From c7523831cb3369442f5449767e8c0fb083980a86 Mon Sep 17 00:00:00 2001 From: Kefu Chai Date: Wed, 27 Nov 2024 15:10:25 +0800 Subject: [PATCH] treewide: migrate from boost::adaptors::transformed to std::views::transform now that we are allowed to use C++23. we now have the luxury of using `std::views::transform`. in this change, we: - replace `boost::adaptors::transformed` with `std::views::transform` - use `fmt::join()` when appropriate where `boost::algorithm::join()` is not applicable to a range view returned by `std::view::transform`. - use `std::ranges::fold_left()` to accumulate the range returned by `std::view::transform` - use `std::ranges::fold_left()` to get the maximum element in the range returned by `std::view::transform` - use `std::ranges::min()` to get the minimal element in the range returned by `std::view::transform` - use `std::ranges::equal()` to compare the range views returned by `std::view::transform` - remove unused `#include ` to reduce the dependency to boost for better maintainability, and leverage standard library features for better long-term support. this change is part of our ongoing effort to modernize our codebase and reduce external dependencies where possible. limitations: there are still a couple places where we are still using `boost::adaptors::transformed` due to the lack of a C++23 alternative for `boost::join()` and `boost::adaptors::uniqued`. Signed-off-by: Kefu Chai --- api/column_family.cc | 14 ++-- auth/resource.cc | 11 ++- compaction/compaction.cc | 25 +++--- compaction/compaction_manager.cc | 8 +- compaction/compaction_strategy.cc | 12 ++- compaction/size_tiered_compaction_strategy.cc | 1 - compaction/task_manager_module.cc | 2 +- compaction/time_window_compaction_strategy.cc | 3 +- cql3/restrictions/statement_restrictions.cc | 1 + cql3/selection/selection.cc | 10 +-- cql3/statements/alter_table_statement.cc | 3 +- cql3/statements/authorization_statement.cc | 7 +- cql3/statements/batch_statement.cc | 1 + cql3/statements/create_aggregate_statement.cc | 3 +- cql3/statements/create_index_statement.cc | 4 +- cql3/statements/create_view_statement.cc | 11 +-- cql3/statements/describe_statement.cc | 18 ++--- cql3/statements/index_target.cc | 13 ++-- cql3/statements/modification_statement.cc | 3 - cql3/statements/select_statement.cc | 7 +- cql3/user_types.cc | 2 - cql3/util.cc | 4 +- db/commitlog/commitlog.cc | 3 - db/consistency_level.cc | 4 +- db/extensions.cc | 2 - db/schema_applier.cc | 4 +- db/schema_tables.cc | 14 ++-- db/size_estimates_virtual_reader.cc | 7 +- db/system_distributed_keyspace.cc | 2 - db/system_keyspace.cc | 7 +- db/view/view.cc | 46 +++++------ db/view/view_update_generator.cc | 4 +- db/virtual_tables.cc | 13 ++-- dht/i_partitioner.cc | 3 - gms/versioned_value.cc | 3 +- keys.cc | 5 +- main.cc | 2 +- mutation/frozen_mutation.cc | 9 +-- mutation/mutation_partition.cc | 4 +- mutation/mutation_partition_v2.cc | 4 +- .../shard_based_splitting_writer.cc | 2 - .../timestamp_based_splitting_writer.cc | 2 - node_ops/task_manager_module.cc | 4 +- query-result-reader.hh | 2 - query-result-set.cc | 2 +- repair/row_level.cc | 5 +- replica/table.cc | 31 ++++---- service/mapreduce_service.cc | 6 +- service/qos/service_level_controller.cc | 2 +- service/raft/group0_state_machine_merger.cc | 4 +- service/storage_service.cc | 27 ++++--- service/topology_mutation.cc | 18 ++--- sstables/mx/parsers.hh | 4 +- sstables/mx/reader.cc | 8 +- sstables/sstable_set.cc | 34 ++++---- sstables/sstables.cc | 16 ++-- sstables/writer.hh | 3 +- sstables_loader.cc | 2 +- tasks/task_handler.cc | 4 +- tasks/task_manager.cc | 8 +- test/boost/cdc_test.cc | 2 +- test/boost/compaction_group_test.cc | 4 +- test/boost/compound_test.cc | 6 +- test/boost/counter_test.cc | 8 +- test/boost/cql_query_test.cc | 12 +-- test/boost/mutation_query_test.cc | 4 - test/boost/mutation_reader_another_test.cc | 2 +- test/boost/mutation_test.cc | 4 +- test/boost/mvcc_test.cc | 2 - test/boost/network_topology_strategy_test.cc | 10 +-- test/boost/partitioner_test.cc | 6 +- test/boost/restrictions_test.cc | 5 +- test/boost/sstable_compaction_test.cc | 14 ++-- test/boost/sstable_datafile_test.cc | 7 +- test/boost/sstable_test.cc | 3 +- test/boost/tablets_test.cc | 2 +- test/boost/transport_test.cc | 19 +---- test/lib/cql_assertions.cc | 15 ++-- test/lib/data_model.cc | 7 +- test/lib/mutation_source_test.cc | 8 +- test/lib/random_schema.cc | 14 ++-- test/lib/simple_schema.hh | 6 +- test/manual/enormous_table_scan_test.cc | 4 - test/perf/perf_commitlog.cc | 10 +-- test/perf/perf_fast_forward.cc | 27 +++---- test/perf/perf_mutation_readers.cc | 78 +++++++++---------- tools/schema_loader.cc | 12 +-- tools/scylla-nodetool.cc | 2 +- tools/scylla-sstable.cc | 4 +- tools/scylla-types.cc | 11 ++- tools/utils.cc | 2 - tracing/trace_state.cc | 3 +- 92 files changed, 339 insertions(+), 457 deletions(-) diff --git a/api/column_family.cc b/api/column_family.cc index 810b9a1f56ac..f79169ffe8f9 100644 --- a/api/column_family.cc +++ b/api/column_family.cc @@ -200,7 +200,7 @@ static future get_cf_histogram(http_context& ctx, utils: }; return ctx.db.map(fun).then([](const std::vector &res) { std::vector r; - boost::copy(res | boost::adaptors::transformed(to_json), std::back_inserter(r)); + std::ranges::copy(res | std::views::transform(to_json), std::back_inserter(r)); return make_ready_future(r); }); } @@ -227,7 +227,7 @@ static future get_cf_rate_and_histogram(http_context& ct }; return ctx.db.map(fun).then([](const std::vector &res) { std::vector r; - boost::copy(res | boost::adaptors::transformed(timer_to_json), std::back_inserter(r)); + std::ranges::copy(res | std::views::transform(timer_to_json), std::back_inserter(r)); return make_ready_future(r); }); } @@ -717,25 +717,25 @@ void set_column_family(http_context& ctx, routes& r, sharded req) { return map_reduce_cf(ctx, req->get_path_param("name"), ratio_holder(), [] (replica::column_family& cf) { - return boost::accumulate(*cf.get_sstables() | boost::adaptors::transformed(filter_false_positive_as_ratio_holder), ratio_holder()); + return std::ranges::fold_left(*cf.get_sstables() | std::views::transform(filter_false_positive_as_ratio_holder), ratio_holder(), std::plus{}); }, std::plus<>()); }); cf::get_all_bloom_filter_false_ratio.set(r, [&ctx] (std::unique_ptr req) { return map_reduce_cf(ctx, ratio_holder(), [] (replica::column_family& cf) { - return boost::accumulate(*cf.get_sstables() | boost::adaptors::transformed(filter_false_positive_as_ratio_holder), ratio_holder()); + return std::ranges::fold_left(*cf.get_sstables() | std::views::transform(filter_false_positive_as_ratio_holder), ratio_holder(), std::plus{}); }, std::plus<>()); }); cf::get_recent_bloom_filter_false_ratio.set(r, [&ctx] (std::unique_ptr req) { return map_reduce_cf(ctx, req->get_path_param("name"), ratio_holder(), [] (replica::column_family& cf) { - return boost::accumulate(*cf.get_sstables() | boost::adaptors::transformed(filter_recent_false_positive_as_ratio_holder), ratio_holder()); + return std::ranges::fold_left(*cf.get_sstables() | std::views::transform(filter_recent_false_positive_as_ratio_holder), ratio_holder(), std::plus{}); }, std::plus<>()); }); cf::get_all_recent_bloom_filter_false_ratio.set(r, [&ctx] (std::unique_ptr req) { return map_reduce_cf(ctx, ratio_holder(), [] (replica::column_family& cf) { - return boost::accumulate(*cf.get_sstables() | boost::adaptors::transformed(filter_recent_false_positive_as_ratio_holder), ratio_holder()); + return std::ranges::fold_left(*cf.get_sstables() | std::views::transform(filter_recent_false_positive_as_ratio_holder), ratio_holder(), std::plus{}); }, std::plus<>()); }); @@ -1089,7 +1089,7 @@ void set_column_family(http_context& ctx, routes& r, sharded future> { auto sstables = co_await db.find_column_family(uuid).get_sstables_by_partition_key(key); - co_return boost::copy_range>(sstables | boost::adaptors::transformed([] (auto s) { return s->get_filename(); })); + co_return sstables | std::views::transform([] (auto s) { return s->get_filename(); }) | std::ranges::to(); }, std::unordered_set(), [](std::unordered_set a, std::unordered_set&& b) mutable { a.merge(b); diff --git a/auth/resource.cc b/auth/resource.cc index 17dc8e94837a..e0c7c1cd5ac0 100644 --- a/auth/resource.cc +++ b/auth/resource.cc @@ -194,7 +194,7 @@ service_level_resource_view::service_level_resource_view(const resource &r) { sstring encode_signature(std::string_view name, std::vector args) { return seastar::format("{}[{}]", name, - fmt::join(args | boost::adaptors::transformed([] (const data_type t) { + fmt::join(args | std::views::transform([] (const data_type t) { return t->name(); }), "^")); } @@ -209,11 +209,10 @@ std::pair> decode_signature(std::string_view enc } std::vector raw_types; boost::split(raw_types, encoded_signature, boost::is_any_of("^")); - std::vector decoded_types = boost::copy_range>( - raw_types | boost::adaptors::transformed([] (std::string_view raw_type) { + std::vector decoded_types = + raw_types | std::views::transform([] (std::string_view raw_type) { return db::marshal::type_parser::parse(raw_type); - }) - ); + }) | std::ranges::to(); return {sstring(function_name), decoded_types}; } @@ -223,7 +222,7 @@ std::pair> decode_signature(std::string_view enc static sstring decoded_signature_string(std::string_view encoded_signature) { auto [function_name, arg_types] = decode_signature(encoded_signature); return seastar::format("{}({})", cql3::util::maybe_quote(sstring(function_name)), - boost::algorithm::join(arg_types | boost::adaptors::transformed([] (data_type t) { + fmt::join(arg_types | std::views::transform([] (data_type t) { return t->cql3_type_name(); }), ", ")); } diff --git a/compaction/compaction.cc b/compaction/compaction.cc index a042d741e502..ffa4e6b6c445 100644 --- a/compaction/compaction.cc +++ b/compaction/compaction.cc @@ -546,12 +546,12 @@ class compaction { } auto owned_ranges = dht::to_partition_ranges(*_owned_ranges, utils::can_yield::yes); - auto non_owned_ranges = boost::copy_range(sstables - | boost::adaptors::transformed([] (const shared_sstable& sst) { + auto non_owned_ranges = sstables + | std::views::transform([] (const shared_sstable& sst) { seastar::thread::maybe_yield(); return dht::partition_range::make({sst->get_first_decorated_key(), true}, {sst->get_last_decorated_key(), true}); - })); + }) | std::ranges::to(); return dht::subtract_ranges(*_schema, non_owned_ranges, std::move(owned_ranges)).get(); } @@ -820,7 +820,7 @@ class compaction { _rp = std::max(_rp, sst_stats.position); } } - log_info("{} [{}]", report_start_desc(), fmt::join(_sstables | boost::adaptors::transformed([] (auto sst) { return to_string(sst, true); }), ",")); + log_info("{} [{}]", report_start_desc(), fmt::join(_sstables | std::views::transform([] (auto sst) { return to_string(sst, true); }), ",")); if (ssts->size() < _sstables.size()) { log_debug("{} out of {} input sstables are fully expired sstables that will not be actually compacted", _sstables.size() - ssts->size(), _sstables.size()); @@ -953,7 +953,7 @@ class compaction { // By the time being, using estimated key count. log_info("{} {} sstables to [{}]. {} to {} (~{}% of original) in {}ms = {}. ~{} total partitions merged to {}.", report_finish_desc(), _input_sstable_generations.size(), - fmt::join(ret.new_sstables | boost::adaptors::transformed([] (auto sst) { return to_string(sst, false); }), ","), + fmt::join(ret.new_sstables | std::views::transform([] (auto sst) { return to_string(sst, false); }), ","), utils::pretty_printed_data_size(_start_size), utils::pretty_printed_data_size(_end_size), int(ratio * 100), std::chrono::duration_cast(duration).count(), utils::pretty_printed_throughput(_start_size, duration), _cdata.total_partitions, _cdata.total_keys_written); @@ -1268,8 +1268,8 @@ class regular_compaction : public compaction { auto exhausted_ssts = std::vector(exhausted, _sstables.end()); log_debug("Replacing earlier exhausted sstable(s) [{}] by new sstable(s) [{}]", - fmt::join(exhausted_ssts | boost::adaptors::transformed([] (auto sst) { return to_string(sst, false); }), ","), - fmt::join(_new_unused_sstables | boost::adaptors::transformed([] (auto sst) { return to_string(sst, true); }), ",")); + fmt::join(exhausted_ssts | std::views::transform([] (auto sst) { return to_string(sst, false); }), ","), + fmt::join(_new_unused_sstables | std::views::transform([] (auto sst) { return to_string(sst, true); }), ",")); _replacer(get_compaction_completion_desc(exhausted_ssts, std::move(_new_unused_sstables))); _sstables.erase(exhausted, _sstables.end()); dynamic_cast(unwrap_monitor_generator()).remove_exhausted_sstables(exhausted_ssts); @@ -1917,7 +1917,7 @@ static future scrub_sstables_validate_mode(sstables::compacti auto permit = table_s.make_compaction_reader_permit(); uint64_t validation_errors = 0; - cdata.compaction_size = boost::accumulate(descriptor.sstables | boost::adaptors::transformed([] (auto& sst) { return sst->data_size(); }), int64_t(0)); + cdata.compaction_size = std::ranges::fold_left(descriptor.sstables | std::views::transform([] (auto& sst) { return sst->data_size(); }), int64_t(0), std::plus{}); for (const auto& sst : descriptor.sstables) { clogger.info("Scrubbing in validate mode {}", sst->get_filename()); @@ -1992,8 +1992,9 @@ get_fully_expired_sstables(const table_state& table_s, const std::vector>(table_s.compacted_undeleted_sstables() - | boost::adaptors::transformed(std::mem_fn(&sstables::sstable::generation))); + auto compacted_undeleted_gens = table_s.compacted_undeleted_sstables() + | std::views::transform(std::mem_fn(&sstables::sstable::generation)) + | std::ranges::to(); auto has_undeleted_ancestor = [&compacted_undeleted_gens] (auto& candidate) { // Get ancestors from sstable which is empty after restart. It works for this purpose because // we only need to check that a sstable compacted *in this instance* hasn't an ancestor undeleted. @@ -2036,11 +2037,11 @@ get_fully_expired_sstables(const table_state& table_s, const std::vector>(sstables | boost::adaptors::transformed(std::mem_fn(&sstables::sstable::run_identifier))).size(); + return std::ranges::distance(sstables | std::views::transform(std::mem_fn(&sstables::sstable::run_identifier)) | std::ranges::to());; } uint64_t compaction_descriptor::sstables_size() const { - return boost::accumulate(sstables | boost::adaptors::transformed(std::mem_fn(&sstables::sstable::data_size)), uint64_t(0)); + return std::ranges::fold_left(sstables | std::views::transform(std::mem_fn(&sstables::sstable::data_size)), uint64_t(0), std::plus{}); } } diff --git a/compaction/compaction_manager.cc b/compaction/compaction_manager.cc index 04d0e63196a4..a48438f221ae 100644 --- a/compaction/compaction_manager.cc +++ b/compaction/compaction_manager.cc @@ -187,7 +187,7 @@ unsigned compaction_manager::current_compaction_fan_in_threshold() const { if (_tasks.empty()) { return 0; } - auto largest_fan_in = std::ranges::max(_tasks | boost::adaptors::transformed([] (auto& task) { + auto largest_fan_in = std::ranges::max(_tasks | std::views::transform([] (auto& task) { return task.compaction_running() ? task.compaction_data().compaction_fan_in : 0; })); // conservatively limit fan-in threshold to 32, such that tons of small sstables won't accumulate if @@ -1364,9 +1364,9 @@ future<> compaction_manager::maybe_wait_for_sstable_count_reduction(table_state& auto num_runs_for_compaction = [&, this] { auto& cs = t.get_compaction_strategy(); auto desc = cs.get_sstables_for_compaction(t, get_strategy_control()); - return boost::copy_range>( - desc.sstables - | boost::adaptors::transformed(std::mem_fn(&sstables::sstable::run_identifier))).size(); + return std::ranges::size(desc.sstables + | std::views::transform(std::mem_fn(&sstables::sstable::run_identifier)) + | std::ranges::to()); }; const auto threshold = size_t(std::max(schema->max_compaction_threshold(), 32)); auto count = num_runs_for_compaction(); diff --git a/compaction/compaction_strategy.cc b/compaction/compaction_strategy.cc index aa52c295f60c..0ef85ec76fdf 100644 --- a/compaction/compaction_strategy.cc +++ b/compaction/compaction_strategy.cc @@ -22,8 +22,6 @@ #include "cql3/statements/property_definitions.hh" #include "schema/schema.hh" #include -#include -#include #include "size_tiered_compaction_strategy.hh" #include "leveled_compaction_strategy.hh" #include "time_window_compaction_strategy.hh" @@ -49,10 +47,10 @@ compaction_descriptor compaction_strategy_impl::make_major_compaction_job(std::v std::vector compaction_strategy_impl::get_cleanup_compaction_jobs(table_state& table_s, std::vector candidates) const { // The default implementation is suboptimal and causes the writeamp problem described issue in #10097. // The compaction strategy relying on it should strive to implement its own method, to make cleanup bucket aware. - return boost::copy_range>(candidates | boost::adaptors::transformed([] (const shared_sstable& sst) { + return candidates | std::views::transform([] (const shared_sstable& sst) { return compaction_descriptor({ sst }, sst->get_sstable_level(), sstables::compaction_descriptor::default_max_sstable_bytes, sst->run_identifier()); - })); + }) | std::ranges::to(); } bool compaction_strategy_impl::worth_dropping_tombstones(const shared_sstable& sst, gc_clock::time_point compaction_time, const table_state& t) { @@ -246,9 +244,9 @@ size_tiered_backlog_tracker::sstables_backlog_contribution size_tiered_backlog_t if (!size_tiered_compaction_strategy::is_bucket_interesting(bucket, threshold)) { continue; } - contrib.value += boost::accumulate(bucket | boost::adaptors::transformed([] (const shared_sstable& sst) -> double { + contrib.value += std::ranges::fold_left(bucket | std::views::transform([] (const shared_sstable& sst) -> double { return sst->data_size() * log4(sst->data_size()); - }), double(0.0f)); + }), double(0.0f), std::plus{}); // Controller is disabled if exception is caught during add / remove calls, so not making any effort to make this exception safe contrib.sstables.insert(bucket.begin(), bucket.end()); } @@ -259,7 +257,7 @@ size_tiered_backlog_tracker::sstables_backlog_contribution size_tiered_backlog_t double size_tiered_backlog_tracker::backlog(const compaction_backlog_tracker::ongoing_writes& ow, const compaction_backlog_tracker::ongoing_compactions& oc) const { inflight_component compacted = compacted_backlog(oc); - auto total_backlog_bytes = boost::accumulate(_contrib.sstables | boost::adaptors::transformed(std::mem_fn(&sstables::sstable::data_size)), uint64_t(0)); + auto total_backlog_bytes = std::ranges::fold_left(_contrib.sstables | std::views::transform(std::mem_fn(&sstables::sstable::data_size)), uint64_t(0), std::plus{}); // Bail out if effective backlog is zero, which happens in a small window where ongoing compaction exhausted // input files but is still sealing output files or doing managerial stuff like updating history table diff --git a/compaction/size_tiered_compaction_strategy.cc b/compaction/size_tiered_compaction_strategy.cc index 33757d19ead5..bf700ff2c44e 100644 --- a/compaction/size_tiered_compaction_strategy.cc +++ b/compaction/size_tiered_compaction_strategy.cc @@ -11,7 +11,6 @@ #include "size_tiered_compaction_strategy.hh" #include "cql3/statements/property_definitions.hh" -#include #include #include diff --git a/compaction/task_manager_module.cc b/compaction/task_manager_module.cc index cdebe2a1be83..483ecf32681a 100644 --- a/compaction/task_manager_module.cc +++ b/compaction/task_manager_module.cc @@ -723,7 +723,7 @@ future<> table_resharding_compaction_task_impl::run() { auto all_jobs = co_await collect_all_shared_sstables(_dir, _db, _status.keyspace, _status.table, _owned_ranges_ptr); auto destinations = co_await distribute_reshard_jobs(std::move(all_jobs)); - uint64_t total_size = boost::accumulate(destinations | boost::adaptors::transformed(std::mem_fn(&replica::reshard_shard_descriptor::size)), uint64_t(0)); + uint64_t total_size = std::ranges::fold_left(destinations | std::views::transform(std::mem_fn(&replica::reshard_shard_descriptor::size)), uint64_t(0), std::plus{}); if (total_size == 0) { co_return; } diff --git a/compaction/time_window_compaction_strategy.cc b/compaction/time_window_compaction_strategy.cc index 1471e08e0814..05958493c41a 100644 --- a/compaction/time_window_compaction_strategy.cc +++ b/compaction/time_window_compaction_strategy.cc @@ -16,7 +16,6 @@ #include #include #include -#include #include @@ -267,7 +266,7 @@ time_window_compaction_strategy::get_reshaping_job(std::vector i single_window.size(), !single_window.empty() && sstable_set_overlapping_count(schema, single_window) == 0); auto get_job_size = [] (const std::vector& ssts) { - return boost::accumulate(ssts | boost::adaptors::transformed(std::mem_fn(&sstable::bytes_on_disk)), uint64_t(0)); + return std::ranges::fold_left(ssts | std::views::transform(std::mem_fn(&sstable::bytes_on_disk)), uint64_t(0), std::plus{}); }; // Targets a space overhead of 10%. All disjoint sstables can be compacted together as long as they won't diff --git a/cql3/restrictions/statement_restrictions.cc b/cql3/restrictions/statement_restrictions.cc index 095dc9bdf22a..edc13cfa9b2e 100644 --- a/cql3/restrictions/statement_restrictions.cc +++ b/cql3/restrictions/statement_restrictions.cc @@ -9,6 +9,7 @@ #include #include +#include #include #include #include diff --git a/cql3/selection/selection.cc b/cql3/selection/selection.cc index be590de3a4db..8e50db3b7132 100644 --- a/cql3/selection/selection.cc +++ b/cql3/selection/selection.cc @@ -494,7 +494,7 @@ ::shared_ptr selection::from_selectors(data_dictionary::database db, auto metadata = collect_metadata(*schema, prepared_selectors); if (processes_selection(prepared_selectors) || prepared_selectors.size() != defs.size()) { return ::make_shared(schema, std::move(defs), std::move(metadata), - boost::copy_range>(prepared_selectors | boost::adaptors::transformed(std::mem_fn(&prepared_selector::expr)))); + prepared_selectors | std::views::transform(std::mem_fn(&prepared_selector::expr)) | std::ranges::to()); } else { return ::make_shared(schema, std::move(defs), std::move(metadata), false); } @@ -583,11 +583,9 @@ bool result_set_builder::last_group_ended() const { if (_last_group.empty()) { return !_selectors->is_aggregate(); } - using boost::adaptors::reversed; - using boost::adaptors::transformed; - return !boost::equal( - _last_group | reversed, - _group_by_cell_indices | reversed | transformed([this](size_t i) { return current[i]; })); + return !std::ranges::equal( + _last_group | std::views::reverse, + _group_by_cell_indices | std::views::reverse | std::views::transform([this](size_t i) { return current[i]; })); } void result_set_builder::flush_selectors() { diff --git a/cql3/statements/alter_table_statement.cc b/cql3/statements/alter_table_statement.cc index e8072a04fb8e..b57696bd7b9b 100644 --- a/cql3/statements/alter_table_statement.cc +++ b/cql3/statements/alter_table_statement.cc @@ -21,7 +21,6 @@ #include "timestamp.hh" #include "validation.hh" #include "db/extensions.hh" -#include #include "cql3/util.hh" #include "view_info.hh" #include "data_dictionary/data_dictionary.hh" @@ -141,7 +140,7 @@ static void validate_column_rename(data_dictionary::database db, const schema& s throw exceptions::invalid_request_exception( seastar::format("Cannot rename column {} because it has dependent secondary indexes ({})", from, - fmt::join(dependent_indices | boost::adaptors::transformed([](const index_metadata& im) { + fmt::join(dependent_indices | std::views::transform([](const index_metadata& im) { return im.name(); }), ", "))); } diff --git a/cql3/statements/authorization_statement.cc b/cql3/statements/authorization_statement.cc index dc19eaee31a9..f6eba3d5cf53 100644 --- a/cql3/statements/authorization_statement.cc +++ b/cql3/statements/authorization_statement.cc @@ -65,12 +65,11 @@ void cql3::statements::authorization_statement::maybe_correct_resource(auth::res auto function_args = functions_view.function_args(); std::vector parsed_types; if (function_args) { - parsed_types = boost::copy_range>( - *function_args | boost::adaptors::transformed([&] (std::string_view raw_type) { + parsed_types = + *function_args | std::views::transform([&] (std::string_view raw_type) { auto parsed = db::cql_type_parser::parse(sstring(keyspace->data(), keyspace->size()), sstring(raw_type.data(), raw_type.size()), utm); return parsed->is_user_type() ? parsed->freeze() : parsed; - }) - ); + }) | std::ranges::to(); } resource = auth::make_functions_resource(*keyspace, auth::encode_signature(function_name, parsed_types)); } diff --git a/cql3/statements/batch_statement.cc b/cql3/statements/batch_statement.cc index e52d29639477..aee5f30b6e04 100644 --- a/cql3/statements/batch_statement.cc +++ b/cql3/statements/batch_statement.cc @@ -19,6 +19,7 @@ #include "service/storage_proxy.hh" #include "tracing/trace_state.hh" +#include #include template diff --git a/cql3/statements/create_aggregate_statement.cc b/cql3/statements/create_aggregate_statement.cc index 6722c0312c14..2d41c2e96c4e 100644 --- a/cql3/statements/create_aggregate_statement.cc +++ b/cql3/statements/create_aggregate_statement.cc @@ -19,7 +19,6 @@ #include "mutation/mutation.hh" #include "cql3/query_processor.hh" #include "gms/feature_service.hh" -#include namespace cql3 { @@ -40,7 +39,7 @@ seastar::future> create_aggregate_statement: acc_types.insert(acc_types.end(), _arg_types.begin(), _arg_types.end()); auto state_func = dynamic_pointer_cast(functions::instance().find(functions::function_name{_name.keyspace, _sfunc}, acc_types)); if (!state_func) { - auto acc_type_names = acc_types | boost::adaptors::transformed([] (auto&& t) { return t->cql3_type_name(); }); + auto acc_type_names = acc_types | std::views::transform([] (auto&& t) { return t->cql3_type_name(); }); throw exceptions::invalid_request_exception(seastar::format("State function {}({}) not found", _sfunc, fmt::join(acc_type_names, ", "))); } if (state_func->return_type() != state_type) { diff --git a/cql3/statements/create_index_statement.cc b/cql3/statements/create_index_statement.cc index f065a75d7087..20f57c02a5ba 100644 --- a/cql3/statements/create_index_statement.cc +++ b/cql3/statements/create_index_statement.cc @@ -27,8 +27,6 @@ #include "index/secondary_index_manager.hh" #include "mutation/mutation.hh" -#include -#include #include namespace cql3 { @@ -179,7 +177,7 @@ std::vector<::shared_ptr> create_index_statement::validate_while_e void create_index_statement::validate_for_local_index(const schema& schema) const { if (!_raw_targets.empty()) { if (const auto* index_pk = std::get_if>>(&_raw_targets.front()->value)) { - auto base_pk_identifiers = *index_pk | boost::adaptors::transformed([&schema] (const ::shared_ptr& raw_ident) { + auto base_pk_identifiers = *index_pk | std::views::transform([&schema] (const ::shared_ptr& raw_ident) { return raw_ident->prepare_column_identifier(schema); }); auto remaining_base_pk_columns = schema.partition_key_columns(); diff --git a/cql3/statements/create_view_statement.cc b/cql3/statements/create_view_statement.cc index d5f3a9598dcb..157768c6ddb9 100644 --- a/cql3/statements/create_view_statement.cc +++ b/cql3/statements/create_view_statement.cc @@ -13,11 +13,6 @@ #include #include -#include -#include -#include -#include - #include #include "cql3/column_identifier.hh" #include "cql3/restrictions/statement_restrictions.hh" @@ -166,7 +161,7 @@ std::pair create_view_statement::prepare_view( } // Gather all included columns, as specified by the select clause - auto included = boost::copy_range>(_select_clause | boost::adaptors::transformed([&](auto&& selector) { + auto included = _select_clause | std::views::transform([&](auto&& selector) { if (selector->alias) { throw exceptions::invalid_request_exception(format("Cannot use alias when defining a materialized view")); } @@ -183,7 +178,7 @@ std::pair create_view_statement::prepare_view( throw exceptions::invalid_request_exception(format("Unknown column name detected in CREATE MATERIALIZED VIEW statement: {}", identifier)); } return def; - })); + }) | std::ranges::to>(); auto parameters = make_lw_shared(raw::select_statement::parameters::orderings_type(), false, true); raw::select_statement raw_select(_base_name, std::move(parameters), _select_clause, _where_clause, std::nullopt, std::nullopt, {}, std::make_unique()); @@ -251,7 +246,7 @@ std::pair create_view_statement::prepare_view( throw exceptions::invalid_request_exception(seastar::format( "Cannot create Materialized View {} without primary key columns from base {} ({})", column_family(), _base_name.get_column_family(), - fmt::join(missing_pk_columns | boost::adaptors::transformed(std::mem_fn(&column_definition::name_as_text)), ", "))); + fmt::join(missing_pk_columns | std::views::transform(std::mem_fn(&column_definition::name_as_text)), ", "))); } if (_partition_keys.empty()) { diff --git a/cql3/statements/describe_statement.cc b/cql3/statements/describe_statement.cc index 7bd78dc82633..a6eabbab643d 100644 --- a/cql3/statements/describe_statement.cc +++ b/cql3/statements/describe_statement.cc @@ -9,8 +9,6 @@ #include #include #include -#include -#include #include "cdc/cdc_options.hh" #include "cdc/log.hh" @@ -331,9 +329,9 @@ future> tables(const data_dictionary::database& db, con } replica::schema_describe_helper describe_helper{db}; - co_return boost::copy_range>(tables | boost::adaptors::transformed([&describe_helper] (auto&& t) { + co_return tables | std::views::transform([&describe_helper] (auto&& t) { return t->describe(describe_helper, describe_option::NO_STMTS); - })); + }) | std::ranges::to(); } // DESCRIBE UTILITY @@ -446,9 +444,9 @@ std::vector> get_element_column_specificatio } std::vector> serialize_descriptions(std::vector&& descs, bool serialize_create_statement = true) { - return boost::copy_range>>(descs | boost::adaptors::transformed([serialize_create_statement] (const description& desc) { + return descs | std::views::transform([serialize_create_statement] (const description& desc) { return desc.serialize(serialize_create_statement); - })); + }) | std::ranges::to(); } @@ -535,15 +533,15 @@ future cluster_describe_statement::range_ownership(const service::sto return std::stol(r._start_token); }); - auto ring_ranges = boost::copy_range>>(ranges | boost::adaptors::transformed([list_type = std::move(list_type)] (auto& range) { + auto ring_ranges = ranges | std::views::transform([list_type = std::move(list_type)] (auto& range) { auto token_end = data_value(range._end_token); - auto endpoints = boost::copy_range>(range._endpoints | boost::adaptors::transformed([] (const auto& endpoint) { + auto endpoints = range._endpoints | std::views::transform([] (const auto& endpoint) { return data_value(endpoint); - })); + }) | std::ranges::to(); auto endpoints_list = make_list_value(list_type, endpoints); return std::pair(token_end, endpoints_list); - })); + }) | std::ranges::to(); co_return make_map_value(map_type, map_type_impl::native_type( std::make_move_iterator(ring_ranges.begin()), diff --git a/cql3/statements/index_target.cc b/cql3/statements/index_target.cc index 96285f171dcd..092c66b8078b 100644 --- a/cql3/statements/index_target.cc +++ b/cql3/statements/index_target.cc @@ -12,8 +12,6 @@ #include #include "index_target.hh" #include "index/secondary_index.hh" -#include -#include namespace cql3 { @@ -29,10 +27,10 @@ sstring index_target::column_name() const { struct as_string_visitor { const index_target* target; sstring operator()(const std::vector<::shared_ptr>& columns) const { - return "(" + boost::algorithm::join(columns | boost::adaptors::transformed( + return seastar::format("({})", fmt::join(columns | std::views::transform( [](const ::shared_ptr& ident) -> sstring { return ident->to_string(); - }), ",") + ")"; + }), ",")); } sstring operator()(const ::shared_ptr& column) const { @@ -138,11 +136,10 @@ index_target::raw::prepare(const schema& s) const { target_type _type; ::shared_ptr operator()(const std::vector<::shared_ptr>& columns) const { - auto prepared_idents = boost::copy_range>>( - columns | boost::adaptors::transformed([this] (const ::shared_ptr& raw_ident) { + auto prepared_idents = + columns | std::views::transform([this] (const ::shared_ptr& raw_ident) { return raw_ident->prepare_column_identifier(_schema); - }) - ); + }) | std::ranges::to(); return ::make_shared(std::move(prepared_idents), _type); } diff --git a/cql3/statements/modification_statement.cc b/cql3/statements/modification_statement.cc index b7885914c4c1..0bbb548f3c21 100644 --- a/cql3/statements/modification_statement.cc +++ b/cql3/statements/modification_statement.cc @@ -21,9 +21,6 @@ #include "db/consistency_level_validations.hh" #include #include -#include -#include -#include #include "transport/messages/result_message.hh" #include "data_dictionary/data_dictionary.hh" #include "replica/database.hh" diff --git a/cql3/statements/select_statement.cc b/cql3/statements/select_statement.cc index 12e7c0002053..40c6d5b6ef36 100644 --- a/cql3/statements/select_statement.cc +++ b/cql3/statements/select_statement.cc @@ -2035,7 +2035,7 @@ std::unique_ptr select_statement::prepare(data_dictionary::d auto all_aggregates = [] (const std::vector& prepared_selectors) { return std::ranges::all_of( - prepared_selectors | boost::adaptors::transformed(std::mem_fn(&selection::prepared_selector::expr)), + prepared_selectors | std::views::transform(std::mem_fn(&selection::prepared_selector::expr)), [] (const expr::expression& e) { auto fn_expr = expr::as_if(&e); if (!fn_expr) { @@ -2587,8 +2587,9 @@ std::unique_ptr build_select_statement( // If the column name is not entirely lowercase (or digits or _), // when output to CQL it must be quoted to preserve case as well // as non alphanumeric characters. - auto cols = boost::copy_range>(selected_columns - | boost::adaptors::transformed(std::mem_fn(&column_definition::name_as_cql_string))); + auto cols = selected_columns + | std::views::transform(std::mem_fn(&column_definition::name_as_cql_string)) + | std::ranges::to(); fmt::print(out, "{}", fmt::join(cols, ", ")); } // Note that cf_name may need to be quoted, just like column names above. diff --git a/cql3/user_types.cc b/cql3/user_types.cc index c24905a0a9ca..02f96a864fb0 100644 --- a/cql3/user_types.cc +++ b/cql3/user_types.cc @@ -14,8 +14,6 @@ #include "cql3/expr/evaluate.hh" #include "cql3/expr/expr-utils.hh" -#include - #include "mutation/mutation.hh" #include "types/user.hh" diff --git a/cql3/util.cc b/cql3/util.cc index e6e9d94c1212..566cff198919 100644 --- a/cql3/util.cc +++ b/cql3/util.cc @@ -138,8 +138,8 @@ sstring relations_to_where_clause(const expr::expression& e) { return fmt::format("{:user}", e); }; auto relations = expr::boolean_factors(e); - auto expressions = relations | boost::adaptors::transformed(expr_to_pretty_string); - return boost::algorithm::join(expressions, " AND "); + auto expressions = relations | std::views::transform(expr_to_pretty_string); + return fmt::to_string(fmt::join(expressions, " AND ")); } expr::expression where_clause_to_relations(const std::string_view& where_clause, dialect d) { diff --git a/db/commitlog/commitlog.cc b/db/commitlog/commitlog.cc index 700b2cdaba01..79b208eba50b 100644 --- a/db/commitlog/commitlog.cc +++ b/db/commitlog/commitlog.cc @@ -55,9 +55,6 @@ #include "commitlog_entry.hh" #include "commitlog_extensions.hh" -#include -#include - #include "checked-file-impl.hh" #include "utils/disk-error-handler.hh" diff --git a/db/consistency_level.cc b/db/consistency_level.cc index 16064bdc8504..4d1ffbf072c3 100644 --- a/db/consistency_level.cc +++ b/db/consistency_level.cc @@ -323,13 +323,13 @@ filter_for_query(consistency_level cl, float ht_min = 1; bool old_node = false; - auto epi = boost::copy_range>>(live_endpoints | boost::adaptors::transformed([&] (gms::inet_address ep) { + auto epi = live_endpoints | std::views::transform([&] (gms::inet_address ep) { auto ht = get_hit_rate(ep); old_node = old_node || ht < 0; ht_max = std::max(ht_max, ht); ht_min = std::min(ht_min, ht); return std::make_pair(ep, ht); - })); + }) | std::ranges::to>>(); if (!old_node && ht_max - ht_min > 0.01) { // if there is old node or hit rates are close skip calculations // local node is always first if present (see storage_proxy::get_endpoints_for_reading) diff --git a/db/extensions.cc b/db/extensions.cc index 02a9afd7e9a6..64aee28e2d7a 100644 --- a/db/extensions.cc +++ b/db/extensions.cc @@ -11,8 +11,6 @@ #include "sstables/sstables.hh" #include "commitlog/commitlog_extensions.hh" #include "schema/schema.hh" -#include -#include db::extensions::extensions() {} diff --git a/db/schema_applier.cc b/db/schema_applier.cc index f02fcae41752..54893460326b 100644 --- a/db/schema_applier.cc +++ b/db/schema_applier.cc @@ -211,10 +211,10 @@ future> static read_table_names_of_keyspace(distributed& proxy, const sstring& keyspace_name, schema_ptr schema_table) { auto pkey = dht::decorate_key(*schema_table, partition_key::from_singular(*schema_table, keyspace_name)); auto&& rs = co_await db::system_keyspace::query(proxy.local().get_db(), schema_table->ks_name(), schema_table->cf_name(), pkey); - co_return boost::copy_range>(rs->rows() | boost::adaptors::transformed([schema_table] (const query::result_set_row& row) { + co_return rs->rows() | std::views::transform([schema_table] (const query::result_set_row& row) { const sstring name = schema_table->clustering_key_columns().begin()->name_as_text(); return row.get_nonnull(name); - })); + }) | std::ranges::to(); } // Applies deletion of the "version" column to system_schema.scylla_tables mutation rows diff --git a/db/schema_tables.cc b/db/schema_tables.cc index 5d2016ed2026..7e621bf4bea6 100644 --- a/db/schema_tables.cc +++ b/db/schema_tables.cc @@ -2346,10 +2346,9 @@ static computed_columns_map get_computed_columns(const schema_mutations& sm) { return {}; } query::result_set computed_result(*sm.computed_columns_mutation()); - return boost::copy_range( - computed_result.rows() | boost::adaptors::transformed([] (const query::result_set_row& row) { + return computed_result.rows() | std::views::transform([] (const query::result_set_row& row) { return computed_columns_map::value_type{to_bytes(row.get_nonnull("column_name")), column_computation::deserialize(row.get_nonnull("computation"))}; - })); + }) | std::ranges::to(); } static std::vector create_columns_from_column_rows(const schema_ctxt& ctxt, @@ -2390,9 +2389,9 @@ static std::vector create_indices_from_index_rows(const query::r const sstring& keyspace, const sstring& table) { - return boost::copy_range>(rows.rows() | boost::adaptors::transformed([&keyspace, &table] (auto&& row) { + return rows.rows() | std::views::transform([&keyspace, &table] (auto&& row) { return create_index_from_index_row(row, keyspace, table); - })); + }) | std::ranges::to>(); } static index_metadata create_index_from_index_row(const query::result_set_row& row, @@ -2622,8 +2621,9 @@ std::vector all_tables(schema_features features) { } std::vector all_table_names(schema_features features) { - return boost::copy_range>(all_tables(features) | - boost::adaptors::transformed([] (auto schema) { return schema->cf_name(); })); + return all_tables(features) + | std::views::transform([] (auto schema) { return schema->cf_name(); }) + | std::ranges::to(); } void check_no_legacy_secondary_index_mv_schema(replica::database& db, const view_ptr& v, schema_ptr base_schema) { diff --git a/db/size_estimates_virtual_reader.cc b/db/size_estimates_virtual_reader.cc index feaa6f2178b8..2e7afbcd5991 100644 --- a/db/size_estimates_virtual_reader.cc +++ b/db/size_estimates_virtual_reader.cc @@ -11,9 +11,6 @@ #include #include "utils/assert.hh" -#include -#include -#include #include "clustering_bounds_comparator.hh" #include "replica/database_fwd.hh" @@ -311,9 +308,9 @@ size_estimates_mutation_reader::estimates_for_current_keyspace(std::vectordecompose(*_current_partition)); auto cfs = _db.find_keyspace(*_current_partition).metadata()->cf_meta_data(); - auto cf_names = boost::copy_range>(cfs | boost::adaptors::transformed([] (auto&& cf) { + auto cf_names = cfs | std::views::transform([] (auto&& cf) { return utf8_type->decompose(cf.first); - })); + }) | std::ranges::to>(); std::ranges::sort(cf_names, [] (auto&& n1, auto&& n2) { return utf8_type->less(n1, n2); }); diff --git a/db/system_distributed_keyspace.cc b/db/system_distributed_keyspace.cc index f4439bdd381e..8c4f39508b89 100644 --- a/db/system_distributed_keyspace.cc +++ b/db/system_distributed_keyspace.cc @@ -31,8 +31,6 @@ #include #include -#include - #include #include diff --git a/db/system_keyspace.cc b/db/system_keyspace.cc index 1ad43de2d84a..7dcd2402b518 100644 --- a/db/system_keyspace.cc +++ b/db/system_keyspace.cc @@ -8,7 +8,6 @@ */ #include -#include #include #include #include @@ -2526,12 +2525,12 @@ future<> system_keyspace::remove_built_view(sstring ks_name, sstring view_name) future> system_keyspace::load_built_views() { return execute_cql(format("SELECT * FROM system.{}", v3::BUILT_VIEWS)).then([] (::shared_ptr cql_result) { - return boost::copy_range>(*cql_result - | boost::adaptors::transformed([] (const cql3::untyped_result_set::row& row) { + return *cql_result + | std::views::transform([] (const cql3::untyped_result_set::row& row) { auto ks_name = row.get_as("keyspace_name"); auto cf_name = row.get_as("view_name"); return std::pair(std::move(ks_name), std::move(cf_name)); - })); + }) | std::ranges::to>(); }); } diff --git a/db/view/view.cc b/db/view/view.cc index a833d6b86a1b..ab33a82c2727 100644 --- a/db/view/view.cc +++ b/db/view/view.cc @@ -8,7 +8,6 @@ * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) */ -#include #include #include #include @@ -17,9 +16,6 @@ #include #include -#include -#include -#include #include #include @@ -720,8 +716,8 @@ view_updates::get_view_rows(const partition_key& base_key, const clustering_or_s std::vector ret; auto compute_row = [&](Range&& pk, Range&& ck) { - partition_key pkey = partition_key::from_range(boost::adaptors::transform(pk, view_managed_key_view_and_action::get_key_view)); - clustering_key ckey = clustering_key::from_range(boost::adaptors::transform(ck, view_managed_key_view_and_action::get_key_view)); + partition_key pkey = partition_key::from_range(std::views::transform(pk, view_managed_key_view_and_action::get_key_view)); + clustering_key ckey = clustering_key::from_range(std::views::transform(ck, view_managed_key_view_and_action::get_key_view)); auto action = (action_column < pk.size() ? pk[action_column] : ck[action_column - pk.size()])._action; mutation_partition& partition = partition_for(std::move(pkey)); @@ -738,11 +734,11 @@ view_updates::get_view_rows(const partition_key& base_key, const clustering_or_s if (had_multiple_values_in_pk) { // cartesian_product expects std::vector>, while we have std::vector. std::vector> pk_elems_, ck_elems_; - auto std_vector_from_small_vector = boost::adaptors::transformed([](const auto& vector) { + auto std_vector_from_small_vector = std::views::transform([](const auto& vector) { return std::vector{vector.begin(), vector.end()}; }); - boost::copy(pk_elems | std_vector_from_small_vector, std::back_inserter(pk_elems_)); - boost::copy(ck_elems | std_vector_from_small_vector, std::back_inserter(ck_elems_)); + std::ranges::copy(pk_elems | std_vector_from_small_vector, std::back_inserter(pk_elems_)); + std::ranges::copy(ck_elems | std_vector_from_small_vector, std::back_inserter(ck_elems_)); auto cartesian_product_pk = cartesian_product(pk_elems_), cartesian_product_ck = cartesian_product(ck_elems_); @@ -776,7 +772,7 @@ view_updates::get_view_rows(const partition_key& base_key, const clustering_or_s } } else { // Here it's the old regular index over regular values. Each vector has just one element. - auto get_front = boost::adaptors::transformed([](const auto& v) { return v.front(); }); + auto get_front = std::views::transform([](const auto& v) { return v.front(); }); compute_row(pk_elems | get_front, ck_elems | get_front); } @@ -1613,14 +1609,14 @@ view_update_builder make_view_update_builder( mutation_reader&& updates, mutation_reader_opt&& existings, gc_clock::time_point now) { - auto vs = boost::copy_range>(views_to_update | boost::adaptors::transformed([&] (view_and_base v) { + auto vs = views_to_update | std::views::transform([&] (view_and_base v) { if (base->version() != v.base->base_schema()->version()) { on_internal_error(vlogger, format("Schema version used for view updates ({}) does not match the current" " base schema version of the view ({}) for view {}.{} of {}.{}", base->version(), v.base->base_schema()->version(), v.view->ks_name(), v.view->cf_name(), base->ks_name(), base->cf_name())); } return view_updates(std::move(v)); - })); + }) | std::ranges::to>(); return view_update_builder(std::move(db), base_table, base, std::move(vs), std::move(updates), std::move(existings), now); } @@ -1678,11 +1674,11 @@ future calculate_affected_clustering_ranges(data_d // this mutation. //FIXME: Unfortunate copy. - co_return boost::copy_range( - interval::deoverlap(std::move(row_ranges), cmp) - | boost::adaptors::transformed([] (auto&& v) { + co_return interval::deoverlap(std::move(row_ranges), cmp) + | std::views::transform([] (auto&& v) { return std::move(v).transform([] (auto&& ckv) { return clustering_key_prefix(ckv); }); - })); + }) + | std::ranges::to(); } @@ -2419,12 +2415,13 @@ view_status_common(cql3::query_processor& qp, sstring ks_name, sstring cf_name, view_builder_query_state(), { std::move(view_ks_name), std::move(view_name) }, cql3::query_processor::cache_internal::no).then([] (::shared_ptr cql_result) { - return boost::copy_range>(*cql_result - | boost::adaptors::transformed([] (const cql3::untyped_result_set::row& row) { + return *cql_result + | std::views::transform([] (const cql3::untyped_result_set::row& row) { auto host_id = locator::host_id(row.get_as("host_id")); auto status = row.get_as("status"); return std::pair(std::move(host_id), std::move(status)); - })); + }) + | std::ranges::to>(); }); } @@ -2659,7 +2656,7 @@ future<> view_builder::migrate_to_v2(locator::token_metadata_ptr tmptr, db::syst co_await utils::get_local_injector().inject("view_builder_pause_in_migrate_v2", utils::wait_for_message(5min)); - auto col_names = boost::copy_range>(schema->all_columns() | boost::adaptors::transformed([] (const auto& col) {return col.name_as_cql_string(); })); + auto col_names = schema->all_columns() | std::views::transform([] (const auto& col) {return col.name_as_cql_string(); }) | std::ranges::to>(); auto col_names_str = boost::algorithm::join(col_names, ", "); sstring val_binders_str = "?"; for (size_t i = 1; i < col_names.size(); ++i) { @@ -2986,10 +2983,9 @@ class view_builder::consumer { built_views consume_end_of_stream() { inject_failure("view_builder_consume_end_of_stream"); if (vlogger.is_enabled(log_level::debug)) { - auto view_names = boost::copy_range>( - _views_to_build | boost::adaptors::transformed([](auto v) { + auto view_names = _views_to_build | std::views::transform([](auto v) { return v->cf_name(); - })); + }) | std::ranges::to>(); vlogger.debug("Completed build step for base {}.{}, at token {}; views={}", _step.base->schema()->ks_name(), _step.base->schema()->cf_name(), _step.current_token(), view_names); } @@ -3231,9 +3227,9 @@ view_updating_consumer::view_updating_consumer(view_update_generator& gen, schem { } std::vector with_base_info_snapshot(std::vector vs) { - return boost::copy_range>(vs | boost::adaptors::transformed([] (const view_ptr& v) { + return vs | std::views::transform([] (const view_ptr& v) { return db::view::view_and_base{v, v->view_info()->base_info()}; - })); + }) | std::ranges::to(); } delete_ghost_rows_visitor::delete_ghost_rows_visitor(service::storage_proxy& proxy, service::query_state& state, view_ptr view, db::timeout_clock::duration timeout_duration) diff --git a/db/view/view_update_generator.cc b/db/view/view_update_generator.cc index 1da294dc32a4..a26278f02dee 100644 --- a/db/view/view_update_generator.cc +++ b/db/view/view_update_generator.cc @@ -312,9 +312,9 @@ void view_update_generator::discover_staging_sstables() { } static size_t memory_usage_of(const utils::chunked_vector& ms) { - return boost::accumulate(ms | boost::adaptors::transformed([] (const frozen_mutation_and_schema& m) { + return std::ranges::fold_left(ms | std::views::transform([] (const frozen_mutation_and_schema& m) { return memory_usage_of(m); - }), 0); + }), 0, std::plus{}); } /** diff --git a/db/virtual_tables.cc b/db/virtual_tables.cc index 804900a7ee4b..11aa6078b38a 100644 --- a/db/virtual_tables.cc +++ b/db/virtual_tables.cc @@ -190,11 +190,11 @@ class token_ring_table : public streaming_virtual_table { dht::decorated_key key; }; - auto keyspace_names = boost::copy_range>( - _db.get_non_local_strategy_keyspaces() - | boost::adaptors::transformed([this] (auto&& ks) { + auto keyspace_names = _db.get_non_local_strategy_keyspaces() + | std::views::transform([this] (auto&& ks) { return decorated_keyspace_name{ks, make_partition_key(ks)}; - })); + }) + | std::ranges::to(); std::ranges::sort(keyspace_names, dht::ring_position_less_comparator(*_s), std::mem_fn(&decorated_keyspace_name::key)); @@ -339,8 +339,9 @@ class protocol_servers_table : public memtable_filling_virtual_table { future<> execute(std::function mutation_sink) override { // Servers are registered on shard 0 only const auto server_infos = co_await smp::submit_to(0ul, [&ss = _ss.container()] { - return boost::copy_range>(ss.local().protocol_servers() - | boost::adaptors::transformed([] (protocol_server* s) { return protocol_server_info(*s); })); + return ss.local().protocol_servers() + | std::views::transform([] (protocol_server* s) { return protocol_server_info(*s); }) + | std::ranges::to(); }); for (auto server : server_infos) { auto dk = dht::decorate_key(*_s, partition_key::from_single_value(*schema(), data_value(server.name).serialize_nonnull())); diff --git a/dht/i_partitioner.cc b/dht/i_partitioner.cc index de0f0616e4f6..5b9dd28122b5 100644 --- a/dht/i_partitioner.cc +++ b/dht/i_partitioner.cc @@ -14,9 +14,6 @@ #include "dht/token-sharding.hh" #include "utils/assert.hh" #include "utils/class_registrator.hh" -#include -#include -#include #include "sstables/key.hh" #include #include diff --git a/gms/versioned_value.cc b/gms/versioned_value.cc index 281c8b2d30e9..4dbc758dcc2c 100644 --- a/gms/versioned_value.cc +++ b/gms/versioned_value.cc @@ -12,7 +12,6 @@ #include #include -#include #include namespace gms { @@ -25,7 +24,7 @@ versioned_value versioned_value::network_version() { } sstring versioned_value::make_full_token_string(const std::unordered_set& tokens) { - return fmt::to_string(fmt::join(tokens | boost::adaptors::transformed([] (const dht::token& t) { + return fmt::to_string(fmt::join(tokens | std::views::transform([] (const dht::token& t) { return t.to_sstring(); }), ";")); } diff --git a/keys.cc b/keys.cc index 067f7678a051..0b67d872e554 100644 --- a/keys.cc +++ b/keys.cc @@ -12,7 +12,6 @@ #include "dht/i_partitioner.hh" #include "clustering_bounds_comparator.hh" #include -#include logging::logger klog("keys"); @@ -111,8 +110,8 @@ const thread_local clustering_key_prefix bound_view::_empty_prefix = clustering_ std::ostream& operator<<(std::ostream& os, const exploded_clustering_prefix& ecp) { - // Can't pass to_hex() to transformed(), since it is overloaded, so wrap: + // Can't pass to_hex() to transform(), since it is overloaded, so wrap: auto enhex = [] (auto&& x) { return fmt_hex(x); }; - fmt::print(os, "prefix{{{}}}", fmt::join(ecp._v | boost::adaptors::transformed(enhex), ":")); + fmt::print(os, "prefix{{{}}}", fmt::join(ecp._v | std::views::transform(enhex), ":")); return os; } diff --git a/main.cc b/main.cc index 4962e4f920d9..e4a38a6787ef 100644 --- a/main.cc +++ b/main.cc @@ -524,7 +524,7 @@ class memory_threshold_guard { // "[key1: value1_1 value1_2 ..., key2: value2_1 value 2_2 ..., (positional) value3, ...]" std::string format_parsed_options(const std::vector& opts) { return fmt::format("[{}]", - boost::algorithm::join(opts | boost::adaptors::transformed([] (const bpo::option& opt) { + fmt::join(opts | std::views::transform([] (const bpo::option& opt) { if (opt.value.empty()) { return opt.string_key; } diff --git a/mutation/frozen_mutation.cc b/mutation/frozen_mutation.cc index 7b4227e58131..759b331aa374 100644 --- a/mutation/frozen_mutation.cc +++ b/mutation/frozen_mutation.cc @@ -7,7 +7,6 @@ */ #include -#include #include "frozen_mutation.hh" #include "schema/schema_registry.hh" #include "mutation_partition.hh" @@ -119,15 +118,15 @@ frozen_mutation freeze(const mutation& m) { } std::vector freeze(const std::vector& muts) { - return boost::copy_range>(muts | boost::adaptors::transformed([] (const mutation& m) { + return muts | std::views::transform([] (const mutation& m) { return freeze(m); - })); + }) | std::ranges::to>(); } std::vector unfreeze(const std::vector& muts) { - return boost::copy_range>(muts | boost::adaptors::transformed([] (const frozen_mutation& fm) { + return muts | std::views::transform([] (const frozen_mutation& fm) { return fm.unfreeze(local_schema_registry().get(fm.schema_version())); - })); + }) | std::ranges::to>(); } diff --git a/mutation/mutation_partition.cc b/mutation/mutation_partition.cc index 6fcaedf4f347..8ec1a1eb25bf 100644 --- a/mutation/mutation_partition.cc +++ b/mutation/mutation_partition.cc @@ -9,8 +9,6 @@ #include #include -#include - #include "mutation_partition.hh" #include "clustering_interval_set.hh" #include "converting_mutation_partition_applier.hh" @@ -838,7 +836,7 @@ operator<<(std::ostream& os, const std::pair static auto prefixed(const sstring& prefix, const RangeOfPrintable& r) { - return r | boost::adaptors::transformed([&] (auto&& e) { return format("{}{}", prefix, e); }); + return r | std::views::transform([&] (auto&& e) { return format("{}{}", prefix, e); }); } std::ostream& diff --git a/mutation/mutation_partition_v2.cc b/mutation/mutation_partition_v2.cc index f4ff45864a79..b62470d8ea75 100644 --- a/mutation/mutation_partition_v2.cc +++ b/mutation/mutation_partition_v2.cc @@ -9,8 +9,6 @@ #include #include -#include - #include "mutation_partition_v2.hh" #include "clustering_interval_set.hh" #include "converting_mutation_partition_applier.hh" @@ -761,7 +759,7 @@ void mutation_partition_v2::for_each_row(const schema& schema, const query::clus // in the original range is prefxied with given string. template static auto prefixed(const sstring& prefix, const RangeOfPrintable& r) { - return r | boost::adaptors::transformed([&] (auto&& e) { return format("{}{}", prefix, e); }); + return r | std::views::transform([&] (auto&& e) { return format("{}{}", prefix, e); }); } auto fmt::formatter::format(const mutation_partition_v2::printer& p, fmt::format_context& ctx) const diff --git a/mutation_writer/shard_based_splitting_writer.cc b/mutation_writer/shard_based_splitting_writer.cc index 4603602a270f..a0e17f12b01e 100644 --- a/mutation_writer/shard_based_splitting_writer.cc +++ b/mutation_writer/shard_based_splitting_writer.cc @@ -8,8 +8,6 @@ #include "mutation_writer/shard_based_splitting_writer.hh" -#include -#include #include #include "dht/i_partitioner.hh" #include "mutation_writer/feed_writers.hh" diff --git a/mutation_writer/timestamp_based_splitting_writer.cc b/mutation_writer/timestamp_based_splitting_writer.cc index 1c059d7cdd90..b4abfa712fef 100644 --- a/mutation_writer/timestamp_based_splitting_writer.cc +++ b/mutation_writer/timestamp_based_splitting_writer.cc @@ -9,8 +9,6 @@ #include "mutation_writer/timestamp_based_splitting_writer.hh" #include -#include -#include #include #include diff --git a/node_ops/task_manager_module.cc b/node_ops/task_manager_module.cc index 04566577ec82..14f72a06319a 100644 --- a/node_ops/task_manager_module.cc +++ b/node_ops/task_manager_module.cc @@ -143,8 +143,8 @@ future<> node_ops_virtual_task::abort(tasks::task_id id) noexcept { future> node_ops_virtual_task::get_stats() { db::system_keyspace& sys_ks = _ss._sys_ks.local(); service::topology& topology = _ss._topology_state_machine._topology; - co_return boost::copy_range>(co_await get_entries(sys_ks, topology, get_task_manager().get_user_task_ttl()) - | boost::adaptors::transformed([] (const auto& e) { + co_return std::ranges::to>(co_await get_entries(sys_ks, topology, get_task_manager().get_user_task_ttl()) + | std::views::transform([] (const auto& e) { auto id = e.first; auto& entry = e.second; return tasks::task_stats { diff --git a/query-result-reader.hh b/query-result-reader.hh index 3fc7095f0273..453b3e8959ad 100644 --- a/query-result-reader.hh +++ b/query-result-reader.hh @@ -9,8 +9,6 @@ #pragma once #include "utils/assert.hh" -#include -#include #include "query-result.hh" #include "full_position.hh" diff --git a/query-result-set.cc b/query-result-set.cc index 9e956d40c46d..f0ea8edf4bd5 100644 --- a/query-result-set.cc +++ b/query-result-set.cc @@ -160,7 +160,7 @@ result_set_builder::deserialize(const result_row_view& row, bool is_static) std::unordered_map cells; auto i = row.iterator(); auto column_ids = is_static ? _slice.static_columns : _slice.regular_columns; - auto columns = column_ids | boost::adaptors::transformed([this, is_static] (column_id id) -> const column_definition& { + auto columns = column_ids | std::views::transform([this, is_static] (column_id id) -> const column_definition& { if (is_static) { return _schema->static_column_at(id); } else { diff --git a/repair/row_level.cc b/repair/row_level.cc index 95d1135c835f..392de27de5bb 100644 --- a/repair/row_level.cc +++ b/repair/row_level.cc @@ -1327,8 +1327,9 @@ class repair_meta { } } if (update_hash_set) { - _peer_row_hash_sets[node_idx] = boost::copy_range(row_diff | - boost::adaptors::transformed([] (repair_row& r) { thread::maybe_yield(); return r.hash(); })); + _peer_row_hash_sets[node_idx] = row_diff + | std::views::transform([] (repair_row& r) { thread::maybe_yield(); return r.hash(); }) + | std::ranges::to(); } // Repair rows in row_diff will be flushed to disk by flush_rows_in_working_row_buf, // so we skip calling do_apply_rows here. diff --git a/replica/table.cc b/replica/table.cc index 70da042fb45a..b9405a9c0400 100644 --- a/replica/table.cc +++ b/replica/table.cc @@ -38,8 +38,6 @@ #include "db/extensions.hh" #include "query-result-writer.hh" #include "db/view/view_update_generator.hh" -#include -#include #include "utils/error_injection.hh" #include "utils/histogram_metrics_helper.hh" #include "mutation/mutation_source_metadata.hh" @@ -202,7 +200,7 @@ table::add_memtables_to_reader_list(std::vector& readers, } auto token_range = range.transform(std::mem_fn(&dht::ring_position::token)); auto cgs = compaction_groups_for_token_range(token_range); - reserve_fn(boost::accumulate(cgs | boost::adaptors::transformed(std::mem_fn(&compaction_group::memtable_count)), uint64_t(0))); + reserve_fn(std::ranges::fold_left(cgs | std::views::transform(std::mem_fn(&compaction_group::memtable_count)), uint64_t(0), std::plus{})); for (auto& cg : cgs) { add_memtables_from_cg(*cg); } @@ -372,12 +370,11 @@ api::timestamp_type compaction_group::min_memtable_timestamp() const { return api::max_timestamp; } - return *boost::range::min_element( + return std::ranges::min( *_memtables - | boost::adaptors::transformed( + | std::views::transform( [](const shared_memtable& m) { return m->get_min_timestamp(); } - ) - ); + )); } api::timestamp_type compaction_group::min_memtable_live_timestamp() const { @@ -385,12 +382,11 @@ api::timestamp_type compaction_group::min_memtable_live_timestamp() const { return api::max_timestamp; } - return *boost::range::min_element( + return std::ranges::min( *_memtables - | boost::adaptors::transformed( + | std::views::transform( [](const shared_memtable& m) { return m->get_min_live_timestamp(); } - ) - ); + )); } api::timestamp_type compaction_group::min_memtable_live_row_marker_timestamp() const { @@ -398,12 +394,11 @@ api::timestamp_type compaction_group::min_memtable_live_row_marker_timestamp() c return api::max_timestamp; } - return *boost::range::min_element( + return std::ranges::min( *_memtables - | boost::adaptors::transformed( + | std::views::transform( [](const shared_memtable& m) { return m->get_min_live_row_marker_timestamp(); } - ) - ); + )); } bool compaction_group::memtable_has_key(const dht::decorated_key& key) const { @@ -1733,11 +1728,11 @@ uint64_t compaction_group::live_disk_space_used() const noexcept { uint64_t storage_group::live_disk_space_used() const noexcept { auto cgs = const_cast(*this).compaction_groups(); - return boost::accumulate(cgs | boost::adaptors::transformed(std::mem_fn(&compaction_group::live_disk_space_used)), uint64_t(0)); + return std::ranges::fold_left(cgs | std::views::transform(std::mem_fn(&compaction_group::live_disk_space_used)), uint64_t(0), std::plus{}); } uint64_t compaction_group::total_disk_space_used() const noexcept { - return live_disk_space_used() + boost::accumulate(_sstables_compacted_but_not_deleted | boost::adaptors::transformed(std::mem_fn(&sstables::sstable::bytes_on_disk)), uint64_t(0)); + return live_disk_space_used() + std::ranges::fold_left(_sstables_compacted_but_not_deleted | std::views::transform(std::mem_fn(&sstables::sstable::bytes_on_disk)), uint64_t(0), std::plus{}); } void table::rebuild_statistics() { @@ -2878,7 +2873,7 @@ size_t compaction_group::memtable_count() const noexcept { size_t storage_group::memtable_count() const noexcept { auto memtable_count = [] (const compaction_group_ptr& cg) { return cg ? cg->memtable_count() : 0; }; return memtable_count(_main_cg) + - boost::accumulate(_split_ready_groups | boost::adaptors::transformed(std::mem_fn(&compaction_group::memtable_count)), size_t(0)); + std::ranges::fold_left(_split_ready_groups | std::views::transform(std::mem_fn(&compaction_group::memtable_count)), size_t(0), std::plus{}); } future<> table::flush(std::optional pos) { diff --git a/service/mapreduce_service.cc b/service/mapreduce_service.cc index 0ae059c3c0dc..e6b13f500320 100644 --- a/service/mapreduce_service.cc +++ b/service/mapreduce_service.cc @@ -173,7 +173,7 @@ static std::vector<::shared_ptr> get_function } } else { auto& info = request.aggregation_infos.value()[i]; - auto types = boost::copy_range>(info.column_names | boost::adaptors::transformed(name_as_type)); + auto types = info.column_names | std::views::transform(name_as_type) | std::ranges::to>(); auto func = cql3::functions::instance().mock_get(info.name, types); if (!func) { @@ -346,7 +346,7 @@ static shared_ptr mock_selection( } auto reducible_aggr = aggr_function->reducible_aggregate_function(); - auto arg_exprs =boost::copy_range>(info->column_names | boost::adaptors::transformed(name_as_expression)); + auto arg_exprs = info->column_names | std::views::transform(name_as_expression) | std::ranges::to>(); auto fc_expr = cql3::expr::function_call{reducible_aggr, arg_exprs}; auto column_identifier = make_shared(info->name.name, false); auto prepared_expr = cql3::expr::prepare_expression(fc_expr, db.as_data_dictionary(), "", schema.get(), nullptr); @@ -506,7 +506,7 @@ future mapreduce_service::execute_on_this_shard( flogger.error("aggregation result column count does not match requested column count"); throw std::runtime_error("aggregation result column count does not match requested column count"); } - query::mapreduce_result res = { .query_results = boost::copy_range>(rows[0] | boost::adaptors::transformed([] (const managed_bytes_opt& x) { return to_bytes_opt(x); })) }; + query::mapreduce_result res = { .query_results = rows[0] | std::views::transform([] (const managed_bytes_opt& x) { return to_bytes_opt(x); }) | std::ranges::to>() }; auto printer = seastar::value_of([&req, &res] { return query::mapreduce_result::printer { diff --git a/service/qos/service_level_controller.cc b/service/qos/service_level_controller.cc index ec3b1c0da7e5..ad712d08346f 100644 --- a/service/qos/service_level_controller.cc +++ b/service/qos/service_level_controller.cc @@ -598,7 +598,7 @@ future<> service_level_controller::migrate_to_v2(size_t nodes_count, db::system_ } - auto col_names = boost::copy_range>(schema->all_columns() | boost::adaptors::transformed([] (const auto& col) {return col.name_as_cql_string(); })); + auto col_names = schema->all_columns() | std::views::transform([] (const auto& col) {return col.name_as_cql_string(); }) | std::ranges::to>(); auto col_names_str = boost::algorithm::join(col_names, ", "); sstring val_binders_str = "?"; for (size_t i = 1; i < col_names.size(); ++i) { diff --git a/service/raft/group0_state_machine_merger.cc b/service/raft/group0_state_machine_merger.cc index f3219ee4636b..37e26f0d807b 100644 --- a/service/raft/group0_state_machine_merger.cc +++ b/service/raft/group0_state_machine_merger.cc @@ -9,8 +9,6 @@ #include "db/system_keyspace.hh" #include "service/raft/group0_state_machine_merger.hh" -#include - namespace service { static logging::logger slogger("group0_raft_sm_merger"); @@ -29,7 +27,7 @@ size_t group0_state_machine_merger::cmd_size(group0_command& cmd) { if (holds_alternative(cmd.change)) { return 0; } - auto r = get_command_mutations(cmd) | boost::adaptors::transformed([] (const canonical_mutation& m) { return m.representation().size(); }); + auto r = get_command_mutations(cmd) | std::views::transform([] (const canonical_mutation& m) { return m.representation().size(); }); return std::accumulate(std::begin(r), std::end(r), size_t(0)); } diff --git a/service/storage_service.cc b/service/storage_service.cc index 14e61f073f2c..372c2755681c 100644 --- a/service/storage_service.cc +++ b/service/storage_service.cc @@ -3000,9 +3000,9 @@ future<> storage_service::join_cluster(sharded& auto seeds = _gossiper.get_seeds(); auto initial_contact_nodes = loaded_endpoints.empty() ? std::unordered_set(seeds.begin(), seeds.end()) : - boost::copy_range>(loaded_endpoints | boost::adaptors::transformed([] (const auto& x) { + loaded_endpoints | std::views::transform([] (const auto& x) { return x.second.endpoint; - })); + }) | std::ranges::to>(); auto loaded_peer_features = co_await _sys_ks.local().load_peer_features(); slogger.info("initial_contact_nodes={}, loaded_endpoints={}, loaded_peer_features={}", initial_contact_nodes, loaded_endpoints | std::views::keys, loaded_peer_features.size()); @@ -3427,7 +3427,7 @@ storage_service::prepare_replacement_info(std::unordered_set } slogger.info("Host {}/{} is replacing {}/{} ignore_nodes={}", get_token_metadata().get_my_id(), get_broadcast_address(), replace_host_id, replace_address, - fmt::join(ri.ignore_nodes | boost::adaptors::transformed ([] (const auto& x) { + fmt::join(ri.ignore_nodes | std::views::transform ([] (const auto& x) { return fmt::format("{}/{}", x.first, x.second.endpoint); }), ",")); co_await _gossiper.reset_endpoint_state_map(); @@ -3955,9 +3955,9 @@ void storage_service::run_replace_ops(std::unordered_set& bootstrap_token auto stop_ctl = deferred_stop(ctl); const auto& uuid = ctl.uuid(); gms::inet_address replace_address = replace_info.address; - ctl.ignore_nodes = boost::copy_range>(replace_info.ignore_nodes | boost::adaptors::transformed([] (const auto& x) { + ctl.ignore_nodes = replace_info.ignore_nodes | std::views::transform([] (const auto& x) { return x.second.endpoint; - })); + }) | std::ranges::to>(); // Step 1: Decide who needs to sync data for replace operation // The replacing node is not a normal token owner yet // Add it back explicitly after checking all other nodes. @@ -3993,9 +3993,9 @@ void storage_service::run_replace_ops(std::unordered_set& bootstrap_token slogger.info("replace[{}]: Using repair based node ops to sync data", uuid); auto ks_erms = _db.local().get_non_local_strategy_keyspaces_erms(); auto tmptr = get_token_metadata_ptr(); - auto ignore_nodes = boost::copy_range>(replace_info.ignore_nodes | boost::adaptors::transformed([] (const auto& x) { + auto ignore_nodes = replace_info.ignore_nodes | std::views::transform([] (const auto& x) { return x.first; - })); + }) | std::ranges::to>(); _repair.local().replace_with_repair(std::move(ks_erms), std::move(tmptr), bootstrap_tokens, std::move(ignore_nodes), replace_info.host_id).get(); } else { slogger.info("replace[{}]: Using streaming based node ops to sync data", uuid); @@ -5678,9 +5678,9 @@ future storage_service::raft_topology_cmd_handler(raft on_internal_error(rtlogger, ::format("Cannot find request_param for node id {}", id)); } if (is_repair_based_node_ops_enabled(streaming::stream_reason::replace)) { - auto ignored_nodes = boost::copy_range>(_topology_state_machine._topology.ignored_nodes | boost::adaptors::transformed([] (const auto& id) { + auto ignored_nodes = _topology_state_machine._topology.ignored_nodes | std::views::transform([] (const auto& id) { return locator::host_id(id.uuid()); - })); + }) | std::ranges::to>(); auto ks_erms = _db.local().get_non_local_strategy_keyspaces_erms(); auto tmptr = get_token_metadata_ptr(); auto replaced_node = locator::host_id(replaced_id.uuid()); @@ -7465,10 +7465,9 @@ bool storage_service::is_normal_state_handled_on_boot(gms::inet_address node) { future<> storage_service::wait_for_normal_state_handled_on_boot() { static logger::rate_limit rate_limit{std::chrono::seconds{5}}; static auto fmt_nodes_with_statuses = [this] (const auto& eps) { - return boost::algorithm::join( - eps | boost::adaptors::transformed([this] (const auto& ep) { + return eps | std::views::transform([this] (const auto& ep) { return ::format("({}, status={})", ep, _gossiper.get_gossip_status(ep)); - }), ", "); + }) | std::views::join_with(','); }; slogger.info("Started waiting for normal state handlers to finish"); @@ -7487,13 +7486,13 @@ future<> storage_service::wait_for_normal_state_handled_on_boot() { if (std::chrono::steady_clock::now() > start_time + std::chrono::seconds(60)) { auto err = ::format("Timed out waiting for normal state handlers to finish for nodes {}", - fmt_nodes_with_statuses(boost::make_iterator_range(it, eps.end()))); + fmt_nodes_with_statuses(std::ranges::subrange(it, eps.end()))); slogger.error("{}", err); throw std::runtime_error{std::move(err)}; } slogger.log(log_level::info, rate_limit, "Normal state handlers not yet finished for nodes {}", - fmt_nodes_with_statuses(boost::make_iterator_range(it, eps.end()))); + fmt_nodes_with_statuses(std::ranges::subrange(it, eps.end()))); co_await sleep_abortable(std::chrono::milliseconds{100}, _abort_source); } diff --git a/service/topology_mutation.cc b/service/topology_mutation.cc index b16e33abd5d3..ab7ab10182f4 100644 --- a/service/topology_mutation.cc +++ b/service/topology_mutation.cc @@ -14,8 +14,6 @@ #include "types/set.hh" #include "types/map.hh" -#include - namespace db { extern thread_local data_type cdc_generation_ts_id_type; } @@ -146,15 +144,15 @@ const schema& topology_node_mutation_builder::schema() const { } topology_node_mutation_builder& topology_node_mutation_builder::set(const char* cell, const std::unordered_set& nodes_ids) { - return apply_set(cell, collection_apply_mode::overwrite, nodes_ids | boost::adaptors::transformed([] (const auto& node_id) { return node_id.id; })); + return apply_set(cell, collection_apply_mode::overwrite, nodes_ids | std::views::transform([] (const auto& node_id) { return node_id.id; })); } topology_node_mutation_builder& topology_node_mutation_builder::set(const char* cell, const std::unordered_set& tokens) { - return apply_set(cell, collection_apply_mode::overwrite, tokens | boost::adaptors::transformed([] (const auto& t) { return t.to_sstring(); })); + return apply_set(cell, collection_apply_mode::overwrite, tokens | std::views::transform([] (const auto& t) { return t.to_sstring(); })); } topology_node_mutation_builder& topology_node_mutation_builder::set(const char* cell, const std::set& features) { - return apply_set(cell, collection_apply_mode::overwrite, features | boost::adaptors::transformed([] (const auto& f) { return sstring(f); })); + return apply_set(cell, collection_apply_mode::overwrite, features | std::views::transform([] (const auto& f) { return sstring(f); })); } canonical_mutation topology_node_mutation_builder::build() { @@ -211,7 +209,7 @@ topology_mutation_builder& topology_mutation_builder::set_new_cdc_generation_dat } topology_mutation_builder& topology_mutation_builder::set_committed_cdc_generations(const std::vector& values) { - auto dv = values | boost::adaptors::transformed([&] (const auto& v) { + auto dv = values | std::views::transform([&] (const auto& v) { return make_tuple_value(db::cdc_generation_ts_id_type, tuple_type_impl::native_type({v.ts, timeuuid_native_type{v.id}})); }); return apply_set("committed_cdc_generations", collection_apply_mode::overwrite, std::move(dv)); @@ -227,7 +225,7 @@ topology_mutation_builder& topology_mutation_builder::set_new_keyspace_rf_change } topology_mutation_builder& topology_mutation_builder::set_unpublished_cdc_generations(const std::vector& values) { - auto dv = values | boost::adaptors::transformed([&] (const auto& v) { + auto dv = values | std::views::transform([&] (const auto& v) { return make_tuple_value(db::cdc_generation_ts_id_type, tuple_type_impl::native_type({v.ts, timeuuid_native_type{v.id}})); }); return apply_set("unpublished_cdc_generations", collection_apply_mode::overwrite, std::move(dv)); @@ -246,7 +244,7 @@ topology_mutation_builder& topology_mutation_builder::set_upgrade_state(topology } topology_mutation_builder& topology_mutation_builder::add_enabled_features(const std::set& features) { - return apply_set("enabled_features", collection_apply_mode::update, features | boost::adaptors::transformed([] (const auto& f) { return sstring(f); })); + return apply_set("enabled_features", collection_apply_mode::update, features | std::views::transform([] (const auto& f) { return sstring(f); })); } topology_mutation_builder& topology_mutation_builder::add_new_committed_cdc_generation(const cdc::generation_id_v2& value) { @@ -257,11 +255,11 @@ topology_mutation_builder& topology_mutation_builder::add_new_committed_cdc_gene } topology_mutation_builder& topology_mutation_builder::add_ignored_nodes(const std::unordered_set& value) { - return apply_set("ignore_nodes", collection_apply_mode::update, value | boost::adaptors::transformed([] (const auto& id) { return id.uuid(); })); + return apply_set("ignore_nodes", collection_apply_mode::update, value | std::views::transform([] (const auto& id) { return id.uuid(); })); } topology_mutation_builder& topology_mutation_builder::set_ignored_nodes(const std::unordered_set& value) { - return apply_set("ignore_nodes", collection_apply_mode::overwrite, value | boost::adaptors::transformed([] (const auto& id) { return id.uuid(); })); + return apply_set("ignore_nodes", collection_apply_mode::overwrite, value | std::views::transform([] (const auto& id) { return id.uuid(); })); } topology_mutation_builder& topology_mutation_builder::del_global_topology_request() { diff --git a/sstables/mx/parsers.hh b/sstables/mx/parsers.hh index 0c3111be4319..d6e74b1e6cbb 100644 --- a/sstables/mx/parsers.hh +++ b/sstables/mx/parsers.hh @@ -15,8 +15,6 @@ #include "sstables/mx/types.hh" #include "mutation/position_in_partition.hh" -#include - namespace sstables { namespace mc { @@ -82,7 +80,7 @@ class clustering_parser { } position_in_partition make_position() { - auto key = clustering_key_prefix::from_range(clustering_key_values | boost::adaptors::transformed( + auto key = clustering_key_prefix::from_range(clustering_key_values | std::views::transform( [] (const FragmentedBuffer & b) { return typename FragmentedBuffer::view(b); })); if (kind == bound_kind_m::clustering) { diff --git a/sstables/mx/reader.cc b/sstables/mx/reader.cc index adc3c1f5dfdb..f1476db94c02 100644 --- a/sstables/mx/reader.cc +++ b/sstables/mx/reader.cc @@ -304,7 +304,7 @@ class mp_row_consumer_m { } row_processing_result consume_row_start(const std::vector& ecp) { - auto key = clustering_key_prefix::from_range(ecp | boost::adaptors::transformed( + auto key = clustering_key_prefix::from_range(ecp | std::views::transform( [] (const fragmented_temporary_buffer& b) { return fragmented_temporary_buffer::view(b); })); _sst->get_stats().on_row_read(); @@ -477,7 +477,7 @@ class mp_row_consumer_m { data_consumer::proceed consume_range_tombstone(const std::vector& ecp, bound_kind kind, tombstone tomb) { - auto ck = clustering_key_prefix::from_range(ecp | boost::adaptors::transformed( + auto ck = clustering_key_prefix::from_range(ecp | std::views::transform( [] (const fragmented_temporary_buffer& b) { return fragmented_temporary_buffer::view(b); })); if (kind == bound_kind::incl_start || kind == bound_kind::excl_start) { return consume_range_tombstone_start(std::move(ck), kind, std::move(tomb)); @@ -490,7 +490,7 @@ class mp_row_consumer_m { sstables::bound_kind_m kind, tombstone end_tombstone, tombstone start_tombstone) { - auto ck = clustering_key_prefix::from_range(ecp | boost::adaptors::transformed( + auto ck = clustering_key_prefix::from_range(ecp | std::views::transform( [] (const fragmented_temporary_buffer& b) { return fragmented_temporary_buffer::view(b); })); switch (kind) { case bound_kind_m::incl_end_excl_start: { @@ -1913,7 +1913,7 @@ class validating_consumer { private: clustering_key from_fragmented_buffer(const std::vector& ecp) { - return clustering_key_prefix::from_range(ecp | boost::adaptors::transformed( + return clustering_key_prefix::from_range(ecp | std::views::transform( [] (const fragmented_temporary_buffer& b) { return fragmented_temporary_buffer::view(b); })); } void validate_fragment_order(mutation_fragment_v2::kind kind, std::optional new_current_tombstone) { diff --git a/sstables/sstable_set.cc b/sstables/sstable_set.cc index bd95ce5f91a1..bfec67f52945 100644 --- a/sstables/sstable_set.cc +++ b/sstables/sstable_set.cc @@ -92,11 +92,11 @@ void sstable_run::erase(shared_sstable sst) { } uint64_t sstable_run::data_size() const { - return boost::accumulate(_all | boost::adaptors::transformed(std::mem_fn(&sstable::data_size)), uint64_t(0)); + return std::ranges::fold_left(_all | std::views::transform(std::mem_fn(&sstable::data_size)), uint64_t(0), std::plus{}); } double sstable_run::estimate_droppable_tombstone_ratio(const gc_clock::time_point& compaction_time, const tombstone_gc_state& gc_state, const schema_ptr& s) const { - auto estimate_sum = boost::accumulate(_all | boost::adaptors::transformed(std::bind(&sstable::estimate_droppable_tombstone_ratio, std::placeholders::_1, compaction_time, gc_state, s)), double(0)); + auto estimate_sum = std::ranges::fold_left(_all | std::views::transform(std::bind(&sstable::estimate_droppable_tombstone_ratio, std::placeholders::_1, compaction_time, gc_state, s)), double(0), std::plus{}); return _all.size() ? estimate_sum / _all.size() : double(0); } @@ -303,9 +303,9 @@ partitioned_sstable_set::partitioned_sstable_set(schema_ptr schema, bool use_lev } static std::unordered_map clone_runs(const std::unordered_map& runs) { - return boost::copy_range>(runs | boost::adaptors::transformed([] (auto& p) { + return runs | std::views::transform([] (auto& p) { return std::make_pair(p.first, make_lw_shared(*p.second)); - })); + }) | std::ranges::to>(); } partitioned_sstable_set::partitioned_sstable_set(schema_ptr schema, const std::vector& unleveled_sstables, const interval_map_type& leveled_sstables, @@ -951,13 +951,12 @@ sstable_set_impl::create_single_key_sstable_reader( if (!num_sstables) { return make_empty_flat_reader_v2(schema, permit); } - auto readers = boost::copy_range>( - filter_sstable_for_reader_by_ck(std::move(selected_sstables), *cf, schema, slice) - | boost::adaptors::transformed([&] (const shared_sstable& sstable) { + auto readers = filter_sstable_for_reader_by_ck(std::move(selected_sstables), *cf, schema, slice) + | std::views::transform([&] (const shared_sstable& sstable) { tracing::trace(trace_state, "Reading key {} from sstable {}", pos, seastar::value_of([&sstable] { return sstable->get_filename(); })); return sstable->make_reader(schema, permit, pr, slice, trace_state, fwd); - }) - ); + }) + | std::ranges::to>(); // If filter_sstable_for_reader_by_ck filtered any sstable that contains the partition // we want to emit partition_start/end if no rows were found, @@ -1150,12 +1149,12 @@ bool compound_sstable_set::erase(shared_sstable sst) { size_t compound_sstable_set::size() const noexcept { - return boost::accumulate(_sets | boost::adaptors::transformed(std::mem_fn(&sstable_set::size)), size_t(0)); + return std::ranges::fold_left(_sets | std::views::transform(std::mem_fn(&sstable_set::size)), size_t(0), std::plus{}); } uint64_t compound_sstable_set::bytes_on_disk() const noexcept { - return boost::accumulate(_sets | boost::adaptors::transformed(std::mem_fn(&sstable_set::bytes_on_disk)), uint64_t(0)); + return std::ranges::fold_left(_sets | std::views::transform(std::mem_fn(&sstable_set::bytes_on_disk)), uint64_t(0), std::plus{}); } class compound_sstable_set::incremental_selector : public incremental_selector_impl { @@ -1164,9 +1163,9 @@ class compound_sstable_set::incremental_selector : public incremental_selector_i std::vector _selectors; private: std::vector make_selectors(const std::vector>& sets) { - return boost::copy_range>(_sets | boost::adaptors::transformed([] (const auto& set) { + return _sets | std::views::transform([] (const auto& set) { return set->make_incremental_selector(); - })); + }) | std::ranges::to>(); } public: incremental_selector(const schema& schema, const std::vector>& sets) @@ -1246,12 +1245,11 @@ compound_sstable_set::create_single_key_sstable_reader( return non_empty_set->create_single_key_sstable_reader(cf, std::move(schema), std::move(permit), sstable_histogram, pr, slice, trace_state, fwd, fwd_mr, predicate); } - auto readers = boost::copy_range>( - boost::make_iterator_range(sets.begin(), it) - | boost::adaptors::transformed([&] (const lw_shared_ptr& non_empty_set) { + auto readers = std::ranges::subrange(sets.begin(), it) + | std::views::transform([&] (const lw_shared_ptr& non_empty_set) { return non_empty_set->create_single_key_sstable_reader(cf, schema, permit, sstable_histogram, pr, slice, trace_state, fwd, fwd_mr, predicate); - }) - ); + }) + | std::ranges::to>(); return make_combined_reader(std::move(schema), std::move(permit), std::move(readers), fwd, fwd_mr); } diff --git a/sstables/sstables.cc b/sstables/sstables.cc index c411ee8dff5f..87db548fc6dc 100644 --- a/sstables/sstables.cc +++ b/sstables/sstables.cc @@ -52,9 +52,6 @@ #include "index_reader.hh" #include "downsampling.hh" #include -#include -#include -#include #include #include #include "mutation/range_tombstone_list.hh" @@ -732,7 +729,7 @@ future<> parse(const schema& s, sstable_version_types v, random_access_reader& i auto transform = [] (auto element) -> std::pair { return { element.key, element.value }; }; - boost::copy(a.elements | boost::adaptors::transformed(transform), std::inserter(sh.bin, sh.bin.end())); + std::ranges::copy(a.elements | std::views::transform(transform), std::inserter(sh.bin, sh.bin.end())); } void write(sstable_version_types v, file_writer& out, const utils::streaming_histogram& sh) { @@ -740,8 +737,9 @@ void write(sstable_version_types v, file_writer& out, const utils::streaming_his check_truncate_and_assign(max_bin_size, sh.max_bin_size); disk_array a; - a.elements = boost::copy_range>(sh.bin - | boost::adaptors::transformed([&] (auto& kv) { return streaming_histogram_element{kv.first, kv.second}; })); + a.elements = sh.bin + | std::views::transform([&] (auto& kv) { return streaming_histogram_element{kv.first, kv.second}; }) + | std::ranges::to>(); write(v, out, max_bin_size, a); } @@ -3022,9 +3020,9 @@ sstable::compute_shards_for_this_sstable(const dht::sharder& sharder_) const { (dtr.left.exclusive ? dht::ring_position::ending_at : dht::ring_position::starting_at)(std::move(t1)), (dtr.right.exclusive ? dht::ring_position::starting_at : dht::ring_position::ending_at)(std::move(t2))); }; - token_ranges = boost::copy_range( - sm->token_ranges.elements - | boost::adaptors::transformed(disk_token_range_to_ring_position_range)); + token_ranges = sm->token_ranges.elements + | std::views::transform(disk_token_range_to_ring_position_range) + | std::ranges::to(); } sstlog.trace("{}: token_ranges={}", get_filename(), token_ranges); auto sharder = dht::ring_position_range_vector_sharder(sharder_, std::move(token_ranges)); diff --git a/sstables/writer.hh b/sstables/writer.hh index 09b46053de2e..404820490768 100644 --- a/sstables/writer.hh +++ b/sstables/writer.hh @@ -10,7 +10,6 @@ #include #include -#include #include "sstables/types.hh" #include "checksum_utils.hh" #include "vint-serialization.hh" @@ -46,7 +45,7 @@ public: return make_ready_future<>(); } virtual future<> put(std::vector> data) override { - _size += boost::accumulate(data | boost::adaptors::transformed(std::mem_fn(&temporary_buffer::size)), 0); + _size += std::ranges::fold_left(data | std::views::transform(std::mem_fn(&temporary_buffer::size)), 0, std::plus{}); return make_ready_future<>(); } virtual future<> put(temporary_buffer buf) override { diff --git a/sstables_loader.cc b/sstables_loader.cc index 5024d65be8b8..79fefc70706e 100644 --- a/sstables_loader.cc +++ b/sstables_loader.cc @@ -278,7 +278,7 @@ future<> sstable_streamer::stream_sstables(const dht::partition_range& pr, std:: auto ops_uuid = streaming::plan_id{utils::make_random_uuid()}; llog.info("load_and_stream: started ops_uuid={}, process [{}-{}] out of {} sstables=[{}]", ops_uuid, nr_sst_current, nr_sst_current + sst_processed.size(), nr_sst_total, - fmt::join(sst_processed | boost::adaptors::transformed([] (auto sst) { return sst->get_filename(); }), ", ")); + fmt::join(sst_processed | std::views::transform([] (auto sst) { return sst->get_filename(); }), ", ")); nr_sst_current += sst_processed.size(); co_await stream_sstable_mutations(ops_uuid, pr, std::move(sst_processed)); if (on_streamed) { diff --git a/tasks/task_handler.cc b/tasks/task_handler.cc index 364119158422..405c25672d09 100644 --- a/tasks/task_handler.cc +++ b/tasks/task_handler.cc @@ -186,12 +186,12 @@ future> task_handler::get_status_recursively( .entity = task.task_status.entity, .progress_units = task.task_status.progress_units, .progress = task.task_progress, - .children = boost::copy_range>(task.failed_children | boost::adaptors::transformed([&tm = _tm] (auto& child) { + .children = task.failed_children | std::views::transform([&tm = _tm] (auto& child) { return task_identity{ .node = tm.get_broadcast_address(), .task_id = child.task_status.id }; - })) + }) | std::ranges::to>() }; res.push_back(status); diff --git a/tasks/task_manager.cc b/tasks/task_manager.cc index 703300accdc4..4e9b4913cc34 100644 --- a/tasks/task_manager.cc +++ b/tasks/task_manager.cc @@ -405,23 +405,23 @@ future> task_manager::virtual_task::impl::get_childre auto ms = module->get_task_manager()._messaging; if (!ms) { auto ids = co_await module->get_task_manager().get_virtual_task_children(parent_id); - co_return boost::copy_range>(ids | boost::adaptors::transformed([&tm = module->get_task_manager()] (auto id) { + co_return ids | std::views::transform([&tm = module->get_task_manager()] (auto id) { return task_identity{ .node = tm.get_broadcast_address(), .task_id = id }; - })); + }) | std::ranges::to>(); } auto nodes = module->get_nodes(); co_return co_await map_reduce(nodes, [ms, parent_id] (auto addr) -> future> { return ms->send_tasks_get_children(netw::msg_addr{addr}, parent_id).then([addr] (auto resp) { - return boost::copy_range>(resp | boost::adaptors::transformed([addr] (auto id) { + return resp | std::views::transform([addr] (auto id) { return task_identity{ .node = addr, .task_id = id }; - })); + }) | std::ranges::to>(); }); }, std::vector{}, concat); } diff --git a/test/boost/cdc_test.cc b/test/boost/cdc_test.cc index dd5176289737..ca2f2207b34b 100644 --- a/test/boost/cdc_test.cc +++ b/test/boost/cdc_test.cc @@ -391,7 +391,7 @@ static std::vector> to_bytes(const cql_transport::message auto rs = rows.rs().result_set().rows(); std::vector> results; for (auto it = rs.begin(); it != rs.end(); ++it) { - results.push_back(boost::copy_range>(*it | boost::adaptors::transformed([] (const managed_bytes_opt& x) { return to_bytes_opt(x); }))); + results.push_back(*it | std::views::transform([] (const managed_bytes_opt& x) { return to_bytes_opt(x); }) | std::ranges::to>()); } return results; } diff --git a/test/boost/compaction_group_test.cc b/test/boost/compaction_group_test.cc index ebe1980d4d63..01460b57ac2f 100644 --- a/test/boost/compaction_group_test.cc +++ b/test/boost/compaction_group_test.cc @@ -155,10 +155,10 @@ SEASTAR_TEST_CASE(basic_compaction_group_splitting_test) { compaction_group->rebuild_main_set(ssts, {}); auto& cm = t->get_compaction_manager(); - auto expected_compaction_size = boost::accumulate(ssts | boost::adaptors::transformed([&] (auto& sst) { + auto expected_compaction_size = std::ranges::fold_left(ssts | std::views::transform([&] (auto& sst) { // sstables that doesn't need split will have compaction bypassed. return sstable_needs_split(sst) ? sst->bytes_on_disk() : size_t(0); - }), int64_t(0)); + }), int64_t(0), std::plus{}); auto ret = cm.perform_split_compaction(*compaction_group, sstables::compaction_type_options::split{classifier}, tasks::task_info{}).get(); BOOST_REQUIRE_EQUAL(ret->start_size, expected_compaction_size); diff --git a/test/boost/compound_test.cc b/test/boost/compound_test.cc index 2f2be0f51687..5caa97c969f1 100644 --- a/test/boost/compound_test.cc +++ b/test/boost/compound_test.cc @@ -18,8 +18,6 @@ #include "schema/schema_builder.hh" #include "dht/murmur3_partitioner.hh" -#include - static std::vector to_bytes_vec(std::vector values) { std::vector result; for (auto&& v : values) { @@ -328,9 +326,9 @@ SEASTAR_THREAD_TEST_CASE(test_composite_from_exploded) { SEASTAR_THREAD_TEST_CASE(test_composite_view_explode) { auto to_owning_vector = [] (std::vector bvs) { - return boost::copy_range>(bvs | boost::adaptors::transformed([] (auto bv) { + return bvs | std::views::transform([] (auto bv) { return bytes(bv.begin(), bv.end()); - })); + }) | std::ranges::to>(); }; { BOOST_REQUIRE_EQUAL(to_owning_vector(composite_view(composite(bytes({'\x00', '\x03', 'e', 'l', '1', '\x00'}))).explode()), diff --git a/test/boost/counter_test.cc b/test/boost/counter_test.cc index 5c8168b7d5a3..ead470adf029 100644 --- a/test/boost/counter_test.cc +++ b/test/boost/counter_test.cc @@ -14,8 +14,6 @@ #include #include -#include - #include "test/lib/scylla_test_case.hh" #include "test/lib/test_utils.hh" #include "schema/schema_builder.hh" @@ -545,11 +543,9 @@ SEASTAR_TEST_CASE(test_counter_id_ordering) { "ffeeddcc-aa99-8878-6655-443322110000", }; - auto counter_ids = boost::copy_range>( - ids | boost::adaptors::transformed([] (auto id) { + auto counter_ids = ids | std::views::transform([] (auto id) { return counter_id(utils::UUID(id)); - }) - ); + }) | std::ranges::to>(); for (auto it = counter_ids.begin(); it != counter_ids.end(); ++it) { BOOST_REQUIRE_EQUAL(*it, *it); diff --git a/test/boost/cql_query_test.cc b/test/boost/cql_query_test.cc index 577882223011..86bd25dcf421 100644 --- a/test/boost/cql_query_test.cc +++ b/test/boost/cql_query_test.cc @@ -2965,16 +2965,16 @@ SEASTAR_TEST_CASE(test_reversed_slice_with_many_clustering_ranges) { // Many singular ranges - to check that the right range is used for // determining the disk read-range upper bound. { - const auto select_query = format( + const auto select_query = fmt::format( "SELECT * FROM test WHERE pk = {} and ck IN ({}) ORDER BY ck DESC BYPASS CACHE;", pk, - boost::algorithm::join(selected_cks | boost::adaptors::transformed([] (int ck) { return format("{}", ck); }), ", ")); + fmt::join(selected_cks | std::views::transform([] (int ck) { return format("{}", ck); }), ", ")); assert_that(e.execute_cql(select_query).get()) .is_rows() - .with_rows(boost::copy_range>>( - selected_cks - | boost::adaptors::reversed - | boost::adaptors::transformed(make_expected_row))); + .with_rows(selected_cks + | std::views::reverse + | std::views::transform(make_expected_row) + | std::ranges::to>>()); } // A single wide range - to check that the right range bound is used for diff --git a/test/boost/mutation_query_test.cc b/test/boost/mutation_query_test.cc index 2e1970515547..9f4c1f70fb1b 100644 --- a/test/boost/mutation_query_test.cc +++ b/test/boost/mutation_query_test.cc @@ -10,10 +10,6 @@ #include "utils/assert.hh" #include -#include -#include -#include - #include #include "query-result-set.hh" #include "query-result-writer.hh" diff --git a/test/boost/mutation_reader_another_test.cc b/test/boost/mutation_reader_another_test.cc index a2c9edeb654b..8bffc422d164 100644 --- a/test/boost/mutation_reader_another_test.cc +++ b/test/boost/mutation_reader_another_test.cc @@ -667,7 +667,7 @@ SEASTAR_THREAD_TEST_CASE(test_make_forwardable) { return s.make_row(permit, s.make_ckey(n), "value"); }) | std::ranges::to>(); - auto ms = keys | boost::adaptors::transformed([&](auto &key) { + auto ms = keys | std::views::transform([&](auto &key) { auto m = mutation(s.schema(), key); for (auto &mf : crs) { m.apply(mf); diff --git a/test/boost/mutation_test.cc b/test/boost/mutation_test.cc index bb0d0e914186..7a2b40997b20 100644 --- a/test/boost/mutation_test.cc +++ b/test/boost/mutation_test.cc @@ -8,8 +8,6 @@ #include -#include -#include #include #include #include "compaction/compaction_garbage_collector.hh" @@ -3761,7 +3759,7 @@ SEASTAR_THREAD_TEST_CASE(test_compactor_validator) { auto msg = fmt::format("expected_is_valid ({}) != is_valid ({}), fragments:\n{}", expected_is_valid, is_valid, - fmt::join(frag_refs | boost::adaptors::transformed([&] (std::reference_wrapper mf) { + fmt::join(frag_refs | std::views::transform([&] (std::reference_wrapper mf) { return fmt::format("{}", mutation_fragment_v2::printer(*s, mf.get())); }), "\n")); BOOST_FAIL(msg); diff --git a/test/boost/mvcc_test.cc b/test/boost/mvcc_test.cc index df6327f820b5..d1f094e289b8 100644 --- a/test/boost/mvcc_test.cc +++ b/test/boost/mvcc_test.cc @@ -8,8 +8,6 @@ #include "utils/assert.hh" -#include -#include #include #include #include diff --git a/test/boost/network_topology_strategy_test.cc b/test/boost/network_topology_strategy_test.cc index 398a1e6be79c..bdc8ffcd80a5 100644 --- a/test/boost/network_topology_strategy_test.cc +++ b/test/boost/network_topology_strategy_test.cc @@ -559,7 +559,7 @@ static void test_random_balancing(sharded& snitch, gms::inet_address } testlog.debug("num_dcs={} num_racks={} nodes_per_rack={} shards_per_node={} ring_points=[{}]", num_dcs, num_racks, nodes_per_rack, shard_count, - fmt::join(ring_points | boost::adaptors::transformed([] (const ring_point& rp) { + fmt::join(ring_points | std::views::transform([] (const ring_point& rp) { return fmt::format("({}, {})", rp.id, rp.host); }), ", ")); @@ -815,12 +815,12 @@ static void test_equivalence(const shared_token_metadata& stm, const locator::to }; my_network_topology_strategy nts(replication_strategy_params( - boost::copy_range>( - datacenters - | boost::adaptors::transformed( + datacenters | std::views::transform( [](const std::pair& p) { return std::make_pair(p.first, to_sstring(p.second)); - })), std::nullopt)); + }) + | std::ranges::to>(), + std::nullopt)); const token_metadata& tm = *stm.get(); for (size_t i = 0; i < 1000; ++i) { diff --git a/test/boost/partitioner_test.cc b/test/boost/partitioner_test.cc index 5a6a2c23a533..9b4419223d0e 100644 --- a/test/boost/partitioner_test.cc +++ b/test/boost/partitioner_test.cc @@ -307,8 +307,7 @@ void test_sharding(const dht::sharder& sharder, unsigned shards, std::vector v) { - return boost::copy_range>( - v | boost::adaptors::transformed(token_from_long)); + return v | std::views::transform(token_from_long) | std::ranges::to>(); }; dht::static_sharder mm3p7s(7); auto mm3p7s_shard_limits = make_token_vector({ @@ -331,8 +330,7 @@ SEASTAR_THREAD_TEST_CASE(test_murmur3_sharding) { SEASTAR_THREAD_TEST_CASE(test_murmur3_sharding_with_ignorebits) { auto make_token_vector = [] (std::vector v) { - return boost::copy_range>( - v | boost::adaptors::transformed(token_from_long)); + return v | std::views::transform(token_from_long) | std::ranges::to>(); }; dht::static_sharder mm3p7s2i(7, 2); auto mm3p7s2i_shard_limits = make_token_vector({ diff --git a/test/boost/restrictions_test.cc b/test/boost/restrictions_test.cc index 60574682f9b7..a632f20746de 100644 --- a/test/boost/restrictions_test.cc +++ b/test/boost/restrictions_test.cc @@ -24,7 +24,6 @@ namespace { using seastar::compat::source_location; -using boost::adaptors::transformed; std::unique_ptr to_options( const cql3::cql_config& cfg, @@ -45,7 +44,7 @@ void require_rows(cql_test_env& e, const std::vector>& expected, const seastar::compat::source_location& loc = source_location::current()) { // This helps compiler pick the right overload for make_value: - const auto rvals = values | transformed([] (const bytes_opt& v) { return cql3::raw_value::make_value(v); }); + const auto rvals = values | std::views::transform([] (const bytes_opt& v) { return cql3::raw_value::make_value(v); }); cql3::cql_config cfg(cql3::cql_config::default_tag{}); auto opts = to_options(cfg, std::move(names), std::vector(rvals.begin(), rvals.end())); try { @@ -746,7 +745,7 @@ SEASTAR_THREAD_TEST_CASE(multi_col_in) { auto bound_tuples = [] (std::vector> tuples) { const auto tuple_type = tuple_type_impl::get_instance({int32_type, float_type}); const auto list_type = list_type_impl::get_instance(tuple_type, true); - const auto tvals = tuples | transformed([&] (const std::tuple& t) { + const auto tvals = tuples | std::views::transform([&] (const std::tuple& t) { return make_tuple_value(tuple_type, tuple_type_impl::native_type({std::get<0>(t), std::get<1>(t)})); }); return list_type->decompose( diff --git a/test/boost/sstable_compaction_test.cc b/test/boost/sstable_compaction_test.cc index c152e97ed7c5..16cf635e5a25 100644 --- a/test/boost/sstable_compaction_test.cc +++ b/test/boost/sstable_compaction_test.cc @@ -429,8 +429,9 @@ static future<> check_compacted_sstables(test_env& env, compact_sstables_result std::sort(keys.begin(), keys.end(), partition_key::less_compare(*s)); BOOST_REQUIRE_EQUAL(keys.size(), res.input_sstables.size()); - auto generations = boost::copy_range>(res.input_sstables | - boost::adaptors::transformed([] (const sstables::shared_sstable& sst) { return sst->generation(); })); + auto generations = res.input_sstables + | std::views::transform([] (const sstables::shared_sstable& sst) { return sst->generation(); }) + | std::ranges::to>(); for (auto& k : keys) { bool found = false; for (auto it = generations.begin(); it != generations.end(); ++it) { @@ -3452,8 +3453,9 @@ SEASTAR_TEST_CASE(sstable_run_based_compaction_test) { }); BOOST_REQUIRE_EQUAL(desc.sstables.size(), expected_input); - auto sstable_run = boost::copy_range>(desc.sstables - | boost::adaptors::transformed([] (auto& sst) { return sst->generation(); })); + auto sstable_run = desc.sstables + | std::views::transform([] (auto& sst) { return sst->generation(); }) + | std::ranges::to>(); auto expected_sst = sstable_run.begin(); auto closed_sstables_tracker = sstable_run.begin(); auto replacer = [&] (sstables::compaction_completion_desc desc) { @@ -4588,7 +4590,7 @@ SEASTAR_TEST_CASE(twcs_reshape_with_disjoint_set_test) { } auto job_size = [] (auto&& sst_range) { - return boost::accumulate(sst_range | boost::adaptors::transformed(std::mem_fn(&sstable::bytes_on_disk)), uint64_t(0)); + return std::ranges::fold_left(sst_range | std::views::transform(std::mem_fn(&sstable::bytes_on_disk)), uint64_t(0), std::plus{}); }; auto free_space_for_reshaping_sstables = [&job_size] (auto&& sst_range) { return job_size(std::move(sst_range)) * (time_window_compaction_strategy::reshape_target_space_overhead * 100); @@ -5264,7 +5266,7 @@ SEASTAR_TEST_CASE(test_compaction_strategy_cleanup_method) { auto [candidates, descriptors] = get_cleanup_jobs(compaction_strategy_type, std::forward(args)...); testlog.info("get_cleanup_jobs() returned {} descriptors; expected={}", descriptors.size(), target_job_count); BOOST_REQUIRE(descriptors.size() == target_job_count); - auto generations = boost::copy_range>(candidates | boost::adaptors::transformed(std::mem_fn(&sstables::sstable::generation))); + auto generations = candidates | std::views::transform(std::mem_fn(&sstables::sstable::generation)) | std::ranges::to>(); auto check_desc = [&] (const auto& desc) { BOOST_REQUIRE(desc.sstables.size() == per_job_files); for (auto& sst: desc.sstables) { diff --git a/test/boost/sstable_datafile_test.cc b/test/boost/sstable_datafile_test.cc index 5dffe8f71888..68541e2f225e 100644 --- a/test/boost/sstable_datafile_test.cc +++ b/test/boost/sstable_datafile_test.cc @@ -2038,8 +2038,9 @@ SEASTAR_TEST_CASE(sstable_owner_shards) { auto mut = [&] (auto shard) { return make_insert(tests::generate_partition_key(key_schema, shard)); }; - auto muts = boost::copy_range>(shards - | boost::adaptors::transformed([&] (auto shard) { return mut(shard); })); + auto muts = shards + | std::views::transform([&] (auto shard) { return mut(shard); }) + | std::ranges::to>(); auto sst_gen = [&] () mutable { auto schema = schema_builder(s).with_sharder(1, ignore_msb).build(); auto sst = env.make_sstable(std::move(schema)); @@ -2718,7 +2719,7 @@ SEASTAR_TEST_CASE(compound_sstable_set_basic_test) { set2->insert(sstable_for_overlapping_test(env, s, keys[0].key(), keys[1].key(), 0)); set2->insert(sstable_for_overlapping_test(env, s, keys[0].key(), keys[1].key(), 0)); - BOOST_REQUIRE(boost::accumulate(*compound->all() | boost::adaptors::transformed([] (const sstables::shared_sstable& sst) { return sst->generation().as_int(); }), unsigned(0)) == 6); + BOOST_REQUIRE(std::ranges::fold_left(*compound->all() | std::views::transform([] (const sstables::shared_sstable& sst) { return sst->generation().as_int(); }), unsigned(0), std::plus{}) == 6); { unsigned found = 0; for (auto sstables = compound->all(); [[maybe_unused]] auto& sst : *sstables) { diff --git a/test/boost/sstable_test.cc b/test/boost/sstable_test.cc index b540099e5468..ba6bcd2c0751 100644 --- a/test/boost/sstable_test.cc +++ b/test/boost/sstable_test.cc @@ -77,8 +77,7 @@ test_using_reusable_sst(schema_ptr s, sstring dir, sstables::generation_type::in future> index_read(schema_ptr schema, sstring path) { return test_using_reusable_sst(std::move(schema), std::move(path), 1, [] (test_env& env, sstable_ptr ptr) { auto indexes = sstables::test(ptr).read_indexes(env.make_reader_permit()).get(); - return boost::copy_range>( - indexes | boost::adaptors::transformed([] (const sstables::test::index_entry& e) { return e.key; })); + return indexes | std::views::transform([] (const sstables::test::index_entry& e) { return e.key; }) | std::ranges::to>(); }); } diff --git a/test/boost/tablets_test.cc b/test/boost/tablets_test.cc index da26a59729ef..c8533ae45151 100644 --- a/test/boost/tablets_test.cc +++ b/test/boost/tablets_test.cc @@ -2824,7 +2824,7 @@ SEASTAR_THREAD_TEST_CASE(test_tablet_range_splitter) { }; check_single(dht::partition_range::make_open_ended_both_sides(), included_ranges); - check(boost::copy_range(included_ranges | boost::adaptors::transformed([&] (auto& r) { return r.range; })), included_ranges); + check(included_ranges | std::views::transform([&] (auto& r) { return r.range; }) | std::ranges::to(), included_ranges); check(excluded_ranges, {}); check_intersection_single({bound{dks[0], true}, bound{dks[1], false}}); diff --git a/test/boost/transport_test.cc b/test/boost/transport_test.cc index f1816461739c..a58b9764fc4a 100644 --- a/test/boost/transport_test.cc +++ b/test/boost/transport_test.cc @@ -17,17 +17,6 @@ #include "test/lib/random_utils.hh" #include "test/lib/test_utils.hh" -namespace boost { - -template -requires fmt::is_formattable>::value -std::ostream& boost_test_print_type(std::ostream& os, const transformed_range& rng) { - fmt::print(os, "{}", rng); - return os; -} - -} - namespace cql3 { bool operator==(const cql3::raw_value_view& a, const cql3::raw_value_view& b) { @@ -126,15 +115,15 @@ SEASTAR_THREAD_TEST_CASE(test_response_request_reader) { cql3::unset_bind_variable_vector unset; req.read_name_and_value_list(version, names, values, unset); BOOST_CHECK(std::none_of(unset.begin(), unset.end(), std::identity())); - BOOST_CHECK_EQUAL(names, names_and_values | boost::adaptors::transformed([] (auto& name_and_value) { + BOOST_CHECK(std::ranges::equal(names, names_and_values | std::views::transform([] (auto& name_and_value) { return std::string_view(name_and_value.first); - })); - BOOST_CHECK_EQUAL(values, names_and_values | boost::adaptors::transformed([] (auto& name_and_value) { + }))); + BOOST_CHECK(std::ranges::equal(values, names_and_values | std::views::transform([] (auto& name_and_value) { if (!name_and_value.second) { return cql3::raw_value_view::make_null(); } return cql3::raw_value_view::make_value(fragmented_temporary_buffer::view(*name_and_value.second)); - })); + }))); auto received_string_list = std::vector(); req.read_string_list(received_string_list); diff --git a/test/lib/cql_assertions.cc b/test/lib/cql_assertions.cc index 71a1e1444855..347e10cf74e1 100644 --- a/test/lib/cql_assertions.cc +++ b/test/lib/cql_assertions.cc @@ -8,7 +8,6 @@ */ #include -#include #include #include #include "test/lib/cql_assertions.hh" @@ -105,7 +104,7 @@ rows_assertions::with_column_types(std::initializer_list column_types rows_assertions rows_assertions::with_row(std::initializer_list values) { const auto& rs = _rows->rs().result_set(); - std::vector expected_row = boost::copy_range>((values | boost::adaptors::transformed(to_managed_bytes_opt))); + std::vector expected_row = values | std::views::transform(to_managed_bytes_opt) | std::ranges::to>(); for (auto&& row : rs.rows()) { if (row == expected_row) { return {*this}; @@ -127,10 +126,10 @@ rows_assertions::with_rows(std::vector> rows) { fail(format("Expected more rows ({:d}), got {:d}", rows.size(), rs.size())); } auto& actual = *actual_i; - auto expected_row = row | boost::adaptors::transformed(to_managed_bytes_opt); - if (!std::equal( - std::begin(expected_row), std::end(expected_row), - std::begin(actual), std::end(actual))) { + auto expected_row = row | std::views::transform(to_managed_bytes_opt); + if (!std::ranges::equal( + expected_row, + actual)) { fail(seastar::format("row {} differs, expected {} got {}", row_nr, row, actual)); } ++actual_i; @@ -149,7 +148,7 @@ rows_assertions::with_rows_ignore_order(std::vector> rows const auto& rs = _rows->rs().result_set(); auto& actual = rs.rows(); for (auto&& expected : rows) { - auto expected_row = expected | boost::adaptors::transformed(to_managed_bytes_opt); + auto expected_row = expected | std::views::transform(to_managed_bytes_opt); auto found = std::find_if(std::begin(actual), std::end(actual), [&] (auto&& row) { return std::equal( std::begin(row), std::end(row), @@ -157,7 +156,7 @@ rows_assertions::with_rows_ignore_order(std::vector> rows }); if (found == std::end(actual)) { fail(seastar::format("row {} not found in result set ({})", expected, - fmt::join(actual | boost::adaptors::transformed([] (auto& r) { return fmt::to_string(r); }), ", "))); + fmt::join(actual | std::views::transform([] (auto& r) { return fmt::to_string(r); }), ", "))); } } if (rs.size() != rows.size()) { diff --git a/test/lib/data_model.cc b/test/lib/data_model.cc index 473920f9a6f6..195e5fc87277 100644 --- a/test/lib/data_model.cc +++ b/test/lib/data_model.cc @@ -12,7 +12,6 @@ #include "test/lib/data_model.hh" #include -#include #include "schema/schema_builder.hh" #include "concrete_types.hh" @@ -276,11 +275,9 @@ schema_ptr table_description::build_schema() const { } std::vector table_description::build_mutations(schema_ptr s) const { - auto ms = boost::copy_range>( - _mutations | boost::adaptors::transformed([&] (const mutation_description& md) { + auto ms = _mutations | std::views::transform([&] (const mutation_description& md) { return md.build(s); - }) - ); + }) | std::ranges::to>(); std::ranges::sort(ms, mutation_decorated_key_less_comparator()); return ms; } diff --git a/test/lib/mutation_source_test.cc b/test/lib/mutation_source_test.cc index aeb29fdb2ca8..d8cc20e31b25 100644 --- a/test/lib/mutation_source_test.cc +++ b/test/lib/mutation_source_test.cc @@ -2100,7 +2100,7 @@ class random_mutation_generator::impl { // The pre-existing assumption here is that the type of all the primary key components is blob. // So we generate partition keys and take the single blob component and save it as a random blob value. auto keys = tests::generate_partition_keys(n_blobs, _schema, _local_shard_only, tests::key_size{_external_blob_size, _external_blob_size}); - _blobs = boost::copy_range>(keys | boost::adaptors::transformed([] (const dht::decorated_key& dk) { return dk.key().explode().front(); })); + _blobs = keys | std::views::transform([] (const dht::decorated_key& dk) { return dk.key().explode().front(); }) | std::ranges::to>(); } void set_key_cardinality(size_t n_keys) { @@ -2558,12 +2558,10 @@ void for_each_schema_change(std::function col_specs; @@ -1050,8 +1050,9 @@ std::vector random_schema::make_pkeys(siz ++i; } - return boost::copy_range>(keys | - boost::adaptors::transformed([] (const dht::decorated_key& dkey) { return dkey.key().explode(); })); + return keys + | std::views::transform([] (const dht::decorated_key& dkey) { return dkey.key().explode(); }) + | std::ranges::to>(); } data_model::mutation_description::key random_schema::make_ckey(uint32_t n) { @@ -1067,8 +1068,9 @@ std::vector random_schema::make_ckeys(siz keys.emplace(clustering_key::from_exploded(make_clustering_key(i, val_gen))); } - return boost::copy_range>(keys | - boost::adaptors::transformed([] (const clustering_key& ckey) { return ckey.explode(); })); + return keys + | std::views::transform([] (const clustering_key& ckey) { return ckey.explode(); }) + | std::ranges::to>(); } data_model::mutation_description random_schema::new_mutation(data_model::mutation_description::key pkey) { diff --git a/test/lib/simple_schema.hh b/test/lib/simple_schema.hh index 85807317b9d3..551cf9d25468 100644 --- a/test/lib/simple_schema.hh +++ b/test/lib/simple_schema.hh @@ -23,8 +23,6 @@ #include "test/lib/key_utils.hh" #include "mutation/atomic_cell_or_collection.hh" -#include - // Helper for working with the following table: // // CREATE TABLE ks.cf (pk text, ck text, v text, s1 text static, PRIMARY KEY (pk, ck)); @@ -273,9 +271,9 @@ public: } static std::vector to_ring_positions(const std::vector& keys) { - return boost::copy_range>(keys | boost::adaptors::transformed([] (const dht::decorated_key& key) { + return keys | std::views::transform([] (const dht::decorated_key& key) { return dht::ring_position(key); - })); + }) | std::ranges::to(); } // Returns n clustering keys in their natural order diff --git a/test/manual/enormous_table_scan_test.cc b/test/manual/enormous_table_scan_test.cc index ff055c3e0774..862d3151439f 100644 --- a/test/manual/enormous_table_scan_test.cc +++ b/test/manual/enormous_table_scan_test.cc @@ -16,10 +16,6 @@ #include "test/lib/cql_assertions.hh" #include "transport/messages/result_message.hh" -#include -#include -#include - #include "dht/i_partitioner.hh" #include "mutation/mutation_fragment.hh" #include "schema/schema_builder.hh" diff --git a/test/perf/perf_commitlog.cc b/test/perf/perf_commitlog.cc index dc7b2ce53236..71047ae68562 100644 --- a/test/perf/perf_commitlog.cc +++ b/test/perf/perf_commitlog.cc @@ -9,10 +9,8 @@ #include #include -#include #include #include -#include #include #include @@ -258,10 +256,10 @@ int main(int argc, char** argv) { auto median = median_result.throughput; auto min = results[0].throughput; auto max = results[results.size() - 1].throughput; - auto absolute_deviations = boost::copy_range>( - results - | boost::adaptors::transformed(std::mem_fn(&perf_result::throughput)) - | boost::adaptors::transformed([&] (double r) { return abs(r - median); })); + auto absolute_deviations = results + | std::views::transform(std::mem_fn(&perf_result::throughput)) + | std::views::transform([&] (double r) { return abs(r - median); }) + | std::ranges::to>(); std::sort(absolute_deviations.begin(), absolute_deviations.end()); auto mad = absolute_deviations[results.size() / 2]; std::cout << format("\nmedian {}\nmedian absolute deviation: {:.2f}\nmaximum: {:.2f}\nminimum: {:.2f}\n", median_result, mad, max, min); diff --git a/test/perf/perf_fast_forward.cc b/test/perf/perf_fast_forward.cc index e4447725a586..ffc4153cd5b9 100644 --- a/test/perf/perf_fast_forward.cc +++ b/test/perf/perf_fast_forward.cc @@ -1284,12 +1284,12 @@ void print_all(const test_result_vector& results) { if (!dump_all_results || results.empty()) { return; } - output_mgr->add_all_test_values(results.back().get_params(), boost::copy_range>( - results - | boost::adaptors::transformed([] (const test_result& tr) { + output_mgr->add_all_test_values(results.back().get_params(), results + | std::views::transform([] (const test_result& tr) { return tr.get_stats_values(); }) - )); + | std::ranges::to>() + ); } class result_collector { @@ -1965,9 +1965,9 @@ int scylla_fast_forward_main(int argc, char** argv) { app.add_options() ("random-seed", boost::program_options::value(), "Random number generator seed") ("run-tests", bpo::value>()->default_value( - boost::copy_range>( - test_groups | boost::adaptors::transformed([] (auto&& tc) { return tc.name; })) - ), + test_groups + | std::views::transform([] (auto&& tc) { return tc.name; }) + | std::ranges::to>()), "Test groups to run") ("datasets", bpo::value>()->default_value( datasets @@ -2069,13 +2069,13 @@ int scylla_fast_forward_main(int argc, char** argv) { output_mgr = std::make_unique(app.configuration()["output-format"].as()); auto enabled_dataset_names = app.configuration()["datasets"].as>(); - auto enabled_datasets = boost::copy_range>(enabled_dataset_names - | boost::adaptors::transformed([&](auto&& name) { + auto enabled_datasets = enabled_dataset_names + | std::views::transform([&](auto&& name) { if (!datasets.contains(name)) { throw std::runtime_error(seastar::format("No such dataset: {}", name)); } return datasets[name].get(); - })); + }) | std::ranges::to>(); if (app.configuration().contains("populate")) { int n_rows = app.configuration()["rows"].as(); @@ -2112,10 +2112,11 @@ int scylla_fast_forward_main(int argc, char** argv) { return requested_test_groups.contains(tc.name); }); - auto compaction_guard = make_compaction_disabling_guard(db, boost::copy_range>( - enabled_datasets | boost::adaptors::transformed([&] (auto&& ds) { + auto compaction_guard = make_compaction_disabling_guard(db, enabled_datasets + | std::views::transform([&] (auto&& ds) { return &find_table(db, *ds); - }))); + }) + | std::ranges::to>()); auto run_tests = [&] (test_group::type type) { std::ranges::for_each( diff --git a/test/perf/perf_mutation_readers.cc b/test/perf/perf_mutation_readers.cc index 812a5e3ae731..9f9999c6fce8 100644 --- a/test/perf/perf_mutation_readers.cc +++ b/test/perf/perf_mutation_readers.cc @@ -71,28 +71,26 @@ class combined { std::vector combined::create_one_row(simple_schema& s, reader_permit permit) { - return boost::copy_range>( - s.make_pkeys(1) - | boost::adaptors::transformed([&] (auto& dkey) { + return s.make_pkeys(1) + | std::views::transform([&] (auto& dkey) { auto m = mutation(s.schema(), dkey); m.apply(s.make_row(permit, s.make_ckey(0), "value")); return m; - }) - ); + }) + | std::ranges::to>(); } std::vector combined::create_single_stream(simple_schema& s, reader_permit permit) { - return boost::copy_range>( - s.make_pkeys(32) - | boost::adaptors::transformed([&] (auto& dkey) { + return s.make_pkeys(32) + | std::views::transform([&] (auto& dkey) { auto m = mutation(s.schema(), dkey); for (auto i = 0; i < 16; i++) { m.apply(s.make_row(permit, s.make_ckey(i), "value")); } return m; - }) - ); + }) + | std::ranges::to>(); } std::vector> combined::create_disjoint_interleaved_streams(simple_schema& s, reader_permit permit) @@ -127,16 +125,15 @@ std::vector> combined::create_overlapping_partitions_disjo auto keys = s.make_pkeys(4); std::vector> mss; for (int i = 0; i < 4; i++) { - mss.emplace_back(boost::copy_range>( - keys - | boost::adaptors::transformed([&] (auto& dkey) { + mss.emplace_back(keys + | std::views::transform([&] (auto& dkey) { auto m = mutation(s.schema(), dkey); for (int j = 0; j < 32; j++) { m.apply(s.make_row(permit, s.make_ckey(32 * i + j), "value")); } return m; - }) - )); + }) + | std::ranges::to>()); } return mss; } @@ -192,36 +189,33 @@ PERF_TEST_F(combined, many_overlapping) PERF_TEST_F(combined, disjoint_interleaved) { return consume_all(make_combined_reader(schema().schema(), permit(), - boost::copy_range>( - disjoint_interleaved_streams() - | boost::adaptors::transformed([this] (auto&& ms) { + disjoint_interleaved_streams() + | std::views::transform([this] (auto&& ms) { return schema().schema(), make_mutation_reader_from_mutations_v2(schema().schema(), permit(), std::move(ms)); - }) - ) + }) + | std::ranges::to>() )); } PERF_TEST_F(combined, disjoint_ranges) { return consume_all(make_combined_reader(schema().schema(), permit(), - boost::copy_range>( - disjoint_ranges_streams() - | boost::adaptors::transformed([this] (auto&& ms) { + disjoint_ranges_streams() + | std::views::transform([this] (auto&& ms) { return make_mutation_reader_from_mutations_v2(schema().schema(), permit(), std::move(ms)); - }) - ) + }) + | std::ranges::to>() )); } PERF_TEST_F(combined, overlapping_partitions_disjoint_rows) { return consume_all(make_combined_reader(schema().schema(), permit(), - boost::copy_range>( overlapping_partitions_disjoint_rows_streams() - | boost::adaptors::transformed([this] (auto&& ms) { + | std::views::transform([this] (auto&& ms) { return make_mutation_reader_from_mutations_v2(schema().schema(), permit(), std::move(ms)); - }) - ) + }) + | std::ranges::to>() )); } @@ -288,23 +282,21 @@ future clustering_combined::consume_all(mutation_reader mr) const PERF_TEST_F(clustering_combined, ranges_generic) { return consume_all(make_combined_reader(schema().schema(), permit(), - boost::copy_range>( - almost_disjoint_clustering_ranges() - | boost::adaptors::transformed([this] (auto&& mb) { + almost_disjoint_clustering_ranges() + | std::views::transform([this] (auto&& mb) { return make_mutation_reader_from_mutations_v2(schema().schema(), permit(), std::move(mb.m)); - }) - ) + }) + | std::ranges::to>() )); } PERF_TEST_F(clustering_combined, ranges_specialized) { - auto rbs = boost::copy_range>( - almost_disjoint_clustering_ranges() | boost::adaptors::transformed([this] (auto&& mb) { + auto rbs = almost_disjoint_clustering_ranges() | std::views::transform([this] (auto&& mb) { return reader_bounds{ make_mutation_reader_from_mutations_v2(schema().schema(), permit(), std::move(mb.m)), std::move(mb.lower), std::move(mb.upper)}; - })); + }) | std::ranges::to>(); auto q = std::make_unique(*schema().schema(), std::move(rbs)); return consume_all(make_clustering_combined_reader( schema().schema(), permit(), streamed_mutation::forwarding::no, std::move(q))); @@ -360,9 +352,9 @@ class memtable_single_row : public memtable { memtable_single_row() : _mt(make_lw_shared(schema())) { - boost::for_each( + std::ranges::for_each( _dkeys - | boost::adaptors::transformed([&] (auto& dkey) { + | std::views::transform([&] (auto& dkey) { auto m = mutation(schema(), dkey); m.apply(_schema.make_row(_permit, _schema.make_ckey(0), "value")); return m; @@ -392,9 +384,9 @@ class memtable_multi_row : public memtable { memtable_multi_row() : _mt(make_lw_shared(_schema.schema())) { - boost::for_each( + std::ranges::for_each( _dkeys - | boost::adaptors::transformed([&] (auto& dkey) { + | std::views::transform([&] (auto& dkey) { auto m = mutation(_schema.schema(), dkey); for (auto i = 0u; i < row_count; i++) { m.apply(_schema.make_row(_permit, _schema.make_ckey(i), "value")); @@ -427,9 +419,9 @@ class memtable_large_partition : public memtable { memtable_large_partition() : _mt(make_lw_shared(_schema.schema())) { - boost::for_each( + std::ranges::for_each( _dkeys - | boost::adaptors::transformed([&] (auto& dkey) { + | std::views::transform([&] (auto& dkey) { auto m = mutation(_schema.schema(), dkey); // Make sure the partition fills buffers in flat mutation reader multiple times for (auto i = 0u; i < 8 * 1024; i++) { diff --git a/tools/schema_loader.cc b/tools/schema_loader.cc index 691c2f239942..6b7d748c475c 100644 --- a/tools/schema_loader.cc +++ b/tools/schema_loader.cc @@ -118,7 +118,7 @@ class data_dictionary_impl : public data_dictionary::impl { return wrap(*it); } virtual std::vector get_keyspaces(data_dictionary::database db) const override { - return boost::copy_range>(unwrap(db).keyspaces | boost::adaptors::transformed([this] (const keyspace& ks) { return wrap(ks); })); + return unwrap(db).keyspaces | std::views::transform([this] (const keyspace& ks) { return wrap(ks); }) | std::ranges::to>(); } virtual std::vector get_user_keyspaces(data_dictionary::database db) const override { return std::ranges::to>( @@ -128,10 +128,10 @@ class data_dictionary_impl : public data_dictionary::impl { ); } virtual std::vector get_all_keyspaces(data_dictionary::database db) const override { - return boost::copy_range>(unwrap(db).keyspaces | boost::adaptors::transformed([] (const keyspace& ks) { return ks.metadata->name(); })); + return unwrap(db).keyspaces | std::views::transform([] (const keyspace& ks) { return ks.metadata->name(); }) | std::ranges::to>(); } virtual std::vector get_tables(data_dictionary::database db) const override { - return boost::copy_range>(unwrap(db).tables | boost::adaptors::transformed([this] (const table& ks) { return wrap(ks); })); + return unwrap(db).tables | std::views::transform([this] (const table& ks) { return wrap(ks); }) | std::ranges::to>(); } virtual std::optional try_find_table(data_dictionary::database db, std::string_view ks, std::string_view tab) const override { auto& tables = unwrap(db).tables; @@ -445,9 +445,9 @@ schema_ptr do_load_schema_from_schema_tables(const db::config& dbcfg, std::files db::cql_type_parser::raw_builder ut_builder(*ks); auto get_list = [] (const query::result_set_row& row, const char* name) { - return boost::copy_range>( - row.get_nonnull(name) - | boost::adaptors::transformed([] (const data_value& v) { return value_cast(v); })); + return row.get_nonnull(name) + | std::views::transform([] (const data_value& v) { return value_cast(v); }) + | std::ranges::to>(); }; for (const auto& row : result.rows()) { diff --git a/tools/scylla-nodetool.cc b/tools/scylla-nodetool.cc index d67d3a1296a0..b85210f043b1 100644 --- a/tools/scylla-nodetool.cc +++ b/tools/scylla-nodetool.cc @@ -4559,7 +4559,7 @@ For more information, see: {})"; const auto operations = get_operations_with_func() | std::views::keys | std::ranges::to(); tool_app_template::config app_cfg{ .name = app_name, - .description = seastar::format(description_template, app_name, nlog.name(), boost::algorithm::join(operations | boost::adaptors::transformed([] (const auto& op) { + .description = seastar::format(description_template, app_name, nlog.name(), fmt::join(operations | std::views::transform([] (const auto& op) { return seastar::format("* {}: {}", op.name(), op.summary()); }), "\n"), doc_link("operating-scylla/nodetool.html")), .logger_name = nlog.name(), diff --git a/tools/scylla-sstable.cc b/tools/scylla-sstable.cc index c38e481b28ac..6db86649cb66 100644 --- a/tools/scylla-sstable.cc +++ b/tools/scylla-sstable.cc @@ -1837,7 +1837,7 @@ class json_mutation_stream_parser { } std::string stack_to_string() const { - return boost::algorithm::join(_state_stack | boost::adaptors::transformed([] (state s) { return std::string(to_string(s)); }), "|"); + return fmt::to_string(fmt::join(_state_stack | std::views::transform([] (state s) { return to_string(s); }), "|")); } template @@ -3142,7 +3142,7 @@ Validate the specified sstables: const auto operations = operations_with_func | std::views::keys | std::ranges::to(); tool_app_template::config app_cfg{ .name = app_name, - .description = seastar::format(description_template, app_name, sst_log.name(), boost::algorithm::join(operations | boost::adaptors::transformed([] (const auto& op) { + .description = seastar::format(description_template, app_name, sst_log.name(), fmt::join(operations | std::views::transform([] (const auto& op) { return seastar::format("* {}: {}", op.name(), op.summary()); }), "\n")), .logger_name = sst_log.name(), diff --git a/tools/scylla-types.cc b/tools/scylla-types.cc index 98c1a275eae7..1f7f1b81d6b3 100644 --- a/tools/scylla-types.cc +++ b/tools/scylla-types.cc @@ -385,7 +385,7 @@ For more information about individual actions, see their specific help: const auto operations = operations_with_func | std::views::keys | std::ranges::to(); tool_app_template::config app_cfg{ .name = app_name, - .description = seastar::format(description_template, app_name, app_name, boost::algorithm::join(operations | boost::adaptors::transformed( + .description = seastar::format(description_template, app_name, app_name, fmt::join(operations | std::views::transform( [] (const operation& op) { return fmt::format("* {} - {}", op.name(), op.summary()); } ), "\n")), .operations = std::move(operations), .global_options = &global_options, @@ -398,8 +398,9 @@ For more information about individual actions, see their specific help: throw std::invalid_argument("error: missing required option '--type'"); } type_variant type = [&app_config] () -> type_variant { - auto types = boost::copy_range>(app_config["type"].as>() - | boost::adaptors::transformed([] (const std::string_view type_name) { return db::marshal::type_parser::parse(type_name); })); + auto types = app_config["type"].as>() + | std::views::transform([] (const std::string_view type_name) { return db::marshal::type_parser::parse(type_name); }) + | std::ranges::to>(); if (app_config.contains("prefix-compound")) { return compound_type(std::move(types)); } else if (app_config.contains("full-compound")) { @@ -420,9 +421,7 @@ For more information about individual actions, see their specific help: switch (handler.index()) { case 0: { - auto values = boost::copy_range>( - app_config["value"].as>() | boost::adaptors::transformed(from_hex)); - + auto values = app_config["value"].as>() | std::views::transform(from_hex) | std::ranges::to(); std::get(handler)(std::move(type), std::move(values), app_config); } break; diff --git a/tools/utils.cc b/tools/utils.cc index 890338bb9e2c..d8076b1c9a39 100644 --- a/tools/utils.cc +++ b/tools/utils.cc @@ -8,9 +8,7 @@ #include #include -#include #include -#include #include "db/config.hh" #include "db/extensions.hh" diff --git a/tracing/trace_state.cc b/tracing/trace_state.cc index 123146cca9fa..d2f4380b6325 100644 --- a/tracing/trace_state.cc +++ b/tracing/trace_state.cc @@ -8,7 +8,6 @@ * SPDX-License-Identifier: (AGPL-3.0-or-later and Apache-2.0) */ #include -#include #include "cql3/statements/prepared_statement.hh" #include "tracing/trace_state.hh" #include "timestamp.hh" @@ -113,7 +112,7 @@ void trace_state::build_parameters_map() { params_values& vals = *_params_ptr; if (vals.batchlog_endpoints) { - auto batch_endpoints = fmt::format("{}", fmt::join(*vals.batchlog_endpoints | boost::adaptors::transformed([](gms::inet_address ep) {return seastar::format("/{}", ep);}), ",")); + auto batch_endpoints = fmt::format("{}", fmt::join(*vals.batchlog_endpoints | std::views::transform([](gms::inet_address ep) {return seastar::format("/{}", ep);}), ",")); params_map.emplace("batch_endpoints", std::move(batch_endpoints)); }