diff --git a/src/v/cloud_topics/BUILD b/src/v/cloud_topics/BUILD index 514355d93551a..f44d69a51e035 100644 --- a/src/v/cloud_topics/BUILD +++ b/src/v/cloud_topics/BUILD @@ -133,7 +133,6 @@ redpanda_cc_library( "//src/v/cloud_topics/level_zero/cluster_services_impl", "//src/v/cloud_topics/level_zero/pipeline:read_pipeline", "//src/v/cloud_topics/level_zero/pipeline:write_pipeline", - "//src/v/cloud_topics/level_zero/read_debounce", "//src/v/cloud_topics/level_zero/read_fanout", "//src/v/cloud_topics/level_zero/read_request_scheduler", "//src/v/cloud_topics/level_zero/reader:fetch_handler", diff --git a/src/v/cloud_topics/data_plane_impl.cc b/src/v/cloud_topics/data_plane_impl.cc index 580a3af3be44c..a6136c0e28061 100644 --- a/src/v/cloud_topics/data_plane_impl.cc +++ b/src/v/cloud_topics/data_plane_impl.cc @@ -19,7 +19,6 @@ #include "cloud_topics/level_zero/cluster_services_impl/cluster_services.h" #include "cloud_topics/level_zero/pipeline/read_pipeline.h" #include "cloud_topics/level_zero/pipeline/write_pipeline.h" -#include "cloud_topics/level_zero/read_debounce/read_debounce.h" #include "cloud_topics/level_zero/read_fanout/read_fanout.h" #include "cloud_topics/level_zero/read_request_scheduler/read_request_scheduler.h" #include "cloud_topics/level_zero/reader/fetch_request_handler.h" @@ -82,12 +81,6 @@ class impl return _read_pipeline.local().register_read_pipeline_stage(); })); } - if (config::shard_local_cfg().cloud_topics_fetch_debounce_enabled()) { - co_await construct_service( - _read_debounce, ss::sharded_parameter([this] { - return _read_pipeline.local().register_read_pipeline_stage(); - })); - } co_await construct_service( _fetch_handler, @@ -113,10 +106,6 @@ class impl co_await _read_request_scheduler.invoke_on_all( [](auto& s) { return s.start(); }); } - if (_read_debounce.local_is_initialized()) { - co_await _read_debounce.invoke_on_all( - [](auto& s) { return s.start(); }); - } co_await _fetch_handler.invoke_on_all( [](auto& s) { return s.start(); }); co_await _batch_cache.invoke_on_all([](auto& s) { return s.start(); }); @@ -185,7 +174,6 @@ class impl ss::sharded> _read_pipeline; ss::sharded _read_fanout; ss::sharded _read_request_scheduler; - ss::sharded> _read_debounce; ss::sharded _fetch_handler; // Batch cache diff --git a/src/v/cloud_topics/level_zero/pipeline/BUILD b/src/v/cloud_topics/level_zero/pipeline/BUILD index 24f9fd8e6315b..d3dddd71aa68a 100644 --- a/src/v/cloud_topics/level_zero/pipeline/BUILD +++ b/src/v/cloud_topics/level_zero/pipeline/BUILD @@ -5,8 +5,6 @@ package(default_visibility = [ "//src/v/cloud_topics:__pkg__", "//src/v/cloud_topics/level_zero/batcher:__pkg__", "//src/v/cloud_topics/level_zero/batcher/tests:__pkg__", - "//src/v/cloud_topics/level_zero/read_debounce:__pkg__", - "//src/v/cloud_topics/level_zero/read_debounce/tests:__pkg__", "//src/v/cloud_topics/level_zero/read_fanout:__pkg__", "//src/v/cloud_topics/level_zero/read_fanout/tests:__pkg__", "//src/v/cloud_topics/level_zero/read_request_scheduler:__pkg__", diff --git a/src/v/cloud_topics/level_zero/read_debounce/BUILD b/src/v/cloud_topics/level_zero/read_debounce/BUILD deleted file mode 100644 index 4fed357ef78db..0000000000000 --- a/src/v/cloud_topics/level_zero/read_debounce/BUILD +++ /dev/null @@ -1,33 +0,0 @@ -load("//bazel:build.bzl", "redpanda_cc_library") - -package(default_visibility = [ - "//src/v/cloud_topics:__pkg__", - "//src/v/cloud_topics/level_zero/read_debounce:__pkg__", - "//src/v/cloud_topics/level_zero/read_debounce/tests:__pkg__", -]) - -redpanda_cc_library( - name = "read_debounce", - srcs = [ - "read_debounce.cc", - ], - hdrs = [ - "read_debounce.h", - ], - deps = [ - "//src/v/base", - "//src/v/bytes", - "//src/v/cloud_topics:logger", - "//src/v/cloud_topics:types", - "//src/v/cloud_topics/level_zero/pipeline:event_filter", - "//src/v/cloud_topics/level_zero/pipeline:read_pipeline", - "//src/v/cloud_topics/level_zero/pipeline:read_request", - "//src/v/container:chunked_vector", - "//src/v/model", - "//src/v/ssx:checkpoint_mutex", - "//src/v/ssx:future_util", - "//src/v/storage", - "//src/v/utils:retry_chain_node", - "@seastar", - ], -) diff --git a/src/v/cloud_topics/level_zero/read_debounce/read_debounce.cc b/src/v/cloud_topics/level_zero/read_debounce/read_debounce.cc deleted file mode 100644 index ba95ed0106a2e..0000000000000 --- a/src/v/cloud_topics/level_zero/read_debounce/read_debounce.cc +++ /dev/null @@ -1,181 +0,0 @@ -/* - * Copyright 2025 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 "cloud_topics/level_zero/read_debounce/read_debounce.h" - -#include "cloud_topics/level_zero/pipeline/read_request.h" -#include "container/chunked_vector.h" -#include "ssx/checkpoint_mutex.h" -#include "ssx/future-util.h" - -#include -#include - -namespace cloud_topics::l0 { - -constexpr size_t max_bytes_per_iter = 10_MiB; -constexpr auto debounce_interval = std::chrono::milliseconds(250); - -template -read_debounce::read_debounce(read_pipeline::stage s) - : _pipeline_stage(s) {} - -template -ss::future<> read_debounce::start() { - ssx::spawn_with_gate(_gate, [this] { return bg_loop(); }); - return ss::now(); -} - -template -ss::future<> read_debounce::stop() { - for (auto& state : _in_flight) { - state.lock.broken(); - } - co_await _gate.close(); -} - -template -ss::future<> read_debounce::bg_loop() { - auto holder = _gate.hold(); - while (!_pipeline_stage.stopped()) { - // Pick up new requests as fast as possible. - // Proxy them forward. - auto fut = co_await ss::coroutine::as_future( - _pipeline_stage.pull_fetch_requests(max_bytes_per_iter)); - - if (fut.failed()) { - auto e = fut.get_exception(); - if (ssx::is_shutdown_exception(e)) { - vlog( - _pipeline_stage.logger().debug, - "Read debounce stopping due to shutdown"); - co_return; - } - vlog( - _pipeline_stage.logger().error, - "Read debounce failed to pull requests: {}", - e); - continue; - } - auto fut_res = std::move(fut).get(); - if (!fut_res.has_value()) { - auto err = fut_res.error(); - if (err == errc::shutting_down) { - vlog( - _pipeline_stage.logger().debug, - "Read debounce stopping due to shutdown"); - co_return; - } - vlog( - _pipeline_stage.logger().error, - "Read debounce received error pulling requests: {}", - fut_res.error()); - continue; - } - auto to_process = std::move(fut_res.value()); - auto queue = std::move(to_process.requests); - while (!queue.empty()) { - auto req = &queue.front(); - queue.pop_front(); - // Safe because the requests are capped by memory use. - ssx::spawn_with_gate(_gate, [this, req]() mutable { - return process_single_request(req); - }); - } - } -} - -template -ss::future<> -read_debounce::process_single_request(read_request* req) { - // The request is expected to target a single L0 object. - try { - // The metadata is supposed to refer to a single L0 object. - // If it doesn't there will be no error but the debouncing - // could be inefficient. Because of that read_debounce should - // always be paired with the read_fanout. The req->query.meta - // may still have more than one extent. - vassert( - req->query.meta.size() > 0, "Empty read queries are not allowed"); - - // Add L0 object UUID to the map of in-flight requests so the subsequent - // requests could be debounced - auto id = req->query.meta.front().id.name; - auto hash = absl::Hash{}(id); - auto ix = hash % debounce_hash_size; - auto u = _in_flight.at(ix).lock.try_get_units(); - if (!u.has_value() && !_pipeline_stage.stopped()) { - try { - u = co_await _in_flight.at(ix).lock.get_units( - debounce_interval); - } catch (const ss::semaphore_timed_out&) { - vlog( - req->rtc_logger.debug, - "Lock timed out, id: {}, proceeding anyway", - id); - } - } else if (_pipeline_stage.stopped()) { - co_return; - } - - // Here, it's not guaranteed that 'u' will actually have - // units. It is possible for the 'get_units' call to time out - // leaving 'u' uninitialized. This is intentional because we don't - // want to block requests indefinitely. The 'get_units' call is - // supposed to debounce requests for limited amount of time. - // In case if 'u' is nullopt we may trigger same download twice - // which is not a problem for correctness. - // It's expected that the majority of GetObject requests will be - // fulfilled within the debounce_interval. - dataplane_query query{ - .output_size_estimate = req->query.output_size_estimate, - .meta = req->query.meta.copy(), - }; - - auto proxy = ss::make_lw_shared>( - req->ntp, - std::move(query), - req->expiration_time, - &_pipeline_stage.get_root_rtc(), - req->stage); - - _pipeline_stage.push_next_stage(*proxy); - - auto holder = _gate.hold(); - proxy->response.get_future() - .finally([proxy, u = std::move(u), h = std::move(holder)] mutable { - // finally is used to capture 'u' and 'proxy' - // while the request is fulfilled. - // This call exits shortly but 'proxy' should - // live until the request is running. The value - // of 'u' could be 'nullopt'. - u.reset(); - }) - .forward_to(std::move(req->response)); - // At this point it's guaranteed that the req->response - // promise will be set. - } catch (...) { - auto ex = std::current_exception(); - if (ssx::is_shutdown_exception(ex)) { - vlog(req->rtc_logger.debug, "Read debounce shutting down"); - req->set_value(errc::shutting_down); - co_return; - } - vlog( - req->rtc_logger.error, - "Unexpected exception in read debounce: {}", - ex); - req->set_value(errc::unexpected_failure); - } -} - -template class read_debounce; -template class read_debounce; - -} // namespace cloud_topics::l0 diff --git a/src/v/cloud_topics/level_zero/read_debounce/read_debounce.h b/src/v/cloud_topics/level_zero/read_debounce/read_debounce.h deleted file mode 100644 index 0fc45e13d5101..0000000000000 --- a/src/v/cloud_topics/level_zero/read_debounce/read_debounce.h +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Copyright 2025 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 "cloud_topics/level_zero/pipeline/read_pipeline.h" -#include "cloud_topics/types.h" -#include "ssx/checkpoint_mutex.h" - -#include -#include -#include - -namespace cloud_topics::l0 { - -template -class read_debounce { -public: - explicit read_debounce(read_pipeline::stage); - ss::future<> start(); - ss::future<> stop(); - -private: - ss::future<> bg_loop(); - ss::future<> process_single_request(read_request* req); - - ss::gate _gate; - read_pipeline::stage _pipeline_stage; - - struct debounce_state { - ssx::basic_checkpoint_mutex lock{"read_debounce"}; - }; - static constexpr size_t debounce_hash_size = 128; - std::array _in_flight{}; -}; -} // namespace cloud_topics::l0 diff --git a/src/v/cloud_topics/level_zero/read_debounce/tests/BUILD b/src/v/cloud_topics/level_zero/read_debounce/tests/BUILD deleted file mode 100644 index db36742345cf0..0000000000000 --- a/src/v/cloud_topics/level_zero/read_debounce/tests/BUILD +++ /dev/null @@ -1,49 +0,0 @@ -load("//bazel:test.bzl", "redpanda_cc_bench", "redpanda_cc_gtest") - -redpanda_cc_gtest( - name = "read_debounce_test", - timeout = "short", - srcs = [ - "read_debounce_test.cc", - ], - deps = [ - "//src/v/base", - "//src/v/cloud_topics:types", - "//src/v/cloud_topics/level_zero/pipeline:event_filter", - "//src/v/cloud_topics/level_zero/pipeline:pipeline_stage", - "//src/v/cloud_topics/level_zero/pipeline:read_pipeline", - "//src/v/cloud_topics/level_zero/pipeline:read_request", - "//src/v/cloud_topics/level_zero/read_debounce", - "//src/v/config", - "//src/v/model", - "//src/v/model/tests:random", - "//src/v/test_utils:gtest", - "//src/v/utils:uuid", - "@googletest//:gtest", - "@seastar", - ], -) - -redpanda_cc_bench( - name = "read_debounce_rpbench", - timeout = "short", - srcs = [ - "read_debounce_bench.cc", - ], - cpu = 1, - deps = [ - "//src/v/base", - "//src/v/cloud_topics/level_zero/pipeline:event_filter", - "//src/v/cloud_topics/level_zero/pipeline:pipeline_stage", - "//src/v/cloud_topics/level_zero/pipeline:read_pipeline", - "//src/v/cloud_topics/level_zero/pipeline:read_request", - "//src/v/cloud_topics/level_zero/read_debounce", - "//src/v/config", - "//src/v/model", - "//src/v/model/tests:random", - "//src/v/ssx:future_util", - "//src/v/test_utils:scoped_config", - "@seastar", - "@seastar//:benchmark", - ], -) diff --git a/src/v/cloud_topics/level_zero/read_debounce/tests/read_debounce_bench.cc b/src/v/cloud_topics/level_zero/read_debounce/tests/read_debounce_bench.cc deleted file mode 100644 index 2024e6a2fbf5a..0000000000000 --- a/src/v/cloud_topics/level_zero/read_debounce/tests/read_debounce_bench.cc +++ /dev/null @@ -1,166 +0,0 @@ -// Copyright 2025 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/level_zero/pipeline/event_filter.h" -#include "cloud_topics/level_zero/pipeline/pipeline_stage.h" -#include "cloud_topics/level_zero/read_debounce/read_debounce.h" -#include "config/configuration.h" -#include "model/fundamental.h" -#include "model/namespace.h" -#include "model/record.h" -#include "model/record_batch_reader.h" -#include "model/tests/random_batch.h" -#include "ssx/future-util.h" - -#include -#include -#include -#include -#include - -#include -#include -#include -#include - -using namespace std::chrono_literals; -namespace cloud_topics { - -struct fetch_handler { - explicit fetch_handler(l0::read_pipeline<>& p) - : stage(p.register_read_pipeline_stage()) { - _batch = model::test::make_random_batch( - model::test::record_batch_spec{ - .offset = model::offset(0), - .allow_compression = false, - .count = 1, - .records = 1, - .record_sizes = std::vector{4096}, - }); - } - - ss::future<> start() { - ssx::background = bg_run(); - return ss::now(); - } - - ss::future<> stop() { co_await _gate.close(); } - - ss::future<> bg_run() { - auto h = _gate.hold(); - while (!stage.stopped()) { - auto result = co_await stage.pull_fetch_requests( - std::numeric_limits::max()); - - if (!result.has_value()) { - // Expected during shutdown - co_return; - } - - for (auto& r : result.value().requests) { - process_single_request(&r); - } - } - } - - void process_single_request(l0::read_request<>* req) { - auto auto_dispose = ss::defer( - [req] { req->set_value(errc::unexpected_failure); }); - - auto meta = std::move(req->query.meta); - chunked_vector data; - for (auto& m : meta) { - std::ignore = m; - data.push_back(_batch->copy()); - } - auto_dispose.cancel(); - req->set_value({{std::move(data)}}); - } - - std::optional _batch; - l0::read_pipeline<>::stage stage; - ss::gate _gate; -}; -} // namespace cloud_topics - -using namespace cloud_topics; - -class read_debounce_bench { -public: - /// Start benchmark fixture. - /// \param enable_debounce enables or disables read debounce - ss::future<> start(bool enable_debounce) { - co_await pipeline.start(); - - if (enable_debounce) { - co_await debounce.start(ss::sharded_parameter([this] { - return pipeline.local().register_read_pipeline_stage(); - })); - - co_await debounce.invoke_on_all([](auto& f) { return f.start(); }); - } - - co_await sink.start( - ss::sharded_parameter([this] { return std::ref(pipeline.local()); })); - - co_await sink.invoke_on_all([](auto& sink) { return sink.start(); }); - } - - ss::future<> stop() { - co_await pipeline.stop(); - co_await sink.stop(); - if (debounce.local_is_initialized()) { - co_await debounce.stop(); - } - } - - // Run requests serially - ss::future<> test_run(int num_requests) { - for (int i = 0; i < num_requests; i++) { - l0::dataplane_query query; - query.output_size_estimate = 1_KiB; - query.meta.push_back( - // The exact values doesn't matter. The only requirement - // is that the id is different for every request. - extent_meta{ - .id = object_id{.name = uuid_t::create()}, - .byte_range_size = byte_range_size_t{1_KiB}}); - - // The requests are processed sequentially, without any concurrency - // so we're measuring a propagation latency and not throughput. The - // latency is expected to be in the ballpark of 10 microseconds end - // to end. The actual I/O (either cache or cloud) will take - // longer. - perf_tests::start_measuring_time(); - perf_tests::do_not_optimize( - co_await pipeline.local().make_reader( - model::controller_ntp, - std::move(query), - ss::lowres_clock::now() + std::chrono::seconds(10))); - perf_tests::stop_measuring_time(); - } - } - - ss::sharded> pipeline; - ss::sharded> debounce; - ss::sharded sink; -}; - -PERF_TEST_C(read_debounce_bench, baseline) { - // This is a baseline. It is measured without the debouncing. - co_await start(false); - co_await test_run(100); - co_await stop(); -} - -PERF_TEST_C(read_debounce_bench, debounce) { - co_await start(true); - co_await test_run(100); - co_await stop(); -} diff --git a/src/v/cloud_topics/level_zero/read_debounce/tests/read_debounce_test.cc b/src/v/cloud_topics/level_zero/read_debounce/tests/read_debounce_test.cc deleted file mode 100644 index b70da40c0b3df..0000000000000 --- a/src/v/cloud_topics/level_zero/read_debounce/tests/read_debounce_test.cc +++ /dev/null @@ -1,156 +0,0 @@ -// Copyright 2025 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/level_zero/pipeline/event_filter.h" -#include "cloud_topics/level_zero/pipeline/pipeline_stage.h" -#include "cloud_topics/level_zero/read_debounce/read_debounce.h" -#include "cloud_topics/types.h" -#include "config/configuration.h" -#include "model/fundamental.h" -#include "model/metadata.h" -#include "model/namespace.h" -#include "model/record.h" -#include "model/record_batch_reader.h" -#include "model/tests/random_batch.h" -#include "test_utils/test.h" -#include "utils/uuid.h" - -#include -#include -#include -#include - -#include -#include -#include -#include - -using namespace std::chrono_literals; - -static ss::logger test_log("L0_read_debounce_test"); - -namespace cloud_topics { - -struct fetch_handler { - explicit fetch_handler(l0::read_pipeline& p) - : stage(p.register_read_pipeline_stage()) {} - - using read_requests_list = l0::requests_list< - l0::read_pipeline, - l0::read_request>; - - ss::future> get_next_requests() { - auto result = co_await stage.pull_fetch_requests( - std::numeric_limits::max()); - - if (!result.has_value()) { - co_return std::unexpected(result.error()); - } - - auto list = std::move(result.value()); - co_return std::move(list); - } - - l0::read_pipeline::stage stage; -}; - -} // namespace cloud_topics - -using namespace cloud_topics; - -class read_debounce_fixture : public seastar_test { -public: - ss::future<> start() { - co_await pipeline.start(); - - co_await debounce.start(ss::sharded_parameter([this] { - return pipeline.local().register_read_pipeline_stage(); - })); - - co_await debounce.invoke_on_all( - [](l0::read_debounce& s) { return s.start(); }); - - co_await fetch_handler.start( - ss::sharded_parameter([this] { return std::ref(pipeline.local()); })); - } - - ss::future<> stop() { - co_await pipeline.invoke_on_all([](auto& s) { return s.shutdown(); }); - co_await fetch_handler.stop(); - co_await debounce.stop(); - co_await pipeline.stop(); - } - - ss::sharded> pipeline; - ss::sharded> debounce; - ss::sharded fetch_handler; -}; - -static const model::topic_namespace - test_topic(model::kafka_namespace, model::topic("tapioca")); - -static const model::ntp - test_ntp(test_topic.ns, test_topic.tp, model::partition_id(0)); - -TEST_F_CORO(read_debounce_fixture, test_happy_path) { - // Check that the read request is going through and the correct results - // are propagated back to the caller. - co_await start(); - l0::dataplane_query query; - query.output_size_estimate = 1_MiB; - query.meta.push_back( - extent_meta{.byte_range_size = byte_range_size_t{1_MiB}}); - auto result_fut = pipeline.local().make_reader( - test_ntp, std::move(query), ss::manual_clock::now() + 10s); - - auto request = co_await fetch_handler.local().get_next_requests(); - ASSERT_TRUE_CORO(request.has_value()); - ASSERT_EQ_CORO(request.value().requests.size(), 1); - - chunked_vector batches; - model::test::record_batch_spec spec{}; - spec.offset = model::offset{0}; - spec.count = 1; - spec.records = 1; - auto rb = model::test::make_random_batch(spec); - batches.push_back(std::move(rb)); - - request.value().requests.front().set_value({{std::move(batches)}}); - - auto result = co_await std::move(result_fut); - - ASSERT_TRUE_CORO(result.has_value()); - ASSERT_EQ_CORO(result.value().results.size(), 1); - ASSERT_EQ_CORO( - result.value().results.front().header().base_offset, model::offset{0}); - co_await stop(); -} - -TEST_F_CORO(read_debounce_fixture, test_error_propagation) { - // Check that the error is propagated to the caller. - co_await start(); - l0::dataplane_query query; - query.output_size_estimate = 1_MiB; - query.meta.push_back( - extent_meta{.byte_range_size = byte_range_size_t{1_MiB}}); - auto result_fut = pipeline.local().make_reader( - test_ntp, std::move(query), ss::manual_clock::now() + 10s); - - auto request = co_await fetch_handler.local().get_next_requests(); - ASSERT_TRUE_CORO(request.has_value()); - ASSERT_EQ_CORO(request.value().requests.size(), 1); - - request.value().requests.front().set_value(errc::timeout); - - auto result = co_await std::move(result_fut); - - ASSERT_FALSE_CORO(result.has_value()); - ASSERT_EQ_CORO(result.error(), errc::timeout); - co_await stop(); -} diff --git a/src/v/config/configuration.cc b/src/v/config/configuration.cc index 4318180ed7cba..9efaa6588709f 100644 --- a/src/v/config/configuration.cc +++ b/src/v/config/configuration.cc @@ -4585,14 +4585,6 @@ configuration::configuration() "request using multiple shards.", {.needs_restart = needs_restart::yes, .visibility = visibility::user}, true) - , cloud_topics_fetch_debounce_enabled( - *this, - "cloud_topics_fetch_debounce_enabled", - "Enables fetch debouncing in cloud topics. This mechanism guarantees " - "that the broker fetches every object only once improving the " - "performance and lowering the cost.", - {.needs_restart = needs_restart::yes, .visibility = visibility::user}, - true) , development_feature_property_testing_only( *this, "development_feature_property_testing_only", diff --git a/src/v/config/configuration.h b/src/v/config/configuration.h index 625c06d4a8f5f..8aad38b4c14c6 100644 --- a/src/v/config/configuration.h +++ b/src/v/config/configuration.h @@ -837,8 +837,6 @@ struct configuration final : public config_store { property cloud_topics_parallel_fetch_enabled; - property cloud_topics_fetch_debounce_enabled; - development_feature_property development_feature_property_testing_only; private: