Skip to content

Commit

Permalink
Merge pull request #23703 from michael-redpanda/debug-bundle/core-785…
Browse files Browse the repository at this point in the history
…9/issue-selector

debug_bundle: Added label_select option
  • Loading branch information
michael-redpanda authored Oct 11, 2024
2 parents 08fbaf3 + 1840b4c commit f0bfb1f
Show file tree
Hide file tree
Showing 9 changed files with 151 additions and 47 deletions.
1 change: 0 additions & 1 deletion src/v/debug_bundle/BUILD
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ redpanda_cc_library(
"//src/v/ssx:future_util",
"//src/v/utils:external_process",
"@boost//:algorithm",
"@re2",
],
include_prefix = "debug_bundle",
visibility = ["//visibility:public"],
Expand Down
27 changes: 8 additions & 19 deletions src/v/debug_bundle/debug_bundle_service.cc
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,6 @@

#include <boost/algorithm/string/join.hpp>
#include <fmt/format.h>
#include <re2/re2.h>

#include <variant>

Expand All @@ -63,6 +62,7 @@ constexpr std::string_view tls_enabled_variable = "-Xtls.enabled";
constexpr std::string_view tls_insecure_skip_verify_variable
= "-Xtls.insecure_skip_verify";
constexpr std::string_view k8s_namespace_variable = "--namespace";
constexpr std::string_view k8s_label_selector = "--label-selector";

bool contains_sensitive_info(const ss::sstring& arg) {
if (arg.find(password_variable) != ss::sstring::npos) {
Expand Down Expand Up @@ -95,20 +95,6 @@ std::filesystem::path form_process_output_file_path(
return base_path / form_process_output_file_name(job_id);
}

bool is_valid_rfc1123(std::string_view ns) {
// Regular expression for RFC1123 hostname validation
constexpr std::string_view rfc1123_pattern
= R"(^([a-zA-Z0-9]([a-zA-Z0-9-]{0,61}[a-zA-Z0-9])?))";

// Validate the hostname against the regular expression using RE2
return RE2::FullMatch(ns, RE2(rfc1123_pattern));
}

bool is_valid_k8s_namespace(std::string_view ns) {
constexpr auto max_ns_length = 63;
return !ns.empty() && ns.size() <= max_ns_length && is_valid_rfc1123(ns);
}

std::filesystem::path form_debug_bundle_storage_directory() {
const auto& debug_bundle_dir
= config::shard_local_cfg().debug_bundle_storage_dir();
Expand Down Expand Up @@ -670,13 +656,16 @@ result<std::vector<ss::sstring>> service::build_rpk_arguments(
*params.tls_insecure_skip_verify));
}
if (params.k8s_namespace.has_value()) {
if (!is_valid_k8s_namespace(params.k8s_namespace.value()())) {
return error_info(
error_code::invalid_parameters, "Invalid k8s namespace name");
}
rv.emplace_back(k8s_namespace_variable);
rv.emplace_back(*params.k8s_namespace);
}
if (
params.label_selector.has_value()
&& !params.label_selector.value().empty()) {
rv.emplace_back(k8s_label_selector);
rv.emplace_back(
ssx::sformat("{}", fmt::join(params.label_selector.value(), ",")));
}

return rv;
}
Expand Down
36 changes: 35 additions & 1 deletion src/v/debug_bundle/json.h
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
#include "json/types.h"
#include "reflection/type_traits.h"
#include "security/types.h"
#include "strings/utf8.h"
#include "utils/functional.h"
#include "utils/uuid.h"

Expand Down Expand Up @@ -83,7 +84,14 @@ debug_bundle::result<T> from_json(const json::Value& v) {
return std::move(r).assume_error();
} else if constexpr (std::is_same_v<T, ss::sstring>) {
if (v.IsString()) {
return T{as_string_view(v)};
auto vv = as_string_view(v);
try {
validate_no_control(vv);
} catch (const std::runtime_error& e) {
return parse_error(
fmt::format(": invalid control character: {}", e.what()));
}
return T{vv};
}
return parse_error(": expected string");
} else if constexpr (std::is_same_v<T, uuid_t>) {
Expand Down Expand Up @@ -208,6 +216,25 @@ debug_bundle::result<T> from_json(const json::Value& v) {
return v.GetBool();
}
return parse_error(": expected bool");
} else if constexpr (std::is_same_v<T, label_selection>) {
if (v.IsObject()) {
auto o = v.GetObject();
label_selection ls;
if (auto r = from_json<decltype(ls.key)>(o, "key", true);
r.has_value()) {
ls.key = std::move(r).assume_value();
} else {
return std::move(r).assume_error();
}
if (auto r = from_json<decltype(ls.value)>(o, "value", true);
r.has_value()) {
ls.value = std::move(r).assume_value();
} else {
return std::move(r).assume_error();
}
return std::move(ls);
}
return parse_error(": expected object");
} else if constexpr (std::is_same_v<T, debug_bundle_parameters>) {
debug_bundle_parameters params;
if (v.IsObject()) {
Expand Down Expand Up @@ -290,6 +317,13 @@ debug_bundle::result<T> from_json(const json::Value& v) {
} else {
return std::move(r).assume_error();
}
if (auto r = from_json<decltype(params.label_selector)>(
obj, "label_selector", false);
r.has_value()) {
params.label_selector = r.assume_value();
} else {
return std::move(r).assume_error();
}

return std::move(params);
}
Expand Down
26 changes: 11 additions & 15 deletions src/v/debug_bundle/tests/debug_bundle_service_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -297,6 +297,9 @@ TEST_F_CORO(debug_bundle_service_started_fixture, test_all_parameters) {
bool tls_enabled = true;
bool tls_insecure_skip_verify = false;
ss::sstring k8s_namespace_name = "redpanda-namespace";
std::vector<debug_bundle::label_selection> label_select{
debug_bundle::label_selection{.key = "test/key1", .value = "value1"},
debug_bundle::label_selection{.key = "key2", .value = "value2"}};

debug_bundle::debug_bundle_parameters params{
.authn_options = debug_bundle::
Expand All @@ -310,15 +313,16 @@ TEST_F_CORO(debug_bundle_service_started_fixture, test_all_parameters) {
.partition = partition,
.tls_enabled = tls_enabled,
.tls_insecure_skip_verify = tls_insecure_skip_verify,
.k8s_namespace = debug_bundle::k8s_namespace{k8s_namespace_name}};
.k8s_namespace = k8s_namespace_name,
.label_selector = label_select};

ss::sstring expected_params(fmt::format(
"debug bundle --output {}/{}.zip --verbose -Xuser={} -Xpass={} "
"-Xsasl.mechanism={} --controller-logs-size-limit {}B "
"--cpu-profiler-wait {}s --logs-since {} --logs-size-limit {}B "
"--logs-until {} --metrics-interval {}s --partition {}/{}/1,2,3 "
"{}/{}/4,5,6 -Xtls.enabled=true -Xtls.insecure_skip_verify=false "
"--namespace {}\n",
"--namespace {} --label-selector {}={},{}={}\n",
(_data_dir / debug_bundle::service::debug_bundle_dir_name).native(),
job_id,
username,
Expand All @@ -334,7 +338,11 @@ TEST_F_CORO(debug_bundle_service_started_fixture, test_all_parameters) {
tn1.tp,
tn2.ns,
tn2.tp,
k8s_namespace_name));
k8s_namespace_name,
label_select[0].key,
label_select[0].value,
label_select[1].key,
label_select[1].value));

auto res = co_await _service.local().initiate_rpk_debug_bundle_collection(
job_id, std::move(params));
Expand Down Expand Up @@ -366,18 +374,6 @@ TEST_F_CORO(debug_bundle_service_started_fixture, test_all_parameters) {
}
}

TEST_F_CORO(debug_bundle_service_started_fixture, test_invalid_k8s_name) {
debug_bundle::job_id_t job_id(uuid_t::create());
ss::sstring invalid_k8s_namespace_name = "redpanda-namespace.*/";
debug_bundle::debug_bundle_parameters params{
.k8s_namespace = debug_bundle::k8s_namespace{invalid_k8s_namespace_name}};
auto res = co_await _service.local().initiate_rpk_debug_bundle_collection(
job_id, std::move(params));
ASSERT_FALSE_CORO(res.has_value());
EXPECT_EQ(
res.assume_error().code(), debug_bundle::error_code::invalid_parameters);
}

TEST_F_CORO(debug_bundle_service_started_fixture, try_running_multiple) {
auto res = co_await _service.invoke_on(
debug_bundle::service_shard, [](debug_bundle::service& s) {
Expand Down
63 changes: 54 additions & 9 deletions src/v/debug_bundle/tests/json_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@

#include <chrono>
#include <optional>
#include <string>
#include <type_traits>

using namespace debug_bundle;
Expand Down Expand Up @@ -59,7 +60,7 @@ using JsonTestTypes = ::testing::Types<
std::vector<int>,
absl::btree_set<int>,
bool,
k8s_namespace>;
label_selection>;
TYPED_TEST_SUITE(JsonTypeTest, JsonTestTypes);

TYPED_TEST(JsonTypeTest, BasicType) {
Expand Down Expand Up @@ -134,7 +135,11 @@ TYPED_TEST(JsonTypeTest, BasicType) {
],
"tls_enabled": true,
"tls_insecure_skip_verify": false,
"namespace": "k8s-namespace"
"namespace": "k8s-namespace",
"label_selector": [
{"key": "test/key1", "value": "value1"},
{"key": "key2", "value": "value2"}
]
})";
const std::string_view test_time = "2024-09-05T14:34:02";
std::istringstream ss(test_time.data());
Expand All @@ -158,7 +163,9 @@ TYPED_TEST(JsonTypeTest, BasicType) {
.partition = std::vector<partition_selection>{{{model::ns{"foo"}, model::topic{"bar"}}, {{model::partition_id{1}, model::partition_id{2}}}}, {{model::kafka_namespace, model::topic{"baz"}}, {{model::partition_id{1}, model::partition_id{2}, model::partition_id{3}}}}},
.tls_enabled = true,
.tls_insecure_skip_verify = false,
.k8s_namespace = debug_bundle::k8s_namespace("k8s-namespace")};
.k8s_namespace = "k8s-namespace",
.label_selector = std::vector<label_selection>{
{"test/key1", "value1"}, {"key2", "value2"}}};
} else if constexpr (detail::
is_specialization_of_v<TypeParam, std::vector>) {
this->json_input = R"([1,2,3])";
Expand All @@ -170,9 +177,9 @@ TYPED_TEST(JsonTypeTest, BasicType) {
} else if constexpr (std::is_same_v<TypeParam, bool>) {
this->json_input = R"(true)";
this->expected = true;
} else if constexpr (std::is_same_v<TypeParam, k8s_namespace>) {
this->json_input = R"("k8s-namespace")";
this->expected = k8s_namespace("k8s-namespace");
} else if constexpr (std::is_same_v<TypeParam, label_selection>) {
this->json_input = R"({"key": "test/key1", "value": "value1"})";
this->expected = label_selection{.key = "test/key1", .value = "value1"};
} else {
static_assert(always_false_v<TypeParam>, "not implemented");
}
Expand Down Expand Up @@ -256,9 +263,9 @@ TYPED_TEST(JsonTypeTest, TypeIsInvalid) {
} else if constexpr (std::is_same_v<TypeParam, bool>) {
this->json_input = R"("blergh")";
this->expected = true;
} else if constexpr (std::is_same_v<TypeParam, k8s_namespace>) {
this->json_input = R"(42)";
this->expected = k8s_namespace("k8s-namespace");
} else if constexpr (std::is_same_v<TypeParam, label_selection>) {
this->json_input = R"("key")";
this->expected = label_selection{.key = "test/key1", .value = "value1"};
} else {
static_assert(always_false_v<TypeParam>, "not implemented");
}
Expand All @@ -279,3 +286,41 @@ TYPED_TEST(JsonTypeTest, TypeIsInvalid) {
EXPECT_TRUE(res.assume_error().message().starts_with("Failed to parse"))
<< res.assume_error().message();
}

TYPED_TEST(JsonTypeTest, ValidateControlCharacters) {
if constexpr (std::is_same_v<TypeParam, ss::sstring>) {
this->json_input = R"("foo\nbar")";
this->expected = "foo\nbar";
} else if constexpr (std::is_same_v<TypeParam, scram_creds>) {
this->json_input
= R"({"username": "user\r", "password": "pass", "mechanism": "SCRAM-SHA-256"})";
this->expected = scram_creds{
.username{"user"}, .password{"pass"}, .mechanism{"SCRAM-SHA-256"}};
} else if constexpr (std::
is_same_v<TypeParam, debug_bundle_authn_options>) {
this->json_input
= R"({"username": "user", "password": "\fpass", "mechanism": "SCRAM-SHA-256"})";
this->expected = TypeParam{scram_creds{
.username{"user"}, .password{"pass"}, .mechanism{"SCRAM-SHA-256"}}};
} else {
return;
}

json::Document doc;
ASSERT_NO_THROW(doc.Parse(this->json_input));
ASSERT_FALSE(doc.HasParseError()) << fmt::format(
"Malformed json schema: {} at offset {}",
rapidjson::GetParseError_En(doc.GetParseError()),
doc.GetErrorOffset());

debug_bundle::result<TypeParam> res{outcome::success()};

ASSERT_NO_THROW(res = from_json<TypeParam>(doc));
ASSERT_TRUE(res.has_error());
ASSERT_FALSE(res.has_exception());
EXPECT_EQ(res.assume_error().code(), error_code::invalid_parameters);
EXPECT_TRUE(
res.assume_error().message().find("invalid control character")
!= std::string::npos)
<< res.assume_error().message();
}
5 changes: 5 additions & 0 deletions src/v/debug_bundle/types.cc
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,11 @@ std::ostream& operator<<(std::ostream& o, const partition_selection& p) {
return o;
}

std::ostream& operator<<(std::ostream& o, const label_selection& l) {
fmt::print(o, "{}={}", l.key, l.value);
return o;
}

std::optional<partition_selection>
partition_selection::from_string_view(std::string_view str) {
try {
Expand Down
13 changes: 11 additions & 2 deletions src/v/debug_bundle/types.h
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,15 @@ struct partition_selection {

std::ostream& operator<<(std::ostream& o, const partition_selection& p);

using k8s_namespace = named_type<ss::sstring, struct k8s_namespace_tag>;
struct label_selection {
ss::sstring key;
ss::sstring value;

friend bool operator==(const label_selection&, const label_selection&)
= default;

friend std::ostream& operator<<(std::ostream& o, const label_selection& l);
};

/// Parameters used to spawn rpk debug bundle
struct debug_bundle_parameters {
Expand All @@ -101,7 +109,8 @@ struct debug_bundle_parameters {
std::optional<std::vector<partition_selection>> partition;
std::optional<bool> tls_enabled;
std::optional<bool> tls_insecure_skip_verify;
std::optional<k8s_namespace> k8s_namespace;
std::optional<ss::sstring> k8s_namespace;
std::optional<std::vector<label_selection>> label_selector;

friend bool
operator==(const debug_bundle_parameters&, const debug_bundle_parameters&)
Expand Down
21 changes: 21 additions & 0 deletions src/v/redpanda/admin/api-doc/debug_bundle.json
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,27 @@
},
"namespace": {
"type": "string"
},
"label_selector": {
"description": "Label selection to filter k8s resources in the debug bundle response",
"type": "array",
"items": {
"type": "object",
"required": [
"key",
"value"
],
"properties": {
"key": {
"description": "The resource label key",
"type": "string"
},
"value": {
"description": "The resource label value",
"type": "string"
}
}
}
}
}
}
Expand Down
6 changes: 6 additions & 0 deletions tests/rptest/services/admin.py
Original file line number Diff line number Diff line change
Expand Up @@ -408,6 +408,11 @@ def encode(self, o: Any) -> str:
return super().encode(self.default(o))


class DebugBundleLabelSelection(NamedTuple):
key: str
value: str


class DebugBundleStartConfigParams(NamedTuple):
authentication: Optional[SaslCredentials] = None
controller_logs_size_limit_bytes: Optional[int] = None
Expand All @@ -420,6 +425,7 @@ class DebugBundleStartConfigParams(NamedTuple):
tls_enabled: Optional[bool] = None
tls_insecure_skip_verify: Optional[bool] = None
namespace: Optional[str] = None
label_selector: Optional[list[DebugBundleLabelSelection]] = None


class DebugBundleStartConfig(NamedTuple):
Expand Down

0 comments on commit f0bfb1f

Please sign in to comment.