diff --git a/proto/redpanda/core/admin/v2/shadow_link.proto b/proto/redpanda/core/admin/v2/shadow_link.proto index 002a8f3c47457..4f582fbf4d32f 100644 --- a/proto/redpanda/core/admin/v2/shadow_link.proto +++ b/proto/redpanda/core/admin/v2/shadow_link.proto @@ -422,14 +422,30 @@ message SecuritySettingsSyncOptions { // ACL filters repeated ACLFilter acl_filters = 4; } -// Authentication config. Currently only supporting SASL/SCRAM, -// however made as a oneof for expansion +// Authentication config. Supports: +// * SASL/SCRAM +// * SASL/PLAIN message AuthenticationConfiguration { oneof authentication { // SASL/SCRAM configuration ScramConfig scram_configuration = 1; + // SASL/PLAIN configuration + PlainConfig plain_configuration = 2; } } +// PLAIN settings +message PlainConfig { + // PLAIN username + string username = 1; + // Password + string password = 2 [(google.api.field_behavior) = INPUT_ONLY]; + // Indicates that the password has been set + bool password_set = 3 [(google.api.field_behavior) = OUTPUT_ONLY]; + // Timestamp of when the password was last set - only valid if password_set + // is true + google.protobuf.Timestamp password_set_at = 4 + [(google.api.field_behavior) = OUTPUT_ONLY]; +} // SCRAM settings message ScramConfig { // SCRAM username diff --git a/src/go/rpk/gen/protocomments/admin/v2/comments.pb.go b/src/go/rpk/gen/protocomments/admin/v2/comments.pb.go index b8096cf25b247..13676c873bcc5 100644 --- a/src/go/rpk/gen/protocomments/admin/v2/comments.pb.go +++ b/src/go/rpk/gen/protocomments/admin/v2/comments.pb.go @@ -26,8 +26,10 @@ var Comments = map[string]string{ that are named ` + "`" + `*` + "`" + ``, "redpanda.core.admin.v2.ACLResourceFilter.pattern_type": "The pattern to apply to name", "redpanda.core.admin.v2.ACLResourceFilter.resource_type": "The ACL resource type to match", - "redpanda.core.admin.v2.AuthenticationConfiguration": `Authentication config. Currently only supporting SASL/SCRAM, - however made as a oneof for expansion`, + "redpanda.core.admin.v2.AuthenticationConfiguration": `Authentication config. Supports: + * SASL/SCRAM + * SASL/PLAIN`, + "redpanda.core.admin.v2.AuthenticationConfiguration.plain_configuration": "SASL/PLAIN configuration", "redpanda.core.admin.v2.AuthenticationConfiguration.scram_configuration": "SASL/SCRAM configuration", "redpanda.core.admin.v2.ConsumerOffsetSyncOptions": "Options for syncing consumer offsets", "redpanda.core.admin.v2.ConsumerOffsetSyncOptions.effective_interval": "The effective interval for the task", @@ -74,10 +76,16 @@ var Comments = map[string]string{ "redpanda.core.admin.v2.NameFilter.name": `The resource name, or "*" Note if "*", must be the _only_ character and ` + "`" + `pattern_type` + "`" + ` must be ` + "`" + `PATTERN_TYPE_LITERAL` + "`" + ``, - "redpanda.core.admin.v2.NameFilter.pattern_type": "Literal or prefix", - "redpanda.core.admin.v2.PATTERN_TYPE_LITERAL": "Must match the filter exactly", - "redpanda.core.admin.v2.PATTERN_TYPE_PREFIX": "Will match anything that starts with filter", - "redpanda.core.admin.v2.PatternType": "The matching pattern type", + "redpanda.core.admin.v2.NameFilter.pattern_type": "Literal or prefix", + "redpanda.core.admin.v2.PATTERN_TYPE_LITERAL": "Must match the filter exactly", + "redpanda.core.admin.v2.PATTERN_TYPE_PREFIX": "Will match anything that starts with filter", + "redpanda.core.admin.v2.PatternType": "The matching pattern type", + "redpanda.core.admin.v2.PlainConfig": "PLAIN settings", + "redpanda.core.admin.v2.PlainConfig.password": "Password", + "redpanda.core.admin.v2.PlainConfig.password_set": "Indicates that the password has been set", + "redpanda.core.admin.v2.PlainConfig.password_set_at": `Timestamp of when the password was last set - only valid if password_set + is true`, + "redpanda.core.admin.v2.PlainConfig.username": "PLAIN username", "redpanda.core.admin.v2.SCRAM_MECHANISM_SCRAM_SHA_256": "SCRAM-SHA-256", "redpanda.core.admin.v2.SCRAM_MECHANISM_SCRAM_SHA_512": "SCRAM-SHA-512", "redpanda.core.admin.v2.SHADOW_LINK_STATE_ACTIVE": "Shadow link is active", diff --git a/src/v/cluster/cluster_link/frontend.cc b/src/v/cluster/cluster_link/frontend.cc index a0062d640bd5f..52d3e8eccc977 100644 --- a/src/v/cluster/cluster_link/frontend.cc +++ b/src/v/cluster/cluster_link/frontend.cc @@ -981,8 +981,8 @@ errc frontend::validator::validate_connection_config( } if ( - c.mechanism != "SCRAM-SHA-256" - && c.mechanism != "SCRAM-SHA-512") { + c.mechanism != "SCRAM-SHA-256" && c.mechanism != "SCRAM-SHA-512" + && c.mechanism != "PLAIN") { vlog( cluster::clusterlog.warn, "Unsupported SCRAM mechanism: {}", diff --git a/src/v/kafka/client/broker.cc b/src/v/kafka/client/broker.cc index 583767b2dc219..7e63c1e044f07 100644 --- a/src/v/kafka/client/broker.cc +++ b/src/v/kafka/client/broker.cc @@ -15,6 +15,7 @@ #include "net/connection.h" #include "random/generators.h" #include "security/oidc_authenticator.h" +#include "security/plain_authenticator.h" #include "security/scram_authenticator.h" #include "thirdparty/c-ares/ares.h" #include "utils/backoff_policy.h" @@ -320,7 +321,8 @@ ss::future<> remote_broker::do_authenticate() { if ( mechanism != security::scram_sha256_authenticator::name && mechanism != security::scram_sha512_authenticator::name - && mechanism != security::oidc::sasl_authenticator::name) { + && mechanism != security::oidc::sasl_authenticator::name + && mechanism != security::plain_authenticator::name) { throw broker_error{ _node_id, error_code::sasl_authentication_failed, @@ -348,11 +350,17 @@ ss::future<> remote_broker::do_authenticate() { if (mechanism == security::scram_sha256_authenticator::name) { co_await do_authenticate_scram256(username, password); - } else if (mechanism == security::scram_sha512_authenticator::name) { co_await do_authenticate_scram512(username, password); } else if (mechanism == security::oidc::sasl_authenticator::name) { co_await do_authenticate_oauthbearer(password); + } else if (mechanism == security::plain_authenticator::name) { + co_await do_authenticate_plain(username, password); + } else { + throw broker_error{ + _node_id, + error_code::sasl_authentication_failed, + fmt_with_ctx(ssx::sformat, "Unknown mechanism: {}", mechanism)}; } } @@ -520,6 +528,27 @@ ss::future<> remote_broker::do_authenticate_oauthbearer(ss::sstring token) { } } +ss::future<> remote_broker::do_authenticate_plain( + ss::sstring username, ss::sstring password) { + sasl_authenticate_request req; + std::string bytes; + // 2 - number of null characters in the PLAIN auth message + bytes.reserve(2 + username.size() + password.size()); + bytes.push_back('\0'); + bytes.append(username.cbegin(), username.cend()); + bytes.push_back('\0'); + bytes.append(password.cbegin(), password.cend()); + req.data.auth_bytes = bytes::from_string(std::move(bytes)); + auto res = co_await do_dispatch( + std::move(req), get_sasl_authenticate_request_version()); + if (res.data.errored()) { + throw broker_error{ + _node_id, + res.data.error_code, + res.data.error_message.value_or("")}; + } +} + namespace { template api_version get_auth_request_version( diff --git a/src/v/kafka/client/broker.h b/src/v/kafka/client/broker.h index 0ffc6379d0dbd..f1d37a62f52be 100644 --- a/src/v/kafka/client/broker.h +++ b/src/v/kafka/client/broker.h @@ -191,6 +191,8 @@ class remote_broker ss::future<> do_authenticate_scram512(ss::sstring username, ss::sstring password); ss::future<> do_authenticate_oauthbearer(ss::sstring token); + ss::future<> + do_authenticate_plain(ss::sstring username, ss::sstring password); ss::future send_scram_client_first(const security::client_first_message& client_first); ss::future diff --git a/src/v/redpanda/admin/services/shadow_link/converter.cc b/src/v/redpanda/admin/services/shadow_link/converter.cc index bf1902b285f5c..276a0ab0ba463 100644 --- a/src/v/redpanda/admin/services/shadow_link/converter.cc +++ b/src/v/redpanda/admin/services/shadow_link/converter.cc @@ -30,6 +30,7 @@ using proto::admin::authentication_configuration; using proto::admin::consumer_offset_sync_options; using proto::admin::create_shadow_link_request; using proto::admin::name_filter; +using proto::admin::plain_config; using proto::admin::schema_registry_sync_options; using proto::admin::schema_registry_sync_options_shadow_schema_registry_topic; using proto::admin::scram_config; @@ -407,6 +408,19 @@ create_authn_settings(const authentication_configuration& authn_config) { scram_mechanism_to_string(scram.get_scram_mechanism())}; return creds; }, + [](const plain_config& plain) + -> cluster_link::model::connection_config::authn_variant { + if (plain.get_username().empty() || plain.get_password().empty()) { + throw std::invalid_argument( + "When setting PLAIN configuration, must provide username and " + "password"); + } + cluster_link::model::scram_credentials creds; + creds.username = plain.get_username(); + creds.password = plain.get_password(); + creds.mechanism = "PLAIN"; + return creds; + }, [](std::monostate) -> cluster_link::model::connection_config::authn_variant { throw std::invalid_argument( @@ -524,26 +538,38 @@ authentication_configuration create_authentication_configuration( authn, [](const cluster_link::model::scram_credentials& scram) -> authentication_configuration { - scram_config scram_proto; - scram_proto.set_username(ss::sstring{scram.username}); - scram_proto.set_password_set(true); - scram_proto.set_password_set_at( - absl::FromChrono( - model::to_time_point(scram.password_last_updated))); - scram_proto.set_scram_mechanism( - proto::admin::scram_mechanism::unspecified); - if (scram.mechanism == "SCRAM-SHA-256") { - scram_proto.set_scram_mechanism( - proto::admin::scram_mechanism::scram_sha_256); - } else if (scram.mechanism == "SCRAM-SHA-512") { - scram_proto.set_scram_mechanism( - proto::admin::scram_mechanism::scram_sha_512); + authentication_configuration authn; + if (scram.mechanism == "PLAIN") { + plain_config plain_proto; + plain_proto.set_username(ss::sstring{scram.username}); + plain_proto.set_password_set(true); + plain_proto.set_password_set_at( + absl::FromChrono( + model::to_time_point(scram.password_last_updated))); + authn.set_plain_configuration(std::move(plain_proto)); } else { - throw std::invalid_argument( - ssx::sformat("Unknown SCRAM mechanism: {}", scram.mechanism)); + scram_config scram_proto; + scram_proto.set_username(ss::sstring{scram.username}); + scram_proto.set_password_set(true); + scram_proto.set_password_set_at( + absl::FromChrono( + model::to_time_point(scram.password_last_updated))); + scram_proto.set_scram_mechanism( + proto::admin::scram_mechanism::unspecified); + if (scram.mechanism == "SCRAM-SHA-256") { + scram_proto.set_scram_mechanism( + proto::admin::scram_mechanism::scram_sha_256); + } else if (scram.mechanism == "SCRAM-SHA-512") { + scram_proto.set_scram_mechanism( + proto::admin::scram_mechanism::scram_sha_512); + } else { + throw std::invalid_argument( + ssx::sformat( + "Unknown SCRAM mechanism: {}", scram.mechanism)); + } + + authn.set_scram_configuration(std::move(scram_proto)); } - authentication_configuration authn; - authn.set_scram_configuration(std::move(scram_proto)); return authn; }); } diff --git a/src/v/redpanda/admin/services/shadow_link/tests/converter_test.cc b/src/v/redpanda/admin/services/shadow_link/tests/converter_test.cc index ed5e082e8e57d..5a0fe1cc5ea5e 100644 --- a/src/v/redpanda/admin/services/shadow_link/tests/converter_test.cc +++ b/src/v/redpanda/admin/services/shadow_link/tests/converter_test.cc @@ -247,6 +247,53 @@ TEST(converter_test, create_with_authn_config_scram_unspecified) { serde::pb::rpc::invalid_argument_exception); } +TEST(converter_test, create_with_plain) { + const auto name = "test-link"; + const auto username = "test-user"; + const auto password = "test-password"; + + proto::admin::shadow_link shadow_link; + proto::admin::create_shadow_link_request req; + proto::admin::shadow_link_configurations shadow_link_configurations; + proto::admin::shadow_link_client_options shadow_link_client_options; + proto::admin::authentication_configuration authn_config; + proto::admin::plain_config plain_config; + + plain_config.set_username(ss::sstring{username}); + plain_config.set_password(ss::sstring{password}); + authn_config.set_plain_configuration(std::move(plain_config)); + shadow_link_client_options.set_authentication_configuration( + std::move(authn_config)); + + shadow_link_client_options.set_bootstrap_servers({"localhost:9092"}); + shadow_link_configurations.set_client_options( + std::move(shadow_link_client_options)); + + shadow_link.set_configurations(std::move(shadow_link_configurations)); + shadow_link.set_name(ss::sstring{name}); + req.set_shadow_link(std::move(shadow_link)); + + auto now = model::to_time_point(model::timestamp::now()); + auto md = admin::convert_create_to_metadata(std::move(req)); + + ASSERT_TRUE(md.connection.authn_config.has_value()); + ASSERT_TRUE( + std::holds_alternative( + md.connection.authn_config.value())); + const auto& md_authn_config + = std::get( + md.connection.authn_config.value()); + + EXPECT_EQ(md_authn_config.username, username); + EXPECT_EQ(md_authn_config.password, password); + EXPECT_EQ(md_authn_config.mechanism, "PLAIN"); + auto pwd_updated = model::to_time_point( + md_authn_config.password_last_updated); + // Expect the password updated time to be within 10s + EXPECT_GE(pwd_updated, now - 5s); + EXPECT_LE(pwd_updated, now + 5s); +} + TEST(converter_test, create_with_tls_flag_only) { const auto name = "test-link"; proto::admin::shadow_link shadow_link; @@ -745,6 +792,26 @@ TEST(converter_test, metadata_to_shadow_link_authn_invalid_scram) { admin::metadata_to_shadow_link(std::move(md), {}), std::invalid_argument); } +TEST(converter_test, metadata_to_shadow_link_authn_plain) { + cluster_link::model::metadata md; + md.connection.authn_config = cluster_link::model::scram_credentials{ + .username = "test-user", + .password = "test-password", + .mechanism = "PLAIN"}; + + auto sl = admin::metadata_to_shadow_link(std::move(md), {}); + + const auto& client_options = sl.get_configurations().get_client_options(); + ASSERT_TRUE(client_options.has_authentication_configuration()); + ASSERT_TRUE(client_options.get_authentication_configuration() + .has_plain_configuration()); + const auto& plain_config = client_options.get_authentication_configuration() + .get_plain_configuration(); + EXPECT_EQ(plain_config.get_username(), "test-user"); + EXPECT_TRUE(plain_config.get_password_set()); + EXPECT_TRUE(plain_config.get_password().empty()); +} + TEST(converter_test, metadata_to_shadow_link_tls_file) { const auto ca_file = "ca_file.pem"; const auto key_file = "key_file.pem"; diff --git a/tests/rptest/clients/admin/proto/redpanda/core/admin/v2/shadow_link_pb2.py b/tests/rptest/clients/admin/proto/redpanda/core/admin/v2/shadow_link_pb2.py index c164802e1688d..71a4d6cc2996d 100644 --- a/tests/rptest/clients/admin/proto/redpanda/core/admin/v2/shadow_link_pb2.py +++ b/tests/rptest/clients/admin/proto/redpanda/core/admin/v2/shadow_link_pb2.py @@ -16,7 +16,7 @@ from google.protobuf import duration_pb2 as google_dot_protobuf_dot_duration__pb2 from google.protobuf import timestamp_pb2 as google_dot_protobuf_dot_timestamp__pb2 from google.protobuf import field_mask_pb2 as google_dot_protobuf_dot_field__mask__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n.proto/redpanda/core/admin/v2/shadow_link.proto\x12\x16redpanda.core.admin.v2\x1a\'proto/redpanda/core/pbgen/options.proto\x1a#proto/redpanda/core/pbgen/rpc.proto\x1a\'proto/redpanda/core/common/v1/acl.proto\x1a\'proto/redpanda/core/common/v1/tls.proto\x1a\x1fgoogle/api/field_behavior.proto\x1a\x1bgoogle/api/field_info.proto\x1a\x19google/api/resource.proto\x1a\x1egoogle/protobuf/duration.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a google/protobuf/field_mask.proto"\xc2\x01\n\nShadowLink\x12\x11\n\x04name\x18\x01 \x01(\tB\x03\xe0A\x02\x12\x18\n\x03uid\x18\x02 \x01(\tB\x0b\xe0A\x03\xe2\x8c\xcf\xd7\x08\x02\x08\x01\x12H\n\x0econfigurations\x18\x03 \x01(\x0b20.redpanda.core.admin.v2.ShadowLinkConfigurations\x12=\n\x06status\x18\x04 \x01(\x0b2(.redpanda.core.admin.v2.ShadowLinkStatusB\x03\xe0A\x03"\xc0\x01\n\x0bShadowTopic\x12\x11\n\x04name\x18\x01 \x01(\tB\x03\xe0A\x03\x12\x1d\n\x08topic_id\x18\x02 \x01(\tB\x0b\xe0A\x03\xe2\x8c\xcf\xd7\x08\x02\x08\x01\x12\x19\n\x11source_topic_name\x18\x03 \x01(\t\x12$\n\x0fsource_topic_id\x18\x04 \x01(\tB\x0b\xe0A\x03\xe2\x8c\xcf\xd7\x08\x02\x08\x01\x12>\n\x06status\x18\x05 \x01(\x0b2).redpanda.core.admin.v2.ShadowTopicStatusB\x03\xe0A\x03"R\n\x17CreateShadowLinkRequest\x127\n\x0bshadow_link\x18\x01 \x01(\x0b2".redpanda.core.admin.v2.ShadowLink"S\n\x18CreateShadowLinkResponse\x127\n\x0bshadow_link\x18\x01 \x01(\x0b2".redpanda.core.admin.v2.ShadowLink"p\n\x17DeleteShadowLinkRequest\x12F\n\x04name\x18\x01 \x01(\tB8\xe0A\x02\xfaA2\n0redpanda.core.admin.ShadowLinkService/ShadowLink\x12\r\n\x05force\x18\x02 \x01(\x08"\x1a\n\x18DeleteShadowLinkResponse"^\n\x14GetShadowLinkRequest\x12F\n\x04name\x18\x01 \x01(\tB8\xe0A\x02\xfaA2\n0redpanda.core.admin.ShadowLinkService/ShadowLink"P\n\x15GetShadowLinkResponse\x127\n\x0bshadow_link\x18\x01 \x01(\x0b2".redpanda.core.admin.v2.ShadowLink"\x18\n\x16ListShadowLinksRequest"S\n\x17ListShadowLinksResponse\x128\n\x0cshadow_links\x18\x01 \x03(\x0b2".redpanda.core.admin.v2.ShadowLink"\x83\x01\n\x17UpdateShadowLinkRequest\x127\n\x0bshadow_link\x18\x01 \x01(\x0b2".redpanda.core.admin.v2.ShadowLink\x12/\n\x0bupdate_mask\x18\x02 \x01(\x0b2\x1a.google.protobuf.FieldMask"S\n\x18UpdateShadowLinkResponse\x127\n\x0bshadow_link\x18\x01 \x01(\x0b2".redpanda.core.admin.v2.ShadowLink"y\n\x0fFailOverRequest\x12F\n\x04name\x18\x01 \x01(\tB8\xe0A\x02\xfaA2\n0redpanda.core.admin.ShadowLinkService/ShadowLink\x12\x1e\n\x11shadow_topic_name\x18\x02 \x01(\tB\x03\xe0A\x01"K\n\x10FailOverResponse\x127\n\x0bshadow_link\x18\x01 \x01(\x0b2".redpanda.core.admin.v2.ShadowLink"I\n\x15GetShadowTopicRequest\x12\x1d\n\x10shadow_link_name\x18\x01 \x01(\tB\x03\xe0A\x02\x12\x11\n\x04name\x18\x02 \x01(\tB\x03\xe0A\x02"S\n\x16GetShadowTopicResponse\x129\n\x0cshadow_topic\x18\x01 \x01(\x0b2#.redpanda.core.admin.v2.ShadowTopic"8\n\x17ListShadowTopicsRequest\x12\x1d\n\x10shadow_link_name\x18\x01 \x01(\tB\x03\xe0A\x02"V\n\x18ListShadowTopicsResponse\x12:\n\rshadow_topics\x18\x01 \x03(\x0b2#.redpanda.core.admin.v2.ShadowTopic"\xc0\x03\n\x18ShadowLinkConfigurations\x12G\n\x0eclient_options\x18\x01 \x01(\x0b2/.redpanda.core.admin.v2.ShadowLinkClientOptions\x12U\n\x1btopic_metadata_sync_options\x18\x02 \x01(\x0b20.redpanda.core.admin.v2.TopicMetadataSyncOptions\x12W\n\x1cconsumer_offset_sync_options\x18\x03 \x01(\x0b21.redpanda.core.admin.v2.ConsumerOffsetSyncOptions\x12R\n\x15security_sync_options\x18\x04 \x01(\x0b23.redpanda.core.admin.v2.SecuritySettingsSyncOptions\x12W\n\x1cschema_registry_sync_options\x18\x05 \x01(\x0b21.redpanda.core.admin.v2.SchemaRegistrySyncOptions"\xb4\x06\n\x17ShadowLinkClientOptions\x12\x1e\n\x11bootstrap_servers\x18\x01 \x03(\tB\x03\xe0A\x02\x12\x16\n\tclient_id\x18\x02 \x01(\tB\x03\xe0A\x03\x12\x19\n\x11source_cluster_id\x18\x03 \x01(\t\x12?\n\x0ctls_settings\x18\x04 \x01(\x0b2$.redpanda.core.common.v1.TLSSettingsH\x00\x88\x01\x01\x12^\n\x1cauthentication_configuration\x18\x05 \x01(\x0b23.redpanda.core.admin.v2.AuthenticationConfigurationH\x01\x88\x01\x01\x12\x1b\n\x13metadata_max_age_ms\x18\x06 \x01(\x05\x12*\n\x1deffective_metadata_max_age_ms\x18\x07 \x01(\x05B\x03\xe0A\x03\x12\x1d\n\x15connection_timeout_ms\x18\x08 \x01(\x05\x12,\n\x1feffective_connection_timeout_ms\x18\t \x01(\x05B\x03\xe0A\x03\x12\x18\n\x10retry_backoff_ms\x18\n \x01(\x05\x12\'\n\x1aeffective_retry_backoff_ms\x18\x0b \x01(\x05B\x03\xe0A\x03\x12\x19\n\x11fetch_wait_max_ms\x18\x0c \x01(\x05\x12(\n\x1beffective_fetch_wait_max_ms\x18\r \x01(\x05B\x03\xe0A\x03\x12\x17\n\x0ffetch_min_bytes\x18\x0e \x01(\x05\x12&\n\x19effective_fetch_min_bytes\x18\x0f \x01(\x05B\x03\xe0A\x03\x12\x17\n\x0ffetch_max_bytes\x18\x10 \x01(\x05\x12&\n\x19effective_fetch_max_bytes\x18\x11 \x01(\x05B\x03\xe0A\x03\x12!\n\x19fetch_partition_max_bytes\x18\x12 \x01(\x05\x120\n#effective_fetch_partition_max_bytes\x18\x13 \x01(\x05B\x03\xe0A\x03B\x0f\n\r_tls_settingsB\x1f\n\x1d_authentication_configuration"\xc6\x04\n\x18TopicMetadataSyncOptions\x12+\n\x08interval\x18\x01 \x01(\x0b2\x19.google.protobuf.Duration\x12:\n\x12effective_interval\x18\x02 \x01(\x0b2\x19.google.protobuf.DurationB\x03\xe0A\x03\x12L\n auto_create_shadow_topic_filters\x18\x03 \x03(\x0b2".redpanda.core.admin.v2.NameFilter\x12&\n\x1esynced_shadow_topic_properties\x18\x04 \x03(\t\x12\x17\n\x0fexclude_default\x18\x05 \x01(\x08\x12\\\n\x11start_at_earliest\x18\x06 \x01(\x0b2?.redpanda.core.admin.v2.TopicMetadataSyncOptions.EarliestOffsetH\x00\x12X\n\x0fstart_at_latest\x18\x07 \x01(\x0b2=.redpanda.core.admin.v2.TopicMetadataSyncOptions.LatestOffsetH\x00\x128\n\x12start_at_timestamp\x18\x08 \x01(\x0b2\x1a.google.protobuf.TimestampH\x00\x12\x0e\n\x06paused\x18\t \x01(\x08\x1a\x10\n\x0eEarliestOffset\x1a\x0e\n\x0cLatestOffsetB\x0e\n\x0cstart_offset"\xcf\x01\n\x19SchemaRegistrySyncOptions\x12s\n\x1cshadow_schema_registry_topic\x18\x01 \x01(\x0b2K.redpanda.core.admin.v2.SchemaRegistrySyncOptions.ShadowSchemaRegistryTopicH\x00\x1a\x1b\n\x19ShadowSchemaRegistryTopicB \n\x1eschema_registry_shadowing_mode"\xcf\x01\n\x19ConsumerOffsetSyncOptions\x12+\n\x08interval\x18\x01 \x01(\x0b2\x19.google.protobuf.Duration\x12:\n\x12effective_interval\x18\x02 \x01(\x0b2\x19.google.protobuf.DurationB\x03\xe0A\x03\x12\x0e\n\x06paused\x18\x03 \x01(\x08\x129\n\rgroup_filters\x18\x04 \x03(\x0b2".redpanda.core.admin.v2.NameFilter"\xce\x01\n\x1bSecuritySettingsSyncOptions\x12+\n\x08interval\x18\x01 \x01(\x0b2\x19.google.protobuf.Duration\x12:\n\x12effective_interval\x18\x02 \x01(\x0b2\x19.google.protobuf.DurationB\x03\xe0A\x03\x12\x0e\n\x06paused\x18\x03 \x01(\x08\x126\n\x0bacl_filters\x18\x04 \x03(\x0b2!.redpanda.core.admin.v2.ACLFilter"s\n\x1bAuthenticationConfiguration\x12B\n\x13scram_configuration\x18\x01 \x01(\x0b2#.redpanda.core.admin.v2.ScramConfigH\x00B\x10\n\x0eauthentication"\xcc\x01\n\x0bScramConfig\x12\x10\n\x08username\x18\x01 \x01(\t\x12\x15\n\x08password\x18\x02 \x01(\tB\x03\xe0A\x04\x12\x19\n\x0cpassword_set\x18\x03 \x01(\x08B\x03\xe0A\x03\x128\n\x0fpassword_set_at\x18\x04 \x01(\x0b2\x1a.google.protobuf.TimestampB\x03\xe0A\x03\x12?\n\x0fscram_mechanism\x18\x05 \x01(\x0e2&.redpanda.core.admin.v2.ScramMechanism"\x8e\x01\n\nNameFilter\x129\n\x0cpattern_type\x18\x01 \x01(\x0e2#.redpanda.core.admin.v2.PatternType\x127\n\x0bfilter_type\x18\x02 \x01(\x0e2".redpanda.core.admin.v2.FilterType\x12\x0c\n\x04name\x18\x03 \x01(\t"\x8f\x01\n\tACLFilter\x12B\n\x0fresource_filter\x18\x01 \x01(\x0b2).redpanda.core.admin.v2.ACLResourceFilter\x12>\n\raccess_filter\x18\x02 \x01(\x0b2\'.redpanda.core.admin.v2.ACLAccessFilter"\x99\x01\n\x11ACLResourceFilter\x12;\n\rresource_type\x18\x01 \x01(\x0e2$.redpanda.core.common.v1.ACLResource\x129\n\x0cpattern_type\x18\x02 \x01(\x0e2#.redpanda.core.common.v1.ACLPattern\x12\x0c\n\x04name\x18\x03 \x01(\t"\xb1\x01\n\x0fACLAccessFilter\x12\x11\n\tprincipal\x18\x01 \x01(\t\x128\n\toperation\x18\x02 \x01(\x0e2%.redpanda.core.common.v1.ACLOperation\x12C\n\x0fpermission_type\x18\x03 \x01(\x0e2*.redpanda.core.common.v1.ACLPermissionType\x12\x0c\n\x04host\x18\x04 \x01(\t"\xf3\x01\n\x10ShadowLinkStatus\x126\n\x05state\x18\x01 \x01(\x0e2\'.redpanda.core.admin.v2.ShadowLinkState\x12C\n\rtask_statuses\x18\x02 \x03(\x0b2,.redpanda.core.admin.v2.ShadowLinkTaskStatus\x12:\n\rshadow_topics\x18\x03 \x03(\x0b2#.redpanda.core.admin.v2.ShadowTopic\x12&\n\x1esynced_shadow_topic_properties\x18\x04 \x03(\t"\x88\x01\n\x14ShadowLinkTaskStatus\x12\x0c\n\x04name\x18\x01 \x01(\t\x120\n\x05state\x18\x02 \x01(\x0e2!.redpanda.core.admin.v2.TaskState\x12\x0e\n\x06reason\x18\x03 \x01(\t\x12\x11\n\tbroker_id\x18\x04 \x01(\x05\x12\r\n\x05shard\x18\x05 \x01(\x05"\x9e\x01\n\x11ShadowTopicStatus\x127\n\x05state\x18\x01 \x01(\x0e2(.redpanda.core.admin.v2.ShadowTopicState\x12P\n\x15partition_information\x18\x02 \x03(\x0b21.redpanda.core.admin.v2.TopicPartitionInformation"\xce\x01\n\x19TopicPartitionInformation\x12\x14\n\x0cpartition_id\x18\x01 \x01(\x03\x12!\n\x19source_last_stable_offset\x18\x02 \x01(\x03\x12\x1d\n\x15source_high_watermark\x18\x03 \x01(\x03\x12\x16\n\x0ehigh_watermark\x18\x04 \x01(\x03\x12A\n\x1dsource_last_updated_timestamp\x18\x05 \x01(\x0b2\x1a.google.protobuf.Timestamp*p\n\x0fShadowLinkState\x12!\n\x1dSHADOW_LINK_STATE_UNSPECIFIED\x10\x00\x12\x1c\n\x18SHADOW_LINK_STATE_ACTIVE\x10\x01\x12\x1c\n\x18SHADOW_LINK_STATE_PAUSED\x10\x02*w\n\x0eScramMechanism\x12\x1f\n\x1bSCRAM_MECHANISM_UNSPECIFIED\x10\x00\x12!\n\x1dSCRAM_MECHANISM_SCRAM_SHA_256\x10\x01\x12!\n\x1dSCRAM_MECHANISM_SCRAM_SHA_512\x10\x02*^\n\x0bPatternType\x12\x1c\n\x18PATTERN_TYPE_UNSPECIFIED\x10\x00\x12\x18\n\x14PATTERN_TYPE_LITERAL\x10\x01\x12\x17\n\x13PATTERN_TYPE_PREFIX\x10\x02*[\n\nFilterType\x12\x1b\n\x17FILTER_TYPE_UNSPECIFIED\x10\x00\x12\x17\n\x13FILTER_TYPE_INCLUDE\x10\x01\x12\x17\n\x13FILTER_TYPE_EXCLUDE\x10\x02*\xaa\x01\n\tTaskState\x12\x1a\n\x16TASK_STATE_UNSPECIFIED\x10\x00\x12\x15\n\x11TASK_STATE_ACTIVE\x10\x01\x12\x15\n\x11TASK_STATE_PAUSED\x10\x02\x12\x1f\n\x1bTASK_STATE_LINK_UNAVAILABLE\x10\x03\x12\x1a\n\x16TASK_STATE_NOT_RUNNING\x10\x04\x12\x16\n\x12TASK_STATE_FAULTED\x10\x05*\xa0\x02\n\x10ShadowTopicState\x12"\n\x1eSHADOW_TOPIC_STATE_UNSPECIFIED\x10\x00\x12\x1d\n\x19SHADOW_TOPIC_STATE_ACTIVE\x10\x01\x12\x1e\n\x1aSHADOW_TOPIC_STATE_FAULTED\x10\x02\x12\x1d\n\x19SHADOW_TOPIC_STATE_PAUSED\x10\x03\x12#\n\x1fSHADOW_TOPIC_STATE_FAILING_OVER\x10\x04\x12"\n\x1eSHADOW_TOPIC_STATE_FAILED_OVER\x10\x05\x12 \n\x1cSHADOW_TOPIC_STATE_PROMOTING\x10\x06\x12\x1f\n\x1bSHADOW_TOPIC_STATE_PROMOTED\x10\x072\xe1\x07\n\x11ShadowLinkService\x12}\n\x10CreateShadowLink\x12/.redpanda.core.admin.v2.CreateShadowLinkRequest\x1a0.redpanda.core.admin.v2.CreateShadowLinkResponse"\x06\xea\x92\x19\x02\x10\x03\x12}\n\x10DeleteShadowLink\x12/.redpanda.core.admin.v2.DeleteShadowLinkRequest\x1a0.redpanda.core.admin.v2.DeleteShadowLinkResponse"\x06\xea\x92\x19\x02\x10\x03\x12t\n\rGetShadowLink\x12,.redpanda.core.admin.v2.GetShadowLinkRequest\x1a-.redpanda.core.admin.v2.GetShadowLinkResponse"\x06\xea\x92\x19\x02\x10\x03\x12z\n\x0fListShadowLinks\x12..redpanda.core.admin.v2.ListShadowLinksRequest\x1a/.redpanda.core.admin.v2.ListShadowLinksResponse"\x06\xea\x92\x19\x02\x10\x03\x12}\n\x10UpdateShadowLink\x12/.redpanda.core.admin.v2.UpdateShadowLinkRequest\x1a0.redpanda.core.admin.v2.UpdateShadowLinkResponse"\x06\xea\x92\x19\x02\x10\x03\x12e\n\x08FailOver\x12\'.redpanda.core.admin.v2.FailOverRequest\x1a(.redpanda.core.admin.v2.FailOverResponse"\x06\xea\x92\x19\x02\x10\x03\x12w\n\x0eGetShadowTopic\x12-.redpanda.core.admin.v2.GetShadowTopicRequest\x1a..redpanda.core.admin.v2.GetShadowTopicResponse"\x06\xea\x92\x19\x02\x10\x03\x12}\n\x10ListShadowTopics\x12/.redpanda.core.admin.v2.ListShadowTopicsRequest\x1a0.redpanda.core.admin.v2.ListShadowTopicsResponse"\x06\xea\x92\x19\x02\x10\x03B\x10\xea\x92\x19\x0cproto::adminb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n.proto/redpanda/core/admin/v2/shadow_link.proto\x12\x16redpanda.core.admin.v2\x1a\'proto/redpanda/core/pbgen/options.proto\x1a#proto/redpanda/core/pbgen/rpc.proto\x1a\'proto/redpanda/core/common/v1/acl.proto\x1a\'proto/redpanda/core/common/v1/tls.proto\x1a\x1fgoogle/api/field_behavior.proto\x1a\x1bgoogle/api/field_info.proto\x1a\x19google/api/resource.proto\x1a\x1egoogle/protobuf/duration.proto\x1a\x1fgoogle/protobuf/timestamp.proto\x1a google/protobuf/field_mask.proto"\xc2\x01\n\nShadowLink\x12\x11\n\x04name\x18\x01 \x01(\tB\x03\xe0A\x02\x12\x18\n\x03uid\x18\x02 \x01(\tB\x0b\xe0A\x03\xe2\x8c\xcf\xd7\x08\x02\x08\x01\x12H\n\x0econfigurations\x18\x03 \x01(\x0b20.redpanda.core.admin.v2.ShadowLinkConfigurations\x12=\n\x06status\x18\x04 \x01(\x0b2(.redpanda.core.admin.v2.ShadowLinkStatusB\x03\xe0A\x03"\xc0\x01\n\x0bShadowTopic\x12\x11\n\x04name\x18\x01 \x01(\tB\x03\xe0A\x03\x12\x1d\n\x08topic_id\x18\x02 \x01(\tB\x0b\xe0A\x03\xe2\x8c\xcf\xd7\x08\x02\x08\x01\x12\x19\n\x11source_topic_name\x18\x03 \x01(\t\x12$\n\x0fsource_topic_id\x18\x04 \x01(\tB\x0b\xe0A\x03\xe2\x8c\xcf\xd7\x08\x02\x08\x01\x12>\n\x06status\x18\x05 \x01(\x0b2).redpanda.core.admin.v2.ShadowTopicStatusB\x03\xe0A\x03"R\n\x17CreateShadowLinkRequest\x127\n\x0bshadow_link\x18\x01 \x01(\x0b2".redpanda.core.admin.v2.ShadowLink"S\n\x18CreateShadowLinkResponse\x127\n\x0bshadow_link\x18\x01 \x01(\x0b2".redpanda.core.admin.v2.ShadowLink"p\n\x17DeleteShadowLinkRequest\x12F\n\x04name\x18\x01 \x01(\tB8\xe0A\x02\xfaA2\n0redpanda.core.admin.ShadowLinkService/ShadowLink\x12\r\n\x05force\x18\x02 \x01(\x08"\x1a\n\x18DeleteShadowLinkResponse"^\n\x14GetShadowLinkRequest\x12F\n\x04name\x18\x01 \x01(\tB8\xe0A\x02\xfaA2\n0redpanda.core.admin.ShadowLinkService/ShadowLink"P\n\x15GetShadowLinkResponse\x127\n\x0bshadow_link\x18\x01 \x01(\x0b2".redpanda.core.admin.v2.ShadowLink"\x18\n\x16ListShadowLinksRequest"S\n\x17ListShadowLinksResponse\x128\n\x0cshadow_links\x18\x01 \x03(\x0b2".redpanda.core.admin.v2.ShadowLink"\x83\x01\n\x17UpdateShadowLinkRequest\x127\n\x0bshadow_link\x18\x01 \x01(\x0b2".redpanda.core.admin.v2.ShadowLink\x12/\n\x0bupdate_mask\x18\x02 \x01(\x0b2\x1a.google.protobuf.FieldMask"S\n\x18UpdateShadowLinkResponse\x127\n\x0bshadow_link\x18\x01 \x01(\x0b2".redpanda.core.admin.v2.ShadowLink"y\n\x0fFailOverRequest\x12F\n\x04name\x18\x01 \x01(\tB8\xe0A\x02\xfaA2\n0redpanda.core.admin.ShadowLinkService/ShadowLink\x12\x1e\n\x11shadow_topic_name\x18\x02 \x01(\tB\x03\xe0A\x01"K\n\x10FailOverResponse\x127\n\x0bshadow_link\x18\x01 \x01(\x0b2".redpanda.core.admin.v2.ShadowLink"I\n\x15GetShadowTopicRequest\x12\x1d\n\x10shadow_link_name\x18\x01 \x01(\tB\x03\xe0A\x02\x12\x11\n\x04name\x18\x02 \x01(\tB\x03\xe0A\x02"S\n\x16GetShadowTopicResponse\x129\n\x0cshadow_topic\x18\x01 \x01(\x0b2#.redpanda.core.admin.v2.ShadowTopic"8\n\x17ListShadowTopicsRequest\x12\x1d\n\x10shadow_link_name\x18\x01 \x01(\tB\x03\xe0A\x02"V\n\x18ListShadowTopicsResponse\x12:\n\rshadow_topics\x18\x01 \x03(\x0b2#.redpanda.core.admin.v2.ShadowTopic"\xc0\x03\n\x18ShadowLinkConfigurations\x12G\n\x0eclient_options\x18\x01 \x01(\x0b2/.redpanda.core.admin.v2.ShadowLinkClientOptions\x12U\n\x1btopic_metadata_sync_options\x18\x02 \x01(\x0b20.redpanda.core.admin.v2.TopicMetadataSyncOptions\x12W\n\x1cconsumer_offset_sync_options\x18\x03 \x01(\x0b21.redpanda.core.admin.v2.ConsumerOffsetSyncOptions\x12R\n\x15security_sync_options\x18\x04 \x01(\x0b23.redpanda.core.admin.v2.SecuritySettingsSyncOptions\x12W\n\x1cschema_registry_sync_options\x18\x05 \x01(\x0b21.redpanda.core.admin.v2.SchemaRegistrySyncOptions"\xb4\x06\n\x17ShadowLinkClientOptions\x12\x1e\n\x11bootstrap_servers\x18\x01 \x03(\tB\x03\xe0A\x02\x12\x16\n\tclient_id\x18\x02 \x01(\tB\x03\xe0A\x03\x12\x19\n\x11source_cluster_id\x18\x03 \x01(\t\x12?\n\x0ctls_settings\x18\x04 \x01(\x0b2$.redpanda.core.common.v1.TLSSettingsH\x00\x88\x01\x01\x12^\n\x1cauthentication_configuration\x18\x05 \x01(\x0b23.redpanda.core.admin.v2.AuthenticationConfigurationH\x01\x88\x01\x01\x12\x1b\n\x13metadata_max_age_ms\x18\x06 \x01(\x05\x12*\n\x1deffective_metadata_max_age_ms\x18\x07 \x01(\x05B\x03\xe0A\x03\x12\x1d\n\x15connection_timeout_ms\x18\x08 \x01(\x05\x12,\n\x1feffective_connection_timeout_ms\x18\t \x01(\x05B\x03\xe0A\x03\x12\x18\n\x10retry_backoff_ms\x18\n \x01(\x05\x12\'\n\x1aeffective_retry_backoff_ms\x18\x0b \x01(\x05B\x03\xe0A\x03\x12\x19\n\x11fetch_wait_max_ms\x18\x0c \x01(\x05\x12(\n\x1beffective_fetch_wait_max_ms\x18\r \x01(\x05B\x03\xe0A\x03\x12\x17\n\x0ffetch_min_bytes\x18\x0e \x01(\x05\x12&\n\x19effective_fetch_min_bytes\x18\x0f \x01(\x05B\x03\xe0A\x03\x12\x17\n\x0ffetch_max_bytes\x18\x10 \x01(\x05\x12&\n\x19effective_fetch_max_bytes\x18\x11 \x01(\x05B\x03\xe0A\x03\x12!\n\x19fetch_partition_max_bytes\x18\x12 \x01(\x05\x120\n#effective_fetch_partition_max_bytes\x18\x13 \x01(\x05B\x03\xe0A\x03B\x0f\n\r_tls_settingsB\x1f\n\x1d_authentication_configuration"\xc6\x04\n\x18TopicMetadataSyncOptions\x12+\n\x08interval\x18\x01 \x01(\x0b2\x19.google.protobuf.Duration\x12:\n\x12effective_interval\x18\x02 \x01(\x0b2\x19.google.protobuf.DurationB\x03\xe0A\x03\x12L\n auto_create_shadow_topic_filters\x18\x03 \x03(\x0b2".redpanda.core.admin.v2.NameFilter\x12&\n\x1esynced_shadow_topic_properties\x18\x04 \x03(\t\x12\x17\n\x0fexclude_default\x18\x05 \x01(\x08\x12\\\n\x11start_at_earliest\x18\x06 \x01(\x0b2?.redpanda.core.admin.v2.TopicMetadataSyncOptions.EarliestOffsetH\x00\x12X\n\x0fstart_at_latest\x18\x07 \x01(\x0b2=.redpanda.core.admin.v2.TopicMetadataSyncOptions.LatestOffsetH\x00\x128\n\x12start_at_timestamp\x18\x08 \x01(\x0b2\x1a.google.protobuf.TimestampH\x00\x12\x0e\n\x06paused\x18\t \x01(\x08\x1a\x10\n\x0eEarliestOffset\x1a\x0e\n\x0cLatestOffsetB\x0e\n\x0cstart_offset"\xcf\x01\n\x19SchemaRegistrySyncOptions\x12s\n\x1cshadow_schema_registry_topic\x18\x01 \x01(\x0b2K.redpanda.core.admin.v2.SchemaRegistrySyncOptions.ShadowSchemaRegistryTopicH\x00\x1a\x1b\n\x19ShadowSchemaRegistryTopicB \n\x1eschema_registry_shadowing_mode"\xcf\x01\n\x19ConsumerOffsetSyncOptions\x12+\n\x08interval\x18\x01 \x01(\x0b2\x19.google.protobuf.Duration\x12:\n\x12effective_interval\x18\x02 \x01(\x0b2\x19.google.protobuf.DurationB\x03\xe0A\x03\x12\x0e\n\x06paused\x18\x03 \x01(\x08\x129\n\rgroup_filters\x18\x04 \x03(\x0b2".redpanda.core.admin.v2.NameFilter"\xce\x01\n\x1bSecuritySettingsSyncOptions\x12+\n\x08interval\x18\x01 \x01(\x0b2\x19.google.protobuf.Duration\x12:\n\x12effective_interval\x18\x02 \x01(\x0b2\x19.google.protobuf.DurationB\x03\xe0A\x03\x12\x0e\n\x06paused\x18\x03 \x01(\x08\x126\n\x0bacl_filters\x18\x04 \x03(\x0b2!.redpanda.core.admin.v2.ACLFilter"\xb7\x01\n\x1bAuthenticationConfiguration\x12B\n\x13scram_configuration\x18\x01 \x01(\x0b2#.redpanda.core.admin.v2.ScramConfigH\x00\x12B\n\x13plain_configuration\x18\x02 \x01(\x0b2#.redpanda.core.admin.v2.PlainConfigH\x00B\x10\n\x0eauthentication"\x8b\x01\n\x0bPlainConfig\x12\x10\n\x08username\x18\x01 \x01(\t\x12\x15\n\x08password\x18\x02 \x01(\tB\x03\xe0A\x04\x12\x19\n\x0cpassword_set\x18\x03 \x01(\x08B\x03\xe0A\x03\x128\n\x0fpassword_set_at\x18\x04 \x01(\x0b2\x1a.google.protobuf.TimestampB\x03\xe0A\x03"\xcc\x01\n\x0bScramConfig\x12\x10\n\x08username\x18\x01 \x01(\t\x12\x15\n\x08password\x18\x02 \x01(\tB\x03\xe0A\x04\x12\x19\n\x0cpassword_set\x18\x03 \x01(\x08B\x03\xe0A\x03\x128\n\x0fpassword_set_at\x18\x04 \x01(\x0b2\x1a.google.protobuf.TimestampB\x03\xe0A\x03\x12?\n\x0fscram_mechanism\x18\x05 \x01(\x0e2&.redpanda.core.admin.v2.ScramMechanism"\x8e\x01\n\nNameFilter\x129\n\x0cpattern_type\x18\x01 \x01(\x0e2#.redpanda.core.admin.v2.PatternType\x127\n\x0bfilter_type\x18\x02 \x01(\x0e2".redpanda.core.admin.v2.FilterType\x12\x0c\n\x04name\x18\x03 \x01(\t"\x8f\x01\n\tACLFilter\x12B\n\x0fresource_filter\x18\x01 \x01(\x0b2).redpanda.core.admin.v2.ACLResourceFilter\x12>\n\raccess_filter\x18\x02 \x01(\x0b2\'.redpanda.core.admin.v2.ACLAccessFilter"\x99\x01\n\x11ACLResourceFilter\x12;\n\rresource_type\x18\x01 \x01(\x0e2$.redpanda.core.common.v1.ACLResource\x129\n\x0cpattern_type\x18\x02 \x01(\x0e2#.redpanda.core.common.v1.ACLPattern\x12\x0c\n\x04name\x18\x03 \x01(\t"\xb1\x01\n\x0fACLAccessFilter\x12\x11\n\tprincipal\x18\x01 \x01(\t\x128\n\toperation\x18\x02 \x01(\x0e2%.redpanda.core.common.v1.ACLOperation\x12C\n\x0fpermission_type\x18\x03 \x01(\x0e2*.redpanda.core.common.v1.ACLPermissionType\x12\x0c\n\x04host\x18\x04 \x01(\t"\xf3\x01\n\x10ShadowLinkStatus\x126\n\x05state\x18\x01 \x01(\x0e2\'.redpanda.core.admin.v2.ShadowLinkState\x12C\n\rtask_statuses\x18\x02 \x03(\x0b2,.redpanda.core.admin.v2.ShadowLinkTaskStatus\x12:\n\rshadow_topics\x18\x03 \x03(\x0b2#.redpanda.core.admin.v2.ShadowTopic\x12&\n\x1esynced_shadow_topic_properties\x18\x04 \x03(\t"\x88\x01\n\x14ShadowLinkTaskStatus\x12\x0c\n\x04name\x18\x01 \x01(\t\x120\n\x05state\x18\x02 \x01(\x0e2!.redpanda.core.admin.v2.TaskState\x12\x0e\n\x06reason\x18\x03 \x01(\t\x12\x11\n\tbroker_id\x18\x04 \x01(\x05\x12\r\n\x05shard\x18\x05 \x01(\x05"\x9e\x01\n\x11ShadowTopicStatus\x127\n\x05state\x18\x01 \x01(\x0e2(.redpanda.core.admin.v2.ShadowTopicState\x12P\n\x15partition_information\x18\x02 \x03(\x0b21.redpanda.core.admin.v2.TopicPartitionInformation"\xce\x01\n\x19TopicPartitionInformation\x12\x14\n\x0cpartition_id\x18\x01 \x01(\x03\x12!\n\x19source_last_stable_offset\x18\x02 \x01(\x03\x12\x1d\n\x15source_high_watermark\x18\x03 \x01(\x03\x12\x16\n\x0ehigh_watermark\x18\x04 \x01(\x03\x12A\n\x1dsource_last_updated_timestamp\x18\x05 \x01(\x0b2\x1a.google.protobuf.Timestamp*p\n\x0fShadowLinkState\x12!\n\x1dSHADOW_LINK_STATE_UNSPECIFIED\x10\x00\x12\x1c\n\x18SHADOW_LINK_STATE_ACTIVE\x10\x01\x12\x1c\n\x18SHADOW_LINK_STATE_PAUSED\x10\x02*w\n\x0eScramMechanism\x12\x1f\n\x1bSCRAM_MECHANISM_UNSPECIFIED\x10\x00\x12!\n\x1dSCRAM_MECHANISM_SCRAM_SHA_256\x10\x01\x12!\n\x1dSCRAM_MECHANISM_SCRAM_SHA_512\x10\x02*^\n\x0bPatternType\x12\x1c\n\x18PATTERN_TYPE_UNSPECIFIED\x10\x00\x12\x18\n\x14PATTERN_TYPE_LITERAL\x10\x01\x12\x17\n\x13PATTERN_TYPE_PREFIX\x10\x02*[\n\nFilterType\x12\x1b\n\x17FILTER_TYPE_UNSPECIFIED\x10\x00\x12\x17\n\x13FILTER_TYPE_INCLUDE\x10\x01\x12\x17\n\x13FILTER_TYPE_EXCLUDE\x10\x02*\xaa\x01\n\tTaskState\x12\x1a\n\x16TASK_STATE_UNSPECIFIED\x10\x00\x12\x15\n\x11TASK_STATE_ACTIVE\x10\x01\x12\x15\n\x11TASK_STATE_PAUSED\x10\x02\x12\x1f\n\x1bTASK_STATE_LINK_UNAVAILABLE\x10\x03\x12\x1a\n\x16TASK_STATE_NOT_RUNNING\x10\x04\x12\x16\n\x12TASK_STATE_FAULTED\x10\x05*\xa0\x02\n\x10ShadowTopicState\x12"\n\x1eSHADOW_TOPIC_STATE_UNSPECIFIED\x10\x00\x12\x1d\n\x19SHADOW_TOPIC_STATE_ACTIVE\x10\x01\x12\x1e\n\x1aSHADOW_TOPIC_STATE_FAULTED\x10\x02\x12\x1d\n\x19SHADOW_TOPIC_STATE_PAUSED\x10\x03\x12#\n\x1fSHADOW_TOPIC_STATE_FAILING_OVER\x10\x04\x12"\n\x1eSHADOW_TOPIC_STATE_FAILED_OVER\x10\x05\x12 \n\x1cSHADOW_TOPIC_STATE_PROMOTING\x10\x06\x12\x1f\n\x1bSHADOW_TOPIC_STATE_PROMOTED\x10\x072\xe1\x07\n\x11ShadowLinkService\x12}\n\x10CreateShadowLink\x12/.redpanda.core.admin.v2.CreateShadowLinkRequest\x1a0.redpanda.core.admin.v2.CreateShadowLinkResponse"\x06\xea\x92\x19\x02\x10\x03\x12}\n\x10DeleteShadowLink\x12/.redpanda.core.admin.v2.DeleteShadowLinkRequest\x1a0.redpanda.core.admin.v2.DeleteShadowLinkResponse"\x06\xea\x92\x19\x02\x10\x03\x12t\n\rGetShadowLink\x12,.redpanda.core.admin.v2.GetShadowLinkRequest\x1a-.redpanda.core.admin.v2.GetShadowLinkResponse"\x06\xea\x92\x19\x02\x10\x03\x12z\n\x0fListShadowLinks\x12..redpanda.core.admin.v2.ListShadowLinksRequest\x1a/.redpanda.core.admin.v2.ListShadowLinksResponse"\x06\xea\x92\x19\x02\x10\x03\x12}\n\x10UpdateShadowLink\x12/.redpanda.core.admin.v2.UpdateShadowLinkRequest\x1a0.redpanda.core.admin.v2.UpdateShadowLinkResponse"\x06\xea\x92\x19\x02\x10\x03\x12e\n\x08FailOver\x12\'.redpanda.core.admin.v2.FailOverRequest\x1a(.redpanda.core.admin.v2.FailOverResponse"\x06\xea\x92\x19\x02\x10\x03\x12w\n\x0eGetShadowTopic\x12-.redpanda.core.admin.v2.GetShadowTopicRequest\x1a..redpanda.core.admin.v2.GetShadowTopicResponse"\x06\xea\x92\x19\x02\x10\x03\x12}\n\x10ListShadowTopics\x12/.redpanda.core.admin.v2.ListShadowTopicsRequest\x1a0.redpanda.core.admin.v2.ListShadowTopicsResponse"\x06\xea\x92\x19\x02\x10\x03B\x10\xea\x92\x19\x0cproto::adminb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) _builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'proto.redpanda.core.admin.v2.shadow_link_pb2', _globals) @@ -75,6 +75,12 @@ _globals['_CONSUMEROFFSETSYNCOPTIONS'].fields_by_name['effective_interval']._serialized_options = b'\xe0A\x03' _globals['_SECURITYSETTINGSSYNCOPTIONS'].fields_by_name['effective_interval']._loaded_options = None _globals['_SECURITYSETTINGSSYNCOPTIONS'].fields_by_name['effective_interval']._serialized_options = b'\xe0A\x03' + _globals['_PLAINCONFIG'].fields_by_name['password']._loaded_options = None + _globals['_PLAINCONFIG'].fields_by_name['password']._serialized_options = b'\xe0A\x04' + _globals['_PLAINCONFIG'].fields_by_name['password_set']._loaded_options = None + _globals['_PLAINCONFIG'].fields_by_name['password_set']._serialized_options = b'\xe0A\x03' + _globals['_PLAINCONFIG'].fields_by_name['password_set_at']._loaded_options = None + _globals['_PLAINCONFIG'].fields_by_name['password_set_at']._serialized_options = b'\xe0A\x03' _globals['_SCRAMCONFIG'].fields_by_name['password']._loaded_options = None _globals['_SCRAMCONFIG'].fields_by_name['password']._serialized_options = b'\xe0A\x04' _globals['_SCRAMCONFIG'].fields_by_name['password_set']._loaded_options = None @@ -97,18 +103,18 @@ _globals['_SHADOWLINKSERVICE'].methods_by_name['GetShadowTopic']._serialized_options = b'\xea\x92\x19\x02\x10\x03' _globals['_SHADOWLINKSERVICE'].methods_by_name['ListShadowTopics']._loaded_options = None _globals['_SHADOWLINKSERVICE'].methods_by_name['ListShadowTopics']._serialized_options = b'\xea\x92\x19\x02\x10\x03' - _globals['_SHADOWLINKSTATE']._serialized_start = 6333 - _globals['_SHADOWLINKSTATE']._serialized_end = 6445 - _globals['_SCRAMMECHANISM']._serialized_start = 6447 - _globals['_SCRAMMECHANISM']._serialized_end = 6566 - _globals['_PATTERNTYPE']._serialized_start = 6568 - _globals['_PATTERNTYPE']._serialized_end = 6662 - _globals['_FILTERTYPE']._serialized_start = 6664 - _globals['_FILTERTYPE']._serialized_end = 6755 - _globals['_TASKSTATE']._serialized_start = 6758 - _globals['_TASKSTATE']._serialized_end = 6928 - _globals['_SHADOWTOPICSTATE']._serialized_start = 6931 - _globals['_SHADOWTOPICSTATE']._serialized_end = 7219 + _globals['_SHADOWLINKSTATE']._serialized_start = 6544 + _globals['_SHADOWLINKSTATE']._serialized_end = 6656 + _globals['_SCRAMMECHANISM']._serialized_start = 6658 + _globals['_SCRAMMECHANISM']._serialized_end = 6777 + _globals['_PATTERNTYPE']._serialized_start = 6779 + _globals['_PATTERNTYPE']._serialized_end = 6873 + _globals['_FILTERTYPE']._serialized_start = 6875 + _globals['_FILTERTYPE']._serialized_end = 6966 + _globals['_TASKSTATE']._serialized_start = 6969 + _globals['_TASKSTATE']._serialized_end = 7139 + _globals['_SHADOWTOPICSTATE']._serialized_start = 7142 + _globals['_SHADOWTOPICSTATE']._serialized_end = 7430 _globals['_SHADOWLINK']._serialized_start = 423 _globals['_SHADOWLINK']._serialized_end = 617 _globals['_SHADOWTOPIC']._serialized_start = 620 @@ -163,25 +169,27 @@ _globals['_CONSUMEROFFSETSYNCOPTIONS']._serialized_end = 4416 _globals['_SECURITYSETTINGSSYNCOPTIONS']._serialized_start = 4419 _globals['_SECURITYSETTINGSSYNCOPTIONS']._serialized_end = 4625 - _globals['_AUTHENTICATIONCONFIGURATION']._serialized_start = 4627 - _globals['_AUTHENTICATIONCONFIGURATION']._serialized_end = 4742 - _globals['_SCRAMCONFIG']._serialized_start = 4745 - _globals['_SCRAMCONFIG']._serialized_end = 4949 - _globals['_NAMEFILTER']._serialized_start = 4952 - _globals['_NAMEFILTER']._serialized_end = 5094 - _globals['_ACLFILTER']._serialized_start = 5097 - _globals['_ACLFILTER']._serialized_end = 5240 - _globals['_ACLRESOURCEFILTER']._serialized_start = 5243 - _globals['_ACLRESOURCEFILTER']._serialized_end = 5396 - _globals['_ACLACCESSFILTER']._serialized_start = 5399 - _globals['_ACLACCESSFILTER']._serialized_end = 5576 - _globals['_SHADOWLINKSTATUS']._serialized_start = 5579 - _globals['_SHADOWLINKSTATUS']._serialized_end = 5822 - _globals['_SHADOWLINKTASKSTATUS']._serialized_start = 5825 - _globals['_SHADOWLINKTASKSTATUS']._serialized_end = 5961 - _globals['_SHADOWTOPICSTATUS']._serialized_start = 5964 - _globals['_SHADOWTOPICSTATUS']._serialized_end = 6122 - _globals['_TOPICPARTITIONINFORMATION']._serialized_start = 6125 - _globals['_TOPICPARTITIONINFORMATION']._serialized_end = 6331 - _globals['_SHADOWLINKSERVICE']._serialized_start = 7222 - _globals['_SHADOWLINKSERVICE']._serialized_end = 8215 \ No newline at end of file + _globals['_AUTHENTICATIONCONFIGURATION']._serialized_start = 4628 + _globals['_AUTHENTICATIONCONFIGURATION']._serialized_end = 4811 + _globals['_PLAINCONFIG']._serialized_start = 4814 + _globals['_PLAINCONFIG']._serialized_end = 4953 + _globals['_SCRAMCONFIG']._serialized_start = 4956 + _globals['_SCRAMCONFIG']._serialized_end = 5160 + _globals['_NAMEFILTER']._serialized_start = 5163 + _globals['_NAMEFILTER']._serialized_end = 5305 + _globals['_ACLFILTER']._serialized_start = 5308 + _globals['_ACLFILTER']._serialized_end = 5451 + _globals['_ACLRESOURCEFILTER']._serialized_start = 5454 + _globals['_ACLRESOURCEFILTER']._serialized_end = 5607 + _globals['_ACLACCESSFILTER']._serialized_start = 5610 + _globals['_ACLACCESSFILTER']._serialized_end = 5787 + _globals['_SHADOWLINKSTATUS']._serialized_start = 5790 + _globals['_SHADOWLINKSTATUS']._serialized_end = 6033 + _globals['_SHADOWLINKTASKSTATUS']._serialized_start = 6036 + _globals['_SHADOWLINKTASKSTATUS']._serialized_end = 6172 + _globals['_SHADOWTOPICSTATUS']._serialized_start = 6175 + _globals['_SHADOWTOPICSTATUS']._serialized_end = 6333 + _globals['_TOPICPARTITIONINFORMATION']._serialized_start = 6336 + _globals['_TOPICPARTITIONINFORMATION']._serialized_end = 6542 + _globals['_SHADOWLINKSERVICE']._serialized_start = 7433 + _globals['_SHADOWLINKSERVICE']._serialized_end = 8426 \ No newline at end of file diff --git a/tests/rptest/clients/admin/proto/redpanda/core/admin/v2/shadow_link_pb2.pyi b/tests/rptest/clients/admin/proto/redpanda/core/admin/v2/shadow_link_pb2.pyi index ddc9d87c1c088..ef2fdfb70a6ca 100644 --- a/tests/rptest/clients/admin/proto/redpanda/core/admin/v2/shadow_link_pb2.pyi +++ b/tests/rptest/clients/admin/proto/redpanda/core/admin/v2/shadow_link_pb2.pyi @@ -905,29 +905,66 @@ Global___SecuritySettingsSyncOptions: typing_extensions.TypeAlias = SecuritySett @typing.final class AuthenticationConfiguration(google.protobuf.message.Message): - """Authentication config. Currently only supporting SASL/SCRAM, - however made as a oneof for expansion + """Authentication config. Supports: + * SASL/SCRAM + * SASL/PLAIN """ DESCRIPTOR: google.protobuf.descriptor.Descriptor SCRAM_CONFIGURATION_FIELD_NUMBER: builtins.int + PLAIN_CONFIGURATION_FIELD_NUMBER: builtins.int @property def scram_configuration(self) -> Global___ScramConfig: """SASL/SCRAM configuration""" - def __init__(self, *, scram_configuration: Global___ScramConfig | None=...) -> None: + @property + def plain_configuration(self) -> Global___PlainConfig: + """SASL/PLAIN configuration""" + + def __init__(self, *, scram_configuration: Global___ScramConfig | None=..., plain_configuration: Global___PlainConfig | None=...) -> None: ... - def HasField(self, field_name: typing.Literal['authentication', b'authentication', 'scram_configuration', b'scram_configuration']) -> builtins.bool: + def HasField(self, field_name: typing.Literal['authentication', b'authentication', 'plain_configuration', b'plain_configuration', 'scram_configuration', b'scram_configuration']) -> builtins.bool: ... - def ClearField(self, field_name: typing.Literal['authentication', b'authentication', 'scram_configuration', b'scram_configuration']) -> None: + def ClearField(self, field_name: typing.Literal['authentication', b'authentication', 'plain_configuration', b'plain_configuration', 'scram_configuration', b'scram_configuration']) -> None: ... - def WhichOneof(self, oneof_group: typing.Literal['authentication', b'authentication']) -> typing.Literal['scram_configuration'] | None: + def WhichOneof(self, oneof_group: typing.Literal['authentication', b'authentication']) -> typing.Literal['scram_configuration', 'plain_configuration'] | None: ... Global___AuthenticationConfiguration: typing_extensions.TypeAlias = AuthenticationConfiguration +@typing.final +class PlainConfig(google.protobuf.message.Message): + """PLAIN settings""" + DESCRIPTOR: google.protobuf.descriptor.Descriptor + USERNAME_FIELD_NUMBER: builtins.int + PASSWORD_FIELD_NUMBER: builtins.int + PASSWORD_SET_FIELD_NUMBER: builtins.int + PASSWORD_SET_AT_FIELD_NUMBER: builtins.int + username: builtins.str + 'PLAIN username' + password: builtins.str + 'Password' + password_set: builtins.bool + 'Indicates that the password has been set' + + @property + def password_set_at(self) -> google.protobuf.timestamp_pb2.Timestamp: + """Timestamp of when the password was last set - only valid if password_set + is true + """ + + def __init__(self, *, username: builtins.str=..., password: builtins.str=..., password_set: builtins.bool=..., password_set_at: google.protobuf.timestamp_pb2.Timestamp | None=...) -> None: + ... + + def HasField(self, field_name: typing.Literal['password_set_at', b'password_set_at']) -> builtins.bool: + ... + + def ClearField(self, field_name: typing.Literal['password', b'password', 'password_set', b'password_set', 'password_set_at', b'password_set_at', 'username', b'username']) -> None: + ... +Global___PlainConfig: typing_extensions.TypeAlias = PlainConfig + @typing.final class ScramConfig(google.protobuf.message.Message): """SCRAM settings""" diff --git a/tests/rptest/tests/cluster_linking_topic_syncing_test.py b/tests/rptest/tests/cluster_linking_topic_syncing_test.py index 18f7c2af9c0b3..ddb33395c65b2 100644 --- a/tests/rptest/tests/cluster_linking_topic_syncing_test.py +++ b/tests/rptest/tests/cluster_linking_topic_syncing_test.py @@ -432,6 +432,88 @@ def setUp(self): ) +class ClusterLinkingTopicSyncingWithPlain(ClusterLinkingTopicSyncingTestBase): + """ + Run the same battery of tests with PLAIN + """ + + def __init__(self, test_context, *args, **kwargs): + security = SecurityConfig() + security.enable_sasl = True + security.sasl_mechanisms = ["SCRAM", "PLAIN"] + secondary_args: SecondaryClusterArgs = SecondaryClusterArgs(security=security) + self.cluster_link_user = "cluster-link-user" + self.cluster_link_password = "cluster-link-password" + + super().__init__( + test_context=test_context, + secondary_cluster_args=secondary_args, + *args, + **kwargs, + ) + + def validate_created_link(self, shadow_link: shadow_link_pb2.ShadowLink) -> None: + now = time.time() + assert ( + shadow_link.configurations.client_options.authentication_configuration.WhichOneof( + "authentication" + ) + == "plain_configuration" + ), ( + f"Expected 'plain_configuration' but got {shadow_link.configurations.client_options.authentication_configuration.WhichOneof('authentication')}" + ) + + plain_config = shadow_link.configurations.client_options.authentication_configuration.plain_configuration + assert plain_config.password_set, "Password not set in plain configuration" + assert plain_config.password == "", "Password should not be set" + assert plain_config.username == self.cluster_link_user, ( + f"Username does not match: {plain_config.username} != {self.cluster_link_user}" + ) + assert ( + plain_config.password_set_at != google.protobuf.timestamp_pb2.Timestamp() + ), "Password set time not set" + + assert now - 5 <= plain_config.password_set_at.seconds <= now + 5, ( + f"Password set time not recent: {plain_config.password_set_at.seconds} vs {now}" + ) + + def add_credentials_to_link( + self, shadow_link: shadow_link_pb2.ShadowLink + ) -> shadow_link_pb2.ShadowLink: + self.logger.debug( + f"Adding PLAIN credentials for user {self.cluster_link_user} to link" + ) + + shadow_link.configurations.client_options.authentication_configuration.plain_configuration.CopyFrom( + shadow_link_pb2.PlainConfig( + username=self.cluster_link_user, password=self.cluster_link_password + ) + ) + return shadow_link + + def get_source_cluster_rpk(self) -> RpkTool: + return RpkTool( + self.source_cluster.service, + username=self.redpanda.SUPERUSER_CREDENTIALS.username, + password=self.redpanda.SUPERUSER_CREDENTIALS.password, + sasl_mechanism=self.redpanda.SUPERUSER_CREDENTIALS.mechanism, + ) + + def setUp(self): + super().setUp() + self.get_source_cluster_rpk().sasl_create_user( + self.cluster_link_user, self.cluster_link_password + ) + self.source_cluster.service.set_cluster_config( + { + "superusers": [ + self.redpanda.SUPERUSER_CREDENTIALS.username, + self.cluster_link_user, + ] + } + ) + + class ClusterLinkingTopicSyncingWithTlsFiles(ClusterLinkingTopicSyncingTestBase): """ Runs the base tests with TLS enabled on both endpoints