From 479beda1e3bc480325cf6c7999b3dc67b00f547e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gell=C3=A9rt=20Peresztegi-Nagy?= Date: Fri, 18 Oct 2024 14:43:53 +0100 Subject: [PATCH 001/229] sr/test: refactor avro verbose compat test Make it easier to extend by simplifying the definition of a test case. --- .../test/compatibility_avro.cc | 55 +++++++++++-------- 1 file changed, 33 insertions(+), 22 deletions(-) diff --git a/src/v/pandaproxy/schema_registry/test/compatibility_avro.cc b/src/v/pandaproxy/schema_registry/test/compatibility_avro.cc index 2c92bd5d1ec3..f9af35ca86ef 100644 --- a/src/v/pandaproxy/schema_registry/test/compatibility_avro.cc +++ b/src/v/pandaproxy/schema_registry/test/compatibility_avro.cc @@ -325,9 +325,7 @@ SEASTAR_THREAD_TEST_CASE(test_avro_alias_resolution_stopgap) { namespace { -const auto schema_old = pps::sanitize_avro_schema_definition( - { - R"({ +const auto schema_old = R"({ "type": "record", "name": "myrecord", "fields": [ @@ -392,13 +390,9 @@ const auto schema_old = pps::sanitize_avro_schema_definition( } } ] -})", - pps::schema_type::avro}) - .value(); +})"; -const auto schema_new = pps::sanitize_avro_schema_definition( - { - R"({ +const auto schema_new = R"({ "type": "record", "name": "myrecord", "fields": [ @@ -464,9 +458,7 @@ const auto schema_new = pps::sanitize_avro_schema_definition( } } ] -})", - pps::schema_type::avro}) - .value(); +})"; using incompatibility = pps::avro_incompatibility; @@ -534,16 +526,22 @@ const absl::flat_hash_set backward_expected{ "expected: someEnum1 (alias resolution is not yet fully supported)"}, }; -const auto compat_data = std::to_array>({ +struct compat_test_case { + std::string reader; + std::string writer; + absl::flat_hash_set expected; +}; + +const auto compat_data = std::to_array({ { - schema_old.share(), - schema_new.share(), - forward_expected, + .reader=schema_old, + .writer=schema_new, + .expected=forward_expected, }, { - schema_new.share(), - schema_old.share(), - backward_expected, + .reader=schema_new, + .writer=schema_old, + .expected=backward_expected, }, }); @@ -555,13 +553,26 @@ std::string format_set(const absl::flat_hash_set& d) { SEASTAR_THREAD_TEST_CASE(test_avro_compat_messages) { for (const auto& cd : compat_data) { - auto compat = check_compatible_verbose(cd.reader, cd.writer); + auto compat = check_compatible_verbose( + pps::sanitize_avro_schema_definition( + {cd.reader, pps::schema_type::avro}) + .value(), + pps::sanitize_avro_schema_definition( + {cd.writer, pps::schema_type::avro}) + .value()); + + pps::raw_compatibility_result raw; + absl::c_for_each(cd.expected, [&raw](auto e) { + raw.emplace(std::move(e)); + }); + auto exp_compat = std::move(raw)(pps::verbose::yes); + absl::flat_hash_set errs{ compat.messages.begin(), compat.messages.end()}; absl::flat_hash_set expected{ - cd.expected.messages.begin(), cd.expected.messages.end()}; + exp_compat.messages.begin(), exp_compat.messages.end()}; - BOOST_CHECK(!compat.is_compat); + BOOST_CHECK_EQUAL(compat.is_compat, exp_compat.is_compat); BOOST_CHECK_EQUAL(errs.size(), expected.size()); BOOST_REQUIRE_MESSAGE( errs == expected, From 6aabcfa22777ee230419933a521455bb7957e740 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gell=C3=A9rt=20Peresztegi-Nagy?= Date: Fri, 18 Oct 2024 13:38:41 +0100 Subject: [PATCH 002/229] sr/test: add more tests for avro non-default field This adds more tests for the avro incompatibility `reader_field_missing_default_value`. All of these tests are already passing, while the tests added in the following commit show the change in behaviour. --- .../test/compatibility_avro.cc | 106 ++++++++++++++++++ 1 file changed, 106 insertions(+) diff --git a/src/v/pandaproxy/schema_registry/test/compatibility_avro.cc b/src/v/pandaproxy/schema_registry/test/compatibility_avro.cc index f9af35ca86ef..7120efa3a39c 100644 --- a/src/v/pandaproxy/schema_registry/test/compatibility_avro.cc +++ b/src/v/pandaproxy/schema_registry/test/compatibility_avro.cc @@ -543,6 +543,112 @@ const auto compat_data = std::to_array({ .writer=schema_old, .expected=backward_expected, }, + { + .reader=R"({ + "type": "record", + "name": "car", + "fields": [ + { + "name": "color", + "type": ["string", "null"] + } + ] + })", + .writer=R"({ + "type": "record", + "name": "car", + "fields": [] + })", + .expected={ + {"/fields/0", + incompatibility::Type::reader_field_missing_default_value, + "color"}, + }, + }, + { + .reader = R"({ + "type": "record", + "name": "car", + "fields": [ + { + "name": "color", + "type": ["null", "string"], + "default": null + } + ] + })", + .writer = R"({ + "type": "record", + "name": "car", + "fields": [] + })", + .expected = {}, + }, + { + .reader=R"({ + "type": "record", + "name": "car", + "fields": [ + { + "name": "color", + "type": "null" + } + ] + })", + .writer=R"({ + "type": "record", + "name": "car", + "fields": [] + })", + .expected={ + {"/fields/0", + incompatibility::Type::reader_field_missing_default_value, + "color"}, + }, + }, + { + .reader=R"({ + "type": "record", + "name": "car", + "fields": [ + { + "name": "color", + "type": "string", + "default": "somevalue" + } + ] + })", + .writer=R"({ + "type": "record", + "name": "car", + "fields": [] + })", + .expected={}, + },{ + .reader=R"({ + "type": "record", + "name": "car", + "fields": [ + { + "name": "color", + "type": "null", + "default": null + } + ] + })", + .writer=R"({ + "type": "record", + "name": "car", + "fields": [] + })", + .expected={ + // Note: this is overly restrictive for null-type fields with null defaults. + // This is because the Avro API is not expressive enough to differentiate the two. + {"/fields/0", + incompatibility::Type::reader_field_missing_default_value, + "color"}, + }, + }, }); std::string format_set(const absl::flat_hash_set& d) { From 3866b1c09faa0a6fd3d8a525589c27fdd55b1a95 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gell=C3=A9rt=20Peresztegi-Nagy?= Date: Wed, 6 Nov 2024 17:35:02 +0000 Subject: [PATCH 003/229] sr/avro: fix missing default compat check for null This fixes a bug in the avro compatibility check for fields present in the reader but missing from the writer. In this case the writer and the reader are compatible only if the reader field has a specified default value. The Avro library sets the default value to `GenericDatum()` which has type `AVRO_NULL` when the default is not specified. This is what we are trying to detect in this check. However, the check was confusing an explicit null default for a union-type as a missing value. The former is represented as `GenericDatum(Union(Null))` and because of the way `GenericDatum::type()` delegates to the type of the first union leaf value when `GenericDatum` contains a union, the check thought that the value was missing, when it was in fact specified as null. To solve for the above, we also check that the type is not a union type to be able to detect when the default is not set. (The same check is used in the Avro library to detect unset default values, for example, when printing an Avro schema as JSON.) The caveat is that we are still overly restrictive for fields of type null, because in that case, the default value will be `GenericDatum()` regardless of whether the default value was explicitly set to null or not. This is a limitation of the Avro library's API and could only be fixed by making breaking changes to the Avro library's API. Also note that the checks against `reader.leafAt(...)` have been removed. They were attempting to compare the type of the field against the type of the default value. This is not necessary because this is ensured by the Avro library's built-in schema validation done while parsing the Avro schema. --- src/v/pandaproxy/schema_registry/avro.cc | 35 ++++++++++++------- .../test/compatibility_avro.cc | 22 ++++++++++++ 2 files changed, 45 insertions(+), 12 deletions(-) diff --git a/src/v/pandaproxy/schema_registry/avro.cc b/src/v/pandaproxy/schema_registry/avro.cc index 9b31f3629bc5..1850ddeb3cb2 100644 --- a/src/v/pandaproxy/schema_registry/avro.cc +++ b/src/v/pandaproxy/schema_registry/avro.cc @@ -116,18 +116,29 @@ avro_compatibility_result check_compatible( *reader.leafAt(int(r_idx)), *writer.leafAt(int(w_idx)), fields_p / std::to_string(r_idx) / "type")); - } else if ( - reader.defaultValueAt(int(r_idx)).type() == avro::AVRO_NULL) { - // if the reader's record schema has a field with no default - // value, and writer's schema does not have a field with the - // same name, an error is signalled. - - // For union, the default must correspond to the first type. - // The default may be null. - const auto& r_leaf = reader.leafAt(int(r_idx)); - if ( - r_leaf->type() != avro::Type::AVRO_UNION - || r_leaf->leafAt(0)->type() != avro::Type::AVRO_NULL) { + } else { + // if the reader's record schema has a field with no + // default value, and writer's schema does not have a + // field with the same name, an error is signalled. + // For union, the default must correspond to the first + // type. + const auto& def = reader.defaultValueAt(int(r_idx)); + + // Note: this code is overly restrictive for null-type + // fields with null defaults. This is because the Avro API + // is not expressive enough to differentiate the two. + // Union type field's default set to null: + // def=GenericDatum(Union(Null)) + // Union type field's default missing: + // def=GenericDatum(Null) + // Null type field's default set to null: + // def=GenericDatum(Null) + // Null type field's default missing: + // def=GenericDatum(Null) + auto default_unset = !def.isUnion() + && def.type() == avro::AVRO_NULL; + + if (default_unset) { compat_result.emplace( fields_p / std::to_string(r_idx), avro_incompatibility::Type:: diff --git a/src/v/pandaproxy/schema_registry/test/compatibility_avro.cc b/src/v/pandaproxy/schema_registry/test/compatibility_avro.cc index 7120efa3a39c..6c927ed866d1 100644 --- a/src/v/pandaproxy/schema_registry/test/compatibility_avro.cc +++ b/src/v/pandaproxy/schema_registry/test/compatibility_avro.cc @@ -543,6 +543,28 @@ const auto compat_data = std::to_array({ .writer=schema_old, .expected=backward_expected, }, + { + .reader=R"({ + "type": "record", + "name": "car", + "fields": [ + { + "name": "color", + "type": ["null", "string"] + } + ] + })", + .writer=R"({ + "type": "record", + "name": "car", + "fields": [] + })", + .expected={ + {"/fields/0", + incompatibility::Type::reader_field_missing_default_value, + "color"}, + }, + }, { .reader=R"({ "type": "record", From 94a0b3a01d508c0733420134d93262d13192a643 Mon Sep 17 00:00:00 2001 From: Chris Lee Date: Mon, 18 Nov 2024 12:56:27 -0500 Subject: [PATCH 004/229] [rptest] Auto-retry in RedpandaInstaller._avail_for_download --- tests/rptest/services/redpanda_installer.py | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/tests/rptest/services/redpanda_installer.py b/tests/rptest/services/redpanda_installer.py index 94af2fab0a7d..dcb19164e771 100644 --- a/tests/rptest/services/redpanda_installer.py +++ b/tests/rptest/services/redpanda_installer.py @@ -15,6 +15,7 @@ import typing import threading from datetime import datetime, timezone, timedelta +from time import sleep import requests @@ -447,7 +448,17 @@ def _avail_for_download(self, version: tuple[int, int, int]): """ r = requests.head(self._version_package_url(version)) # allow 403 ClientError, it usually indicates Unauthorized get and can happen on S3 while dealing with old releases - if r.status_code not in (200, 403, 404): + allowed = (200, 403, 404) + if r.status_code not in allowed: + num_retries = 3 + while num_retries > 0: + sleep(5.0**(4 - num_retries)) + r = requests.head(self._version_package_url(version)) + if r.status_code in allowed: + break + num_retries -= 1 + + if r.status_code not in allowed: r.raise_for_status() if r.status_code == 403: From 85b0d80e2a4781d0c38ed74bf5280fa2420c3929 Mon Sep 17 00:00:00 2001 From: Chris Lee Date: Mon, 25 Nov 2024 12:45:08 -0700 Subject: [PATCH 005/229] [rptest] unbreak Azure T4/T5 tests --- tests/rptest/clients/kubectl.py | 8 ++++++-- tests/rptest/services/cloud_broker.py | 2 +- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/tests/rptest/clients/kubectl.py b/tests/rptest/clients/kubectl.py index 80e0e47842e5..435167e74152 100644 --- a/tests/rptest/clients/kubectl.py +++ b/tests/rptest/clients/kubectl.py @@ -450,11 +450,13 @@ class KubeNodeShell(): def __init__(self, kubectl: KubectlTool, node_name: str, + namespace: str = 'redpanda-node-setup', clean=False) -> None: self.kubectl = kubectl self.node_name = node_name # It is bad, but it works self.logger = self.kubectl._redpanda.logger + self.namespace = namespace self.current_context = self.kubectl.cmd( f"config current-context").strip() # Make sure that name is not longer that 63 chars @@ -528,6 +530,7 @@ def initialize_nodeshell(self): # Feel free to uncomment _out = self.kubectl.cmd([ f"--context={self.current_context}", + f"--namespace={self.namespace}", "run", "--image docker.io/library/alpine", "--restart=Never", @@ -541,7 +544,8 @@ def initialize_nodeshell(self): def destroy_nodeshell(self): if self._is_shell_running(): try: - self.kubectl.cmd(f"delete pod {self.pod_name}") + self.kubectl.cmd( + f"-n {self.namespace} delete pod {self.pod_name}") except Exception as e: self.logger.warning("Failed to delete node shell pod " f"'{self.pod_name}': {e}") @@ -558,7 +562,7 @@ def __exit__(self, *args, **kwargs): def __call__(self, cmd: list[str] | str, capture=False): self.logger.info(f"Running command inside node '{self.node_name}'") # Prefix for running inside proper pod - _kcmd = ["exec", self.pod_name, "--"] + _kcmd = ["-n", f"{self.namespace}", "exec", self.pod_name, "--"] # Universal for list and str _cmd = cmd if isinstance(cmd, list) else cmd.split() _kcmd += _cmd diff --git a/tests/rptest/services/cloud_broker.py b/tests/rptest/services/cloud_broker.py index a389a0c7cad8..c171fbf847e5 100644 --- a/tests/rptest/services/cloud_broker.py +++ b/tests/rptest/services/cloud_broker.py @@ -83,7 +83,7 @@ def inject_script(self, script_name): # Copy agent -> broker node remote_path = os.path.join("/tmp", script_name) _cp_cmd = self._kubeclient._ssh_prefix() + [ - 'kubectl', 'cp', script_name, + 'kubectl', '-n', 'redpanda-node-setup', 'cp', script_name, f"{self.nodeshell.pod_name}:{remote_path}" ] self.logger.debug(_cp_cmd) From 96b0318e95596db6986e08fe03340899205137f8 Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Tue, 26 Nov 2024 14:16:03 -0500 Subject: [PATCH 006/229] `storage`: remove unnecessary `if()` --- src/v/storage/disk_log_impl.cc | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/v/storage/disk_log_impl.cc b/src/v/storage/disk_log_impl.cc index 59bda0620e6d..da04c1a7cfa1 100644 --- a/src/v/storage/disk_log_impl.cc +++ b/src/v/storage/disk_log_impl.cc @@ -551,11 +551,8 @@ segment_set disk_log_impl::find_sliding_range( buf.emplace_back(seg); } - segment_set segs(std::move(buf)); - if (segs.empty()) { - return segs; - } + segment_set segs(std::move(buf)); return segs; } From cba889288a6227a2083af2758941975cf571f1a2 Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Tue, 26 Nov 2024 15:07:26 -0500 Subject: [PATCH 007/229] `storage`: reset compaction window start offset when offset map isn't built Consider the following situation: [0] [1] [2] [3] [4] [5] ^ |----- _last_compaction_window_start_offset In the event that segment [2] cannot have its keys completely added to the offset map due to memory constraints, if we do not reset the window start offset, compaction will become somewhat stuck as it repeatedly tries to index segment [2] in future compaction rounds. By resetting the window's value in this situation, we can allow compaction to continue to make forward progression. --- src/v/storage/disk_log_impl.cc | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/v/storage/disk_log_impl.cc b/src/v/storage/disk_log_impl.cc index da04c1a7cfa1..2214f557e921 100644 --- a/src/v/storage/disk_log_impl.cc +++ b/src/v/storage/disk_log_impl.cc @@ -659,6 +659,10 @@ ss::future disk_log_impl::sliding_window_compact( "[{}] failed to build offset map. Stopping compaction: {}", config().ntp(), std::current_exception()); + // Reset the sliding window start offset so that compaction may still + // make progress in the future. Otherwise, we will fail to build the + // offset map for the same segment over and over. + _last_compaction_window_start_offset.reset(); co_return false; } vlog( From 72118b92065643da72ace80366f792f34c075c77 Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Tue, 26 Nov 2024 15:12:56 -0500 Subject: [PATCH 008/229] `storage`: add `TestFailToIndexOneSegmentResetWindow` --- src/v/storage/tests/compaction_e2e_test.cc | 57 ++++++++++++++++++++++ 1 file changed, 57 insertions(+) diff --git a/src/v/storage/tests/compaction_e2e_test.cc b/src/v/storage/tests/compaction_e2e_test.cc index f927a9788d67..ba7a8fee2a85 100644 --- a/src/v/storage/tests/compaction_e2e_test.cc +++ b/src/v/storage/tests/compaction_e2e_test.cc @@ -363,6 +363,63 @@ TEST_F(CompactionFixtureTest, TestDedupeMultiPass) { ASSERT_NO_FATAL_FAILURE(check_records(cardinality, num_segments - 1).get()); } +// Test that failing to index a single segment will result in the compaction +// sliding window being reset, allowing compaction to potentially make progress +// in the future instead of repeatedly failing to index the same segment. +TEST_F(CompactionFixtureTest, TestFailToIndexOneSegmentResetWindow) { + constexpr auto num_segments = 1; + constexpr auto cardinality = 10; + size_t records_per_segment = cardinality; + generate_data(num_segments, cardinality, records_per_segment).get(); + + ss::abort_source never_abort; + auto& disk_log = dynamic_cast(*log); + + // Set the last compaction window start offset, just to be sure that its + // value is reset by failing to index a segment. + disk_log.set_last_compaction_window_start_offset(model::offset::max()); + + bool did_compact = do_sliding_window_compact( + log->segments().back()->offsets().get_base_offset(), + std::nullopt, + cardinality - 1) + .get(); + + ASSERT_FALSE(did_compact); + ASSERT_FALSE( + disk_log.get_last_compaction_window_start_offset().has_value()); + + // Generate some more data in such a way that the first segment can be fully + // indexed after the next round of compaction. + generate_data(num_segments, cardinality, records_per_segment / 2).get(); + + // This round of compaction will fully index the last added segment, and + // deduplicate the keys in the first segment enough that it will no longer + // fail to be indexed. + did_compact = do_sliding_window_compact( + log->segments().back()->offsets().get_base_offset(), + std::nullopt, + cardinality - 1) + .get(); + + ASSERT_TRUE(did_compact); + ASSERT_TRUE(disk_log.get_last_compaction_window_start_offset().has_value()); + ASSERT_EQ( + disk_log.get_last_compaction_window_start_offset().value(), + disk_log.segments()[1]->offsets().get_base_offset()); + + // Now, try to compact one last time. + did_compact = do_sliding_window_compact( + log->segments().back()->offsets().get_base_offset(), + std::nullopt, + cardinality - 1) + .get(); + + ASSERT_TRUE(did_compact); + ASSERT_FALSE( + disk_log.get_last_compaction_window_start_offset().has_value()); +} + TEST_F(CompactionFixtureTest, TestDedupeMultiPassAddedSegment) { constexpr auto duplicates_per_key = 10; constexpr auto num_segments = 25; From 4bf9b74a21f59c0672abcffe771e2a7177400792 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Tue, 29 Oct 2024 14:21:14 +0000 Subject: [PATCH 009/229] ct: add snapshotting --- src/v/cloud_topics/BUILD | 34 ++++- src/v/cloud_topics/CMakeLists.txt | 1 + src/v/cloud_topics/dl_snapshot.cc | 16 ++ src/v/cloud_topics/dl_snapshot.h | 55 +++++++ src/v/cloud_topics/dl_stm/BUILD | 4 + src/v/cloud_topics/dl_stm/dl_stm.cc | 11 ++ src/v/cloud_topics/dl_stm/dl_stm_api.cc | 87 +++++++++++ src/v/cloud_topics/dl_stm/dl_stm_api.h | 33 ++++ src/v/cloud_topics/dl_stm/dl_stm_commands.h | 26 ++++ src/v/cloud_topics/dl_stm/dl_stm_state.cc | 76 ++++++++++ src/v/cloud_topics/dl_stm/dl_stm_state.h | 46 +++++- src/v/cloud_topics/dl_stm/tests/BUILD | 1 + .../dl_stm/tests/dl_stm_state_test.cc | 143 ++++++++++++++++++ .../cloud_topics/dl_stm/tests/dl_stm_test.cc | 34 +++++ src/v/cloud_topics/types.cc | 5 + src/v/cloud_topics/types.h | 2 + src/v/cluster/partition.cc | 8 + 17 files changed, 571 insertions(+), 11 deletions(-) create mode 100644 src/v/cloud_topics/dl_snapshot.cc create mode 100644 src/v/cloud_topics/dl_snapshot.h diff --git a/src/v/cloud_topics/BUILD b/src/v/cloud_topics/BUILD index e3e559be99e3..f1a4f10b0c86 100644 --- a/src/v/cloud_topics/BUILD +++ b/src/v/cloud_topics/BUILD @@ -41,13 +41,6 @@ redpanda_cc_library( ], ) -redpanda_cc_library( - name = "dl_version", - hdrs = ["dl_version.h"], - include_prefix = "cloud_topics", - deps = ["//src/v/utils:named_type"], -) - redpanda_cc_library( name = "dl_overlay", srcs = [ @@ -82,6 +75,33 @@ redpanda_cc_library( ], ) +redpanda_cc_library( + name = "dl_version", + hdrs = ["dl_version.h"], + include_prefix = "cloud_topics", + deps = ["//src/v/utils:named_type"], +) + +redpanda_cc_library( + name = "dl_snapshot", + srcs = [ + "dl_snapshot.cc", + ], + hdrs = [ + "dl_snapshot.h", + ], + implementation_deps = [ + "@fmt", + ], + include_prefix = "cloud_topics", + deps = [ + ":dl_overlay", + ":dl_version", + "//src/v/container:fragmented_vector", + "//src/v/serde", + ], +) + redpanda_cc_library( name = "app", srcs = [ diff --git a/src/v/cloud_topics/CMakeLists.txt b/src/v/cloud_topics/CMakeLists.txt index f95db8d50ab5..65e5bf75a2f7 100644 --- a/src/v/cloud_topics/CMakeLists.txt +++ b/src/v/cloud_topics/CMakeLists.txt @@ -3,6 +3,7 @@ v_cc_library( NAME cloud_topics_base SRCS dl_overlay.cc + dl_snapshot.cc logger.cc types.cc DEPS diff --git a/src/v/cloud_topics/dl_snapshot.cc b/src/v/cloud_topics/dl_snapshot.cc new file mode 100644 index 000000000000..95a64ab13760 --- /dev/null +++ b/src/v/cloud_topics/dl_snapshot.cc @@ -0,0 +1,16 @@ +// Copyright 2024 Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 + +#include "cloud_topics/dl_snapshot.h" + +auto fmt::formatter::format( + const experimental::cloud_topics::dl_snapshot_id& id, + fmt::format_context& ctx) const -> decltype(ctx.out()) { + return fmt::format_to(ctx.out(), "{}", id.version); +} diff --git a/src/v/cloud_topics/dl_snapshot.h b/src/v/cloud_topics/dl_snapshot.h new file mode 100644 index 000000000000..498870802aac --- /dev/null +++ b/src/v/cloud_topics/dl_snapshot.h @@ -0,0 +1,55 @@ +// Copyright 2024 Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 + +#pragma once + +#include "cloud_topics/dl_overlay.h" +#include "cloud_topics/dl_version.h" +#include "container/fragmented_vector.h" +#include "serde/envelope.h" + +namespace experimental::cloud_topics { + +struct dl_snapshot_id + : serde:: + envelope, serde::compat_version<0>> { + dl_snapshot_id() noexcept = default; + + explicit dl_snapshot_id(dl_version version) noexcept + : version(version) {} + + auto serde_fields() { return std::tie(version); } + + bool operator==(const dl_snapshot_id& other) const noexcept = default; + + /// Version for which the snapshot is created. + dl_version version; +}; + +struct dl_snapshot_payload + : serde::checksum_envelope< + dl_snapshot_id, + serde::version<0>, + serde::compat_version<0>> { + /// Version for which the snapshot is created. + dl_snapshot_id id; + + /// Overlays visible at the snapshot version. + fragmented_vector overlays; +}; + +}; // namespace experimental::cloud_topics + +template<> +struct fmt::formatter + : fmt::formatter { + auto format( + const experimental::cloud_topics::dl_snapshot_id&, + fmt::format_context& ctx) const -> decltype(ctx.out()); +}; diff --git a/src/v/cloud_topics/dl_stm/BUILD b/src/v/cloud_topics/dl_stm/BUILD index ba2447e7edd1..5eb84bc75b42 100644 --- a/src/v/cloud_topics/dl_stm/BUILD +++ b/src/v/cloud_topics/dl_stm/BUILD @@ -7,6 +7,7 @@ redpanda_cc_library( visibility = [":__subpackages__"], deps = [ "//src/v/cloud_topics:dl_overlay", + "//src/v/cloud_topics:dl_version", "//src/v/serde", ], ) @@ -19,6 +20,7 @@ redpanda_cc_library( visibility = [":__subpackages__"], deps = [ "//src/v/cloud_topics:dl_overlay", + "//src/v/cloud_topics:dl_snapshot", "//src/v/cloud_topics:dl_version", "//src/v/container:fragmented_vector", "//src/v/model", @@ -58,6 +60,8 @@ redpanda_cc_library( deps = [ "//src/v/base", "//src/v/cloud_topics:dl_overlay", + "//src/v/cloud_topics:dl_snapshot", + "//src/v/cloud_topics:dl_version", "//src/v/model", "@seastar", ], diff --git a/src/v/cloud_topics/dl_stm/dl_stm.cc b/src/v/cloud_topics/dl_stm/dl_stm.cc index db91721310ce..5532f2921d21 100644 --- a/src/v/cloud_topics/dl_stm/dl_stm.cc +++ b/src/v/cloud_topics/dl_stm/dl_stm.cc @@ -46,6 +46,17 @@ ss::future<> dl_stm::do_apply(const model::record_batch& batch) { _state.push_overlay(new_dl_version, std::move(cmd.overlay)); break; } + case dl_stm_key::start_snapshot: { + std::ignore = serde::from_iobuf( + r.release_value()); + _state.start_snapshot(new_dl_version); + break; + } + case dl_stm_key::remove_snapshots_before_version: + auto cmd = serde::from_iobuf( + r.release_value()); + _state.remove_snapshots_before(cmd.last_version_to_keep); + break; } }); diff --git a/src/v/cloud_topics/dl_stm/dl_stm_api.cc b/src/v/cloud_topics/dl_stm/dl_stm_api.cc index 016ec4f10f16..7febb26e160f 100644 --- a/src/v/cloud_topics/dl_stm/dl_stm_api.cc +++ b/src/v/cloud_topics/dl_stm/dl_stm_api.cc @@ -17,6 +17,8 @@ #include "serde/rw/uuid.h" #include "storage/record_batch_builder.h" +#include + namespace experimental::cloud_topics { std::ostream& operator<<(std::ostream& o, dl_stm_api_errc errc) { @@ -32,8 +34,12 @@ dl_stm_api::dl_stm_api(ss::logger& logger, ss::shared_ptr stm) : _logger(logger) , _stm(std::move(stm)) {} +ss::future<> dl_stm_api::stop() { co_await _gate.close(); } + ss::future> dl_stm_api::push_overlay(dl_overlay overlay) { + auto h = _gate.hold(); + // TODO: Sync state and consider whether we need to encode invariants in the // command. model::term_id term = _stm->_raft->term(); @@ -68,4 +74,85 @@ std::optional dl_stm_api::lower_bound(kafka::offset offset) const { return _stm->_state.lower_bound(offset); } +ss::future> +dl_stm_api::start_snapshot() { + vlog(_logger.debug, "Replicating dl_stm_cmd::start_snapshot_cmd"); + auto h = _gate.hold(); + + storage::record_batch_builder builder( + model::record_batch_type::dl_stm_command, model::offset(0)); + builder.add_raw_kv( + serde::to_iobuf(dl_stm_key::start_snapshot), + serde::to_iobuf(start_snapshot_cmd())); + + auto batch = std::move(builder).build(); + + auto apply_result = co_await replicated_apply(std::move(batch)); + if (apply_result.has_failure()) { + co_return apply_result.error(); + } + + // We abuse knowledge of implementation detail here to construct the + // dl_snapshot_id without having to setup listeners and notifiers of command + // apply. + auto expected_id = dl_snapshot_id(dl_version(apply_result.value())); + + // Ensure that the expected snapshot was created. + if (!_stm->_state.snapshot_exists(expected_id)) { + throw std::runtime_error(fmt::format( + "Snapshot with expected id not found after waiting for command to be " + "applied: {}", + expected_id)); + } + + co_return outcome::success(expected_id); +} + +std::optional +dl_stm_api::read_snapshot(dl_snapshot_id id) { + return _stm->_state.read_snapshot(id); +} + +ss::future> +dl_stm_api::remove_snapshots_before(dl_version last_version_to_keep) { + vlog(_logger.debug, "Replicating dl_stm_cmd::remove_snapshots_cmd"); + auto h = _gate.hold(); + + storage::record_batch_builder builder( + model::record_batch_type::dl_stm_command, model::offset(0)); + builder.add_raw_kv( + serde::to_iobuf(dl_stm_key::remove_snapshots_before_version), + serde::to_iobuf( + remove_snapshots_before_version_cmd(last_version_to_keep))); + + auto batch = std::move(builder).build(); + auto apply_result = co_await replicated_apply(std::move(batch)); + if (apply_result.has_failure()) { + co_return apply_result.error(); + } + + co_return outcome::success(); +} + +ss::future> +dl_stm_api::replicated_apply(model::record_batch&& batch) { + model::term_id term = _stm->_raft->term(); + + auto reader = model::make_memory_record_batch_reader(std::move(batch)); + + auto opts = raft::replicate_options(raft::consistency_level::quorum_ack); + opts.set_force_flush(); + auto res = co_await _stm->_raft->replicate(term, std::move(reader), opts); + + if (res.has_error()) { + throw std::runtime_error( + fmt::format("Failed to replicate overlay: {}", res.error())); + } + + co_await _stm->wait( + res.value().last_offset, model::timeout_clock::now() + 30s); + + co_return res.value().last_offset; +} + }; // namespace experimental::cloud_topics diff --git a/src/v/cloud_topics/dl_stm/dl_stm_api.h b/src/v/cloud_topics/dl_stm/dl_stm_api.h index a47d59a2caa6..efbcba1fa139 100644 --- a/src/v/cloud_topics/dl_stm/dl_stm_api.h +++ b/src/v/cloud_topics/dl_stm/dl_stm_api.h @@ -11,7 +11,11 @@ #include "base/outcome.h" #include "cloud_topics/dl_overlay.h" +#include "cloud_topics/dl_snapshot.h" +#include "cloud_topics/dl_version.h" +#include "model/record.h" +#include #include #include @@ -32,6 +36,13 @@ class dl_stm_api { dl_stm_api(ss::logger& logger, ss::shared_ptr stm); dl_stm_api(dl_stm_api&&) noexcept = default; + ~dl_stm_api() { + vassert(_gate.is_closed(), "object destroyed before calling stop()"); + } + +public: + ss::future<> stop(); + public: /// Attempt to add a new overlay. ss::future> push_overlay(dl_overlay overlay); @@ -41,9 +52,31 @@ class dl_stm_api { /// available offset. std::optional lower_bound(kafka::offset offset) const; + /// Request a new snapshot to be created. + ss::future> start_snapshot(); + + /// Read the payload of a snapshot. + std::optional read_snapshot(dl_snapshot_id id); + + /// Remove all snapshots with version less than the given version. + /// This must be called periodically as new snapshots are being created + /// to avoid the state growing indefinitely. + ss::future> + remove_snapshots_before(dl_version last_version_to_keep); + +private: + /// Replicate a record batch and wait for it to be applied to the dl_stm. + /// Returns the offset at which the batch was applied. + ss::future> + replicated_apply(model::record_batch&& batch); + private: ss::logger& _logger; + /// Gate held by async operations to ensure that the API is not destroyed + /// while an operation is in progress. + ss::gate _gate; + /// The API can only read the state of the stm. The state can be mutated /// only via \ref consensus::replicate calls. ss::shared_ptr _stm; diff --git a/src/v/cloud_topics/dl_stm/dl_stm_commands.h b/src/v/cloud_topics/dl_stm/dl_stm_commands.h index 7f51c6c15ed9..0791be2e9ef6 100644 --- a/src/v/cloud_topics/dl_stm/dl_stm_commands.h +++ b/src/v/cloud_topics/dl_stm/dl_stm_commands.h @@ -10,6 +10,7 @@ #pragma once #include "cloud_topics/dl_overlay.h" +#include "cloud_topics/dl_version.h" #include "model/fundamental.h" #include "model/timestamp.h" #include "serde/envelope.h" @@ -28,4 +29,29 @@ struct push_overlay_cmd dl_overlay overlay; }; +struct start_snapshot_cmd + : public serde::envelope< + start_snapshot_cmd, + serde::version<0>, + serde::compat_version<0>> { + start_snapshot_cmd() noexcept = default; + + auto serde_fields() { return std::tie(); } +}; + +struct remove_snapshots_before_version_cmd + : public serde::envelope< + remove_snapshots_before_version_cmd, + serde::version<0>, + serde::compat_version<0>> { + remove_snapshots_before_version_cmd() noexcept = default; + explicit remove_snapshots_before_version_cmd( + dl_version last_version_to_keep) + : last_version_to_keep(last_version_to_keep) {} + + auto serde_fields() { return std::tie(last_version_to_keep); } + + dl_version last_version_to_keep{}; +}; + } // namespace experimental::cloud_topics diff --git a/src/v/cloud_topics/dl_stm/dl_stm_state.cc b/src/v/cloud_topics/dl_stm/dl_stm_state.cc index 37dc4669346d..7b5d2e56c93c 100644 --- a/src/v/cloud_topics/dl_stm/dl_stm_state.cc +++ b/src/v/cloud_topics/dl_stm/dl_stm_state.cc @@ -10,6 +10,7 @@ #include "cloud_topics/dl_stm/dl_stm_state.h" #include "cloud_topics/dl_overlay.h" +#include "cloud_topics/dl_snapshot.h" #include "model/fundamental.h" #include @@ -71,4 +72,79 @@ dl_stm_state::lower_bound(kafka::offset offset) const { return best_match; } +dl_snapshot_id dl_stm_state::start_snapshot(dl_version version) noexcept { + _version_invariant.set_last_snapshot_version(version); + + auto id = dl_snapshot_id(version); + _snapshots.push_back(id); + + return id; +} + +bool dl_stm_state::snapshot_exists(dl_snapshot_id id) const noexcept { + return std::binary_search( + _snapshots.begin(), + _snapshots.end(), + id, + [](const dl_snapshot_id& a, const dl_snapshot_id& b) { + return a.version < b.version; + }); +} + +std::optional +dl_stm_state::read_snapshot(dl_snapshot_id id) const { + auto it = std::find_if( + _snapshots.begin(), _snapshots.end(), [&id](const dl_snapshot_id& s) { + return s.version == id.version; + }); + + // Snapshot not found. + if (it == _snapshots.end()) { + return std::nullopt; + } + + // Collect overlays that are visible at the snapshot version. + fragmented_vector overlays; + for (const auto& entry : _overlays) { + if ( + entry.added_at <= id.version + && (entry.removed_at == dl_version{} || entry.removed_at > id.version)) { + overlays.push_back(entry.overlay); + } + } + + return dl_snapshot_payload{ + .id = *it, + .overlays = std::move(overlays), + }; +} + +void dl_stm_state::remove_snapshots_before(dl_version last_version_to_keep) { + if (_snapshots.empty()) { + throw std::runtime_error(fmt::format( + "Attempt to remove snapshots before version {} but no snapshots " + "exist", + last_version_to_keep)); + } + + // Find the first snapshot to keep. It is the first snapshot with a version + // equal or greater than the version to keep. + auto it = std::lower_bound( + _snapshots.begin(), + _snapshots.end(), + last_version_to_keep, + [](const dl_snapshot_id& a, dl_version b) { return a.version < b; }); + + if (it == _snapshots.begin()) { + // Short circuit if there are no snapshots to remove + return; + } else if (it == _snapshots.end()) { + throw std::runtime_error(fmt::format( + "Trying to remove snapshots before an non-existent snapshot", + last_version_to_keep)); + } else { + _snapshots.erase(_snapshots.begin(), it); + } +} + } // namespace experimental::cloud_topics diff --git a/src/v/cloud_topics/dl_stm/dl_stm_state.h b/src/v/cloud_topics/dl_stm/dl_stm_state.h index 09b876dd91b0..442c0b57ec0f 100644 --- a/src/v/cloud_topics/dl_stm/dl_stm_state.h +++ b/src/v/cloud_topics/dl_stm/dl_stm_state.h @@ -10,9 +10,12 @@ #pragma once #include "cloud_topics/dl_overlay.h" +#include "cloud_topics/dl_snapshot.h" #include "cloud_topics/dl_version.h" #include "container/fragmented_vector.h" +#include + namespace experimental::cloud_topics { struct dl_overlay_entry { @@ -25,17 +28,34 @@ struct dl_overlay_entry { class dl_version_monotonic_invariant { public: void set_version(dl_version version) noexcept { - // Greater or equal is required to handle retries. + // Greater or equal for `_last_version` is required to handle retries. + // Greater for `_last_snapshot_version` to avoid mutating an existing + // snapshot. vassert( - version >= _last_version, - "Version can't go backwards. Current version: {}, new version: {}", + version >= _last_version && version > _last_snapshot_version, + "Version can't go backwards. Current version: {}, new version: {}, " + "last snapshot version: {}", _last_version, - version); + version, + _last_snapshot_version); _last_version = version; } + void set_last_snapshot_version(dl_version version) noexcept { + // Greater or equal is required to handle retries. + vassert( + version >= _last_snapshot_version, + "Snapshot version can't go backwards. Current snapshot version: {}, " + "new snapshot version: {}", + _last_snapshot_version, + version); + set_version(version); + _last_snapshot_version = version; + } + private: dl_version _last_version; + dl_version _last_snapshot_version; }; /// In-memory state of the data layout state machine (dl_stm). @@ -55,11 +75,29 @@ class dl_stm_state { /// available offset. std::optional lower_bound(kafka::offset offset) const; + /// Create a handle to a snapshot of the state at the current version. + /// The snapshot id can be used later to read snapshot contents. + dl_snapshot_id start_snapshot(dl_version version) noexcept; + + bool snapshot_exists(dl_snapshot_id id) const noexcept; + + /// Snapshot of the state at the given version. + std::optional read_snapshot(dl_snapshot_id id) const; + + /// Remove all snapshots with version less than the given version. + void remove_snapshots_before(dl_version last_version_to_keep); + private: // A list of overlays that are stored in the cloud storage. // The order of elements is undefined. std::deque _overlays; + // A list of snapshot handles that are currently open. + // The list is ordered by version in ascending order to efficiently find the + // oldest snapshot when running state garbage collection and to remove + // closed snapshots. + std::deque _snapshots; + dl_version_monotonic_invariant _version_invariant; }; diff --git a/src/v/cloud_topics/dl_stm/tests/BUILD b/src/v/cloud_topics/dl_stm/tests/BUILD index becec908c7f5..f9644e11b3cb 100644 --- a/src/v/cloud_topics/dl_stm/tests/BUILD +++ b/src/v/cloud_topics/dl_stm/tests/BUILD @@ -8,6 +8,7 @@ redpanda_cc_gtest( ], deps = [ "//src/v/cloud_topics:dl_overlay", + "//src/v/cloud_topics:dl_version", "//src/v/cloud_topics:logger", "//src/v/cloud_topics:types", "//src/v/cloud_topics/dl_stm:dl_stm_state", diff --git a/src/v/cloud_topics/dl_stm/tests/dl_stm_state_test.cc b/src/v/cloud_topics/dl_stm/tests/dl_stm_state_test.cc index 8c7a6b410e2c..95b573354001 100644 --- a/src/v/cloud_topics/dl_stm/tests/dl_stm_state_test.cc +++ b/src/v/cloud_topics/dl_stm/tests/dl_stm_state_test.cc @@ -9,6 +9,8 @@ #include "cloud_topics/dl_overlay.h" #include "cloud_topics/dl_stm/dl_stm_state.h" +#include "cloud_topics/dl_version.h" +#include "gmock/gmock.h" #include "gtest/gtest.h" #include "random/generators.h" #include "test_utils/test.h" @@ -167,3 +169,144 @@ TEST(dl_stm_state, lower_bound) { } while (std::next_permutation( push_order.begin(), push_order.end(), base_offset_less_cmp)); } + +TEST(dl_stm_state_death, start_snapshot) { + ct::dl_stm_state state; + + auto snapshot_id1 = state.start_snapshot(ct::dl_version(1)); + ASSERT_EQ(snapshot_id1.version, ct::dl_version(1)); + + auto snapshot1 = state.read_snapshot(snapshot_id1); + ASSERT_TRUE(snapshot1.has_value()); + ASSERT_EQ(snapshot1->id, snapshot_id1); + ASSERT_TRUE(snapshot1->overlays.empty()); + + // This does not exist yet. + ASSERT_FALSE( + state.read_snapshot(ct::dl_snapshot_id(ct::dl_version(2))).has_value()); + + auto snapshot_id2 = state.start_snapshot(ct::dl_version(2)); + ASSERT_EQ(snapshot_id2.version, ct::dl_version(2)); + + auto snapshot2 = state.read_snapshot(snapshot_id2); + ASSERT_TRUE(snapshot2.has_value()); + ASSERT_EQ(snapshot2->id, snapshot_id2); + ASSERT_TRUE(snapshot2->overlays.empty()); + + // Starting a snapshot without advancing the version should throw. + ASSERT_DEATH( + { state.start_snapshot(ct::dl_version(1)); }, + "Snapshot version can't go backwards. Current snapshot version: 2, new " + "snapshot version: 1"); + + ASSERT_DEATH( + { + state.push_overlay( + ct::dl_version(2), + make_overlay(kafka::offset(0), kafka::offset(10))); + }, + "Version can't go backwards. Current version: 2, new version: 2, last " + "snapshot version: 2"); +} + +TEST(dl_stm_state, start_snapshot) { + ct::dl_stm_state state; + + auto overlay0 = make_overlay(kafka::offset(0), kafka::offset(10)); + state.push_overlay(ct::dl_version(1), overlay0); + + auto snapshot_id0 = state.start_snapshot(ct::dl_version(1)); + + // Mark the overlay as removed. + q::overlays(state).front().removed_at = ct::dl_version(2); + + auto snapshot_id1 = state.start_snapshot(ct::dl_version(2)); + + auto overlay1 = make_overlay(kafka::offset(5), kafka::offset(8)); + state.push_overlay(ct::dl_version(3), overlay1); + + auto snapshot_id2 = state.start_snapshot(ct::dl_version(3)); + + auto overlay2 = make_overlay(kafka::offset(6), kafka::offset(20)); + state.push_overlay(ct::dl_version(4), overlay2); + + auto snapshot_id3 = state.start_snapshot(ct::dl_version(5)); + + auto snapshot0 = state.read_snapshot(snapshot_id0); + ASSERT_TRUE(snapshot0.has_value()); + ASSERT_EQ(snapshot0->id, snapshot_id0); + ASSERT_EQ(snapshot0->overlays.size(), 1) << snapshot0->overlays; + ASSERT_EQ(snapshot0->overlays[0], overlay0); + + auto snapshot1 = state.read_snapshot(snapshot_id1); + ASSERT_TRUE(snapshot1.has_value()); + ASSERT_EQ(snapshot1->id, snapshot_id1); + ASSERT_EQ(snapshot1->overlays.size(), 0); + + auto snapshot2 = state.read_snapshot(snapshot_id2); + ASSERT_TRUE(snapshot2.has_value()); + ASSERT_EQ(snapshot2->id, snapshot_id2); + ASSERT_EQ(snapshot2->overlays.size(), 1) << snapshot2->overlays; + ASSERT_EQ(snapshot2->overlays[0], overlay1); + + auto snapshot3 = state.read_snapshot(snapshot_id3); + ASSERT_TRUE(snapshot3.has_value()); + ASSERT_EQ(snapshot3->id, snapshot_id3); + ASSERT_EQ(snapshot3->overlays.size(), 2); + ASSERT_EQ(snapshot3->overlays[0], overlay1) << snapshot3->overlays; + ASSERT_EQ(snapshot3->overlays[1], overlay2) << snapshot3->overlays; +} + +TEST(dl_stm_state, remove_snapshots_before) { + ct::dl_stm_state state; + + EXPECT_THAT( + [&]() { state.remove_snapshots_before(ct::dl_version(42)); }, + ThrowsMessage( + testing::HasSubstr("Attempt to remove snapshots before version 42 but " + "no snapshots exist"))); + + auto overlay0 = make_overlay(kafka::offset(0), kafka::offset(10)); + state.push_overlay(ct::dl_version(1), overlay0); + + auto snapshot_id0 = state.start_snapshot(ct::dl_version(1)); + + // Mark the overlay as removed. + q::overlays(state).front().removed_at = ct::dl_version(2); + + auto snapshot_id1 = state.start_snapshot(ct::dl_version(2)); + + auto overlay1 = make_overlay(kafka::offset(5), kafka::offset(8)); + state.push_overlay(ct::dl_version(3), overlay1); + + auto snapshot_id2 = state.start_snapshot(ct::dl_version(3)); + + auto overlay2 = make_overlay(kafka::offset(6), kafka::offset(20)); + state.push_overlay(ct::dl_version(4), overlay2); + + auto snapshot_id3 = state.start_snapshot(ct::dl_version(5)); + + // Test that operation is idempotent. + for (auto i = 0; i < 3; ++i) { + state.remove_snapshots_before(ct::dl_version(3)); + + ASSERT_FALSE(state.snapshot_exists(snapshot_id0)); + ASSERT_FALSE(state.snapshot_exists(snapshot_id1)); + ASSERT_TRUE(state.snapshot_exists(snapshot_id2)); + ASSERT_TRUE(state.snapshot_exists(snapshot_id3)); + + // Retrying the an out-of-date version is an idempotent operation too. + state.remove_snapshots_before(ct::dl_version(2)); + } + + // It should be impossible to make a call like this because the contract + // with the callers is that they should first call `start_snapshot` and can + // call remove_snapshots_before only with the result of the `start_snapshot` + // call. + // In case this bug is introduced we want to throw an exception instead of + // failing silently. + EXPECT_THAT( + [&]() { state.remove_snapshots_before(ct::dl_version::max()); }, + ThrowsMessage(testing::HasSubstr( + "Trying to remove snapshots before an non-existent snapshot"))); +} diff --git a/src/v/cloud_topics/dl_stm/tests/dl_stm_test.cc b/src/v/cloud_topics/dl_stm/tests/dl_stm_test.cc index 8cd93dafcbde..2fb819b50e73 100644 --- a/src/v/cloud_topics/dl_stm/tests/dl_stm_test.cc +++ b/src/v/cloud_topics/dl_stm/tests/dl_stm_test.cc @@ -24,6 +24,12 @@ class dl_stm_fixture : public raft::raft_fixture { public: static constexpr auto node_count = 3; + ~dl_stm_fixture() override { + for (auto& entry : api_by_vnode) { + entry.second->stop().get(); + } + }; + ss::future<> start() { for (auto i = 0; i < node_count; ++i) { add_node(model::node_id(i), model::revision_id(0)); @@ -87,4 +93,32 @@ TEST_F_CORO(dl_stm_fixture, test_basic) { ASSERT_TRUE_CORO( api(node(*get_leader())).lower_bound(kafka::offset(0)).has_value()); + + auto snapshot_res = co_await api(node(*get_leader())).start_snapshot(); + ASSERT_FALSE_CORO(res.has_error()); + + ASSERT_TRUE_CORO( + api(node(*get_leader())).read_snapshot(snapshot_res.value()).has_value()); + ASSERT_EQ_CORO( + api(node(*get_leader())) + .read_snapshot(snapshot_res.value()) + ->overlays.size(), + 1); + + auto snapshot_res2 = co_await api(node(*get_leader())).start_snapshot(); + + ASSERT_TRUE_CORO(api(node(*get_leader())) + .read_snapshot(snapshot_res2.value()) + .has_value()); + + auto remove_res = co_await api(node(*get_leader())) + .remove_snapshots_before(snapshot_res2.value().version); + ASSERT_FALSE_CORO(remove_res.has_error()); + + ASSERT_FALSE_CORO( + api(node(*get_leader())).read_snapshot(snapshot_res.value()).has_value()); + + ASSERT_TRUE_CORO(api(node(*get_leader())) + .read_snapshot(snapshot_res2.value()) + .has_value()); } diff --git a/src/v/cloud_topics/types.cc b/src/v/cloud_topics/types.cc index 3f6e50d7b113..785a1b6a2d19 100644 --- a/src/v/cloud_topics/types.cc +++ b/src/v/cloud_topics/types.cc @@ -17,6 +17,11 @@ auto fmt::formatter::format( switch (key) { case experimental::cloud_topics::dl_stm_key::push_overlay: return fmt::format_to(ctx.out(), "push_overlay"); + case experimental::cloud_topics::dl_stm_key::start_snapshot: + return fmt::format_to(ctx.out(), "start_snapshot"); + case experimental::cloud_topics::dl_stm_key:: + remove_snapshots_before_version: + return fmt::format_to(ctx.out(), "remove_snapshots_before_version"); } return fmt::format_to( ctx.out(), "unknown dl_stm_key({})", static_cast(key)); diff --git a/src/v/cloud_topics/types.h b/src/v/cloud_topics/types.h index 7ef75f7adc22..bd1e16bf9d10 100644 --- a/src/v/cloud_topics/types.h +++ b/src/v/cloud_topics/types.h @@ -21,6 +21,8 @@ namespace experimental::cloud_topics { enum class dl_stm_key { push_overlay = 0, + start_snapshot = 1, + remove_snapshots_before_version = 2, // TODO: add all commands }; diff --git a/src/v/cluster/partition.cc b/src/v/cluster/partition.cc index 8febc0d33439..d8337ad611f9 100644 --- a/src/v/cluster/partition.cc +++ b/src/v/cluster/partition.cc @@ -585,6 +585,14 @@ ss::future<> partition::stop() { co_await _cloud_storage_manifest_view->stop(); } + if (_dl_stm_api) { + vlog( + clusterlog.debug, + "Stopping dl_stm_api on partition: {}", + partition_ntp); + co_await _dl_stm_api->stop(); + } + _probe.clear_metrics(); vlog(clusterlog.debug, "Stopped partition {}", partition_ntp); } From b711212ebb1c90543011638370d980dee274b4e0 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Thu, 28 Nov 2024 17:10:41 +0000 Subject: [PATCH 010/229] ct: use replicated_apply for push_overlay --- src/v/cloud_topics/dl_stm/dl_stm_api.cc | 19 ++++--------------- 1 file changed, 4 insertions(+), 15 deletions(-) diff --git a/src/v/cloud_topics/dl_stm/dl_stm_api.cc b/src/v/cloud_topics/dl_stm/dl_stm_api.cc index 7febb26e160f..3cd1fcb8f25e 100644 --- a/src/v/cloud_topics/dl_stm/dl_stm_api.cc +++ b/src/v/cloud_topics/dl_stm/dl_stm_api.cc @@ -38,13 +38,11 @@ ss::future<> dl_stm_api::stop() { co_await _gate.close(); } ss::future> dl_stm_api::push_overlay(dl_overlay overlay) { + vlog(_logger.debug, "Replicating dl_stm_cmd::push_overlay_cmd"); auto h = _gate.hold(); // TODO: Sync state and consider whether we need to encode invariants in the // command. - model::term_id term = _stm->_raft->term(); - - vlog(_logger.debug, "Replicating dl_stm_cmd::push_overlay_cmd"); storage::record_batch_builder builder( model::record_batch_type::dl_stm_command, model::offset(0)); @@ -53,20 +51,11 @@ dl_stm_api::push_overlay(dl_overlay overlay) { serde::to_iobuf(push_overlay_cmd(std::move(overlay)))); auto batch = std::move(builder).build(); - auto reader = model::make_memory_record_batch_reader(std::move(batch)); - - auto opts = raft::replicate_options(raft::consistency_level::quorum_ack); - opts.set_force_flush(); - auto res = co_await _stm->_raft->replicate(term, std::move(reader), opts); - - if (res.has_error()) { - throw std::runtime_error( - fmt::format("Failed to replicate overlay: {}", res.error())); + auto apply_result = co_await replicated_apply(std::move(batch)); + if (apply_result.has_failure()) { + co_return apply_result.error(); } - co_await _stm->wait_no_throw( - res.value().last_offset, model::timeout_clock::now() + 30s); - co_return outcome::success(true); } From 5532d3241b0ce74be9fb096cea9b5d5f0dc60ee8 Mon Sep 17 00:00:00 2001 From: Stephan Dollberg Date: Tue, 8 Oct 2024 15:28:19 +0100 Subject: [PATCH 011/229] bazel: Enable LTO for redpanda This patch enables LTO for redpanda by default (main binary only). We add `-flto=thin` and `-ffat-lto-objects` to the compile flags. This allows for the final link step to chose whether to use LTO or not (by passing `flto` to the linker step). Hence, we can selectively enable LTO only for the redpanda binary and not for the test binaries. Note this isn't a massive advantage yet for the bazel build because of the thing where it builds everything twice anyway). It is however already useful for not using LTO for initial local dev (lto=False) but then enabling it for an eventual microbench or a package build. We chose thin-lto over full lto as compile times are a lot slower with that (~5x) and additional perf improvements are small. Also we can always change later. As is the final link step is about a minute. This will go up once we enable debug info as that has quite the impact on compile and link times. Note we can't use the clang thin-lto cache option because it would require a bunch of bazel work to use a cache dir. In a librdkafka bench this change shows a 2-3% improvement. There is some microbench results in CORE-327 but note that microbenches aren't really that representative for LTO. Ref CORE-7789 --- .bazelrc | 3 ++- bazel/build.bzl | 2 ++ bazel/thirdparty/krb5.BUILD | 4 ++++ src/v/redpanda/BUILD | 24 ++++++++++++++++++++++++ 4 files changed, 32 insertions(+), 1 deletion(-) diff --git a/.bazelrc b/.bazelrc index a6e0ff7ff663..35e5e9ed9d39 100644 --- a/.bazelrc +++ b/.bazelrc @@ -89,8 +89,9 @@ build:secure --config=relro build:release --compilation_mode opt build:release --config=secure build:release --copt -mllvm --copt -inline-threshold=2500 -build:release --linkopt=-flto build:release --@seastar//:stack_guards=False +build:release --//src/v/redpanda:lto=True +build:release --copt -flto=thin --copt -ffat-lto-objects build:stamp --stamp --workspace_status_command=./bazel/stamp_vars.sh diff --git a/bazel/build.bzl b/bazel/build.bzl index e5fb7000f0f4..e41375c7cc70 100644 --- a/bazel/build.bzl +++ b/bazel/build.bzl @@ -51,6 +51,7 @@ def redpanda_cc_binary( local_defines = [], visibility = None, copts = [], + linkopts = [], deps = []): """ Define a Redpanda C++ binary. @@ -64,6 +65,7 @@ def redpanda_cc_binary( visibility = visibility, deps = deps, copts = redpanda_copts() + copts, + linkopts = linkopts, features = [ "layering_check", ], diff --git a/bazel/thirdparty/krb5.BUILD b/bazel/thirdparty/krb5.BUILD index e2c6d7554968..c7aa78b1bcd4 100644 --- a/bazel/thirdparty/krb5.BUILD +++ b/bazel/thirdparty/krb5.BUILD @@ -48,6 +48,10 @@ configure_make( "--disable-static", "--enable-asan=$(SANITIZERS)", ], + # Need to pass this additionally here because of a bug in the kerberos build where it doesn't properly pass the linker flag down + copts = [ + "-fuse-ld=lld", + ], env = { "KRB5_BUILD_JOBS": "$(BUILD_JOBS)", }, diff --git a/src/v/redpanda/BUILD b/src/v/redpanda/BUILD index 845d3a246a74..32c9d3308ac4 100644 --- a/src/v/redpanda/BUILD +++ b/src/v/redpanda/BUILD @@ -1,3 +1,4 @@ +load("@bazel_skylib//rules:common_settings.bzl", "bool_flag") load("//bazel:build.bzl", "redpanda_cc_binary", "redpanda_cc_library") redpanda_cc_library( @@ -95,11 +96,34 @@ redpanda_cc_library( ], ) +# Enable or disable (thin) LTO for the redpanda binary (only) +bool_flag( + name = "lto", + build_setting_default = False, +) + +config_setting( + name = "use_lto", + flag_values = { + ":lto": "true", + }, + visibility = ["//visibility:public"], +) + redpanda_cc_binary( name = "redpanda", srcs = [ "main.cc", ], + linkopts = + select({ + ":use_lto": [ + "-flto=thin", + "-ffat-lto-objects", + ], + "//conditions:default": [ + ], + }), visibility = ["//visibility:public"], deps = [ ":application", From cccb53dffd49fe594f8efcb78998d1365c17fe76 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gell=C3=A9rt=20Peresztegi-Nagy?= Date: Mon, 2 Dec 2024 20:00:26 +0000 Subject: [PATCH 012/229] cluster/test: configure reserve_shard0 in fixtures --- src/v/cluster/tests/partition_allocator_fixture.h | 6 +++--- src/v/cluster/tests/partition_allocator_tests.cc | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/v/cluster/tests/partition_allocator_fixture.h b/src/v/cluster/tests/partition_allocator_fixture.h index fcf56e7b07b3..2e815ade1405 100644 --- a/src/v/cluster/tests/partition_allocator_fixture.h +++ b/src/v/cluster/tests/partition_allocator_fixture.h @@ -32,7 +32,6 @@ struct partition_allocator_fixture { static constexpr uint32_t partitions_per_shard = 1000; - static constexpr uint32_t partitions_reserve_shard0 = 2; partition_allocator_fixture() : partition_allocator_fixture(std::nullopt, std::nullopt) {} @@ -68,7 +67,7 @@ struct partition_allocator_fixture { broker.id(), broker.properties().cores, config::mock_binding(uint32_t{partitions_per_shard}), - config::mock_binding(uint32_t{partitions_reserve_shard0}), + partitions_reserve_shard0.bind(), kafka_internal_topics.bind())); } @@ -135,6 +134,7 @@ struct partition_allocator_fixture { cluster::partition_allocator& allocator() { return _allocator.local(); } config::mock_property> kafka_internal_topics{{}}; + config::mock_property partitions_reserve_shard0{2}; model::topic_namespace tn{model::kafka_namespace, model::topic{"test"}}; ss::sharded members; ss::sharded features; @@ -155,7 +155,7 @@ struct partition_allocator_fixture { config::mock_binding>(memory_per_partition), config::mock_binding>(fds_per_partition), config::mock_binding(uint32_t{partitions_per_shard}), - config::mock_binding(uint32_t{partitions_reserve_shard0}), + partitions_reserve_shard0.bind(), kafka_internal_topics.bind(), config::mock_binding(true)) .get(); diff --git a/src/v/cluster/tests/partition_allocator_tests.cc b/src/v/cluster/tests/partition_allocator_tests.cc index 9db4351aec74..80ac265f1b95 100644 --- a/src/v/cluster/tests/partition_allocator_tests.cc +++ b/src/v/cluster/tests/partition_allocator_tests.cc @@ -525,7 +525,7 @@ FIXTURE_TEST(updating_nodes_properties, partition_allocator_fixture) { BOOST_REQUIRE_EQUAL( it->second->max_capacity(), 10 * partition_allocator_fixture::partitions_per_shard - - partition_allocator_fixture::partitions_reserve_shard0); + - partitions_reserve_shard0()); } FIXTURE_TEST(change_replication_factor, partition_allocator_fixture) { From f35fda78da814e0bb2b3d1110f82044468bdf49d Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Mon, 2 Dec 2024 19:47:46 +0000 Subject: [PATCH 013/229] rptest: await all replicas to recover before validation --- tests/rptest/tests/topic_recovery_test.py | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/tests/rptest/tests/topic_recovery_test.py b/tests/rptest/tests/topic_recovery_test.py index ccf4351e5d50..966907e06a34 100644 --- a/tests/rptest/tests/topic_recovery_test.py +++ b/tests/rptest/tests/topic_recovery_test.py @@ -1211,6 +1211,7 @@ def __init__(self, test_context: TestContext, *args, **kwargs): self._started = True self.rpk = RpkTool(self.redpanda) + self.admin = Admin(self.redpanda) def rpk_producer_maker(self, topic: str, @@ -1467,6 +1468,17 @@ def _wait_for_topic(self, expected_num_leaders = sum( [t.partition_count for t in recovered_topics]) + def all_replicas_in_sync(topic, *, partition, num_replicas): + partition_state = self.admin.get_partition_state( + "kafka", topic, partition) + if len(partition_state["replicas"]) != num_replicas: + return False + hwms = [ + replica["high_watermark"] + for replica in partition_state["replicas"] + ] + return all([hwm == hwms[0] for hwm in hwms]) + def verify(): num_leaders = 0 try: @@ -1478,6 +1490,15 @@ def verify(): self.logger.info(f"partition: {partition}") if partition.leader in partition.replicas: num_leaders += 1 + + # If we have a leader, we can check if all replicas are in sync + if not all_replicas_in_sync( + topic.name, + partition=partition.id, + num_replicas=len(partition.replicas)): + self.logger.debug( + "partition replicas are not in sync yet") + return False except: return False return num_leaders == expected_num_leaders From 2d6baaac02a17cc14f79ecad6137a7376c7027a8 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Mon, 2 Dec 2024 20:39:15 +0000 Subject: [PATCH 014/229] rptest: let wait_until handle exception retries Much more helpful error messages as the exceptions are stored and re thrown at the end --- tests/rptest/tests/topic_recovery_test.py | 42 +++++++++++------------ 1 file changed, 21 insertions(+), 21 deletions(-) diff --git a/tests/rptest/tests/topic_recovery_test.py b/tests/rptest/tests/topic_recovery_test.py index 966907e06a34..b41ea25249a5 100644 --- a/tests/rptest/tests/topic_recovery_test.py +++ b/tests/rptest/tests/topic_recovery_test.py @@ -1481,29 +1481,29 @@ def all_replicas_in_sync(topic, *, partition, num_replicas): def verify(): num_leaders = 0 - try: - for topic in recovered_topics: - topic_state = self.rpk.describe_topic(topic.name) - # Describe topics only works after leader election succeded. - # We can use it to wait until the recovery is completed. - for partition in topic_state: - self.logger.info(f"partition: {partition}") - if partition.leader in partition.replicas: - num_leaders += 1 - - # If we have a leader, we can check if all replicas are in sync - if not all_replicas_in_sync( - topic.name, - partition=partition.id, - num_replicas=len(partition.replicas)): - self.logger.debug( - "partition replicas are not in sync yet") - return False - except: - return False + for topic in recovered_topics: + topic_state = self.rpk.describe_topic(topic.name) + # Describe topics only works after leader election succeded. + # We can use it to wait until the recovery is completed. + for partition in topic_state: + self.logger.info(f"partition: {partition}") + if partition.leader in partition.replicas: + num_leaders += 1 + + # If we have a leader, we can check if all replicas are in sync + if not all_replicas_in_sync(topic.name, + partition=partition.id, + num_replicas=len( + partition.replicas)): + self.logger.debug( + "partition replicas are not in sync yet") + return False return num_leaders == expected_num_leaders - wait_until(verify, timeout_sec=timeout.total_seconds(), backoff_sec=1) + wait_until(verify, + timeout_sec=timeout.total_seconds(), + backoff_sec=1, + retry_on_exc=True) def do_run(self, test_case: BaseCase, upload_delay_sec=60): """Template method invoked by all tests.""" From f9477eb612e6a13edbcdb79a8b93c6b02f92eaf5 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Mon, 2 Dec 2024 21:15:45 +0000 Subject: [PATCH 015/229] rptest: stricter verify_file_layout Catches the case of a replica missing all segments. --- tests/rptest/utils/si_utils.py | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/tests/rptest/utils/si_utils.py b/tests/rptest/utils/si_utils.py index 1c2b2889dee4..aed9f50a4af9 100644 --- a/tests/rptest/utils/si_utils.py +++ b/tests/rptest/utils/si_utils.py @@ -407,8 +407,10 @@ def get_ntp_sizes(fdata_per_host, hosts_can_vary=True): that maps host to dict of ntps where each ntp is mapped to the list of segments. The result is a map from ntp to the partition size on disk. """ + first_host = None + ntps = defaultdict(int) - for _, fdata in fdata_per_host.items(): + for host, fdata in fdata_per_host.items(): ntp_size = defaultdict(int) for path, entry in fdata.items(): it = _parse_checksum_entry(path, entry, ignore_rev=True) @@ -418,6 +420,13 @@ def get_ntp_sizes(fdata_per_host, hosts_can_vary=True): # which are created after recovery ntp_size[it.ntp] += it.size + if first_host is None: + first_host = host + else: + assert set(ntps.keys()) == set( + ntp_size.keys() + ), f"NTPs on {host} differ from first host {first_host}: {set(ntps.keys())} vs {host}: {set(ntp_size.keys())}" + for ntp, total_size in ntp_size.items(): if ntp in ntps and not hosts_can_vary: # the size of the partition should be the From 0abd1f6306ce3eaa1a9ab2ae101e496994c05365 Mon Sep 17 00:00:00 2001 From: Ivo Jimenez Date: Tue, 3 Dec 2024 08:17:08 -0700 Subject: [PATCH 016/229] gha: use full major.minor.patch version ID This was changed in a previous commit and while it was working OK, it is now failing. Signed-off-by: Ivo Jimenez --- .github/workflows/promote.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/promote.yml b/.github/workflows/promote.yml index 142cf32abb76..581cd17993cc 100644 --- a/.github/workflows/promote.yml +++ b/.github/workflows/promote.yml @@ -19,7 +19,7 @@ jobs: secret-ids: | ,sdlc/prod/github/buildkite_token parse-json-secrets: true - - uses: buildkite/trigger-pipeline-action@v2 + - uses: buildkite/trigger-pipeline-action@v2.0.0 with: buildkite_api_access_token: ${{ env.BUILDKITE_TOKEN }} pipeline: "redpanda/redpanda" From 83c7c90acae967cea0036356b2f2bb7ea4eecfc3 Mon Sep 17 00:00:00 2001 From: Oren Leiman Date: Tue, 3 Dec 2024 09:53:33 -0800 Subject: [PATCH 017/229] config: iceberg_enabled should not be enterprise For beta version, should not require a license. Signed-off-by: Oren Leiman --- src/v/config/configuration.cc | 1 - src/v/config/configuration.h | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/v/config/configuration.cc b/src/v/config/configuration.cc index 42a3c93f411c..d9a064eecb5a 100644 --- a/src/v/config/configuration.cc +++ b/src/v/config/configuration.cc @@ -3682,7 +3682,6 @@ configuration::configuration() tls_version::v1_3}) , iceberg_enabled( *this, - true, "iceberg_enabled", "Enables the translation of topic data into Iceberg tables. Setting " "iceberg_enabled to true activates the feature at the cluster level, but " diff --git a/src/v/config/configuration.h b/src/v/config/configuration.h index 651afe7f638d..4ccfac65e40f 100644 --- a/src/v/config/configuration.h +++ b/src/v/config/configuration.h @@ -701,7 +701,7 @@ struct configuration final : public config_store { enum_property tls_min_version; // datalake configurations - enterprise> iceberg_enabled; + property iceberg_enabled; bounded_property iceberg_catalog_commit_interval_ms; property iceberg_catalog_base_location; From b773a06a6a134546da0e79dafd39fa695ecf16ca Mon Sep 17 00:00:00 2001 From: gene-redpanda <123959009+gene-redpanda@users.noreply.github.com> Date: Mon, 7 Oct 2024 17:57:34 -0500 Subject: [PATCH 018/229] rpk: add --format to rpk topic describe Adds format flag to rpk topic describe. Supports JSON and YAML --- src/go/rpk/pkg/cli/topic/BUILD | 1 + src/go/rpk/pkg/cli/topic/describe.go | 437 ++++++++---- src/go/rpk/pkg/cli/topic/describe_test.go | 817 +++++++++++++++++++--- src/go/rpk/pkg/config/format.go | 4 + 4 files changed, 1021 insertions(+), 238 deletions(-) diff --git a/src/go/rpk/pkg/cli/topic/BUILD b/src/go/rpk/pkg/cli/topic/BUILD index 3e611e0f3955..c4aa966bcf45 100644 --- a/src/go/rpk/pkg/cli/topic/BUILD +++ b/src/go/rpk/pkg/cli/topic/BUILD @@ -55,6 +55,7 @@ go_test( deps = [ "//src/go/rpk/pkg/config", "@com_github_spf13_afero//:afero", + "@com_github_stretchr_testify//assert", "@com_github_stretchr_testify//require", "@com_github_twmb_franz_go//pkg/kerr", "@com_github_twmb_franz_go_pkg_kadm//:kadm", diff --git a/src/go/rpk/pkg/cli/topic/describe.go b/src/go/rpk/pkg/cli/topic/describe.go index 060ab0f6d854..709439eef302 100644 --- a/src/go/rpk/pkg/cli/topic/describe.go +++ b/src/go/rpk/pkg/cli/topic/describe.go @@ -13,7 +13,10 @@ import ( "context" "errors" "fmt" + "io" + "os" "sort" + "strconv" "github.com/redpanda-data/redpanda/src/go/rpk/pkg/config" "github.com/redpanda-data/redpanda/src/go/rpk/pkg/kafka" @@ -45,6 +48,9 @@ This command prints detailed information about topics. The output contains up to three sections: a summary of the topic, the topic configs, and a detailed partitions section. By default, the summary and configs sections are printed. +Using the --format flag with either JSON or YAML will default in printing +all the topic information (--all). + The --regex flag (-r) parses arguments as regular expressions and describes topics that match any of the expressions. @@ -54,11 +60,13 @@ For example, describe -r '^f.*' '.*r$' # describe any topic starting with f and any topics ending in r describe -r '*' # describe all topics describe -r . # describe any one-character topics - `, - Args: cobra.MinimumNArgs(1), - Run: func(_ *cobra.Command, topicArg []string) { + Run: func(cmd *cobra.Command, topicArg []string) { + f := p.Formatter + if h, ok := f.Help([]describedTopic{}); ok { + out.Exit(h) + } p, err := p.LoadVirtualProfile(fs) out.MaybeDie(err, "rpk unable to load config: %v", err) @@ -84,7 +92,8 @@ For example, // We show all sections if: // - "print-all" is used or // - more than one topic are specified or matched. - if all || len(topicArg) > 1 { + // - the formatter is not text (json/yaml). + if all || len(topicArg) > 1 || !f.IsText() { summary, configs, partitions = true, true, true } else if len(topicArg) == 0 { out.Exit("did not match any topics, exiting.") @@ -96,94 +105,41 @@ For example, reqTopic.Topic = kmsg.StringPtr(topic) req.Topics = append(req.Topics, reqTopic) } - resp, err := req.RequestWith(context.Background(), cl) + resp, err := req.RequestWith(cmd.Context(), cl) out.MaybeDie(err, "unable to request topic metadata: %v", err) - const ( - secSummary = "summary" - secConfigs = "configs" - secPart = "partitions" - ) - - for i, topic := range resp.Topics { - sections := out.NewMaybeHeaderSections( - out.ConditionalSectionHeaders(map[string]bool{ - secSummary: summary, - secConfigs: configs, - secPart: partitions, - })..., - ) - - sections.Add(secSummary, func() { - tw := out.NewTabWriter() - defer tw.Flush() - tw.PrintColumn("NAME", *topic.Topic) - if topic.IsInternal { - tw.PrintColumn("INTERNAL", topic.IsInternal) - } - tw.PrintColumn("PARTITIONS", len(topic.Partitions)) - if len(topic.Partitions) > 0 { - p0 := &topic.Partitions[0] - tw.PrintColumn("REPLICAS", len(p0.Replicas)) - } - if err := kerr.ErrorForCode(topic.ErrorCode); err != nil { - tw.PrintColumn("ERROR", err) - } - }) - - sections.Add(secConfigs, func() { - req := kmsg.NewPtrDescribeConfigsRequest() - reqResource := kmsg.NewDescribeConfigsRequestResource() - reqResource.ResourceType = kmsg.ConfigResourceTypeTopic - reqResource.ResourceName = *topic.Topic - req.Resources = append(req.Resources, reqResource) - - resp, err := req.RequestWith(context.Background(), cl) - out.MaybeDie(err, "unable to request configs: %v", err) - if len(resp.Resources) != 1 { - out.Die("config response returned %d resources when we asked for 1", len(resp.Resources)) - } - err = kerr.ErrorForCode(resp.Resources[0].ErrorCode) - out.MaybeDie(err, "config response contained error: %v", err) - - tw := out.NewTable("KEY", "VALUE", "SOURCE") - defer tw.Flush() - types.Sort(resp) - for _, config := range resp.Resources[0].Configs { - var val string - if config.IsSensitive { - val = "(sensitive)" - } else if config.Value != nil { - val = *config.Value - } - tw.Print(config.Name, val, config.Source) - } - }) - - sections.Add(secPart, func() { - offsets := listStartEndOffsets(cl, *topic.Topic, len(topic.Partitions), stable) - - tw := out.NewTable(describePartitionsHeaders( - topic.Partitions, - offsets, - )...) - defer tw.Flush() - for _, row := range describePartitionsRows( - topic.Partitions, - offsets, - ) { - tw.Print(row...) + var topicDescriptions []describedTopic + for _, topic := range resp.Topics { + var t describedTopic + if summary { + t.Summary = buildDescribeTopicSummary(topic) + } + if configs { + cfgResp, cfgErr := prepDescribeTopicConfig(cmd.Context(), topic, cl) + out.MaybeDieErr(cfgErr) + err = kerr.ErrorForCode(cfgResp.ErrorCode) + if err != nil { + t.cfgErr = err } - }) - - i++ - if i < len(resp.Topics) { - fmt.Println() + t.Configs = buildDescribeTopicConfig(cfgResp.Configs) + } + if partitions { + offsets := listStartEndOffsets(cmd.Context(), cl, *topic.Topic, len(topic.Partitions), stable) + u := getDescribeUsed(topic.Partitions, offsets) + t.Partitions = buildDescribeTopicPartitions(topic.Partitions, offsets, u) + t.u = u } + topicDescriptions = append(topicDescriptions, t) } + + if printDescribedTopicsFormatter(f, topicDescriptions, os.Stdout) { + return + } + printDescribedTopics(summary, configs, partitions, topicDescriptions) }, } + p.InstallFormatFlag(cmd) cmd.Flags().IntVar(new(int), "page", -1, "deprecated") cmd.Flags().IntVar(new(int), "page-size", 20, "deprecated") cmd.Flags().BoolVar(new(bool), "watermarks", true, "deprecated") @@ -204,106 +160,291 @@ For example, return cmd } -// We optionally include the following columns: -// - offline-replicas, if any are offline -// - load-error, if metadata indicates load errors any partitions -// - last-stable-offset, if it is ever not equal to the high watermark (transactions) -func getDescribeUsed(partitions []kmsg.MetadataResponseTopicPartition, offsets []startStableEndOffset) (useOffline, useErr, useStable bool) { - for _, p := range partitions { - if len(p.OfflineReplicas) > 0 { - useOffline = true - } - if p.ErrorCode != 0 { - useErr = true - } +func printDescribedTopicsFormatter(f config.OutFormatter, topics []describedTopic, w io.Writer) bool { + if isText, _, t, err := f.Format(topics); !isText { + out.MaybeDie(err, "unable to print in the requested format %v", err) + fmt.Fprintln(w, t) + return true } - for _, o := range offsets { - // The default stableErr is errUnlisted. We avoid listing - // stable offsets unless the user asks, so by default, we do - // not print the stable column. - if o.stableErr == nil && o.endErr == nil && o.stable != o.end { - useStable = true + return false +} + +func printDescribedTopics(summary, configs, partitions bool, topics []describedTopic) { + const ( + secSummary = "summary" + secConfigs = "configs" + secPart = "partitions" + ) + + for _, topic := range topics { + sections := out.NewMaybeHeaderSections( + out.ConditionalSectionHeaders(map[string]bool{ + secSummary: summary, + secConfigs: configs, + secPart: partitions, + })..., + ) + + sections.Add(secSummary, func() { + tw := out.NewTabWriter() + defer tw.Flush() + tw.PrintColumn("NAME", topic.Summary.Name) + if topic.Summary.Internal { + tw.PrintColumn("INTERNAL", topic.Summary.Internal) + } + tw.PrintColumn("PARTITIONS", topic.Summary.Partitions) + if topic.Summary.Partitions > 0 { + tw.PrintColumn("REPLICAS", topic.Summary.Replicas) + } + if topic.Summary.Error != "" { + tw.PrintColumn("ERROR", topic.Summary.Error) + } + }) + sections.Add(secConfigs, func() { + out.MaybeDie(topic.cfgErr, "config response contained error: %v", topic.cfgErr) + tw := out.NewTable("KEY", "VALUE", "SOURCE") + defer tw.Flush() + for _, c := range topic.Configs { + tw.Print(c.Key, c.Value, c.Source) + } + }) + sections.Add(secPart, func() { + tw := out.NewTable(partitionHeader(topic.u)...) + defer tw.Flush() + for _, row := range topic.Partitions { + tw.PrintStrings(row.Row(topic.u)...) + } + }) + } +} + +type describedTopic struct { + Summary describeTopicSummary `json:"summary" yaml:"summary"` + Configs []describeTopicConfig `json:"configs" yaml:"configs"` + Partitions []describeTopicPartition `json:"partitions" yaml:"partitions"` + u uses + cfgErr error +} + +type describeTopicSummary struct { + Name string `json:"name" yaml:"name"` + Internal bool `json:"internal" yaml:"internal"` + Partitions int `json:"partitions" yaml:"partitions"` + Replicas int `json:"replicas" yaml:"replicas"` + Error string `json:"error" yaml:"error"` +} + +func buildDescribeTopicSummary(topic kmsg.MetadataResponseTopic) describeTopicSummary { + resp := describeTopicSummary{ + Name: *topic.Topic, + Internal: topic.IsInternal, + Partitions: len(topic.Partitions), + } + if len(topic.Partitions) > 0 { + resp.Replicas = len(topic.Partitions[0].Replicas) + } + if err := kerr.ErrorForCode(topic.ErrorCode); err != nil { + resp.Error = err.Error() + } + return resp +} + +type describeTopicConfig struct { + Key string `json:"key" yaml:"key"` + Value string `json:"value" yaml:"value"` + Source string `json:"source" yaml:"source"` +} + +func prepDescribeTopicConfig(ctx context.Context, topic kmsg.MetadataResponseTopic, cl *kgo.Client) (*kmsg.DescribeConfigsResponseResource, error) { + req := kmsg.NewPtrDescribeConfigsRequest() + reqResource := kmsg.NewDescribeConfigsRequestResource() + reqResource.ResourceType = kmsg.ConfigResourceTypeTopic + reqResource.ResourceName = *topic.Topic + req.Resources = append(req.Resources, reqResource) + + resp, err := req.RequestWith(ctx, cl) + if err != nil { + return nil, fmt.Errorf("unable to request configs: %v", err) + } + if len(resp.Resources) != 1 { + return nil, fmt.Errorf("config response returned %d resources when we asked for 1", len(resp.Resources)) + } + return &resp.Resources[0], nil +} + +func buildDescribeTopicConfig(configs []kmsg.DescribeConfigsResponseResourceConfig) []describeTopicConfig { + output := make([]describeTopicConfig, 0, len(configs)) + types.Sort(configs) + for _, cfg := range configs { + d := describeTopicConfig{ + Key: cfg.Name, + Source: cfg.Source.String(), + } + if cfg.IsSensitive { + d.Value = "(sensitive)" + } else if cfg.Value != nil { + d.Value = *cfg.Value } + output = append(output, d) } - return + return output +} + +type describeTopicPartition struct { + Partition int32 `json:"partition" yaml:"partition"` + Leader int32 `json:"leader" yaml:"leader"` + Epoch int32 `json:"epoch" yaml:"epoch"` + Replicas []int32 `json:"replicas" yaml:"replicas"` + OfflineReplicas []int32 `json:"offline_replicas,omitempty" yaml:"offline_replicas,omitempty"` + LoadError string `json:"load_error,omitempty" yaml:"load_error,omitempty"` + LogStartOffset int64 `json:"log_start_offset" yaml:"log_start_offset"` + logStartOffsetText any + LastStableOffset int64 `json:"last_stable_offset,omitempty" yaml:"last_stable_offset,omitempty"` + lastStableOffsetText any + HighWatermark int64 `json:"high_watermark" yaml:"high_watermark"` + highWatermarkText any + Errors []string `json:"error,omitempty" yaml:"error,omitempty"` } -func describePartitionsHeaders( - partitions []kmsg.MetadataResponseTopicPartition, - offsets []startStableEndOffset, -) []string { - offline, err, stable := getDescribeUsed(partitions, offsets) - headers := []string{"partition", "leader", "epoch"} - headers = append(headers, "replicas") // TODO add isr see #1928 - if offline { +func partitionHeader(u uses) []string { + headers := []string{ + "partition", + "leader", + "epoch", + "replicas", + } + + if u.Offline { headers = append(headers, "offline-replicas") } - if err { + if u.LoadErr { headers = append(headers, "load-error") } headers = append(headers, "log-start-offset") - if stable { + if u.Stable { headers = append(headers, "last-stable-offset") } headers = append(headers, "high-watermark") return headers } -func describePartitionsRows( - partitions []kmsg.MetadataResponseTopicPartition, - offsets []startStableEndOffset, -) [][]interface{} { +type uses struct { + Offline bool + LoadErr bool + Stable bool +} + +func (dp describeTopicPartition) Row(u uses) []string { + row := []string{ + strconv.FormatInt(int64(dp.Partition), 10), + strconv.FormatInt(int64(dp.Leader), 10), + strconv.FormatInt(int64(dp.Epoch), 10), + fmt.Sprintf("%v", dp.Replicas), + } + + if u.Offline { + row = append(row, fmt.Sprintf("%v", dp.OfflineReplicas)) + } + + if u.LoadErr { + row = append(row, dp.LoadError) + } + row = append(row, fmt.Sprintf("%v", dp.logStartOffsetText)) + + if u.Stable { + row = append(row, fmt.Sprintf("%v", dp.lastStableOffsetText)) + } + row = append(row, fmt.Sprintf("%v", dp.highWatermarkText)) + return row +} + +func buildDescribeTopicPartitions(partitions []kmsg.MetadataResponseTopicPartition, offsets []startStableEndOffset, u uses) (resp []describeTopicPartition) { sort.Slice(partitions, func(i, j int) bool { return partitions[i].Partition < partitions[j].Partition }) - - offline, err, stable := getDescribeUsed(partitions, offsets) - var rows [][]interface{} for _, p := range partitions { - row := []interface{}{p.Partition, p.Leader, p.LeaderEpoch} - row = append(row, int32s(p.Replicas).sort()) - if offline { - row = append(row, int32s(p.OfflineReplicas).sort()) + row := describeTopicPartition{ + Partition: p.Partition, + Leader: p.Leader, + Epoch: p.LeaderEpoch, + Replicas: int32s(p.Replicas).sort(), + } + if u.Offline { + row.OfflineReplicas = int32s(p.OfflineReplicas).sort() } - if err { + if u.LoadErr { if err := kerr.ErrorForCode(p.ErrorCode); err != nil { - row = append(row, err) + row.LoadError = err.Error() } else { - row = append(row, "-") + row.LoadError = "-" } } - - // For offsets, we have three options: - // - we listed the offset successfully, we write the number - // - list offsets, we write "-" - // - the partition had a partition error, we write the kerr.Error message o := offsets[p.Partition] if o.startErr == nil { - row = append(row, o.start) + row.LogStartOffset = o.start + row.logStartOffsetText = o.start } else if errors.Is(o.startErr, errUnlisted) { - row = append(row, "-") + row.LogStartOffset = -1 + row.logStartOffsetText = "-" } else { - row = append(row, o.startErr.(*kerr.Error).Message) //nolint:errorlint // This error must be kerr.Error, and we want the message + row.LogStartOffset = -1 + err := o.startErr.(*kerr.Error).Message //nolint:errorlint // This error must be kerr.Error, and we want the message + row.logStartOffsetText = err + row.Errors = append(row.Errors, err) } - if stable { + if u.Stable { if o.stableErr == nil { - row = append(row, o.stable) + row.LastStableOffset = o.stable + row.lastStableOffsetText = o.stable } else if errors.Is(o.stableErr, errUnlisted) { - row = append(row, "-") + row.LastStableOffset = -1 + row.lastStableOffsetText = "-" } else { - row = append(row, o.stableErr.(*kerr.Error).Message) //nolint:errorlint // This error must be kerr.Error, and we want the message + row.LastStableOffset = -1 + err := o.stableErr.(*kerr.Error).Message //nolint:errorlint // This error must be kerr.Error, and we want the message + row.lastStableOffsetText = err + row.Errors = append(row.Errors, err) } } if o.endErr == nil { - row = append(row, o.end) + row.HighWatermark = o.end + row.highWatermarkText = o.end } else if errors.Is(o.endErr, errUnlisted) { - row = append(row, "-") + row.HighWatermark = -1 + row.highWatermarkText = "-" } else { - row = append(row, o.endErr.(*kerr.Error).Message) //nolint:errorlint // This error must be kerr.Error, and we want the message + row.HighWatermark = -1 + err := o.endErr.(*kerr.Error).Message //nolint:errorlint // This error must be kerr.Error, and we want the message + row.highWatermarkText = err + row.Errors = append(row.Errors, err) } - rows = append(rows, row) + resp = append(resp, row) } - return rows + return resp +} + +// We optionally include the following columns: +// - offline-replicas, if any are offline +// - load-error, if metadata indicates load errors any partitions +// - last-stable-offset, if it is ever not equal to the high watermark (transactions) +func getDescribeUsed(partitions []kmsg.MetadataResponseTopicPartition, offsets []startStableEndOffset) (u uses) { + for _, p := range partitions { + if len(p.OfflineReplicas) > 0 { + u.Offline = true + } + if p.ErrorCode != 0 { + u.LoadErr = true + } + } + for _, o := range offsets { + // The default stableErr is errUnlisted. We avoid listing + // stable offsets unless the user asks, so by default, we do + // not print the stable column. + if o.stableErr == nil && o.endErr == nil && o.stable != o.end { + u.Stable = true + } + } + return } type startStableEndOffset struct { @@ -325,7 +466,7 @@ var errUnlisted = errors.New("list failed") // always contain the one topic we asked for, and it will contain all // partitions we asked for. The logic below will panic redpanda replies // incorrectly. -func listStartEndOffsets(cl *kgo.Client, topic string, numPartitions int, stable bool) []startStableEndOffset { +func listStartEndOffsets(ctx context.Context, cl *kgo.Client, topic string, numPartitions int, stable bool) []startStableEndOffset { offsets := make([]startStableEndOffset, 0, numPartitions) for i := 0; i < numPartitions; i++ { @@ -351,7 +492,7 @@ func listStartEndOffsets(cl *kgo.Client, topic string, numPartitions int, stable reqTopic.Partitions = append(reqTopic.Partitions, part) } req.Topics = append(req.Topics, reqTopic) - shards := cl.RequestSharded(context.Background(), req) + shards := cl.RequestSharded(ctx, req) allFailed := kafka.EachShard(req, shards, func(shard kgo.ResponseShard) { resp := shard.Resp.(*kmsg.ListOffsetsResponse) if len(resp.Topics) > 0 { @@ -382,7 +523,7 @@ func listStartEndOffsets(cl *kgo.Client, topic string, numPartitions int, stable // transactions are in play. if stable { req.IsolationLevel = 1 - shards = cl.RequestSharded(context.Background(), req) + shards = cl.RequestSharded(ctx, req) allFailed = kafka.EachShard(req, shards, func(shard kgo.ResponseShard) { resp := shard.Resp.(*kmsg.ListOffsetsResponse) if len(resp.Topics) > 0 { @@ -399,7 +540,7 @@ func listStartEndOffsets(cl *kgo.Client, topic string, numPartitions int, stable } // Finally, the HWM. - shards = cl.RequestSharded(context.Background(), req) + shards = cl.RequestSharded(ctx, req) kafka.EachShard(req, shards, func(shard kgo.ResponseShard) { resp := shard.Resp.(*kmsg.ListOffsetsResponse) if len(resp.Topics) > 0 { diff --git a/src/go/rpk/pkg/cli/topic/describe_test.go b/src/go/rpk/pkg/cli/topic/describe_test.go index d7a66df04653..68917fa27be5 100644 --- a/src/go/rpk/pkg/cli/topic/describe_test.go +++ b/src/go/rpk/pkg/cli/topic/describe_test.go @@ -1,136 +1,773 @@ package topic import ( + "bytes" + "encoding/json" + "io" + "os" + "strings" "testing" + "github.com/redpanda-data/redpanda/src/go/rpk/pkg/config" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/twmb/franz-go/pkg/kerr" "github.com/twmb/franz-go/pkg/kmsg" ) -func TestDescribePartitions(t *testing.T) { - // inputs: conditionals for what columns, as well as the rows - // test: ensure which headers are returned, and args - - for _, test := range []struct { - name string - - inMeta []kmsg.MetadataResponseTopicPartition - inOffsets []startStableEndOffset - - expHeaders []string - expRows [][]interface{} +func TestBuildDescribeTopicPartitions(t *testing.T) { + testCases := []struct { + name string + partitions []kmsg.MetadataResponseTopicPartition + offsets []startStableEndOffset + uses uses + expUseErr bool + expected []describeTopicPartition }{ { - name: "all ok, no optional columns, one partition", - - inMeta: []kmsg.MetadataResponseTopicPartition{ + name: "Normal case", + partitions: []kmsg.MetadataResponseTopicPartition{ + { + Partition: 0, + Leader: 1, + LeaderEpoch: 5, + Replicas: []int32{1, 2, 3}, + }, + { + Partition: 1, + Leader: 2, + LeaderEpoch: 3, + Replicas: []int32{1, 2, 3}, + OfflineReplicas: []int32{3}, + }, + }, + offsets: []startStableEndOffset{ + {start: 0, stable: 100, end: 100, startErr: nil, stableErr: nil, endErr: nil}, + {start: 50, stable: 150, end: 200, startErr: nil, stableErr: nil, endErr: nil}, + }, + uses: uses{Offline: true, Stable: true}, + expected: []describeTopicPartition{ + { + Partition: 0, + Leader: 1, + Epoch: 5, + Replicas: []int32{1, 2, 3}, + OfflineReplicas: []int32{}, + LogStartOffset: 0, + logStartOffsetText: int64(0), + LastStableOffset: 100, + lastStableOffsetText: int64(100), + HighWatermark: 100, + highWatermarkText: int64(100), + }, + { + Partition: 1, + Leader: 2, + Epoch: 3, + Replicas: []int32{1, 2, 3}, + OfflineReplicas: []int32{3}, + LogStartOffset: 50, + logStartOffsetText: int64(50), + LastStableOffset: 150, + lastStableOffsetText: int64(150), + HighWatermark: 200, + highWatermarkText: int64(200), + }, + }, + }, + { + name: "With errors", + partitions: []kmsg.MetadataResponseTopicPartition{ { Partition: 0, - Leader: 0, - ErrorCode: 0, - LeaderEpoch: -1, - Replicas: []int32{0, 1, 2}, + Leader: 1, + LeaderEpoch: 5, + Replicas: []int32{1, 2, 3}, + ErrorCode: 9, // REPLICA_NOT_AVAILABLE error code + }, + }, + offsets: []startStableEndOffset{ + { + start: -1, stable: -1, end: -1, + startErr: kerr.ErrorForCode(9), stableErr: errUnlisted, endErr: kerr.ErrorForCode(9), + }, + }, + uses: uses{LoadErr: true}, + expUseErr: true, + expected: []describeTopicPartition{ + { + Partition: 0, + Leader: 1, + Epoch: 5, + Replicas: []int32{1, 2, 3}, + LoadError: "REPLICA_NOT_AVAILABLE: The replica is not available for the requested topic-partition.", + LogStartOffset: -1, + logStartOffsetText: "REPLICA_NOT_AVAILABLE", + HighWatermark: -1, + highWatermarkText: "REPLICA_NOT_AVAILABLE", + Errors: []string{"REPLICA_NOT_AVAILABLE", "REPLICA_NOT_AVAILABLE"}, + }, + }, + }, + { + name: "Recovery failure - Unknown topic or partition", + partitions: []kmsg.MetadataResponseTopicPartition{ + { + Partition: 0, + Leader: -1, // No leader due to failed recovery + ErrorCode: 3, // UNKNOWN_TOPIC_OR_PARTITION error code + }, + }, + offsets: []startStableEndOffset{ + { + start: -1, + startErr: kerr.ErrorForCode(3), // Set the error + stable: -1, + end: -1, + endErr: kerr.ErrorForCode(3), + }, + }, + uses: uses{LoadErr: true}, + expUseErr: true, + expected: []describeTopicPartition{ + { + Partition: 0, + Leader: -1, + LoadError: "UNKNOWN_TOPIC_OR_PARTITION: This server does not host this topic-partition.", + LogStartOffset: -1, + logStartOffsetText: "UNKNOWN_TOPIC_OR_PARTITION", + Replicas: []int32{}, + HighWatermark: -1, + highWatermarkText: "UNKNOWN_TOPIC_OR_PARTITION", + Errors: []string{"UNKNOWN_TOPIC_OR_PARTITION", "UNKNOWN_TOPIC_OR_PARTITION"}, }, }, - inOffsets: []startStableEndOffset{{ - start: 0, - stable: 1, - end: 1, - }}, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + result := buildDescribeTopicPartitions(tc.partitions, tc.offsets, tc.uses) + assert.Equal(t, tc.expected, result) + }) + } +} - expHeaders: []string{ - "partition", - "leader", - "epoch", - "replicas", - "log-start-offset", - "high-watermark", +func TestPartitionHeaderAndRow(t *testing.T) { + tests := []struct { + name string + partition describeTopicPartition + uses uses + expectedHeader []string + expectedRow []string + }{ + { + name: "all fields", + partition: describeTopicPartition{ + Partition: 0, + Leader: 1, + Epoch: 5, + Replicas: []int32{1, 2, 3}, + OfflineReplicas: []int32{3}, + LoadError: "REPLICA_NOT_AVAILABLE", + LogStartOffset: 100, + logStartOffsetText: int64(100), + LastStableOffset: 200, + lastStableOffsetText: int64(200), + HighWatermark: 300, + highWatermarkText: int64(300), + Errors: []string{"Error1", "Error2"}, }, - expRows: [][]interface{}{ - {int32(0), int32(0), int32(-1), []int32{0, 1, 2}, int64(0), int64(1)}, + uses: uses{Offline: true, LoadErr: true, Stable: true}, + expectedHeader: []string{ + "partition", "leader", "epoch", "replicas", "offline-replicas", + "load-error", "log-start-offset", "last-stable-offset", "high-watermark", + }, + expectedRow: []string{ + "0", "1", "5", "[1 2 3]", "[3]", "REPLICA_NOT_AVAILABLE", + "100", "200", "300", }, }, + { + name: "minimal fields", + partition: describeTopicPartition{ + Partition: 1, + Leader: 2, + Epoch: 3, + Replicas: []int32{1, 2}, + LogStartOffset: 50, + logStartOffsetText: int64(50), + HighWatermark: 150, + highWatermarkText: int64(150), + }, + uses: uses{}, + expectedHeader: []string{ + "partition", "leader", "epoch", "replicas", "log-start-offset", "high-watermark", + }, + expectedRow: []string{ + "1", "2", "3", "[1 2]", "50", "150", + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + header := partitionHeader(tt.uses) + assert.Equal(t, tt.expectedHeader, header, "Headers do not match expected") + + row := tt.partition.Row(tt.uses) + assert.Equal(t, tt.expectedRow, row, "Row does not match expected") + }) + } +} +func TestGetDescribeUsed(t *testing.T) { + testCases := []struct { + name string + partitions []kmsg.MetadataResponseTopicPartition + offsets []startStableEndOffset + expected uses + }{ + { + name: "No special cases", + partitions: []kmsg.MetadataResponseTopicPartition{ + {Partition: 0, ErrorCode: 0}, + }, + offsets: []startStableEndOffset{ + {start: 0, stable: 100, end: 100}, + }, + expected: uses{}, + }, + { + name: "With offline replicas", + partitions: []kmsg.MetadataResponseTopicPartition{ + {Partition: 0, OfflineReplicas: []int32{1}}, + }, + offsets: []startStableEndOffset{ + {start: 0, stable: 100, end: 100}, + }, + expected: uses{Offline: true}, + }, { - name: "all ok, all extra columns, out of order partitions, errors", + name: "With load error", + partitions: []kmsg.MetadataResponseTopicPartition{ + {Partition: 0, ErrorCode: 1}, + }, + offsets: []startStableEndOffset{ + {start: 0, stable: 100, end: 100}, + }, + expected: uses{LoadErr: true}, + }, + { + name: "With stable offset different from end", + partitions: []kmsg.MetadataResponseTopicPartition{ + {Partition: 0}, + }, + offsets: []startStableEndOffset{ + {start: 0, stable: 50, end: 100, stableErr: nil, endErr: nil}, + }, + expected: uses{Stable: true}, + }, + { + name: "All cases", + partitions: []kmsg.MetadataResponseTopicPartition{ + {Partition: 0, ErrorCode: 1, OfflineReplicas: []int32{1}}, + }, + offsets: []startStableEndOffset{ + {start: 0, stable: 50, end: 100, stableErr: nil, endErr: nil}, + }, + expected: uses{Offline: true, LoadErr: true, Stable: true}, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + result := getDescribeUsed(tc.partitions, tc.offsets) + assert.Equal(t, tc.expected, result) + }) + } +} - inMeta: []kmsg.MetadataResponseTopicPartition{ +func TestPrintDescribedTopicsFormatter(t *testing.T) { + testCases := []struct { + name string + format string + topics []describedTopic + expectedOutput string + expectedReturn bool + }{ + { + name: "JSON format - single topic", + format: "json", + topics: []describedTopic{ { - Partition: 1, - Leader: 0, - ErrorCode: 1, - LeaderEpoch: 0, // optional, used - Replicas: []int32{0, 1}, - OfflineReplicas: []int32{2, 3}, // optional, used + Summary: describeTopicSummary{ + Name: "test-topic", + Internal: false, + Partitions: 3, + Replicas: 2, + }, + Configs: []describeTopicConfig{ + {Key: "retention.ms", Value: "604800000", Source: "DEFAULT_CONFIG"}, + }, + Partitions: []describeTopicPartition{ + {Partition: 0, Leader: 1, Replicas: []int32{1, 2}}, + {Partition: 1, Leader: 2, Replicas: []int32{2, 1}}, + {Partition: 2, Leader: 1, Replicas: []int32{1, 2}}, + }, }, - + }, + expectedOutput: `[{"summary":{"name":"test-topic","internal":false,"partitions":3,"replicas":2,"error":""},"configs":[{"key":"retention.ms","value":"604800000","source":"DEFAULT_CONFIG"}],"partitions":[{"partition":0,"leader":1,"epoch":0,"replicas":[1,2],"log_start_offset":0,"high_watermark":0},{"partition":1,"leader":2,"epoch":0,"replicas":[2,1],"log_start_offset":0,"high_watermark":0},{"partition":2,"leader":1,"epoch":0,"replicas":[1,2],"log_start_offset":0,"high_watermark":0}]}]`, + expectedReturn: true, + }, + { + name: "JSON format - multiple topics", + format: "json", + topics: []describedTopic{ { - Partition: 0, - Leader: 1, - LeaderEpoch: -1, - Replicas: []int32{0}, + Summary: describeTopicSummary{ + Name: "topic1", + Internal: false, + Partitions: 2, + Replicas: 2, + }, + Configs: []describeTopicConfig{ + {Key: "retention.ms", Value: "86400000", Source: "DYNAMIC_TOPIC_CONFIG"}, + }, + Partitions: []describeTopicPartition{ + {Partition: 0, Leader: 1, Replicas: []int32{1, 2}}, + {Partition: 1, Leader: 2, Replicas: []int32{2, 1}}, + }, + }, + { + Summary: describeTopicSummary{ + Name: "topic2", + Internal: true, + Partitions: 1, + Replicas: 3, + }, + Configs: []describeTopicConfig{ + {Key: "cleanup.policy", Value: "compact", Source: "STATIC_BROKER_CONFIG"}, + }, + Partitions: []describeTopicPartition{ + {Partition: 0, Leader: 3, Replicas: []int32{1, 2, 3}}, + }, }, }, - inOffsets: []startStableEndOffset{ + expectedOutput: `[{"summary":{"name":"topic1","internal":false,"partitions":2,"replicas":2,"error":""},"configs":[{"key":"retention.ms","value":"86400000","source":"DYNAMIC_TOPIC_CONFIG"}],"partitions":[{"partition":0,"leader":1,"epoch":0,"replicas":[1,2],"log_start_offset":0,"high_watermark":0},{"partition":1,"leader":2,"epoch":0,"replicas":[2,1],"log_start_offset":0,"high_watermark":0}]},{"summary":{"name":"topic2","internal":true,"partitions":1,"replicas":3,"error":""},"configs":[{"key":"cleanup.policy","value":"compact","source":"STATIC_BROKER_CONFIG"}],"partitions":[{"partition":0,"leader":3,"epoch":0,"replicas":[1,2,3],"log_start_offset":0,"high_watermark":0}]}]`, + expectedReturn: true, + }, + { + name: "JSON format - topics with errors", + format: "json", + topics: []describedTopic{ + { + Summary: describeTopicSummary{ + Name: "error-topic-1", + Internal: false, + Partitions: 0, + Replicas: 0, + Error: "UNKNOWN_TOPIC_OR_PARTITION", + }, + Configs: []describeTopicConfig{}, + Partitions: []describeTopicPartition{}, + }, { - start: 0, - stable: 1, - end: 1, + Summary: describeTopicSummary{ + Name: "partial-error-topic", + Internal: false, + Partitions: 2, + Replicas: 3, + Error: "", + }, + Configs: []describeTopicConfig{ + {Key: "min.insync.replicas", Value: "2", Source: "DYNAMIC_TOPIC_CONFIG"}, + }, + Partitions: []describeTopicPartition{ + { + Partition: 0, + Leader: 1, + Replicas: []int32{1, 2, 3}, + OfflineReplicas: []int32{3}, + LogStartOffset: 100, + HighWatermark: 200, + }, + { + Partition: 1, + Leader: -1, + Replicas: []int32{1, 2, 3}, + LogStartOffset: -1, + HighWatermark: -1, + LoadError: "LEADER_NOT_AVAILABLE", + Errors: []string{"LEADER_NOT_AVAILABLE"}, + }, + }, }, { - startErr: kerr.ErrorForCode(9), - stable: 1, - end: 2, + Summary: describeTopicSummary{ + Name: "normal-topic", + Internal: false, + Partitions: 1, + Replicas: 1, + }, + Configs: []describeTopicConfig{ + {Key: "retention.ms", Value: "86400000", Source: "DEFAULT_CONFIG"}, + }, + Partitions: []describeTopicPartition{ + { + Partition: 0, + Leader: 1, + Replicas: []int32{1}, + LogStartOffset: 0, + HighWatermark: 150, + }, + }, }, }, - - expHeaders: []string{ - "partition", - "leader", - "epoch", - "replicas", - "offline-replicas", - "load-error", - "log-start-offset", - "last-stable-offset", - "high-watermark", + expectedOutput: `[ + { + "summary": { + "name": "error-topic-1", + "internal": false, + "partitions": 0, + "replicas": 0, + "error": "UNKNOWN_TOPIC_OR_PARTITION" + }, + "configs": [], + "partitions": [] + }, + { + "summary": { + "name": "partial-error-topic", + "internal": false, + "partitions": 2, + "replicas": 3, + "error": "" + }, + "configs": [ + { + "key": "min.insync.replicas", + "value": "2", + "source": "DYNAMIC_TOPIC_CONFIG" + } + ], + "partitions": [ + { + "partition": 0, + "leader": 1, + "epoch": 0, + "replicas": [1, 2, 3], + "offline_replicas": [3], + "log_start_offset": 100, + "high_watermark": 200 + }, + { + "partition": 1, + "leader": -1, + "epoch": 0, + "replicas": [1, 2, 3], + "log_start_offset": -1, + "high_watermark": -1, + "load_error": "LEADER_NOT_AVAILABLE", + "error": ["LEADER_NOT_AVAILABLE"] + } + ] + }, + { + "summary": { + "name": "normal-topic", + "internal": false, + "partitions": 1, + "replicas": 1, + "error": "" + }, + "configs": [ + { + "key": "retention.ms", + "value": "86400000", + "source": "DEFAULT_CONFIG" + } + ], + "partitions": [ + { + "partition": 0, + "leader": 1, + "epoch": 0, + "replicas": [1], + "log_start_offset": 0, + "high_watermark": 150 + } + ] + } + ]`, + expectedReturn: true, + }, + { + name: "YAML format - single topic", + format: "yaml", + topics: []describedTopic{ + { + Summary: describeTopicSummary{ + Name: "test-topic", + Internal: false, + Partitions: 1, + Replicas: 1, + }, + Configs: []describeTopicConfig{ + {Key: "compression.type", Value: "producer", Source: "DEFAULT_CONFIG"}, + }, + Partitions: []describeTopicPartition{ + {Partition: 0, Leader: 1, Replicas: []int32{1}}, + }, + }, }, - expRows: [][]interface{}{ - {int32(0), int32(1), int32(-1), []int32{0}, []int32{}, "-", int64(0), int64(1), int64(1)}, - {int32(1), int32(0), int32(0), []int32{0, 1}, []int32{2, 3}, kerr.ErrorForCode(1), kerr.TypedErrorForCode(9).Message, int64(1), int64(2)}, + expectedOutput: `- summary: + name: test-topic + internal: false + partitions: 1 + replicas: 1 + error: "" + configs: + - key: compression.type + value: producer + source: DEFAULT_CONFIG + partitions: + - partition: 0 + leader: 1 + epoch: 0 + replicas: + - 1 + log_start_offset: 0 + high_watermark: 0`, + expectedReturn: true, + }, + { + name: "YAML format - topic with error", + format: "yaml", + topics: []describedTopic{ + { + Summary: describeTopicSummary{ + Name: "error-topic", + Internal: false, + Partitions: 0, + Replicas: 0, + Error: "UNKNOWN_TOPIC_OR_PARTITION", + }, + }, }, + expectedOutput: `- summary: + name: error-topic + internal: false + partitions: 0 + replicas: 0 + error: UNKNOWN_TOPIC_OR_PARTITION + configs: [] + partitions: []`, + expectedReturn: true, }, + { + name: "Text format - should return false", + format: "text", + topics: []describedTopic{ + { + Summary: describeTopicSummary{ + Name: "test-topic", + Internal: false, + Partitions: 1, + Replicas: 1, + }, + }, + }, + expectedOutput: "", + expectedReturn: false, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + var buf bytes.Buffer + f := config.OutFormatter{Kind: tc.format} + result := printDescribedTopicsFormatter(f, tc.topics, &buf) + + assert.Equal(t, tc.expectedReturn, result) + + if tc.expectedReturn { + switch tc.format { + case "json": + var expected, actual interface{} + err := json.Unmarshal([]byte(tc.expectedOutput), &expected) + require.NoError(t, err) + err = json.Unmarshal(buf.Bytes(), &actual) + require.NoError(t, err) + assert.Equal(t, expected, actual) + case "yaml": + assert.Equal(t, strings.TrimRight(tc.expectedOutput, "\n"), strings.TrimRight(buf.String(), "\n")) + default: + assert.Equal(t, tc.expectedOutput, buf.String()) + } + } else { + assert.Empty(t, buf.String()) + } + }) + } +} + +func TestPrintDescribedTopics(t *testing.T) { + testCases := []struct { + name string + summary bool + configs bool + partitions bool + topics []describedTopic + expectedOutput string + }{ { - name: "no rows", + name: "Print all sections", + summary: true, + configs: true, + partitions: true, + topics: []describedTopic{ + { + Summary: describeTopicSummary{ + Name: "test-topic", + Internal: false, + Partitions: 2, + Replicas: 3, + }, + Configs: []describeTopicConfig{ + {Key: "retention.ms", Value: "604800000", Source: "DEFAULT_CONFIG"}, + }, + Partitions: []describeTopicPartition{ + {Partition: 0, Leader: 1, Replicas: []int32{1, 2, 3}}, + {Partition: 1, Leader: 2, Replicas: []int32{2, 3, 1}}, + }, + }, + }, + expectedOutput: `SUMMARY +======= +NAME test-topic +PARTITIONS 2 +REPLICAS 3 - inMeta: []kmsg.MetadataResponseTopicPartition{}, - inOffsets: []startStableEndOffset{}, - expHeaders: []string{ - "partition", - "leader", - "epoch", - "replicas", - "log-start-offset", - "high-watermark", +CONFIGS +======= +KEY VALUE SOURCE +retention.ms 604800000 DEFAULT_CONFIG + +PARTITIONS +========== +PARTITION LEADER EPOCH REPLICAS LOG-START-OFFSET HIGH-WATERMARK +0 1 0 [1 2 3] +1 2 0 [2 3 1] +`, + }, + { + name: "Print only summary", + summary: true, + configs: false, + partitions: false, + topics: []describedTopic{ + { + Summary: describeTopicSummary{ + Name: "test-topic", + Internal: true, + Partitions: 1, + Replicas: 1, + }, + }, + }, + expectedOutput: `NAME test-topic +INTERNAL true +PARTITIONS 1 +REPLICAS 1 +`, + }, + { + name: "Print summary and configs", + summary: true, + configs: true, + partitions: false, + topics: []describedTopic{ + { + Summary: describeTopicSummary{ + Name: "test-topic", + Internal: false, + Partitions: 1, + Replicas: 1, + }, + Configs: []describeTopicConfig{ + {Key: "cleanup.policy", Value: "delete", Source: "DEFAULT_CONFIG"}, + {Key: "compression.type", Value: "producer", Source: "DYNAMIC_TOPIC_CONFIG"}, + }, + }, + }, + expectedOutput: `SUMMARY +======= +NAME test-topic +PARTITIONS 1 +REPLICAS 1 + +CONFIGS +======= +KEY VALUE SOURCE +cleanup.policy delete DEFAULT_CONFIG +compression.type producer DYNAMIC_TOPIC_CONFIG +`, + }, + { + name: "Print with errors", + summary: true, + configs: true, + partitions: true, + topics: []describedTopic{ + { + Summary: describeTopicSummary{ + Name: "error-topic", + Internal: false, + Partitions: 0, + Replicas: 0, + Error: "UNKNOWN_TOPIC_OR_PARTITION", + }, + }, }, + expectedOutput: `SUMMARY +======= +NAME error-topic +PARTITIONS 0 +ERROR UNKNOWN_TOPIC_OR_PARTITION + +CONFIGS +======= +KEY VALUE SOURCE + +PARTITIONS +========== +PARTITION LEADER EPOCH REPLICAS LOG-START-OFFSET HIGH-WATERMARK +`, }, + } + + for _, tc := range testCases { + // Janky way to test the output of a function that prints to stdout. Would be preferable to just pass in a buf and check that. + t.Run(tc.name, func(t *testing.T) { + // Redirect stdout to capture output + old := os.Stdout + r, w, _ := os.Pipe() + os.Stdout = w + + printDescribedTopics(tc.summary, tc.configs, tc.partitions, tc.topics) + + // Restore stdout + w.Close() + os.Stdout = old - // - } { - t.Run(test.name, func(t *testing.T) { - headers := describePartitionsHeaders( - test.inMeta, - test.inOffsets, - ) - rows := describePartitionsRows( - test.inMeta, - test.inOffsets, - ) + var buf bytes.Buffer + io.Copy(&buf, r) + output := buf.String() - require.Equal(t, test.expHeaders, headers, "headers") - require.Equal(t, test.expRows, rows, "rows") + // Compare output + assert.Equal(t, tc.expectedOutput, output) }) } } diff --git a/src/go/rpk/pkg/config/format.go b/src/go/rpk/pkg/config/format.go index 8f82f558aab0..37e03a088bad 100644 --- a/src/go/rpk/pkg/config/format.go +++ b/src/go/rpk/pkg/config/format.go @@ -63,6 +63,10 @@ func (f *OutFormatter) Help(t any) (string, bool) { return s, true } +func (f *OutFormatter) IsText() bool { + return f.Kind != "json" && f.Kind != "yaml" +} + func formatType(t any, includeTypeName bool) (string, error) { types := make(map[reflect.Type]struct{}) From 0e20ff3b43b9e827f3d81abc44593b1b7850baed Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Tue, 3 Dec 2024 16:19:37 -0500 Subject: [PATCH 019/229] `rptest`: fix condition in `wait_for_translation_until_offset()` The wrong variable was being checked for `None` here. --- tests/rptest/tests/datalake/datalake_services.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/rptest/tests/datalake/datalake_services.py b/tests/rptest/tests/datalake/datalake_services.py index 14351355d8dd..5e7f728b2041 100644 --- a/tests/rptest/tests/datalake/datalake_services.py +++ b/tests/rptest/tests/datalake/datalake_services.py @@ -160,7 +160,7 @@ def translation_done(): self.redpanda.logger.debug( f"Current translated offsets: {offsets}") return all([ - offset and offset <= max_offset + max_offset and offset <= max_offset for _, max_offset in offsets.items() ]) From 4adc8058ab0b566b691cfa860ac7ff2b51679fe2 Mon Sep 17 00:00:00 2001 From: Brandon Allard Date: Tue, 3 Dec 2024 16:23:34 -0500 Subject: [PATCH 020/229] tests/docker: update omb ducktape dep --- tests/docker/ducktape-deps/omb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/docker/ducktape-deps/omb b/tests/docker/ducktape-deps/omb index e52bf40e4ddb..844014e1ddf8 100644 --- a/tests/docker/ducktape-deps/omb +++ b/tests/docker/ducktape-deps/omb @@ -2,5 +2,5 @@ set -e git -C /opt clone https://github.com/redpanda-data/openmessaging-benchmark.git cd /opt/openmessaging-benchmark -git reset --hard 50aba841760313fdd86586f0e74484d1721513e4 +git reset --hard 092eb7c78a106906f21288b99b1e2a4c3cfc652a mvn clean package -DskipTests From f70962c2e199f90a94ce3b66eef2e471c1a2034f Mon Sep 17 00:00:00 2001 From: Noah Watkins Date: Tue, 3 Dec 2024 17:21:26 -0800 Subject: [PATCH 021/229] bazel: add configuration for coverage Signed-off-by: Noah Watkins --- .bazelrc | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/.bazelrc b/.bazelrc index a6e0ff7ff663..b6b8c00e1ca6 100644 --- a/.bazelrc +++ b/.bazelrc @@ -94,6 +94,22 @@ build:release --@seastar//:stack_guards=False build:stamp --stamp --workspace_status_command=./bazel/stamp_vars.sh +# ================================= +# Coverage +# ================================= +coverage --config=coverage +build:coverage --action_env=BAZEL_USE_LLVM_NATIVE_COVERAGE=1 +build:coverage --action_env=GCOV=llvm-profdata +build:coverage --copt=-DNDEBUG +build:coverage --define=dynamic_link_tests=true +build:coverage --combined_report=lcov +build:coverage --experimental_use_llvm_covmap +build:coverage --experimental_generate_llvm_lcov +build:coverage --experimental_split_coverage_postprocessing +build:coverage --experimental_fetch_all_coverage_outputs +build:coverage --collect_code_coverage +build:coverage --instrumentation_filter="^//src/v[/:]" + # ================================= # Testing # ================================= From 1e7f4b7157e596c7fabec775a01bb8c2741118b6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gell=C3=A9rt=20Peresztegi-Nagy?= Date: Wed, 4 Dec 2024 10:49:13 +0000 Subject: [PATCH 022/229] dt/license_enforcement: delay disabling trial license Previously the test relied on the fact that the previous major version (v24.2) did not do strict license enforcement. We can no longer rely on that since the head version became v25.1 and the previous major version v24.3 which does do strict license enforcement. --- .../rptest/tests/license_enforcement_test.py | 26 ++++++++++++++----- 1 file changed, 20 insertions(+), 6 deletions(-) diff --git a/tests/rptest/tests/license_enforcement_test.py b/tests/rptest/tests/license_enforcement_test.py index bb6f12bef562..19abda43f9ae 100644 --- a/tests/rptest/tests/license_enforcement_test.py +++ b/tests/rptest/tests/license_enforcement_test.py @@ -49,9 +49,6 @@ def setUp(self): ) def test_license_enforcement(self, clean_node_before_recovery, clean_node_after_recovery): - self.redpanda.set_environment( - {'__REDPANDA_DISABLE_BUILTIN_TRIAL_LICENSE': True}) - installer = self.redpanda._installer prev_version = installer.highest_from_prior_feature_version( RedpandaInstaller.HEAD) @@ -73,6 +70,16 @@ def test_license_enforcement(self, clean_node_before_recovery, self.redpanda.set_cluster_config( {"partition_autobalancing_mode": "continuous"}) + self.logger.info( + "Disabling the trial license to simulate that the license expired") + self.redpanda.set_environment( + {'__REDPANDA_DISABLE_BUILTIN_TRIAL_LICENSE': True}) + self.redpanda.restart_nodes(self.redpanda.nodes) + self.redpanda.wait_until(self.redpanda.healthy, + timeout_sec=60, + backoff_sec=1, + err_msg="The cluster hasn't stabilized") + first_upgraded = self.redpanda.nodes[0] self.logger.info( f"Upgrading node {first_upgraded} expecting it to crash") @@ -112,9 +119,6 @@ def test_license_enforcement(self, clean_node_before_recovery, @cluster(num_nodes=5, log_allow_list=LOG_ALLOW_LIST) @matrix(clean_node_before_upgrade=[False, True]) def test_escape_hatch_license_variable(self, clean_node_before_upgrade): - self.redpanda.set_environment( - {'__REDPANDA_DISABLE_BUILTIN_TRIAL_LICENSE': True}) - installer = self.redpanda._installer prev_version = installer.highest_from_prior_feature_version( RedpandaInstaller.HEAD) @@ -136,6 +140,16 @@ def test_escape_hatch_license_variable(self, clean_node_before_upgrade): self.redpanda.set_cluster_config( {"partition_autobalancing_mode": "continuous"}) + self.logger.info( + "Disabling the trial license to simulate that the license expired") + self.redpanda.set_environment( + {'__REDPANDA_DISABLE_BUILTIN_TRIAL_LICENSE': True}) + self.redpanda.restart_nodes(self.redpanda.nodes) + self.redpanda.wait_until(self.redpanda.healthy, + timeout_sec=60, + backoff_sec=1, + err_msg="The cluster hasn't stabilized") + first_upgraded = self.redpanda.nodes[0] first_upgraded_id = self.redpanda.node_id(first_upgraded) self.logger.info( From 8852d2e94e4ec348ce1eb6ecc9852fd68e4c6404 Mon Sep 17 00:00:00 2001 From: Noah Watkins Date: Wed, 4 Dec 2024 07:43:06 -0800 Subject: [PATCH 023/229] tests: add reasoning for skip fips mode Signed-off-by: Noah Watkins --- tests/rptest/utils/mode_checks.py | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/tests/rptest/utils/mode_checks.py b/tests/rptest/utils/mode_checks.py index 40de4d2f6ece..2962f0e8119d 100644 --- a/tests/rptest/utils/mode_checks.py +++ b/tests/rptest/utils/mode_checks.py @@ -84,7 +84,20 @@ def in_fips_environment() -> bool: def skip_fips_mode(*args, **kwargs): """ - Test method decorator which signals to the test runner to ignore a given test. + Decorator indicating that the test should not run in FIPS mode. + + Ideally all tests should run in FIPS mode. The following are some situations + in which skipping FIPS mode is required. + + * Exercising a known non-FIPS condition (e.g. virtual-host vs path style + testing). + + * We can't test it in FIPS mode because of infrastructure issues, but the + implementation doesn't change between FIPS and non-FIPS (auditing & OCSF + server). + + * Certain license tests (since enabling FIPS mode enables enterprise license + requirement). Example:: From f0a44288ec55d80d27551d6db61956ee02b9863a Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Mon, 2 Dec 2024 23:36:18 +0100 Subject: [PATCH 024/229] c/leader_balancer: implement and use even_node_load_constraint Leader balancer treats all CPU cores available in the cluster as independent and tries to balance leadership among them and not among nodes (i.e. the objective is that each core has the same number of leaders, and not each node). This leads to unintuitive results, especially when the number of raft groups is comparable to the number of available shards. Add a low-priority node balancing constraint to fix that. --- .../scheduling/leader_balancer_constraints.cc | 26 +++++++++++++++++++ .../scheduling/leader_balancer_constraints.h | 22 ++++++++++++++++ .../scheduling/leader_balancer_random.h | 20 ++++++++++---- 3 files changed, 63 insertions(+), 5 deletions(-) diff --git a/src/v/cluster/scheduling/leader_balancer_constraints.cc b/src/v/cluster/scheduling/leader_balancer_constraints.cc index b6b45b5b29e2..9c8c2358c0c1 100644 --- a/src/v/cluster/scheduling/leader_balancer_constraints.cc +++ b/src/v/cluster/scheduling/leader_balancer_constraints.cc @@ -253,4 +253,30 @@ double pinning_constraint::evaluate_internal(const reassignment& r) { return diff; } +even_node_load_constraint::even_node_load_constraint(const shard_index& si) { + for (const auto& [bs, leaders] : si.shards()) { + auto& info = _node2info[bs.node_id]; + info.shards += 1; + info.leaders += leaders.size(); + } +} + +void even_node_load_constraint::update_index(const reassignment& r) { + _node2info[r.from.node_id].leaders -= 1; + _node2info[r.to.node_id].leaders += 1; +} + +double even_node_load_constraint::evaluate_internal(const reassignment& r) { + // Positive if the reassignment makes the weighted distribution more + // balanced. In particular, it is positive iff + // (from_info.leaders/from_info.shards - to_info.leaders/to_info.shards)^2 + // decreases as a result of the reassignment (showing equivalence is + // straightforward with some algebraic transforms). + const auto& from_info = _node2info[r.from.node_id]; + const auto& to_info = _node2info[r.to.node_id]; + return 2 * (double(from_info.leaders) * to_info.shards + - double(to_info.leaders) * from_info.shards) - from_info.shards + - to_info.shards; +} + } // namespace cluster::leader_balancer_types diff --git a/src/v/cluster/scheduling/leader_balancer_constraints.h b/src/v/cluster/scheduling/leader_balancer_constraints.h index f53f3809acb6..f503ef6ada99 100644 --- a/src/v/cluster/scheduling/leader_balancer_constraints.h +++ b/src/v/cluster/scheduling/leader_balancer_constraints.h @@ -288,4 +288,26 @@ class pinning_constraint final : public soft_constraint { preference_index _preference_idx; }; +// Constraint implementing node-wise balanced load objective (i.e. equal number +// of leaders on each node as opposed to on each shard). +class even_node_load_constraint final + : public soft_constraint + , public index { +public: + explicit even_node_load_constraint(const shard_index& si); + + void update_index(const reassignment& r) override; + +private: + double evaluate_internal(const reassignment& r) override; + +private: + struct node_info { + int32_t leaders = 0; + uint32_t shards = 0; + }; + + absl::flat_hash_map _node2info; +}; + } // namespace cluster::leader_balancer_types diff --git a/src/v/cluster/scheduling/leader_balancer_random.h b/src/v/cluster/scheduling/leader_balancer_random.h index a97858a31e1c..71c0b29d6074 100644 --- a/src/v/cluster/scheduling/leader_balancer_random.h +++ b/src/v/cluster/scheduling/leader_balancer_random.h @@ -112,7 +112,8 @@ class random_hill_climbing_strategy final : public leader_balancer_strategy { , _si(std::make_unique(std::move(index))) , _reassignments(_si->shards()) , _etdc(*_group2topic, *_si, *_mi) - , _eslc(*_si, *_mi) { + , _eslc(*_si, *_mi) + , _enlc(*_si) { if (preference_idx) { _pinning_constr.emplace( *_group2topic, std::move(preference_idx.value())); @@ -156,13 +157,20 @@ class random_hill_climbing_strategy final : public leader_balancer_strategy { } } - auto balancing_diff = _etdc.evaluate(reassignment) - + _eslc.evaluate(reassignment); - if (balancing_diff <= error_jitter) { + auto shard_load_diff = _etdc.evaluate(reassignment) + + _eslc.evaluate(reassignment); + if (shard_load_diff < -error_jitter) { continue; + } else if (shard_load_diff > error_jitter) { + return reassignment_opt; } - return reassignment_opt; + auto node_load_diff = _enlc.evaluate(reassignment); + if (node_load_diff < -error_jitter) { + continue; + } else if (node_load_diff > error_jitter) { + return reassignment_opt; + } } return std::nullopt; @@ -171,6 +179,7 @@ class random_hill_climbing_strategy final : public leader_balancer_strategy { void apply_movement(const reassignment& reassignment) override { _etdc.update_index(reassignment); _eslc.update_index(reassignment); + _enlc.update_index(reassignment); _mi->update_index(reassignment); _si->update_index(reassignment); _reassignments.update_index(reassignment); @@ -192,6 +201,7 @@ class random_hill_climbing_strategy final : public leader_balancer_strategy { std::optional _pinning_constr; even_topic_distribution_constraint _etdc; even_shard_load_constraint _eslc; + even_node_load_constraint _enlc; }; } // namespace cluster::leader_balancer_types From f93b43a41d609f408548d40036b4a753dd34c0d0 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Mon, 2 Dec 2024 23:43:29 +0100 Subject: [PATCH 025/229] c/leader_balancer/ut: fix copy_cluster_index helper Previously it didn't copy empty shards in the index. --- src/v/cluster/tests/leader_balancer_test_utils.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/v/cluster/tests/leader_balancer_test_utils.h b/src/v/cluster/tests/leader_balancer_test_utils.h index d17729ed9943..f15b089df81b 100644 --- a/src/v/cluster/tests/leader_balancer_test_utils.h +++ b/src/v/cluster/tests/leader_balancer_test_utils.h @@ -56,8 +56,9 @@ inline cluster::leader_balancer_strategy::index_type copy_cluster_index( cluster::leader_balancer_strategy::index_type index; for (const auto& [bs, leaders] : c_index) { + auto& gid2replicas = index[bs]; for (const auto& [group_id, replicas] : leaders) { - index[bs][group_id] = replicas; + gid2replicas[group_id] = replicas; } } From 6db17303f1737096103bc759aae1562ed7be5acc Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 4 Dec 2024 00:12:04 +0100 Subject: [PATCH 026/229] c/leader_balancer: sort stats by shard Makes it easier to read the stats. --- src/v/cluster/scheduling/leader_balancer_constraints.cc | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/v/cluster/scheduling/leader_balancer_constraints.cc b/src/v/cluster/scheduling/leader_balancer_constraints.cc index 9c8c2358c0c1..e3dd9d96f3bb 100644 --- a/src/v/cluster/scheduling/leader_balancer_constraints.cc +++ b/src/v/cluster/scheduling/leader_balancer_constraints.cc @@ -218,6 +218,10 @@ std::vector even_shard_load_constraint::stats() const { // oddly, absl::btree::size returns a signed type return shard_load{e->first, static_cast(e->second.size())}; }); + std::sort( + ret.begin(), ret.end(), [](const shard_load& l, const shard_load& r) { + return l.shard < r.shard; + }); return ret; } From 5695541ea81a28b258c739005becde5d7bfd97dc Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Tue, 3 Dec 2024 00:27:03 +0100 Subject: [PATCH 027/229] c/leader_balancer: add even shard load/uneven node load utest --- .../tests/leader_balancer_constraints_test.cc | 71 +++++++++++++++++++ 1 file changed, 71 insertions(+) diff --git a/src/v/cluster/tests/leader_balancer_constraints_test.cc b/src/v/cluster/tests/leader_balancer_constraints_test.cc index a6ddec80e83d..491eac52e8d7 100644 --- a/src/v/cluster/tests/leader_balancer_constraints_test.cc +++ b/src/v/cluster/tests/leader_balancer_constraints_test.cc @@ -347,3 +347,74 @@ BOOST_AUTO_TEST_CASE(topic_skew_error) { BOOST_REQUIRE(post_topic_error <= pre_topic_error); BOOST_REQUIRE(post_shard_error <= pre_shard_error); } + +BOOST_AUTO_TEST_CASE(even_shard_uneven_node_load) { + index_type idx; + cluster::leader_balancer_types::group_id_to_topic_id g2topic; + using cluster::leader_balancer_types::topic_id_t; + + // 3-node cluster, node 2 has twice as many shards as nodes 0 and 1 + const int n_partitions = 10; + for (int n = 0; n < 3; ++n) { + uint32_t n_shards = (n <= 1 ? n_partitions : n_partitions * 2); + for (uint32_t s = 0; s < n_shards; ++s) { + idx[model::broker_shard{model::node_id(n), s}]; + } + } + + // 2 topics, n_partitions partitions each + for (int g = 0; g < n_partitions; ++g) { + raft::group_id group_id(g); + g2topic[group_id] = topic_id_t{123}; + + cluster::replicas_t replicas; + for (int n = 0; n < 3; ++n) { + // each partition on its own shard + uint32_t shard = g; + replicas.push_back(model::broker_shard{model::node_id{n}, shard}); + } + // all leaders on on node 0 + idx[replicas[0]][group_id] = replicas; + } + + for (int g = n_partitions; g < 2 * n_partitions; ++g) { + raft::group_id group_id(g); + g2topic[group_id] = topic_id_t{345}; + + cluster::replicas_t replicas; + for (int n = 0; n < 3; ++n) { + // each partition on its own shard + uint32_t shard = (n <= 1 ? (g - n_partitions) : g); + replicas.push_back(model::broker_shard{model::node_id{n}, shard}); + } + // all leaders on on node 1 + idx[replicas[1]][group_id] = replicas; + } + + cluster::leader_balancer_types::shard_index shard_idx(std::move(idx)); + + auto strategy = lbt::random_hill_climbing_strategy( + leader_balancer_test_utils::copy_cluster_index(shard_idx.shards()), + std::move(g2topic), + cluster::leader_balancer_types::muted_index({}, {}), + std::nullopt); + + // If we view shards as independent, this distribution is perfectly balanced + // (each shard has either 1 or 0 leaders), but node 2 has 0 leaders. Check + // that the balancing strategy fixes this. + + while (auto movement_opt = strategy.find_movement({})) { + strategy.apply_movement(*movement_opt); + shard_idx.update_index(*movement_opt); + } + + std::map node_stats; + for (const auto& s : strategy.stats()) { + node_stats[s.shard.node_id] += s.leaders; + } + + // node 2 is twice bigger so hosts twice as many partitions. + BOOST_CHECK_EQUAL(node_stats[model::node_id{0}], n_partitions / 2); + BOOST_CHECK_EQUAL(node_stats[model::node_id{1}], n_partitions / 2); + BOOST_CHECK_EQUAL(node_stats[model::node_id{2}], n_partitions); +} From 049e25874c3068b8300c742a2ec87de209218718 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Tue, 3 Dec 2024 00:28:08 +0100 Subject: [PATCH 028/229] tests: increase wait_for_racks timeout in leadership_transfer_test Give more time for muted groups to unmute. --- tests/rptest/tests/leadership_transfer_test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/rptest/tests/leadership_transfer_test.py b/tests/rptest/tests/leadership_transfer_test.py index f16eac5ee369..27d8e6563ca4 100644 --- a/tests/rptest/tests/leadership_transfer_test.py +++ b/tests/rptest/tests/leadership_transfer_test.py @@ -474,7 +474,7 @@ def test_leadership_pinning(self): "foo": {"A"}, "bar": {"B", "C"} }, - timeout_sec=30) + timeout_sec=60) # Decrease idle timeout to not wait too long after nodes are killed self.redpanda.set_cluster_config({"enable_leader_balancer": False}) @@ -504,7 +504,7 @@ def test_leadership_pinning(self): "bar": {"C"} }, check_balance=False, - timeout_sec=30) + timeout_sec=60) self.logger.info("unset topic configs") @@ -515,7 +515,7 @@ def test_leadership_pinning(self): "foo": {"A"}, "bar": {"A"} }, - timeout_sec=30) + timeout_sec=60) self.logger.info("unset default preference") From 4b4f6a2ac8e28e007983de7803e99dc6eefa3bd1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gell=C3=A9rt=20Peresztegi-Nagy?= Date: Wed, 4 Dec 2024 17:07:21 +0000 Subject: [PATCH 029/229] cluster: extract `allocation_node::is_internal_topic` Pure refactor. Extract for reuse in the next commit. --- src/v/cluster/scheduling/allocation_node.cc | 30 +++++++++++---------- src/v/cluster/scheduling/allocation_node.h | 7 +++++ 2 files changed, 23 insertions(+), 14 deletions(-) diff --git a/src/v/cluster/scheduling/allocation_node.cc b/src/v/cluster/scheduling/allocation_node.cc index 2cf774570fcb..2ac023bbc17f 100644 --- a/src/v/cluster/scheduling/allocation_node.cc +++ b/src/v/cluster/scheduling/allocation_node.cc @@ -50,24 +50,26 @@ allocation_node::allocation_node( }); } -bool allocation_node::is_full( - const model::ntp& ntp, bool will_add_allocation) const { - // Internal topics are excluded from checks to prevent allocation failures - // when creating them. This is okay because they are fairly small in number - // compared to kafka user topic partitions. +bool allocation_node::is_internal_topic( + const config::binding>& internal_kafka_topics, + model::topic_namespace_view ntp) { auto is_internal_ns = ntp.ns == model::redpanda_ns || ntp.ns == model::kafka_internal_namespace; if (is_internal_ns) { - return false; + return true; } - const auto& internal_topics = _internal_kafka_topics(); - auto is_internal_topic = ntp.ns == model::kafka_namespace - && std::any_of( - internal_topics.cbegin(), - internal_topics.cend(), - [&ntp](const ss::sstring& topic) { - return topic == ntp.tp.topic(); - }); + const auto& internal_topics = internal_kafka_topics(); + return ntp.ns == model::kafka_namespace + && std::any_of( + internal_topics.cbegin(), + internal_topics.cend(), + [&ntp](const ss::sstring& topic) { return topic == ntp.tp; }); +} + +bool allocation_node::is_full( + const model::ntp& ntp, bool will_add_allocation) const { + auto is_internal_topic = allocation_node::is_internal_topic( + _internal_kafka_topics, model::topic_namespace_view{ntp}); auto count = _allocated_partitions; if (will_add_allocation) { diff --git a/src/v/cluster/scheduling/allocation_node.h b/src/v/cluster/scheduling/allocation_node.h index 57bb20eaad1c..9c0d2f8f6f27 100644 --- a/src/v/cluster/scheduling/allocation_node.h +++ b/src/v/cluster/scheduling/allocation_node.h @@ -95,6 +95,13 @@ class allocation_node { } bool is_full(const model::ntp&, bool will_add_allocation) const; + // Internal topics are excluded from checks to prevent allocation failures + // when creating them. This is okay because they are fairly small in number + // compared to kafka user topic partitions. + static bool is_internal_topic( + const config::binding>& internal_kafka_topics, + model::topic_namespace_view ntp); + private: friend allocation_state; From 19bc4f200c296576b5eebd6e3f0ab2bab02ead67 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gell=C3=A9rt=20Peresztegi-Nagy?= Date: Mon, 2 Dec 2024 19:46:32 +0000 Subject: [PATCH 030/229] cluster: exclude internal topics from `check_cluster_limits` Internal topics are excluded from checks to prevent allocation failures when creating them. This is to ensure that lazy-allocated internal topics (eg. the transactions topic) can always be created. This excludes them from the global `check_cluster_limits`. There has already been a fixture test to effectively test that internal topics are excluded from the limit checks, however, it erroniously relied on the fact that the shard0 reservations were not considered in `check_cluster_limits` to allow the test to pass. (See `allocation_over_capacity` and the previous commit.) This adds a new test to validate that internal topics can be created even with partitions that are above the global shard0 reservation. --- .../cluster/scheduling/partition_allocator.cc | 11 +++++++--- .../cluster/scheduling/partition_allocator.h | 4 +++- .../tests/partition_allocator_tests.cc | 22 +++++++++++++++++++ 3 files changed, 33 insertions(+), 4 deletions(-) diff --git a/src/v/cluster/scheduling/partition_allocator.cc b/src/v/cluster/scheduling/partition_allocator.cc index f45df6f08fae..8e501f702f16 100644 --- a/src/v/cluster/scheduling/partition_allocator.cc +++ b/src/v/cluster/scheduling/partition_allocator.cc @@ -108,11 +108,16 @@ allocation_constraints partition_allocator::default_constraints() { * with partitions that cannot be re-accommodated on smaller peers). */ std::error_code partition_allocator::check_cluster_limits( - const uint64_t new_partitions_replicas_requested) const { + const uint64_t new_partitions_replicas_requested, + const model::topic_namespace& topic) const { if (_members.local().nodes().empty()) { // Empty members table, we're probably running in a unit test return errc::success; } + if (allocation_node::is_internal_topic(_internal_kafka_topics, topic)) { + return errc::success; + } + // Calculate how many partition-replicas already exist, so that we can // check if the new topic would take us past any limits. uint64_t existent_partitions{0}; @@ -242,7 +247,7 @@ partition_allocator::allocate(simple_allocation_request simple_req) { const uint64_t create_count = static_cast(simple_req.additional_partitions) * static_cast(simple_req.replication_factor); - auto cluster_errc = check_cluster_limits(create_count); + auto cluster_errc = check_cluster_limits(create_count, simple_req.tp_ns); if (cluster_errc) { co_return cluster_errc; } @@ -274,7 +279,7 @@ partition_allocator::allocate(allocation_request request) { } } - auto cluster_errc = check_cluster_limits(create_count); + auto cluster_errc = check_cluster_limits(create_count, request._nt); if (cluster_errc) { co_return cluster_errc; } diff --git a/src/v/cluster/scheduling/partition_allocator.h b/src/v/cluster/scheduling/partition_allocator.h index 14545c5951b4..2574da6f30b7 100644 --- a/src/v/cluster/scheduling/partition_allocator.h +++ b/src/v/cluster/scheduling/partition_allocator.h @@ -18,6 +18,7 @@ #include "cluster/scheduling/types.h" #include "config/property.h" #include "features/fwd.h" +#include "model/metadata.h" namespace cluster { @@ -145,7 +146,8 @@ class partition_allocator { // new_partitions_replicas_requested represents the total number of // partitions requested by a request. i.e. partitions * replicas requested. std::error_code check_cluster_limits( - const uint64_t new_partitions_replicas_requested) const; + const uint64_t new_partitions_replicas_requested, + const model::topic_namespace& topic) const; ss::future> do_allocate(allocation_request); diff --git a/src/v/cluster/tests/partition_allocator_tests.cc b/src/v/cluster/tests/partition_allocator_tests.cc index 80ac265f1b95..8ef338221367 100644 --- a/src/v/cluster/tests/partition_allocator_tests.cc +++ b/src/v/cluster/tests/partition_allocator_tests.cc @@ -194,6 +194,28 @@ FIXTURE_TEST(allocation_over_capacity, partition_allocator_fixture) { allocator().allocate(make_allocation_request(int_2, 1, 1)).get()); } +FIXTURE_TEST( + allocation_over_capacity_without_shard0, partition_allocator_fixture) { + // Disable shard0 reservations + partitions_reserve_shard0.update(0); + + register_node(0, 6); + register_node(1, 6); + register_node(2, 6); + + saturate_all_machines(); + auto gr = allocator().state().last_group_id(); + BOOST_REQUIRE( + allocator().allocate(make_allocation_request(1, 1)).get().has_error()); + // group id hasn't changed + BOOST_REQUIRE_EQUAL(allocator().state().last_group_id(), gr); + + // Make the topic internal and retry, should work. + kafka_internal_topics.update({tn.tp()}); + BOOST_REQUIRE(allocator().allocate(make_allocation_request(1, 1)).get()); + BOOST_REQUIRE_GT(allocator().state().last_group_id(), gr); +} + FIXTURE_TEST(max_allocation, partition_allocator_fixture) { register_node(0, 2); register_node(1, 2); From a7d42d2779c3f8ee53ee04198852f1f118405957 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Wed, 4 Dec 2024 17:38:12 +0000 Subject: [PATCH 031/229] archival: calculate time based retention from first addressable segment apply_retention() and garbage_collect() are 2 distinct routines. The former computes the new start offset while the later removes the physical files and removes the segment entries from the manifest. However, if we bail out of the second routine (e.g. timeout, leadership lost) we will call apply_retention() again. If we apply the retention rules starting with the first segment entry in the manifest it can happen that we reach the same result (i.e. same new start offset). When this happens, we replicate a new update_start_offset command to the archival stm which tries to advance_start_offset to the current start offset. Currently, we log an error in that case from `archival_metadata_stm::apply_update_start_offset`. Nothing is broken beside printing that error log line. ``` if (!_manifest->advance_start_offset(so.start_offset)) { vlog( _logger.error, "Can't truncate manifest up to offset {}, offset out of range", so.start_offset); } ``` This commit avoids this annoyance and avoids unnecessary redundant work for retention computation and replicating commands by computing retention from the first addressable segment rather than from the first segment in the manifest. --- src/v/cluster/archival/retention_calculator.cc | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/v/cluster/archival/retention_calculator.cc b/src/v/cluster/archival/retention_calculator.cc index 544bec8e3891..65d922ec20b9 100644 --- a/src/v/cluster/archival/retention_calculator.cc +++ b/src/v/cluster/archival/retention_calculator.cc @@ -109,7 +109,8 @@ std::optional retention_calculator::factory( if ( manifest.size() > 0 - && manifest.begin()->max_timestamp < oldest_allowed_timestamp) { + && manifest.first_addressable_segment()->max_timestamp + < oldest_allowed_timestamp) { strats.push_back( std::make_unique(oldest_allowed_timestamp)); } From 3b80f02a8145c084b81abd3339fdd9a8bec6a055 Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Wed, 4 Dec 2024 10:38:17 -0800 Subject: [PATCH 032/229] [CORE-8081] rptest: fix config_profile_verify_test null handling We see the given error if a given profile's config contains a 'null' string value, as is done by install pack YAML files. ``` [DEBUG - 2024-10-29 10:04:39,665 - config_profile_verify_test - _check_rp_config - lineno:72]: asserting cluster config key kafka_rpc_server_tcp_recv_buf has expected value: null actual: None ... [INFO - 2024-10-29 10:04:39,667 - runner_client - log - lineno:294]: RunnerClient: rptest.redpanda_cloud_tests.config_profile_verify_test.ConfigProfileVerifyTest.test_config_profile_verify: FAIL: AssertionError() Traceback (most recent call last): File "/opt/.ducktape-venv/lib/python3.10/site-packages/ducktape/tests/runner_client.py", line 186, in _do_run data = self.run_test() File "/opt/.ducktape-venv/lib/python3.10/site-packages/ducktape/tests/runner_client.py", line 246, in run_test return self.test_context.function(self.test) File "/home/ubuntu/redpanda/tests/rptest/redpanda_cloud_tests/config_profile_verify_test.py", line 59, in test_config_profile_verify self._check_rp_config() File "/home/ubuntu/redpanda/tests/rptest/redpanda_cloud_tests/config_profile_verify_test.py", line 76, in _check_rp_config assert False AssertionError ``` This commit updates the check to expect None if the expected value is "null". --- .../config_profile_verify_test.py | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/tests/rptest/redpanda_cloud_tests/config_profile_verify_test.py b/tests/rptest/redpanda_cloud_tests/config_profile_verify_test.py index 4cabdf768e87..64631a2c2b20 100644 --- a/tests/rptest/redpanda_cloud_tests/config_profile_verify_test.py +++ b/tests/rptest/redpanda_cloud_tests/config_profile_verify_test.py @@ -68,12 +68,15 @@ def _check_rp_config(self): assert clusterConfig['cluster_id'] in (self._clusterId, f'rp-{self._clusterId}') - for k, v in self._configProfile["cluster_config"].items(): + for k, expected_v in self._configProfile["cluster_config"].items(): + actual_v = clusterConfig[k] self.logger.debug( "asserting cluster config key {} has expected value: {} actual: {}" - .format(k, v, clusterConfig[k])) - if clusterConfig[k] != v and "{}".format(clusterConfig[k]) != v: - assert False + .format(k, expected_v, actual_v)) + if expected_v == "null": + expected_v = None + if actual_v != expected_v and "{}".format(actual_v) != expected_v: + assert False, f"incorrect config value for key '{k}': {actual_v} != {expected_v}" def _check_aws_nodes(self): cmd = self.redpanda.kubectl._ssh_prefix() + [ From 688480dfda87e8da1bded8f28126e5ce10e0a3ff Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Wed, 4 Dec 2024 11:50:25 -0800 Subject: [PATCH 033/229] cloud_storage_clients/abs_client: use item filter in list_objects This seems like an oversight. Noticed in an ABS run of a test: ``` INFO 2024-11-12 02:16:37,966 [shard 1:main] cloud_storage - topic_manifest_downloader.cc:102 - Labeled topic manifest download resulted in 2 matching manifests with prefix '', printing first 2 INFO 2024-11-12 02:16:37,966 [shard 1:main] cloud_storage - topic_manifest_downloader.cc:112 - Match for hint '': meta/kafka/topic-a/83123841-40c1-47e4-92ed-5ac8d1f536c7/8/topic_manifest.bin INFO 2024-11-12 02:16:37,966 [shard 1:main] cloud_storage - topic_manifest_downloader.cc:112 - Match for hint '': meta/kafka/topic-a/eb0359d1-a0fb-45aa-8995-d938a34caeef/8_lifecycle.bin WARN 2024-11-12 02:16:37,966 [shard 1:main] kafka - create_topics.cc:185 - Failed to create topic(s) {{kafka/topic-a}} error_code observed: cluster::errc::topic_operation_error ``` In this snippet, the list results contain a lifecycle marker, despite the filter passed in during manifest download being ends_with("topic_manifest.bin"). Auditing the codebase a bit, there don't appear to be any other usages of list_objects() with a filter. --- src/v/cloud_storage_clients/abs_client.cc | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/v/cloud_storage_clients/abs_client.cc b/src/v/cloud_storage_clients/abs_client.cc index 06583bfa24e7..38f2b5b8cb52 100644 --- a/src/v/cloud_storage_clients/abs_client.cc +++ b/src/v/cloud_storage_clients/abs_client.cc @@ -858,7 +858,7 @@ ss::future abs_client::do_list_objects( std::optional marker, ss::lowres_clock::duration timeout, std::optional delimiter, - std::optional) { + std::optional collect_item_if) { auto header = _requestor.make_list_blobs_request( name, _adls_client.has_value(), @@ -891,8 +891,9 @@ ss::future abs_client::do_list_objects( co_return co_await ss::do_with( response_stream->as_input_stream(), xml_sax_parser{}, - [](ss::input_stream& stream, xml_sax_parser& p) mutable { - p.start_parse(std::make_unique()); + [pred = std::move(collect_item_if)]( + ss::input_stream& stream, xml_sax_parser& p) mutable { + p.start_parse(std::make_unique(std::move(pred))); return ss::do_until( [&stream] { return stream.eof(); }, [&stream, &p] { From f92108bcb0c58b169c57c93d0982b4484ff2aa0b Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Wed, 4 Dec 2024 12:56:47 -0800 Subject: [PATCH 034/229] rptest: paramterize test_clusters_share_bucket Testing with ABS would have caught a bug in the ABS client during topic recovery when there are multiple objects within a topic's manifest path prefix (e.g. from different clusters). --- tests/rptest/tests/remote_label_test.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/rptest/tests/remote_label_test.py b/tests/rptest/tests/remote_label_test.py index 5148fdf42c81..faa4f8343a7e 100644 --- a/tests/rptest/tests/remote_label_test.py +++ b/tests/rptest/tests/remote_label_test.py @@ -11,11 +11,12 @@ from rptest.clients.types import TopicSpec from rptest.services.cluster import cluster from rptest.clients.rpk import RpkTool -from rptest.services.redpanda import RedpandaService, SISettings, make_redpanda_service +from rptest.services.redpanda import RedpandaService, SISettings, get_cloud_storage_type, make_redpanda_service from rptest.services.kgo_verifier_services import KgoVerifierProducer from rptest.tests.redpanda_test import RedpandaTest from rptest.tests.read_replica_e2e_test import hwms_are_identical, create_read_replica_topic from rptest.util import wait_until +from ducktape.mark import matrix from ducktape.tests.test import TestContext from rptest.utils.si_utils import BucketView, NT, quiesce_uploads @@ -90,7 +91,8 @@ def produce(self, cluster: RedpandaService, topic_name: str, producer.free() @cluster(num_nodes=3) - def test_clusters_share_bucket(self) -> None: + @matrix(cloud_storage_type=get_cloud_storage_type()) + def test_clusters_share_bucket(self, cloud_storage_type) -> None: """ cluster 1 creates topic_a cluster 2 creates topic_a From 3395f7656ee13073d24f7996dc7dfbe66c571170 Mon Sep 17 00:00:00 2001 From: rpdevmp Date: Wed, 4 Dec 2024 16:30:33 -0800 Subject: [PATCH 035/229] High priority alerts detection --- .../redpanda_cloud_tests/observe_test.py | 22 +++++++++++++------ 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/tests/rptest/redpanda_cloud_tests/observe_test.py b/tests/rptest/redpanda_cloud_tests/observe_test.py index bb27d81d5862..dc6b00d83e67 100644 --- a/tests/rptest/redpanda_cloud_tests/observe_test.py +++ b/tests/rptest/redpanda_cloud_tests/observe_test.py @@ -59,15 +59,23 @@ def cluster_alerts(self, rule_groups): @cluster(num_nodes=0, check_allowed_error_logs=False) def test_cloud_observe(self): - self.logger.debug("Here we go") + self.logger.debug("Starting cloud observe test") rule_groups = self.load_grafana_rules() - alerts = self.cluster_alerts(rule_groups.data.groups) + high_priority_alerts = [] for alert in alerts: - self.logger.warn( - f'alert firing for cluster: {alert.labels.grafana_folder} / {alert.labels.alertname}' - ) - - assert len(alerts) == 0 + alert_message = f"alert firing for cluster: {alert.labels.grafana_folder} / {alert.labels.alertname}" + if "high priority" in alert_message.lower(): + self.logger.error(f"High priority - {alert_message}") + high_priority_alerts.append(alert_message) + else: + self.logger.warning(f"Low priority - {alert_message}") + + # Fail the test if high-priority alerts are present + assert not high_priority_alerts, ( + f"Test failed due to high-priority alerts:\n{high_priority_alerts}" + ) + + self.logger.info("Cloud observe test completed successfully.") From fbaa5e0f05551912a8728dfe8a6d9a8bcbdc887f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Tue, 19 Nov 2024 13:05:42 +0100 Subject: [PATCH 036/229] datalake: added assertion validating that the memory reservation was set MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Added assert validating if Redpanda memory reservation was correctly configured when creating `datalake_manager` service. As an addition to that the log entry informing about configured memory parameters was added. Signed-off-by: Michał Maślanka --- src/v/datalake/datalake_manager.cc | 20 +++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/src/v/datalake/datalake_manager.cc b/src/v/datalake/datalake_manager.cc index f67804f77d54..8a97ff5932c6 100644 --- a/src/v/datalake/datalake_manager.cc +++ b/src/v/datalake/datalake_manager.cc @@ -93,12 +93,22 @@ datalake_manager::datalake_manager( , _sg(sg) , _effective_max_translator_buffered_data( std::min(memory_limit, max_translator_buffered_data)) - , _parallel_translations(std::make_unique( - size_t( - std::floor(memory_limit / _effective_max_translator_buffered_data)), - "datalake_parallel_translations")) , _iceberg_commit_interval( - config::shard_local_cfg().iceberg_catalog_commit_interval_ms.bind()) {} + config::shard_local_cfg().iceberg_catalog_commit_interval_ms.bind()) { + vassert(memory_limit > 0, "Memory limit must be greater than 0"); + auto max_parallel = static_cast( + std::floor(memory_limit / _effective_max_translator_buffered_data)); + vlog( + datalake_log.debug, + "Creating datalake manager with memory limit: {}, effective max " + "translator buffered data: {} and max parallel translations: {}", + memory_limit, + _effective_max_translator_buffered_data, + max_parallel); + + _parallel_translations = std::make_unique( + size_t(max_parallel), "datalake_parallel_translations"); +} datalake_manager::~datalake_manager() = default; ss::future<> datalake_manager::start() { From e6c6445b3199eb852c5434227499c2ede793484a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Tue, 19 Nov 2024 14:26:16 +0100 Subject: [PATCH 037/229] tests: enable datalake in random node operation tests MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Added a parameter that enables Iceberg in RandomNodeOperations test to validate the behavior with node operations and failures. Signed-off-by: Michał Maślanka --- .../tests/random_node_operations_test.py | 83 +++++++++++++++---- tests/rptest/utils/node_operations.py | 11 ++- 2 files changed, 72 insertions(+), 22 deletions(-) diff --git a/tests/rptest/tests/random_node_operations_test.py b/tests/rptest/tests/random_node_operations_test.py index e01b085a1690..e9a8c3b4ba32 100644 --- a/tests/rptest/tests/random_node_operations_test.py +++ b/tests/rptest/tests/random_node_operations_test.py @@ -13,16 +13,17 @@ import threading from rptest.clients.rpk import RpkTool from rptest.services.admin import Admin +from rptest.services.apache_iceberg_catalog import IcebergRESTCatalog from rptest.tests.prealloc_nodes import PreallocNodesTest -from ducktape.mark import matrix +from ducktape.mark import matrix, ignore from ducktape.utils.util import wait_until from rptest.services.admin_ops_fuzzer import AdminOperationsFuzzer from rptest.services.cluster import cluster from rptest.clients.types import TopicSpec from rptest.clients.default import DefaultClient from rptest.services.kgo_verifier_services import KgoVerifierConsumerGroupConsumer, KgoVerifierProducer -from rptest.services.redpanda import CHAOS_LOG_ALLOW_LIST, PREV_VERSION_LOG_ALLOW_LIST, SISettings +from rptest.services.redpanda import CHAOS_LOG_ALLOW_LIST, PREV_VERSION_LOG_ALLOW_LIST, CloudStorageType, PandaproxyConfig, SISettings, SchemaRegistryConfig from rptest.services.redpanda_installer import RedpandaInstaller from rptest.utils.mode_checks import cleanup_on_early_exit, skip_debug_mode, skip_fips_mode from rptest.utils.node_operations import FailureInjectorBackgroundThread, NodeOpsExecutor, generate_random_workload @@ -59,12 +60,22 @@ def __init__(self, test_context, *args, **kwargs): }, # 2 nodes for kgo producer/consumer workloads node_prealloc_count=3, + schema_registry_config=SchemaRegistryConfig(), + pandaproxy_config=PandaproxyConfig(), *args, **kwargs) self.nodes_with_prev_version = [] self.installer = self.redpanda._installer self.previous_version = self.installer.highest_from_prior_feature_version( RedpandaInstaller.HEAD) + self._si_settings = SISettings(self.test_context, + cloud_storage_enable_remote_read=True, + cloud_storage_enable_remote_write=True, + fast_uploads=True) + self.catalog_service = IcebergRESTCatalog( + test_context, + cloud_storage_bucket=self._si_settings.cloud_storage_bucket, + filesystem_wrapper_mode=False) def min_producer_records(self): return 20 * self.producer_throughput @@ -95,8 +106,7 @@ def early_exit_hook(self): self.redpanda.set_skip_if_no_redpanda_log(True) def setUp(self): - # defer starting redpanda to test body - pass + self.catalog_service.start() def _setup_test_scale(self): # test setup @@ -132,17 +142,28 @@ def _setup_test_scale(self): f"running test with: [message_size {self.msg_size}, total_bytes: {self.total_data}, message_count: {self.msg_count}, rate_limit: {self.rate_limit}, cluster_operations: {self.node_operations}]" ) - def _start_redpanda(self, mixed_versions, with_tiered_storage): + def _start_redpanda(self, mixed_versions, with_tiered_storage, + with_iceberg): - if with_tiered_storage: - si_settings = SISettings(self.test_context, - cloud_storage_enable_remote_read=True, - cloud_storage_enable_remote_write=True, - fast_uploads=True) + if with_tiered_storage or with_iceberg: # since this test is deleting topics we must tolerate missing manifests - si_settings.set_expected_damage( + self._si_settings.set_expected_damage( {"ntr_no_topic_manifest", "ntpr_no_manifest"}) - self.redpanda.set_si_settings(si_settings) + self.redpanda.set_si_settings(self._si_settings) + + if with_iceberg: + self.redpanda.add_extra_rp_conf({ + "iceberg_enabled": + "true", + "iceberg_catalog_type": + "rest", + "iceberg_rest_catalog_endpoint": + self.catalog_service.catalog_url, + "iceberg_rest_catalog_client_id": + "panda-user", + "iceberg_rest_catalog_client_secret": + "panda-secret", + }) self.redpanda.set_seed_servers(self.redpanda.nodes) if mixed_versions: @@ -276,17 +297,28 @@ def verify(self): assert self.consumer.consumer_status.validator.invalid_reads == 0, f"Invalid reads in topic: {self.topic}, invalid reads count: {self.consumer.consumer_status.validator.invalid_reads}" + def maybe_enable_iceberg_for_topic(self, topic_spec: TopicSpec, + iceberg_enabled: bool): + if iceberg_enabled: + client = DefaultClient(self.redpanda) + client.alter_topic_config(topic_spec.name, + TopicSpec.PROPERTY_ICEBERG_MODE, + "key_value") + # before v24.2, dns query to s3 endpoint do not include the bucketname, which is required for AWS S3 fips endpoints @skip_fips_mode @skip_debug_mode - @cluster(num_nodes=8, + @cluster(num_nodes=9, log_allow_list=CHAOS_LOG_ALLOW_LIST + PREV_VERSION_LOG_ALLOW_LIST + TS_LOG_ALLOW_LIST) @matrix(enable_failures=[True, False], mixed_versions=[True, False], - with_tiered_storage=[True, False]) + with_tiered_storage=[True, False], + with_iceberg=[True, False], + cloud_storage_type=[CloudStorageType.S3]) def test_node_operations(self, enable_failures, mixed_versions, - with_tiered_storage): + with_tiered_storage, with_iceberg, + cloud_storage_type): # In order to reduce the number of parameters and at the same time cover # as many use cases as possible this test uses 3 topics which 3 separate # producer/consumer pairs: @@ -294,6 +326,11 @@ def test_node_operations(self, enable_failures, mixed_versions, # tp-workload-deletion - topic with delete cleanup policy # tp-workload-compaction - topic with compaction # tp-workload-fast - topic with fast partition movements enabled + if with_iceberg and mixed_versions: + self.should_skip = True + self.logger.info( + "Skipping test with iceberg and mixed versions as it is not supported" + ) def enable_fast_partition_movement(): if not with_tiered_storage: @@ -328,11 +365,16 @@ def enable_write_caching_testing(): # start redpanda process self._start_redpanda(mixed_versions, - with_tiered_storage=with_tiered_storage) + with_tiered_storage=with_tiered_storage, + with_iceberg=with_iceberg) self.redpanda.set_cluster_config( {"controller_snapshot_max_age_sec": 1}) + if with_iceberg: + self.redpanda.set_cluster_config( + {"iceberg_catalog_commit_interval_ms": 10000}) + client = DefaultClient(self.redpanda) # create some initial topics @@ -345,6 +387,7 @@ def enable_write_caching_testing(): redpanda_remote_read=with_tiered_storage, redpanda_remote_write=with_tiered_storage) client.create_topic(regular_topic) + self.maybe_enable_iceberg_for_topic(regular_topic, with_iceberg) if with_tiered_storage: # change local retention policy to make some local segments will be deleted during the test @@ -370,6 +413,7 @@ def enable_write_caching_testing(): redpanda_remote_read=with_tiered_storage, redpanda_remote_write=with_tiered_storage) client.create_topic(compacted_topic) + self.maybe_enable_iceberg_for_topic(compacted_topic, with_iceberg) compacted_producer_consumer = RandomNodeOperationsTest.producer_consumer( test_context=self.test_context, @@ -404,7 +448,7 @@ def enable_write_caching_testing(): default_segment_size) self._alter_local_topic_retention_bytes(fast_topic.name, 8 * default_segment_size) - + self.maybe_enable_iceberg_for_topic(fast_topic, with_iceberg) fast_producer_consumer = RandomNodeOperationsTest.producer_consumer( test_context=self.test_context, logger=self.logger, @@ -433,7 +477,8 @@ def enable_write_caching_testing(): client.create_topic(write_caching_topic) client.alter_topic_config(write_caching_topic.name, TopicSpec.PROPERTY_WRITE_CACHING, "true") - + self.maybe_enable_iceberg_for_topic(write_caching_topic, + with_iceberg) write_caching_producer_consumer = RandomNodeOperationsTest.producer_consumer( test_context=self.test_context, logger=self.logger, @@ -473,6 +518,8 @@ def enable_write_caching_testing(): self.logger, lock, progress_timeout=120 if enable_failures else 60) + if with_iceberg: + executor.override_config_params = {"iceberg_enabled": True} for i, op in enumerate( generate_random_workload( available_nodes=self.active_node_idxs)): diff --git a/tests/rptest/utils/node_operations.py b/tests/rptest/utils/node_operations.py index 224f52791654..816d2dbe9472 100644 --- a/tests/rptest/utils/node_operations.py +++ b/tests/rptest/utils/node_operations.py @@ -221,6 +221,7 @@ def __init__(self, self.timeout = 360 self.lock = lock self.progress_timeout = progress_timeout + self.override_config_params: None | dict = None def node_id(self, idx): return self.redpanda.node_id(self.redpanda.get_node(idx), @@ -373,10 +374,12 @@ def add(self, idx: int): node = self.redpanda.get_node(idx) - self.redpanda.start_node(node, - timeout=self.timeout, - auto_assign_node_id=True, - omit_seeds_on_idx_one=False) + self.redpanda.start_node( + node, + timeout=self.timeout, + auto_assign_node_id=True, + omit_seeds_on_idx_one=False, + override_cfg_params=self.override_config_params) self.logger.info( f"added node: {idx} with new node id: {self.node_id(idx)}") From b632190e2596c3bd651d3e0e21818e2a5f3b2c10 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gell=C3=A9rt=20Peresztegi-Nagy?= Date: Fri, 29 Nov 2024 13:19:32 +0000 Subject: [PATCH 038/229] cluster: consider shard0 reserve in check_cluster_limits Improve the user error feedback when the `topic_partitions_reserve_shard0` cluster config is used and a user tried to allocate a topic that is above the partition limits. Previously this check was only considered as part of the `max_final_capacity` hard constraint, which meant that the kafka error message was more vague (No nodes are available to perform allocation after hard constraints were solved) and there were no clear broker logs to indicate this. Now this is also considered inside `check_cluster_limits` which leads to more specific error messages on both the kafka api (unable to create topic with 20 partitions due to hardware constraints) and in broker logs: ``` WARN 2024-11-29 13:18:13,907 [shard 0:main] cluster - partition_allocator.cc:183 - Refusing to create 20 partitions as total partition count 20 would exceed the core-based limit 18 (per-shard limit: 20, shard0 reservation: 2) ``` --- src/v/cluster/scheduling/partition_allocator.cc | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/v/cluster/scheduling/partition_allocator.cc b/src/v/cluster/scheduling/partition_allocator.cc index 8e501f702f16..2f748b968436 100644 --- a/src/v/cluster/scheduling/partition_allocator.cc +++ b/src/v/cluster/scheduling/partition_allocator.cc @@ -173,15 +173,19 @@ std::error_code partition_allocator::check_cluster_limits( // Refuse to create a partition count that would violate the per-core // limit. - const uint64_t core_limit = (effective_cpu_count * _partitions_per_shard()); + const uint64_t core_limit = (effective_cpu_count * _partitions_per_shard()) + - (broker_count * _partitions_reserve_shard0()); if (proposed_total_partitions > core_limit) { vlog( clusterlog.warn, "Refusing to create {} partitions as total partition count {} would " - "exceed core limit {}", + "exceed the core-based limit {} (per-shard limit: {}, shard0 " + "reservation: {})", new_partitions_replicas_requested, proposed_total_partitions, - effective_cpu_count * _partitions_per_shard()); + core_limit, + _partitions_per_shard(), + _partitions_reserve_shard0()); return errc::topic_invalid_partitions_core_limit; } From c7e355b7f0d54502d17892acc36c944a5aa7c8cb Mon Sep 17 00:00:00 2001 From: Ben Pope Date: Thu, 5 Dec 2024 11:49:29 +0000 Subject: [PATCH 039/229] bazel: Update protobuf to v29.0 Remove the version checks vtools is now updated. Signed-off-by: Ben Pope --- MODULE.bazel | 2 +- MODULE.bazel.lock | 103 ++++++++++++++++--- src/v/pandaproxy/schema_registry/protobuf.cc | 49 --------- src/v/redpanda/application.cc | 4 - 4 files changed, 87 insertions(+), 71 deletions(-) diff --git a/MODULE.bazel b/MODULE.bazel index 1377d9cddc80..ba7f987408bc 100644 --- a/MODULE.bazel +++ b/MODULE.bazel @@ -41,7 +41,7 @@ bazel_dep(name = "googletest", version = "1.15.2") bazel_dep(name = "liburing", version = "2.5") bazel_dep(name = "lz4", version = "1.9.4") bazel_dep(name = "platforms", version = "0.0.10") -bazel_dep(name = "protobuf", version = "27.3") +bazel_dep(name = "protobuf", version = "29.0") bazel_dep(name = "re2", version = "2024-07-02") bazel_dep(name = "rules_foreign_cc", version = "0.12.0") bazel_dep(name = "rules_go", version = "0.50.1") diff --git a/MODULE.bazel.lock b/MODULE.bazel.lock index 97b77eca5e62..8bf56d500765 100644 --- a/MODULE.bazel.lock +++ b/MODULE.bazel.lock @@ -7,6 +7,8 @@ "https://bcr.bazel.build/modules/abseil-cpp/20230125.1/MODULE.bazel": "89047429cb0207707b2dface14ba7f8df85273d484c2572755be4bab7ce9c3a0", "https://bcr.bazel.build/modules/abseil-cpp/20230802.0.bcr.1/MODULE.bazel": "1c8cec495288dccd14fdae6e3f95f772c1c91857047a098fad772034264cc8cb", "https://bcr.bazel.build/modules/abseil-cpp/20230802.0/MODULE.bazel": "d253ae36a8bd9ee3c5955384096ccb6baf16a1b1e93e858370da0a3b94f77c16", + "https://bcr.bazel.build/modules/abseil-cpp/20230802.1/MODULE.bazel": "fa92e2eb41a04df73cdabeec37107316f7e5272650f81d6cc096418fe647b915", + "https://bcr.bazel.build/modules/abseil-cpp/20240116.1/MODULE.bazel": "37bcdb4440fbb61df6a1c296ae01b327f19e9bb521f9b8e26ec854b6f97309ed", "https://bcr.bazel.build/modules/abseil-cpp/20240116.2/MODULE.bazel": "73939767a4686cd9a520d16af5ab440071ed75cec1a876bf2fcfaf1f71987a16", "https://bcr.bazel.build/modules/abseil-cpp/20240722.0/MODULE.bazel": "88668a07647adbdc14cb3a7cd116fb23c9dda37a90a1681590b6c9d8339a5b84", "https://bcr.bazel.build/modules/abseil-cpp/20240722.0/source.json": "59af9f8a8a4817092624e21263fe1fb7d7951a3b06f0570c610c7e5a9caf5f29", @@ -28,7 +30,10 @@ "https://bcr.bazel.build/modules/bazel_features/1.1.1/MODULE.bazel": "27b8c79ef57efe08efccbd9dd6ef70d61b4798320b8d3c134fd571f78963dbcd", "https://bcr.bazel.build/modules/bazel_features/1.11.0/MODULE.bazel": "f9382337dd5a474c3b7d334c2f83e50b6eaedc284253334cf823044a26de03e8", "https://bcr.bazel.build/modules/bazel_features/1.15.0/MODULE.bazel": "d38ff6e517149dc509406aca0db3ad1efdd890a85e049585b7234d04238e2a4d", - "https://bcr.bazel.build/modules/bazel_features/1.15.0/source.json": "483d0d339c23732f16b995fab1323147c1d2cc4b1c6ca4f832373aff028fdea9", + "https://bcr.bazel.build/modules/bazel_features/1.17.0/MODULE.bazel": "039de32d21b816b47bd42c778e0454217e9c9caac4a3cf8e15c7231ee3ddee4d", + "https://bcr.bazel.build/modules/bazel_features/1.18.0/MODULE.bazel": "1be0ae2557ab3a72a57aeb31b29be347bcdc5d2b1eb1e70f39e3851a7e97041a", + "https://bcr.bazel.build/modules/bazel_features/1.19.0/MODULE.bazel": "59adcdf28230d220f0067b1f435b8537dd033bfff8db21335ef9217919c7fb58", + "https://bcr.bazel.build/modules/bazel_features/1.19.0/source.json": "d7bf14517c1b25b9d9c580b0f8795fceeae08a7590f507b76aace528e941375d", "https://bcr.bazel.build/modules/bazel_features/1.4.1/MODULE.bazel": "e45b6bb2350aff3e442ae1111c555e27eac1d915e77775f6fdc4b351b758b5d7", "https://bcr.bazel.build/modules/bazel_features/1.9.1/MODULE.bazel": "8f679097876a9b609ad1f60249c49d68bfab783dd9be012faf9d82547b14815a", "https://bcr.bazel.build/modules/bazel_skylib/1.0.3/MODULE.bazel": "bcb0fd896384802d1ad283b4e4eb4d718eebd8cb820b0a2c3a347fb971afd9d8", @@ -40,6 +45,7 @@ "https://bcr.bazel.build/modules/bazel_skylib/1.4.2/MODULE.bazel": "3bd40978e7a1fac911d5989e6b09d8f64921865a45822d8b09e815eaa726a651", "https://bcr.bazel.build/modules/bazel_skylib/1.5.0/MODULE.bazel": "32880f5e2945ce6a03d1fbd588e9198c0a959bb42297b2cfaf1685b7bc32e138", "https://bcr.bazel.build/modules/bazel_skylib/1.6.1/MODULE.bazel": "8fdee2dbaace6c252131c00e1de4b165dc65af02ea278476187765e1a617b917", + "https://bcr.bazel.build/modules/bazel_skylib/1.7.0/MODULE.bazel": "0db596f4563de7938de764cc8deeabec291f55e8ec15299718b93c4423e9796d", "https://bcr.bazel.build/modules/bazel_skylib/1.7.1/MODULE.bazel": "3120d80c5861aa616222ec015332e5f8d3171e062e3e804a2a0253e1be26e59b", "https://bcr.bazel.build/modules/bazel_skylib/1.7.1/source.json": "f121b43eeefc7c29efbd51b83d08631e2347297c95aac9764a701f2a6a2bb953", "https://bcr.bazel.build/modules/boringssl/0.0.0-20240530-2db0eb3/MODULE.bazel": "d0405b762c5e87cd445b7015f2b8da5400ef9a8dbca0bfefa6c1cea79d528a97", @@ -84,8 +90,9 @@ "https://bcr.bazel.build/modules/platforms/0.0.8/MODULE.bazel": "9f142c03e348f6d263719f5074b21ef3adf0b139ee4c5133e2aa35664da9eb2d", "https://bcr.bazel.build/modules/platforms/0.0.9/MODULE.bazel": "4a87a60c927b56ddd67db50c89acaa62f4ce2a1d2149ccb63ffd871d5ce29ebc", "https://bcr.bazel.build/modules/protobuf/21.7/MODULE.bazel": "a5a29bb89544f9b97edce05642fac225a808b5b7be74038ea3640fae2f8e66a7", - "https://bcr.bazel.build/modules/protobuf/27.3/MODULE.bazel": "d94898cbf9d6d25c0edca2521211413506b68a109a6b01776832ed25154d23d7", - "https://bcr.bazel.build/modules/protobuf/27.3/source.json": "d6fdc641a99c600df6eb0fa5b99879ca497dbcf6fd1287372576a83f82dd93b6", + "https://bcr.bazel.build/modules/protobuf/27.0/MODULE.bazel": "7873b60be88844a0a1d8f80b9d5d20cfbd8495a689b8763e76c6372998d3f64c", + "https://bcr.bazel.build/modules/protobuf/29.0/MODULE.bazel": "319dc8bf4c679ff87e71b1ccfb5a6e90a6dbc4693501d471f48662ac46d04e4e", + "https://bcr.bazel.build/modules/protobuf/29.0/source.json": "b857f93c796750eef95f0d61ee378f3420d00ee1dd38627b27193aa482f4f981", "https://bcr.bazel.build/modules/protobuf/3.19.0/MODULE.bazel": "6b5fbb433f760a99a22b18b6850ed5784ef0e9928a72668b66e4d7ccd47db9b0", "https://bcr.bazel.build/modules/protobuf/3.19.2/MODULE.bazel": "532ffe5f2186b69fdde039efe6df13ba726ff338c6bc82275ad433013fa10573", "https://bcr.bazel.build/modules/protobuf/3.19.6/MODULE.bazel": "9233edc5e1f2ee276a60de3eaa47ac4132302ef9643238f23128fea53ea12858", @@ -95,17 +102,23 @@ "https://bcr.bazel.build/modules/re2/2023-09-01/MODULE.bazel": "cb3d511531b16cfc78a225a9e2136007a48cf8a677e4264baeab57fe78a80206", "https://bcr.bazel.build/modules/re2/2024-07-02/MODULE.bazel": "0eadc4395959969297cbcf31a249ff457f2f1d456228c67719480205aa306daa", "https://bcr.bazel.build/modules/re2/2024-07-02/source.json": "547d0111a9d4f362db32196fef805abbf3676e8d6afbe44d395d87816c1130ca", + "https://bcr.bazel.build/modules/rules_android/0.1.1/MODULE.bazel": "48809ab0091b07ad0182defb787c4c5328bd3a278938415c00a7b69b50c4d3a8", + "https://bcr.bazel.build/modules/rules_android/0.1.1/source.json": "e6986b41626ee10bdc864937ffb6d6bf275bb5b9c65120e6137d56e6331f089e", "https://bcr.bazel.build/modules/rules_buf/0.1.1/MODULE.bazel": "6189aec18a4f7caff599ad41b851ab7645d4f1e114aa6431acf9b0666eb92162", "https://bcr.bazel.build/modules/rules_buf/0.1.1/source.json": "021363d254f7438f3f10725355969c974bb2c67e0c28667782ade31a9cdb747f", "https://bcr.bazel.build/modules/rules_cc/0.0.1/MODULE.bazel": "cb2aa0747f84c6c3a78dad4e2049c154f08ab9d166b1273835a8174940365647", + "https://bcr.bazel.build/modules/rules_cc/0.0.10/MODULE.bazel": "ec1705118f7eaedd6e118508d3d26deba2a4e76476ada7e0e3965211be012002", + "https://bcr.bazel.build/modules/rules_cc/0.0.16/MODULE.bazel": "7661303b8fc1b4d7f532e54e9d6565771fea666fbdf839e0a86affcd02defe87", + "https://bcr.bazel.build/modules/rules_cc/0.0.16/source.json": "227e83737046aa4f50015da48e98e0d8ab42fd0ec74d8d653b6cc9f9a357f200", "https://bcr.bazel.build/modules/rules_cc/0.0.2/MODULE.bazel": "6915987c90970493ab97393024c156ea8fb9f3bea953b2f3ec05c34f19b5695c", "https://bcr.bazel.build/modules/rules_cc/0.0.6/MODULE.bazel": "abf360251023dfe3efcef65ab9d56beefa8394d4176dd29529750e1c57eaa33f", "https://bcr.bazel.build/modules/rules_cc/0.0.8/MODULE.bazel": "964c85c82cfeb6f3855e6a07054fdb159aced38e99a5eecf7bce9d53990afa3e", "https://bcr.bazel.build/modules/rules_cc/0.0.9/MODULE.bazel": "836e76439f354b89afe6a911a7adf59a6b2518fafb174483ad78a2a2fde7b1c5", - "https://bcr.bazel.build/modules/rules_cc/0.0.9/source.json": "1f1ba6fea244b616de4a554a0f4983c91a9301640c8fe0dd1d410254115c8430", "https://bcr.bazel.build/modules/rules_foreign_cc/0.12.0/MODULE.bazel": "d850fab025ce79a845077035861034393f1cc1efc1d9d58d766272a26ba67def", "https://bcr.bazel.build/modules/rules_foreign_cc/0.12.0/source.json": "c97ddc022179fe30d1a9b94425d1e56d0a633f72332c55463e584a52ce1b38ac", "https://bcr.bazel.build/modules/rules_foreign_cc/0.9.0/MODULE.bazel": "c9e8c682bf75b0e7c704166d79b599f93b72cfca5ad7477df596947891feeef6", + "https://bcr.bazel.build/modules/rules_fuzzing/0.5.2/MODULE.bazel": "40c97d1144356f52905566c55811f13b299453a14ac7769dfba2ac38192337a8", + "https://bcr.bazel.build/modules/rules_fuzzing/0.5.2/source.json": "c8b1e2c717646f1702290959a3302a178fb639d987ab61d548105019f11e527e", "https://bcr.bazel.build/modules/rules_go/0.33.0/MODULE.bazel": "a2b11b64cd24bf94f57454f53288a5dacfe6cb86453eee7761b7637728c1910c", "https://bcr.bazel.build/modules/rules_go/0.38.1/MODULE.bazel": "fb8e73dd3b6fc4ff9d260ceacd830114891d49904f5bda1c16bc147bcc254f71", "https://bcr.bazel.build/modules/rules_go/0.39.1/MODULE.bazel": "d34fb2a249403a5f4339c754f1e63dc9e5ad70b47c5e97faee1441fc6636cd61", @@ -117,15 +130,25 @@ "https://bcr.bazel.build/modules/rules_go/0.50.1/source.json": "205765fd30216c70321f84c9a967267684bdc74350af3f3c46c857d9f80a4fa2", "https://bcr.bazel.build/modules/rules_java/4.0.0/MODULE.bazel": "5a78a7ae82cd1a33cef56dc578c7d2a46ed0dca12643ee45edbb8417899e6f74", "https://bcr.bazel.build/modules/rules_java/5.3.5/MODULE.bazel": "a4ec4f2db570171e3e5eb753276ee4b389bae16b96207e9d3230895c99644b86", + "https://bcr.bazel.build/modules/rules_java/6.5.2/MODULE.bazel": "1d440d262d0e08453fa0c4d8f699ba81609ed0e9a9a0f02cd10b3e7942e61e31", + "https://bcr.bazel.build/modules/rules_java/7.10.0/MODULE.bazel": "530c3beb3067e870561739f1144329a21c851ff771cd752a49e06e3dc9c2e71a", + "https://bcr.bazel.build/modules/rules_java/7.12.2/MODULE.bazel": "579c505165ee757a4280ef83cda0150eea193eed3bef50b1004ba88b99da6de6", + "https://bcr.bazel.build/modules/rules_java/7.12.2/source.json": "b0890f9cda8ff1b8e691a3ac6037b5c14b7fd4134765a3946b89f31ea02e5884", + "https://bcr.bazel.build/modules/rules_java/7.2.0/MODULE.bazel": "06c0334c9be61e6cef2c8c84a7800cef502063269a5af25ceb100b192453d4ab", + "https://bcr.bazel.build/modules/rules_java/7.6.1/MODULE.bazel": "2f14b7e8a1aa2f67ae92bc69d1ec0fa8d9f827c4e17ff5e5f02e91caa3b2d0fe", "https://bcr.bazel.build/modules/rules_java/7.6.5/MODULE.bazel": "481164be5e02e4cab6e77a36927683263be56b7e36fef918b458d7a8a1ebadb1", - "https://bcr.bazel.build/modules/rules_java/7.6.5/source.json": "a805b889531d1690e3c72a7a7e47a870d00323186a9904b36af83aa3d053ee8d", "https://bcr.bazel.build/modules/rules_jvm_external/4.4.2/MODULE.bazel": "a56b85e418c83eb1839819f0b515c431010160383306d13ec21959ac412d2fe7", "https://bcr.bazel.build/modules/rules_jvm_external/5.1/MODULE.bazel": "33f6f999e03183f7d088c9be518a63467dfd0be94a11d0055fe2d210f89aa909", - "https://bcr.bazel.build/modules/rules_jvm_external/5.1/source.json": "5abb45cc9beb27b77aec6a65a11855ef2b55d95dfdc358e9f312b78ae0ba32d5", + "https://bcr.bazel.build/modules/rules_jvm_external/5.2/MODULE.bazel": "d9351ba35217ad0de03816ef3ed63f89d411349353077348a45348b096615036", + "https://bcr.bazel.build/modules/rules_jvm_external/6.3/MODULE.bazel": "c998e060b85f71e00de5ec552019347c8bca255062c990ac02d051bb80a38df0", + "https://bcr.bazel.build/modules/rules_jvm_external/6.3/source.json": "6f5f5a5a4419ae4e37c35a5bb0a6ae657ed40b7abc5a5189111b47fcebe43197", + "https://bcr.bazel.build/modules/rules_kotlin/1.9.6/MODULE.bazel": "d269a01a18ee74d0335450b10f62c9ed81f2321d7958a2934e44272fe82dcef3", + "https://bcr.bazel.build/modules/rules_kotlin/1.9.6/source.json": "2faa4794364282db7c06600b7e5e34867a564ae91bda7cae7c29c64e9466b7d5", "https://bcr.bazel.build/modules/rules_license/0.0.3/MODULE.bazel": "627e9ab0247f7d1e05736b59dbb1b6871373de5ad31c3011880b4133cafd4bd0", "https://bcr.bazel.build/modules/rules_license/0.0.7/MODULE.bazel": "088fbeb0b6a419005b89cf93fe62d9517c0a2b8bb56af3244af65ecfe37e7d5d", "https://bcr.bazel.build/modules/rules_license/0.0.8/MODULE.bazel": "5669c6fe49b5134dbf534db681ad3d67a2d49cfc197e4a95f1ca2fd7f3aebe96", - "https://bcr.bazel.build/modules/rules_license/0.0.8/source.json": "ccfd3964cd0cd1739202efb8dbf9a06baab490e61e174b2ad4790f9c4e610beb", + "https://bcr.bazel.build/modules/rules_license/1.0.0/MODULE.bazel": "a7fda60eefdf3d8c827262ba499957e4df06f659330bbe6cdbdb975b768bb65c", + "https://bcr.bazel.build/modules/rules_license/1.0.0/source.json": "a52c89e54cc311196e478f8382df91c15f7a2bfdf4c6cd0e2675cc2ff0b56efb", "https://bcr.bazel.build/modules/rules_nodejs/5.8.2/MODULE.bazel": "6bc03c8f37f69401b888023bf511cb6ee4781433b0cb56236b2e55a21e3a026a", "https://bcr.bazel.build/modules/rules_nodejs/5.8.2/source.json": "6e82cf5753d835ea18308200bc79b9c2e782efe2e2a4edc004a9162ca93382ca", "https://bcr.bazel.build/modules/rules_pkg/0.7.0/MODULE.bazel": "df99f03fc7934a4737122518bb87e667e62d780b610910f0447665a7e2be62dc", @@ -141,19 +164,23 @@ "https://bcr.bazel.build/modules/rules_python/0.22.1/MODULE.bazel": "26114f0c0b5e93018c0c066d6673f1a2c3737c7e90af95eff30cfee38d0bbac7", "https://bcr.bazel.build/modules/rules_python/0.23.1/MODULE.bazel": "49ffccf0511cb8414de28321f5fcf2a31312b47c40cc21577144b7447f2bf300", "https://bcr.bazel.build/modules/rules_python/0.25.0/MODULE.bazel": "72f1506841c920a1afec76975b35312410eea3aa7b63267436bfb1dd91d2d382", + "https://bcr.bazel.build/modules/rules_python/0.28.0/MODULE.bazel": "cba2573d870babc976664a912539b320cbaa7114cd3e8f053c720171cde331ed", "https://bcr.bazel.build/modules/rules_python/0.31.0/MODULE.bazel": "93a43dc47ee570e6ec9f5779b2e64c1476a6ce921c48cc9a1678a91dd5f8fd58", "https://bcr.bazel.build/modules/rules_python/0.33.2/MODULE.bazel": "3e036c4ad8d804a4dad897d333d8dce200d943df4827cb849840055be8d2e937", "https://bcr.bazel.build/modules/rules_python/0.33.2/source.json": "e539592cd3aae4492032cecea510e46ca16eeb972271560b922cae9893944e2f", "https://bcr.bazel.build/modules/rules_python/0.4.0/MODULE.bazel": "9208ee05fd48bf09ac60ed269791cf17fb343db56c8226a720fbb1cdf467166c", "https://bcr.bazel.build/modules/rules_rust/0.49.3/MODULE.bazel": "7c747ca20606b61fdb3c99c537a97a7cc89ac48482c0f25b3e70787297b0ec46", "https://bcr.bazel.build/modules/rules_rust/0.49.3/source.json": "0f4627d0ed4cd0d5af58f0162f87dcdf38fe4578e5308a3d7dca4c68cb13e323", + "https://bcr.bazel.build/modules/rules_shell/0.2.0/MODULE.bazel": "fda8a652ab3c7d8fee214de05e7a9916d8b28082234e8d2c0094505c5268ed3c", + "https://bcr.bazel.build/modules/rules_shell/0.2.0/source.json": "7f27af3c28037d9701487c4744b5448d26537cc66cdef0d8df7ae85411f8de95", "https://bcr.bazel.build/modules/snappy/1.2.1/MODULE.bazel": "ccfc05c2f321f33fa4190a57280f3b0b428982f7c66618f2acadf162fa0bbb95", "https://bcr.bazel.build/modules/snappy/1.2.1/source.json": "9a3e0181edc27543b4304f377a216ad09e014859db57921261552d0a4939ee1d", "https://bcr.bazel.build/modules/stardoc/0.5.0/MODULE.bazel": "f9f1f46ba8d9c3362648eea571c6f9100680efc44913618811b58cc9c02cd678", "https://bcr.bazel.build/modules/stardoc/0.5.1/MODULE.bazel": "1a05d92974d0c122f5ccf09291442580317cdd859f07a8655f1db9a60374f9f8", "https://bcr.bazel.build/modules/stardoc/0.5.3/MODULE.bazel": "c7f6948dae6999bf0db32c1858ae345f112cacf98f174c7a8bb707e41b974f1c", "https://bcr.bazel.build/modules/stardoc/0.5.4/MODULE.bazel": "6569966df04610b8520957cb8e97cf2e9faac2c0309657c537ab51c16c18a2a4", - "https://bcr.bazel.build/modules/stardoc/0.5.4/source.json": "a961f58a71e735aa9dcb2d79b288e06b0a2d860ba730302c8f11be411b76631e", + "https://bcr.bazel.build/modules/stardoc/0.7.0/MODULE.bazel": "05e3d6d30c099b6770e97da986c53bd31844d7f13d41412480ea265ac9e8079c", + "https://bcr.bazel.build/modules/stardoc/0.7.0/source.json": "e3c524bf2ef20992539ce2bc4a2243f4853130209ee831689983e28d05769099", "https://bcr.bazel.build/modules/toolchains_llvm/1.1.2/MODULE.bazel": "402101d6f73115ec49a3a765a3361c1dd90ba3959fa688ccdcd465c36dbbbc52", "https://bcr.bazel.build/modules/toolchains_llvm/1.1.2/source.json": "27f3cf531bc654c719b50411cac94613b7676d63e60962243d485af63e13b9ff", "https://bcr.bazel.build/modules/upb/0.0.0-20220923-a547704/MODULE.bazel": "7298990c00040a0e2f121f6c32544bab27d4452f80d9ce51349b1a28f3005c43", @@ -165,6 +192,7 @@ "https://bcr.bazel.build/modules/zlib/1.2.12/MODULE.bazel": "3b1a8834ada2a883674be8cbd36ede1b6ec481477ada359cd2d3ddc562340b27", "https://bcr.bazel.build/modules/zlib/1.3.1.bcr.3/MODULE.bazel": "af322bc08976524477c79d1e45e241b6efbeb918c497e8840b8ab116802dda79", "https://bcr.bazel.build/modules/zlib/1.3.1.bcr.3/source.json": "2be409ac3c7601245958cd4fcdff4288be79ed23bd690b4b951f500d54ee6e7d", + "https://bcr.bazel.build/modules/zlib/1.3.1/MODULE.bazel": "751c9940dcfe869f5f7274e1295422a34623555916eb98c174c1e945594bf198", "https://bcr.bazel.build/modules/zstd/1.5.6/MODULE.bazel": "471ebe7d3cdd8c6469390fcf623eb4779ff55fbee0a87f1dc57a1def468b96d4", "https://bcr.bazel.build/modules/zstd/1.5.6/source.json": "02010c3333fc89b44fe861db049968decb6e688411f7f9d4f6791d74f9adfb51" }, @@ -2599,28 +2627,69 @@ ] } }, - "@@rules_jvm_external~//:non-module-deps.bzl%non_module_deps": { + "@@rules_kotlin~//src/main/starlark/core/repositories:bzlmod_setup.bzl%rules_kotlin_extensions": { "general": { - "bzlTransitiveDigest": "l6SlNloqPvd60dcuPdWiJNi3g3jfK76fcZc0i/Yr0dQ=", - "usagesDigest": "pX61d12AFioOtqChQDmxvlNGDYT69e5MrKT2E/S6TeQ=", + "bzlTransitiveDigest": "fus14IFJ/1LGWWGKPH/U18VnJCoMjfDt1ckahqCnM0A=", + "usagesDigest": "aJF6fLy82rR95Ff5CZPAqxNoFgOMLMN5ImfBS0nhnkg=", "recordedFileInputs": {}, "recordedDirentsInputs": {}, "envVariables": {}, "generatedRepoSpecs": { - "io_bazel_rules_kotlin": { + "com_github_jetbrains_kotlin_git": { + "bzlFile": "@@rules_kotlin~//src/main/starlark/core/repositories:compiler.bzl", + "ruleClassName": "kotlin_compiler_git_repository", + "attributes": { + "urls": [ + "https://github.com/JetBrains/kotlin/releases/download/v1.9.23/kotlin-compiler-1.9.23.zip" + ], + "sha256": "93137d3aab9afa9b27cb06a824c2324195c6b6f6179d8a8653f440f5bd58be88" + } + }, + "com_github_jetbrains_kotlin": { + "bzlFile": "@@rules_kotlin~//src/main/starlark/core/repositories:compiler.bzl", + "ruleClassName": "kotlin_capabilities_repository", + "attributes": { + "git_repository_name": "com_github_jetbrains_kotlin_git", + "compiler_version": "1.9.23" + } + }, + "com_github_google_ksp": { + "bzlFile": "@@rules_kotlin~//src/main/starlark/core/repositories:ksp.bzl", + "ruleClassName": "ksp_compiler_plugin_repository", + "attributes": { + "urls": [ + "https://github.com/google/ksp/releases/download/1.9.23-1.0.20/artifacts.zip" + ], + "sha256": "ee0618755913ef7fd6511288a232e8fad24838b9af6ea73972a76e81053c8c2d", + "strip_version": "1.9.23-1.0.20" + } + }, + "com_github_pinterest_ktlint": { + "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", + "ruleClassName": "http_file", + "attributes": { + "sha256": "01b2e0ef893383a50dbeb13970fe7fa3be36ca3e83259e01649945b09d736985", + "urls": [ + "https://github.com/pinterest/ktlint/releases/download/1.3.0/ktlint" + ], + "executable": true + } + }, + "rules_android": { "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", "ruleClassName": "http_archive", "attributes": { - "sha256": "946747acdbeae799b085d12b240ec346f775ac65236dfcf18aa0cd7300f6de78", + "sha256": "cd06d15dd8bb59926e4d65f9003bfc20f9da4b2519985c27e190cddc8b7a7806", + "strip_prefix": "rules_android-0.1.1", "urls": [ - "https://github.com/bazelbuild/rules_kotlin/releases/download/v1.7.0-RC-2/rules_kotlin_release.tgz" + "https://github.com/bazelbuild/rules_android/archive/v0.1.1.zip" ] } } }, "recordedRepoMappingEntries": [ [ - "rules_jvm_external~", + "rules_kotlin~", "bazel_tools", "bazel_tools" ] @@ -5191,7 +5260,7 @@ }, "@@rules_rust~//rust:extensions.bzl%rust": { "general": { - "bzlTransitiveDigest": "crzY7WoW/k3FLAYAyu4GS2ZfnEC8wZqRqEfBUkOb6Ho=", + "bzlTransitiveDigest": "e2mzUPIPy3NKky6vadyIVx99U0LKUcD6IGN5l5tNdpY=", "usagesDigest": "kIFjD0j3UyHq8EH2APrpeiquQWPbz1TAnKvNNG68hSY=", "recordedFileInputs": {}, "recordedDirentsInputs": {}, @@ -6778,7 +6847,7 @@ }, "@@rules_rust~//rust/private:extensions.bzl%i": { "general": { - "bzlTransitiveDigest": "Y5CXpUvQT2qwri9vgBWZflpTG2iRVkGVJarCQEtlNks=", + "bzlTransitiveDigest": "vNAsGXSLCBX2N+xGaONrm7ih3WcPqIRr1NUoyDgnqIk=", "usagesDigest": "7xz5OGpAOJhc1oS49mGGVmKImfhZig3f4BlAterVYVM=", "recordedFileInputs": {}, "recordedDirentsInputs": {}, diff --git a/src/v/pandaproxy/schema_registry/protobuf.cc b/src/v/pandaproxy/schema_registry/protobuf.cc index fae4beeb8c5c..a0ff8f5244bd 100644 --- a/src/v/pandaproxy/schema_registry/protobuf.cc +++ b/src/v/pandaproxy/schema_registry/protobuf.cc @@ -143,14 +143,6 @@ class io_error_collector final : public pb::io::ErrorCollector { }; public: -#if PROTOBUF_VERSION < 5027000 - void AddError(int line, int column, const std::string& message) final { - _errors.emplace_back(err{level::error, line, column, message}); - } - void AddWarning(int line, int column, const std::string& message) final { - _errors.emplace_back(err{level::warn, line, column, message}); - } -#else void RecordError(int line, int column, std::string_view message) final { _errors.emplace_back( err{level::error, line, column, ss::sstring{message}}); @@ -159,7 +151,6 @@ class io_error_collector final : public pb::io::ErrorCollector { _errors.emplace_back( err{level::warn, line, column, ss::sstring{message}}); } -#endif error_info error() const; @@ -171,41 +162,6 @@ class io_error_collector final : public pb::io::ErrorCollector { class dp_error_collector final : public pb::DescriptorPool::ErrorCollector { public: -#if PROTOBUF_VERSION < 5027000 - void AddError( - const std::string& filename, - const std::string& element_name, - const pb::Message* descriptor, - ErrorLocation location, - const std::string& message) final { - _errors.emplace_back(err{ - level::error, - ss::sstring{filename}, - ss::sstring{element_name}, - descriptor, - location, - ss::sstring { - message - }}); - } - - void AddWarning( - const std::string& filename, - const std::string& element_name, - const pb::Message* descriptor, - ErrorLocation location, - const std::string& message) final { - _errors.emplace_back(err{ - level::warn, - ss::sstring{filename}, - ss::sstring{element_name}, - descriptor, - location, - ss::sstring { - message - }}); - } -#else void RecordError( std::string_view filename, std::string_view element_name, @@ -235,7 +191,6 @@ class dp_error_collector final : public pb::DescriptorPool::ErrorCollector { location, ss::sstring{message}}); } -#endif error_info error() const; @@ -302,11 +257,7 @@ class parser { } } const auto& sub = schema.sub()(); -#if PROTOBUF_VERSION < 5027000 - _fdp.set_name(sub.data(), sub.size()); -#else _fdp.set_name(std::string_view(sub)); -#endif return _fdp; } diff --git a/src/v/redpanda/application.cc b/src/v/redpanda/application.cc index 0e562fb5143c..3f836ce0a1b0 100644 --- a/src/v/redpanda/application.cc +++ b/src/v/redpanda/application.cc @@ -582,12 +582,8 @@ void application::initialize( * Disable the logger for protobuf; some interfaces don't allow a pluggable * error collector. */ -#if PROTOBUF_VERSION < 5027000 - google::protobuf::SetLogHandler(nullptr); -#else // Protobuf uses absl logging in the latest version absl::SetMinLogLevel(absl::LogSeverityAtLeast::kInfinity); -#endif /* * allocate per-core zstd decompression workspace and per-core From 404c7a2fb0a0781b421b77e41d19f1308587ee12 Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Mon, 2 Dec 2024 16:21:45 -0800 Subject: [PATCH 040/229] raft/recovery: log line with more context --- src/v/raft/recovery_stm.cc | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/v/raft/recovery_stm.cc b/src/v/raft/recovery_stm.cc index 692c661c693e..4efe0e54c878 100644 --- a/src/v/raft/recovery_stm.cc +++ b/src/v/raft/recovery_stm.cc @@ -48,8 +48,9 @@ recovery_stm::recovery_stm( , _ctxlog( raftlog, ssx::sformat( - "[follower: {}] [group_id:{}, {}]", + "[follower: {}, term: {}] [group_id:{}, {}]", _node_id, + _term, _ptr->group(), _ptr->ntp())) , _memory_quota(quota) {} From d0a562be9d4341bcb100a070efa4ef890c16cdf7 Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Tue, 3 Dec 2024 14:18:56 -0800 Subject: [PATCH 041/229] raft/c: notify waiters on follower state reset This typically happens when there is a stepdown and the downstream consumers like recovery need to know about it. --- src/v/raft/types.cc | 1 + 1 file changed, 1 insertion(+) diff --git a/src/v/raft/types.cc b/src/v/raft/types.cc index 3ec57a6bd576..e5c675e01b9e 100644 --- a/src/v/raft/types.cc +++ b/src/v/raft/types.cc @@ -142,6 +142,7 @@ void follower_index_metadata::reset() { last_successful_received_seq = follower_req_seq{0}; inflight_append_request_count = 0; last_sent_protocol_meta.reset(); + follower_state_change.broadcast(); } std::ostream& operator<<(std::ostream& o, const vnode& id) { From 72a82cf47752069a6d1d2fbfa17e0c2bef32b5d0 Mon Sep 17 00:00:00 2001 From: Alexey Bashtanov Date: Tue, 3 Dec 2024 11:04:39 +0000 Subject: [PATCH 042/229] features/snapshot: figure out state for features missing in snapshot Depending on the agreed version in the snapshot, a feature is known to be either not yet available, or retired. Use this logic when applying a snapshot. --- src/v/features/feature_table_snapshot.cc | 22 +++++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/src/v/features/feature_table_snapshot.cc b/src/v/features/feature_table_snapshot.cc index 7113d5f4755d..564aaeba4643 100644 --- a/src/v/features/feature_table_snapshot.cc +++ b/src/v/features/feature_table_snapshot.cc @@ -54,13 +54,25 @@ void feature_table_snapshot::apply(feature_table& ft) const { }); if (snap_state_iter == states.end()) { // The feature table refers to a feature name that the snapshot - // doesn't mention: this is normal on upgrade. The feature will - // remain in its default-initialized state. + // doesn't mention: this is normal on upgrade. + if (spec.require_version <= version) { + // The feature was introduced no later than the agreed version, + // which is no later than the version of the broker that took + // the snapshot. So it only can be missing because it has been + // retired and thus deemed active. + cur_state._state = feature_state::state::active; + } else { + // Otherwise the feature was introduced after the agreed + // version, so it can only be disabled before we reach it. + cur_state._state = feature_state::state::unavailable; + } vlog( featureslog.debug, - "No state for feature '{}' in snapshot, upgrade in progress?", - spec.name); - continue; + "No state for feature '{}' in snapshot v{}, upgrade in progress? " + "Assuming the feature state is {}", + spec.name, + version, + cur_state._state); } else { if ( spec.require_version From 7c63a297db228c4601c49f412d1202d0e4b9756f Mon Sep 17 00:00:00 2001 From: Alexey Bashtanov Date: Wed, 4 Dec 2024 16:23:12 +0000 Subject: [PATCH 043/229] features/table: improve comments Warn that it is not safe to retire a feature together with its checks in the same version. --- src/v/cluster/feature_backend.h | 2 +- src/v/features/feature_table.h | 7 ++++++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/v/cluster/feature_backend.h b/src/v/cluster/feature_backend.h index b9d8f51c4487..231bb631409d 100644 --- a/src/v/cluster/feature_backend.h +++ b/src/v/cluster/feature_backend.h @@ -41,7 +41,7 @@ class feature_backend { ss::future<> fill_snapshot(controller_snapshot&) const; ss::future<> apply_snapshot(model::offset, const controller_snapshot&); - /// this functions deal with the snapshot stored in local kvstore (in + /// these functions deal with the snapshot stored in local kvstore (in /// contrast to fill/apply_snapshot which deal with the feature table data /// in the replicated controller snapshot). bool has_local_snapshot(); diff --git a/src/v/features/feature_table.h b/src/v/features/feature_table.h index b9e08df0fe9c..db3d8ad010f6 100644 --- a/src/v/features/feature_table.h +++ b/src/v/features/feature_table.h @@ -82,8 +82,13 @@ enum class feature : std::uint64_t { // controller messages for unknown features (unexpected), and controller // messages that refer to features that have been retired. // -// retired does *not* mean the functionality is gone: it just means it +// Retired does *not* mean the functionality is gone: it just means it // is no longer guarded by a feature flag. +// +// All feature checks need to be removed one version before the feature is +// retired. That's because during upgrade, when a mixed version cluster is +// running, the older version nodes may read a snaphot from the newer version +// and get the feature automatically enabled. inline const std::unordered_set retired_features = { "central_config", "consumer_offsets", From 3ad5aeb97786198d0faeae5d333d3d2919072392 Mon Sep 17 00:00:00 2001 From: Alexey Bashtanov Date: Thu, 5 Dec 2024 14:17:43 +0000 Subject: [PATCH 044/229] c/migrations/frontend: only remember the local instance of components to avoid calling .local() every time we use them --- src/v/cluster/data_migration_frontend.cc | 55 +++++++++++------------- src/v/cluster/data_migration_frontend.h | 6 +-- 2 files changed, 28 insertions(+), 33 deletions(-) diff --git a/src/v/cluster/data_migration_frontend.cc b/src/v/cluster/data_migration_frontend.cc index c20f08a69d21..8269b5bc68ca 100644 --- a/src/v/cluster/data_migration_frontend.cc +++ b/src/v/cluster/data_migration_frontend.cc @@ -48,10 +48,10 @@ frontend::frontend( : _self(self) , _cloud_storage_api_initialized(cloud_storage_api_initialized) , _table(table) - , _features(features) + , _features(features.local()) , _controller(stm) - , _leaders_table(leaders) - , _connections(connections) + , _leaders_table(leaders.local()) + , _connections(connections.local()) , _topic_mount_handler(topic_mount_handler) , _as(as) , _operation_timeout(10s) {} @@ -69,8 +69,7 @@ frontend::process_or_dispatch( DispatchFunc dispatch, ProcessFunc process, ReplyMapperFunc reply_mapper) { - auto controller_leader = _leaders_table.local().get_leader( - model::controller_ntp); + auto controller_leader = _leaders_table.get_leader(model::controller_ntp); /// Return early if there is no controller leader if (!controller_leader) { vlog( @@ -102,18 +101,17 @@ frontend::process_or_dispatch( req, *controller_leader); /// If leader is somewhere else, dispatch RPC request to current leader - auto reply = co_await _connections.local() - .with_node_client( - _self, - ss::this_shard_id(), - *controller_leader, - _operation_timeout, - [req = std::move(req), - dispatch = std::forward(dispatch)]( - data_migrations_client_protocol client) mutable { - return dispatch(std::move(req), client) - .then(&rpc::get_ctx_data); - }); + auto reply + = co_await _connections.with_node_client( + _self, + ss::this_shard_id(), + *controller_leader, + _operation_timeout, + [req = std::move(req), dispatch = std::forward(dispatch)]( + data_migrations_client_protocol client) mutable { + return dispatch(std::move(req), client) + .then(&rpc::get_ctx_data); + }); vlog( dm_log.debug, "got reply {} from controller leader at {}", @@ -123,7 +121,7 @@ frontend::process_or_dispatch( } bool frontend::data_migrations_active() const { - return _features.local().is_active(features::feature::data_migrations) + return _features.is_active(features::feature::data_migrations) && _cloud_storage_api_initialized; } @@ -233,7 +231,7 @@ ss::future frontend::check_ntp_states_on_foreign_node( model::node_id node, check_ntp_states_request&& req) { vlog(dm_log.debug, "dispatching node request {} to node {}", req, node); - return _connections.local() + return _connections .with_node_client( _self, ss::this_shard_id(), @@ -295,21 +293,17 @@ ss::future> frontend::do_create_migration(data_migration migration) { } ss::future> frontend::list_migrations() { - return container().invoke_on(data_migrations_shard, [](frontend& local) { - return local._table.local().list_migrations(); - }); + return _table.invoke_on_instance(&migrations_table::list_migrations); } ss::future> frontend::get_migration(id migration_id) { - return container().invoke_on( - data_migrations_shard, [migration_id](frontend& local) { - auto maybe_migration = local._table.local().get_migration( - migration_id); - return maybe_migration - ? result(maybe_migration->get().copy()) - : errc::data_migration_not_exists; - }); + return _table.invoke_on_instance([migration_id](migrations_table& table) { + auto maybe_migration = table.get_migration(migration_id); + return maybe_migration + ? result(maybe_migration->get().copy()) + : errc::data_migration_not_exists; + }); } ss::future @@ -333,6 +327,7 @@ ss::future frontend::insert_barrier() { * required for correctness but allows the fronted to do more accurate * preliminary validation. */ + static_assert(controller_stm_shard == data_migrations_shard); auto barrier_result = co_await _controller.local().insert_linearizable_barrier( _operation_timeout + model::timeout_clock::now()); diff --git a/src/v/cluster/data_migration_frontend.h b/src/v/cluster/data_migration_frontend.h index 4eee0b84d547..d4329637130a 100644 --- a/src/v/cluster/data_migration_frontend.h +++ b/src/v/cluster/data_migration_frontend.h @@ -98,10 +98,10 @@ class frontend : public ss::peering_sharded_service { model::node_id _self; bool _cloud_storage_api_initialized; ssx::single_sharded& _table; - ss::sharded& _features; + features::feature_table& _features; ss::sharded& _controller; - ss::sharded& _leaders_table; - ss::sharded& _connections; + partition_leaders_table& _leaders_table; + rpc::connection_cache& _connections; std::optional> _topic_mount_handler; ss::sharded& _as; From f4237231390969c27d31b702570a8828690e2334 Mon Sep 17 00:00:00 2001 From: Alexey Bashtanov Date: Thu, 5 Dec 2024 14:26:37 +0000 Subject: [PATCH 045/229] c/migrations/frontend: check license on migration create --- src/v/cluster/data_migration_frontend.cc | 11 ++++++----- src/v/cluster/data_migration_frontend.h | 2 +- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/v/cluster/data_migration_frontend.cc b/src/v/cluster/data_migration_frontend.cc index 8269b5bc68ca..8ca7964d15d7 100644 --- a/src/v/cluster/data_migration_frontend.cc +++ b/src/v/cluster/data_migration_frontend.cc @@ -120,14 +120,15 @@ frontend::process_or_dispatch( co_return reply_mapper(std::move(reply)); } -bool frontend::data_migrations_active() const { +bool frontend::data_migrations_active(bool check_license) const { return _features.is_active(features::feature::data_migrations) - && _cloud_storage_api_initialized; + && _cloud_storage_api_initialized + && !(check_license && _features.should_sanction()); } ss::future> frontend::create_migration( data_migration migration, can_dispatch_to_leader can_dispatch) { - if (!data_migrations_active()) { + if (!data_migrations_active(true)) { return ssx::now>(errc::feature_disabled); } vlog(dm_log.debug, "creating migration: {}", migration); @@ -162,7 +163,7 @@ ss::future> frontend::create_migration( ss::future frontend::update_migration_state( id id, state state, can_dispatch_to_leader can_dispatch) { - if (!data_migrations_active()) { + if (!data_migrations_active(false)) { return ssx::now(errc::feature_disabled); } vlog(dm_log.debug, "updating migration: {} state with: {}", id, state); @@ -196,7 +197,7 @@ ss::future frontend::update_migration_state( ss::future frontend::remove_migration(id id, can_dispatch_to_leader can_dispatch) { - if (!data_migrations_active()) { + if (!data_migrations_active(false)) { return ssx::now(errc::feature_disabled); } vlog(dm_log.debug, "removing migration: {}", id); diff --git a/src/v/cluster/data_migration_frontend.h b/src/v/cluster/data_migration_frontend.h index d4329637130a..04b025049f13 100644 --- a/src/v/cluster/data_migration_frontend.h +++ b/src/v/cluster/data_migration_frontend.h @@ -92,7 +92,7 @@ class frontend : public ss::peering_sharded_service { "This method can only be called on data migration shard"); } - bool data_migrations_active() const; + bool data_migrations_active(bool check_license) const; private: model::node_id _self; From 89b490894b3aaab80256e0c08deaf3952762472d Mon Sep 17 00:00:00 2001 From: Alexey Bashtanov Date: Thu, 5 Dec 2024 14:29:26 +0000 Subject: [PATCH 046/229] tests/migrations: check behaviour when there's no enterprise license --- .../rptest/tests/data_migrations_api_test.py | 61 +++++++++++++++++-- 1 file changed, 56 insertions(+), 5 deletions(-) diff --git a/tests/rptest/tests/data_migrations_api_test.py b/tests/rptest/tests/data_migrations_api_test.py index ea42f8b6e21e..447219bf198f 100644 --- a/tests/rptest/tests/data_migrations_api_test.py +++ b/tests/rptest/tests/data_migrations_api_test.py @@ -581,8 +581,29 @@ def test_mount_inexistent(self): self.admin.delete_data_migration(in_migration_id) self.wait_migration_disappear(in_migration_id) + def toggle_license(self, on: bool): + ENV_KEY = '__REDPANDA_DISABLE_BUILTIN_TRIAL_LICENSE' + if on: + self.redpanda.unset_environment([ENV_KEY]) + else: + self.redpanda.set_environment({ENV_KEY: '1'}) + self.redpanda.rolling_restart_nodes(self.redpanda.nodes, + use_maintenance_mode=False) + @cluster(num_nodes=3, log_allow_list=MIGRATION_LOG_ALLOW_LIST) def test_creating_and_listing_migrations(self): + self.do_test_creating_and_listing_migrations(False) + + @cluster( + num_nodes=3, + log_allow_list=MIGRATION_LOG_ALLOW_LIST + [ + # license violation + r'/v1/migrations.*Requested feature is disabled', + ]) + def test_creating_and_listing_migrations_wo_license(self): + self.do_test_creating_and_listing_migrations(True) + + def do_test_creating_and_listing_migrations(self, try_wo_license: bool): topics = [TopicSpec(partition_count=3) for i in range(5)] for t in topics: @@ -591,13 +612,27 @@ def test_creating_and_listing_migrations(self): migrations_map = self.get_migrations_map() assert len(migrations_map) == 0, "There should be no data migrations" - with self.finj_thread(): + if try_wo_license: + time.sleep(2) # make sure test harness can see Redpanda is live + self.toggle_license(on=False) + self.assure_not_migratable( + topics[0], { + "message": + "Unexpected cluster error: Requested feature is disabled", + "code": 500 + }) + self.toggle_license(on=True) + + with nullcontext() if try_wo_license else self.finj_thread(): # out outbound_topics = [make_namespaced_topic(t.name) for t in topics] out_migration = OutboundDataMigration(outbound_topics, consumer_groups=[]) - out_migration_id = self.create_and_wait(out_migration) + + if try_wo_license: + self.toggle_license(on=False) + self.check_migrations(out_migration_id, len(topics), 1) self.execute_data_migration_action_flaky(out_migration_id, @@ -605,6 +640,7 @@ def test_creating_and_listing_migrations(self): self.wait_for_migration_states(out_migration_id, ['preparing', 'prepared']) self.wait_for_migration_states(out_migration_id, ['prepared']) + self.execute_data_migration_action_flaky(out_migration_id, MigrationAction.execute) self.wait_for_migration_states(out_migration_id, @@ -631,7 +667,12 @@ def test_creating_and_listing_migrations(self): ] in_migration = InboundDataMigration(topics=inbound_topics, consumer_groups=["g-1", "g-2"]) + self.logger.info(f'{try_wo_license=}') + if try_wo_license: + self.toggle_license(on=True) in_migration_id = self.create_and_wait(in_migration) + if try_wo_license: + self.toggle_license(on=False) self.check_migrations(in_migration_id, len(inbound_topics), 2) self.log_topics(t.source_topic_reference.topic @@ -639,9 +680,19 @@ def test_creating_and_listing_migrations(self): self.execute_data_migration_action_flaky(in_migration_id, MigrationAction.prepare) - self.wait_for_migration_states(in_migration_id, - ['preparing', 'prepared']) - self.wait_for_migration_states(in_migration_id, ['prepared']) + if try_wo_license: + self.wait_for_migration_states(in_migration_id, ['preparing']) + time.sleep(5) + # stuck as a topic cannot be created without license + self.wait_for_migration_states(in_migration_id, ['preparing']) + self.toggle_license(on=True) + self.wait_for_migration_states(in_migration_id, ['prepared']) + self.toggle_license(on=False) + else: + self.wait_for_migration_states(in_migration_id, + ['preparing', 'prepared']) + self.wait_for_migration_states(in_migration_id, ['prepared']) + self.execute_data_migration_action_flaky(in_migration_id, MigrationAction.execute) self.wait_for_migration_states(in_migration_id, From 02ec796b0cc82657a7ad2ec393840c94415f78b8 Mon Sep 17 00:00:00 2001 From: Alexey Bashtanov Date: Thu, 5 Dec 2024 17:23:13 +0000 Subject: [PATCH 047/229] features/tests: create a constant for test version --- src/v/features/feature_table.cc | 6 +++--- src/v/features/feature_table.h | 2 ++ src/v/features/tests/feature_table_test.cc | 6 +++--- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/src/v/features/feature_table.cc b/src/v/features/feature_table.cc index 3b47fa299056..346b3813ae2b 100644 --- a/src/v/features/feature_table.cc +++ b/src/v/features/feature_table.cc @@ -197,7 +197,7 @@ bool is_major_version_upgrade( static std::array test_extra_schema{ // For testing, a feature that does not auto-activate feature_spec{ - cluster::cluster_version{2001}, + TEST_VERSION, "__test_alpha", feature::test_alpha, feature_spec::available_policy::explicit_only, @@ -205,7 +205,7 @@ static std::array test_extra_schema{ // For testing, a feature that auto-activates feature_spec{ - cluster::cluster_version{2001}, + TEST_VERSION, "__test_bravo", feature::test_bravo, feature_spec::available_policy::always, @@ -213,7 +213,7 @@ static std::array test_extra_schema{ // For testing, a feature that auto-activates feature_spec{ - cluster::cluster_version{2001}, + TEST_VERSION, "__test_charlie", feature::test_charlie, feature_spec::available_policy::new_clusters_only, diff --git a/src/v/features/feature_table.h b/src/v/features/feature_table.h index db3d8ad010f6..cef32382f01f 100644 --- a/src/v/features/feature_table.h +++ b/src/v/features/feature_table.h @@ -163,6 +163,8 @@ constexpr cluster::cluster_version to_cluster_version(release_version rv) { vassert(false, "Invalid release_version"); } +constexpr cluster::cluster_version TEST_VERSION{2001}; + bool is_major_version_upgrade( cluster::cluster_version from, cluster::cluster_version to); diff --git a/src/v/features/tests/feature_table_test.cc b/src/v/features/tests/feature_table_test.cc index 77174d346a69..31803e7c9ed5 100644 --- a/src/v/features/tests/feature_table_test.cc +++ b/src/v/features/tests/feature_table_test.cc @@ -121,10 +121,10 @@ FIXTURE_TEST(feature_table_basic, feature_table_fixture) { ft.get_state(feature::test_alpha).get_state() == feature_state::state::unavailable); - // The dummy test features requires version 2001. The feature + // The dummy test features requires version TEST_VERSION. The feature // should go available, but not any further: the feature table // relies on external stimulus to actually activate features. - set_active_version(cluster_version{2001}); + set_active_version(TEST_VERSION); BOOST_REQUIRE( ft.get_state(feature::test_alpha).get_state() @@ -274,7 +274,7 @@ FIXTURE_TEST(feature_uniqueness, feature_table_fixture) { * but also activates elegible features. */ FIXTURE_TEST(feature_table_bootstrap, feature_table_fixture) { - bootstrap_active_version(cluster_version{2001}); + bootstrap_active_version(TEST_VERSION); // A non-auto-activating feature should remain in available state: // explicit_only features always require explicit activation, even From 8e53b6a65f94ecca4b0bfb5a2c16d9632fd851ff Mon Sep 17 00:00:00 2001 From: Alexey Bashtanov Date: Thu, 5 Dec 2024 17:27:10 +0000 Subject: [PATCH 048/229] features/test: check behavior on missing features in snapshot --- src/v/features/tests/feature_table_test.cc | 34 ++++++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/src/v/features/tests/feature_table_test.cc b/src/v/features/tests/feature_table_test.cc index 31803e7c9ed5..aea777e63580 100644 --- a/src/v/features/tests/feature_table_test.cc +++ b/src/v/features/tests/feature_table_test.cc @@ -334,6 +334,40 @@ FIXTURE_TEST(feature_table_old_snapshot, feature_table_fixture) { == feature_state::state::active); } +// Test that applying an old snapshot disables features that we only enabled in +// this version. +FIXTURE_TEST(feature_table_old_snapshot_missing, feature_table_fixture) { + bootstrap_active_version(TEST_VERSION); + + features::feature_table_snapshot snapshot; + snapshot.version = cluster::cluster_version{ft.get_active_version() - 1}; + snapshot.states = {}; + snapshot.apply(ft); + + // A feature with explicit available_policy should be activated by the + // snapshot. + BOOST_CHECK( + ft.get_state(feature::test_alpha).get_state() + == feature_state::state::unavailable); +} + +// Test that applying a snapshot of the same version with a missing feature +// enables it, as we assume it was retired in the next version. +FIXTURE_TEST(feature_table_new_snapshot_missing, feature_table_fixture) { + bootstrap_active_version(TEST_VERSION); + + features::feature_table_snapshot snapshot; + snapshot.version = cluster::cluster_version{ft.get_active_version()}; + snapshot.states = {}; + snapshot.apply(ft); + + // A feature with explicit available_policy should be activated by the + // snapshot. + BOOST_CHECK( + ft.get_state(feature::test_alpha).get_state() + == feature_state::state::active); +} + FIXTURE_TEST(feature_table_trial_license_test, feature_table_fixture) { const char* sample_valid_license = std::getenv("REDPANDA_SAMPLE_LICENSE"); if (sample_valid_license == nullptr) { From d2d6f85dd0e42718694e95207de6487559b2161a Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Thu, 5 Dec 2024 15:26:25 -0500 Subject: [PATCH 049/229] `rptest`: adjust `num_segments_deleted()` condition This can race with a segment upload and end up deleting more segments than anticipated: ``` [DEBUG - 2024-12-05 19:07:50,121] waiting until 143 segments will be removed [DEBUG - 2024-12-05 19:07:50,747] remote.cc:236 - Uploading segment to path ... [INFO - 2024-12-05 19:09:47,435] redpanda_cloud_storage_deleted_segments_total = 144 ``` Adjust the value to account for `segments_deleted` >= than what is expected. --- tests/rptest/tests/archive_retention_test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/rptest/tests/archive_retention_test.py b/tests/rptest/tests/archive_retention_test.py index 9873532d69ab..a84b811f38d8 100644 --- a/tests/rptest/tests/archive_retention_test.py +++ b/tests/rptest/tests/archive_retention_test.py @@ -162,11 +162,11 @@ def new_manifest_spilled(): f"waiting until {segments_to_delete} segments will be removed") # Wait for the first truncation to the middle of the archive wait_until( - lambda: self.num_segments_deleted() == segments_to_delete, + lambda: self.num_segments_deleted() >= segments_to_delete, timeout_sec=100, backoff_sec=5, err_msg= - f"Segments were not removed from the cloud, expected {segments_to_delete} deletions" + f"Segments were not removed from the cloud, expected at least {segments_to_delete} deletions" ) view.reset() @@ -210,11 +210,11 @@ def new_manifest_spilled(): f"waiting until {segments_to_delete} segments will be removed") # Wait for the second truncation of the entire archive wait_until( - lambda: self.num_segments_deleted() == segments_to_delete, + lambda: self.num_segments_deleted() >= segments_to_delete, timeout_sec=120, backoff_sec=5, err_msg= - f"Segments were not removed from the cloud, expected {segments_to_delete} " + f"Segments were not removed from the cloud, expected at least {segments_to_delete} " f"segments to be removed but only {self.num_segments_deleted()} was actually removed" ) From a037d8e0b0b5d4a5f10e977c9160f5c4c35359a2 Mon Sep 17 00:00:00 2001 From: rpdevmp Date: Thu, 5 Dec 2024 14:07:26 -0800 Subject: [PATCH 050/229] Inverting logic for alerts priority --- tests/rptest/redpanda_cloud_tests/observe_test.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/tests/rptest/redpanda_cloud_tests/observe_test.py b/tests/rptest/redpanda_cloud_tests/observe_test.py index dc6b00d83e67..4d9223512159 100644 --- a/tests/rptest/redpanda_cloud_tests/observe_test.py +++ b/tests/rptest/redpanda_cloud_tests/observe_test.py @@ -67,15 +67,16 @@ def test_cloud_observe(self): for alert in alerts: alert_message = f"alert firing for cluster: {alert.labels.grafana_folder} / {alert.labels.alertname}" - if "high priority" in alert_message.lower(): - self.logger.error(f"High priority - {alert_message}") - high_priority_alerts.append(alert_message) + # Treat all alerts not explicitly marked as "low priority" as high priority + if "low priority" in alert_message.lower(): + self.logger.warn(f"Low priority alert - {alert_message}") else: - self.logger.warning(f"Low priority - {alert_message}") + self.logger.error(f"High priority alert - {alert_message}") + high_priority_alerts.append(alert_message) # Fail the test if high-priority alerts are present assert not high_priority_alerts, ( - f"Test failed due to high-priority alerts:\n{high_priority_alerts}" + f"Test failed due to potential high-priority alerts:\n{high_priority_alerts}" ) self.logger.info("Cloud observe test completed successfully.") From f3f28f58fca64f4f96d3eae71d47e347971e3f92 Mon Sep 17 00:00:00 2001 From: Oren Leiman Date: Thu, 5 Dec 2024 14:41:33 -0800 Subject: [PATCH 051/229] iceberg: fix decimal type json writer Missing fmt::format call resulting in ill-formed serialization of decimal iceberg field type. --- src/v/iceberg/datatypes_json.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/v/iceberg/datatypes_json.cc b/src/v/iceberg/datatypes_json.cc index 0b6dc06fc527..a95c0b22cabe 100644 --- a/src/v/iceberg/datatypes_json.cc +++ b/src/v/iceberg/datatypes_json.cc @@ -142,7 +142,7 @@ class rjson_visitor { void operator()(const iceberg::float_type&) { w.String("float"); } void operator()(const iceberg::double_type&) { w.String("double"); } void operator()(const iceberg::decimal_type& t) { - w.String("decimal({}, {})", t.precision, t.scale); + w.String(fmt::format("decimal({}, {})", t.precision, t.scale)); } void operator()(const iceberg::date_type&) { w.String("date"); } void operator()(const iceberg::time_type&) { w.String("time"); } From fedfb66ca5e1c29680e00caa86e1ce6863499e0d Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Thu, 5 Dec 2024 19:13:47 -0500 Subject: [PATCH 052/229] `rptest`: remove unused variable and extra assert statement --- tests/rptest/utils/si_utils.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/rptest/utils/si_utils.py b/tests/rptest/utils/si_utils.py index aed9f50a4af9..81d3582c5379 100644 --- a/tests/rptest/utils/si_utils.py +++ b/tests/rptest/utils/si_utils.py @@ -1731,10 +1731,8 @@ def check_archive_integrity(self, ntp: NTP): summaries = self.segment_summaries(ntp) if len(summaries) == 0: assert 'archive_start_offset' not in manifest - assert 'archive_start_offset' not in manifest else: next_base_offset = manifest.get('archive_start_offset') - stm_start_offset = manifest.get('start_offset') expected_last = manifest.get('last_offset') for summary in summaries: From 686623fce0104f4054ddae5b0edecb50fe0f60db Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Thu, 5 Dec 2024 19:14:45 -0500 Subject: [PATCH 053/229] `rptest`: deflake `si_utils` assertion functions These functions `is_archive_cleanup_complete()` and `check_archive_integrity()` are used in `archive_retention_test.py`. For context, the function `quiesce_uploads()` is called before these assertion functions. This function waits for segments to be uploaded to cloud storage until the remote high watermark is up to date with the local high watermark, and the partition manifest also reflects this flushing operation. Afterwards, we assert on some expected properties using the manifest offsets as well as the segments. Because `quiesce_uploads()` doesn't take into account/forcefully flush segments composed of entirely non-data batches, these assertions can be quite race-y (and would be race-y even if it did). Add `are_all_segments_config_batches()` to `BucketView` and check the condition in `is_archive_cleanup_complete()` as well as `check_archive_integrity()` before performing any assertions that might flake depending on the sequence of previous events. --- tests/rptest/utils/si_utils.py | 23 +++++++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/tests/rptest/utils/si_utils.py b/tests/rptest/utils/si_utils.py index 81d3582c5379..55c68245adad 100644 --- a/tests/rptest/utils/si_utils.py +++ b/tests/rptest/utils/si_utils.py @@ -1697,6 +1697,15 @@ def segment_summaries(self, ntp: NTP) -> list[SegmentSummary]: return [] + def are_all_segments_config_batches(self, summaries): + all_config_batches = all( + [summary.num_data_records == 0 for summary in summaries]) + if all_config_batches: + self.logger.debug( + "Segments left in archive are composed of all non-data batches" + ) + return all_config_batches + def is_archive_cleanup_complete(self, ntp: NTP): self._ensure_listing() manifest = self.manifest_for_ntp(ntp.topic, ntp.partition) @@ -1715,6 +1724,13 @@ def is_archive_cleanup_complete(self, ntp: NTP): f"archive is empty, start: {aso}, clean: {aco}, len: {num}") return True + if self.are_all_segments_config_batches(summaries): + # quiesce_uploads() would not have waited for these segments + # to have been uploaded and the partition manifest + # to have been marked clean and reuploaded, since they + # didn't contribute to the HWM. Treat this race-y case as successful. + return True + first_segment = min(summaries, key=lambda seg: seg.base_offset) if first_segment.base_offset != aso: @@ -1732,6 +1748,13 @@ def check_archive_integrity(self, ntp: NTP): if len(summaries) == 0: assert 'archive_start_offset' not in manifest else: + if self.are_all_segments_config_batches(summaries): + # quiesce_uploads() would not have waited for these segments + # to have been uploaded and the partition manifest + # to have been marked clean and reuploaded, since they + # didn't contribute to the HWM. Treat this race-y case as successful. + return + next_base_offset = manifest.get('archive_start_offset') expected_last = manifest.get('last_offset') From 97c689ac7fa2d323ff47c94fc8353b12bdc11438 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Fri, 6 Dec 2024 09:37:59 +0100 Subject: [PATCH 054/229] chore: removed arrow library dependency MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Datalake module now uses the `serde::parquet` as the parquet format serializer. This allow us to remove the Arror and Parquet library dependency. Signed-off-by: Michał Maślanka --- install-dependencies.sh | 9 - licenses/third_party.md | 1 - src/v/datalake/CMakeLists.txt | 7 - src/v/datalake/arrow_translator.cc | 780 ------------------ src/v/datalake/arrow_translator.h | 43 - src/v/datalake/arrow_writer.cc | 119 --- src/v/datalake/arrow_writer.h | 38 - src/v/datalake/batching_parquet_writer.cc | 160 ---- src/v/datalake/batching_parquet_writer.h | 82 -- src/v/datalake/errors.h | 11 - src/v/datalake/tests/CMakeLists.txt | 31 +- src/v/datalake/tests/arrow_writer_test.cc | 441 ---------- .../tests/batching_parquet_writer_test.cc | 73 -- .../tests/gtest_record_multiplexer_test.cc | 72 +- src/v/datalake/tests/parquet_writer_test.cc | 79 -- src/v/datalake/tests/translation_task_test.cc | 5 +- ubsan_suppressions.txt | 1 - 17 files changed, 6 insertions(+), 1946 deletions(-) delete mode 100644 src/v/datalake/arrow_translator.cc delete mode 100644 src/v/datalake/arrow_translator.h delete mode 100644 src/v/datalake/arrow_writer.cc delete mode 100644 src/v/datalake/arrow_writer.h delete mode 100644 src/v/datalake/batching_parquet_writer.cc delete mode 100644 src/v/datalake/batching_parquet_writer.h delete mode 100644 src/v/datalake/tests/arrow_writer_test.cc delete mode 100644 src/v/datalake/tests/batching_parquet_writer_test.cc delete mode 100644 src/v/datalake/tests/parquet_writer_test.cc diff --git a/install-dependencies.sh b/install-dependencies.sh index 1bdd3309b117..368ac13a6e42 100755 --- a/install-dependencies.sh +++ b/install-dependencies.sh @@ -32,13 +32,11 @@ deb_deps=( cmake git golang - libarrow-dev libboost-all-dev libc-ares-dev libgssapi-krb5-2 libkrb5-dev liblz4-dev - libparquet-dev libprotobuf-dev libprotoc-dev libre2-dev @@ -73,7 +71,6 @@ fedora_deps=( golang hwloc-devel krb5-devel - libarrow-devel libxml2-devel libzstd-devel lksctp-tools-devel @@ -85,7 +82,6 @@ fedora_deps=( numactl-devel openssl openssl-devel - parquet-libs-devel procps protobuf-devel python3 @@ -130,11 +126,6 @@ case "$ID" in ubuntu | debian | pop) export DEBIAN_FRONTEND=noninteractive apt update - apt install -y -V ca-certificates lsb-release wget - wget https://apache.jfrog.io/artifactory/arrow/$(lsb_release --id --short | tr 'A-Z' 'a-z')/apache-arrow-apt-source-latest-$(lsb_release --codename --short).deb - apt install -y -V ./apache-arrow-apt-source-latest-$(lsb_release --codename --short).deb - rm apache-arrow-apt-source-latest-$(lsb_release --codename --short).deb - apt update apt-get install -y "${deb_deps[@]}" if [[ $CLEAN_PKG_CACHE == true ]]; then rm -rf /var/lib/apt/lists/* diff --git a/licenses/third_party.md b/licenses/third_party.md index 9689cd35a647..b792ff0a92a8 100644 --- a/licenses/third_party.md +++ b/licenses/third_party.md @@ -9,7 +9,6 @@ please keep this up to date with every new library use. | :---------- | :------------ | | abseil | Apache License 2 | | ada | Apache License 2 / MIT | -| arrow | Apache License 2 / MIT / Boost / BSD 2 & 3 clause / ZPL / LLVM / | | avro | Apache License 2 | | base64 | BSD 2 | | boost libraries | Boost Software License Version 1.0 | diff --git a/src/v/datalake/CMakeLists.txt b/src/v/datalake/CMakeLists.txt index 9463a0dc2395..30dcf2640e73 100644 --- a/src/v/datalake/CMakeLists.txt +++ b/src/v/datalake/CMakeLists.txt @@ -1,5 +1,3 @@ -find_package(Arrow REQUIRED) -find_package(Parquet REQUIRED) find_package(Protobuf REQUIRED) add_subdirectory(coordinator) @@ -30,11 +28,8 @@ v_cc_library( v_cc_library( NAME datalake_writer SRCS - arrow_translator.cc - batching_parquet_writer.cc catalog_schema_manager.cc data_writer_interface.cc - arrow_writer.cc record_multiplexer.cc record_schema_resolver.cc record_translator.cc @@ -61,8 +56,6 @@ v_cc_library( v::storage v::schema Seastar::seastar - Arrow::arrow_shared - Parquet::parquet_shared protobuf::libprotobuf v::iceberg v::serde_protobuf diff --git a/src/v/datalake/arrow_translator.cc b/src/v/datalake/arrow_translator.cc deleted file mode 100644 index 67e8dae7347e..000000000000 --- a/src/v/datalake/arrow_translator.cc +++ /dev/null @@ -1,780 +0,0 @@ -/* - * Copyright 2024 Redpanda Data, Inc. - * - * Licensed as a Redpanda Enterprise file under the Redpanda Community - * License (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md - */ -#include "datalake/arrow_translator.h" - -#include "container/fragmented_vector.h" -#include "iceberg/datatypes.h" -#include "iceberg/values.h" - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - -namespace datalake { -class converter_interface { -public: - virtual ~converter_interface() noexcept = default; - // Return an Arrow field descriptor for this converter. This is needed both - // for schema generation and for converters for compound types: list, - // struct, map. - virtual std::shared_ptr field( - const std::string& name, - iceberg::field_required required, - iceberg::nested_field::id_t field_id) - = 0; - - // Return the underlying builder. This is needed to get the child builders - // for compound types. - virtual std::shared_ptr builder() = 0; - - // The add_data and add_optional_data methods add a data item to the - // converter. They throw an exception on error. - // - invalid_argument: a bad data type in input data - // - runtime_error: an error in underlying Arrow library - // Data should be validated before calling this method: if an error occurs - // the converter is in an indeterminate state. - - // There are two methods because in most cases we need to be able to add an - // optional value, but the key of a map field is a corner case where the - // value is not optional. - virtual void - add_optional_data(const std::optional& /*value*/) - = 0; - virtual void add_data(const iceberg::value& /*value*/) = 0; -}; - -namespace { -static constexpr auto parquet_field_id_metadata_key = "PARQUET:field_id"; -// Iceberg uses field_ids to identify columns in parquet files. -auto default_field_metadata(iceberg::nested_field::id_t id) { - return std::make_shared( - std::unordered_map( - {{parquet_field_id_metadata_key, fmt::to_string(id)}})); -} -} // namespace - -template -class scalar_converter : public converter_interface { - using BuilderType = arrow::TypeTraits::BuilderType; - -public: - explicit scalar_converter( - const std::shared_ptr& data_type, - std::shared_ptr builder); - - std::shared_ptr field( - const std::string& name, - iceberg::field_required required, - iceberg::nested_field::id_t field_id) override; - - std::shared_ptr builder() override; - - void - add_optional_data(const std::optional& /*value*/) override; - void add_data(const iceberg::value& /*value*/) override; - void add_primitive(const iceberg::primitive_value& prim_value); - -private: - std::shared_ptr _arrow_data_type; - std::shared_ptr _builder; -}; - -class struct_converter : public converter_interface { -public: - explicit struct_converter(const iceberg::struct_type& schema); - - std::shared_ptr field( - const std::string& name, - iceberg::field_required required, - iceberg::nested_field::id_t field_id) override; - - std::shared_ptr builder() override; - - void add_optional_data( - const std::optional& maybe_value) override; - - void add_data(const iceberg::value& value) override; - - void add_struct_data(const iceberg::struct_value value); - - arrow::FieldVector get_field_vector(); - - std::shared_ptr take_chunk(); - -private: - fragmented_vector _field_required; - arrow::FieldVector _fields; - std::shared_ptr _arrow_data_type; - - fragmented_vector> _child_converters; - std::shared_ptr _builder; -}; - -class list_converter : public converter_interface { -public: - explicit list_converter(const iceberg::list_type& schema); - - std::shared_ptr field( - const std::string& name, - iceberg::field_required required, - iceberg::nested_field::id_t field_id) override; - - std::shared_ptr builder() override; - - void add_optional_data( - const std::optional& maybe_value) override; - - void add_data(const iceberg::value& value) override; - -private: - std::shared_ptr _arrow_data_type; - - std::unique_ptr _child_converter; - std::shared_ptr _builder; -}; - -class map_converter : public converter_interface { -public: - explicit map_converter(const iceberg::map_type& schema); - - std::shared_ptr field( - const std::string& name, - iceberg::field_required required, - iceberg::nested_field::id_t field_id) override; - - std::shared_ptr builder() override; - - void add_optional_data( - const std::optional& maybe_value) override; - - void add_data(const iceberg::value& value) override; - -private: - std::shared_ptr _arrow_data_type; - - std::unique_ptr _key_converter; - std::unique_ptr _value_converter; - std::shared_ptr _builder; -}; - -// Helper method to throw nicer error messages if an std::get would fail. -// std::get throws a bad_variant_access, but without much useful information. -template -const ValueT& get_or_throw(const SourceT& v, const std::string& on_error) { - if (!std::holds_alternative(v)) { - throw std::invalid_argument(on_error); - } - return std::get(v); -} - -template -void append_or_throw(BuilderT& builder, const ValueT& val) { - auto status = builder->Append(val); - if (!status.ok()) { - throw std::runtime_error( - fmt::format("Unable to append to builder: {}", status.ToString())); - } -} - -struct converter_builder_visitor { - std::unique_ptr - operator()(const iceberg::struct_type& s) { - return std::make_unique(s); - } - - std::unique_ptr - operator()(const iceberg::list_type& s) { - return std::make_unique(s); - } - - std::unique_ptr - operator()(const iceberg::map_type& m) { - return std::make_unique(m); - } - - std::unique_ptr - operator()(const iceberg::primitive_type& p) { - return std::visit(*this, p); - } - - std::unique_ptr - operator()(const iceberg::boolean_type& /*t*/) { - return std::make_unique>( - arrow::boolean(), std::make_shared()); - } - std::unique_ptr - operator()(const iceberg::int_type& /*t*/) { - return std::make_unique>( - arrow::int32(), std::make_shared()); - } - std::unique_ptr - operator()(const iceberg::long_type& /*t*/) { - return std::make_unique>( - arrow::int64(), std::make_shared()); - } - std::unique_ptr - operator()(const iceberg::float_type& /*t*/) { - return std::make_unique>( - arrow::float32(), std::make_shared()); - } - std::unique_ptr - operator()(const iceberg::double_type& /*t*/) { - return std::make_unique>( - arrow::float64(), std::make_shared()); - } - std::unique_ptr - operator()(const iceberg::decimal_type& t) { - if (t.precision == 0) { - return nullptr; - } - return std::make_unique>( - arrow::decimal(t.precision, t.scale), - std::make_shared( - arrow::decimal(t.precision, t.scale))); - } - std::unique_ptr - operator()(const iceberg::date_type& /*t*/) { - return std::make_unique>( - arrow::date32(), std::make_shared()); - } - std::unique_ptr - operator()(const iceberg::time_type& /*t*/) { - return std::make_unique>( - arrow::time64(arrow::TimeUnit::MICRO), - std::make_shared( - arrow::time64(arrow::TimeUnit::MICRO), - arrow::default_memory_pool())); - } - std::unique_ptr - operator()(const iceberg::timestamp_type& /*t*/) { - return std::make_unique>( - arrow::timestamp(arrow::TimeUnit::MICRO), - std::make_shared( - arrow::timestamp(arrow::TimeUnit::MICRO), - arrow::default_memory_pool())); - } - std::unique_ptr - operator()(const iceberg::timestamptz_type& /*t*/) { - // Iceberg timestamps are UTC - return std::make_unique>( - arrow::timestamp(arrow::TimeUnit::MICRO, "UTC"), - std::make_shared( - arrow::timestamp(arrow::TimeUnit::MICRO, "UTC"), - arrow::default_memory_pool())); - } - std::unique_ptr - operator()(const iceberg::string_type& /*t*/) { - return std::make_unique>( - arrow::utf8(), std::make_shared()); - } - std::unique_ptr - operator()(const iceberg::uuid_type& /*t*/) { - // TODO: there's an arrow extension for a uuid logical type. - // Under-the-hood it's stored as a fixed(16). Do we want to use the - // logical type here? - return std::make_unique>( - arrow::fixed_size_binary(16), - std::make_shared( - arrow::fixed_size_binary(16))); - } - std::unique_ptr - operator()(const iceberg::fixed_type& t) { - return std::make_unique>( - arrow::fixed_size_binary(static_cast(t.length)), - std::make_shared( - arrow::fixed_size_binary(t.length))); - } - std::unique_ptr - operator()(const iceberg::binary_type& /*t*/) { - return std::make_unique>( - arrow::binary(), std::make_shared()); - } -}; - -template -scalar_converter::scalar_converter( - const std::shared_ptr& data_type, - std::shared_ptr builder) - : _arrow_data_type{data_type} - , _builder{builder} {} - -template -std::shared_ptr scalar_converter::field( - const std::string& name, - iceberg::field_required required, - iceberg::nested_field::id_t field_id) { - return arrow::field( - name, - _arrow_data_type, - required == iceberg::field_required::no, - default_field_metadata(field_id)); -} - -template -std::shared_ptr scalar_converter::builder() { - return _builder; -}; - -template -void scalar_converter::add_optional_data( - const std::optional& maybe_value) { - if (!maybe_value.has_value()) { - auto status = _builder->AppendNull(); - if (!status.ok()) { - throw std::runtime_error(fmt::format( - "Unable to append null to scalar converter: {}", - status.ToString())); - } - } else { - add_data(maybe_value.value()); - } -} - -template -void scalar_converter::add_data(const iceberg::value& value) { - add_primitive(get_or_throw( - value, "Scalar converter got non-scalar data type")); -} - -template<> -void scalar_converter::add_primitive( - const iceberg::primitive_value& prim_value) { - auto val = get_or_throw( - prim_value, "Scalar converter expected a boolean"); - append_or_throw(_builder, val.val); -} - -template<> -void scalar_converter::add_primitive( - const iceberg::primitive_value& prim_value) { - auto val = get_or_throw( - prim_value, "Scalar converter expected an int32"); - append_or_throw(_builder, val.val); -} - -template<> -void scalar_converter::add_primitive( - const iceberg::primitive_value& prim_value) { - auto val = get_or_throw( - prim_value, "Scalar converter expected an int64"); - append_or_throw(_builder, val.val); -} - -template<> -void scalar_converter::add_primitive( - const iceberg::primitive_value& prim_value) { - auto val = get_or_throw( - prim_value, "Scalar converter expected a float"); - append_or_throw(_builder, val.val); -} - -template<> -void scalar_converter::add_primitive( - const iceberg::primitive_value& prim_value) { - auto val = get_or_throw( - prim_value, "Scalar converter expected a double"); - append_or_throw(_builder, val.val); -} - -template<> -void scalar_converter::add_primitive( - const iceberg::primitive_value& prim_value) { - auto val = get_or_throw( - prim_value, "Scalar converter expected a decimal") - .val; - - auto status = _builder->Append( - arrow::Decimal128(absl::Int128High64(val), absl::Int128Low64(val))); - if (!status.ok()) { - throw std::runtime_error( - fmt::format("Unable to append to builder: {}", status.ToString())); - } -} - -template<> -void scalar_converter::add_primitive( - const iceberg::primitive_value& prim_value) { - auto val = get_or_throw( - prim_value, "Scalar converter expected a date"); - append_or_throw(_builder, val.val); -} - -template<> -void scalar_converter::add_primitive( - const iceberg::primitive_value& prim_value) { - auto val = get_or_throw( - prim_value, "Scalar converter expected a time"); - append_or_throw(_builder, val.val); -} - -template<> -void scalar_converter::add_primitive( - const iceberg::primitive_value& prim_value) { - if (std::holds_alternative(prim_value)) { - append_or_throw( - _builder, std::get(prim_value).val); - } else if (std::holds_alternative(prim_value)) { - append_or_throw( - _builder, std::get(prim_value).val); - } else { - throw std::invalid_argument("Scalar converter expected a timestamp"); - } -} - -template<> -void scalar_converter::add_primitive( - const iceberg::primitive_value& prim_value) { - auto& val = get_or_throw( - prim_value, "Scalar converter expected a string"); - auto status = _builder->AppendEmptyValue(); - if (!status.ok()) { - throw std::runtime_error("Unable to add value to string builder"); - } - for (const auto& frag : val.val) { - status = _builder->ExtendCurrent( - reinterpret_cast(frag.get()), frag.size()); - if (!status.ok()) { - throw std::runtime_error("Unable to add value to string builder"); - } - } -} - -template<> -void scalar_converter::add_primitive( - const iceberg::primitive_value& prim_value) { - if (std::holds_alternative(prim_value)) { - // TODO: This copies the entire string into a std::string. - // FixedSizeBinaryBuilder doesn't have ExtendCurrent. For small - // values that's probably ok, but this will cause problems with - // seastar allocation limits. If we're going to continue using Arrow - // we should make an ArrayBuilder for iobuf and/or ss:sstring. - const auto& val = std::get(prim_value).val; - auto begin = iobuf::byte_iterator(val.cbegin(), val.cend()); - auto end = iobuf::byte_iterator(val.cend(), val.cend()); - std::string str; - std::copy(begin, end, std::back_inserter(str)); - if (str.size() != static_cast(_builder->byte_width())) { - throw std::invalid_argument(fmt::format( - "Expected {} byte value. Got {}.", - static_cast(_builder->byte_width()), - str.size())); - } - append_or_throw(_builder, str); - } else if (std::holds_alternative(prim_value)) { - const auto& val - = std::get(prim_value).val.to_vector(); - append_or_throw(_builder, val.data()); - } else { - throw std::invalid_argument( - "Scalar converter expected a fixed or uuid"); - } -} - -template<> -void scalar_converter::add_primitive( - const iceberg::primitive_value& prim_value) { - const auto& val = get_or_throw( - prim_value, "Scalar converter expected a binary field") - .val; - auto status = _builder->AppendEmptyValue(); - if (!status.ok()) { - throw std::runtime_error("Unable to add value to binary builder"); - } - for (const auto& frag : val) { - status = _builder->ExtendCurrent( - reinterpret_cast(frag.get()), frag.size()); - if (!status.ok()) { - throw std::runtime_error("Unable to add value to binary builder"); - } - } -} - -struct_converter::struct_converter(const iceberg::struct_type& schema) { - _fields.reserve(schema.fields.size()); - std::vector> child_builders; - child_builders.reserve(schema.fields.size()); - for (const auto& field : schema.fields) { - std::unique_ptr child = std::visit( - converter_builder_visitor{}, field->type); - if (!child) { - // We shouldn't get here. The child should throw if it can't be - // constructed correctly. - throw std::invalid_argument("Unable to construct child converter"); - } - _fields.push_back( - child->field(field->name, field->required, field->id)); - _field_required.push_back(bool(field->required)); - child_builders.push_back(child->builder()); - _child_converters.push_back(std::move(child)); - } - _arrow_data_type = arrow::struct_(_fields); - _builder = std::make_shared( - _arrow_data_type, arrow::default_memory_pool(), child_builders); -} - -std::shared_ptr struct_converter::field( - const std::string& name, - iceberg::field_required required, - iceberg::nested_field::id_t field_id) { - return arrow::field( - name, - _arrow_data_type, - required == iceberg::field_required::no, - default_field_metadata(field_id)); -} - -std::shared_ptr struct_converter::builder() { - return _builder; -}; - -void struct_converter::add_optional_data( - const std::optional& maybe_value) { - if (!maybe_value.has_value()) { - // "Automatically appends an empty value to each child builder." - auto status = _builder->AppendNull(); - if (!status.ok()) { - throw std::runtime_error(fmt::format( - "Unable to append null to struct builder: {}", - status.ToString())); - } - } else { - return add_data(maybe_value.value()); - } -} - -void struct_converter::add_struct_data(const iceberg::struct_value struct_val) { - if (struct_val.fields.size() != _child_converters.size()) { - throw std::invalid_argument(fmt::format( - "Got incorrect number of fields in struct converter. Expected {} " - "got {}", - _child_converters.size(), - struct_val.fields.size())); - } - for (size_t i = 0; i < _child_converters.size(); i++) { - if (_field_required[i] && !struct_val.fields[i].has_value()) { - throw std::invalid_argument("Missing required field"); - } - _child_converters[i]->add_optional_data(struct_val.fields[i]); - } - auto status = _builder->Append(); - if (!status.ok()) { - // This is unexpected: if we were able to add to all of the child - // builders, this append should be fine. - throw std::runtime_error(fmt::format( - "Unable to append to struct builder: {}", status.ToString())); - } -} - -void struct_converter::add_data(const iceberg::value& value) { - // 1. Call add_data on all children - // 2. Call append on our builder - const auto& struct_val - = get_or_throw>( - value, "Scalar converter expected a binary field"); - - add_struct_data(std::move(*struct_val)); -} - -arrow::FieldVector struct_converter::get_field_vector() { return _fields; } - -std::shared_ptr struct_converter::take_chunk() { - arrow::Result> builder_result - = _builder->Finish(); - if (!builder_result.status().ok()) { - throw std::runtime_error(fmt::format( - "Unable to create Arrow array: {}", - builder_result.status().ToString())); - } - return builder_result.ValueUnsafe(); -} - -list_converter::list_converter(const iceberg::list_type& schema) { - _child_converter = std::visit( - converter_builder_visitor{}, schema.element_field->type); - if (!_child_converter) { - _arrow_data_type = nullptr; - // We shouldn't get here. The child should throw if it can't be - // constructed correctly. - throw std::runtime_error("Unable to construct child converter"); - } - std::shared_ptr child_field = _child_converter->field( - schema.element_field->name, - schema.element_field->required, - schema.element_field->id); - if (child_field) { - _arrow_data_type = arrow::list(child_field); - } else { - throw std::runtime_error("Child converter has null field"); - } - _builder = std::make_shared( - arrow::default_memory_pool(), - _child_converter->builder(), - _arrow_data_type); -} - -std::shared_ptr list_converter::field( - const std::string& name, - iceberg::field_required required, - iceberg::nested_field::id_t field_id) { - return arrow::field( - name, - _arrow_data_type, - required == iceberg::field_required::no, - default_field_metadata(field_id)); -} - -std::shared_ptr list_converter::builder() { - return _builder; -}; - -void list_converter::add_optional_data( - const std::optional& maybe_value) { - if (!maybe_value.has_value()) { - // This represents adding a null list to the builder, rather than - // a null element to a list - auto status = _builder->AppendNull(); - if (!status.ok()) { - // This is unexpected: if we were able to add to all of the - // child builders, this append should be fine. - throw std::runtime_error(fmt::format( - "Unable to append null to list builder: {}", status.ToString())); - } - } else { - return add_data(maybe_value.value()); - } -} -void list_converter::add_data(const iceberg::value& value) { - if (std::holds_alternative>(value)) { - // List API is: - // 1. Append elements to child - // 2. Append to the parent to start a new list - const iceberg::list_value& list_value - = *get_or_throw>( - value, "List converter expected a list"); - for (const auto& elem : list_value.elements) { - _child_converter->add_optional_data(elem); - } - auto status = _builder->Append(); - if (!status.ok()) { - // This is unexpected: if we were able to add to all of the - // child builders, this append should be fine. - throw std::runtime_error(fmt::format( - "Unable to append to list builder: {}", status.ToString())); - } - } else { - throw std::invalid_argument("Got invalid type for list converter"); - } -} - -map_converter::map_converter(const iceberg::map_type& schema) { - _key_converter = std::visit( - converter_builder_visitor{}, schema.key_field->type); - _value_converter = std::visit( - converter_builder_visitor{}, schema.value_field->type); - if (!_key_converter || !_value_converter) { - _arrow_data_type = nullptr; - // We shouldn't get here. The child should throw if it can't be - // constructed correctly. - throw std::runtime_error("Unable to construct child converter"); - } - - std::shared_ptr key_field = _key_converter->field( - schema.key_field->name, schema.key_field->required, schema.key_field->id); - std::shared_ptr value_field = _value_converter->field( - schema.value_field->name, - schema.value_field->required, - schema.value_field->id); - if (!key_field || !value_field) { - _arrow_data_type = nullptr; - throw std::runtime_error("Child converter has null field"); - } - - _arrow_data_type = arrow::map(key_field->type(), value_field->type()); - _builder = std::make_shared( - arrow::default_memory_pool(), - _key_converter->builder(), - _value_converter->builder()); -} - -std::shared_ptr map_converter::field( - const std::string& name, - iceberg::field_required required, - iceberg::nested_field::id_t field_id) { - return arrow::field( - name, - _arrow_data_type, - required == iceberg::field_required::no, - default_field_metadata(field_id)); -} - -std::shared_ptr map_converter::builder() { - return _builder; -}; - -void map_converter::add_optional_data( - const std::optional& maybe_value) { - // The order of this is reversed from ListBuilder: - // 1. Call Append on the parent builder to create a new slot - // 2. Call append on the child builders to add data - if (!maybe_value.has_value()) { - auto status = _builder->AppendNull(); - if (!status.ok()) { - // This is unexpected: if we were able to add to all of the - // child builders, this append should be fine. - throw std::runtime_error(fmt::format( - "Unable to append null to list builder: {}", status.ToString())); - } - } else { - add_data(maybe_value.value()); - } -} - -void map_converter::add_data(const iceberg::value& value) { - const auto& map_val = *get_or_throw>( - value, "Got invalid type for map converter"); - auto status = _builder->Append(); - if (!status.ok()) { - throw std::runtime_error( - fmt::format("Unable to append to map builder {}", status.ToString())); - } - - for (const iceberg::kv_value& kv : map_val.kvs) { - _key_converter->add_data(kv.key); - _value_converter->add_optional_data(kv.val); - } -} - -arrow_translator::arrow_translator(const iceberg::struct_type& schema) - : _struct_converter(std::make_unique(schema)) {} - -arrow_translator::~arrow_translator() = default; - -std::shared_ptr arrow_translator::build_arrow_schema() { - return arrow::schema(_struct_converter->get_field_vector()); -} - -void arrow_translator::add_data(iceberg::struct_value value) { - _struct_converter->add_struct_data(std::move(value)); -} - -std::shared_ptr arrow_translator::take_chunk() { - return _struct_converter->take_chunk(); -} -} // namespace datalake diff --git a/src/v/datalake/arrow_translator.h b/src/v/datalake/arrow_translator.h deleted file mode 100644 index 2a1e056d058f..000000000000 --- a/src/v/datalake/arrow_translator.h +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Copyright 2024 Redpanda Data, Inc. - * - * Licensed as a Redpanda Enterprise file under the Redpanda Community - * License (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md - */ -#pragma once - -#include "container/fragmented_vector.h" -#include "iceberg/datatypes.h" -#include "iceberg/values.h" - -#include -#include - -#include - -namespace datalake { - -class struct_converter; - -class arrow_translator { -public: - explicit arrow_translator(const iceberg::struct_type& schema); - - ~arrow_translator(); - - std::shared_ptr build_arrow_schema(); - void add_data(iceberg::struct_value value); - - // Returns an arrow:Array for all of the data that has been added since the - // translator was created or the last take_chunk call. It then clears the - // data in the translator so that it's ready to accept more. - std::shared_ptr take_chunk(); - -private: - // Top-level struct that represents the whole schema. - std::unique_ptr _struct_converter; -}; -} // namespace datalake diff --git a/src/v/datalake/arrow_writer.cc b/src/v/datalake/arrow_writer.cc deleted file mode 100644 index 2cc288c7c270..000000000000 --- a/src/v/datalake/arrow_writer.cc +++ /dev/null @@ -1,119 +0,0 @@ -/* - * Copyright 2024 Redpanda Data, Inc. - * - * Licensed as a Redpanda Enterprise file under the Redpanda Community - * License (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md - */ -#include "datalake/arrow_writer.h" - -#include "bytes/iobuf.h" - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - -namespace datalake { - -class arrow_to_iobuf::iobuf_output_stream : public arrow::io::OutputStream { -public: - iobuf_output_stream() = default; - - ~iobuf_output_stream() override = default; - - // Close the stream cleanly. - arrow::Status Close() override { - _closed = true; - return arrow::Status::OK(); - } - - // Return the position in this stream - arrow::Result Tell() const override { return _position; } - - // Return whether the stream is closed - bool closed() const override { return _closed; } - - arrow::Status Write(const void* data, int64_t nbytes) override { - _current_iobuf.append(reinterpret_cast(data), nbytes); - _position += nbytes; - return arrow::Status::OK(); - } - - // TODO: implement this to avoid copying data multiple times - // virtual Status Write(const std::shared_ptr& data); - - // Take the data from the iobuf and clear the internal state. - iobuf take_iobuf() { return std::exchange(_current_iobuf, {}); } - -private: - iobuf _current_iobuf; - bool _closed = false; - int64_t _position = 0; -}; - -arrow_to_iobuf::arrow_to_iobuf(std::shared_ptr schema) { - // TODO: make the compression algorithm configurable. - std::shared_ptr writer_props - = parquet::WriterProperties::Builder() - .compression(arrow::Compression::SNAPPY) - ->build(); - - // Opt to store Arrow schema for easier reads back into Arrow - std::shared_ptr arrow_props - = parquet::ArrowWriterProperties::Builder().store_schema()->build(); - - _ostream = std::make_shared(); - - auto writer_result = parquet::arrow::FileWriter::Open( - *schema, - arrow::default_memory_pool(), - _ostream, - std::move(writer_props), - std::move(arrow_props)); - if (!writer_result.ok()) { - throw std::runtime_error(fmt::format( - "Failed to create Arrow writer: {}", writer_result.status())); - } - _writer = std::move(writer_result.ValueUnsafe()); -} - -void arrow_to_iobuf::add_arrow_array(std::shared_ptr data) { - arrow::ArrayVector data_av = {std::move(data)}; - - std::shared_ptr chunked_data - = std::make_shared(std::move(data_av)); - auto table_result = arrow::Table::FromChunkedStructArray(chunked_data); - if (!table_result.ok()) { - throw std::runtime_error(fmt::format( - "Failed to create arrow table: {}", table_result.status())); - } - auto table = table_result.ValueUnsafe(); - auto write_result = _writer->WriteTable(*table); - if (!write_result.ok()) { - throw std::runtime_error(fmt::format( - "Failed to write arrow table: {}", write_result.ToString())); - } -} - -iobuf arrow_to_iobuf::take_iobuf() { return _ostream->take_iobuf(); } - -iobuf arrow_to_iobuf::close_and_take_iobuf() { - auto status = _writer->Close(); - if (!status.ok()) { - throw std::runtime_error( - fmt::format("Failed to close FileWriter: {}", status.ToString())); - } - return take_iobuf(); -} -} // namespace datalake diff --git a/src/v/datalake/arrow_writer.h b/src/v/datalake/arrow_writer.h deleted file mode 100644 index 219c34342c5c..000000000000 --- a/src/v/datalake/arrow_writer.h +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Copyright 2024 Redpanda Data, Inc. - * - * Licensed as a Redpanda Enterprise file under the Redpanda Community - * License (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md - */ -#pragma once - -#include "bytes/iobuf.h" - -#include -#include - -namespace datalake { - -class arrow_to_iobuf { -public: - explicit arrow_to_iobuf(std::shared_ptr schema); - - void add_arrow_array(std::shared_ptr data); - - // Get the current pending data to be written and clear the internal state. - iobuf take_iobuf(); - - // Close the writer and get any remaining data that is pending. - iobuf close_and_take_iobuf(); - -private: - class iobuf_output_stream; - - std::shared_ptr _ostream; - std::unique_ptr _writer; -}; - -} // namespace datalake diff --git a/src/v/datalake/batching_parquet_writer.cc b/src/v/datalake/batching_parquet_writer.cc deleted file mode 100644 index 32ab4a69de36..000000000000 --- a/src/v/datalake/batching_parquet_writer.cc +++ /dev/null @@ -1,160 +0,0 @@ -/* - * Copyright 2024 Redpanda Data, Inc. - * - * Licensed as a Redpanda Enterprise file under the Redpanda Community - * License (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md - */ - -#include "datalake/batching_parquet_writer.h" - -#include "base/vlog.h" -#include "bytes/iostream.h" -#include "datalake/arrow_translator.h" -#include "datalake/data_writer_interface.h" -#include "datalake/logger.h" - -#include -#include -#include -#include -#include -#include - -#include -#include -#include - -namespace datalake { - -batching_parquet_writer::batching_parquet_writer( - const iceberg::struct_type& schema, - size_t row_count_threshold, - size_t byte_count_threshold, - ss::output_stream output) - : _iceberg_to_arrow(schema) - , _arrow_to_iobuf(_iceberg_to_arrow.build_arrow_schema()) - , _row_count_threshold{row_count_threshold} - , _byte_count_threshold{byte_count_threshold} - , _output_stream(std::move(output)) {} - -ss::future batching_parquet_writer::add_data_struct( - iceberg::struct_value data, size_t approx_size) { - bool error = false; - try { - _iceberg_to_arrow.add_data(std::move(data)); - } catch (...) { - vlog( - datalake_log.error, - "Error adding data value to Arrow table: {}", - std::current_exception()); - error = true; - } - if (error) { - co_return writer_error::parquet_conversion_error; - } - _row_count++; - _byte_count += approx_size; - - if ( - _row_count >= _row_count_threshold - || _byte_count > _byte_count_threshold) { - co_return co_await write_row_group(); - } - - co_return writer_error::ok; -} - -ss::future batching_parquet_writer::finish() { - auto write_result = co_await write_row_group(); - if (write_result != writer_error::ok) { - co_return writer_error::ok; - } - bool error = false; - iobuf out; - try { - out = _arrow_to_iobuf.close_and_take_iobuf(); - - } catch (...) { - vlog( - datalake_log.error, - "Error closing arrow_to_iobuf stream: {}", - std::current_exception()); - error = true; - } - if (error) { - co_return writer_error::parquet_conversion_error; - } - - try { - co_await write_iobuf_to_output_stream(std::move(out), _output_stream); - co_await _output_stream.close(); - } catch (...) { - vlog( - datalake_log.error, - "Error closing output stream: {}", - std::current_exception()); - error = true; - } - if (error) { - co_return writer_error::file_io_error; - } - - co_return writer_error::ok; -} - -ss::future batching_parquet_writer::write_row_group() { - if (_row_count == 0) { - // This can happen if finish() is called when there is no new data. - co_return writer_error::ok; - } - bool error = false; - iobuf out; - try { - auto chunk = _iceberg_to_arrow.take_chunk(); - _row_count = 0; - _byte_count = 0; - _arrow_to_iobuf.add_arrow_array(chunk); - out = _arrow_to_iobuf.take_iobuf(); - } catch (...) { - vlog( - datalake_log.error, - "Error converting Arrow to Parquet iobuf: {}", - std::current_exception()); - error = true; - } - if (error) { - co_return writer_error::parquet_conversion_error; - } - try { - co_await write_iobuf_to_output_stream(std::move(out), _output_stream); - } catch (...) { - vlog( - datalake_log.error, - "Error writing to output stream: {}", - std::current_exception()); - error = true; - } - if (error) { - co_return writer_error::file_io_error; - } - co_return writer_error::ok; -} - -batching_parquet_writer_factory::batching_parquet_writer_factory( - size_t row_count_threshold, size_t byte_count_threshold) - : _row_count_threshold{row_count_threshold} - , _byte_count_threshold{byte_count_threshold} {} - -ss::future> -batching_parquet_writer_factory::create_writer( - const iceberg::struct_type& schema, ss::output_stream output) { - co_return std::make_unique( - std::move(schema), - _row_count_threshold, - _byte_count_threshold, - std::move(output)); -} -} // namespace datalake diff --git a/src/v/datalake/batching_parquet_writer.h b/src/v/datalake/batching_parquet_writer.h deleted file mode 100644 index c2abfd33bf27..000000000000 --- a/src/v/datalake/batching_parquet_writer.h +++ /dev/null @@ -1,82 +0,0 @@ -/* - * Copyright 2024 Redpanda Data, Inc. - * - * Licensed as a Redpanda Enterprise file under the Redpanda Community - * License (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md - */ - -#pragma once -#include "base/outcome.h" -#include "datalake/arrow_translator.h" -#include "datalake/arrow_writer.h" -#include "datalake/data_writer_interface.h" -#include "iceberg/datatypes.h" - -#include -#include - -#include - -#include -#include - -namespace datalake { -// batching_parquet_writer ties together the low-level components for iceberg to -// parquet translation to provide a high-level interface for creating parquet -// files from iceberg::value. It: -// 1. Accepts iceberg::value and collects them in an arrow_translator -// 2. Once the row count or size threshold is reached it writes data to the -// output stream: -// 1. takes a chunk from the arrow_translator -// 2. Adds the chunk to the parquet_writer -// 3. Extracts iobufs from the parquet_writer -// 4. Writes them to the stream -// 4. When finish() is called it flushes all remaining data and closes the -// stream. -class batching_parquet_writer : public parquet_ostream { -public: - batching_parquet_writer( - const iceberg::struct_type& schema, - size_t row_count_threshold, - size_t byte_count_threshold, - ss::output_stream output_stream); - - ss::future - add_data_struct(iceberg::struct_value data, size_t approx_size) override; - - ss::future finish() override; - -private: - ss::future write_row_group(); - - // translating - arrow_translator _iceberg_to_arrow; - arrow_to_iobuf _arrow_to_iobuf; - - // batching - size_t _row_count_threshold; - size_t _byte_count_threshold; - size_t _row_count = 0; - size_t _byte_count = 0; - // Output - ss::output_stream _output_stream; -}; - -class batching_parquet_writer_factory : public parquet_ostream_factory { -public: - batching_parquet_writer_factory( - size_t row_count_threshold, size_t byte_count_threshold); - - ss::future> create_writer( - const iceberg::struct_type& schema, - ss::output_stream output) override; - -private: - size_t _row_count_threshold; - size_t _byte_count_threshold; -}; - -} // namespace datalake diff --git a/src/v/datalake/errors.h b/src/v/datalake/errors.h index 2a67ceec281a..fbdde3fbb21f 100644 --- a/src/v/datalake/errors.h +++ b/src/v/datalake/errors.h @@ -12,17 +12,6 @@ #include namespace datalake { -// TODO: Make an std::error_category instance for this -enum class arrow_converter_status { - ok, - - // User errors - parse_error, - - // System Errors - internal_error, -}; - class initialization_error : public std::runtime_error { public: explicit initialization_error(const std::string& what_arg) diff --git a/src/v/datalake/tests/CMakeLists.txt b/src/v/datalake/tests/CMakeLists.txt index 0bd902a797c6..e92a4e79e78f 100644 --- a/src/v/datalake/tests/CMakeLists.txt +++ b/src/v/datalake/tests/CMakeLists.txt @@ -33,22 +33,7 @@ rp_test( ARGS "-- -c 1" ) -rp_test( - UNIT_TEST - GTEST - BINARY_NAME gtest_arrow_writer - SOURCES arrow_writer_test.cc - LIBRARIES - v::application - v::features - v::gtest_main - v::kafka_test_utils - v::datalake_writer - v::model_test_utils - v::iceberg_test_utils - LABELS datalake - ARGS "-- -c 1" -) + rp_test( FIXTURE_TEST @@ -56,7 +41,6 @@ rp_test( BINARY_NAME datalake SOURCES record_schema_resolver_test.cc - parquet_writer_test.cc partitioning_writer_test.cc LIBRARIES v::bytes @@ -174,19 +158,6 @@ target_include_directories(v_datalake_test_proto_cc_files PUBLIC ${CMAKE_CURRENT_BINARY_DIR} ) -rp_test( - UNIT_TEST - GTEST - BINARY_NAME batching_parquet_writer - SOURCES batching_parquet_writer_test.cc - LIBRARIES - v::gtest_main - v::datalake_writer - v::iceberg_test_utils - LABELS datalake - ARGS "-- -c 1" -) - rp_test( UNIT_TEST GTEST diff --git a/src/v/datalake/tests/arrow_writer_test.cc b/src/v/datalake/tests/arrow_writer_test.cc deleted file mode 100644 index 1fb834fcc022..000000000000 --- a/src/v/datalake/tests/arrow_writer_test.cc +++ /dev/null @@ -1,441 +0,0 @@ -/* - * Copyright 2024 Redpanda Data, Inc. - * - * Licensed as a Redpanda Enterprise file under the Redpanda Community - * License (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md - */ -#include "datalake/arrow_translator.h" -#include "datalake/tests/test_data.h" -#include "iceberg/datatypes.h" -#include "iceberg/tests/value_generator.h" -#include "iceberg/values.h" - -#include -#include - -#include -#include -#include - -TEST(ArrowWriterTest, TranslatesSchemas) { - datalake::arrow_translator writer(test_schema(iceberg::field_required::no)); - auto schema = writer.build_arrow_schema(); - ASSERT_NE(schema, nullptr); - std::string expected_schema = R"(test_bool: bool -test_int: int32 -test_long: int64 -test_float: float -test_double: double -test_decimal: decimal128(16, 8) -test_date: date32[day] -test_time: time64[us] -test_timestamp: timestamp[us] -test_timestamptz: timestamp[us, tz=UTC] -test_string: string -test_uuid: fixed_size_binary[16] -test_fixed: fixed_size_binary[11] -test_binary: binary -test_struct: struct -test_list: list -test_map: map)"; - EXPECT_EQ(schema->ToString(), expected_schema); -} - -TEST(ArrowWriterTest, TranslatesSchemasWithRequired) { - datalake::arrow_translator writer( - test_schema(iceberg::field_required::yes)); - auto schema = writer.build_arrow_schema(); - ASSERT_NE(schema, nullptr); - std::string expected_schema = R"(test_bool: bool not null -test_int: int32 not null -test_long: int64 not null -test_float: float not null -test_double: double not null -test_decimal: decimal128(16, 8) not null -test_date: date32[day] not null -test_time: time64[us] not null -test_timestamp: timestamp[us] not null -test_timestamptz: timestamp[us, tz=UTC] not null -test_string: string not null -test_uuid: fixed_size_binary[16] not null -test_fixed: fixed_size_binary[11] not null -test_binary: binary not null -test_struct: struct not null -test_list: list not null -test_map: map not null)"; - EXPECT_EQ(schema->ToString(), expected_schema); -} - -// putting this at the end so it doesn't clutter up the test. -std::string get_expected_translation_output(); - -TEST(ArrowWriterTest, TranslatesData) { - datalake::arrow_translator schema_translator( - test_schema(iceberg::field_required::no)); - - for (int i = 0; i < 5; i++) { - auto data = iceberg::tests::make_struct_value( - iceberg::tests::value_spec{ - .forced_fixed_val = iobuf::from("Hello world")}, - test_schema(iceberg::field_required::no)); - schema_translator.add_data(std::move(data)); - } - - std::string result_string = schema_translator.take_chunk()->ToString(); - - EXPECT_EQ(result_string, get_expected_translation_output()); -} - -TEST(ArrowWriterTest, FailsOnWrongFieldCount) { - using namespace iceberg; - struct_type schema; - schema.fields.emplace_back(nested_field::create( - 1, "test_bool_1", field_required::no, boolean_type{})); - schema.fields.emplace_back(nested_field::create( - 2, "test_bool_2", field_required::no, boolean_type{})); - - datalake::arrow_translator schema_translator(std::move(schema)); - - std::unique_ptr too_little_data - = std::make_unique(); - too_little_data->fields.emplace_back(boolean_value{true}); - - EXPECT_THROW( - schema_translator.add_data(std::move(*too_little_data)), - std::invalid_argument); -} - -struct required_field_test_params { - iceberg::field_required required; - bool present; - bool should_fail; -}; - -class RequiredFieldTest - : public ::testing::TestWithParam {}; - -TEST_P(RequiredFieldTest, DoRequiredFieldTest) { - // void missing_require_field_test( - // iceberg::field_required required, bool present, bool should_fail) { - const auto params = GetParam(); - using namespace iceberg; - struct_type schema; - schema.fields.emplace_back(nested_field::create( - 1, "test_bool_1", field_required::no, boolean_type{})); - schema.fields.emplace_back( - nested_field::create(2, "test_bool_2", params.required, boolean_type{})); - - datalake::arrow_translator schema_translator(std::move(schema)); - - std::unique_ptr missing_field - = std::make_unique(); - missing_field->fields.emplace_back(boolean_value{true}); - - if (params.present) { - missing_field->fields.emplace_back(boolean_value{true}); - } else { - missing_field->fields.emplace_back(std::nullopt); - } - - if (params.should_fail) { - EXPECT_THROW( - schema_translator.add_data(std::move(*missing_field)), - std::invalid_argument); - } else { - schema_translator.add_data(std::move(*missing_field)); - } -} - -INSTANTIATE_TEST_SUITE_P( - RequiredFieldTestInst, - RequiredFieldTest, - ::testing::Values( - required_field_test_params{ - .required = iceberg::field_required::no, - .present = true, - .should_fail = false}, - required_field_test_params{ - .required = iceberg::field_required::no, - .present = false, - .should_fail = false}, - required_field_test_params{ - .required = iceberg::field_required::yes, - .present = true, - .should_fail = false}, - required_field_test_params{ - .required = iceberg::field_required::yes, - .present = false, - .should_fail = true})); - -TEST(ArrowWriterTest, BadSchema) { - using namespace iceberg; - struct_type input_schema; - - // A decimal type with a precision of 0 will fail. - input_schema.fields.emplace_back(nested_field::create( - 6, "test_decimal", field_required::yes, decimal_type{0, 16})); - - EXPECT_THROW( - datalake::arrow_translator(std::move(input_schema)), - std::invalid_argument); -} - -std::string get_expected_translation_output() { - return R"(-- is_valid: all not null --- child 0 type: bool - [ - false, - false, - false, - false, - false - ] --- child 1 type: int32 - [ - 0, - 0, - 0, - 0, - 0 - ] --- child 2 type: int64 - [ - 0, - 0, - 0, - 0, - 0 - ] --- child 3 type: float - [ - 0, - 0, - 0, - 0, - 0 - ] --- child 4 type: double - [ - 0, - 0, - 0, - 0, - 0 - ] --- child 5 type: decimal128(16, 8) - [ - 0.E-8, - 0.E-8, - 0.E-8, - 0.E-8, - 0.E-8 - ] --- child 6 type: date32[day] - [ - 1970-01-01, - 1970-01-01, - 1970-01-01, - 1970-01-01, - 1970-01-01 - ] --- child 7 type: time64[us] - [ - 00:00:00.000000, - 00:00:00.000000, - 00:00:00.000000, - 00:00:00.000000, - 00:00:00.000000 - ] --- child 8 type: timestamp[us] - [ - 1970-01-01 00:00:00.000000, - 1970-01-01 00:00:00.000000, - 1970-01-01 00:00:00.000000, - 1970-01-01 00:00:00.000000, - 1970-01-01 00:00:00.000000 - ] --- child 9 type: timestamp[us, tz=UTC] - [ - 1970-01-01 00:00:00.000000Z, - 1970-01-01 00:00:00.000000Z, - 1970-01-01 00:00:00.000000Z, - 1970-01-01 00:00:00.000000Z, - 1970-01-01 00:00:00.000000Z - ] --- child 10 type: string - [ - "", - "", - "", - "", - "" - ] --- child 11 type: fixed_size_binary[16] - [ - 00000000000000000000000000000000, - 00000000000000000000000000000000, - 00000000000000000000000000000000, - 00000000000000000000000000000000, - 00000000000000000000000000000000 - ] --- child 12 type: fixed_size_binary[11] - [ - 48656C6C6F20776F726C64, - 48656C6C6F20776F726C64, - 48656C6C6F20776F726C64, - 48656C6C6F20776F726C64, - 48656C6C6F20776F726C64 - ] --- child 13 type: binary - [ - , - , - , - , - - ] --- child 14 type: struct - -- is_valid: all not null - -- child 0 type: bool - [ - false, - false, - false, - false, - false - ] - -- child 1 type: int32 - [ - 0, - 0, - 0, - 0, - 0 - ] - -- child 2 type: int64 - [ - 0, - 0, - 0, - 0, - 0 - ] --- child 15 type: list - [ - [ - "", - "", - "", - "", - "" - ], - [ - "", - "", - "", - "", - "" - ], - [ - "", - "", - "", - "", - "" - ], - [ - "", - "", - "", - "", - "" - ], - [] - ] --- child 16 type: map - [ - keys: - [ - "", - "", - "", - "", - "" - ] - values: - [ - 0, - 0, - 0, - 0, - 0 - ], - keys: - [ - "", - "", - "", - "", - "" - ] - values: - [ - 0, - 0, - 0, - 0, - 0 - ], - keys: - [ - "", - "", - "", - "", - "" - ] - values: - [ - 0, - 0, - 0, - 0, - 0 - ], - keys: - [ - "", - "", - "", - "", - "" - ] - values: - [ - 0, - 0, - 0, - 0, - 0 - ], - keys: - [ - "", - "", - "", - "", - "" - ] - values: - [ - 0, - 0, - 0, - 0, - 0 - ] - ])"; -} diff --git a/src/v/datalake/tests/batching_parquet_writer_test.cc b/src/v/datalake/tests/batching_parquet_writer_test.cc deleted file mode 100644 index 82d668e68918..000000000000 --- a/src/v/datalake/tests/batching_parquet_writer_test.cc +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Copyright 2024 Redpanda Data, Inc. - * - * Licensed as a Redpanda Enterprise file under the Redpanda Community - * License (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md - */ -#include "datalake/batching_parquet_writer.h" -#include "datalake/local_parquet_file_writer.h" -#include "datalake/tests/test_data.h" -#include "iceberg/tests/value_generator.h" -#include "iceberg/values.h" -#include "test_utils/tmp_dir.h" - -#include - -#include -#include -#include -#include - -#include - -namespace datalake { - -TEST(BatchingParquetWriterTest, WritesParquetFiles) { - temporary_dir tmp_dir("batching_parquet_writer"); - std::filesystem::path file_path = "test_file.parquet"; - std::filesystem::path full_path = tmp_dir.get_path() / file_path; - int num_rows = 1000; - - local_parquet_file_writer file_writer( - local_path(full_path), - ss::make_shared(500, 100000)); - - file_writer.initialize(test_schema(iceberg::field_required::no)).get(); - - for (int i = 0; i < num_rows; i++) { - auto data = iceberg::tests::make_struct_value( - iceberg::tests::value_spec{ - .forced_fixed_val = iobuf::from("Hello world")}, - test_schema(iceberg::field_required::no)); - file_writer.add_data_struct(std::move(data), 1000).get(); - } - - auto result = file_writer.finish().get0(); - ASSERT_TRUE(result.has_value()); - EXPECT_EQ(result.value().path, full_path); - EXPECT_EQ(result.value().row_count, num_rows); - auto true_file_size = std::filesystem::file_size(full_path); - EXPECT_EQ(result.value().size_bytes, true_file_size); - - // Read the file and check the contents - auto reader = arrow::io::ReadableFile::Open(full_path).ValueUnsafe(); - - // Open Parquet file reader - std::unique_ptr arrow_reader; - ASSERT_TRUE(parquet::arrow::OpenFile( - reader, arrow::default_memory_pool(), &arrow_reader) - .ok()); - - // Read entire file as a single Arrow table - std::shared_ptr table; - auto r = arrow_reader->ReadTable(&table); - ASSERT_TRUE(r.ok()); - - EXPECT_EQ(table->num_rows(), num_rows); - EXPECT_EQ(table->num_columns(), 17); -} - -} // namespace datalake diff --git a/src/v/datalake/tests/gtest_record_multiplexer_test.cc b/src/v/datalake/tests/gtest_record_multiplexer_test.cc index 0cb9c7cb0465..c79da1106484 100644 --- a/src/v/datalake/tests/gtest_record_multiplexer_test.cc +++ b/src/v/datalake/tests/gtest_record_multiplexer_test.cc @@ -8,12 +8,12 @@ * https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md */ #include "datalake/base_types.h" -#include "datalake/batching_parquet_writer.h" #include "datalake/catalog_schema_manager.h" #include "datalake/local_parquet_file_writer.h" #include "datalake/record_multiplexer.h" #include "datalake/record_schema_resolver.h" #include "datalake/record_translator.h" +#include "datalake/serde_parquet_writer.h" #include "datalake/table_creator.h" #include "datalake/tests/catalog_and_registry_fixture.h" #include "datalake/tests/record_generator.h" @@ -25,10 +25,7 @@ #include "storage/record_batch_builder.h" #include "test_utils/tmp_dir.h" -#include -#include #include -#include #include @@ -128,7 +125,7 @@ TEST(DatalakeMultiplexerTest, WritesDataFiles) { auto writer_factory = std::make_unique( datalake::local_path(tmp_dir.get_path()), "data", - ss::make_shared(100, 10000)); + ss::make_shared()); datalake::record_multiplexer multiplexer( ntp, @@ -164,34 +161,6 @@ TEST(DatalakeMultiplexerTest, WritesDataFiles) { EXPECT_EQ( result.value().last_offset(), start_offset + record_count * batch_count - 1); - - // Open the resulting file and check that it has data in it with the - // appropriate counts. - int file_count = 0; - for (const auto& entry : - std::filesystem::directory_iterator(tmp_dir.get_path())) { - file_count++; - auto arrow_file_reader - = arrow::io::ReadableFile::Open(entry.path()).ValueUnsafe(); - - // Open Parquet file reader - std::unique_ptr arrow_reader; - ASSERT_TRUE( - parquet::arrow::OpenFile( - arrow_file_reader, arrow::default_memory_pool(), &arrow_reader) - .ok()); - - // Read entire file as a single Arrow table - std::shared_ptr table; - auto r = arrow_reader->ReadTable(&table); - ASSERT_TRUE(r.ok()); - - EXPECT_EQ(table->num_rows(), record_count * batch_count); - // Expect one nested column and one value column. - EXPECT_EQ(table->num_columns(), 2); - } - // Expect this test to create exactly 1 file - EXPECT_EQ(file_count, 1); } namespace { @@ -274,7 +243,7 @@ TEST_F(RecordMultiplexerParquetTest, TestSimple) { auto writer_factory = std::make_unique( datalake::local_path(tmp_dir.get_path()), "data", - ss::make_shared(100, 10000)); + ss::make_shared()); record_multiplexer mux( ntp, rev, @@ -289,39 +258,4 @@ TEST_F(RecordMultiplexerParquetTest, TestSimple) { const auto num_records = num_hrs * batches_per_hr * records_per_batch; EXPECT_EQ(res.value().last_offset(), start_offset() + num_records - 1); - int file_count = 0; - for (const auto& entry : - std::filesystem::directory_iterator(tmp_dir.get_path())) { - file_count++; - auto arrow_file_reader - = arrow::io::ReadableFile::Open(entry.path()).ValueUnsafe(); - std::unique_ptr arrow_reader; - ASSERT_TRUE( - parquet::arrow::OpenFile( - arrow_file_reader, arrow::default_memory_pool(), &arrow_reader) - .ok()); - - // Read entire file as a single Arrow table - std::shared_ptr table; - auto r = arrow_reader->ReadTable(&table); - ASSERT_TRUE(r.ok()); - - // Records should be split across hours. - EXPECT_EQ(table->num_rows(), num_records / num_hrs); - - // Default columns + a nested struct. - EXPECT_EQ(table->num_columns(), 8); - auto expected_type - = R"(redpanda: struct not null>, key: binary> not null -mylong: int64 not null -nestedrecord: struct not null -myarray: list not null -mybool: bool not null -myfixed: fixed_size_binary[16] not null -anotherint: int32 not null -bytes: binary not null)"; - EXPECT_EQ(expected_type, table->schema()->ToString()); - } - // Expect this test to create exactly 1 file - EXPECT_EQ(file_count, num_hrs); } diff --git a/src/v/datalake/tests/parquet_writer_test.cc b/src/v/datalake/tests/parquet_writer_test.cc deleted file mode 100644 index 5501fcc3d795..000000000000 --- a/src/v/datalake/tests/parquet_writer_test.cc +++ /dev/null @@ -1,79 +0,0 @@ -/* - * Copyright 2024 Redpanda Data, Inc. - * - * Licensed as a Redpanda Enterprise file under the Redpanda Community - * License (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md - */ - -#include "bytes/bytes.h" -#include "datalake/arrow_translator.h" -#include "datalake/arrow_writer.h" -#include "datalake/tests/test_data.h" -#include "iceberg/tests/value_generator.h" -#include "utils/file_io.h" - -#include -#include -#include -#include -#include -#include -#include -#include - -TEST(ParquetWriter, CreatesValidParquetData) { - datalake::arrow_translator translator( - test_schema(iceberg::field_required::no)); - - iobuf full_result; - - for (int i = 0; i < 5; i++) { - auto data = iceberg::tests::make_struct_value( - iceberg::tests::value_spec{ - .forced_fixed_val = iobuf::from("Hello world")}, - test_schema(iceberg::field_required::no)); - translator.add_data(std::move(data)); - } - - std::shared_ptr result = translator.take_chunk(); - ASSERT_NE(result, nullptr); - - datalake::arrow_to_iobuf writer(translator.build_arrow_schema()); - - for (int i = 0; i < 10; i++) { - writer.add_arrow_array(result); - iobuf serialized = writer.take_iobuf(); - // Sizes are not consistent between writes, but should be about right. - EXPECT_NEAR(serialized.size_bytes(), 3300, 200); - full_result.append_fragments(std::move(serialized)); - } - - // The last write is long. This is probably Parquet footer information. - auto serialized = writer.close_and_take_iobuf(); - EXPECT_NEAR(serialized.size_bytes(), 22000, 1000); - full_result.append_fragments(std::move(serialized)); - - EXPECT_NEAR(full_result.size_bytes(), 57000, 1000); - - // Check that the data is a valid parquet file. Convert the iobuf to a - // single buffer then import that into an arrow::io::BufferReader - auto full_result_bytes = iobuf_to_bytes(full_result); - auto reader = std::make_shared( - full_result_bytes.data(), full_result_bytes.size()); - - // Open Parquet file reader - std::unique_ptr arrow_reader; - ASSERT_TRUE(parquet::arrow::OpenFile( - reader, arrow::default_memory_pool(), &arrow_reader) - .ok()); - - // Read entire file as a single Arrow table - std::shared_ptr table; - ASSERT_TRUE(arrow_reader->ReadTable(&table).ok()); - - EXPECT_EQ(table->num_rows(), 10 * 5); - EXPECT_EQ(table->num_columns(), 17); -} diff --git a/src/v/datalake/tests/translation_task_test.cc b/src/v/datalake/tests/translation_task_test.cc index 69aef68f782a..11341aef61f0 100644 --- a/src/v/datalake/tests/translation_task_test.cc +++ b/src/v/datalake/tests/translation_task_test.cc @@ -10,12 +10,12 @@ #include "cloud_io/tests/scoped_remote.h" #include "cloud_storage/tests/s3_imposter.h" -#include "datalake/batching_parquet_writer.h" #include "datalake/catalog_schema_manager.h" #include "datalake/cloud_data_io.h" #include "datalake/local_parquet_file_writer.h" #include "datalake/record_schema_resolver.h" #include "datalake/record_translator.h" +#include "datalake/serde_parquet_writer.h" #include "datalake/table_creator.h" #include "datalake/translation_task.h" #include "model/record_batch_reader.h" @@ -84,8 +84,7 @@ class TranslateTaskTest return std::make_unique( datalake::local_path(tmp_dir.get_path()), "test-prefix", - ss::make_shared( - row_threshold, bytes_threshold)); + ss::make_shared()); } lazy_abort_source& never_abort() { diff --git a/ubsan_suppressions.txt b/ubsan_suppressions.txt index 744b232ea4e8..35bfc76406c5 100644 --- a/ubsan_suppressions.txt +++ b/ubsan_suppressions.txt @@ -3,4 +3,3 @@ nonnull-attribute:aead.c alignment:crc32c_arm64.cc function:openssl/*.c function:openssl-fips/*.c -alignment:arrow/util/ubsan.h From 55a498710b5bad4f20e66b5a936aa4e677e216dd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Fri, 6 Dec 2024 09:39:47 +0100 Subject: [PATCH 055/229] redpanda: remove datalake bazel build defines MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Michał Maślanka --- src/v/datalake/BUILD | 2 +- src/v/redpanda/BUILD | 8 ++------ src/v/redpanda/application.cc | 13 +------------ 3 files changed, 4 insertions(+), 19 deletions(-) diff --git a/src/v/datalake/BUILD b/src/v/datalake/BUILD index 362f3089cbd5..24908e6c8609 100644 --- a/src/v/datalake/BUILD +++ b/src/v/datalake/BUILD @@ -293,7 +293,7 @@ redpanda_cc_library( "cloud_data_io.h", ], include_prefix = "datalake", - visibility = [":__subpackages__"], + visibility = ["//visibility:public"], deps = [ ":base_types", ":logger", diff --git a/src/v/redpanda/BUILD b/src/v/redpanda/BUILD index 32c9d3308ac4..6d8f889bd0d7 100644 --- a/src/v/redpanda/BUILD +++ b/src/v/redpanda/BUILD @@ -14,12 +14,6 @@ redpanda_cc_library( "monitor_unsafe.h", ], include_prefix = "redpanda", - # TODO(iceberg): Remove once arrow is bazel friendly - # and all the datalake components are ported over to - # bazel. - local_defines = [ - "BAZEL_DISABLE_DATALAKE_FEATURE=1", - ], visibility = ["//visibility:public"], deps = [ "//src/v/base", @@ -38,6 +32,8 @@ redpanda_cc_library( "//src/v/crypto", # "//src/v/datalake:manager", "//src/v/datalake:types", + "//src/v/datalake:cloud_data_io", + "//src/v/datalake:manager", "//src/v/datalake/coordinator:coordinator_manager", "//src/v/datalake/coordinator:frontend", "//src/v/datalake/coordinator:stm", diff --git a/src/v/redpanda/application.cc b/src/v/redpanda/application.cc index 0e562fb5143c..8443216db7e2 100644 --- a/src/v/redpanda/application.cc +++ b/src/v/redpanda/application.cc @@ -87,16 +87,14 @@ #include "config/seed_server.h" #include "config/types.h" #include "crypto/ossl_context_service.h" -#include "datalake/coordinator/catalog_factory.h" -#ifndef BAZEL_DISABLE_DATALAKE_FEATURE #include "datalake/cloud_data_io.h" +#include "datalake/coordinator/catalog_factory.h" #include "datalake/coordinator/coordinator_manager.h" #include "datalake/coordinator/frontend.h" #include "datalake/coordinator/service.h" #include "datalake/coordinator/state_machine.h" #include "datalake/datalake_manager.h" #include "datalake/translation/state_machine.h" -#endif #include "debug_bundle/debug_bundle_service.h" #include "features/feature_table_snapshot.h" #include "features/fwd.h" @@ -1422,7 +1420,6 @@ void application::wire_up_runtime_services( .get(); } -#ifndef BAZEL_DISABLE_DATALAKE_FEATURE if (datalake_enabled()) { vassert( bucket.has_value(), @@ -1489,7 +1486,6 @@ void application::wire_up_runtime_services( _datalake_manager.invoke_on_all(&datalake::datalake_manager::start) .get(); } -#endif construct_single_service(_monitor_unsafe, std::ref(feature_table)); construct_service(_debug_bundle_service, &storage.local().kvs()).get(); @@ -2955,10 +2951,8 @@ void application::start_runtime_services( pm.register_factory( storage.local().kvs(), config::shard_local_cfg().rm_sync_timeout_ms.bind()); -#ifndef BAZEL_DISABLE_DATALAKE_FEATURE pm.register_factory(); pm.register_factory(); -#endif if (config::shard_local_cfg().development_enable_cloud_topics()) { pm.register_factory(); } @@ -3005,7 +2999,6 @@ void application::start_runtime_services( if (offsets_recovery_router.local_is_initialized()) { offsets_recovery_requestor = offsets_recovery_manager; } -#ifndef BAZEL_DISABLE_DATALAKE_FEATURE if (_datalake_coordinator_mgr.local_is_initialized()) { // Before starting the controller, start the coordinator manager so we // don't miss any partition/leadership notifications. @@ -3013,7 +3006,6 @@ void application::start_runtime_services( .invoke_on_all(&datalake::coordinator::coordinator_manager::start) .get(); } -#endif controller ->start( cd, @@ -3155,14 +3147,11 @@ void application::start_runtime_services( smp_service_groups.cluster_smp_sg(), std::ref(controller->get_data_migration_frontend()), std::ref(controller->get_data_migration_irpc_frontend()))); - -#ifndef BAZEL_DISABLE_DATALAKE_FEATURE runtime_services.push_back( std::make_unique( sched_groups.datalake_sg(), smp_service_groups.datalake_sg(), &_datalake_coordinator_fe)); -#endif s.add_services(std::move(runtime_services)); From 8f1d8098d09bc817510eae94f34bbf3bac31fa66 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Fri, 6 Dec 2024 11:01:50 +0100 Subject: [PATCH 056/229] ducktape: applied formatting to bash scripts MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Michał Maślanka --- tests/docker/ducktape-deps/ocsf-server | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/docker/ducktape-deps/ocsf-server b/tests/docker/ducktape-deps/ocsf-server index 1ecfdd245985..c55ca8a782fa 100644 --- a/tests/docker/ducktape-deps/ocsf-server +++ b/tests/docker/ducktape-deps/ocsf-server @@ -1,7 +1,7 @@ #!/usr/bin/env bash set -e -OCSF_SCHEMA_VERSION=9608805fe0b61035cb821bb9068096fe47fed12d # tip of v1.0.0 branch +OCSF_SCHEMA_VERSION=9608805fe0b61035cb821bb9068096fe47fed12d # tip of v1.0.0 branch OCSF_SERVER_VERSION=d3b26de39df9eb33c6d63e34a126c77c0811c7a0 wget "https://github.com/redpanda-data/ocsf-schema/archive/${OCSF_SCHEMA_VERSION}.tar.gz" From 1f5f3633ee14522c0962daf18500a972228c2f60 Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Thu, 5 Dec 2024 19:57:35 -0500 Subject: [PATCH 057/229] `storage`: check for `may_have_tombstones` in `self_compact_segment` Segments that may have tombstone records in them should still be considered eligible for self compaction. An early return statement was missing a check for this condition. Add it so that tombstones will be properly removed for a segment eligible for removal. Also adjusts a `vassert()` to account for this case. --- src/v/storage/segment_utils.cc | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/src/v/storage/segment_utils.cc b/src/v/storage/segment_utils.cc index 7cfcca47b9dd..3d130303518a 100644 --- a/src/v/storage/segment_utils.cc +++ b/src/v/storage/segment_utils.cc @@ -725,9 +725,10 @@ ss::future self_compact_segment( "Cannot compact an active segment. cfg:{} - segment:{}", cfg, s)); } + const bool may_remove_tombstones = may_have_removable_tombstones(s, cfg); if ( !s->has_compactible_offsets(cfg) - || (s->finished_self_compaction() && !may_have_removable_tombstones(s, cfg))) { + || (s->finished_self_compaction() && !may_remove_tombstones)) { co_return compaction_result{s->size_bytes()}; } @@ -736,7 +737,11 @@ ss::future self_compact_segment( = co_await maybe_rebuild_compaction_index( s, stm_manager, cfg, read_holder, resources, pb); - if (state == compacted_index::recovery_state::already_compacted) { + const bool segment_already_compacted + = (state == compacted_index::recovery_state::already_compacted) + && !may_remove_tombstones; + + if (segment_already_compacted) { vlog( gclog.debug, "detected {} is already compacted", @@ -745,10 +750,10 @@ ss::future self_compact_segment( co_return compaction_result{s->size_bytes()}; } - vassert( - state == compacted_index::recovery_state::index_recovered, - "Unexpected state {}", - state); + const bool is_valid_index_state + = (state == compacted_index::recovery_state::index_recovered) + || (state == compacted_index::recovery_state::already_compacted); + vassert(is_valid_index_state, "Unexpected state {}", state); auto sz_before = s->size_bytes(); auto apply_offset = should_apply_delta_time_offset(feature_table); From 5b50f25e9be30bb58bbffc371323da39d2718b05 Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Tue, 26 Nov 2024 13:11:37 -0500 Subject: [PATCH 058/229] `storage`: add `_segment_cleanly_compacted` to `probe` A metric that measures the number of segments that have been cleanly compacted (i.e, had their keys de-duplicated with all previous segments before them to the front of the log). --- src/v/storage/disk_log_impl.cc | 8 ++++---- src/v/storage/probe.cc | 8 ++++++++ src/v/storage/probe.h | 2 ++ src/v/storage/segment_utils.cc | 3 ++- src/v/storage/segment_utils.h | 2 +- 5 files changed, 17 insertions(+), 6 deletions(-) diff --git a/src/v/storage/disk_log_impl.cc b/src/v/storage/disk_log_impl.cc index 6a138a938fa4..6c5a9c9d314c 100644 --- a/src/v/storage/disk_log_impl.cc +++ b/src/v/storage/disk_log_impl.cc @@ -616,7 +616,7 @@ ss::future disk_log_impl::sliding_window_compact( // compacted. auto seg = segs.front(); co_await internal::mark_segment_as_finished_window_compaction( - seg, true); + seg, true, *_probe); segs.pop_front(); } if (segs.empty()) { @@ -707,7 +707,7 @@ ss::future disk_log_impl::sliding_window_compact( // entirely comprised of non-data batches. Mark it as compacted so // we can progress through compactions. co_await internal::mark_segment_as_finished_window_compaction( - seg, is_clean_compacted); + seg, is_clean_compacted, *_probe); vlog( gclog.debug, @@ -722,7 +722,7 @@ ss::future disk_log_impl::sliding_window_compact( // All data records are already compacted away. Skip to avoid a // needless rewrite. co_await internal::mark_segment_as_finished_window_compaction( - seg, is_clean_compacted); + seg, is_clean_compacted, *_probe); vlog( gclog.trace, @@ -822,7 +822,7 @@ ss::future disk_log_impl::sliding_window_compact( // Mark the segment as completed window compaction, and possibly set the // clean_compact_timestamp in it's index. co_await internal::mark_segment_as_finished_window_compaction( - seg, is_clean_compacted); + seg, is_clean_compacted, *_probe); co_await seg->index().flush(); co_await ss::rename_file( diff --git a/src/v/storage/probe.cc b/src/v/storage/probe.cc index ed650aba358c..ef4a1328401f 100644 --- a/src/v/storage/probe.cc +++ b/src/v/storage/probe.cc @@ -201,6 +201,14 @@ void probe::setup_metrics(const model::ntp& ntp) { sm::description("Number of tombstone records removed by compaction " "due to the delete.retention.ms setting."), labels), + sm::make_counter( + "cleanly_compacted_segment", + [this] { return _segment_cleanly_compacted; }, + sm::description( + "Number of segments cleanly compacted (i.e, had their " + "keys de-duplicated with all previous segments " + "before them to the front of the log)"), + labels), }, {}, {sm::shard_label, partition_label}); diff --git a/src/v/storage/probe.h b/src/v/storage/probe.h index dcc464dd740e..88608ba5c212 100644 --- a/src/v/storage/probe.h +++ b/src/v/storage/probe.h @@ -98,6 +98,7 @@ class probe { } void add_removed_tombstone() { ++_tombstones_removed; } + void add_cleanly_compacted_segment() { ++_segment_cleanly_compacted; } void batch_parse_error() { ++_batch_parse_errors; } @@ -140,6 +141,7 @@ class probe { uint32_t _batch_write_errors = 0; double _compaction_ratio = 1.0; uint64_t _tombstones_removed = 0; + uint64_t _segment_cleanly_compacted = 0; ssize_t _compaction_removed_bytes = 0; diff --git a/src/v/storage/segment_utils.cc b/src/v/storage/segment_utils.cc index 3d130303518a..6cba638d9cd2 100644 --- a/src/v/storage/segment_utils.cc +++ b/src/v/storage/segment_utils.cc @@ -1142,12 +1142,13 @@ offset_delta_time should_apply_delta_time_offset( } ss::future<> mark_segment_as_finished_window_compaction( - ss::lw_shared_ptr seg, bool set_clean_compact_timestamp) { + ss::lw_shared_ptr seg, bool set_clean_compact_timestamp, probe& pb) { seg->mark_as_finished_windowed_compaction(); if (set_clean_compact_timestamp) { bool did_set = seg->index().maybe_set_clean_compact_timestamp( model::timestamp::now()); if (did_set) { + pb.add_cleanly_compacted_segment(); return seg->index().flush(); } } diff --git a/src/v/storage/segment_utils.h b/src/v/storage/segment_utils.h index 51f6e51041ea..df60ec3a07e4 100644 --- a/src/v/storage/segment_utils.h +++ b/src/v/storage/segment_utils.h @@ -284,7 +284,7 @@ bool may_have_removable_tombstones( // Also potentially issues a call to seg->index()->flush(), if the // `clean_compact_timestamp` was set in the index. ss::future<> mark_segment_as_finished_window_compaction( - ss::lw_shared_ptr seg, bool set_clean_compact_timestamp); + ss::lw_shared_ptr seg, bool set_clean_compact_timestamp, probe& pb); template auto with_segment_reader_handle(segment_reader_handle handle, Func func) { From 8d66388ecf56ff7e1a10f0df62f4b56312bdf8a6 Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Tue, 26 Nov 2024 13:07:19 -0500 Subject: [PATCH 059/229] `storage`: add `_segments_marked_tombstone_free` to `probe` A metric that measures the number of segments that have been verified through the compaction process to be tombstone free. --- src/v/storage/probe.cc | 6 ++++++ src/v/storage/probe.h | 4 ++++ src/v/storage/segment_deduplication_utils.cc | 6 ++++++ src/v/storage/segment_utils.cc | 6 ++++++ 4 files changed, 22 insertions(+) diff --git a/src/v/storage/probe.cc b/src/v/storage/probe.cc index ef4a1328401f..2e50738a4360 100644 --- a/src/v/storage/probe.cc +++ b/src/v/storage/probe.cc @@ -209,6 +209,12 @@ void probe::setup_metrics(const model::ntp& ntp) { "keys de-duplicated with all previous segments " "before them to the front of the log)"), labels), + sm::make_counter( + "segments_marked_tombstone_free", + [this] { return _segments_marked_tombstone_free; }, + sm::description("Number of segments that have been verified through " + "the compaction process to be tombstone free."), + labels), }, {}, {sm::shard_label, partition_label}); diff --git a/src/v/storage/probe.h b/src/v/storage/probe.h index 88608ba5c212..5579995d6bec 100644 --- a/src/v/storage/probe.h +++ b/src/v/storage/probe.h @@ -99,6 +99,9 @@ class probe { void add_removed_tombstone() { ++_tombstones_removed; } void add_cleanly_compacted_segment() { ++_segment_cleanly_compacted; } + void add_segment_marked_tombstone_free() { + ++_segments_marked_tombstone_free; + } void batch_parse_error() { ++_batch_parse_errors; } @@ -142,6 +145,7 @@ class probe { double _compaction_ratio = 1.0; uint64_t _tombstones_removed = 0; uint64_t _segment_cleanly_compacted = 0; + uint64_t _segments_marked_tombstone_free = 0; ssize_t _compaction_removed_bytes = 0; diff --git a/src/v/storage/segment_deduplication_utils.cc b/src/v/storage/segment_deduplication_utils.cc index fd4d058e77ad..62511506afdd 100644 --- a/src/v/storage/segment_deduplication_utils.cc +++ b/src/v/storage/segment_deduplication_utils.cc @@ -245,6 +245,12 @@ ss::future deduplicate_segment( // Set may_have_tombstone_records new_idx.may_have_tombstone_records = may_have_tombstone_records; + if ( + seg->index().may_have_tombstone_records() + && !may_have_tombstone_records) { + probe.add_segment_marked_tombstone_free(); + } + co_return new_idx; } diff --git a/src/v/storage/segment_utils.cc b/src/v/storage/segment_utils.cc index 6cba638d9cd2..5ab8341a7d56 100644 --- a/src/v/storage/segment_utils.cc +++ b/src/v/storage/segment_utils.cc @@ -460,6 +460,12 @@ ss::future do_copy_segment_data( // Set may_have_tombstone_records new_index.may_have_tombstone_records = may_have_tombstone_records; + if ( + seg->index().may_have_tombstone_records() + && !may_have_tombstone_records) { + pb.add_segment_marked_tombstone_free(); + } + co_return new_index; } From 87d442bf2a992eae2a139b8f33d122a19094ec1a Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Tue, 26 Nov 2024 13:08:43 -0500 Subject: [PATCH 060/229] `storage`: add `_num_rounds_window_compaction` to `probe` A metric that measures the number of rounds of sliding window compaction that have been driven to completion. --- src/v/storage/disk_log_impl.cc | 4 ++-- src/v/storage/probe.cc | 6 ++++++ src/v/storage/probe.h | 4 ++++ 3 files changed, 12 insertions(+), 2 deletions(-) diff --git a/src/v/storage/disk_log_impl.cc b/src/v/storage/disk_log_impl.cc index 6c5a9c9d314c..0c68787b89e3 100644 --- a/src/v/storage/disk_log_impl.cc +++ b/src/v/storage/disk_log_impl.cc @@ -522,7 +522,7 @@ segment_set disk_log_impl::find_sliding_range( config().ntp(), _last_compaction_window_start_offset.value(), _segs.front()->offsets().get_base_offset()); - + _probe->add_sliding_window_round_complete(); _last_compaction_window_start_offset.reset(); } @@ -687,7 +687,7 @@ ss::future disk_log_impl::sliding_window_compact( "{}, resetting sliding window start offset", config().ntp(), idx_start_offset); - + _probe->add_sliding_window_round_complete(); next_window_start_offset.reset(); } diff --git a/src/v/storage/probe.cc b/src/v/storage/probe.cc index 2e50738a4360..de7c96109fb6 100644 --- a/src/v/storage/probe.cc +++ b/src/v/storage/probe.cc @@ -215,6 +215,12 @@ void probe::setup_metrics(const model::ntp& ntp) { sm::description("Number of segments that have been verified through " "the compaction process to be tombstone free."), labels), + sm::make_counter( + "complete_sliding_window_rounds", + [this] { return _num_rounds_window_compaction; }, + sm::description("Number of rounds of sliding window compaction that " + "have been driven to completion."), + labels), }, {}, {sm::shard_label, partition_label}); diff --git a/src/v/storage/probe.h b/src/v/storage/probe.h index 5579995d6bec..449e6aefeff2 100644 --- a/src/v/storage/probe.h +++ b/src/v/storage/probe.h @@ -102,6 +102,9 @@ class probe { void add_segment_marked_tombstone_free() { ++_segments_marked_tombstone_free; } + void add_sliding_window_round_complete() { + ++_num_rounds_window_compaction; + } void batch_parse_error() { ++_batch_parse_errors; } @@ -146,6 +149,7 @@ class probe { uint64_t _tombstones_removed = 0; uint64_t _segment_cleanly_compacted = 0; uint64_t _segments_marked_tombstone_free = 0; + uint64_t _num_rounds_window_compaction = 0; ssize_t _compaction_removed_bytes = 0; From 85bea2b00b2ec5035d7f3a679e555f9c227db950 Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Wed, 4 Dec 2024 16:42:55 -0500 Subject: [PATCH 061/229] `tests`: bump `kgo-verifier` version and support new features See https://github.com/redpanda-data/kgo-verifier/pull/60, which added `--tombstone-probability`, `--compacted`, `--validate-latest-values` as input parameters. --- tests/docker/ducktape-deps/kgo-verifier | 2 +- .../rptest/services/kgo_verifier_services.py | 51 +++++++++++++++---- 2 files changed, 43 insertions(+), 10 deletions(-) diff --git a/tests/docker/ducktape-deps/kgo-verifier b/tests/docker/ducktape-deps/kgo-verifier index 176b425778ad..75b4be9f8d29 100644 --- a/tests/docker/ducktape-deps/kgo-verifier +++ b/tests/docker/ducktape-deps/kgo-verifier @@ -2,6 +2,6 @@ set -e git -C /opt clone https://github.com/redpanda-data/kgo-verifier.git cd /opt/kgo-verifier -git reset --hard 7bbf8c883d1807cdf297fdb589d92f436604772b +git reset --hard bffac1f1358875ee6e91308229d908f40d5fe18e go mod tidy make diff --git a/tests/rptest/services/kgo_verifier_services.py b/tests/rptest/services/kgo_verifier_services.py index efbd4cab4393..286d904c5383 100644 --- a/tests/rptest/services/kgo_verifier_services.py +++ b/tests/rptest/services/kgo_verifier_services.py @@ -228,6 +228,7 @@ def clean_node(self, node: ClusterNode): self._redpanda.logger.info(f"{self.__class__.__name__}.clean_node") node.account.kill_process("kgo-verifier", clean_shutdown=False) node.account.remove("valid_offsets*json", True) + node.account.remove("latest_value*json", True) node.account.remove(f"/tmp/{self.__class__.__name__}*", True) def _remote(self, node, action, timeout=60): @@ -475,8 +476,8 @@ class ValidatorStatus: def __init__(self, name: str, valid_reads: int, invalid_reads: int, out_of_scope_invalid_reads: int, - max_offsets_consumed: Optional[int], lost_offsets: Dict[str, - int]): + max_offsets_consumed: Optional[int], + lost_offsets: Dict[str, int], tombstones_consumed: int): # Validator name is just a unique name per worker thread in kgo-verifier: useful in logging # but we mostly don't care self.name = name @@ -486,6 +487,7 @@ def __init__(self, name: str, valid_reads: int, invalid_reads: int, self.out_of_scope_invalid_reads = out_of_scope_invalid_reads self.max_offsets_consumed = max_offsets_consumed self.lost_offsets = lost_offsets + self.tombstones_consumed = tombstones_consumed @property def total_reads(self): @@ -510,7 +512,8 @@ def __str__(self): f"valid_reads={self.valid_reads}, " \ f"invalid_reads={self.invalid_reads}, " \ f"out_of_scope_invalid_reads={self.out_of_scope_invalid_reads}, " \ - f"lost_offsets={self.lost_offsets}>" + f"lost_offsets={self.lost_offsets}, " \ + f"tombstones_consumed={self.tombstones_consumed}>" class ConsumerStatus: @@ -531,7 +534,8 @@ def __init__(self, 'out_of_scope_invalid_reads': 0, 'name': "", 'max_offsets_consumed': dict(), - 'lost_offsets': dict() + 'lost_offsets': dict(), + 'tombstones_consumed': 0 } self.validator = ValidatorStatus(**validator) @@ -571,7 +575,9 @@ def __init__(self, msgs_per_producer_id=None, max_buffered_records=None, tolerate_data_loss=False, - tolerate_failed_produce=False): + tolerate_failed_produce=False, + tombstone_probability=0.0, + validate_latest_values=False): super(KgoVerifierProducer, self).__init__(context, redpanda, topic, msg_size, custom_node, debug_logs, trace_logs, username, password, @@ -590,6 +596,8 @@ def __init__(self, self._max_buffered_records = max_buffered_records self._tolerate_data_loss = tolerate_data_loss self._tolerate_failed_produce = tolerate_failed_produce + self._tombstone_probability = tombstone_probability + self._validate_latest_values = validate_latest_values @property def produce_status(self): @@ -697,6 +705,11 @@ def start_node(self, node, clean=False): if self._tolerate_failed_produce: cmd += " --tolerate-failed-produce" + if self._tombstone_probability is not None: + cmd += f" --tombstone-probability {self._tombstone_probability}" + if self._validate_latest_values: + cmd += " --validate-latest-values" + self.spawn(cmd, node) self._status_thread = StatusThread(self, node, ProduceStatus) @@ -745,7 +758,9 @@ def __init__( username: Optional[str] = None, password: Optional[str] = None, enable_tls: Optional[bool] = False, - use_transactions: Optional[bool] = False): + use_transactions: Optional[bool] = False, + compacted: Optional[bool] = False, + validate_latest_values: Optional[bool] = False): super().__init__(context, redpanda, topic, msg_size, nodes, debug_logs, trace_logs, username, password, enable_tls) self._max_msgs = max_msgs @@ -755,6 +770,8 @@ def __init__( self._tolerate_data_loss = tolerate_data_loss self._producer = producer self._use_transactions = use_transactions + self._compacted = compacted + self._validate_latest_values = validate_latest_values def start_node(self, node, clean=False): if clean: @@ -778,6 +795,11 @@ def start_node(self, node, clean=False): cmd += " --tolerate-data-loss" if self._use_transactions: cmd += " --use-transactions" + if self._compacted: + cmd += " --compacted" + if self._validate_latest_values: + cmd += " --validate-latest-values" + self.spawn(cmd, node) self._status_thread = StatusThread(self, node, ConsumerStatus) @@ -877,7 +899,9 @@ def __init__(self, continuous=False, tolerate_data_loss=False, group_name=None, - use_transactions=False): + use_transactions=False, + compacted=False, + validate_latest_values=False): super().__init__(context, redpanda, topic, msg_size, nodes, debug_logs, trace_logs, username, password, enable_tls) @@ -889,6 +913,8 @@ def __init__(self, self._continuous = continuous self._tolerate_data_loss = tolerate_data_loss self._use_transactions = use_transactions + self._compacted = compacted + self._validate_latest_values = validate_latest_values def start_node(self, node, clean=False): if clean: @@ -915,6 +941,11 @@ def start_node(self, node, clean=False): cmd += f" --consumer_group_name {self._group_name}" if self._use_transactions: cmd += " --use-transactions" + if self._compacted: + cmd += " --compacted" + if self._validate_latest_values: + cmd += " --validate-latest-values" + self.spawn(cmd, node) self._status_thread = StatusThread(self, node, ConsumerStatus) @@ -933,7 +964,8 @@ def __init__(self, active=False, failed_transactions=0, aborted_transaction_msgs=0, - fails=0): + fails=0, + tombstones_produced=0): self.topic = topic self.sent = sent self.acked = acked @@ -947,7 +979,8 @@ def __init__(self, self.failed_transactions = failed_transactions self.aborted_transaction_messages = aborted_transaction_msgs self.fails = fails + self.tombstones_produced = tombstones_produced def __str__(self): l = self.latency - return f"ProduceStatus<{self.sent} {self.acked} {self.bad_offsets} {self.restarts} {self.failed_transactions} {self.aborted_transaction_messages} {self.fails} {l['p50']}/{l['p90']}/{l['p99']}>" + return f"ProduceStatus<{self.sent} {self.acked} {self.bad_offsets} {self.restarts} {self.failed_transactions} {self.aborted_transaction_messages} {self.fails} {self.tombstones_produced} {l['p50']}/{l['p90']}/{l['p99']}>" From 15ab6a85579871913a9e30280d3052ebc0a3e5c9 Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Mon, 18 Nov 2024 09:32:23 -0500 Subject: [PATCH 062/229] `rptest`: add `wait_for_latest_value_map()` to `kgo` --- tests/rptest/services/kgo_verifier_services.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/tests/rptest/services/kgo_verifier_services.py b/tests/rptest/services/kgo_verifier_services.py index 286d904c5383..42a283b05c1e 100644 --- a/tests/rptest/services/kgo_verifier_services.py +++ b/tests/rptest/services/kgo_verifier_services.py @@ -652,6 +652,15 @@ def wait_for_offset_map(self): backoff_sec=1) self._status_thread.raise_on_error() + def wait_for_latest_value_map(self): + # Producer worker aims to checkpoint every 5 seconds, so we should see this promptly. + self._redpanda.wait_until(lambda: self._status_thread.errored or all( + node.account.exists(f"latest_value_{self._topic}.json") + for node in self.nodes), + timeout_sec=15, + backoff_sec=1) + self._status_thread.raise_on_error() + def is_complete(self): return self._status.acked >= self._msg_count From ac273cc3213c5512988a880f12324f2b0cd38ebd Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Fri, 15 Nov 2024 10:49:06 -0500 Subject: [PATCH 063/229] `rptest`: use `delete.retention.ms` in `KafkaCliTools.create_topic()` --- tests/rptest/clients/kafka_cli_tools.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/rptest/clients/kafka_cli_tools.py b/tests/rptest/clients/kafka_cli_tools.py index f5b707c8221b..3f6ae9884f80 100644 --- a/tests/rptest/clients/kafka_cli_tools.py +++ b/tests/rptest/clients/kafka_cli_tools.py @@ -152,6 +152,10 @@ def create_topic(self, spec: TopicSpec): args += ["--config", f"retention.ms={spec.retention_ms}"] if spec.max_message_bytes: args += ["--config", f"max.message.bytes={spec.max_message_bytes}"] + if spec.delete_retention_ms: + args += [ + "--config", f"delete.retention.ms={spec.delete_retention_ms}" + ] return self._run("kafka-topics.sh", args, desc="create_topic") def create_topic_partitions(self, topic: str, partitions: int): From 985beffa2034e543d8685e9196a390f5e03772c0 Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Thu, 28 Nov 2024 09:56:38 -0500 Subject: [PATCH 064/229] `rptest`: add `_wait_for_file_on_nodes()` to `KgoVerifierProducer` Also add an error message to the `wait_until()` call. --- .../rptest/services/kgo_verifier_services.py | 25 ++++++++++--------- 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/tests/rptest/services/kgo_verifier_services.py b/tests/rptest/services/kgo_verifier_services.py index 42a283b05c1e..dc2c6c2017e5 100644 --- a/tests/rptest/services/kgo_verifier_services.py +++ b/tests/rptest/services/kgo_verifier_services.py @@ -643,23 +643,24 @@ def wait_for_acks(self, count, timeout_sec, backoff_sec): backoff_sec=backoff_sec) self._status_thread.raise_on_error() + def _wait_for_file_on_nodes(self, file_name): + self._redpanda.wait_until( + lambda: self._status_thread.errored or all( + node.account.exists(file_name) for node in self.nodes), + timeout_sec=15, + backoff_sec=1, + err_msg=f"Timed out waiting for {file_name} to be created") + self._status_thread.raise_on_error() + def wait_for_offset_map(self): # Producer worker aims to checkpoint every 5 seconds, so we should see this promptly. - self._redpanda.wait_until(lambda: self._status_thread.errored or all( - node.account.exists(f"valid_offsets_{self._topic}.json") - for node in self.nodes), - timeout_sec=15, - backoff_sec=1) - self._status_thread.raise_on_error() + offset_map_file_name = f"valid_offsets_{self._topic}.json" + self._wait_for_file_on_nodes(offset_map_file_name) def wait_for_latest_value_map(self): # Producer worker aims to checkpoint every 5 seconds, so we should see this promptly. - self._redpanda.wait_until(lambda: self._status_thread.errored or all( - node.account.exists(f"latest_value_{self._topic}.json") - for node in self.nodes), - timeout_sec=15, - backoff_sec=1) - self._status_thread.raise_on_error() + value_map_file_name = f"latest_value_{self._topic}.json" + self._wait_for_file_on_nodes(value_map_file_name) def is_complete(self): return self._status.acked >= self._msg_count From d238b1c9b606758520149e2f17aee241a8368a2d Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Fri, 15 Nov 2024 11:24:34 -0500 Subject: [PATCH 065/229] `rptest`: add `log_compaction_test.py` --- tests/rptest/tests/log_compaction_test.py | 262 ++++++++++++++++++++++ 1 file changed, 262 insertions(+) create mode 100644 tests/rptest/tests/log_compaction_test.py diff --git a/tests/rptest/tests/log_compaction_test.py b/tests/rptest/tests/log_compaction_test.py new file mode 100644 index 000000000000..61171447ac4c --- /dev/null +++ b/tests/rptest/tests/log_compaction_test.py @@ -0,0 +1,262 @@ +# Copyright 2024 Redpanda Data, Inc. +# +# Use of this software is governed by the Business Source License +# included in the file licenses/BSL.md +# +# As of the Change Date specified in that file, in accordance with +# the Business Source License, use of this software will be governed +# by the Apache License, Version 2.0 +import time +import threading + +from ducktape.mark import matrix +from ducktape.utils.util import wait_until +from rptest.clients.types import TopicSpec +from rptest.services.cluster import cluster +from rptest.services.kgo_verifier_services import KgoVerifierProducer, KgoVerifierConsumerGroupConsumer, KgoVerifierSeqConsumer +from rptest.services.redpanda import MetricsEndpoint +from rptest.tests.partition_movement import PartitionMovementMixin +from rptest.tests.redpanda_test import RedpandaTest +from rptest.utils.mode_checks import skip_debug_mode +from rptest.tests.prealloc_nodes import PreallocNodesTest + + +class LogCompactionTest(PreallocNodesTest, PartitionMovementMixin): + def __init__(self, test_context): + self.test_context = test_context + # Run with small segments, a low retention value and a very frequent compaction interval. + self.extra_rp_conf = { + 'log_compaction_interval_ms': 4000, + 'log_segment_size': 2 * 1024**2, # 2 MiB + 'retention_bytes': 25 * 1024**2, # 25 MiB + 'compacted_log_segment_size': 1024**2 # 1 MiB + } + super().__init__(test_context=test_context, + num_brokers=3, + node_prealloc_count=1, + extra_rp_conf=self.extra_rp_conf) + + def topic_setup(self, cleanup_policy, key_set_cardinality): + """ + Sets variables and creates topic. + """ + self.msg_size = 1024 # 1 KiB + self.rate_limit = 50 * 1024**2 # 50 MiBps + self.total_data = 100 * 1024**2 # 100 MiB + self.msg_count = int(self.total_data / self.msg_size) + self.tombstone_probability = 0.4 + self.partition_count = 10 + self.cleanup_policy = cleanup_policy + self.key_set_cardinality = key_set_cardinality + + # A value below log_compaction_interval_ms (therefore, tombstones that would be compacted away during deduplication will be visibly removed instead) + self.delete_retention_ms = 3000 + self.topic_spec = TopicSpec( + name="tapioca", + delete_retention_ms=self.delete_retention_ms, + partition_count=self.partition_count, + cleanup_policy=self.cleanup_policy) + self.client().create_topic(self.topic_spec) + + def get_removed_tombstones(self): + return self.redpanda.metric_sum( + metric_name="vectorized_storage_log_tombstones_removed_total", + metrics_endpoint=MetricsEndpoint.METRICS, + topic=self.topic_spec.name) + + def get_cleanly_compacted_segments(self): + return self.redpanda.metric_sum( + metric_name= + "vectorized_storage_log_cleanly_compacted_segment_total", + metrics_endpoint=MetricsEndpoint.METRICS, + topic=self.topic_spec.name) + + def get_segments_marked_tombstone_free(self): + return self.redpanda.metric_sum( + metric_name= + "vectorized_storage_log_segments_marked_tombstone_free_total", + metrics_endpoint=MetricsEndpoint.METRICS, + topic=self.topic_spec.name) + + def get_complete_sliding_window_rounds(self): + return self.redpanda.metric_sum( + metric_name= + "vectorized_storage_log_complete_sliding_window_rounds_total", + metrics_endpoint=MetricsEndpoint.METRICS, + topic=self.topic_spec.name) + + def produce_and_consume(self): + """ + Creates producer and consumer. Asserts that tombstones are seen + in the consumer log. + """ + + producer = KgoVerifierProducer( + context=self.test_context, + redpanda=self.redpanda, + topic=self.topic_spec.name, + debug_logs=True, + trace_logs=True, + msg_size=self.msg_size, + msg_count=self.msg_count, + rate_limit_bps=self.rate_limit, + key_set_cardinality=self.key_set_cardinality, + tolerate_data_loss=False, + tombstone_probability=self.tombstone_probability, + validate_latest_values=True, + custom_node=self.preallocated_nodes) + + # Produce and wait + producer.start() + producer.wait_for_latest_value_map() + producer.wait(timeout_sec=180) + + assert producer.produce_status.tombstones_produced > 0 + assert producer.produce_status.bad_offsets == 0 + + consumer = KgoVerifierSeqConsumer(self.test_context, + self.redpanda, + self.topic_spec.name, + self.msg_size, + debug_logs=True, + trace_logs=True, + compacted=True, + loop=False, + nodes=self.preallocated_nodes) + + # Consume and wait. clean=False to not accidentally remove latest value map. + consumer.start(clean=False) + consumer.wait(timeout_sec=180) + + # Clean up + producer.stop() + consumer.stop() + + assert consumer.consumer_status.validator.tombstones_consumed > 0 + assert consumer.consumer_status.validator.invalid_reads == 0 + + def validate_log(self): + """ + After several rounds of compaction, restart the brokers, + create a consumer, and assert that no tombstones are consumed. + Latest key-value pairs in the log are verified in KgoVerifier. + """ + + # Restart each redpanda broker to force roll segments + self.redpanda.restart_nodes(self.redpanda.nodes) + + # Sleep until the log has been fully compacted. + self.prev_sliding_window_rounds = -1 + self.prev_tombstones_removed = -1 + + def compaction_has_completed(): + # In order to be confident that compaction has settled, + # we check that the number of compaction rounds that + # have occured as well as the number of tombstones records + # removed have stabilized over some period longer than + # log_compaction_interval_ms (and expected time for compaction to complete). + new_sliding_window_rounds = self.get_complete_sliding_window_rounds( + ) + new_tombstones_removed = self.get_removed_tombstones() + res = self.prev_sliding_window_rounds == new_sliding_window_rounds and self.prev_tombstones_removed == new_tombstones_removed + self.prev_sliding_window_rounds = new_sliding_window_rounds + self.prev_tombstones_removed = new_tombstones_removed + return res + + wait_until( + compaction_has_completed, + timeout_sec=120, + backoff_sec=self.extra_rp_conf['log_compaction_interval_ms'] / + 1000 * 4, + err_msg="Compaction did not stabilize.") + + assert self.get_complete_sliding_window_rounds() > 0 + assert self.get_cleanly_compacted_segments() > 0 + assert self.get_segments_marked_tombstone_free() > 0 + + consumer = KgoVerifierSeqConsumer(self.test_context, + self.redpanda, + self.topic_spec.name, + self.msg_size, + debug_logs=True, + trace_logs=True, + compacted=True, + loop=False, + validate_latest_values=True, + nodes=self.preallocated_nodes) + + # Consume and wait. clean=False to not accidentally remove latest value map. + consumer.start(clean=False) + consumer.wait(timeout_sec=180) + + consumer.stop() + + # Expect to see 0 tombstones consumed + assert consumer.consumer_status.validator.tombstones_consumed == 0 + assert consumer.consumer_status.validator.invalid_reads == 0 + + @skip_debug_mode + @cluster(num_nodes=4) + @matrix( + cleanup_policy=[ + TopicSpec.CLEANUP_COMPACT, TopicSpec.CLEANUP_COMPACT_DELETE + ], + key_set_cardinality=[100, 1000], + ) + def compaction_stress_test(self, cleanup_policy, key_set_cardinality): + """ + Uses partition movement and frequent compaction/garbage collecting to + validate tombstone removal and general compaction behavior. + """ + self.topic_setup(cleanup_policy, key_set_cardinality) + + class PartitionMoveExceptionReporter: + exc = None + + def background_test_loop(reporter, + fn, + iterations=10, + sleep_sec=1, + allowable_retries=3): + try: + while iterations > 0: + try: + fn() + except Exception as e: + if allowable_retries == 0: + raise e + time.sleep(sleep_sec) + iterations -= 1 + allowable_retries -= 1 + except Exception as e: + reporter.exc = e + + def issue_partition_move(): + try: + self._dispatch_random_partition_move(self.topic_spec.name, 0) + self._wait_for_move_in_progress(self.topic_spec.name, + 0, + timeout=5) + except Exception as e: + reporter.exc = e + + partition_move_thread = threading.Thread( + target=background_test_loop, + args=(PartitionMoveExceptionReporter, issue_partition_move), + kwargs={ + 'iterations': 5, + 'sleep_sec': 1 + }) + + # Start partition movement thread + partition_move_thread.start() + + self.produce_and_consume() + + self.validate_log() + + # Clean up partition movement thread + partition_move_thread.join() + + if PartitionMoveExceptionReporter.exc is not None: + raise PartitionMoveExceptionReporter.exc From 2d4906e99b4cc1e16e612e1e5f681116d6620288 Mon Sep 17 00:00:00 2001 From: Michael Boquard Date: Fri, 6 Dec 2024 09:50:42 -0500 Subject: [PATCH 066/229] bazel: Updated seastar ref Pull in changes from https://github.com/redpanda-data/seastar/pull/158 Signed-off-by: Michael Boquard --- MODULE.bazel.lock | 8 ++++---- bazel/repositories.bzl | 6 +++--- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/MODULE.bazel.lock b/MODULE.bazel.lock index 97b77eca5e62..4c011d5f4672 100644 --- a/MODULE.bazel.lock +++ b/MODULE.bazel.lock @@ -172,7 +172,7 @@ "moduleExtensions": { "//bazel:extensions.bzl%non_module_dependencies": { "general": { - "bzlTransitiveDigest": "TvXu6DtfWgMGFFU5j/KjnvJsrxMUxNyLFZxqRWHgAxw=", + "bzlTransitiveDigest": "IQG/FXTm4steXpXTdw3O4KW8OvtPDvBzPK9dFKzQz68=", "usagesDigest": "8eEn6X1e7HKkx2OPWUwQBOEnT9TIkMhEcXbu/wRjGno=", "recordedFileInputs": {}, "recordedDirentsInputs": {}, @@ -359,9 +359,9 @@ "ruleClassName": "http_archive", "attributes": { "build_file": "@@//bazel/thirdparty:seastar.BUILD", - "sha256": "8b71096e8c57a5a1a447638fc64b05686ecd7f079861de7a3fe3112fd0e4d6f7", - "strip_prefix": "seastar-ef24a8bc3f7dca212dfe982911bb726e5d37cef5", - "url": "https://github.com/redpanda-data/seastar/archive/ef24a8bc3f7dca212dfe982911bb726e5d37cef5.tar.gz", + "sha256": "254bd1c2d26bb2d9df8e5217e1c50a2f08bce58b0a6d13cdc692192ee8b79a56", + "strip_prefix": "seastar-f194d34ef62b5a64364291b941f97a9cd19b7835", + "url": "https://github.com/redpanda-data/seastar/archive/f194d34ef62b5a64364291b941f97a9cd19b7835.tar.gz", "patches": [ "@@//bazel/thirdparty:seastar-fortify-source.patch" ], diff --git a/bazel/repositories.bzl b/bazel/repositories.bzl index 86a9e9ae8eb0..20bc51a601f1 100644 --- a/bazel/repositories.bzl +++ b/bazel/repositories.bzl @@ -167,9 +167,9 @@ def data_dependency(): http_archive( name = "seastar", build_file = "//bazel/thirdparty:seastar.BUILD", - sha256 = "8b71096e8c57a5a1a447638fc64b05686ecd7f079861de7a3fe3112fd0e4d6f7", - strip_prefix = "seastar-ef24a8bc3f7dca212dfe982911bb726e5d37cef5", - url = "https://github.com/redpanda-data/seastar/archive/ef24a8bc3f7dca212dfe982911bb726e5d37cef5.tar.gz", + sha256 = "254bd1c2d26bb2d9df8e5217e1c50a2f08bce58b0a6d13cdc692192ee8b79a56", + strip_prefix = "seastar-f194d34ef62b5a64364291b941f97a9cd19b7835", + url = "https://github.com/redpanda-data/seastar/archive/f194d34ef62b5a64364291b941f97a9cd19b7835.tar.gz", patches = ["//bazel/thirdparty:seastar-fortify-source.patch"], patch_args = ["-p1"], ) From 7deb38ba5165252bda0c4ad5d0c54b4731bbbec0 Mon Sep 17 00:00:00 2001 From: Travis Downs Date: Fri, 6 Dec 2024 11:58:14 -0300 Subject: [PATCH 067/229] iobuf: update the allocator schedule iobuf uses a growing schedule of allocations (growth factor 1.5) for each fragment. This schedule was not aware of the seastar allocation sizes, so it would request, for example 66K allocation, but seastar internally rounds that up to 128K, so for that allocation we waste ~50% of the memory which is invisible to iobuf. In this change we update the schedule to be seastar allocator aware, i.e., using the same ~1.5x growth factor, but rounding up to the next seastar allocator boundary. At 16K or below, these boundaries are log-linear: every 2^n size, plus 3 evenly spread sizes in between each power of 2. Above 16K, sizes are 2^n. This change slightly reduces the total number of steps until we get to the max size of 128K, as from 32K to 128K we use doubling steps instead of 1.5. For large iobufs this results in 1 fewer total which is why some tests which tested the exact number of fragments needed to be decreased by 1. Fixes CORE-8478. --- src/v/bytes/details/io_allocation_size.h | 56 +++++++++++++++-------- src/v/bytes/tests/iobuf_tests.cc | 58 ++++++++++++------------ 2 files changed, 65 insertions(+), 49 deletions(-) diff --git a/src/v/bytes/details/io_allocation_size.h b/src/v/bytes/details/io_allocation_size.h index d46d6f5cf82c..c7cd938f956c 100644 --- a/src/v/bytes/details/io_allocation_size.h +++ b/src/v/bytes/details/io_allocation_size.h @@ -29,28 +29,46 @@ class io_allocation_size { static constexpr size_t ss_max_small_allocation = 16384; public: - // >>> x=512 - // >>> while x < int((1024*128)): - // ... print(x) - // ... x=int(((x*3)+1)/2) - // ... x=int(min(1024*128,x)) - // print(1024*128) + // This script computes the table immediately below, which are "ideal" + // allocation sizes: rounded up to the next true size supported by + // the seastar allocator. At <= 16K we apply the small pool indexing + // logic, and above we use powers of 2 since we are in the buddy allocator + // which only supports power-of-two sizes. + // + // We scale the target size by 1.47, i.e., 1.5 tweaked slightly to ensure + // we hit 16K at the small<->big pool boundary. + // + // def lg2(size: int): + // return size.bit_length() - 1 + // def p(v: object): + // print(f"{v},") + // s = 512 + // fb = 2 # idx_frac_bits + // while s <= 2**14: + // # The size calculation below is doing idx_to_size(size_to_idx(s)), + // # i.e., figuring out which small point index the allocation falls in + // # then seeing what the size of that small pool is, i.e., the size of + // # the smallest small pool that can fix this allocation. + // # See the corresponding routines in src/core/memory.cc: + // #https://github.com/scylladb/seastar/blob/f840b860432e7e716e3cfc004690897b50dc122c/src/core/memory.cc#L478-L499 + // idx = ((lg2(s) << fb) - ((1 << fb) - 1)) + ((s - 1) >> (lg2(s) - fb)) + // p((((1 << fb) | (idx & ((1 << fb) - 1))) << (idx >> fb)) >> fb) + // s = int(s * 1.47) + // for e in [15, 16, 17]: + // p(2**e) static constexpr auto alloc_table = std::to_array( - // computed from a python script above {512, 768, - 1152, - 1728, - 2592, - 3888, - 5832, - 8748, - 13122, - 19683, - 29525, - 44288, - 66432, - 99648, + 1280, + 1792, + 2560, + 3584, + 6144, + 8192, + 12288, + 16384, + 32768, + 65536, 131072}); static size_t next_allocation_size(size_t data_size); diff --git a/src/v/bytes/tests/iobuf_tests.cc b/src/v/bytes/tests/iobuf_tests.cc index 54dda05a5e7d..101195e6bcc2 100644 --- a/src/v/bytes/tests/iobuf_tests.cc +++ b/src/v/bytes/tests/iobuf_tests.cc @@ -379,38 +379,36 @@ SEASTAR_THREAD_TEST_CASE(iobuf_as_ostream) { } SEASTAR_THREAD_TEST_CASE(alloctor_forward_progress) { - static constexpr std::array src = {{ + static constexpr auto src = std::to_array({ 512, 768, - 1152, - 1728, - 2592, - 3888, - 5832, - 8748, - 13122, - 19683, - 29525, - 44288, - 66432, - 99648, - }}; - static constexpr std::array expected = {{ + 1280, + 1792, + 2560, + 3584, + 6144, + 8192, + 12288, + 16384, + 32768, + 65536, + 131072, + }); + static constexpr auto expected = std::to_array({ 768, - 1152, - 1728, - 2592, - 3888, - 5832, - 8748, - 13122, - 19683, - 29525, - 44288, - 66432, - 99648, + 1280, + 1792, + 2560, + 3584, + 6144, + 8192, + 12288, + 16384, + 32768, + 65536, + 131072, 131072, - }}; + }); BOOST_REQUIRE_EQUAL(src.size(), expected.size()); for (size_t i = 0; i < src.size(); ++i) { BOOST_REQUIRE_EQUAL( @@ -429,7 +427,7 @@ SEASTAR_THREAD_TEST_CASE(test_next_chunk_allocation_append_temp_buf) { } // slow but tha'ts life. auto distance = std::distance(buf.begin(), buf.end()); - BOOST_REQUIRE_EQUAL(distance, 324); + BOOST_REQUIRE_EQUAL(distance, 323); constexpr size_t sz = 40000 * 1024; auto msg = iobuf_as_scattered(std::move(buf)); BOOST_REQUIRE_EQUAL(msg.size(), sz); @@ -445,7 +443,7 @@ SEASTAR_THREAD_TEST_CASE(test_next_chunk_allocation_append_iobuf) { } // slow but tha'ts life. auto distance = std::distance(buf.begin(), buf.end()); - BOOST_REQUIRE_EQUAL(distance, 324); + BOOST_REQUIRE_EQUAL(distance, 323); constexpr size_t sz = 40000 * 1024; auto msg = iobuf_as_scattered(std::move(buf)); BOOST_REQUIRE_EQUAL(msg.size(), sz); From c931fc60f9b1b450a07f81872891338b39f73457 Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Mon, 2 Dec 2024 19:05:49 -0800 Subject: [PATCH 068/229] datalake/tests: move counter stream utility to DatalakeServices --- .../tests/datalake/datalake_services.py | 31 +++++++++++++++++++ .../tests/datalake/datalake_verifier_test.py | 29 +---------------- 2 files changed, 32 insertions(+), 28 deletions(-) diff --git a/tests/rptest/tests/datalake/datalake_services.py b/tests/rptest/tests/datalake/datalake_services.py index 14351355d8dd..ac3bf9133d7c 100644 --- a/tests/rptest/tests/datalake/datalake_services.py +++ b/tests/rptest/tests/datalake/datalake_services.py @@ -17,6 +17,7 @@ from rptest.services.spark_service import SparkService from rptest.services.trino_service import TrinoService from rptest.tests.datalake.query_engine_base import QueryEngineType +from rptest.services.redpanda_connect import RedpandaConnectService from rptest.tests.datalake.query_engine_factory import get_query_engine_by_type @@ -94,6 +95,36 @@ def spark(self) -> SparkService: assert spark, "Missing Spark service" return spark + def start_counter_stream(self, + topic: str, + name: str = "ducky_stream", + count: int = 100, + interval: str = "") -> RedpandaConnectService: + stream_conf = { + "input": { + "generate": { + "mapping": "root = counter()", + "interval": interval, + "count": count, + "batch_size": 1 + } + }, + "pipeline": { + "processors": [] + }, + "output": { + "redpanda": { + "seed_brokers": self.redpanda.brokers_list(), + "topic": topic, + } + } + } + connect = RedpandaConnectService(self.test_ctx, self.redpanda) + connect.start() + # create a stream + connect.start_stream(name, config=stream_conf) + return connect + def service(self, engine_type: QueryEngineType): for e in self.query_engines: if e.engine_name() == engine_type: diff --git a/tests/rptest/tests/datalake/datalake_verifier_test.py b/tests/rptest/tests/datalake/datalake_verifier_test.py index 974a0736c8a6..1087aba19d6c 100644 --- a/tests/rptest/tests/datalake/datalake_verifier_test.py +++ b/tests/rptest/tests/datalake/datalake_verifier_test.py @@ -40,27 +40,6 @@ def __init__(self, test_context): }, schema_registry_config=SchemaRegistryConfig()) - def simple_stream(self, topic, subject): - return { - "input": { - "generate": { - "mapping": "root = counter()", - "interval": "", - "count": 100, - "batch_size": 1 - } - }, - "pipeline": { - "processors": [] - }, - "output": { - "redpanda": { - "seed_brokers": self.redpanda.brokers_list(), - "topic": topic, - } - } - } - def setUp(self): pass @@ -70,13 +49,7 @@ def _prepare_test_data(self, topic_name: str, dl: DatalakeServices): partitions=1, replicas=1, iceberg_mode="key_value") - - connect = RedpandaConnectService(self.test_context, self.redpanda) - connect.start() - # create a stream - connect.start_stream(name="ducky_stream", - config=self.simple_stream(topic_name, - "verifier_schema")) + connect = dl.start_counter_stream(topic=topic_name) dl.wait_for_translation(topic_name, 100) connect.wait_for_stream_to_finish("ducky_stream") From 27a27aae181f3835098829455208273a3cee6f9b Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Mon, 2 Dec 2024 22:18:39 -0800 Subject: [PATCH 069/229] datalake/tests: utility to scrape stream metrics .. also renames stop method for readability. --- tests/rptest/services/redpanda_connect.py | 35 ++++++++++++------- .../tests/datalake/datalake_verifier_test.py | 2 +- .../tests/datalake/simple_connect_test.py | 2 +- tests/rptest/tests/redpanda_connect_test.py | 2 +- 4 files changed, 26 insertions(+), 15 deletions(-) diff --git a/tests/rptest/services/redpanda_connect.py b/tests/rptest/services/redpanda_connect.py index fed5de46b823..eca3c63dfe05 100644 --- a/tests/rptest/services/redpanda_connect.py +++ b/tests/rptest/services/redpanda_connect.py @@ -18,6 +18,7 @@ import requests from rptest.services.redpanda import RedpandaService from ducktape.utils.util import wait_until +from prometheus_client.parser import text_string_to_metric_families from ducktape.cluster.cluster import ClusterNode @@ -119,25 +120,35 @@ def start_stream(self, name: str, config: dict): def remove_stream(self, name: str): self._request("DELETE", f"streams/{name}") - def wait_for_stream_to_finish(self, - name: str, - timeout_sec=60, - remove=True): + def stream_metrics(self, name: str): + metrics_resp = self._request("GET", "metrics") + assert metrics_resp.status_code == 200 + families = text_string_to_metric_families(metrics_resp.text) + metrics = dict() + for family in families: + for sample in family.samples: + if sample.labels.get('stream') == name: + family_metrics = metrics.get(family.name, []) + family_metrics.append(sample) + metrics[family.name] = family_metrics + return metrics + + def stop_stream(self, name: str, wait_to_finish=True, timeout_sec=60): """ - Waits for all streams to finish and then removes the stream + Optionally waits for the stream to finish and then removes the stream """ def _finished(): streams = self._request("GET", f"streams").json() return name not in streams or streams[name]["active"] == False - wait_until(_finished, - timeout_sec=timeout_sec, - backoff_sec=0.5, - err_msg=f"Timeout waiting for {name} stream to finish", - retry_on_exc=True) + if wait_to_finish: + wait_until(_finished, + timeout_sec=timeout_sec, + backoff_sec=0.5, + err_msg=f"Timeout waiting for {name} stream to finish", + retry_on_exc=True) - if remove: - self.remove_stream(name) + self.remove_stream(name) def _request(self, method, endpoint, **kwargs): self.logger.debug(f"Executing request {method} {self.url}/{endpoint}") diff --git a/tests/rptest/tests/datalake/datalake_verifier_test.py b/tests/rptest/tests/datalake/datalake_verifier_test.py index 1087aba19d6c..cef9273bb777 100644 --- a/tests/rptest/tests/datalake/datalake_verifier_test.py +++ b/tests/rptest/tests/datalake/datalake_verifier_test.py @@ -51,7 +51,7 @@ def _prepare_test_data(self, topic_name: str, dl: DatalakeServices): iceberg_mode="key_value") connect = dl.start_counter_stream(topic=topic_name) dl.wait_for_translation(topic_name, 100) - connect.wait_for_stream_to_finish("ducky_stream") + connect.stop_stream("ducky_stream") @cluster(num_nodes=4) @matrix(cloud_storage_type=supported_storage_types()) diff --git a/tests/rptest/tests/datalake/simple_connect_test.py b/tests/rptest/tests/datalake/simple_connect_test.py index 0c117fdcaccc..d9f867640e1e 100644 --- a/tests/rptest/tests/datalake/simple_connect_test.py +++ b/tests/rptest/tests/datalake/simple_connect_test.py @@ -135,5 +135,5 @@ def test_translating_avro_serialized_records(self, cloud_storage_type): topic_name, "verifier_schema")) verifier.start() - connect.wait_for_stream_to_finish("ducky_stream") + connect.stop_stream("ducky_stream") verifier.wait() diff --git a/tests/rptest/tests/redpanda_connect_test.py b/tests/rptest/tests/redpanda_connect_test.py index 4c9e27db86ab..809d5d79bb61 100644 --- a/tests/rptest/tests/redpanda_connect_test.py +++ b/tests/rptest/tests/redpanda_connect_test.py @@ -100,7 +100,7 @@ def test_redpanda_connect_producer(self): "test")) # wait for the stream to finish - connect.wait_for_stream_to_finish(name="ducky_stream") + connect.stop_stream(name="ducky_stream") connect.wait() # check if the messages are in the topic From 32973fb82a84731efe4b8d5cca371941ff650431 Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Mon, 2 Dec 2024 23:08:36 -0800 Subject: [PATCH 070/229] datalake/verifier/tests: reduce the frequency of a chatty log line --- tests/rptest/tests/datalake/datalake_verifier.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/rptest/tests/datalake/datalake_verifier.py b/tests/rptest/tests/datalake/datalake_verifier.py index 00f1c13fadf2..61b04c560587 100644 --- a/tests/rptest/tests/datalake/datalake_verifier.py +++ b/tests/rptest/tests/datalake/datalake_verifier.py @@ -85,9 +85,10 @@ def _consumer_thread(self): with self._lock: self._total_msgs_cnt += 1 - self.logger.debug( - f"Consumed message partition: {msg.partition()} at offset {msg.offset()}" - ) + if self._total_msgs_cnt % 100 == 0: + self.logger.debug( + f"Consumed message partition: {msg.partition()} at offset {msg.offset()}" + ) self._consumed_messages[msg.partition()].append(msg) if len(self._errors) > 0: return From 92e5aa5719a2aa3dd450cdc49f8982494cf4f725 Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Mon, 2 Dec 2024 23:17:18 -0800 Subject: [PATCH 071/229] datalake/tests: verification test with cross core movements --- .../tests/datalake/partition_movement_test.py | 109 ++++++++++++++++++ 1 file changed, 109 insertions(+) create mode 100644 tests/rptest/tests/datalake/partition_movement_test.py diff --git a/tests/rptest/tests/datalake/partition_movement_test.py b/tests/rptest/tests/datalake/partition_movement_test.py new file mode 100644 index 000000000000..f2e075ad67e5 --- /dev/null +++ b/tests/rptest/tests/datalake/partition_movement_test.py @@ -0,0 +1,109 @@ +# Copyright 2024 Redpanda Data, Inc. +# +# Use of this software is governed by the Business Source License +# included in the file licenses/BSL.md +# +# As of the Change Date specified in that file, in accordance with +# the Business Source License, use of this software will be governed +# by the Apache License, Version 2.0 + +from rptest.services.spark_service import QueryEngineType +from rptest.tests.datalake.datalake_services import DatalakeServices +from rptest.tests.datalake.datalake_verifier import DatalakeVerifier +from rptest.tests.partition_movement import PartitionMovementMixin +from rptest.tests.redpanda_test import RedpandaTest +from rptest.services.redpanda import PandaproxyConfig, SchemaRegistryConfig, SISettings +from ducktape.mark import matrix +from ducktape.utils.util import wait_until +from rptest.services.cluster import cluster +from rptest.tests.datalake.utils import supported_storage_types + + +class PartitionMovementTest(PartitionMovementMixin, RedpandaTest): + def __init__(self, test_ctx, *args, **kwargs): + super(PartitionMovementTest, + self).__init__(test_ctx, + num_brokers=1, + si_settings=SISettings(test_context=test_ctx), + extra_rp_conf={ + "iceberg_enabled": "true", + "iceberg_catalog_commit_interval_ms": 5000 + }, + schema_registry_config=SchemaRegistryConfig(), + pandaproxy_config=PandaproxyConfig(), + *args, + **kwargs) + self.test_ctx = test_ctx + self.topic_name = "test" + + def setUp(self): + pass + + @cluster(num_nodes=4) + @matrix(cloud_storage_type=supported_storage_types()) + def test_cross_core_movements(self, cloud_storage_type): + """Tests interaction between cross core partition movement and iceberg translation. + Cross core partition movement involves shutting down the partition replica machinery on one + core and restarting it on another core. This test ensures the translation can make progress + during these cross core movements.""" + + moves = 15 + admin = self.redpanda._admin + topic = self.topic_name + partition = 0 + stream = "cross_core_test" + + with DatalakeServices(self.test_context, + redpanda=self.redpanda, + filesystem_catalog_mode=False, + include_query_engines=[QueryEngineType.TRINO + ]) as dl: + dl.create_iceberg_enabled_topic(topic) + # A long running counter that runs until stopped + connect = dl.start_counter_stream(name=stream, + topic=topic, + count=0, + interval="1ms") + + def total_records_ingested(): + metrics = connect.stream_metrics(name=stream) + samples = metrics["output_sent"] + for s in samples: + if s.name == "output_sent_total": + return s.value + assert False, f"Unable to probe metrics for stream {stream}" + + def ensure_stream_progress(target: int): + wait_until( + lambda: total_records_ingested() >= target, + timeout_sec=20, + backoff_sec=5, + err_msg= + f"Timed out waiting for stream producer to reach target: {target}" + ) + + for _ in range(moves): + assignments = self._get_current_node_cores( + admin, topic, partition) + for a in assignments: + a['core'] = (a['core'] + + 1) % self.redpanda.get_node_cpu_count() + + counter_before = total_records_ingested() + + self._set_partition_assignments(topic, + partition, + assignments, + admin=admin) + self._wait_post_move(topic, partition, assignments, 180) + # Make sure the stream is not stuck + ensure_stream_progress(counter_before + 500) + + connect.stop_stream(name=stream, wait_to_finish=False) + + total_row_count = total_records_ingested() + dl.wait_for_translation_until_offset(topic, total_row_count - 1) + + verifier = DatalakeVerifier(self.redpanda, topic, dl.trino()) + verifier.start() + verifier.wait() From b6b9a60dc2bf7fae3958447a4b8c46ef37c72fa8 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Wed, 4 Dec 2024 18:58:00 +0000 Subject: [PATCH 072/229] cluster: reject writes only if data disk is degraded Health monitor tracks only the data disk now as the only use of that state is for rejecting writes. Cache disk state is irrelevant at cluster level. This was tested manually by creating a cluster with custom cache disk mountpoint and trying to produce to it. Before this commit, producing would have failed with a full cache disk. After this commit, producing fails only if the data disk is full. --- src/v/cluster/health_monitor_backend.cc | 14 +++++++------- src/v/cluster/health_monitor_backend.h | 4 ++-- src/v/cluster/health_monitor_frontend.cc | 20 +++++++++++--------- src/v/cluster/health_monitor_frontend.h | 4 ++-- src/v/cluster/metadata_cache.cc | 2 +- tests/rptest/tests/full_disk_test.py | 2 +- 6 files changed, 24 insertions(+), 22 deletions(-) diff --git a/src/v/cluster/health_monitor_backend.cc b/src/v/cluster/health_monitor_backend.cc index 0ad5f74b9681..88ef8b740085 100644 --- a/src/v/cluster/health_monitor_backend.cc +++ b/src/v/cluster/health_monitor_backend.cc @@ -295,7 +295,7 @@ health_monitor_backend::get_cluster_health( } ss::future -health_monitor_backend::get_cluster_disk_health( +health_monitor_backend::get_cluster_data_disk_health( force_refresh refresh, model::timeout_clock::time_point deadline) { auto ec = co_await maybe_refresh_cluster_health(refresh, deadline); if (ec) { @@ -308,7 +308,7 @@ health_monitor_backend::get_cluster_disk_health( // operate, I guess. co_return storage::disk_space_alert::ok; } - co_return _reports_disk_health; + co_return _reports_data_disk_health; } ss::future @@ -444,8 +444,8 @@ ss::future health_monitor_backend::collect_cluster_health() { auto old_reports = std::exchange(_reports, {}); - // update nodes reports and cache cluster-level disk health - storage::disk_space_alert cluster_disk_health + // update nodes reports and cache cluster-level data disk health + storage::disk_space_alert cluster_data_disk_health = storage::disk_space_alert::ok; for (auto& r : reports) { if (r) { @@ -471,14 +471,14 @@ ss::future health_monitor_backend::collect_cluster_health() { for (auto& cb : _node_callbacks) { cb.second(r.value(), old_report); } - cluster_disk_health = storage::max_severity( - r.value().local_state.get_disk_alert(), cluster_disk_health); + cluster_data_disk_health = storage::max_severity( + r.value().local_state.data_disk.alert, cluster_data_disk_health); _reports.emplace( id, ss::make_lw_shared(std::move(r.value()))); } } - _reports_disk_health = cluster_disk_health; + _reports_data_disk_health = cluster_data_disk_health; if (config::shard_local_cfg().enable_usage()) { vlog(clusterlog.info, "collecting cloud health statistics"); diff --git a/src/v/cluster/health_monitor_backend.h b/src/v/cluster/health_monitor_backend.h index 470178deb9f4..dc55fb82e0f9 100644 --- a/src/v/cluster/health_monitor_backend.h +++ b/src/v/cluster/health_monitor_backend.h @@ -65,7 +65,7 @@ class health_monitor_backend { ss::future> get_cluster_health( cluster_report_filter, force_refresh, model::timeout_clock::time_point); - ss::future get_cluster_disk_health( + ss::future get_cluster_data_disk_health( force_refresh refresh, model::timeout_clock::time_point deadline); ss::future> collect_current_node_health(); @@ -182,7 +182,7 @@ class health_monitor_backend { status_cache_t _status; report_cache_t _reports; - storage::disk_space_alert _reports_disk_health + storage::disk_space_alert _reports_data_disk_health = storage::disk_space_alert::ok; std::optional _bytes_in_cloud_storage; diff --git a/src/v/cluster/health_monitor_frontend.cc b/src/v/cluster/health_monitor_frontend.cc index 756109975c00..f5906bc5743c 100644 --- a/src/v/cluster/health_monitor_frontend.cc +++ b/src/v/cluster/health_monitor_frontend.cc @@ -60,8 +60,9 @@ health_monitor_frontend::get_cluster_health( }); } -storage::disk_space_alert health_monitor_frontend::get_cluster_disk_health() { - return _cluster_disk_health; +storage::disk_space_alert +health_monitor_frontend::get_cluster_data_disk_health() { + return _cluster_data_disk_health; } /** @@ -100,23 +101,24 @@ health_monitor_frontend::get_cluster_health_overview( ss::future<> health_monitor_frontend::update_other_shards( const storage::disk_space_alert dsa) { - co_await container().invoke_on_others( - [dsa](health_monitor_frontend& fe) { fe._cluster_disk_health = dsa; }); + co_await container().invoke_on_others([dsa](health_monitor_frontend& fe) { + fe._cluster_data_disk_health = dsa; + }); } ss::future<> health_monitor_frontend::update_frontend_and_backend_cache() { auto deadline = model::time_from_now(default_timeout); auto disk_health = co_await dispatch_to_backend( [deadline](health_monitor_backend& be) { - return be.get_cluster_disk_health(force_refresh::no, deadline); + return be.get_cluster_data_disk_health(force_refresh::no, deadline); }); - if (disk_health != _cluster_disk_health) { + if (disk_health != _cluster_data_disk_health) { vlog( clusterlog.debug, - "Update disk health cache {} -> {}", - _cluster_disk_health, + "Update data disk health cache {} -> {}", + _cluster_data_disk_health, disk_health); - _cluster_disk_health = disk_health; + _cluster_data_disk_health = disk_health; co_await update_other_shards(disk_health); } } diff --git a/src/v/cluster/health_monitor_frontend.h b/src/v/cluster/health_monitor_frontend.h index 7ea0df709bc7..018200b5f266 100644 --- a/src/v/cluster/health_monitor_frontend.h +++ b/src/v/cluster/health_monitor_frontend.h @@ -57,7 +57,7 @@ class health_monitor_frontend ss::future> get_cluster_health( cluster_report_filter, force_refresh, model::timeout_clock::time_point); - storage::disk_space_alert get_cluster_disk_health(); + storage::disk_space_alert get_cluster_data_disk_health(); // Collects or return cached version of current node health report. ss::future> get_current_node_health(); @@ -101,7 +101,7 @@ class health_monitor_frontend config::binding _alive_timeout; // Currently the worst / max of all nodes' disk space state - storage::disk_space_alert _cluster_disk_health{ + storage::disk_space_alert _cluster_data_disk_health{ storage::disk_space_alert::ok}; ss::timer _refresh_timer; ss::gate _refresh_gate; diff --git a/src/v/cluster/metadata_cache.cc b/src/v/cluster/metadata_cache.cc index 034583c438cc..c49ceb6d2742 100644 --- a/src/v/cluster/metadata_cache.cc +++ b/src/v/cluster/metadata_cache.cc @@ -149,7 +149,7 @@ std::vector metadata_cache::node_ids() const { } bool metadata_cache::should_reject_writes() const { - return _health_monitor.local().get_cluster_disk_health() + return _health_monitor.local().get_cluster_data_disk_health() == storage::disk_space_alert::degraded; } diff --git a/tests/rptest/tests/full_disk_test.py b/tests/rptest/tests/full_disk_test.py index 79397282a1bf..19954d56a21e 100644 --- a/tests/rptest/tests/full_disk_test.py +++ b/tests/rptest/tests/full_disk_test.py @@ -147,7 +147,7 @@ def check_health_monitor_frontend(disk_space_change: str): # Looking for a log statement about a change in disk space. # This is a check for the health monitor frontend because # that structure logs disk space alerts. - pattern = f"Update disk health cache {disk_space_change}" + pattern = f"Update data disk health cache {disk_space_change}" wait_until( lambda: self.redpanda.search_log_any(pattern), timeout_sec=5, From 259b4876574aa64f4d86cc8e0729f587bf48a8d7 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Mon, 2 Dec 2024 17:57:08 +0000 Subject: [PATCH 073/229] bazel/thirdparty: expose openssl executable --- bazel/thirdparty/openssl.BUILD | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/bazel/thirdparty/openssl.BUILD b/bazel/thirdparty/openssl.BUILD index dba38b38397f..80f8f6eebbe0 100644 --- a/bazel/thirdparty/openssl.BUILD +++ b/bazel/thirdparty/openssl.BUILD @@ -1,4 +1,6 @@ load("@bazel_skylib//rules:common_settings.bzl", "int_flag", "string_flag") +load("@bazel_skylib//rules:copy_file.bzl", "copy_file") +load("@bazel_skylib//rules:select_file.bzl", "select_file") load("@rules_foreign_cc//foreign_cc:defs.bzl", "configure_make") # Make this build faster by setting `build --@openssl//:build_jobs=16` in user.bazelrc @@ -55,6 +57,9 @@ configure_make( "OPENSSL_BUILD_JOBS": "$(BUILD_JOBS)", }, lib_source = ":srcs", + out_binaries = [ + "openssl", + ], out_shared_libs = [ "libssl.so.3", "libcrypto.so.3", @@ -64,3 +69,26 @@ configure_make( "//visibility:public", ], ) + +filegroup( + name = "gen_dir", + srcs = [":openssl"], + output_group = "gen_dir", +) + +select_file( + name = "openssl_exe_file", + srcs = ":openssl", + subpath = "bin/openssl", +) + +copy_file( + name = "openssl_exe", + src = ":openssl_exe_file", + out = "openssl.exe", + allow_symlink = True, + is_executable = True, + visibility = [ + "//visibility:public", + ], +) From 8f456fc54e46b7411d4824d055395ad46a804a4b Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Mon, 2 Dec 2024 19:35:51 +0000 Subject: [PATCH 074/229] rpc: add dependencies for redpanda_cc_rpc_library --- src/v/rpc/compiler.bzl | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/v/rpc/compiler.bzl b/src/v/rpc/compiler.bzl index c98dad3f9ef0..ce53be135b5f 100644 --- a/src/v/rpc/compiler.bzl +++ b/src/v/rpc/compiler.bzl @@ -4,7 +4,7 @@ This module contains functions for working with Redpanda RPC system. load("//bazel:build.bzl", "redpanda_cc_library") -def redpanda_cc_rpc_library(name, src, out = None, include_prefix = None, visibility = None): +def redpanda_cc_rpc_library(name, src, out = None, deps = [], include_prefix = None, visibility = None): """ Generate Redpanda RPC library. @@ -12,6 +12,7 @@ def redpanda_cc_rpc_library(name, src, out = None, include_prefix = None, visibi name: name of the library src: rpc specification json file out: output header name. defaults to src_service.h (without .json extension) + deps: dependencies defined in the json src file include_prefix: include_prefix of generated header visibility: visibility setting """ @@ -30,9 +31,18 @@ def redpanda_cc_rpc_library(name, src, out = None, include_prefix = None, visibi tools = ["//src/v/rpc:compiler"], ) + rpc_template_deps = [ + "//src/v/config", + "//src/v/metrics", + "//src/v/rpc", + "//src/v/finjector", + "//src/v/random:fast_prng", + ] + redpanda_cc_library( name = name, hdrs = [out], + deps = rpc_template_deps + deps, include_prefix = include_prefix, visibility = visibility, ) From fed15c409ce0efed2deb43014c30a29964afe8c5 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Mon, 2 Dec 2024 19:45:46 +0000 Subject: [PATCH 075/229] bazel: add utility functions to generate certificates --- bazel/cert.bzl | 144 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 144 insertions(+) create mode 100644 bazel/cert.bzl diff --git a/bazel/cert.bzl b/bazel/cert.bzl new file mode 100644 index 000000000000..367348c8031d --- /dev/null +++ b/bazel/cert.bzl @@ -0,0 +1,144 @@ +""" +This module contains functions to generate a simple CA +""" + +load("@bazel_skylib//rules:run_binary.bzl", "run_binary") + +# buildifier: disable=function-docstring-args +def _redpanda_private_key(name, certificate): + private_key = certificate + ".key" + + run_binary( + name = name + "_key_gen", + srcs = [], + outs = [private_key], + args = [ + "ecparam", + "-name", + "prime256v1", + "-genkey", + "-noout", + "-out", + "$(execpath :{})".format(private_key), + ], + tool = "@openssl//:openssl_exe", + ) + + return private_key + +def redpanda_selfsigned_cert(name, certificate, common_name, visibility = None): + """ + Generate a Redpanda self-signed certificate. + + Args: + name: name of the target + certificate: name to use for output files (crt, key, and csr) + common_name: the CN to use when setting the subject name + visibility: visibility setting + """ + + cert = certificate + ".crt" + subj = "/C=US/ST=California/L=San Francisco/O=Redpanda Data/OU=Core/CN=" + common_name + + private_key = _redpanda_private_key(name, certificate) + + run_binary( + name = name + "_crt_gen", + srcs = [private_key], + outs = [cert], + args = [ + "req", + "-new", + "-x509", + "-sha256", + "-key", + "$(execpath :{})".format(private_key), + "-out", + "$(execpath :{})".format(cert), + "-subj", + subj, + "-addext", + "subjectAltName = IP:127.0.0.1", + ], + tool = "@openssl//:openssl_exe", + ) + + native.filegroup( + name = name, + srcs = [private_key, cert], + visibility = visibility, + ) + +def redpanda_signed_cert(name, certificate, common_name, ca, serial_number, visibility = None): + """ + Generate a Redpanda signed certificate. + + Args: + name: name of the target + certificate: name to use for output files (crt, key, and csr) + common_name: the CN to use when setting the subject name + ca: the certificate to be used as the signing CA + serial_number: the serial number of cert when issued by CA + visibility: visibility setting + """ + + subj = "/C=US/ST=California/L=San Francisco/O=Redpanda Data/OU=Core/CN=" + common_name + + private_key = _redpanda_private_key(name, certificate) + csr = certificate + ".csr" + cert = certificate + ".crt" + + run_binary( + name = name + "_csr_gen", + srcs = [private_key], + outs = [csr], + args = [ + "req", + "-new", + "-sha256", + "-key", + "$(execpath :{})".format(private_key), + "-out", + "$(execpath :{})".format(csr), + "-subj", + subj, + ], + tool = "@openssl//:openssl_exe", + ) + + ca_cert = ca + ".crt" + ca_private_key = ca + ".key" + + run_binary( + name = name + "_crt_gen", + srcs = [ + ca_cert, + ca_private_key, + csr, + ], + outs = [cert], + args = [ + "x509", + "-req", + "-days", + "1000", + "-sha256", + "-set_serial", + "{}".format(serial_number), + "-in", + "$(execpath :{})".format(csr), + "-CA", + "$(execpaths :{})".format(ca_cert), + "-CAkey", + "$(execpaths :{})".format(ca_private_key), + "-out", + "$(execpath :{})".format(cert), + ], + tool = "@openssl//:openssl_exe", + ) + + native.filegroup( + name = name, + srcs = [private_key, csr, cert], + visibility = visibility, + ) From 5073e705befac8d0081f46269437af91f66605de Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Mon, 2 Dec 2024 19:46:01 +0000 Subject: [PATCH 076/229] rpc: bazelize rpc_gen_cycling_test --- src/v/rpc/test/BUILD | 110 ++++++++++++++++++++++++- src/v/rpc/test/cycling_service.json | 2 +- src/v/rpc/test/echo_service.json | 2 +- src/v/rpc/test/echo_v2_service.json | 2 +- src/v/rpc/test/rpc_gen_cycling_test.cc | 69 ++++++++++++---- 5 files changed, 164 insertions(+), 21 deletions(-) diff --git a/src/v/rpc/test/BUILD b/src/v/rpc/test/BUILD index 480dbf94a229..6c2105f3468c 100644 --- a/src/v/rpc/test/BUILD +++ b/src/v/rpc/test/BUILD @@ -1,6 +1,23 @@ -load("//bazel:test.bzl", "redpanda_cc_bench", "redpanda_cc_btest") +load("//bazel:build.bzl", "redpanda_cc_library") +load("//bazel:cert.bzl", "redpanda_selfsigned_cert", "redpanda_signed_cert") +load("//bazel:test.bzl", "redpanda_cc_bench", "redpanda_cc_btest", "redpanda_test_cc_library") load("//src/v/rpc:compiler.bzl", "redpanda_cc_rpc_library") +redpanda_test_cc_library( + name = "rpc_integration_fixture", + hdrs = [ + "rpc_integration_fixture.h", + ], + include_prefix = "rpc/test", + deps = [ + "//src/v/base", + "//src/v/config", + "//src/v/net", + "//src/v/rpc", + "@seastar", + ], +) + redpanda_cc_btest( name = "netbuf_test", timeout = "short", @@ -101,6 +118,37 @@ redpanda_cc_btest( ], ) +redpanda_cc_btest( + name = "rpc_gen_cycling_test", + timeout = "short", + srcs = [ + "rpc_gen_cycling_test.cc", + ], + cpu = 1, + data = [ + ":cert", + ":cert_ca", + ":other_cert", + ":other_cert_ca", + ], + tags = ["exclusive"], + deps = [ + ":cycling_rpc", + ":echo_rpc", + ":echo_v2_rpc", + ":rpc_integration_fixture", + "//src/v/bytes:random", + "//src/v/model", + "//src/v/random:generators", + "//src/v/rpc", + "//src/v/test_utils:fixture", + "//src/v/test_utils:seastar_boost", + "@boost//:test", + "@seastar", + "@seastar//:testing", + ], +) + redpanda_cc_bench( name = "rpc_bench", timeout = "short", @@ -114,17 +162,77 @@ redpanda_cc_bench( ], ) +redpanda_cc_library( + name = "rpc_gen_types", + hdrs = [ + "rpc_gen_types.h", + ], + include_prefix = "rpc/test", + deps = [ + "//src/v/base", + "//src/v/reflection:adl", + "//src/v/rpc", + "//src/v/serde", + "//src/v/serde:enum", + "//src/v/serde:sstring", + "@seastar", + ], +) + redpanda_cc_rpc_library( name = "cycling_rpc", src = "cycling_service.json", + out = "cycling_service.h", + include_prefix = "rpc/test", + deps = [ + ":rpc_gen_types", + ], ) redpanda_cc_rpc_library( name = "echo_rpc", src = "echo_service.json", + out = "echo_service.h", + include_prefix = "rpc/test", + deps = [ + ":rpc_gen_types", + ], ) redpanda_cc_rpc_library( name = "echo_v2_rpc", src = "echo_v2_service.json", + out = "echo_v2_service.h", + include_prefix = "rpc/test", + deps = [ + ":rpc_gen_types", + ], +) + +redpanda_selfsigned_cert( + name = "cert_ca", + certificate = "root_certificate_authority", + common_name = "redpanda.com", +) + +redpanda_selfsigned_cert( + name = "other_cert_ca", + certificate = "root_certificate_authority.other", + common_name = "redpanda.other.com", +) + +redpanda_signed_cert( + name = "cert", + ca = "root_certificate_authority", + certificate = "redpanda", + common_name = "cert.com", + serial_number = 1, +) + +redpanda_signed_cert( + name = "other_cert", + ca = "root_certificate_authority.other", + certificate = "redpanda.other", + common_name = "cert.other.com", + serial_number = 2, ) diff --git a/src/v/rpc/test/cycling_service.json b/src/v/rpc/test/cycling_service.json index 3921be13bdbf..ce47533c05c9 100644 --- a/src/v/rpc/test/cycling_service.json +++ b/src/v/rpc/test/cycling_service.json @@ -2,7 +2,7 @@ "namespace": "cycling", "service_name": "team_movistar", "includes": [ - "rpc_gen_types.h" + "rpc/test/rpc_gen_types.h" ], "methods": [ { diff --git a/src/v/rpc/test/echo_service.json b/src/v/rpc/test/echo_service.json index cde1d3e5e817..b73f8eb669f1 100644 --- a/src/v/rpc/test/echo_service.json +++ b/src/v/rpc/test/echo_service.json @@ -2,7 +2,7 @@ "namespace": "echo", "service_name": "echo", "includes": [ - "rpc_gen_types.h" + "rpc/test/rpc_gen_types.h" ], "methods": [ { diff --git a/src/v/rpc/test/echo_v2_service.json b/src/v/rpc/test/echo_v2_service.json index 288d9829c4a7..2d77c67af0d9 100644 --- a/src/v/rpc/test/echo_v2_service.json +++ b/src/v/rpc/test/echo_v2_service.json @@ -2,7 +2,7 @@ "namespace": "echo_v2", "service_name": "echo", "includes": [ - "rpc_gen_types.h" + "rpc/test/rpc_gen_types.h" ], "methods": [ { diff --git a/src/v/rpc/test/rpc_gen_cycling_test.cc b/src/v/rpc/test/rpc_gen_cycling_test.cc index 6b5088a7c07c..f1b87a445e92 100644 --- a/src/v/rpc/test/rpc_gen_cycling_test.cc +++ b/src/v/rpc/test/rpc_gen_cycling_test.cc @@ -18,10 +18,8 @@ #include "rpc/test/cycling_service.h" #include "rpc/test/echo_service.h" #include "rpc/test/echo_v2_service.h" +#include "rpc/test/rpc_integration_fixture.h" #include "rpc/types.h" -#include "rpc_gen_types.h" -#include "rpc_integration_fixture.h" -#include "serde/rw/iobuf.h" #include "test_utils/async.h" #include "test_utils/fixture.h" @@ -135,6 +133,41 @@ class rpc_integration_fixture : public rpc_simple_integration_fixture { static constexpr uint16_t redpanda_rpc_port = 32147; }; +struct certificate { + ss::sstring key; + ss::sstring crt; + ss::sstring ca; +}; + +namespace { +ss::sstring root_path() { + // if this file exists, we are running in cmake + if (std::filesystem::exists("redpanda.key")) { + return ""; + } + + // otherwise we are running in bazel and we need + // the full path + return "src/v/rpc/test/"; +} + +certificate redpanda_cert() { + const auto root = root_path(); + return { + .key = root + "redpanda.key", + .crt = root + "redpanda.crt", + .ca = root + "root_certificate_authority.crt"}; +} + +certificate redpanda_other_cert() { + const auto root = root_path(); + return { + .key = root + "redpanda.other.key", + .crt = root + "redpanda.other.crt", + .ca = root + "root_certificate_authority.other.crt"}; +} +} // namespace + FIXTURE_TEST(echo_round_trip, rpc_integration_fixture) { configure_server(); register_services(); @@ -237,7 +270,7 @@ FIXTURE_TEST(echo_from_cache, rpc_integration_fixture) { // Check that we can create connections from a cache, and moreover that we // can run several clients targeted at the same server, if we provide // multiple node IDs to the cache. - constexpr const size_t num_nodes_ids = 10; + constexpr const int num_nodes_ids = 10; const auto ccfg = client_config(); for (int i = 0; i < num_nodes_ids; ++i) { const auto payload = random_generators::gen_alphanum_string(100); @@ -301,10 +334,11 @@ FIXTURE_TEST(rpc_abort_from_cache, rpc_integration_fixture) { } FIXTURE_TEST(echo_round_trip_tls, rpc_integration_fixture) { + const auto cert = redpanda_cert(); auto creds_builder = config::tls_config( true, - config::key_cert{"redpanda.key", "redpanda.crt"}, - "root_certificate_authority.crt", + config::key_cert{cert.key, cert.crt}, + cert.ca, std::nullopt, /* CRL */ false) .get_credentials_builder() @@ -367,16 +401,18 @@ struct certificate_reload_ctx { }; FIXTURE_TEST(rpcgen_reload_credentials_integration, rpc_integration_fixture) { + const certificate cert = redpanda_cert(); + const certificate other_cert = redpanda_other_cert(); + // Server starts with bad credentials, files are updated on disk and then // client connects. Expected behavior is that client can connect without // issues. Condition variable is used to wait for credentials to reload. auto context = ss::make_lw_shared(); temporary_dir tmp; // client credentials - auto client_key = tmp.copy_file("redpanda.key", "client.key"); - auto client_crt = tmp.copy_file("redpanda.crt", "client.crt"); - auto client_ca = tmp.copy_file( - "root_certificate_authority.crt", "ca_client.pem"); + auto client_key = tmp.copy_file(cert.key.c_str(), "client.key"); + auto client_crt = tmp.copy_file(cert.crt.c_str(), "client.crt"); + auto client_ca = tmp.copy_file(cert.ca.c_str(), "ca_client.pem"); auto client_creds_builder = config::tls_config( true, config::key_cert{ @@ -387,10 +423,9 @@ FIXTURE_TEST(rpcgen_reload_credentials_integration, rpc_integration_fixture) { .get_credentials_builder() .get(); // server credentials - auto server_key = tmp.copy_file("redpanda.other.key", "server.key"); - auto server_crt = tmp.copy_file("redpanda.other.crt", "server.crt"); - auto server_ca = tmp.copy_file( - "root_certificate_authority.other.crt", "ca_server.pem"); + auto server_key = tmp.copy_file(other_cert.key.c_str(), "server.key"); + auto server_crt = tmp.copy_file(other_cert.crt.c_str(), "server.crt"); + auto server_ca = tmp.copy_file(other_cert.ca.c_str(), "ca_server.pem"); auto server_creds_builder = config::tls_config( true, config::key_cert{ @@ -432,9 +467,9 @@ FIXTURE_TEST(rpcgen_reload_credentials_integration, rpc_integration_fixture) { // fix client credentials and reconnect info("replacing files"); - tmp.copy_file("redpanda.key", "server.key"); - tmp.copy_file("redpanda.crt", "server.crt"); - tmp.copy_file("root_certificate_authority.crt", "ca_server.pem"); + tmp.copy_file(cert.key.c_str(), "server.key"); + tmp.copy_file(cert.crt.c_str(), "server.crt"); + tmp.copy_file(cert.ca.c_str(), "ca_server.pem"); context->cvar.wait([context] { return context->updated.size() == 3; }) .get(); From 4713734af3d225e1e5d577d6c454e9028d35d7af Mon Sep 17 00:00:00 2001 From: Andrew Hsu Date: Fri, 6 Dec 2024 16:07:30 -0600 Subject: [PATCH 077/229] gha: use setup-python v5 --- .github/workflows/render-pr-body-release-notes.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/render-pr-body-release-notes.yml b/.github/workflows/render-pr-body-release-notes.yml index f2a2e3e8a0ec..b71db1b7fecf 100644 --- a/.github/workflows/render-pr-body-release-notes.yml +++ b/.github/workflows/render-pr-body-release-notes.yml @@ -22,7 +22,7 @@ jobs: curl -s -S -f -L -o rpchangelog/requirements.txt https://vectorized-public.s3.us-west-2.amazonaws.com/rpchangelog/requirements.txt curl -s -S -f -L -o rpchangelog/rpchangelog.py https://vectorized-public.s3.us-west-2.amazonaws.com/rpchangelog/rpchangelog.py chmod +x rpchangelog/rpchangelog.py - - uses: actions/setup-python@v4 + - uses: actions/setup-python@v5 with: python-version: '3.10' cache: 'pip' From 7da88fa63bf879cc642f77b67659a8ec0050d899 Mon Sep 17 00:00:00 2001 From: Noah Watkins Date: Fri, 6 Dec 2024 11:09:31 -0800 Subject: [PATCH 078/229] bazel: patch protobuf warning Signed-off-by: Noah Watkins --- MODULE.bazel | 9 +++++++++ bazel/thirdparty/protobuf_capture_warning.patch | 13 +++++++++++++ 2 files changed, 22 insertions(+) create mode 100644 bazel/thirdparty/protobuf_capture_warning.patch diff --git a/MODULE.bazel b/MODULE.bazel index ba7f987408bc..fbe8f6790b9b 100644 --- a/MODULE.bazel +++ b/MODULE.bazel @@ -42,6 +42,15 @@ bazel_dep(name = "liburing", version = "2.5") bazel_dep(name = "lz4", version = "1.9.4") bazel_dep(name = "platforms", version = "0.0.10") bazel_dep(name = "protobuf", version = "29.0") +single_version_override( + module_name = "protobuf", + patch_strip = 1, + # this patch is in upstream protobuf, so should be in an upcoming release. + patches = [ + "//bazel/thirdparty:protobuf_capture_warning.patch", + ], +) + bazel_dep(name = "re2", version = "2024-07-02") bazel_dep(name = "rules_foreign_cc", version = "0.12.0") bazel_dep(name = "rules_go", version = "0.50.1") diff --git a/bazel/thirdparty/protobuf_capture_warning.patch b/bazel/thirdparty/protobuf_capture_warning.patch new file mode 100644 index 000000000000..b9662efa8073 --- /dev/null +++ b/bazel/thirdparty/protobuf_capture_warning.patch @@ -0,0 +1,13 @@ +diff --git a/src/google/protobuf/map.cc b/src/google/protobuf/map.cc +index 570b61bec86f..da6ceb99d993 100644 +--- a/src/google/protobuf/map.cc ++++ b/src/google/protobuf/map.cc +@@ -116,7 +116,7 @@ void UntypedMapBase::ClearTable(const ClearInput input) { + ABSL_DCHECK_NE(num_buckets_, kGlobalEmptyTableSize); + + if (alloc_.arena() == nullptr) { +- const auto loop = [=](auto destroy_node) { ++ const auto loop = [&, this](auto destroy_node) { + const TableEntryPtr* table = table_; + for (map_index_t b = index_of_first_non_null_, end = num_buckets_; + b < end; ++b) { From 7880674620bce6a95f7d42d58b3d5746c88dd3fe Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Thu, 5 Dec 2024 16:00:47 +0000 Subject: [PATCH 079/229] bazel: bump version --- .bazelversion | 2 +- MODULE.bazel.lock | 2451 ++++++++++++++------------------------------- 2 files changed, 772 insertions(+), 1681 deletions(-) diff --git a/.bazelversion b/.bazelversion index ba7f754d0c33..815da58b7a9e 100644 --- a/.bazelversion +++ b/.bazelversion @@ -1 +1 @@ -7.4.0 +7.4.1 diff --git a/MODULE.bazel.lock b/MODULE.bazel.lock index b976a1062298..e8cbe0767700 100644 --- a/MODULE.bazel.lock +++ b/MODULE.bazel.lock @@ -802,37 +802,12 @@ }, "@@aspect_rules_js~//npm:extensions.bzl%pnpm": { "general": { - "bzlTransitiveDigest": "YizhiqMlh5caC++5y82GH8DLaoGFbw6GwoQK/xU/KvA=", - "usagesDigest": "v3KREGSYZf2uOgAr3qIjjiGR9FyKSmZ1NevfULvrOJw=", + "bzlTransitiveDigest": "roscSBlY/YuE46w1gEYQIkORMqkGdIyJVkxDfq/ZAtw=", + "usagesDigest": "1hU324o/rWis1wprOwPM+3YiIXklZvJ5jfmEbzKAClo=", "recordedFileInputs": {}, "recordedDirentsInputs": {}, "envVariables": {}, "generatedRepoSpecs": { - "pnpm__links": { - "bzlFile": "@@aspect_rules_js~//npm/private:npm_import.bzl", - "ruleClassName": "npm_import_links", - "attributes": { - "package": "pnpm", - "version": "8.6.7", - "dev": false, - "root_package": "", - "link_packages": {}, - "deps": {}, - "transitive_closure": {}, - "lifecycle_build_target": false, - "lifecycle_hooks_env": [], - "lifecycle_hooks_execution_requirements": [ - "no-sandbox" - ], - "lifecycle_hooks_use_default_shell_env": false, - "bins": {}, - "npm_translate_lock_repo": "", - "package_visibility": [ - "//visibility:public" - ], - "replace_package": "" - } - }, "pnpm": { "bzlFile": "@@aspect_rules_js~//npm/private:npm_import.bzl", "ruleClassName": "npm_import_rule", @@ -858,6 +833,31 @@ "extra_build_content": "load(\"@aspect_rules_js//js:defs.bzl\", \"js_binary\")\njs_binary(name = \"pnpm\", data = glob([\"package/**\"]), entry_point = \"package/dist/pnpm.cjs\", visibility = [\"//visibility:public\"])", "generate_bzl_library_targets": false } + }, + "pnpm__links": { + "bzlFile": "@@aspect_rules_js~//npm/private:npm_import.bzl", + "ruleClassName": "npm_import_links", + "attributes": { + "package": "pnpm", + "version": "8.6.7", + "dev": false, + "root_package": "", + "link_packages": {}, + "deps": {}, + "transitive_closure": {}, + "lifecycle_build_target": false, + "lifecycle_hooks_env": [], + "lifecycle_hooks_execution_requirements": [ + "no-sandbox" + ], + "lifecycle_hooks_use_default_shell_env": false, + "bins": {}, + "npm_translate_lock_repo": "", + "package_visibility": [ + "//visibility:public" + ], + "replace_package": "" + } } }, "recordedRepoMappingEntries": [ @@ -1051,8 +1051,8 @@ }, "@@hedron_compile_commands~//:workspace_setup.bzl%hedron_compile_commands_extension": { "general": { - "bzlTransitiveDigest": "dt88TFdQv/FH3VMUEhxvtc8bsKnxEN2hrpeC8gEyHw4=", - "usagesDigest": "TTuQdozEYhaN7+oiVYMFloEWxoZfdKcT7Sfz3s7VepA=", + "bzlTransitiveDigest": "3h/6zUQnbEGP9byyn34wt0pCl5EMkq5XfvLx4aN++z8=", + "usagesDigest": "ZDVGDCmInjpiNhgRA6J4I3Wttz9QywT/gtb9prOy598=", "recordedFileInputs": {}, "recordedDirentsInputs": {}, "envVariables": {}, @@ -1069,7 +1069,7 @@ "@@hedron_compile_commands~//:workspace_setup_transitive.bzl%hedron_compile_commands_extension": { "general": { "bzlTransitiveDigest": "IfDf0vEa2jjQ11RNpUM0u4xftPXIs+pyM8IMVkRqVMk=", - "usagesDigest": "SbxmRqzxHy+fuSIIxYlFqlmPJ3V0Wk+4XjQQhrhTIgo=", + "usagesDigest": "QQepffNx1f2xTIC7Qgebko1RCgkU9T5OwNhDa1sNpVw=", "recordedFileInputs": {}, "recordedDirentsInputs": {}, "envVariables": {}, @@ -1080,7 +1080,7 @@ "@@hedron_compile_commands~//:workspace_setup_transitive_transitive.bzl%hedron_compile_commands_extension": { "general": { "bzlTransitiveDigest": "1p58k3o2Jgjt/pBE7cb8WmmkplrSguIKma/h32x7X10=", - "usagesDigest": "JJJti4hzFY57HoGTLSrlLzOfmDqmd4Tk20aRYmWkkoU=", + "usagesDigest": "oSAtVdFq01jLEStKb6PfuaxobaJnGcKsuG82E+6VVS8=", "recordedFileInputs": {}, "recordedDirentsInputs": {}, "envVariables": {}, @@ -1091,7 +1091,7 @@ "@@hedron_compile_commands~//:workspace_setup_transitive_transitive_transitive.bzl%hedron_compile_commands_extension": { "general": { "bzlTransitiveDigest": "arNWX4EleUjJxqkM5nCRTj+ce05Zz1gSdGH1DCKOoLs=", - "usagesDigest": "BsZ4Efn1w1bRgZQeNwryYQHDfBfnD1GzDQtpP27craI=", + "usagesDigest": "6CbeqZpNc+5lPZBP7Qcm+MQVDTBMHHMCfiCI9qaKPYk=", "recordedFileInputs": {}, "recordedDirentsInputs": {}, "envVariables": {}, @@ -1118,8 +1118,8 @@ }, "@@pybind11_bazel~//:internal_configure.bzl%internal_configure_extension": { "general": { - "bzlTransitiveDigest": "+F47SE20NlARCHVGbd4r7kkjg4OA0eCJcOd5fqKq4fQ=", - "usagesDigest": "iH2lKTfsNEpn2MqtGpBNwJrxbb2C7DiYmh/XuKgDtr8=", + "bzlTransitiveDigest": "CyAKLVVonohnkTSqg9II/HA7M49sOlnMkgMHL3CmDuc=", + "usagesDigest": "mFrTHX5eCiNU/OIIGVHH3cOILY9Zmjqk8RQYv8o6Thk=", "recordedFileInputs": { "@@pybind11_bazel~//MODULE.bazel": "e6f4c20442eaa7c90d7190d8dc539d0ab422f95c65a57cc59562170c58ae3d34" }, @@ -1612,1173 +1612,249 @@ ] } }, - "@@rules_jvm_external~//:extensions.bzl%maven": { + "@@rules_fuzzing~//fuzzing/private:extensions.bzl%non_module_dependencies": { "general": { - "bzlTransitiveDigest": "4ijz6uc3T4E+d+U8LQv4EAt+8OqZNVY/lzvhLx3y1yg=", - "usagesDigest": "OjLvK9v56sSYg9fWBGDp03uaz8IwSP9Vg23Iv73BRdY=", - "recordedFileInputs": { - "@@rules_jvm_external~//rules_jvm_external_deps_install.json": "3ab1f67b0de4815df110bc72ccd6c77882b3b21d3d1e0a84445847b6ce3235a3" - }, + "bzlTransitiveDigest": "hVgJRQ3Er45/UUAgNn1Yp2Khcp/Y8WyafA2kXIYmQ5M=", + "usagesDigest": "YnIrdgwnf3iCLfChsltBdZ7yOJh706lpa2vww/i2pDI=", + "recordedFileInputs": {}, "recordedDirentsInputs": {}, "envVariables": {}, "generatedRepoSpecs": { - "org_slf4j_slf4j_api_1_7_30": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "cdba07964d1bb40a0761485c6b1e8c2f8fd9eb1d19c53928ac0d7f9510105c57", - "urls": [ - "https://repo1.maven.org/maven2/org/slf4j/slf4j-api/1.7.30/slf4j-api-1.7.30.jar", - "https://maven.google.com/org/slf4j/slf4j-api/1.7.30/slf4j-api-1.7.30.jar" - ], - "downloaded_file_path": "org/slf4j/slf4j-api/1.7.30/slf4j-api-1.7.30.jar" - } - }, - "com_google_api_grpc_proto_google_common_protos_2_0_1": { + "platforms": { "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", + "ruleClassName": "http_archive", "attributes": { - "sha256": "5ce71656118618731e34a5d4c61aa3a031be23446dc7de8b5a5e77b66ebcd6ef", "urls": [ - "https://repo1.maven.org/maven2/com/google/api/grpc/proto-google-common-protos/2.0.1/proto-google-common-protos-2.0.1.jar", - "https://maven.google.com/com/google/api/grpc/proto-google-common-protos/2.0.1/proto-google-common-protos-2.0.1.jar" + "https://mirror.bazel.build/github.com/bazelbuild/platforms/releases/download/0.0.8/platforms-0.0.8.tar.gz", + "https://github.com/bazelbuild/platforms/releases/download/0.0.8/platforms-0.0.8.tar.gz" ], - "downloaded_file_path": "com/google/api/grpc/proto-google-common-protos/2.0.1/proto-google-common-protos-2.0.1.jar" + "sha256": "8150406605389ececb6da07cbcb509d5637a3ab9a24bc69b1101531367d89d74" } }, - "com_google_api_gax_1_60_0": { + "rules_python": { "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", + "ruleClassName": "http_archive", "attributes": { - "sha256": "02f37d4ff1a7b8d71dff8064cf9568aa4f4b61bcc4485085d16130f32afa5a79", - "urls": [ - "https://repo1.maven.org/maven2/com/google/api/gax/1.60.0/gax-1.60.0.jar", - "https://maven.google.com/com/google/api/gax/1.60.0/gax-1.60.0.jar" - ], - "downloaded_file_path": "com/google/api/gax/1.60.0/gax-1.60.0.jar" + "sha256": "d70cd72a7a4880f0000a6346253414825c19cdd40a28289bdf67b8e6480edff8", + "strip_prefix": "rules_python-0.28.0", + "url": "https://github.com/bazelbuild/rules_python/releases/download/0.28.0/rules_python-0.28.0.tar.gz" } }, - "com_google_guava_failureaccess_1_0_1": { + "bazel_skylib": { "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", + "ruleClassName": "http_archive", "attributes": { - "sha256": "a171ee4c734dd2da837e4b16be9df4661afab72a41adaf31eb84dfdaf936ca26", + "sha256": "cd55a062e763b9349921f0f5db8c3933288dc8ba4f76dd9416aac68acee3cb94", "urls": [ - "https://repo1.maven.org/maven2/com/google/guava/failureaccess/1.0.1/failureaccess-1.0.1.jar", - "https://maven.google.com/com/google/guava/failureaccess/1.0.1/failureaccess-1.0.1.jar" - ], - "downloaded_file_path": "com/google/guava/failureaccess/1.0.1/failureaccess-1.0.1.jar" + "https://mirror.bazel.build/github.com/bazelbuild/bazel-skylib/releases/download/1.5.0/bazel-skylib-1.5.0.tar.gz", + "https://github.com/bazelbuild/bazel-skylib/releases/download/1.5.0/bazel-skylib-1.5.0.tar.gz" + ] } }, - "commons_logging_commons_logging_1_2": { + "com_google_absl": { "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", + "ruleClassName": "http_archive", "attributes": { - "sha256": "daddea1ea0be0f56978ab3006b8ac92834afeefbd9b7e4e6316fca57df0fa636", "urls": [ - "https://repo1.maven.org/maven2/commons-logging/commons-logging/1.2/commons-logging-1.2.jar", - "https://maven.google.com/commons-logging/commons-logging/1.2/commons-logging-1.2.jar" + "https://github.com/abseil/abseil-cpp/archive/refs/tags/20240116.1.zip" ], - "downloaded_file_path": "commons-logging/commons-logging/1.2/commons-logging-1.2.jar" + "strip_prefix": "abseil-cpp-20240116.1", + "integrity": "sha256-7capMWOvWyoYbUaHF/b+I2U6XLMaHmky8KugWvfXYuk=" } }, - "com_google_http_client_google_http_client_appengine_1_38_0": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "f97b495fd97ac3a3d59099eb2b55025f4948230da15a076f189b9cff37c6b4d2", - "urls": [ - "https://repo1.maven.org/maven2/com/google/http-client/google-http-client-appengine/1.38.0/google-http-client-appengine-1.38.0.jar", - "https://maven.google.com/com/google/http-client/google-http-client-appengine/1.38.0/google-http-client-appengine-1.38.0.jar" - ], - "downloaded_file_path": "com/google/http-client/google-http-client-appengine/1.38.0/google-http-client-appengine-1.38.0.jar" - } + "rules_fuzzing_oss_fuzz": { + "bzlFile": "@@rules_fuzzing~//fuzzing/private/oss_fuzz:repository.bzl", + "ruleClassName": "oss_fuzz_repository", + "attributes": {} }, - "com_google_cloud_google_cloud_storage_1_113_4": { + "honggfuzz": { "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", + "ruleClassName": "http_archive", "attributes": { - "sha256": "796833e9bdab80c40bbc820e65087eb8f28c6bfbca194d2e3e00d98cb5bc55d6", - "urls": [ - "https://repo1.maven.org/maven2/com/google/cloud/google-cloud-storage/1.113.4/google-cloud-storage-1.113.4.jar", - "https://maven.google.com/com/google/cloud/google-cloud-storage/1.113.4/google-cloud-storage-1.113.4.jar" - ], - "downloaded_file_path": "com/google/cloud/google-cloud-storage/1.113.4/google-cloud-storage-1.113.4.jar" + "build_file": "@@rules_fuzzing~//:honggfuzz.BUILD", + "sha256": "6b18ba13bc1f36b7b950c72d80f19ea67fbadc0ac0bb297ec89ad91f2eaa423e", + "url": "https://github.com/google/honggfuzz/archive/2.5.zip", + "strip_prefix": "honggfuzz-2.5" } }, - "io_grpc_grpc_context_1_33_1": { + "rules_fuzzing_jazzer": { "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", + "ruleClassName": "http_jar", "attributes": { - "sha256": "99b8aea2b614fe0e61c3676e681259dc43c2de7f64620998e1a8435eb2976496", - "urls": [ - "https://repo1.maven.org/maven2/io/grpc/grpc-context/1.33.1/grpc-context-1.33.1.jar", - "https://maven.google.com/io/grpc/grpc-context/1.33.1/grpc-context-1.33.1.jar" - ], - "downloaded_file_path": "io/grpc/grpc-context/1.33.1/grpc-context-1.33.1.jar" + "sha256": "ee6feb569d88962d59cb59e8a31eb9d007c82683f3ebc64955fd5b96f277eec2", + "url": "https://repo1.maven.org/maven2/com/code-intelligence/jazzer/0.20.1/jazzer-0.20.1.jar" } }, - "com_google_api_grpc_proto_google_iam_v1_1_0_3": { + "rules_fuzzing_jazzer_api": { "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", + "ruleClassName": "http_jar", "attributes": { - "sha256": "64cee7383a97e846da8d8e160e6c8fe30561e507260552c59e6ccfc81301fdc8", - "urls": [ - "https://repo1.maven.org/maven2/com/google/api/grpc/proto-google-iam-v1/1.0.3/proto-google-iam-v1-1.0.3.jar", - "https://maven.google.com/com/google/api/grpc/proto-google-iam-v1/1.0.3/proto-google-iam-v1-1.0.3.jar" - ], - "downloaded_file_path": "com/google/api/grpc/proto-google-iam-v1/1.0.3/proto-google-iam-v1-1.0.3.jar" + "sha256": "f5a60242bc408f7fa20fccf10d6c5c5ea1fcb3c6f44642fec5af88373ae7aa1b", + "url": "https://repo1.maven.org/maven2/com/code-intelligence/jazzer-api/0.20.1/jazzer-api-0.20.1.jar" } - }, - "com_google_api_api_common_1_10_1": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", + } + }, + "recordedRepoMappingEntries": [ + [ + "rules_fuzzing~", + "bazel_tools", + "bazel_tools" + ] + ] + } + }, + "@@rules_kotlin~//src/main/starlark/core/repositories:bzlmod_setup.bzl%rules_kotlin_extensions": { + "general": { + "bzlTransitiveDigest": "fus14IFJ/1LGWWGKPH/U18VnJCoMjfDt1ckahqCnM0A=", + "usagesDigest": "aJF6fLy82rR95Ff5CZPAqxNoFgOMLMN5ImfBS0nhnkg=", + "recordedFileInputs": {}, + "recordedDirentsInputs": {}, + "envVariables": {}, + "generatedRepoSpecs": { + "com_github_jetbrains_kotlin_git": { + "bzlFile": "@@rules_kotlin~//src/main/starlark/core/repositories:compiler.bzl", + "ruleClassName": "kotlin_compiler_git_repository", "attributes": { - "sha256": "2a033f24bb620383eda440ad307cb8077cfec1c7eadc684d65216123a1b9613a", "urls": [ - "https://repo1.maven.org/maven2/com/google/api/api-common/1.10.1/api-common-1.10.1.jar", - "https://maven.google.com/com/google/api/api-common/1.10.1/api-common-1.10.1.jar" + "https://github.com/JetBrains/kotlin/releases/download/v1.9.23/kotlin-compiler-1.9.23.zip" ], - "downloaded_file_path": "com/google/api/api-common/1.10.1/api-common-1.10.1.jar" + "sha256": "93137d3aab9afa9b27cb06a824c2324195c6b6f6179d8a8653f440f5bd58be88" } }, - "com_google_auth_google_auth_library_oauth2_http_0_22_0": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", + "com_github_jetbrains_kotlin": { + "bzlFile": "@@rules_kotlin~//src/main/starlark/core/repositories:compiler.bzl", + "ruleClassName": "kotlin_capabilities_repository", "attributes": { - "sha256": "1722d895c42dc42ea1d1f392ddbec1fbb28f7a979022c3a6c29acc39cc777ad1", - "urls": [ - "https://repo1.maven.org/maven2/com/google/auth/google-auth-library-oauth2-http/0.22.0/google-auth-library-oauth2-http-0.22.0.jar", - "https://maven.google.com/com/google/auth/google-auth-library-oauth2-http/0.22.0/google-auth-library-oauth2-http-0.22.0.jar" - ], - "downloaded_file_path": "com/google/auth/google-auth-library-oauth2-http/0.22.0/google-auth-library-oauth2-http-0.22.0.jar" + "git_repository_name": "com_github_jetbrains_kotlin_git", + "compiler_version": "1.9.23" } }, - "com_typesafe_netty_netty_reactive_streams_2_0_5": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", + "com_github_google_ksp": { + "bzlFile": "@@rules_kotlin~//src/main/starlark/core/repositories:ksp.bzl", + "ruleClassName": "ksp_compiler_plugin_repository", "attributes": { - "sha256": "f949849fc8ee75fde468ba3a35df2e04577fa31a2940b83b2a7dc9d14dac13d6", "urls": [ - "https://repo1.maven.org/maven2/com/typesafe/netty/netty-reactive-streams/2.0.5/netty-reactive-streams-2.0.5.jar", - "https://maven.google.com/com/typesafe/netty/netty-reactive-streams/2.0.5/netty-reactive-streams-2.0.5.jar" + "https://github.com/google/ksp/releases/download/1.9.23-1.0.20/artifacts.zip" ], - "downloaded_file_path": "com/typesafe/netty/netty-reactive-streams/2.0.5/netty-reactive-streams-2.0.5.jar" + "sha256": "ee0618755913ef7fd6511288a232e8fad24838b9af6ea73972a76e81053c8c2d", + "strip_version": "1.9.23-1.0.20" } }, - "com_typesafe_netty_netty_reactive_streams_http_2_0_5": { + "com_github_pinterest_ktlint": { "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", "ruleClassName": "http_file", "attributes": { - "sha256": "b39224751ad936758176e9d994230380ade5e9079e7c8ad778e3995779bcf303", + "sha256": "01b2e0ef893383a50dbeb13970fe7fa3be36ca3e83259e01649945b09d736985", "urls": [ - "https://repo1.maven.org/maven2/com/typesafe/netty/netty-reactive-streams-http/2.0.5/netty-reactive-streams-http-2.0.5.jar", - "https://maven.google.com/com/typesafe/netty/netty-reactive-streams-http/2.0.5/netty-reactive-streams-http-2.0.5.jar" + "https://github.com/pinterest/ktlint/releases/download/1.3.0/ktlint" ], - "downloaded_file_path": "com/typesafe/netty/netty-reactive-streams-http/2.0.5/netty-reactive-streams-http-2.0.5.jar" + "executable": true } }, - "javax_annotation_javax_annotation_api_1_3_2": { + "rules_android": { "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", + "ruleClassName": "http_archive", "attributes": { - "sha256": "e04ba5195bcd555dc95650f7cc614d151e4bcd52d29a10b8aa2197f3ab89ab9b", + "sha256": "cd06d15dd8bb59926e4d65f9003bfc20f9da4b2519985c27e190cddc8b7a7806", + "strip_prefix": "rules_android-0.1.1", "urls": [ - "https://repo1.maven.org/maven2/javax/annotation/javax.annotation-api/1.3.2/javax.annotation-api-1.3.2.jar", - "https://maven.google.com/javax/annotation/javax.annotation-api/1.3.2/javax.annotation-api-1.3.2.jar" - ], - "downloaded_file_path": "javax/annotation/javax.annotation-api/1.3.2/javax.annotation-api-1.3.2.jar" + "https://github.com/bazelbuild/rules_android/archive/v0.1.1.zip" + ] } - }, - "com_google_j2objc_j2objc_annotations_1_3": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", + } + }, + "recordedRepoMappingEntries": [ + [ + "rules_kotlin~", + "bazel_tools", + "bazel_tools" + ] + ] + } + }, + "@@rules_nodejs~//nodejs:extensions.bzl%node": { + "general": { + "bzlTransitiveDigest": "xRRX0NuyvfLtjtzM4AqJgxdMSWWnLIw28rUUi10y6k0=", + "usagesDigest": "9IUJvk13jWE1kE+N3sP2y0mw9exjO9CGQ2oAgwKTNK4=", + "recordedFileInputs": {}, + "recordedDirentsInputs": {}, + "envVariables": {}, + "generatedRepoSpecs": { + "nodejs_linux_amd64": { + "bzlFile": "@@rules_nodejs~//nodejs:repositories.bzl", + "ruleClassName": "node_repositories", "attributes": { - "sha256": "21af30c92267bd6122c0e0b4d20cccb6641a37eaf956c6540ec471d584e64a7b", - "urls": [ - "https://repo1.maven.org/maven2/com/google/j2objc/j2objc-annotations/1.3/j2objc-annotations-1.3.jar", - "https://maven.google.com/com/google/j2objc/j2objc-annotations/1.3/j2objc-annotations-1.3.jar" - ], - "downloaded_file_path": "com/google/j2objc/j2objc-annotations/1.3/j2objc-annotations-1.3.jar" + "platform": "linux_amd64", + "node_version": "16.19.0" } }, - "software_amazon_awssdk_metrics_spi_2_17_183": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", + "nodejs_linux_arm64": { + "bzlFile": "@@rules_nodejs~//nodejs:repositories.bzl", + "ruleClassName": "node_repositories", "attributes": { - "sha256": "08a11dc8c4ba464beafbcc7ac05b8c724c1ccb93da99482e82a68540ac704e4a", - "urls": [ - "https://repo1.maven.org/maven2/software/amazon/awssdk/metrics-spi/2.17.183/metrics-spi-2.17.183.jar", - "https://maven.google.com/software/amazon/awssdk/metrics-spi/2.17.183/metrics-spi-2.17.183.jar" - ], - "downloaded_file_path": "software/amazon/awssdk/metrics-spi/2.17.183/metrics-spi-2.17.183.jar" + "platform": "linux_arm64", + "node_version": "16.19.0" } }, - "org_reactivestreams_reactive_streams_1_0_3": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", + "nodejs_linux_s390x": { + "bzlFile": "@@rules_nodejs~//nodejs:repositories.bzl", + "ruleClassName": "node_repositories", "attributes": { - "sha256": "1dee0481072d19c929b623e155e14d2f6085dc011529a0a0dbefc84cf571d865", - "urls": [ - "https://repo1.maven.org/maven2/org/reactivestreams/reactive-streams/1.0.3/reactive-streams-1.0.3.jar", - "https://maven.google.com/org/reactivestreams/reactive-streams/1.0.3/reactive-streams-1.0.3.jar" - ], - "downloaded_file_path": "org/reactivestreams/reactive-streams/1.0.3/reactive-streams-1.0.3.jar" + "platform": "linux_s390x", + "node_version": "16.19.0" } }, - "com_google_http_client_google_http_client_jackson2_1_38_0": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", + "nodejs_linux_ppc64le": { + "bzlFile": "@@rules_nodejs~//nodejs:repositories.bzl", + "ruleClassName": "node_repositories", "attributes": { - "sha256": "e6504a82425fcc2168a4ca4175138ddcc085168daed8cdedb86d8f6fdc296e1e", - "urls": [ - "https://repo1.maven.org/maven2/com/google/http-client/google-http-client-jackson2/1.38.0/google-http-client-jackson2-1.38.0.jar", - "https://maven.google.com/com/google/http-client/google-http-client-jackson2/1.38.0/google-http-client-jackson2-1.38.0.jar" - ], - "downloaded_file_path": "com/google/http-client/google-http-client-jackson2/1.38.0/google-http-client-jackson2-1.38.0.jar" + "platform": "linux_ppc64le", + "node_version": "16.19.0" } }, - "io_netty_netty_transport_4_1_72_Final": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", + "nodejs_darwin_amd64": { + "bzlFile": "@@rules_nodejs~//nodejs:repositories.bzl", + "ruleClassName": "node_repositories", "attributes": { - "sha256": "c5fb68e9a65b6e8a516adfcb9fa323479ee7b4d9449d8a529d2ecab3d3711d5a", - "urls": [ - "https://repo1.maven.org/maven2/io/netty/netty-transport/4.1.72.Final/netty-transport-4.1.72.Final.jar", - "https://maven.google.com/io/netty/netty-transport/4.1.72.Final/netty-transport-4.1.72.Final.jar" - ], - "downloaded_file_path": "io/netty/netty-transport/4.1.72.Final/netty-transport-4.1.72.Final.jar" + "platform": "darwin_amd64", + "node_version": "16.19.0" } }, - "io_netty_netty_codec_http2_4_1_72_Final": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", + "nodejs_darwin_arm64": { + "bzlFile": "@@rules_nodejs~//nodejs:repositories.bzl", + "ruleClassName": "node_repositories", "attributes": { - "sha256": "c89a70500f59e8563e720aaa808263a514bd9e2bd91ba84eab8c2ccb45f234b2", - "urls": [ - "https://repo1.maven.org/maven2/io/netty/netty-codec-http2/4.1.72.Final/netty-codec-http2-4.1.72.Final.jar", - "https://maven.google.com/io/netty/netty-codec-http2/4.1.72.Final/netty-codec-http2-4.1.72.Final.jar" - ], - "downloaded_file_path": "io/netty/netty-codec-http2/4.1.72.Final/netty-codec-http2-4.1.72.Final.jar" + "platform": "darwin_arm64", + "node_version": "16.19.0" } }, - "io_opencensus_opencensus_api_0_24_0": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", + "nodejs_windows_amd64": { + "bzlFile": "@@rules_nodejs~//nodejs:repositories.bzl", + "ruleClassName": "node_repositories", "attributes": { - "sha256": "f561b1cc2673844288e596ddf5bb6596868a8472fd2cb8993953fc5c034b2352", - "urls": [ - "https://repo1.maven.org/maven2/io/opencensus/opencensus-api/0.24.0/opencensus-api-0.24.0.jar", - "https://maven.google.com/io/opencensus/opencensus-api/0.24.0/opencensus-api-0.24.0.jar" - ], - "downloaded_file_path": "io/opencensus/opencensus-api/0.24.0/opencensus-api-0.24.0.jar" + "platform": "windows_amd64", + "node_version": "16.19.0" } }, - "rules_jvm_external_deps": { - "bzlFile": "@@rules_jvm_external~//:coursier.bzl", - "ruleClassName": "pinned_coursier_fetch", + "nodejs": { + "bzlFile": "@@rules_nodejs~//nodejs/private:nodejs_repo_host_os_alias.bzl", + "ruleClassName": "nodejs_repo_host_os_alias", "attributes": { - "repositories": [ - "{ \"repo_url\": \"https://repo1.maven.org/maven2\" }" - ], - "artifacts": [ - "{ \"group\": \"com.google.auth\", \"artifact\": \"google-auth-library-credentials\", \"version\": \"0.22.0\" }", - "{ \"group\": \"com.google.auth\", \"artifact\": \"google-auth-library-oauth2-http\", \"version\": \"0.22.0\" }", - "{ \"group\": \"com.google.cloud\", \"artifact\": \"google-cloud-core\", \"version\": \"1.93.10\" }", - "{ \"group\": \"com.google.cloud\", \"artifact\": \"google-cloud-storage\", \"version\": \"1.113.4\" }", - "{ \"group\": \"com.google.code.gson\", \"artifact\": \"gson\", \"version\": \"2.9.0\" }", - "{ \"group\": \"com.google.googlejavaformat\", \"artifact\": \"google-java-format\", \"version\": \"1.15.0\" }", - "{ \"group\": \"com.google.guava\", \"artifact\": \"guava\", \"version\": \"31.1-jre\" }", - "{ \"group\": \"org.apache.maven\", \"artifact\": \"maven-artifact\", \"version\": \"3.8.6\" }", - "{ \"group\": \"software.amazon.awssdk\", \"artifact\": \"s3\", \"version\": \"2.17.183\" }" - ], - "fetch_sources": true, - "fetch_javadoc": false, - "generate_compat_repositories": false, - "maven_install_json": "@@rules_jvm_external~//:rules_jvm_external_deps_install.json", - "override_targets": {}, - "strict_visibility": false, - "strict_visibility_value": [ - "@@//visibility:private" - ], - "jetify": false, - "jetify_include_list": [ - "*" - ], - "additional_netrc_lines": [], - "fail_if_repin_required": false, - "use_starlark_android_rules": false, - "aar_import_bzl_label": "@build_bazel_rules_android//android:rules.bzl", - "duplicate_version_warning": "warn" + "user_node_repository_name": "nodejs" } }, - "org_threeten_threetenbp_1_5_0": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", + "nodejs_host": { + "bzlFile": "@@rules_nodejs~//nodejs/private:nodejs_repo_host_os_alias.bzl", + "ruleClassName": "nodejs_repo_host_os_alias", "attributes": { - "sha256": "dcf9c0f940739f2a825cd8626ff27113459a2f6eb18797c7152f93fff69c264f", - "urls": [ - "https://repo1.maven.org/maven2/org/threeten/threetenbp/1.5.0/threetenbp-1.5.0.jar", - "https://maven.google.com/org/threeten/threetenbp/1.5.0/threetenbp-1.5.0.jar" - ], - "downloaded_file_path": "org/threeten/threetenbp/1.5.0/threetenbp-1.5.0.jar" + "user_node_repository_name": "nodejs" } }, - "software_amazon_awssdk_http_client_spi_2_17_183": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", + "nodejs_toolchains": { + "bzlFile": "@@rules_nodejs~//nodejs/private:toolchains_repo.bzl", + "ruleClassName": "toolchains_repo", "attributes": { - "sha256": "fe7120f175df9e47ebcc5d946d7f40110faf2ba0a30364f3b935d5b8a5a6c3c6", - "urls": [ - "https://repo1.maven.org/maven2/software/amazon/awssdk/http-client-spi/2.17.183/http-client-spi-2.17.183.jar", - "https://maven.google.com/software/amazon/awssdk/http-client-spi/2.17.183/http-client-spi-2.17.183.jar" - ], - "downloaded_file_path": "software/amazon/awssdk/http-client-spi/2.17.183/http-client-spi-2.17.183.jar" - } - }, - "software_amazon_awssdk_third_party_jackson_core_2_17_183": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "1bc27c9960993c20e1ab058012dd1ae04c875eec9f0f08f2b2ca41e578dee9a4", - "urls": [ - "https://repo1.maven.org/maven2/software/amazon/awssdk/third-party-jackson-core/2.17.183/third-party-jackson-core-2.17.183.jar", - "https://maven.google.com/software/amazon/awssdk/third-party-jackson-core/2.17.183/third-party-jackson-core-2.17.183.jar" - ], - "downloaded_file_path": "software/amazon/awssdk/third-party-jackson-core/2.17.183/third-party-jackson-core-2.17.183.jar" - } - }, - "software_amazon_eventstream_eventstream_1_0_1": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "0c37d8e696117f02c302191b8110b0d0eb20fa412fce34c3a269ec73c16ce822", - "urls": [ - "https://repo1.maven.org/maven2/software/amazon/eventstream/eventstream/1.0.1/eventstream-1.0.1.jar", - "https://maven.google.com/software/amazon/eventstream/eventstream/1.0.1/eventstream-1.0.1.jar" - ], - "downloaded_file_path": "software/amazon/eventstream/eventstream/1.0.1/eventstream-1.0.1.jar" - } - }, - "com_google_oauth_client_google_oauth_client_1_31_1": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "4ed4e2948251dbda66ce251bd7f3b32cd8570055e5cdb165a3c7aea8f43da0ff", - "urls": [ - "https://repo1.maven.org/maven2/com/google/oauth-client/google-oauth-client/1.31.1/google-oauth-client-1.31.1.jar", - "https://maven.google.com/com/google/oauth-client/google-oauth-client/1.31.1/google-oauth-client-1.31.1.jar" - ], - "downloaded_file_path": "com/google/oauth-client/google-oauth-client/1.31.1/google-oauth-client-1.31.1.jar" - } - }, - "software_amazon_awssdk_aws_xml_protocol_2_17_183": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "566bba05d49256fa6994efd68fa625ae05a62ea45ee74bb9130d20ea20988363", - "urls": [ - "https://repo1.maven.org/maven2/software/amazon/awssdk/aws-xml-protocol/2.17.183/aws-xml-protocol-2.17.183.jar", - "https://maven.google.com/software/amazon/awssdk/aws-xml-protocol/2.17.183/aws-xml-protocol-2.17.183.jar" - ], - "downloaded_file_path": "software/amazon/awssdk/aws-xml-protocol/2.17.183/aws-xml-protocol-2.17.183.jar" - } - }, - "software_amazon_awssdk_annotations_2_17_183": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "8e4d72361ca805a0bd8bbd9017cd7ff77c8d170f2dd469c7d52d5653330bb3fd", - "urls": [ - "https://repo1.maven.org/maven2/software/amazon/awssdk/annotations/2.17.183/annotations-2.17.183.jar", - "https://maven.google.com/software/amazon/awssdk/annotations/2.17.183/annotations-2.17.183.jar" - ], - "downloaded_file_path": "software/amazon/awssdk/annotations/2.17.183/annotations-2.17.183.jar" - } - }, - "software_amazon_awssdk_netty_nio_client_2_17_183": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "a6d356f364c56d7b90006b0b7e503b8630010993a5587ce42e74b10b8dca2238", - "urls": [ - "https://repo1.maven.org/maven2/software/amazon/awssdk/netty-nio-client/2.17.183/netty-nio-client-2.17.183.jar", - "https://maven.google.com/software/amazon/awssdk/netty-nio-client/2.17.183/netty-nio-client-2.17.183.jar" - ], - "downloaded_file_path": "software/amazon/awssdk/netty-nio-client/2.17.183/netty-nio-client-2.17.183.jar" - } - }, - "com_google_guava_guava_31_1_jre": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "a42edc9cab792e39fe39bb94f3fca655ed157ff87a8af78e1d6ba5b07c4a00ab", - "urls": [ - "https://repo1.maven.org/maven2/com/google/guava/guava/31.1-jre/guava-31.1-jre.jar", - "https://maven.google.com/com/google/guava/guava/31.1-jre/guava-31.1-jre.jar" - ], - "downloaded_file_path": "com/google/guava/guava/31.1-jre/guava-31.1-jre.jar" - } - }, - "com_google_auto_value_auto_value_annotations_1_7_4": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "fedd59b0b4986c342f6ab2d182f2a4ee9fceb2c7e2d5bdc4dc764c92394a23d3", - "urls": [ - "https://repo1.maven.org/maven2/com/google/auto/value/auto-value-annotations/1.7.4/auto-value-annotations-1.7.4.jar", - "https://maven.google.com/com/google/auto/value/auto-value-annotations/1.7.4/auto-value-annotations-1.7.4.jar" - ], - "downloaded_file_path": "com/google/auto/value/auto-value-annotations/1.7.4/auto-value-annotations-1.7.4.jar" - } - }, - "io_netty_netty_transport_native_unix_common_4_1_72_Final": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "6f8f1cc29b5a234eeee9439a63eb3f03a5994aa540ff555cb0b2c88cefaf6877", - "urls": [ - "https://repo1.maven.org/maven2/io/netty/netty-transport-native-unix-common/4.1.72.Final/netty-transport-native-unix-common-4.1.72.Final.jar", - "https://maven.google.com/io/netty/netty-transport-native-unix-common/4.1.72.Final/netty-transport-native-unix-common-4.1.72.Final.jar" - ], - "downloaded_file_path": "io/netty/netty-transport-native-unix-common/4.1.72.Final/netty-transport-native-unix-common-4.1.72.Final.jar" - } - }, - "io_opencensus_opencensus_contrib_http_util_0_24_0": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "7155273bbb1ed3d477ea33cf19d7bbc0b285ff395f43b29ae576722cf247000f", - "urls": [ - "https://repo1.maven.org/maven2/io/opencensus/opencensus-contrib-http-util/0.24.0/opencensus-contrib-http-util-0.24.0.jar", - "https://maven.google.com/io/opencensus/opencensus-contrib-http-util/0.24.0/opencensus-contrib-http-util-0.24.0.jar" - ], - "downloaded_file_path": "io/opencensus/opencensus-contrib-http-util/0.24.0/opencensus-contrib-http-util-0.24.0.jar" - } - }, - "com_fasterxml_jackson_core_jackson_core_2_11_3": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "78cd0a6b936232e06dd3e38da8a0345348a09cd1ff9c4d844c6ee72c75cfc402", - "urls": [ - "https://repo1.maven.org/maven2/com/fasterxml/jackson/core/jackson-core/2.11.3/jackson-core-2.11.3.jar", - "https://maven.google.com/com/fasterxml/jackson/core/jackson-core/2.11.3/jackson-core-2.11.3.jar" - ], - "downloaded_file_path": "com/fasterxml/jackson/core/jackson-core/2.11.3/jackson-core-2.11.3.jar" - } - }, - "com_google_cloud_google_cloud_core_1_93_10": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "832d74eca66f4601e162a8460d6f59f50d1d23f93c18b02654423b6b0d67c6ea", - "urls": [ - "https://repo1.maven.org/maven2/com/google/cloud/google-cloud-core/1.93.10/google-cloud-core-1.93.10.jar", - "https://maven.google.com/com/google/cloud/google-cloud-core/1.93.10/google-cloud-core-1.93.10.jar" - ], - "downloaded_file_path": "com/google/cloud/google-cloud-core/1.93.10/google-cloud-core-1.93.10.jar" - } - }, - "com_google_auth_google_auth_library_credentials_0_22_0": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "42c76031276de5b520909e9faf88c5b3c9a722d69ee9cfdafedb1c52c355dfc5", - "urls": [ - "https://repo1.maven.org/maven2/com/google/auth/google-auth-library-credentials/0.22.0/google-auth-library-credentials-0.22.0.jar", - "https://maven.google.com/com/google/auth/google-auth-library-credentials/0.22.0/google-auth-library-credentials-0.22.0.jar" - ], - "downloaded_file_path": "com/google/auth/google-auth-library-credentials/0.22.0/google-auth-library-credentials-0.22.0.jar" - } - }, - "software_amazon_awssdk_profiles_2_17_183": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "78833b32fde3f1c5320373b9ea955c1bbc28f2c904010791c4784e610193ee56", - "urls": [ - "https://repo1.maven.org/maven2/software/amazon/awssdk/profiles/2.17.183/profiles-2.17.183.jar", - "https://maven.google.com/software/amazon/awssdk/profiles/2.17.183/profiles-2.17.183.jar" - ], - "downloaded_file_path": "software/amazon/awssdk/profiles/2.17.183/profiles-2.17.183.jar" - } - }, - "org_apache_httpcomponents_httpcore_4_4_13": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "e06e89d40943245fcfa39ec537cdbfce3762aecde8f9c597780d2b00c2b43424", - "urls": [ - "https://repo1.maven.org/maven2/org/apache/httpcomponents/httpcore/4.4.13/httpcore-4.4.13.jar", - "https://maven.google.com/org/apache/httpcomponents/httpcore/4.4.13/httpcore-4.4.13.jar" - ], - "downloaded_file_path": "org/apache/httpcomponents/httpcore/4.4.13/httpcore-4.4.13.jar" - } - }, - "io_netty_netty_common_4_1_72_Final": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "8adb4c291260ceb2859a68c49f0adeed36bf49587608e2b81ecff6aaf06025e9", - "urls": [ - "https://repo1.maven.org/maven2/io/netty/netty-common/4.1.72.Final/netty-common-4.1.72.Final.jar", - "https://maven.google.com/io/netty/netty-common/4.1.72.Final/netty-common-4.1.72.Final.jar" - ], - "downloaded_file_path": "io/netty/netty-common/4.1.72.Final/netty-common-4.1.72.Final.jar" - } - }, - "io_netty_netty_transport_classes_epoll_4_1_72_Final": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "e1528a9751c1285aa7beaf3a1eb0597151716426ce38598ac9bc0891209b9e68", - "urls": [ - "https://repo1.maven.org/maven2/io/netty/netty-transport-classes-epoll/4.1.72.Final/netty-transport-classes-epoll-4.1.72.Final.jar", - "https://maven.google.com/io/netty/netty-transport-classes-epoll/4.1.72.Final/netty-transport-classes-epoll-4.1.72.Final.jar" - ], - "downloaded_file_path": "io/netty/netty-transport-classes-epoll/4.1.72.Final/netty-transport-classes-epoll-4.1.72.Final.jar" - } - }, - "org_checkerframework_checker_qual_3_12_0": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "ff10785ac2a357ec5de9c293cb982a2cbb605c0309ea4cc1cb9b9bc6dbe7f3cb", - "urls": [ - "https://repo1.maven.org/maven2/org/checkerframework/checker-qual/3.12.0/checker-qual-3.12.0.jar", - "https://maven.google.com/org/checkerframework/checker-qual/3.12.0/checker-qual-3.12.0.jar" - ], - "downloaded_file_path": "org/checkerframework/checker-qual/3.12.0/checker-qual-3.12.0.jar" - } - }, - "com_google_cloud_google_cloud_core_http_1_93_10": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "81ac67c14c7c4244d2b7db2607ad352416aca8d3bb2adf338964e8fea25b1b3c", - "urls": [ - "https://repo1.maven.org/maven2/com/google/cloud/google-cloud-core-http/1.93.10/google-cloud-core-http-1.93.10.jar", - "https://maven.google.com/com/google/cloud/google-cloud-core-http/1.93.10/google-cloud-core-http-1.93.10.jar" - ], - "downloaded_file_path": "com/google/cloud/google-cloud-core-http/1.93.10/google-cloud-core-http-1.93.10.jar" - } - }, - "software_amazon_awssdk_utils_2_17_183": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "7bd849bb5aa71bfdf6b849643736ecab3a7b3f204795804eefe5754104231ec6", - "urls": [ - "https://repo1.maven.org/maven2/software/amazon/awssdk/utils/2.17.183/utils-2.17.183.jar", - "https://maven.google.com/software/amazon/awssdk/utils/2.17.183/utils-2.17.183.jar" - ], - "downloaded_file_path": "software/amazon/awssdk/utils/2.17.183/utils-2.17.183.jar" - } - }, - "org_apache_commons_commons_lang3_3_8_1": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "dac807f65b07698ff39b1b07bfef3d87ae3fd46d91bbf8a2bc02b2a831616f68", - "urls": [ - "https://repo1.maven.org/maven2/org/apache/commons/commons-lang3/3.8.1/commons-lang3-3.8.1.jar", - "https://maven.google.com/org/apache/commons/commons-lang3/3.8.1/commons-lang3-3.8.1.jar" - ], - "downloaded_file_path": "org/apache/commons/commons-lang3/3.8.1/commons-lang3-3.8.1.jar" - } - }, - "software_amazon_awssdk_aws_core_2_17_183": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "bccbdbea689a665a702ff19828662d87fb7fe81529df13f02ef1e4c474ea9f93", - "urls": [ - "https://repo1.maven.org/maven2/software/amazon/awssdk/aws-core/2.17.183/aws-core-2.17.183.jar", - "https://maven.google.com/software/amazon/awssdk/aws-core/2.17.183/aws-core-2.17.183.jar" - ], - "downloaded_file_path": "software/amazon/awssdk/aws-core/2.17.183/aws-core-2.17.183.jar" - } - }, - "com_google_api_gax_httpjson_0_77_0": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "fd4dae47fa016d3b26e8d90b67ddc6c23c4c06e8bcdf085c70310ab7ef324bd6", - "urls": [ - "https://repo1.maven.org/maven2/com/google/api/gax-httpjson/0.77.0/gax-httpjson-0.77.0.jar", - "https://maven.google.com/com/google/api/gax-httpjson/0.77.0/gax-httpjson-0.77.0.jar" - ], - "downloaded_file_path": "com/google/api/gax-httpjson/0.77.0/gax-httpjson-0.77.0.jar" - } - }, - "unpinned_rules_jvm_external_deps": { - "bzlFile": "@@rules_jvm_external~//:coursier.bzl", - "ruleClassName": "coursier_fetch", - "attributes": { - "repositories": [ - "{ \"repo_url\": \"https://repo1.maven.org/maven2\" }" - ], - "artifacts": [ - "{ \"group\": \"com.google.auth\", \"artifact\": \"google-auth-library-credentials\", \"version\": \"0.22.0\" }", - "{ \"group\": \"com.google.auth\", \"artifact\": \"google-auth-library-oauth2-http\", \"version\": \"0.22.0\" }", - "{ \"group\": \"com.google.cloud\", \"artifact\": \"google-cloud-core\", \"version\": \"1.93.10\" }", - "{ \"group\": \"com.google.cloud\", \"artifact\": \"google-cloud-storage\", \"version\": \"1.113.4\" }", - "{ \"group\": \"com.google.code.gson\", \"artifact\": \"gson\", \"version\": \"2.9.0\" }", - "{ \"group\": \"com.google.googlejavaformat\", \"artifact\": \"google-java-format\", \"version\": \"1.15.0\" }", - "{ \"group\": \"com.google.guava\", \"artifact\": \"guava\", \"version\": \"31.1-jre\" }", - "{ \"group\": \"org.apache.maven\", \"artifact\": \"maven-artifact\", \"version\": \"3.8.6\" }", - "{ \"group\": \"software.amazon.awssdk\", \"artifact\": \"s3\", \"version\": \"2.17.183\" }" - ], - "fail_on_missing_checksum": true, - "fetch_sources": true, - "fetch_javadoc": false, - "excluded_artifacts": [], - "generate_compat_repositories": false, - "version_conflict_policy": "default", - "override_targets": {}, - "strict_visibility": false, - "strict_visibility_value": [ - "@@//visibility:private" - ], - "maven_install_json": "@@rules_jvm_external~//:rules_jvm_external_deps_install.json", - "resolve_timeout": 600, - "jetify": false, - "jetify_include_list": [ - "*" - ], - "use_starlark_android_rules": false, - "aar_import_bzl_label": "@build_bazel_rules_android//android:rules.bzl", - "duplicate_version_warning": "warn" - } - }, - "com_google_errorprone_error_prone_annotations_2_11_0": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "721cb91842b46fa056847d104d5225c8b8e1e8b62263b993051e1e5a0137b7ec", - "urls": [ - "https://repo1.maven.org/maven2/com/google/errorprone/error_prone_annotations/2.11.0/error_prone_annotations-2.11.0.jar", - "https://maven.google.com/com/google/errorprone/error_prone_annotations/2.11.0/error_prone_annotations-2.11.0.jar" - ], - "downloaded_file_path": "com/google/errorprone/error_prone_annotations/2.11.0/error_prone_annotations-2.11.0.jar" - } - }, - "software_amazon_awssdk_regions_2_17_183": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "d3079395f3ffc07d04ffcce16fca29fb5968197f6e9ea3dbff6be297102b40a5", - "urls": [ - "https://repo1.maven.org/maven2/software/amazon/awssdk/regions/2.17.183/regions-2.17.183.jar", - "https://maven.google.com/software/amazon/awssdk/regions/2.17.183/regions-2.17.183.jar" - ], - "downloaded_file_path": "software/amazon/awssdk/regions/2.17.183/regions-2.17.183.jar" - } - }, - "io_netty_netty_handler_4_1_72_Final": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "9cb6012af7e06361d738ac4e3bdc49a158f8cf87d9dee0f2744056b7d99c28d5", - "urls": [ - "https://repo1.maven.org/maven2/io/netty/netty-handler/4.1.72.Final/netty-handler-4.1.72.Final.jar", - "https://maven.google.com/io/netty/netty-handler/4.1.72.Final/netty-handler-4.1.72.Final.jar" - ], - "downloaded_file_path": "io/netty/netty-handler/4.1.72.Final/netty-handler-4.1.72.Final.jar" - } - }, - "software_amazon_awssdk_aws_query_protocol_2_17_183": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "4dace03c76f80f3dec920cb3dedb2a95984c4366ef4fda728660cb90bed74848", - "urls": [ - "https://repo1.maven.org/maven2/software/amazon/awssdk/aws-query-protocol/2.17.183/aws-query-protocol-2.17.183.jar", - "https://maven.google.com/software/amazon/awssdk/aws-query-protocol/2.17.183/aws-query-protocol-2.17.183.jar" - ], - "downloaded_file_path": "software/amazon/awssdk/aws-query-protocol/2.17.183/aws-query-protocol-2.17.183.jar" - } - }, - "io_netty_netty_codec_http_4_1_72_Final": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "fa6fec88010bfaf6a7415b5364671b6b18ffb6b35a986ab97b423fd8c3a0174b", - "urls": [ - "https://repo1.maven.org/maven2/io/netty/netty-codec-http/4.1.72.Final/netty-codec-http-4.1.72.Final.jar", - "https://maven.google.com/io/netty/netty-codec-http/4.1.72.Final/netty-codec-http-4.1.72.Final.jar" - ], - "downloaded_file_path": "io/netty/netty-codec-http/4.1.72.Final/netty-codec-http-4.1.72.Final.jar" - } - }, - "io_netty_netty_resolver_4_1_72_Final": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "6474598aab7cc9d8d6cfa06c05bd1b19adbf7f8451dbdd73070b33a6c60b1b90", - "urls": [ - "https://repo1.maven.org/maven2/io/netty/netty-resolver/4.1.72.Final/netty-resolver-4.1.72.Final.jar", - "https://maven.google.com/io/netty/netty-resolver/4.1.72.Final/netty-resolver-4.1.72.Final.jar" - ], - "downloaded_file_path": "io/netty/netty-resolver/4.1.72.Final/netty-resolver-4.1.72.Final.jar" - } - }, - "software_amazon_awssdk_protocol_core_2_17_183": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "10e7c4faa1f05e2d73055d0390dbd0bb6450e2e6cb85beda051b1e4693c826ce", - "urls": [ - "https://repo1.maven.org/maven2/software/amazon/awssdk/protocol-core/2.17.183/protocol-core-2.17.183.jar", - "https://maven.google.com/software/amazon/awssdk/protocol-core/2.17.183/protocol-core-2.17.183.jar" - ], - "downloaded_file_path": "software/amazon/awssdk/protocol-core/2.17.183/protocol-core-2.17.183.jar" - } - }, - "org_checkerframework_checker_compat_qual_2_5_5": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "11d134b245e9cacc474514d2d66b5b8618f8039a1465cdc55bbc0b34e0008b7a", - "urls": [ - "https://repo1.maven.org/maven2/org/checkerframework/checker-compat-qual/2.5.5/checker-compat-qual-2.5.5.jar", - "https://maven.google.com/org/checkerframework/checker-compat-qual/2.5.5/checker-compat-qual-2.5.5.jar" - ], - "downloaded_file_path": "org/checkerframework/checker-compat-qual/2.5.5/checker-compat-qual-2.5.5.jar" - } - }, - "com_google_apis_google_api_services_storage_v1_rev20200927_1_30_10": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "52d26a9d105f8d8a0850807285f307a76cea8f3e0cdb2be4d3b15b1adfa77351", - "urls": [ - "https://repo1.maven.org/maven2/com/google/apis/google-api-services-storage/v1-rev20200927-1.30.10/google-api-services-storage-v1-rev20200927-1.30.10.jar", - "https://maven.google.com/com/google/apis/google-api-services-storage/v1-rev20200927-1.30.10/google-api-services-storage-v1-rev20200927-1.30.10.jar" - ], - "downloaded_file_path": "com/google/apis/google-api-services-storage/v1-rev20200927-1.30.10/google-api-services-storage-v1-rev20200927-1.30.10.jar" - } - }, - "com_google_api_client_google_api_client_1_30_11": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "ee6f97865cc7de6c7c80955c3f37372cf3887bd75e4fc06f1058a6b4cd9bf4da", - "urls": [ - "https://repo1.maven.org/maven2/com/google/api-client/google-api-client/1.30.11/google-api-client-1.30.11.jar", - "https://maven.google.com/com/google/api-client/google-api-client/1.30.11/google-api-client-1.30.11.jar" - ], - "downloaded_file_path": "com/google/api-client/google-api-client/1.30.11/google-api-client-1.30.11.jar" - } - }, - "software_amazon_awssdk_s3_2_17_183": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "ab073b91107a9e4ed9f030314077d137fe627e055ad895fabb036980a050e360", - "urls": [ - "https://repo1.maven.org/maven2/software/amazon/awssdk/s3/2.17.183/s3-2.17.183.jar", - "https://maven.google.com/software/amazon/awssdk/s3/2.17.183/s3-2.17.183.jar" - ], - "downloaded_file_path": "software/amazon/awssdk/s3/2.17.183/s3-2.17.183.jar" - } - }, - "org_apache_maven_maven_artifact_3_8_6": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "de22a4c6f54fe31276a823b1bbd3adfd6823529e732f431b5eff0852c2b9252b", - "urls": [ - "https://repo1.maven.org/maven2/org/apache/maven/maven-artifact/3.8.6/maven-artifact-3.8.6.jar", - "https://maven.google.com/org/apache/maven/maven-artifact/3.8.6/maven-artifact-3.8.6.jar" - ], - "downloaded_file_path": "org/apache/maven/maven-artifact/3.8.6/maven-artifact-3.8.6.jar" - } - }, - "com_google_googlejavaformat_google_java_format_1_15_0": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "4f546cfe159547ac3b9547daa9649e728f6abc254979c975f1cb9971793692c3", - "urls": [ - "https://repo1.maven.org/maven2/com/google/googlejavaformat/google-java-format/1.15.0/google-java-format-1.15.0.jar", - "https://maven.google.com/com/google/googlejavaformat/google-java-format/1.15.0/google-java-format-1.15.0.jar" - ], - "downloaded_file_path": "com/google/googlejavaformat/google-java-format/1.15.0/google-java-format-1.15.0.jar" - } - }, - "org_apache_httpcomponents_httpclient_4_5_13": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "6fe9026a566c6a5001608cf3fc32196641f6c1e5e1986d1037ccdbd5f31ef743", - "urls": [ - "https://repo1.maven.org/maven2/org/apache/httpcomponents/httpclient/4.5.13/httpclient-4.5.13.jar", - "https://maven.google.com/org/apache/httpcomponents/httpclient/4.5.13/httpclient-4.5.13.jar" - ], - "downloaded_file_path": "org/apache/httpcomponents/httpclient/4.5.13/httpclient-4.5.13.jar" - } - }, - "com_google_guava_listenablefuture_9999_0_empty_to_avoid_conflict_with_guava": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "b372a037d4230aa57fbeffdef30fd6123f9c0c2db85d0aced00c91b974f33f99", - "urls": [ - "https://repo1.maven.org/maven2/com/google/guava/listenablefuture/9999.0-empty-to-avoid-conflict-with-guava/listenablefuture-9999.0-empty-to-avoid-conflict-with-guava.jar", - "https://maven.google.com/com/google/guava/listenablefuture/9999.0-empty-to-avoid-conflict-with-guava/listenablefuture-9999.0-empty-to-avoid-conflict-with-guava.jar" - ], - "downloaded_file_path": "com/google/guava/listenablefuture/9999.0-empty-to-avoid-conflict-with-guava/listenablefuture-9999.0-empty-to-avoid-conflict-with-guava.jar" - } - }, - "com_google_http_client_google_http_client_1_38_0": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "411f4a42519b6b78bdc0fcfdf74c9edcef0ee97afa4a667abe04045a508d6302", - "urls": [ - "https://repo1.maven.org/maven2/com/google/http-client/google-http-client/1.38.0/google-http-client-1.38.0.jar", - "https://maven.google.com/com/google/http-client/google-http-client/1.38.0/google-http-client-1.38.0.jar" - ], - "downloaded_file_path": "com/google/http-client/google-http-client/1.38.0/google-http-client-1.38.0.jar" - } - }, - "software_amazon_awssdk_apache_client_2_17_183": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "78ceae502fce6a97bbe5ff8f6a010a52ab7ea3ae66cb1a4122e18185fce45022", - "urls": [ - "https://repo1.maven.org/maven2/software/amazon/awssdk/apache-client/2.17.183/apache-client-2.17.183.jar", - "https://maven.google.com/software/amazon/awssdk/apache-client/2.17.183/apache-client-2.17.183.jar" - ], - "downloaded_file_path": "software/amazon/awssdk/apache-client/2.17.183/apache-client-2.17.183.jar" - } - }, - "software_amazon_awssdk_arns_2_17_183": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "659a185e191d66c71de81209490e66abeaccae208ea7b2831a738670823447aa", - "urls": [ - "https://repo1.maven.org/maven2/software/amazon/awssdk/arns/2.17.183/arns-2.17.183.jar", - "https://maven.google.com/software/amazon/awssdk/arns/2.17.183/arns-2.17.183.jar" - ], - "downloaded_file_path": "software/amazon/awssdk/arns/2.17.183/arns-2.17.183.jar" - } - }, - "com_google_code_gson_gson_2_9_0": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "c96d60551331a196dac54b745aa642cd078ef89b6f267146b705f2c2cbef052d", - "urls": [ - "https://repo1.maven.org/maven2/com/google/code/gson/gson/2.9.0/gson-2.9.0.jar", - "https://maven.google.com/com/google/code/gson/gson/2.9.0/gson-2.9.0.jar" - ], - "downloaded_file_path": "com/google/code/gson/gson/2.9.0/gson-2.9.0.jar" - } - }, - "io_netty_netty_buffer_4_1_72_Final": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "568ff7cd9d8e2284ec980730c88924f686642929f8f219a74518b4e64755f3a1", - "urls": [ - "https://repo1.maven.org/maven2/io/netty/netty-buffer/4.1.72.Final/netty-buffer-4.1.72.Final.jar", - "https://maven.google.com/io/netty/netty-buffer/4.1.72.Final/netty-buffer-4.1.72.Final.jar" - ], - "downloaded_file_path": "io/netty/netty-buffer/4.1.72.Final/netty-buffer-4.1.72.Final.jar" - } - }, - "com_google_code_findbugs_jsr305_3_0_2": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "766ad2a0783f2687962c8ad74ceecc38a28b9f72a2d085ee438b7813e928d0c7", - "urls": [ - "https://repo1.maven.org/maven2/com/google/code/findbugs/jsr305/3.0.2/jsr305-3.0.2.jar", - "https://maven.google.com/com/google/code/findbugs/jsr305/3.0.2/jsr305-3.0.2.jar" - ], - "downloaded_file_path": "com/google/code/findbugs/jsr305/3.0.2/jsr305-3.0.2.jar" - } - }, - "commons_codec_commons_codec_1_11": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "e599d5318e97aa48f42136a2927e6dfa4e8881dff0e6c8e3109ddbbff51d7b7d", - "urls": [ - "https://repo1.maven.org/maven2/commons-codec/commons-codec/1.11/commons-codec-1.11.jar", - "https://maven.google.com/commons-codec/commons-codec/1.11/commons-codec-1.11.jar" - ], - "downloaded_file_path": "commons-codec/commons-codec/1.11/commons-codec-1.11.jar" - } - }, - "software_amazon_awssdk_auth_2_17_183": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "8820c6636e5c14efc29399fb5565ce50212b0c1f4ed720a025a2c402d54e0978", - "urls": [ - "https://repo1.maven.org/maven2/software/amazon/awssdk/auth/2.17.183/auth-2.17.183.jar", - "https://maven.google.com/software/amazon/awssdk/auth/2.17.183/auth-2.17.183.jar" - ], - "downloaded_file_path": "software/amazon/awssdk/auth/2.17.183/auth-2.17.183.jar" - } - }, - "software_amazon_awssdk_json_utils_2_17_183": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "51ab7f550adc06afcb49f5270cdf690f1bfaaee243abaa5d978095e2a1e4e1a5", - "urls": [ - "https://repo1.maven.org/maven2/software/amazon/awssdk/json-utils/2.17.183/json-utils-2.17.183.jar", - "https://maven.google.com/software/amazon/awssdk/json-utils/2.17.183/json-utils-2.17.183.jar" - ], - "downloaded_file_path": "software/amazon/awssdk/json-utils/2.17.183/json-utils-2.17.183.jar" - } - }, - "org_codehaus_plexus_plexus_utils_3_3_1": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "4b570fcdbe5a894f249d2eb9b929358a9c88c3e548d227a80010461930222f2a", - "urls": [ - "https://repo1.maven.org/maven2/org/codehaus/plexus/plexus-utils/3.3.1/plexus-utils-3.3.1.jar", - "https://maven.google.com/org/codehaus/plexus/plexus-utils/3.3.1/plexus-utils-3.3.1.jar" - ], - "downloaded_file_path": "org/codehaus/plexus/plexus-utils/3.3.1/plexus-utils-3.3.1.jar" - } - }, - "com_google_protobuf_protobuf_java_util_3_13_0": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "d9de66b8c9445905dfa7064f6d5213d47ce88a20d34e21d83c4a94a229e14e62", - "urls": [ - "https://repo1.maven.org/maven2/com/google/protobuf/protobuf-java-util/3.13.0/protobuf-java-util-3.13.0.jar", - "https://maven.google.com/com/google/protobuf/protobuf-java-util/3.13.0/protobuf-java-util-3.13.0.jar" - ], - "downloaded_file_path": "com/google/protobuf/protobuf-java-util/3.13.0/protobuf-java-util-3.13.0.jar" - } - }, - "io_netty_netty_codec_4_1_72_Final": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "5d8591ca271a1e9c224e8de3873aa9936acb581ee0db514e7dc18523df36d16c", - "urls": [ - "https://repo1.maven.org/maven2/io/netty/netty-codec/4.1.72.Final/netty-codec-4.1.72.Final.jar", - "https://maven.google.com/io/netty/netty-codec/4.1.72.Final/netty-codec-4.1.72.Final.jar" - ], - "downloaded_file_path": "io/netty/netty-codec/4.1.72.Final/netty-codec-4.1.72.Final.jar" - } - }, - "com_google_protobuf_protobuf_java_3_13_0": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "97d5b2758408690c0dc276238707492a0b6a4d71206311b6c442cdc26c5973ff", - "urls": [ - "https://repo1.maven.org/maven2/com/google/protobuf/protobuf-java/3.13.0/protobuf-java-3.13.0.jar", - "https://maven.google.com/com/google/protobuf/protobuf-java/3.13.0/protobuf-java-3.13.0.jar" - ], - "downloaded_file_path": "com/google/protobuf/protobuf-java/3.13.0/protobuf-java-3.13.0.jar" - } - }, - "io_netty_netty_tcnative_classes_2_0_46_Final": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "d3ec888dcc4ac7915bf88b417c5e04fd354f4311032a748a6882df09347eed9a", - "urls": [ - "https://repo1.maven.org/maven2/io/netty/netty-tcnative-classes/2.0.46.Final/netty-tcnative-classes-2.0.46.Final.jar", - "https://maven.google.com/io/netty/netty-tcnative-classes/2.0.46.Final/netty-tcnative-classes-2.0.46.Final.jar" - ], - "downloaded_file_path": "io/netty/netty-tcnative-classes/2.0.46.Final/netty-tcnative-classes-2.0.46.Final.jar" - } - }, - "software_amazon_awssdk_sdk_core_2_17_183": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "677e9cc90fdd82c1f40f97b99cb115b13ad6c3f58beeeab1c061af6954d64c77", - "urls": [ - "https://repo1.maven.org/maven2/software/amazon/awssdk/sdk-core/2.17.183/sdk-core-2.17.183.jar", - "https://maven.google.com/software/amazon/awssdk/sdk-core/2.17.183/sdk-core-2.17.183.jar" - ], - "downloaded_file_path": "software/amazon/awssdk/sdk-core/2.17.183/sdk-core-2.17.183.jar" - } - } - }, - "recordedRepoMappingEntries": [ - [ - "rules_jvm_external~", - "bazel_tools", - "bazel_tools" - ], - [ - "rules_jvm_external~", - "rules_jvm_external", - "rules_jvm_external~" - ] - ] - } - }, - "@@rules_kotlin~//src/main/starlark/core/repositories:bzlmod_setup.bzl%rules_kotlin_extensions": { - "general": { - "bzlTransitiveDigest": "fus14IFJ/1LGWWGKPH/U18VnJCoMjfDt1ckahqCnM0A=", - "usagesDigest": "aJF6fLy82rR95Ff5CZPAqxNoFgOMLMN5ImfBS0nhnkg=", - "recordedFileInputs": {}, - "recordedDirentsInputs": {}, - "envVariables": {}, - "generatedRepoSpecs": { - "com_github_jetbrains_kotlin_git": { - "bzlFile": "@@rules_kotlin~//src/main/starlark/core/repositories:compiler.bzl", - "ruleClassName": "kotlin_compiler_git_repository", - "attributes": { - "urls": [ - "https://github.com/JetBrains/kotlin/releases/download/v1.9.23/kotlin-compiler-1.9.23.zip" - ], - "sha256": "93137d3aab9afa9b27cb06a824c2324195c6b6f6179d8a8653f440f5bd58be88" - } - }, - "com_github_jetbrains_kotlin": { - "bzlFile": "@@rules_kotlin~//src/main/starlark/core/repositories:compiler.bzl", - "ruleClassName": "kotlin_capabilities_repository", - "attributes": { - "git_repository_name": "com_github_jetbrains_kotlin_git", - "compiler_version": "1.9.23" - } - }, - "com_github_google_ksp": { - "bzlFile": "@@rules_kotlin~//src/main/starlark/core/repositories:ksp.bzl", - "ruleClassName": "ksp_compiler_plugin_repository", - "attributes": { - "urls": [ - "https://github.com/google/ksp/releases/download/1.9.23-1.0.20/artifacts.zip" - ], - "sha256": "ee0618755913ef7fd6511288a232e8fad24838b9af6ea73972a76e81053c8c2d", - "strip_version": "1.9.23-1.0.20" - } - }, - "com_github_pinterest_ktlint": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_file", - "attributes": { - "sha256": "01b2e0ef893383a50dbeb13970fe7fa3be36ca3e83259e01649945b09d736985", - "urls": [ - "https://github.com/pinterest/ktlint/releases/download/1.3.0/ktlint" - ], - "executable": true - } - }, - "rules_android": { - "bzlFile": "@@bazel_tools//tools/build_defs/repo:http.bzl", - "ruleClassName": "http_archive", - "attributes": { - "sha256": "cd06d15dd8bb59926e4d65f9003bfc20f9da4b2519985c27e190cddc8b7a7806", - "strip_prefix": "rules_android-0.1.1", - "urls": [ - "https://github.com/bazelbuild/rules_android/archive/v0.1.1.zip" - ] - } - } - }, - "recordedRepoMappingEntries": [ - [ - "rules_kotlin~", - "bazel_tools", - "bazel_tools" - ] - ] - } - }, - "@@rules_nodejs~//nodejs:extensions.bzl%node": { - "general": { - "bzlTransitiveDigest": "xRRX0NuyvfLtjtzM4AqJgxdMSWWnLIw28rUUi10y6k0=", - "usagesDigest": "9IUJvk13jWE1kE+N3sP2y0mw9exjO9CGQ2oAgwKTNK4=", - "recordedFileInputs": {}, - "recordedDirentsInputs": {}, - "envVariables": {}, - "generatedRepoSpecs": { - "nodejs_linux_amd64": { - "bzlFile": "@@rules_nodejs~//nodejs:repositories.bzl", - "ruleClassName": "node_repositories", - "attributes": { - "platform": "linux_amd64", - "node_version": "16.19.0" - } - }, - "nodejs_linux_arm64": { - "bzlFile": "@@rules_nodejs~//nodejs:repositories.bzl", - "ruleClassName": "node_repositories", - "attributes": { - "platform": "linux_arm64", - "node_version": "16.19.0" - } - }, - "nodejs_linux_s390x": { - "bzlFile": "@@rules_nodejs~//nodejs:repositories.bzl", - "ruleClassName": "node_repositories", - "attributes": { - "platform": "linux_s390x", - "node_version": "16.19.0" - } - }, - "nodejs_linux_ppc64le": { - "bzlFile": "@@rules_nodejs~//nodejs:repositories.bzl", - "ruleClassName": "node_repositories", - "attributes": { - "platform": "linux_ppc64le", - "node_version": "16.19.0" - } - }, - "nodejs_darwin_amd64": { - "bzlFile": "@@rules_nodejs~//nodejs:repositories.bzl", - "ruleClassName": "node_repositories", - "attributes": { - "platform": "darwin_amd64", - "node_version": "16.19.0" - } - }, - "nodejs_darwin_arm64": { - "bzlFile": "@@rules_nodejs~//nodejs:repositories.bzl", - "ruleClassName": "node_repositories", - "attributes": { - "platform": "darwin_arm64", - "node_version": "16.19.0" - } - }, - "nodejs_windows_amd64": { - "bzlFile": "@@rules_nodejs~//nodejs:repositories.bzl", - "ruleClassName": "node_repositories", - "attributes": { - "platform": "windows_amd64", - "node_version": "16.19.0" - } - }, - "nodejs": { - "bzlFile": "@@rules_nodejs~//nodejs/private:nodejs_repo_host_os_alias.bzl", - "ruleClassName": "nodejs_repo_host_os_alias", - "attributes": { - "user_node_repository_name": "nodejs" - } - }, - "nodejs_host": { - "bzlFile": "@@rules_nodejs~//nodejs/private:nodejs_repo_host_os_alias.bzl", - "ruleClassName": "nodejs_repo_host_os_alias", - "attributes": { - "user_node_repository_name": "nodejs" - } - }, - "nodejs_toolchains": { - "bzlFile": "@@rules_nodejs~//nodejs/private:toolchains_repo.bzl", - "ruleClassName": "toolchains_repo", - "attributes": { - "user_node_repository_name": "nodejs" + "user_node_repository_name": "nodejs" } } }, @@ -2798,8 +1874,8 @@ }, "@@rules_python~//python/private/bzlmod:pip.bzl%pip_internal": { "general": { - "bzlTransitiveDigest": "/9ui63Nm1hs0hLrSI41GQeUSUOB52YeVYBf/FBGjaUQ=", - "usagesDigest": "spYTfmH5Mw462zD7a6kcJXCuJA9q0cpZZc4yYSSxiPw=", + "bzlTransitiveDigest": "HhVh+gh1Hx2G9hPmU88Qzz2vyadG+CawqwqPo/UTYPU=", + "usagesDigest": "Ytl7Xydpzlx/9E3VePX4x952J6xMN3xDnNsv/9gX+Zw=", "recordedFileInputs": { "@@rules_python~//tools/publish/requirements.txt": "8ced1e640eab3ee44298590e5ad88cd612f5bf96245af1981709f7a8884a982b", "@@rules_python~//tools/publish/requirements_windows.txt": "0b7327c4f5751dc429bf53d21fc0797a7a0a6ac468ddcb38e238bed90ef0a7da", @@ -2808,32 +1884,7 @@ "recordedDirentsInputs": {}, "envVariables": {}, "generatedRepoSpecs": { - "rules_python_publish_deps_311_cffi_cp311_cp311_manylinux_2_17_aarch64_3548db28": { - "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", - "ruleClassName": "whl_library", - "attributes": { - "dep_template": "@rules_python_publish_deps//{name}:{target}", - "experimental_target_platforms": [ - "linux_aarch64", - "linux_arm", - "linux_ppc", - "linux_s390x", - "linux_x86_64" - ], - "filename": "cffi-1.15.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", - "isolated": true, - "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", - "quiet": true, - "repo": "rules_python_publish_deps_311", - "requirement": "cffi==1.15.1", - "sha256": "3548db281cd7d2561c9ad9984681c95f7b0e38881201e157833a2342c30d5e8c", - "timeout": 600, - "urls": [ - "https://files.pythonhosted.org/packages/91/bc/b7723c2fe7a22eee71d7edf2102cd43423d5f95ff3932ebaa2f82c7ec8d0/cffi-1.15.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl" - ] - } - }, - "rules_python_publish_deps_311_zipp_sdist_a7a22e05": { + "rules_python_publish_deps_311_six_py2_none_any_8abb2f1d": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -2848,20 +1899,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "zipp-3.11.0.tar.gz", + "filename": "six-1.16.0-py2.py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "zipp==3.11.0", - "sha256": "a7a22e05929290a67401440b39690ae6563279bced5f314609d9d03798f56766", + "requirement": "six==1.16.0", + "sha256": "8abb2f1d86890a2dfb989f9a77cfcfd3e47c2a354b01111771326f8aa26e0254", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/8e/b3/8b16a007184714f71157b1a71bbe632c5d66dd43bc8152b3c799b13881e1/zipp-3.11.0.tar.gz" + "https://files.pythonhosted.org/packages/d9/5a/e7c31adbe875f2abbb91bd84cf2dc52d792b5a01506781dbcf25c91daf11/six-1.16.0-py2.py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_urllib3_sdist_076907bf": { + "rules_python_publish_deps_311_six_sdist_1e61c374": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -2871,22 +1922,25 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64" + "linux_x86_64", + "osx_aarch64", + "osx_x86_64", + "windows_x86_64" ], - "filename": "urllib3-1.26.14.tar.gz", + "filename": "six-1.16.0.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "urllib3==1.26.14", - "sha256": "076907bf8fd355cde77728471316625a4d2f7e713c125f51953bb5b3eecf4f72", + "requirement": "six==1.16.0", + "sha256": "1e61c37477a1626458e36f7b1d82aa5c9b094fa4802892072e49de9c60c4c926", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/c5/52/fe421fb7364aa738b3506a2d99e4f3a56e079c0a798e9f4fa5e14c60922f/urllib3-1.26.14.tar.gz" + "https://files.pythonhosted.org/packages/71/39/171f1c67cd00715f190ba0b100d606d440a28c93c7714febeca8b79af85e/six-1.16.0.tar.gz" ] } }, - "rules_python_publish_deps_311_cryptography_cp37_abi3_manylinux_2_17_aarch64_afda76d8": { + "rules_python_publish_deps_311_cffi_cp311_cp311_manylinux_2_17_aarch64_3548db28": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -2898,16 +1952,16 @@ "linux_s390x", "linux_x86_64" ], - "filename": "cryptography-41.0.6-cp37-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", + "filename": "cffi-1.15.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "cryptography==41.0.6", - "sha256": "afda76d84b053923c27ede5edc1ed7d53e3c9f475ebaf63c68e69f1403c405a8", + "requirement": "cffi==1.15.1", + "sha256": "3548db281cd7d2561c9ad9984681c95f7b0e38881201e157833a2342c30d5e8c", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/88/bd/0c1dc2d29a6eed5ac0491d9b0ba3e118ac8d36b532bb812b3047e3b87a1e/cryptography-41.0.6-cp37-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl" + "https://files.pythonhosted.org/packages/91/bc/b7723c2fe7a22eee71d7edf2102cd43423d5f95ff3932ebaa2f82c7ec8d0/cffi-1.15.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl" ] } }, @@ -2936,35 +1990,7 @@ ] } }, - "rules_python_publish_deps_311_requests_py3_none_any_64299f49": { - "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", - "ruleClassName": "whl_library", - "attributes": { - "dep_template": "@rules_python_publish_deps//{name}:{target}", - "experimental_target_platforms": [ - "linux_aarch64", - "linux_arm", - "linux_ppc", - "linux_s390x", - "linux_x86_64", - "osx_aarch64", - "osx_x86_64", - "windows_x86_64" - ], - "filename": "requests-2.28.2-py3-none-any.whl", - "isolated": true, - "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", - "quiet": true, - "repo": "rules_python_publish_deps_311", - "requirement": "requests==2.28.2", - "sha256": "64299f4909223da747622c030b781c0d7811e359c37124b4bd368fb8c6518baa", - "timeout": 600, - "urls": [ - "https://files.pythonhosted.org/packages/d2/f4/274d1dbe96b41cf4e0efb70cbced278ffd61b5c7bb70338b62af94ccb25b/requests-2.28.2-py3-none-any.whl" - ] - } - }, - "rules_python_publish_deps_311_certifi_sdist_35824b4c": { + "rules_python_publish_deps_311_cffi_cp311_cp311_manylinux_2_17_x86_64_94411f22": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -2976,20 +2002,20 @@ "linux_s390x", "linux_x86_64" ], - "filename": "certifi-2022.12.7.tar.gz", + "filename": "cffi-1.15.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "certifi==2022.12.7", - "sha256": "35824b4c3a97115964b408844d64aa14db1cc518f6562e8d7261699d1350a9e3", + "requirement": "cffi==1.15.1", + "sha256": "94411f22c3985acaec6f83c6df553f2dbe17b698cc7f8ae751ff2237d96b9e3c", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/37/f7/2b1b0ec44fdc30a3d31dfebe52226be9ddc40cd6c0f34ffc8923ba423b69/certifi-2022.12.7.tar.gz" + "https://files.pythonhosted.org/packages/37/5a/c37631a86be838bdd84cc0259130942bf7e6e32f70f4cab95f479847fb91/cffi-1.15.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl" ] } }, - "rules_python_publish_deps_311_readme_renderer_py3_none_any_f67a16ca": { + "rules_python_publish_deps_311_cffi_cp311_cp311_musllinux_1_1_x86_64_cc4d65ae": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -2999,21 +2025,18 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64", - "osx_aarch64", - "osx_x86_64", - "windows_x86_64" + "linux_x86_64" ], - "filename": "readme_renderer-37.3-py3-none-any.whl", + "filename": "cffi-1.15.1-cp311-cp311-musllinux_1_1_x86_64.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "readme-renderer==37.3", - "sha256": "f67a16caedfa71eef48a31b39708637a6f4664c4394801a7b0d6432d13907343", + "requirement": "cffi==1.15.1", + "sha256": "cc4d65aeeaa04136a12677d3dd0b1c0c94dc43abac5860ab33cceb42b801c1e8", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/97/52/fd8a77d6f0a9ddeb26ed8fb334e01ac546106bf0c5b8e40dc826c5bd160f/readme_renderer-37.3-py3-none-any.whl" + "https://files.pythonhosted.org/packages/d3/56/3e94aa719ae96eeda8b68b3ec6e347e0a23168c6841dc276ccdcdadc9f32/cffi-1.15.1-cp311-cp311-musllinux_1_1_x86_64.whl" ] } }, @@ -3042,7 +2065,7 @@ ] } }, - "rules_python_publish_deps_311_cryptography_cp37_abi3_musllinux_1_1_aarch64_5daeb18e": { + "rules_python_publish_deps_311_idna_py3_none_any_90b77e79": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3052,22 +2075,25 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64" + "linux_x86_64", + "osx_aarch64", + "osx_x86_64", + "windows_x86_64" ], - "filename": "cryptography-41.0.6-cp37-abi3-musllinux_1_1_aarch64.whl", + "filename": "idna-3.4-py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "cryptography==41.0.6", - "sha256": "5daeb18e7886a358064a68dbcaf441c036cbdb7da52ae744e7b9207b04d3908c", + "requirement": "idna==3.4", + "sha256": "90b77e79eaa3eba6de819a0c442c0b4ceefc341a7a2ab77d7562bf49f425c5c2", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/07/68/d41ba60a16ff4e64965a857fcf2041f893362ae62c5b88d8b958196e2bed/cryptography-41.0.6-cp37-abi3-musllinux_1_1_aarch64.whl" + "https://files.pythonhosted.org/packages/fc/34/3030de6f1370931b9dbb4dad48f6ab1015ab1d32447850b9fc94e60097be/idna-3.4-py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_requests_toolbelt_py2_none_any_18565aa5": { + "rules_python_publish_deps_311_idna_sdist_814f528e": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3082,20 +2108,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "requests_toolbelt-0.10.1-py2.py3-none-any.whl", + "filename": "idna-3.4.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "requests-toolbelt==0.10.1", - "sha256": "18565aa58116d9951ac39baa288d3adb5b3ff975c4f25eee78555d89e8f247f7", + "requirement": "idna==3.4", + "sha256": "814f528e8dead7d329833b91c5faa87d60bf71824cd12a7530b5526063d02cb4", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/05/d3/bf87a36bff1cb88fd30a509fd366c70ec30676517ee791b2f77e0e29817a/requests_toolbelt-0.10.1-py2.py3-none-any.whl" + "https://files.pythonhosted.org/packages/8b/e1/43beb3d38dba6cb420cefa297822eac205a277ab43e5ba5d5c46faf96438/idna-3.4.tar.gz" ] } }, - "rules_python_publish_deps_311_cryptography_cp37_abi3_manylinux_2_28_x86_64_b648fe2a": { + "rules_python_publish_deps_311_rich_py3_none_any_7c963f0d": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3105,22 +2131,25 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64" + "linux_x86_64", + "osx_aarch64", + "osx_x86_64", + "windows_x86_64" ], - "filename": "cryptography-41.0.6-cp37-abi3-manylinux_2_28_x86_64.whl", + "filename": "rich-13.2.0-py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "cryptography==41.0.6", - "sha256": "b648fe2a45e426aaee684ddca2632f62ec4613ef362f4d681a9a6283d10e079d", + "requirement": "rich==13.2.0", + "sha256": "7c963f0d03819221e9ac561e1bc866e3f95a02248c1234daa48954e6d381c003", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/1e/7a/22192740f36448bb763846da291c13fa66dae92917b5a1cd032ea5dfe2d1/cryptography-41.0.6-cp37-abi3-manylinux_2_28_x86_64.whl" + "https://files.pythonhosted.org/packages/0e/cf/a6369a2aee266c2d7604230f083d4bd14b8f69bc69eb25b3da63b9f2f853/rich-13.2.0-py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_cffi_cp311_cp311_manylinux_2_17_x86_64_94411f22": { + "rules_python_publish_deps_311_rich_sdist_f1a00cdd": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3130,22 +2159,25 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64" + "linux_x86_64", + "osx_aarch64", + "osx_x86_64", + "windows_x86_64" ], - "filename": "cffi-1.15.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", + "filename": "rich-13.2.0.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "cffi==1.15.1", - "sha256": "94411f22c3985acaec6f83c6df553f2dbe17b698cc7f8ae751ff2237d96b9e3c", + "requirement": "rich==13.2.0", + "sha256": "f1a00cdd3eebf999a15d85ec498bfe0b1a77efe9b34f645768a54132ef444ac5", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/37/5a/c37631a86be838bdd84cc0259130942bf7e6e32f70f4cab95f479847fb91/cffi-1.15.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl" + "https://files.pythonhosted.org/packages/9e/5e/c3dc3ea32e2c14bfe46e48de954dd175bff76bcc549dd300acb9689521ae/rich-13.2.0.tar.gz" ] } }, - "rules_python_publish_deps_311_pygments_py3_none_any_fa7bd7bd": { + "rules_python_publish_deps_311_zipp_py3_none_any_83a28fcb": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3160,20 +2192,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "Pygments-2.14.0-py3-none-any.whl", + "filename": "zipp-3.11.0-py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "pygments==2.14.0", - "sha256": "fa7bd7bd2771287c0de303af8bfdfc731f51bd2c6a47ab69d117138893b82717", + "requirement": "zipp==3.11.0", + "sha256": "83a28fcb75844b5c0cdaf5aa4003c2d728c77e05f5aeabe8e95e56727005fbaa", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/0b/42/d9d95cc461f098f204cd20c85642ae40fbff81f74c300341b8d0e0df14e0/Pygments-2.14.0-py3-none-any.whl" + "https://files.pythonhosted.org/packages/d8/20/256eb3f3f437c575fb1a2efdce5e801a5ce3162ea8117da96c43e6ee97d8/zipp-3.11.0-py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_bleach_py3_none_any_33c16e33": { + "rules_python_publish_deps_311_zipp_sdist_a7a22e05": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3188,20 +2220,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "bleach-6.0.0-py3-none-any.whl", + "filename": "zipp-3.11.0.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "bleach==6.0.0", - "sha256": "33c16e3353dbd13028ab4799a0f89a83f113405c766e9c122df8a06f5b85b3f4", + "requirement": "zipp==3.11.0", + "sha256": "a7a22e05929290a67401440b39690ae6563279bced5f314609d9d03798f56766", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/ac/e2/dfcab68c9b2e7800c8f06b85c76e5f978d05b195a958daa9b1dda54a1db6/bleach-6.0.0-py3-none-any.whl" + "https://files.pythonhosted.org/packages/8e/b3/8b16a007184714f71157b1a71bbe632c5d66dd43bc8152b3c799b13881e1/zipp-3.11.0.tar.gz" ] } }, - "rules_python_publish_deps_311_keyring_py3_none_any_771ed2a9": { + "rules_python_publish_deps_311_mdurl_py3_none_any_84008a41": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3216,20 +2248,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "keyring-23.13.1-py3-none-any.whl", + "filename": "mdurl-0.1.2-py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "keyring==23.13.1", - "sha256": "771ed2a91909389ed6148631de678f82ddc73737d85a927f382a8a1b157898cd", + "requirement": "mdurl==0.1.2", + "sha256": "84008a41e51615a49fc9966191ff91509e3c40b939176e643fd50a5c2196b8f8", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/62/db/0e9a09b2b95986dcd73ac78be6ed2bd73ebe8bac65cba7add5b83eb9d899/keyring-23.13.1-py3-none-any.whl" + "https://files.pythonhosted.org/packages/b3/38/89ba8ad64ae25be8de66a6d463314cf1eb366222074cfda9ee839c56a4b4/mdurl-0.1.2-py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_jaraco_classes_sdist_89559fa5": { + "rules_python_publish_deps_311_mdurl_sdist_bb413d29": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3244,20 +2276,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "jaraco.classes-3.2.3.tar.gz", + "filename": "mdurl-0.1.2.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "jaraco-classes==3.2.3", - "sha256": "89559fa5c1d3c34eff6f631ad80bb21f378dbcbb35dd161fd2c6b93f5be2f98a", + "requirement": "mdurl==0.1.2", + "sha256": "bb413d29f5eea38f31dd4754dd7377d4465116fb207585f97bf925588687c1ba", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/bf/02/a956c9bfd2dfe60b30c065ed8e28df7fcf72b292b861dca97e951c145ef6/jaraco.classes-3.2.3.tar.gz" + "https://files.pythonhosted.org/packages/d6/54/cfe61301667036ec958cb99bd3efefba235e65cdeb9c84d24a8293ba1d90/mdurl-0.1.2.tar.gz" ] } }, - "rules_python_publish_deps_311_rich_py3_none_any_7c963f0d": { + "rules_python_publish_deps_311_twine_py3_none_any_929bc3c2": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3272,20 +2304,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "rich-13.2.0-py3-none-any.whl", + "filename": "twine-4.0.2-py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "rich==13.2.0", - "sha256": "7c963f0d03819221e9ac561e1bc866e3f95a02248c1234daa48954e6d381c003", + "requirement": "twine==4.0.2", + "sha256": "929bc3c280033347a00f847236564d1c52a3e61b1ac2516c97c48f3ceab756d8", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/0e/cf/a6369a2aee266c2d7604230f083d4bd14b8f69bc69eb25b3da63b9f2f853/rich-13.2.0-py3-none-any.whl" + "https://files.pythonhosted.org/packages/3a/38/a3f27a9e8ce45523d7d1e28c09e9085b61a98dab15d35ec086f36a44b37c/twine-4.0.2-py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_manylinux_2_17_s390x_8c7fe7af": { + "rules_python_publish_deps_311_twine_sdist_9e102ef5": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3300,20 +2332,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", + "filename": "twine-4.0.2.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "charset-normalizer==3.0.1", - "sha256": "8c7fe7afa480e3e82eed58e0ca89f751cd14d767638e2550c77a92a9e749c317", + "requirement": "twine==4.0.2", + "sha256": "9e102ef5fdd5a20661eb88fad46338806c3bd32cf1db729603fe3697b1bc83c8", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/df/c5/dd3a17a615775d0ffc3e12b0e47833d8b7e0a4871431dad87a3f92382a19/charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl" + "https://files.pythonhosted.org/packages/b7/1a/a7884359429d801cd63c2c5512ad0a337a509994b0e42d9696d4778d71f6/twine-4.0.2.tar.gz" ] } }, - "rules_python_publish_deps_311_secretstorage_sdist_2403533e": { + "rules_python_publish_deps_311_bleach_py3_none_any_33c16e33": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3323,22 +2355,25 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64" + "linux_x86_64", + "osx_aarch64", + "osx_x86_64", + "windows_x86_64" ], - "filename": "SecretStorage-3.3.3.tar.gz", + "filename": "bleach-6.0.0-py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "secretstorage==3.3.3", - "sha256": "2403533ef369eca6d2ba81718576c5e0f564d5cca1b58f73a8b23e7d4eeebd77", + "requirement": "bleach==6.0.0", + "sha256": "33c16e3353dbd13028ab4799a0f89a83f113405c766e9c122df8a06f5b85b3f4", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/53/a4/f48c9d79cb507ed1373477dbceaba7401fd8a23af63b837fa61f1dcd3691/SecretStorage-3.3.3.tar.gz" + "https://files.pythonhosted.org/packages/ac/e2/dfcab68c9b2e7800c8f06b85c76e5f978d05b195a958daa9b1dda54a1db6/bleach-6.0.0-py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_cryptography_cp37_abi3_musllinux_1_1_x86_64_068bc551": { + "rules_python_publish_deps_311_bleach_sdist_1a1a85c1": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3348,22 +2383,25 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64" + "linux_x86_64", + "osx_aarch64", + "osx_x86_64", + "windows_x86_64" ], - "filename": "cryptography-41.0.6-cp37-abi3-musllinux_1_1_x86_64.whl", + "filename": "bleach-6.0.0.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "cryptography==41.0.6", - "sha256": "068bc551698c234742c40049e46840843f3d98ad7ce265fd2bd4ec0d11306596", + "requirement": "bleach==6.0.0", + "sha256": "1a1a85c1595e07d8db14c5f09f09e6433502c51c595970edc090551f0db99414", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/a4/37/38d1340a8eb720dac78fde5d14742c6bb22a4b5f750ac869ef4eaaa795e0/cryptography-41.0.6-cp37-abi3-musllinux_1_1_x86_64.whl" + "https://files.pythonhosted.org/packages/7e/e6/d5f220ca638f6a25557a611860482cb6e54b2d97f0332966b1b005742e1f/bleach-6.0.0.tar.gz" ] } }, - "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_musllinux_1_1_ppc64le_5995f016": { + "rules_python_publish_deps_311_certifi_py3_none_any_4ad3232f": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3373,25 +2411,22 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64", - "osx_aarch64", - "osx_x86_64", - "windows_x86_64" + "linux_x86_64" ], - "filename": "charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_ppc64le.whl", + "filename": "certifi-2022.12.7-py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "charset-normalizer==3.0.1", - "sha256": "5995f0164fa7df59db4746112fec3f49c461dd6b31b841873443bdb077c13cfc", + "requirement": "certifi==2022.12.7", + "sha256": "4ad3232f5e926d6718ec31cfc1fcadfde020920e278684144551c91769c7bc18", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/86/eb/31c9025b4ed7eddd930c5f2ac269efb953de33140608c7539675d74a2081/charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_ppc64le.whl" + "https://files.pythonhosted.org/packages/71/4c/3db2b8021bd6f2f0ceb0e088d6b2d49147671f25832fb17970e9b583d742/certifi-2022.12.7-py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_more_itertools_sdist_5a6257e4": { + "rules_python_publish_deps_311_certifi_sdist_35824b4c": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3401,53 +2436,68 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64", + "linux_x86_64" + ], + "filename": "certifi-2022.12.7.tar.gz", + "isolated": true, + "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", + "quiet": true, + "repo": "rules_python_publish_deps_311", + "requirement": "certifi==2022.12.7", + "sha256": "35824b4c3a97115964b408844d64aa14db1cc518f6562e8d7261699d1350a9e3", + "timeout": 600, + "urls": [ + "https://files.pythonhosted.org/packages/37/f7/2b1b0ec44fdc30a3d31dfebe52226be9ddc40cd6c0f34ffc8923ba423b69/certifi-2022.12.7.tar.gz" + ] + } + }, + "rules_python_publish_deps_311_certifi_py3_none_any_92d60375": { + "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", + "ruleClassName": "whl_library", + "attributes": { + "dep_template": "@rules_python_publish_deps//{name}:{target}", + "experimental_target_platforms": [ "osx_aarch64", "osx_x86_64", "windows_x86_64" ], - "filename": "more-itertools-9.0.0.tar.gz", + "filename": "certifi-2023.7.22-py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "more-itertools==9.0.0", - "sha256": "5a6257e40878ef0520b1803990e3e22303a41b5714006c32a3fd8304b26ea1ab", + "requirement": "certifi==2023.7.22", + "sha256": "92d6037539857d8206b8f6ae472e8b77db8058fec5937a1ef3f54304089edbb9", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/13/b3/397aa9668da8b1f0c307bc474608653d46122ae0563d1d32f60e24fa0cbd/more-itertools-9.0.0.tar.gz" + "https://files.pythonhosted.org/packages/4c/dd/2234eab22353ffc7d94e8d13177aaa050113286e93e7b40eae01fbf7c3d9/certifi-2023.7.22-py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_importlib_metadata_py3_none_any_7efb448e": { + "rules_python_publish_deps_311_certifi_sdist_539cc1d1": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { "dep_template": "@rules_python_publish_deps//{name}:{target}", "experimental_target_platforms": [ - "linux_aarch64", - "linux_arm", - "linux_ppc", - "linux_s390x", - "linux_x86_64", "osx_aarch64", "osx_x86_64", "windows_x86_64" ], - "filename": "importlib_metadata-6.0.0-py3-none-any.whl", + "filename": "certifi-2023.7.22.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "importlib-metadata==6.0.0", - "sha256": "7efb448ec9a5e313a57655d35aa54cd3e01b7e1fbcf72dce1bf06119420f5bad", + "requirement": "certifi==2023.7.22", + "sha256": "539cc1d13202e33ca466e88b2807e29f4c13049d6d87031a3c110744495cb082", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/26/a7/9da7d5b23fc98ab3d424ac2c65613d63c1f401efb84ad50f2fa27b2caab4/importlib_metadata-6.0.0-py3-none-any.whl" + "https://files.pythonhosted.org/packages/98/98/c2ff18671db109c9f10ed27f5ef610ae05b73bd876664139cf95bd1429aa/certifi-2023.7.22.tar.gz" ] } }, - "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_win_amd64_9ab77acb": { + "rules_python_publish_deps_311_jeepney_py3_none_any_c0a454ad": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3457,25 +2507,22 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64", - "osx_aarch64", - "osx_x86_64", - "windows_x86_64" + "linux_x86_64" ], - "filename": "charset_normalizer-3.0.1-cp311-cp311-win_amd64.whl", + "filename": "jeepney-0.8.0-py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "charset-normalizer==3.0.1", - "sha256": "9ab77acb98eba3fd2a85cd160851816bfce6871d944d885febf012713f06659c", + "requirement": "jeepney==0.8.0", + "sha256": "c0a454ad016ca575060802ee4d590dd912e35c122fa04e70306de3d076cce755", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/2e/7b/5053a4a46fac017fd2aea3dc9abdd9983fd4cef153b6eb6aedcb0d7cb6e3/charset_normalizer-3.0.1-cp311-cp311-win_amd64.whl" + "https://files.pythonhosted.org/packages/ae/72/2a1e2290f1ab1e06f71f3d0f1646c9e4634e70e1d37491535e19266e8dc9/jeepney-0.8.0-py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_importlib_metadata_sdist_e354bede": { + "rules_python_publish_deps_311_jeepney_sdist_5efe48d2": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3485,25 +2532,22 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64", - "osx_aarch64", - "osx_x86_64", - "windows_x86_64" + "linux_x86_64" ], - "filename": "importlib_metadata-6.0.0.tar.gz", + "filename": "jeepney-0.8.0.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "importlib-metadata==6.0.0", - "sha256": "e354bedeb60efa6affdcc8ae121b73544a7aa74156d047311948f6d711cd378d", + "requirement": "jeepney==0.8.0", + "sha256": "5efe48d255973902f6badc3ce55e2aa6c5c3b3bc642059ef3a91247bcfcc5806", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/90/07/6397ad02d31bddf1841c9ad3ec30a693a3ff208e09c2ef45c9a8a5f85156/importlib_metadata-6.0.0.tar.gz" + "https://files.pythonhosted.org/packages/d6/f4/154cf374c2daf2020e05c3c6a03c91348d59b23c5366e968feb198306fdf/jeepney-0.8.0.tar.gz" ] } }, - "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_macosx_11_0_arm64_87701167": { + "rules_python_publish_deps_311_keyring_py3_none_any_771ed2a9": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3518,20 +2562,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "charset_normalizer-3.0.1-cp311-cp311-macosx_11_0_arm64.whl", + "filename": "keyring-23.13.1-py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "charset-normalizer==3.0.1", - "sha256": "87701167f2a5c930b403e9756fab1d31d4d4da52856143b609e30a1ce7160f3c", + "requirement": "keyring==23.13.1", + "sha256": "771ed2a91909389ed6148631de678f82ddc73737d85a927f382a8a1b157898cd", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/02/49/78b4c1bc8b1b0e0fc66fb31ce30d8302f10a1412ba75de72c57532f0beb0/charset_normalizer-3.0.1-cp311-cp311-macosx_11_0_arm64.whl" + "https://files.pythonhosted.org/packages/62/db/0e9a09b2b95986dcd73ac78be6ed2bd73ebe8bac65cba7add5b83eb9d899/keyring-23.13.1-py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_musllinux_1_1_x86_64_761e8904": { + "rules_python_publish_deps_311_keyring_sdist_ba2e15a9": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3546,20 +2590,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_x86_64.whl", + "filename": "keyring-23.13.1.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "charset-normalizer==3.0.1", - "sha256": "761e8904c07ad053d285670f36dd94e1b6ab7f16ce62b9805c475b7aa1cffde6", + "requirement": "keyring==23.13.1", + "sha256": "ba2e15a9b35e21908d0aaf4e0a47acc52d6ae33444df0da2b49d41a46ef6d678", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/82/49/ab81421d5aa25bc8535896a017c93204cb4051f2a4e72b1ad8f3b594e072/charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_x86_64.whl" + "https://files.pythonhosted.org/packages/55/fe/282f4c205add8e8bb3a1635cbbac59d6def2e0891b145aa553a0e40dd2d0/keyring-23.13.1.tar.gz" ] } }, - "rules_python_publish_deps_311_certifi_py3_none_any_4ad3232f": { + "rules_python_publish_deps_311_pkginfo_py3_none_any_4b7a555a": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3569,22 +2613,25 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64" + "linux_x86_64", + "osx_aarch64", + "osx_x86_64", + "windows_x86_64" ], - "filename": "certifi-2022.12.7-py3-none-any.whl", + "filename": "pkginfo-1.9.6-py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "certifi==2022.12.7", - "sha256": "4ad3232f5e926d6718ec31cfc1fcadfde020920e278684144551c91769c7bc18", + "requirement": "pkginfo==1.9.6", + "sha256": "4b7a555a6d5a22169fcc9cf7bfd78d296b0361adad412a346c1226849af5e546", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/71/4c/3db2b8021bd6f2f0ceb0e088d6b2d49147671f25832fb17970e9b583d742/certifi-2022.12.7-py3-none-any.whl" + "https://files.pythonhosted.org/packages/b3/f2/6e95c86a23a30fa205ea6303a524b20cbae27fbee69216377e3d95266406/pkginfo-1.9.6-py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_jeepney_py3_none_any_c0a454ad": { + "rules_python_publish_deps_311_pkginfo_sdist_8fd5896e": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3594,22 +2641,25 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64" + "linux_x86_64", + "osx_aarch64", + "osx_x86_64", + "windows_x86_64" ], - "filename": "jeepney-0.8.0-py3-none-any.whl", + "filename": "pkginfo-1.9.6.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "jeepney==0.8.0", - "sha256": "c0a454ad016ca575060802ee4d590dd912e35c122fa04e70306de3d076cce755", + "requirement": "pkginfo==1.9.6", + "sha256": "8fd5896e8718a4372f0ea9cc9d96f6417c9b986e23a4d116dda26b62cc29d046", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/ae/72/2a1e2290f1ab1e06f71f3d0f1646c9e4634e70e1d37491535e19266e8dc9/jeepney-0.8.0-py3-none-any.whl" + "https://files.pythonhosted.org/packages/b4/1c/89b38e431c20d6b2389ed8b3926c2ab72f58944733ba029354c6d9f69129/pkginfo-1.9.6.tar.gz" ] } }, - "rules_python_publish_deps_311_secretstorage_py3_none_any_f356e662": { + "rules_python_publish_deps_311_rfc3986_py2_none_any_50b1502b": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3619,22 +2669,25 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64" + "linux_x86_64", + "osx_aarch64", + "osx_x86_64", + "windows_x86_64" ], - "filename": "SecretStorage-3.3.3-py3-none-any.whl", + "filename": "rfc3986-2.0.0-py2.py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "secretstorage==3.3.3", - "sha256": "f356e6628222568e3af06f2eba8df495efa13b3b63081dafd4f7d9a7b7bc9f99", + "requirement": "rfc3986==2.0.0", + "sha256": "50b1502b60e289cb37883f3dfd34532b8873c7de9f49bb546641ce9cbd256ebd", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/54/24/b4293291fa1dd830f353d2cb163295742fa87f179fcc8a20a306a81978b7/SecretStorage-3.3.3-py3-none-any.whl" + "https://files.pythonhosted.org/packages/ff/9a/9afaade874b2fa6c752c36f1548f718b5b83af81ed9b76628329dab81c1b/rfc3986-2.0.0-py2.py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_idna_py3_none_any_90b77e79": { + "rules_python_publish_deps_311_rfc3986_sdist_97aacf9d": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3649,16 +2702,16 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "idna-3.4-py3-none-any.whl", + "filename": "rfc3986-2.0.0.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "idna==3.4", - "sha256": "90b77e79eaa3eba6de819a0c442c0b4ceefc341a7a2ab77d7562bf49f425c5c2", + "requirement": "rfc3986==2.0.0", + "sha256": "97aacf9dbd4bfd829baad6e6309fa6573aaf1be3f6fa735c8ab05e46cecb261c", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/fc/34/3030de6f1370931b9dbb4dad48f6ab1015ab1d32447850b9fc94e60097be/idna-3.4-py3-none-any.whl" + "https://files.pythonhosted.org/packages/85/40/1520d68bfa07ab5a6f065a186815fb6610c86fe957bc065754e47f7b0840/rfc3986-2.0.0.tar.gz" ] } }, @@ -3687,7 +2740,7 @@ ] } }, - "rules_python_publish_deps_311_charset_normalizer_py3_none_any_7e189e2e": { + "rules_python_publish_deps_311_urllib3_sdist_076907bf": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3697,74 +2750,68 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64", - "osx_aarch64", - "osx_x86_64", - "windows_x86_64" + "linux_x86_64" ], - "filename": "charset_normalizer-3.0.1-py3-none-any.whl", + "filename": "urllib3-1.26.14.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "charset-normalizer==3.0.1", - "sha256": "7e189e2e1d3ed2f4aebabd2d5b0f931e883676e51c7624826e0a4e5fe8a0bf24", + "requirement": "urllib3==1.26.14", + "sha256": "076907bf8fd355cde77728471316625a4d2f7e713c125f51953bb5b3eecf4f72", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/68/2b/02e9d6a98ddb73fa238d559a9edcc30b247b8dc4ee848b6184c936e99dc0/charset_normalizer-3.0.1-py3-none-any.whl" + "https://files.pythonhosted.org/packages/c5/52/fe421fb7364aa738b3506a2d99e4f3a56e079c0a798e9f4fa5e14c60922f/urllib3-1.26.14.tar.gz" ] } }, - "rules_python_publish_deps_311_twine_sdist_9e102ef5": { + "rules_python_publish_deps_311_urllib3_py2_none_any_34b97092": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { "dep_template": "@rules_python_publish_deps//{name}:{target}", "experimental_target_platforms": [ - "linux_aarch64", - "linux_arm", - "linux_ppc", - "linux_s390x", - "linux_x86_64", "osx_aarch64", "osx_x86_64", "windows_x86_64" ], - "filename": "twine-4.0.2.tar.gz", + "filename": "urllib3-1.26.18-py2.py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "twine==4.0.2", - "sha256": "9e102ef5fdd5a20661eb88fad46338806c3bd32cf1db729603fe3697b1bc83c8", + "requirement": "urllib3==1.26.18", + "sha256": "34b97092d7e0a3a8cf7cd10e386f401b3737364026c45e622aa02903dffe0f07", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/b7/1a/a7884359429d801cd63c2c5512ad0a337a509994b0e42d9696d4778d71f6/twine-4.0.2.tar.gz" + "https://files.pythonhosted.org/packages/b0/53/aa91e163dcfd1e5b82d8a890ecf13314e3e149c05270cc644581f77f17fd/urllib3-1.26.18-py2.py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_pywin32_ctypes_py2_none_any_9dc2d991": { + "rules_python_publish_deps_311_urllib3_sdist_f8ecc1bb": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { "dep_template": "@rules_python_publish_deps//{name}:{target}", "experimental_target_platforms": [ + "osx_aarch64", + "osx_x86_64", "windows_x86_64" ], - "filename": "pywin32_ctypes-0.2.0-py2.py3-none-any.whl", + "filename": "urllib3-1.26.18.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "pywin32-ctypes==0.2.0", - "sha256": "9dc2d991b3479cc2df15930958b674a48a227d5361d413827a4cfd0b5876fc98", + "requirement": "urllib3==1.26.18", + "sha256": "f8ecc1bba5667413457c529ab955bf8c67b45db799d159066261719e328580a0", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/9e/4b/3ab2720f1fa4b4bc924ef1932b842edf10007e4547ea8157b0b9fc78599a/pywin32_ctypes-0.2.0-py2.py3-none-any.whl" + "https://files.pythonhosted.org/packages/0c/39/64487bf07df2ed854cc06078c27c0d0abc59bd27b32232876e403c333a08/urllib3-1.26.18.tar.gz" ] } }, - "rules_python_publish_deps_311_pkginfo_py3_none_any_4b7a555a": { + "rules_python_publish_deps_311_docutils_py3_none_any_5e1de4d8": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3779,20 +2826,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "pkginfo-1.9.6-py3-none-any.whl", + "filename": "docutils-0.19-py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "pkginfo==1.9.6", - "sha256": "4b7a555a6d5a22169fcc9cf7bfd78d296b0361adad412a346c1226849af5e546", + "requirement": "docutils==0.19", + "sha256": "5e1de4d849fee02c63b040a4a3fd567f4ab104defd8a5511fbbc24a8a017efbc", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/b3/f2/6e95c86a23a30fa205ea6303a524b20cbae27fbee69216377e3d95266406/pkginfo-1.9.6-py3-none-any.whl" + "https://files.pythonhosted.org/packages/93/69/e391bd51bc08ed9141ecd899a0ddb61ab6465309f1eb470905c0c8868081/docutils-0.19-py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_cffi_cp311_cp311_musllinux_1_1_x86_64_cc4d65ae": { + "rules_python_publish_deps_311_docutils_sdist_33995a67": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3802,22 +2849,25 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64" + "linux_x86_64", + "osx_aarch64", + "osx_x86_64", + "windows_x86_64" ], - "filename": "cffi-1.15.1-cp311-cp311-musllinux_1_1_x86_64.whl", + "filename": "docutils-0.19.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "cffi==1.15.1", - "sha256": "cc4d65aeeaa04136a12677d3dd0b1c0c94dc43abac5860ab33cceb42b801c1e8", + "requirement": "docutils==0.19", + "sha256": "33995a6753c30b7f577febfc2c50411fec6aac7f7ffeb7c4cfe5991072dcf9e6", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/d3/56/3e94aa719ae96eeda8b68b3ec6e347e0a23168c6841dc276ccdcdadc9f32/cffi-1.15.1-cp311-cp311-musllinux_1_1_x86_64.whl" + "https://files.pythonhosted.org/packages/6b/5c/330ea8d383eb2ce973df34d1239b3b21e91cd8c865d21ff82902d952f91f/docutils-0.19.tar.gz" ] } }, - "rules_python_publish_deps_311_mdurl_py3_none_any_84008a41": { + "rules_python_publish_deps_311_pygments_py3_none_any_fa7bd7bd": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3832,20 +2882,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "mdurl-0.1.2-py3-none-any.whl", + "filename": "Pygments-2.14.0-py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "mdurl==0.1.2", - "sha256": "84008a41e51615a49fc9966191ff91509e3c40b939176e643fd50a5c2196b8f8", + "requirement": "pygments==2.14.0", + "sha256": "fa7bd7bd2771287c0de303af8bfdfc731f51bd2c6a47ab69d117138893b82717", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/b3/38/89ba8ad64ae25be8de66a6d463314cf1eb366222074cfda9ee839c56a4b4/mdurl-0.1.2-py3-none-any.whl" + "https://files.pythonhosted.org/packages/0b/42/d9d95cc461f098f204cd20c85642ae40fbff81f74c300341b8d0e0df14e0/Pygments-2.14.0-py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_more_itertools_py3_none_any_250e83d7": { + "rules_python_publish_deps_311_pygments_sdist_b3ed06a9": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3860,20 +2910,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "more_itertools-9.0.0-py3-none-any.whl", + "filename": "Pygments-2.14.0.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "more-itertools==9.0.0", - "sha256": "250e83d7e81d0c87ca6bd942e6aeab8cc9daa6096d12c5308f3f92fa5e5c1f41", + "requirement": "pygments==2.14.0", + "sha256": "b3ed06a9e8ac9a9aae5a6f5dbe78a8a58655d17b43b93c078f094ddc476ae297", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/5d/87/1ec3fcc09d2c04b977eabf8a1083222f82eaa2f46d5a4f85f403bf8e7b30/more_itertools-9.0.0-py3-none-any.whl" + "https://files.pythonhosted.org/packages/da/6a/c427c06913204e24de28de5300d3f0e809933f376e0b7df95194b2bb3f71/Pygments-2.14.0.tar.gz" ] } }, - "rules_python_publish_deps_311_mdurl_sdist_bb413d29": { + "rules_python_publish_deps_311_requests_py3_none_any_64299f49": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3888,20 +2938,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "mdurl-0.1.2.tar.gz", + "filename": "requests-2.28.2-py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "mdurl==0.1.2", - "sha256": "bb413d29f5eea38f31dd4754dd7377d4465116fb207585f97bf925588687c1ba", + "requirement": "requests==2.28.2", + "sha256": "64299f4909223da747622c030b781c0d7811e359c37124b4bd368fb8c6518baa", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/d6/54/cfe61301667036ec958cb99bd3efefba235e65cdeb9c84d24a8293ba1d90/mdurl-0.1.2.tar.gz" + "https://files.pythonhosted.org/packages/d2/f4/274d1dbe96b41cf4e0efb70cbced278ffd61b5c7bb70338b62af94ccb25b/requests-2.28.2-py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_keyring_sdist_ba2e15a9": { + "rules_python_publish_deps_311_requests_sdist_98b1b278": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3916,20 +2966,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "keyring-23.13.1.tar.gz", + "filename": "requests-2.28.2.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "keyring==23.13.1", - "sha256": "ba2e15a9b35e21908d0aaf4e0a47acc52d6ae33444df0da2b49d41a46ef6d678", + "requirement": "requests==2.28.2", + "sha256": "98b1b2782e3c6c4904938b84c0eb932721069dfdb9134313beff7c83c2df24bf", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/55/fe/282f4c205add8e8bb3a1635cbbac59d6def2e0891b145aa553a0e40dd2d0/keyring-23.13.1.tar.gz" + "https://files.pythonhosted.org/packages/9d/ee/391076f5937f0a8cdf5e53b701ffc91753e87b07d66bae4a09aa671897bf/requests-2.28.2.tar.gz" ] } }, - "rules_python_publish_deps_311_rfc3986_sdist_97aacf9d": { + "rules_python_publish_deps_311_pycparser_py2_none_any_8ee45429": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3939,25 +2989,22 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64", - "osx_aarch64", - "osx_x86_64", - "windows_x86_64" + "linux_x86_64" ], - "filename": "rfc3986-2.0.0.tar.gz", + "filename": "pycparser-2.21-py2.py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "rfc3986==2.0.0", - "sha256": "97aacf9dbd4bfd829baad6e6309fa6573aaf1be3f6fa735c8ab05e46cecb261c", + "requirement": "pycparser==2.21", + "sha256": "8ee45429555515e1f6b185e78100aea234072576aa43ab53aefcae078162fca9", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/85/40/1520d68bfa07ab5a6f065a186815fb6610c86fe957bc065754e47f7b0840/rfc3986-2.0.0.tar.gz" + "https://files.pythonhosted.org/packages/62/d5/5f610ebe421e85889f2e55e33b7f9a6795bd982198517d912eb1c76e1a53/pycparser-2.21-py2.py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_six_py2_none_any_8abb2f1d": { + "rules_python_publish_deps_311_pycparser_sdist_e644fdec": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3967,25 +3014,22 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64", - "osx_aarch64", - "osx_x86_64", - "windows_x86_64" + "linux_x86_64" ], - "filename": "six-1.16.0-py2.py3-none-any.whl", + "filename": "pycparser-2.21.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "six==1.16.0", - "sha256": "8abb2f1d86890a2dfb989f9a77cfcfd3e47c2a354b01111771326f8aa26e0254", + "requirement": "pycparser==2.21", + "sha256": "e644fdec12f7872f86c58ff790da456218b10f863970249516d60a5eaca77206", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/d9/5a/e7c31adbe875f2abbb91bd84cf2dc52d792b5a01506781dbcf25c91daf11/six-1.16.0-py2.py3-none-any.whl" + "https://files.pythonhosted.org/packages/5e/0b/95d387f5f4433cb0f53ff7ad859bd2c6051051cebbb564f139a999ab46de/pycparser-2.21.tar.gz" ] } }, - "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_manylinux_2_17_aarch64_14e76c0f": { + "rules_python_publish_deps_311_cryptography_cp37_abi3_musllinux_1_1_x86_64_068bc551": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -3995,25 +3039,22 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64", - "osx_aarch64", - "osx_x86_64", - "windows_x86_64" + "linux_x86_64" ], - "filename": "charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", + "filename": "cryptography-41.0.6-cp37-abi3-musllinux_1_1_x86_64.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "charset-normalizer==3.0.1", - "sha256": "14e76c0f23218b8f46c4d87018ca2e441535aed3632ca134b10239dfb6dadd6b", + "requirement": "cryptography==41.0.6", + "sha256": "068bc551698c234742c40049e46840843f3d98ad7ce265fd2bd4ec0d11306596", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/c0/4d/6b82099e3f25a9ed87431e2f51156c14f3a9ce8fad73880a3856cd95f1d5/charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl" + "https://files.pythonhosted.org/packages/a4/37/38d1340a8eb720dac78fde5d14742c6bb22a4b5f750ac869ef4eaaa795e0/cryptography-41.0.6-cp37-abi3-musllinux_1_1_x86_64.whl" ] } }, - "rules_python_publish_deps_311_readme_renderer_sdist_cd653186": { + "rules_python_publish_deps_311_cryptography_cp37_abi3_musllinux_1_1_aarch64_5daeb18e": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4023,25 +3064,22 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64", - "osx_aarch64", - "osx_x86_64", - "windows_x86_64" + "linux_x86_64" ], - "filename": "readme_renderer-37.3.tar.gz", + "filename": "cryptography-41.0.6-cp37-abi3-musllinux_1_1_aarch64.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "readme-renderer==37.3", - "sha256": "cd653186dfc73055656f090f227f5cb22a046d7f71a841dfa305f55c9a513273", + "requirement": "cryptography==41.0.6", + "sha256": "5daeb18e7886a358064a68dbcaf441c036cbdb7da52ae744e7b9207b04d3908c", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/81/c3/d20152fcd1986117b898f66928938f329d0c91ddc47f081c58e64e0f51dc/readme_renderer-37.3.tar.gz" + "https://files.pythonhosted.org/packages/07/68/d41ba60a16ff4e64965a857fcf2041f893362ae62c5b88d8b958196e2bed/cryptography-41.0.6-cp37-abi3-musllinux_1_1_aarch64.whl" ] } }, - "rules_python_publish_deps_311_markdown_it_py_py3_none_any_93de681e": { + "rules_python_publish_deps_311_cryptography_cp37_abi3_manylinux_2_17_aarch64_afda76d8": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4051,25 +3089,47 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64", - "osx_aarch64", - "osx_x86_64", - "windows_x86_64" + "linux_x86_64" + ], + "filename": "cryptography-41.0.6-cp37-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", + "isolated": true, + "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", + "quiet": true, + "repo": "rules_python_publish_deps_311", + "requirement": "cryptography==41.0.6", + "sha256": "afda76d84b053923c27ede5edc1ed7d53e3c9f475ebaf63c68e69f1403c405a8", + "timeout": 600, + "urls": [ + "https://files.pythonhosted.org/packages/88/bd/0c1dc2d29a6eed5ac0491d9b0ba3e118ac8d36b532bb812b3047e3b87a1e/cryptography-41.0.6-cp37-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl" + ] + } + }, + "rules_python_publish_deps_311_cryptography_cp37_abi3_manylinux_2_28_x86_64_b648fe2a": { + "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", + "ruleClassName": "whl_library", + "attributes": { + "dep_template": "@rules_python_publish_deps//{name}:{target}", + "experimental_target_platforms": [ + "linux_aarch64", + "linux_arm", + "linux_ppc", + "linux_s390x", + "linux_x86_64" ], - "filename": "markdown_it_py-2.1.0-py3-none-any.whl", + "filename": "cryptography-41.0.6-cp37-abi3-manylinux_2_28_x86_64.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "markdown-it-py==2.1.0", - "sha256": "93de681e5c021a432c63147656fe21790bc01231e0cd2da73626f1aa3ac0fe27", + "requirement": "cryptography==41.0.6", + "sha256": "b648fe2a45e426aaee684ddca2632f62ec4613ef362f4d681a9a6283d10e079d", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/f9/3f/ecd1b708973b9a3e4574b43cffc1ce8eb98696da34f1a1c44a68c3c0d737/markdown_it_py-2.1.0-py3-none-any.whl" + "https://files.pythonhosted.org/packages/1e/7a/22192740f36448bb763846da291c13fa66dae92917b5a1cd032ea5dfe2d1/cryptography-41.0.6-cp37-abi3-manylinux_2_28_x86_64.whl" ] } }, - "rules_python_publish_deps_311_six_sdist_1e61c374": { + "rules_python_publish_deps_311_cryptography_cp37_abi3_manylinux_2_17_x86_64_da46e2b5": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4079,48 +3139,47 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64", - "osx_aarch64", - "osx_x86_64", - "windows_x86_64" + "linux_x86_64" ], - "filename": "six-1.16.0.tar.gz", + "filename": "cryptography-41.0.6-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "six==1.16.0", - "sha256": "1e61c37477a1626458e36f7b1d82aa5c9b094fa4802892072e49de9c60c4c926", + "requirement": "cryptography==41.0.6", + "sha256": "da46e2b5df770070412c46f87bac0849b8d685c5f2679771de277a422c7d0b86", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/71/39/171f1c67cd00715f190ba0b100d606d440a28c93c7714febeca8b79af85e/six-1.16.0.tar.gz" + "https://files.pythonhosted.org/packages/ce/4e/54960380dda23ceb2027500e568aeafd6f06ce031847d7f2d3157f2bd12b/cryptography-41.0.6-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl" ] } }, - "rules_python_publish_deps_311_urllib3_py2_none_any_34b97092": { + "rules_python_publish_deps_311_cryptography_cp37_abi3_manylinux_2_28_aarch64_ff369dd1": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { "dep_template": "@rules_python_publish_deps//{name}:{target}", "experimental_target_platforms": [ - "osx_aarch64", - "osx_x86_64", - "windows_x86_64" + "linux_aarch64", + "linux_arm", + "linux_ppc", + "linux_s390x", + "linux_x86_64" ], - "filename": "urllib3-1.26.18-py2.py3-none-any.whl", + "filename": "cryptography-41.0.6-cp37-abi3-manylinux_2_28_aarch64.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "urllib3==1.26.18", - "sha256": "34b97092d7e0a3a8cf7cd10e386f401b3737364026c45e622aa02903dffe0f07", + "requirement": "cryptography==41.0.6", + "sha256": "ff369dd19e8fe0528b02e8df9f2aeb2479f89b1270d90f96a63500afe9af5cae", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/b0/53/aa91e163dcfd1e5b82d8a890ecf13314e3e149c05270cc644581f77f17fd/urllib3-1.26.18-py2.py3-none-any.whl" + "https://files.pythonhosted.org/packages/0e/dd/6043bf697d30dc4277cc1608af1145d6076fdd0f00283626bf916d7cde8f/cryptography-41.0.6-cp37-abi3-manylinux_2_28_aarch64.whl" ] } }, - "rules_python_publish_deps_311_twine_py3_none_any_929bc3c2": { + "rules_python_publish_deps_311_cryptography_sdist_422e3e31": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4130,25 +3189,22 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64", - "osx_aarch64", - "osx_x86_64", - "windows_x86_64" + "linux_x86_64" ], - "filename": "twine-4.0.2-py3-none-any.whl", + "filename": "cryptography-41.0.6.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "twine==4.0.2", - "sha256": "929bc3c280033347a00f847236564d1c52a3e61b1ac2516c97c48f3ceab756d8", + "requirement": "cryptography==41.0.6", + "sha256": "422e3e31d63743855e43e5a6fcc8b4acab860f560f9321b0ee6269cc7ed70cc3", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/3a/38/a3f27a9e8ce45523d7d1e28c09e9085b61a98dab15d35ec086f36a44b37c/twine-4.0.2-py3-none-any.whl" + "https://files.pythonhosted.org/packages/4d/b4/828991d82d3f1b6f21a0f8cfa54337ed33fdb52135f694130060839cfc33/cryptography-41.0.6.tar.gz" ] } }, - "rules_python_publish_deps_311_webencodings_sdist_b36a1c24": { + "rules_python_publish_deps_311_webencodings_py2_none_any_a0af1213": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4163,20 +3219,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "webencodings-0.5.1.tar.gz", + "filename": "webencodings-0.5.1-py2.py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", "requirement": "webencodings==0.5.1", - "sha256": "b36a1c245f2d304965eb4e0a82848379241dc04b865afcc4aab16748587e1923", + "sha256": "a0af1213f3c2226497a97e2b3aa01a7e4bee4f403f95be16fc9acd2947514a78", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/0b/02/ae6ceac1baeda530866a85075641cec12989bd8d31af6d5ab4a3e8c92f47/webencodings-0.5.1.tar.gz" + "https://files.pythonhosted.org/packages/f4/24/2a3e3df732393fed8b3ebf2ec078f05546de641fe1b667ee316ec1dcf3b7/webencodings-0.5.1-py2.py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_musllinux_1_1_s390x_4a8fcf28": { + "rules_python_publish_deps_311_webencodings_sdist_b36a1c24": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4191,20 +3247,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_s390x.whl", + "filename": "webencodings-0.5.1.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "charset-normalizer==3.0.1", - "sha256": "4a8fcf28c05c1f6d7e177a9a46a1c52798bfe2ad80681d275b10dcf317deaf0b", + "requirement": "webencodings==0.5.1", + "sha256": "b36a1c245f2d304965eb4e0a82848379241dc04b865afcc4aab16748587e1923", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/80/54/183163f9910936e57a60ee618f4f5cc91c2f8333ee2d4ebc6c50f6c8684d/charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_s390x.whl" + "https://files.pythonhosted.org/packages/0b/02/ae6ceac1baeda530866a85075641cec12989bd8d31af6d5ab4a3e8c92f47/webencodings-0.5.1.tar.gz" ] } }, - "rules_python_publish_deps_311_markdown_it_py_sdist_cf7e59fe": { + "rules_python_publish_deps_311_secretstorage_py3_none_any_f356e662": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4214,25 +3270,22 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64", - "osx_aarch64", - "osx_x86_64", - "windows_x86_64" + "linux_x86_64" ], - "filename": "markdown-it-py-2.1.0.tar.gz", + "filename": "SecretStorage-3.3.3-py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "markdown-it-py==2.1.0", - "sha256": "cf7e59fed14b5ae17c0006eff14a2d9a00ed5f3a846148153899a0224e2c07da", + "requirement": "secretstorage==3.3.3", + "sha256": "f356e6628222568e3af06f2eba8df495efa13b3b63081dafd4f7d9a7b7bc9f99", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/33/e9/ac8a93e9eda3891ecdfecf5e01c060bbd2c44d4e3e77efc83b9c7ce9db32/markdown-it-py-2.1.0.tar.gz" + "https://files.pythonhosted.org/packages/54/24/b4293291fa1dd830f353d2cb163295742fa87f179fcc8a20a306a81978b7/SecretStorage-3.3.3-py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_cryptography_cp37_abi3_manylinux_2_17_x86_64_da46e2b5": { + "rules_python_publish_deps_311_secretstorage_sdist_2403533e": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4244,20 +3297,20 @@ "linux_s390x", "linux_x86_64" ], - "filename": "cryptography-41.0.6-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", + "filename": "SecretStorage-3.3.3.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "cryptography==41.0.6", - "sha256": "da46e2b5df770070412c46f87bac0849b8d685c5f2679771de277a422c7d0b86", + "requirement": "secretstorage==3.3.3", + "sha256": "2403533ef369eca6d2ba81718576c5e0f564d5cca1b58f73a8b23e7d4eeebd77", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/ce/4e/54960380dda23ceb2027500e568aeafd6f06ce031847d7f2d3157f2bd12b/cryptography-41.0.6-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl" + "https://files.pythonhosted.org/packages/53/a4/f48c9d79cb507ed1373477dbceaba7401fd8a23af63b837fa61f1dcd3691/SecretStorage-3.3.3.tar.gz" ] } }, - "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_manylinux_2_17_x86_64_79909e27": { + "rules_python_publish_deps_311_jaraco_classes_py3_none_any_2353de32": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4272,20 +3325,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", + "filename": "jaraco.classes-3.2.3-py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "charset-normalizer==3.0.1", - "sha256": "79909e27e8e4fcc9db4addea88aa63f6423ebb171db091fb4373e3312cb6d603", + "requirement": "jaraco-classes==3.2.3", + "sha256": "2353de3288bc6b82120752201c6b1c1a14b058267fa424ed5ce5984e3b922158", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/d9/7a/60d45c9453212b30eebbf8b5cddbdef330eebddfcf335bce7920c43fb72e/charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl" + "https://files.pythonhosted.org/packages/60/28/220d3ae0829171c11e50dded4355d17824d60895285631d7eb9dee0ab5e5/jaraco.classes-3.2.3-py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_idna_sdist_814f528e": { + "rules_python_publish_deps_311_jaraco_classes_sdist_89559fa5": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4300,20 +3353,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "idna-3.4.tar.gz", + "filename": "jaraco.classes-3.2.3.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "idna==3.4", - "sha256": "814f528e8dead7d329833b91c5faa87d60bf71824cd12a7530b5526063d02cb4", + "requirement": "jaraco-classes==3.2.3", + "sha256": "89559fa5c1d3c34eff6f631ad80bb21f378dbcbb35dd161fd2c6b93f5be2f98a", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/8b/e1/43beb3d38dba6cb420cefa297822eac205a277ab43e5ba5d5c46faf96438/idna-3.4.tar.gz" + "https://files.pythonhosted.org/packages/bf/02/a956c9bfd2dfe60b30c065ed8e28df7fcf72b292b861dca97e951c145ef6/jaraco.classes-3.2.3.tar.gz" ] } }, - "rules_python_publish_deps_311_jaraco_classes_py3_none_any_2353de32": { + "rules_python_publish_deps_311_markdown_it_py_py3_none_any_93de681e": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4328,20 +3381,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "jaraco.classes-3.2.3-py3-none-any.whl", + "filename": "markdown_it_py-2.1.0-py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "jaraco-classes==3.2.3", - "sha256": "2353de3288bc6b82120752201c6b1c1a14b058267fa424ed5ce5984e3b922158", + "requirement": "markdown-it-py==2.1.0", + "sha256": "93de681e5c021a432c63147656fe21790bc01231e0cd2da73626f1aa3ac0fe27", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/60/28/220d3ae0829171c11e50dded4355d17824d60895285631d7eb9dee0ab5e5/jaraco.classes-3.2.3-py3-none-any.whl" + "https://files.pythonhosted.org/packages/f9/3f/ecd1b708973b9a3e4574b43cffc1ce8eb98696da34f1a1c44a68c3c0d737/markdown_it_py-2.1.0-py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_pycparser_py2_none_any_8ee45429": { + "rules_python_publish_deps_311_markdown_it_py_sdist_cf7e59fe": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4351,22 +3404,25 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64" + "linux_x86_64", + "osx_aarch64", + "osx_x86_64", + "windows_x86_64" ], - "filename": "pycparser-2.21-py2.py3-none-any.whl", + "filename": "markdown-it-py-2.1.0.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "pycparser==2.21", - "sha256": "8ee45429555515e1f6b185e78100aea234072576aa43ab53aefcae078162fca9", + "requirement": "markdown-it-py==2.1.0", + "sha256": "cf7e59fed14b5ae17c0006eff14a2d9a00ed5f3a846148153899a0224e2c07da", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/62/d5/5f610ebe421e85889f2e55e33b7f9a6795bd982198517d912eb1c76e1a53/pycparser-2.21-py2.py3-none-any.whl" + "https://files.pythonhosted.org/packages/33/e9/ac8a93e9eda3891ecdfecf5e01c060bbd2c44d4e3e77efc83b9c7ce9db32/markdown-it-py-2.1.0.tar.gz" ] } }, - "rules_python_publish_deps_311_cryptography_cp37_abi3_manylinux_2_28_aarch64_ff369dd1": { + "rules_python_publish_deps_311_more_itertools_py3_none_any_250e83d7": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4376,22 +3432,25 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64" + "linux_x86_64", + "osx_aarch64", + "osx_x86_64", + "windows_x86_64" ], - "filename": "cryptography-41.0.6-cp37-abi3-manylinux_2_28_aarch64.whl", + "filename": "more_itertools-9.0.0-py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "cryptography==41.0.6", - "sha256": "ff369dd19e8fe0528b02e8df9f2aeb2479f89b1270d90f96a63500afe9af5cae", + "requirement": "more-itertools==9.0.0", + "sha256": "250e83d7e81d0c87ca6bd942e6aeab8cc9daa6096d12c5308f3f92fa5e5c1f41", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/0e/dd/6043bf697d30dc4277cc1608af1145d6076fdd0f00283626bf916d7cde8f/cryptography-41.0.6-cp37-abi3-manylinux_2_28_aarch64.whl" + "https://files.pythonhosted.org/packages/5d/87/1ec3fcc09d2c04b977eabf8a1083222f82eaa2f46d5a4f85f403bf8e7b30/more_itertools-9.0.0-py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_rich_sdist_f1a00cdd": { + "rules_python_publish_deps_311_more_itertools_sdist_5a6257e4": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4406,41 +3465,48 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "rich-13.2.0.tar.gz", + "filename": "more-itertools-9.0.0.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "rich==13.2.0", - "sha256": "f1a00cdd3eebf999a15d85ec498bfe0b1a77efe9b34f645768a54132ef444ac5", + "requirement": "more-itertools==9.0.0", + "sha256": "5a6257e40878ef0520b1803990e3e22303a41b5714006c32a3fd8304b26ea1ab", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/9e/5e/c3dc3ea32e2c14bfe46e48de954dd175bff76bcc549dd300acb9689521ae/rich-13.2.0.tar.gz" + "https://files.pythonhosted.org/packages/13/b3/397aa9668da8b1f0c307bc474608653d46122ae0563d1d32f60e24fa0cbd/more-itertools-9.0.0.tar.gz" ] } }, - "rules_python_publish_deps_311_pywin32_ctypes_sdist_24ffc3b3": { + "rules_python_publish_deps_311_readme_renderer_py3_none_any_f67a16ca": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { "dep_template": "@rules_python_publish_deps//{name}:{target}", "experimental_target_platforms": [ + "linux_aarch64", + "linux_arm", + "linux_ppc", + "linux_s390x", + "linux_x86_64", + "osx_aarch64", + "osx_x86_64", "windows_x86_64" ], - "filename": "pywin32-ctypes-0.2.0.tar.gz", + "filename": "readme_renderer-37.3-py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "pywin32-ctypes==0.2.0", - "sha256": "24ffc3b341d457d48e8922352130cf2644024a4ff09762a2261fd34c36ee5942", + "requirement": "readme-renderer==37.3", + "sha256": "f67a16caedfa71eef48a31b39708637a6f4664c4394801a7b0d6432d13907343", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/7a/7d/0dbc4c99379452a819b0fb075a0ffbb98611df6b6d59f54db67367af5bc0/pywin32-ctypes-0.2.0.tar.gz" + "https://files.pythonhosted.org/packages/97/52/fd8a77d6f0a9ddeb26ed8fb334e01ac546106bf0c5b8e40dc826c5bd160f/readme_renderer-37.3-py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_pkginfo_sdist_8fd5896e": { + "rules_python_publish_deps_311_readme_renderer_sdist_cd653186": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4455,20 +3521,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "pkginfo-1.9.6.tar.gz", + "filename": "readme_renderer-37.3.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "pkginfo==1.9.6", - "sha256": "8fd5896e8718a4372f0ea9cc9d96f6417c9b986e23a4d116dda26b62cc29d046", + "requirement": "readme-renderer==37.3", + "sha256": "cd653186dfc73055656f090f227f5cb22a046d7f71a841dfa305f55c9a513273", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/b4/1c/89b38e431c20d6b2389ed8b3926c2ab72f58944733ba029354c6d9f69129/pkginfo-1.9.6.tar.gz" + "https://files.pythonhosted.org/packages/81/c3/d20152fcd1986117b898f66928938f329d0c91ddc47f081c58e64e0f51dc/readme_renderer-37.3.tar.gz" ] } }, - "rules_python_publish_deps_311_pygments_sdist_b3ed06a9": { + "rules_python_publish_deps_311_requests_toolbelt_py2_none_any_18565aa5": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4483,43 +3549,48 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "Pygments-2.14.0.tar.gz", + "filename": "requests_toolbelt-0.10.1-py2.py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "pygments==2.14.0", - "sha256": "b3ed06a9e8ac9a9aae5a6f5dbe78a8a58655d17b43b93c078f094ddc476ae297", + "requirement": "requests-toolbelt==0.10.1", + "sha256": "18565aa58116d9951ac39baa288d3adb5b3ff975c4f25eee78555d89e8f247f7", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/da/6a/c427c06913204e24de28de5300d3f0e809933f376e0b7df95194b2bb3f71/Pygments-2.14.0.tar.gz" + "https://files.pythonhosted.org/packages/05/d3/bf87a36bff1cb88fd30a509fd366c70ec30676517ee791b2f77e0e29817a/requests_toolbelt-0.10.1-py2.py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_certifi_py3_none_any_92d60375": { + "rules_python_publish_deps_311_requests_toolbelt_sdist_62e09f7f": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { "dep_template": "@rules_python_publish_deps//{name}:{target}", "experimental_target_platforms": [ + "linux_aarch64", + "linux_arm", + "linux_ppc", + "linux_s390x", + "linux_x86_64", "osx_aarch64", "osx_x86_64", "windows_x86_64" ], - "filename": "certifi-2023.7.22-py3-none-any.whl", + "filename": "requests-toolbelt-0.10.1.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "certifi==2023.7.22", - "sha256": "92d6037539857d8206b8f6ae472e8b77db8058fec5937a1ef3f54304089edbb9", + "requirement": "requests-toolbelt==0.10.1", + "sha256": "62e09f7ff5ccbda92772a29f394a49c3ad6cb181d568b1337626b2abb628a63d", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/4c/dd/2234eab22353ffc7d94e8d13177aaa050113286e93e7b40eae01fbf7c3d9/certifi-2023.7.22-py3-none-any.whl" + "https://files.pythonhosted.org/packages/0c/4c/07f01c6ac44f7784fa399137fbc8d0cdc1b5d35304e8c0f278ad82105b58/requests-toolbelt-0.10.1.tar.gz" ] } }, - "rules_python_publish_deps_311_zipp_py3_none_any_83a28fcb": { + "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_macosx_10_9_universal2_0298eaff": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4534,20 +3605,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "zipp-3.11.0-py3-none-any.whl", + "filename": "charset_normalizer-3.0.1-cp311-cp311-macosx_10_9_universal2.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "zipp==3.11.0", - "sha256": "83a28fcb75844b5c0cdaf5aa4003c2d728c77e05f5aeabe8e95e56727005fbaa", + "requirement": "charset-normalizer==3.0.1", + "sha256": "0298eafff88c99982a4cf66ba2efa1128e4ddaca0b05eec4c456bbc7db691d8d", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/d8/20/256eb3f3f437c575fb1a2efdce5e801a5ce3162ea8117da96c43e6ee97d8/zipp-3.11.0-py3-none-any.whl" + "https://files.pythonhosted.org/packages/37/00/ca188e0a2b3cd3184cdd2521b8765cf579327d128caa8aedc3dc7614020a/charset_normalizer-3.0.1-cp311-cp311-macosx_10_9_universal2.whl" ] } }, - "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_macosx_10_9_x86_64_a8d0fc94": { + "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_manylinux_2_17_ppc64le_0c0a5902": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4562,20 +3633,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "charset_normalizer-3.0.1-cp311-cp311-macosx_10_9_x86_64.whl", + "filename": "charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", "requirement": "charset-normalizer==3.0.1", - "sha256": "a8d0fc946c784ff7f7c3742310cc8a57c5c6dc31631269876a88b809dbeff3d3", + "sha256": "0c0a590235ccd933d9892c627dec5bc7511ce6ad6c1011fdf5b11363022746c1", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/90/59/941e2e5ae6828a688c6437ad16e026eb3606d0cfdd13ea5c9090980f3ffd/charset_normalizer-3.0.1-cp311-cp311-macosx_10_9_x86_64.whl" + "https://files.pythonhosted.org/packages/12/e5/aa09a1c39c3e444dd223d63e2c816c18ed78d035cff954143b2a539bdc9e/charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl" ] } }, - "rules_python_publish_deps_311_docutils_py3_none_any_5e1de4d8": { + "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_manylinux_2_17_aarch64_14e76c0f": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4590,20 +3661,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "docutils-0.19-py3-none-any.whl", + "filename": "charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "docutils==0.19", - "sha256": "5e1de4d849fee02c63b040a4a3fd567f4ab104defd8a5511fbbc24a8a017efbc", + "requirement": "charset-normalizer==3.0.1", + "sha256": "14e76c0f23218b8f46c4d87018ca2e441535aed3632ca134b10239dfb6dadd6b", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/93/69/e391bd51bc08ed9141ecd899a0ddb61ab6465309f1eb470905c0c8868081/docutils-0.19-py3-none-any.whl" + "https://files.pythonhosted.org/packages/c0/4d/6b82099e3f25a9ed87431e2f51156c14f3a9ce8fad73880a3856cd95f1d5/charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl" ] } }, - "rules_python_publish_deps_311_docutils_sdist_33995a67": { + "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_musllinux_1_1_s390x_4a8fcf28": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4618,20 +3689,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "docutils-0.19.tar.gz", + "filename": "charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_s390x.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "docutils==0.19", - "sha256": "33995a6753c30b7f577febfc2c50411fec6aac7f7ffeb7c4cfe5991072dcf9e6", + "requirement": "charset-normalizer==3.0.1", + "sha256": "4a8fcf28c05c1f6d7e177a9a46a1c52798bfe2ad80681d275b10dcf317deaf0b", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/6b/5c/330ea8d383eb2ce973df34d1239b3b21e91cd8c865d21ff82902d952f91f/docutils-0.19.tar.gz" + "https://files.pythonhosted.org/packages/80/54/183163f9910936e57a60ee618f4f5cc91c2f8333ee2d4ebc6c50f6c8684d/charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_s390x.whl" ] } }, - "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_macosx_10_9_universal2_0298eaff": { + "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_musllinux_1_1_ppc64le_5995f016": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4646,20 +3717,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "charset_normalizer-3.0.1-cp311-cp311-macosx_10_9_universal2.whl", + "filename": "charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_ppc64le.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", "requirement": "charset-normalizer==3.0.1", - "sha256": "0298eafff88c99982a4cf66ba2efa1128e4ddaca0b05eec4c456bbc7db691d8d", + "sha256": "5995f0164fa7df59db4746112fec3f49c461dd6b31b841873443bdb077c13cfc", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/37/00/ca188e0a2b3cd3184cdd2521b8765cf579327d128caa8aedc3dc7614020a/charset_normalizer-3.0.1-cp311-cp311-macosx_10_9_universal2.whl" + "https://files.pythonhosted.org/packages/86/eb/31c9025b4ed7eddd930c5f2ac269efb953de33140608c7539675d74a2081/charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_ppc64le.whl" ] } }, - "rules_python_publish_deps_311_jeepney_sdist_5efe48d2": { + "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_musllinux_1_1_aarch64_72966d1b": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4669,22 +3740,25 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64" + "linux_x86_64", + "osx_aarch64", + "osx_x86_64", + "windows_x86_64" ], - "filename": "jeepney-0.8.0.tar.gz", + "filename": "charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_aarch64.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "jeepney==0.8.0", - "sha256": "5efe48d255973902f6badc3ce55e2aa6c5c3b3bc642059ef3a91247bcfcc5806", + "requirement": "charset-normalizer==3.0.1", + "sha256": "72966d1b297c741541ca8cf1223ff262a6febe52481af742036a0b296e35fa5a", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/d6/f4/154cf374c2daf2020e05c3c6a03c91348d59b23c5366e968feb198306fdf/jeepney-0.8.0.tar.gz" + "https://files.pythonhosted.org/packages/01/ff/9ee4a44e8c32fe96dfc12daa42f29294608a55eadc88f327939327fb20fb/charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_aarch64.whl" ] } }, - "rules_python_publish_deps_311_requests_toolbelt_sdist_62e09f7f": { + "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_musllinux_1_1_x86_64_761e8904": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4699,43 +3773,48 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "requests-toolbelt-0.10.1.tar.gz", + "filename": "charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_x86_64.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "requests-toolbelt==0.10.1", - "sha256": "62e09f7ff5ccbda92772a29f394a49c3ad6cb181d568b1337626b2abb628a63d", + "requirement": "charset-normalizer==3.0.1", + "sha256": "761e8904c07ad053d285670f36dd94e1b6ab7f16ce62b9805c475b7aa1cffde6", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/0c/4c/07f01c6ac44f7784fa399137fbc8d0cdc1b5d35304e8c0f278ad82105b58/requests-toolbelt-0.10.1.tar.gz" + "https://files.pythonhosted.org/packages/82/49/ab81421d5aa25bc8535896a017c93204cb4051f2a4e72b1ad8f3b594e072/charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_x86_64.whl" ] } }, - "rules_python_publish_deps_311_certifi_sdist_539cc1d1": { + "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_manylinux_2_17_x86_64_79909e27": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { "dep_template": "@rules_python_publish_deps//{name}:{target}", "experimental_target_platforms": [ + "linux_aarch64", + "linux_arm", + "linux_ppc", + "linux_s390x", + "linux_x86_64", "osx_aarch64", "osx_x86_64", "windows_x86_64" ], - "filename": "certifi-2023.7.22.tar.gz", + "filename": "charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "certifi==2023.7.22", - "sha256": "539cc1d13202e33ca466e88b2807e29f4c13049d6d87031a3c110744495cb082", + "requirement": "charset-normalizer==3.0.1", + "sha256": "79909e27e8e4fcc9db4addea88aa63f6423ebb171db091fb4373e3312cb6d603", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/98/98/c2ff18671db109c9f10ed27f5ef610ae05b73bd876664139cf95bd1429aa/certifi-2023.7.22.tar.gz" + "https://files.pythonhosted.org/packages/d9/7a/60d45c9453212b30eebbf8b5cddbdef330eebddfcf335bce7920c43fb72e/charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl" ] } }, - "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_manylinux_2_17_ppc64le_0c0a5902": { + "rules_python_publish_deps_311_charset_normalizer_py3_none_any_7e189e2e": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4750,20 +3829,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", + "filename": "charset_normalizer-3.0.1-py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", "requirement": "charset-normalizer==3.0.1", - "sha256": "0c0a590235ccd933d9892c627dec5bc7511ce6ad6c1011fdf5b11363022746c1", + "sha256": "7e189e2e1d3ed2f4aebabd2d5b0f931e883676e51c7624826e0a4e5fe8a0bf24", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/12/e5/aa09a1c39c3e444dd223d63e2c816c18ed78d035cff954143b2a539bdc9e/charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl" + "https://files.pythonhosted.org/packages/68/2b/02e9d6a98ddb73fa238d559a9edcc30b247b8dc4ee848b6184c936e99dc0/charset_normalizer-3.0.1-py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_rfc3986_py2_none_any_50b1502b": { + "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_macosx_11_0_arm64_87701167": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4778,20 +3857,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "rfc3986-2.0.0-py2.py3-none-any.whl", + "filename": "charset_normalizer-3.0.1-cp311-cp311-macosx_11_0_arm64.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "rfc3986==2.0.0", - "sha256": "50b1502b60e289cb37883f3dfd34532b8873c7de9f49bb546641ce9cbd256ebd", + "requirement": "charset-normalizer==3.0.1", + "sha256": "87701167f2a5c930b403e9756fab1d31d4d4da52856143b609e30a1ce7160f3c", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/ff/9a/9afaade874b2fa6c752c36f1548f718b5b83af81ed9b76628329dab81c1b/rfc3986-2.0.0-py2.py3-none-any.whl" + "https://files.pythonhosted.org/packages/02/49/78b4c1bc8b1b0e0fc66fb31ce30d8302f10a1412ba75de72c57532f0beb0/charset_normalizer-3.0.1-cp311-cp311-macosx_11_0_arm64.whl" ] } }, - "rules_python_publish_deps_311_requests_sdist_98b1b278": { + "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_manylinux_2_17_s390x_8c7fe7af": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4806,60 +3885,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "requests-2.28.2.tar.gz", + "filename": "charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "requests==2.28.2", - "sha256": "98b1b2782e3c6c4904938b84c0eb932721069dfdb9134313beff7c83c2df24bf", + "requirement": "charset-normalizer==3.0.1", + "sha256": "8c7fe7afa480e3e82eed58e0ca89f751cd14d767638e2550c77a92a9e749c317", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/9d/ee/391076f5937f0a8cdf5e53b701ffc91753e87b07d66bae4a09aa671897bf/requests-2.28.2.tar.gz" + "https://files.pythonhosted.org/packages/df/c5/dd3a17a615775d0ffc3e12b0e47833d8b7e0a4871431dad87a3f92382a19/charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl" ] } }, - "rules_python_publish_deps": { - "bzlFile": "@@rules_python~//python/private/bzlmod:pip_repository.bzl", - "ruleClassName": "pip_repository", - "attributes": { - "repo_name": "rules_python_publish_deps", - "whl_map": { - "six": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"six-1.16.0-py2.py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_six_py2_none_any_8abb2f1d\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"six-1.16.0.tar.gz\",\"repo\":\"rules_python_publish_deps_311_six_sdist_1e61c374\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "cffi": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cffi-1.15.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl\",\"repo\":\"rules_python_publish_deps_311_cffi_cp311_cp311_manylinux_2_17_aarch64_3548db28\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cffi-1.15.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl\",\"repo\":\"rules_python_publish_deps_311_cffi_cp311_cp311_manylinux_2_17_ppc64le_91fc98ad\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cffi-1.15.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl\",\"repo\":\"rules_python_publish_deps_311_cffi_cp311_cp311_manylinux_2_17_x86_64_94411f22\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cffi-1.15.1-cp311-cp311-musllinux_1_1_x86_64.whl\",\"repo\":\"rules_python_publish_deps_311_cffi_cp311_cp311_musllinux_1_1_x86_64_cc4d65ae\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cffi-1.15.1.tar.gz\",\"repo\":\"rules_python_publish_deps_311_cffi_sdist_d400bfb9\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "idna": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"idna-3.4-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_idna_py3_none_any_90b77e79\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"idna-3.4.tar.gz\",\"repo\":\"rules_python_publish_deps_311_idna_sdist_814f528e\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "rich": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"rich-13.2.0-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_rich_py3_none_any_7c963f0d\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"rich-13.2.0.tar.gz\",\"repo\":\"rules_python_publish_deps_311_rich_sdist_f1a00cdd\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "zipp": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"zipp-3.11.0-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_zipp_py3_none_any_83a28fcb\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"zipp-3.11.0.tar.gz\",\"repo\":\"rules_python_publish_deps_311_zipp_sdist_a7a22e05\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "mdurl": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"mdurl-0.1.2-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_mdurl_py3_none_any_84008a41\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"mdurl-0.1.2.tar.gz\",\"repo\":\"rules_python_publish_deps_311_mdurl_sdist_bb413d29\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "twine": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"twine-4.0.2-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_twine_py3_none_any_929bc3c2\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"twine-4.0.2.tar.gz\",\"repo\":\"rules_python_publish_deps_311_twine_sdist_9e102ef5\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "bleach": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"bleach-6.0.0-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_bleach_py3_none_any_33c16e33\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"bleach-6.0.0.tar.gz\",\"repo\":\"rules_python_publish_deps_311_bleach_sdist_1a1a85c1\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "certifi": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"certifi-2022.12.7-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_certifi_py3_none_any_4ad3232f\",\"target_platforms\":[\"linux_aarch64\",\"linux_arm\",\"linux_ppc\",\"linux_s390x\",\"linux_x86_64\"],\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"certifi-2022.12.7.tar.gz\",\"repo\":\"rules_python_publish_deps_311_certifi_sdist_35824b4c\",\"target_platforms\":[\"linux_aarch64\",\"linux_arm\",\"linux_ppc\",\"linux_s390x\",\"linux_x86_64\"],\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"certifi-2023.7.22-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_certifi_py3_none_any_92d60375\",\"target_platforms\":[\"osx_aarch64\",\"osx_x86_64\",\"windows_x86_64\"],\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"certifi-2023.7.22.tar.gz\",\"repo\":\"rules_python_publish_deps_311_certifi_sdist_539cc1d1\",\"target_platforms\":[\"osx_aarch64\",\"osx_x86_64\",\"windows_x86_64\"],\"version\":\"3.11\"}]", - "jeepney": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"jeepney-0.8.0-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_jeepney_py3_none_any_c0a454ad\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"jeepney-0.8.0.tar.gz\",\"repo\":\"rules_python_publish_deps_311_jeepney_sdist_5efe48d2\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "keyring": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"keyring-23.13.1-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_keyring_py3_none_any_771ed2a9\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"keyring-23.13.1.tar.gz\",\"repo\":\"rules_python_publish_deps_311_keyring_sdist_ba2e15a9\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "pkginfo": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"pkginfo-1.9.6-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_pkginfo_py3_none_any_4b7a555a\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"pkginfo-1.9.6.tar.gz\",\"repo\":\"rules_python_publish_deps_311_pkginfo_sdist_8fd5896e\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "rfc3986": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"rfc3986-2.0.0-py2.py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_rfc3986_py2_none_any_50b1502b\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"rfc3986-2.0.0.tar.gz\",\"repo\":\"rules_python_publish_deps_311_rfc3986_sdist_97aacf9d\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "urllib3": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"urllib3-1.26.14-py2.py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_urllib3_py2_none_any_75edcdc2\",\"target_platforms\":[\"linux_aarch64\",\"linux_arm\",\"linux_ppc\",\"linux_s390x\",\"linux_x86_64\"],\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"urllib3-1.26.14.tar.gz\",\"repo\":\"rules_python_publish_deps_311_urllib3_sdist_076907bf\",\"target_platforms\":[\"linux_aarch64\",\"linux_arm\",\"linux_ppc\",\"linux_s390x\",\"linux_x86_64\"],\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"urllib3-1.26.18-py2.py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_urllib3_py2_none_any_34b97092\",\"target_platforms\":[\"osx_aarch64\",\"osx_x86_64\",\"windows_x86_64\"],\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"urllib3-1.26.18.tar.gz\",\"repo\":\"rules_python_publish_deps_311_urllib3_sdist_f8ecc1bb\",\"target_platforms\":[\"osx_aarch64\",\"osx_x86_64\",\"windows_x86_64\"],\"version\":\"3.11\"}]", - "docutils": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"docutils-0.19-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_docutils_py3_none_any_5e1de4d8\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"docutils-0.19.tar.gz\",\"repo\":\"rules_python_publish_deps_311_docutils_sdist_33995a67\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "pygments": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"Pygments-2.14.0-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_pygments_py3_none_any_fa7bd7bd\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"Pygments-2.14.0.tar.gz\",\"repo\":\"rules_python_publish_deps_311_pygments_sdist_b3ed06a9\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "requests": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"requests-2.28.2-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_requests_py3_none_any_64299f49\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"requests-2.28.2.tar.gz\",\"repo\":\"rules_python_publish_deps_311_requests_sdist_98b1b278\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "pycparser": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"pycparser-2.21-py2.py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_pycparser_py2_none_any_8ee45429\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"pycparser-2.21.tar.gz\",\"repo\":\"rules_python_publish_deps_311_pycparser_sdist_e644fdec\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "cryptography": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cryptography-41.0.6-cp37-abi3-musllinux_1_1_x86_64.whl\",\"repo\":\"rules_python_publish_deps_311_cryptography_cp37_abi3_musllinux_1_1_x86_64_068bc551\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cryptography-41.0.6-cp37-abi3-musllinux_1_1_aarch64.whl\",\"repo\":\"rules_python_publish_deps_311_cryptography_cp37_abi3_musllinux_1_1_aarch64_5daeb18e\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cryptography-41.0.6-cp37-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl\",\"repo\":\"rules_python_publish_deps_311_cryptography_cp37_abi3_manylinux_2_17_aarch64_afda76d8\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cryptography-41.0.6-cp37-abi3-manylinux_2_28_x86_64.whl\",\"repo\":\"rules_python_publish_deps_311_cryptography_cp37_abi3_manylinux_2_28_x86_64_b648fe2a\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cryptography-41.0.6-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl\",\"repo\":\"rules_python_publish_deps_311_cryptography_cp37_abi3_manylinux_2_17_x86_64_da46e2b5\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cryptography-41.0.6-cp37-abi3-manylinux_2_28_aarch64.whl\",\"repo\":\"rules_python_publish_deps_311_cryptography_cp37_abi3_manylinux_2_28_aarch64_ff369dd1\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cryptography-41.0.6.tar.gz\",\"repo\":\"rules_python_publish_deps_311_cryptography_sdist_422e3e31\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "webencodings": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"webencodings-0.5.1-py2.py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_webencodings_py2_none_any_a0af1213\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"webencodings-0.5.1.tar.gz\",\"repo\":\"rules_python_publish_deps_311_webencodings_sdist_b36a1c24\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "secretstorage": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"SecretStorage-3.3.3-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_secretstorage_py3_none_any_f356e662\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"SecretStorage-3.3.3.tar.gz\",\"repo\":\"rules_python_publish_deps_311_secretstorage_sdist_2403533e\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "jaraco_classes": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"jaraco.classes-3.2.3-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_jaraco_classes_py3_none_any_2353de32\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"jaraco.classes-3.2.3.tar.gz\",\"repo\":\"rules_python_publish_deps_311_jaraco_classes_sdist_89559fa5\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "markdown_it_py": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"markdown_it_py-2.1.0-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_markdown_it_py_py3_none_any_93de681e\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"markdown-it-py-2.1.0.tar.gz\",\"repo\":\"rules_python_publish_deps_311_markdown_it_py_sdist_cf7e59fe\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "more_itertools": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"more_itertools-9.0.0-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_more_itertools_py3_none_any_250e83d7\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"more-itertools-9.0.0.tar.gz\",\"repo\":\"rules_python_publish_deps_311_more_itertools_sdist_5a6257e4\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "readme_renderer": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"readme_renderer-37.3-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_readme_renderer_py3_none_any_f67a16ca\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"readme_renderer-37.3.tar.gz\",\"repo\":\"rules_python_publish_deps_311_readme_renderer_sdist_cd653186\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "requests_toolbelt": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"requests_toolbelt-0.10.1-py2.py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_requests_toolbelt_py2_none_any_18565aa5\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"requests-toolbelt-0.10.1.tar.gz\",\"repo\":\"rules_python_publish_deps_311_requests_toolbelt_sdist_62e09f7f\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "charset_normalizer": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-macosx_10_9_universal2.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_macosx_10_9_universal2_0298eaff\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_manylinux_2_17_ppc64le_0c0a5902\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_manylinux_2_17_aarch64_14e76c0f\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_s390x.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_musllinux_1_1_s390x_4a8fcf28\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_ppc64le.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_musllinux_1_1_ppc64le_5995f016\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_aarch64.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_musllinux_1_1_aarch64_72966d1b\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_x86_64.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_musllinux_1_1_x86_64_761e8904\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_manylinux_2_17_x86_64_79909e27\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_py3_none_any_7e189e2e\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-macosx_11_0_arm64.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_macosx_11_0_arm64_87701167\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_manylinux_2_17_s390x_8c7fe7af\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-win_amd64.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_win_amd64_9ab77acb\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-macosx_10_9_x86_64.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_macosx_10_9_x86_64_a8d0fc94\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset-normalizer-3.0.1.tar.gz\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_sdist_ebea339a\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "importlib_metadata": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"importlib_metadata-6.0.0-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_importlib_metadata_py3_none_any_7efb448e\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"importlib_metadata-6.0.0.tar.gz\",\"repo\":\"rules_python_publish_deps_311_importlib_metadata_sdist_e354bede\",\"target_platforms\":null,\"version\":\"3.11\"}]", - "pywin32_ctypes": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"pywin32_ctypes-0.2.0-py2.py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_pywin32_ctypes_py2_none_any_9dc2d991\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"pywin32-ctypes-0.2.0.tar.gz\",\"repo\":\"rules_python_publish_deps_311_pywin32_ctypes_sdist_24ffc3b3\",\"target_platforms\":null,\"version\":\"3.11\"}]" - }, - "default_version": "3.12", - "groups": {} - } - }, - "rules_python_publish_deps_311_webencodings_py2_none_any_a0af1213": { + "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_win_amd64_9ab77acb": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4874,20 +3913,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "webencodings-0.5.1-py2.py3-none-any.whl", + "filename": "charset_normalizer-3.0.1-cp311-cp311-win_amd64.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "webencodings==0.5.1", - "sha256": "a0af1213f3c2226497a97e2b3aa01a7e4bee4f403f95be16fc9acd2947514a78", + "requirement": "charset-normalizer==3.0.1", + "sha256": "9ab77acb98eba3fd2a85cd160851816bfce6871d944d885febf012713f06659c", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/f4/24/2a3e3df732393fed8b3ebf2ec078f05546de641fe1b667ee316ec1dcf3b7/webencodings-0.5.1-py2.py3-none-any.whl" + "https://files.pythonhosted.org/packages/2e/7b/5053a4a46fac017fd2aea3dc9abdd9983fd4cef153b6eb6aedcb0d7cb6e3/charset_normalizer-3.0.1-cp311-cp311-win_amd64.whl" ] } }, - "rules_python_publish_deps_311_charset_normalizer_sdist_ebea339a": { + "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_macosx_10_9_x86_64_a8d0fc94": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4902,20 +3941,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "charset-normalizer-3.0.1.tar.gz", + "filename": "charset_normalizer-3.0.1-cp311-cp311-macosx_10_9_x86_64.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", "requirement": "charset-normalizer==3.0.1", - "sha256": "ebea339af930f8ca5d7a699b921106c6e29c617fe9606fa7baa043c1cdae326f", + "sha256": "a8d0fc946c784ff7f7c3742310cc8a57c5c6dc31631269876a88b809dbeff3d3", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/96/d7/1675d9089a1f4677df5eb29c3f8b064aa1e70c1251a0a8a127803158942d/charset-normalizer-3.0.1.tar.gz" + "https://files.pythonhosted.org/packages/90/59/941e2e5ae6828a688c6437ad16e026eb3606d0cfdd13ea5c9090980f3ffd/charset_normalizer-3.0.1-cp311-cp311-macosx_10_9_x86_64.whl" ] } }, - "rules_python_publish_deps_311_cryptography_sdist_422e3e31": { + "rules_python_publish_deps_311_charset_normalizer_sdist_ebea339a": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4925,22 +3964,25 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64" + "linux_x86_64", + "osx_aarch64", + "osx_x86_64", + "windows_x86_64" ], - "filename": "cryptography-41.0.6.tar.gz", + "filename": "charset-normalizer-3.0.1.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "cryptography==41.0.6", - "sha256": "422e3e31d63743855e43e5a6fcc8b4acab860f560f9321b0ee6269cc7ed70cc3", + "requirement": "charset-normalizer==3.0.1", + "sha256": "ebea339af930f8ca5d7a699b921106c6e29c617fe9606fa7baa043c1cdae326f", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/4d/b4/828991d82d3f1b6f21a0f8cfa54337ed33fdb52135f694130060839cfc33/cryptography-41.0.6.tar.gz" + "https://files.pythonhosted.org/packages/96/d7/1675d9089a1f4677df5eb29c3f8b064aa1e70c1251a0a8a127803158942d/charset-normalizer-3.0.1.tar.gz" ] } }, - "rules_python_publish_deps_311_charset_normalizer_cp311_cp311_musllinux_1_1_aarch64_72966d1b": { + "rules_python_publish_deps_311_importlib_metadata_py3_none_any_7efb448e": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4955,20 +3997,20 @@ "osx_x86_64", "windows_x86_64" ], - "filename": "charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_aarch64.whl", + "filename": "importlib_metadata-6.0.0-py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "charset-normalizer==3.0.1", - "sha256": "72966d1b297c741541ca8cf1223ff262a6febe52481af742036a0b296e35fa5a", + "requirement": "importlib-metadata==6.0.0", + "sha256": "7efb448ec9a5e313a57655d35aa54cd3e01b7e1fbcf72dce1bf06119420f5bad", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/01/ff/9ee4a44e8c32fe96dfc12daa42f29294608a55eadc88f327939327fb20fb/charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_aarch64.whl" + "https://files.pythonhosted.org/packages/26/a7/9da7d5b23fc98ab3d424ac2c65613d63c1f401efb84ad50f2fa27b2caab4/importlib_metadata-6.0.0-py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_pycparser_sdist_e644fdec": { + "rules_python_publish_deps_311_importlib_metadata_sdist_e354bede": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { @@ -4978,71 +4020,105 @@ "linux_arm", "linux_ppc", "linux_s390x", - "linux_x86_64" + "linux_x86_64", + "osx_aarch64", + "osx_x86_64", + "windows_x86_64" ], - "filename": "pycparser-2.21.tar.gz", + "filename": "importlib_metadata-6.0.0.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "pycparser==2.21", - "sha256": "e644fdec12f7872f86c58ff790da456218b10f863970249516d60a5eaca77206", + "requirement": "importlib-metadata==6.0.0", + "sha256": "e354bedeb60efa6affdcc8ae121b73544a7aa74156d047311948f6d711cd378d", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/5e/0b/95d387f5f4433cb0f53ff7ad859bd2c6051051cebbb564f139a999ab46de/pycparser-2.21.tar.gz" + "https://files.pythonhosted.org/packages/90/07/6397ad02d31bddf1841c9ad3ec30a693a3ff208e09c2ef45c9a8a5f85156/importlib_metadata-6.0.0.tar.gz" ] } }, - "rules_python_publish_deps_311_bleach_sdist_1a1a85c1": { + "rules_python_publish_deps_311_pywin32_ctypes_py2_none_any_9dc2d991": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { "dep_template": "@rules_python_publish_deps//{name}:{target}", "experimental_target_platforms": [ - "linux_aarch64", - "linux_arm", - "linux_ppc", - "linux_s390x", - "linux_x86_64", - "osx_aarch64", - "osx_x86_64", "windows_x86_64" ], - "filename": "bleach-6.0.0.tar.gz", + "filename": "pywin32_ctypes-0.2.0-py2.py3-none-any.whl", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "bleach==6.0.0", - "sha256": "1a1a85c1595e07d8db14c5f09f09e6433502c51c595970edc090551f0db99414", + "requirement": "pywin32-ctypes==0.2.0", + "sha256": "9dc2d991b3479cc2df15930958b674a48a227d5361d413827a4cfd0b5876fc98", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/7e/e6/d5f220ca638f6a25557a611860482cb6e54b2d97f0332966b1b005742e1f/bleach-6.0.0.tar.gz" + "https://files.pythonhosted.org/packages/9e/4b/3ab2720f1fa4b4bc924ef1932b842edf10007e4547ea8157b0b9fc78599a/pywin32_ctypes-0.2.0-py2.py3-none-any.whl" ] } }, - "rules_python_publish_deps_311_urllib3_sdist_f8ecc1bb": { + "rules_python_publish_deps_311_pywin32_ctypes_sdist_24ffc3b3": { "bzlFile": "@@rules_python~//python/pip_install:pip_repository.bzl", "ruleClassName": "whl_library", "attributes": { "dep_template": "@rules_python_publish_deps//{name}:{target}", "experimental_target_platforms": [ - "osx_aarch64", - "osx_x86_64", "windows_x86_64" ], - "filename": "urllib3-1.26.18.tar.gz", + "filename": "pywin32-ctypes-0.2.0.tar.gz", "isolated": true, "python_interpreter_target": "@@rules_python~~python~python_3_11_host//:python", "quiet": true, "repo": "rules_python_publish_deps_311", - "requirement": "urllib3==1.26.18", - "sha256": "f8ecc1bba5667413457c529ab955bf8c67b45db799d159066261719e328580a0", + "requirement": "pywin32-ctypes==0.2.0", + "sha256": "24ffc3b341d457d48e8922352130cf2644024a4ff09762a2261fd34c36ee5942", "timeout": 600, "urls": [ - "https://files.pythonhosted.org/packages/0c/39/64487bf07df2ed854cc06078c27c0d0abc59bd27b32232876e403c333a08/urllib3-1.26.18.tar.gz" + "https://files.pythonhosted.org/packages/7a/7d/0dbc4c99379452a819b0fb075a0ffbb98611df6b6d59f54db67367af5bc0/pywin32-ctypes-0.2.0.tar.gz" ] } + }, + "rules_python_publish_deps": { + "bzlFile": "@@rules_python~//python/private/bzlmod:pip_repository.bzl", + "ruleClassName": "pip_repository", + "attributes": { + "repo_name": "rules_python_publish_deps", + "whl_map": { + "six": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"six-1.16.0-py2.py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_six_py2_none_any_8abb2f1d\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"six-1.16.0.tar.gz\",\"repo\":\"rules_python_publish_deps_311_six_sdist_1e61c374\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "cffi": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cffi-1.15.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl\",\"repo\":\"rules_python_publish_deps_311_cffi_cp311_cp311_manylinux_2_17_aarch64_3548db28\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cffi-1.15.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl\",\"repo\":\"rules_python_publish_deps_311_cffi_cp311_cp311_manylinux_2_17_ppc64le_91fc98ad\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cffi-1.15.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl\",\"repo\":\"rules_python_publish_deps_311_cffi_cp311_cp311_manylinux_2_17_x86_64_94411f22\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cffi-1.15.1-cp311-cp311-musllinux_1_1_x86_64.whl\",\"repo\":\"rules_python_publish_deps_311_cffi_cp311_cp311_musllinux_1_1_x86_64_cc4d65ae\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cffi-1.15.1.tar.gz\",\"repo\":\"rules_python_publish_deps_311_cffi_sdist_d400bfb9\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "idna": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"idna-3.4-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_idna_py3_none_any_90b77e79\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"idna-3.4.tar.gz\",\"repo\":\"rules_python_publish_deps_311_idna_sdist_814f528e\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "rich": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"rich-13.2.0-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_rich_py3_none_any_7c963f0d\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"rich-13.2.0.tar.gz\",\"repo\":\"rules_python_publish_deps_311_rich_sdist_f1a00cdd\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "zipp": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"zipp-3.11.0-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_zipp_py3_none_any_83a28fcb\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"zipp-3.11.0.tar.gz\",\"repo\":\"rules_python_publish_deps_311_zipp_sdist_a7a22e05\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "mdurl": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"mdurl-0.1.2-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_mdurl_py3_none_any_84008a41\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"mdurl-0.1.2.tar.gz\",\"repo\":\"rules_python_publish_deps_311_mdurl_sdist_bb413d29\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "twine": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"twine-4.0.2-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_twine_py3_none_any_929bc3c2\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"twine-4.0.2.tar.gz\",\"repo\":\"rules_python_publish_deps_311_twine_sdist_9e102ef5\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "bleach": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"bleach-6.0.0-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_bleach_py3_none_any_33c16e33\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"bleach-6.0.0.tar.gz\",\"repo\":\"rules_python_publish_deps_311_bleach_sdist_1a1a85c1\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "certifi": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"certifi-2022.12.7-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_certifi_py3_none_any_4ad3232f\",\"target_platforms\":[\"linux_aarch64\",\"linux_arm\",\"linux_ppc\",\"linux_s390x\",\"linux_x86_64\"],\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"certifi-2022.12.7.tar.gz\",\"repo\":\"rules_python_publish_deps_311_certifi_sdist_35824b4c\",\"target_platforms\":[\"linux_aarch64\",\"linux_arm\",\"linux_ppc\",\"linux_s390x\",\"linux_x86_64\"],\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"certifi-2023.7.22-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_certifi_py3_none_any_92d60375\",\"target_platforms\":[\"osx_aarch64\",\"osx_x86_64\",\"windows_x86_64\"],\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"certifi-2023.7.22.tar.gz\",\"repo\":\"rules_python_publish_deps_311_certifi_sdist_539cc1d1\",\"target_platforms\":[\"osx_aarch64\",\"osx_x86_64\",\"windows_x86_64\"],\"version\":\"3.11\"}]", + "jeepney": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"jeepney-0.8.0-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_jeepney_py3_none_any_c0a454ad\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"jeepney-0.8.0.tar.gz\",\"repo\":\"rules_python_publish_deps_311_jeepney_sdist_5efe48d2\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "keyring": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"keyring-23.13.1-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_keyring_py3_none_any_771ed2a9\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"keyring-23.13.1.tar.gz\",\"repo\":\"rules_python_publish_deps_311_keyring_sdist_ba2e15a9\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "pkginfo": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"pkginfo-1.9.6-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_pkginfo_py3_none_any_4b7a555a\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"pkginfo-1.9.6.tar.gz\",\"repo\":\"rules_python_publish_deps_311_pkginfo_sdist_8fd5896e\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "rfc3986": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"rfc3986-2.0.0-py2.py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_rfc3986_py2_none_any_50b1502b\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"rfc3986-2.0.0.tar.gz\",\"repo\":\"rules_python_publish_deps_311_rfc3986_sdist_97aacf9d\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "urllib3": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"urllib3-1.26.14-py2.py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_urllib3_py2_none_any_75edcdc2\",\"target_platforms\":[\"linux_aarch64\",\"linux_arm\",\"linux_ppc\",\"linux_s390x\",\"linux_x86_64\"],\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"urllib3-1.26.14.tar.gz\",\"repo\":\"rules_python_publish_deps_311_urllib3_sdist_076907bf\",\"target_platforms\":[\"linux_aarch64\",\"linux_arm\",\"linux_ppc\",\"linux_s390x\",\"linux_x86_64\"],\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"urllib3-1.26.18-py2.py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_urllib3_py2_none_any_34b97092\",\"target_platforms\":[\"osx_aarch64\",\"osx_x86_64\",\"windows_x86_64\"],\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"urllib3-1.26.18.tar.gz\",\"repo\":\"rules_python_publish_deps_311_urllib3_sdist_f8ecc1bb\",\"target_platforms\":[\"osx_aarch64\",\"osx_x86_64\",\"windows_x86_64\"],\"version\":\"3.11\"}]", + "docutils": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"docutils-0.19-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_docutils_py3_none_any_5e1de4d8\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"docutils-0.19.tar.gz\",\"repo\":\"rules_python_publish_deps_311_docutils_sdist_33995a67\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "pygments": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"Pygments-2.14.0-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_pygments_py3_none_any_fa7bd7bd\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"Pygments-2.14.0.tar.gz\",\"repo\":\"rules_python_publish_deps_311_pygments_sdist_b3ed06a9\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "requests": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"requests-2.28.2-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_requests_py3_none_any_64299f49\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"requests-2.28.2.tar.gz\",\"repo\":\"rules_python_publish_deps_311_requests_sdist_98b1b278\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "pycparser": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"pycparser-2.21-py2.py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_pycparser_py2_none_any_8ee45429\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"pycparser-2.21.tar.gz\",\"repo\":\"rules_python_publish_deps_311_pycparser_sdist_e644fdec\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "cryptography": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cryptography-41.0.6-cp37-abi3-musllinux_1_1_x86_64.whl\",\"repo\":\"rules_python_publish_deps_311_cryptography_cp37_abi3_musllinux_1_1_x86_64_068bc551\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cryptography-41.0.6-cp37-abi3-musllinux_1_1_aarch64.whl\",\"repo\":\"rules_python_publish_deps_311_cryptography_cp37_abi3_musllinux_1_1_aarch64_5daeb18e\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cryptography-41.0.6-cp37-abi3-manylinux_2_17_aarch64.manylinux2014_aarch64.whl\",\"repo\":\"rules_python_publish_deps_311_cryptography_cp37_abi3_manylinux_2_17_aarch64_afda76d8\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cryptography-41.0.6-cp37-abi3-manylinux_2_28_x86_64.whl\",\"repo\":\"rules_python_publish_deps_311_cryptography_cp37_abi3_manylinux_2_28_x86_64_b648fe2a\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cryptography-41.0.6-cp37-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl\",\"repo\":\"rules_python_publish_deps_311_cryptography_cp37_abi3_manylinux_2_17_x86_64_da46e2b5\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cryptography-41.0.6-cp37-abi3-manylinux_2_28_aarch64.whl\",\"repo\":\"rules_python_publish_deps_311_cryptography_cp37_abi3_manylinux_2_28_aarch64_ff369dd1\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"cryptography-41.0.6.tar.gz\",\"repo\":\"rules_python_publish_deps_311_cryptography_sdist_422e3e31\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "webencodings": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"webencodings-0.5.1-py2.py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_webencodings_py2_none_any_a0af1213\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"webencodings-0.5.1.tar.gz\",\"repo\":\"rules_python_publish_deps_311_webencodings_sdist_b36a1c24\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "secretstorage": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"SecretStorage-3.3.3-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_secretstorage_py3_none_any_f356e662\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"SecretStorage-3.3.3.tar.gz\",\"repo\":\"rules_python_publish_deps_311_secretstorage_sdist_2403533e\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "jaraco_classes": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"jaraco.classes-3.2.3-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_jaraco_classes_py3_none_any_2353de32\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"jaraco.classes-3.2.3.tar.gz\",\"repo\":\"rules_python_publish_deps_311_jaraco_classes_sdist_89559fa5\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "markdown_it_py": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"markdown_it_py-2.1.0-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_markdown_it_py_py3_none_any_93de681e\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"markdown-it-py-2.1.0.tar.gz\",\"repo\":\"rules_python_publish_deps_311_markdown_it_py_sdist_cf7e59fe\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "more_itertools": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"more_itertools-9.0.0-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_more_itertools_py3_none_any_250e83d7\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"more-itertools-9.0.0.tar.gz\",\"repo\":\"rules_python_publish_deps_311_more_itertools_sdist_5a6257e4\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "readme_renderer": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"readme_renderer-37.3-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_readme_renderer_py3_none_any_f67a16ca\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"readme_renderer-37.3.tar.gz\",\"repo\":\"rules_python_publish_deps_311_readme_renderer_sdist_cd653186\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "requests_toolbelt": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"requests_toolbelt-0.10.1-py2.py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_requests_toolbelt_py2_none_any_18565aa5\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"requests-toolbelt-0.10.1.tar.gz\",\"repo\":\"rules_python_publish_deps_311_requests_toolbelt_sdist_62e09f7f\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "charset_normalizer": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-macosx_10_9_universal2.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_macosx_10_9_universal2_0298eaff\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_manylinux_2_17_ppc64le_0c0a5902\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_manylinux_2_17_aarch64_14e76c0f\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_s390x.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_musllinux_1_1_s390x_4a8fcf28\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_ppc64le.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_musllinux_1_1_ppc64le_5995f016\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_aarch64.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_musllinux_1_1_aarch64_72966d1b\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_x86_64.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_musllinux_1_1_x86_64_761e8904\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_manylinux_2_17_x86_64_79909e27\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_py3_none_any_7e189e2e\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-macosx_11_0_arm64.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_macosx_11_0_arm64_87701167\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_manylinux_2_17_s390x_8c7fe7af\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-win_amd64.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_win_amd64_9ab77acb\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset_normalizer-3.0.1-cp311-cp311-macosx_10_9_x86_64.whl\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_cp311_cp311_macosx_10_9_x86_64_a8d0fc94\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"charset-normalizer-3.0.1.tar.gz\",\"repo\":\"rules_python_publish_deps_311_charset_normalizer_sdist_ebea339a\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "importlib_metadata": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"importlib_metadata-6.0.0-py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_importlib_metadata_py3_none_any_7efb448e\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"importlib_metadata-6.0.0.tar.gz\",\"repo\":\"rules_python_publish_deps_311_importlib_metadata_sdist_e354bede\",\"target_platforms\":null,\"version\":\"3.11\"}]", + "pywin32_ctypes": "[{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"pywin32_ctypes-0.2.0-py2.py3-none-any.whl\",\"repo\":\"rules_python_publish_deps_311_pywin32_ctypes_py2_none_any_9dc2d991\",\"target_platforms\":null,\"version\":\"3.11\"},{\"config_setting\":\"//_config:is_python_3.11\",\"filename\":\"pywin32-ctypes-0.2.0.tar.gz\",\"repo\":\"rules_python_publish_deps_311_pywin32_ctypes_sdist_24ffc3b3\",\"target_platforms\":null,\"version\":\"3.11\"}]" + }, + "default_version": "3.12", + "groups": {} + } } }, "recordedRepoMappingEntries": [ @@ -5151,6 +4227,11 @@ "pythons_hub", "rules_python~~python~pythons_hub" ], + [ + "rules_python~~python~pythons_hub", + "python_3_10_host", + "rules_python~~python~python_3_10_host" + ], [ "rules_python~~python~pythons_hub", "python_3_11_host", @@ -5160,14 +4241,24 @@ "rules_python~~python~pythons_hub", "python_3_12_host", "rules_python~~python~python_3_12_host" + ], + [ + "rules_python~~python~pythons_hub", + "python_3_8_host", + "rules_python~~python~python_3_8_host" + ], + [ + "rules_python~~python~pythons_hub", + "python_3_9_host", + "rules_python~~python~python_3_9_host" ] ] } }, "@@rules_rust~//crate_universe/private/module_extensions:cargo_bazel_bootstrap.bzl%cargo_bazel_bootstrap": { "general": { - "bzlTransitiveDigest": "m6F2qRqOBDs5A6SrcVYRfwgtk1LmyMhyFuO30tUiqf8=", - "usagesDigest": "pKIPfG2qGDQGtWz+JyxH3I8o4VLbXQjm6ZggdFBBFbU=", + "bzlTransitiveDigest": "ItqpUbtb76p93mlsRnbmY2Q5D4g1+fON7mwlSq3+nr8=", + "usagesDigest": "77j42jOHe6DTKp5n5N7Vg7YAYJ1O1eJBbpYdH/Yws1E=", "recordedFileInputs": {}, "recordedDirentsInputs": {}, "envVariables": {}, @@ -5260,7 +4351,7 @@ }, "@@rules_rust~//rust:extensions.bzl%rust": { "general": { - "bzlTransitiveDigest": "e2mzUPIPy3NKky6vadyIVx99U0LKUcD6IGN5l5tNdpY=", + "bzlTransitiveDigest": "cu87e2DBOEvyxYRiP9I1Fn8YCkA9jLwDzC9aFrdcXqA=", "usagesDigest": "kIFjD0j3UyHq8EH2APrpeiquQWPbz1TAnKvNNG68hSY=", "recordedFileInputs": {}, "recordedDirentsInputs": {}, From e52d194136deeadf18bf51944d8a47b0d5ac54c8 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Fri, 6 Dec 2024 02:38:50 +0000 Subject: [PATCH 080/229] cluster: use scoped enum In C++23 we check for scoped enums in serde, we need to make this enum a class enum. --- src/v/cluster/tm_stm_types.cc | 35 ++++++++++++++-------------- src/v/cluster/tm_stm_types.h | 2 +- src/v/cluster/tx_gateway_frontend.cc | 32 ++++++++++++------------- src/v/cluster/tx_protocol_types.cc | 2 +- src/v/cluster/tx_protocol_types.h | 2 +- 5 files changed, 37 insertions(+), 36 deletions(-) diff --git a/src/v/cluster/tm_stm_types.cc b/src/v/cluster/tm_stm_types.cc index 66842ca1f52e..513861a88a9d 100644 --- a/src/v/cluster/tm_stm_types.cc +++ b/src/v/cluster/tm_stm_types.cc @@ -38,41 +38,41 @@ bool is_state_transition_valid( * precursor of the requested one. */ switch (target_status) { - case empty: + case tx_status::empty: // ready is an initial state a transaction can never go back to that // state return is_one_of(current.status, tx_status::empty); - case ongoing: + case tx_status::ongoing: return is_one_of(current.status, tx_status::empty, tx_status::ongoing); - case preparing_commit: + case tx_status::preparing_commit: return is_one_of( current.status, tx_status::empty, tx_status::ongoing, tx_status::preparing_commit); - case completed_commit: + case tx_status::completed_commit: return is_one_of( current.status, tx_status::preparing_commit, tx_status::completed_commit); - case preparing_abort: + case tx_status::preparing_abort: return is_one_of( current.status, tx_status::empty, tx_status::ongoing, tx_status::preparing_abort); - case preparing_internal_abort: + case tx_status::preparing_internal_abort: return is_one_of( current.status, tx_status::ongoing, tx_status::preparing_internal_abort); - case tombstone: + case tx_status::tombstone: return is_one_of( current.status, tx_status::tombstone, tx_status::completed_commit, tx_status::completed_abort); - case completed_abort: + case tx_status::completed_abort: return is_one_of( current.status, tx_status::preparing_internal_abort, @@ -84,7 +84,8 @@ bool is_state_transition_valid( } bool tx_metadata::is_finished() const { - return status == completed_commit || status == completed_abort; + return status == tx_status::completed_commit + || status == tx_status::completed_abort; } std::string_view tx_metadata::get_status() const { @@ -164,21 +165,21 @@ tx_metadata::try_update_status(tx_status requested) { std::ostream& operator<<(std::ostream& o, tx_status status) { switch (status) { - case ongoing: + case tx_status::ongoing: return o << "ongoing"; - case preparing_abort: + case tx_status::preparing_abort: return o << "preparing_abort"; - case preparing_commit: + case tx_status::preparing_commit: return o << "preparing_commit"; - case completed_commit: + case tx_status::completed_commit: return o << "completed_commit"; - case preparing_internal_abort: + case tx_status::preparing_internal_abort: return o << "expired"; - case empty: + case tx_status::empty: return o << "empty"; - case tombstone: + case tx_status::tombstone: return o << "tombstone"; - case completed_abort: + case tx_status::completed_abort: return o << "completed_abort"; } } diff --git a/src/v/cluster/tm_stm_types.h b/src/v/cluster/tm_stm_types.h index c3e217462bad..1e10e720624c 100644 --- a/src/v/cluster/tm_stm_types.h +++ b/src/v/cluster/tm_stm_types.h @@ -58,7 +58,7 @@ namespace cluster { * | | * +---------------+ */ -enum tx_status : int32_t { +enum class tx_status : int32_t { /** * When transactional id to producer id mapping is added to coordinator it * starts in this state diff --git a/src/v/cluster/tx_gateway_frontend.cc b/src/v/cluster/tx_gateway_frontend.cc index b8c92b1f3e7e..d9ad91b5220a 100644 --- a/src/v/cluster/tx_gateway_frontend.cc +++ b/src/v/cluster/tx_gateway_frontend.cc @@ -577,9 +577,9 @@ ss::future tx_gateway_frontend::do_try_abort( } vlog(txlog.info, "[tx_id={}] found transaction {} to abort", tx_id, tx); switch (tx.status) { - case empty: + case tx_status::empty: [[fallthrough]]; - case ongoing: { + case tx_status::ongoing: { vlog(txlog.trace, "[tx_id={}] aborting transaction: {}", tx_id, tx); auto killed_tx = co_await stm->update_transaction_status( term, tx.id, tx_status::preparing_internal_abort); @@ -595,22 +595,22 @@ ss::future tx_gateway_frontend::do_try_abort( } co_return try_abort_reply::make_aborted(); } - case preparing_commit: + case tx_status::preparing_commit: [[fallthrough]]; - case completed_commit: + case tx_status::completed_commit: vlog( txlog.trace, "[tx_id={}] transaction: {} is already committed", tx_id, tx); co_return try_abort_reply::make_committed(); - case preparing_abort: + case tx_status::preparing_abort: [[fallthrough]]; - case preparing_internal_abort: + case tx_status::preparing_internal_abort: [[fallthrough]]; - case completed_abort: + case tx_status::completed_abort: [[fallthrough]]; - case tombstone: + case tx_status::tombstone: vlog( txlog.trace, "[tx_id={}] transaction: {} is already aborted", @@ -1005,7 +1005,7 @@ ss::future tx_gateway_frontend::do_init_tm_tx( } switch (tx.status) { - case ongoing: { + case tx_status::ongoing: { vlog(txlog.info, "[tx_id={}] tx is ongoing, aborting", tx_id); auto abort_result = co_await do_abort_tm_tx(term, stm, tx, timeout); if (!abort_result) { @@ -1019,10 +1019,10 @@ ss::future tx_gateway_frontend::do_init_tm_tx( } co_return init_tm_tx_reply{tx::errc::concurrent_transactions}; } - case empty: - case tombstone: - case completed_commit: - case completed_abort: { + case tx_status::empty: + case tx_status::tombstone: + case tx_status::completed_commit: + case tx_status::completed_abort: { co_return co_await increase_producer_epoch( tx.id, tx.pid, @@ -1033,9 +1033,9 @@ ss::future tx_gateway_frontend::do_init_tm_tx( transaction_timeout_ms, timeout); } - case preparing_abort: - case preparing_internal_abort: - case preparing_commit: + case tx_status::preparing_abort: + case tx_status::preparing_internal_abort: + case tx_status::preparing_commit: co_return init_tm_tx_reply{tx::errc::concurrent_transactions}; } } diff --git a/src/v/cluster/tx_protocol_types.cc b/src/v/cluster/tx_protocol_types.cc index d6558453823f..c396c7aaee04 100644 --- a/src/v/cluster/tx_protocol_types.cc +++ b/src/v/cluster/tx_protocol_types.cc @@ -138,7 +138,7 @@ std::ostream& operator<<(std::ostream& o, const fetch_tx_reply& r) { r.last_pid, r.tx_seq, r.timeout_ms.count(), - r.status, + static_cast(r.status), r.partitions, r.groups); return o; diff --git a/src/v/cluster/tx_protocol_types.h b/src/v/cluster/tx_protocol_types.h index 426793471995..f3bca4f8e465 100644 --- a/src/v/cluster/tx_protocol_types.h +++ b/src/v/cluster/tx_protocol_types.h @@ -224,7 +224,7 @@ struct fetch_tx_reply envelope, serde::compat_version<0>> { using rpc_adl_exempt = std::true_type; - enum tx_status : int32_t { + enum class tx_status : int32_t { ongoing, preparing, prepared, From e51b0a4c50d89c39437807a0233a4676f7ac8433 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Fri, 6 Dec 2024 02:44:16 +0000 Subject: [PATCH 081/229] cloud_topics: remove unused variable --- src/v/cloud_topics/core/pipeline_stage.h | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/v/cloud_topics/core/pipeline_stage.h b/src/v/cloud_topics/core/pipeline_stage.h index 8b10c76952d9..9c935a781f0e 100644 --- a/src/v/cloud_topics/core/pipeline_stage.h +++ b/src/v/cloud_topics/core/pipeline_stage.h @@ -47,9 +47,6 @@ class pipeline_stage_container { pipeline_stage register_pipeline_stage() noexcept; private: - // Pipeline stage counter used to assign ids to stages - int _next_stage_id_to_alloc{0}; - std::vector _stages; }; From 55e7a180ef560b4456489d8c7a9a29f7008af80f Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Fri, 6 Dec 2024 02:55:36 +0000 Subject: [PATCH 082/229] crypto: use deleter after class is defined Fixes a compilation error in c++23 where we declare the deleter for a class that hasn't yet been defined. --- src/v/crypto/ossl_context_service.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/v/crypto/ossl_context_service.cc b/src/v/crypto/ossl_context_service.cc index 40e8e05528ed..ec65bc1b93f1 100644 --- a/src/v/crypto/ossl_context_service.cc +++ b/src/v/crypto/ossl_context_service.cc @@ -162,8 +162,6 @@ void finalize_worker_thread(OSSL_LIB_CTX* orig_ctx) { } } // namespace -ossl_context_service::~ossl_context_service() noexcept = default; - class ossl_context_service::impl final { friend class ossl_context_test_class; @@ -328,6 +326,8 @@ class ossl_context_service::impl final { OSSL_LIB_CTX* _old_context{nullptr}; }; +ossl_context_service::~ossl_context_service() noexcept = default; + ossl_context_service::ossl_context_service( ssx::singleton_thread_worker& thread_worker, ss::sstring config_file, From 59e4de141bd848840e3402402d47e7853d43b700 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Fri, 6 Dec 2024 03:28:03 +0000 Subject: [PATCH 083/229] serde: correct feature check When this macro is actually defined, you get an error. Fix the check to use defined instead --- src/v/serde/serde_is_enum.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/v/serde/serde_is_enum.h b/src/v/serde/serde_is_enum.h index 9f534b89291e..3993b788135e 100644 --- a/src/v/serde/serde_is_enum.h +++ b/src/v/serde/serde_is_enum.h @@ -17,7 +17,7 @@ using serde_enum_serialized_t = int32_t; template inline constexpr bool serde_is_enum_v = -#if __has_cpp_attribute(__cpp_lib_is_scoped_enum) +#if defined(__cpp_lib_is_scoped_enum) && __cpp_lib_is_scoped_enum >= 202011L std::is_scoped_enum_v && sizeof(std::decay_t) <= sizeof(serde_enum_serialized_t); #else From d6641bbca15acbb5248f0d27f7cf7a1af46fab7e Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Thu, 5 Dec 2024 15:58:57 +0000 Subject: [PATCH 084/229] build: upgrade to c++23 --- .bazelrc | 2 +- CMakeLists.txt | 2 +- MODULE.bazel | 2 +- MODULE.bazel.lock | 6 +++--- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/.bazelrc b/.bazelrc index ca889b4f027c..e2f60eff9f58 100644 --- a/.bazelrc +++ b/.bazelrc @@ -9,7 +9,7 @@ common:clang-19 --extra_toolchains=@llvm_19_toolchain//:all build:system-clang --extra_toolchains=@local_config_cc_toolchains//:all build:system-clang --action_env=BAZEL_COMPILER=clang -build:system-clang --cxxopt=-std=c++20 --host_cxxopt=-std=c++20 +build:system-clang --cxxopt=-std=c++23 --host_cxxopt=-std=c++23 build:system-clang --linkopt -fuse-ld=lld # use a compiler name that doesn't symlink to ccache build:system-clang-18 --config=system-clang diff --git a/CMakeLists.txt b/CMakeLists.txt index 4fea6e40eba9..961894e49e8a 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -12,7 +12,7 @@ project(redpanda LANGUAGES CXX) list(APPEND CMAKE_MODULE_PATH ${CMAKE_CURRENT_SOURCE_DIR}/cmake) set(CMAKE_C_STANDARD 11) -set(CMAKE_CXX_STANDARD 20) +set(CMAKE_CXX_STANDARD 23) set(CMAKE_CXX_STANDARD_REQUIRED ON) set(CMAKE_CXX_EXTENSIONS OFF) set(CMAKE_EXPORT_COMPILE_COMMANDS ON) diff --git a/MODULE.bazel b/MODULE.bazel index fbe8f6790b9b..f4204fd868a2 100644 --- a/MODULE.bazel +++ b/MODULE.bazel @@ -123,7 +123,7 @@ single_version_override( # NOTE: We build our toolchains on ubuntu:jammy so you're going to need a distro at least that old. llvm = use_extension("@toolchains_llvm//toolchain/extensions:llvm.bzl", "llvm") -CXX_STANDARD = "c++20" +CXX_STANDARD = "c++23" llvm.toolchain( name = "llvm_19_toolchain", diff --git a/MODULE.bazel.lock b/MODULE.bazel.lock index e8cbe0767700..cd93ebf14edd 100644 --- a/MODULE.bazel.lock +++ b/MODULE.bazel.lock @@ -16740,7 +16740,7 @@ "@@toolchains_llvm~//toolchain/extensions:llvm.bzl%llvm": { "general": { "bzlTransitiveDigest": "y9h5L2NtWbogyWSOJgqnUaU50MTPWAW+waelXSirMVg=", - "usagesDigest": "3FwiQRD+JaBtqxv5ml//TWatmQnOHlCzhQj+xgAKCNA=", + "usagesDigest": "vaOqtBKVYh0KUH0biytmgYkovSqWiLA9X1XBqLx7yw8=", "recordedFileInputs": {}, "recordedDirentsInputs": {}, "envVariables": {}, @@ -16785,7 +16785,7 @@ "cxx_builtin_include_directories": {}, "cxx_flags": {}, "cxx_standard": { - "": "c++20" + "": "c++23" }, "dbg_compile_flags": {}, "exec_arch": "", @@ -16844,7 +16844,7 @@ "cxx_builtin_include_directories": {}, "cxx_flags": {}, "cxx_standard": { - "": "c++20" + "": "c++23" }, "dbg_compile_flags": {}, "exec_arch": "", From 06db8e06f02ca15290f66ab2af5c3897a85a8d62 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Sun, 8 Dec 2024 12:17:11 +0000 Subject: [PATCH 085/229] tracking_allocator: support c++23 There was a new allocate_at_least method added --- src/v/utils/tests/tracking_allocator_tests.cc | 6 ++++-- src/v/utils/tracking_allocator.h | 8 ++++++++ 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/src/v/utils/tests/tracking_allocator_tests.cc b/src/v/utils/tests/tracking_allocator_tests.cc index 2120e2266fb2..76de6ca47fc5 100644 --- a/src/v/utils/tests/tracking_allocator_tests.cc +++ b/src/v/utils/tests/tracking_allocator_tests.cc @@ -91,9 +91,10 @@ BOOST_AUTO_TEST_CASE(allocator_list_vector) { allocator alloc{tracker}; std::vector> v{alloc}; v.push_back(1); - BOOST_REQUIRE_EQUAL(tracker->consumption(), sizeof(int)); + BOOST_REQUIRE_GE(tracker->consumption(), sizeof(int)); v.push_back(2); - BOOST_REQUIRE_EQUAL(tracker->consumption(), 2 * sizeof(int)); + BOOST_REQUIRE_GE(tracker->consumption(), 2 * sizeof(int)); + BOOST_REQUIRE_EQUAL(v.size(), 2); } BOOST_REQUIRE_EQUAL(tracker->consumption(), 0); { @@ -101,6 +102,7 @@ BOOST_AUTO_TEST_CASE(allocator_list_vector) { std::list> v{alloc}; v.push_back(1); BOOST_REQUIRE_GE(tracker->consumption(), 0); + BOOST_REQUIRE_EQUAL(v.size(), 1); } BOOST_REQUIRE_EQUAL(tracker->consumption(), 0); } diff --git a/src/v/utils/tracking_allocator.h b/src/v/utils/tracking_allocator.h index f208c894059b..2ef1ad60ef79 100644 --- a/src/v/utils/tracking_allocator.h +++ b/src/v/utils/tracking_allocator.h @@ -121,6 +121,14 @@ class tracking_allocator : public allocator { return allocator::allocate(n); } +#if __cplusplus >= 202302L + [[nodiscard]] std::allocation_result allocate_at_least(std::size_t n) { + auto result = allocator::allocate_at_least(n); + _tracker->allocate(result.count * sizeof(T)); + return result; + } +#endif + void deallocate(T* p, std::size_t n) { allocator::deallocate(p, n); _tracker->deallocate(n * sizeof(T)); From e513ac58fa208fee193fdf3570af65267caad685 Mon Sep 17 00:00:00 2001 From: Travis Downs Date: Sun, 8 Dec 2024 10:49:22 -0300 Subject: [PATCH 086/229] test: redirect boost test output under bazel The bazel test runner sets an XML_OUTPUT_FILE environment variable so that the test writes the results in a local that bazel can subsequently consume. If the test does not write anything to that location bazel instead builds a "default" result file which treats the entire test run as a single test case, and uses the stdout of the test binary as the output of the test case. In addition to not exposing the test cases contained in the test correctly, this process is extremely slow when there is a lot of test output (as many redpanda fixture tests have) since the output is injected via shell variable interpolation which apparently doesn't scale up to variables several GB in size. This change introduces a boost global test configuration object which looks for the XML_OUTPUT_FILE variable and if set enables JUNIT output redirected to the specified path. Fixes CORE-8013. --- bazel/test.bzl | 5 ++- src/v/test_utils/BUILD | 17 +++++++++- src/v/test_utils/boost_result_redirect.cc | 39 +++++++++++++++++++++++ 3 files changed, 59 insertions(+), 2 deletions(-) create mode 100644 src/v/test_utils/boost_result_redirect.cc diff --git a/bazel/test.bzl b/bazel/test.bzl index 4642d8320a7a..7af017422c8b 100644 --- a/bazel/test.bzl +++ b/bazel/test.bzl @@ -249,7 +249,10 @@ def redpanda_cc_btest_no_seastar( srcs = srcs, defines = defines, copts = redpanda_copts(), - deps = ["@boost//:test.so"] + deps, + deps = [ + "//src/v/test_utils:boost_result_redirect", + "@boost//:test.so", + ] + deps, ) def redpanda_test_cc_library( diff --git a/src/v/test_utils/BUILD b/src/v/test_utils/BUILD index 666e71280cb6..25cd35e1443b 100644 --- a/src/v/test_utils/BUILD +++ b/src/v/test_utils/BUILD @@ -61,11 +61,26 @@ redpanda_test_cc_library( include_prefix = "test_utils", visibility = ["//visibility:public"], deps = [ - "//src/v/test_utils:fixture", + ":boost_result_redirect", + ":fixture", "@seastar//:testing", ], ) +redpanda_test_cc_library( + name = "boost_result_redirect", + srcs = [ + "boost_result_redirect.cc", + ], + include_prefix = "test_utils", + visibility = ["//visibility:public"], + deps = [ + "@boost//:config", + "@boost//:core", + "@boost//:test", + ], +) + redpanda_test_cc_library( name = "random", hdrs = [ diff --git a/src/v/test_utils/boost_result_redirect.cc b/src/v/test_utils/boost_result_redirect.cc new file mode 100644 index 000000000000..9407686ffb0a --- /dev/null +++ b/src/v/test_utils/boost_result_redirect.cc @@ -0,0 +1,39 @@ +// Copyright 2024 Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 + +#include +#include +#include + +#include +#include + +/** + * This test configuration object is used to redirect the JUNIT result output + * (if enabled) to the location specified by the XML_OUTPUT_FILE environment + * variable. This variable is set by the bazel test runner so that the test + * writes the results in a local that bazel can subsequently consume. + * Ref: CORE-8013. + */ +struct bazel_result_handler_for_boost { + bazel_result_handler_for_boost() { + using namespace boost::unit_test; + if (auto xml_path = std::getenv("XML_OUTPUT_FILE")) { + _out = std::ofstream(xml_path); + // use fixed format to avoid scientific notation as a work-around + // to https://github.com/bazelbuild/bazel/issues/24605 + _out << std::fixed << std::setprecision(6); + unit_test_log.add_format(OF_JUNIT); + unit_test_log.set_stream(OF_JUNIT, _out); + } + } + std::ofstream _out; +}; + +BOOST_TEST_GLOBAL_CONFIGURATION(bazel_result_handler_for_boost); From 48371f454085826420b6de81da9198ae2be212bf Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Sun, 8 Dec 2024 17:19:42 -0800 Subject: [PATCH 087/229] producer_state_manager: make period configurable for testing --- src/v/cluster/producer_state_manager.cc | 10 ++++++++-- src/v/cluster/producer_state_manager.h | 6 ++++-- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/src/v/cluster/producer_state_manager.cc b/src/v/cluster/producer_state_manager.cc index 2e3b79795228..2cd7dd72c52e 100644 --- a/src/v/cluster/producer_state_manager.cc +++ b/src/v/cluster/producer_state_manager.cc @@ -41,7 +41,7 @@ producer_state_manager::producer_state_manager( ss::future<> producer_state_manager::start() { _reaper.set_callback([this] { evict_excess_producers(); }); - _reaper.arm(period); + _reaper.arm(_reaper_period); vlog(clusterlog.info, "Started producer state manager"); return ss::now(); } @@ -69,6 +69,12 @@ void producer_state_manager::setup_metrics() { sm::description("Number of evicted producers so far."))}); } +void producer_state_manager::rearm_eviction_timer_for_testing( + std::chrono::milliseconds new_period) { + _reaper_period = new_period; + _reaper.rearm(ss::lowres_clock::now() + _reaper_period); +} + void producer_state_manager::register_producer( producer_state& state, std::optional vcluster) { vlog( @@ -97,7 +103,7 @@ void producer_state_manager::evict_excess_producers() { _cache.evict_older_than( ss::lowres_system_clock::now() - _producer_expiration_ms()); if (!_gate.is_closed()) { - _reaper.arm(period); + _reaper.arm(_reaper_period); } } diff --git a/src/v/cluster/producer_state_manager.h b/src/v/cluster/producer_state_manager.h index b414dc74b915..434ad6bf84b8 100644 --- a/src/v/cluster/producer_state_manager.h +++ b/src/v/cluster/producer_state_manager.h @@ -45,8 +45,10 @@ class producer_state_manager { */ void touch(producer_state&, std::optional); + void rearm_eviction_timer_for_testing(std::chrono::milliseconds); + private: - static constexpr std::chrono::seconds period{5}; + std::chrono::milliseconds _reaper_period{5000}; /** * Constant to be used when a partition has no vcluster_id assigned. */ @@ -83,7 +85,7 @@ class producer_state_manager { config::binding _virtual_cluster_min_producer_ids; // cache of all producers on this shard cache_t _cache; - ss::timer _reaper; + ss::timer _reaper; ss::gate _gate; metrics::internal_metric_groups _metrics; From 7cfbcb05d210652292673090c468a32acf8ebdb1 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Thu, 5 Dec 2024 17:53:40 +0100 Subject: [PATCH 088/229] application: don't enable datalake services in recovery mode --- src/v/redpanda/application.cc | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/src/v/redpanda/application.cc b/src/v/redpanda/application.cc index 67cdf5b9f738..a040d6cc55e1 100644 --- a/src/v/redpanda/application.cc +++ b/src/v/redpanda/application.cc @@ -2402,7 +2402,8 @@ bool application::wasm_data_transforms_enabled() { } bool application::datalake_enabled() { - return config::shard_local_cfg().iceberg_enabled(); + return config::shard_local_cfg().iceberg_enabled() + && !config::node().recovery_mode_enabled(); } ss::future<> @@ -3143,11 +3144,13 @@ void application::start_runtime_services( smp_service_groups.cluster_smp_sg(), std::ref(controller->get_data_migration_frontend()), std::ref(controller->get_data_migration_irpc_frontend()))); - runtime_services.push_back( - std::make_unique( - sched_groups.datalake_sg(), - smp_service_groups.datalake_sg(), - &_datalake_coordinator_fe)); + if (datalake_enabled()) { + runtime_services.push_back( + std::make_unique( + sched_groups.datalake_sg(), + smp_service_groups.datalake_sg(), + &_datalake_coordinator_fe)); + } s.add_services(std::move(runtime_services)); From 159a2722d7e0f46f81239bd24c95c31cb707b8c8 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Thu, 5 Dec 2024 17:55:03 +0100 Subject: [PATCH 089/229] tests/datalake: add test in recovery mode --- .../tests/datalake/recovery_mode_test.py | 89 +++++++++++++++++++ 1 file changed, 89 insertions(+) create mode 100644 tests/rptest/tests/datalake/recovery_mode_test.py diff --git a/tests/rptest/tests/datalake/recovery_mode_test.py b/tests/rptest/tests/datalake/recovery_mode_test.py new file mode 100644 index 000000000000..e154e558e9ff --- /dev/null +++ b/tests/rptest/tests/datalake/recovery_mode_test.py @@ -0,0 +1,89 @@ +# Copyright 2024 Redpanda Data, Inc. +# +# Use of this software is governed by the Business Source License +# included in the file licenses/BSL.md +# +# As of the Change Date specified in that file, in accordance with +# the Business Source License, use of this software will be governed +# by the Apache License, Version 2.0 + +import time +import random + +from ducktape.mark import matrix +from ducktape.utils.util import wait_until + +from rptest.clients.rpk import RpkTool +from rptest.services.cluster import cluster +from rptest.clients.types import TopicSpec +from rptest.services.redpanda import SISettings +from rptest.services.admin import Admin +from rptest.tests.redpanda_test import RedpandaTest +from rptest.tests.datalake.datalake_services import DatalakeServices +from rptest.tests.datalake.query_engine_base import QueryEngineType +from rptest.tests.datalake.utils import supported_storage_types + + +class DatalakeRecoveryModeTest(RedpandaTest): + def __init__(self, test_ctx, *args, **kwargs): + super(DatalakeRecoveryModeTest, + self).__init__(test_ctx, + num_brokers=3, + si_settings=SISettings(test_ctx), + extra_rp_conf={ + "iceberg_enabled": "true", + "iceberg_catalog_commit_interval_ms": 5000 + }, + *args, + **kwargs) + + def setUp(self): + # redpanda will be started by DatalakeServices + pass + + @cluster(num_nodes=6) + @matrix(cloud_storage_type=supported_storage_types(), + filesystem_catalog_mode=[True, False]) + def test_recovery_mode(self, cloud_storage_type, filesystem_catalog_mode): + with DatalakeServices(self.test_context, + redpanda=self.redpanda, + filesystem_catalog_mode=filesystem_catalog_mode, + include_query_engines=[QueryEngineType.SPARK + ]) as dl: + count = 1000 + rpk = RpkTool(self.redpanda) + + dl.create_iceberg_enabled_topic("foo", partitions=10) + rpk.create_topic("bar", partitions=10, replicas=3) + + dl.produce_to_topic("foo", 1024, count) + + # test partial recovery mode + self.redpanda.restart_nodes( + random.sample(self.redpanda.nodes, 1), + override_cfg_params={"recovery_mode_enabled": True}) + + time.sleep(15) + + self.redpanda.restart_nodes( + self.redpanda.nodes, + override_cfg_params={"recovery_mode_enabled": True}) + self.redpanda.wait_for_membership(first_start=False) + + admin = Admin(self.redpanda) + admin.await_stable_leader(namespace="redpanda", topic="controller") + + rpk.alter_topic_config("bar", TopicSpec.PROPERTY_ICEBERG_MODE, + "key_value") + time.sleep(15) + + self.redpanda.restart_nodes( + self.redpanda.nodes, + override_cfg_params={"recovery_mode_enabled": False}) + self.redpanda.wait_for_membership(first_start=False) + + dl.produce_to_topic("foo", 1024, count) + dl.produce_to_topic("bar", 1024, count) + + dl.wait_for_translation("foo", msg_count=2 * count) + dl.wait_for_translation("bar", msg_count=count) From 80d3ac2076c12646810ac61948159dc1ecd053ec Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Thu, 5 Dec 2024 17:56:33 +0100 Subject: [PATCH 090/229] tests/datalake: add test with disabled partitions --- .../tests/datalake/recovery_mode_test.py | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/tests/rptest/tests/datalake/recovery_mode_test.py b/tests/rptest/tests/datalake/recovery_mode_test.py index e154e558e9ff..4f4f881caf93 100644 --- a/tests/rptest/tests/datalake/recovery_mode_test.py +++ b/tests/rptest/tests/datalake/recovery_mode_test.py @@ -87,3 +87,39 @@ def test_recovery_mode(self, cloud_storage_type, filesystem_catalog_mode): dl.wait_for_translation("foo", msg_count=2 * count) dl.wait_for_translation("bar", msg_count=count) + + @cluster(num_nodes=6) + @matrix(cloud_storage_type=supported_storage_types(), + filesystem_catalog_mode=[True, False]) + def test_disabled_partitions(self, cloud_storage_type, + filesystem_catalog_mode): + with DatalakeServices(self.test_context, + redpanda=self.redpanda, + filesystem_catalog_mode=filesystem_catalog_mode, + include_query_engines=[QueryEngineType.SPARK + ]) as dl: + + count = 1000 + rpk = RpkTool(self.redpanda) + + dl.create_iceberg_enabled_topic("foo", partitions=10) + rpk.create_topic("bar", partitions=10, replicas=3) + + dl.produce_to_topic("foo", 1024, count) + + admin = Admin(self.redpanda) + admin.set_partitions_disabled(ns="kafka", topic="foo") + admin.set_partitions_disabled(ns="kafka", topic="bar") + + rpk.alter_topic_config("bar", TopicSpec.PROPERTY_ICEBERG_MODE, + "key_value") + + time.sleep(15) + admin.set_partitions_disabled(ns="kafka", topic="foo", value=False) + admin.set_partitions_disabled(ns="kafka", topic="bar", value=False) + + dl.produce_to_topic("foo", 1024, count) + dl.produce_to_topic("bar", 1024, count) + + dl.wait_for_translation("foo", msg_count=2 * count) + dl.wait_for_translation("bar", msg_count=count) From 28ec1df5d0f2121b482ff6311e373ac0c1154980 Mon Sep 17 00:00:00 2001 From: Alexey Bashtanov Date: Fri, 6 Dec 2024 15:42:35 +0000 Subject: [PATCH 091/229] c/partition_recovery_manager: avoid oversized allocations when logging Do not dump partition manifest JSON as it can be very large --- src/v/cluster/partition_recovery_manager.cc | 8 -------- 1 file changed, 8 deletions(-) diff --git a/src/v/cluster/partition_recovery_manager.cc b/src/v/cluster/partition_recovery_manager.cc index 4f516fb670d9..94c656ce2685 100644 --- a/src/v/cluster/partition_recovery_manager.cc +++ b/src/v/cluster/partition_recovery_manager.cc @@ -316,14 +316,6 @@ ss::future partition_downloader::download_log() { _ntpc.get_revision(), retention); auto mat = co_await find_recovery_material(); - if (cst_log.is_enabled(ss::log_level::debug)) { - std::stringstream ostr; - mat.partition_manifest.serialize_json(ostr); - vlog( - _ctxlog.debug, - "Partition manifest used for recovery: {}", - ostr.str()); - } if (mat.partition_manifest.size() == 0) { // If the downloaded manifest doesn't have any segments log_recovery_result result{ From 4bd5b4bb16e43b7bbb075ebaab73a2e7086d15a4 Mon Sep 17 00:00:00 2001 From: Chris Lee Date: Mon, 9 Dec 2024 10:16:28 -0700 Subject: [PATCH 092/229] [rptest] Use "redpanda" namespace for KubeNodeShell --- tests/rptest/clients/kubectl.py | 2 +- tests/rptest/services/cloud_broker.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/rptest/clients/kubectl.py b/tests/rptest/clients/kubectl.py index 435167e74152..a3686f53c644 100644 --- a/tests/rptest/clients/kubectl.py +++ b/tests/rptest/clients/kubectl.py @@ -450,7 +450,7 @@ class KubeNodeShell(): def __init__(self, kubectl: KubectlTool, node_name: str, - namespace: str = 'redpanda-node-setup', + namespace: str = 'redpanda', clean=False) -> None: self.kubectl = kubectl self.node_name = node_name diff --git a/tests/rptest/services/cloud_broker.py b/tests/rptest/services/cloud_broker.py index c171fbf847e5..06a120cf48ac 100644 --- a/tests/rptest/services/cloud_broker.py +++ b/tests/rptest/services/cloud_broker.py @@ -83,7 +83,7 @@ def inject_script(self, script_name): # Copy agent -> broker node remote_path = os.path.join("/tmp", script_name) _cp_cmd = self._kubeclient._ssh_prefix() + [ - 'kubectl', '-n', 'redpanda-node-setup', 'cp', script_name, + 'kubectl', '-n', 'redpanda', 'cp', script_name, f"{self.nodeshell.pod_name}:{remote_path}" ] self.logger.debug(_cp_cmd) From 404d5e7d1278b80055f70afaf404604ef0fbcf25 Mon Sep 17 00:00:00 2001 From: Travis Downs Date: Tue, 22 Oct 2024 10:16:45 -0300 Subject: [PATCH 093/229] storage/tests: add bazel test libraries Add bazel test libraries needed for the storage e2e test: - common - storage_test_fixture - log_gap_analysis Ref CORE-7649. --- src/v/storage/tests/BUILD | 42 +++++++++++++++++++++++++++++++++++++++ 1 file changed, 42 insertions(+) diff --git a/src/v/storage/tests/BUILD b/src/v/storage/tests/BUILD index ac0971ff1093..6b179f0849da 100644 --- a/src/v/storage/tests/BUILD +++ b/src/v/storage/tests/BUILD @@ -63,6 +63,48 @@ redpanda_test_cc_library( ], ) +redpanda_test_cc_library( + name = "common", + hdrs = [ + "common.h", + ], + include_prefix = "storage/tests", + deps = [ + "//src/v/storage", + ], +) + +redpanda_test_cc_library( + name = "storage_test_fixture", + hdrs = [ + "storage_test_fixture.h", + ], + include_prefix = "storage/tests", + deps = [ + "//src/v/compression", + "//src/v/config", + "//src/v/features", + "//src/v/model/tests:random", + "//src/v/storage", + "//src/v/test_utils:seastar_boost", + ], +) + +redpanda_test_cc_library( + name = "log_gap_analysis", + srcs = [ + "utils/log_gap_analysis.cc", + ], + hdrs = [ + "utils/log_gap_analysis.h", + ], + include_prefix = "storage/tests", + deps = [ + "//src/v/model", + "//src/v/storage:parser_utils", + ], +) + redpanda_cc_gtest( name = "segment_offset_tracker_test", timeout = "short", From 68db0406e875544f82aac705aa9f9846336ba9b3 Mon Sep 17 00:00:00 2001 From: Travis Downs Date: Tue, 22 Oct 2024 10:16:45 -0300 Subject: [PATCH 094/229] storage/tests/e2e: fix compile errors Fix compile warnings (as errors) which appear only in the bazel build, prior to enabling this test in bazel. Ref CORE-7649. --- src/v/storage/tests/storage_e2e_test.cc | 26 ++++++++++++------------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/src/v/storage/tests/storage_e2e_test.cc b/src/v/storage/tests/storage_e2e_test.cc index cfc3fda99c6d..2f6272467240 100644 --- a/src/v/storage/tests/storage_e2e_test.cc +++ b/src/v/storage/tests/storage_e2e_test.cc @@ -380,7 +380,7 @@ FIXTURE_TEST(test_truncation_with_write_caching, storage_test_fixture) { const auto truncate_batch_ix = headers.size() - 2; const auto truncate_offset = [&]() { model::offset o{0}; - for (auto i = 0; i < truncate_batch_ix; i++) { + for (size_t i = 0; i < truncate_batch_ix; i++) { o += headers[i].record_count; } return o; @@ -937,7 +937,7 @@ ss::future append_exactly( val_sz -= real_batch_size; - for (int i = 0; i < batch_count; ++i) { + for (size_t i = 0; i < batch_count; ++i) { storage::record_batch_builder builder(batch_type, model::offset{}); iobuf value = bytes_to_iobuf(random_generators::get_bytes(val_sz)); builder.add_raw_kv(key_buf.copy(), std::move(value)); @@ -1050,7 +1050,7 @@ FIXTURE_TEST(append_concurrent_with_prefix_truncate, storage_test_fixture) { auto log = mgr.manage(std::move(ntp_cfg)).get0(); bool stop = false; - int cnt = 0; + size_t cnt = 0; std::vector types{ model::record_batch_type::raft_data, model::record_batch_type::raft_configuration}; @@ -2445,7 +2445,7 @@ FIXTURE_TEST(changing_cleanup_policy_back_and_forth, storage_test_fixture) { // add a segment, some of the record keys in batches are random and some of // them are the same to generate offset gaps after compaction - auto add_segment = [&log, disk_log](size_t size, model::term_id term) { + auto add_segment = [&log, disk_log](size_t size) { do { // 10 records per batch for (int i = 0; i < 10; ++i) { @@ -2481,11 +2481,11 @@ FIXTURE_TEST(changing_cleanup_policy_back_and_forth, storage_test_fixture) { } while (disk_log->segments().back()->size_bytes() < size); }; // add 2 log segments - add_segment(1_MiB, model::term_id(1)); + add_segment(1_MiB); disk_log->force_roll(ss::default_priority_class()).get(); - add_segment(1_MiB, model::term_id(1)); + add_segment(1_MiB); disk_log->force_roll(ss::default_priority_class()).get(); - add_segment(1_MiB, model::term_id(1)); + add_segment(1_MiB); log->flush().get(); BOOST_REQUIRE_EQUAL(disk_log->segment_count(), 3); @@ -3840,7 +3840,7 @@ FIXTURE_TEST(test_bytes_eviction_overrides, storage_test_fixture) { auto deferred_rm = ss::defer( [&mgr, ntp]() mutable { mgr.remove(ntp).get(); }); - for (int i = 0; i < batch_cnt; ++i) { + for (size_t i = 0; i < batch_cnt; ++i) { append_exactly(log, 1, batch_size).get(); } auto disk_log = log; @@ -4261,7 +4261,7 @@ FIXTURE_TEST(test_offset_range_size, storage_test_fixture) { auto log = mgr.manage(std::move(ntp_cfg)).get(); model::offset first_segment_last_offset; - for (int i = 0; i < num_segments; i++) { + for (size_t i = 0; i < num_segments; i++) { append_random_batches( log, 10, @@ -4374,7 +4374,7 @@ FIXTURE_TEST(test_offset_range_size2, storage_test_fixture) { auto log = mgr.manage(std::move(ntp_cfg)).get(); model::offset first_segment_last_offset; - for (int i = 0; i < num_segments; i++) { + for (size_t i = 0; i < num_segments; i++) { append_random_batches( log, 10, @@ -4578,7 +4578,7 @@ FIXTURE_TEST(test_offset_range_size_compacted, storage_test_fixture) { auto log = mgr.manage(std::move(ntp_cfg)).get(); model::offset first_segment_last_offset; - for (int i = 0; i < num_segments; i++) { + for (size_t i = 0; i < num_segments; i++) { append_random_batches( log, 10, model::term_id(i), key_limited_random_batch_generator()); if (first_segment_last_offset == model::offset{}) { @@ -4777,7 +4777,7 @@ FIXTURE_TEST(test_offset_range_size2_compacted, storage_test_fixture) { auto log = mgr.manage(std::move(ntp_cfg)).get(); model::offset first_segment_last_offset; - for (int i = 0; i < num_segments; i++) { + for (size_t i = 0; i < num_segments; i++) { append_random_batches( log, 10, model::term_id(0), key_limited_random_batch_generator()); if (first_segment_last_offset == model::offset{}) { @@ -5079,7 +5079,7 @@ FIXTURE_TEST(test_offset_range_size_incremental, storage_test_fixture) { auto log = mgr.manage(std::move(ntp_cfg)).get(); model::offset first_segment_last_offset; - for (int i = 0; i < num_segments; i++) { + for (size_t i = 0; i < num_segments; i++) { append_random_batches( log, 10, From c4ce3d7f0bac2eac54a57dff880ea5897cf122cc Mon Sep 17 00:00:00 2001 From: Paulo Borges Date: Mon, 9 Dec 2024 17:02:58 -0300 Subject: [PATCH 095/229] rpk: update help description to match docs Update the help text for rpk partitions move to match docs style of not using 'topic-partition' hyphenated. --- src/go/rpk/pkg/cli/cluster/partitions/move.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/go/rpk/pkg/cli/cluster/partitions/move.go b/src/go/rpk/pkg/cli/cluster/partitions/move.go index 54ef9c0c8503..8c82097b3680 100644 --- a/src/go/rpk/pkg/cli/cluster/partitions/move.go +++ b/src/go/rpk/pkg/cli/cluster/partitions/move.go @@ -148,7 +148,7 @@ func newMovePartitionReplicasCommand(fs afero.Fs, p *config.Params) *cobra.Comma } }, } - cmd.Flags().StringArrayVarP(&partitionsFlag, "partition", "p", nil, "Topic-partitions to move and new replica locations (repeatable)") + cmd.Flags().StringArrayVarP(&partitionsFlag, "partition", "p", nil, "Topic partitions to move and new replica locations (repeatable)") cmd.MarkFlagRequired("partition") p.InstallFormatFlag(cmd) From a235e0a0138c1dd4561559c121b80021bdd53981 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Mon, 9 Dec 2024 18:07:18 +0000 Subject: [PATCH 096/229] bazel: update target for openssl executable Previous solutions was a workaround and would potentially use system's openssl libraries and config data. Replacing it with foreign_cc:runnable_binary which is properly aware of the executable dependencies. --- bazel/cert.bzl | 23 ++++++++++++++++++++--- bazel/thirdparty/openssl.BUILD | 25 +++++++++++++------------ 2 files changed, 33 insertions(+), 15 deletions(-) diff --git a/bazel/cert.bzl b/bazel/cert.bzl index 367348c8031d..062172f5d1f5 100644 --- a/bazel/cert.bzl +++ b/bazel/cert.bzl @@ -4,13 +4,19 @@ This module contains functions to generate a simple CA load("@bazel_skylib//rules:run_binary.bzl", "run_binary") +openssl_env = { + "OPENSSL_CONF": "$(execpath @openssl//:openssl_data)/openssl.cnf", +} + # buildifier: disable=function-docstring-args def _redpanda_private_key(name, certificate): private_key = certificate + ".key" run_binary( name = name + "_key_gen", - srcs = [], + srcs = [ + "@openssl//:openssl_data", + ], outs = [private_key], args = [ "ecparam", @@ -21,6 +27,7 @@ def _redpanda_private_key(name, certificate): "-out", "$(execpath :{})".format(private_key), ], + env = openssl_env, tool = "@openssl//:openssl_exe", ) @@ -44,7 +51,10 @@ def redpanda_selfsigned_cert(name, certificate, common_name, visibility = None): run_binary( name = name + "_crt_gen", - srcs = [private_key], + srcs = [ + private_key, + "@openssl//:openssl_data", + ], outs = [cert], args = [ "req", @@ -60,6 +70,7 @@ def redpanda_selfsigned_cert(name, certificate, common_name, visibility = None): "-addext", "subjectAltName = IP:127.0.0.1", ], + env = openssl_env, tool = "@openssl//:openssl_exe", ) @@ -90,7 +101,10 @@ def redpanda_signed_cert(name, certificate, common_name, ca, serial_number, visi run_binary( name = name + "_csr_gen", - srcs = [private_key], + srcs = [ + private_key, + "@openssl//:openssl_data", + ], outs = [csr], args = [ "req", @@ -103,6 +117,7 @@ def redpanda_signed_cert(name, certificate, common_name, ca, serial_number, visi "-subj", subj, ], + env = openssl_env, tool = "@openssl//:openssl_exe", ) @@ -115,6 +130,7 @@ def redpanda_signed_cert(name, certificate, common_name, ca, serial_number, visi ca_cert, ca_private_key, csr, + "@openssl//:openssl_data", ], outs = [cert], args = [ @@ -134,6 +150,7 @@ def redpanda_signed_cert(name, certificate, common_name, ca, serial_number, visi "-out", "$(execpath :{})".format(cert), ], + env = openssl_env, tool = "@openssl//:openssl_exe", ) diff --git a/bazel/thirdparty/openssl.BUILD b/bazel/thirdparty/openssl.BUILD index 80f8f6eebbe0..40da3ef6e841 100644 --- a/bazel/thirdparty/openssl.BUILD +++ b/bazel/thirdparty/openssl.BUILD @@ -1,7 +1,6 @@ load("@bazel_skylib//rules:common_settings.bzl", "int_flag", "string_flag") -load("@bazel_skylib//rules:copy_file.bzl", "copy_file") load("@bazel_skylib//rules:select_file.bzl", "select_file") -load("@rules_foreign_cc//foreign_cc:defs.bzl", "configure_make") +load("@rules_foreign_cc//foreign_cc:defs.bzl", "configure_make", "runnable_binary") # Make this build faster by setting `build --@openssl//:build_jobs=16` in user.bazelrc # if you have the cores to spare. @@ -60,6 +59,9 @@ configure_make( out_binaries = [ "openssl", ], + out_data_dirs = [ + "ssl", + ], out_shared_libs = [ "libssl.so.3", "libcrypto.so.3", @@ -77,18 +79,17 @@ filegroup( ) select_file( - name = "openssl_exe_file", + name = "openssl_data", srcs = ":openssl", - subpath = "bin/openssl", -) - -copy_file( - name = "openssl_exe", - src = ":openssl_exe_file", - out = "openssl.exe", - allow_symlink = True, - is_executable = True, + subpath = "ssl", visibility = [ "//visibility:public", ], ) + +runnable_binary( + name = "openssl_exe", + binary = "openssl", + foreign_cc_target = ":openssl", + visibility = ["//visibility:public"], +) From 6d829767d3455ddd6d548903a17348c2b11cdad2 Mon Sep 17 00:00:00 2001 From: Travis Downs Date: Tue, 22 Oct 2024 10:16:45 -0300 Subject: [PATCH 097/229] storage/tests/e2e: add test to bazel Add the storage_e2e_single_threaded test to the bazel build. Ref CORE-7649. --- src/v/storage/tests/BUILD | 38 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/src/v/storage/tests/BUILD b/src/v/storage/tests/BUILD index 6b179f0849da..5912b9922f87 100644 --- a/src/v/storage/tests/BUILD +++ b/src/v/storage/tests/BUILD @@ -503,3 +503,41 @@ redpanda_cc_gtest( "@seastar", ], ) + +redpanda_cc_btest( + name = "storage_e2e_single_thread", + timeout = "moderate", + srcs = [ + "storage_e2e_test.cc", + ], + cpu = 1, + deps = [ + ":common", + ":disk_log_builder", + ":disk_log_builder_fixture", + ":log_gap_analysis", + ":storage_test_fixture", + "//src/v/base", + "//src/v/bytes", + "//src/v/bytes:random", + "//src/v/compression", + "//src/v/config", + "//src/v/features", + "//src/v/finjector", + "//src/v/model", + "//src/v/model/tests:random", + "//src/v/random:generators", + "//src/v/reflection:adl", + "//src/v/storage", + "//src/v/storage:batch_cache", + "//src/v/storage:record_batch_builder", + "//src/v/storage:resources", + "//src/v/test_utils:random", + "//src/v/test_utils:seastar_boost", + "//src/v/utils:directory_walker", + "//src/v/utils:to_string", + "@boost//:test", + "@fmt", + "@seastar", + ], +) From 5a4bc8031b28aeeea1c10ae47412121245322d8e Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Mon, 9 Dec 2024 19:32:50 -0500 Subject: [PATCH 098/229] `rptest`: use proper `WriteCachingMode` string for parameterization `"on"` and `"off"` are not valid strings for write caching mode. Replace the values used for write caching parameterization in `omb_validation_test.py` with the strings coded in the `WriteCachingMode` class. --- tests/rptest/redpanda_cloud_tests/omb_validation_test.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/rptest/redpanda_cloud_tests/omb_validation_test.py b/tests/rptest/redpanda_cloud_tests/omb_validation_test.py index 87f01c10e0ec..426df9f070c7 100644 --- a/tests/rptest/redpanda_cloud_tests/omb_validation_test.py +++ b/tests/rptest/redpanda_cloud_tests/omb_validation_test.py @@ -24,6 +24,7 @@ OMBSampleConfigurations from rptest.services.machinetype import get_machine_info from rptest.utils.type_utils import rcast +from rptest.tests.write_caching_test import WriteCachingMode # pyright: strict @@ -231,7 +232,7 @@ def _mb_to_mib(self, mb: float | int): return math.floor(0.9537 * mb) @cluster(num_nodes=CLUSTER_NODES) - @matrix(write_caching=["on", "off"]) + @matrix(write_caching=[WriteCachingMode.TRUE, WriteCachingMode.FALSE]) def test_max_connections(self, write_caching: str): tier_limits = self.tier_limits @@ -564,7 +565,7 @@ def kv_str(k: str, v: Any): self.logger.warn(str(results)) @cluster(num_nodes=CLUSTER_NODES) - @matrix(write_caching=["on", "off"]) + @matrix(write_caching=[WriteCachingMode.TRUE, WriteCachingMode.FALSE]) def test_max_partitions(self, write_caching: str): tier_limits = self.tier_limits @@ -670,7 +671,7 @@ def test_max_partitions(self, write_caching: str): self.redpanda.assert_cluster_is_reusable() @cluster(num_nodes=CLUSTER_NODES) - @matrix(write_caching=["on", "off"]) + @matrix(write_caching=[WriteCachingMode.TRUE, WriteCachingMode.FALSE]) def test_common_workload(self, write_caching: str): tier_limits = self.tier_limits @@ -728,7 +729,7 @@ def test_common_workload(self, write_caching: str): self.redpanda.assert_cluster_is_reusable() @cluster(num_nodes=CLUSTER_NODES) - @matrix(write_caching=["on", "off"]) + @matrix(write_caching=[WriteCachingMode.TRUE, WriteCachingMode.FALSE]) def test_retention(self, write_caching: str): tier_limits = self.tier_limits From a0bc940e961c88ff0f534087862c8fa399808097 Mon Sep 17 00:00:00 2001 From: Chris Lee Date: Fri, 6 Dec 2024 12:46:59 -0700 Subject: [PATCH 099/229] [rptest] improve asserts, Azure path in ConfigProfileVerify --- .../config_profile_verify_test.py | 39 ++++++++----------- 1 file changed, 17 insertions(+), 22 deletions(-) diff --git a/tests/rptest/redpanda_cloud_tests/config_profile_verify_test.py b/tests/rptest/redpanda_cloud_tests/config_profile_verify_test.py index 64631a2c2b20..c8955cfdf566 100644 --- a/tests/rptest/redpanda_cloud_tests/config_profile_verify_test.py +++ b/tests/rptest/redpanda_cloud_tests/config_profile_verify_test.py @@ -60,13 +60,11 @@ def test_config_profile_verify(self): def _check_rp_config(self): confRes = self.redpanda.kubectl.exec( - "rpk redpanda admin config print --host 0") + 'rpk redpanda admin config print --host 0') clusterConfig = json.loads(confRes) - self.logger.debug( - "asserting we got the config for the right cluster: expected rp-{}, actual: {}" - .format(self._clusterId, clusterConfig["cluster_id"])) - assert clusterConfig['cluster_id'] in (self._clusterId, - f'rp-{self._clusterId}') + assert clusterConfig['cluster_id'] in ( + self._clusterId, f'rp-{self._clusterId}' + ), f'asserting we got the config for the right cluster: expected {self._clusterId} to contain {clusterConfig["cluster_id"]}' for k, expected_v in self._configProfile["cluster_config"].items(): actual_v = clusterConfig[k] @@ -99,27 +97,24 @@ def _check_aws_nodes(self): assert resd[0] == self._configProfile['machine_type'] def _check_azure_nodes(self): - # currently, we have to override the PATH for azure because - # az-cli is installed via snap and /snap/bin only appears in - # $PATH on *interactive* shells + jsonpath = '{..labels.node\\.kubernetes\\.io/instance-type}' cmd = self.redpanda.kubectl._ssh_prefix() + [ - 'env', 'PATH=/usr/local/bin:/usr/bin:/bin:/snap/bin', - 'az', 'aks', 'nodepool', 'list', - '--cluster-name', f'aks-rpcloud-{self._clusterId}', - '--resource-group', f'rg-rpcloud-{self._clusterId}', - '--query', "'[?starts_with(name,`redpanda`)].vmSize'", - '--output', 'json' + 'kubectl', 'get', 'nodes', + '--selector', 'redpanda-node=true', + '--output', f'jsonpath-as-json="{jsonpath}"' ] # yapf: disable - res = subprocess.check_output(cmd) - resd = json.loads(res) + output = subprocess.check_output(cmd) + self.logger.debug(f'nodes: {output}') + nodes = json.loads(output) - nc = self._configProfile['nodes_count'] - assert len( - resd) == nc, f"expected nodes_count: {nc}, actual: {len(resd)}" + config_nodes_count = self._configProfile['nodes_count'] + actual_nodes_count = len(nodes) + assert actual_nodes_count == config_nodes_count, f"expected nodes_count: {config_nodes_count}, actual: {actual_nodes_count}" - mt = self._configProfile['machine_type'] - assert resd[1] == mt, f"expected machineType: {mt}, actual: {resd[1]}" + config_machine_type = self._configProfile['machine_type'] + actual_machine_type = nodes[0] + assert actual_machine_type == config_machine_type, f"expected machineType: {config_machine_type}, actual: {actual_machine_type}" def _check_gcp_nodes(self): cmd = self.redpanda.kubectl._ssh_prefix() + [ From 95100b9a6c30827d6a3cbb30a037756e93e334a8 Mon Sep 17 00:00:00 2001 From: Alexey Bashtanov Date: Tue, 10 Dec 2024 10:30:13 +0000 Subject: [PATCH 100/229] cloud_storage/partition_manifest/serialize_json: option to omit segments In partition_manifest::serialize_json add an option to dump without segments data. This is to limit the JSON size where it is used for logging. --- src/v/cloud_storage/partition_manifest.cc | 17 ++++++++++------- src/v/cloud_storage/partition_manifest.h | 2 +- 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/src/v/cloud_storage/partition_manifest.cc b/src/v/cloud_storage/partition_manifest.cc index faed1e8b7730..e363a4b51dda 100644 --- a/src/v/cloud_storage/partition_manifest.cc +++ b/src/v/cloud_storage/partition_manifest.cc @@ -2033,15 +2033,18 @@ ss::future partition_manifest::serialize_buf() const { return ss::make_ready_future(to_iobuf()); } -void partition_manifest::serialize_json(std::ostream& out) const { +void partition_manifest::serialize_json( + std::ostream& out, bool include_segments) const { serialization_cursor_ptr c = make_cursor(out); serialize_begin(c); - while (!c->segments_done) { - serialize_segments(c); - } - serialize_replaced(c); - while (!c->spillover_done) { - serialize_spillover(c); + if (include_segments) { + while (!c->segments_done) { + serialize_segments(c); + } + serialize_replaced(c); + while (!c->spillover_done) { + serialize_spillover(c); + } } serialize_end(c); } diff --git a/src/v/cloud_storage/partition_manifest.h b/src/v/cloud_storage/partition_manifest.h index c48a05ff6ab0..c3c04c94b6aa 100644 --- a/src/v/cloud_storage/partition_manifest.h +++ b/src/v/cloud_storage/partition_manifest.h @@ -448,7 +448,7 @@ class partition_manifest : public base_manifest { /// Serialize manifest object /// /// \param out output stream that should be used to output the json - void serialize_json(std::ostream& out) const; + void serialize_json(std::ostream& out, bool include_segments = true) const; // Serialize the manifest to an ss::output_stream in JSON format /// \param out output stream to serialize into; must be kept alive From d7c7b7e94c3394ab01c1c989a3d5d97fc79f9296 Mon Sep 17 00:00:00 2001 From: Alexey Bashtanov Date: Tue, 10 Dec 2024 10:32:21 +0000 Subject: [PATCH 101/229] cloud_storage/partition_manifest: make it printable add operator<< --- src/v/cloud_storage/partition_manifest.cc | 5 +++++ src/v/cloud_storage/partition_manifest.h | 2 ++ 2 files changed, 7 insertions(+) diff --git a/src/v/cloud_storage/partition_manifest.cc b/src/v/cloud_storage/partition_manifest.cc index e363a4b51dda..7b1af4a9fdde 100644 --- a/src/v/cloud_storage/partition_manifest.cc +++ b/src/v/cloud_storage/partition_manifest.cc @@ -2713,4 +2713,9 @@ void partition_manifest::process_anomalies( _last_scrubbed_offset); } +std::ostream& operator<<(std::ostream& o, const partition_manifest& pm) { + pm.serialize_json(o, false); + return o; +} + } // namespace cloud_storage diff --git a/src/v/cloud_storage/partition_manifest.h b/src/v/cloud_storage/partition_manifest.h index c3c04c94b6aa..e03eeef345de 100644 --- a/src/v/cloud_storage/partition_manifest.h +++ b/src/v/cloud_storage/partition_manifest.h @@ -700,4 +700,6 @@ class partition_manifest : public base_manifest { model::offset _applied_offset; }; +std::ostream& operator<<(std::ostream& o, const partition_manifest& f); + } // namespace cloud_storage From 4319c28ca9c07db5a4b1dffaa2dda973e8b78db9 Mon Sep 17 00:00:00 2001 From: Solonas Gousteris Date: Fri, 6 Dec 2024 16:55:41 +0200 Subject: [PATCH 102/229] tests/setup: update ducktape to 0.12 --- tests/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/setup.py b/tests/setup.py index d8dc450fb205..dd8ea2f93ecb 100644 --- a/tests/setup.py +++ b/tests/setup.py @@ -13,7 +13,7 @@ package_data={'': ['*.md']}, include_package_data=True, install_requires=[ - 'ducktape@git+https://github.com/redpanda-data/ducktape.git@0b128ce0aaa1815db9880fb171a4f90cf7b74e43', + 'ducktape@git+https://github.com/redpanda-data/ducktape.git@f51d83f1c035cf77f9389139646f5abdaa6d3648', 'prometheus-client==0.9.0', 'kafka-python==2.0.2', 'crc32c==2.2', From 77c6021d4769a601b71ed87ab34d791237bd155d Mon Sep 17 00:00:00 2001 From: Solonas Gousteris Date: Fri, 6 Dec 2024 19:19:20 +0200 Subject: [PATCH 103/229] update kafkatest version to match with 0.12 --- tests/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/setup.py b/tests/setup.py index dd8ea2f93ecb..dd19a5036073 100644 --- a/tests/setup.py +++ b/tests/setup.py @@ -29,7 +29,7 @@ 'pytest==7.1.2', 'jump-consistent-hash==3.2.0', 'azure-storage-blob==12.14.1', - 'kafkatest@git+https://github.com/apache/kafka.git@e146c7c9164c4e8817fe891ac9d7d1661889b2b4#egg=kafkatest&subdirectory=tests', + 'kafkatest@git+https://github.com/apache/kafka.git@b3939f7901470048a70ee7cbaaa587f32cfac50e#egg=kafkatest&subdirectory=tests', 'grpcio==1.57.0', 'grpcio-tools==1.57', 'grpcio-status==1.57.0', From e4fc766bfa02e112f0195ffe221b63a0a8d3e1ab Mon Sep 17 00:00:00 2001 From: Stephan Dollberg Date: Tue, 10 Dec 2024 09:22:32 +0000 Subject: [PATCH 104/229] storage: Reduce default read-ahead count Lower `storage_read_readahead_count` from 10 to 1: - 1 read ahead at 128KiB readahead size (which is the default for storage_read_buffer_size) should be more than good enough to get good read throughput. Especially given this is per partition so in practice you'd have more than 1 in any case - On the non-TS path the memory used for read ahead is unaccounted for in the kafka/fetch semaphore. Hence this is a OOM risk. Obviously the real fix would be to account for it but the kafka/fetch semaphore accounting is already quite weird anyway so fixing that isn't immediately trivial - On the TS path we do account for the readahead in memory accounting. So that's good. At the same time I think we have lowered it in the past during some incidents to avoid overreading. --- src/v/cloud_storage/tests/remote_partition_fuzz_test.cc | 2 +- src/v/config/configuration.cc | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/v/cloud_storage/tests/remote_partition_fuzz_test.cc b/src/v/cloud_storage/tests/remote_partition_fuzz_test.cc index 1091ec2cd362..356e9c51be1a 100644 --- a/src/v/cloud_storage/tests/remote_partition_fuzz_test.cc +++ b/src/v/cloud_storage/tests/remote_partition_fuzz_test.cc @@ -362,7 +362,7 @@ FIXTURE_TEST( try { auto headers_read = scan_remote_partition_incrementally_with_closest_lso( - *this, base, max, 5, 5); + *this, base, max, 5, 25); vlog(test_log.debug, "{} record batches consumed", headers_read.size()); model::offset expected_offset{0}; size_t ix_header = 0; diff --git a/src/v/config/configuration.cc b/src/v/config/configuration.cc index d9a064eecb5a..2164cb3138f6 100644 --- a/src/v/config/configuration.cc +++ b/src/v/config/configuration.cc @@ -1340,7 +1340,7 @@ configuration::configuration() {.needs_restart = needs_restart::no, .example = "1", .visibility = visibility::tunable}, - 10) + 1) , segment_fallocation_step( *this, "segment_fallocation_step", From 5bfec5452c3b9e509012f7142ffb27e38a93a590 Mon Sep 17 00:00:00 2001 From: Alexey Bashtanov Date: Tue, 10 Dec 2024 10:33:04 +0000 Subject: [PATCH 105/229] c/partition_recovery_manager: restore partition manifest logging Partition manifest logging was removed in #24476 to avoid oversized allocations and very long log lines. This is to re-add it, with segment data limited to last segment only. --- src/v/cloud_storage/partition_manifest.cc | 2 ++ src/v/cluster/partition_recovery_manager.cc | 4 ++++ 2 files changed, 6 insertions(+) diff --git a/src/v/cloud_storage/partition_manifest.cc b/src/v/cloud_storage/partition_manifest.cc index 7b1af4a9fdde..550a48c6a93a 100644 --- a/src/v/cloud_storage/partition_manifest.cc +++ b/src/v/cloud_storage/partition_manifest.cc @@ -2714,7 +2714,9 @@ void partition_manifest::process_anomalies( } std::ostream& operator<<(std::ostream& o, const partition_manifest& pm) { + o << "{manifest: "; pm.serialize_json(o, false); + o << "; last segment: " << pm.last_segment() << "}"; return o; } diff --git a/src/v/cluster/partition_recovery_manager.cc b/src/v/cluster/partition_recovery_manager.cc index 94c656ce2685..fdaba3fec8a3 100644 --- a/src/v/cluster/partition_recovery_manager.cc +++ b/src/v/cluster/partition_recovery_manager.cc @@ -316,6 +316,10 @@ ss::future partition_downloader::download_log() { _ntpc.get_revision(), retention); auto mat = co_await find_recovery_material(); + vlog( + _ctxlog.debug, + "Partition manifest used for recovery: {}", + mat.partition_manifest); if (mat.partition_manifest.size() == 0) { // If the downloaded manifest doesn't have any segments log_recovery_result result{ From 156aeeae425d2ff0cc426387b33e07aff85386d8 Mon Sep 17 00:00:00 2001 From: Evgeny Lazin <4lazin@gmail.com> Date: Tue, 10 Dec 2024 07:46:45 -0500 Subject: [PATCH 106/229] base: Rename assert-unreachable logger to assert Signed-off-by: Evgeny Lazin <4lazin@gmail.com> --- src/v/base/unreachable.h | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/src/v/base/unreachable.h b/src/v/base/unreachable.h index 6149774d92c5..242c407dfcbf 100644 --- a/src/v/base/unreachable.h +++ b/src/v/base/unreachable.h @@ -11,21 +11,17 @@ #pragma once -#include -#include +#include "base/vassert.h" -namespace details { -// NOLINTNEXTLINE -inline seastar::logger _g_unreachable_assert_logger("assert-unreachable"); -} // namespace details +#include // NOLINTNEXTLINE #define unreachable() \ /* NOLINTNEXTLINE(cppcoreguidelines-avoid-do-while) */ \ do { \ - ::details::_g_unreachable_assert_logger.error( \ + ::detail::g_assert_log.l.error( \ "This code should not be reached ({}:{})", __FILE__, __LINE__); \ - ::details::_g_unreachable_assert_logger.error( \ + ::detail::g_assert_log.l.error( \ "Backtrace below:\n{}", seastar::current_backtrace()); \ __builtin_trap(); \ } while (0) From b51d23ef8212c313b4990a21de7662a9c9084a4e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Tue, 10 Dec 2024 16:29:50 +0100 Subject: [PATCH 107/229] ducktape: updated minio MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit We used MinIO version that awas 2 years old. Updated the minio used in ducktape tests. Signed-off-by: Michał Maślanka --- tests/docker/docker-compose.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/docker/docker-compose.yml b/tests/docker/docker-compose.yml index 7357c694f0e2..f14c8df066ec 100644 --- a/tests/docker/docker-compose.yml +++ b/tests/docker/docker-compose.yml @@ -39,7 +39,7 @@ services: - -f - http://localhost:9000/minio/health/live timeout: 20s - image: minio/minio:RELEASE.2023-01-25T00-19-54Z + image: minio/minio:RELEASE.2024-10-29T16-01-48Z networks: redpanda-test: ipv4_address: 192.168.215.125 From 88e21321ab60471a0eb08695a70b74cbbd096c54 Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Tue, 10 Dec 2024 11:13:13 -0500 Subject: [PATCH 108/229] `rptest`: make check for `None` explicit In the case that `offset == max_offset == 0`, checking the boolean `max_offset` will return `False`. Since the intention here is just to check for `None`, make it explicit and avoid this mishandled corner case. --- tests/rptest/tests/datalake/datalake_services.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/rptest/tests/datalake/datalake_services.py b/tests/rptest/tests/datalake/datalake_services.py index 9696860f6b51..5b52f4394732 100644 --- a/tests/rptest/tests/datalake/datalake_services.py +++ b/tests/rptest/tests/datalake/datalake_services.py @@ -191,7 +191,7 @@ def translation_done(): self.redpanda.logger.debug( f"Current translated offsets: {offsets}") return all([ - max_offset and offset <= max_offset + max_offset is not None and offset <= max_offset for _, max_offset in offsets.items() ]) From 4015d4df0ff2c2d1c284cc3e5fcd49533cd71960 Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Sun, 8 Dec 2024 19:41:00 -0800 Subject: [PATCH 109/229] rm_stm/tests: add a high concurrency test for producer eviction --- src/v/cluster/tests/rm_stm_test_fixture.h | 50 +++++++++-- src/v/cluster/tests/rm_stm_tests.cc | 100 +++++++++++++++++++++ src/v/cluster/tests/tx_compaction_tests.cc | 2 + 3 files changed, 147 insertions(+), 5 deletions(-) diff --git a/src/v/cluster/tests/rm_stm_test_fixture.h b/src/v/cluster/tests/rm_stm_test_fixture.h index fc86cdc53e13..2637de1a83e6 100644 --- a/src/v/cluster/tests/rm_stm_test_fixture.h +++ b/src/v/cluster/tests/rm_stm_test_fixture.h @@ -10,7 +10,7 @@ #pragma once #include "cluster/producer_state_manager.h" #include "cluster/rm_stm.h" -#include "config/property.h" +#include "config/mock_property.h" #include "raft/tests/simple_raft_fixture.h" #include @@ -23,12 +23,14 @@ static prefix_logger ctx_logger{logger, ""}; struct rm_stm_test_fixture : simple_raft_fixture { void create_stm_and_start_raft( storage::ntp_config::default_overrides overrides = {}) { + max_concurent_producers.start(std::numeric_limits::max()).get(); + producer_expiration_ms.start(std::chrono::milliseconds::max()).get(); producer_state_manager .start( - config::mock_binding(std::numeric_limits::max()), - config::mock_binding( - std::chrono::duration_cast( - std::chrono::microseconds::max())), + ss::sharded_parameter( + [this] { return max_concurent_producers.local().bind(); }), + ss::sharded_parameter( + [this] { return producer_expiration_ms.local().bind(); }), config::mock_binding(std::numeric_limits::max())) .get(); producer_state_manager @@ -55,6 +57,8 @@ struct rm_stm_test_fixture : simple_raft_fixture { if (_started) { stop_all(); producer_state_manager.stop().get(); + producer_expiration_ms.stop().get(); + max_concurent_producers.stop().get(); } } @@ -66,6 +70,17 @@ struct rm_stm_test_fixture : simple_raft_fixture { return _stm->do_take_local_snapshot(version, {}); } + void update_producer_expiration(std::chrono::milliseconds value) { + producer_expiration_ms + .invoke_on_all( + [value](auto& local) mutable { local.update(std::move(value)); }) + .get(); + } + + auto apply_raft_snapshot(const iobuf& buf) { + return _stm->apply_raft_snapshot(buf); + } + auto apply_snapshot(raft::stm_snapshot_header hdr, iobuf buf) { return _stm->apply_local_snapshot(hdr, std::move(buf)); } @@ -77,6 +92,31 @@ struct rm_stm_test_fixture : simple_raft_fixture { auto get_expired_producers() const { return _stm->get_expired_producers(); } + auto stm_read_lock() { return _stm->_state_lock.hold_read_lock(); } + + auto maybe_create_producer(model::producer_identity pid) { + return stm_read_lock().then([pid, this](auto /*units*/) { + return _stm->maybe_create_producer(pid); + }); + } + + auto reset_producers() { + return _stm->_state_lock.hold_write_lock().then([this](auto units) { + return _stm->reset_producers().then([units = std::move(units)] {}); + }); + } + + auto rearm_eviction_timer(std::chrono::milliseconds period) { + return producer_state_manager + .invoke_on_all([period](auto& mgr) { + return mgr.rearm_eviction_timer_for_testing(period); + }) + .get(); + } + + ss::sharded> max_concurent_producers; + ss::sharded> + producer_expiration_ms; ss::sharded tx_gateway_frontend; ss::sharded producer_state_manager; ss::shared_ptr _stm; diff --git a/src/v/cluster/tests/rm_stm_tests.cc b/src/v/cluster/tests/rm_stm_tests.cc index 8477c91ce4ff..4cbf67791039 100644 --- a/src/v/cluster/tests/rm_stm_tests.cc +++ b/src/v/cluster/tests/rm_stm_tests.cc @@ -12,6 +12,7 @@ #include "cluster/tests/randoms.h" #include "cluster/tests/rm_stm_test_fixture.h" #include "finjector/hbadger.h" +#include "finjector/stress_fiber.h" #include "model/fundamental.h" #include "model/metadata.h" #include "model/record.h" @@ -931,3 +932,102 @@ FIXTURE_TEST(test_tx_expiration_without_data_batches, rm_stm_test_fixture) { != expired.end(); }).get(); } + +/* + * This test ensures concurrent evictions can happen in the presence of + * replication operations and operations that reset the state (snapshots, + * partition stop). + */ +FIXTURE_TEST(test_concurrent_producer_evictions, rm_stm_test_fixture) { + create_stm_and_start_raft(); + auto& stm = *_stm; + stm.start().get0(); + stm.testing_only_disable_auto_abort(); + + wait_for_confirmed_leader(); + wait_for_meta_initialized(); + + // Ensure eviction runs with higher frequency + // and evicts everything possible. + update_producer_expiration(0ms); + rearm_eviction_timer(1ms); + + stress_fiber_manager stress_mgr; + stress_mgr.start( + {.min_spins_per_scheduling_point = random_generators::get_int(50, 100), + .max_spins_per_scheduling_point = random_generators::get_int(500, 1000), + .num_fibers = random_generators::get_int(5, 10)}); + auto stop = ss::defer([&stress_mgr] { stress_mgr.stop().get(); }); + + int64_t counter = 0; + ss::abort_source as; + ss::gate gate; + size_t max_replication_fibers = 1000; + + // simulates replication. + // In each iteration of the loop, we create some producers and randomly + // hold the producer state lock on some of them(thus preventing eviction). + // This is roughly the lifecycle of replicate requests using a producer + // state. This creates stream of producer states in a tight loop, some + // evictable and some non evictable while eviction constantly runs in the + // background. + auto replicate_f = ss::do_until( + [&as] { return as.abort_requested(); }, + [&, this] { + std::vector> spawn_replicate_futures; + for (int i = 0; i < 5; i++) { + auto maybe_replicate_f + = maybe_create_producer(model::producer_identity{counter++, 0}) + .then([&, this](auto result) { + auto producer = result.first; + if ( + gate.get_count() < max_replication_fibers + && tests::random_bool()) { + // simulates replication. + ssx::spawn_with_gate(gate, [this, producer] { + return stm_read_lock().then([producer]( + auto stm_units) { + return producer + ->run_with_lock([](auto units) { + auto sleep_ms + = std::chrono::milliseconds{ + random_generators::get_int(3)}; + return ss::sleep(sleep_ms).finally( + [units = std::move(units)] {}); + }) + .handle_exception_type( + [producer]( + const ss::gate_closed_exception&) { + vlog( + logger.info, + "producer {} already evicted, " + "ignoring", + producer->id()); + }) + .finally( + [producer, + stm_units = std::move(stm_units)] {}); + }); + }); + } + }); + spawn_replicate_futures.push_back(std::move(maybe_replicate_f)); + } + + return ss::when_all_succeed(std::move(spawn_replicate_futures)) + .then([]() { return ss::sleep(1ms); }); + }); + + // simulates raft snapshot application / partition shutdown + // applying a snapshot is stop the world operation that resets + // all the producers. + auto reset_f = ss::do_until( + [&as] { return as.abort_requested(); }, + [&, this] { + return reset_producers().then([] { return ss::sleep(3ms); }); + }); + + ss::sleep(20s).finally([&as] { as.request_abort(); }).get(); + ss::when_all_succeed(std::move(replicate_f), std::move(reset_f)).get(); + gate.close().get(); +} diff --git a/src/v/cluster/tests/tx_compaction_tests.cc b/src/v/cluster/tests/tx_compaction_tests.cc index 41c230b3ab05..3f91805e4577 100644 --- a/src/v/cluster/tests/tx_compaction_tests.cc +++ b/src/v/cluster/tests/tx_compaction_tests.cc @@ -25,6 +25,8 @@ using cluster::tx_executor; _data_dir = "test_dir_" + random_generators::gen_alphanum_string(6); \ stop_all(); \ producer_state_manager.stop().get(); \ + producer_expiration_ms.stop().get(); \ + max_concurent_producers.stop().get(); \ _stm = nullptr; \ }); \ wait_for_confirmed_leader(); \ From 3c473925beee08dbc1c23c2a4ae796588641fa3e Mon Sep 17 00:00:00 2001 From: Chris Lee Date: Tue, 10 Dec 2024 10:27:34 -0700 Subject: [PATCH 110/229] [rptest] Fix "ContainerAlreadyExists" in test_recovery_mode --- tests/rptest/services/redpanda.py | 2 +- tests/rptest/tests/recovery_mode_test.py | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/rptest/services/redpanda.py b/tests/rptest/services/redpanda.py index 9d07dfd43d79..51f80ff0604e 100644 --- a/tests/rptest/services/redpanda.py +++ b/tests/rptest/services/redpanda.py @@ -3383,7 +3383,7 @@ def start_si(self): ) else: raise RuntimeError( - f"Unsuported cloud_storage_type: {self.si_settings.cloud_storage_type}" + f"Unsupported cloud_storage_type: {self.si_settings.cloud_storage_type}" ) if not self.si_settings.bypass_bucket_creation: diff --git a/tests/rptest/tests/recovery_mode_test.py b/tests/rptest/tests/recovery_mode_test.py index 568ef5d032e7..26dbbccdcf20 100644 --- a/tests/rptest/tests/recovery_mode_test.py +++ b/tests/rptest/tests/recovery_mode_test.py @@ -159,7 +159,9 @@ def test_recovery_mode(self): # check that a new node can join the cluster - self.redpanda.start(nodes=[joiner_node], auto_assign_node_id=True) + self.redpanda.start(nodes=[joiner_node], + auto_assign_node_id=True, + start_si=False) self.redpanda.wait_for_membership(first_start=True) # restart the cluster back in normal mode From 1fd8856d4831ca7e3846900f119b2d89a5c6c66a Mon Sep 17 00:00:00 2001 From: Brandon Allard Date: Sun, 8 Dec 2024 19:45:47 -0500 Subject: [PATCH 111/229] treewide: move `s3_fifo` into `src/v/utils` --- src/v/io/BUILD | 2 +- src/v/io/page.cc | 2 +- src/v/io/page.h | 6 +-- src/v/io/page_cache.h | 5 ++- src/v/io/pager.cc | 4 +- src/v/io/pager.h | 5 ++- src/v/io/tests/BUILD | 10 ----- src/v/io/tests/CMakeLists.txt | 1 - src/v/utils/BUILD | 12 ++++++ src/v/{io/cache.h => utils/s3_fifo.h} | 14 +++---- src/v/utils/tests/BUILD | 10 +++++ src/v/utils/tests/CMakeLists.txt | 1 + .../tests/s3_fifo_test.cc} | 42 ++++++++++--------- 13 files changed, 66 insertions(+), 48 deletions(-) rename src/v/{io/cache.h => utils/s3_fifo.h} (98%) rename src/v/{io/tests/cache_test.cc => utils/tests/s3_fifo_test.cc} (95%) diff --git a/src/v/io/BUILD b/src/v/io/BUILD index ad58ee0ba1dc..33463300263c 100644 --- a/src/v/io/BUILD +++ b/src/v/io/BUILD @@ -15,7 +15,6 @@ redpanda_cc_library( "scheduler.cc", ], hdrs = [ - "cache.h", "interval_map.h", "io_queue.h", "page.h", @@ -33,6 +32,7 @@ redpanda_cc_library( "//src/v/container:intrusive", "//src/v/ssx:future_util", "//src/v/ssx:semaphore", + "//src/v/utils:s3_fifo", "@abseil-cpp//absl/container:btree", "@abseil-cpp//absl/container:flat_hash_map", "@boost//:intrusive", diff --git a/src/v/io/page.cc b/src/v/io/page.cc index a1e2439b9804..8439c25a60c9 100644 --- a/src/v/io/page.cc +++ b/src/v/io/page.cc @@ -22,7 +22,7 @@ page::page(uint64_t offset, seastar::temporary_buffer data) page::page( uint64_t offset, seastar::temporary_buffer data, - const class cache_hook& hook) + const class utils::s3_fifo::cache_hook& hook) : cache_hook(hook) , offset_(offset) , size_(data.size()) diff --git a/src/v/io/page.h b/src/v/io/page.h index 4a711e6ba542..8c811311b8c3 100644 --- a/src/v/io/page.h +++ b/src/v/io/page.h @@ -11,7 +11,7 @@ #pragma once #include "container/intrusive_list_helpers.h" -#include "io/cache.h" +#include "utils/s3_fifo.h" #include #include @@ -40,7 +40,7 @@ class page : public seastar::enable_lw_shared_from_this { page( uint64_t offset, seastar::temporary_buffer data, - const cache_hook& hook); + const utils::s3_fifo::cache_hook& hook); page(const page&) = delete; page& operator=(const page&) = delete; @@ -123,7 +123,7 @@ class page : public seastar::enable_lw_shared_from_this { * Page cache entry intrusive list hook. */ // NOLINTNEXTLINE(*-non-private-member-variables-in-classes) - cache_hook cache_hook; + utils::s3_fifo::cache_hook cache_hook; struct waiter { intrusive_list_hook waiter; diff --git a/src/v/io/page_cache.h b/src/v/io/page_cache.h index dee1d7bdb96d..e4792f6f40df 100644 --- a/src/v/io/page_cache.h +++ b/src/v/io/page_cache.h @@ -9,8 +9,8 @@ * by the Apache License, Version 2.0 */ #pragma once -#include "io/cache.h" #include "io/page.h" +#include "utils/s3_fifo.h" namespace experimental::io { @@ -26,7 +26,8 @@ class page_cache { size_t operator()(const page&) noexcept; }; - using cache_type = cache; + using cache_type + = utils::s3_fifo::cache; public: using config = cache_type::config; diff --git a/src/v/io/pager.cc b/src/v/io/pager.cc index eaab1a6a55a7..8eb8ec532953 100644 --- a/src/v/io/pager.cc +++ b/src/v/io/pager.cc @@ -45,8 +45,8 @@ seastar::future<> pager::close() noexcept { } } -seastar::lw_shared_ptr -pager::alloc_page(uint64_t offset, std::optional hook) noexcept { +seastar::lw_shared_ptr pager::alloc_page( + uint64_t offset, std::optional hook) noexcept { auto buf = seastar::temporary_buffer::aligned(page_size, page_size); if (hook.has_value()) { return seastar::make_lw_shared( diff --git a/src/v/io/pager.h b/src/v/io/pager.h index f26b5521992a..10e5aa0879ee 100644 --- a/src/v/io/pager.h +++ b/src/v/io/pager.h @@ -10,9 +10,9 @@ */ #pragma once -#include "io/cache.h" #include "io/page_set.h" #include "io/scheduler.h" +#include "utils/s3_fifo.h" #include #include @@ -92,7 +92,8 @@ class pager { private: static seastar::lw_shared_ptr alloc_page( - uint64_t offset, std::optional hook = std::nullopt) noexcept; + uint64_t offset, + std::optional hook = std::nullopt) noexcept; /* * Read a page from the underlying file. diff --git a/src/v/io/tests/BUILD b/src/v/io/tests/BUILD index 1cf407432fbb..851e19422dc4 100644 --- a/src/v/io/tests/BUILD +++ b/src/v/io/tests/BUILD @@ -29,16 +29,6 @@ redpanda_cc_gtest( ], ) -redpanda_cc_gtest( - name = "cache_test", - timeout = "short", - srcs = ["cache_test.cc"], - deps = [ - "//src/v/io", - "//src/v/test_utils:gtest", - ], -) - redpanda_cc_gtest( name = "interval_map_test", timeout = "short", diff --git a/src/v/io/tests/CMakeLists.txt b/src/v/io/tests/CMakeLists.txt index 3c219aa7ed42..83a491f42f27 100644 --- a/src/v/io/tests/CMakeLists.txt +++ b/src/v/io/tests/CMakeLists.txt @@ -6,7 +6,6 @@ rp_test( SOURCES common.cc common_test.cc - cache_test.cc interval_map_test.cc persistence_test.cc page_test.cc diff --git a/src/v/utils/BUILD b/src/v/utils/BUILD index 8d8901611e74..a1a317a944c7 100644 --- a/src/v/utils/BUILD +++ b/src/v/utils/BUILD @@ -652,3 +652,15 @@ redpanda_cc_library( "@seastar", ], ) + +redpanda_cc_library( + name = "s3_fifo", + hdrs = [ + "s3_fifo.h", + ], + include_prefix = "utils", + deps = [ + "@boost//:intrusive", + "@fmt", + ], +) diff --git a/src/v/io/cache.h b/src/v/utils/s3_fifo.h similarity index 98% rename from src/v/io/cache.h rename to src/v/utils/s3_fifo.h index d2f82eea84e1..c29cf737b751 100644 --- a/src/v/io/cache.h +++ b/src/v/utils/s3_fifo.h @@ -175,7 +175,7 @@ * @{ */ -namespace experimental::io { +namespace utils::s3_fifo { namespace testing_details { class cache_hook_accessor; @@ -674,18 +674,18 @@ bool cache::evict_main() noexcept { * @} */ -} // namespace experimental::io +} // namespace utils::s3_fifo template< typename T, - experimental::io::cache_hook T::*Hook, - experimental::io::cache_evictor Evictor, - experimental::io::cache_cost Cost> -struct fmt::formatter> + utils::s3_fifo::cache_hook T::*Hook, + utils::s3_fifo::cache_evictor Evictor, + utils::s3_fifo::cache_cost Cost> +struct fmt::formatter> : fmt::formatter { template auto format( - const experimental::io::cache& cache, + const utils::s3_fifo::cache& cache, FormatContext& ctx) const { const auto stat = cache.stat(); return fmt::format_to( diff --git a/src/v/utils/tests/BUILD b/src/v/utils/tests/BUILD index d607bbec47e3..4d7f75a3644a 100644 --- a/src/v/utils/tests/BUILD +++ b/src/v/utils/tests/BUILD @@ -513,3 +513,13 @@ redpanda_cc_gtest( "@seastar", ], ) + +redpanda_cc_gtest( + name = "s3_fifo_test", + timeout = "short", + srcs = ["s3_fifo_test.cc"], + deps = [ + "//src/v/test_utils:gtest", + "//src/v/utils:s3_fifo", + ], +) diff --git a/src/v/utils/tests/CMakeLists.txt b/src/v/utils/tests/CMakeLists.txt index 3623a6c27be1..3bd22b260095 100644 --- a/src/v/utils/tests/CMakeLists.txt +++ b/src/v/utils/tests/CMakeLists.txt @@ -31,6 +31,7 @@ rp_test( BINARY_NAME gtest_utils_single_thread SOURCES external_process_test.cc + s3_fifo_test.cc LIBRARIES v::utils v::gtest_main diff --git a/src/v/io/tests/cache_test.cc b/src/v/utils/tests/s3_fifo_test.cc similarity index 95% rename from src/v/io/tests/cache_test.cc rename to src/v/utils/tests/s3_fifo_test.cc index a8b5af31b9ad..3580613a141e 100644 --- a/src/v/io/tests/cache_test.cc +++ b/src/v/utils/tests/s3_fifo_test.cc @@ -8,29 +8,29 @@ * the Business Source License, use of this software will be governed * by the Apache License, Version 2.0 */ -#include "io/cache.h" #include "test_utils/test.h" +#include "utils/s3_fifo.h" #include -namespace experimental::io::testing_details { +namespace utils::s3_fifo::testing_details { class cache_hook_accessor { public: static std::optional - get_hook_insertion_time(const io::cache_hook& hook) { + get_hook_insertion_time(const utils::s3_fifo::cache_hook& hook) { return hook.ghost_insertion_time_; } - static void - set_hook_insertion_time(io::cache_hook& hook, std::optional time) { + static void set_hook_insertion_time( + utils::s3_fifo::cache_hook& hook, std::optional time) { hook.ghost_insertion_time_ = time; } - static uint8_t get_hook_freq(io::cache_hook& hook) { return hook.freq_; } + static uint8_t get_hook_freq(utils::s3_fifo::cache_hook& hook) { + return hook.freq_; + } }; -} // namespace experimental::io::testing_details - -namespace io = experimental::io; +} // namespace utils::s3_fifo::testing_details class CacheTest : public ::testing::Test { public: @@ -39,7 +39,7 @@ class CacheTest : public ::testing::Test { static constexpr auto main_size = cache_size - small_size; struct entry { - io::cache_hook hook; + utils::s3_fifo::cache_hook hook; bool may_evict{true}; bool evicted{false}; }; @@ -58,7 +58,8 @@ class CacheTest : public ::testing::Test { size_t operator()(const entry& /*entry*/) noexcept { return 1; } }; - using cache_type = io::cache; + using cache_type + = utils::s3_fifo::cache; void SetUp() override { cache = std::make_unique(cache_type::config{ @@ -78,19 +79,19 @@ class CacheTest : public ::testing::Test { } static std::optional get_hook_insertion_time(const entry& entry) { - return io::testing_details::cache_hook_accessor:: + return utils::s3_fifo::testing_details::cache_hook_accessor:: get_hook_insertion_time(entry.hook); } static void set_hook_insertion_time(entry& entry, std::optional time) { - io::testing_details::cache_hook_accessor::set_hook_insertion_time( - entry.hook, time); + utils::s3_fifo::testing_details::cache_hook_accessor:: + set_hook_insertion_time(entry.hook, time); } static uint8_t get_hook_freq(entry& entry) { - return io::testing_details::cache_hook_accessor::get_hook_freq( - entry.hook); + return utils::s3_fifo::testing_details::cache_hook_accessor:: + get_hook_freq(entry.hook); } template @@ -700,7 +701,7 @@ TEST_F(CacheTest, Formattable) { TEST(CacheTestCustom, CustomCost) { struct entry { - io::cache_hook hook; + utils::s3_fifo::cache_hook hook; std::string data; }; @@ -713,8 +714,11 @@ TEST(CacheTestCustom, CustomCost) { constexpr auto cache_size = 100; constexpr auto small_size = 5; - using cache_type - = io::cache; + using cache_type = utils::s3_fifo::cache< + entry, + &entry::hook, + utils::s3_fifo::default_cache_evictor, + entry_cost>; cache_type cache( cache_type::config{.cache_size = cache_size, .small_size = small_size}); From f98cb9c4069ad3c9940b433133917a719a7a440d Mon Sep 17 00:00:00 2001 From: Brandon Allard Date: Mon, 9 Dec 2024 11:28:47 -0500 Subject: [PATCH 112/229] utils: add chunked_kv_cache --- src/v/utils/BUILD | 13 ++ src/v/utils/chunked_kv_cache.h | 195 +++++++++++++++++++++ src/v/utils/tests/BUILD | 11 ++ src/v/utils/tests/CMakeLists.txt | 1 + src/v/utils/tests/chunked_kv_cache_test.cc | 122 +++++++++++++ 5 files changed, 342 insertions(+) create mode 100644 src/v/utils/chunked_kv_cache.h create mode 100644 src/v/utils/tests/chunked_kv_cache_test.cc diff --git a/src/v/utils/BUILD b/src/v/utils/BUILD index a1a317a944c7..39549b844910 100644 --- a/src/v/utils/BUILD +++ b/src/v/utils/BUILD @@ -664,3 +664,16 @@ redpanda_cc_library( "@fmt", ], ) + +redpanda_cc_library( + name = "chunked_kv_cache", + hdrs = [ + "chunked_kv_cache.h", + ], + include_prefix = "utils", + deps = [ + "//src/v/container:chunked_hash_map", + "//src/v/utils:s3_fifo", + "@boost//:intrusive", + ], +) diff --git a/src/v/utils/chunked_kv_cache.h b/src/v/utils/chunked_kv_cache.h new file mode 100644 index 000000000000..aa0a2c8e0c19 --- /dev/null +++ b/src/v/utils/chunked_kv_cache.h @@ -0,0 +1,195 @@ +/* + * Copyright 2024 Redpanda Data, Inc. + * + * Use of this software is governed by the Business Source License + * included in the file licenses/BSL.md + * + * As of the Change Date specified in that file, in accordance with + * the Business Source License, use of this software will be governed + * by the Apache License, Version 2.0 + */ + +#pragma once + +#include "container/chunked_hash_map.h" +#include "utils/s3_fifo.h" + +#include +#include + +#include +#include +#include + +namespace utils { + +/** + * A basic key-value cache implementation built on top of the s3_fifo::cache. + */ +template< + typename Key, + typename Value, + typename Hash = std::conditional_t< + detail::has_absl_hash, + detail::avalanching_absl_hash, + ankerl::unordered_dense::hash>, + typename EqualTo = std::equal_to> +class chunked_kv_cache { + struct cached_value; + struct evict; + using cache_t = s3_fifo::cache< + cached_value, + &cached_value::hook, + evict, + s3_fifo::default_cache_cost>; + +public: + using config = cache_t::config; + + explicit chunked_kv_cache(config config) + : _cache{config, evict{*this}} {} + + ~chunked_kv_cache() noexcept = default; + + // These contructors need to be deleted to ensure a stable `this` pointer. + chunked_kv_cache(chunked_kv_cache&&) = delete; + chunked_kv_cache& operator=(chunked_kv_cache&&) noexcept = delete; + chunked_kv_cache(const chunked_kv_cache&) = delete; + chunked_kv_cache& operator=(const chunked_kv_cache&) noexcept = delete; + + /** + * Inserts a value for a given key into the cache. + * + * Returns true if the value was inserted and false if there was already a + * value for the given key in the cache. + */ + bool try_insert(const Key& key, ss::shared_ptr val); + + /** + * Gets the key's corresponding value from the cache. + * + * Returns std::nullopt if the key doesn't have a value in the cache. + */ + ss::optimized_optional> get_value(const Key& key); + + using cache_stat = struct cache_t::stat; + /** + * Cache statistics. + */ + struct stat : public cache_stat { + /// Current size of the cache index. + size_t index_size; + }; + + /** + * Returns the current cache statistics. + */ + [[nodiscard]] stat stat() const noexcept; + +private: + using ghost_hook_t = boost::intrusive::list_member_hook< + boost::intrusive::link_mode>; + + struct cached_value { + Key key; + ss::shared_ptr value; + s3_fifo::cache_hook hook; + ghost_hook_t ghost_hook; + }; + + using entry_t = std::unique_ptr; + using ghost_fifo_t = boost::intrusive::list< + cached_value, + boost::intrusive:: + member_hook>; + + chunked_hash_map _map; + cache_t _cache; + ghost_fifo_t _ghost_fifo; + + void gc_ghost_fifo(); +}; + +template +struct chunked_kv_cache::evict { + chunked_kv_cache& kv_c; + + bool operator()(cached_value& e) noexcept { + e.value = nullptr; + kv_c._ghost_fifo.push_back(e); + return true; + } +}; + +template +bool chunked_kv_cache::try_insert( + const Key& key, ss::shared_ptr val) { + gc_ghost_fifo(); + + auto e_it = _map.find(key); + if (e_it == _map.end()) { + auto [e_it, succ] = _map.try_emplace( + key, std::make_unique(key, std::move(val))); + if (!succ) { + return false; + } + + _cache.insert(*e_it->second); + return true; + } + + auto& entry = *e_it->second; + if (entry.hook.evicted()) { + entry.value = std::move(val); + _ghost_fifo.erase(_ghost_fifo.iterator_to(entry)); + _cache.insert(entry); + return true; + } + + return false; +} + +template +ss::optimized_optional> +chunked_kv_cache::get_value(const Key& key) { + gc_ghost_fifo(); + + auto e_it = _map.find(key); + if (e_it == _map.end()) { + return std::nullopt; + } + + auto& entry = *e_it->second; + if (entry.hook.evicted()) { + return std::nullopt; + } + + entry.hook.touch(); + return entry.value; +} + +template +void chunked_kv_cache::gc_ghost_fifo() { + for (auto it = _ghost_fifo.begin(); it != _ghost_fifo.end();) { + auto& entry = *it; + if (_cache.ghost_queue_contains(entry)) { + // The ghost queue is in fifo-order so any entry that comes after an + // entry that hasn't been evicted will also not be evicted. + return; + } + + it = _ghost_fifo.erase(it); + _map.erase(entry.key); + } +} +template +struct chunked_kv_cache::stat +chunked_kv_cache::stat() const noexcept { + struct stat s { + _cache.stat() + }; + s.index_size = _map.size(); + return s; +} + +} // namespace utils diff --git a/src/v/utils/tests/BUILD b/src/v/utils/tests/BUILD index 4d7f75a3644a..b8b72f887522 100644 --- a/src/v/utils/tests/BUILD +++ b/src/v/utils/tests/BUILD @@ -523,3 +523,14 @@ redpanda_cc_gtest( "//src/v/utils:s3_fifo", ], ) + +redpanda_cc_gtest( + name = "chunked_kv_cache_test", + timeout = "short", + srcs = ["chunked_kv_cache_test.cc"], + deps = [ + "//src/v/test_utils:gtest", + "//src/v/utils:chunked_kv_cache", + "@seastar", + ], +) diff --git a/src/v/utils/tests/CMakeLists.txt b/src/v/utils/tests/CMakeLists.txt index 3bd22b260095..af5f850fb5f8 100644 --- a/src/v/utils/tests/CMakeLists.txt +++ b/src/v/utils/tests/CMakeLists.txt @@ -32,6 +32,7 @@ rp_test( SOURCES external_process_test.cc s3_fifo_test.cc + chunked_kv_cache_test.cc LIBRARIES v::utils v::gtest_main diff --git a/src/v/utils/tests/chunked_kv_cache_test.cc b/src/v/utils/tests/chunked_kv_cache_test.cc new file mode 100644 index 000000000000..cdafdbe70225 --- /dev/null +++ b/src/v/utils/tests/chunked_kv_cache_test.cc @@ -0,0 +1,122 @@ +/* + * Copyright 2024 Redpanda Data, Inc. + * + * Use of this software is governed by the Business Source License + * included in the file licenses/BSL.md + * + * As of the Change Date specified in that file, in accordance with + * the Business Source License, use of this software will be governed + * by the Apache License, Version 2.0 + */ +#include "test_utils/test.h" +#include "utils/chunked_kv_cache.h" + +#include + +TEST(ChunkedKVTest, InsertGetTest) { + using cache_type = utils::chunked_kv_cache; + + cache_type cache(cache_type::config{.cache_size = 2, .small_size = 1}); + auto str = "avaluestr"; + + EXPECT_EQ(cache.try_insert(1, ss::make_shared(str)), true); + auto v = cache.get_value(1); + EXPECT_TRUE(v); + EXPECT_EQ(**v, str); +} + +TEST(ChunkedKVTest, InvalidGetTest) { + using cache_type = utils::chunked_kv_cache; + + cache_type cache(cache_type::config{.cache_size = 2, .small_size = 1}); + + EXPECT_EQ(cache.get_value(0), std::nullopt); +} + +TEST(ChunkedKVTest, ReinsertionTest) { + using cache_type = utils::chunked_kv_cache; + + cache_type cache(cache_type::config{.cache_size = 2, .small_size = 1}); + auto str = "avaluestr"; + + EXPECT_EQ(cache.try_insert(0, ss::make_shared(str)), true); + auto stat = cache.stat(); + EXPECT_EQ(stat.main_queue_size, 0); + EXPECT_EQ(stat.small_queue_size, 1); + EXPECT_EQ(stat.index_size, 1); + + EXPECT_EQ(cache.try_insert(0, ss::make_shared(str)), false); + stat = cache.stat(); + EXPECT_EQ(stat.main_queue_size, 0); + EXPECT_EQ(stat.small_queue_size, 1); + EXPECT_EQ(stat.index_size, 1); +} + +TEST(ChunkedKVTest, EvictionTest) { + using cache_type = utils::chunked_kv_cache; + + cache_type cache(cache_type::config{.cache_size = 2, .small_size = 1}); + auto str = "avaluestr"; + + // Initial phase without any eviction. The `s3_fifo` cache allows for its + // max size to be exceeded by one. + for (int i = 0; i < 3; i++) { + EXPECT_EQ(cache.try_insert(i, ss::make_shared(str)), true); + auto stat = cache.stat(); + EXPECT_EQ(stat.main_queue_size, 0); + EXPECT_EQ(stat.small_queue_size, i + 1); + EXPECT_EQ(stat.index_size, i + 1); + } + + // Next phase with evictions and one ghost queue entry. Ensures that entries + // that have been removed from the ghost queue are also removed from the + // index. + for (int i = 3; i < 10; i++) { + EXPECT_EQ(cache.try_insert(i, ss::make_shared(str)), true); + auto stat = cache.stat(); + EXPECT_EQ(stat.main_queue_size, 0); + EXPECT_EQ(stat.small_queue_size, 3); + EXPECT_EQ(stat.index_size, 4); + } +} + +TEST(ChunkedKVTest, GhostToMainTest) { + using cache_type = utils::chunked_kv_cache; + + cache_type cache(cache_type::config{.cache_size = 4, .small_size = 1}); + auto str = "avaluestr"; + + // Fill the cache + for (int i = 0; i < 5; i++) { + EXPECT_EQ(cache.try_insert(i, ss::make_shared(str)), true); + auto stat = cache.stat(); + EXPECT_EQ(stat.main_queue_size, 0); + EXPECT_EQ(stat.small_queue_size, i + 1); + EXPECT_EQ(stat.index_size, i + 1); + } + + // Move one entry to the ghost queue + EXPECT_EQ(cache.try_insert(5, ss::make_shared(str)), true); + auto stat = cache.stat(); + EXPECT_EQ(stat.main_queue_size, 0); + EXPECT_EQ(stat.small_queue_size, 5); + EXPECT_EQ(stat.index_size, 6); + + // Get the key for the entry in the ghost queue. + int key = -1; + for (int i = 0; i < 5; i++) { + if (!cache.get_value(i)) { + key = i; + break; + } + } + + EXPECT_NE(key, -1); + + // Move entry from ghost queue to main queue. + EXPECT_EQ(cache.try_insert(key, ss::make_shared(str)), true); + stat = cache.stat(); + EXPECT_EQ(stat.main_queue_size, 1); + EXPECT_EQ(stat.small_queue_size, 4); + EXPECT_EQ(stat.index_size, 6); +} From d03a8466ff076cb47afbc1cd099604493465a06e Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Thu, 5 Dec 2024 21:44:59 -0800 Subject: [PATCH 113/229] datalake/verifier: add ability to handle transactions with gaps Currently verifier expects to be able to consume/query until hwm. This does not hold true for transactions due to the offset gaps created by control batches (commit/batches) and aborted data batches. A simple example is when the topic data ends with an aborted transaction resulting in a range of offsets being non consumable. This commit changes the invariants that mark completion of translation to the following 1. All offsets until hwm should be consumed 2. All consumed messages must be translated 3. Max consumption offset should be same max translated offset (1) is where most of the change is as we cannot rely on message offsets to determine if the consumption is complete. Eg: the topic ends with an aborted transaction (which includes 1 or more aborted batches and a control batch that is not propagated to the app layer) --- .../tests/datalake/datalake_verifier.py | 71 +++++++++++++++---- 1 file changed, 59 insertions(+), 12 deletions(-) diff --git a/tests/rptest/tests/datalake/datalake_verifier.py b/tests/rptest/tests/datalake/datalake_verifier.py index 61b04c560587..5ae1676394fc 100644 --- a/tests/rptest/tests/datalake/datalake_verifier.py +++ b/tests/rptest/tests/datalake/datalake_verifier.py @@ -11,13 +11,15 @@ from concurrent.futures import ThreadPoolExecutor import random import threading -import time +from time import time, sleep +from typing import Optional from rptest.clients.rpk import RpkTool from rptest.services.redpanda import RedpandaService from rptest.tests.datalake.query_engine_base import QueryEngineBase from rptest.util import wait_until from confluent_kafka import Consumer +from confluent_kafka import TopicPartition class DatalakeVerifier(): @@ -41,7 +43,18 @@ def __init__(self, redpanda: RedpandaService, topic: str, self.redpanda = redpanda self.topic = topic self.logger = redpanda.logger + # Map from partition id to list of messages consumed + # by from the partition self._consumed_messages = defaultdict(list) + # Maximum offset consumed from the partition. + # Consumed here refers to consumption by the app layer, meaning + # there has to be a valid batch at the offset returned by the + # kafka consume API. + self._max_consumed_offsets = {} + # Next position to be consumed from the partition. This may be + # > max_consumed_offset + 1 if there are gaps from non consumable + # batches like aborted data batches / control batches + self._next_positions = defaultdict(lambda: -1) self._query: QueryEngineBase = query_engine self._cg = f"verifier-group-{random.randint(0, 1000000)}" @@ -71,12 +84,38 @@ def create_consumer(self): def _consumer_thread(self): self.logger.info("Starting consumer thread") consumer = self.create_consumer() - while not self._stop.is_set(): + last_position_update = time() + + def maybe_update_positions(): + nonlocal last_position_update + if time() < last_position_update + 3: + # periodically sweep through all partitions + # and update positions + return + with self._lock: + partitions = [ + TopicPartition(topic=self.topic, partition=p) + for p in self._consumed_messages.keys() + ] + positions = consumer.position(partitions) + for p in positions: + if p.error is not None: + self.logger.warning( + f"Erorr querying position for partition {p.partition}" + ) + else: + self.logger.debug( + f"next position for {p.partition} is {p.offset}") + self._next_positions[p.partition] = p.offset + last_position_update = time() + + while not self._stop.is_set(): self._msg_semaphore.acquire() if self._stop.is_set(): break msg = consumer.poll(1.0) + maybe_update_positions() if msg is None: continue if msg.error(): @@ -90,6 +129,9 @@ def _consumer_thread(self): f"Consumed message partition: {msg.partition()} at offset {msg.offset()}" ) self._consumed_messages[msg.partition()].append(msg) + self._max_consumed_offsets[msg.partition()] = max( + self._max_consumed_offsets.get(msg.partition(), -1), + msg.offset()) if len(self._errors) > 0: return @@ -139,9 +181,7 @@ def _verify_next_message(self, partition, iceberg_offset): def _get_partition_offsets_list(self): with self._lock: - return [(partition, messages[-1].offset()) - for partition, messages in self._consumed_messages.items() - if len(messages) > 0] + return self._next_positions.copy() def _query_thread(self): self.logger.info("Starting query thread") @@ -149,10 +189,11 @@ def _query_thread(self): try: partitions = self._get_partition_offsets_list() - for partition, max_consumed in partitions: + for partition, next_consume_offset in partitions.items(): last_queried_offset = self._max_queried_offsets[ partition] if partition in self._max_queried_offsets else -1 + max_consumed = next_consume_offset - 1 # no new messages consumed, skip query if max_consumed <= last_queried_offset: continue @@ -178,7 +219,7 @@ def _query_thread(self): except Exception as e: self.logger.error(f"Error querying iceberg table: {e}") - time.sleep(2) + sleep(2) def start(self): self._executor.submit(self._consumer_thread) @@ -194,11 +235,16 @@ def _all_offsets_translated(self): ) return False - if self._max_queried_offsets[p.id] < p.high_watermark - 1: + if self._next_positions[p.id] < p.high_watermark: self.logger.debug( - f"partition {p.id} high watermark: {p.high_watermark}, max offset: {self._max_queried_offsets[p.id]}" + f"partition {p.id} high watermark: {p.high_watermark} max offset: {self._next_positions[p.id]} has not been consumed fully" ) return False + # Ensure all the consumed messages are drained. + return all( + len(messages) == 0 + for messages in self._consumed_messages.values()) + return True def _made_progress(self): @@ -237,6 +283,7 @@ def stop(self): self._errors ) == 0, f"Topic {self.topic} validation errors: {self._errors}" - assert all( - len(messages) == 0 for messages in self._consumed_messages.values( - )), f"Partition left with consumed but not translated messages" + self.logger.debug(f"consumed offsets: {self._max_consumed_offsets}") + self.logger.debug(f"queried offsets: {self._max_queried_offsets}") + + assert self._max_queried_offsets == self._max_consumed_offsets, "Mismatch between maximum offsets in topic vs iceberg table" From a145396c39b9a0c67f678791a5625ce0df2ff748 Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Tue, 10 Dec 2024 07:43:29 -0800 Subject: [PATCH 114/229] datalake/tests: test with transactions --- .../tests/datalake/transactions_test.py | 105 ++++++++++++++++++ .../verifiers/compacted_verifier.py | 1 + 2 files changed, 106 insertions(+) create mode 100644 tests/rptest/tests/datalake/transactions_test.py diff --git a/tests/rptest/tests/datalake/transactions_test.py b/tests/rptest/tests/datalake/transactions_test.py new file mode 100644 index 000000000000..4b3b9c3daea2 --- /dev/null +++ b/tests/rptest/tests/datalake/transactions_test.py @@ -0,0 +1,105 @@ +# Copyright 2024 Redpanda Data, Inc. +# +# Use of this software is governed by the Business Source License +# included in the file licenses/BSL.md +# +# As of the Change Date specified in that file, in accordance with +# the Business Source License, use of this software will be governed +# by the Apache License, Version 2.0 + +from rptest.clients.kafka_cat import KafkaCat +from rptest.clients.rpk import TopicSpec +from rptest.services.spark_service import QueryEngineType +from rptest.tests.datalake.datalake_e2e_test import DatalakeServices +from rptest.tests.datalake.datalake_verifier import DatalakeVerifier +from rptest.tests.redpanda_test import RedpandaTest +from rptest.services.redpanda import PandaproxyConfig, SchemaRegistryConfig, SISettings +from rptest.services.redpanda import SISettings +from ducktape.mark import matrix +import confluent_kafka as ck +from rptest.transactions.verifiers.compacted_verifier import CompactedVerifier, Workload +from rptest.services.cluster import cluster +from rptest.tests.datalake.utils import supported_storage_types + + +class DatalakeTransactionTests(RedpandaTest): + def __init__(self, test_ctx, *args, **kwargs): + super(DatalakeTransactionTests, + self).__init__(test_ctx, + num_brokers=1, + si_settings=SISettings(test_context=test_ctx), + extra_rp_conf={ + "iceberg_enabled": "true", + "iceberg_catalog_commit_interval_ms": 5000, + "log_compaction_interval_ms": 1000, + }, + schema_registry_config=SchemaRegistryConfig(), + pandaproxy_config=PandaproxyConfig(), + *args, + **kwargs) + self.test_ctx = test_ctx + self.topic_name = "test" + + def setUp(self): + pass + + def verify_no_aborted_batches( + self, + dl: DatalakeServices, + query_engine: QueryEngineType = QueryEngineType.TRINO): + qe_svc = dl.service(query_engine) + assert qe_svc, f"No query engine of type {query_engine} found" + table = f"redpanda.{qe_svc.escape_identifier(self.topic_name)}" + # This pattern is taken from the format of the abort records + # generated by the verifier. The verifier injects a known pattern + # into all aborted records for future verification. + abort_value_pattern = "'\ta\t'" + query = f"select count(*) from {table} where regexp_like(from_utf8(value), {abort_value_pattern}) = TRUE" + abort_count = qe_svc.run_query_fetch_all(query)[0][0] + if abort_count > 0: + # Dump some diagnostics + query = f"select redpanda.offset from {table} where regexp_like(from_utf8(value), {abort_value_pattern}) = TRUE limit 5" + offsets = qe_svc.run_query_fetch_all(query) + self.redpanda.logger.error( + f"A few offsets with aborted records {offsets}") + assert abort_count == 0, f"{abort_count} aborted records found in iceberg table" + + @cluster(num_nodes=4) + @matrix(cloud_storage_type=supported_storage_types(), compaction=[False]) + def test_with_transactions(self, cloud_storage_type, compaction): + """Test ensures correctness of translation when running with transactions. + Ensures no aborted transactions make it to the iceberg table""" + + tx_verifier = CompactedVerifier(self.test_context, self.redpanda, + Workload.TX) + min_num_records = 1000 + with DatalakeServices(self.test_context, + redpanda=self.redpanda, + filesystem_catalog_mode=False, + include_query_engines=[QueryEngineType.TRINO + ]) as dl: + topic_config = dict() + topic_config[TopicSpec.PROPERTY_SEGMENT_SIZE] = 1 * 1024 * 1024 + if compaction: + topic_config[ + TopicSpec. + PROPERTY_CLEANUP_POLICY] = TopicSpec.CLEANUP_COMPACT + dl.create_iceberg_enabled_topic(self.topic_name, + config=topic_config) + + tx_verifier.start() + # Produces transactional data into the topic with 30% aborted batches + tx_verifier.remote_start_producer(self.redpanda.brokers(), + self.topic_name, + partitions=1) + tx_verifier.ensure_progress(delta=min_num_records, timeout_sec=180) + tx_verifier.remote_stop_producer() + tx_verifier.remote_wait_producer() + tx_verifier.stop() + + # Verify the committed offsets + committed_offset_verifier = DatalakeVerifier( + self.redpanda, self.topic_name, dl.trino()) + committed_offset_verifier.start() + committed_offset_verifier.wait() + self.verify_no_aborted_batches(dl) diff --git a/tests/rptest/transactions/verifiers/compacted_verifier.py b/tests/rptest/transactions/verifiers/compacted_verifier.py index a3341d087049..2e5585e1de70 100644 --- a/tests/rptest/transactions/verifiers/compacted_verifier.py +++ b/tests/rptest/transactions/verifiers/compacted_verifier.py @@ -136,6 +136,7 @@ def start_node(self, node, timeout_sec=10): def stop_node(self, node): node.account.ssh("bash /opt/remote/control/stop.sh rw") + self.wait_node(node, timeout_sec=30) self.raise_on_violation(node) def clean_node(self, node): From 4ee6b02436cdf44d85e6b81f40f6bdfd6647637b Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Tue, 10 Dec 2024 08:16:46 -0800 Subject: [PATCH 115/229] rm_stm: fix a race condition between async cleanup and reset This is a classic iterator invalidation caught by the test added in the previous commit. Cleanup could race with reset thus invalidating the iterator used in max_concurrent_for_each(). --- src/v/cluster/rm_stm.cc | 59 ++++++++++++++++++++++++++++++++--------- src/v/cluster/rm_stm.h | 5 +++- 2 files changed, 50 insertions(+), 14 deletions(-) diff --git a/src/v/cluster/rm_stm.cc b/src/v/cluster/rm_stm.cc index aecb81d61d8f..e74f7cfda543 100644 --- a/src/v/cluster/rm_stm.cc +++ b/src/v/cluster/rm_stm.cc @@ -90,7 +90,8 @@ rm_stm::rm_stm( , _feature_table(feature_table) , _ctx_log(txlog, ssx::sformat("[{}]", c->ntp())) , _producer_state_manager(producer_state_manager) - , _vcluster_id(vcluster_id) { + , _vcluster_id(vcluster_id) + , _producers_pending_cleanup(std::numeric_limits::max()) { setup_metrics(); if (!_is_tx_enabled) { _is_autoabort_enabled = false; @@ -114,6 +115,18 @@ rm_stm::rm_stm( e); }); }); + + ssx::repeat_until_gate_closed_or_aborted(_gate, _as, [this] { + return cleanup_evicted_producers().handle_exception( + [h = _gate.hold(), this](const std::exception_ptr& ex) { + if (!ssx::is_shutdown_exception(ex)) { + vlog( + _ctx_log.warn, + "encountered an exception while cleaning producers: {}", + ex); + } + }); + }); } ss::future rm_stm::bootstrap_committed_offset() { @@ -129,6 +142,8 @@ ss::future rm_stm::bootstrap_committed_offset() { std::pair rm_stm::maybe_create_producer(model::producer_identity pid) { + // note: must be called under state_lock in shared/read mode. + // Double lookup because of two reasons // 1. we are forced to use a ptr as map value_type because producer_state is // not movable @@ -148,23 +163,37 @@ rm_stm::maybe_create_producer(model::producer_identity pid) { return std::make_pair(producer, producer_previously_known::no); } -void rm_stm::cleanup_producer_state(model::producer_identity pid) { - auto it = _producers.find(pid.get_id()); - if (it != _producers.end() && it->second->id() == pid) { - const auto& producer = *(it->second); - if (producer._active_transaction_hook.is_linked()) { - vlog( - _ctx_log.error, - "Ignoring cleanup request of producer {} due to in progress " - "transaction.", - producer); - return; +ss::future<> rm_stm::cleanup_evicted_producers() { + while (!_as.abort_requested() && !_gate.is_closed()) { + auto pid = co_await _producers_pending_cleanup.pop_eventually(); + auto units = co_await _state_lock.hold_read_lock(); + auto it = _producers.find(pid.get_id()); + if (it != _producers.end() && it->second->id() == pid) { + const auto& producer = *(it->second); + if (producer._active_transaction_hook.is_linked()) { + vlog( + _ctx_log.error, + "Ignoring cleanup request of producer {} due to in progress " + "transaction.", + producer); + co_return; + } + _producers.erase(it); + vlog(_ctx_log.trace, "removed producer: {}", pid); } - _producers.erase(it); } +} + +void rm_stm::cleanup_producer_state(model::producer_identity pid) noexcept { + if (_as.abort_requested() || _gate.is_closed()) { + return; + } + _producers_pending_cleanup.push(std::move(pid)); }; ss::future<> rm_stm::reset_producers() { + // note: must always be called under exlusive write lock to + // avoid concurrrent state changes to _producers. co_await ss::max_concurrent_for_each( _producers.begin(), _producers.end(), 32, [this](auto& it) { auto& producer = it.second; @@ -738,6 +767,8 @@ ss::future> rm_stm::do_replicate( ss::future<> rm_stm::stop() { _as.request_abort(); + _producers_pending_cleanup.abort( + std::make_exception_ptr(ss::abort_requested_exception{})); auto_abort_timer.cancel(); co_await _gate.close(); co_await reset_producers(); @@ -1655,6 +1686,8 @@ model::offset rm_stm::to_log_offset(kafka::offset k_offset) const { ss::future<> rm_stm::apply_local_snapshot(raft::stm_snapshot_header hdr, iobuf&& tx_ss_buf) { + auto units = co_await _state_lock.hold_write_lock(); + vlog( _ctx_log.trace, "applying snapshot with last included offset: {}", diff --git a/src/v/cluster/rm_stm.h b/src/v/cluster/rm_stm.h index f31cc1e97b6e..4fe2cd6ee067 100644 --- a/src/v/cluster/rm_stm.h +++ b/src/v/cluster/rm_stm.h @@ -242,7 +242,8 @@ class rm_stm final : public raft::persisted_stm<> { = ss::bool_class; std::pair maybe_create_producer(model::producer_identity); - void cleanup_producer_state(model::producer_identity); + void cleanup_producer_state(model::producer_identity) noexcept; + ss::future<> cleanup_evicted_producers(); ss::future<> reset_producers(); ss::future> do_begin_tx( model::term_id, @@ -414,6 +415,8 @@ class rm_stm final : public raft::persisted_stm<> { // producers because epoch is unused. producers_t _producers; + ss::queue _producers_pending_cleanup; + // All the producers with open transactions in this partition. // The list is sorted by the open transaction begin offset, so // the first entry in the list is the earliest open transaction From 6befcccf1cb4637e395725abfd999b73dfde7c8f Mon Sep 17 00:00:00 2001 From: Andrew Hsu Date: Tue, 10 Dec 2024 19:22:52 -0600 Subject: [PATCH 116/229] gha: add permissions for jira_issue_manage to write issue --- .github/workflows/jira_issue_manage.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/jira_issue_manage.yml b/.github/workflows/jira_issue_manage.yml index 52f088ec5811..88b7a3568997 100644 --- a/.github/workflows/jira_issue_manage.yml +++ b/.github/workflows/jira_issue_manage.yml @@ -16,6 +16,7 @@ jobs: permissions: id-token: write contents: read + issues: write steps: - uses: actions/checkout@v4 - uses: actions/setup-python@v5 From 4e188456fbff2d43242685e0c81e80b0c4eb27dd Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Tue, 10 Dec 2024 12:53:10 -0500 Subject: [PATCH 117/229] `storage`: log on clamped `segment.ms` value Users can configure `log_segment_ms` at the cluster level or `segment.ms` at the topic level. However, the cluster properties `log_segment_ms_min` and `log_segment_ms_max` will quietly clamp the value. Add warn level logging to `disk_log_impl::apply_segment_ms()` so that this clamping is no longer silent, and warns users that they may have a misconfiguration between their topic and cluster level properties. This warning is rate limited to once every 5 minutes to avoid potential spam in the logs. --- src/v/storage/disk_log_impl.cc | 36 ++++++++++++++++++++++++++++------ 1 file changed, 30 insertions(+), 6 deletions(-) diff --git a/src/v/storage/disk_log_impl.cc b/src/v/storage/disk_log_impl.cc index 6a138a938fa4..e27be55ad4b0 100644 --- a/src/v/storage/disk_log_impl.cc +++ b/src/v/storage/disk_log_impl.cc @@ -58,6 +58,7 @@ #include #include +#include #include #include #include @@ -1819,13 +1820,36 @@ ss::future<> disk_log_impl::apply_segment_ms() { auto& local_config = config::shard_local_cfg(); // clamp user provided value with (hopefully sane) server min and max // values, this should protect against overflow UB - if ( - first_write_ts.value() - + std::clamp( + const auto clamped_seg_ms = std::clamp( + seg_ms.value(), + local_config.log_segment_ms_min(), + local_config.log_segment_ms_max()); + + static constexpr auto rate_limit = std::chrono::minutes(5); + thread_local static ss::logger::rate_limit rate(rate_limit); + if (seg_ms < clamped_seg_ms) { + stlog.log( + ss::log_level::warn, + rate, + "Configured segment.ms={} is lower than the configured cluster " + "bound {}={}", + seg_ms.value(), + local_config.log_segment_ms_min.name(), + local_config.log_segment_ms_min()); + } + + if (seg_ms > clamped_seg_ms) { + stlog.log( + ss::log_level::warn, + rate, + "Configured segment.ms={} is higher than the configured cluster " + "bound {}={}", seg_ms.value(), - local_config.log_segment_ms_min(), - local_config.log_segment_ms_max()) - > ss::lowres_clock::now()) { + local_config.log_segment_ms_max.name(), + local_config.log_segment_ms_max()); + } + + if (first_write_ts.value() + clamped_seg_ms > ss::lowres_clock::now()) { // skip, time hasn't expired co_return; } From b90389ec1990d3aec3c3565f869874db6c1450a4 Mon Sep 17 00:00:00 2001 From: Alexey Bashtanov Date: Wed, 11 Dec 2024 09:53:55 +0000 Subject: [PATCH 118/229] raft/tests/reconfiguration: allow more memory raft_fixture.test_force_reconfiguration often fails with failed allocations within 2-4 seconds from start --- src/v/raft/tests/BUILD | 2 +- src/v/raft/tests/CMakeLists.txt | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/v/raft/tests/BUILD b/src/v/raft/tests/BUILD index b371936d996a..73707bd1dae5 100644 --- a/src/v/raft/tests/BUILD +++ b/src/v/raft/tests/BUILD @@ -610,7 +610,7 @@ redpanda_cc_gtest( "raft_reconfiguration_test.cc", ], cpu = 4, - memory = "2GiB", + memory = "4GiB", tags = ["exclusive"], deps = [ "//src/v/base", diff --git a/src/v/raft/tests/CMakeLists.txt b/src/v/raft/tests/CMakeLists.txt index 75f3e84f5100..819e8a1e947f 100644 --- a/src/v/raft/tests/CMakeLists.txt +++ b/src/v/raft/tests/CMakeLists.txt @@ -65,7 +65,7 @@ rp_test( SOURCES ${gsrcs} LIBRARIES v::raft v::storage_test_utils v::model_test_utils v::features v::gtest_main LABELS raft - ARGS "-- -c 8 -m 2G" + ARGS "-- -c 8 -m 4G" ) rp_test( From f34856c2b74c41f55c64f80c78499665bba652fa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Wed, 11 Dec 2024 13:15:38 +0100 Subject: [PATCH 119/229] raft/tests: added option to enable offset trannslation in tests MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Michał Maślanka --- src/v/raft/tests/raft_fixture.cc | 24 +++++++++++++++++------- src/v/raft/tests/raft_fixture.h | 10 ++++++++-- 2 files changed, 25 insertions(+), 9 deletions(-) diff --git a/src/v/raft/tests/raft_fixture.cc b/src/v/raft/tests/raft_fixture.cc index c9649f6f62a4..c4f632ec9cab 100644 --- a/src/v/raft/tests/raft_fixture.cc +++ b/src/v/raft/tests/raft_fixture.cc @@ -346,7 +346,8 @@ raft_node_instance::raft_node_instance( leader_update_clb_t leader_update_clb, bool enable_longest_log_detection, config::binding election_timeout, - config::binding heartbeat_interval) + config::binding heartbeat_interval, + bool with_offset_translation) : raft_node_instance( id, revision, @@ -357,7 +358,8 @@ raft_node_instance::raft_node_instance( std::move(leader_update_clb), enable_longest_log_detection, std::move(election_timeout), - std::move(heartbeat_interval)) {} + std::move(heartbeat_interval), + with_offset_translation) {} raft_node_instance::raft_node_instance( model::node_id id, @@ -368,7 +370,8 @@ raft_node_instance::raft_node_instance( leader_update_clb_t leader_update_clb, bool enable_longest_log_detection, config::binding election_timeout, - config::binding heartbeat_interval) + config::binding heartbeat_interval, + bool with_offset_translation) : _id(id) , _revision(revision) , _logger(test_log, fmt::format("[node: {}]", _id)) @@ -391,7 +394,8 @@ raft_node_instance::raft_node_instance( , _leader_clb(std::move(leader_update_clb)) , _enable_longest_log_detection(enable_longest_log_detection) , _election_timeout(std::move(election_timeout)) - , _heartbeat_interval(std::move(heartbeat_interval)) { + , _heartbeat_interval(std::move(heartbeat_interval)) + , _with_offset_translation(with_offset_translation) { config::shard_local_cfg().disable_metrics.set_value(true); } @@ -424,7 +428,11 @@ raft_node_instance::initialise(std::vector initial_nodes) { co_await _storage.invoke_on_all(&storage::api::start); storage::ntp_config ntp_cfg(ntp(), _base_directory); - auto log = co_await _storage.local().log_mgr().manage(std::move(ntp_cfg)); + auto log = co_await _storage.local().log_mgr().manage( + std::move(ntp_cfg), + test_group, + _with_offset_translation ? model::offset_translator_batch_types() + : std::vector{}); _raft = ss::make_lw_shared( _id, @@ -591,7 +599,8 @@ raft_fixture::add_node(model::node_id id, model::revision_id rev) { }, _enable_longest_log_detection, _election_timeout.bind(), - _heartbeat_interval.bind()); + _heartbeat_interval.bind(), + _with_offset_translation); auto [it, success] = _nodes.emplace(id, std::move(instance)); return *it->second; @@ -613,7 +622,8 @@ raft_node_instance& raft_fixture::add_node( }, _enable_longest_log_detection, _election_timeout.bind(), - _heartbeat_interval.bind()); + _heartbeat_interval.bind(), + _with_offset_translation); auto [it, success] = _nodes.emplace(id, std::move(instance)); return *it->second; diff --git a/src/v/raft/tests/raft_fixture.h b/src/v/raft/tests/raft_fixture.h index 1b85afb943d3..3baf664a2b46 100644 --- a/src/v/raft/tests/raft_fixture.h +++ b/src/v/raft/tests/raft_fixture.h @@ -158,7 +158,8 @@ class raft_node_instance : public ss::weakly_referencable { leader_update_clb_t leader_update_clb, bool enable_longest_log_detection, config::binding election_timeout, - config::binding heartbeat_interval); + config::binding heartbeat_interval, + bool with_offset_translation = false); raft_node_instance( model::node_id id, @@ -168,7 +169,8 @@ class raft_node_instance : public ss::weakly_referencable { leader_update_clb_t leader_update_clb, bool enable_longest_log_detection, config::binding election_timeout, - config::binding heartbeat_interval); + config::binding heartbeat_interval, + bool with_offset_translation = false); raft_node_instance(const raft_node_instance&) = delete; raft_node_instance(raft_node_instance&&) noexcept = delete; @@ -265,6 +267,7 @@ class raft_node_instance : public ss::weakly_referencable { bool _enable_longest_log_detection; config::binding _election_timeout; config::binding _heartbeat_interval; + bool _with_offset_translation; }; class raft_fixture @@ -530,6 +533,8 @@ class raft_fixture _heartbeat_interval.update(std::move(timeout)); } + void enable_offset_translation() { _with_offset_translation = true; } + protected: class raft_not_leader_exception : std::exception {}; @@ -561,6 +566,7 @@ class raft_fixture std::optional _leader_clb; config::mock_property _election_timeout{500ms}; config::mock_property _heartbeat_interval{50ms}; + bool _with_offset_translation = false; }; template From f454ef290e3d3a476f9a10158281f390acf8efa6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Wed, 11 Dec 2024 13:16:44 +0100 Subject: [PATCH 120/229] datalake/tests: enable offset translation in state machine test MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Michał Maślanka --- src/v/datalake/translation/tests/state_machine_test.cc | 1 + 1 file changed, 1 insertion(+) diff --git a/src/v/datalake/translation/tests/state_machine_test.cc b/src/v/datalake/translation/tests/state_machine_test.cc index dc026b9abe67..b276a55e3ba6 100644 --- a/src/v/datalake/translation/tests/state_machine_test.cc +++ b/src/v/datalake/translation/tests/state_machine_test.cc @@ -112,6 +112,7 @@ struct translator_stm_fixture : stm_raft_fixture { }; TEST_F_CORO(translator_stm_fixture, state_machine_ops) { + enable_offset_translation(); co_await initialize_state_machines(); co_await wait_for_leader(5s); scoped_config config; From 8985f5762f42b0326e0a3f444b1d18cd1e0de8a3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Wed, 11 Dec 2024 13:17:20 +0100 Subject: [PATCH 121/229] translation_stm: reset highest translated offset on raft snapshot MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit When an stm receives Raft snapshot it indicates the whole in memory state of that state machine should be replaced with the state from the snapshot. The datalake translation state machine was incorrectly handling raft snapshot which lead to its state being out of date after the snapshot is applied. Raft snapshot for translation_stm is empty so the correct action is to reset the state machine state and wait for the update to be applied. Fixes: CORE-8485 Signed-off-by: Michał Maślanka --- src/v/datalake/translation/state_machine.cc | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/v/datalake/translation/state_machine.cc b/src/v/datalake/translation/state_machine.cc index 07b0741393a4..a4064c5325b3 100644 --- a/src/v/datalake/translation/state_machine.cc +++ b/src/v/datalake/translation/state_machine.cc @@ -138,7 +138,14 @@ translation_stm::take_local_snapshot(ssx::semaphore_units apply_units) { co_return raft::stm_snapshot::create(0, snapshot_offset, std::move(result)); } -ss::future<> translation_stm::apply_raft_snapshot(const iobuf&) { co_return; } +ss::future<> translation_stm::apply_raft_snapshot(const iobuf&) { + // reset offset to not initalized when handling Raft snapshot, this way + // state machine will not hold any obsolete state that should be overriden + // with the snapshot. + vlog(_log.debug, "Applying raft snapshot, resetting state"); + _highest_translated_offset = kafka::offset{}; + co_return; +} ss::future translation_stm::take_snapshot(model::offset) { co_return iobuf{}; From 8885778cca24c6d133d5bbd2abfc6ab07c0d6e98 Mon Sep 17 00:00:00 2001 From: Ben Pope Date: Mon, 2 Dec 2024 20:37:36 +0000 Subject: [PATCH 122/229] security: Extract validate_scram_credential Pure refactor, no change in behaviour. Signed-off-by: Ben Pope --- src/v/security/request_auth.cc | 25 ++++--------------------- src/v/security/scram_authenticator.cc | 14 ++++++++++++++ src/v/security/scram_authenticator.h | 4 ++++ 3 files changed, 22 insertions(+), 21 deletions(-) diff --git a/src/v/security/request_auth.cc b/src/v/security/request_auth.cc index eaf12e5eb8e6..63743bad2ac4 100644 --- a/src/v/security/request_auth.cc +++ b/src/v/security/request_auth.cc @@ -17,7 +17,6 @@ #include "seastar/http/exception.hh" #include "security/credential_store.h" #include "security/oidc_authenticator.h" -#include "security/scram_algorithm.h" #include "security/scram_authenticator.h" #include "security/types.h" @@ -124,25 +123,9 @@ request_auth_result request_authenticator::do_authenticate( throw unauthorized_user_exception( std::move(username), "Unauthorized"); } else { - const auto& cred = cred_opt.value(); - ss::sstring sasl_mechanism; - bool is_valid{false}; - if (security::scram_sha256::validate_password( - password, - cred.stored_key(), - cred.salt(), - cred.iterations())) { - is_valid = true; - sasl_mechanism = security::scram_sha256_authenticator::name; - } else if (security::scram_sha512::validate_password( - password, - cred.stored_key(), - cred.salt(), - cred.iterations())) { - is_valid = true; - sasl_mechanism = security::scram_sha512_authenticator::name; - } - if (!is_valid) { + auto sasl_mechanism = validate_scram_credential( + *cred_opt, password); + if (!sasl_mechanism.has_value()) { // User found, password doesn't match vlog( logger.warn, @@ -159,7 +142,7 @@ request_auth_result request_authenticator::do_authenticate( return request_auth_result( std::move(username), std::move(password), - std::move(sasl_mechanism), + ss::sstring{*sasl_mechanism}, request_auth_result::superuser(superuser)); } } diff --git a/src/v/security/scram_authenticator.cc b/src/v/security/scram_authenticator.cc index 9dc87af60fa3..833334249927 100644 --- a/src/v/security/scram_authenticator.cc +++ b/src/v/security/scram_authenticator.cc @@ -137,4 +137,18 @@ scram_authenticator::authenticate(bytes auth_bytes) { template class scram_authenticator; template class scram_authenticator; +std::optional validate_scram_credential( + const scram_credential& cred, const credential_password& password) { + std::optional sasl_mechanism; + if (security::scram_sha256::validate_password( + password, cred.stored_key(), cred.salt(), cred.iterations())) { + sasl_mechanism = security::scram_sha256_authenticator::name; + } else if (security::scram_sha512::validate_password( + password, cred.stored_key(), cred.salt(), cred.iterations())) { + sasl_mechanism = security::scram_sha512_authenticator::name; + } + + return sasl_mechanism; +} + } // namespace security diff --git a/src/v/security/scram_authenticator.h b/src/v/security/scram_authenticator.h index a3509cddb6f5..31858390e6f3 100644 --- a/src/v/security/scram_authenticator.h +++ b/src/v/security/scram_authenticator.h @@ -13,6 +13,7 @@ #include "security/fwd.h" #include "security/sasl_authentication.h" #include "security/scram_algorithm.h" +#include "security/types.h" namespace security { @@ -80,4 +81,7 @@ struct scram_sha512_authenticator { static constexpr const char* name = "SCRAM-SHA-512"; }; +std::optional validate_scram_credential( + const scram_credential& cred, const credential_password& password); + } // namespace security From 848a3ffaa432b42d74b82ef55819a97c29df4b39 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Wed, 11 Dec 2024 15:35:36 +0100 Subject: [PATCH 123/229] tests: reduce logging in random_node_operations_test MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Random node operations tests logs can grow into multiple gigabytes. Reduced the logging level to make the test logs easier to handle. Signed-off-by: Michał Maślanka --- tests/rptest/tests/random_node_operations_test.py | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/tests/rptest/tests/random_node_operations_test.py b/tests/rptest/tests/random_node_operations_test.py index e9a8c3b4ba32..b59526912bec 100644 --- a/tests/rptest/tests/random_node_operations_test.py +++ b/tests/rptest/tests/random_node_operations_test.py @@ -23,7 +23,7 @@ from rptest.clients.types import TopicSpec from rptest.clients.default import DefaultClient from rptest.services.kgo_verifier_services import KgoVerifierConsumerGroupConsumer, KgoVerifierProducer -from rptest.services.redpanda import CHAOS_LOG_ALLOW_LIST, PREV_VERSION_LOG_ALLOW_LIST, CloudStorageType, PandaproxyConfig, SISettings, SchemaRegistryConfig +from rptest.services.redpanda import CHAOS_LOG_ALLOW_LIST, PREV_VERSION_LOG_ALLOW_LIST, CloudStorageType, LoggingConfig, PandaproxyConfig, SISettings, SchemaRegistryConfig from rptest.services.redpanda_installer import RedpandaInstaller from rptest.utils.mode_checks import cleanup_on_early_exit, skip_debug_mode, skip_fips_mode from rptest.utils.node_operations import FailureInjectorBackgroundThread, NodeOpsExecutor, generate_random_workload @@ -62,6 +62,14 @@ def __init__(self, test_context, *args, **kwargs): node_prealloc_count=3, schema_registry_config=SchemaRegistryConfig(), pandaproxy_config=PandaproxyConfig(), + log_config=LoggingConfig( + 'info', { + 'storage-resources': 'warn', + 'storage-gc': 'warn', + 'raft': 'debug', + 'cluster': 'debug', + 'datalake': 'debug', + }), *args, **kwargs) self.nodes_with_prev_version = [] From 201739c6e3607ca4429856d1e745d68328d87bbe Mon Sep 17 00:00:00 2001 From: Evgeny Lazin <4lazin@gmail.com> Date: Wed, 11 Dec 2024 11:35:44 -0500 Subject: [PATCH 124/229] http: Handle nested exception correctly If the nested exception contains simple connection errors (broken pipe/connection refused/connection reset) don't log it on ERROR level and treat it as recoverable. Signed-off-by: Evgeny Lazin <4lazin@gmail.com> --- src/v/cloud_storage_clients/util.cc | 35 ++++++++++++++++++++++++++--- 1 file changed, 32 insertions(+), 3 deletions(-) diff --git a/src/v/cloud_storage_clients/util.cc b/src/v/cloud_storage_clients/util.cc index f3e6ccef3fe0..62442b206971 100644 --- a/src/v/cloud_storage_clients/util.cc +++ b/src/v/cloud_storage_clients/util.cc @@ -15,8 +15,13 @@ #include "net/connection.h" #include "utils/retry_chain_node.h" +#include + #include +#include +#include + namespace { bool is_abort_or_gate_close_exception(const std::exception_ptr& ex) { @@ -40,6 +45,28 @@ bool has_abort_or_gate_close_exception(const ss::nested_exception& ex) { || is_abort_or_gate_close_exception(ex.outer); } +bool is_nested_reconnect_error(const ss::nested_exception& ex) { + try { + std::rethrow_exception(ex.inner); + } catch (const std::system_error& e) { + if (!net::is_reconnect_error(e)) { + return false; + } + } catch (...) { + return false; + } + try { + std::rethrow_exception(ex.outer); + } catch (const std::system_error& e) { + if (!net::is_reconnect_error(e)) { + return false; + } + } catch (...) { + return false; + } + return true; +} + template error_outcome handle_client_transport_error( std::exception_ptr current_exception, Logger& logger) { @@ -103,10 +130,12 @@ error_outcome handle_client_transport_error( if (has_abort_or_gate_close_exception(ex)) { vlog(logger.debug, "Nested abort or gate closed: {}", ex); throw; + } else if (is_nested_reconnect_error(ex)) { + vlog(logger.warn, "Connection error {}", std::current_exception()); + } else { + vlog(logger.error, "Unexpected error {}", std::current_exception()); + outcome = error_outcome::fail; } - - vlog(logger.error, "Unexpected error {}", std::current_exception()); - outcome = error_outcome::fail; } catch (...) { vlog(logger.error, "Unexpected error {}", std::current_exception()); outcome = error_outcome::fail; From 26d4939d9172ff704584bce30df0b63faa4937cb Mon Sep 17 00:00:00 2001 From: Evgeny Lazin <4lazin@gmail.com> Date: Wed, 11 Dec 2024 11:39:10 -0500 Subject: [PATCH 125/229] cloud_io: Treat 'unknown' backend as a valid case The switch stmnt in the 'remote' class triggers 'unreachable' error in case if the backend is unknonw. This is not valid because the backend type is detected from the configuration and we shouldn't crash because configuration is unexpected. Also, it's safe to assume that many cloud storage providers which implement S3 compatible REST API will be detected as 'unknown' and should be treated as S3 compactible. Signed-off-by: Evgeny Lazin <4lazin@gmail.com> --- src/v/cloud_io/remote.cc | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/v/cloud_io/remote.cc b/src/v/cloud_io/remote.cc index 2857c31949af..7dcdc7a3da79 100644 --- a/src/v/cloud_io/remote.cc +++ b/src/v/cloud_io/remote.cc @@ -66,6 +66,9 @@ cloud_io::provider infer_provider( model::cloud_storage_backend backend, const cloud_storage_clients::client_configuration& conf) { switch (backend) { + case model::cloud_storage_backend::unknown: + // NOTE: treat unknown cloud storage backend as a valid case + // in which we're assuming S3 compatible storage. case model::cloud_storage_backend::aws: case model::cloud_storage_backend::minio: case model::cloud_storage_backend::oracle_s3_compat: @@ -78,8 +81,6 @@ cloud_io::provider infer_provider( .account_name = abs.storage_account_name(), }; } - case model::cloud_storage_backend::unknown: - unreachable(); } } From 44af2e38fd76a365147d7361be4ba7b3b67a66cc Mon Sep 17 00:00:00 2001 From: Stephan Dollberg Date: Mon, 9 Dec 2024 10:44:29 +0000 Subject: [PATCH 126/229] chunk_cache: Add metrics Adds metrics to track chunk_cache usage: - total & available size: These export the sizes which are already tracked by the chunk cache. Difference between the two is the actual in-use memory. - wait count: Counts the amount of times we had to wait to get a chunk from the chunk cache. This is important because the above gauges might not represent micro-behaviour that happens inbetween scrapes. --- src/v/redpanda/application.cc | 5 ++ src/v/storage/BUILD | 1 + src/v/storage/chunk_cache.cc | 51 +++++++++++++++++-- src/v/storage/chunk_cache.h | 7 +++ .../tests/log_segment_appender_test.cc | 2 +- 5 files changed, 61 insertions(+), 5 deletions(-) diff --git a/src/v/redpanda/application.cc b/src/v/redpanda/application.cc index 67cdf5b9f738..42ee030f2abe 100644 --- a/src/v/redpanda/application.cc +++ b/src/v/redpanda/application.cc @@ -2455,6 +2455,11 @@ void application::wire_up_bootstrap_services() { ss::smp::invoke_on_all([] { return storage::internal::chunks().start(); }).get(); + _deferred.emplace_back([] { + ss::smp::invoke_on_all([] { + return storage::internal::chunks().stop(); + }).get(); + }); construct_service(stress_fiber_manager).get(); syschecks::systemd_message("Constructing storage services").get(); construct_single_service_sharded( diff --git a/src/v/storage/BUILD b/src/v/storage/BUILD index 6c6b890cf5ae..cf2cc9bde624 100644 --- a/src/v/storage/BUILD +++ b/src/v/storage/BUILD @@ -187,6 +187,7 @@ redpanda_cc_library( deps = [ "//src/v/base", "//src/v/container:intrusive", + "//src/v/metrics", "//src/v/ssx:semaphore", "//src/v/utils:named_type", "@seastar", diff --git a/src/v/storage/chunk_cache.cc b/src/v/storage/chunk_cache.cc index 19d3d840843d..1709d400bb7c 100644 --- a/src/v/storage/chunk_cache.cc +++ b/src/v/storage/chunk_cache.cc @@ -11,6 +11,7 @@ #include "storage/chunk_cache.h" #include "config/configuration.h" +#include "metrics/prometheus_sanitize.h" #include "resource_mgmt/memory_groups.h" #include @@ -25,6 +26,7 @@ chunk_cache::chunk_cache() noexcept , _chunk_size(config::shard_local_cfg().append_chunk_size()) {} ss::future<> chunk_cache::start() { + setup_metrics(); const auto num_chunks = memory_groups().chunk_cache_min_memory() / _chunk_size; return ss::do_for_each( @@ -37,6 +39,38 @@ ss::future<> chunk_cache::start() { }); } +ss::future<> chunk_cache::stop() { + _metrics.clear(); + return ss::now(); +} + +void chunk_cache::setup_metrics() { + if (config::shard_local_cfg().disable_metrics()) { + return; + } + + namespace sm = ss::metrics; + _metrics.add_group( + prometheus_sanitize::metrics_name("chunk_cache"), + { + sm::make_gauge( + "total_size_bytes", + [this] { return _size_total; }, + sm::description("Total size of all segment appender chunks in any " + "state, in bytes.")), + sm::make_gauge( + "available_size_bytes", + [this] { return _size_available; }, + sm::description("Total size of all free segment appender chunks in " + "the cache, in bytes.")), + sm::make_counter( + "wait_count", + [this] { return _wait_for_chunk_count; }, + sm::description("Count of how many times we had to wait for a chunk " + "to become available")), + }); +} + void chunk_cache::add(const chunk_ptr& chunk) { if (_size_available >= _size_target) { _size_total -= _chunk_size; @@ -54,16 +88,25 @@ ss::future chunk_cache::get() { if (!_sem.waiters()) { return do_get(); } - return ss::get_units(_sem, 1).then( - [this](ssx::semaphore_units) { return do_get(); }); + + return wait_and_get(); } ss::future chunk_cache::do_get() { if (auto c = pop_or_allocate(); c) { return ss::make_ready_future(c); } - return ss::get_units(_sem, 1).then( - [this](ssx::semaphore_units) { return do_get(); }); + + return wait_and_get(); +} + +ss::future chunk_cache::wait_and_get() { + auto fut = ss::get_units(_sem, 1); + if (_sem.waiters()) { + _wait_for_chunk_count++; + } + + return fut.then([this](ssx::semaphore_units) { return do_get(); }); } chunk_cache::chunk_ptr chunk_cache::pop_or_allocate() { diff --git a/src/v/storage/chunk_cache.h b/src/v/storage/chunk_cache.h index 2371fedf5aa0..e48050548508 100644 --- a/src/v/storage/chunk_cache.h +++ b/src/v/storage/chunk_cache.h @@ -11,6 +11,7 @@ #pragma once #include "base/seastarx.h" +#include "metrics/metrics.h" #include "ssx/semaphore.h" #include "storage/segment_appender_chunk.h" @@ -42,6 +43,7 @@ class chunk_cache { ~chunk_cache() noexcept = default; ss::future<> start(); + ss::future<> stop(); void add(const chunk_ptr& chunk); @@ -51,6 +53,8 @@ class chunk_cache { private: ss::future do_get(); + void setup_metrics(); + ss::future wait_and_get(); chunk_ptr pop_or_allocate(); @@ -62,6 +66,9 @@ class chunk_cache { const size_t _size_limit; const size_t _chunk_size{0}; + + size_t _wait_for_chunk_count{0}; + metrics::internal_metric_groups _metrics; }; chunk_cache& chunks(); diff --git a/src/v/storage/tests/log_segment_appender_test.cc b/src/v/storage/tests/log_segment_appender_test.cc index ce1008bef9da..7c8f4eb97e70 100644 --- a/src/v/storage/tests/log_segment_appender_test.cc +++ b/src/v/storage/tests/log_segment_appender_test.cc @@ -123,7 +123,7 @@ iobuf make_iobuf_with_char(size_t len, unsigned char c) { return ret; } -size_t default_chunk_size() { return internal::chunks().chunk_size(); } +size_t default_chunk_size() { return storage::internal::chunks().chunk_size(); } } // namespace From 53736c377461f4faa05d7f310e49acd5a1cc57a6 Mon Sep 17 00:00:00 2001 From: Stephan Dollberg Date: Mon, 9 Dec 2024 10:46:06 +0000 Subject: [PATCH 127/229] memory_groups: 10x mem group shares Multiply all shares by 10 such that we can represent "half" shares. --- src/v/resource_mgmt/memory_groups.cc | 14 ++++++------ .../resource_mgmt/tests/memory_groups_test.cc | 22 +++++++++---------- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/src/v/resource_mgmt/memory_groups.cc b/src/v/resource_mgmt/memory_groups.cc index 94abd65d26e1..f1f5bf18ac51 100644 --- a/src/v/resource_mgmt/memory_groups.cc +++ b/src/v/resource_mgmt/memory_groups.cc @@ -31,13 +31,13 @@ bool datalake_enabled() { } struct memory_shares { - constexpr static size_t chunk_cache = 3; - constexpr static size_t kafka = 3; - constexpr static size_t rpc = 2; - constexpr static size_t recovery = 1; - constexpr static size_t tiered_storage = 1; - constexpr static size_t data_transforms = 1; - constexpr static size_t datalake = 1; + constexpr static size_t chunk_cache = 30; + constexpr static size_t kafka = 30; + constexpr static size_t rpc = 20; + constexpr static size_t recovery = 10; + constexpr static size_t tiered_storage = 10; + constexpr static size_t data_transforms = 10; + constexpr static size_t datalake = 10; static size_t total_shares(bool with_wasm, bool with_datalake) { size_t total = chunk_cache + kafka + rpc + recovery + tiered_storage; diff --git a/src/v/resource_mgmt/tests/memory_groups_test.cc b/src/v/resource_mgmt/tests/memory_groups_test.cc index 6801210b1b83..ca5d1146ff59 100644 --- a/src/v/resource_mgmt/tests/memory_groups_test.cc +++ b/src/v/resource_mgmt/tests/memory_groups_test.cc @@ -16,9 +16,9 @@ #include #include -static constexpr size_t total_shares_without_optionals = 10; -static constexpr size_t total_wasm_shares = 1; -static constexpr size_t total_datalake_shares = 1; +static constexpr size_t total_shares_without_optionals = 100; +static constexpr size_t total_wasm_shares = 10; +static constexpr size_t total_datalake_shares = 10; // It's not really useful to know the exact byte values for each of these // numbers so we just make sure we're within a MB @@ -68,33 +68,33 @@ TEST_P(MemoryGroupSharesTest, DividesSharesCorrectly) { } EXPECT_THAT( groups.chunk_cache_min_memory(), - IsApprox(total_available_memory * 1.0 / total_shares)); + IsApprox(total_available_memory * 10.0 / total_shares)); EXPECT_THAT( groups.chunk_cache_max_memory(), - IsApprox(total_available_memory * 3.0 / total_shares)); + IsApprox(total_available_memory * 30.0 / total_shares)); EXPECT_THAT( groups.tiered_storage_max_memory(), - IsApprox(total_available_memory * 1.0 / total_shares)); + IsApprox(total_available_memory * 10.0 / total_shares)); EXPECT_THAT( groups.recovery_max_memory(), - IsApprox(total_available_memory * 1.0 / total_shares)); + IsApprox(total_available_memory * 10.0 / total_shares)); EXPECT_THAT( groups.kafka_total_memory(), - IsApprox(total_available_memory * 3.0 / total_shares)); + IsApprox(total_available_memory * 30.0 / total_shares)); EXPECT_THAT( groups.rpc_total_memory(), - IsApprox(total_available_memory * 2.0 / total_shares)); + IsApprox(total_available_memory * 20.0 / total_shares)); if (wasm_enabled()) { EXPECT_THAT( groups.data_transforms_max_memory(), - IsApprox(total_available_memory * 1.0 / total_shares)); + IsApprox(total_available_memory * 10.0 / total_shares)); } else { EXPECT_THAT(groups.data_transforms_max_memory(), 0); } if (datalake_enabled()) { EXPECT_THAT( groups.datalake_max_memory(), - IsApprox(total_available_memory * 1.0 / total_shares)); + IsApprox(total_available_memory * 10.0 / total_shares)); } else { EXPECT_THAT(groups.datalake_max_memory(), 0); } From 032d8bbf1cf3a9d0b9bc67a436530b1a00b7cc45 Mon Sep 17 00:00:00 2001 From: Stephan Dollberg Date: Mon, 9 Dec 2024 10:46:54 +0000 Subject: [PATCH 128/229] chunk_cache: Half chunk cache size Assuming wasm and iceberg being disabled the chunkcache currently gets assigned 30% of the total memory as per memory groups. At rest it only preallocates a lower watermark of a third of its share (aka. 10% of total memory) but the other two thirds are still reserved away from other subsystems. Even 10% of total memory is a massive overallocation for the chunk cache. Looking at some benchmarks the per shard usage is usually below 1MB. On a 4GB per shard system the chunk cache preallocates 400MB of chunks with allowance to go up to 1.2GB. The worst case scenario is a high partition scenario. Even at rest each partition uses one chunk (to store the last 4KB I think). In a super high partition density scenario of lets say 10k partitions per core (this isn't really realistic) this would only take 160MB of chunks and hence be still way below the 400MB low water mark in this case. Write caching can also increase the chunk cache usage but even in that case we write chunks as soon as the chunk is full so chunk throughput is high and we don't want to get too far behind in any case as that might mean bursting above disk throughput. Hence this patch halfs the current reserve amounts to free up some space. As per the above numbers we could go even lower but we can gain some further insight first with the metrics added in a previous patch. --- src/v/resource_mgmt/memory_groups.cc | 2 +- src/v/resource_mgmt/tests/memory_groups_test.cc | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/v/resource_mgmt/memory_groups.cc b/src/v/resource_mgmt/memory_groups.cc index f1f5bf18ac51..1e51c62b1b63 100644 --- a/src/v/resource_mgmt/memory_groups.cc +++ b/src/v/resource_mgmt/memory_groups.cc @@ -31,7 +31,7 @@ bool datalake_enabled() { } struct memory_shares { - constexpr static size_t chunk_cache = 30; + constexpr static size_t chunk_cache = 15; constexpr static size_t kafka = 30; constexpr static size_t rpc = 20; constexpr static size_t recovery = 10; diff --git a/src/v/resource_mgmt/tests/memory_groups_test.cc b/src/v/resource_mgmt/tests/memory_groups_test.cc index ca5d1146ff59..2027fd86c3e3 100644 --- a/src/v/resource_mgmt/tests/memory_groups_test.cc +++ b/src/v/resource_mgmt/tests/memory_groups_test.cc @@ -16,7 +16,7 @@ #include #include -static constexpr size_t total_shares_without_optionals = 100; +static constexpr size_t total_shares_without_optionals = 85; static constexpr size_t total_wasm_shares = 10; static constexpr size_t total_datalake_shares = 10; @@ -68,10 +68,10 @@ TEST_P(MemoryGroupSharesTest, DividesSharesCorrectly) { } EXPECT_THAT( groups.chunk_cache_min_memory(), - IsApprox(total_available_memory * 10.0 / total_shares)); + IsApprox(total_available_memory * 5.0 / total_shares)); EXPECT_THAT( groups.chunk_cache_max_memory(), - IsApprox(total_available_memory * 30.0 / total_shares)); + IsApprox(total_available_memory * 15.0 / total_shares)); EXPECT_THAT( groups.tiered_storage_max_memory(), IsApprox(total_available_memory * 10.0 / total_shares)); From d1827f93ca9e747c787042c32480e1686b439ae3 Mon Sep 17 00:00:00 2001 From: Chris Lee Date: Tue, 10 Dec 2024 15:54:13 -0700 Subject: [PATCH 129/229] [rptest] Add Azure V3 machinetypes --- tests/rptest/services/machinetype.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/tests/rptest/services/machinetype.py b/tests/rptest/services/machinetype.py index 8a6c92584c19..ce3454ab5ec9 100644 --- a/tests/rptest/services/machinetype.py +++ b/tests/rptest/services/machinetype.py @@ -24,6 +24,9 @@ class MachineTypeName(str, Enum): # Azure X86 STANDARD_L8S_V3 = 'Standard_L8s_v3' STANDARD_L8AS_V3 = 'Standard_L8as_v3' + STANDARD_D2D_V5 = 'Standard_D2d_v5' + STANDARD_D4D_V5 = 'Standard_D4d_v5' + STANDARD_D32D_V5 = 'Standard_D32d_v5' # GCP X86 N2_STANDARD_2 = 'n2-standard-2' @@ -86,6 +89,12 @@ class MachineTypeConfig: MachineTypeConfig(num_shards=7, memory=64 * GiB), MachineTypeName.STANDARD_L8AS_V3: MachineTypeConfig(num_shards=7, memory=64 * GiB), + MachineTypeName.STANDARD_D2D_V5: + MachineTypeConfig(num_shards=1, memory=8 * GiB), + MachineTypeName.STANDARD_D4D_V5: + MachineTypeConfig(num_shards=3, memory=16 * GiB), + MachineTypeName.STANDARD_D32D_V5: + MachineTypeConfig(num_shards=30, memory=128 * GiB), # GCP X86 MachineTypeName.N2_STANDARD_2: From edc0ab3c101ce93d0298d3dc8ac20d66c69f356a Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Mon, 9 Dec 2024 14:37:56 -0500 Subject: [PATCH 130/229] `storage`: fix race condition in `segment::release_appender_in_background()` This function caused race conditions between `segment::close()` and a segment roll. Consider the following sequence of events: 1. `_gate.close()` called in `segment::close()` 2. `auto a = std::exchange(_appender, nullptr)` called in `segment::release_appender_in_background()` 3. `ssx::spawn_with_gate()` called in `segment::release_appender_in_background()` 4. `return ignore_shutdown_exceptions()` in `ssx::spawn_with_gate()` 5. rest of `release_appender_in_background()` is ignored 6. `a` goes out of scope in `release_appender_in_background()` without `close()`ing the `appender` 7. one sad panda Add an explicit check to `_gate.check()` in `release_appender_in_background()` to throw an exception in case the gate is closed, and defer the closing of the appender to `segment::close()` in order to avoid the potential race condition here. --- src/v/storage/segment.cc | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/v/storage/segment.cc b/src/v/storage/segment.cc index ec731353f0b8..93d37e780036 100644 --- a/src/v/storage/segment.cc +++ b/src/v/storage/segment.cc @@ -297,6 +297,8 @@ ss::future<> segment::release_appender(readers_cache* readers_cache) { } void segment::release_appender_in_background(readers_cache* readers_cache) { + _gate.check(); + auto a = std::exchange(_appender, nullptr); auto c = config::shard_local_cfg().release_cache_on_segment_roll() ? std::exchange(_cache, std::nullopt) From 7a29f24eaf13735b96c65fe05941d9062d9a1f03 Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Mon, 9 Dec 2024 15:49:32 -0500 Subject: [PATCH 131/229] `storage`: use `model::next_offset()` in `segment::force_roll()` If the dirty offset is default initialized to `int64_t::min()` and not updated before a segment is force rolled, an assert will fail in `disk_log_impl::new_segment()` for the offset being < 0. Use `model::next_offset()` instead of simply adding `1` to the dirty offset to avoid this case. --- src/v/storage/disk_log_impl.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/v/storage/disk_log_impl.cc b/src/v/storage/disk_log_impl.cc index 6a138a938fa4..817e3d861984 100644 --- a/src/v/storage/disk_log_impl.cc +++ b/src/v/storage/disk_log_impl.cc @@ -1747,7 +1747,7 @@ void disk_log_impl::bg_checkpoint_offset_translator() { ss::future<> disk_log_impl::force_roll(ss::io_priority_class iopc) { auto roll_lock_holder = co_await _segments_rolling_lock.get_units(); auto t = term(); - auto next_offset = offsets().dirty_offset + model::offset(1); + auto next_offset = model::next_offset(offsets().dirty_offset); if (_segs.empty()) { co_return co_await new_segment(next_offset, t, iopc); } From c0d76d1afbccecc6762efc365d185b70feb3fdb6 Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Fri, 6 Dec 2024 20:05:02 -0500 Subject: [PATCH 132/229] `storage`: move `storage_e2e_fixture` to its own header file --- src/v/storage/disk_log_impl.h | 2 + src/v/storage/tests/storage_e2e_fixture.h | 50 +++++++++++++++++++ .../storage/tests/storage_e2e_fixture_test.cc | 31 ++---------- 3 files changed, 57 insertions(+), 26 deletions(-) create mode 100644 src/v/storage/tests/storage_e2e_fixture.h diff --git a/src/v/storage/disk_log_impl.h b/src/v/storage/disk_log_impl.h index 9a32445c80d8..00b6f3ed511e 100644 --- a/src/v/storage/disk_log_impl.h +++ b/src/v/storage/disk_log_impl.h @@ -31,6 +31,7 @@ #include +struct storage_e2e_fixture; namespace storage { /// \brief offset boundary type @@ -248,6 +249,7 @@ class disk_log_impl final : public log { private: friend class disk_log_appender; // for multi-term appends friend class disk_log_builder; // for tests + friend ::storage_e2e_fixture; friend std::ostream& operator<<(std::ostream& o, const disk_log_impl& d); /// Compute file offset of the batch inside the segment diff --git a/src/v/storage/tests/storage_e2e_fixture.h b/src/v/storage/tests/storage_e2e_fixture.h new file mode 100644 index 000000000000..8f7e3352e2f9 --- /dev/null +++ b/src/v/storage/tests/storage_e2e_fixture.h @@ -0,0 +1,50 @@ +// Copyright 2024 Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 + +#include "redpanda/tests/fixture.h" +#include "storage/disk_log_impl.h" +#include "storage/segment.h" +#include "test_utils/scoped_config.h" + +#include +#include + +struct storage_e2e_fixture : public redpanda_thread_fixture { + scoped_config test_local_cfg; + + // Produces to the given fixture's partition for 10 seconds. + ss::future<> produce_to_fixture(model::topic topic_name, int* incomplete) { + tests::kafka_produce_transport producer(co_await make_kafka_client()); + co_await producer.start(); + const int cardinality = 10; + auto now = ss::lowres_clock::now(); + while (ss::lowres_clock::now() < now + 5s) { + for (int i = 0; i < cardinality; i++) { + co_await producer.produce_to_partition( + topic_name, + model::partition_id(0), + tests::kv_t::sequence(i, 1)); + } + } + *incomplete -= 1; + } + + ss::future<> remove_segment_permanently( + storage::disk_log_impl* log, ss::lw_shared_ptr seg) { + return log->remove_segment_permanently(seg, "storage_e2e_fixture") + .then([&, log, seg]() { + auto& segs = log->segments(); + auto it = std::find(segs.begin(), segs.end(), seg); + if (it == segs.end()) { + return; + } + segs.erase(it, std::next(it)); + }); + } +}; diff --git a/src/v/storage/tests/storage_e2e_fixture_test.cc b/src/v/storage/tests/storage_e2e_fixture_test.cc index 6c3a8c38ba17..2ccc1a52400f 100644 --- a/src/v/storage/tests/storage_e2e_fixture_test.cc +++ b/src/v/storage/tests/storage_e2e_fixture_test.cc @@ -10,10 +10,12 @@ #include "kafka/server/tests/produce_consume_utils.h" #include "model/fundamental.h" #include "random/generators.h" -#include "redpanda/tests/fixture.h" +#include "storage/disk_log_impl.h" +#include "storage/segment.h" +#include "storage/tests/storage_e2e_fixture.h" #include "test_utils/fixture.h" -#include "test_utils/scoped_config.h" +#include #include #include @@ -23,29 +25,6 @@ using namespace std::chrono_literals; -struct storage_e2e_fixture : public redpanda_thread_fixture { - scoped_config test_local_cfg; -}; - -namespace { - -// Produces to the given fixture's partition for 10 seconds. -ss::future<> produce_to_fixture( - storage_e2e_fixture* fix, model::topic topic_name, int* incomplete) { - tests::kafka_produce_transport producer(co_await fix->make_kafka_client()); - co_await producer.start(); - const int cardinality = 10; - auto now = ss::lowres_clock::now(); - while (ss::lowres_clock::now() < now + 5s) { - for (int i = 0; i < cardinality; i++) { - co_await producer.produce_to_partition( - topic_name, model::partition_id(0), tests::kv_t::sequence(i, 1)); - } - } - *incomplete -= 1; -} -} // namespace - FIXTURE_TEST(test_compaction_segment_ms, storage_e2e_fixture) { test_local_cfg.get("log_segment_ms_min") .set_value(std::chrono::duration_cast(1ms)); @@ -69,7 +48,7 @@ FIXTURE_TEST(test_compaction_segment_ms, storage_e2e_fixture) { produces.reserve(5); int incomplete = 5; for (int i = 0; i < 5; i++) { - auto fut = produce_to_fixture(this, topic_name, &incomplete); + auto fut = produce_to_fixture(topic_name, &incomplete); produces.emplace_back(std::move(fut)); } auto partition = app.partition_manager.local().get(ntp); From 24ec83438505276dcba709beac3677a50212e55c Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Fri, 6 Dec 2024 20:10:01 -0500 Subject: [PATCH 133/229] `storage`: add `test_concurrent_segment_roll_and_close` To test race conditions between `segment::close()` and a segment roll, particularly one which goes through `release_appender_in_background()`. --- .../storage/tests/storage_e2e_fixture_test.cc | 39 +++++++++++++++++++ 1 file changed, 39 insertions(+) diff --git a/src/v/storage/tests/storage_e2e_fixture_test.cc b/src/v/storage/tests/storage_e2e_fixture_test.cc index 2ccc1a52400f..a9299cbfece8 100644 --- a/src/v/storage/tests/storage_e2e_fixture_test.cc +++ b/src/v/storage/tests/storage_e2e_fixture_test.cc @@ -15,6 +15,7 @@ #include "storage/tests/storage_e2e_fixture.h" #include "test_utils/fixture.h" +#include #include #include @@ -25,6 +26,16 @@ using namespace std::chrono_literals; +namespace { +ss::future<> force_roll_log(storage::disk_log_impl* log) { + try { + co_await log->force_roll(ss::default_priority_class()); + } catch (...) { + } +} + +} // namespace + FIXTURE_TEST(test_compaction_segment_ms, storage_e2e_fixture) { test_local_cfg.get("log_segment_ms_min") .set_value(std::chrono::duration_cast(1ms)); @@ -156,3 +167,31 @@ FIXTURE_TEST(test_concurrent_log_eviction_and_append, storage_e2e_fixture) { // final round of eviction. BOOST_REQUIRE_LE(log->segment_count(), 1); } + +FIXTURE_TEST(test_concurrent_segment_roll_and_close, storage_e2e_fixture) { + const auto topic_name = model::topic("tapioca"); + const auto ntp = model::ntp(model::kafka_namespace, topic_name, 0); + + cluster::topic_properties props; + add_topic({model::kafka_namespace, topic_name}, 1, props).get(); + wait_for_leader(ntp).get(); + + auto partition = app.partition_manager.local().get(ntp); + auto* log = dynamic_cast(partition->log().get()); + auto seg = log->segments().back(); + + // Hold a read lock, which will force release_appender() to go through + // release_appender_in_background() + auto read_lock_holder = seg->read_lock().get(); + + auto roll_fut = force_roll_log(log); + auto release_holder_fut = ss::sleep(100ms).then( + [read_locker_holder = std::move(read_lock_holder)] {}); + auto remove_segment_fut = remove_segment_permanently(log, seg); + + ss::when_all( + std::move(roll_fut), + std::move(remove_segment_fut), + std::move(release_holder_fut)) + .get(); +} From 1dab32e29ba08b8406df7bcc8cd92f174397a4bc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Mon, 2 Dec 2024 11:09:03 +0100 Subject: [PATCH 134/229] r/tests: refactored group configuration tests MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Changed group configuration tests to use vnode based api as the broker based one is deprecated. Signed-off-by: Michał Maślanka --- src/v/raft/tests/BUILD | 5 +- src/v/raft/tests/CMakeLists.txt | 12 +-- src/v/raft/tests/group_configuration_tests.cc | 99 +++++++------------ 3 files changed, 38 insertions(+), 78 deletions(-) diff --git a/src/v/raft/tests/BUILD b/src/v/raft/tests/BUILD index b371936d996a..20fc85316209 100644 --- a/src/v/raft/tests/BUILD +++ b/src/v/raft/tests/BUILD @@ -240,7 +240,7 @@ redpanda_cc_btest( ], ) -redpanda_cc_btest( +redpanda_cc_gtest( name = "group_configuration_test", timeout = "short", srcs = [ @@ -256,11 +256,12 @@ redpanda_cc_btest( "//src/v/resource_mgmt:io_priority", "//src/v/storage", "//src/v/storage:record_batch_builder", + "//src/v/test_utils:gtest", "//src/v/test_utils:random", "//src/v/test_utils:seastar_boost", "//src/v/utils:unresolved_address", - "@boost//:test", "@fmt", + "@googletest//:gtest", "@seastar", "@seastar//:testing", ], diff --git a/src/v/raft/tests/CMakeLists.txt b/src/v/raft/tests/CMakeLists.txt index 75f3e84f5100..334d2d98d560 100644 --- a/src/v/raft/tests/CMakeLists.txt +++ b/src/v/raft/tests/CMakeLists.txt @@ -8,17 +8,6 @@ rp_test( ARGS "-- -c 8" ) -rp_test( - UNIT_TEST - BINARY_NAME group_configuration_tests - SOURCES group_configuration_tests.cc - DEFINITIONS BOOST_TEST_DYN_LINK - LIBRARIES Boost::unit_test_framework v::raft - LABELS raft - ARGS "-- -c 8" -) - - set(srcs jitter_tests.cc bootstrap_configuration_test.cc @@ -55,6 +44,7 @@ set(gsrcs replication_monitor_tests.cc mux_state_machine_test.cc snapshot_recovery_test.cc + group_configuration_tests.cc ) rp_test( diff --git a/src/v/raft/tests/group_configuration_tests.cc b/src/v/raft/tests/group_configuration_tests.cc index c766bbf7aabd..badf50268514 100644 --- a/src/v/raft/tests/group_configuration_tests.cc +++ b/src/v/raft/tests/group_configuration_tests.cc @@ -9,112 +9,81 @@ #include "model/metadata.h" #include "raft/group_configuration.h" +#include "test_utils/test.h" #include "utils/unresolved_address.h" -#include #include -#define BOOST_TEST_MODULE raft -#include "raft/types.h" - -#include - -model::broker create_broker(int32_t id) { - return model::broker( - model::node_id{id}, - net::unresolved_address("127.0.0.1", 9002), - net::unresolved_address("127.0.0.1", 1234), - std::nullopt, - model::broker_properties{}); -} - -BOOST_AUTO_TEST_CASE(should_return_true_as_it_contains_learner) { - raft::group_configuration test_grp = raft::group_configuration( - {create_broker(1)}, model::revision_id(0)); - auto contains = test_grp.contains_broker(model::node_id(1)); - BOOST_REQUIRE_EQUAL(contains, true); +namespace { +raft::vnode create_vnode(int32_t id) { + return {model::node_id(id), model::revision_id(0)}; } -BOOST_AUTO_TEST_CASE(should_return_true_as_it_contains_voter) { - raft::group_configuration test_grp = raft::group_configuration( - {create_broker(1)}, model::revision_id(0)); - - auto contains = test_grp.contains_broker(model::node_id(1)); - BOOST_REQUIRE_EQUAL(contains, true); +raft::group_configuration create_configuration(std::vector nodes) { + return {std::move(nodes), model::revision_id(0)}; } -BOOST_AUTO_TEST_CASE(should_return_false_as_it_does_not_contain_machine) { - raft::group_configuration test_grp = raft::group_configuration( - {create_broker(3)}, model::revision_id(0)); - - auto contains = test_grp.contains_broker(model::node_id(1)); - BOOST_REQUIRE_EQUAL(contains, false); -} +} // namespace -BOOST_AUTO_TEST_CASE(test_demoting_removed_voters) { - raft::group_configuration test_grp = raft::group_configuration( - std::vector{create_broker(3)}, model::revision_id(0)); +TEST(test_raft_group_configuration, test_demoting_removed_voters) { + raft::group_configuration test_grp = create_configuration( + {create_vnode(3)}); - // add brokers - test_grp.add_broker(create_broker(1), model::revision_id{0}); + // add nodes + test_grp.add(create_vnode(1), model::revision_id{0}, std::nullopt); test_grp.promote_to_voter( raft::vnode(model::node_id{1}, model::revision_id(0))); test_grp.finish_configuration_transition(); - test_grp.add_broker(create_broker(2), model::revision_id{0}); + test_grp.add(create_vnode(2), model::revision_id{0}, std::nullopt); test_grp.promote_to_voter( raft::vnode(model::node_id{2}, model::revision_id(0))); test_grp.finish_configuration_transition(); test_grp.finish_configuration_transition(); // remove single broker - test_grp.remove_broker(model::node_id(1)); + test_grp.remove(create_vnode(1), model::revision_id{0}); // finish configuration transition - auto demoted = test_grp.maybe_demote_removed_voters(); - BOOST_REQUIRE_EQUAL(demoted, true); - BOOST_REQUIRE_EQUAL(test_grp.old_config()->voters.size(), 2); + ASSERT_TRUE(test_grp.maybe_demote_removed_voters()); + ASSERT_EQ(test_grp.old_config()->voters.size(), 2); // node 0 was demoted since it was removed from the cluster - BOOST_REQUIRE_EQUAL( - test_grp.old_config()->learners[0], - raft::vnode(model::node_id{1}, model::revision_id(0))); + ASSERT_EQ(test_grp.old_config()->learners[0], create_vnode(1)); // assert that operation is idempotent - demoted = test_grp.maybe_demote_removed_voters(); - BOOST_REQUIRE_EQUAL(demoted, false); + ASSERT_FALSE(test_grp.maybe_demote_removed_voters()); } -BOOST_AUTO_TEST_CASE(test_aborting_configuration_change) { +TEST(test_raft_group_configuration, test_aborting_configuration_change) { raft::group_configuration test_grp = raft::group_configuration( - std::vector{create_broker(3)}, model::revision_id(0)); + {create_vnode(3)}, model::revision_id(0)); - auto original_brokers = test_grp.brokers(); auto original_voters = test_grp.current_config().voters; - + auto original_nodes = test_grp.all_nodes(); // add brokers - test_grp.add_broker(create_broker(1), model::revision_id{0}); + test_grp.add(create_vnode(1), model::revision_id{0}, std::nullopt); // abort change test_grp.abort_configuration_change(model::revision_id{1}); - BOOST_REQUIRE_EQUAL( - test_grp.get_state(), raft::configuration_state::simple); - BOOST_REQUIRE_EQUAL(test_grp.brokers(), original_brokers); - BOOST_REQUIRE_EQUAL(test_grp.current_config().voters, original_voters); + ASSERT_EQ(test_grp.get_state(), raft::configuration_state::simple); + ASSERT_EQ(test_grp.current_config().voters, original_voters); + ASSERT_EQ(test_grp.all_nodes(), original_nodes); } -BOOST_AUTO_TEST_CASE(test_reverting_configuration_change_when_adding) { +TEST( + test_raft_group_configuration, + test_reverting_configuration_change_when_adding) { raft::group_configuration test_grp = raft::group_configuration( - std::vector{create_broker(3)}, model::revision_id(0)); + {create_vnode(3)}, model::revision_id(0)); // add brokers - test_grp.add_broker(create_broker(1), model::revision_id{0}); + test_grp.add(create_vnode(1), model::revision_id{0}, std::nullopt); // abort change test_grp.cancel_configuration_change(model::revision_id{1}); - BOOST_REQUIRE_EQUAL( - test_grp.get_state(), raft::configuration_state::simple); - BOOST_REQUIRE_EQUAL(test_grp.brokers().size(), 1); - BOOST_REQUIRE_EQUAL(test_grp.current_config().voters.size(), 1); - BOOST_REQUIRE_EQUAL(test_grp.current_config().learners.size(), 0); + ASSERT_EQ(test_grp.get_state(), raft::configuration_state::simple); + ASSERT_EQ(test_grp.all_nodes().size(), 1); + ASSERT_EQ(test_grp.current_config().voters.size(), 1); + ASSERT_EQ(test_grp.current_config().learners.size(), 0); } From b2fc1be49e9f61abe2cdd1c5a5bbc60a57e55ae6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Tue, 3 Dec 2024 09:32:47 +0100 Subject: [PATCH 135/229] raft: added new strategy to make reconfiguration symetric MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit In order to support cancellation of raft group reconfiguration cancellation we must make the cancell operation symmetric. This means that when reconfiguration is cancelled twice it must lead to the same outcome as it would not be cancelled at all. This commit introduces a new `raft::group_configuration` version together with a new reconfiguration strategy. This is a first step towards allowing cancelling cancellations in controller. Signed-off-by: Michał Maślanka --- src/v/raft/group_configuration.cc | 288 +++++++++- src/v/raft/group_configuration.h | 19 +- .../tests/configuration_serialization_test.cc | 4 +- src/v/raft/tests/group_configuration_tests.cc | 515 +++++++++++++++++- 4 files changed, 815 insertions(+), 11 deletions(-) diff --git a/src/v/raft/group_configuration.cc b/src/v/raft/group_configuration.cc index 5ce96af60d10..e93e46b9f9bf 100644 --- a/src/v/raft/group_configuration.cc +++ b/src/v/raft/group_configuration.cc @@ -146,6 +146,49 @@ class configuration_change_strategy_v5 void cancel_configuration_change(model::revision_id) final; void finish_configuration_transition() final; +private: + void cancel_update_in_transitional_state(); + void cancel_update_in_joint_state(); + + group_configuration& _cfg; +}; + +class configuration_change_strategy_v6 + : public group_configuration::configuration_change_strategy { +public: + explicit configuration_change_strategy_v6(group_configuration& cfg) + : _cfg(cfg) {} + + void add_broker(model::broker, model::revision_id) final { + vassert( + false, + "broker based api is not supported in configuration version 6"); + } + void remove_broker(model::node_id) final { + vassert( + false, + "broker based api is not supported in configuration version 6"); + } + void + replace_brokers(std::vector, model::revision_id) final { + vassert( + false, + "broker based api is not supported in configuration version 6"); + } + + void add(vnode, model::revision_id, std::optional) final; + void replace( + std::vector, + model::revision_id, + std::optional) final; + void remove(vnode, model::revision_id) final; + + void discard_old_config() final; + void abort_configuration_change(model::revision_id) final; + void cancel_configuration_change(model::revision_id) final; + void finish_configuration_transition() final; + void fill_learners_with_nodes_to_add(); + private: void cancel_update_in_transitional_state(); void cancel_update_in_joint_state(); @@ -310,7 +353,9 @@ group_configuration::group_configuration( std::unique_ptr group_configuration::make_change_strategy() { - if (_version >= v_5) { + if (_version >= v_7) { + return std::make_unique(*this); + } else if (_version >= v_5) { return std::make_unique(*this); } else if (_version == v_4) { return std::make_unique(*this); @@ -1187,6 +1232,192 @@ void configuration_change_strategy_v5::discard_old_config() { _cfg._configuration_update.reset(); } +/** + * configuration_change_strategy_v6 differs from the previous versions as it + * makes the reconfiguration cancellation process symmetric. With this strategy + * when configuration is representing reconfiguration from replicas set A to + * replica set B (A -> B) the cancellation process will make the reconfiguration + * to change direction (B -> A). Reconfiguration strategy v6 allows to toggle + * the direction again i.e. cancellation of reconfiguration from B to A will + * make the reconfiguration to go from A to B again. + * + * The difference between v6 and v5 is that v6 does not loose the information + * about the update even if it is not strictly required to finish + * reconfiguration. The information is kept in the `_configuration_update` field + * even when the configuration in in joint state. + */ +void configuration_change_strategy_v6::replace( + std::vector replicas, + model::revision_id rev, + std::optional learner_start_offset) { + _cfg._revision = rev; + + /** + * If configurations are identical do nothing. Configurations are considered + * equal if requested nodes are voters + */ + bool are_equal = _cfg._current.voters.size() == replicas.size() + && std::all_of( + replicas.begin(), + replicas.end(), + [this](const vnode& vn) { return _cfg.contains(vn); }); + + // configurations are identical, do nothing + if (are_equal) { + return; + } + // calculate configuration update + _cfg._configuration_update = calculate_configuration_update( + _cfg._current.voters, replicas); + // set learner start offset + _cfg._configuration_update->learner_start_offset = learner_start_offset; + + // add replicas to current configuration + for (auto& vn : replicas) { + if (_cfg._configuration_update->is_to_add(vn)) { + _cfg._current.learners.push_back(vn); + } + } + + // optimization: when there are only nodes to be deleted we may go straight + // to the joint configuration + if (_cfg._configuration_update->replicas_to_add.empty()) { + finish_configuration_transition(); + } +} + +void configuration_change_strategy_v6::add( + vnode node, + model::revision_id rev, + std::optional learner_start_offset) { + if (_cfg._current.contains(node)) { + throw std::invalid_argument(fmt::format( + "replica {} already found in current configuration {}", node, _cfg)); + } + auto new_replicas = _cfg.all_nodes(); + new_replicas.push_back(node); + replace(new_replicas, rev, learner_start_offset); +} + +void configuration_change_strategy_v6::remove( + vnode node, model::revision_id rev) { + if (!_cfg._current.contains(node)) { + throw std::invalid_argument(fmt::format( + "replica {} not found in current configuration {}", node, _cfg)); + } + auto new_replicas = _cfg.all_nodes(); + std::erase_if(new_replicas, [node](const vnode& v) { return v == node; }); + replace(new_replicas, rev, std::nullopt); +} + +void configuration_change_strategy_v6::discard_old_config() { + _cfg._old.reset(); + _cfg._configuration_update.reset(); +} + +void configuration_change_strategy_v6::abort_configuration_change( + model::revision_id revision) { + // collect all node ids that the configuration either contains or the one + // that were removed + absl::flat_hash_set all_node_ids; + + _cfg.for_each_learner( + [&all_node_ids](const vnode& learner) { all_node_ids.insert(learner); }); + + _cfg.for_each_voter( + [&all_node_ids](const vnode& voter) { all_node_ids.insert(voter); }); + + for (auto& to_remove : _cfg._configuration_update->replicas_to_remove) { + all_node_ids.insert(to_remove); + } + + // clear the configuration + _cfg._current.voters.clear(); + _cfg._current.learners.clear(); + _cfg._old.reset(); + + // rebuild configuration + for (auto& vn : all_node_ids) { + if (!_cfg._configuration_update->is_to_add(vn)) { + _cfg._current.voters.push_back(vn); + } + } + + // finally reset configuration update and set the revision + _cfg._configuration_update.reset(); + _cfg._revision = revision; +} + +void configuration_change_strategy_v6::cancel_configuration_change( + model::revision_id revision) { + switch (_cfg.get_state()) { + case configuration_state::simple: + vassert( + false, + "can not cancel, configuration change is not in progress - {}", + _cfg); + case configuration_state::transitional: + cancel_update_in_transitional_state(); + break; + case configuration_state::joint: + cancel_update_in_joint_state(); + break; + } + _cfg._revision = revision; +} + +void configuration_change_strategy_v6::cancel_update_in_transitional_state() { + std::swap( + _cfg._configuration_update->replicas_to_add, + _cfg._configuration_update->replicas_to_remove); + + // remove all learners that were added + std::erase_if(_cfg._current.learners, [this](const vnode& learner) { + return _cfg._configuration_update->is_to_remove(learner); + }); + + fill_learners_with_nodes_to_add(); + finish_configuration_transition(); +} +void configuration_change_strategy_v6::fill_learners_with_nodes_to_add() { + for (auto& to_add : _cfg._configuration_update->replicas_to_add) { + if (!_cfg._current.contains(to_add)) { + _cfg._current.learners.push_back(to_add); + } + } +} +void configuration_change_strategy_v6::cancel_update_in_joint_state() { + std::swap( + _cfg._configuration_update->replicas_to_add, + _cfg._configuration_update->replicas_to_remove); + _cfg._current = *_cfg._old; + _cfg._old.reset(); + + fill_learners_with_nodes_to_add(); +} + +void configuration_change_strategy_v6::finish_configuration_transition() { + if (_cfg.get_state() != configuration_state::transitional) { + return; + } + + auto has_replicas_to_remove = std::any_of( + _cfg._configuration_update->replicas_to_remove.begin(), + _cfg._configuration_update->replicas_to_remove.end(), + [this](const vnode& v) { return _cfg._current.contains(v); }); + + if (!has_replicas_to_remove) { + _cfg._configuration_update.reset(); + return; + } + + _cfg._old = _cfg._current; + + std::erase_if(_cfg._current.voters, [this](const vnode& voter) { + return _cfg._configuration_update->is_to_remove(voter); + }); +} + std::vector with_revisions_assigned( const std::vector& vnodes, model::revision_id new_revision) { std::vector with_rev; @@ -1286,15 +1517,68 @@ group_configuration group_configuration::serde_direct_read( auto old = serde::read_nested>( p, h._bytes_left_limit); auto rev = serde::read_nested(p, h._bytes_left_limit); - return {std::move(current), rev, std::move(update), std::move(old)}; + group_configuration ret{ + std::move(current), rev, std::move(update), std::move(old)}; + + // if no version is set we assume that configuration is in version 6 (first + // that was serde serialized) + ret._version = group_configuration::v_6; + + if (h._version >= 7) { + auto version = serde::read_nested(p, h._bytes_left_limit); + ret._version = version; + } + + if (p.bytes_left() > h._bytes_left_limit) { + p.skip(p.bytes_left() - h._bytes_left_limit); + } + + return ret; } void group_configuration::serde_write(iobuf& out) { + using serde::write; + serde_write_v6(out); + write(out, _version); +} + +void group_configuration::serde_write_v6(iobuf& out) { using serde::write; write(out, _current); write(out, _configuration_update); write(out, _old); write(out, _revision); } + +/** + * We need custom serde implementation for group_configuration as we can not + * simply add a field to it because of the bug in deserialization logic (missing + * skip) + */ +void tag_invoke( + serde::tag_t, iobuf& out, group_configuration cfg) { + using serde::write; + const bool newer_than_v6 = cfg.version() >= group_configuration::v_7; + std::uint8_t s_version + = newer_than_v6 ? raft::group_configuration::redpanda_serde_version : 6; + write(out, s_version); + write(out, group_configuration::redpanda_serde_compat_version); + + auto size_placeholder = out.reserve(sizeof(serde::serde_size_t)); + const auto size_before = out.size_bytes(); + + if (newer_than_v6) { + cfg.serde_write(out); + } else { + cfg.serde_write_v6(out); + } + + const auto written_size = out.size_bytes() - size_before; + + const auto size = ss::cpu_to_le( + static_cast(written_size)); + size_placeholder.write( + reinterpret_cast(&size), sizeof(serde::serde_size_t)); +} } // namespace raft namespace reflection { diff --git a/src/v/raft/group_configuration.h b/src/v/raft/group_configuration.h index 654ac0f1a530..1cebe42070b8 100644 --- a/src/v/raft/group_configuration.h +++ b/src/v/raft/group_configuration.h @@ -108,7 +108,7 @@ struct configuration_update class group_configuration : public serde::envelope< group_configuration, - serde::version<6>, + serde::version<7>, serde::compat_version<6>> { public: using version_t @@ -124,7 +124,10 @@ class group_configuration // serde serialized configuration static constexpr version_t v_6{6}; - static constexpr version_t current_version = v_6; + // version with symmetric cancellations (configuration_change_strategy_v6) + static constexpr version_t v_7{7}; + + static constexpr version_t current_version = v_7; /** * creates a configuration where all provided brokers are current @@ -386,6 +389,7 @@ class group_configuration bool is_with_brokers() const { return _version < v_5; } void serde_write(iobuf& out); + void serde_write_v6(iobuf& out); static group_configuration serde_direct_read(iobuf_parser&, const serde::header&); @@ -397,11 +401,14 @@ class group_configuration friend class configuration_change_strategy_v5; + friend class configuration_change_strategy_v6; + std::vector unique_voter_ids() const; std::vector unique_learner_ids() const; std::unique_ptr make_change_strategy(); - - version_t _version = current_version; + // initialize version to the v_6 in order to correctly handle + // deserialization when version field is missing + version_t _version = v_6; std::vector _brokers; group_nodes _current; std::optional _configuration_update; @@ -409,8 +416,10 @@ class group_configuration model::revision_id _revision; }; -namespace details { +void tag_invoke( + serde::tag_t, iobuf& out, group_configuration t); +namespace details { template auto quorum_match(ValueProvider&& f, Range&& range) { using ret_t = std::invoke_result_t; diff --git a/src/v/raft/tests/configuration_serialization_test.cc b/src/v/raft/tests/configuration_serialization_test.cc index bf1c9131ed27..ea9a2a270fbf 100644 --- a/src/v/raft/tests/configuration_serialization_test.cc +++ b/src/v/raft/tests/configuration_serialization_test.cc @@ -92,7 +92,9 @@ raft::group_configuration random_configuration() { SEASTAR_THREAD_TEST_CASE(roundtrip_raft_configuration_entry) { for (auto v : - {raft::group_configuration::v_5, raft::group_configuration::v_6}) { + {raft::group_configuration::v_5, + raft::group_configuration::v_6, + raft::group_configuration::v_7}) { auto cfg = random_configuration(); cfg.set_version(v); diff --git a/src/v/raft/tests/group_configuration_tests.cc b/src/v/raft/tests/group_configuration_tests.cc index badf50268514..c53a7b02a9f7 100644 --- a/src/v/raft/tests/group_configuration_tests.cc +++ b/src/v/raft/tests/group_configuration_tests.cc @@ -7,13 +7,14 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0 -#include "model/metadata.h" +#include "gmock/gmock.h" #include "raft/group_configuration.h" -#include "test_utils/test.h" -#include "utils/unresolved_address.h" +#include "test_utils/randoms.h" #include +#include + namespace { raft::vnode create_vnode(int32_t id) { return {model::node_id(id), model::revision_id(0)}; @@ -87,3 +88,511 @@ TEST( ASSERT_EQ(test_grp.current_config().voters.size(), 1); ASSERT_EQ(test_grp.current_config().learners.size(), 0); } +namespace { +std::vector +diff(const std::vector& lhs, const std::vector& rhs) { + std::vector result; + for (auto& lhs_node : lhs) { + auto it = std::find(rhs.begin(), rhs.end(), lhs_node); + if (it == rhs.end()) { + result.push_back(lhs_node); + } + } + return result; +} + +void transition_configuration_update(raft::group_configuration& cfg) { + while (cfg.get_state() != raft::configuration_state::simple) { + if (cfg.get_state() == raft::configuration_state::joint) { + cfg.maybe_demote_removed_voters(); + cfg.discard_old_config(); + } + + if (cfg.get_state() == raft::configuration_state::transitional) { + if (!cfg.current_config().learners.empty()) { + auto learners = cfg.current_config().learners; + for (const auto& vnode : learners) { + cfg.promote_to_voter(vnode); + } + } + cfg.finish_configuration_transition(); + } + } +} + +} // namespace +using namespace ::testing; +struct configuration_cancel_test_params { + configuration_cancel_test_params( + std::initializer_list source, std::initializer_list target) { + for (auto& s : source) { + initial_replica_set.push_back(create_vnode(s)); + } + for (auto t : target) { + target_replica_set.push_back(create_vnode(t)); + } + } + std::vector initial_replica_set; + std::vector target_replica_set; + + friend std::ostream& + operator<<(std::ostream&, const configuration_cancel_test_params&); +}; + +std::ostream& +operator<<(std::ostream& o, const configuration_cancel_test_params& p) { + fmt::print( + o, + "[{}] -> [{}]", + fmt::join( + std::ranges::views::transform( + p.initial_replica_set, [](auto& v) { return v.id(); }), + ","), + fmt::join( + std::ranges::views::transform( + p.target_replica_set, [](auto& v) { return v.id(); }), + ",")); + return o; +} + +class ConfigurationCancellationTest + : public TestWithParam {}; + +/** + * This test verifies if cancelling the configuration change twice at any point + * will lead to the cancellation being reverted. + * example: + * replicas are updated from set A to set B + * 1. A->B + * then a reconfiguration is cancelled + * 2. B->A + * then the reconfiguration is cancelled again + * 3. A->B + * Finally the replica set must be equal to B + */ +TEST_P(ConfigurationCancellationTest, TestEvenNumberOfCancellations) { + const auto params = GetParam(); + + const std::vector original_replicas + = params.initial_replica_set; + const std::vector target_replicas = params.target_replica_set; + const auto to_add = diff(target_replicas, original_replicas); + const auto to_remove = diff(original_replicas, target_replicas); + + raft::group_configuration test_cfg = raft::group_configuration( + original_replicas, model::revision_id(0)); + test_cfg.set_version(raft::group_configuration::v_7); + + // trigger reconfiguration + test_cfg.replace(target_replicas, model::revision_id{0}, std::nullopt); + + ASSERT_THAT( + test_cfg.get_configuration_update()->replicas_to_add, + ElementsAreArray(to_add)); + ASSERT_THAT( + test_cfg.get_configuration_update()->replicas_to_remove, + ElementsAreArray(to_remove)); + + // CASE 1. Cancel right after change was requested + + // cancel configuration change, goes straight to simple state as learner can + // be removed immediately + auto cfg_1 = test_cfg; + cfg_1.cancel_configuration_change(model::revision_id{0}); + // check if reconfiguration is finished by optimizations + if (cfg_1.get_state() == raft::configuration_state::simple) { + ASSERT_EQ(cfg_1.current_config().voters, original_replicas); + ASSERT_TRUE(cfg_1.current_config().learners.empty()); + } else { + cfg_1.cancel_configuration_change(model::revision_id{0}); + transition_configuration_update(cfg_1); + ASSERT_EQ(cfg_1.current_config().voters, target_replicas); + ASSERT_EQ(cfg_1.get_state(), raft::configuration_state::simple); + ASSERT_TRUE(cfg_1.current_config().learners.empty()); + } + // CASE 2. Cancel after learners promotion + for (size_t cancel_after = 1; cancel_after <= to_add.size(); + ++cancel_after) { + // create a copy of the configuration for each round + auto cfg = test_cfg; + for (size_t i = 0; i < cancel_after; ++i) { + cfg.promote_to_voter(to_add[i]); + } + // update the original configuration for the next step + if (cancel_after == to_add.size()) { + test_cfg = cfg; + } + cfg.cancel_configuration_change(model::revision_id{0}); + cfg.cancel_configuration_change(model::revision_id{0}); + + transition_configuration_update(cfg); + + ASSERT_EQ(cfg.current_config().voters, target_replicas); + ASSERT_EQ(cfg.get_state(), raft::configuration_state::simple); + ASSERT_TRUE(cfg.current_config().learners.empty()); + } + + // now finish the configuration transition as all learners were promoted + test_cfg.finish_configuration_transition(); + + // CASE 3. Cancel after leaving transitional state + + // check if reconfiguration is finished by optimizations + if (test_cfg.get_state() == raft::configuration_state::simple) { + ASSERT_EQ(test_cfg.current_config().voters, target_replicas); + ASSERT_TRUE(cfg_1.current_config().learners.empty()); + return; + } + // at every step create a copy of test_cfg and execute cancellations against + // the copy + auto cfg_2 = test_cfg; + + cfg_2.cancel_configuration_change(model::revision_id{0}); + if (cfg_2.get_state() == raft::configuration_state::simple) { + ASSERT_EQ(cfg_2.current_config().voters, original_replicas); + ASSERT_TRUE(cfg_2.current_config().learners.empty()); + } else { + cfg_2.cancel_configuration_change(model::revision_id{0}); + + transition_configuration_update(cfg_2); + + ASSERT_EQ(cfg_2.get_state(), raft::configuration_state::simple); + ASSERT_EQ(cfg_2.current_config().voters, target_replicas); + ASSERT_TRUE(cfg_2.current_config().learners.empty()); + } + + test_cfg.maybe_demote_removed_voters(); + + ASSERT_EQ(test_cfg.get_state(), raft::configuration_state::joint); + + // CASE 4. Cancel after demoting removed voters + + test_cfg.cancel_configuration_change(model::revision_id{0}); + // check if reconfiguration is finished by optimizations + if (test_cfg.get_state() == raft::configuration_state::simple) { + ASSERT_EQ(test_cfg.current_config().voters, original_replicas); + ASSERT_TRUE(test_cfg.current_config().learners.empty()); + } else { + test_cfg.cancel_configuration_change(model::revision_id{0}); + transition_configuration_update(test_cfg); + ASSERT_EQ(test_cfg.get_state(), raft::configuration_state::simple); + ASSERT_EQ(test_cfg.current_config().voters, target_replicas); + ASSERT_TRUE(test_cfg.current_config().learners.empty()); + } +} + +/** + * This test verifies if cancelling the configuration change twice at any point + * will lead to the cancellation being reverted. + * example: + * replicas are updated from set A to set B + * 1. A->B + * then a reconfiguration is cancelled + * 2. B->A + * then the reconfiguration is cancelled again + * 3. A->B + * finally after last cancellation + * 4. B->A + * + */ +TEST_P(ConfigurationCancellationTest, TestOddNumberOfCancellations) { + const auto params = GetParam(); + + const std::vector original_replicas + = params.initial_replica_set; + const std::vector target_replicas = params.target_replica_set; + + const auto to_add = diff(target_replicas, original_replicas); + const auto to_remove = diff(original_replicas, target_replicas); + + raft::group_configuration test_cfg = raft::group_configuration( + original_replicas, model::revision_id(0)); + test_cfg.set_version(raft::group_configuration::v_7); + + // trigger reconfiguration + test_cfg.replace(target_replicas, model::revision_id{0}, std::nullopt); + + ASSERT_THAT( + test_cfg.get_configuration_update()->replicas_to_add, + ElementsAreArray(to_add)); + ASSERT_THAT( + test_cfg.get_configuration_update()->replicas_to_remove, + ElementsAreArray(to_remove)); + + // CASE 1. Cancel right after change was requested + + // cancel configuration change, goes straight to simple state as learner can + // be removed immediately + auto cfg_1 = test_cfg; + cfg_1.cancel_configuration_change(model::revision_id{0}); + // check if reconfiguration is finished by optimizations + if (cfg_1.get_state() == raft::configuration_state::simple) { + ASSERT_EQ(cfg_1.current_config().voters, original_replicas); + } else { + cfg_1.cancel_configuration_change(model::revision_id{0}); + cfg_1.cancel_configuration_change(model::revision_id{0}); + + transition_configuration_update(cfg_1); + ASSERT_EQ(cfg_1.current_config().voters, original_replicas); + ASSERT_EQ(cfg_1.get_state(), raft::configuration_state::simple); + ASSERT_TRUE(cfg_1.current_config().learners.empty()); + } + // CASE 2. Cancel after learners promotion + for (size_t cancel_after = 1; cancel_after <= to_add.size(); + ++cancel_after) { + // create a copy of the configuration for each round + auto cfg = test_cfg; + for (size_t i = 0; i < cancel_after; ++i) { + cfg.promote_to_voter(to_add[i]); + } + // update the original configuration for the next step + if (cancel_after == to_add.size()) { + test_cfg = cfg; + } + cfg.cancel_configuration_change(model::revision_id{0}); + cfg.cancel_configuration_change(model::revision_id{0}); + cfg.cancel_configuration_change(model::revision_id{0}); + + transition_configuration_update(cfg); + + ASSERT_EQ(cfg.current_config().voters, original_replicas); + ASSERT_EQ(cfg.get_state(), raft::configuration_state::simple); + ASSERT_TRUE(cfg.current_config().learners.empty()); + } + + // now finish the configuration transition as all learners were promoted + test_cfg.finish_configuration_transition(); + + // CASE 3. Cancel after leaving transitional state + + // check if reconfiguration is finished by optimizations + if (test_cfg.get_state() == raft::configuration_state::simple) { + ASSERT_EQ(test_cfg.current_config().voters, target_replicas); + return; + } + // at every step create a copy of test_cfg and execute cancellations against + // the copy + auto cfg_2 = test_cfg; + + cfg_2.cancel_configuration_change(model::revision_id{0}); + if (cfg_2.get_state() == raft::configuration_state::simple) { + ASSERT_EQ(cfg_2.current_config().voters, original_replicas); + ASSERT_TRUE(cfg_2.current_config().learners.empty()); + } else { + cfg_2.cancel_configuration_change(model::revision_id{0}); + cfg_2.cancel_configuration_change(model::revision_id{0}); + + transition_configuration_update(cfg_2); + + ASSERT_EQ(cfg_2.get_state(), raft::configuration_state::simple); + ASSERT_EQ(cfg_2.current_config().voters, original_replicas); + ASSERT_TRUE(cfg_2.current_config().learners.empty()); + } + + test_cfg.maybe_demote_removed_voters(); + + ASSERT_EQ(test_cfg.get_state(), raft::configuration_state::joint); + + // CASE 4. Cancel after demoting removed voters + + test_cfg.cancel_configuration_change(model::revision_id{0}); + // check if reconfiguration is finished by optimizations + if (test_cfg.get_state() == raft::configuration_state::simple) { + ASSERT_EQ(test_cfg.current_config().voters, original_replicas); + ASSERT_TRUE(test_cfg.current_config().learners.empty()); + return; + } + test_cfg.cancel_configuration_change(model::revision_id{0}); + if (test_cfg.get_state() == raft::configuration_state::simple) { + ASSERT_EQ(test_cfg.current_config().voters, target_replicas); + ASSERT_TRUE(test_cfg.current_config().learners.empty()); + return; + } + test_cfg.cancel_configuration_change(model::revision_id{0}); + transition_configuration_update(test_cfg); + ASSERT_EQ(test_cfg.get_state(), raft::configuration_state::simple); + ASSERT_EQ(test_cfg.current_config().voters, original_replicas); + ASSERT_TRUE(test_cfg.current_config().learners.empty()); +} + +// Simple helper class to handle raft group configuration advancement +struct configuration_advancement_state_machine { + explicit configuration_advancement_state_machine( + raft::group_configuration& cfg) + : cfg(cfg) { + reconcile_state(); + } + + enum class state { + done, + learners_promotion, + exiting_transitional_state, + demoting_voters, + exiting_joint_state, + }; + + enum class direction { + original_to_target, + target_to_original, + }; + + friend std::ostream& operator<<(std::ostream& o, state s) { + switch (s) { + case state::done: + return o << "done"; + case state::learners_promotion: + return o << "learners_promotion"; + case state::demoting_voters: + return o << "demoting_voters"; + case state::exiting_transitional_state: + return o << "exiting_transitional_state"; + case state::exiting_joint_state: + return o << "exiting_joint_state"; + } + } + + void advance_state() { + switch (current_state) { + case state::done: + break; + case state::learners_promotion: + cfg.promote_to_voter(cfg.current_config().learners[0]); + reconcile_state(); + break; + case state::exiting_transitional_state: + cfg.finish_configuration_transition(); + reconcile_state(); + break; + case state::demoting_voters: + cfg.maybe_demote_removed_voters(); + reconcile_state(); + break; + case state::exiting_joint_state: + cfg.discard_old_config(); + reconcile_state(); + break; + } + } + + void cancel_reconfiguration() { + vlog( + logger.info, + "Cancelling reconfiguration in state: {}", + current_state); + cfg.cancel_configuration_change(model::revision_id{0}); + if (dir == direction::original_to_target) { + dir = direction::target_to_original; + } else { + dir = direction::original_to_target; + } + reconcile_state(); + } + void abort_reconfiguration() { + vlog( + logger.info, "Aborting reconfiguration in state: {}", current_state); + cfg.abort_configuration_change(model::revision_id{0}); + + if (dir == direction::original_to_target) { + dir = direction::target_to_original; + } else { + dir = direction::original_to_target; + } + reconcile_state(); + } + void reconcile_state() { + vlog(logger.info, "C: {}", cfg); + if (cfg.get_state() == raft::configuration_state::simple) { + current_state = state::done; + } else if (cfg.get_state() == raft::configuration_state::transitional) { + if (!cfg.current_config().learners.empty()) { + current_state = state::learners_promotion; + } else { + current_state = state::exiting_transitional_state; + } + } else if (cfg.get_state() == raft::configuration_state::joint) { + if (cfg.old_config()->learners.empty()) { + current_state = state::demoting_voters; + } else { + current_state = state::exiting_joint_state; + } + } + } + ss::logger logger = ss::logger("test-config-stm"); + direction dir = direction::original_to_target; + state current_state = state::done; + raft::group_configuration& cfg; +}; + +TEST_P(ConfigurationCancellationTest, TestCancellationAfterAdvancement) { + const auto params = GetParam(); + + const std::vector original_replicas + = params.initial_replica_set; + const std::vector target_replicas = params.target_replica_set; + + const auto to_add = diff(target_replicas, original_replicas); + const auto to_remove = diff(original_replicas, target_replicas); + // execute the test multiple times + for (int i = 0; i < 5000; ++i) { + raft::group_configuration test_cfg = raft::group_configuration( + original_replicas, model::revision_id(0)); + test_cfg.set_version(raft::group_configuration::v_7); + + // trigger reconfiguration + test_cfg.replace(target_replicas, model::revision_id{0}, std::nullopt); + + configuration_advancement_state_machine stm(test_cfg); + /** + * Try advancing state until the reconfiguration is done, cancel in + * random points + */ + while (stm.current_state + != configuration_advancement_state_machine::state::done) { + if (tests::random_bool()) { + if (tests::random_bool()) { + stm.cancel_reconfiguration(); + } else { + stm.abort_reconfiguration(); + } + } + stm.advance_state(); + } + // at some point the configuration will end up in simple state, the + // direction defines if the expected replica set is the original or the + // target + if ( + stm.dir + == configuration_advancement_state_machine::direction:: + original_to_target) { + ASSERT_THAT( + test_cfg.current_config().voters, + UnorderedElementsAreArray(target_replicas)); + } else { + ASSERT_THAT( + test_cfg.current_config().voters, + UnorderedElementsAreArray(original_replicas)); + } + ASSERT_TRUE(test_cfg.current_config().learners.empty()); + } +} + +auto params = Values( + configuration_cancel_test_params({0}, {1}), + configuration_cancel_test_params({0, 1, 2}, {0, 1, 3}), + configuration_cancel_test_params({0, 1, 2}, {10, 11, 12}), + configuration_cancel_test_params({0, 1, 2}, {0}), + configuration_cancel_test_params({0}, {0, 1, 3}), + configuration_cancel_test_params({10}, {0, 1, 3}), + configuration_cancel_test_params({0, 1, 2}, {10}), + configuration_cancel_test_params({0, 1, 2, 3, 4}, {3, 10, 11}), + configuration_cancel_test_params({0, 1, 2}, {0, 1, 2, 3, 4}), + configuration_cancel_test_params({0, 1}, {10, 11})); + +INSTANTIATE_TEST_SUITE_P( + TestEvenNumberOfCancellations, ConfigurationCancellationTest, params); + +INSTANTIATE_TEST_SUITE_P( + TestOddNumberOfCancellations, ConfigurationCancellationTest, params); + +INSTANTIATE_TEST_SUITE_P( + TestCancellationAfterAdvancement, ConfigurationCancellationTest, params); From 5d38b593878f52797e6bee819a6183734acf9e79 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Tue, 3 Dec 2024 10:48:53 +0100 Subject: [PATCH 136/229] feature: introduced a feature for symmetic reconfiguration cancel MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Michał Maślanka --- src/v/features/feature_table.cc | 2 ++ src/v/features/feature_table.h | 8 +++++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/src/v/features/feature_table.cc b/src/v/features/feature_table.cc index 346b3813ae2b..44ae82e2ec36 100644 --- a/src/v/features/feature_table.cc +++ b/src/v/features/feature_table.cc @@ -98,6 +98,8 @@ std::string_view to_string_view(feature f) { return "shadow_indexing_split_topic_property_update"; case feature::datalake_iceberg: return "datalake_iceberg"; + case feature::raft_symmetric_reconfiguration_cancel: + return "raft_symmetric_reconfiguration_cancel"; /* * testing features diff --git a/src/v/features/feature_table.h b/src/v/features/feature_table.h index cef32382f01f..f67e951a00bf 100644 --- a/src/v/features/feature_table.h +++ b/src/v/features/feature_table.h @@ -69,7 +69,7 @@ enum class feature : std::uint64_t { partition_properties_stm = 1ULL << 52U, shadow_indexing_split_topic_property_update = 1ULL << 53U, datalake_iceberg = 1ULL << 54U, - + raft_symmetric_reconfiguration_cancel = 1ULL << 55U, // Dummy features for testing only test_alpha = 1ULL << 61U, test_bravo = 1ULL << 62U, @@ -414,6 +414,12 @@ inline constexpr std::array feature_schema{ feature::datalake_iceberg, feature_spec::available_policy::always, feature_spec::prepare_policy::always}, + feature_spec{ + release_version::v25_1_1, + "raft_symmetric_reconfiguration_cancel", + feature::raft_symmetric_reconfiguration_cancel, + feature_spec::available_policy::always, + feature_spec::prepare_policy::always}, }; std::string_view to_string_view(feature); From 5e4fa7f19e58d7e9715ef4d6e903d411394758cc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Tue, 3 Dec 2024 10:52:36 +0100 Subject: [PATCH 137/229] feature_table: retire serde_raft_configuration feature MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Michał Maślanka --- src/v/features/feature_table.cc | 2 -- src/v/features/feature_table.h | 8 +------- src/v/raft/consensus.cc | 14 ++------------ src/v/raft/consensus.h | 4 ---- src/v/raft/group_configuration.cc | 8 ++------ 5 files changed, 5 insertions(+), 31 deletions(-) diff --git a/src/v/features/feature_table.cc b/src/v/features/feature_table.cc index 44ae82e2ec36..eae3596b6c59 100644 --- a/src/v/features/feature_table.cc +++ b/src/v/features/feature_table.cc @@ -62,8 +62,6 @@ std::string_view to_string_view(feature f) { return "broker_time_based_retention"; case feature::wasm_transforms: return "wasm_transforms"; - case feature::raft_config_serde: - return "raft_config_serde"; case feature::fast_partition_reconfiguration: return "fast_partition_reconfiguration"; case feature::disabling_partitions: diff --git a/src/v/features/feature_table.h b/src/v/features/feature_table.h index f67e951a00bf..c6fc643a6aa0 100644 --- a/src/v/features/feature_table.h +++ b/src/v/features/feature_table.h @@ -51,7 +51,6 @@ enum class feature : std::uint64_t { enhanced_force_reconfiguration = 1ULL << 33U, broker_time_based_retention = 1ULL << 34U, wasm_transforms = 1ULL << 35U, - raft_config_serde = 1ULL << 36U, fast_partition_reconfiguration = 1ULL << 38U, disabling_partitions = 1ULL << 39U, cloud_metadata_cluster_recovery = 1ULL << 40U, @@ -113,6 +112,7 @@ inline const std::unordered_set retired_features = { "partition_move_revert_cancel", "rpc_transport_unknown_errc", "raft_append_entries_serde", + "raft_config_serde", }; // The latest_version associated with past releases. Increment this @@ -306,12 +306,6 @@ inline constexpr std::array feature_schema{ feature::wasm_transforms, feature_spec::available_policy::always, feature_spec::prepare_policy::always}, - feature_spec{ - release_version::v23_3_1, - "raft_config_serde", - feature::raft_config_serde, - feature_spec::available_policy::always, - feature_spec::prepare_policy::always}, feature_spec{ release_version::v23_3_1, "fast_partition_reconfiguration", diff --git a/src/v/raft/consensus.cc b/src/v/raft/consensus.cc index 550da57110fc..1074593ecbdc 100644 --- a/src/v/raft/consensus.cc +++ b/src/v/raft/consensus.cc @@ -1354,15 +1354,7 @@ ss::future consensus::force_replace_configuration_locally( auto units = co_await _op_lock.get_units(); auto new_cfg = group_configuration( std::move(voters), std::move(learners), new_revision); - if ( - new_cfg.version() == group_configuration::v_5 - && use_serde_configuration()) { - vlog( - _ctxlog.debug, - "Upgrading configuration {} version to 6", - new_cfg); - new_cfg.set_version(group_configuration::v_6); - } + try_updating_configuration_version(new_cfg); vlog(_ctxlog.info, "Force replacing configuration with: {}", new_cfg); update_follower_stats(new_cfg); @@ -2619,9 +2611,7 @@ ss::future consensus::replicate_configuration( return ss::with_gate( _bg, [this, u = std::move(u), cfg = std::move(cfg)]() mutable { maybe_upgrade_configuration_to_v4(cfg); - if ( - cfg.version() == group_configuration::v_5 - && use_serde_configuration()) { + if (cfg.version() == group_configuration::v_5) { vlog( _ctxlog.debug, "Upgrading configuration {} version to 6", cfg); cfg.set_version(group_configuration::v_6); diff --git a/src/v/raft/consensus.h b/src/v/raft/consensus.h index 7bb049b90a58..15a7b94569cb 100644 --- a/src/v/raft/consensus.h +++ b/src/v/raft/consensus.h @@ -776,10 +776,6 @@ class consensus { std::optional get_learner_start_offset() const; - bool use_serde_configuration() const { - return _features.is_active(features::feature::raft_config_serde); - } - flush_delay_t compute_max_flush_delay() const; ss::future<> do_flush(); diff --git a/src/v/raft/group_configuration.cc b/src/v/raft/group_configuration.cc index e93e46b9f9bf..849beccfc9b0 100644 --- a/src/v/raft/group_configuration.cc +++ b/src/v/raft/group_configuration.cc @@ -311,17 +311,13 @@ group_configuration::group_configuration( group_configuration::group_configuration( std::vector initial_nodes, model::revision_id rev) - : _version(v_5) - , _revision(rev) { - _current.voters = std::move(initial_nodes); -} + : group_configuration(std::move(initial_nodes), {}, rev) {} group_configuration::group_configuration( std::vector voters, std::vector learners, model::revision_id rev) - : _version(v_5) - , _revision(rev) { + : _revision(rev) { _current.voters = std::move(voters); _current.learners = std::move(learners); } From 9ba29da4e3f9b49e7a7c59708f86f181b1bae704 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Tue, 3 Dec 2024 11:12:16 +0100 Subject: [PATCH 138/229] raft: update configuration to version 7 when possible MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Added code that updates raft group configuration to version 7 if it is possible to do so. The change simillar to change from v3 to v4 can only be done when configuration is in simple state and all nodes supports v7. Signed-off-by: Michał Maślanka --- src/v/raft/consensus.cc | 59 +++++++++++++++++++++++++++-------------- src/v/raft/consensus.h | 6 +++++ 2 files changed, 45 insertions(+), 20 deletions(-) diff --git a/src/v/raft/consensus.cc b/src/v/raft/consensus.cc index 1074593ecbdc..0c59f3297695 100644 --- a/src/v/raft/consensus.cc +++ b/src/v/raft/consensus.cc @@ -1106,7 +1106,7 @@ ss::future consensus::change_configuration(Func&& f) { return ss::make_ready_future( errc::configuration_change_in_progress); } - maybe_upgrade_configuration_to_v4(latest_cfg); + try_updating_configuration_version(latest_cfg); result res = f(std::move(latest_cfg)); if (res) { if (res.value().revision_id() < config().revision_id()) { @@ -1176,29 +1176,30 @@ ss::future consensus::add_group_member( model::revision_id new_revision, std::optional learner_start_offset) { vlog(_ctxlog.trace, "Adding member: {}", node); - return change_configuration([node, new_revision, learner_start_offset]( - group_configuration current) mutable { - using ret_t = result; - if (current.contains(node)) { - return ret_t{errc::node_already_exists}; - } - current.set_version(raft::group_configuration::v_5); - current.add(node, new_revision, learner_start_offset); + return change_configuration( + [this, node, new_revision, learner_start_offset]( + group_configuration current) mutable { + using ret_t = result; + if (current.contains(node)) { + return ret_t{errc::node_already_exists}; + } + try_updating_configuration_version(current); + current.add(node, new_revision, learner_start_offset); - return ret_t{std::move(current)}; - }); + return ret_t{std::move(current)}; + }); } ss::future consensus::remove_member(vnode node, model::revision_id new_revision) { vlog(_ctxlog.trace, "Removing member: {}", node); return change_configuration( - [node, new_revision](group_configuration current) { + [this, node, new_revision](group_configuration current) { using ret_t = result; if (!current.contains(node)) { return ret_t{errc::node_does_not_exists}; } - current.set_version(raft::group_configuration::v_5); + try_updating_configuration_version(current); current.remove(node, new_revision); if (current.current_config().voters.empty()) { @@ -1216,7 +1217,7 @@ ss::future consensus::replace_configuration( [this, nodes = std::move(nodes), new_revision, learner_start_offset]( group_configuration current) mutable { auto old = current; - current.set_version(raft::group_configuration::v_5); + try_updating_configuration_version(current); current.replace(nodes, new_revision, learner_start_offset); vlog( _ctxlog.debug, @@ -1367,6 +1368,29 @@ ss::future consensus::force_replace_configuration_locally( co_return errc::success; } +void consensus::try_updating_configuration_version(group_configuration& cfg) { + maybe_upgrade_configuration_to_v4(cfg); + + auto version = cfg.version(); + if ( + version >= group_configuration::v_4 + && version < group_configuration::v_7) { + version = supports_symmetric_reconfiguration_cancel() + && cfg.get_state() == configuration_state::simple + ? group_configuration::v_7 + : group_configuration::v_6; + if (version == cfg.version()) { + return; + } + vlog( + _ctxlog.debug, + "Upgrading configuration {} version to {}", + cfg, + version); + cfg.set_version(version); + } +} + ss::future<> consensus::start( std::optional stm_manager_builder, std::optional xst_state) { @@ -2610,12 +2634,7 @@ ss::future consensus::replicate_configuration( vlog(_ctxlog.debug, "Replicating group configuration {}", cfg); return ss::with_gate( _bg, [this, u = std::move(u), cfg = std::move(cfg)]() mutable { - maybe_upgrade_configuration_to_v4(cfg); - if (cfg.version() == group_configuration::v_5) { - vlog( - _ctxlog.debug, "Upgrading configuration {} version to 6", cfg); - cfg.set_version(group_configuration::v_6); - } + try_updating_configuration_version(cfg); auto batches = details::serialize_configuration_as_batches( std::move(cfg)); diff --git a/src/v/raft/consensus.h b/src/v/raft/consensus.h index 15a7b94569cb..08ac762286f1 100644 --- a/src/v/raft/consensus.h +++ b/src/v/raft/consensus.h @@ -779,6 +779,12 @@ class consensus { flush_delay_t compute_max_flush_delay() const; ss::future<> do_flush(); + bool supports_symmetric_reconfiguration_cancel() const { + return _features.is_active( + features::feature::raft_symmetric_reconfiguration_cancel); + } + + void try_updating_configuration_version(group_configuration& cfg); // args vnode _self; raft::group_id _group; From 70af61fc236bf43ca357ae37f17afb22847b169b Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Thu, 12 Dec 2024 08:58:10 -0500 Subject: [PATCH 139/229] `rptest`: fix `cloud_storage_type` parameterization Hard coding this to `CloudStorageType.S3` will fail for Azure CDT. Use the `get_cloud_storage_type()` function to determine the type instead per the environment. --- tests/rptest/tests/random_node_operations_test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/rptest/tests/random_node_operations_test.py b/tests/rptest/tests/random_node_operations_test.py index b59526912bec..0055b08d6e47 100644 --- a/tests/rptest/tests/random_node_operations_test.py +++ b/tests/rptest/tests/random_node_operations_test.py @@ -23,7 +23,7 @@ from rptest.clients.types import TopicSpec from rptest.clients.default import DefaultClient from rptest.services.kgo_verifier_services import KgoVerifierConsumerGroupConsumer, KgoVerifierProducer -from rptest.services.redpanda import CHAOS_LOG_ALLOW_LIST, PREV_VERSION_LOG_ALLOW_LIST, CloudStorageType, LoggingConfig, PandaproxyConfig, SISettings, SchemaRegistryConfig +from rptest.services.redpanda import CHAOS_LOG_ALLOW_LIST, PREV_VERSION_LOG_ALLOW_LIST, CloudStorageType, LoggingConfig, PandaproxyConfig, SISettings, SchemaRegistryConfig, get_cloud_storage_type from rptest.services.redpanda_installer import RedpandaInstaller from rptest.utils.mode_checks import cleanup_on_early_exit, skip_debug_mode, skip_fips_mode from rptest.utils.node_operations import FailureInjectorBackgroundThread, NodeOpsExecutor, generate_random_workload @@ -323,7 +323,7 @@ def maybe_enable_iceberg_for_topic(self, topic_spec: TopicSpec, mixed_versions=[True, False], with_tiered_storage=[True, False], with_iceberg=[True, False], - cloud_storage_type=[CloudStorageType.S3]) + cloud_storage_type=get_cloud_storage_type()) def test_node_operations(self, enable_failures, mixed_versions, with_tiered_storage, with_iceberg, cloud_storage_type): From fca1898ddae6736732a1afec29a0db50294edf49 Mon Sep 17 00:00:00 2001 From: Stephan Dollberg Date: Thu, 12 Dec 2024 15:24:38 +0000 Subject: [PATCH 140/229] bazel: Enable SEASTAR_DEBUG_SHARED_PTR in debug Adds seastar define to debug builds which enables `ss::(lw)_shared_ptr` owner checks. --- bazel/thirdparty/seastar.BUILD | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/bazel/thirdparty/seastar.BUILD b/bazel/thirdparty/seastar.BUILD index aa84d81988a6..0de73d6f40ce 100644 --- a/bazel/thirdparty/seastar.BUILD +++ b/bazel/thirdparty/seastar.BUILD @@ -570,7 +570,10 @@ cc_library( ":use_logger_compile_time_fmt": ["SEASTAR_LOGGER_COMPILE_TIME_FMT"], "//conditions:default": [], }) + select({ - ":with_debug": ["SEASTAR_DEBUG"], + ":with_debug": [ + "SEASTAR_DEBUG", + "SEASTAR_DEBUG_SHARED_PTR", + ], "//conditions:default": [], }), includes = [ From 05cd9280627149f20881f176f1b629462ebd88fa Mon Sep 17 00:00:00 2001 From: r-vasquez Date: Fri, 6 Dec 2024 12:24:01 -0800 Subject: [PATCH 141/229] rpk: add well-known protobuf types to rpk This is the same set of well-known types that both Redpanda and Redpanda Console already support This allows users to use the whole suite and encode/decode using these well-known types. One rpk unit tests verify that the embedded types are the same as the ones in the Redpanda repo, hence the need to trigger rpk tests on changes in pandaproxy/schema_registry/protobuf --- .github/workflows/rpk-build.yml | 2 + src/go/rpk/pkg/serde/BUILD | 1 + src/go/rpk/pkg/serde/embed/BUILD | 38 +++ src/go/rpk/pkg/serde/embed/embed.go | 62 +++++ src/go/rpk/pkg/serde/embed/embed_test.go | 42 +++ .../pkg/serde/embed/protobuf/.clang-format | 2 + .../serde/embed/protobuf/confluent/meta.proto | 27 ++ .../protobuf/confluent/types/decimal.proto | 17 ++ .../embed/protobuf/google/type/README.md | 16 ++ .../google/type/calendar_period.proto | 56 ++++ .../embed/protobuf/google/type/color.proto | 174 ++++++++++++ .../embed/protobuf/google/type/date.proto | 52 ++++ .../embed/protobuf/google/type/datetime.proto | 104 +++++++ .../protobuf/google/type/dayofweek.proto | 50 ++++ .../embed/protobuf/google/type/decimal.proto | 95 +++++++ .../embed/protobuf/google/type/expr.proto | 73 +++++ .../embed/protobuf/google/type/fraction.proto | 33 +++ .../embed/protobuf/google/type/interval.proto | 46 +++ .../embed/protobuf/google/type/latlng.proto | 37 +++ .../protobuf/google/type/localized_text.proto | 36 +++ .../embed/protobuf/google/type/money.proto | 42 +++ .../embed/protobuf/google/type/month.proto | 65 +++++ .../protobuf/google/type/phone_number.proto | 113 ++++++++ .../protobuf/google/type/postal_address.proto | 134 +++++++++ .../protobuf/google/type/quaternion.proto | 94 +++++++ .../protobuf/google/type/timeofday.proto | 44 +++ src/go/rpk/pkg/serde/proto.go | 25 +- src/go/rpk/pkg/serde/proto_test.go | 262 ++++++++++++++++++ src/v/pandaproxy/BUILD | 1 + tests/rptest/clients/rpk.py | 6 +- tests/rptest/tests/rpk_registry_test.py | 75 ++++- 31 files changed, 1813 insertions(+), 11 deletions(-) create mode 100644 src/go/rpk/pkg/serde/embed/BUILD create mode 100644 src/go/rpk/pkg/serde/embed/embed.go create mode 100644 src/go/rpk/pkg/serde/embed/embed_test.go create mode 100644 src/go/rpk/pkg/serde/embed/protobuf/.clang-format create mode 100644 src/go/rpk/pkg/serde/embed/protobuf/confluent/meta.proto create mode 100644 src/go/rpk/pkg/serde/embed/protobuf/confluent/types/decimal.proto create mode 100644 src/go/rpk/pkg/serde/embed/protobuf/google/type/README.md create mode 100644 src/go/rpk/pkg/serde/embed/protobuf/google/type/calendar_period.proto create mode 100644 src/go/rpk/pkg/serde/embed/protobuf/google/type/color.proto create mode 100644 src/go/rpk/pkg/serde/embed/protobuf/google/type/date.proto create mode 100644 src/go/rpk/pkg/serde/embed/protobuf/google/type/datetime.proto create mode 100644 src/go/rpk/pkg/serde/embed/protobuf/google/type/dayofweek.proto create mode 100644 src/go/rpk/pkg/serde/embed/protobuf/google/type/decimal.proto create mode 100644 src/go/rpk/pkg/serde/embed/protobuf/google/type/expr.proto create mode 100644 src/go/rpk/pkg/serde/embed/protobuf/google/type/fraction.proto create mode 100644 src/go/rpk/pkg/serde/embed/protobuf/google/type/interval.proto create mode 100644 src/go/rpk/pkg/serde/embed/protobuf/google/type/latlng.proto create mode 100644 src/go/rpk/pkg/serde/embed/protobuf/google/type/localized_text.proto create mode 100644 src/go/rpk/pkg/serde/embed/protobuf/google/type/money.proto create mode 100644 src/go/rpk/pkg/serde/embed/protobuf/google/type/month.proto create mode 100644 src/go/rpk/pkg/serde/embed/protobuf/google/type/phone_number.proto create mode 100644 src/go/rpk/pkg/serde/embed/protobuf/google/type/postal_address.proto create mode 100644 src/go/rpk/pkg/serde/embed/protobuf/google/type/quaternion.proto create mode 100644 src/go/rpk/pkg/serde/embed/protobuf/google/type/timeofday.proto diff --git a/.github/workflows/rpk-build.yml b/.github/workflows/rpk-build.yml index b762a90a4991..1b42e4113b09 100644 --- a/.github/workflows/rpk-build.yml +++ b/.github/workflows/rpk-build.yml @@ -13,10 +13,12 @@ on: branches: [dev] paths: - 'src/go/rpk/**' + - 'src/v/pandaproxy/schema_registry/protobuf/**' - '.github/workflows/rpk-build.yml' pull_request: paths: - 'src/go/rpk/**' + - 'src/v/pandaproxy/schema_registry/protobuf/**' - '.github/workflows/rpk-build.yml' jobs: test: diff --git a/src/go/rpk/pkg/serde/BUILD b/src/go/rpk/pkg/serde/BUILD index 240f96c40e00..a7fb6b167a98 100644 --- a/src/go/rpk/pkg/serde/BUILD +++ b/src/go/rpk/pkg/serde/BUILD @@ -11,6 +11,7 @@ go_library( importpath = "github.com/redpanda-data/redpanda/src/go/rpk/pkg/serde", visibility = ["//visibility:public"], deps = [ + "//src/go/rpk/pkg/serde/embed", "@com_github_bufbuild_protocompile//:protocompile", "@com_github_bufbuild_protocompile//linker", "@com_github_hamba_avro_v2//:avro", diff --git a/src/go/rpk/pkg/serde/embed/BUILD b/src/go/rpk/pkg/serde/embed/BUILD new file mode 100644 index 000000000000..303033b8187c --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/BUILD @@ -0,0 +1,38 @@ +load("@rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "embed", + srcs = ["embed.go"], + embedsrcs = [ + "protobuf/confluent/meta.proto", + "protobuf/confluent/types/decimal.proto", + "protobuf/google/type/calendar_period.proto", + "protobuf/google/type/color.proto", + "protobuf/google/type/date.proto", + "protobuf/google/type/datetime.proto", + "protobuf/google/type/dayofweek.proto", + "protobuf/google/type/decimal.proto", + "protobuf/google/type/expr.proto", + "protobuf/google/type/fraction.proto", + "protobuf/google/type/interval.proto", + "protobuf/google/type/latlng.proto", + "protobuf/google/type/localized_text.proto", + "protobuf/google/type/money.proto", + "protobuf/google/type/month.proto", + "protobuf/google/type/phone_number.proto", + "protobuf/google/type/postal_address.proto", + "protobuf/google/type/quaternion.proto", + "protobuf/google/type/timeofday.proto", + ], + importpath = "github.com/redpanda-data/redpanda/src/go/rpk/pkg/serde/embed", + visibility = ["//visibility:public"], +) + +go_test( + name = "embed_test", + size = "small", + srcs = ["embed_test.go"], + data = ["//src/v/pandaproxy:schema_registry_protos"], + embed = [":embed"], + deps = ["@com_github_stretchr_testify//require"], +) diff --git a/src/go/rpk/pkg/serde/embed/embed.go b/src/go/rpk/pkg/serde/embed/embed.go new file mode 100644 index 000000000000..07d7bd1ba502 --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/embed.go @@ -0,0 +1,62 @@ +// Copyright 2024 Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 + +package embed + +import ( + "embed" + "io/fs" + "path/filepath" + "sync" +) + +//go:embed protobuf/google/type/*.proto protobuf/confluent/*.proto protobuf/confluent/types/*.proto +var content embed.FS + +var ( + once sync.Once + protoMap map[string]string +) + +// CommonProtoFiles returns the file system representation of the common +// protobuf types. +func CommonProtoFiles() (fs.FS, error) { + return fs.Sub(content, "protobuf") +} + +// CommonProtoFileMap returns the map representation of the common protobuf +// types. This is useful for protoreflect parsing. +func CommonProtoFileMap() (map[string]string, error) { + protoFS, err := CommonProtoFiles() + if err != nil { + return nil, err + } + + once.Do(func() { + protoMap = make(map[string]string) + err = fs.WalkDir(protoFS, ".", func(path string, d fs.DirEntry, err error) error { + if err != nil { + return err + } + if d.IsDir() || filepath.Ext(path) != ".proto" { + return nil + } + data, err := fs.ReadFile(protoFS, path) + if err == nil { + protoMap[path] = string(data) + } + return nil + }) + }) + + if err != nil { + return nil, err + } + return protoMap, err +} diff --git a/src/go/rpk/pkg/serde/embed/embed_test.go b/src/go/rpk/pkg/serde/embed/embed_test.go new file mode 100644 index 000000000000..e96c728f3d0b --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/embed_test.go @@ -0,0 +1,42 @@ +package embed + +import ( + "io/fs" + "os" + "path/filepath" + "testing" + + "github.com/stretchr/testify/require" +) + +func TestEmbeddedFiles(t *testing.T) { + t.Run("Test Embedded files in rpk, equal to Redpanda", func(t *testing.T) { + // /src/v/pandaproxy/schema_registry/protobuf + redpandaProtoFS := os.DirFS("../../../../../v/pandaproxy/schema_registry/protobuf/") + redpandaMap := make(map[string]string) + err := fs.WalkDir(redpandaProtoFS, ".", func(path string, d fs.DirEntry, err error) error { + if err != nil { + return err + } + if d.IsDir() || filepath.Ext(path) != ".proto" { + return nil + } + data, err := fs.ReadFile(redpandaProtoFS, path) + if err == nil { + redpandaMap[path] = string(data) + } + return nil + }) + + embeddedMap, err := CommonProtoFileMap() + require.NoError(t, err) + + for path, embedContent := range embeddedMap { + if rpContent, ok := redpandaMap[path]; ok { + require.Equalf(t, rpContent, embedContent, "Contents of %v have changed vs the embedded rpk files", path) + } else { + t.Fatalf("%s not found in Redpanda files", path) + } + } + }) +} diff --git a/src/go/rpk/pkg/serde/embed/protobuf/.clang-format b/src/go/rpk/pkg/serde/embed/protobuf/.clang-format new file mode 100644 index 000000000000..ef9566ac8a7a --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/protobuf/.clang-format @@ -0,0 +1,2 @@ +Language: Proto +DisableFormat: true diff --git a/src/go/rpk/pkg/serde/embed/protobuf/confluent/meta.proto b/src/go/rpk/pkg/serde/embed/protobuf/confluent/meta.proto new file mode 100644 index 000000000000..4439139d0c68 --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/protobuf/confluent/meta.proto @@ -0,0 +1,27 @@ +syntax = "proto3"; +package confluent; + +import "google/protobuf/descriptor.proto"; + +option go_package="../confluent"; + +message Meta { + string doc = 1; + map params = 2; +} + +extend google.protobuf.FileOptions { + Meta file_meta = 1088; +} +extend google.protobuf.MessageOptions { + Meta message_meta = 1088; +} +extend google.protobuf.FieldOptions { + Meta field_meta = 1088; +} +extend google.protobuf.EnumOptions { + Meta enum_meta = 1088; +} +extend google.protobuf.EnumValueOptions { + Meta enum_value_meta = 1088; +} diff --git a/src/go/rpk/pkg/serde/embed/protobuf/confluent/types/decimal.proto b/src/go/rpk/pkg/serde/embed/protobuf/confluent/types/decimal.proto new file mode 100644 index 000000000000..75d8b9b46f0a --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/protobuf/confluent/types/decimal.proto @@ -0,0 +1,17 @@ +syntax = "proto3"; + +package confluent.type; + +option go_package="../types"; + +message Decimal { + + // The two's-complement representation of the unscaled integer value in big-endian byte order + bytes value = 1; + + // The precision + uint32 precision = 2; + + // The scale + int32 scale = 3; +} \ No newline at end of file diff --git a/src/go/rpk/pkg/serde/embed/protobuf/google/type/README.md b/src/go/rpk/pkg/serde/embed/protobuf/google/type/README.md new file mode 100644 index 000000000000..6caf02cf1f5e --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/protobuf/google/type/README.md @@ -0,0 +1,16 @@ +# Google Common Types + +This package contains definitions of common types for Google APIs. +All types defined in this package are suitable for different APIs to +exchange data, and will never break binary compatibility. They should +have design quality comparable to major programming languages like +Java and C#. + +NOTE: Some common types are defined in the package `google.protobuf` +as they are directly supported by Protocol Buffers compiler and +runtime. Those types are called Well-Known Types. + +## Java Utilities + +A set of Java utilities for the Common Types are provided in the +`//java/com/google/type/util/` package. \ No newline at end of file diff --git a/src/go/rpk/pkg/serde/embed/protobuf/google/type/calendar_period.proto b/src/go/rpk/pkg/serde/embed/protobuf/google/type/calendar_period.proto new file mode 100644 index 000000000000..82f5690b752b --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/protobuf/google/type/calendar_period.proto @@ -0,0 +1,56 @@ +// Copyright 2021 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto3"; + +package google.type; + +option go_package = "google.golang.org/genproto/googleapis/type/calendarperiod;calendarperiod"; +option java_multiple_files = true; +option java_outer_classname = "CalendarPeriodProto"; +option java_package = "com.google.type"; +option objc_class_prefix = "GTP"; + +// A `CalendarPeriod` represents the abstract concept of a time period that has +// a canonical start. Grammatically, "the start of the current +// `CalendarPeriod`." All calendar times begin at midnight UTC. +enum CalendarPeriod { + // Undefined period, raises an error. + CALENDAR_PERIOD_UNSPECIFIED = 0; + + // A day. + DAY = 1; + + // A week. Weeks begin on Monday, following + // [ISO 8601](https://en.wikipedia.org/wiki/ISO_week_date). + WEEK = 2; + + // A fortnight. The first calendar fortnight of the year begins at the start + // of week 1 according to + // [ISO 8601](https://en.wikipedia.org/wiki/ISO_week_date). + FORTNIGHT = 3; + + // A month. + MONTH = 4; + + // A quarter. Quarters start on dates 1-Jan, 1-Apr, 1-Jul, and 1-Oct of each + // year. + QUARTER = 5; + + // A half-year. Half-years start on dates 1-Jan and 1-Jul. + HALF = 6; + + // A year. + YEAR = 7; +} diff --git a/src/go/rpk/pkg/serde/embed/protobuf/google/type/color.proto b/src/go/rpk/pkg/serde/embed/protobuf/google/type/color.proto new file mode 100644 index 000000000000..5dc85a6a3856 --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/protobuf/google/type/color.proto @@ -0,0 +1,174 @@ +// Copyright 2021 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto3"; + +package google.type; + +import "google/protobuf/wrappers.proto"; + +option cc_enable_arenas = true; +option go_package = "google.golang.org/genproto/googleapis/type/color;color"; +option java_multiple_files = true; +option java_outer_classname = "ColorProto"; +option java_package = "com.google.type"; +option objc_class_prefix = "GTP"; + +// Represents a color in the RGBA color space. This representation is designed +// for simplicity of conversion to/from color representations in various +// languages over compactness. For example, the fields of this representation +// can be trivially provided to the constructor of `java.awt.Color` in Java; it +// can also be trivially provided to UIColor's `+colorWithRed:green:blue:alpha` +// method in iOS; and, with just a little work, it can be easily formatted into +// a CSS `rgba()` string in JavaScript. +// +// This reference page doesn't carry information about the absolute color +// space +// that should be used to interpret the RGB value (e.g. sRGB, Adobe RGB, +// DCI-P3, BT.2020, etc.). By default, applications should assume the sRGB color +// space. +// +// When color equality needs to be decided, implementations, unless +// documented otherwise, treat two colors as equal if all their red, +// green, blue, and alpha values each differ by at most 1e-5. +// +// Example (Java): +// +// import com.google.type.Color; +// +// // ... +// public static java.awt.Color fromProto(Color protocolor) { +// float alpha = protocolor.hasAlpha() +// ? protocolor.getAlpha().getValue() +// : 1.0; +// +// return new java.awt.Color( +// protocolor.getRed(), +// protocolor.getGreen(), +// protocolor.getBlue(), +// alpha); +// } +// +// public static Color toProto(java.awt.Color color) { +// float red = (float) color.getRed(); +// float green = (float) color.getGreen(); +// float blue = (float) color.getBlue(); +// float denominator = 255.0; +// Color.Builder resultBuilder = +// Color +// .newBuilder() +// .setRed(red / denominator) +// .setGreen(green / denominator) +// .setBlue(blue / denominator); +// int alpha = color.getAlpha(); +// if (alpha != 255) { +// result.setAlpha( +// FloatValue +// .newBuilder() +// .setValue(((float) alpha) / denominator) +// .build()); +// } +// return resultBuilder.build(); +// } +// // ... +// +// Example (iOS / Obj-C): +// +// // ... +// static UIColor* fromProto(Color* protocolor) { +// float red = [protocolor red]; +// float green = [protocolor green]; +// float blue = [protocolor blue]; +// FloatValue* alpha_wrapper = [protocolor alpha]; +// float alpha = 1.0; +// if (alpha_wrapper != nil) { +// alpha = [alpha_wrapper value]; +// } +// return [UIColor colorWithRed:red green:green blue:blue alpha:alpha]; +// } +// +// static Color* toProto(UIColor* color) { +// CGFloat red, green, blue, alpha; +// if (![color getRed:&red green:&green blue:&blue alpha:&alpha]) { +// return nil; +// } +// Color* result = [[Color alloc] init]; +// [result setRed:red]; +// [result setGreen:green]; +// [result setBlue:blue]; +// if (alpha <= 0.9999) { +// [result setAlpha:floatWrapperWithValue(alpha)]; +// } +// [result autorelease]; +// return result; +// } +// // ... +// +// Example (JavaScript): +// +// // ... +// +// var protoToCssColor = function(rgb_color) { +// var redFrac = rgb_color.red || 0.0; +// var greenFrac = rgb_color.green || 0.0; +// var blueFrac = rgb_color.blue || 0.0; +// var red = Math.floor(redFrac * 255); +// var green = Math.floor(greenFrac * 255); +// var blue = Math.floor(blueFrac * 255); +// +// if (!('alpha' in rgb_color)) { +// return rgbToCssColor(red, green, blue); +// } +// +// var alphaFrac = rgb_color.alpha.value || 0.0; +// var rgbParams = [red, green, blue].join(','); +// return ['rgba(', rgbParams, ',', alphaFrac, ')'].join(''); +// }; +// +// var rgbToCssColor = function(red, green, blue) { +// var rgbNumber = new Number((red << 16) | (green << 8) | blue); +// var hexString = rgbNumber.toString(16); +// var missingZeros = 6 - hexString.length; +// var resultBuilder = ['#']; +// for (var i = 0; i < missingZeros; i++) { +// resultBuilder.push('0'); +// } +// resultBuilder.push(hexString); +// return resultBuilder.join(''); +// }; +// +// // ... +message Color { + // The amount of red in the color as a value in the interval [0, 1]. + float red = 1; + + // The amount of green in the color as a value in the interval [0, 1]. + float green = 2; + + // The amount of blue in the color as a value in the interval [0, 1]. + float blue = 3; + + // The fraction of this color that should be applied to the pixel. That is, + // the final pixel color is defined by the equation: + // + // `pixel color = alpha * (this color) + (1.0 - alpha) * (background color)` + // + // This means that a value of 1.0 corresponds to a solid color, whereas + // a value of 0.0 corresponds to a completely transparent color. This + // uses a wrapper message rather than a simple float scalar so that it is + // possible to distinguish between a default value and the value being unset. + // If omitted, this color object is rendered as a solid color + // (as if the alpha value had been explicitly given a value of 1.0). + google.protobuf.FloatValue alpha = 4; +} diff --git a/src/go/rpk/pkg/serde/embed/protobuf/google/type/date.proto b/src/go/rpk/pkg/serde/embed/protobuf/google/type/date.proto new file mode 100644 index 000000000000..e4e730e6f5a9 --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/protobuf/google/type/date.proto @@ -0,0 +1,52 @@ +// Copyright 2021 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto3"; + +package google.type; + +option cc_enable_arenas = true; +option go_package = "google.golang.org/genproto/googleapis/type/date;date"; +option java_multiple_files = true; +option java_outer_classname = "DateProto"; +option java_package = "com.google.type"; +option objc_class_prefix = "GTP"; + +// Represents a whole or partial calendar date, such as a birthday. The time of +// day and time zone are either specified elsewhere or are insignificant. The +// date is relative to the Gregorian Calendar. This can represent one of the +// following: +// +// * A full date, with non-zero year, month, and day values +// * A month and day value, with a zero year, such as an anniversary +// * A year on its own, with zero month and day values +// * A year and month value, with a zero day, such as a credit card expiration +// date +// +// Related types are [google.type.TimeOfDay][google.type.TimeOfDay] and +// `google.protobuf.Timestamp`. +message Date { + // Year of the date. Must be from 1 to 9999, or 0 to specify a date without + // a year. + int32 year = 1; + + // Month of a year. Must be from 1 to 12, or 0 to specify a year without a + // month and day. + int32 month = 2; + + // Day of a month. Must be from 1 to 31 and valid for the year and month, or 0 + // to specify a year by itself or a year and month where the day isn't + // significant. + int32 day = 3; +} diff --git a/src/go/rpk/pkg/serde/embed/protobuf/google/type/datetime.proto b/src/go/rpk/pkg/serde/embed/protobuf/google/type/datetime.proto new file mode 100644 index 000000000000..cfed85d70a10 --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/protobuf/google/type/datetime.proto @@ -0,0 +1,104 @@ +// Copyright 2021 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto3"; + +package google.type; + +import "google/protobuf/duration.proto"; + +option cc_enable_arenas = true; +option go_package = "google.golang.org/genproto/googleapis/type/datetime;datetime"; +option java_multiple_files = true; +option java_outer_classname = "DateTimeProto"; +option java_package = "com.google.type"; +option objc_class_prefix = "GTP"; + +// Represents civil time (or occasionally physical time). +// +// This type can represent a civil time in one of a few possible ways: +// +// * When utc_offset is set and time_zone is unset: a civil time on a calendar +// day with a particular offset from UTC. +// * When time_zone is set and utc_offset is unset: a civil time on a calendar +// day in a particular time zone. +// * When neither time_zone nor utc_offset is set: a civil time on a calendar +// day in local time. +// +// The date is relative to the Proleptic Gregorian Calendar. +// +// If year is 0, the DateTime is considered not to have a specific year. month +// and day must have valid, non-zero values. +// +// This type may also be used to represent a physical time if all the date and +// time fields are set and either case of the `time_offset` oneof is set. +// Consider using `Timestamp` message for physical time instead. If your use +// case also would like to store the user's timezone, that can be done in +// another field. +// +// This type is more flexible than some applications may want. Make sure to +// document and validate your application's limitations. +message DateTime { + // Optional. Year of date. Must be from 1 to 9999, or 0 if specifying a + // datetime without a year. + int32 year = 1; + + // Required. Month of year. Must be from 1 to 12. + int32 month = 2; + + // Required. Day of month. Must be from 1 to 31 and valid for the year and + // month. + int32 day = 3; + + // Required. Hours of day in 24 hour format. Should be from 0 to 23. An API + // may choose to allow the value "24:00:00" for scenarios like business + // closing time. + int32 hours = 4; + + // Required. Minutes of hour of day. Must be from 0 to 59. + int32 minutes = 5; + + // Required. Seconds of minutes of the time. Must normally be from 0 to 59. An + // API may allow the value 60 if it allows leap-seconds. + int32 seconds = 6; + + // Required. Fractions of seconds in nanoseconds. Must be from 0 to + // 999,999,999. + int32 nanos = 7; + + // Optional. Specifies either the UTC offset or the time zone of the DateTime. + // Choose carefully between them, considering that time zone data may change + // in the future (for example, a country modifies their DST start/end dates, + // and future DateTimes in the affected range had already been stored). + // If omitted, the DateTime is considered to be in local time. + oneof time_offset { + // UTC offset. Must be whole seconds, between -18 hours and +18 hours. + // For example, a UTC offset of -4:00 would be represented as + // { seconds: -14400 }. + google.protobuf.Duration utc_offset = 8; + + // Time zone. + TimeZone time_zone = 9; + } +} + +// Represents a time zone from the +// [IANA Time Zone Database](https://www.iana.org/time-zones). +message TimeZone { + // IANA Time Zone Database time zone, e.g. "America/New_York". + string id = 1; + + // Optional. IANA Time Zone Database version number, e.g. "2019a". + string version = 2; +} diff --git a/src/go/rpk/pkg/serde/embed/protobuf/google/type/dayofweek.proto b/src/go/rpk/pkg/serde/embed/protobuf/google/type/dayofweek.proto new file mode 100644 index 000000000000..4c80c62ec0b4 --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/protobuf/google/type/dayofweek.proto @@ -0,0 +1,50 @@ +// Copyright 2021 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto3"; + +package google.type; + +option go_package = "google.golang.org/genproto/googleapis/type/dayofweek;dayofweek"; +option java_multiple_files = true; +option java_outer_classname = "DayOfWeekProto"; +option java_package = "com.google.type"; +option objc_class_prefix = "GTP"; + +// Represents a day of the week. +enum DayOfWeek { + // The day of the week is unspecified. + DAY_OF_WEEK_UNSPECIFIED = 0; + + // Monday + MONDAY = 1; + + // Tuesday + TUESDAY = 2; + + // Wednesday + WEDNESDAY = 3; + + // Thursday + THURSDAY = 4; + + // Friday + FRIDAY = 5; + + // Saturday + SATURDAY = 6; + + // Sunday + SUNDAY = 7; +} diff --git a/src/go/rpk/pkg/serde/embed/protobuf/google/type/decimal.proto b/src/go/rpk/pkg/serde/embed/protobuf/google/type/decimal.proto new file mode 100644 index 000000000000..beb18a5d8dd2 --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/protobuf/google/type/decimal.proto @@ -0,0 +1,95 @@ +// Copyright 2021 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto3"; + +package google.type; + +option cc_enable_arenas = true; +option go_package = "google.golang.org/genproto/googleapis/type/decimal;decimal"; +option java_multiple_files = true; +option java_outer_classname = "DecimalProto"; +option java_package = "com.google.type"; +option objc_class_prefix = "GTP"; + +// A representation of a decimal value, such as 2.5. Clients may convert values +// into language-native decimal formats, such as Java's [BigDecimal][] or +// Python's [decimal.Decimal][]. +// +// [BigDecimal]: +// https://docs.oracle.com/en/java/javase/11/docs/api/java.base/java/math/BigDecimal.html +// [decimal.Decimal]: https://docs.python.org/3/library/decimal.html +message Decimal { + // The decimal value, as a string. + // + // The string representation consists of an optional sign, `+` (`U+002B`) + // or `-` (`U+002D`), followed by a sequence of zero or more decimal digits + // ("the integer"), optionally followed by a fraction, optionally followed + // by an exponent. + // + // The fraction consists of a decimal point followed by zero or more decimal + // digits. The string must contain at least one digit in either the integer + // or the fraction. The number formed by the sign, the integer and the + // fraction is referred to as the significand. + // + // The exponent consists of the character `e` (`U+0065`) or `E` (`U+0045`) + // followed by one or more decimal digits. + // + // Services **should** normalize decimal values before storing them by: + // + // - Removing an explicitly-provided `+` sign (`+2.5` -> `2.5`). + // - Replacing a zero-length integer value with `0` (`.5` -> `0.5`). + // - Coercing the exponent character to lower-case (`2.5E8` -> `2.5e8`). + // - Removing an explicitly-provided zero exponent (`2.5e0` -> `2.5`). + // + // Services **may** perform additional normalization based on its own needs + // and the internal decimal implementation selected, such as shifting the + // decimal point and exponent value together (example: `2.5e-1` <-> `0.25`). + // Additionally, services **may** preserve trailing zeroes in the fraction + // to indicate increased precision, but are not required to do so. + // + // Note that only the `.` character is supported to divide the integer + // and the fraction; `,` **should not** be supported regardless of locale. + // Additionally, thousand separators **should not** be supported. If a + // service does support them, values **must** be normalized. + // + // The ENBF grammar is: + // + // DecimalString = + // [Sign] Significand [Exponent]; + // + // Sign = '+' | '-'; + // + // Significand = + // Digits ['.'] [Digits] | [Digits] '.' Digits; + // + // Exponent = ('e' | 'E') [Sign] Digits; + // + // Digits = { '0' | '1' | '2' | '3' | '4' | '5' | '6' | '7' | '8' | '9' }; + // + // Services **should** clearly document the range of supported values, the + // maximum supported precision (total number of digits), and, if applicable, + // the scale (number of digits after the decimal point), as well as how it + // behaves when receiving out-of-bounds values. + // + // Services **may** choose to accept values passed as input even when the + // value has a higher precision or scale than the service supports, and + // **should** round the value to fit the supported scale. Alternatively, the + // service **may** error with `400 Bad Request` (`INVALID_ARGUMENT` in gRPC) + // if precision would be lost. + // + // Services **should** error with `400 Bad Request` (`INVALID_ARGUMENT` in + // gRPC) if the service receives a value outside of the supported range. + string value = 1; +} diff --git a/src/go/rpk/pkg/serde/embed/protobuf/google/type/expr.proto b/src/go/rpk/pkg/serde/embed/protobuf/google/type/expr.proto new file mode 100644 index 000000000000..af0778cf958c --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/protobuf/google/type/expr.proto @@ -0,0 +1,73 @@ +// Copyright 2021 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto3"; + +package google.type; + +option go_package = "google.golang.org/genproto/googleapis/type/expr;expr"; +option java_multiple_files = true; +option java_outer_classname = "ExprProto"; +option java_package = "com.google.type"; +option objc_class_prefix = "GTP"; + +// Represents a textual expression in the Common Expression Language (CEL) +// syntax. CEL is a C-like expression language. The syntax and semantics of CEL +// are documented at https://github.com/google/cel-spec. +// +// Example (Comparison): +// +// title: "Summary size limit" +// description: "Determines if a summary is less than 100 chars" +// expression: "document.summary.size() < 100" +// +// Example (Equality): +// +// title: "Requestor is owner" +// description: "Determines if requestor is the document owner" +// expression: "document.owner == request.auth.claims.email" +// +// Example (Logic): +// +// title: "Public documents" +// description: "Determine whether the document should be publicly visible" +// expression: "document.type != 'private' && document.type != 'internal'" +// +// Example (Data Manipulation): +// +// title: "Notification string" +// description: "Create a notification string with a timestamp." +// expression: "'New message received at ' + string(document.create_time)" +// +// The exact variables and functions that may be referenced within an expression +// are determined by the service that evaluates it. See the service +// documentation for additional information. +message Expr { + // Textual representation of an expression in Common Expression Language + // syntax. + string expression = 1; + + // Optional. Title for the expression, i.e. a short string describing + // its purpose. This can be used e.g. in UIs which allow to enter the + // expression. + string title = 2; + + // Optional. Description of the expression. This is a longer text which + // describes the expression, e.g. when hovered over it in a UI. + string description = 3; + + // Optional. String indicating the location of the expression for error + // reporting, e.g. a file name and a position in the file. + string location = 4; +} diff --git a/src/go/rpk/pkg/serde/embed/protobuf/google/type/fraction.proto b/src/go/rpk/pkg/serde/embed/protobuf/google/type/fraction.proto new file mode 100644 index 000000000000..6c5ae6e2a25d --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/protobuf/google/type/fraction.proto @@ -0,0 +1,33 @@ +// Copyright 2021 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto3"; + +package google.type; + +option go_package = "google.golang.org/genproto/googleapis/type/fraction;fraction"; +option java_multiple_files = true; +option java_outer_classname = "FractionProto"; +option java_package = "com.google.type"; +option objc_class_prefix = "GTP"; + +// Represents a fraction in terms of a numerator divided by a denominator. +message Fraction { + // The numerator in the fraction, e.g. 2 in 2/3. + int64 numerator = 1; + + // The value by which the numerator is divided, e.g. 3 in 2/3. Must be + // positive. + int64 denominator = 2; +} diff --git a/src/go/rpk/pkg/serde/embed/protobuf/google/type/interval.proto b/src/go/rpk/pkg/serde/embed/protobuf/google/type/interval.proto new file mode 100644 index 000000000000..9702324cd4e8 --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/protobuf/google/type/interval.proto @@ -0,0 +1,46 @@ +// Copyright 2021 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto3"; + +package google.type; + +import "google/protobuf/timestamp.proto"; + +option cc_enable_arenas = true; +option go_package = "google.golang.org/genproto/googleapis/type/interval;interval"; +option java_multiple_files = true; +option java_outer_classname = "IntervalProto"; +option java_package = "com.google.type"; +option objc_class_prefix = "GTP"; + +// Represents a time interval, encoded as a Timestamp start (inclusive) and a +// Timestamp end (exclusive). +// +// The start must be less than or equal to the end. +// When the start equals the end, the interval is empty (matches no time). +// When both start and end are unspecified, the interval matches any time. +message Interval { + // Optional. Inclusive start of the interval. + // + // If specified, a Timestamp matching this interval will have to be the same + // or after the start. + google.protobuf.Timestamp start_time = 1; + + // Optional. Exclusive end of the interval. + // + // If specified, a Timestamp matching this interval will have to be before the + // end. + google.protobuf.Timestamp end_time = 2; +} diff --git a/src/go/rpk/pkg/serde/embed/protobuf/google/type/latlng.proto b/src/go/rpk/pkg/serde/embed/protobuf/google/type/latlng.proto new file mode 100644 index 000000000000..9231456e328f --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/protobuf/google/type/latlng.proto @@ -0,0 +1,37 @@ +// Copyright 2021 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto3"; + +package google.type; + +option cc_enable_arenas = true; +option go_package = "google.golang.org/genproto/googleapis/type/latlng;latlng"; +option java_multiple_files = true; +option java_outer_classname = "LatLngProto"; +option java_package = "com.google.type"; +option objc_class_prefix = "GTP"; + +// An object that represents a latitude/longitude pair. This is expressed as a +// pair of doubles to represent degrees latitude and degrees longitude. Unless +// specified otherwise, this must conform to the +// WGS84 +// standard. Values must be within normalized ranges. +message LatLng { + // The latitude in degrees. It must be in the range [-90.0, +90.0]. + double latitude = 1; + + // The longitude in degrees. It must be in the range [-180.0, +180.0]. + double longitude = 2; +} diff --git a/src/go/rpk/pkg/serde/embed/protobuf/google/type/localized_text.proto b/src/go/rpk/pkg/serde/embed/protobuf/google/type/localized_text.proto new file mode 100644 index 000000000000..5c6922b8c0cd --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/protobuf/google/type/localized_text.proto @@ -0,0 +1,36 @@ +// Copyright 2021 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto3"; + +package google.type; + +option cc_enable_arenas = true; +option go_package = "google.golang.org/genproto/googleapis/type/localized_text;localized_text"; +option java_multiple_files = true; +option java_outer_classname = "LocalizedTextProto"; +option java_package = "com.google.type"; +option objc_class_prefix = "GTP"; + +// Localized variant of a text in a particular language. +message LocalizedText { + // Localized string in the language corresponding to `language_code' below. + string text = 1; + + // The text's BCP-47 language code, such as "en-US" or "sr-Latn". + // + // For more information, see + // http://www.unicode.org/reports/tr35/#Unicode_locale_identifier. + string language_code = 2; +} diff --git a/src/go/rpk/pkg/serde/embed/protobuf/google/type/money.proto b/src/go/rpk/pkg/serde/embed/protobuf/google/type/money.proto new file mode 100644 index 000000000000..98d6494e4210 --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/protobuf/google/type/money.proto @@ -0,0 +1,42 @@ +// Copyright 2021 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto3"; + +package google.type; + +option cc_enable_arenas = true; +option go_package = "google.golang.org/genproto/googleapis/type/money;money"; +option java_multiple_files = true; +option java_outer_classname = "MoneyProto"; +option java_package = "com.google.type"; +option objc_class_prefix = "GTP"; + +// Represents an amount of money with its currency type. +message Money { + // The three-letter currency code defined in ISO 4217. + string currency_code = 1; + + // The whole units of the amount. + // For example if `currencyCode` is `"USD"`, then 1 unit is one US dollar. + int64 units = 2; + + // Number of nano (10^-9) units of the amount. + // The value must be between -999,999,999 and +999,999,999 inclusive. + // If `units` is positive, `nanos` must be positive or zero. + // If `units` is zero, `nanos` can be positive, zero, or negative. + // If `units` is negative, `nanos` must be negative or zero. + // For example $-1.75 is represented as `units`=-1 and `nanos`=-750,000,000. + int32 nanos = 3; +} diff --git a/src/go/rpk/pkg/serde/embed/protobuf/google/type/month.proto b/src/go/rpk/pkg/serde/embed/protobuf/google/type/month.proto new file mode 100644 index 000000000000..99e7551b1416 --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/protobuf/google/type/month.proto @@ -0,0 +1,65 @@ +// Copyright 2021 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto3"; + +package google.type; + +option go_package = "google.golang.org/genproto/googleapis/type/month;month"; +option java_multiple_files = true; +option java_outer_classname = "MonthProto"; +option java_package = "com.google.type"; +option objc_class_prefix = "GTP"; + +// Represents a month in the Gregorian calendar. +enum Month { + // The unspecified month. + MONTH_UNSPECIFIED = 0; + + // The month of January. + JANUARY = 1; + + // The month of February. + FEBRUARY = 2; + + // The month of March. + MARCH = 3; + + // The month of April. + APRIL = 4; + + // The month of May. + MAY = 5; + + // The month of June. + JUNE = 6; + + // The month of July. + JULY = 7; + + // The month of August. + AUGUST = 8; + + // The month of September. + SEPTEMBER = 9; + + // The month of October. + OCTOBER = 10; + + // The month of November. + NOVEMBER = 11; + + // The month of December. + DECEMBER = 12; +} diff --git a/src/go/rpk/pkg/serde/embed/protobuf/google/type/phone_number.proto b/src/go/rpk/pkg/serde/embed/protobuf/google/type/phone_number.proto new file mode 100644 index 000000000000..7bbb7d873229 --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/protobuf/google/type/phone_number.proto @@ -0,0 +1,113 @@ +// Copyright 2021 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto3"; + +package google.type; + +option cc_enable_arenas = true; +option go_package = "google.golang.org/genproto/googleapis/type/phone_number;phone_number"; +option java_multiple_files = true; +option java_outer_classname = "PhoneNumberProto"; +option java_package = "com.google.type"; +option objc_class_prefix = "GTP"; + +// An object representing a phone number, suitable as an API wire format. +// +// This representation: +// +// - should not be used for locale-specific formatting of a phone number, such +// as "+1 (650) 253-0000 ext. 123" +// +// - is not designed for efficient storage +// - may not be suitable for dialing - specialized libraries (see references) +// should be used to parse the number for that purpose +// +// To do something meaningful with this number, such as format it for various +// use-cases, convert it to an `i18n.phonenumbers.PhoneNumber` object first. +// +// For instance, in Java this would be: +// +// com.google.type.PhoneNumber wireProto = +// com.google.type.PhoneNumber.newBuilder().build(); +// com.google.i18n.phonenumbers.Phonenumber.PhoneNumber phoneNumber = +// PhoneNumberUtil.getInstance().parse(wireProto.getE164Number(), "ZZ"); +// if (!wireProto.getExtension().isEmpty()) { +// phoneNumber.setExtension(wireProto.getExtension()); +// } +// +// Reference(s): +// - https://github.com/google/libphonenumber +message PhoneNumber { + // An object representing a short code, which is a phone number that is + // typically much shorter than regular phone numbers and can be used to + // address messages in MMS and SMS systems, as well as for abbreviated dialing + // (e.g. "Text 611 to see how many minutes you have remaining on your plan."). + // + // Short codes are restricted to a region and are not internationally + // dialable, which means the same short code can exist in different regions, + // with different usage and pricing, even if those regions share the same + // country calling code (e.g. US and CA). + message ShortCode { + // Required. The BCP-47 region code of the location where calls to this + // short code can be made, such as "US" and "BB". + // + // Reference(s): + // - http://www.unicode.org/reports/tr35/#unicode_region_subtag + string region_code = 1; + + // Required. The short code digits, without a leading plus ('+') or country + // calling code, e.g. "611". + string number = 2; + } + + // Required. Either a regular number, or a short code. New fields may be + // added to the oneof below in the future, so clients should ignore phone + // numbers for which none of the fields they coded against are set. + oneof kind { + // The phone number, represented as a leading plus sign ('+'), followed by a + // phone number that uses a relaxed ITU E.164 format consisting of the + // country calling code (1 to 3 digits) and the subscriber number, with no + // additional spaces or formatting, e.g.: + // - correct: "+15552220123" + // - incorrect: "+1 (555) 222-01234 x123". + // + // The ITU E.164 format limits the latter to 12 digits, but in practice not + // all countries respect that, so we relax that restriction here. + // National-only numbers are not allowed. + // + // References: + // - https://www.itu.int/rec/T-REC-E.164-201011-I + // - https://en.wikipedia.org/wiki/E.164. + // - https://en.wikipedia.org/wiki/List_of_country_calling_codes + string e164_number = 1; + + // A short code. + // + // Reference(s): + // - https://en.wikipedia.org/wiki/Short_code + ShortCode short_code = 2; + } + + // The phone number's extension. The extension is not standardized in ITU + // recommendations, except for being defined as a series of numbers with a + // maximum length of 40 digits. Other than digits, some other dialing + // characters such as ',' (indicating a wait) or '#' may be stored here. + // + // Note that no regions currently use extensions with short codes, so this + // field is normally only set in conjunction with an E.164 number. It is held + // separately from the E.164 number to allow for short code extensions in the + // future. + string extension = 3; +} diff --git a/src/go/rpk/pkg/serde/embed/protobuf/google/type/postal_address.proto b/src/go/rpk/pkg/serde/embed/protobuf/google/type/postal_address.proto new file mode 100644 index 000000000000..c57c7c31a2cc --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/protobuf/google/type/postal_address.proto @@ -0,0 +1,134 @@ +// Copyright 2021 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto3"; + +package google.type; + +option cc_enable_arenas = true; +option go_package = "google.golang.org/genproto/googleapis/type/postaladdress;postaladdress"; +option java_multiple_files = true; +option java_outer_classname = "PostalAddressProto"; +option java_package = "com.google.type"; +option objc_class_prefix = "GTP"; + +// Represents a postal address, e.g. for postal delivery or payments addresses. +// Given a postal address, a postal service can deliver items to a premise, P.O. +// Box or similar. +// It is not intended to model geographical locations (roads, towns, +// mountains). +// +// In typical usage an address would be created via user input or from importing +// existing data, depending on the type of process. +// +// Advice on address input / editing: +// - Use an i18n-ready address widget such as +// https://github.com/google/libaddressinput) +// - Users should not be presented with UI elements for input or editing of +// fields outside countries where that field is used. +// +// For more guidance on how to use this schema, please see: +// https://support.google.com/business/answer/6397478 +message PostalAddress { + // The schema revision of the `PostalAddress`. This must be set to 0, which is + // the latest revision. + // + // All new revisions **must** be backward compatible with old revisions. + int32 revision = 1; + + // Required. CLDR region code of the country/region of the address. This + // is never inferred and it is up to the user to ensure the value is + // correct. See http://cldr.unicode.org/ and + // http://www.unicode.org/cldr/charts/30/supplemental/territory_information.html + // for details. Example: "CH" for Switzerland. + string region_code = 2; + + // Optional. BCP-47 language code of the contents of this address (if + // known). This is often the UI language of the input form or is expected + // to match one of the languages used in the address' country/region, or their + // transliterated equivalents. + // This can affect formatting in certain countries, but is not critical + // to the correctness of the data and will never affect any validation or + // other non-formatting related operations. + // + // If this value is not known, it should be omitted (rather than specifying a + // possibly incorrect default). + // + // Examples: "zh-Hant", "ja", "ja-Latn", "en". + string language_code = 3; + + // Optional. Postal code of the address. Not all countries use or require + // postal codes to be present, but where they are used, they may trigger + // additional validation with other parts of the address (e.g. state/zip + // validation in the U.S.A.). + string postal_code = 4; + + // Optional. Additional, country-specific, sorting code. This is not used + // in most regions. Where it is used, the value is either a string like + // "CEDEX", optionally followed by a number (e.g. "CEDEX 7"), or just a number + // alone, representing the "sector code" (Jamaica), "delivery area indicator" + // (Malawi) or "post office indicator" (e.g. Côte d'Ivoire). + string sorting_code = 5; + + // Optional. Highest administrative subdivision which is used for postal + // addresses of a country or region. + // For example, this can be a state, a province, an oblast, or a prefecture. + // Specifically, for Spain this is the province and not the autonomous + // community (e.g. "Barcelona" and not "Catalonia"). + // Many countries don't use an administrative area in postal addresses. E.g. + // in Switzerland this should be left unpopulated. + string administrative_area = 6; + + // Optional. Generally refers to the city/town portion of the address. + // Examples: US city, IT comune, UK post town. + // In regions of the world where localities are not well defined or do not fit + // into this structure well, leave locality empty and use address_lines. + string locality = 7; + + // Optional. Sublocality of the address. + // For example, this can be neighborhoods, boroughs, districts. + string sublocality = 8; + + // Unstructured address lines describing the lower levels of an address. + // + // Because values in address_lines do not have type information and may + // sometimes contain multiple values in a single field (e.g. + // "Austin, TX"), it is important that the line order is clear. The order of + // address lines should be "envelope order" for the country/region of the + // address. In places where this can vary (e.g. Japan), address_language is + // used to make it explicit (e.g. "ja" for large-to-small ordering and + // "ja-Latn" or "en" for small-to-large). This way, the most specific line of + // an address can be selected based on the language. + // + // The minimum permitted structural representation of an address consists + // of a region_code with all remaining information placed in the + // address_lines. It would be possible to format such an address very + // approximately without geocoding, but no semantic reasoning could be + // made about any of the address components until it was at least + // partially resolved. + // + // Creating an address only containing a region_code and address_lines, and + // then geocoding is the recommended way to handle completely unstructured + // addresses (as opposed to guessing which parts of the address should be + // localities or administrative areas). + repeated string address_lines = 9; + + // Optional. The recipient at the address. + // This field may, under certain circumstances, contain multiline information. + // For example, it might contain "care of" information. + repeated string recipients = 10; + + // Optional. The name of the organization at the address. + string organization = 11; +} diff --git a/src/go/rpk/pkg/serde/embed/protobuf/google/type/quaternion.proto b/src/go/rpk/pkg/serde/embed/protobuf/google/type/quaternion.proto new file mode 100644 index 000000000000..dfb822deff16 --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/protobuf/google/type/quaternion.proto @@ -0,0 +1,94 @@ +// Copyright 2021 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto3"; + +package google.type; + +option cc_enable_arenas = true; +option go_package = "google.golang.org/genproto/googleapis/type/quaternion;quaternion"; +option java_multiple_files = true; +option java_outer_classname = "QuaternionProto"; +option java_package = "com.google.type"; +option objc_class_prefix = "GTP"; + +// A quaternion is defined as the quotient of two directed lines in a +// three-dimensional space or equivalently as the quotient of two Euclidean +// vectors (https://en.wikipedia.org/wiki/Quaternion). +// +// Quaternions are often used in calculations involving three-dimensional +// rotations (https://en.wikipedia.org/wiki/Quaternions_and_spatial_rotation), +// as they provide greater mathematical robustness by avoiding the gimbal lock +// problems that can be encountered when using Euler angles +// (https://en.wikipedia.org/wiki/Gimbal_lock). +// +// Quaternions are generally represented in this form: +// +// w + xi + yj + zk +// +// where x, y, z, and w are real numbers, and i, j, and k are three imaginary +// numbers. +// +// Our naming choice `(x, y, z, w)` comes from the desire to avoid confusion for +// those interested in the geometric properties of the quaternion in the 3D +// Cartesian space. Other texts often use alternative names or subscripts, such +// as `(a, b, c, d)`, `(1, i, j, k)`, or `(0, 1, 2, 3)`, which are perhaps +// better suited for mathematical interpretations. +// +// To avoid any confusion, as well as to maintain compatibility with a large +// number of software libraries, the quaternions represented using the protocol +// buffer below *must* follow the Hamilton convention, which defines `ij = k` +// (i.e. a right-handed algebra), and therefore: +// +// i^2 = j^2 = k^2 = ijk = −1 +// ij = −ji = k +// jk = −kj = i +// ki = −ik = j +// +// Please DO NOT use this to represent quaternions that follow the JPL +// convention, or any of the other quaternion flavors out there. +// +// Definitions: +// +// - Quaternion norm (or magnitude): `sqrt(x^2 + y^2 + z^2 + w^2)`. +// - Unit (or normalized) quaternion: a quaternion whose norm is 1. +// - Pure quaternion: a quaternion whose scalar component (`w`) is 0. +// - Rotation quaternion: a unit quaternion used to represent rotation. +// - Orientation quaternion: a unit quaternion used to represent orientation. +// +// A quaternion can be normalized by dividing it by its norm. The resulting +// quaternion maintains the same direction, but has a norm of 1, i.e. it moves +// on the unit sphere. This is generally necessary for rotation and orientation +// quaternions, to avoid rounding errors: +// https://en.wikipedia.org/wiki/Rotation_formalisms_in_three_dimensions +// +// Note that `(x, y, z, w)` and `(-x, -y, -z, -w)` represent the same rotation, +// but normalization would be even more useful, e.g. for comparison purposes, if +// it would produce a unique representation. It is thus recommended that `w` be +// kept positive, which can be achieved by changing all the signs when `w` is +// negative. +// +message Quaternion { + // The x component. + double x = 1; + + // The y component. + double y = 2; + + // The z component. + double z = 3; + + // The scalar component. + double w = 4; +} diff --git a/src/go/rpk/pkg/serde/embed/protobuf/google/type/timeofday.proto b/src/go/rpk/pkg/serde/embed/protobuf/google/type/timeofday.proto new file mode 100644 index 000000000000..5cb48aa936f3 --- /dev/null +++ b/src/go/rpk/pkg/serde/embed/protobuf/google/type/timeofday.proto @@ -0,0 +1,44 @@ +// Copyright 2021 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +syntax = "proto3"; + +package google.type; + +option cc_enable_arenas = true; +option go_package = "google.golang.org/genproto/googleapis/type/timeofday;timeofday"; +option java_multiple_files = true; +option java_outer_classname = "TimeOfDayProto"; +option java_package = "com.google.type"; +option objc_class_prefix = "GTP"; + +// Represents a time of day. The date and time zone are either not significant +// or are specified elsewhere. An API may choose to allow leap seconds. Related +// types are [google.type.Date][google.type.Date] and +// `google.protobuf.Timestamp`. +message TimeOfDay { + // Hours of day in 24 hour format. Should be from 0 to 23. An API may choose + // to allow the value "24:00:00" for scenarios like business closing time. + int32 hours = 1; + + // Minutes of hour of day. Must be from 0 to 59. + int32 minutes = 2; + + // Seconds of minutes of the time. Must normally be from 0 to 59. An API may + // allow the value 60 if it allows leap-seconds. + int32 seconds = 3; + + // Fractions of seconds in nanoseconds. Must be from 0 to 999,999,999. + int32 nanos = 4; +} diff --git a/src/go/rpk/pkg/serde/proto.go b/src/go/rpk/pkg/serde/proto.go index 3604b9b20a6a..93c0f17c7829 100644 --- a/src/go/rpk/pkg/serde/proto.go +++ b/src/go/rpk/pkg/serde/proto.go @@ -16,6 +16,7 @@ import ( "github.com/bufbuild/protocompile" "github.com/bufbuild/protocompile/linker" + "github.com/redpanda-data/redpanda/src/go/rpk/pkg/serde/embed" "github.com/twmb/franz-go/pkg/sr" "google.golang.org/protobuf/encoding/protojson" "google.golang.org/protobuf/proto" @@ -154,18 +155,32 @@ func compileSchema(ctx context.Context, cl *sr.Client, schema *sr.Schema) (linke // This is the original schema. accessorMap[inMemFileName] = schema.Schema - // And we add the rest of schemas if we have references. + // Add Schema References. if len(schema.References) > 0 { err := mapReferences(ctx, cl, schema, accessorMap) if err != nil { return nil, fmt.Errorf("unable to map references: %v", err) } } - + // Redpanda includes another set of common protobuf types in + // https://github.com/redpanda-data/redpanda/tree/dev/src/v/pandaproxy/schema_registry/protobuf + // We embed it into our source accessor map to keep parity with Redpanda. + commonMap, err := embed.CommonProtoFileMap() + if err != nil { + return nil, fmt.Errorf("unable to get common protobuf types: %v", err) + } + for commonPath, commonSchema := range commonMap { + if _, exists := accessorMap[commonPath]; !exists { + accessorMap[commonPath] = commonSchema + } + } + resolver := &protocompile.SourceResolver{ + Accessor: protocompile.SourceAccessorFromMap(accessorMap), + } compiler := protocompile.Compiler{ - Resolver: &protocompile.SourceResolver{ - Accessor: protocompile.SourceAccessorFromMap(accessorMap), - }, + // And we finally add a set of well-known protobuf types: + // https://github.com/bufbuild/protocompile/blob/main/std_imports.go#L42. + Resolver: protocompile.WithStandardImports(resolver), SourceInfoMode: protocompile.SourceInfoStandard, } compiled, err := compiler.Compile(ctx, inMemFileName) diff --git a/src/go/rpk/pkg/serde/proto_test.go b/src/go/rpk/pkg/serde/proto_test.go index 2b344e901898..3b1934680d87 100644 --- a/src/go/rpk/pkg/serde/proto_test.go +++ b/src/go/rpk/pkg/serde/proto_test.go @@ -105,6 +105,95 @@ message Person { .Person.PhoneNumber phone = 2; }` + // https://github.com/redpanda-data/redpanda/blob/dev/src/v/pandaproxy/schema_registry/test/compatibility_protobuf.cc + const testWellKnownSchema = `syntax = "proto3"; +package test; +import "google/protobuf/any.proto"; +import "google/protobuf/api.proto"; +import "google/protobuf/duration.proto"; +import "google/protobuf/empty.proto"; +import "google/protobuf/field_mask.proto"; +import "google/protobuf/source_context.proto"; +import "google/protobuf/struct.proto"; +import "google/protobuf/timestamp.proto"; +import "google/protobuf/type.proto"; +import "google/protobuf/wrappers.proto"; +import "google/type/calendar_period.proto"; +import "google/type/color.proto"; +import "google/type/date.proto"; +import "google/type/datetime.proto"; +import "google/type/dayofweek.proto"; +import "google/type/decimal.proto"; +import "google/type/expr.proto"; +import "google/type/fraction.proto"; +import "google/type/interval.proto"; +import "google/type/latlng.proto"; +import "google/type/localized_text.proto"; +import "google/type/money.proto"; +import "google/type/month.proto"; +import "google/type/phone_number.proto"; +import "google/type/postal_address.proto"; +import "google/type/quaternion.proto"; +import "google/type/timeofday.proto"; +import "confluent/meta.proto"; +import "confluent/types/decimal.proto"; + +message well_known_types { + google.protobuf.Any any = 1; + google.protobuf.Api api = 2; + google.protobuf.BoolValue bool_value = 3; + google.protobuf.BytesValue bytes_value = 4; + google.protobuf.DoubleValue double_value = 5; + google.protobuf.Duration duration = 6; + google.protobuf.Empty empty = 7; + google.protobuf.Enum enum = 8; + google.protobuf.EnumValue enum_value = 9; + google.protobuf.Field field = 10; + google.protobuf.FieldMask field_mask = 11; + google.protobuf.FloatValue float_value = 12; + google.protobuf.Int32Value int32_value = 13; + google.protobuf.Int64Value int64_value = 14; + google.protobuf.ListValue list_value = 15; + google.protobuf.Method method = 16; + google.protobuf.Mixin mixin = 17; + google.protobuf.NullValue null_value = 18; + google.protobuf.Option option = 19; + google.protobuf.SourceContext source_context = 20; + google.protobuf.StringValue string_value = 21; + google.protobuf.Struct struct = 22; + google.protobuf.Syntax syntax = 23; + google.protobuf.Timestamp timestamp = 24; + google.protobuf.Type type = 25; + google.protobuf.UInt32Value uint32_value = 26; + google.protobuf.UInt64Value uint64_value = 27; + google.protobuf.Value value = 28; + google.type.CalendarPeriod calendar_period = 29; + google.type.Color color = 30; + google.type.Date date = 31; + google.type.DateTime date_time = 32; + google.type.DayOfWeek day_of_week = 33; + google.type.Decimal decimal = 34; + google.type.Expr expr = 35; + google.type.Fraction fraction = 36; + google.type.Interval interval = 37; + google.type.LatLng lat_lng = 39; + google.type.LocalizedText localized_text = 40; + google.type.Money money = 41; + google.type.Month month = 42; + google.type.PhoneNumber phone_number = 43; + google.type.PostalAddress postal_address = 44; + google.type.Quaternion quaternion = 45; + google.type.TimeOfDay time_of_day = 46; + confluent.Meta c_meta = 47; + confluent.type.Decimal c_decimal = 48; +} + +message Person { + string first_name = 1; + string last_name = 2; +} +` + tests := []struct { name string schema string @@ -224,6 +313,14 @@ message Person { record: `{"brand":"pandaPhone","year":2023}`, expRecord: `{"brand":"pandaPhone","year":2023}`, expIdx: []int{0, 1, 1}, + }, { + name: "wellknown - All Fields", + schema: testWellKnownSchema, + msgType: "test.well_known_types", + schemaID: 11, + record: messageAllWellKnown, + expRecord: messageAllWellKnown, + expIdx: []int{0}, }, } for _, tt := range tests { @@ -454,3 +551,168 @@ func protoReferenceHandler() http.HandlerFunc { } } } + +// Long message, including all well-known types baked in rpk that we can +// encode/decode. +const messageAllWellKnown = `{ + "any": { + "@type": "type.googleapis.com/test.Person", + "firstName": "foo", + "lastName": "bar" + }, + "api": { + "version": "v1", + "methods": [ + { + "name": "GetMethod", + "requestTypeUrl": "type.googleapis.com/google.protobuf.Empty", + "responseTypeUrl": "type.googleapis.com/google.protobuf.StringValue" + } + ] + }, + "boolValue": true, + "bytesValue": "bGVasG6gd11ybGQ=", + "doubleValue": 3.14159, + "duration": "3600s", + "empty": {}, + "enum": { + "name": "ZERO" + }, + "enumValue": { + "name": "MY_ENUM_VALUE", + "number": 1 + }, + "field": { + "name": "fieldName" + }, + "fieldMask": "field", + "floatValue": 1.23, + "int32Value": 42, + "int64Value": "123456789", + "listValue": [ + { "stringValue": "Item 1" }, + { "int32Value": 100 } + ], + "method": { + "name": "MethodName", + "requestTypeUrl": "type.googleapis.com/google.protobuf.StringValue", + "responseTypeUrl": "type.googleapis.com/google.protobuf.StringValue" + }, + "mixin": { + "name": "MixinName", + "root": "rootPath" + }, + "option": { + "name": "optionName", + "value": { + "@type": "type.googleapis.com/test.Person", + "firstName": "foo", + "lastName": "bar" + } + }, + "sourceContext": { + "fileName": "fileName.proto" + }, + "stringValue": "This is a string", + "struct": { + "fields": { + "field1": { "stringValue": "value1" } + } + }, + "syntax": "SYNTAX_PROTO3", + "timestamp": "2020-05-22T20:32:05Z", + "type": { + "name": "TypeName", + "fields": [ + { "name": "field1", "typeUrl": "type.googleapis.com/google.protobuf.StringValue" } + ] + }, + "uint32Value": 9876521, + "uint64Value": "9876543210", + "value": { + "stringValue": "A value example" + }, + "calendarPeriod": "DAY", + "color": { + "red": 255, + "green": 100, + "blue": 50, + "alpha": 0.8 + }, + "date": { + "year": 2024, + "month": 12, + "day": 5 + }, + "dateTime": { + "year": 2024, + "month": 12, + "day": 5, + "hours": 14, + "minutes": 30, + "seconds": 15, + "nanos": 123456789, + "utcOffset": "3600s" + }, + "dayOfWeek": "WEDNESDAY", + "decimal": { + "value": "123.456" + }, + "expr": { + "expression": "a + b", + "title": "Sample Expression" + }, + "fraction": { + "numerator": "3", + "denominator": "4" + }, + "interval": { + "startTime": "2020-05-22T20:32:05Z", + "endTime": "2023-01-01T20:32:05Z" + }, + "latLng": { + "latitude": 37.7749, + "longitude": -122.4194 + }, + "localizedText": { + "text": "Hello", + "languageCode": "en-US" + }, + "money": { + "currencyCode": "USD", + "units": "100", + "nanos": 99 + }, + "month": "DECEMBER", + "phoneNumber": { + "e164Number": "+15552220123", + "extension": "1234" + }, + "postalAddress": { + "regionCode": "US", + "postalCode": "94105", + "locality": "San Francisco", + "addressLines": ["123 Main St", "Suite 456"] + }, + "quaternion": { + "x": 1.0, + "y": 1.0, + "z": 1.0, + "w": 1.0 + }, + "timeOfDay": { + "hours": 14, + "minutes": 30, + "seconds": 1, + "nanos": 2 + }, + "cMeta": { + "doc": "v2" + }, + "cDecimal": { + "value": "AQAAAG9wI1Xh", + "precision": 10, + "scale": 2 + } +} +` diff --git a/src/v/pandaproxy/BUILD b/src/v/pandaproxy/BUILD index 65eeaa5002b2..5f523b5fc0c7 100644 --- a/src/v/pandaproxy/BUILD +++ b/src/v/pandaproxy/BUILD @@ -35,6 +35,7 @@ proto_library( "schema_registry/protobuf/google/type/quaternion.proto", "schema_registry/protobuf/google/type/timeofday.proto", ], + visibility = ["//src/go/rpk/pkg/serde:__subpackages__"], deps = [ "@protobuf//:any_proto", "@protobuf//:descriptor_proto", diff --git a/tests/rptest/clients/rpk.py b/tests/rptest/clients/rpk.py index bad49f3cf930..a57c14ccb99f 100644 --- a/tests/rptest/clients/rpk.py +++ b/tests/rptest/clients/rpk.py @@ -1624,11 +1624,7 @@ def _schema_registry_conn_settings(self): ] return flags - def _run_registry(self, - cmd, - stdin=None, - timeout=None, - output_format="json"): + def _run_registry(self, cmd, stdin=None, timeout=60, output_format="json"): cmd = [self._rpk_binary(), "registry", "--format", output_format ] + self._schema_registry_conn_settings() + cmd out = self._execute(cmd, stdin=stdin, timeout=timeout) diff --git a/tests/rptest/tests/rpk_registry_test.py b/tests/rptest/tests/rpk_registry_test.py index ce9028831ef6..bf9c7818e103 100644 --- a/tests/rptest/tests/rpk_registry_test.py +++ b/tests/rptest/tests/rpk_registry_test.py @@ -11,10 +11,12 @@ import socket import json +from ducktape.utils.util import wait_until from rptest.services.cluster import cluster from rptest.tests.redpanda_test import RedpandaTest from rptest.services.redpanda import SchemaRegistryConfig, SecurityConfig from rptest.clients.rpk import RpkTool, RpkException +from rptest.services.admin import Admin from rptest.services import tls from rptest.tests.pandaproxy_test import User, PandaProxyTLSProvider from rptest.util import expect_exception @@ -51,6 +53,7 @@ def __init__(self, ctx, schema_registry_config=SchemaRegistryConfig()): super(RpkRegistryTest, self).__init__( test_context=ctx, schema_registry_config=schema_registry_config, + node_ready_timeout_s=60, ) # SASL Config self.security = SecurityConfig() @@ -86,7 +89,6 @@ def setUp(self): self.schema_registry_config.client_crt = self.admin_user.certificate.crt self.redpanda.set_security_settings(self.security) self.redpanda.set_schema_registry_settings(self.schema_registry_config) - self.redpanda.start() self._rpk = RpkTool(self.redpanda, @@ -100,6 +102,24 @@ def setUp(self): self.admin_user.password, self.admin_user.algorithm) + # wait for users to propagate to nodes + admin = Admin(self.redpanda) + + def auth_user_propagated(): + for node in self.redpanda.nodes: + users = admin.list_users(node=node) + if self.admin_user.username not in users: + return False + return True + + wait_until(auth_user_propagated, timeout_sec=60, backoff_sec=3) + + # Wait until Schema Registry is ready. + wait_until(self._schema_topic_created, + timeout_sec=60, + backoff_sec=3, + retry_on_exc=True) + def create_schema(self, subject, schema, suffix, references=None): with tempfile.NamedTemporaryFile(suffix=suffix) as tf: tf.write(bytes(schema, 'UTF-8')) @@ -109,6 +129,11 @@ def create_schema(self, subject, schema, suffix, references=None): references=references) assert out["subject"] == subject + def _schema_topic_created(self): + # SR should create the topic the first time its accessed. + self._rpk.list_schemas() + return "_schemas" in self._rpk.list_topics() + @cluster(num_nodes=3) def test_registry_schema(self): subject_1 = "test_subject_1" @@ -361,6 +386,7 @@ def test_produce_consume_avro(self): @cluster(num_nodes=3) def test_produce_consume_proto(self): + # First we register the schemas with their references. subject_1 = "subject_for_person" proto_person = """ @@ -448,6 +474,53 @@ def test_produce_consume_proto(self): assert json.loads(msg["value"]) == expected_msg_1 assert json.loads(msg["key"]) == expected_msg_2 + # Testing the same as above but using a well-known protobuf type + subject_3 = "subject_for_well_known_types" + well_known_proto_def = """ +syntax = "proto3"; + +import "google/protobuf/timestamp.proto"; +import "google/type/month.proto"; + +message Test3 { + google.protobuf.Timestamp timestamp = 1; + google.type.Month month = 2; +} +""" + self.create_schema(subject_3, well_known_proto_def, ".proto") # ID 3 + + msg_3 = '{"timestamp":"2024-12-18T20:32:05Z","month":"DECEMBER"}' + key_3 = "somekey" + expected_msg_3 = json.loads(msg_3) + # Produce: unencoded key, encoded value: + self._rpk.produce( + test_topic, msg=msg_3, key=key_3, schema_id=3, + proto_msg="") # For single-message, it should default to it. + + # We consume as is, i.e: it will show the encoded value. + out = self._rpk.consume(test_topic, offset="2:3") + msg = json.loads(out) + + raw_bytes_string = msg["value"] + assert raw_bytes_string != expected_msg_3, f'expected to have raw bytes {raw_bytes_string} to be different than {expected_msg_3}' + assert msg["key"] == key_3, f'got key: {msg["key"]}; expected {key_3}' + + bytes_from_string = bytes( + raw_bytes_string.encode().decode('unicode-escape'), 'utf-8') + assert bytes_from_string[ + 0] == 0, f'expected encoding to contain magic byte (0)' + + # Now we decode the same message: + out = self._rpk.consume(test_topic, + offset="2:3", + use_schema_registry="value") + msg = json.loads(out) + + assert json.loads( + msg["value"] + ) == expected_msg_3, f'got: {json.loads(msg["value"])}; expected {expected_msg_3}' + assert msg["key"] == key_3, f'got key: {msg["key"]}; expected {key_3}' + @cluster(num_nodes=3) def test_produce_consume_json(self): # First we register the schemas with their references. From a4044af67842a6f48036c8095d781d98c3d2f346 Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Wed, 11 Dec 2024 22:46:24 -0500 Subject: [PATCH 142/229] `config`: disable bounded property checks with env variable For the purpose of testing certain configurations outside the bounds imposed by `bounded_property`. A new function `bounds_checking_disabled()` is adding, which checks for the presence of the environmental variable `__REDPANDA_TEST_DISABLE_BOUNDED_PROPERTY_CHECKS`. Another function, `bounded_property::clamp_with_bounds()` is added to de-duplicate checks of this value before applying `_bounds.clamp()`. If this env variable is set, `bounded_property` will neither validate nor clamp any values per the bounds set for it. This is intended for test use only and should not be used or set in a production environment. --- src/v/config/bounded_property.h | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/src/v/config/bounded_property.h b/src/v/config/bounded_property.h index e978a5f8e1db..541c85066a60 100644 --- a/src/v/config/bounded_property.h +++ b/src/v/config/bounded_property.h @@ -12,6 +12,7 @@ #include "config/base_property.h" #include "config/property.h" +#include #include namespace config { @@ -84,6 +85,10 @@ concept bounds = requires(T bounds, const typename T::underlying_t& value) { { bounds.clamp(value) } -> std::same_as; }; +inline bool bounds_checking_disabled() { + return std::getenv("__REDPANDA_TEST_DISABLE_BOUNDED_PROPERTY_CHECKS"); +} + } // namespace detail /** @@ -197,6 +202,9 @@ class bounded_property : public property { meta, def, [this](T new_value) -> std::optional { + if (detail::bounds_checking_disabled()) { + return std::nullopt; + } // Extract inner value if we are an optional<>, // and pass through into numeric_bounds::validate using outer_type = std::decay_t; @@ -235,6 +243,13 @@ class bounded_property : public property { } private: + I clamp_with_bounds(I val) { + if (detail::bounds_checking_disabled()) { + return val; + } + return _bounds.clamp(val); + } + bool clamp_and_update(T val) { using outer_type = std::decay_t; @@ -248,13 +263,13 @@ class bounded_property : public property { if constexpr (reflection::is_std_optional) { if (val.has_value()) { return property::update_value( - std::move(_bounds.clamp(val.value()))); + std::move(clamp_with_bounds(val.value()))); } else { // nullopt is always valid, never clamped. Pass it through. return property::update_value(std::move(val)); } } else { - return property::update_value(std::move(_bounds.clamp(val))); + return property::update_value(std::move(clamp_with_bounds(val))); } } From a9ce489c750c84c3075dafc63937a8244ee00399 Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Wed, 11 Dec 2024 23:25:06 -0500 Subject: [PATCH 143/229] `rptest`: add `test_disable_bounded_property_checks` --- tests/rptest/tests/cluster_config_test.py | 29 +++++++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/tests/rptest/tests/cluster_config_test.py b/tests/rptest/tests/cluster_config_test.py index c9f3f5c60c42..16aba8a3c0f6 100644 --- a/tests/rptest/tests/cluster_config_test.py +++ b/tests/rptest/tests/cluster_config_test.py @@ -1668,6 +1668,35 @@ def test_validate_cloud_storage_cache_size_config(self): lambda e: e.response.status_code == 400): self.admin.patch_cluster_config(upsert=upsert) + @cluster(num_nodes=1) + def test_disable_bounded_property_checks(self): + """ + Test that the environmental variable __REDPANDA_TEST_DISABLE_BOUNDED_PROPERTY_CHECKS + being set disables bounded property checks for cluster properties. + """ + out_of_bound_properties = { + "storage_compaction_key_map_memory": 1, + "log_segment_size": 2, + "log_segment_ms": 10 + } + + # Check that these out of bounds value updates for bounded properties are properly rejected + with expect_exception(requests.exceptions.HTTPError, + lambda e: e.response.status_code == 400): + self.redpanda.set_cluster_config(out_of_bound_properties, + expect_restart=True) + + environment = {"__REDPANDA_TEST_DISABLE_BOUNDED_PROPERTY_CHECKS": "ON"} + self.redpanda.set_environment(environment) + self.redpanda.restart_nodes(self.redpanda.nodes) + + # Expect these out of bound value updates to succeed. + # expect_restart=True due to some of the properties used. + self.redpanda.set_cluster_config(out_of_bound_properties, + expect_restart=True) + for prop, value in out_of_bound_properties.items(): + self._check_value_everywhere(prop, value) + """ PropertyAliasData: From ad8a5b0bd7dd6b2987f5ec09e6edd3fe5d60be91 Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Thu, 12 Dec 2024 13:22:58 -0800 Subject: [PATCH 144/229] datalake: fix potential skipped leadership notification Previously the following sequence seemed possible: - a replica has partition translator of term 9 - leadership is lost for term 9 - stop_translator() is called and a task is scheduled to remove the translator from the map and stop it, but doesn't run right away - the replica becomes leader for term 10 - the leadership notification runs for term 10 and does a lookup of the translators, but one is still there from term 9, so the notification is a no-op - stop_translator() task completes, removing the translator from term 9 - term 10 has no translator and never gets another notification to start a new one This commit tweaks the code to make the notifications synchronously add and remove translators from the map, rather than deferring that to background work. The actual stopping of the translators is still backgrounded, but that shouldn't be problematic. --- src/v/datalake/datalake_manager.cc | 21 +++++++++++---------- src/v/datalake/datalake_manager.h | 2 +- 2 files changed, 12 insertions(+), 11 deletions(-) diff --git a/src/v/datalake/datalake_manager.cc b/src/v/datalake/datalake_manager.cc index 8a97ff5932c6..1db9321dd210 100644 --- a/src/v/datalake/datalake_manager.cc +++ b/src/v/datalake/datalake_manager.cc @@ -126,9 +126,7 @@ ss::future<> datalake_manager::start() { = _partition_mgr->local().register_unmanage_notification( model::kafka_namespace, [this](model::topic_partition_view tp) { model::ntp ntp{model::kafka_namespace, tp.topic, tp.partition}; - ssx::spawn_with_gate(_gate, [this, ntp = std::move(ntp)] { - return stop_translator(ntp); - }); + stop_translator(ntp); }); // Handle leadership changes auto leadership_registration @@ -217,9 +215,7 @@ void datalake_manager::on_group_notification(const model::ntp& ntp) { == model::iceberg_mode::disabled; if (!partition->is_leader() || iceberg_disabled) { if (it != _translators.end()) { - ssx::spawn_with_gate(_gate, [this, partition] { - return stop_translator(partition->ntp()); - }); + stop_translator(partition->ntp()); } return; } @@ -260,14 +256,19 @@ void datalake_manager::start_translator( _translators.emplace(partition->ntp(), std::move(translator)); } -ss::future<> datalake_manager::stop_translator(const model::ntp& ntp) { +void datalake_manager::stop_translator(const model::ntp& ntp) { auto it = _translators.find(ntp); if (it == _translators.end()) { - co_return; + return; } - auto translator = std::move(it->second); + auto t = std::move(it->second); _translators.erase(it); - co_await translator->stop(); + ssx::spawn_with_gate(_gate, [t = std::move(t)]() mutable { + // Keep 't' alive by capturing it into the finally below. Use the raw + // pointer here to avoid a user-after-move. + auto* t_ptr = t.get(); + return t_ptr->stop().finally([_ = std::move(t)] {}); + }); } } // namespace datalake diff --git a/src/v/datalake/datalake_manager.h b/src/v/datalake/datalake_manager.h index 7c562f0af72e..45e8a258327a 100644 --- a/src/v/datalake/datalake_manager.h +++ b/src/v/datalake/datalake_manager.h @@ -75,7 +75,7 @@ class datalake_manager : public ss::peering_sharded_service { void on_group_notification(const model::ntp&); void start_translator( ss::lw_shared_ptr, model::iceberg_mode); - ss::future<> stop_translator(const model::ntp&); + void stop_translator(const model::ntp&); model::node_id _self; ss::sharded* _group_mgr; From 44354fb2dd9e61c323b520393f4d6a2ecf22f230 Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Thu, 12 Dec 2024 16:07:24 -0800 Subject: [PATCH 145/229] datalake: opt out of stop_translator when stopping Previously stop_translator() seemed to allow erased translators to go un-stopped if the datalake_manager's gate is closed. This commit adds an early return to avoid this. It's not clear that this is actually problematic in practice because the notification deregistration and gate close should happen atomically in stop(), so it seems like notifications _should_ be guaranteed to run without thinking about the gate. But it's an easy enough check to add in. --- src/v/datalake/datalake_manager.cc | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/v/datalake/datalake_manager.cc b/src/v/datalake/datalake_manager.cc index 1db9321dd210..cccb79c6eef1 100644 --- a/src/v/datalake/datalake_manager.cc +++ b/src/v/datalake/datalake_manager.cc @@ -257,6 +257,10 @@ void datalake_manager::start_translator( } void datalake_manager::stop_translator(const model::ntp& ntp) { + if (_gate.is_closed()) { + // Cleanup should be deferred to stop(). + return; + } auto it = _translators.find(ntp); if (it == _translators.end()) { return; From 1db9eeec588c10e994d5342a3530b649bb5ae48e Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Wed, 11 Dec 2024 14:56:33 -0800 Subject: [PATCH 146/229] dl/coordinator: add logging of applied STM update Adds logging of the STM updates for the coordinator when applying. It's otherwise very difficult to infer what the current state tracked by the coordinator is. Sample outputs for adding entries: DEBUG 2024-12-12 20:36:50,031 [shard 0:main] datalake - [{kafka/node_1/0} (datalake_coordinator_stm.snapshot)] - state_machine.cc:113 - Applying update_key::add_files from offset 44: {tp: {test_topic_0/0}, revision: 1, entries: [{start_offset: 1270, last_offset: 1279, files: []}, {start_offset: 1280, last_offset: 1289, files: []}, {start_offset: 1290, last_offse t: 1299, files: []}, {start_offset: 1300, last_offset: 1309, files: []}, {start_offset: 1310, last_offset: 1319, files: []}, {start_offset: 1320, last_offset: 1329, files: []}] (6 entries)} DEBUG 2024-12-12 20:36:50,171 [shard 0:main] datalake - [{kafka/node_2/0} (datalake_coordinator_stm.snapshot)] - state_machine.cc:113 - Applying update_key::add_files from offset 63: {tp: {test_topic_0/0}, revision: 1, entries: [{start_offset: 1820, last_offset: 1829, files: []}, {start_offset: 1830, last_offset: 1839, files: []}, {start_offset: 1840, last_offset: 1849, files: []}...{start_offset: 1880, last_offset: 1889, files: []}, {start_offset: 1890, last_offset: 1899, files: []}, {start_offset: 1900, last_offset: 1909, files: []}] (9 entries)} NOTE: the above snippet is from a test that added empty files lists NOTE: we log at most 6 entries to avoid oversized log messages Sample outputs for committing offsets: DEBUG 2024-12-12 20:36:50,314 [shard 0:main] datalake - [{kafka/node_0/0} (datalake_coordinator_stm.snapshot)] - state_machine.cc:121 - Applying update_key::mark_files_committed from offset 71: {tp: {test_topic_0/0}, revision: 1, new_committed: 1999} --- src/v/datalake/coordinator/state_machine.cc | 6 ++--- src/v/datalake/coordinator/state_update.cc | 29 ++++++++++++++++++++- src/v/datalake/coordinator/state_update.h | 6 ++++- 3 files changed, 35 insertions(+), 6 deletions(-) diff --git a/src/v/datalake/coordinator/state_machine.cc b/src/v/datalake/coordinator/state_machine.cc index 3b91dd10e871..93fd76895a8f 100644 --- a/src/v/datalake/coordinator/state_machine.cc +++ b/src/v/datalake/coordinator/state_machine.cc @@ -110,8 +110,7 @@ ss::future<> coordinator_stm::do_apply(const model::record_batch& b) { // TODO: make updates a variant so we can share code more easily? case update_key::add_files: { auto update = co_await serde::read_async(val_p); - vlog( - _log.debug, "Applying {} from offset {}: {}", key, o, update.tp); + vlog(_log.debug, "Applying {} from offset {}: {}", key, o, update); auto res = update.apply(state_, o); maybe_log_update_error(_log, key, o, res); continue; @@ -119,8 +118,7 @@ ss::future<> coordinator_stm::do_apply(const model::record_batch& b) { case update_key::mark_files_committed: { auto update = co_await serde::read_async(val_p); - vlog( - _log.debug, "Applying {} from offset {}: {}", key, o, update.tp); + vlog(_log.debug, "Applying {} from offset {}: {}", key, o, update); auto res = update.apply(state_); maybe_log_update_error(_log, key, o, res); continue; diff --git a/src/v/datalake/coordinator/state_update.cc b/src/v/datalake/coordinator/state_update.cc index 8c0a4e99efcd..34b7558288bd 100644 --- a/src/v/datalake/coordinator/state_update.cc +++ b/src/v/datalake/coordinator/state_update.cc @@ -284,7 +284,34 @@ topic_lifecycle_update::apply(topics_state& state) { return true; } -std::ostream& operator<<(std::ostream& o, topic_lifecycle_update u) { +std::ostream& operator<<(std::ostream& o, const add_files_update& u) { + fmt::print(o, "{{tp: {}, revision: {}, entries: [", u.tp, u.topic_revision); + static constexpr size_t max_to_log = 6; + static constexpr size_t halved = max_to_log / 2; + const auto& e = u.entries; + if (e.size() <= max_to_log) { + fmt::print(o, "{}", fmt::join(e, ", ")); + } else { + fmt::print(o, "{}", fmt::join(e.begin(), e.begin() + halved, ", ")); + o << "..."; + fmt::print(o, "{}", fmt::join(e.end() - halved, e.end(), ", ")); + } + fmt::print(o, "] ({} entries)}}", e.size()); + return o; +} + +std::ostream& +operator<<(std::ostream& o, const mark_files_committed_update& u) { + fmt::print( + o, + "{{tp: {}, revision: {}, new_committed: {}}}", + u.tp, + u.topic_revision, + u.new_committed); + return o; +} + +std::ostream& operator<<(std::ostream& o, const topic_lifecycle_update& u) { fmt::print( o, "{{topic: {}, revision: {}, new_state: {}}}", diff --git a/src/v/datalake/coordinator/state_update.h b/src/v/datalake/coordinator/state_update.h index fbae9a5f52a5..cfa77ed05816 100644 --- a/src/v/datalake/coordinator/state_update.h +++ b/src/v/datalake/coordinator/state_update.h @@ -45,6 +45,7 @@ struct add_files_update checked can_apply(const topics_state&); checked apply(topics_state&, model::offset); + friend std::ostream& operator<<(std::ostream&, const add_files_update&); model::topic_partition tp; model::revision_id topic_revision; @@ -68,6 +69,8 @@ struct mark_files_committed_update checked can_apply(const topics_state&); checked apply(topics_state&); + friend std::ostream& + operator<<(std::ostream&, const mark_files_committed_update&); model::topic_partition tp; model::revision_id topic_revision; @@ -90,7 +93,8 @@ struct topic_lifecycle_update checked can_apply(const topics_state&); checked apply(topics_state&); - friend std::ostream& operator<<(std::ostream&, topic_lifecycle_update); + friend std::ostream& + operator<<(std::ostream&, const topic_lifecycle_update&); model::topic topic; model::revision_id revision; From 1ad12700ba0dafca534d7e0f57e534851f47728c Mon Sep 17 00:00:00 2001 From: Andrew Wong Date: Thu, 12 Dec 2024 13:12:46 -0800 Subject: [PATCH 147/229] dl/coordinator: add more logging for file committer The file committer makes many decisions about what to commit to the table and what to replicate. This commit adds a few log messages when decisions are made. --- .../coordinator/iceberg_file_committer.cc | 40 +++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/src/v/datalake/coordinator/iceberg_file_committer.cc b/src/v/datalake/coordinator/iceberg_file_committer.cc index 3d1835eda165..71c154586752 100644 --- a/src/v/datalake/coordinator/iceberg_file_committer.cc +++ b/src/v/datalake/coordinator/iceberg_file_committer.cc @@ -99,10 +99,12 @@ ss::future< checked, file_committer::errc>> iceberg_file_committer::commit_topic_files_to_catalog( model::topic topic, const topics_state& state) const { + vlog(datalake_log.debug, "Beginning commit for topic {}", topic); auto tp_it = state.topic_to_state.find(topic); if ( tp_it == state.topic_to_state.end() || !tp_it->second.has_pending_entries()) { + vlog(datalake_log.debug, "Topic {} has no pending entries", topic); co_return chunked_vector{}; } auto topic_revision = tp_it->second.revision; @@ -110,6 +112,11 @@ iceberg_file_committer::commit_topic_files_to_catalog( auto table_id = table_id_for_topic(topic); auto table_res = co_await load_table(table_id); if (table_res.has_error()) { + vlog( + datalake_log.warn, + "Error loading table {} for committing from topic {}", + table_id, + topic); co_return table_res.error(); } auto& table = table_res.value(); @@ -130,6 +137,13 @@ iceberg_file_committer::commit_topic_files_to_catalog( if ( tp_it == state.topic_to_state.end() || tp_it->second.revision != topic_revision) { + vlog( + datalake_log.debug, + "Commit returning early, topic {} state is missing or revision has " + "changed: current {} vs expected {}", + topic, + tp_it->second.revision, + topic_revision); co_return chunked_vector{}; } @@ -148,6 +162,16 @@ iceberg_file_committer::commit_topic_files_to_catalog( // replicate the fact that it was committed previously, but // don't construct a data_file to send to Iceberg as it is // already committed. + vlog( + datalake_log.debug, + "Skipping entry for topic {} revision {} added at " + "coordinator offset {} because table {} has data including " + "coordinator offset {}", + topic, + topic_revision, + e.added_pending_at, + table_id, + *iceberg_commit_meta_opt); continue; } new_committed_offset = std::max( @@ -168,6 +192,11 @@ iceberg_file_committer::commit_topic_files_to_catalog( } } if (pending_commits.empty()) { + vlog( + datalake_log.debug, + "No new data to mark committed for topic {} revision {}", + topic, + topic_revision); co_return chunked_vector{}; } chunked_vector updates; @@ -187,6 +216,12 @@ iceberg_file_committer::commit_topic_files_to_catalog( } if (icb_files.empty()) { // All files are deduplicated. + vlog( + datalake_log.debug, + "All committed files were deduplicated for topic {} revision {}, " + "returning without updating Iceberg catalog", + topic, + topic_revision); co_return updates; } vassert( @@ -195,6 +230,11 @@ iceberg_file_committer::commit_topic_files_to_catalog( const auto commit_meta = commit_offset_metadata{ .offset = *new_committed_offset, }; + vlog( + datalake_log.debug, + "Adding {} files to Iceberg table {}", + icb_files.size(), + table_id); iceberg::transaction txn(std::move(table)); auto icb_append_res = co_await txn.merge_append( io_, From b90c9f0f8d739e2c39c13480aa8f6c382ca7b222 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Thu, 12 Dec 2024 13:43:09 +0000 Subject: [PATCH 148/229] pandaproxy: merge internal and public metrics --- src/v/pandaproxy/probe.cc | 162 ++++++++++++++++++++++---------------- src/v/pandaproxy/probe.h | 1 - 2 files changed, 93 insertions(+), 70 deletions(-) diff --git a/src/v/pandaproxy/probe.cc b/src/v/pandaproxy/probe.cc index a5dd7bb237d3..c73d027eb3d5 100644 --- a/src/v/pandaproxy/probe.cc +++ b/src/v/pandaproxy/probe.cc @@ -16,6 +16,7 @@ #include #include +#include namespace pandaproxy { @@ -26,82 +27,105 @@ probe::probe( , _group_name(group_name) , _metrics() { setup_metrics(); - setup_public_metrics(); } void probe::setup_metrics() { namespace sm = ss::metrics; - if (config::shard_local_cfg().disable_metrics()) { - return; + using is_internal = ss::bool_class; + + struct Labels { + sm::label_instance label; + std::vector agg; + sm::label status; + }; + const auto make_labels = [this](const is_internal internal) -> Labels { + const auto make_label = + [](const ss::sstring& key, const is_internal internal) { + return internal ? sm::label(key) + : metrics::make_namespaced_label(key); + }; + const auto operation_label = make_label("operation", internal); + const auto agg = internal ? std::vector{sm::shard_label} + : std::vector{ + sm::shard_label, operation_label}; + const auto status = make_label("status", internal); + return { + .label = operation_label(_path.operations.nickname), + .agg = agg, + .status = status}; + }; + + const auto internal_labels = make_labels(is_internal::yes); + const auto public_labels = make_labels(is_internal::no); + + const auto make_internal_request_latency = [this](const Labels& l) { + return sm::make_histogram( + "request_latency", + sm::description("Request latency"), + {l.label}, + [this] { + return _request_metrics.hist().internal_histogram_logform(); + }); + }; + + const auto make_public_request_latency = [this](const Labels& l) { + return sm::make_histogram( + "request_latency_seconds", + sm::description( + ssx::sformat("Internal latency of request for {}", _group_name)), + {l.label}, + [this] { + return _request_metrics.hist().public_histogram_logform(); + }); + }; + + const auto make_request_errors_total_5xx = [this](const Labels& l) { + return sm::make_counter( + "request_errors_total", + [this] { return _request_metrics._5xx_count; }, + sm::description( + ssx::sformat("Total number of {} server errors", _group_name)), + {l.label, l.status("5xx")}); + }; + + const auto make_request_errors_total_4xx = [this](const Labels& l) { + return sm::make_counter( + "request_errors_total", + [this] { return _request_metrics._4xx_count; }, + sm::description( + ssx::sformat("Total number of {} client errors", _group_name)), + {l.label, l.status("4xx")}); + }; + + const auto make_request_errors_total_3xx = [this](const Labels& l) { + return sm::make_counter( + "request_errors_total", + [this] { return _request_metrics._3xx_count; }, + sm::description( + ssx::sformat("Total number of {} redirection errors", _group_name)), + {l.label, l.status("3xx")}); + }; + + if (!config::shard_local_cfg().disable_metrics()) { + _metrics.add_group( + "pandaproxy", + {make_internal_request_latency(internal_labels)}, + {}, + internal_labels.agg); } - - auto operation_label = sm::label("operation"); - std::vector labels{ - operation_label(_path.operations.nickname)}; - - _metrics.add_group( - "pandaproxy", - {sm::make_histogram( - "request_latency", - sm::description("Request latency"), - labels, - [this] { - return _request_metrics.hist().internal_histogram_logform(); - })}, - {}, - {sm::shard_label}); -} - -void probe::setup_public_metrics() { - namespace sm = ss::metrics; - - if (config::shard_local_cfg().disable_public_metrics()) { - return; + if (!config::shard_local_cfg().disable_public_metrics()) { + _public_metrics.add_group( + _group_name, + {make_public_request_latency(public_labels) + .aggregate(public_labels.agg), + make_request_errors_total_5xx(public_labels) + .aggregate(public_labels.agg), + make_request_errors_total_4xx(public_labels) + .aggregate(public_labels.agg), + make_request_errors_total_3xx(public_labels) + .aggregate(public_labels.agg)}); } - - auto operation_label = metrics::make_namespaced_label("operation"); - auto status_label = metrics::make_namespaced_label("status"); - - std::vector labels{ - operation_label(_path.operations.nickname)}; - - auto aggregate_labels = std::vector{ - sm::shard_label, operation_label}; - - _public_metrics.add_group( - _group_name, - {sm::make_histogram( - "request_latency_seconds", - sm::description( - ssx::sformat("Internal latency of request for {}", _group_name)), - labels, - [this] { return _request_metrics.hist().public_histogram_logform(); }) - .aggregate(aggregate_labels), - - sm::make_counter( - "request_errors_total", - [this] { return _request_metrics._5xx_count; }, - sm::description( - ssx::sformat("Total number of {} server errors", _group_name)), - {operation_label(_path.operations.nickname), status_label("5xx")}) - .aggregate(aggregate_labels), - - sm::make_counter( - "request_errors_total", - [this] { return _request_metrics._4xx_count; }, - sm::description( - ssx::sformat("Total number of {} client errors", _group_name)), - {operation_label(_path.operations.nickname), status_label("4xx")}) - .aggregate(aggregate_labels), - - sm::make_counter( - "request_errors_total", - [this] { return _request_metrics._3xx_count; }, - sm::description( - ssx::sformat("Total number of {} redirection errors", _group_name)), - {operation_label(_path.operations.nickname), status_label("3xx")}) - .aggregate(aggregate_labels)}); } } // namespace pandaproxy diff --git a/src/v/pandaproxy/probe.h b/src/v/pandaproxy/probe.h index 70270b2b6708..9074dbccb4e8 100644 --- a/src/v/pandaproxy/probe.h +++ b/src/v/pandaproxy/probe.h @@ -72,7 +72,6 @@ class probe { private: void setup_metrics(); - void setup_public_metrics(); private: http_status_metric _request_metrics; From 939face46b0edc0824be83b5fa7ace01b17fe664 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Thu, 12 Dec 2024 14:45:12 +0000 Subject: [PATCH 149/229] pandaproxy: add request_errors_total metric to interal metrics --- src/v/pandaproxy/probe.cc | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/v/pandaproxy/probe.cc b/src/v/pandaproxy/probe.cc index c73d027eb3d5..43f146198f93 100644 --- a/src/v/pandaproxy/probe.cc +++ b/src/v/pandaproxy/probe.cc @@ -110,7 +110,10 @@ void probe::setup_metrics() { if (!config::shard_local_cfg().disable_metrics()) { _metrics.add_group( "pandaproxy", - {make_internal_request_latency(internal_labels)}, + {make_internal_request_latency(internal_labels), + make_request_errors_total_5xx(internal_labels), + make_request_errors_total_4xx(internal_labels), + make_request_errors_total_3xx(internal_labels)}, {}, internal_labels.agg); } From 843b549eea7426468cfe92ff2a6350aa3439a7af Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Fri, 13 Dec 2024 14:18:04 +0100 Subject: [PATCH 150/229] admin: show leader id in /v1/cluster/partitions response --- src/v/redpanda/admin/server.cc | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/v/redpanda/admin/server.cc b/src/v/redpanda/admin/server.cc index 16dd2b0c0ba2..bb1b18b5336a 100644 --- a/src/v/redpanda/admin/server.cc +++ b/src/v/redpanda/admin/server.cc @@ -3345,6 +3345,7 @@ struct cluster_partition_info { ss::lw_shared_ptr ns_tp; model::partition_id id; std::vector replicas; + std::optional leader_id; bool disabled = false; ss::httpd::cluster_json::cluster_partition to_json() const { @@ -3358,6 +3359,9 @@ struct cluster_partition_info { a.core = r.shard; ret.replicas.push(a); } + if (leader_id) { + ret.leader_id = leader_id.value(); + } ret.disabled = disabled; return ret; } @@ -3371,6 +3375,7 @@ using cluster_partitions_t cluster_partitions_t topic2cluster_partitions( model::topic_namespace ns_tp, const cluster::assignments_set& assignments, + const cluster::metadata_cache& md_cache, const cluster::topic_disabled_partitions_set* disabled_set, std::optional disabled_filter) { cluster_partitions_t ret; @@ -3412,6 +3417,7 @@ cluster_partitions_t topic2cluster_partitions( .ns_tp = shared_ns_tp, .id = id, .replicas = as_it->second.replicas, + .leader_id = md_cache.get_leader_id(*shared_ns_tp, id), .disabled = true, }); } @@ -3429,6 +3435,7 @@ cluster_partitions_t topic2cluster_partitions( .ns_tp = shared_ns_tp, .id = p_as.id, .replicas = p_as.replicas, + .leader_id = md_cache.get_leader_id(*shared_ns_tp, p_as.id), .disabled = disabled, }); } @@ -3534,6 +3541,7 @@ admin_server::get_cluster_partitions_handler( auto topic_partitions = topic2cluster_partitions( ns_tp, topic_it->second.get_assignments(), + _metadata_cache.local(), topics_state.get_topic_disabled_set(ns_tp), disabled_filter); @@ -3577,6 +3585,7 @@ admin_server::get_cluster_partitions_topic_handler( auto partitions = topic2cluster_partitions( ns_tp, topic_it->second.get_assignments(), + _metadata_cache.local(), topics_state.get_topic_disabled_set(ns_tp), disabled_filter); From 0f982748658f045002d74c53c8cc654094267c7a Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Thu, 12 Dec 2024 18:31:00 -0500 Subject: [PATCH 151/229] `rptest`: bump `kgo-verifier` version --- tests/docker/ducktape-deps/kgo-verifier | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/docker/ducktape-deps/kgo-verifier b/tests/docker/ducktape-deps/kgo-verifier index 75b4be9f8d29..80379af3f692 100644 --- a/tests/docker/ducktape-deps/kgo-verifier +++ b/tests/docker/ducktape-deps/kgo-verifier @@ -2,6 +2,6 @@ set -e git -C /opt clone https://github.com/redpanda-data/kgo-verifier.git cd /opt/kgo-verifier -git reset --hard bffac1f1358875ee6e91308229d908f40d5fe18e +git reset --hard 3c87c1c53a150ead7b53e6179c2d6105f53bd1cf go mod tidy make From cfa02dc3b035712cf65d4401c4529bf0d5a63929 Mon Sep 17 00:00:00 2001 From: Michael Boquard Date: Fri, 13 Dec 2024 14:49:33 -0500 Subject: [PATCH 152/229] config: Added is_restricted() method to enterprise property Signed-off-by: Michael Boquard --- src/v/config/property.h | 5 ++++ .../config/tests/enterprise_property_test.cc | 30 +++++++++++++++++++ 2 files changed, 35 insertions(+) diff --git a/src/v/config/property.h b/src/v/config/property.h index c2900d932400..a118e2c6da46 100644 --- a/src/v/config/property.h +++ b/src/v/config/property.h @@ -1091,6 +1091,11 @@ class enterprise : public P { return std::nullopt; } + /** + * @brief Checks current value of property to see if it is restricted + */ + bool is_restricted() const { return do_check_restricted(this->value()); } + private: bool do_check_restricted(const T& setting) const final { // depending on how the restriction was defined, construct an applicable diff --git a/src/v/config/tests/enterprise_property_test.cc b/src/v/config/tests/enterprise_property_test.cc index 3bda3965fba5..65ea48fcb592 100644 --- a/src/v/config/tests/enterprise_property_test.cc +++ b/src/v/config/tests/enterprise_property_test.cc @@ -109,4 +109,34 @@ TEST(EnterprisePropertyTest, TestTypeName) { EXPECT_EQ(cfg.enterprise_enum.type_name(), "string"); } +TEST(EnterprisePropertyTest, TestIsRestricted) { + test_config cfg; + cfg.enterprise_bool.set_value(false); + EXPECT_FALSE(cfg.enterprise_bool.is_restricted()); + cfg.enterprise_bool.set_value(true); + EXPECT_TRUE(cfg.enterprise_bool.is_restricted()); + + cfg.enterprise_str_enum.set_value("foo"); + EXPECT_FALSE(cfg.enterprise_str_enum.is_restricted()); + cfg.enterprise_str_enum.set_value("bar"); + EXPECT_TRUE(cfg.enterprise_str_enum.is_restricted()); + + cfg.enterprise_str_vec.set_value( + std::vector{"foo", "bar", "baz"}); + EXPECT_FALSE(cfg.enterprise_str_vec.is_restricted()); + cfg.enterprise_str_vec.set_value( + std::vector{"foo", "bar", "baz", "GSSAPI"}); + EXPECT_TRUE(cfg.enterprise_str_vec.is_restricted()); + + cfg.enterprise_opt_int.set_value(10); + EXPECT_FALSE(cfg.enterprise_opt_int.is_restricted()); + cfg.enterprise_opt_int.set_value(10000); + EXPECT_TRUE(cfg.enterprise_opt_int.is_restricted()); + + cfg.enterprise_enum.set_value(tls_version::v1_0); + EXPECT_FALSE(cfg.enterprise_enum.is_restricted()); + cfg.enterprise_enum.set_value(tls_version::v1_3); + EXPECT_TRUE(cfg.enterprise_enum.is_restricted()); +} + } // namespace config From 3a5c67fef772f43de5dfc8a378e11a0eded174af Mon Sep 17 00:00:00 2001 From: Willem Kaufmann Date: Fri, 13 Dec 2024 17:14:27 -0500 Subject: [PATCH 153/229] `rptest`: fix `random_node_operations_test` parameterization If this test is ran `with_iceberg`, we must check if the `cloud_storage_type` parameter is supported for the current environment via `supported_storage_types()`. If it is not, we skip the test. --- .../tests/random_node_operations_test.py | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/tests/rptest/tests/random_node_operations_test.py b/tests/rptest/tests/random_node_operations_test.py index 0055b08d6e47..c152396bc20a 100644 --- a/tests/rptest/tests/random_node_operations_test.py +++ b/tests/rptest/tests/random_node_operations_test.py @@ -29,6 +29,7 @@ from rptest.utils.node_operations import FailureInjectorBackgroundThread, NodeOpsExecutor, generate_random_workload from rptest.clients.offline_log_viewer import OfflineLogViewer +from rptest.tests.datalake.utils import supported_storage_types TS_LOG_ALLOW_LIST = [ re.compile( @@ -334,11 +335,18 @@ def test_node_operations(self, enable_failures, mixed_versions, # tp-workload-deletion - topic with delete cleanup policy # tp-workload-compaction - topic with compaction # tp-workload-fast - topic with fast partition movements enabled - if with_iceberg and mixed_versions: - self.should_skip = True - self.logger.info( - "Skipping test with iceberg and mixed versions as it is not supported" - ) + if with_iceberg: + if mixed_versions: + self.should_skip = True + self.logger.info( + "Skipping test with iceberg and mixed versions as it is not supported" + ) + cloud_storage_types = supported_storage_types() + if cloud_storage_type not in cloud_storage_types: + self.should_skip = True + self.logger.info( + "Skipping test with iceberg and unsupported cloud storage type" + ) def enable_fast_partition_movement(): if not with_tiered_storage: From 454ca47d9560e770c6ba3430487c49f05c3e6153 Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Fri, 13 Dec 2024 15:27:12 -0800 Subject: [PATCH 154/229] datalake/translator: ensure cleanup of log reader in all cases The reader is cleaned up in consume(). Other exit points between reader creation and consumption can potentially trigger an assert. The change moves the reader creation as close to the translation as possible. --- .../translation/partition_translator.cc | 60 +++++++++++-------- .../translation/partition_translator.h | 4 +- 2 files changed, 36 insertions(+), 28 deletions(-) diff --git a/src/v/datalake/translation/partition_translator.cc b/src/v/datalake/translation/partition_translator.cc index 5de00fa1531d..516263ec2872 100644 --- a/src/v/datalake/translation/partition_translator.cc +++ b/src/v/datalake/translation/partition_translator.cc @@ -210,13 +210,13 @@ partition_translator::max_offset_for_translation() const { ss::future> partition_translator::do_translation_for_range( retry_chain_node& parent, - model::record_batch_reader rdr, - kafka::offset begin_offset) { + kafka::offset read_begin_offset, + kafka::offset read_end_offset) { // This configuration only writes a single row group per file but we limit // the bytes via the reader max_bytes. auto writer_factory = std::make_unique( - local_path{_writer_scratch_space}, // storage temp files are written to - fmt::format("{}", begin_offset), // file prefix + local_path{_writer_scratch_space}, // storage temp files are written to + fmt::format("{}", read_begin_offset), // file prefix ss::make_shared()); auto task = translation_task{ @@ -232,11 +232,36 @@ partition_translator::do_translation_for_range( return can_continue() ? std::nullopt : std::make_optional("translator stopping"); }}; + + vlog( + _logger.debug, + "translating data in kafka range: [{}, {}]", + read_begin_offset, + read_end_offset); + + auto log_reader = co_await _partition_proxy->make_reader( + {kafka::offset_cast(read_begin_offset), + kafka::offset_cast(read_end_offset), + 0, + _max_bytes_per_reader, + datalake_priority(), + std::nullopt, + std::nullopt, + _as}); + auto tracker = kafka::aborted_transaction_tracker::create_default( + _partition_proxy.get(), std::move(log_reader.ot_state)); + auto kafka_reader + = model::make_record_batch_reader( + std::move(tracker), std::move(log_reader.reader)); + // Be wary of introducing abortable code here that can skip cleanup + // of kafka_reader. The reader is cleaned up along with consumption, + // so we need to ensure that the reader is dispatched to translation + // in all cases. auto result = co_await task.translate( ntp, _partition->get_topic_revision_id(), std::move(writer_factory), - std::move(rdr), + std::move(kafka_reader), remote_path_prefix, parent, las); @@ -289,29 +314,12 @@ partition_translator::do_translate_once(retry_chain_node& parent_rcn) { // accumulate. The resulting parquet files are only performant // if there is a big chunk of data in them. Smaller parquet files // are an overhead for iceberg metadata. - auto log_reader = co_await _partition_proxy->make_reader( - {kafka::offset_cast(read_begin_offset), - kafka::offset_cast(read_end_offset), - 0, - _max_bytes_per_reader, - datalake_priority(), - std::nullopt, - std::nullopt, - _as}); - auto units = co_await ss::get_units(**_parallel_translations, 1, _as); - vlog( - _logger.debug, - "translating data in kafka range: [{}, {}]", - read_begin_offset, - read_end_offset); - auto tracker = kafka::aborted_transaction_tracker::create_default( - _partition_proxy.get(), std::move(log_reader.ot_state)); - auto kafka_reader - = model::make_record_batch_reader( - std::move(tracker), std::move(log_reader.reader)); + auto translation_result = co_await do_translation_for_range( - parent_rcn, std::move(kafka_reader), read_begin_offset); + parent_rcn, read_begin_offset, read_end_offset); + + // release units and checkpoint outside of the lock. units.return_all(); vlog(_logger.debug, "translation result: {}", translation_result); auto result = translation_success::no; diff --git a/src/v/datalake/translation/partition_translator.h b/src/v/datalake/translation/partition_translator.h index e8902b6a914e..50fd01eba141 100644 --- a/src/v/datalake/translation/partition_translator.h +++ b/src/v/datalake/translation/partition_translator.h @@ -105,8 +105,8 @@ class partition_translator { ss::future> do_translation_for_range( retry_chain_node& parent, - model::record_batch_reader, - kafka::offset begin_offset); + kafka::offset read_begin, + kafka::offset read_end); using checkpoint_result = ss::bool_class; ss::future checkpoint_translated_data( From bef32546a3ece0764c51e2eb881411c799b1427d Mon Sep 17 00:00:00 2001 From: Bharath Vissapragada Date: Thu, 12 Dec 2024 15:36:59 -0800 Subject: [PATCH 155/229] datalake/metrics: miscellaneous improvements to lag metrics This is based on our experience debugging Brandon's perf setup. - Changes the metric reporting to report lag only on leaders. This makes it easy to monitor the metric using an aggregate across all replicas without having to worry about the current leader. - Fixed a bug where lag entry was not added to serde fields, adjusted the test coverage to catch this scenario, refactored the test slightly while I'm there. --- src/v/cluster/partition_probe.cc | 36 ++++++-- src/v/cluster/partition_probe.h | 2 + src/v/datalake/coordinator/types.h | 4 +- .../tests/datalake/datalake_e2e_test.py | 89 ++++++++++++------- 4 files changed, 94 insertions(+), 37 deletions(-) diff --git a/src/v/cluster/partition_probe.cc b/src/v/cluster/partition_probe.cc index 2607ccba0675..6c33f6522104 100644 --- a/src/v/cluster/partition_probe.cc +++ b/src/v/cluster/partition_probe.cc @@ -24,6 +24,8 @@ namespace cluster { static const ss::sstring cluster_metrics_name = prometheus_sanitize::metrics_name("cluster:partition"); +static constexpr int64_t follower_iceberg_lag_metric = 0; + replicated_partition_probe::replicated_partition_probe( const partition& p) noexcept : _partition(p) { @@ -46,6 +48,16 @@ void replicated_partition_probe::setup_metrics(const model::ntp& ntp) { setup_public_metrics(ntp); } +int64_t replicated_partition_probe::iceberg_translation_offset_lag() const { + return _partition.is_leader() ? _iceberg_translation_offset_lag + : follower_iceberg_lag_metric; +} + +int64_t replicated_partition_probe::iceberg_commit_offset_lag() const { + return _partition.is_leader() ? _iceberg_commit_offset_lag + : follower_iceberg_lag_metric; +} + void replicated_partition_probe::setup_internal_metrics(const model::ntp& ntp) { namespace sm = ss::metrics; @@ -168,6 +180,11 @@ void replicated_partition_probe::setup_internal_metrics(const model::ntp& ntp) { {sm::shard_label, partition_label}); if (model::is_user_topic(_partition.ntp())) { + // Metrics are reported as follows + // -2 (default initialized state) + // -1 (iceberg disabled state) + // 0 (iceberg enabled but follower replicas) + // leader replicas _metrics.add_group( cluster_metrics_name, { @@ -175,21 +192,28 @@ void replicated_partition_probe::setup_internal_metrics(const model::ntp& ntp) { "iceberg_offsets_pending_translation", [this] { return _partition.log()->config().iceberg_enabled() - ? _iceberg_translation_offset_lag + ? iceberg_translation_offset_lag() : metric_feature_disabled_state; }, - sm::description("Total number of offsets that are pending " - "translation to iceberg."), + sm::description( + "Total number of offsets that are pending " + "translation to iceberg. Lag is reported only on leader " + "replicas while followers report 0. -1 is reported if iceberg " + "is disabled while -2 indicates the lag is " + "not yet computed."), labels), sm::make_gauge( "iceberg_offsets_pending_commit", [this] { return _partition.log()->config().iceberg_enabled() - ? _iceberg_commit_offset_lag + ? iceberg_commit_offset_lag() : metric_feature_disabled_state; }, - sm::description("Total number of offsets that are pending " - "commit to iceberg catalog."), + sm::description( + "Total number of offsets that are pending " + "commit to iceberg catalog. Lag is reported only on leader " + "while followers report 0. -1 is reported if iceberg is " + "disabled while -2 indicates the lag is not yet computed."), labels), }, {}, diff --git a/src/v/cluster/partition_probe.h b/src/v/cluster/partition_probe.h index 7d8c10aaaf6a..32acc0da9ea9 100644 --- a/src/v/cluster/partition_probe.h +++ b/src/v/cluster/partition_probe.h @@ -109,6 +109,8 @@ class replicated_partition_probe : public partition_probe::impl { void clear_metrics() final; private: + int64_t iceberg_translation_offset_lag() const; + int64_t iceberg_commit_offset_lag() const; void reconfigure_metrics(); void setup_public_metrics(const model::ntp&); void setup_internal_metrics(const model::ntp&); diff --git a/src/v/datalake/coordinator/types.h b/src/v/datalake/coordinator/types.h index ba37346bfa4d..d25c9ea70c31 100644 --- a/src/v/datalake/coordinator/types.h +++ b/src/v/datalake/coordinator/types.h @@ -186,7 +186,9 @@ struct fetch_latest_translated_offset_reply friend std::ostream& operator<<(std::ostream&, const fetch_latest_translated_offset_reply&); - auto serde_fields() { return std::tie(last_added_offset, errc); } + auto serde_fields() { + return std::tie(last_added_offset, errc, last_iceberg_committed_offset); + } }; // For a given topic/partition fetches the latest translated offset from diff --git a/tests/rptest/tests/datalake/datalake_e2e_test.py b/tests/rptest/tests/datalake/datalake_e2e_test.py index 8029166e32e3..0664e44ee0c5 100644 --- a/tests/rptest/tests/datalake/datalake_e2e_test.py +++ b/tests/rptest/tests/datalake/datalake_e2e_test.py @@ -24,11 +24,6 @@ from ducktape.utils.util import wait_until from rptest.services.metrics_check import MetricCheck -NO_SCHEMA_ERRORS = [ - r'Must have parsed schema when using structured data mode', - r'Error translating data to binary record' -] - class DatalakeE2ETests(RedpandaTest): def __init__(self, test_ctx, *args, **kwargs): @@ -190,45 +185,79 @@ def table_deleted(): dl.produce_to_topic(self.topic_name, 1024, count) dl.wait_for_translation(self.topic_name, msg_count=count) - @cluster(num_nodes=3, log_allow_list=NO_SCHEMA_ERRORS) - @matrix(cloud_storage_type=supported_storage_types()) - def test_metrics(self, cloud_storage_type): - commit_lag = 'vectorized_cluster_partition_iceberg_offsets_pending_commit' - translation_lag = 'vectorized_cluster_partition_iceberg_offsets_pending_translation' +class DatalakeMetricsTest(RedpandaTest): + + commit_lag = 'vectorized_cluster_partition_iceberg_offsets_pending_commit' + translation_lag = 'vectorized_cluster_partition_iceberg_offsets_pending_translation' + + def __init__(self, test_ctx, *args, **kwargs): + super(DatalakeMetricsTest, + self).__init__(test_ctx, + num_brokers=3, + si_settings=SISettings(test_context=test_ctx), + extra_rp_conf={ + "iceberg_enabled": "true", + "iceberg_catalog_commit_interval_ms": "5000", + "enable_leader_balancer": False + }, + schema_registry_config=SchemaRegistryConfig(), + pandaproxy_config=PandaproxyConfig(), + *args, + **kwargs) + self.test_ctx = test_ctx + self.topic_name = "test" + + def setUp(self): + pass + + def wait_for_lag(self, metric_check: MetricCheck, metric_name: str, + count: int): + wait_until( + lambda: metric_check.evaluate([(metric_name, lambda _, val: val == + count)]), + timeout_sec=30, + backoff_sec=5, + err_msg=f"Timed out waiting for {metric_name} to reach: {count}") + + @cluster(num_nodes=5) + @matrix(cloud_storage_type=supported_storage_types()) + def test_lag_metrics(self, cloud_storage_type): with DatalakeServices(self.test_ctx, redpanda=self.redpanda, filesystem_catalog_mode=False, include_query_engines=[]) as dl: - dl.create_iceberg_enabled_topic( - self.topic_name, - partitions=1, - replicas=1, - iceberg_mode="value_schema_id_prefix") + # Stop the catalog to halt the translation flow + dl.catalog_service.stop() + + dl.create_iceberg_enabled_topic(self.topic_name, + partitions=1, + replicas=3) + topic_leader = self.redpanda.partitions(self.topic_name)[0].leader count = randint(12, 21) - # Populate schemaless messages in schema-ed mode, this should - # hold up translation and commits - dl.produce_to_topic(self.topic_name, 1024, msg_count=count) + dl.produce_to_topic(self.topic_name, 1, msg_count=count) m = MetricCheck(self.redpanda.logger, self.redpanda, - self.redpanda.nodes[0], - [commit_lag, translation_lag], + topic_leader, [ + DatalakeMetricsTest.commit_lag, + DatalakeMetricsTest.translation_lag + ], labels={ 'namespace': 'kafka', 'topic': self.topic_name, 'partition': '0' }, reduce=sum) - expectations = [] - for metric in [commit_lag, translation_lag]: - expectations.append([metric, lambda _, val: val == count]) - - # Ensure lag metric builds up as expected. - wait_until( - lambda: m.evaluate(expectations), - timeout_sec=30, - backoff_sec=5, - err_msg=f"Timed out waiting for metrics to reach: {count}") + + # Wait for lag build up + self.wait_for_lag(m, DatalakeMetricsTest.translation_lag, count) + self.wait_for_lag(m, DatalakeMetricsTest.commit_lag, count) + + # Resume iceberg translation + dl.catalog_service.start() + + self.wait_for_lag(m, DatalakeMetricsTest.translation_lag, 0) + self.wait_for_lag(m, DatalakeMetricsTest.commit_lag, 0) From aba90afb53736add657cb414912c8aaf767e8592 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Thu, 12 Dec 2024 17:51:52 +0100 Subject: [PATCH 156/229] r/buffered_protocol: added a missing call to setup metrics MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Michał Maślanka --- src/v/raft/buffered_protocol.cc | 1 + 1 file changed, 1 insertion(+) diff --git a/src/v/raft/buffered_protocol.cc b/src/v/raft/buffered_protocol.cc index 44d4849345bc..c662bbacb9b2 100644 --- a/src/v/raft/buffered_protocol.cc +++ b/src/v/raft/buffered_protocol.cc @@ -253,6 +253,7 @@ append_entries_queue::append_entries_queue( } _current_max_inflight_requests = new_value; }); + setup_internal_metrics(); // start dispatch loop ssx::repeat_until_gate_closed( _gate, From 710449fd0ac3eac5847d8ddce993bee7acac2a11 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Fri, 13 Dec 2024 11:20:56 +0100 Subject: [PATCH 157/229] r/buffered_protocol: removed double colons in metric names MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Michał Maślanka --- src/v/raft/buffered_protocol.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/v/raft/buffered_protocol.cc b/src/v/raft/buffered_protocol.cc index c662bbacb9b2..60c96d2bf650 100644 --- a/src/v/raft/buffered_protocol.cc +++ b/src/v/raft/buffered_protocol.cc @@ -358,7 +358,7 @@ void append_entries_queue::setup_internal_metrics() { } sm::label_instance target_node_id_label("target_node_id", _target_node); _internal_metrics.add_group( - prometheus_sanitize::metrics_name("raft::buffered::protocol"), + prometheus_sanitize::metrics_name("raft:buffered:protocol"), {sm::make_gauge( "inflight_requests", [this] { return inflight_requests(); }, From e80fbd69f9b8103d5720266a0fd02ba856ffbfaa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Fri, 13 Dec 2024 11:21:53 +0100 Subject: [PATCH 158/229] r/buffered_protocol: removed append entries latency histogram MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit We already track RPC methods latency so the histogram can be removed. Signed-off-by: Michał Maślanka --- src/v/raft/buffered_protocol.cc | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/src/v/raft/buffered_protocol.cc b/src/v/raft/buffered_protocol.cc index 60c96d2bf650..aa1a312493ab 100644 --- a/src/v/raft/buffered_protocol.cc +++ b/src/v/raft/buffered_protocol.cc @@ -375,12 +375,7 @@ void append_entries_queue::setup_internal_metrics() { [this] { return _requests.size(); }, sm::description( "Total number of append entries requests in the queue"), - {target_node_id_label}), - sm::make_histogram( - "append_entries_request_latency", - sm::description("Latency of append entries requests"), - {target_node_id_label}, - [this] { return _hist.internal_histogram_logform(); })}); + {target_node_id_label})}); } void append_entries_queue::setup_public_metrics() { From efe96248c16a58382a81f2a5613728c5cf9fddf4 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Wed, 11 Dec 2024 20:07:35 +0000 Subject: [PATCH 159/229] pandaproxy: add max memory check for incoming requests --- src/v/pandaproxy/reply.h | 4 ++ src/v/pandaproxy/rest/proxy.cc | 2 +- src/v/pandaproxy/schema_registry/service.cc | 2 +- src/v/pandaproxy/server.cc | 6 ++ src/v/pandaproxy/server.h | 1 + tests/rptest/tests/pandaproxy_test.py | 63 +++++++++++++++++++++ 6 files changed, 76 insertions(+), 2 deletions(-) diff --git a/src/v/pandaproxy/reply.h b/src/v/pandaproxy/reply.h index a69703286cf1..d9fe375af42a 100644 --- a/src/v/pandaproxy/reply.h +++ b/src/v/pandaproxy/reply.h @@ -65,6 +65,10 @@ inline ss::http::reply& set_reply_too_many_requests(ss::http::reply& rep) { .add_header("Retry-After", "0"); } +inline ss::http::reply& set_reply_payload_too_large(ss::http::reply& rep) { + return rep.set_status(ss::http::reply::status_type::payload_too_large); +} + inline std::unique_ptr reply_unavailable() { auto rep = std::make_unique(ss::http::reply{}); set_reply_unavailable(*rep); diff --git a/src/v/pandaproxy/rest/proxy.cc b/src/v/pandaproxy/rest/proxy.cc index 811aee79bd58..911c4aef1e66 100644 --- a/src/v/pandaproxy/rest/proxy.cc +++ b/src/v/pandaproxy/rest/proxy.cc @@ -113,7 +113,7 @@ proxy::proxy( , _inflight_config_binding(config::shard_local_cfg().max_in_flight_pandaproxy_requests_per_shard.bind()) , _client(client) , _client_cache(client_cache) - , _ctx{{{{}, _mem_sem, _inflight_sem, {}, smp_sg}, *this}, + , _ctx{{{{}, max_memory, _mem_sem, _inflight_sem, {}, smp_sg}, *this}, {config::always_true(), config::shard_local_cfg().superusers.bind(), controller}, _config.pandaproxy_api.value()} , _server( diff --git a/src/v/pandaproxy/schema_registry/service.cc b/src/v/pandaproxy/schema_registry/service.cc index 9e99cf408074..77502d30c5e7 100644 --- a/src/v/pandaproxy/schema_registry/service.cc +++ b/src/v/pandaproxy/schema_registry/service.cc @@ -614,7 +614,7 @@ service::service( config::shard_local_cfg() .max_in_flight_schema_registry_requests_per_shard.bind()) , _client(client) - , _ctx{{{}, _mem_sem, _inflight_sem, {}, smp_sg}, *this} + , _ctx{{{}, max_memory, _mem_sem, _inflight_sem, {}, smp_sg}, *this} , _server( "schema_registry", // server_name "schema_registry", // public_metric_group_name diff --git a/src/v/pandaproxy/server.cc b/src/v/pandaproxy/server.cc index 22507fdc5135..ea8ab73cfa63 100644 --- a/src/v/pandaproxy/server.cc +++ b/src/v/pandaproxy/server.cc @@ -104,6 +104,12 @@ struct handler_adaptor : ss::httpd::handler_base { co_return std::move(rp.rep); } auto req_size = get_request_size(*rq.req); + if (req_size > _ctx.max_memory) { + set_reply_payload_too_large(*rp.rep); + rp.mime_type = _exceptional_mime_type; + set_and_measure_response(rp); + co_return std::move(rp.rep); + } auto sem_units = co_await ss::get_units(_ctx.mem_sem, req_size); if (_ctx.as.abort_requested()) { set_reply_unavailable(*rp.rep); diff --git a/src/v/pandaproxy/server.h b/src/v/pandaproxy/server.h index 813eb0a40f51..7bca466fc052 100644 --- a/src/v/pandaproxy/server.h +++ b/src/v/pandaproxy/server.h @@ -69,6 +69,7 @@ class server { public: struct context_t { std::vector advertised_listeners; + size_t max_memory; ssx::semaphore& mem_sem; adjustable_semaphore& inflight_sem; ss::abort_source as; diff --git a/tests/rptest/tests/pandaproxy_test.py b/tests/rptest/tests/pandaproxy_test.py index dd1895ae1ece..017a17e43a48 100644 --- a/tests/rptest/tests/pandaproxy_test.py +++ b/tests/rptest/tests/pandaproxy_test.py @@ -35,12 +35,15 @@ from rptest.services import tls from rptest.utils.utf8 import CONTROL_CHARS_MAP from typing import Optional, List, Dict, Union +from rptest.utils.mode_checks import skip_debug_mode def create_topic_names(count): return list(f"pandaproxy-topic-{uuid.uuid4()}" for _ in range(count)) +PAYLOAD_TOO_LARGE_HTTP_ERROR_CODE = 413 + HTTP_GET_BROKERS_HEADERS = { "Accept": "application/vnd.kafka.v2+json", "Content-Type": "application/vnd.kafka.v2+json" @@ -1284,6 +1287,66 @@ def test_invalid_topics_fetch(self): assert sc_res.json( )["message"] == f'Invalid parameter \'topic_name\' got \'{topic_name.translate(CONTROL_CHARS_MAP)}\'' + #Memory tracking is disabled in debug + @skip_debug_mode + @cluster(num_nodes=3) + def test_topic_produce_request_too_big(self): + """ + Create a topic and post a request larger than the total available memory. + """ + + self.redpanda.set_resource_settings( + ResourceSettings(memory_mb=256, num_cpus=1)) + self.redpanda.start() + + name = create_topic_names(1)[0] + + self.logger.info("Generating request larger than the available memory") + value = { + "value": + ("TWVzc2FnZSBTdGFydC4gVXNpbmcgYSBsb25nIHNlbnRlbmNlIHRvIGJlIGFibGUgdG8gcmVhY2ggdGhlIGF2YWlsYWJsZSB" + "tZW1vcnkgbGltaXQgd2l0aG91dCBoYXZpbmcgdG8gdXNlIHRvbyBtYW55IHJlY29yZHMuIEV2ZXJ5IHJlY29yZCBvYmplY3" + "QgaXMgOTYgYnl0ZXMgKyBoZWFwLiBJZiBhIHNtYWxsIHZhbHVlIHN0cmluZyBpcyB1c2VkIHBlciBvYmplY3QsIHdoZW4gd" + "GhpcyBqc29uIGlzIHBhcnNlZCwgdGhlIG1lbW9yeSByZXF1aXJlbWVudHMgYXJlIG11Y2ggbW9yZSB0aGFuIHRoZSByZXF1" + "ZXN0IGl0c2VsZi4gTWVzc2FnZSBFbmQuIE1lc3NhZ2UgU3RhcnQuIFVzaW5nIGEgbG9uZyBzZW50ZW5jZSB0byBiZSBhYmx" + "lIHRvIHJlYWNoIHRoZSBhdmFpbGFibGUgbWVtb3J5IGxpbWl0IHdpdGhvdXQgaGF2aW5nIHRvIHVzZSB0b28gbWFueSByZW" + "NvcmRzLiBFdmVyeSByZWNvcmQgb2JqZWN0IGlzIDk2IGJ5dGVzICsgaGVhcC4gSWYgYSBzbWFsbCB2YWx1ZSBzdHJpbmcga" + "XMgdXNlZCBwZXIgb2JqZWN0LCB3aGVuIHRoaXMganNvbiBpcyBwYXJzZWQsIHRoZSBtZW1vcnkgcmVxdWlyZW1lbnRzIGFy" + "ZSBtdWNoIG1vcmUgdGhhbiB0aGUgcmVxdWVzdCBpdHNlbGYuIE1lc3NhZ2UgRW5kLiBNZXNzYWdlIFN0YXJ0LiBVc2luZyB" + "hIGxvbmcgc2VudGVuY2UgdG8gYmUgYWJsZSB0byByZWFjaCB0aGUgYXZhaWxhYmxlIG1lbW9yeSBsaW1pdCB3aXRob3V0IG" + "hhdmluZyB0byB1c2UgdG9vIG1hbnkgcmVjb3Jkcy4gRXZlcnkgcmVjb3JkIG9iamVjdCBpcyA5NiBieXRlcyArIGhlYXAuI" + "ElmIGEgc21hbGwgdmFsdWUgc3RyaW5nIGlzIHVzZWQgcGVyIG9iamVjdCwgd2hlbiB0aGlzIGpzb24gaXMgcGFyc2VkLCB0" + "aGUgbWVtb3J5IHJlcXVpcmVtZW50cyBhcmUgbXVjaCBtb3JlIHRoYW4gdGhlIHJlcXVlc3QgaXRzZWxmLiBNZXNzYWdlIEV" + "uZC4gTWVzc2FnZSBTdGFydC4gVXNpbmcgYSBsb25nIHNlbnRlbmNlIHRvIGJlIGFibGUgdG8gcmVhY2ggdGhlIGF2YWlsYW" + "JsZSBtZW1vcnkgbGltaXQgd2l0aG91dCBoYXZpbmcgdG8gdXNlIHRvbyBtYW55IHJlY29yZHMuIEV2ZXJ5IHJlY29yZCBvY" + "mplY3QgaXMgOTYgYnl0ZXMgKyBoZWFwLiBJZiBhIHNtYWxsIHZhbHVlIHN0cmluZyBpcyB1c2VkIHBlciBvYmplY3QsIHdo" + "ZW4gdGhpcyBqc29uIGlzIHBhcnNlZCwgdGhlIG1lbW9yeSByZXF1aXJlbWVudHMgYXJlIG11Y2ggbW9yZSB0aGFuIHRoZSB" + "yZXF1ZXN0IGl0c2VsZi4gTWVzc2FnZSBFbmQuIE1lc3NhZ2UgU3RhcnQuIFVzaW5nIGEgbG9uZyBzZW50ZW5jZSB0byBiZS" + "BhYmxlIHRvIHJlYWNoIHRoZSBhdmFpbGFibGUgbWVtb3J5IGxpbWl0IHdpdGhvdXQgaGF2aW5nIHRvIHVzZSB0b28gbWFue" + "SByZWNvcmRzLiBFdmVyeSByZWNvcmQgb2JqZWN0IGlzIDk2IGJ5dGVzICsgaGVhcC4gSWYgYSBzbWFsbCB2YWx1ZSBzdHJp" + "bmcgaXMgdXNlZCBwZXIgb2JqZWN0LCB3aGVuIHRoaXMganNvbiBpcyBwYXJzZWQsIHRoZSBtZW1vcnkgcmVxdWlyZW1lbnR" + "zIGFyZSBtdWNoIG1vcmUgdGhhbiB0aGUgcmVxdWVzdCBpdHNlbGYuIE1lc3NhZ2UgRW5kLg==" + ) + } + values = [value for _ in range(50000)] + data = {"records": values} + data_json = json.dumps(data) + + # With 256Mb available per core, the available memory for the kafka services + # is 90.4Mb at most. We want to ensure that this request is larger than this + memory_limit = 90.4 * 1024 * 1024 + assert len(data_json) > memory_limit, \ + f"Expected request larger than {memory_limit}b. Got {len(data_json)}b, instead" + + self.logger.info(f"Creating test topic: {name}") + self._create_topics([name], partitions=3) + + self.logger.info(f"Producing to topic: {name}") + produce_result_raw = self._produce_topic(name, data_json) + assert produce_result_raw.status_code == PAYLOAD_TOO_LARGE_HTTP_ERROR_CODE, \ + f"Expected '{PAYLOAD_TOO_LARGE_HTTP_ERROR_CODE}' " \ + f"but got '{produce_result_raw.status_code}' instead" + class PandaProxySASLTest(PandaProxyEndpoints): """ From bea1bed4ab7cbd4327a4075cbfb3b94f413a2273 Mon Sep 17 00:00:00 2001 From: Travis Downs Date: Mon, 16 Dec 2024 10:11:51 -0300 Subject: [PATCH 160/229] fmt: remove formatter workaround for old fmt We had a workaround for fmt versions less than 9.0.1 to avoid sstring being formatted using operator<< and to use the formatter<> specialization instead, but this is not needed after version 9.0.1, and it so was protected by #if FMT_VERSION < 090100. We can just remove this case entirely on dev as we have upgraded format to 9.0.1 on both the bazel and cmake sides. We leave in place only a check that the FMT version is high enough. --- src/v/ssx/sformat.h | 37 +------------------------------------ 1 file changed, 1 insertion(+), 36 deletions(-) diff --git a/src/v/ssx/sformat.h b/src/v/ssx/sformat.h index 92adece8a545..342481666847 100644 --- a/src/v/ssx/sformat.h +++ b/src/v/ssx/sformat.h @@ -17,42 +17,7 @@ #include #if FMT_VERSION < 90100 -template<> -struct fmt::formatter final : fmt::formatter { - template - auto format(const seastar::sstring& s, FormatContext& ctx) const { - return formatter::format( - string_view(s.data(), s.size()), ctx); - } -}; - -namespace fmt::detail { - -// TODO(BenP): Remove this hack with next libfmt upgrade -// -// If a type that has both operator<< and formatter<>, the former is chosen, -// which is not what is desired. This is a regression betwen 7.0.3 and 8.1.1 -// -// This hack disables picking operator<<, enabling the above formatter. -// -// A future version of fmtlib will require explicit opt-in: -// template<> struct formatter : ostream_formatter {}; -// -// Without this hack: -// test iterations median mad min max -// std_std_fmt_1K.join 11739 75.312us 1.073us 74.156us 97.341us -// ss_ss_fmt_1K.join 5855 155.362us 925.138ns 154.437us 160.463us -// ss_ss_ssx_1K.join 6642 148.931us 1.317us 147.613us 155.382us -// -// With this hack: -// test iterations median mad min max -// std_std_fmt_1K.join 11684 71.891us 218.588ns 71.652us 72.854us -// ss_ss_fmt_1K.join 10300 86.037us 444.582ns 84.775us 88.985us -// ss_ss_ssx_1K.join 12311 72.840us 2.224us 70.616us 101.076us -template<> -struct is_streamable : std::false_type {}; - -} // namespace fmt::detail +#error fmt library version is too old, must be >= 90100 #endif namespace ssx { From 2ed2a65eeee9bbc3cea6d8ec7c66293de80086b4 Mon Sep 17 00:00:00 2001 From: Michael Boquard Date: Fri, 13 Dec 2024 14:50:09 -0500 Subject: [PATCH 161/229] kafka: Relaxed topic property enforcement when license invalid Topic properties related to tiered storage and schema ID validation will be permitted to be set/changed even when a license is not valid provided that the overriding cluster config (`cloud_storage_enabled` and `enable_schema_id_validation`) are disabled. Signed-off-by: Michael Boquard --- src/v/cluster/topics_frontend.cc | 62 +- src/v/kafka/server/tests/alter_config_test.cc | 709 +++++++++--------- .../server/tests/create_partition_test.cc | 51 ++ .../kafka/server/tests/create_topics_test.cc | 49 ++ 4 files changed, 516 insertions(+), 355 deletions(-) diff --git a/src/v/cluster/topics_frontend.cc b/src/v/cluster/topics_frontend.cc index 44ac566f0cb3..b4061d837b10 100644 --- a/src/v/cluster/topics_frontend.cc +++ b/src/v/cluster/topics_frontend.cc @@ -74,18 +74,30 @@ std::vector get_enterprise_features(const cluster::topic_configuration& cfg) { std::vector features; const auto si_disabled = model::shadow_indexing_mode::disabled; - if (cfg.properties.shadow_indexing.value_or(si_disabled) != si_disabled) { - features.emplace_back("tiered storage"); - } - if (cfg.is_recovery_enabled()) { - features.emplace_back("topic recovery"); - } - if (cfg.is_read_replica()) { - features.emplace_back("remote read replicas"); + // Only enforce tiered storage topic config sanctions when cloud storage is + // enabled for the cluster + if (config::shard_local_cfg().cloud_storage_enabled.is_restricted()) { + if ( + cfg.properties.shadow_indexing.value_or(si_disabled) != si_disabled) { + features.emplace_back("tiered storage"); + } + if (cfg.is_recovery_enabled()) { + features.emplace_back("topic recovery"); + } + if (cfg.is_read_replica()) { + features.emplace_back("remote read replicas"); + } } - if (cfg.is_schema_id_validation_enabled()) { - features.emplace_back("schema ID validation"); + + // Only enforce schema ID validation topic configs if Schema ID validation + // is enabled for the cluster + if (config::shard_local_cfg().enable_schema_id_validation.is_restricted()) { + if (cfg.is_schema_id_validation_enabled()) { + features.emplace_back("schema ID validation"); + } } + + // We are always enforcing leadership preference restrictions if (const auto& leaders_pref = cfg.properties.leaders_preference; leaders_pref.has_value() && config::shard_local_cfg() @@ -111,11 +123,15 @@ std::vector get_enterprise_features( std::vector features; const auto si_disabled = model::shadow_indexing_mode::disabled; - if ( - (properties.shadow_indexing.value_or(si_disabled) - < updated_properties.shadow_indexing.value_or(si_disabled)) - || (properties.remote_delete < updated_properties.remote_delete)) { - features.emplace_back("tiered storage"); + // Only enforce tiered storage topic config sanctions when cloud storage is + // enabled for the cluster + if (config::shard_local_cfg().cloud_storage_enabled.is_restricted()) { + if ( + (properties.shadow_indexing.value_or(si_disabled) + < updated_properties.shadow_indexing.value_or(si_disabled)) + || (properties.remote_delete < updated_properties.remote_delete)) { + features.emplace_back("tiered storage"); + } } static constexpr auto key_schema_id_validation_enabled = @@ -163,12 +179,16 @@ std::vector get_enterprise_features( up.record_value_subject_name_strategy_compat)); }; - if ( - ((key_schema_id_validation_enabled(properties) - < key_schema_id_validation_enabled(updated_properties)) - || (value_schema_id_validation_enabled(properties) < value_schema_id_validation_enabled(updated_properties))) - || (schema_id_validation_enabled(updated_properties) && sns_modified())) { - features.emplace_back("schema id validation"); + // Only enforce schema ID validation topic configs if Schema ID validation + // is enabled for the cluster + if (config::shard_local_cfg().enable_schema_id_validation.is_restricted()) { + if ( + ((key_schema_id_validation_enabled(properties) + < key_schema_id_validation_enabled(updated_properties)) + || (value_schema_id_validation_enabled(properties) < value_schema_id_validation_enabled(updated_properties))) + || (schema_id_validation_enabled(updated_properties) && sns_modified())) { + features.emplace_back("schema id validation"); + } } if (const auto& updated_pref = updated_properties.leaders_preference; diff --git a/src/v/kafka/server/tests/alter_config_test.cc b/src/v/kafka/server/tests/alter_config_test.cc index 9182065020e3..0017e2a53b8d 100644 --- a/src/v/kafka/server/tests/alter_config_test.cc +++ b/src/v/kafka/server/tests/alter_config_test.cc @@ -234,6 +234,376 @@ class alter_config_test_fixture : public topic_properties_test_fixture { BOOST_CHECK_EQUAL(cfg_it->value, value); } } + + void alter_config_no_license_test(bool enable_cluster_config) { + using props_t = absl::flat_hash_map; + using alter_props_t = absl::flat_hash_map< + ss::sstring, + std:: + pair, kafka::config_resource_operation>>; + using skip_create = ss::bool_class; + struct test_case { + ss::sstring tp_raw; + props_t props; + alter_props_t alteration; + kafka::error_code expected; + skip_create skip{skip_create::no}; + }; + std::vector test_cases; + + constexpr auto success = kafka::error_code::none; + constexpr auto failure = kafka::error_code::invalid_config; + + constexpr auto with = + [](std::string_view prop, auto val) -> props_t::value_type { + return {ss::sstring{prop}, ssx::sformat("{}", val)}; + }; + + constexpr auto set = + [](std::string_view prop, auto val) -> alter_props_t::value_type { + return { + ss::sstring{prop}, + {ssx::sformat("{}", val), kafka::config_resource_operation::set}}; + }; + + constexpr auto remove = + [](std::string_view prop) -> alter_props_t::value_type { + return { + ss::sstring{prop}, + {std::nullopt, kafka::config_resource_operation::remove}}; + }; + + const auto enterprise_props = + [enable_cluster_config]() -> std::vector { + // If we aren't enabling Schema ID validation cluster config, + // then skip testing those topic properties + if (enable_cluster_config) { + return { + kafka::topic_property_remote_read, + kafka::topic_property_remote_write, + kafka::topic_property_record_key_schema_id_validation, + kafka::topic_property_record_key_schema_id_validation_compat, + kafka::topic_property_record_value_schema_id_validation, + kafka:: + topic_property_record_value_schema_id_validation_compat, + }; + } else { + return { + kafka::topic_property_remote_read, + kafka::topic_property_remote_write, + }; + } + }(); + + const auto non_enterprise_prop = props_t::value_type{ + kafka::topic_property_max_message_bytes, "4096"}; + + for (const auto& p : enterprise_props) { + // A topic without an enterprise property set, and then enable it + test_cases.emplace_back( + ssx::sformat("enable_{}", p), + props_t{}, + alter_props_t{{set(p, true)}}, + enable_cluster_config ? failure : success); + // A topic with an enterprise property set, and then set it to false + test_cases.emplace_back( + ssx::sformat("set_false_{}", p), + props_t{with(p, true)}, + alter_props_t{{set(p, false)}}, + success); + // A topic with an enterprise property set, and then remove it + test_cases.emplace_back( + ssx::sformat("remove_{}", p), + props_t{with(p, true)}, + alter_props_t{{remove(p)}}, + success); + // A topic with an enterprise property set, and then change + // non-enterprise property + test_cases.emplace_back( + ssx::sformat("set_other_{}", p), + props_t{with(p, true)}, + alter_props_t{{std::apply(set, non_enterprise_prop)}}, + success); + // A topic with an enterprise property set, and then remove + // non-enterprise property + test_cases.emplace_back( + ssx::sformat("remove_other_{}", p), + props_t{with(p, true), non_enterprise_prop}, + alter_props_t{{remove(non_enterprise_prop.first)}}, + success); + + // Skip creating topic. Expect no sanctions. + // Alter operations should fail downstream. + test_cases.emplace_back( + ssx::sformat("skip_create_{}", p), + props_t{}, + alter_props_t{{set(p, true)}}, + kafka::error_code::unknown_topic_or_partition, + skip_create::yes); + } + + // Specific tests for tiered storage + { + const auto full_si = props_t{ + with(kafka::topic_property_remote_read, true), + with(kafka::topic_property_remote_write, true), + with(kafka::topic_property_remote_delete, true)}; + test_cases.emplace_back( + "remove_remote.read_from_full", + full_si, + alter_props_t{{remove(kafka::topic_property_remote_read)}}, + success); + test_cases.emplace_back( + "remove_remote.write_from_full", + full_si, + alter_props_t{{remove(kafka::topic_property_remote_write)}}, + success); + test_cases.emplace_back( + "remove_remote.delete_from_full", + full_si, + alter_props_t{{remove(kafka::topic_property_remote_delete)}}, + success); + test_cases.emplace_back( + "enable_remote.delete", + props_t{with(kafka::topic_property_remote_delete, false)}, + alter_props_t{{set(kafka::topic_property_remote_delete, true)}}, + enable_cluster_config ? failure : success); + } + + // Specific tests for schema id validation subject name strategy + if (enable_cluster_config) { + using sns = pandaproxy::schema_registry::subject_name_strategy; + + const auto full_validation = props_t{ + with(kafka::topic_property_record_key_schema_id_validation, true), + with( + kafka::topic_property_record_value_schema_id_validation, true), + }; + test_cases.emplace_back( + "set_key_sns", + full_validation, + alter_props_t{ + {set( + kafka::topic_property_record_key_subject_name_strategy, + sns::topic_name)}, + }, + failure); + test_cases.emplace_back( + "set_value_sns", + full_validation, + alter_props_t{ + {set( + kafka::topic_property_record_value_subject_name_strategy, + sns::topic_name)}, + }, + failure); + + const auto key_validation = props_t{ + with(kafka::topic_property_record_key_schema_id_validation, true), + }; + test_cases.emplace_back( + "set_value_after_key", + key_validation, + alter_props_t{{set( + kafka::topic_property_record_value_schema_id_validation_compat, + true)}}, + failure); + test_cases.emplace_back( + "unset_key", + key_validation, + alter_props_t{{set( + kafka::topic_property_record_key_schema_id_validation, false)}}, + success); + + const auto value_validation = props_t{ + with( + kafka::topic_property_record_value_schema_id_validation, true), + }; + test_cases.emplace_back( + "set_key_after_value", + value_validation, + alter_props_t{{set( + kafka::topic_property_record_key_schema_id_validation_compat, + true)}}, + failure); + test_cases.emplace_back( + "unset_value", + value_validation, + alter_props_t{{set( + kafka::topic_property_record_value_schema_id_validation, + false)}}, + success); + + const auto validation_with_strat = props_t{ + with(kafka::topic_property_record_key_schema_id_validation, true), + with( + kafka::topic_property_record_value_schema_id_validation, true), + with( + kafka::topic_property_record_key_subject_name_strategy, + sns::topic_name), + with( + kafka::topic_property_record_value_subject_name_strategy, + sns::topic_name), + }; + test_cases.emplace_back( + "change_key_sns", + validation_with_strat, + alter_props_t{ + {set( + kafka::topic_property_record_key_subject_name_strategy, + sns::record_name)}, + }, + failure); + test_cases.emplace_back( + "change_value_sns", + validation_with_strat, + alter_props_t{ + {set( + kafka::topic_property_record_value_subject_name_strategy, + sns::record_name)}, + }, + failure); + test_cases.emplace_back( + "remove_key_sns", + validation_with_strat, + alter_props_t{{remove( + kafka::topic_property_record_key_subject_name_strategy)}}, + success); + + test_cases.emplace_back( + "remove_value_sns", + validation_with_strat, + alter_props_t{{remove( + kafka::topic_property_record_value_subject_name_strategy)}}, + success); + } + + // NOTE(oren): w/o schema validation enabled at the cluster level, + // related properties will be ignored on the topic create path. stick to + // COMPAT here because it's a superset of REDPANDA. + if (enable_cluster_config) { + update_cluster_config("enable_schema_id_validation", "compat"); + update_cluster_config("cloud_storage_enabled", "true"); + } + auto unset_cluster_config = ss::defer([&] { + update_cluster_config("enable_schema_id_validation", "none"); + update_cluster_config("cloud_storage_enabled", "false"); + }); + + // Specific tests for leadership pinning + { + const config::leaders_preference no_preference{}; + const config::leaders_preference pref_a{ + .type = config::leaders_preference::type_t::racks, + .racks = {model::rack_id{"A"}}}; + const config::leaders_preference pref_b{ + .type = config::leaders_preference::type_t::racks, + .racks = {model::rack_id{"A"}, model::rack_id{"B"}}}; + + test_cases.emplace_back( + "leaders_preference.enable", + props_t{}, + alter_props_t{ + {set(kafka::topic_property_leaders_preference, pref_a)}}, + failure); + test_cases.emplace_back( + "leaders_preference.change", + props_t{with(kafka::topic_property_leaders_preference, pref_a)}, + alter_props_t{ + {set(kafka::topic_property_leaders_preference, pref_b)}}, + failure); + test_cases.emplace_back( + "leaders_preference.no_change", + props_t{with(kafka::topic_property_leaders_preference, pref_a)}, + alter_props_t{ + {set(kafka::topic_property_leaders_preference, pref_a)}}, + success); + test_cases.emplace_back( + "leaders_preference.unset", + props_t{with(kafka::topic_property_leaders_preference, pref_a)}, + alter_props_t{{remove(kafka::topic_property_leaders_preference)}}, + success); + test_cases.emplace_back( + "leaders_preference.disable", + props_t{with(kafka::topic_property_leaders_preference, pref_a)}, + alter_props_t{ + {set(kafka::topic_property_leaders_preference, no_preference)}}, + success); + } + + // Create the topics for the tests + constexpr auto inc_alter_topic = [](std::string_view tp_raw) { + return model::topic{ssx::sformat("incremental_alter_{}", tp_raw)}; + }; + constexpr auto alter_topic = [](std::string_view tp_raw) { + return model::topic{ssx::sformat("alter_{}", tp_raw)}; + }; + + for (const auto& [tp_raw, props, alteration, expected, skip] : + test_cases) { + BOOST_TEST_CONTEXT(fmt::format("topic: {}", tp_raw)) { + BOOST_REQUIRE( + skip + || !create_topic(inc_alter_topic(tp_raw), props, 3) + .data.errored()); + BOOST_REQUIRE( + skip + || !create_topic(alter_topic(tp_raw), props, 3) + .data.errored()); + } + + revoke_license(); + + // Test incremental alter config + auto tp = inc_alter_topic(tp_raw); + BOOST_TEST_CONTEXT(tp) { + auto resp = incremental_alter_configs( + make_incremental_alter_topic_config_resource_cv( + tp, alteration)); + BOOST_REQUIRE_EQUAL(resp.data.responses.size(), 1); + BOOST_CHECK_EQUAL(resp.data.responses[0].error_code, expected); + if (expected == failure) { + BOOST_CHECK( + resp.data.responses[0] + .error_message.value_or("") + .contains( + features::enterprise_error_message::required)); + } + } + + delete_topic( + model::topic_namespace{model::kafka_namespace, std::move(tp)}) + .get(); + + // Test alter config + tp = alter_topic(tp_raw); + BOOST_TEST_CONTEXT(tp) { + auto properties = props; + for (const auto& a : alteration) { + if ( + a.second.second + == kafka::config_resource_operation::remove) { + properties.erase(a.first); + } else if ( + a.second.second + == kafka::config_resource_operation::set) { + properties.insert_or_assign( + a.first, a.second.first.value()); + }; + } + + auto resp = alter_configs( + make_alter_topic_config_resource_cv(tp, properties)); + BOOST_REQUIRE_EQUAL(resp.data.responses.size(), 1); + BOOST_CHECK_EQUAL(resp.data.responses[0].error_code, expected); + } + delete_topic( + model::topic_namespace{model::kafka_namespace, std::move(tp)}) + .get(); + + reinstall_license(); + } + } }; FIXTURE_TEST( @@ -1202,339 +1572,10 @@ FIXTURE_TEST( } FIXTURE_TEST(test_unlicensed_alter_configs, alter_config_test_fixture) { - using props_t = absl::flat_hash_map; - using alter_props_t = absl::flat_hash_map< - ss::sstring, - std::pair, kafka::config_resource_operation>>; - using skip_create = ss::bool_class; - struct test_case { - ss::sstring tp_raw; - props_t props; - alter_props_t alteration; - kafka::error_code expected; - skip_create skip{skip_create::no}; - }; - std::vector test_cases; - - constexpr auto success = kafka::error_code::none; - constexpr auto failure = kafka::error_code::invalid_config; - - constexpr auto with = - [](std::string_view prop, auto val) -> props_t::value_type { - return {ss::sstring{prop}, ssx::sformat("{}", val)}; - }; - - constexpr auto set = - [](std::string_view prop, auto val) -> alter_props_t::value_type { - return { - ss::sstring{prop}, - {ssx::sformat("{}", val), kafka::config_resource_operation::set}}; - }; - - constexpr auto remove = - [](std::string_view prop) -> alter_props_t::value_type { - return { - ss::sstring{prop}, - {std::nullopt, kafka::config_resource_operation::remove}}; - }; - - const auto enterprise_props = { - kafka::topic_property_remote_read, - kafka::topic_property_remote_write, - kafka::topic_property_record_key_schema_id_validation, - kafka::topic_property_record_key_schema_id_validation_compat, - kafka::topic_property_record_value_schema_id_validation, - kafka::topic_property_record_value_schema_id_validation_compat, - }; - - const auto non_enterprise_prop = props_t::value_type{ - kafka::topic_property_max_message_bytes, "4096"}; - - for (const auto& p : enterprise_props) { - // A topic without an enterprise property set, and then enable it - test_cases.emplace_back( - ssx::sformat("enable_{}", p), - props_t{}, - alter_props_t{{set(p, true)}}, - failure); - // A topic with an enterprise property set, and then set it to false - test_cases.emplace_back( - ssx::sformat("set_false_{}", p), - props_t{with(p, true)}, - alter_props_t{{set(p, false)}}, - success); - // A topic with an enterprise property set, and then remove it - test_cases.emplace_back( - ssx::sformat("remove_{}", p), - props_t{with(p, true)}, - alter_props_t{{remove(p)}}, - success); - // A topic with an enterprise property set, and then change - // non-enterprise property - test_cases.emplace_back( - ssx::sformat("set_other_{}", p), - props_t{with(p, true)}, - alter_props_t{{std::apply(set, non_enterprise_prop)}}, - success); - // A topic with an enterprise property set, and then remove - // non-enterprise property - test_cases.emplace_back( - ssx::sformat("remove_other_{}", p), - props_t{with(p, true), non_enterprise_prop}, - alter_props_t{{remove(non_enterprise_prop.first)}}, - success); - - // Skip creating topic. Expect no sanctions. - // Alter operations should fail downstream. - test_cases.emplace_back( - ssx::sformat("skip_create_{}", p), - props_t{}, - alter_props_t{{set(p, true)}}, - kafka::error_code::unknown_topic_or_partition, - skip_create::yes); - } - - // Specific tests for tiered storage - { - const auto full_si = props_t{ - with(kafka::topic_property_remote_read, true), - with(kafka::topic_property_remote_write, true), - with(kafka::topic_property_remote_delete, true)}; - test_cases.emplace_back( - "remove_remote.read_from_full", - full_si, - alter_props_t{{remove(kafka::topic_property_remote_read)}}, - success); - test_cases.emplace_back( - "remove_remote.write_from_full", - full_si, - alter_props_t{{remove(kafka::topic_property_remote_write)}}, - success); - test_cases.emplace_back( - "remove_remote.delete_from_full", - full_si, - alter_props_t{{remove(kafka::topic_property_remote_delete)}}, - success); - test_cases.emplace_back( - "enable_remote.delete", - props_t{with(kafka::topic_property_remote_delete, false)}, - alter_props_t{{set(kafka::topic_property_remote_delete, true)}}, - failure); - } - - // Specific tests for schema id validation subject name strategy - { - using sns = pandaproxy::schema_registry::subject_name_strategy; - - const auto full_validation = props_t{ - with(kafka::topic_property_record_key_schema_id_validation, true), - with(kafka::topic_property_record_value_schema_id_validation, true), - }; - test_cases.emplace_back( - "set_key_sns", - full_validation, - alter_props_t{ - {set( - kafka::topic_property_record_key_subject_name_strategy, - sns::topic_name)}, - }, - failure); - test_cases.emplace_back( - "set_value_sns", - full_validation, - alter_props_t{ - {set( - kafka::topic_property_record_value_subject_name_strategy, - sns::topic_name)}, - }, - failure); - - const auto key_validation = props_t{ - with(kafka::topic_property_record_key_schema_id_validation, true), - }; - test_cases.emplace_back( - "set_value_after_key", - key_validation, - alter_props_t{{set( - kafka::topic_property_record_value_schema_id_validation_compat, - true)}}, - failure); - test_cases.emplace_back( - "unset_key", - key_validation, - alter_props_t{{set( - kafka::topic_property_record_key_schema_id_validation, false)}}, - success); - - const auto value_validation = props_t{ - with(kafka::topic_property_record_value_schema_id_validation, true), - }; - test_cases.emplace_back( - "set_key_after_value", - value_validation, - alter_props_t{{set( - kafka::topic_property_record_key_schema_id_validation_compat, - true)}}, - failure); - test_cases.emplace_back( - "unset_value", - value_validation, - alter_props_t{{set( - kafka::topic_property_record_value_schema_id_validation, false)}}, - success); - - const auto validation_with_strat = props_t{ - with(kafka::topic_property_record_key_schema_id_validation, true), - with(kafka::topic_property_record_value_schema_id_validation, true), - with( - kafka::topic_property_record_key_subject_name_strategy, - sns::topic_name), - with( - kafka::topic_property_record_value_subject_name_strategy, - sns::topic_name), - }; - test_cases.emplace_back( - "change_key_sns", - validation_with_strat, - alter_props_t{ - {set( - kafka::topic_property_record_key_subject_name_strategy, - sns::record_name)}, - }, - failure); - test_cases.emplace_back( - "change_value_sns", - validation_with_strat, - alter_props_t{ - {set( - kafka::topic_property_record_value_subject_name_strategy, - sns::record_name)}, - }, - failure); - test_cases.emplace_back( - "remove_key_sns", - validation_with_strat, - alter_props_t{ - {remove(kafka::topic_property_record_key_subject_name_strategy)}}, - success); - - test_cases.emplace_back( - "remove_value_sns", - validation_with_strat, - alter_props_t{ - {remove(kafka::topic_property_record_value_subject_name_strategy)}}, - success); - } - - // NOTE(oren): w/o schema validation enabled at the cluster level, related - // properties will be ignored on the topic create path. stick to COMPAT here - // because it's a superset of REDPANDA. - update_cluster_config("enable_schema_id_validation", "compat"); - - // Specific tests for leadership pinning - { - const config::leaders_preference no_preference{}; - const config::leaders_preference pref_a{ - .type = config::leaders_preference::type_t::racks, - .racks = {model::rack_id{"A"}}}; - const config::leaders_preference pref_b{ - .type = config::leaders_preference::type_t::racks, - .racks = {model::rack_id{"A"}, model::rack_id{"B"}}}; - - test_cases.emplace_back( - "leaders_preference.enable", - props_t{}, - alter_props_t{ - {set(kafka::topic_property_leaders_preference, pref_a)}}, - failure); - test_cases.emplace_back( - "leaders_preference.change", - props_t{with(kafka::topic_property_leaders_preference, pref_a)}, - alter_props_t{ - {set(kafka::topic_property_leaders_preference, pref_b)}}, - failure); - test_cases.emplace_back( - "leaders_preference.no_change", - props_t{with(kafka::topic_property_leaders_preference, pref_a)}, - alter_props_t{ - {set(kafka::topic_property_leaders_preference, pref_a)}}, - success); - test_cases.emplace_back( - "leaders_preference.unset", - props_t{with(kafka::topic_property_leaders_preference, pref_a)}, - alter_props_t{{remove(kafka::topic_property_leaders_preference)}}, - success); - test_cases.emplace_back( - "leaders_preference.disable", - props_t{with(kafka::topic_property_leaders_preference, pref_a)}, - alter_props_t{ - {set(kafka::topic_property_leaders_preference, no_preference)}}, - success); - } - - // Create the topics for the tests - constexpr auto inc_alter_topic = [](std::string_view tp_raw) { - return model::topic{ssx::sformat("incremental_alter_{}", tp_raw)}; - }; - constexpr auto alter_topic = [](std::string_view tp_raw) { - return model::topic{ssx::sformat("alter_{}", tp_raw)}; - }; - - for (const auto& [tp_raw, props, alteration, expected, skip] : test_cases) { - BOOST_TEST_CONTEXT(fmt::format("topic: {}", tp_raw)) { - BOOST_REQUIRE( - skip - || !create_topic(inc_alter_topic(tp_raw), props, 3) - .data.errored()); - BOOST_REQUIRE( - skip - || !create_topic(alter_topic(tp_raw), props, 3).data.errored()); - } - - revoke_license(); - - // Test incremental alter config - auto tp = inc_alter_topic(tp_raw); - BOOST_TEST_CONTEXT(tp) { - auto resp = incremental_alter_configs( - make_incremental_alter_topic_config_resource_cv(tp, alteration)); - BOOST_REQUIRE_EQUAL(resp.data.responses.size(), 1); - BOOST_CHECK_EQUAL(resp.data.responses[0].error_code, expected); - if (expected == failure) { - BOOST_CHECK( - resp.data.responses[0].error_message.value_or("").contains( - features::enterprise_error_message::required)); - } - } - - delete_topic( - model::topic_namespace{model::kafka_namespace, std::move(tp)}) - .get(); - - // Test alter config - tp = alter_topic(tp_raw); - BOOST_TEST_CONTEXT(tp) { - auto properties = props; - for (const auto& a : alteration) { - if ( - a.second.second == kafka::config_resource_operation::remove) { - properties.erase(a.first); - } else if ( - a.second.second == kafka::config_resource_operation::set) { - properties.insert_or_assign( - a.first, a.second.first.value()); - }; - } - - auto resp = alter_configs( - make_alter_topic_config_resource_cv(tp, properties)); - BOOST_REQUIRE_EQUAL(resp.data.responses.size(), 1); - BOOST_CHECK_EQUAL(resp.data.responses[0].error_code, expected); - } - delete_topic( - model::topic_namespace{model::kafka_namespace, std::move(tp)}) - .get(); + alter_config_no_license_test(true); +} - reinstall_license(); - } +FIXTURE_TEST( + test_unlicensed_alter_configs_no_cluster_config, alter_config_test_fixture) { + alter_config_no_license_test(false); } diff --git a/src/v/kafka/server/tests/create_partition_test.cc b/src/v/kafka/server/tests/create_partition_test.cc index fc1228c85d04..b88f335a924b 100644 --- a/src/v/kafka/server/tests/create_partition_test.cc +++ b/src/v/kafka/server/tests/create_partition_test.cc @@ -35,7 +35,13 @@ FIXTURE_TEST( prop, props_t{{ss::sstring{prop}, ssx::sformat("{}", value)}}); }; + update_cluster_config(lconf().cloud_storage_enabled.name(), "true"); update_cluster_config(lconf().enable_schema_id_validation.name(), "compat"); + auto unset_cluster_config = ss::defer([&] { + update_cluster_config(lconf().cloud_storage_enabled.name(), "false"); + update_cluster_config( + lconf().enable_schema_id_validation.name(), "none"); + }); std::initializer_list enterprise_props{ // si_props @@ -85,3 +91,48 @@ FIXTURE_TEST( } } } + +FIXTURE_TEST( + test_unlicensed_topic_prop_create_partition_no_cluster_config, + topic_properties_test_fixture) { + using props_t = absl::flat_hash_map; + using test_t = std::pair; + const auto with = [](const std::string_view prop, const auto value) { + return std::make_pair( + prop, props_t{{ss::sstring{prop}, ssx::sformat("{}", value)}}); + }; + + std::initializer_list enterprise_props{ + // si_props + // Exclude these; setting up s3_imposter is too complex for this test + // * kafka::topic_property_recovery + // * kafka::topic_property_read_replica + with(kafka::topic_property_remote_read, true), + with(kafka::topic_property_remote_write, true)}; + + const int32_t partitions = 3; + + for (const auto& [prop, props] : enterprise_props) { + BOOST_TEST_CONTEXT(fmt::format("property: {}", prop)) { + auto tp = model::topic{ssx::sformat("{}", prop)}; + + auto c_res = create_topic(tp, props, 3).data; + BOOST_REQUIRE_EQUAL(c_res.topics.size(), 1); + BOOST_REQUIRE_EQUAL( + c_res.topics[0].error_code, kafka::error_code::none); + + revoke_license(); + + auto res = create_partitions(tp, partitions + 1).data; + BOOST_REQUIRE_EQUAL(res.results.size(), 1); + BOOST_CHECK_EQUAL( + res.results[0].error_code, kafka::error_code::none); + + delete_topic( + model::topic_namespace{model::kafka_namespace, std::move(tp)}) + .get(); + + reinstall_license(); + } + } +} diff --git a/src/v/kafka/server/tests/create_topics_test.cc b/src/v/kafka/server/tests/create_topics_test.cc index be837ae83046..066c0970d664 100644 --- a/src/v/kafka/server/tests/create_topics_test.cc +++ b/src/v/kafka/server/tests/create_topics_test.cc @@ -7,6 +7,7 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0 +#include "config/configuration.h" #include "config/leaders_preference.h" #include "container/fragmented_vector.h" #include "features/enterprise_feature_messages.h" @@ -490,12 +491,57 @@ FIXTURE_TEST(case_insensitive_boolean_property, create_topic_fixture) { BOOST_CHECK_EQUAL(resp.data.topics[0].name, "topic1"); } +FIXTURE_TEST(unlicensed_permit_if_config_disabled, create_topic_fixture) { + lconf().enable_schema_id_validation.set_value( + pandaproxy::schema_registry::schema_id_validation_mode::none); + lconf().cloud_storage_enabled.set_value(false); + + revoke_license(); + + using prop_t = std::map; + const auto with = [](const std::string_view prop, const auto value) { + return std::make_pair( + prop, prop_t{{ss::sstring{prop}, ssx::sformat("{}", value)}}); + }; + std::vector> enterprise_props{ + // si_props + with(kafka::topic_property_remote_read, true), + with(kafka::topic_property_remote_write, true), + // schema id validation + with(kafka::topic_property_record_key_schema_id_validation, true), + with(kafka::topic_property_record_key_schema_id_validation_compat, true), + with(kafka::topic_property_record_value_schema_id_validation, true), + with( + kafka::topic_property_record_value_schema_id_validation_compat, true)}; + + auto client = make_kafka_client().get(); + client.connect().get(); + + for (const auto& [name, props] : enterprise_props) { + auto topic = make_topic( + ssx::sformat("topic_{}", name), std::nullopt, std::nullopt, props); + + auto resp + = client.dispatch(make_req({topic}), kafka::api_version(5)).get(); + + BOOST_CHECK_EQUAL( + resp.data.topics[0].error_code, kafka::error_code::none); + } +} + FIXTURE_TEST(unlicensed_rejected, create_topic_fixture) { // NOTE(oren): w/o schema validation enabled at the cluster level, related // properties will be ignored on the topic create path. stick to COMPAT here // because it's a superset of REDPANDA. lconf().enable_schema_id_validation.set_value( pandaproxy::schema_registry::schema_id_validation_mode::compat); + lconf().cloud_storage_enabled.set_value(true); + + auto unset_cloud_storage = ss::defer([&] { + lconf().enable_schema_id_validation.set_value( + pandaproxy::schema_registry::schema_id_validation_mode::none); + lconf().cloud_storage_enabled.set_value(false); + }); revoke_license(); using prop_t = std::map; @@ -545,6 +591,9 @@ FIXTURE_TEST(unlicensed_rejected, create_topic_fixture) { } FIXTURE_TEST(unlicensed_reject_defaults, create_topic_fixture) { + lconf().cloud_storage_enabled.set_value(true); + auto unset_cloud_storage = ss::defer( + [&] { lconf().cloud_storage_enabled.set_value(false); }); revoke_license(); const std::initializer_list si_configs{ From 98a08b26a9accad05e49e81ca489534bdedb4de8 Mon Sep 17 00:00:00 2001 From: Michael Boquard Date: Fri, 13 Dec 2024 15:36:42 -0500 Subject: [PATCH 162/229] dt: Test to verify relaxed tiered storage topic configs Signed-off-by: Michael Boquard --- .../rptest/tests/license_enforcement_test.py | 87 ++++++++++++++++++- 1 file changed, 85 insertions(+), 2 deletions(-) diff --git a/tests/rptest/tests/license_enforcement_test.py b/tests/rptest/tests/license_enforcement_test.py index 19abda43f9ae..47512c583a2c 100644 --- a/tests/rptest/tests/license_enforcement_test.py +++ b/tests/rptest/tests/license_enforcement_test.py @@ -12,9 +12,9 @@ from ducktape.mark import matrix from rptest.services.cluster import cluster -from rptest.clients.rpk import RpkTool +from rptest.clients.rpk import RpkTool, RpkException from rptest.services.admin import Admin -from rptest.services.redpanda import LoggingConfig +from rptest.services.redpanda import LoggingConfig, SISettings from rptest.tests.redpanda_test import RedpandaTest from rptest.services.redpanda_installer import RedpandaInstaller from rptest.utils.mode_checks import skip_fips_mode @@ -201,3 +201,86 @@ def test_enterprise_cluster_bootstrap(self, root_driven_bootstrap): timeout_sec=60, backoff_sec=1, err_msg="The cluster hasn't stabilized") + + +class LicenseEnforcementPermittedTopicParams(RedpandaTest): + """ + Tests that validate that topics properties whose controlling cluster config + is disabled do not cause any issues in regards to license enforcement. + """ + def __init__(self, *args, **kwargs): + super().__init__(*args, **kwargs) + + self.rpk = RpkTool(self.redpanda) + + def setUp(self): + pass + + @cluster(num_nodes=3) + @matrix(enable_cloud_storage=[False, True]) + def test_cloud_storage_topic_params(self, enable_cloud_storage): + """ + This test verifies that if a license isn't installed and `cloud_storage_enabled` + is set to `False`, then topics may be created with TS settingss set to true, e.g. + `redpanda.remote.write`. + """ + if enable_cloud_storage: + si_settings = SISettings(self.test_context) + self.redpanda.set_si_settings(si_settings) + + super().setUp() + + self.redpanda.set_environment( + {'__REDPANDA_DISABLE_BUILTIN_TRIAL_LICENSE': True}) + self.redpanda.restart_nodes(self.redpanda.nodes) + self.redpanda.wait_until(self.redpanda.healthy, + timeout_sec=60, + backoff_sec=1, + err_msg="The cluster hasn't stabilized") + + try: + self.rpk.create_topic("test", + config={"redpanda.remote.write": "true"}) + assert not enable_cloud_storage, "Should have failed to create topic with redpanda.remote.write set and cloud_storage_enabled set to True" + except RpkException as e: + assert enable_cloud_storage, f"Should not have failed to create topic with redpanda.remote.write set and cloud_storage_enabled set to False: {e}" + + @cluster(num_nodes=3) + def test_upgrade_with_topic_configs(self): + """ + This test verifies that if a license isn't installed and `cloud_storage_enabled` + is set to `False` and topics exist with tiered storage capabilities, the upgrade + will still succeed + """ + installer = self.redpanda._installer + prev_version = installer.highest_from_prior_feature_version( + RedpandaInstaller.HEAD) + latest_version = installer.head_version() + self.logger.info( + f"Testing with versions: {prev_version=} {latest_version=}") + + self.logger.info(f"Starting all nodes with version: {prev_version}") + installer.install(self.redpanda.nodes, prev_version) + self.redpanda.start(nodes=self.redpanda.nodes, + omit_seeds_on_idx_one=False) + self.redpanda.wait_until(self.redpanda.healthy, + timeout_sec=60, + backoff_sec=1, + err_msg="The cluster hasn't stabilized") + self.logger.debug( + "Creating a topic with redpanda.remote.write set to true") + self.rpk.create_topic("test", config={"redpanda.remote.write": "true"}) + self.logger.info( + "Disabling the trial license to simulate that the license expired") + self.redpanda.set_environment( + {'__REDPANDA_DISABLE_BUILTIN_TRIAL_LICENSE': True}) + self.redpanda.restart_nodes(self.redpanda.nodes) + self.redpanda.wait_until(self.redpanda.healthy, + timeout_sec=60, + backoff_sec=1, + err_msg="The cluster hasn't stabilized") + + installer.install(self.redpanda.nodes, latest_version) + self.redpanda.start(nodes=self.redpanda.nodes, + auto_assign_node_id=True, + omit_seeds_on_idx_one=False) From 7f8322ca0d5e2a5905311c9a62782be03a55e1cd Mon Sep 17 00:00:00 2001 From: r-vasquez Date: Fri, 13 Dec 2024 11:50:44 -0800 Subject: [PATCH 163/229] rpk remote debug bundle: job-id help text change This string must be a UUID and it was not clear in the help text. --- src/go/rpk/pkg/cli/debug/remotebundle/start.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/go/rpk/pkg/cli/debug/remotebundle/start.go b/src/go/rpk/pkg/cli/debug/remotebundle/start.go index ba477a28956c..bec3b1b5086d 100644 --- a/src/go/rpk/pkg/cli/debug/remotebundle/start.go +++ b/src/go/rpk/pkg/cli/debug/remotebundle/start.go @@ -155,7 +155,7 @@ status, run: }, } f := cmd.Flags() - f.StringVar(&jobID, "job-id", "", "ID of the job to start debug bundle in") + f.StringVar(&jobID, "job-id", "", "Custom UUID to assign to the job that generates the debug bundle") f.BoolVar(&noConfirm, "no-confirm", false, "Disable confirmation prompt") // Debug bundle options: opts.InstallFlags(f) From 3f60aed21804a1e382695e2098d39e2c6d5bea15 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Mon, 16 Dec 2024 16:50:02 +0100 Subject: [PATCH 164/229] tests/polaris: use curl to query polaris catalog health check MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Using `curl` which executes on the node where Polaris server runs so no additional ports have to be open. Fixes: CORE-8338 Signed-off-by: Michał Maślanka --- tests/rptest/services/polaris_catalog.py | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/tests/rptest/services/polaris_catalog.py b/tests/rptest/services/polaris_catalog.py index 3254d0823c00..5fe7882ad791 100644 --- a/tests/rptest/services/polaris_catalog.py +++ b/tests/rptest/services/polaris_catalog.py @@ -137,12 +137,14 @@ def _polaris_ready(): self.logger.debug( f"Querying polaris healthcheck on http://{node.account.hostname}:8182/healthcheck" ) - r = requests.get( - f"http://{node.account.hostname}:8182/healthcheck", timeout=10) - self.logger.info( - f"health check result status code: {r.status_code}") - return r.status_code == 200 + out = node.account.ssh_output( + "curl -s -m 10 -o /dev/null -w '%{http_code}' http://localhost:8182/healthcheck" + ) + status_code = int(out.decode('utf-8')) + self.logger.debug( + f"health check result status code: {status_code}") + return status_code == 200 wait_until(_polaris_ready, timeout_sec=timeout_sec, From 8ace98e9e916bb1902ef4d8489286c92407a92b9 Mon Sep 17 00:00:00 2001 From: Ben Pope Date: Mon, 2 Dec 2024 20:40:42 +0000 Subject: [PATCH 165/229] security: Improve validate_scram_credential Optimise the password validation when SCRAM-SHA-512 is in use, by avoiding the validation against SCRAM-SHA-256 that will fail. Signed-off-by: Ben Pope --- src/v/security/scram_algorithm.h | 2 ++ src/v/security/scram_authenticator.cc | 12 ++++++++---- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/src/v/security/scram_algorithm.h b/src/v/security/scram_algorithm.h index 25c4dfc3f9c2..bfa631d5f65b 100644 --- a/src/v/security/scram_algorithm.h +++ b/src/v/security/scram_algorithm.h @@ -196,6 +196,8 @@ class scram_algorithm { static constexpr int min_iterations = MinIterations; static_assert(min_iterations > 0, "Minimum iterations must be positive"); + static constexpr auto key_size = HashType::digest_size; + static bytes client_signature( bytes_view stored_key, const client_first_message& client_first, diff --git a/src/v/security/scram_authenticator.cc b/src/v/security/scram_authenticator.cc index 833334249927..7247d54f8f20 100644 --- a/src/v/security/scram_authenticator.cc +++ b/src/v/security/scram_authenticator.cc @@ -140,11 +140,15 @@ template class scram_authenticator; std::optional validate_scram_credential( const scram_credential& cred, const credential_password& password) { std::optional sasl_mechanism; - if (security::scram_sha256::validate_password( - password, cred.stored_key(), cred.salt(), cred.iterations())) { + if ( + cred.stored_key().size() == security::scram_sha256::key_size + && security::scram_sha256::validate_password( + password, cred.stored_key(), cred.salt(), cred.iterations())) { sasl_mechanism = security::scram_sha256_authenticator::name; - } else if (security::scram_sha512::validate_password( - password, cred.stored_key(), cred.salt(), cred.iterations())) { + } else if ( + cred.stored_key().size() == security::scram_sha512::key_size + && security::scram_sha512::validate_password( + password, cred.stored_key(), cred.salt(), cred.iterations())) { sasl_mechanism = security::scram_sha512_authenticator::name; } From 6cab505cc903cd20fa4edd18743418c29d155f4b Mon Sep 17 00:00:00 2001 From: Ben Pope Date: Mon, 2 Dec 2024 19:53:26 +0000 Subject: [PATCH 166/229] security: Introduce plain_authenticator SUCCESS-02 Signed-off-by: Ben Pope --- src/v/security/BUILD | 2 + src/v/security/CMakeLists.txt | 1 + src/v/security/plain_authenticator.cc | 108 ++++++++++++++++++++++++++ src/v/security/plain_authenticator.h | 62 +++++++++++++++ 4 files changed, 173 insertions(+) create mode 100644 src/v/security/plain_authenticator.cc create mode 100644 src/v/security/plain_authenticator.h diff --git a/src/v/security/BUILD b/src/v/security/BUILD index 71a5bc1242cb..74f58a80d099 100644 --- a/src/v/security/BUILD +++ b/src/v/security/BUILD @@ -91,6 +91,7 @@ redpanda_cc_library( "oidc_authenticator.cc", "oidc_principal_mapping_applicator.cc", "oidc_service.cc", + "plain_authenticator.cc", "role.cc", "scram_algorithm.cc", "scram_authenticator.cc", @@ -118,6 +119,7 @@ redpanda_cc_library( "oidc_error.h", "oidc_principal_mapping_applicator.h", "oidc_service.h", + "plain_authenticator.h", "role.h", "role_store.h", "sasl_authentication.h", diff --git a/src/v/security/CMakeLists.txt b/src/v/security/CMakeLists.txt index 4471df8ca281..2add10498168 100644 --- a/src/v/security/CMakeLists.txt +++ b/src/v/security/CMakeLists.txt @@ -29,6 +29,7 @@ v_cc_library( oidc_authenticator.cc oidc_principal_mapping_applicator.cc oidc_service.cc + plain_authenticator.cc request_auth.cc role.cc scram_algorithm.cc diff --git a/src/v/security/plain_authenticator.cc b/src/v/security/plain_authenticator.cc new file mode 100644 index 000000000000..21d048954ce7 --- /dev/null +++ b/src/v/security/plain_authenticator.cc @@ -0,0 +1,108 @@ +/* + * Copyright 2024 Redpanda Data, Inc. + * + * Use of this software is governed by the Business Source License + * included in the file licenses/BSL.md + * + * As of the Change Date specified in that file, in accordance with + * the Business Source License, use of this software will be governed + * by the Apache License, Version 2.0 + */ +#include "security/plain_authenticator.h" + +#include "base/vlog.h" +#include "security/acl.h" +#include "security/credential_store.h" +#include "security/errc.h" +#include "security/logger.h" +#include "security/scram_authenticator.h" +#include "security/types.h" +#include "strings/utf8.h" + +#include + +namespace security { + +ss::future> plain_authenticator::authenticate(bytes auth_bytes) { + constexpr size_t max_length{255}; + constexpr std::string_view sep{"\0", 1}; + + auto make_failed = ss::defer([this] { _state = state::failed; }); + + if (_state != state::init) { + vlog( + seclog.warn, + "invalid plain state: {}", + _state == state::failed ? "failed" : "complete"); + co_return errc::invalid_credentials; + } + + auto auth_str = std::string_view( + reinterpret_cast(auth_bytes.data()), auth_bytes.size()); + + if (!is_valid_utf8(auth_str)) { + vlog(seclog.warn, "invalid utf8"); + co_return errc::invalid_credentials; + } + + // [authorization identity] not supported + if (!auth_str.starts_with(sep)) { + vlog(seclog.warn, "[authorization identity] not supported"); + co_return errc::invalid_credentials; + } + auth_str = auth_str.substr(sep.length()); + auto it = auth_str.find(sep); + if (std::string_view::npos == it) { + vlog(seclog.warn, "seperator not found"); + co_return errc::invalid_credentials; + } + + credential_user username{auth_str.substr(0, it)}; + credential_password password{auth_str.substr(it + sep.length())}; + + if (username().empty()) { + vlog(seclog.warn, "username not found"); + co_return errc::invalid_credentials; + } + + if (username().length() > max_length) { + vlog(seclog.warn, "username too long"); + co_return errc::invalid_credentials; + } + + if (password().empty()) { + vlog(seclog.warn, "password not found"); + co_return errc::invalid_credentials; + } + + if (password().length() > max_length) { + vlog(seclog.warn, "password too long"); + co_return errc::invalid_credentials; + } + + _audit_user.name = username; + auto cred = _credentials.get(username); + if (!cred.has_value()) { + vlog(seclog.warn, "credential not found"); + co_return errc::invalid_credentials; + } + + if (!validate_scram_credential(*cred, password).has_value()) { + vlog(seclog.warn, "scram authentication failed"); + co_return errc::invalid_credentials; + } + + vlog(seclog.trace, "Authenticated user {}", username); + + make_failed.cancel(); + + _principal = cred->principal().value_or( + acl_principal{principal_type::user, username()}); + _audit_user.name = _principal.name(); + _audit_user.type_id = audit::user::type::user; + + _state = state::complete; + co_return bytes{}; +} + +} // namespace security diff --git a/src/v/security/plain_authenticator.h b/src/v/security/plain_authenticator.h new file mode 100644 index 000000000000..468a3b8ce3e9 --- /dev/null +++ b/src/v/security/plain_authenticator.h @@ -0,0 +1,62 @@ +/* + * Copyright 2024 Redpanda Data, Inc. + * + * Use of this software is governed by the Business Source License + * included in the file licenses/BSL.md + * + * As of the Change Date specified in that file, in accordance with + * the Business Source License, use of this software will be governed + * by the Apache License, Version 2.0 + */ +#pragma once +#include "security/acl.h" +#include "security/fwd.h" +#include "security/sasl_authentication.h" + +namespace security { + +/** + * @class plain_authenticator + * @brief A class that implements SASL/PLAIN authentication mechanism. + * + * This class is responsible for handling the SASL/PLAIN authentication process. + * It authenticates the username and password provided by the client against + * SCRAM users in the credential store. + */ +class plain_authenticator final : public sasl_mechanism { +public: + static constexpr const char* name = "PLAIN"; + + explicit plain_authenticator(credential_store& credentials) + : _credentials(credentials) {} + + ss::future> authenticate(bytes auth_bytes) override; + + bool complete() const override { return _state == state::complete; } + bool failed() const override { return _state == state::failed; } + + const acl_principal& principal() const override { + vassert( + _state == state::complete, + "Authentication id is not valid until auth process complete"); + return _principal; + } + + const audit::user& audit_user() const override { return _audit_user; } + + const char* mechanism_name() const override { return "SASL-PLAIN"; } + +private: + enum class state { + init, + complete, + failed, + }; + + state _state{state::init}; + credential_store& _credentials; + acl_principal _principal; + security::audit::user _audit_user; +}; + +} // namespace security From 08d63c9224d65c05d8d7f4a94caf137d418c165c Mon Sep 17 00:00:00 2001 From: Ben Pope Date: Tue, 3 Dec 2024 03:30:01 +0000 Subject: [PATCH 167/229] config: Add configuration for SASL/PLAIN Signed-off-by: Ben Pope --- src/v/config/configuration.cc | 2 +- src/v/config/validators.cc | 8 +++++++- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/src/v/config/configuration.cc b/src/v/config/configuration.cc index d9a064eecb5a..d0ed6d851da9 100644 --- a/src/v/config/configuration.cc +++ b/src/v/config/configuration.cc @@ -1500,7 +1500,7 @@ configuration::configuration() std::vector{"GSSAPI", "OAUTHBEARER"}, "sasl_mechanisms", "A list of supported SASL mechanisms. Accepted values: `SCRAM`, " - "`GSSAPI`, `OAUTHBEARER`.", + "`GSSAPI`, `OAUTHBEARER`, `PLAIN`.", meta{ .needs_restart = needs_restart::no, .visibility = visibility::user, diff --git a/src/v/config/validators.cc b/src/v/config/validators.cc index d5576ac3b0b7..3365e132a6d7 100644 --- a/src/v/config/validators.cc +++ b/src/v/config/validators.cc @@ -115,7 +115,7 @@ std::optional validate_client_groups_byte_rate_quota( std::optional validate_sasl_mechanisms(const std::vector& mechanisms) { constexpr auto supported = std::to_array( - {"GSSAPI", "SCRAM", "OAUTHBEARER"}); + {"GSSAPI", "SCRAM", "OAUTHBEARER", "PLAIN"}); // Validate results for (const auto& m : mechanisms) { @@ -124,6 +124,12 @@ validate_sasl_mechanisms(const std::vector& mechanisms) { return ssx::sformat("'{}' is not a supported SASL mechanism", m); } } + + if (mechanisms.size() == 1 && mechanisms[0] == "PLAIN") { + return "When PLAIN is enabled, at least one other mechanism must be " + "enabled"; + } + return std::nullopt; } From 4beb86aa0d202ed8749be82f47b09ebe662c4274 Mon Sep 17 00:00:00 2001 From: Ben Pope Date: Tue, 3 Dec 2024 03:30:32 +0000 Subject: [PATCH 168/229] kafka: Wire up SASL/PLAIN Signed-off-by: Ben Pope --- src/v/kafka/server/server.cc | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/v/kafka/server/server.cc b/src/v/kafka/server/server.cc index ba04ef527d65..9781434facd5 100644 --- a/src/v/kafka/server/server.cc +++ b/src/v/kafka/server/server.cc @@ -65,6 +65,7 @@ #include "security/gssapi_authenticator.h" #include "security/mtls.h" #include "security/oidc_authenticator.h" +#include "security/plain_authenticator.h" #include "security/scram_algorithm.h" #include "security/scram_authenticator.h" #include "ssx/future-util.h" @@ -720,6 +721,16 @@ ss::future sasl_handshake_handler::handle( } } + if (supports("PLAIN")) { + supported_sasl_mechanisms.emplace_back( + security::plain_authenticator::name); + if (request.data.mechanism == security::plain_authenticator::name) { + ctx.sasl()->set_mechanism( + std::make_unique( + ctx.credentials())); + } + } + if (supports("GSSAPI")) { supported_sasl_mechanisms.emplace_back( security::gssapi_authenticator::name); From ca1b2ec65e1bf7d5283cb16721d4b58dff98210a Mon Sep 17 00:00:00 2001 From: Ben Pope Date: Tue, 3 Dec 2024 03:37:14 +0000 Subject: [PATCH 169/229] feature_manager: Refactor any_of from absl to std::ranges No change in behaviour Signed-off-by: Ben Pope --- src/v/cluster/feature_manager.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/v/cluster/feature_manager.cc b/src/v/cluster/feature_manager.cc index 97457501c73e..3e6dc81ee8dd 100644 --- a/src/v/cluster/feature_manager.cc +++ b/src/v/cluster/feature_manager.cc @@ -34,9 +34,9 @@ #include -#include #include +#include #include namespace cluster { @@ -208,7 +208,7 @@ feature_manager::report_enterprise_features() const { const auto& cfg = config::shard_local_cfg(); const auto& node_cfg = config::node(); auto has_gssapi = [&cfg]() { - return absl::c_any_of( + return std::ranges::any_of( cfg.sasl_mechanisms(), [](const auto& m) { return m == "GSSAPI"; }); }; auto has_oidc = []() { From e9419b0a25c033aacf9a5019d1ed90c7893846f4 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Mon, 16 Dec 2024 17:33:14 +0000 Subject: [PATCH 170/229] config: bazelize enterprise_property_test --- src/v/config/tests/BUILD | 15 +++++++++++++++ src/v/config/tests/enterprise_property_test.cc | 4 ---- 2 files changed, 15 insertions(+), 4 deletions(-) create mode 100644 src/v/config/tests/BUILD diff --git a/src/v/config/tests/BUILD b/src/v/config/tests/BUILD new file mode 100644 index 000000000000..dca33f7a5cd3 --- /dev/null +++ b/src/v/config/tests/BUILD @@ -0,0 +1,15 @@ +load("//bazel:test.bzl", "redpanda_cc_gtest") + +redpanda_cc_gtest( + name = "enterprise_property_test", + timeout = "short", + srcs = [ + "enterprise_property_test.cc", + ], + cpu = 1, + deps = [ + "//src/v/config", + "//src/v/test_utils:gtest", + "@googletest//:gtest", + ], +) diff --git a/src/v/config/tests/enterprise_property_test.cc b/src/v/config/tests/enterprise_property_test.cc index 3bda3965fba5..a0c27a5658ee 100644 --- a/src/v/config/tests/enterprise_property_test.cc +++ b/src/v/config/tests/enterprise_property_test.cc @@ -14,10 +14,6 @@ #include #include -#include - -#include -#include namespace config { namespace { From fedb768807626abbeb8a575d39f9673e717da71b Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Mon, 16 Dec 2024 17:36:27 +0000 Subject: [PATCH 171/229] config: bazelize bounded_property_test --- src/v/config/tests/BUILD | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/src/v/config/tests/BUILD b/src/v/config/tests/BUILD index dca33f7a5cd3..9e4d9ef01019 100644 --- a/src/v/config/tests/BUILD +++ b/src/v/config/tests/BUILD @@ -1,4 +1,19 @@ -load("//bazel:test.bzl", "redpanda_cc_gtest") +load("//bazel:test.bzl", "redpanda_cc_btest", "redpanda_cc_gtest") + +redpanda_cc_btest( + name = "bounded_property_test", + timeout = "short", + srcs = [ + "bounded_property_test.cc", + ], + cpu = 1, + deps = [ + "//src/v/config", + "//src/v/test_utils:seastar_boost", + "@seastar", + "@seastar//:testing", + ], +) redpanda_cc_gtest( name = "enterprise_property_test", From 1422fd28aa0c798d127ba2a4b4b8a67c007c6342 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Mon, 16 Dec 2024 17:38:51 +0000 Subject: [PATCH 172/229] config: bazelize enum_property_test --- src/v/config/tests/BUILD | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/src/v/config/tests/BUILD b/src/v/config/tests/BUILD index 9e4d9ef01019..5c63561e04a8 100644 --- a/src/v/config/tests/BUILD +++ b/src/v/config/tests/BUILD @@ -15,6 +15,21 @@ redpanda_cc_btest( ], ) +redpanda_cc_btest( + name = "enum_property_test", + timeout = "short", + srcs = [ + "enum_property_test.cc", + ], + cpu = 1, + deps = [ + "//src/v/config", + "//src/v/test_utils:seastar_boost", + "@seastar", + "@seastar//:testing", + ], +) + redpanda_cc_gtest( name = "enterprise_property_test", timeout = "short", From 359fd9100b4a62ad3c873ebb54572f197a80ed4f Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Mon, 16 Dec 2024 17:42:35 +0000 Subject: [PATCH 173/229] config: bazelize retention_property_test --- src/v/config/tests/BUILD | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/src/v/config/tests/BUILD b/src/v/config/tests/BUILD index 5c63561e04a8..0fa8cd865822 100644 --- a/src/v/config/tests/BUILD +++ b/src/v/config/tests/BUILD @@ -30,6 +30,21 @@ redpanda_cc_btest( ], ) +redpanda_cc_btest( + name = "retention_property_test", + timeout = "short", + srcs = [ + "retention_property_test.cc", + ], + cpu = 1, + deps = [ + "//src/v/config", + "//src/v/test_utils:seastar_boost", + "@seastar", + "@seastar//:testing", + ], +) + redpanda_cc_gtest( name = "enterprise_property_test", timeout = "short", From 2043a702d863a9fe64ffb0f8a1c729d189166e4f Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Mon, 16 Dec 2024 17:44:06 +0000 Subject: [PATCH 174/229] config: bazelize config_store_test --- src/v/config/tests/BUILD | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/src/v/config/tests/BUILD b/src/v/config/tests/BUILD index 0fa8cd865822..05b6581ddf51 100644 --- a/src/v/config/tests/BUILD +++ b/src/v/config/tests/BUILD @@ -45,6 +45,22 @@ redpanda_cc_btest( ], ) +redpanda_cc_btest( + name = "config_store_test", + timeout = "short", + srcs = [ + "config_store_test.cc", + ], + cpu = 1, + deps = [ + "//src/v/config", + "//src/v/json", + "//src/v/test_utils:seastar_boost", + "@seastar", + "@seastar//:testing", + ], +) + redpanda_cc_gtest( name = "enterprise_property_test", timeout = "short", From 7add4d6df4581d68c7ee93a1ea27beec5588d775 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Mon, 16 Dec 2024 17:46:24 +0000 Subject: [PATCH 175/229] config: bazelize socket_address_convert_test --- src/v/config/tests/BUILD | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/src/v/config/tests/BUILD b/src/v/config/tests/BUILD index 05b6581ddf51..7355262f1c67 100644 --- a/src/v/config/tests/BUILD +++ b/src/v/config/tests/BUILD @@ -61,6 +61,22 @@ redpanda_cc_btest( ], ) +redpanda_cc_btest( + name = "socket_address_convert_test", + timeout = "short", + srcs = [ + "socket_address_convert_test.cc", + ], + cpu = 1, + deps = [ + "//src/v/config", + "//src/v/test_utils:seastar_boost", + "@seastar", + "@seastar//:testing", + "@yaml-cpp", + ], +) + redpanda_cc_gtest( name = "enterprise_property_test", timeout = "short", From 7aea33337ac75f50dc9b0fc395294da8532222a3 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Mon, 16 Dec 2024 17:48:07 +0000 Subject: [PATCH 176/229] config: bazelize tls_config_convert_test --- src/v/config/tests/BUILD | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/src/v/config/tests/BUILD b/src/v/config/tests/BUILD index 7355262f1c67..73999b1f5e05 100644 --- a/src/v/config/tests/BUILD +++ b/src/v/config/tests/BUILD @@ -77,6 +77,23 @@ redpanda_cc_btest( ], ) +redpanda_cc_btest( + name = "tls_config_convert_test", + timeout = "short", + srcs = [ + "tls_config_convert_test.cc", + ], + cpu = 1, + deps = [ + "//src/v/config", + "//src/v/test_utils:seastar_boost", + "//src/v/utils:to_string", + "@seastar", + "@seastar//:testing", + "@yaml-cpp", + ], +) + redpanda_cc_gtest( name = "enterprise_property_test", timeout = "short", From 6b88b41a31da3facecf5ff6669c186fd769d905e Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Mon, 16 Dec 2024 17:49:46 +0000 Subject: [PATCH 177/229] config: bazelize scoped_config_test --- src/v/config/tests/BUILD | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/src/v/config/tests/BUILD b/src/v/config/tests/BUILD index 73999b1f5e05..14301eae387d 100644 --- a/src/v/config/tests/BUILD +++ b/src/v/config/tests/BUILD @@ -94,6 +94,23 @@ redpanda_cc_btest( ], ) +redpanda_cc_btest( + name = "scoped_config_test", + timeout = "short", + srcs = [ + "scoped_config_test.cc", + ], + cpu = 1, + deps = [ + "//src/v/config", + "//src/v/test_utils:scoped_config", + "//src/v/test_utils:seastar_boost", + "@seastar", + "@seastar//:testing", + "@yaml-cpp", + ], +) + redpanda_cc_gtest( name = "enterprise_property_test", timeout = "short", From 230247a7eb63029ecb135382dfe86e2f76184bfd Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Mon, 16 Dec 2024 17:51:23 +0000 Subject: [PATCH 178/229] config: bazelize advertised_kafka_api_test --- src/v/config/tests/BUILD | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/src/v/config/tests/BUILD b/src/v/config/tests/BUILD index 14301eae387d..5128c9f71724 100644 --- a/src/v/config/tests/BUILD +++ b/src/v/config/tests/BUILD @@ -111,6 +111,21 @@ redpanda_cc_btest( ], ) +redpanda_cc_btest( + name = "advertised_kafka_api_test", + timeout = "short", + srcs = [ + "advertised_kafka_api_test.cc", + ], + cpu = 1, + deps = [ + "//src/v/config", + "//src/v/test_utils:seastar_boost", + "@seastar", + "@seastar//:testing", + ], +) + redpanda_cc_gtest( name = "enterprise_property_test", timeout = "short", From 1d163b9c1775753ba420f03e594721697b772dca Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Mon, 16 Dec 2024 17:55:09 +0000 Subject: [PATCH 179/229] config: bazelize seed_server_property_test --- src/v/config/tests/BUILD | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/src/v/config/tests/BUILD b/src/v/config/tests/BUILD index 5128c9f71724..b020866bf25d 100644 --- a/src/v/config/tests/BUILD +++ b/src/v/config/tests/BUILD @@ -126,6 +126,22 @@ redpanda_cc_btest( ], ) +redpanda_cc_btest( + name = "seed_server_property_test", + timeout = "short", + srcs = [ + "seed_server_property_test.cc", + ], + cpu = 1, + deps = [ + "//src/v/config", + "//src/v/test_utils:seastar_boost", + "//src/v/utils:unresolved_address", + "@seastar", + "@seastar//:testing", + ], +) + redpanda_cc_gtest( name = "enterprise_property_test", timeout = "short", From 83e41494eae7d548ae11b867e39d5daa2f603d5e Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Mon, 16 Dec 2024 17:56:40 +0000 Subject: [PATCH 180/229] config: bazelize cloud_credentials_source_test --- src/v/config/tests/BUILD | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/src/v/config/tests/BUILD b/src/v/config/tests/BUILD index b020866bf25d..4c9c445ee45c 100644 --- a/src/v/config/tests/BUILD +++ b/src/v/config/tests/BUILD @@ -142,6 +142,22 @@ redpanda_cc_btest( ], ) +redpanda_cc_btest( + name = "cloud_credentials_source_test", + timeout = "short", + srcs = [ + "cloud_credentials_source_test.cc", + ], + cpu = 1, + deps = [ + "//src/v/config", + "//src/v/test_utils:seastar_boost", + "@seastar", + "@seastar//:testing", + "@yaml-cpp", + ], +) + redpanda_cc_gtest( name = "enterprise_property_test", timeout = "short", From a5a41111ae4d9cff45603eebfb401a5ecb7066c3 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Mon, 16 Dec 2024 17:58:07 +0000 Subject: [PATCH 181/229] config: bazelize validator_tests --- src/v/config/tests/BUILD | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/src/v/config/tests/BUILD b/src/v/config/tests/BUILD index 4c9c445ee45c..b5380b549f6c 100644 --- a/src/v/config/tests/BUILD +++ b/src/v/config/tests/BUILD @@ -158,6 +158,21 @@ redpanda_cc_btest( ], ) +redpanda_cc_btest( + name = "validator_test", + timeout = "short", + srcs = [ + "validator_tests.cc", + ], + cpu = 1, + deps = [ + "//src/v/config", + "//src/v/test_utils:seastar_boost", + "@seastar", + "@seastar//:testing", + ], +) + redpanda_cc_gtest( name = "enterprise_property_test", timeout = "short", From 31d4676385d7fe9f0a07bf964e1728a95d3f8fae Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Mon, 16 Dec 2024 18:00:07 +0000 Subject: [PATCH 182/229] config: bazelize throughput_control_group_test --- src/v/config/tests/BUILD | 17 +++++++++++++++++ .../tests/throughput_control_group_test.cc | 3 --- 2 files changed, 17 insertions(+), 3 deletions(-) diff --git a/src/v/config/tests/BUILD b/src/v/config/tests/BUILD index b5380b549f6c..79a3c3c79bbe 100644 --- a/src/v/config/tests/BUILD +++ b/src/v/config/tests/BUILD @@ -173,6 +173,23 @@ redpanda_cc_btest( ], ) +redpanda_cc_btest( + name = "throughput_control_group_test", + timeout = "short", + srcs = [ + "throughput_control_group_test.cc", + ], + cpu = 1, + deps = [ + "//src/v/config", + "//src/v/test_utils:seastar_boost", + "@boost//:test", + "@seastar", + "@seastar//:testing", + "@yaml-cpp", + ], +) + redpanda_cc_gtest( name = "enterprise_property_test", timeout = "short", diff --git a/src/v/config/tests/throughput_control_group_test.cc b/src/v/config/tests/throughput_control_group_test.cc index 729ef71eb771..fefb914bf36f 100644 --- a/src/v/config/tests/throughput_control_group_test.cc +++ b/src/v/config/tests/throughput_control_group_test.cc @@ -21,10 +21,7 @@ #include #include -#include -#include #include -#include #include using namespace std::string_literals; From e6846377f4989141e14a94a6c0c3462eb85d7fee Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Mon, 16 Dec 2024 18:02:56 +0000 Subject: [PATCH 183/229] config: bazelize node_override_test --- src/v/config/tests/BUILD | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/src/v/config/tests/BUILD b/src/v/config/tests/BUILD index 79a3c3c79bbe..0c6244dcb1ab 100644 --- a/src/v/config/tests/BUILD +++ b/src/v/config/tests/BUILD @@ -190,6 +190,24 @@ redpanda_cc_btest( ], ) +redpanda_cc_btest( + name = "node_override_test", + timeout = "short", + srcs = [ + "node_override_test.cc", + ], + cpu = 1, + deps = [ + "//src/v/config", + "//src/v/model", + "//src/v/test_utils:seastar_boost", + "//src/v/utils:uuid", + "@seastar", + "@seastar//:testing", + "@yaml-cpp", + ], +) + redpanda_cc_gtest( name = "enterprise_property_test", timeout = "short", From 09a84e8fbcdf9ea569b306e1d40e9ae0dda7a7c0 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Mon, 16 Dec 2024 18:04:25 +0000 Subject: [PATCH 184/229] config: bazelize leaders_preference_test --- src/v/config/tests/BUILD | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/src/v/config/tests/BUILD b/src/v/config/tests/BUILD index 0c6244dcb1ab..aaf5518ea3bf 100644 --- a/src/v/config/tests/BUILD +++ b/src/v/config/tests/BUILD @@ -208,6 +208,21 @@ redpanda_cc_btest( ], ) +redpanda_cc_btest( + name = "leaders_preference_test", + timeout = "short", + srcs = [ + "leaders_preference_test.cc", + ], + cpu = 1, + deps = [ + "//src/v/config", + "//src/v/test_utils:seastar_boost", + "@seastar", + "@seastar//:testing", + ], +) + redpanda_cc_gtest( name = "enterprise_property_test", timeout = "short", From 067fc7b90da47a2c24d6eb96abc1ac0925610cf0 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Thu, 12 Dec 2024 18:36:43 +0000 Subject: [PATCH 185/229] pandaproxy: add semamphore usage metrics --- src/v/pandaproxy/probe.cc | 75 +++++++++++++++++++++ src/v/pandaproxy/probe.h | 19 ++++++ src/v/pandaproxy/rest/proxy.cc | 9 ++- src/v/pandaproxy/schema_registry/service.cc | 9 ++- src/v/pandaproxy/server.cc | 18 +++-- src/v/pandaproxy/server.h | 9 ++- 6 files changed, 127 insertions(+), 12 deletions(-) diff --git a/src/v/pandaproxy/probe.cc b/src/v/pandaproxy/probe.cc index a5dd7bb237d3..396156ff531c 100644 --- a/src/v/pandaproxy/probe.cc +++ b/src/v/pandaproxy/probe.cc @@ -104,4 +104,79 @@ void probe::setup_public_metrics() { .aggregate(aggregate_labels)}); } +server_probe::server_probe( + server::context_t& ctx, const ss::sstring& group_name) + : _ctx(ctx) + , _group_name(group_name) + , _metrics() + , _public_metrics() { + setup_metrics(); +} + +void server_probe::setup_metrics() { + namespace sm = ss::metrics; + + auto setup_common = [this]() { + const auto usage = [](const size_t current, const size_t max) { + constexpr double divide_by_zero = -1.; + constexpr double invalid_values = -2.; + if (max == 0) { + return divide_by_zero; + } + if (current > max) { + return invalid_values; + } + const auto max_d = static_cast(max); + const auto current_d = static_cast(current); + return (max_d - current_d) / max_d; + }; + + std::vector defs; + defs.reserve(3); + defs.emplace_back( + sm::make_gauge( + "inflight_requests_usage_ratio", + [this, usage] { + return usage(_ctx.inflight_sem.current(), _ctx.max_inflight); + }, + sm::description(ssx::sformat( + "Usage ratio of in-flight requests in the {}", _group_name))) + .aggregate({})); + defs.emplace_back( + sm::make_gauge( + "inflight_requests_memory_usage_ratio", + [this, usage] { + return usage(_ctx.mem_sem.current(), _ctx.max_memory); + }, + sm::description(ssx::sformat( + "Memory usage ratio of in-flight requests in the {}", + _group_name))) + .aggregate({})); + defs.emplace_back( + sm::make_gauge( + "queued_requests_memory_blocked", + [this] { return _ctx.mem_sem.waiters(); }, + sm::description(ssx::sformat( + "Number of requests queued in {}, due to memory limitations", + _group_name))) + .aggregate({})); + return defs; + }; + + if (!config::shard_local_cfg().disable_metrics()) { + _metrics.add_group( + _group_name, + setup_common + .template operator()(), + {}, + {}); + } + + if (!config::shard_local_cfg().disable_public_metrics()) { + _public_metrics.add_group( + _group_name, + setup_common.template operator()()); + } +} + } // namespace pandaproxy diff --git a/src/v/pandaproxy/probe.h b/src/v/pandaproxy/probe.h index 70270b2b6708..97a251b32f1f 100644 --- a/src/v/pandaproxy/probe.h +++ b/src/v/pandaproxy/probe.h @@ -12,6 +12,7 @@ #pragma once #include "metrics/metrics.h" +#include "pandaproxy/server.h" #include "utils/log_hist.h" #include @@ -82,4 +83,22 @@ class probe { metrics::public_metric_groups _public_metrics; }; +class server_probe { +public: + server_probe(server::context_t& ctx, const ss::sstring& group_name); + server_probe(const server_probe&) = delete; + server_probe& operator=(const server_probe&) = delete; + server_probe(server_probe&&) = delete; + server_probe& operator=(server_probe&&) = delete; + ~server_probe() = default; + +private: + void setup_metrics(); + + server::context_t& _ctx; + const ss::sstring& _group_name; + metrics::internal_metric_groups _metrics; + metrics::public_metric_groups _public_metrics; +}; + } // namespace pandaproxy diff --git a/src/v/pandaproxy/rest/proxy.cc b/src/v/pandaproxy/rest/proxy.cc index 911c4aef1e66..7eb5eca96f08 100644 --- a/src/v/pandaproxy/rest/proxy.cc +++ b/src/v/pandaproxy/rest/proxy.cc @@ -113,7 +113,7 @@ proxy::proxy( , _inflight_config_binding(config::shard_local_cfg().max_in_flight_pandaproxy_requests_per_shard.bind()) , _client(client) , _client_cache(client_cache) - , _ctx{{{{}, max_memory, _mem_sem, _inflight_sem, {}, smp_sg}, *this}, + , _ctx{{{{}, max_memory, _mem_sem, _inflight_config_binding(), _inflight_sem, {}, smp_sg}, *this}, {config::always_true(), config::shard_local_cfg().superusers.bind(), controller}, _config.pandaproxy_api.value()} , _server( @@ -126,8 +126,11 @@ proxy::proxy( json::serialization_format::application_json) , _ensure_started{[this]() { return do_start(); }} , _controller(controller) { - _inflight_config_binding.watch( - [this]() { _inflight_sem.set_capacity(_inflight_config_binding()); }); + _inflight_config_binding.watch([this]() { + const size_t capacity = _inflight_config_binding(); + _inflight_sem.set_capacity(capacity); + _ctx.max_inflight = capacity; + }); } ss::future<> proxy::start() { diff --git a/src/v/pandaproxy/schema_registry/service.cc b/src/v/pandaproxy/schema_registry/service.cc index 77502d30c5e7..056f38a73386 100644 --- a/src/v/pandaproxy/schema_registry/service.cc +++ b/src/v/pandaproxy/schema_registry/service.cc @@ -614,7 +614,7 @@ service::service( config::shard_local_cfg() .max_in_flight_schema_registry_requests_per_shard.bind()) , _client(client) - , _ctx{{{}, max_memory, _mem_sem, _inflight_sem, {}, smp_sg}, *this} + , _ctx{{{}, max_memory, _mem_sem, _inflight_config_binding(), _inflight_sem, {}, smp_sg}, *this} , _server( "schema_registry", // server_name "schema_registry", // public_metric_group_name @@ -632,8 +632,11 @@ service::service( config::always_true(), config::shard_local_cfg().superusers.bind(), controller.get()} { - _inflight_config_binding.watch( - [this]() { _inflight_sem.set_capacity(_inflight_config_binding()); }); + _inflight_config_binding.watch([this]() { + const size_t capacity = _inflight_config_binding(); + _inflight_sem.set_capacity(capacity); + _ctx.max_inflight = capacity; + }); } ss::future<> service::start() { diff --git a/src/v/pandaproxy/server.cc b/src/v/pandaproxy/server.cc index ea8ab73cfa63..09b3072829bd 100644 --- a/src/v/pandaproxy/server.cc +++ b/src/v/pandaproxy/server.cc @@ -28,6 +28,7 @@ #include #include +#include namespace pandaproxy { @@ -156,7 +157,8 @@ server::server( , _api20(std::move(api20)) , _has_routes(false) , _ctx(ctx) - , _exceptional_mime_type(exceptional_mime_type) { + , _exceptional_mime_type(exceptional_mime_type) + , _probe{} { _api20.set_api_doc(_server._routes); _api20.register_api_file(_server._routes, header); _api20.add_definitions_file(_server._routes, definitions); @@ -201,6 +203,9 @@ ss::future<> server::start( const std::vector& advertised) { _server._routes.register_exeption_handler( exception_replier{ss::sstring{name(_exceptional_mime_type)}}); + + _probe = std::make_unique(_ctx, _public_metrics_group_name); + _ctx.advertised_listeners.reserve(endpoints.size()); for (auto& server_endpoint : endpoints) { auto addr = co_await net::resolve_dns(server_endpoint.address); @@ -240,13 +245,18 @@ ss::future<> server::start( } co_await _server.listen(addr, cred); } + co_return; } ss::future<> server::stop() { - return _pending_reqs.close() - .finally([this]() { return _ctx.as.request_abort(); }) - .finally([this]() mutable { return _server.stop(); }); + return _pending_reqs.close().finally([this]() { + _ctx.as.request_abort(); + _probe.reset(nullptr); + return _server.stop(); + }); } +server::~server() noexcept = default; + } // namespace pandaproxy diff --git a/src/v/pandaproxy/server.h b/src/v/pandaproxy/server.h index 7bca466fc052..e854b0a8e86a 100644 --- a/src/v/pandaproxy/server.h +++ b/src/v/pandaproxy/server.h @@ -26,6 +26,7 @@ #include #include #include +#include #include #include #include @@ -40,6 +41,8 @@ namespace pandaproxy { +class server_probe; + inline ss::shard_id user_shard(const ss::sstring& name) { auto hash = xxhash_64(name.data(), name.length()); return jump_consistent_hash(hash, ss::smp::count); @@ -71,6 +74,7 @@ class server { std::vector advertised_listeners; size_t max_memory; ssx::semaphore& mem_sem; + size_t max_inflight; adjustable_semaphore& inflight_sem; ss::abort_source as; ss::smp_service_group smp_sg; @@ -104,9 +108,9 @@ class server { }; server() = delete; - ~server() = default; + ~server() noexcept; server(const server&) = delete; - server(server&&) noexcept = default; + server(server&&) noexcept = delete; server& operator=(const server&) = delete; server& operator=(server&&) = delete; @@ -136,6 +140,7 @@ class server { bool _has_routes; context_t& _ctx; json::serialization_format _exceptional_mime_type; + std::unique_ptr _probe; }; template From 27a6f15bfd28d1e130866bd201a338e21a435cf1 Mon Sep 17 00:00:00 2001 From: Stephan Dollberg Date: Mon, 16 Dec 2024 11:41:24 +0000 Subject: [PATCH 186/229] omb: Limit cloud read throughput in TS read test With the switch to m7gd we are now maxing out disk throughput when catching up from the backlog. This makes produce latency go bad. To avoid this limit cloud readthroughput (as we do in cloud). We use the same formula as in the cloud: 150MB/s / 12 * 2 (throughput / total_shard_count * 2). --- tests/rptest/perf/ts_read_openmessaging_test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/rptest/perf/ts_read_openmessaging_test.py b/tests/rptest/perf/ts_read_openmessaging_test.py index e09f79101f27..26aad6febeb2 100644 --- a/tests/rptest/perf/ts_read_openmessaging_test.py +++ b/tests/rptest/perf/ts_read_openmessaging_test.py @@ -23,6 +23,8 @@ def __init__(self, ctx): "retention_local_strict": True, "retention_local_target_bytes_default": 16 * 1_000_000, # 16 MB "cloud_storage_spillover_manifest_size": None, + # approximate limit for m7gd.xlarge + "cloud_storage_max_throughput_per_shard": 25 * 1_000_000, } si_settings = SISettings( test_context=ctx, From 5b515d3860160a45ad1db429fc33667b2d052eba Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Tue, 17 Dec 2024 15:32:20 +0000 Subject: [PATCH 187/229] pandaproxy: bazelize json/requests/test/fetch test --- src/v/kafka/client/test/BUILD | 1 + src/v/pandaproxy/json/requests/test/BUILD | 22 ++++++++++++++++++++++ 2 files changed, 23 insertions(+) create mode 100644 src/v/pandaproxy/json/requests/test/BUILD diff --git a/src/v/kafka/client/test/BUILD b/src/v/kafka/client/test/BUILD index 0a9797c4ccc5..c7ef249a04b4 100644 --- a/src/v/kafka/client/test/BUILD +++ b/src/v/kafka/client/test/BUILD @@ -24,6 +24,7 @@ redpanda_test_cc_library( "//src/v/random:generators", ], include_prefix = "kafka/client/test", + visibility = ["//visibility:public"], deps = [ "//src/v/reflection:adl", "//src/v/storage:record_batch_builder", diff --git a/src/v/pandaproxy/json/requests/test/BUILD b/src/v/pandaproxy/json/requests/test/BUILD new file mode 100644 index 000000000000..d3d38d07af63 --- /dev/null +++ b/src/v/pandaproxy/json/requests/test/BUILD @@ -0,0 +1,22 @@ +load("//bazel:test.bzl", "redpanda_cc_btest") + +redpanda_cc_btest( + name = "fetch", + timeout = "short", + srcs = [ + "fetch.cc", + ], + deps = [ + "//src/v/base", + "//src/v/container:fragmented_vector", + "//src/v/json", + "//src/v/kafka/client/test:utils", + "//src/v/kafka/protocol", + "//src/v/model", + "//src/v/pandaproxy", + "//src/v/test_utils:seastar_boost", + "@boost//:test", + "@seastar", + "@seastar//:testing", + ], +) From f16fb6be15c59ed86424c8bb87903de5e8836752 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Tue, 17 Dec 2024 15:38:51 +0000 Subject: [PATCH 188/229] pandaproxy: bazelize json/requests/test/produce test --- src/v/pandaproxy/json/requests/test/BUILD | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/src/v/pandaproxy/json/requests/test/BUILD b/src/v/pandaproxy/json/requests/test/BUILD index d3d38d07af63..af2aa0a3f709 100644 --- a/src/v/pandaproxy/json/requests/test/BUILD +++ b/src/v/pandaproxy/json/requests/test/BUILD @@ -20,3 +20,22 @@ redpanda_cc_btest( "@seastar//:testing", ], ) + +redpanda_cc_btest( + name = "produce", + timeout = "short", + srcs = [ + "produce.cc", + ], + deps = [ + "//src/v/base", + "//src/v/kafka/protocol:produce", + "//src/v/kafka/protocol/schemata:produce_response", + "//src/v/model", + "//src/v/pandaproxy", + "//src/v/test_utils:seastar_boost", + "//src/v/utils:to_string", + "@seastar", + "@seastar//:testing", + ], +) From 57ef5040177918b9ffd6be0f6e3edd208efa6b44 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Tue, 17 Dec 2024 15:45:26 +0000 Subject: [PATCH 189/229] pandaproxy: bazelize json/test/iobuf test --- src/v/pandaproxy/json/test/BUILD | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) create mode 100644 src/v/pandaproxy/json/test/BUILD diff --git a/src/v/pandaproxy/json/test/BUILD b/src/v/pandaproxy/json/test/BUILD new file mode 100644 index 000000000000..e967ee8ffe32 --- /dev/null +++ b/src/v/pandaproxy/json/test/BUILD @@ -0,0 +1,17 @@ +load("//bazel:test.bzl", "redpanda_cc_btest") + +redpanda_cc_btest( + name = "iobuf", + timeout = "short", + srcs = [ + "iobuf.cc", + ], + deps = [ + "//src/v/bytes:iobuf_parser", + "//src/v/json", + "//src/v/pandaproxy", + "//src/v/test_utils:seastar_boost", + "@seastar", + "@seastar//:testing", + ], +) From 7e4e84cb74fd18350a953a2c56db7124804211e1 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Tue, 17 Dec 2024 15:52:02 +0000 Subject: [PATCH 190/229] pandaproxy: bazelize parsing/test/from_chars test --- src/v/pandaproxy/parsing/test/BUILD | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) create mode 100644 src/v/pandaproxy/parsing/test/BUILD diff --git a/src/v/pandaproxy/parsing/test/BUILD b/src/v/pandaproxy/parsing/test/BUILD new file mode 100644 index 000000000000..2f0963e657c3 --- /dev/null +++ b/src/v/pandaproxy/parsing/test/BUILD @@ -0,0 +1,18 @@ +load("//bazel:test.bzl", "redpanda_cc_btest") + +redpanda_cc_btest( + name = "from_chars", + timeout = "short", + srcs = [ + "from_chars.cc", + ], + deps = [ + "//src/v/pandaproxy", + "//src/v/test_utils:seastar_boost", + "//src/v/utils:named_type", + "@boost//:mpl", + "@boost//:test", + "@seastar", + "@seastar//:testing", + ], +) From 56d258146f3d2180b3d5072f7b250b0b1dcde7dc Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Tue, 17 Dec 2024 15:53:55 +0000 Subject: [PATCH 191/229] pandaproxy: bazelize parsing/test/httpd test --- src/v/pandaproxy/parsing/test/BUILD | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/src/v/pandaproxy/parsing/test/BUILD b/src/v/pandaproxy/parsing/test/BUILD index 2f0963e657c3..4ce8458808c8 100644 --- a/src/v/pandaproxy/parsing/test/BUILD +++ b/src/v/pandaproxy/parsing/test/BUILD @@ -16,3 +16,20 @@ redpanda_cc_btest( "@seastar//:testing", ], ) + +redpanda_cc_btest( + name = "httpd", + timeout = "short", + srcs = [ + "httpd.cc", + ], + deps = [ + "//src/v/pandaproxy", + "//src/v/test_utils:seastar_boost", + "@boost//:test", + "@boost//:tuple", + "@boost//:utility", + "@seastar", + "@seastar//:testing", + ], +) From 82f40f45315a20bee1cb69214465e6e31fba6a0a Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Tue, 17 Dec 2024 17:03:15 +0000 Subject: [PATCH 192/229] pandaproxy: bazelize rest/test/consumer_group test --- src/v/pandaproxy/rest/test/BUILD | 23 ++++++++++++++++++++++ src/v/pandaproxy/test/BUILD | 16 +++++++++++++++ src/v/pandaproxy/test/pandaproxy_fixture.h | 5 ----- 3 files changed, 39 insertions(+), 5 deletions(-) create mode 100644 src/v/pandaproxy/rest/test/BUILD diff --git a/src/v/pandaproxy/rest/test/BUILD b/src/v/pandaproxy/rest/test/BUILD new file mode 100644 index 000000000000..96ae5cf92bb5 --- /dev/null +++ b/src/v/pandaproxy/rest/test/BUILD @@ -0,0 +1,23 @@ +load("//bazel:test.bzl", "redpanda_cc_btest") + +redpanda_cc_btest( + name = "consumer_group", + timeout = "short", + srcs = [ + "consumer_group.cc", + ], + tags = ["exclusive"], + deps = [ + "//src/v/http", + "//src/v/kafka/protocol:join_group", + "//src/v/pandaproxy", + "//src/v/pandaproxy/test:fixture", + "//src/v/pandaproxy/test:utils", + "//src/v/test_utils:seastar_boost", + "//src/v/utils:unresolved_address", + "@boost//:beast", + "@boost//:test", + "@seastar", + "@seastar//:testing", + ], +) diff --git a/src/v/pandaproxy/test/BUILD b/src/v/pandaproxy/test/BUILD index 3fe96e84b65a..dae11606383a 100644 --- a/src/v/pandaproxy/test/BUILD +++ b/src/v/pandaproxy/test/BUILD @@ -14,6 +14,22 @@ redpanda_test_cc_library( ], ) +redpanda_test_cc_library( + name = "fixture", + hdrs = [ + "pandaproxy_fixture.h", + ], + implementation_deps = [ + ], + include_prefix = "pandaproxy/test", + visibility = ["//visibility:public"], + deps = [ + "//src/v/http", + "//src/v/pandaproxy", + "//src/v/redpanda/tests:fixture_btest", + ], +) + redpanda_cc_btest( name = "errors_test", timeout = "short", diff --git a/src/v/pandaproxy/test/pandaproxy_fixture.h b/src/v/pandaproxy/test/pandaproxy_fixture.h index 904f33dc5069..c4fb0f7e1286 100644 --- a/src/v/pandaproxy/test/pandaproxy_fixture.h +++ b/src/v/pandaproxy/test/pandaproxy_fixture.h @@ -11,12 +11,7 @@ #pragma once -#include "config/configuration.h" #include "http/client.h" -#include "kafka/client/client.h" -#include "kafka/client/configuration.h" -#include "kafka/protocol/metadata.h" -#include "pandaproxy/rest/configuration.h" #include "redpanda/tests/fixture.h" class pandaproxy_test_fixture : public redpanda_thread_fixture { From be5aca7bc142459afa95f5f2c4672af44682357c Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Tue, 17 Dec 2024 17:06:33 +0000 Subject: [PATCH 193/229] pandaproxy: bazelize rest/test/fetch test --- src/v/pandaproxy/rest/test/BUILD | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/src/v/pandaproxy/rest/test/BUILD b/src/v/pandaproxy/rest/test/BUILD index 96ae5cf92bb5..0f3eb8e17e9c 100644 --- a/src/v/pandaproxy/rest/test/BUILD +++ b/src/v/pandaproxy/rest/test/BUILD @@ -21,3 +21,23 @@ redpanda_cc_btest( "@seastar//:testing", ], ) + +redpanda_cc_btest( + name = "fetch", + timeout = "short", + srcs = [ + "fetch.cc", + ], + tags = ["exclusive"], + deps = [ + "//src/v/http", + "//src/v/pandaproxy", + "//src/v/pandaproxy/test:fixture", + "//src/v/pandaproxy/test:utils", + "//src/v/test_utils:seastar_boost", + "@boost//:beast", + "@boost//:test", + "@seastar", + "@seastar//:testing", + ], +) From 8ef4e31150f32494a63d50448900e987daa0bb5f Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Tue, 17 Dec 2024 17:08:11 +0000 Subject: [PATCH 194/229] pandaproxy: bazelize rest/test/list_topics test --- src/v/pandaproxy/rest/test/BUILD | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/src/v/pandaproxy/rest/test/BUILD b/src/v/pandaproxy/rest/test/BUILD index 0f3eb8e17e9c..1554c208c944 100644 --- a/src/v/pandaproxy/rest/test/BUILD +++ b/src/v/pandaproxy/rest/test/BUILD @@ -41,3 +41,22 @@ redpanda_cc_btest( "@seastar//:testing", ], ) + +redpanda_cc_btest( + name = "list_topics", + timeout = "short", + srcs = [ + "list_topics.cc", + ], + tags = ["exclusive"], + deps = [ + "//src/v/http", + "//src/v/pandaproxy/test:fixture", + "//src/v/pandaproxy/test:utils", + "//src/v/test_utils:seastar_boost", + "@boost//:beast", + "@boost//:test", + "@seastar", + "@seastar//:testing", + ], +) From 0c63b53e7429c57edf2f9dec5983d170e1261961 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Tue, 17 Dec 2024 17:11:10 +0000 Subject: [PATCH 195/229] pandaproxy: bazelize rest/test/produce test --- src/v/pandaproxy/rest/test/BUILD | 19 +++++++++++++++++++ src/v/pandaproxy/rest/test/produce.cc | 1 - 2 files changed, 19 insertions(+), 1 deletion(-) diff --git a/src/v/pandaproxy/rest/test/BUILD b/src/v/pandaproxy/rest/test/BUILD index 1554c208c944..c0aef904701a 100644 --- a/src/v/pandaproxy/rest/test/BUILD +++ b/src/v/pandaproxy/rest/test/BUILD @@ -60,3 +60,22 @@ redpanda_cc_btest( "@seastar//:testing", ], ) + +redpanda_cc_btest( + name = "produce", + timeout = "short", + srcs = [ + "produce.cc", + ], + tags = ["exclusive"], + deps = [ + "//src/v/http", + "//src/v/pandaproxy/test:fixture", + "//src/v/pandaproxy/test:utils", + "//src/v/test_utils:seastar_boost", + "@boost//:beast", + "@boost//:test", + "@seastar", + "@seastar//:testing", + ], +) diff --git a/src/v/pandaproxy/rest/test/produce.cc b/src/v/pandaproxy/rest/test/produce.cc index 0e2c431f2482..2088b881bd7a 100644 --- a/src/v/pandaproxy/rest/test/produce.cc +++ b/src/v/pandaproxy/rest/test/produce.cc @@ -8,7 +8,6 @@ // by the Apache License, Version 2.0 #include "http/client.h" -#include "pandaproxy/rest/configuration.h" #include "pandaproxy/test/pandaproxy_fixture.h" #include "pandaproxy/test/utils.h" From efdfbae42660d45ba10e63dfd95dd149d4d20b42 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Tue, 17 Dec 2024 17:15:57 +0000 Subject: [PATCH 196/229] pandaproxy: bazelize sr/requests/test/get_subject_versions_version --- .../schema_registry/requests/test/BUILD | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) create mode 100644 src/v/pandaproxy/schema_registry/requests/test/BUILD diff --git a/src/v/pandaproxy/schema_registry/requests/test/BUILD b/src/v/pandaproxy/schema_registry/requests/test/BUILD new file mode 100644 index 000000000000..44a83922ef1d --- /dev/null +++ b/src/v/pandaproxy/schema_registry/requests/test/BUILD @@ -0,0 +1,16 @@ +load("//bazel:test.bzl", "redpanda_cc_btest") + +redpanda_cc_btest( + name = "get_subject_versions_version", + timeout = "short", + srcs = [ + "get_subject_versions_version.cc", + ], + deps = [ + "//src/v/base", + "//src/v/pandaproxy", + "//src/v/test_utils:seastar_boost", + "@seastar", + "@seastar//:testing", + ], +) From eac95ccae6fd6a114ddde898db806453be3f9b9e Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Tue, 17 Dec 2024 17:17:48 +0000 Subject: [PATCH 197/229] pandaproxy: bazelize sr/requests/test/post_subject_versions --- .../schema_registry/requests/test/BUILD | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/src/v/pandaproxy/schema_registry/requests/test/BUILD b/src/v/pandaproxy/schema_registry/requests/test/BUILD index 44a83922ef1d..3917262c592a 100644 --- a/src/v/pandaproxy/schema_registry/requests/test/BUILD +++ b/src/v/pandaproxy/schema_registry/requests/test/BUILD @@ -14,3 +14,19 @@ redpanda_cc_btest( "@seastar//:testing", ], ) + +redpanda_cc_btest( + name = "post_subject_versions", + timeout = "short", + srcs = [ + "post_subject_versions.cc", + ], + deps = [ + "//src/v/base", + "//src/v/pandaproxy", + "//src/v/test_utils:seastar_boost", + "@fmt", + "@seastar", + "@seastar//:testing", + ], +) From 8a4f49539d2246915708d9ecc92ae173ad5ad0f9 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Tue, 17 Dec 2024 17:35:30 +0000 Subject: [PATCH 198/229] pandaproxy: bazelize sr/test/get_schema_types test --- src/v/pandaproxy/schema_registry/test/BUILD | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/src/v/pandaproxy/schema_registry/test/BUILD b/src/v/pandaproxy/schema_registry/test/BUILD index 9e492ab340d0..3c386a1a629e 100644 --- a/src/v/pandaproxy/schema_registry/test/BUILD +++ b/src/v/pandaproxy/schema_registry/test/BUILD @@ -227,3 +227,22 @@ redpanda_cc_btest( "@seastar//:testing", ], ) + +redpanda_cc_btest( + name = "get_schema_types", + timeout = "short", + srcs = [ + "get_schema_types.cc", + ], + tags = ["exclusive"], + deps = [ + "//src/v/http", + "//src/v/pandaproxy/test:fixture", + "//src/v/pandaproxy/test:utils", + "//src/v/test_utils:seastar_boost", + "@boost//:beast", + "@boost//:test", + "@seastar", + "@seastar//:testing", + ], +) From 5f48741194c84efc19811e56249fb4e31b8d7c15 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Tue, 17 Dec 2024 20:28:22 +0100 Subject: [PATCH 199/229] c/leader_balancer: improve log message wording --- src/v/cluster/scheduling/leader_balancer.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/v/cluster/scheduling/leader_balancer.cc b/src/v/cluster/scheduling/leader_balancer.cc index 828294fff4ef..5b6fcead6aef 100644 --- a/src/v/cluster/scheduling/leader_balancer.cc +++ b/src/v/cluster/scheduling/leader_balancer.cc @@ -732,7 +732,7 @@ ss::future leader_balancer::balance() { vlog( clusterlog.info, "balancer iteration finished: " - "total error: {:.4}, number of muted groups: {}, " + "objective: {:.4}, number of muted groups: {}, " "number in flight: {}, dispatched in this tick: {}", strategy->error(), _muted.size(), From 57184137c8478ec28fbce1daaead60c6d9632e54 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Tue, 17 Dec 2024 17:44:05 +0000 Subject: [PATCH 200/229] pandaproxy: bazelize sr/test/post_subjects_subject_version test --- src/v/pandaproxy/schema_registry/test/BUILD | 46 +++++++++++++++++++ .../schema_registry/test/avro_payloads.h | 3 -- 2 files changed, 46 insertions(+), 3 deletions(-) diff --git a/src/v/pandaproxy/schema_registry/test/BUILD b/src/v/pandaproxy/schema_registry/test/BUILD index 3c386a1a629e..29bdf8eebecc 100644 --- a/src/v/pandaproxy/schema_registry/test/BUILD +++ b/src/v/pandaproxy/schema_registry/test/BUILD @@ -1,5 +1,32 @@ load("//bazel:test.bzl", "redpanda_cc_btest", "redpanda_test_cc_library") +redpanda_test_cc_library( + name = "avro_payloads", + hdrs = [ + "avro_payloads.h", + ], + include_prefix = "pandaproxy/schema_registry/test", + deps = [ + "//src/v/pandaproxy", + ], +) + +redpanda_test_cc_library( + name = "client_utils", + hdrs = [ + "client_utils.h", + ], + include_prefix = "pandaproxy/schema_registry/test", + deps = [ + "//src/v/http", + "//src/v/json", + "//src/v/pandaproxy", + "//src/v/pandaproxy/test:utils", + "@abseil-cpp//absl/algorithm:container", + "@boost//:beast", + ], +) + redpanda_test_cc_library( name = "compatibility_avro", hdrs = [ @@ -246,3 +273,22 @@ redpanda_cc_btest( "@seastar//:testing", ], ) + +redpanda_cc_btest( + name = "post_subjects_subject_version", + timeout = "short", + srcs = [ + "post_subjects_subject_version.cc", + ], + tags = ["exclusive"], + deps = [ + ":avro_payloads", + ":client_utils", + "//src/v/pandaproxy", + "//src/v/pandaproxy/test:fixture", + "//src/v/pandaproxy/test:utils", + "//src/v/test_utils:seastar_boost", + "@seastar", + "@seastar//:testing", + ], +) diff --git a/src/v/pandaproxy/schema_registry/test/avro_payloads.h b/src/v/pandaproxy/schema_registry/test/avro_payloads.h index 556af0089042..b25c10b68317 100644 --- a/src/v/pandaproxy/schema_registry/test/avro_payloads.h +++ b/src/v/pandaproxy/schema_registry/test/avro_payloads.h @@ -11,9 +11,6 @@ #include "pandaproxy/schema_registry/avro.h" -namespace pp = pandaproxy; -namespace pps = pp::schema_registry; - const ss::sstring avro_string_payload{ R"( { From 3bc01f5d168b6b6c294af5d34190e9ea68a89387 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Tue, 17 Dec 2024 17:46:12 +0000 Subject: [PATCH 201/229] pandaproxy: bazelize sr/test/delete_subject_endpoints --- src/v/pandaproxy/schema_registry/test/BUILD | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/src/v/pandaproxy/schema_registry/test/BUILD b/src/v/pandaproxy/schema_registry/test/BUILD index 29bdf8eebecc..5084a96d4df2 100644 --- a/src/v/pandaproxy/schema_registry/test/BUILD +++ b/src/v/pandaproxy/schema_registry/test/BUILD @@ -292,3 +292,24 @@ redpanda_cc_btest( "@seastar//:testing", ], ) + +redpanda_cc_btest( + name = "delete_subject_endpoints", + timeout = "short", + srcs = [ + "delete_subject_endpoints.cc", + ], + tags = ["exclusive"], + deps = [ + ":avro_payloads", + ":client_utils", + "//src/v/pandaproxy", + "//src/v/pandaproxy/test:fixture", + "//src/v/pandaproxy/test:utils", + "//src/v/test_utils:seastar_boost", + "@boost//:beast", + "@boost//:test", + "@seastar", + "@seastar//:testing", + ], +) From 8b8fef2e6467782288d1e2fcaf72f9091e1de1c8 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Tue, 17 Dec 2024 17:48:35 +0000 Subject: [PATCH 202/229] pandaproxy: add missing single-cpu argument to sr tests --- src/v/pandaproxy/schema_registry/test/BUILD | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/v/pandaproxy/schema_registry/test/BUILD b/src/v/pandaproxy/schema_registry/test/BUILD index 5084a96d4df2..74c153da063c 100644 --- a/src/v/pandaproxy/schema_registry/test/BUILD +++ b/src/v/pandaproxy/schema_registry/test/BUILD @@ -159,6 +159,7 @@ redpanda_cc_btest( srcs = [ "sharded_store.cc", ], + cpu = 1, deps = [ "//src/v/pandaproxy", "//src/v/pandaproxy/schema_registry/test:compatibility_protobuf", @@ -175,6 +176,7 @@ redpanda_cc_btest( srcs = [ "consume_to_store.cc", ], + cpu = 1, deps = [ "//src/v/model", "//src/v/pandaproxy", @@ -191,6 +193,7 @@ redpanda_cc_btest( srcs = [ "compatibility_store.cc", ], + cpu = 1, deps = [ "//src/v/pandaproxy", "//src/v/pandaproxy/schema_registry/test:compatibility_avro", @@ -207,6 +210,7 @@ redpanda_cc_btest( srcs = [ "compatibility_3rdparty.cc", ], + cpu = 1, deps = [ "//src/v/model", "//src/v/pandaproxy", @@ -223,6 +227,7 @@ redpanda_cc_btest( srcs = [ "compatibility_avro.cc", ], + cpu = 1, deps = [ "//src/v/pandaproxy", "//src/v/pandaproxy/schema_registry/test:compatibility_avro", @@ -242,6 +247,7 @@ redpanda_cc_btest( srcs = [ "test_json_schema.cc", ], + cpu = 1, deps = [ "//src/v/pandaproxy", "//src/v/pandaproxy/schema_registry/test:compatibility_common", From 588ec77af989fa04233fb3aea64e01527c52bc39 Mon Sep 17 00:00:00 2001 From: Ben Pope Date: Tue, 3 Dec 2024 03:20:03 +0000 Subject: [PATCH 203/229] feature_manager: Log bad SASL/PLAIN config REQ-02 Signed-off-by: Ben Pope --- src/v/cluster/feature_manager.cc | 46 ++++++++++++++++---- src/v/cluster/feature_manager.h | 4 +- tests/rptest/tests/rbac_test.py | 18 +++----- tests/rptest/tests/redpanda_kerberos_test.py | 18 +++----- tests/rptest/tests/redpanda_oauth_test.py | 20 ++++----- tests/rptest/tests/redpanda_startup_test.py | 12 ++--- tests/rptest/tests/schema_registry_test.py | 18 +++----- tests/rptest/utils/log_utils.py | 25 +++++++++++ 8 files changed, 98 insertions(+), 63 deletions(-) create mode 100644 tests/rptest/utils/log_utils.py diff --git a/src/v/cluster/feature_manager.cc b/src/v/cluster/feature_manager.cc index 3e6dc81ee8dd..5df81b80ad2f 100644 --- a/src/v/cluster/feature_manager.cc +++ b/src/v/cluster/feature_manager.cc @@ -20,7 +20,9 @@ #include "cluster/logger.h" #include "cluster/members_table.h" #include "config/configuration.h" +#include "config/endpoint_tls_config.h" #include "config/node_config.h" +#include "config/tls_config.h" #include "config/types.h" #include "config/validators.h" #include "features/enterprise_feature_messages.h" @@ -184,7 +186,7 @@ feature_manager::start(std::vector&& cluster_founder_nodes) { ssx::background = ssx::spawn_with_gate_then(_gate, [this] { return ss::do_until( [this] { return _as.local().abort_requested(); }, - [this] { return maybe_log_license_check_info(); }); + [this] { return maybe_log_periodic_reminders(); }); }); for (const model::node_id n : cluster_founder_nodes) { @@ -274,32 +276,38 @@ feature_manager::report_enterprise_features() const { return report; } -ss::future<> feature_manager::maybe_log_license_check_info() { - auto license_check_retry = std::chrono::seconds(60 * 5); +ss::future<> feature_manager::maybe_log_periodic_reminders() { + auto reminder_period = std::chrono::seconds(60 * 5); auto interval_override = std::getenv( "__REDPANDA_LICENSE_CHECK_INTERVAL_SEC"); if (interval_override != nullptr) { try { - license_check_retry = std::min( - std::chrono::seconds{license_check_retry}, + reminder_period = std::min( + std::chrono::seconds{reminder_period}, std::chrono::seconds{std::stoi(interval_override)}); vlog( clusterlog.info, - "Overriding default license log annoy interval to: {}s", - license_check_retry.count()); + "Overriding default reminder period interval to: {}s", + reminder_period.count()); } catch (...) { vlog( clusterlog.error, - "Invalid license check interval override '{}'", + "Invalid reminder period interval override '{}'", interval_override); } } try { - co_await ss::sleep_abortable(license_check_retry, _as.local()); + co_await ss::sleep_abortable(reminder_period, _as.local()); } catch (const ss::sleep_aborted&) { // Shutting down - next iteration will drop out co_return; } + + maybe_log_license_nag(); + maybe_log_security_nag(); +} + +void feature_manager::maybe_log_license_nag() { auto enterprise_features = report_enterprise_features(); if (enterprise_features.any()) { if (_feature_table.local().should_sanction()) { @@ -312,6 +320,26 @@ ss::future<> feature_manager::maybe_log_license_check_info() { } } +void feature_manager::maybe_log_security_nag() { + if (std::ranges::any_of( + config::shard_local_cfg().sasl_mechanisms(), + [](const auto& m) { return m == "PLAIN"; })) { + const bool any_tls_disabled + = std::ranges::any_of( + config::node_config().kafka_api_tls.value(), + [](const config::endpoint_tls_config& cfg) { + return !cfg.config.is_enabled(); + }) + || config::node_config().kafka_api_tls.value().empty(); + + vlogl( + clusterlog, + any_tls_disabled ? ss::log_level::error : ss::log_level::warn, + "SASL/PLAIN is enabled. This is insecure and not recommended for " + "production."); + } +} + bool feature_manager::need_to_verify_enterprise_license() { return features::is_major_version_upgrade( _feature_table.local().get_active_version(), diff --git a/src/v/cluster/feature_manager.h b/src/v/cluster/feature_manager.h index c2a4deea6e4e..d6ebfe94912e 100644 --- a/src/v/cluster/feature_manager.h +++ b/src/v/cluster/feature_manager.h @@ -142,7 +142,9 @@ class feature_manager { && _am_controller_leader; } - ss::future<> maybe_log_license_check_info(); + ss::future<> maybe_log_periodic_reminders(); + void maybe_log_license_nag(); + void maybe_log_security_nag(); bool need_to_verify_enterprise_license(); // Compose a command struct, replicate it via raft and wait for apply. diff --git a/tests/rptest/tests/rbac_test.py b/tests/rptest/tests/rbac_test.py index 28c6f35d7740..66f6bb462310 100644 --- a/tests/rptest/tests/rbac_test.py +++ b/tests/rptest/tests/rbac_test.py @@ -30,6 +30,7 @@ from rptest.tests.admin_api_auth_test import create_user_and_wait from rptest.tests.metrics_reporter_test import MetricsReporterServer from rptest.util import expect_exception, expect_http_error, wait_until_result +from rptest.utils.log_utils import wait_until_nag_is_set from rptest.utils.mode_checks import skip_fips_mode ALICE = SaslCredentials("alice", "itsMeH0nest", "SCRAM-SHA-256") @@ -633,17 +634,12 @@ def __init__(self, test_ctx, **kwargs): f'{self.LICENSE_CHECK_INTERVAL_SEC}' }) - def _license_nag_is_set(self): - return self.redpanda.search_log_all( - f"Overriding default license log annoy interval to: {self.LICENSE_CHECK_INTERVAL_SEC}s" - ) - @cluster(num_nodes=1) @skip_fips_mode # See NOTE below def test_license_nag(self): - wait_until(self._license_nag_is_set, - timeout_sec=30, - err_msg="Failed to set license nag internal") + wait_until_nag_is_set( + redpanda=self.redpanda, + check_interval_sec=self.LICENSE_CHECK_INTERVAL_SEC) self.logger.debug("Ensuring no license nag") time.sleep(self.LICENSE_CHECK_INTERVAL_SEC * 2) @@ -658,9 +654,9 @@ def test_license_nag(self): {'__REDPANDA_DISABLE_BUILTIN_TRIAL_LICENSE': '1'}) self.redpanda.rolling_restart_nodes(self.redpanda.nodes, use_maintenance_mode=False) - wait_until(self._license_nag_is_set, - timeout_sec=30, - err_msg="Failed to set license nag internal") + wait_until_nag_is_set( + redpanda=self.redpanda, + check_interval_sec=self.LICENSE_CHECK_INTERVAL_SEC) self.logger.debug("Waiting for license nag") wait_until(self.redpanda.has_license_nag, diff --git a/tests/rptest/tests/redpanda_kerberos_test.py b/tests/rptest/tests/redpanda_kerberos_test.py index ee56a0864ca4..a764651201ea 100644 --- a/tests/rptest/tests/redpanda_kerberos_test.py +++ b/tests/rptest/tests/redpanda_kerberos_test.py @@ -23,6 +23,7 @@ from rptest.services.kerberos import KrbKdc, KrbClient, RedpandaKerberosNode, AuthenticationError, KRB5_CONF_PATH, render_krb5_config, ActiveDirectoryKdc from rptest.services.redpanda import LoggingConfig, RedpandaService, SecurityConfig from rptest.tests.sasl_reauth_test import get_sasl_metrics, REAUTH_METRIC, EXPIRATION_METRIC +from rptest.utils.log_utils import wait_until_nag_is_set from rptest.utils.mode_checks import skip_fips_mode from rptest.utils.rpenv import IsCIOrNotEmpty @@ -170,17 +171,12 @@ def __init__(self, test_context, num_nodes=3, **kwargs): f'{self.LICENSE_CHECK_INTERVAL_SEC}', }) - def _license_nag_is_set(self): - return self.redpanda.search_log_all( - f"Overriding default license log annoy interval to: {self.LICENSE_CHECK_INTERVAL_SEC}s" - ) - @cluster(num_nodes=3) @skip_fips_mode # See NOTE below def test_license_nag(self): - wait_until(self._license_nag_is_set, - timeout_sec=30, - err_msg="Failed to set license nag internal") + wait_until_nag_is_set( + redpanda=self.redpanda, + check_interval_sec=self.LICENSE_CHECK_INTERVAL_SEC) self.logger.debug("Ensuring no license nag") time.sleep(self.LICENSE_CHECK_INTERVAL_SEC * 2) @@ -197,9 +193,9 @@ def test_license_nag(self): self.redpanda.stop() self.redpanda.start(clean_nodes=False) - wait_until(self._license_nag_is_set, - timeout_sec=30, - err_msg="Failed to set license nag internal") + wait_until_nag_is_set( + redpanda=self.redpanda, + check_interval_sec=self.LICENSE_CHECK_INTERVAL_SEC) self.logger.debug("Waiting for license nag") wait_until(self.redpanda.has_license_nag, diff --git a/tests/rptest/tests/redpanda_oauth_test.py b/tests/rptest/tests/redpanda_oauth_test.py index 41bcf8c0eaa5..b28a983044b0 100644 --- a/tests/rptest/tests/redpanda_oauth_test.py +++ b/tests/rptest/tests/redpanda_oauth_test.py @@ -20,9 +20,10 @@ from rptest.services.cluster import cluster from rptest.services.tls import TLSCertManager from rptest.tests.sasl_reauth_test import get_sasl_metrics, REAUTH_METRIC, EXPIRATION_METRIC +from rptest.tests.tls_metrics_test import FaketimeTLSProvider from rptest.util import expect_exception +from rptest.utils.log_utils import wait_until_nag_is_set from rptest.utils.mode_checks import skip_fips_mode -from rptest.tests.tls_metrics_test import FaketimeTLSProvider import requests import time @@ -612,19 +613,14 @@ def __init__(self, test_context, num_nodes=3, **kwargs): f'{self.LICENSE_CHECK_INTERVAL_SEC}', }) - def _license_nag_is_set(self): - return self.redpanda.search_log_all( - f"Overriding default license log annoy interval to: {self.LICENSE_CHECK_INTERVAL_SEC}s" - ) - @cluster(num_nodes=3) @skip_fips_mode # See NOTE below @parametrize(authn_config={"sasl_mechanisms": ["OAUTHBEARER", "SCRAM"]}) @parametrize(authn_config={"http_authentication": ["OIDC", "BASIC"]}) def test_license_nag(self, authn_config): - wait_until(self._license_nag_is_set, - timeout_sec=30, - err_msg="Failed to set license nag internal") + wait_until_nag_is_set( + redpanda=self.redpanda, + check_interval_sec=self.LICENSE_CHECK_INTERVAL_SEC) self.logger.debug("Ensuring no license nag") time.sleep(self.LICENSE_CHECK_INTERVAL_SEC * 2) @@ -640,9 +636,9 @@ def test_license_nag(self, authn_config): self.redpanda.rolling_restart_nodes(self.redpanda.nodes, use_maintenance_mode=False) - wait_until(self._license_nag_is_set, - timeout_sec=30, - err_msg="Failed to set license nag internal") + wait_until_nag_is_set( + redpanda=self.redpanda, + check_interval_sec=self.LICENSE_CHECK_INTERVAL_SEC) self.logger.debug("Waiting for license nag") wait_until(self.redpanda.has_license_nag, diff --git a/tests/rptest/tests/redpanda_startup_test.py b/tests/rptest/tests/redpanda_startup_test.py index 8256adccff9d..818f16403b6f 100644 --- a/tests/rptest/tests/redpanda_startup_test.py +++ b/tests/rptest/tests/redpanda_startup_test.py @@ -17,6 +17,7 @@ from rptest.services.cluster import cluster from rptest.services.redpanda import MetricsEndpoint, MetricSamples, RedpandaServiceBase from rptest.tests.redpanda_test import RedpandaTest +from rptest.utils.log_utils import wait_until_nag_is_set from rptest.utils.mode_checks import in_fips_environment @@ -249,17 +250,12 @@ def __init__(self, test_context): True }) - def _license_nag_is_set(self) -> bool: - return self.redpanda.search_log_all( - f"Overriding default license log annoy interval to: {self.LICENSE_CHECK_INTERVAL_SEC}s" - ) - @ignore # https://redpandadata.atlassian.net/browse/CORE-4283 @cluster(num_nodes=3) def test_fips_license_nag(self): - wait_until(self._license_nag_is_set, - timeout_sec=30, - err_msg="Failed to set license nag interval") + wait_until_nag_is_set( + redpanda=self.redpanda, + check_interval_sec=self.LICENSE_CHECK_INTERVAL_SEC) self.logger.debug("Ensuring no license nag") sleep(self.LICENSE_CHECK_INTERVAL_SEC * 2) diff --git a/tests/rptest/tests/schema_registry_test.py b/tests/rptest/tests/schema_registry_test.py index 53c57544a1fe..31e1b04a0730 100644 --- a/tests/rptest/tests/schema_registry_test.py +++ b/tests/rptest/tests/schema_registry_test.py @@ -39,6 +39,7 @@ from rptest.tests.pandaproxy_test import User, PandaProxyTLSProvider from rptest.tests.redpanda_test import RedpandaTest from rptest.util import expect_exception, inject_remote_script, search_logs_with_timeout +from rptest.utils.log_utils import wait_until_nag_is_set from rptest.utils.mode_checks import skip_fips_mode @@ -3843,19 +3844,14 @@ def __init__(self, *args, **kwargs): f'{self.LICENSE_CHECK_INTERVAL_SEC}', }) - def _license_nag_is_set(self): - return self.redpanda.search_log_all( - f"Overriding default license log annoy interval to: {self.LICENSE_CHECK_INTERVAL_SEC}s" - ) - @cluster(num_nodes=3) @skip_fips_mode # See NOTE below @parametrize(mode=SchemaIdValidationMode.REDPANDA) @parametrize(mode=SchemaIdValidationMode.COMPAT) def test_license_nag(self, mode): - wait_until(self._license_nag_is_set, - timeout_sec=30, - err_msg="Failed to set license nag internal") + wait_until_nag_is_set( + redpanda=self.redpanda, + check_interval_sec=self.LICENSE_CHECK_INTERVAL_SEC) self.logger.debug("Ensuring no license nag") time.sleep(self.LICENSE_CHECK_INTERVAL_SEC * 2) @@ -3871,9 +3867,9 @@ def test_license_nag(self, mode): self.redpanda.rolling_restart_nodes(self.redpanda.nodes, use_maintenance_mode=False) - wait_until(self._license_nag_is_set, - timeout_sec=30, - err_msg="Failed to set license nag internal") + wait_until_nag_is_set( + redpanda=self.redpanda, + check_interval_sec=self.LICENSE_CHECK_INTERVAL_SEC) self.logger.debug("Waiting for license nag") wait_until(self.redpanda.has_license_nag, diff --git a/tests/rptest/utils/log_utils.py b/tests/rptest/utils/log_utils.py new file mode 100644 index 000000000000..933ae5989833 --- /dev/null +++ b/tests/rptest/utils/log_utils.py @@ -0,0 +1,25 @@ +from ducktape.utils.util import wait_until + +from rptest.services.redpanda import RedpandaService + + +def wait_until_nag_is_set(redpanda: RedpandaService, + check_interval_sec: int, + timeout_sec: int = 30): + """ + Waits until the log message indicating override of the periodic reminder + interval has been set + + Parameters: + redpanda (RedpandaService): Redpanda service instance to query the logs from + check_interval_sec (int): The interval in seconds that should be logged by Redpanda + timeotu_sec (int): The maximum time to wait for the log message to appear + """ + def nag_check() -> bool: + return redpanda.search_log_all( + f"Overriding default reminder period interval to: {check_interval_sec}s" + ) + + wait_until(nag_check, + timeout_sec=timeout_sec, + err_msg="Failed to set periodic reminder interval") From ef9dcc57d81ab8afe93c6e0205112b8c3307a95d Mon Sep 17 00:00:00 2001 From: Michael Boquard Date: Mon, 16 Dec 2024 15:10:01 -0500 Subject: [PATCH 204/229] fm/dt: Renamed __REDPANDA_LICENSE_CHECK_INTERVAL_SEC Renamed `__REDPANDA_LICENSE_CHECK_INTERVAL_SEC` with `__REDPANDA_PERIODIC_REMINDER_INTERVAL_SEC` to better reflect what messages this environmental variable is controlling. Signed-off-by: Michael Boquard --- src/v/cluster/feature_manager.cc | 2 +- tests/rptest/tests/rbac_test.py | 2 +- tests/rptest/tests/rbac_upgrade_test.py | 2 +- tests/rptest/tests/redpanda_kerberos_test.py | 2 +- tests/rptest/tests/redpanda_oauth_test.py | 2 +- tests/rptest/tests/redpanda_startup_test.py | 2 +- tests/rptest/tests/schema_registry_test.py | 2 +- tests/rptest/tests/workload_license.py | 4 ++-- 8 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/v/cluster/feature_manager.cc b/src/v/cluster/feature_manager.cc index 5df81b80ad2f..2cd2aa05af09 100644 --- a/src/v/cluster/feature_manager.cc +++ b/src/v/cluster/feature_manager.cc @@ -279,7 +279,7 @@ feature_manager::report_enterprise_features() const { ss::future<> feature_manager::maybe_log_periodic_reminders() { auto reminder_period = std::chrono::seconds(60 * 5); auto interval_override = std::getenv( - "__REDPANDA_LICENSE_CHECK_INTERVAL_SEC"); + "__REDPANDA_PERIODIC_REMINDER_INTERVAL_SEC"); if (interval_override != nullptr) { try { reminder_period = std::min( diff --git a/tests/rptest/tests/rbac_test.py b/tests/rptest/tests/rbac_test.py index 66f6bb462310..3f472e311248 100644 --- a/tests/rptest/tests/rbac_test.py +++ b/tests/rptest/tests/rbac_test.py @@ -630,7 +630,7 @@ class RBACLicenseTest(RBACTestBase): def __init__(self, test_ctx, **kwargs): super().__init__(test_ctx, **kwargs) self.redpanda.set_environment({ - '__REDPANDA_LICENSE_CHECK_INTERVAL_SEC': + '__REDPANDA_PERIODIC_REMINDER_INTERVAL_SEC': f'{self.LICENSE_CHECK_INTERVAL_SEC}' }) diff --git a/tests/rptest/tests/rbac_upgrade_test.py b/tests/rptest/tests/rbac_upgrade_test.py index 8bc41b63d150..c1a6f2d5f3ca 100644 --- a/tests/rptest/tests/rbac_upgrade_test.py +++ b/tests/rptest/tests/rbac_upgrade_test.py @@ -28,7 +28,7 @@ class UpgradeMigrationCreatingDefaultRole(RedpandaTest): def __init__(self, test_ctx, **kwargs): super().__init__(test_ctx, **kwargs) self.redpanda.set_environment({ - '__REDPANDA_LICENSE_CHECK_INTERVAL_SEC': + '__REDPANDA_PERIODIC_REMINDER_INTERVAL_SEC': f'{self.LICENSE_CHECK_INTERVAL_SEC}', '__REDPANDA_DISABLE_BUILTIN_TRIAL_LICENSE': True diff --git a/tests/rptest/tests/redpanda_kerberos_test.py b/tests/rptest/tests/redpanda_kerberos_test.py index a764651201ea..8532db5b4d97 100644 --- a/tests/rptest/tests/redpanda_kerberos_test.py +++ b/tests/rptest/tests/redpanda_kerberos_test.py @@ -167,7 +167,7 @@ def __init__(self, test_context, num_nodes=3, **kwargs): sasl_mechanisms=["SCRAM"], **kwargs) self.redpanda.set_environment({ - '__REDPANDA_LICENSE_CHECK_INTERVAL_SEC': + '__REDPANDA_PERIODIC_REMINDER_INTERVAL_SEC': f'{self.LICENSE_CHECK_INTERVAL_SEC}', }) diff --git a/tests/rptest/tests/redpanda_oauth_test.py b/tests/rptest/tests/redpanda_oauth_test.py index b28a983044b0..70498c42c309 100644 --- a/tests/rptest/tests/redpanda_oauth_test.py +++ b/tests/rptest/tests/redpanda_oauth_test.py @@ -609,7 +609,7 @@ def __init__(self, test_context, num_nodes=3, **kwargs): http_authentication=["BASIC"], **kwargs) self.redpanda.set_environment({ - '__REDPANDA_LICENSE_CHECK_INTERVAL_SEC': + '__REDPANDA_PERIODIC_REMINDER_INTERVAL_SEC': f'{self.LICENSE_CHECK_INTERVAL_SEC}', }) diff --git a/tests/rptest/tests/redpanda_startup_test.py b/tests/rptest/tests/redpanda_startup_test.py index 818f16403b6f..a020b6337a65 100644 --- a/tests/rptest/tests/redpanda_startup_test.py +++ b/tests/rptest/tests/redpanda_startup_test.py @@ -244,7 +244,7 @@ def __init__(self, test_context): fips_mode=RedpandaServiceBase.FIPSMode.disabled) self.redpanda.set_environment({ - '__REDPANDA_LICENSE_CHECK_INTERVAL_SEC': + '__REDPANDA_PERIODIC_REMINDER_INTERVAL_SEC': f'{self.LICENSE_CHECK_INTERVAL_SEC}', '__REDPANDA_DISABLE_BUILTIN_TRIAL_LICENSE': True diff --git a/tests/rptest/tests/schema_registry_test.py b/tests/rptest/tests/schema_registry_test.py index 31e1b04a0730..75de78d1a1b2 100644 --- a/tests/rptest/tests/schema_registry_test.py +++ b/tests/rptest/tests/schema_registry_test.py @@ -3840,7 +3840,7 @@ def __init__(self, *args, **kwargs): schema_registry_config=SchemaRegistryConfig(), **kwargs) self.redpanda.set_environment({ - '__REDPANDA_LICENSE_CHECK_INTERVAL_SEC': + '__REDPANDA_PERIODIC_REMINDER_INTERVAL_SEC': f'{self.LICENSE_CHECK_INTERVAL_SEC}', }) diff --git a/tests/rptest/tests/workload_license.py b/tests/rptest/tests/workload_license.py index 41c2850305c2..47ffd47639be 100644 --- a/tests/rptest/tests/workload_license.py +++ b/tests/rptest/tests/workload_license.py @@ -43,7 +43,7 @@ def begin(self): return self.ctx.redpanda.set_environment({ - '__REDPANDA_LICENSE_CHECK_INTERVAL_SEC': + '__REDPANDA_PERIODIC_REMINDER_INTERVAL_SEC': f'{LicenseWorkload.LICENSE_CHECK_INTERVAL_SEC}' }) @@ -107,7 +107,7 @@ def on_cluster_upgraded(self, version: tuple[int, int, int]) -> int: # Install license assert admin.put_license(self.license).status_code == 200 self.ctx.redpanda.unset_environment( - ['__REDPANDA_LICENSE_CHECK_INTERVAL_SEC']) + ['__REDPANDA_PERIODIC_REMINDER_INTERVAL_SEC']) self.license_installed = True return PWorkload.DONE From 176570ea3f19898a635d980f365df3f52c9aa354 Mon Sep 17 00:00:00 2001 From: Michael Boquard Date: Mon, 9 Dec 2024 09:22:09 -0500 Subject: [PATCH 205/229] dt: Added 'PLAIN' to list of simple SASL mechanisms This change allows for the selection of SASL/PLAIN as an authentication method using the Python RdKafka library. Signed-off-by: Michael Boquard --- tests/rptest/services/redpanda_types.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/rptest/services/redpanda_types.py b/tests/rptest/services/redpanda_types.py index 202f7184834e..ca484b4beb53 100644 --- a/tests/rptest/services/redpanda_types.py +++ b/tests/rptest/services/redpanda_types.py @@ -59,7 +59,7 @@ def __str__(self): return self.name -SIMPLE_SASL_MECHANISMS = ['SCRAM-SHA-256', 'SCRAM-SHA-512'] +SIMPLE_SASL_MECHANISMS = ['PLAIN', 'SCRAM-SHA-256', 'SCRAM-SHA-512'] COMPLEX_SASL_MECHANISMS = ['GSSAPI', 'OAUTHBEARER'] From fe9df923773520d2acaa26fb92e2cab714fe4447 Mon Sep 17 00:00:00 2001 From: Brandon Allard Date: Mon, 18 Nov 2024 01:29:25 -0500 Subject: [PATCH 206/229] treewide: refactor avro data_generator --- src/v/datalake/tests/BUILD | 2 +- src/v/datalake/tests/datalake_avro_tests.cc | 4 +- src/v/datalake/tests/record_generator.cc | 8 +-- src/v/datalake/tests/record_generator.h | 4 +- src/v/serde/avro/tests/data_generator.cc | 56 +++++++-------------- src/v/serde/avro/tests/data_generator.h | 26 +++++++--- src/v/serde/avro/tests/parser_test.cc | 16 +++--- 7 files changed, 54 insertions(+), 62 deletions(-) diff --git a/src/v/datalake/tests/BUILD b/src/v/datalake/tests/BUILD index f3ec6640bb19..5300d189ac20 100644 --- a/src/v/datalake/tests/BUILD +++ b/src/v/datalake/tests/BUILD @@ -101,7 +101,6 @@ redpanda_test_cc_library( implementation_deps = [ "//src/v/container:chunked_hash_map", "//src/v/schema:registry", - "//src/v/serde/avro/tests:data_generator", "@avro", ], include_prefix = "datalake/tests", @@ -112,6 +111,7 @@ redpanda_test_cc_library( "//src/v/container:fragmented_vector", "//src/v/model", "//src/v/pandaproxy", + "//src/v/serde/avro/tests:data_generator", "//src/v/storage:record_batch_builder", "//src/v/utils:named_type", "@seastar", diff --git a/src/v/datalake/tests/datalake_avro_tests.cc b/src/v/datalake/tests/datalake_avro_tests.cc index 2554ae517b5b..e2de9adea1ce 100644 --- a/src/v/datalake/tests/datalake_avro_tests.cc +++ b/src/v/datalake/tests/datalake_avro_tests.cc @@ -509,8 +509,8 @@ prepare_avro_test(std::string_view schema) { // Convert to iceberg schema auto iceberg_struct_res = datalake::type_to_iceberg(valid_schema.root()); // Generate random generic datum - generator_state state{0}; - avro::GenericDatum datum = generate_datum(valid_schema.root(), state, 10); + avro_generator gen({}); + avro::GenericDatum datum = gen.generate_datum(valid_schema.root()); // Serialize using avro library auto buffer = serialize_with_avro(datum, valid_schema); diff --git a/src/v/datalake/tests/record_generator.cc b/src/v/datalake/tests/record_generator.cc index fa0f50e0af41..b3dcdfb1ef41 100644 --- a/src/v/datalake/tests/record_generator.cc +++ b/src/v/datalake/tests/record_generator.cc @@ -10,7 +10,6 @@ #include "datalake/tests/record_generator.h" #include "schema/registry.h" -#include "serde/avro/tests/data_generator.h" #include "storage/record_batch_builder.h" #include @@ -45,7 +44,8 @@ ss::future> record_generator::add_random_avro_record( storage::record_batch_builder& b, std::string_view name, - std::optional key) { + std::optional key, + testing::avro_generator_config config) { using namespace pandaproxy::schema_registry; auto it = _id_by_name.find(name); if (it == _id_by_name.end()) { @@ -83,8 +83,8 @@ record_generator::add_random_avro_record( co_return error{ fmt::format("Schema {} didn't resolve Avro node", name)}; } - testing::generator_state gs; - auto datum = generate_datum(node_ptr, gs, 10); + testing::avro_generator gen(config); + auto datum = gen.generate_datum(node_ptr); std::unique_ptr out = avro::memoryOutputStream(); avro::EncoderPtr e = avro::binaryEncoder(); e->init(*out); diff --git a/src/v/datalake/tests/record_generator.h b/src/v/datalake/tests/record_generator.h index 7d961ad6a07d..f830af72e6a8 100644 --- a/src/v/datalake/tests/record_generator.h +++ b/src/v/datalake/tests/record_generator.h @@ -15,6 +15,7 @@ #include "model/record.h" #include "model/timestamp.h" #include "pandaproxy/schema_registry/types.h" +#include "serde/avro/tests/data_generator.h" #include "storage/record_batch_builder.h" #include "utils/named_type.h" @@ -40,7 +41,8 @@ class record_generator { ss::future> add_random_avro_record( storage::record_batch_builder&, std::string_view schema_name, - std::optional key); + std::optional key, + testing::avro_generator_config config = {}); private: chunked_hash_map diff --git a/src/v/serde/avro/tests/data_generator.cc b/src/v/serde/avro/tests/data_generator.cc index 370ead6075be..f4d18f27725a 100644 --- a/src/v/serde/avro/tests/data_generator.cc +++ b/src/v/serde/avro/tests/data_generator.cc @@ -28,21 +28,19 @@ std::vector generate_decimal(size_t max_size = 16) { return bytes; } -::avro::GenericDatum generate_datum( - const avro::NodePtr& node, - generator_state& state, - int max_nesting_level, - std::optional elements_in_collection) { - state.level++; - - auto decrement_level = ss::defer([&state] { state.level--; }); +ss::sstring random_string(const avro_generator_config& config) { + auto [min, max] = config.string_length_range; + return random_generators::gen_alphanum_string( + random_generators::get_int(min, max)); +} - auto get_elements_count = - [&state, elements_in_collection, max_nesting_level]() -> size_t { - if (state.level >= max_nesting_level) { +::avro::GenericDatum +avro_generator::generate_datum_impl(int level, const avro::NodePtr& node) { + auto get_elements_count = [this, level]() -> size_t { + if (level >= _config.max_nesting_level) { return 0; } - return elements_in_collection.value_or( + return _config.elements_in_collection.value_or( random_generators::get_int(10)); }; ::avro::GenericDatum datum{node}; @@ -52,8 +50,7 @@ ::avro::GenericDatum generate_datum( datum.value() = fmt::to_string(uuid_t::create()); return datum; } - auto v = random_generators::gen_alphanum_string( - random_generators::get_int(32)); + auto v = random_string(_config); datum.value() = v; return datum; } @@ -113,11 +110,7 @@ ::avro::GenericDatum generate_datum( case avro::AVRO_RECORD: { ::avro::GenericRecord record{node}; for (size_t i = 0; i < record.fieldCount(); ++i) { - record.fieldAt(i) = generate_datum( - node->leafAt(i), - state, - max_nesting_level, - elements_in_collection); + record.fieldAt(i) = generate_datum_impl(level + 1, node->leafAt(i)); } return {node, record}; } @@ -126,11 +119,8 @@ ::avro::GenericDatum generate_datum( auto sz = get_elements_count(); for (size_t i = 0; i < sz; ++i) { - array.value().push_back(generate_datum( - array.schema()->leafAt(0), - state, - max_nesting_level, - elements_in_collection)); + array.value().push_back( + generate_datum_impl(level + 1, array.schema()->leafAt(0))); } return {node, array}; } @@ -142,26 +132,20 @@ ::avro::GenericDatum generate_datum( random_generators::get_int(16)); map.value().emplace_back( - key, - generate_datum( - map.schema()->leafAt(1), - state, - max_nesting_level, - elements_in_collection)); + key, generate_datum_impl(level + 1, map.schema()->leafAt(1))); } return {node, map}; } case avro::AVRO_SYMBOLIC: { auto resolved = ::avro::resolveSymbol(node); - return generate_datum( - resolved, state, max_nesting_level, elements_in_collection); + return generate_datum_impl(level + 1, resolved); } case avro::AVRO_UNION: { ::avro::GenericUnion u{node}; int branch = 0; // some of the unions are recursive, prevent infinite recursion by // choosing a plain type instead of the record - if (state.level >= max_nesting_level) { + if (level >= _config.max_nesting_level) { for (size_t i = 0; i < node->leaves(); i++) { if (node->leafAt(i)->type() != avro::AVRO_RECORD) { branch = i; @@ -173,11 +157,7 @@ ::avro::GenericDatum generate_datum( } u.selectBranch(branch); - u.datum() = generate_datum( - u.schema()->leafAt(branch), - state, - max_nesting_level, - elements_in_collection); + u.datum() = generate_datum_impl(level + 1, u.schema()->leafAt(branch)); return {node, u}; } case avro::AVRO_FIXED: { diff --git a/src/v/serde/avro/tests/data_generator.h b/src/v/serde/avro/tests/data_generator.h index be391da04e0e..731c42515301 100644 --- a/src/v/serde/avro/tests/data_generator.h +++ b/src/v/serde/avro/tests/data_generator.h @@ -19,13 +19,25 @@ namespace testing { -struct generator_state { - int level{0}; +struct avro_generator_config { + std::optional elements_in_collection{}; + std::pair string_length_range{0, 32}; + int max_nesting_level{10}; +}; + +class avro_generator { +public: + explicit avro_generator(avro_generator_config c) + : _config(c) {} + + avro::GenericDatum generate_datum(const avro::NodePtr& node) { + return generate_datum_impl(1, node); + } + +private: + avro::GenericDatum + generate_datum_impl(int level, const avro::NodePtr& node); + avro_generator_config _config; }; -avro::GenericDatum generate_datum( - const avro::NodePtr& node, - generator_state& state, - int max_nesting_level, - std::optional elements_in_collection = std::nullopt); } // namespace testing diff --git a/src/v/serde/avro/tests/parser_test.cc b/src/v/serde/avro/tests/parser_test.cc index 68b68f1bb069..69cace05285e 100644 --- a/src/v/serde/avro/tests/parser_test.cc +++ b/src/v/serde/avro/tests/parser_test.cc @@ -305,9 +305,9 @@ TEST_P(AvroParserTest, RoundtripTest) { for (int i = 0; i < 500; ++i) { // Generate random value - generator_state state; - ::avro::GenericDatum random_value = generate_datum( - valid_schema.root(), state, 10); + avro_generator gen({}); + ::avro::GenericDatum random_value = gen.generate_datum( + valid_schema.root()); // serialize data with AVRO library iobuf buffer = serialize_with_avro(random_value, valid_schema); // read using serde::avro @@ -358,9 +358,8 @@ INSTANTIATE_TEST_SUITE_P( TEST_F(AvroParserTest, TestTooManyBytes) { auto valid_schema = load_json_schema("record2"); - generator_state state; - ::avro::GenericDatum random_value = generate_datum( - valid_schema.root(), state, 10); + avro_generator gen({}); + ::avro::GenericDatum random_value = gen.generate_datum(valid_schema.root()); iobuf buffer = serialize_with_avro(random_value, valid_schema); buffer.append(random_generators::make_iobuf(128)); @@ -415,9 +414,8 @@ bool try_deserialize_with_avro_lib( TEST_F(AvroParserTest, TestIncorrectSchema) { auto valid_schema = load_json_schema("record2"); - generator_state state; - ::avro::GenericDatum random_value = generate_datum( - valid_schema.root(), state, 10); + avro_generator gen({}); + ::avro::GenericDatum random_value = gen.generate_datum(valid_schema.root()); iobuf buffer = serialize_with_avro(random_value, valid_schema); auto invalid_schema = load_json_schema("tree2"); auto success = try_deserialize_with_avro_lib(invalid_schema, buffer); From 2e24c6f203533cdc4df3b37ffa18f45e77ace184 Mon Sep 17 00:00:00 2001 From: Brandon Allard Date: Mon, 18 Nov 2024 01:31:22 -0500 Subject: [PATCH 207/229] serde/protobuf: add protobuf data_generator --- src/v/serde/protobuf/CMakeLists.txt | 2 + src/v/serde/protobuf/tests/BUILD | 20 ++- src/v/serde/protobuf/tests/CMakeLists.txt | 12 ++ src/v/serde/protobuf/tests/data_generator.cc | 175 +++++++++++++++++++ src/v/serde/protobuf/tests/data_generator.h | 48 +++++ 5 files changed, 256 insertions(+), 1 deletion(-) create mode 100644 src/v/serde/protobuf/tests/CMakeLists.txt create mode 100644 src/v/serde/protobuf/tests/data_generator.cc create mode 100644 src/v/serde/protobuf/tests/data_generator.h diff --git a/src/v/serde/protobuf/CMakeLists.txt b/src/v/serde/protobuf/CMakeLists.txt index 1df0e397f9d8..60b29749ca7d 100644 --- a/src/v/serde/protobuf/CMakeLists.txt +++ b/src/v/serde/protobuf/CMakeLists.txt @@ -11,3 +11,5 @@ v_cc_library( v::utils protobuf::libprotobuf ) + +add_subdirectory(tests) diff --git a/src/v/serde/protobuf/tests/BUILD b/src/v/serde/protobuf/tests/BUILD index 53ec7e5b7f45..f46063e93031 100644 --- a/src/v/serde/protobuf/tests/BUILD +++ b/src/v/serde/protobuf/tests/BUILD @@ -1,5 +1,5 @@ load("@rules_proto//proto:defs.bzl", "proto_library") -load("//bazel:test.bzl", "redpanda_cc_gtest") +load("//bazel:test.bzl", "redpanda_cc_gtest", "redpanda_test_cc_library") proto_library( name = "two_proto", @@ -58,3 +58,21 @@ redpanda_cc_gtest( "@seastar", ], ) + +redpanda_test_cc_library( + name = "data_generator", + srcs = [ + "data_generator.cc", + ], + hdrs = [ + "data_generator.h", + ], + implementation_deps = [ + "//src/v/random:generators", + ], + include_prefix = "serde/protobuf/tests", + visibility = ["//visibility:public"], + deps = [ + "@protobuf", + ], +) diff --git a/src/v/serde/protobuf/tests/CMakeLists.txt b/src/v/serde/protobuf/tests/CMakeLists.txt new file mode 100644 index 000000000000..e63431b82f52 --- /dev/null +++ b/src/v/serde/protobuf/tests/CMakeLists.txt @@ -0,0 +1,12 @@ +find_package(Protobuf REQUIRED) + +v_cc_library( + NAME protobuf_test_utils + HDRS + data_generator.h + SRCS + data_generator.cc + DEPS + protobuf::libprotobuf + v::random +) diff --git a/src/v/serde/protobuf/tests/data_generator.cc b/src/v/serde/protobuf/tests/data_generator.cc new file mode 100644 index 000000000000..f8c4402e6555 --- /dev/null +++ b/src/v/serde/protobuf/tests/data_generator.cc @@ -0,0 +1,175 @@ +/* + * Copyright 2024 Redpanda Data, Inc. + * + * Use of this software is governed by the Business Source License + * included in the file licenses/BSL.md + * + * As of the Change Date specified in that file, in accordance with + * the Business Source License, use of this software will be governed + * by the Apache License, Version 2.0 + */ + +#include "serde/protobuf/tests/data_generator.h" + +#include "random/generators.h" + +#include +#include + +namespace testing { +ss::sstring random_string(const protobuf_generator_config& config) { + auto [min, max] = config.string_length_range; + return random_generators::gen_alphanum_string( + random_generators::get_int(min, max)); +} + +std::unique_ptr +protobuf_generator::generate_protobuf_message_impl( + int level, const google::protobuf::Descriptor* d) { + auto& factory = _factory; + auto get_elements_count = [level, this]() -> size_t { + if (level >= _config.max_nesting_level) { + return 0; + } + return _config.elements_in_collection.value_or( + random_generators::get_int(10)); + }; + + auto message = std::unique_ptr( + factory.GetPrototype(d)->New()); + const google::protobuf::Reflection* r = message->GetReflection(); + for (int i = 0; i < d->field_count(); i++) { + const auto* field_d = d->field(i); + + // if its a union type select a random field in it, randomize its value, + // then jump to the end of all fields in the oneof. + if (auto oneof_d = field_d->real_containing_oneof()) { + auto num_fields = oneof_d->field_count(); + field_d = d->field( + random_generators::get_int(i, i + (num_fields - 1))); + i += (num_fields - 1); + + if ( + _config.randomize_optional_fields && field_d->is_optional() + && random_generators::get_int(0, 1) == 1) { + continue; + } + } + + using fdns = google::protobuf::FieldDescriptor; + + if (field_d->is_repeated()) { + for (size_t i = 0; i < get_elements_count(); i++) { + switch (field_d->cpp_type()) { + case fdns::CPPTYPE_INT32: { + auto v = random_generators::get_int(); + r->AddInt32(message.get(), field_d, v); + } break; + case fdns::CPPTYPE_INT64: { + auto v = random_generators::get_int(); + r->AddInt64(message.get(), field_d, v); + } break; + case fdns::CPPTYPE_UINT32: { + auto v = random_generators::get_int(); + r->AddUInt32(message.get(), field_d, v); + } break; + case fdns::CPPTYPE_UINT64: { + auto v = random_generators::get_int(); + r->AddUInt64(message.get(), field_d, v); + } break; + case fdns::CPPTYPE_DOUBLE: { + auto v = random_generators::get_real(); + r->AddDouble(message.get(), field_d, v); + } break; + case fdns::CPPTYPE_FLOAT: { + auto v = random_generators::get_real(); + r->AddFloat(message.get(), field_d, v); + } break; + case fdns::CPPTYPE_BOOL: { + auto v = random_generators::random_choice({true, false}); + r->AddBool(message.get(), field_d, v); + } break; + case fdns::CPPTYPE_ENUM: { + const auto* enum_d = field_d->enum_type(); + auto ri = random_generators::get_int( + 0, enum_d->value_count() - 1); + r->AddEnum(message.get(), field_d, enum_d->value(ri)); + } break; + case fdns::CPPTYPE_STRING: { + r->AddString( + message.get(), field_d, random_string(_config)); + } break; + case fdns::CPPTYPE_MESSAGE: { + // TODO: does this work with maps? + auto* sub_messsage = generate_protobuf_message_impl( + level + 1, field_d->message_type()) + .release(); + r->AddAllocatedMessage( + message.get(), field_d, sub_messsage); + } break; + } + } + } else { + if ( + _config.randomize_optional_fields && field_d->is_optional() + && random_generators::get_int(0, 1) == 1) { + continue; + } + + switch (field_d->cpp_type()) { + case fdns::CPPTYPE_INT32: { + auto v = random_generators::get_int(); + r->SetInt32(message.get(), field_d, v); + } break; + case fdns::CPPTYPE_INT64: { + auto v = random_generators::get_int(); + r->SetInt64(message.get(), field_d, v); + } break; + case fdns::CPPTYPE_UINT32: { + auto v = random_generators::get_int(); + r->SetUInt32(message.get(), field_d, v); + } break; + case fdns::CPPTYPE_UINT64: { + auto v = random_generators::get_int(); + r->SetUInt64(message.get(), field_d, v); + } break; + case fdns::CPPTYPE_DOUBLE: { + auto v = random_generators::get_real(); + r->SetDouble(message.get(), field_d, v); + } break; + case fdns::CPPTYPE_FLOAT: { + auto v = random_generators::get_real(); + r->SetFloat(message.get(), field_d, v); + } break; + case fdns::CPPTYPE_BOOL: { + auto v = random_generators::random_choice({true, false}); + r->SetBool(message.get(), field_d, v); + } break; + case fdns::CPPTYPE_ENUM: { + const auto* enum_d = field_d->enum_type(); + auto ri = random_generators::get_int( + 0, enum_d->value_count() - 1); + r->SetEnum(message.get(), field_d, enum_d->value(ri)); + } break; + case fdns::CPPTYPE_STRING: { + r->SetString(message.get(), field_d, random_string(_config)); + } break; + case fdns::CPPTYPE_MESSAGE: { + if ( + field_d->is_optional() + && level >= _config.max_nesting_level) { + break; + } + auto* sub_messsage = generate_protobuf_message_impl( + level + 1, field_d->message_type()) + .release(); + r->SetAllocatedMessage(message.get(), sub_messsage, field_d); + } break; + } + } + } + + return message; +} + +} // namespace testing diff --git a/src/v/serde/protobuf/tests/data_generator.h b/src/v/serde/protobuf/tests/data_generator.h new file mode 100644 index 000000000000..85618d98c8ea --- /dev/null +++ b/src/v/serde/protobuf/tests/data_generator.h @@ -0,0 +1,48 @@ +/* + * Copyright 2024 Redpanda Data, Inc. + * + * Use of this software is governed by the Business Source License + * included in the file licenses/BSL.md + * + * As of the Change Date specified in that file, in accordance with + * the Business Source License, use of this software will be governed + * by the Apache License, Version 2.0 + */ +#pragma once + +#include +#include +#include +#include +#include + +#include +#include + +namespace testing { + +struct protobuf_generator_config { + std::optional elements_in_collection{}; + std::pair string_length_range{0, 32}; + int max_nesting_level{10}; + // Enable randomly not setting values for optional fields in the message. + bool randomize_optional_fields{false}; +}; + +class protobuf_generator { +public: + explicit protobuf_generator(protobuf_generator_config c) + : _config(c) {} + + std::unique_ptr + generate_protobuf_message(const google::protobuf::Descriptor* d) { + return generate_protobuf_message_impl(1, d); + } + +private: + std::unique_ptr generate_protobuf_message_impl( + int level, const google::protobuf::Descriptor* d); + protobuf_generator_config _config; + google::protobuf::DynamicMessageFactory _factory{}; +}; +} // namespace testing From a0bd72b0039ab8ea44ae3d61aba6f853ad4e3614 Mon Sep 17 00:00:00 2001 From: Brandon Allard Date: Mon, 18 Nov 2024 01:33:13 -0500 Subject: [PATCH 208/229] datalake: add protobuf support to record_generator --- src/v/datalake/tests/BUILD | 3 + src/v/datalake/tests/CMakeLists.txt | 2 + src/v/datalake/tests/record_generator.cc | 108 +++++++++++++++++++++++ src/v/datalake/tests/record_generator.h | 14 ++- 4 files changed, 126 insertions(+), 1 deletion(-) diff --git a/src/v/datalake/tests/BUILD b/src/v/datalake/tests/BUILD index 5300d189ac20..1aa3d35fe79b 100644 --- a/src/v/datalake/tests/BUILD +++ b/src/v/datalake/tests/BUILD @@ -101,7 +101,9 @@ redpanda_test_cc_library( implementation_deps = [ "//src/v/container:chunked_hash_map", "//src/v/schema:registry", + "//src/v/utils:vint", "@avro", + "@protobuf", ], include_prefix = "datalake/tests", visibility = ["//visibility:public"], @@ -112,6 +114,7 @@ redpanda_test_cc_library( "//src/v/model", "//src/v/pandaproxy", "//src/v/serde/avro/tests:data_generator", + "//src/v/serde/protobuf/tests:data_generator", "//src/v/storage:record_batch_builder", "//src/v/utils:named_type", "@seastar", diff --git a/src/v/datalake/tests/CMakeLists.txt b/src/v/datalake/tests/CMakeLists.txt index e92a4e79e78f..fea737bf7ede 100644 --- a/src/v/datalake/tests/CMakeLists.txt +++ b/src/v/datalake/tests/CMakeLists.txt @@ -10,7 +10,9 @@ v_cc_library( DEPS Avro::avro Seastar::seastar + protobuf::libprotobuf v::avro_test_utils + v::protobuf_test_utils v::schema v::schema_test_fixture v::storage diff --git a/src/v/datalake/tests/record_generator.cc b/src/v/datalake/tests/record_generator.cc index b3dcdfb1ef41..b436815f088d 100644 --- a/src/v/datalake/tests/record_generator.cc +++ b/src/v/datalake/tests/record_generator.cc @@ -9,15 +9,24 @@ */ #include "datalake/tests/record_generator.h" +#include "pandaproxy/schema_registry/protobuf.h" +#include "pandaproxy/schema_registry/types.h" #include "schema/registry.h" #include "storage/record_batch_builder.h" +#include "utils/vint.h" +#include #include +#include #include #include #include #include +#include +#include +#include +#include namespace datalake::tests { @@ -40,6 +49,105 @@ record_generator::register_avro_schema( co_return std::nullopt; } +ss::future> +record_generator::register_protobuf_schema( + std::string_view name, std::string_view schema) { + using namespace pandaproxy::schema_registry; + auto id = co_await ss::coroutine::as_future( + _sr->create_schema(unparsed_schema{ + subject{"foo"}, + unparsed_schema_definition{schema, schema_type::protobuf}})); + if (id.failed()) { + co_return error{fmt::format( + "Error creating schema {}: {}", name, id.get_exception())}; + } + auto [_, added] = _id_by_name.emplace(name, id.get()); + if (!added) { + co_return error{fmt::format("Failed to add schema {} to map", name)}; + } + co_return std::nullopt; +} + +iobuf encode_protobuf_message_index(const std::vector& message_index) { + iobuf ret; + if (message_index.size() == 1 && message_index[0] == 0) { + ret.append("\0", 1); + return ret; + } + + std::array bytes{0}; + size_t res_size = vint::serialize(message_index.size(), &bytes[0]); + ret.append(&bytes[0], res_size); + + for (const auto& o : message_index) { + size_t res_size = vint::serialize(o, &bytes[0]); + ret.append(&bytes[0], res_size); + } + + return ret; +} + +ss::future> +record_generator::add_random_protobuf_record( + storage::record_batch_builder& b, + std::string_view name, + const std::vector& message_index, + std::optional key, + testing::protobuf_generator_config config) { + using namespace pandaproxy::schema_registry; + auto it = _id_by_name.find(name); + if (it == _id_by_name.end()) { + co_return error{fmt::format("Schema {} is missing", name)}; + } + auto schema_id = it->second; + auto schema_def = co_await _sr->get_valid_schema(schema_id); + if (!schema_def) { + co_return error{ + fmt::format("Unable to find schema def for id: {}", schema_id)}; + } + if (schema_def->type() != schema_type::protobuf) { + co_return error{fmt::format( + "Schema {} has wrong type: {}", name, schema_def->type())}; + } + + auto protobuf_def = schema_def + ->visit(ss::make_visitor( + [](const avro_schema_definition&) + -> std::optional { + return std::nullopt; + }, + [](const protobuf_schema_definition& pb_def) + -> std::optional { + return {pb_def}; + }, + [](const json_schema_definition&) + -> std::optional { + return std::nullopt; + })) + .value(); + auto md_res = pandaproxy::schema_registry::descriptor( + protobuf_def, message_index); + if (md_res.has_error()) { + co_return error{fmt::format( + "Wasn't able to get descriptor for protobuf def with id: {}", + schema_id)}; + } + + iobuf val; + val.append("\0", 1); + int32_t encoded_id = ss::cpu_to_be(schema_id()); + val.append((const uint8_t*)(&encoded_id), 4); + + testing::protobuf_generator pb_gen(config); + auto msg = pb_gen.generate_protobuf_message(&md_res.value().get()); + + val.append(encode_protobuf_message_index(message_index)); + val.append(iobuf::from(msg->SerializeAsString())); + + b.add_raw_kv(std::move(key), std::move(val)); + co_return std::nullopt; +} + ss::future> record_generator::add_random_avro_record( storage::record_batch_builder& b, diff --git a/src/v/datalake/tests/record_generator.h b/src/v/datalake/tests/record_generator.h index f830af72e6a8..06288d3c76bb 100644 --- a/src/v/datalake/tests/record_generator.h +++ b/src/v/datalake/tests/record_generator.h @@ -12,10 +12,10 @@ #include "base/seastarx.h" #include "bytes/iobuf.h" #include "container/chunked_hash_map.h" -#include "model/record.h" #include "model/timestamp.h" #include "pandaproxy/schema_registry/types.h" #include "serde/avro/tests/data_generator.h" +#include "serde/protobuf/tests/data_generator.h" #include "storage/record_batch_builder.h" #include "utils/named_type.h" @@ -37,6 +37,10 @@ class record_generator { ss::future> register_avro_schema(std::string_view name, std::string_view schema); + // Registers the given schema with the given name. + ss::future> + register_protobuf_schema(std::string_view name, std::string_view schema); + // Adds a record of the given schema to the builder. ss::future> add_random_avro_record( storage::record_batch_builder&, @@ -44,6 +48,14 @@ class record_generator { std::optional key, testing::avro_generator_config config = {}); + // Adds a record of the given schema to the builder. + ss::future> add_random_protobuf_record( + storage::record_batch_builder&, + std::string_view schema_name, + const std::vector& message_index, + std::optional key, + testing::protobuf_generator_config config = {}); + private: chunked_hash_map _id_by_name; From c77ce4bc4af0ddcf429b47b20117f04b9ada7703 Mon Sep 17 00:00:00 2001 From: Brandon Allard Date: Thu, 21 Nov 2024 22:42:56 -0500 Subject: [PATCH 209/229] utils: move null_output_stream to utils --- src/v/storage/BUILD | 1 + src/v/storage/offset_to_filepos.cc | 32 +++------------------- src/v/utils/BUILD | 12 +++++++++ src/v/utils/null_output_stream.h | 43 ++++++++++++++++++++++++++++++ 4 files changed, 59 insertions(+), 29 deletions(-) create mode 100644 src/v/utils/null_output_stream.h diff --git a/src/v/storage/BUILD b/src/v/storage/BUILD index cf2cc9bde624..468975b3a899 100644 --- a/src/v/storage/BUILD +++ b/src/v/storage/BUILD @@ -420,6 +420,7 @@ redpanda_cc_library( "//src/v/utils:mutex", "//src/v/utils:named_type", "//src/v/utils:notification_list", + "//src/v/utils:null_output_stream", "//src/v/utils:prefix_logger", "//src/v/utils:stream_provider", "//src/v/utils:to_string", diff --git a/src/v/storage/offset_to_filepos.cc b/src/v/storage/offset_to_filepos.cc index c4b93d7cfab6..0081ff5012df 100644 --- a/src/v/storage/offset_to_filepos.cc +++ b/src/v/storage/offset_to_filepos.cc @@ -16,36 +16,10 @@ #include "storage/parser.h" #include "storage/segment.h" #include "storage/segment_utils.h" +#include "utils/null_output_stream.h" #include -namespace { -// Data sink for noop output_stream instance -// needed to implement scanning -struct null_data_sink final : ss::data_sink_impl { - ss::future<> put(ss::net::packet data) final { return put(data.release()); } - ss::future<> put(std::vector> all) final { - return ss::do_with( - std::move(all), [this](std::vector>& all) { - return ss::do_for_each( - all, [this](ss::temporary_buffer& buf) { - return put(std::move(buf)); - }); - }); - } - ss::future<> put(ss::temporary_buffer) final { return ss::now(); } - ss::future<> flush() final { return ss::now(); } - ss::future<> close() final { return ss::now(); } -}; - -ss::output_stream make_null_output_stream() { - auto ds = ss::data_sink(std::make_unique()); - ss::output_stream ostr(std::move(ds), 4_KiB); - return ostr; -} - -} // namespace - namespace storage { namespace internal { @@ -120,7 +94,7 @@ ss::future> convert_begin_offset_to_file_pos( std::move(handle), [&begin_inclusive, &sto, &offset_found, &ts, &offset_inside_batch]( segment_reader_handle& reader_handle) { - auto ostr = make_null_output_stream(); + auto ostr = utils::make_null_output_stream(); return transform_stream( reader_handle.take_stream(), std::move(ostr), @@ -228,7 +202,7 @@ ss::future> convert_end_offset_to_file_pos( &offset_found, &ts, &offset_inside_batch](segment_reader_handle& handle) { - auto ostr = make_null_output_stream(); + auto ostr = utils::make_null_output_stream(); return transform_stream( handle.take_stream(), std::move(ostr), diff --git a/src/v/utils/BUILD b/src/v/utils/BUILD index 39549b844910..023f8c94c460 100644 --- a/src/v/utils/BUILD +++ b/src/v/utils/BUILD @@ -677,3 +677,15 @@ redpanda_cc_library( "@boost//:intrusive", ], ) + +redpanda_cc_library( + name = "null_output_stream", + hdrs = [ + "null_output_stream.h", + ], + include_prefix = "utils", + deps = [ + "//src/v/base", + "@seastar", + ], +) diff --git a/src/v/utils/null_output_stream.h b/src/v/utils/null_output_stream.h new file mode 100644 index 000000000000..47455f5a089b --- /dev/null +++ b/src/v/utils/null_output_stream.h @@ -0,0 +1,43 @@ +/* + * Copyright 2024 Redpanda Data, Inc. + * + * Use of this software is governed by the Business Source License + * included in the file licenses/BSL.md + * + * As of the Change Date specified in that file, in accordance with + * the Business Source License, use of this software will be governed + * by the Apache License, Version 2.0 + */ + +#include "base/seastarx.h" +#include "base/units.h" + +#include +#include + +namespace utils { +// Data sink for noop output_stream instance +// needed to implement scanning +struct null_data_sink final : ss::data_sink_impl { + ss::future<> put(ss::net::packet data) final { return put(data.release()); } + ss::future<> put(std::vector> all) final { + return ss::do_with( + std::move(all), [this](std::vector>& all) { + return ss::do_for_each( + all, [this](ss::temporary_buffer& buf) { + return put(std::move(buf)); + }); + }); + } + ss::future<> put(ss::temporary_buffer) final { return ss::now(); } + ss::future<> flush() final { return ss::now(); } + ss::future<> close() final { return ss::now(); } +}; + +ss::output_stream make_null_output_stream() { + auto ds = ss::data_sink(std::make_unique()); + ss::output_stream ostr(std::move(ds), 4_KiB); + return ostr; +} + +} // namespace utils From b187703a06cd02ab5288eb2001bc4aeddfd4c88f Mon Sep 17 00:00:00 2001 From: Brandon Allard Date: Fri, 22 Nov 2024 00:41:18 -0500 Subject: [PATCH 210/229] datalake: add test_serde_parquet_data_writer This data writer is for tests. It doesn't write to any files, however, it does go through the process of converting the ostream to parqueat. --- src/v/datalake/tests/BUILD | 2 ++ src/v/datalake/tests/test_data_writer.h | 42 +++++++++++++++++++++++++ 2 files changed, 44 insertions(+) diff --git a/src/v/datalake/tests/BUILD b/src/v/datalake/tests/BUILD index 1aa3d35fe79b..467f001106be 100644 --- a/src/v/datalake/tests/BUILD +++ b/src/v/datalake/tests/BUILD @@ -128,9 +128,11 @@ redpanda_test_cc_library( ], include_prefix = "datalake/tests", deps = [ + "//src/v/datalake:serde_parquet_writer", "//src/v/datalake:writer", "//src/v/iceberg:datatypes", "//src/v/iceberg:values", + "//src/v/utils:null_output_stream", "@seastar", ], ) diff --git a/src/v/datalake/tests/test_data_writer.h b/src/v/datalake/tests/test_data_writer.h index 1db0e87444ae..61d11e3d2811 100644 --- a/src/v/datalake/tests/test_data_writer.h +++ b/src/v/datalake/tests/test_data_writer.h @@ -9,8 +9,10 @@ #pragma once #include "datalake/data_writer_interface.h" +#include "datalake/serde_parquet_writer.h" #include "iceberg/datatypes.h" #include "iceberg/values.h" +#include "utils/null_output_stream.h" #include #include @@ -62,4 +64,44 @@ class test_data_writer_factory : public parquet_file_writer_factory { bool _return_error; }; +class test_serde_parquet_data_writer : public parquet_file_writer { +public: + explicit test_serde_parquet_data_writer( + std::unique_ptr writer) + : _writer(std::move(writer)) + , _result{} {} + + ss::future + add_data_struct(iceberg::struct_value data, int64_t sz) override { + auto write_result = co_await _writer->add_data_struct( + std::move(data), sz); + _result.row_count++; + co_return write_result; + } + + ss::future> finish() override { + return ss::make_ready_future>( + _result); + } + +private: + std::unique_ptr _writer; + local_file_metadata _result; +}; + +class test_serde_parquet_writer_factory : public parquet_file_writer_factory { +public: + ss::future, writer_error>> + create_writer(const iceberg::struct_type& schema) override { + auto ostream_writer = co_await _serde_parquet_factory.create_writer( + schema, utils::make_null_output_stream()); + + co_return std::make_unique( + std::move(ostream_writer)); + } + +private: + serde_parquet_writer_factory _serde_parquet_factory; +}; + } // namespace datalake From 1c29911e06c2d66798b15bfe4e2c1ab5b89cd832 Mon Sep 17 00:00:00 2001 From: Brandon Allard Date: Mon, 18 Nov 2024 01:50:25 -0500 Subject: [PATCH 211/229] datalake: add record_multiplexer microbenchmark --- src/v/datalake/tests/CMakeLists.txt | 17 + .../tests/record_multiplexer_bench.cc | 535 ++++++++++++++++++ 2 files changed, 552 insertions(+) create mode 100644 src/v/datalake/tests/record_multiplexer_bench.cc diff --git a/src/v/datalake/tests/CMakeLists.txt b/src/v/datalake/tests/CMakeLists.txt index fea737bf7ede..d9d2dbe5c6ed 100644 --- a/src/v/datalake/tests/CMakeLists.txt +++ b/src/v/datalake/tests/CMakeLists.txt @@ -204,3 +204,20 @@ rp_test( LABELS datalake ARGS "-- -c 1" ) + +rp_test( + BENCHMARK_TEST + BINARY_NAME record_multiplexer + SOURCES record_multiplexer_bench.cc + LIBRARIES + Seastar::seastar_perf_testing + Boost::unit_test_framework + v::cloud_io_utils + v::application + v::datalake_test_utils + v::iceberg_test_utils + v::schema + v::s3_imposter + ARGS "-c 1 --duration=1 --runs=1 --memory=4G" + LABELS datalake +) diff --git a/src/v/datalake/tests/record_multiplexer_bench.cc b/src/v/datalake/tests/record_multiplexer_bench.cc new file mode 100644 index 000000000000..3acf78338546 --- /dev/null +++ b/src/v/datalake/tests/record_multiplexer_bench.cc @@ -0,0 +1,535 @@ +/* + * Copyright 2024 Redpanda Data, Inc. + * + * Use of this software is governed by the Business Source License + * included in the file licenses/BSL.md + * + * As of the Change Date specified in that file, in accordance with + * the Business Source License, use of this software will be governed + * by the Apache License, Version 2.0 + */ +#include "container/fragmented_vector.h" +#include "datalake/catalog_schema_manager.h" +#include "datalake/record_multiplexer.h" +#include "datalake/record_schema_resolver.h" +#include "datalake/record_translator.h" +#include "datalake/table_creator.h" +#include "datalake/tests/catalog_and_registry_fixture.h" +#include "datalake/tests/record_generator.h" +#include "datalake/tests/test_data_writer.h" +#include "model/record.h" +#include "model/record_batch_reader.h" +#include "serde/avro/tests/data_generator.h" +#include "serde/protobuf/tests/data_generator.h" + +#include +#include + +#include + +#include +#include + +namespace { +std::string generate_nested_proto_internal(size_t total_depth) { + constexpr auto proto_template = R"( + message Foo{} {{ + {} + string a{} = {}; + {} + }})"; + + if (total_depth == 0) { + return ""; + } + std::string member = ""; + if (total_depth > 1) { + member = std::format( + "Foo{} b{} = {}; ", + total_depth - 1, + total_depth, + 2 * total_depth + 1); + } + return std::format( + proto_template, + total_depth, + generate_nested_proto_internal(total_depth - 1), + total_depth, + 2 * total_depth, + member); +} + +/** + * Generates a nested protobuf schema. + * + * I.e, if total_depth=3 then the following would be generated; + * + * syntax = "proto3"; + * message Foo3 { + * message Foo2 { + * message Foo1 { + * string a1 = 2; + * } + * string a2 = 4; + * Foo1 b2 = 5; + * } + * string a3 = 6; + * Foo2 b3 = 7; + * } + */ +std::string generate_nested_proto(size_t total_depth) { + return std::format( + "syntax = \"proto3\"; {}", generate_nested_proto_internal(total_depth)); +} + +/** + * Generates a linear protobuf schema. + * + * I.e, if total_fields=3 then the following would be generated; + * + * syntax = "proto3"; + * message Linear { + * string a1 = 1; + * string a2 = 2; + * string a3 = 3; + * } + */ +std::string generate_linear_proto(size_t total_fields) { + constexpr auto proto_template = R"( + syntax = "proto3"; + message Linear {{ + {} + }})"; + + std::string fields = ""; + for (int i = 1; i <= total_fields; i++) { + fields += std::format("string a{} = {};\n", i, i); + } + + return std::format(proto_template, fields); +} + +std::string generate_nested_avro_internal(size_t total_depth) { + constexpr auto avro_template = R"( + {{ + "name": "nestedval{}", + "type": {{ + "type": "record", + "name": "nestedrecord{}", + "fields": [ + {} + {} + ] + }} + }})"; + + if (total_depth == 0) { + return ""; + } + + std::string string_field = std::format( + R"({{ "name": "inval{}", "type": "string" }})", total_depth); + if (total_depth != 1) { + string_field += ","; + }; + + return std::format( + avro_template, + total_depth, + total_depth, + string_field, + generate_nested_avro_internal(total_depth - 1)); +} + +/** + * Generates a nested avro schema; + * + * I.e, if total_depth=2 then the following would be generated; + * { + * "name": "base", + * "type": "record", + * "fields": [ + * { + * "name": "nestedval2", + * "type": { + * "type": "record", + * "name": "nestedrecord2", + * "fields": [ + * { + * "name": "inval2", + * "type": "string" + * }, + * { + * "name": "nestedval1", + * "type": { + * "type": "record", + * "name": "nestedrecord1", + * "fields": [ + * { + * "name": "inval1", + * "type": "string" + * } + * ] + * } + * } + * ] + * } + * } + * ] + *} + * + */ +std::string generate_nested_avro(size_t total_depth) { + constexpr auto avro_template = R"({{ + "name": "base", + "type": "record", + "fields": [ + {} + ]}})"; + + return std::format( + avro_template, generate_nested_avro_internal(total_depth)); +} + +/** + * Generates a linear avro schema. + * + * I.e, if total_fields=3 then the following would be generated; + * { + * "name": "base", + * "type": "record", + * "fields": [ + * { + * "name": "field0", + * "type": "string" + * }, + * { + * "name": "field1", + * "type": "string" + * }, + * { + * "name": "field2", + * "type": "string" + * } + * ] + * } + * + */ +std::string generate_linear_avro(size_t total_fields) { + constexpr auto avro_template = R"({{ + "name": "base", + "type": "record", + "fields": [ + {} + ]}})"; + constexpr auto field_template + = R"({{ "name": "field{}", "type": "string" }})"; + std::string ret = ""; + + for (size_t i = 0; i < total_fields; i++) { + ret += std::format(field_template, i); + if (i != total_fields - 1) { + ret += ","; + } + } + + ret = std::format(avro_template, ret); + return ret; +} + +chunked_vector +share_batches(chunked_vector& batches) { + chunked_vector ret; + for (auto& batch : batches) { + ret.push_back(batch.share()); + } + return ret; +} + +struct counting_consumer { + size_t total_bytes = 0; + datalake::record_multiplexer mux; + ss::future operator()(model::record_batch&& batch) { + total_bytes += batch.size_bytes(); + return mux(std::move(batch)); + } + ss::future end_of_stream() { + auto res = co_await mux.end_of_stream(); + BOOST_REQUIRE(!res.has_error()); + co_return std::move(*this); + } +}; + +} // namespace + +class record_multiplexer_bench_fixture + : public datalake::tests::catalog_and_registry_fixture { +public: + record_multiplexer_bench_fixture() + : _schema_mgr(catalog) + , _type_resolver(registry) + , _record_gen(®istry) + , _table_creator(_type_resolver, _schema_mgr) {} + + template + requires std::same_as + || std::same_as + ss::future<> configure_bench( + T gen_config, + std::string schema, + size_t batches, + size_t records_per_batch) { + if constexpr (std::is_same_v) { + _batch_data = co_await generate_protobuf_batches( + records_per_batch, + batches, + "proto_schema", + schema, + {0}, + gen_config); + } else { + _batch_data = co_await generate_avro_batches( + records_per_batch, batches, "avro_schema", schema, gen_config); + } + } + + ss::future run_bench() { + auto reader = model::make_fragmented_memory_record_batch_reader( + share_batches(_batch_data)); + auto consumer = counting_consumer{.mux = create_mux()}; + + perf_tests::start_measuring_time(); + auto res = co_await reader.consume( + std::move(consumer), model::no_timeout); + perf_tests::stop_measuring_time(); + + co_return res.total_bytes; + } + +private: + std::unordered_set _added_names; + datalake::catalog_schema_manager _schema_mgr; + datalake::record_schema_resolver _type_resolver; + datalake::tests::record_generator _record_gen; + datalake::default_translator _translator; + datalake::direct_table_creator _table_creator; + chunked_vector _batch_data; + + const model::ntp ntp{ + model::ns{"rp"}, model::topic{"t"}, model::partition_id{0}}; + const model::revision_id topic_rev{123}; + + datalake::record_multiplexer create_mux() { + return datalake::record_multiplexer( + ntp, + topic_rev, + std::make_unique(), + _schema_mgr, + _type_resolver, + _translator, + _table_creator); + } + + ss::future<> + try_add_avro_schema(std::string_view name, std::string_view schema) { + auto [_, added] = _added_names.emplace(name); + if (!added) { + co_return; + } + + auto reg_res = co_await _record_gen.register_avro_schema(name, schema); + BOOST_REQUIRE(reg_res.has_value()); + } + + ss::future<> + try_add_protobuf_schema(std::string_view name, std::string_view schema) { + auto [_, added] = _added_names.emplace(name); + if (!added) { + co_return; + } + + auto reg_res = co_await _record_gen.register_protobuf_schema( + name, schema); + BOOST_REQUIRE(reg_res.has_value()); + } + + ss::future> generate_batches( + size_t records_per_batch, + size_t batches, + std::function>( + storage::record_batch_builder&)> add_batch) { + chunked_vector ret; + ret.reserve(batches); + + model::offset o{0}; + for (size_t i = 0; i < batches; ++i) { + storage::record_batch_builder batch_builder( + model::record_batch_type::raft_data, o); + + // Add some records per batch. + for (size_t r = 0; r < records_per_batch; ++r) { + auto res = co_await add_batch(batch_builder); + ++o; + + BOOST_REQUIRE(!res.has_error()); + } + auto batch = std::move(batch_builder).build(); + ret.emplace_back(std::move(batch)); + } + + co_return ret; + } + + ss::future> generate_protobuf_batches( + size_t records_per_batch, + size_t batches, + std::string schema_name, + std::string proto_schema, + std::vector msg_idx, + ::testing::protobuf_generator_config gen_config) { + co_await try_add_protobuf_schema(schema_name, proto_schema); + co_return co_await generate_batches( + records_per_batch, batches, [&](auto& bb) { + return _record_gen.add_random_protobuf_record( + bb, schema_name, msg_idx, std::nullopt, gen_config); + }); + } + + ss::future> generate_avro_batches( + size_t records_per_batch, + size_t batches, + std::string schema_name, + std::string avro_schema, + ::testing::avro_generator_config gen_config) { + co_await try_add_avro_schema(schema_name, avro_schema); + co_return co_await generate_batches( + records_per_batch, batches, [&](auto& bb) { + return _record_gen.add_random_avro_record( + bb, schema_name, std::nullopt, gen_config); + }); + } +}; + +namespace { + +// Specifies how many batches should be in the test dataset. +static constexpr size_t batches = 1000; +// Specifies how many records should be in each batch of the test dataset. +static constexpr size_t records_per_batch = 10; + +} // namespace + +PERF_TEST_CN( + record_multiplexer_bench_fixture, protobuf_381_byte_message_linear_1_field) { + static ::testing::protobuf_generator_config gen_config = { + .string_length_range{302, 302}}; + static std::string proto_schema = generate_linear_proto(1); + + co_await configure_bench( + gen_config, proto_schema, batches, records_per_batch); + co_return co_await run_bench(); +} + +PERF_TEST_CN( + record_multiplexer_bench_fixture, + protobuf_381_byte_message_linear_40_fields) { + static ::testing::protobuf_generator_config gen_config = { + .string_length_range{5, 5}}; + static std::string proto_schema = generate_linear_proto(40); + + co_await configure_bench( + gen_config, proto_schema, batches, records_per_batch); + co_return co_await run_bench(); +} + +PERF_TEST_CN( + record_multiplexer_bench_fixture, + protobuf_381_byte_message_linear_80_fields) { + static ::testing::protobuf_generator_config gen_config = { + .string_length_range{1, 1}}; + static std::string proto_schema = generate_linear_proto(80); + + co_await configure_bench( + gen_config, proto_schema, batches, records_per_batch); + co_return co_await run_bench(); +} + +PERF_TEST_CN( + record_multiplexer_bench_fixture, + protobuf_384_byte_message_nested_24_levels) { + static ::testing::protobuf_generator_config gen_config = { + .string_length_range{7, 7}, .max_nesting_level = 40}; + static std::string proto_schema = generate_nested_proto(24); + + co_await configure_bench( + gen_config, proto_schema, batches, records_per_batch); + co_return co_await run_bench(); +} + +PERF_TEST_CN( + record_multiplexer_bench_fixture, + protobuf_386_byte_message_nested_31_levels) { + static ::testing::protobuf_generator_config gen_config = { + .string_length_range{4, 4}, .max_nesting_level = 40}; + static std::string proto_schema = generate_nested_proto(31); + + co_await configure_bench( + gen_config, proto_schema, batches, records_per_batch); + co_return co_await run_bench(); +} + +PERF_TEST_CN( + record_multiplexer_bench_fixture, avro_385_byte_message_linear_1_field) { + static ::testing::avro_generator_config gen_config = { + .string_length_range{308, 308}}; + static std::string avro_schema = generate_linear_avro(1); + + co_await configure_bench( + gen_config, avro_schema, batches, records_per_batch); + co_return co_await run_bench(); +} + +PERF_TEST_CN( + record_multiplexer_bench_fixture, avro_385_byte_message_linear_31_fields) { + static ::testing::avro_generator_config gen_config = { + .string_length_range{9, 9}}; + static std::string avro_schema = generate_linear_avro(31); + + co_await configure_bench( + gen_config, avro_schema, batches, records_per_batch); + co_return co_await run_bench(); +} + +PERF_TEST_CN( + record_multiplexer_bench_fixture, avro_385_byte_message_linear_62_fields) { + static ::testing::avro_generator_config gen_config = { + .string_length_range{4, 4}}; + static std::string avro_schema = generate_linear_avro(62); + + co_await configure_bench( + gen_config, avro_schema, batches, records_per_batch); + co_return co_await run_bench(); +} + +PERF_TEST_CN( + record_multiplexer_bench_fixture, avro_385_byte_message_nested_31_levels) { + static ::testing::avro_generator_config gen_config = { + .string_length_range{9, 9}, .max_nesting_level = 40}; + static std::string avro_schema = generate_nested_avro(31); + + co_await configure_bench( + gen_config, avro_schema, batches, records_per_batch); + co_return co_await run_bench(); +} + +PERF_TEST_CN( + record_multiplexer_bench_fixture, avro_385_byte_message_nested_62_levels) { + static ::testing::avro_generator_config gen_config = { + .string_length_range{4, 4}, .max_nesting_level = 40}; + static std::string avro_schema = generate_nested_avro(62); + + co_await configure_bench( + gen_config, avro_schema, batches, records_per_batch); + co_return co_await run_bench(); +} From 1c7c283f28966d9b8985ceadc420f65185335189 Mon Sep 17 00:00:00 2001 From: Oren Leiman Date: Thu, 12 Dec 2024 18:12:45 -0800 Subject: [PATCH 212/229] iceberg: Introduce compatibility module Initially only supports checking for primitive type promotions. iceberg::check_types(field_type s, field_type d) returns - type_promoted::no - if the types are identical (or non-primitive and equal) - type_promoted::yes - if s was promoted to d and the promotion is legal - an error - if s was promoted to d and the promotion is illegal Signed-off-by: Oren Leiman --- src/v/iceberg/BUILD | 19 +++ src/v/iceberg/CMakeLists.txt | 1 + src/v/iceberg/compatibility.cc | 109 ++++++++++++++ src/v/iceberg/compatibility.h | 49 +++++++ src/v/iceberg/tests/BUILD | 18 +++ src/v/iceberg/tests/CMakeLists.txt | 11 ++ src/v/iceberg/tests/compatibility_test.cc | 167 ++++++++++++++++++++++ 7 files changed, 374 insertions(+) create mode 100644 src/v/iceberg/compatibility.cc create mode 100644 src/v/iceberg/compatibility.h create mode 100644 src/v/iceberg/tests/compatibility_test.cc diff --git a/src/v/iceberg/BUILD b/src/v/iceberg/BUILD index 5de1fab42252..ccce33e960b6 100644 --- a/src/v/iceberg/BUILD +++ b/src/v/iceberg/BUILD @@ -120,6 +120,25 @@ redpanda_cc_library( ], ) +redpanda_cc_library( + name = "compatibility", + srcs = [ + "compatibility.cc", + ], + hdrs = [ + "compatibility.h", + ], + include_prefix = "iceberg", + visibility = ["//visibility:public"], + deps = [ + ":datatypes", + "//src/v/base", + "@abseil-cpp//absl/container:btree", + "@fmt", + "@seastar", + ], +) + redpanda_cc_library( name = "datatypes_json", srcs = [ diff --git a/src/v/iceberg/CMakeLists.txt b/src/v/iceberg/CMakeLists.txt index eaa1fa0103fc..1ffc034fc57a 100644 --- a/src/v/iceberg/CMakeLists.txt +++ b/src/v/iceberg/CMakeLists.txt @@ -30,6 +30,7 @@ v_cc_library( ${avro_hdrs} action.cc catalog.cc + compatibility.cc datatypes.cc datatypes_json.cc filesystem_catalog.cc diff --git a/src/v/iceberg/compatibility.cc b/src/v/iceberg/compatibility.cc new file mode 100644 index 000000000000..9d3bd3af99e4 --- /dev/null +++ b/src/v/iceberg/compatibility.cc @@ -0,0 +1,109 @@ +// Copyright 2024 Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 + +#include "iceberg/compatibility.h" + +#include "iceberg/datatypes.h" + +#include +#include + +#include + +namespace iceberg { + +namespace { +struct primitive_type_promotion_policy_visitor { + template + requires(!std::is_same_v) + type_check_result operator()(const T&, const U&) const { + return compat_errc::mismatch; + } + + template + type_check_result operator()(const T&, const T&) const { + return type_promoted::no; + } + + type_check_result + operator()(const iceberg::int_type&, const iceberg::long_type&) const { + return type_promoted::yes; + } + + type_check_result operator()( + const iceberg::date_type&, const iceberg::timestamp_type&) const { + return type_promoted::yes; + } + + type_check_result + operator()(const iceberg::float_type&, const iceberg::double_type&) { + return type_promoted::yes; + } + + type_check_result operator()( + const iceberg::decimal_type& src, const iceberg::decimal_type& dst) { + if (iceberg::primitive_type{src} == iceberg::primitive_type{dst}) { + return type_promoted::no; + } + if ((dst.scale == src.scale && dst.precision > src.precision)) { + return type_promoted::yes; + } + return compat_errc::mismatch; + } + + type_check_result + operator()(const iceberg::fixed_type& src, const iceberg::fixed_type& dst) { + if (iceberg::primitive_type{src} == iceberg::primitive_type{dst}) { + return type_promoted::no; + } + return compat_errc::mismatch; + } +}; + +struct field_type_check_visitor { + explicit field_type_check_visitor( + primitive_type_promotion_policy_visitor policy) + : policy(policy) {} + + template + requires(!std::is_same_v) + type_check_result operator()(const T&, const U&) const { + return compat_errc::mismatch; + } + + // For non-primitives, type identity is sufficient to pass this check. + // e.g. any two struct types will pass w/o indicating type promotion, + // whereas a struct and, say, a list would produce a mismatch error code. + // The member fields of two such structs (or the element fields of two + // lists, k/v for a map, etc.) will be checked elsewhere. + template + type_check_result operator()(const T&, const T&) const { + return type_promoted::no; + } + + type_check_result + operator()(const primitive_type& src, const primitive_type& dest) { + return std::visit(policy, src, dest); + } + +private: + primitive_type_promotion_policy_visitor policy; +}; + +} // namespace + +type_check_result +check_types(const iceberg::field_type& src, const iceberg::field_type& dest) { + return std::visit( + field_type_check_visitor{primitive_type_promotion_policy_visitor{}}, + src, + dest); +} + +} // namespace iceberg diff --git a/src/v/iceberg/compatibility.h b/src/v/iceberg/compatibility.h new file mode 100644 index 000000000000..d90f44701d1a --- /dev/null +++ b/src/v/iceberg/compatibility.h @@ -0,0 +1,49 @@ +// Copyright 2024 Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 + +#pragma once + +#include "base/outcome.h" +#include "iceberg/datatypes.h" + +#include + +namespace iceberg { + +enum class compat_errc { + mismatch, +}; + +using type_promoted = ss::bool_class; +using type_check_result = checked; + +/** + check_types - Performs a basic type check between two Iceberg field types, + enforcing the Primitive Type Promotion policy laid out in + https://iceberg.apache.org/spec/#schema-evolution + + - For non-primitive types, checks strict equality - i.e. struct == struct, + list != map + - Unwraps and compares input types, returns the result. Does not account for + nesting. + + @param src - The type of some field in an existing schema + @param dest - The type of some field in a new schema, possibly compatible + with src. + @return The result of the type check: + - type_promoted::yes - if src -> dest is a valid type promotion + e.g. int -> long + - type_promoted::no - if src == dest + - compat_errc::mismatch - if src != dest but src -> dest is not + permitted e.g. int -> string or struct -> list + */ +type_check_result +check_types(const iceberg::field_type& src, const iceberg::field_type& dest); + +} // namespace iceberg diff --git a/src/v/iceberg/tests/BUILD b/src/v/iceberg/tests/BUILD index e82b2329f487..df359e39c4b9 100644 --- a/src/v/iceberg/tests/BUILD +++ b/src/v/iceberg/tests/BUILD @@ -613,6 +613,24 @@ redpanda_cc_gtest( ], ) +redpanda_cc_gtest( + name = "compatibility_test", + timeout = "short", + srcs = [ + "compatibility_test.cc", + ], + cpu = 1, + deps = [ + "//src/v/iceberg:compatibility", + "//src/v/iceberg:datatypes", + "//src/v/iceberg:field_collecting_visitor", + "//src/v/test_utils:gtest", + "@fmt", + "@googletest//:gtest", + "@seastar", + ], +) + redpanda_cc_bench( name = "uri_bench", timeout = "short", diff --git a/src/v/iceberg/tests/CMakeLists.txt b/src/v/iceberg/tests/CMakeLists.txt index 4fd44d281bd0..d0f9046e34e7 100644 --- a/src/v/iceberg/tests/CMakeLists.txt +++ b/src/v/iceberg/tests/CMakeLists.txt @@ -14,6 +14,17 @@ v_cc_library( v::iceberg ) +rp_test( + UNIT_TEST + GTEST + BINARY_NAME iceberg_compatibility + SOURCES + compatibility_test.cc + LIBRARIES + v::gtest_main + v::iceberg_test_utils +) + rp_test( FIXTURE_TEST GTEST diff --git a/src/v/iceberg/tests/compatibility_test.cc b/src/v/iceberg/tests/compatibility_test.cc new file mode 100644 index 000000000000..a5bfceaaad4d --- /dev/null +++ b/src/v/iceberg/tests/compatibility_test.cc @@ -0,0 +1,167 @@ +// Copyright 2024 Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 + +#include "iceberg/compatibility.h" +#include "iceberg/datatypes.h" + +#include +#include + +#include +#include + +using namespace iceberg; + +namespace { + +using compat = ss::bool_class; + +struct field_test_case { + field_test_case( + field_type source, field_type dest, type_check_result expected) + : source(std::move(source)) + , dest(std::move(dest)) + , expected(expected) {} + + field_test_case(const field_test_case& other) + : source(make_copy(other.source)) + , dest(make_copy(other.dest)) + , expected( + other.expected.has_error() + ? type_check_result{other.expected.error()} + : type_check_result{other.expected.value()}) {} + + field_test_case(field_test_case&&) = default; + field_test_case& operator=(const field_test_case& other) = delete; + field_test_case& operator=(field_test_case&&) = delete; + ~field_test_case() = default; + + field_type source; + field_type dest; + type_check_result expected{compat_errc::mismatch}; +}; + +std::ostream& operator<<(std::ostream& os, const field_test_case& ftc) { + fmt::print( + os, + "{}->{} [expected: {}]", + ftc.source, + ftc.dest, + ftc.expected.has_error() + ? std::string{"ERROR"} + : fmt::format("promoted={}", ftc.expected.value())); + return os; +} +} // namespace + +std::vector generate_test_cases() { + std::vector test_data{}; + + test_data.emplace_back(int_type{}, long_type{}, type_promoted::yes); + test_data.emplace_back(int_type{}, boolean_type{}, compat_errc::mismatch); + + test_data.emplace_back(date_type{}, timestamp_type{}, type_promoted::yes); + test_data.emplace_back(date_type{}, long_type{}, compat_errc::mismatch); + + test_data.emplace_back(float_type{}, double_type{}, type_promoted::yes); + test_data.emplace_back( + float_type{}, fixed_type{.length = 64}, compat_errc::mismatch); + + test_data.emplace_back( + decimal_type{.precision = 10, .scale = 2}, + decimal_type{.precision = 20, .scale = 2}, + type_promoted::yes); + test_data.emplace_back( + decimal_type{.precision = 10, .scale = 2}, + decimal_type{.precision = 10, .scale = 2}, + type_promoted::no); + test_data.emplace_back( + decimal_type{.precision = 20, .scale = 2}, + decimal_type{.precision = 10, .scale = 2}, + compat_errc::mismatch); + + test_data.emplace_back( + fixed_type{.length = 32}, fixed_type{.length = 32}, type_promoted::no); + test_data.emplace_back( + fixed_type{.length = 32}, + fixed_type{.length = 64}, + compat_errc::mismatch); + test_data.emplace_back( + fixed_type{.length = 64}, + fixed_type{.length = 32}, + compat_errc::mismatch); + + struct_type s1{}; + struct_type s2{}; + s2.fields.emplace_back( + nested_field::create(0, "foo", field_required::yes, int_type{})); + field_type l1 = list_type::create(0, field_required::yes, int_type{}); + field_type l2 = list_type::create(0, field_required::no, string_type{}); + field_type m1 = map_type::create( + 0, int_type{}, 0, field_required::yes, date_type{}); + field_type m2 = map_type::create( + 0, string_type{}, 0, field_required::no, timestamptz_type{}); + + // NOTE: basic type check doesn't descend into non-primitive types + // Checking stops at type ID - i.e. compat(struct, struct) == true, + // compat(struct, list) == false. + test_data.emplace_back(s1.copy(), s1.copy(), type_promoted::no); + test_data.emplace_back(s1.copy(), s2.copy(), type_promoted::no); + test_data.emplace_back(make_copy(l1), make_copy(l1), type_promoted::no); + test_data.emplace_back(make_copy(l1), make_copy(l2), type_promoted::no); + test_data.emplace_back(make_copy(m1), make_copy(m1), type_promoted::no); + test_data.emplace_back(make_copy(m1), make_copy(m2), type_promoted::no); + + std::vector non_promotable_types; + non_promotable_types.emplace_back(boolean_type{}); + non_promotable_types.emplace_back(long_type{}); + non_promotable_types.emplace_back(double_type{}); + non_promotable_types.emplace_back(time_type{}); + non_promotable_types.emplace_back(timestamp_type{}); + non_promotable_types.emplace_back(timestamptz_type{}); + non_promotable_types.emplace_back(string_type{}); + non_promotable_types.emplace_back(uuid_type{}); + non_promotable_types.emplace_back(binary_type{}); + non_promotable_types.emplace_back(s1.copy()); + non_promotable_types.emplace_back(make_copy(l1)); + non_promotable_types.emplace_back(make_copy(m1)); + + for (const auto& fta : non_promotable_types) { + for (const auto& ftb : non_promotable_types) { + if (fta == ftb) { + continue; + } + test_data.emplace_back( + make_copy(fta), make_copy(ftb), compat_errc::mismatch); + } + } + + return test_data; +} + +struct CompatibilityTest + : ::testing::Test + , testing::WithParamInterface {}; + +INSTANTIATE_TEST_SUITE_P( + PrimitiveTypeCompatibilityTest, + CompatibilityTest, + ::testing::ValuesIn(generate_test_cases())); + +TEST_P(CompatibilityTest, CompatibleTypesAreCompatible) { + const auto& p = GetParam(); + + auto res = check_types(p.source, p.dest); + ASSERT_EQ(res.has_error(), p.expected.has_error()); + if (res.has_error()) { + ASSERT_EQ(res.error(), p.expected.error()); + } else { + ASSERT_EQ(res.value(), p.expected.value()); + } +} From 3eee4de8af8e506e231d9100bde5eca4953728cd Mon Sep 17 00:00:00 2001 From: Oren Leiman Date: Thu, 12 Dec 2024 18:19:50 -0800 Subject: [PATCH 213/229] datalake: Update catalog_schema_manager to support type promotion Works similarly at the interface, but instead of checking for field-wise type identity check for primitive type promotion. If one or more fields was promoted, consider the resulting field assignment "incomplete" to trigger a schema update operation. Signed-off-by: Oren Leiman --- src/v/datalake/BUILD | 1 + src/v/datalake/catalog_schema_manager.cc | 30 ++++++--- src/v/datalake/tests/BUILD | 1 + .../tests/catalog_schema_manager_test.cc | 61 +++++++++++++++++++ 4 files changed, 83 insertions(+), 10 deletions(-) diff --git a/src/v/datalake/BUILD b/src/v/datalake/BUILD index 24908e6c8609..fb128df36b2b 100644 --- a/src/v/datalake/BUILD +++ b/src/v/datalake/BUILD @@ -12,6 +12,7 @@ redpanda_cc_library( ":logger", ":table_definition", "//src/v/base", + "//src/v/iceberg:compatibility", "//src/v/iceberg:field_collecting_visitor", "//src/v/iceberg:schema", "//src/v/iceberg:transaction", diff --git a/src/v/datalake/catalog_schema_manager.cc b/src/v/datalake/catalog_schema_manager.cc index d07809d0410d..5490f6085808 100644 --- a/src/v/datalake/catalog_schema_manager.cc +++ b/src/v/datalake/catalog_schema_manager.cc @@ -12,6 +12,8 @@ #include "base/vlog.h" #include "datalake/logger.h" #include "datalake/table_definition.h" +#include "iceberg/compatibility.h" +#include "iceberg/datatypes.h" #include "iceberg/field_collecting_visitor.h" #include "iceberg/table_identifier.h" #include "iceberg/transaction.h" @@ -36,10 +38,13 @@ schema_manager::errc log_and_convert_catalog_err( } enum class fill_errc { // There is a mismatch in a field's type, name, or required. - mismatch, + invalid_schema, // We couldn't fill all the columns, but the ones we could all matched. - incomplete, + // Or one or more columns were type-promoted (legally), so we'll need to + // push an update into the catalog. + schema_evolution_needed, }; + // Performs a simultaneous, depth-first iteration through fields of the two // schemas, filling dest's field IDs with those from the source. Returns // successfully if all the field IDs in the destination type are filled. @@ -56,23 +61,28 @@ fill_field_ids(iceberg::struct_type& dest, const iceberg::struct_type& source) { for (auto& f : std::ranges::reverse_view(source.fields)) { source_stack.emplace_back(f.get()); } + bool has_primitive_type_promotion{false}; while (!source_stack.empty() && !dest_stack.empty()) { auto* dst = dest_stack.back(); auto* src = source_stack.back(); - if ( - dst->name != src->name || dst->required != src->required - || dst->type.index() != src->type.index()) { - return fill_errc::mismatch; + if (auto compatibility = check_types(src->type, dst->type); + dst->name != src->name || dst->required != src->required + || compatibility.has_error()) { + return fill_errc::invalid_schema; + } else if (compatibility.value() == type_promoted::yes) { + has_primitive_type_promotion = true; } + dst->id = src->id; dest_stack.pop_back(); source_stack.pop_back(); std::visit(reverse_field_collecting_visitor(dest_stack), dst->type); std::visit(reverse_field_collecting_visitor(source_stack), src->type); } - if (!dest_stack.empty()) { + + if (!dest_stack.empty() || has_primitive_type_promotion) { // There are more fields to fill. - return fill_errc::incomplete; + return fill_errc::schema_evolution_needed; } // We successfully filled all the fields in the destination. return std::nullopt; @@ -212,10 +222,10 @@ catalog_schema_manager::get_ids_from_table_meta( auto fill_res = fill_field_ids(dest_type, schema_iter->schema_struct); if (fill_res.has_error()) { switch (fill_res.error()) { - case fill_errc::mismatch: + case fill_errc::invalid_schema: vlog(datalake_log.warn, "Type mismatch with table {}", table_id); return errc::not_supported; - case fill_errc::incomplete: + case fill_errc::schema_evolution_needed: return false; } } diff --git a/src/v/datalake/tests/BUILD b/src/v/datalake/tests/BUILD index f3ec6640bb19..9592a841175c 100644 --- a/src/v/datalake/tests/BUILD +++ b/src/v/datalake/tests/BUILD @@ -144,6 +144,7 @@ redpanda_cc_gtest( "//src/v/cloud_io/tests:scoped_remote", "//src/v/cloud_storage/tests:s3_imposter_gtest", "//src/v/datalake:catalog_schema_manager", + "//src/v/iceberg:datatypes", "//src/v/iceberg:field_collecting_visitor", "//src/v/iceberg:filesystem_catalog", "//src/v/iceberg:table_identifier", diff --git a/src/v/datalake/tests/catalog_schema_manager_test.cc b/src/v/datalake/tests/catalog_schema_manager_test.cc index bf5e378ad91e..bc77e322d248 100644 --- a/src/v/datalake/tests/catalog_schema_manager_test.cc +++ b/src/v/datalake/tests/catalog_schema_manager_test.cc @@ -11,6 +11,7 @@ #include "cloud_io/tests/scoped_remote.h" #include "cloud_storage/tests/s3_imposter.h" #include "datalake/catalog_schema_manager.h" +#include "iceberg/datatypes.h" #include "iceberg/field_collecting_visitor.h" #include "iceberg/filesystem_catalog.h" #include "iceberg/table_identifier.h" @@ -272,3 +273,63 @@ TEST_F(CatalogSchemaManagerTest, TestTypeMismatch) { ASSERT_TRUE(res.has_error()); EXPECT_EQ(res.error(), schema_manager::errc::not_supported); } + +TEST_F(CatalogSchemaManagerTest, AcceptsValidTypePromotion) { + auto original_type = std::get(test_nested_schema_type()); + original_type.fields.emplace_back( + nested_field::create(18, "some_field", field_required::yes, int_type{})); + create_table(original_type); + + auto type = original_type.copy(); + // int->long is a valid primitive type promotion + type.fields.back()->type = long_type{}; + reset_field_ids(type); + + // so schema_mgr should accept the new schema + auto ensure_res + = schema_mgr.ensure_table_schema(model::topic{"foo"}, type).get(); + ASSERT_FALSE(ensure_res.has_error()) << ensure_res.error(); + + auto fill_res + = schema_mgr.get_registered_ids(model::topic{"foo"}, type).get(); + ASSERT_FALSE(fill_res.has_error()) << fill_res.error(); + + auto loaded_table = load_table_schema(table_ident).get(); + ASSERT_TRUE(loaded_table.has_value()); + ASSERT_EQ(loaded_table.value().schema_struct, type); +} + +TEST_F(CatalogSchemaManagerTest, RejectsInvalidTypePromotion) { + auto original_type = std::get(test_nested_schema_type()); + original_type.fields.emplace_back( + nested_field::create(18, "some_field", field_required::yes, int_type{})); + create_table(original_type); + + auto type = original_type.copy(); + // int->string is not a valid primitive type promotion + type.fields.back()->type = string_type{}; + reset_field_ids(type); + + // so schema_mgr should reject the new schema + auto ensure_res + = schema_mgr.ensure_table_schema(model::topic{"foo"}, type).get(); + ASSERT_TRUE(ensure_res.has_error()); + EXPECT_EQ(ensure_res.error(), schema_manager::errc::not_supported) + << ensure_res.error(); + + auto fill_res + = schema_mgr.get_registered_ids(model::topic{"foo"}, type).get(); + ASSERT_TRUE(fill_res.has_error()); + EXPECT_EQ(fill_res.error(), schema_manager::errc::not_supported) + << fill_res.error(); + + // check that the table still holds the original schema + reset_field_ids(original_type); + fill_res + = schema_mgr.get_registered_ids(model::topic{"foo"}, original_type).get(); + ASSERT_FALSE(fill_res.has_error()) << fill_res.error(); + + auto loaded_table = load_table_schema(table_ident).get(); + ASSERT_TRUE(loaded_table.has_value()); + ASSERT_EQ(loaded_table.value().schema_struct, original_type); +} From 63dec6a1ec7c45a91137149913f3e7eb51c64652 Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Mon, 9 Dec 2024 12:58:43 +0000 Subject: [PATCH 214/229] bazel: add target for fuzzing testing A new has_sanitizers build flag is added. This flag needs to be set to show presence of sanitizers. Added extra global linking argument "-stdlib=libc++". Without this, "libfuzzer.a", which is included by adding "-fsanitizer=fuzzer", introduces a dependency on "stdlibc++.so". This ends up with both "libc++.a" and "stdlibc++.so" being linked. Added a config target to include debug symbols. Added a test target for fuzz testing. Fuzz tests are only available when the has_sanitizers flag is set. --- .bazelrc | 2 ++ bazel/BUILD | 13 ++++++++++ bazel/test.bzl | 67 ++++++++++++++++++++++++++++++++++++++++++++++++++ 3 files changed, 82 insertions(+) diff --git a/.bazelrc b/.bazelrc index e2f60eff9f58..5cabd87c0e54 100644 --- a/.bazelrc +++ b/.bazelrc @@ -4,6 +4,7 @@ common --extra_toolchains=@llvm_18_toolchain//:all common --@toolchains_llvm//toolchain/config:libunwind=False common --@toolchains_llvm//toolchain/config:compiler-rt=False build --linkopt --unwindlib=libgcc +build --linkopt -stdlib=libc++ common:clang-19 --extra_toolchains=@llvm_19_toolchain//:all @@ -47,6 +48,7 @@ build:sanitizer --linkopt -fsanitize=address,undefined,vptr,function build:sanitizer --linkopt --rtlib=compiler-rt build:sanitizer --linkopt -fsanitize-link-c++-runtime build:sanitizer --copt -O1 +build:sanitizer --//bazel:has_sanitizers=True # seastar has to be run with system allocator when using sanitizers build:sanitizer --@seastar//:system_allocator=True # krb5 is a foreign cc build and needs explicit list of sanitizers to build the shared library diff --git a/bazel/BUILD b/bazel/BUILD index e69de29bb2d1..9b4763c5e1b8 100644 --- a/bazel/BUILD +++ b/bazel/BUILD @@ -0,0 +1,13 @@ +load("@bazel_skylib//rules:common_settings.bzl", "bool_flag") + +bool_flag( + name = "has_sanitizers", + build_setting_default = False, +) + +config_setting( + name = "enable_fuzz_testing", + flag_values = { + ":has_sanitizers": "true", + }, +) diff --git a/bazel/test.bzl b/bazel/test.bzl index 7af017422c8b..99fa03cc2231 100644 --- a/bazel/test.bzl +++ b/bazel/test.bzl @@ -130,6 +130,53 @@ def _redpanda_cc_test( local_defines = local_defines, ) +def _redpanda_cc_fuzz_test( + name, + timeout, + srcs = [], + defines = [], + deps = [], + custom_args = [], + env = {}, + data = []): + """ + Helper to define a Redpanda C++ fuzzing test. + + Args: + name: name of the test + timeout: same as native cc_test + srcs: test source files + defines: definitions of object-like macros + deps: test dependencies + custom_args: arguments from cc_test users + env: environment variables + data: data file dependencies + """ + native.cc_test( + name = name, + timeout = timeout, + srcs = srcs, + defines = defines, + deps = deps, + copts = redpanda_copts(), + args = custom_args, + features = [ + "layering_check", + ], + tags = [ + "fuzz", + ], + env = env, + data = data, + linkopts = [ + "-fsanitize=fuzzer", + ], + target_compatible_with = select({ + "//bazel:enable_fuzz_testing": [], + "//conditions:default": ["@platforms//:incompatible"], + }), + ) + def _redpanda_cc_unit_test(cpu, memory, **kwargs): extra_args = [ "--unsafe-bypass-fsync 1", @@ -231,6 +278,26 @@ def redpanda_cc_bench( ], ) +def redpanda_cc_fuzz_test( + name, + timeout, + srcs = [], + defines = [], + deps = [], + args = [], + env = {}, + data = []): + _redpanda_cc_fuzz_test( + data = data, + env = env, + name = name, + timeout = timeout, + srcs = srcs, + defines = defines, + deps = deps, + custom_args = args, + ) + def redpanda_cc_btest_no_seastar( name, timeout, From 2992a1a6fdff8b93a1d4e7b980a7c51606f4775e Mon Sep 17 00:00:00 2001 From: Ioannis Kavvadias Date: Tue, 26 Nov 2024 16:48:51 +0000 Subject: [PATCH 215/229] bytes: bazelize iobuf_fuzz --- src/v/bytes/tests/BUILD | 17 ++++++++++++++++- src/v/bytes/tests/iobuf_fuzz.cc | 4 ++-- 2 files changed, 18 insertions(+), 3 deletions(-) diff --git a/src/v/bytes/tests/BUILD b/src/v/bytes/tests/BUILD index d4a3869cc5ff..80ade1d1558b 100644 --- a/src/v/bytes/tests/BUILD +++ b/src/v/bytes/tests/BUILD @@ -1,4 +1,4 @@ -load("//bazel:test.bzl", "redpanda_cc_btest", "redpanda_cc_gtest") +load("//bazel:test.bzl", "redpanda_cc_btest", "redpanda_cc_fuzz_test", "redpanda_cc_gtest") redpanda_cc_btest( name = "iobuf_test", @@ -76,3 +76,18 @@ redpanda_cc_btest( "@seastar//:testing", ], ) + +redpanda_cc_fuzz_test( + name = "iobuf_fuzz", + timeout = "short", + srcs = ["iobuf_fuzz.cc"], + args = [ + "-max_total_time=30", + "-rss_limit_mb=8192", + ], + deps = [ + "//src/v/base", + "//src/v/bytes:iobuf", + "//src/v/bytes:scattered_message", + ], +) diff --git a/src/v/bytes/tests/iobuf_fuzz.cc b/src/v/bytes/tests/iobuf_fuzz.cc index 6b9ccaa6ed01..0dd492495558 100644 --- a/src/v/bytes/tests/iobuf_fuzz.cc +++ b/src/v/bytes/tests/iobuf_fuzz.cc @@ -17,10 +17,10 @@ * -format=html ../src/v/bytes/iobuf.h ../src/v/bytes/iobuf.cc > cov.html */ #include "base/vassert.h" -#include "bytes/bytes.h" #include "bytes/iobuf.h" #include "bytes/scattered_message.h" +#include #include /* @@ -508,7 +508,7 @@ class driver { template T read() { - if (std::distance(pc_, program_.cend()) < sizeof(T)) { + if (std::cmp_less(std::distance(pc_, program_.cend()), sizeof(T))) { throw end_of_program(); } T ret; From c4afa35fc8bf3c4f350905b1d2ceff14cbb9dafb Mon Sep 17 00:00:00 2001 From: Michael Boquard Date: Mon, 9 Dec 2024 16:23:48 -0500 Subject: [PATCH 216/229] dt: Permitted selection of SASL algorithm with kafka cli Signed-off-by: Michael Boquard --- tests/rptest/clients/kafka_cli_tools.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/rptest/clients/kafka_cli_tools.py b/tests/rptest/clients/kafka_cli_tools.py index 3f6ae9884f80..01bc0d13e5a3 100644 --- a/tests/rptest/clients/kafka_cli_tools.py +++ b/tests/rptest/clients/kafka_cli_tools.py @@ -72,6 +72,7 @@ def __init__(self, version: str | None = None, user: str | None = None, passwd: str | None = None, + algorithm: str | None = 'SCRAM-SHA-256', protocol: str = 'SASL_PLAINTEXT', oauth_cfg: OAuthConfig | None = None): self._redpanda = redpanda @@ -94,7 +95,7 @@ def __init__(self, if user: security = security.override(user, passwd, - 'SCRAM-SHA-256', + algorithm, tls_enabled=None) if sasl := security.simple_credentials(): From 7e072e3ac6a860f331f3be7a9d548d2253faefea Mon Sep 17 00:00:00 2001 From: Michael Boquard Date: Mon, 9 Dec 2024 09:22:52 -0500 Subject: [PATCH 217/229] dt: Added test to exercise SASL/PLAIN using various clients Added SASL/PLAIN verification tests for: * KCL * RPK * Python RdKafka * Kafka CLI tools Validated that both SCRAM-256 and SCRAM-512 users worked. Fixes: CORE-8458 Fixes: CORE-8459 Fixes: CORE-8460 Tested that a log message appears when SASL/PLAIN is enabled and that PLAIN cannot be the only SASL mechanism in the mechanisms list. Signed-off-by: Michael Boquard --- tests/rptest/tests/scram_test.py | 278 ++++++++++++++++++++++++++++++- 1 file changed, 274 insertions(+), 4 deletions(-) diff --git a/tests/rptest/tests/scram_test.py b/tests/rptest/tests/scram_test.py index 5ea97d254c5d..5563bfff8fed 100644 --- a/tests/rptest/tests/scram_test.py +++ b/tests/rptest/tests/scram_test.py @@ -6,28 +6,39 @@ # As of the Change Date specified in that file, in accordance with # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0 +from enum import IntEnum +import json import random import socket import string import requests from requests.exceptions import HTTPError +import socket import time import urllib.parse import re -from ducktape.mark import parametrize +from confluent_kafka import KafkaException, KafkaError + +from ducktape.cluster.cluster import ClusterNode +from ducktape.mark import parametrize, matrix from ducktape.utils.util import wait_until from ducktape.errors import TimeoutError +from ducktape.services.service import Service from rptest.services.cluster import cluster from rptest.tests.redpanda_test import RedpandaTest +from rptest.clients.kcl import RawKCL +from rptest.clients.kafka_cli_tools import KafkaCliTools, KafkaCliToolsError from rptest.clients.types import TopicSpec -from rptest.clients.rpk import RpkTool +from rptest.clients.rpk import RpkTool, RpkException from rptest.clients.python_librdkafka import PythonLibrdkafka from rptest.services.admin import Admin -from rptest.services.redpanda import SecurityConfig, SaslCredentials, SecurityConfig +from rptest.services.redpanda import SecurityConfig, SaslCredentials, SecurityConfig, TLSProvider +from rptest.services.tls import Certificate, CertificateAuthority, TLSCertManager from rptest.tests.sasl_reauth_test import get_sasl_metrics, REAUTH_METRIC, EXPIRATION_METRIC from rptest.util import expect_http_error +from rptest.utils.log_utils import wait_until_nag_is_set from rptest.utils.utf8 import CONTROL_CHARS, CONTROL_CHARS_MAP, generate_string_with_control_character @@ -102,9 +113,12 @@ def gen(length): return password - def make_superuser_client(self, password_override=None): + def make_superuser_client(self, + password_override=None, + algorithm_override=None): username, password, algorithm = self.redpanda.SUPERUSER_CREDENTIALS password = password_override or password + algorithm = algorithm_override or algorithm return PythonLibrdkafka(self.redpanda, username=username, password=password, @@ -294,6 +308,262 @@ def test_scram(self): assert username in users +class SaslPlainTest(BaseScramTest): + """ + These tests validate the functionality of the SASL/PLAIN + authentication mechanism. + """ + class ClientType(IntEnum): + KCL = 1 + RPK = 2 + PYTHON_RDKAFKA = 3 + KCLI = 4 + + class ScramType(IntEnum): + SCRAM_SHA_256 = 1 + SCRAM_SHA_512 = 2 + + def __str__(self): + return self.name.replace("_", "-") + + def __init__(self, test_context): + security = SecurityConfig() + security.enable_sasl = True + super(SaslPlainTest, + self).__init__(test_context, + num_brokers=3, + security=security, + extra_node_conf={'developer_mode': True}) + + def _enable_plain_authn(self): + self.logger.debug("Enabling SASL PLAIN and disabling SCRAM") + admin = Admin(self.redpanda) + # PLAIN cannot be on by itself, so we will enable OAUTHBEARER as well + # but keep SCRAM disabled to ensure we are validating the PLAIN authentication + # mechanism + admin.patch_cluster_config( + upsert={'sasl_mechanisms': ['PLAIN', 'OAUTHBEARER']}) + + def _make_client( + self, + client_type: ClientType, + username_override: str | None = None, + password_override: str | None = None, + algorithm_override: str | None = None + ) -> PythonLibrdkafka | RawKCL | RpkTool | KafkaCliTools: + username, password, algorithm = self.redpanda.SUPERUSER_CREDENTIALS + username = username_override or username + password = password_override or password + algorithm = algorithm_override or algorithm + + if client_type == self.ClientType.PYTHON_RDKAFKA: + return PythonLibrdkafka(self.redpanda, + username=username, + password=password, + algorithm=algorithm) + elif client_type == self.ClientType.KCL: + return RawKCL(self.redpanda, + username=username, + password=password, + sasl_mechanism=algorithm) + elif client_type == self.ClientType.RPK: + return RpkTool(self.redpanda, + username=username, + password=password, + sasl_mechanism=algorithm) + elif client_type == self.ClientType.KCLI: + return KafkaCliTools(self.redpanda, + user=username, + passwd=password, + algorithm=algorithm) + else: + assert False, f'Unknown client type: {client_type}' + + def _make_topic(self, client: PythonLibrdkafka | RawKCL | RpkTool + | KafkaCliTools, expect_success: bool) -> TopicSpec: + topic_name = "test-topic" + topic = TopicSpec(name=topic_name) + try: + if isinstance(client, PythonLibrdkafka): + client.create_topic(topic) + elif isinstance(client, RawKCL): + resp = client.create_topics(6, [{"name": topic_name}]) + self.logger.info(f"RESP: {resp}") + if expect_success: + assert len( + resp + ) != 0, "Should have received response with SASL/PLAIN enabled" + assert resp[0][ + 'ErrorCode'] == 0, f"Expected error code 0, got {resp[0]['ErrorCode']}" + return + else: + assert len( + resp + ) == 0, "Should not have received response with SASL/PLAIN disabled" + return + elif isinstance(client, RpkTool): + client.create_topic(topic=topic_name) + elif isinstance(client, KafkaCliTools): + client.create_topic(topic) + else: + assert False, f'Unknown client type: {client} ({type(client)})' + assert expect_success, "Should have failed with SASL/PLAIN disabled" + except RpkException as e: + assert isinstance( + client, RpkTool + ), f"Should not have received an RPK exception from {client} ({type(client)})" + assert not expect_success, f"Should not have failed with SASL/PLAIN enabled: {e}" + assert "UNSUPPORTED_SASL_MECHANISM" in str( + e), f"Expected UNSUPPORTED_SASL_MECHANISM, got {e}" + except KafkaException as e: + assert isinstance( + client, PythonLibrdkafka + ), f"Should not have received a KafkaException from {client} ({type(client)})" + assert not expect_success, f"Should not have failed with SASL/PLAIN enabled: {e}" + assert e.args[0].code( + ) == KafkaError._TIMED_OUT, f"Expected KafkaError._TIMED_OUT, got {e.args[0].code()}" + except KafkaCliToolsError as e: + assert isinstance( + client, KafkaCliTools + ), f"Should not have received a KafkaCliToolsError from {client} ({type(client)})" + assert not expect_success, f"Should not have failed with SASL/PLAIN enabled: {e}" + assert "UnsupportedSaslMechanismException" in str( + e + ), f"Expected to see UnsupportedSaslMechanismException, got {e}" + + @cluster(num_nodes=3) + @matrix(client_type=list(ClientType), + scram_type=list(ScramType), + sasl_plain_enabled=[True, False]) + def test_plain_authn(self, client_type, scram_type, sasl_plain_enabled): + """ + This test validates that SASL/PLAIN works with common kafka client + libraries: + - Python librdkafka + - Raw KCL + - RPK + - Kafka CLI tools + + This test will validate that SASL/PLAIN works with both SCRAM-SHA-256 + and SCRAM-SHA-512 users. + """ + username = "test-user" + password = "test-password" + RpkTool(self.redpanda, + username=self.redpanda.SUPERUSER_CREDENTIALS.username, + password=self.redpanda.SUPERUSER_CREDENTIALS.password, + sasl_mechanism=self.redpanda.SUPERUSER_CREDENTIALS.algorithm + ).sasl_allow_principal(principal=username, + operations=["all"], + resource="topic", + resource_name="*") + self.create_user(username=username, + algorithm=str(scram_type), + password=password) + + if sasl_plain_enabled: + self._enable_plain_authn() + + client = self._make_client(client_type, + username_override=username, + password_override=password, + algorithm_override="PLAIN") + self._make_topic(client, sasl_plain_enabled) + + +class SaslPlainTLSProvider(TLSProvider): + def __init__(self, tls: TLSCertManager): + self._tls = tls + + @property + def ca(self) -> CertificateAuthority: + return self._tls.ca + + def create_broker_cert(self, service: Service, + node: ClusterNode) -> Certificate: + assert node in service.nodes + return self._tls.create_cert(node.name) + + def create_service_client_cert(self, _: Service, name: str) -> Certificate: + return self._tls.create_cert(socket.gethostname(), name=name) + + +class SaslPlainConfigTest(BaseScramTest): + """ + These tests verify the behavior of Redpanda in different + configurations with SASL/PLAIN enabled + """ + + LICENSE_CHECK_INTERVAL_SEC = 1 + + def __init__(self, test_context): + self.security = SecurityConfig() + self.security.enable_sasl = True + super(SaslPlainConfigTest, self).__init__(test_context, + num_brokers=3, + security=self.security) + self.redpanda.set_environment({ + '__REDPANDA_PERIODIC_REMINDER_INTERVAL_SEC': + f'{self.LICENSE_CHECK_INTERVAL_SEC}' + }) + self.tls = TLSCertManager(self.logger) + + def setUp(self): + pass + + def _start_cluster(self, enable_tls: bool): + if enable_tls: + self.security.tls_provider = SaslPlainTLSProvider(tls=self.tls) + self.redpanda.set_security_settings(self.security) + super().setUp() + + @cluster(num_nodes=3) + def test_cannot_enable_only_plain(self): + """ + This test verifies that a user cannot select PLAIN as the only + sasl_mechanism + """ + self._start_cluster(enable_tls=False) + admin = Admin(self.redpanda) + try: + admin.patch_cluster_config(upsert={'sasl_mechanisms': ['PLAIN']}) + assert False, "Should not be able to enable only PLAIN" + except HTTPError as e: + assert e.response.status_code == 400, f"Expected 400, got {e.response.status_code}" + response = json.loads(e.response.text) + assert 'sasl_mechanisms' in response, f'Response missing "sasl_mechanisms": {response}' + assert "When PLAIN is enabled, at least one other mechanism must be enabled" == response[ + 'sasl_mechanisms'], f"Invalid message in response: {response['sasl_mechanisms']}" + + @cluster(num_nodes=3, log_allow_list=[re.compile('SASL/PLAIN is enabled')]) + @parametrize(enable_tls=True) + @parametrize(enable_tls=False) + def test_sasl_plain_log(self, enable_tls: bool): + """ + This test verifies that a log message is emitted when SASL/PLAIN is enabled + """ + self._start_cluster(enable_tls=enable_tls) + wait_until_nag_is_set( + redpanda=self.redpanda, + check_interval_sec=self.LICENSE_CHECK_INTERVAL_SEC) + admin = Admin(self.redpanda) + admin.patch_cluster_config( + upsert={'sasl_mechanisms': ['SCRAM', 'PLAIN']}) + + self.logger.debug("Waiting for SASL/PLAIN message") + + def has_sasl_plain_log(): + # There is always at least one Kafka API with TLS disabled meaning + # this will always log at the error level + return self.redpanda.search_log_all( + r"^ERROR.*SASL/PLAIN is enabled\. This is insecure and not recommended for production\.$" + ) + + wait_until(has_sasl_plain_log, + timeout_sec=self.LICENSE_CHECK_INTERVAL_SEC * 2, + err_msg="Failed to find SASL/PLAIN log message") + + class ScramLiveUpdateTest(RedpandaTest): def __init__(self, test_context): super(ScramLiveUpdateTest, self).__init__(test_context, num_brokers=1) From b2d8fa9fe7c984e1b9cb53026427761b72466900 Mon Sep 17 00:00:00 2001 From: Michael Boquard Date: Tue, 10 Dec 2024 21:04:04 -0500 Subject: [PATCH 218/229] security/test: Added unit tests for plain authenticator Signed-off-by: Michael Boquard --- src/v/security/tests/BUILD | 16 +- src/v/security/tests/CMakeLists.txt | 13 + .../tests/plain_authenticator_test.cc | 225 ++++++++++++++++++ 3 files changed, 253 insertions(+), 1 deletion(-) create mode 100644 src/v/security/tests/plain_authenticator_test.cc diff --git a/src/v/security/tests/BUILD b/src/v/security/tests/BUILD index 895f830a48a2..2fabe6c06070 100644 --- a/src/v/security/tests/BUILD +++ b/src/v/security/tests/BUILD @@ -1,4 +1,4 @@ -load("//bazel:test.bzl", "redpanda_cc_bench", "redpanda_cc_btest", "redpanda_test_cc_library") +load("//bazel:test.bzl", "redpanda_cc_bench", "redpanda_cc_btest", "redpanda_cc_gtest", "redpanda_test_cc_library") redpanda_test_cc_library( name = "utils", @@ -30,6 +30,20 @@ redpanda_cc_btest( ], ) +redpanda_cc_gtest( + name = "plain_authenticator_test", + timeout = "short", + srcs = [ + "plain_authenticator_test.cc", + ], + deps = [ + "//src/v/security", + "//src/v/test_utils:gtest", + "@googletest//:gtest", + "@seastar", + ], +) + redpanda_cc_btest( name = "credential_store_test", timeout = "short", diff --git a/src/v/security/tests/CMakeLists.txt b/src/v/security/tests/CMakeLists.txt index e1e28568d7d8..ecf41dee1e84 100644 --- a/src/v/security/tests/CMakeLists.txt +++ b/src/v/security/tests/CMakeLists.txt @@ -17,6 +17,19 @@ rp_test( LABELS kafka ) +rp_test( + UNIT_TEST + GTEST + BINARY_NAME test_authenticators + SOURCES + plain_authenticator_test.cc + LIBRARIES + v::security + v::gtest_main + LABELS kafka + ARGS "-- -c 1" +) + rp_test( UNIT_TEST BINARY_NAME test_kafka_security_single_thread diff --git a/src/v/security/tests/plain_authenticator_test.cc b/src/v/security/tests/plain_authenticator_test.cc new file mode 100644 index 000000000000..9acee7a7b650 --- /dev/null +++ b/src/v/security/tests/plain_authenticator_test.cc @@ -0,0 +1,225 @@ +// Copyright 2024 Redpanda Data, Inc. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.md +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0 +#include "security/acl.h" +#include "security/credential_store.h" +#include "security/errc.h" +#include "security/plain_authenticator.h" +#include "security/scram_algorithm.h" +#include "security/scram_credential.h" +#include "test_utils/test.h" + +class sasl_plain_test_fixture : public seastar_test { +public: + ss::future<> SetUpAsync() override { + authn = std::make_unique(store); + ASSERT_FALSE_CORO(authn->complete()); + ASSERT_FALSE_CORO(authn->failed()); + } + + ss::future<> TearDownAsync() override { + authn.reset(); + co_return; + } + + security::credential_store store; + std::unique_ptr authn; + + enum class scram_type_t { + scram_sha256, + scram_sha512, + }; + + security::scram_credential make_scram_credential( + scram_type_t scram_type, + const ss::sstring& username, + const ss::sstring& password) { + auto iterations = [scram_type]() { + if (scram_type == scram_type_t::scram_sha256) { + return security::scram_sha256::min_iterations; + } + return security::scram_sha512::min_iterations; + }(); + + security::acl_principal principal( + security::principal_type::user, username); + + return scram_type == scram_type_t::scram_sha256 + ? security::scram_sha256::make_credentials( + std::move(principal), password, iterations) + : security::scram_sha512::make_credentials( + std::move(principal), password, iterations); + } + + bytes create_authn_message( + const ss::sstring& username, const ss::sstring& password) { + bytes rv; + rv.push_back('\0'); + std::ranges::move(bytes::from_string(username), std::back_inserter(rv)); + rv.push_back('\0'); + std::ranges::move(bytes::from_string(password), std::back_inserter(rv)); + return rv; + } +}; + +TEST_F_CORO(sasl_plain_test_fixture, test_success_scram256) { + ss::sstring username("user"); + ss::sstring password("password"); + + auto creds = make_scram_credential( + scram_type_t::scram_sha256, username, password); + store.put(security::credential_user{username}, creds); + + auto res = co_await authn->authenticate( + create_authn_message(username, password)); + ASSERT_TRUE_CORO(res.has_value()) + << "Error during authenticate: " << res.assume_error(); + EXPECT_TRUE(authn->complete()); + EXPECT_FALSE(authn->failed()); + EXPECT_EQ(authn->principal().name(), username); +} + +TEST_F_CORO(sasl_plain_test_fixture, test_success_scram512) { + ss::sstring username("user"); + ss::sstring password("password"); + + auto creds = make_scram_credential( + scram_type_t::scram_sha512, username, password); + store.put(security::credential_user{username}, creds); + + auto res = co_await authn->authenticate( + create_authn_message(username, password)); + ASSERT_TRUE_CORO(res.has_value()) + << "Error during authenticate: " << res.assume_error(); + EXPECT_TRUE(authn->complete()); + EXPECT_FALSE(authn->failed()); + EXPECT_EQ(authn->principal().name(), username); +} + +TEST_F_CORO(sasl_plain_test_fixture, test_invalid_password) { + ss::sstring username("user"); + ss::sstring password("password"); + ss::sstring wrong_password("wrong_password"); + auto creds = make_scram_credential( + scram_type_t::scram_sha256, username, password); + store.put(security::credential_user{username}, creds); + + auto res = co_await authn->authenticate( + create_authn_message(username, wrong_password)); + ASSERT_FALSE_CORO(res.has_value()) << "Should not have authenticated"; + EXPECT_EQ(res.assume_error(), security::errc::invalid_credentials); + EXPECT_FALSE(authn->complete()); + EXPECT_TRUE(authn->failed()); +} + +TEST_F_CORO(sasl_plain_test_fixture, test_invalid_user) { + ss::sstring username("user"); + ss::sstring password("password"); + ss::sstring wrong_username("wrong_user"); + + auto creds = make_scram_credential( + scram_type_t::scram_sha256, username, password); + store.put(security::credential_user{username}, creds); + + auto res = co_await authn->authenticate( + create_authn_message(wrong_username, password)); + ASSERT_FALSE_CORO(res.has_value()) << "Should not have authenticated"; + EXPECT_EQ(res.assume_error(), security::errc::invalid_credentials); + EXPECT_FALSE(authn->complete()); + EXPECT_TRUE(authn->failed()); +} + +TEST_F_CORO(sasl_plain_test_fixture, test_resuse_authn) { + ss::sstring username("user"); + ss::sstring password("password"); + + auto creds = make_scram_credential( + scram_type_t::scram_sha256, username, password); + store.put(security::credential_user{username}, creds); + { + auto res = co_await authn->authenticate( + create_authn_message(username, password)); + ASSERT_TRUE_CORO(res.has_value()) + << "Error during authenticate: " << res.assume_error(); + EXPECT_TRUE(authn->complete()); + EXPECT_FALSE(authn->failed()); + } + + { + auto res = co_await authn->authenticate( + create_authn_message(username, password)); + ASSERT_FALSE_CORO(res.has_value()) << "Should not have authenticated"; + EXPECT_EQ(res.assume_error(), security::errc::invalid_credentials); + EXPECT_FALSE(authn->complete()); + EXPECT_TRUE(authn->failed()); + } +} + +TEST_F_CORO(sasl_plain_test_fixture, test_authz_id) { + ss::sstring username("user"); + ss::sstring password("password"); + auto authn_msg = bytes::from_string("authz"); + auto authn_msg_tmp = create_authn_message(username, password); + std::ranges::move(authn_msg_tmp, std::back_inserter(authn_msg)); + + auto res = co_await authn->authenticate(authn_msg); + ASSERT_FALSE_CORO(res.has_value()) << "Should not have authenticated"; + EXPECT_EQ(res.assume_error(), security::errc::invalid_credentials); + EXPECT_FALSE(authn->complete()); + EXPECT_TRUE(authn->failed()); +} + +TEST_F_CORO(sasl_plain_test_fixture, no_username) { + ss::sstring password("password"); + auto res = co_await authn->authenticate(create_authn_message("", password)); + ASSERT_FALSE_CORO(res.has_value()) << "Should not have authenticated"; + EXPECT_EQ(res.assume_error(), security::errc::invalid_credentials); + EXPECT_FALSE(authn->complete()); + EXPECT_TRUE(authn->failed()); +} + +TEST_F_CORO(sasl_plain_test_fixture, no_password) { + ss::sstring username("user"); + auto res = co_await authn->authenticate(create_authn_message(username, "")); + ASSERT_FALSE_CORO(res.has_value()) << "Should not have authenticated"; + EXPECT_EQ(res.assume_error(), security::errc::invalid_credentials); + EXPECT_FALSE(authn->complete()); + EXPECT_TRUE(authn->failed()); +} + +TEST_F_CORO(sasl_plain_test_fixture, massive_username) { + ss::sstring username(1024, 'a'); + ss::sstring password("password"); + + auto creds = make_scram_credential( + scram_type_t::scram_sha256, username, password); + store.put(security::credential_user{username}, creds); + + auto res = co_await authn->authenticate( + create_authn_message(username, password)); + ASSERT_FALSE_CORO(res.has_value()) << "Should not have authenticated"; + EXPECT_EQ(res.assume_error(), security::errc::invalid_credentials); + EXPECT_FALSE(authn->complete()); + EXPECT_TRUE(authn->failed()); +} + +TEST_F_CORO(sasl_plain_test_fixture, massive_password) { + ss::sstring username("user"); + ss::sstring password(1024, 'a'); + + auto creds = make_scram_credential( + scram_type_t::scram_sha256, username, password); + store.put(security::credential_user{username}, creds); + + auto res = co_await authn->authenticate( + create_authn_message(username, password)); + ASSERT_FALSE_CORO(res.has_value()) << "Should not have authenticated"; + EXPECT_EQ(res.assume_error(), security::errc::invalid_credentials); + EXPECT_FALSE(authn->complete()); + EXPECT_TRUE(authn->failed()); +} From 672e64a32981d831f07f7dfffd0f3a40db19ce18 Mon Sep 17 00:00:00 2001 From: Oren Leiman Date: Wed, 18 Dec 2024 08:29:09 -0800 Subject: [PATCH 219/229] dt/datalake: Basic integration test for schema evolution Uses RPCN + filesystem catalog to check that a simple primitive type promotion behaves as expected, end-to-end. Signed-off-by: Oren Leiman --- .../tests/datalake/schema_evolution_test.py | 229 ++++++++++++++++++ 1 file changed, 229 insertions(+) create mode 100644 tests/rptest/tests/datalake/schema_evolution_test.py diff --git a/tests/rptest/tests/datalake/schema_evolution_test.py b/tests/rptest/tests/datalake/schema_evolution_test.py new file mode 100644 index 000000000000..6a6543d3d000 --- /dev/null +++ b/tests/rptest/tests/datalake/schema_evolution_test.py @@ -0,0 +1,229 @@ +# Copyright 2024 Redpanda Data, Inc. +# +# Use of this software is governed by the Business Source License +# included in the file licenses/BSL.md +# +# As of the Change Date specified in that file, in accordance with +# the Business Source License, use of this software will be governed +# by the Apache License, Version 2.0 +import tempfile + +from rptest.clients.rpk import RpkTool +from rptest.services.cluster import cluster +from rptest.services.redpanda import SISettings, SchemaRegistryConfig +from rptest.services.redpanda_connect import RedpandaConnectService +from rptest.tests.datalake.datalake_services import DatalakeServices +from rptest.tests.datalake.datalake_verifier import DatalakeVerifier +from rptest.tests.datalake.query_engine_base import QueryEngineType +from rptest.tests.redpanda_test import RedpandaTest +from rptest.tests.datalake.utils import supported_storage_types +from rptest.util import expect_exception +from ducktape.mark import matrix + +schema_avro = """ +{ + "type": "record", + "name": "VerifierRecord", + "fields": [ + { + "name": "verifier_string", + "type": "string" + }, + { + "name": "ordinal", + "type": "int" + } + ] +} +""" + +legal_evo_schema_avro = """ +{ + "type": "record", + "name": "VerifierRecord", + "fields": [ + { + "name": "verifier_string", + "type": "string" + }, + { + "name": "ordinal", + "type": "long" + }, + { + "name": "another", + "type": "float" + } + ] +} +""" + +legal_evo_schema_avro_2 = """ +{ + "type": "record", + "name": "VerifierRecord", + "fields": [ + { + "name": "verifier_string", + "type": "string" + }, + { + "name": "ordinal", + "type": "long" + } + ] +} +""" + +illegal_evo_schema_avro = """ +{ + "type": "record", + "name": "VerifierRecord", + "fields": [ + { + "name": "verifier_string", + "type": "double" + }, + { + "name": "ordinal", + "type": "long" + } + ] +} +""" + + +class SchemaEvolutionTest(RedpandaTest): + def __init__(self, test_context): + super(SchemaEvolutionTest, self).__init__( + test_context=test_context, + num_brokers=1, + si_settings=SISettings(test_context, + cloud_storage_enable_remote_read=False, + cloud_storage_enable_remote_write=False), + extra_rp_conf={ + "iceberg_enabled": True, + "iceberg_catalog_commit_interval_ms": 5000, + }, + schema_registry_config=SchemaRegistryConfig()) + + def avro_stream_config(self, topic, subject, mapping: str): + return { + "input": { + "generate": { + "mapping": mapping, + "interval": "", + "count": 1000, + "batch_size": 1 + } + }, + "pipeline": { + "processors": [{ + "schema_registry_encode": { + "url": self.redpanda.schema_reg().split(",")[0], + "subject": subject, + "refresh_period": "10s" + } + }] + }, + "output": { + "redpanda": { + "seed_brokers": self.redpanda.brokers_list(), + "topic": topic, + } + } + } + + def setUp(self): + pass + + def _create_schema(self, subject: str, schema: str, schema_type="avro"): + rpk = RpkTool(self.redpanda) + with tempfile.NamedTemporaryFile(suffix=f".{schema_type}") as tf: + tf.write(bytes(schema, 'UTF-8')) + tf.flush() + rpk.create_schema(subject, tf.name) + + @cluster(num_nodes=4, log_allow_list=["Schema \d+ already exists"]) + @matrix( + cloud_storage_type=supported_storage_types(), + query_engine=[ + QueryEngineType.SPARK, + QueryEngineType.TRINO, + ], + ) + def test_evolving_avro_schemas(self, cloud_storage_type, query_engine): + topic_name = "ducky-topic" + with DatalakeServices(self.test_context, + redpanda=self.redpanda, + filesystem_catalog_mode=True, + include_query_engines=[ + query_engine, + ]) as dl: + + dl.create_iceberg_enabled_topic( + topic_name, + partitions=5, + replicas=1, + iceberg_mode="value_schema_id_prefix") + + def get_qe(): + if query_engine == QueryEngineType.SPARK: + return dl.spark() + else: + return dl.trino() + + self._create_schema("schema_avro", schema_avro) + self._create_schema("legal_evo_schema_avro", legal_evo_schema_avro) + self._create_schema("legal_evo_schema_avro_2", + legal_evo_schema_avro_2) + self._create_schema("illegal_evo_schema_avro", + illegal_evo_schema_avro) + + connect = RedpandaConnectService(self.test_context, self.redpanda) + connect.start() + + def run_verifier(schema: str, mapping: str): + + # create verifier + verifier = DatalakeVerifier(self.redpanda, topic_name, + get_qe()) + # create a stream + connect.start_stream(name="ducky_stream", + config=self.avro_stream_config( + topic_name, schema, mapping=mapping)) + + verifier.start() + connect.stop_stream("ducky_stream", wait_to_finish=True) + verifier.wait() + + run_verifier("schema_avro", + mapping=f""" + root.verifier_string = uuid_v4() + root.ordinal = counter() + """) + + # again w/ a new schema (extra field and promoted ordinal) + run_verifier("legal_evo_schema_avro", + mapping=f""" + root.verifier_string = uuid_v4() + root.ordinal = counter() + root.another = 12.0 + """) + + # remove the extra field and do it one more time + run_verifier("legal_evo_schema_avro_2", + mapping=f""" + root.verifier_string = uuid_v4() + root.ordinal = counter() + """) + + # finally once w/ a non-compatible schema, verifier should time out and fail its assertion + with expect_exception( + AssertionError, + lambda e: "Mismatch between maximum offsets" in str(e)): + run_verifier("illegal_evo_schema_avro", + mapping=f""" + root.verifier_string = 23.0 + root.ordinal = counter() + """) From 4e7ad61e82641b804697358eb9895c49f527f880 Mon Sep 17 00:00:00 2001 From: Paulo Borges Date: Thu, 19 Dec 2024 13:59:40 -0300 Subject: [PATCH 220/229] rpk: update topic describe help text --- src/go/rpk/pkg/cli/topic/describe.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/go/rpk/pkg/cli/topic/describe.go b/src/go/rpk/pkg/cli/topic/describe.go index 709439eef302..6ff8cb999fe4 100644 --- a/src/go/rpk/pkg/cli/topic/describe.go +++ b/src/go/rpk/pkg/cli/topic/describe.go @@ -48,8 +48,7 @@ This command prints detailed information about topics. The output contains up to three sections: a summary of the topic, the topic configs, and a detailed partitions section. By default, the summary and configs sections are printed. -Using the --format flag with either JSON or YAML will default in printing -all the topic information (--all). +Using the --format flag with either JSON or YAML prints all the topic information. The --regex flag (-r) parses arguments as regular expressions and describes topics that match any of the expressions. From 026fa9b14ebedaff9b5ca2d5b77a7c7288617489 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Wed, 18 Dec 2024 18:02:26 +0100 Subject: [PATCH 221/229] tests: added validation of offset translator consistency in ducktape MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Added code that validates if deltas are consistent across different replicas. Signed-off-by: Michał Maślanka --- tests/rptest/tests/ot_consistency_test.py | 50 +++++++++++++++++++++++ tests/rptest/utils/node_operations.py | 47 +++++++++++++++++++++ 2 files changed, 97 insertions(+) create mode 100644 tests/rptest/tests/ot_consistency_test.py diff --git a/tests/rptest/tests/ot_consistency_test.py b/tests/rptest/tests/ot_consistency_test.py new file mode 100644 index 000000000000..de9b401a1c7d --- /dev/null +++ b/tests/rptest/tests/ot_consistency_test.py @@ -0,0 +1,50 @@ +# Copyright 2020 Redpanda Data, Inc. +# +# Use of this software is governed by the Business Source License +# included in the file licenses/BSL.md +# +# As of the Change Date specified in that file, in accordance with +# the Business Source License, use of this software will be governed +# by the Apache License, Version 2.0 +import json +import os +import re +import sys +import time +import traceback +from collections import namedtuple, defaultdict +from typing import DefaultDict, List, Optional + +from ducktape.mark import matrix +from ducktape.utils.util import wait_until + +from rptest.clients.kafka_cat import KafkaCat +from rptest.clients.kafka_cli_tools import KafkaCliTools +from rptest.clients.rpk import RpkTool +from rptest.clients.types import TopicSpec +from rptest.services.cluster import cluster +from rptest.services.redpanda import RedpandaService, SISettings, CloudStorageTypeAndUrlStyle, get_cloud_storage_type, get_cloud_storage_type_and_url_style +from rptest.tests.redpanda_test import RedpandaTest +from rptest.utils.node_operations import verify_offset_translator_state_consistent + + +class OffsetTranslatorConsistencyTest(RedpandaTest): + def __init__(self, test_ctx, *args, **kwargs): + self._ctx = test_ctx + super(OffsetTranslatorConsistencyTest, self).__init__( + test_ctx, + si_settings=SISettings(test_ctx, + log_segment_size=1024 * 1024, + fast_uploads=True), + *args, + **kwargs, + ) + + @cluster(num_nodes=3) + def test_offset_translator_state_consistent(self): + cli = KafkaCliTools(self.redpanda) + topic = TopicSpec(partition_count=3, replication_factor=3) + + cli.create_topic(topic) + cli.produce(topic.name, 1000, 100) + verify_offset_translator_state_consistent(self.redpanda) diff --git a/tests/rptest/utils/node_operations.py b/tests/rptest/utils/node_operations.py index 816d2dbe9472..3b00581d1811 100644 --- a/tests/rptest/utils/node_operations.py +++ b/tests/rptest/utils/node_operations.py @@ -10,6 +10,7 @@ from collections import defaultdict from enum import Enum import random +import re import threading import time import requests @@ -20,6 +21,7 @@ from rptest.services.failure_injector import FailureInjector, FailureSpec from rptest.services.redpanda import RedpandaService from rptest.services.redpanda_installer import VERSION_RE, int_tuple +from rptest.util import wait_until_result class OperationType(Enum): @@ -79,6 +81,51 @@ def add(idx): yield NodeOperation(op, idx, random.choice([True, False])) +def verify_offset_translator_state_consistent(redpanda: RedpandaService): + logger = redpanda.logger + last_delta_pattern = re.compile('^\\{.*, last delta: (?P\\d+)\\}$') + admin = Admin(redpanda) + + for n in redpanda.started_nodes(): + node_id = redpanda.node_id(n) + all_partitions = admin.get_partitions(node=n) + + def _state_consistent(ns, topic, partition): + + state = admin.get_partition_state(ns, topic, partition, node=n) + dirty_offset = state['replicas'][0]['dirty_offset'] + if all(r['dirty_offset'] == dirty_offset + for r in state['replicas']): + return True, state + return False, None + + for p in all_partitions: + namespace = p['ns'] + topic = p['topic'] + partition = p['partition_id'] + partition_name = f"{namespace}/{topic}/{partition}" + state = wait_until_result( + lambda: _state_consistent(namespace, topic, partition), + timeout_sec=180, + backoff_sec=1, + err_msg="Error waiting for offsets to be consistent") + + logger.debug( + f"debug state of {partition_name} replica on node {node_id}: {state}" + ) + last_deltas = set() + for r_state in state['replicas']: + ot_state = r_state['raft_state']['offset_translator_state'] + if "empty" in ot_state: + continue + m = last_delta_pattern.match(ot_state) + assert m, f"offset translator state {ot_state} does not match expected pattern" + last_deltas.add(m['delta']) + assert len( + last_deltas + ) <= 1, f"partition {p} has inconsistent offset translation. Last deltas: {last_deltas}" + + class NodeDecommissionWaiter(): def __init__(self, redpanda, From 59eba45a567add9689a2bf5a4d8bbab56660b3e7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Wed, 18 Dec 2024 18:03:55 +0100 Subject: [PATCH 222/229] tests: validate offset translator state in random_node_operations_test MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Michał Maślanka --- tests/rptest/tests/random_node_operations_test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/rptest/tests/random_node_operations_test.py b/tests/rptest/tests/random_node_operations_test.py index c152396bc20a..223672e795c2 100644 --- a/tests/rptest/tests/random_node_operations_test.py +++ b/tests/rptest/tests/random_node_operations_test.py @@ -26,7 +26,7 @@ from rptest.services.redpanda import CHAOS_LOG_ALLOW_LIST, PREV_VERSION_LOG_ALLOW_LIST, CloudStorageType, LoggingConfig, PandaproxyConfig, SISettings, SchemaRegistryConfig, get_cloud_storage_type from rptest.services.redpanda_installer import RedpandaInstaller from rptest.utils.mode_checks import cleanup_on_early_exit, skip_debug_mode, skip_fips_mode -from rptest.utils.node_operations import FailureInjectorBackgroundThread, NodeOpsExecutor, generate_random_workload +from rptest.utils.node_operations import FailureInjectorBackgroundThread, NodeOpsExecutor, generate_random_workload, verify_offset_translator_state_consistent from rptest.clients.offline_log_viewer import OfflineLogViewer from rptest.tests.datalake.utils import supported_storage_types @@ -593,6 +593,7 @@ def cluster_version_updated(): err_msg="Error waiting for cluster to report consistent version" ) + verify_offset_translator_state_consistent(self.redpanda) # Validate that the controller log written during the test is readable by offline log viewer log_viewer = OfflineLogViewer(self.redpanda) for node in self.redpanda.started_nodes(): From 9d885f5a4e93493559d52adacd65041ea6b2b777 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Wed, 18 Dec 2024 18:50:39 +0100 Subject: [PATCH 223/229] tests: made segment size smaller in random_node_operations_test MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Michał Maślanka --- tests/rptest/tests/random_node_operations_test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/rptest/tests/random_node_operations_test.py b/tests/rptest/tests/random_node_operations_test.py index 223672e795c2..8ff8fc9ad924 100644 --- a/tests/rptest/tests/random_node_operations_test.py +++ b/tests/rptest/tests/random_node_operations_test.py @@ -58,6 +58,8 @@ def __init__(self, test_context, *args, **kwargs): # set disk timeout to value greater than max suspend time # not to emit spurious errors "raft_io_timeout_ms": 20000, + "compacted_log_segment_size": 1024 * 1024, + "log_segment_size": 2 * 1024 * 1024, }, # 2 nodes for kgo producer/consumer workloads node_prealloc_count=3, From ff96d021822d74a99550e0487646ef5f1aa70c8e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Fri, 20 Dec 2024 09:02:55 +0100 Subject: [PATCH 224/229] r/consensus: added prev_log_delta to protocol metadata MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Added a `prev_log_delta` field to `raft::protocol_metadata`. The `prev_log_delta` contains an offset translator delta corresponding to metadata `prev_log_index`. Including the delta in metadata will allow follower to detect offset translator state inconsistency. Signed-off-by: Michał Maślanka --- src/v/raft/consensus.cc | 13 ++++++++++++- src/v/raft/consensus.h | 2 ++ src/v/raft/recovery_stm.cc | 4 +++- src/v/raft/types.cc | 6 ++++-- src/v/raft/types.h | 7 +++++-- 5 files changed, 26 insertions(+), 6 deletions(-) diff --git a/src/v/raft/consensus.cc b/src/v/raft/consensus.cc index 0c59f3297695..5b1851f8d1c5 100644 --- a/src/v/raft/consensus.cc +++ b/src/v/raft/consensus.cc @@ -2923,7 +2923,18 @@ protocol_metadata consensus::meta() const { .prev_log_index = lstats.dirty_offset, .prev_log_term = prev_log_term, .last_visible_index = last_visible_index(), - .dirty_offset = lstats.dirty_offset}; + .dirty_offset = lstats.dirty_offset, + .prev_log_delta = get_offset_delta(lstats, lstats.dirty_offset), + }; +} + +model::offset_delta consensus::get_offset_delta( + const storage::offset_stats& lstats, model::offset offset) const { + if (offset < model::offset{0} || offset < lstats.start_offset) { + return model::offset_delta{}; + } + + return _log->offset_delta(offset); } void consensus::update_node_append_timestamp(vnode id) { diff --git a/src/v/raft/consensus.h b/src/v/raft/consensus.h index 08ac762286f1..d68f0b058665 100644 --- a/src/v/raft/consensus.h +++ b/src/v/raft/consensus.h @@ -662,6 +662,8 @@ class consensus { template ss::future change_configuration(Func&&); + model::offset_delta + get_offset_delta(const storage::offset_stats&, model::offset) const; template ss::future interrupt_configuration_change(model::revision_id, Func); diff --git a/src/v/raft/recovery_stm.cc b/src/v/raft/recovery_stm.cc index 4efe0e54c878..4fc184a459b8 100644 --- a/src/v/raft/recovery_stm.cc +++ b/src/v/raft/recovery_stm.cc @@ -567,7 +567,9 @@ ss::future<> recovery_stm::replicate( .prev_log_index = prev_log_idx, .prev_log_term = prev_log_term, .last_visible_index = last_visible_idx, - .dirty_offset = lstats.dirty_offset}, + .dirty_offset = lstats.dirty_offset, + .prev_log_delta = _ptr->get_offset_delta(lstats, prev_log_idx), + }, std::move(reader), range_size, flush); diff --git a/src/v/raft/types.cc b/src/v/raft/types.cc index e5c675e01b9e..45779f0a3958 100644 --- a/src/v/raft/types.cc +++ b/src/v/raft/types.cc @@ -247,14 +247,16 @@ std::ostream& operator<<(std::ostream& o, const protocol_metadata& m) { fmt::print( o, "{{group: {}, commit_index: {}, term: {}, prev_log_index: {}, " - "prev_log_term: {}, last_visible_index: {}, dirty_offset: {}}}", + "prev_log_term: {}, last_visible_index: {}, dirty_offset: {}, " + "prev_log_delta: {}}}", m.group, m.commit_index, m.term, m.prev_log_index, m.prev_log_term, m.last_visible_index, - m.dirty_offset); + m.dirty_offset, + m.prev_log_delta); return o; } diff --git a/src/v/raft/types.h b/src/v/raft/types.h index bf554426e042..89f8f9bde7f3 100644 --- a/src/v/raft/types.h +++ b/src/v/raft/types.h @@ -39,7 +39,7 @@ using election_success = ss::bool_class; struct protocol_metadata : serde:: - envelope, serde::compat_version<0>> { + envelope, serde::compat_version<0>> { group_id group; model::offset commit_index; model::term_id term; @@ -47,6 +47,8 @@ struct protocol_metadata model::term_id prev_log_term; model::offset last_visible_index; model::offset dirty_offset; + // offset delta corresponding to the prev_log_index + model::offset_delta prev_log_delta{}; friend std::ostream& operator<<(std::ostream& o, const protocol_metadata& m); @@ -62,7 +64,8 @@ struct protocol_metadata prev_log_index, prev_log_term, last_visible_index, - dirty_offset); + dirty_offset, + prev_log_delta); } }; From 97d3d4f7b730b2a9ec1588b530bb01dfbf99e0dd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Fri, 20 Dec 2024 09:05:01 +0100 Subject: [PATCH 225/229] r/consensus: validate offset delta in append entries handler MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Added offset translator state validation to append entries handler. The validation doesn't prevent Raft group from operation but emits an error log message which can be easily observed. Signed-off-by: Michał Maślanka --- src/v/raft/consensus.cc | 26 +++++++++++++++++++++++++- src/v/raft/consensus.h | 3 +++ 2 files changed, 28 insertions(+), 1 deletion(-) diff --git a/src/v/raft/consensus.cc b/src/v/raft/consensus.cc index 5b1851f8d1c5..34de3aac5d18 100644 --- a/src/v/raft/consensus.cc +++ b/src/v/raft/consensus.cc @@ -2201,7 +2201,7 @@ consensus::do_append_entries(append_entries_request&& r) { // the leader vote timeout _hbeat = clock_type::now(); }); - + validate_offset_translator_delta(request_metadata, lstats); storage::append_result ofs = co_await disk_append( std::move(r).release_batches(), update_last_quorum_index::no); auto last_visible = std::min( @@ -2250,6 +2250,30 @@ consensus::do_append_entries(append_entries_request&& r) { } } +void consensus::validate_offset_translator_delta( + const protocol_metadata& meta, const storage::offset_stats& lstats) { + // do not validate if prev_log_delta is not set + if (meta.prev_log_delta < model::offset_delta{0}) { + return; + } + /** + * If request contain valid information and it is about to be appended + * to the log validate the offset translator delta consistency. + */ + const auto last_delta = get_offset_delta(lstats, meta.prev_log_index); + if ( + last_delta >= model::offset_delta{0} + && last_delta != meta.prev_log_delta) { + vlog( + _ctxlog.error, + "Offset translator state inconsistency detected. Received " + "append entries request {} with last offset delta different " + "than expected: {}", + meta, + last_delta); + } +} + void consensus::maybe_update_leader(vnode request_node) { if (unlikely(_leader_id != request_node)) { _leader_id = request_node; diff --git a/src/v/raft/consensus.h b/src/v/raft/consensus.h index d68f0b058665..0bf3dfb7dfcd 100644 --- a/src/v/raft/consensus.h +++ b/src/v/raft/consensus.h @@ -787,6 +787,9 @@ class consensus { } void try_updating_configuration_version(group_configuration& cfg); + + void validate_offset_translator_delta( + const protocol_metadata&, const storage::offset_stats& lstats); // args vnode _self; raft::group_id _group; From 3dc54705463c8bcc4a45d74a8809299345127cd9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Fri, 20 Dec 2024 09:21:38 +0100 Subject: [PATCH 226/229] r/probe: added offset translator inconsistency error metric MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Michał Maślanka --- src/v/raft/consensus.cc | 1 + src/v/raft/probe.cc | 6 ++++++ src/v/raft/probe.h | 5 ++++- 3 files changed, 11 insertions(+), 1 deletion(-) diff --git a/src/v/raft/consensus.cc b/src/v/raft/consensus.cc index 34de3aac5d18..28f92308b34f 100644 --- a/src/v/raft/consensus.cc +++ b/src/v/raft/consensus.cc @@ -2271,6 +2271,7 @@ void consensus::validate_offset_translator_delta( "than expected: {}", meta, last_delta); + _probe->offset_translator_inconsistency_error(); } } diff --git a/src/v/raft/probe.cc b/src/v/raft/probe.cc index 3584d4c8fa4f..e3178a658735 100644 --- a/src/v/raft/probe.cc +++ b/src/v/raft/probe.cc @@ -162,6 +162,12 @@ void probe::setup_metrics(const model::ntp& ntp) { [this] { return _full_heartbeat_requests; }, sm::description("Number of full heartbeats sent by the leader"), labels), + sm::make_counter( + "offset_translator_inconsistency_errors", + [this] { return _offset_translator_inconsistency_error; }, + sm::description("Number of append entries requests that failed the " + "offset translator consistency check"), + labels), }, {}, {sm::shard_label, sm::label("partition")}); diff --git a/src/v/raft/probe.h b/src/v/raft/probe.h index ff7f751ee6da..37fa18cc406b 100644 --- a/src/v/raft/probe.h +++ b/src/v/raft/probe.h @@ -63,6 +63,9 @@ class probe { void full_heartbeat() { ++_full_heartbeat_requests; } void lw_heartbeat() { ++_lw_heartbeat_requests; } + void offset_translator_inconsistency_error() { + ++_offset_translator_inconsistency_error; + } void clear() { _metrics.clear(); @@ -89,7 +92,7 @@ class probe { uint64_t _recovery_request_error = 0; uint64_t _full_heartbeat_requests = 0; uint64_t _lw_heartbeat_requests = 0; - + uint64_t _offset_translator_inconsistency_error = 0; metrics::internal_metric_groups _metrics; metrics::public_metric_groups _public_metrics; }; From e7ea155a4aba1d502aae36c12875c696743114ba Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Micha=C5=82=20Ma=C5=9Blanka?= Date: Fri, 20 Dec 2024 11:55:07 +0100 Subject: [PATCH 227/229] r/utils: fixed setting log start delta in snapshot MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit When topic partition is recovered from cloud data Raft protocol is seeded with special snapshot created during recovery. The snapshot may then be used to seed out of date followers. The snapshot content must be valid in terms of offsets and offset translator deltas. Previously the delta set in the snapshot created during recovery was set to 0. When this snapshot is delivered to the follower it will reset the follower offset translation state accordingly. This may lead to inconsistent deltas across replicas. Signed-off-by: Michał Maślanka --- src/v/raft/consensus_utils.cc | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/v/raft/consensus_utils.cc b/src/v/raft/consensus_utils.cc index 597f52c20290..a5fc64f41448 100644 --- a/src/v/raft/consensus_utils.cc +++ b/src/v/raft/consensus_utils.cc @@ -377,7 +377,8 @@ ss::future<> create_raft_state_for_pre_existing_partition( model::offset min_rp_offset, model::offset max_rp_offset, model::term_id last_included_term, - std::vector initial_nodes) { + std::vector initial_nodes, + model::offset_delta log_start_delta) { // Prepare Raft state in kvstore vlog( raftlog.debug, @@ -403,7 +404,7 @@ ss::future<> create_raft_state_for_pre_existing_partition( .version = raft::snapshot_metadata::current_version, .latest_configuration = std::move(group_config), .cluster_time = ss::lowres_clock::now(), - .log_start_delta = raft::offset_translator_delta{0}, + .log_start_delta = offset_translator_delta{log_start_delta}, }; vlog( @@ -458,7 +459,8 @@ ss::future<> bootstrap_pre_existing_partition( min_rp_offset, max_rp_offset, last_included_term, - initial_nodes); + initial_nodes, + model::offset_delta{ot_state->delta(min_rp_offset)}); } } // namespace raft::details From ee7918a301c818ee712463477f85de79563985ed Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Fri, 20 Dec 2024 14:47:58 +0000 Subject: [PATCH 228/229] rptest: disable leader balancer in cache stress test --- tests/rptest/scale_tests/tiered_storage_cache_stress_test.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/rptest/scale_tests/tiered_storage_cache_stress_test.py b/tests/rptest/scale_tests/tiered_storage_cache_stress_test.py index c1102ec25687..e25e0e41c2a9 100644 --- a/tests/rptest/scale_tests/tiered_storage_cache_stress_test.py +++ b/tests/rptest/scale_tests/tiered_storage_cache_stress_test.py @@ -65,6 +65,10 @@ def setUp(self): self.manifest_upload_interval, 'disable_public_metrics': False, 'cloud_storage_cache_check_interval': 500, + # We're consuming from leader node and expect specific changes in + # its local cache. Disable leader balancer so that our expectations + # are valid. + "enable_leader_balancer": False, } self.redpanda.set_extra_rp_conf(extra_rp_conf) From 834d8aa4c0fdb2c045736ae09c4c3ed02aead37c Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Thu, 19 Dec 2024 18:40:44 +0000 Subject: [PATCH 229/229] rptest/s: always cleanup azurite containers to avoid slowdown In https://github.com/redpanda-data/redpanda/pull/24550 a test was enabled to also run with Azure/Azurite cloud storage. Azurite is a docker container shared across all tests of a ducktape run. It is a single threaded nodejs service. It seems that its performance degrades when buckets aren't removed and some tests become flaky as a result. We have somewhat reasonable time budgets for test operations but Azurite becomes unreasonable slow. To overcome this we always cleanup Azurite buckets at end of tests. I did run release tests with and without cleanup and didn't observe significant change in the runtime of the test suite. Minio (s3 api equivalent) doesn't seem to have this problem. The check now will also be skipped for real Azure cloud too. Deleting a bucket does not require emptying it first so it is a cheap operation. --- tests/rptest/services/redpanda.py | 46 ++++++++++++++++++------------- 1 file changed, 27 insertions(+), 19 deletions(-) diff --git a/tests/rptest/services/redpanda.py b/tests/rptest/services/redpanda.py index 51f80ff0604e..4ef24beda33a 100644 --- a/tests/rptest/services/redpanda.py +++ b/tests/rptest/services/redpanda.py @@ -3411,30 +3411,38 @@ def delete_bucket_from_si(self): ) if self.si_settings.cloud_storage_cleanup_strategy == CloudStorageCleanupStrategy.ALWAYS_SMALL_BUCKETS_ONLY: - bucket_is_small = True - max_object_count = 3000 - - # See if the bucket is small enough - t = time.time() - for i, m in enumerate( - self.cloud_storage_client.list_objects( - self.si_settings.cloud_storage_bucket)): - if i >= max_object_count: - bucket_is_small = False - break - self.logger.info( - f"Determining bucket count for {self.si_settings.cloud_storage_bucket} up to {max_object_count} objects took {time.time() - t}s" - ) - if bucket_is_small: - # Log grep hint: "a small bucket" + if self.si_settings.cloud_storage_type == CloudStorageType.ABS: + # ABS buckets can be deleted without emptying so no need to check size. + # Also leaving buckets around when using local instance of Azurite causes + # performance issues and test flakiness. self.logger.info( - f"Bucket {self.si_settings.cloud_storage_bucket} is a small bucket (deleting it)" + "Always deleting ABS buckets as they don't have to be emptied first." ) else: + bucket_is_small = True + max_object_count = 3000 + + # See if the bucket is small enough + t = time.time() + for i, m in enumerate( + self.cloud_storage_client.list_objects( + self.si_settings.cloud_storage_bucket)): + if i >= max_object_count: + bucket_is_small = False + break self.logger.info( - f"Bucket {self.si_settings.cloud_storage_bucket} is NOT a small bucket (NOT deleting it)" + f"Determining bucket count for {self.si_settings.cloud_storage_bucket} up to {max_object_count} objects took {time.time() - t}s" ) - return + if bucket_is_small: + # Log grep hint: "a small bucket" + self.logger.info( + f"Bucket {self.si_settings.cloud_storage_bucket} is a small bucket (deleting it)" + ) + else: + self.logger.info( + f"Bucket {self.si_settings.cloud_storage_bucket} is NOT a small bucket (NOT deleting it)" + ) + return elif self.si_settings.cloud_storage_cleanup_strategy == CloudStorageCleanupStrategy.IF_NOT_USING_LIFECYCLE_RULE: if self.si_settings.use_bucket_cleanup_policy: