Skip to content

Commit c752383

Browse files
committed
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 <boost/range/adaptor/transformed.hpp>` 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 <[email protected]>
1 parent 40abebe commit c752383

File tree

92 files changed

+339
-457
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

92 files changed

+339
-457
lines changed

api/column_family.cc

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -200,7 +200,7 @@ static future<json::json_return_type> get_cf_histogram(http_context& ctx, utils:
200200
};
201201
return ctx.db.map(fun).then([](const std::vector<utils::ihistogram> &res) {
202202
std::vector<httpd::utils_json::histogram> r;
203-
boost::copy(res | boost::adaptors::transformed(to_json), std::back_inserter(r));
203+
std::ranges::copy(res | std::views::transform(to_json), std::back_inserter(r));
204204
return make_ready_future<json::json_return_type>(r);
205205
});
206206
}
@@ -227,7 +227,7 @@ static future<json::json_return_type> get_cf_rate_and_histogram(http_context& ct
227227
};
228228
return ctx.db.map(fun).then([](const std::vector<utils::rate_moving_average_and_histogram> &res) {
229229
std::vector<httpd::utils_json::rate_moving_average_and_histogram> r;
230-
boost::copy(res | boost::adaptors::transformed(timer_to_json), std::back_inserter(r));
230+
std::ranges::copy(res | std::views::transform(timer_to_json), std::back_inserter(r));
231231
return make_ready_future<json::json_return_type>(r);
232232
});
233233
}
@@ -717,25 +717,25 @@ void set_column_family(http_context& ctx, routes& r, sharded<db::system_keyspace
717717

718718
cf::get_bloom_filter_false_ratio.set(r, [&ctx] (std::unique_ptr<http::request> req) {
719719
return map_reduce_cf(ctx, req->get_path_param("name"), ratio_holder(), [] (replica::column_family& cf) {
720-
return boost::accumulate(*cf.get_sstables() | boost::adaptors::transformed(filter_false_positive_as_ratio_holder), ratio_holder());
720+
return std::ranges::fold_left(*cf.get_sstables() | std::views::transform(filter_false_positive_as_ratio_holder), ratio_holder(), std::plus{});
721721
}, std::plus<>());
722722
});
723723

724724
cf::get_all_bloom_filter_false_ratio.set(r, [&ctx] (std::unique_ptr<http::request> req) {
725725
return map_reduce_cf(ctx, ratio_holder(), [] (replica::column_family& cf) {
726-
return boost::accumulate(*cf.get_sstables() | boost::adaptors::transformed(filter_false_positive_as_ratio_holder), ratio_holder());
726+
return std::ranges::fold_left(*cf.get_sstables() | std::views::transform(filter_false_positive_as_ratio_holder), ratio_holder(), std::plus{});
727727
}, std::plus<>());
728728
});
729729

730730
cf::get_recent_bloom_filter_false_ratio.set(r, [&ctx] (std::unique_ptr<http::request> req) {
731731
return map_reduce_cf(ctx, req->get_path_param("name"), ratio_holder(), [] (replica::column_family& cf) {
732-
return boost::accumulate(*cf.get_sstables() | boost::adaptors::transformed(filter_recent_false_positive_as_ratio_holder), ratio_holder());
732+
return std::ranges::fold_left(*cf.get_sstables() | std::views::transform(filter_recent_false_positive_as_ratio_holder), ratio_holder(), std::plus{});
733733
}, std::plus<>());
734734
});
735735

736736
cf::get_all_recent_bloom_filter_false_ratio.set(r, [&ctx] (std::unique_ptr<http::request> req) {
737737
return map_reduce_cf(ctx, ratio_holder(), [] (replica::column_family& cf) {
738-
return boost::accumulate(*cf.get_sstables() | boost::adaptors::transformed(filter_recent_false_positive_as_ratio_holder), ratio_holder());
738+
return std::ranges::fold_left(*cf.get_sstables() | std::views::transform(filter_recent_false_positive_as_ratio_holder), ratio_holder(), std::plus{});
739739
}, std::plus<>());
740740
});
741741

@@ -1089,7 +1089,7 @@ void set_column_family(http_context& ctx, routes& r, sharded<db::system_keyspace
10891089

10901090
return ctx.db.map_reduce0([key, uuid] (replica::database& db) -> future<std::unordered_set<sstring>> {
10911091
auto sstables = co_await db.find_column_family(uuid).get_sstables_by_partition_key(key);
1092-
co_return boost::copy_range<std::unordered_set<sstring>>(sstables | boost::adaptors::transformed([] (auto s) { return s->get_filename(); }));
1092+
co_return sstables | std::views::transform([] (auto s) { return s->get_filename(); }) | std::ranges::to<std::unordered_set>();
10931093
}, std::unordered_set<sstring>(),
10941094
[](std::unordered_set<sstring> a, std::unordered_set<sstring>&& b) mutable {
10951095
a.merge(b);

auth/resource.cc

Lines changed: 5 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -194,7 +194,7 @@ service_level_resource_view::service_level_resource_view(const resource &r) {
194194

195195
sstring encode_signature(std::string_view name, std::vector<data_type> args) {
196196
return seastar::format("{}[{}]", name,
197-
fmt::join(args | boost::adaptors::transformed([] (const data_type t) {
197+
fmt::join(args | std::views::transform([] (const data_type t) {
198198
return t->name();
199199
}), "^"));
200200
}
@@ -209,11 +209,10 @@ std::pair<sstring, std::vector<data_type>> decode_signature(std::string_view enc
209209
}
210210
std::vector<std::string_view> raw_types;
211211
boost::split(raw_types, encoded_signature, boost::is_any_of("^"));
212-
std::vector<data_type> decoded_types = boost::copy_range<std::vector<data_type>>(
213-
raw_types | boost::adaptors::transformed([] (std::string_view raw_type) {
212+
std::vector<data_type> decoded_types =
213+
raw_types | std::views::transform([] (std::string_view raw_type) {
214214
return db::marshal::type_parser::parse(raw_type);
215-
})
216-
);
215+
}) | std::ranges::to<std::vector>();
217216
return {sstring(function_name), decoded_types};
218217
}
219218

@@ -223,7 +222,7 @@ std::pair<sstring, std::vector<data_type>> decode_signature(std::string_view enc
223222
static sstring decoded_signature_string(std::string_view encoded_signature) {
224223
auto [function_name, arg_types] = decode_signature(encoded_signature);
225224
return seastar::format("{}({})", cql3::util::maybe_quote(sstring(function_name)),
226-
boost::algorithm::join(arg_types | boost::adaptors::transformed([] (data_type t) {
225+
fmt::join(arg_types | std::views::transform([] (data_type t) {
227226
return t->cql3_type_name();
228227
}), ", "));
229228
}

compaction/compaction.cc

Lines changed: 13 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -546,12 +546,12 @@ class compaction {
546546
}
547547
auto owned_ranges = dht::to_partition_ranges(*_owned_ranges, utils::can_yield::yes);
548548

549-
auto non_owned_ranges = boost::copy_range<dht::partition_range_vector>(sstables
550-
| boost::adaptors::transformed([] (const shared_sstable& sst) {
549+
auto non_owned_ranges = sstables
550+
| std::views::transform([] (const shared_sstable& sst) {
551551
seastar::thread::maybe_yield();
552552
return dht::partition_range::make({sst->get_first_decorated_key(), true},
553553
{sst->get_last_decorated_key(), true});
554-
}));
554+
}) | std::ranges::to<dht::partition_range_vector>();
555555

556556
return dht::subtract_ranges(*_schema, non_owned_ranges, std::move(owned_ranges)).get();
557557
}
@@ -820,7 +820,7 @@ class compaction {
820820
_rp = std::max(_rp, sst_stats.position);
821821
}
822822
}
823-
log_info("{} [{}]", report_start_desc(), fmt::join(_sstables | boost::adaptors::transformed([] (auto sst) { return to_string(sst, true); }), ","));
823+
log_info("{} [{}]", report_start_desc(), fmt::join(_sstables | std::views::transform([] (auto sst) { return to_string(sst, true); }), ","));
824824
if (ssts->size() < _sstables.size()) {
825825
log_debug("{} out of {} input sstables are fully expired sstables that will not be actually compacted",
826826
_sstables.size() - ssts->size(), _sstables.size());
@@ -953,7 +953,7 @@ class compaction {
953953
// By the time being, using estimated key count.
954954
log_info("{} {} sstables to [{}]. {} to {} (~{}% of original) in {}ms = {}. ~{} total partitions merged to {}.",
955955
report_finish_desc(), _input_sstable_generations.size(),
956-
fmt::join(ret.new_sstables | boost::adaptors::transformed([] (auto sst) { return to_string(sst, false); }), ","),
956+
fmt::join(ret.new_sstables | std::views::transform([] (auto sst) { return to_string(sst, false); }), ","),
957957
utils::pretty_printed_data_size(_start_size), utils::pretty_printed_data_size(_end_size), int(ratio * 100),
958958
std::chrono::duration_cast<std::chrono::milliseconds>(duration).count(), utils::pretty_printed_throughput(_start_size, duration),
959959
_cdata.total_partitions, _cdata.total_keys_written);
@@ -1268,8 +1268,8 @@ class regular_compaction : public compaction {
12681268

12691269
auto exhausted_ssts = std::vector<shared_sstable>(exhausted, _sstables.end());
12701270
log_debug("Replacing earlier exhausted sstable(s) [{}] by new sstable(s) [{}]",
1271-
fmt::join(exhausted_ssts | boost::adaptors::transformed([] (auto sst) { return to_string(sst, false); }), ","),
1272-
fmt::join(_new_unused_sstables | boost::adaptors::transformed([] (auto sst) { return to_string(sst, true); }), ","));
1271+
fmt::join(exhausted_ssts | std::views::transform([] (auto sst) { return to_string(sst, false); }), ","),
1272+
fmt::join(_new_unused_sstables | std::views::transform([] (auto sst) { return to_string(sst, true); }), ","));
12731273
_replacer(get_compaction_completion_desc(exhausted_ssts, std::move(_new_unused_sstables)));
12741274
_sstables.erase(exhausted, _sstables.end());
12751275
dynamic_cast<compaction_read_monitor_generator&>(unwrap_monitor_generator()).remove_exhausted_sstables(exhausted_ssts);
@@ -1917,7 +1917,7 @@ static future<compaction_result> scrub_sstables_validate_mode(sstables::compacti
19171917
auto permit = table_s.make_compaction_reader_permit();
19181918

19191919
uint64_t validation_errors = 0;
1920-
cdata.compaction_size = boost::accumulate(descriptor.sstables | boost::adaptors::transformed([] (auto& sst) { return sst->data_size(); }), int64_t(0));
1920+
cdata.compaction_size = std::ranges::fold_left(descriptor.sstables | std::views::transform([] (auto& sst) { return sst->data_size(); }), int64_t(0), std::plus{});
19211921

19221922
for (const auto& sst : descriptor.sstables) {
19231923
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<sstable
19921992
}
19931993
}
19941994

1995-
auto compacted_undeleted_gens = boost::copy_range<std::unordered_set<generation_type>>(table_s.compacted_undeleted_sstables()
1996-
| boost::adaptors::transformed(std::mem_fn(&sstables::sstable::generation)));
1995+
auto compacted_undeleted_gens = table_s.compacted_undeleted_sstables()
1996+
| std::views::transform(std::mem_fn(&sstables::sstable::generation))
1997+
| std::ranges::to<std::unordered_set>();
19971998
auto has_undeleted_ancestor = [&compacted_undeleted_gens] (auto& candidate) {
19981999
// Get ancestors from sstable which is empty after restart. It works for this purpose because
19992000
// 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<sstable
20362037
}
20372038

20382039
unsigned compaction_descriptor::fan_in() const {
2039-
return boost::copy_range<std::unordered_set<run_id>>(sstables | boost::adaptors::transformed(std::mem_fn(&sstables::sstable::run_identifier))).size();
2040+
return std::ranges::distance(sstables | std::views::transform(std::mem_fn(&sstables::sstable::run_identifier)) | std::ranges::to<std::unordered_set>());;
20402041
}
20412042

20422043
uint64_t compaction_descriptor::sstables_size() const {
2043-
return boost::accumulate(sstables | boost::adaptors::transformed(std::mem_fn(&sstables::sstable::data_size)), uint64_t(0));
2044+
return std::ranges::fold_left(sstables | std::views::transform(std::mem_fn(&sstables::sstable::data_size)), uint64_t(0), std::plus{});
20442045
}
20452046

20462047
}

compaction/compaction_manager.cc

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -187,7 +187,7 @@ unsigned compaction_manager::current_compaction_fan_in_threshold() const {
187187
if (_tasks.empty()) {
188188
return 0;
189189
}
190-
auto largest_fan_in = std::ranges::max(_tasks | boost::adaptors::transformed([] (auto& task) {
190+
auto largest_fan_in = std::ranges::max(_tasks | std::views::transform([] (auto& task) {
191191
return task.compaction_running() ? task.compaction_data().compaction_fan_in : 0;
192192
}));
193193
// 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&
13641364
auto num_runs_for_compaction = [&, this] {
13651365
auto& cs = t.get_compaction_strategy();
13661366
auto desc = cs.get_sstables_for_compaction(t, get_strategy_control());
1367-
return boost::copy_range<std::unordered_set<sstables::run_id>>(
1368-
desc.sstables
1369-
| boost::adaptors::transformed(std::mem_fn(&sstables::sstable::run_identifier))).size();
1367+
return std::ranges::size(desc.sstables
1368+
| std::views::transform(std::mem_fn(&sstables::sstable::run_identifier))
1369+
| std::ranges::to<std::unordered_set>());
13701370
};
13711371
const auto threshold = size_t(std::max(schema->max_compaction_threshold(), 32));
13721372
auto count = num_runs_for_compaction();

compaction/compaction_strategy.cc

Lines changed: 5 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -22,8 +22,6 @@
2222
#include "cql3/statements/property_definitions.hh"
2323
#include "schema/schema.hh"
2424
#include <boost/range/algorithm/find.hpp>
25-
#include <boost/range/algorithm/remove_if.hpp>
26-
#include <boost/range/adaptor/transformed.hpp>
2725
#include "size_tiered_compaction_strategy.hh"
2826
#include "leveled_compaction_strategy.hh"
2927
#include "time_window_compaction_strategy.hh"
@@ -49,10 +47,10 @@ compaction_descriptor compaction_strategy_impl::make_major_compaction_job(std::v
4947
std::vector<compaction_descriptor> compaction_strategy_impl::get_cleanup_compaction_jobs(table_state& table_s, std::vector<shared_sstable> candidates) const {
5048
// The default implementation is suboptimal and causes the writeamp problem described issue in #10097.
5149
// The compaction strategy relying on it should strive to implement its own method, to make cleanup bucket aware.
52-
return boost::copy_range<std::vector<compaction_descriptor>>(candidates | boost::adaptors::transformed([] (const shared_sstable& sst) {
50+
return candidates | std::views::transform([] (const shared_sstable& sst) {
5351
return compaction_descriptor({ sst },
5452
sst->get_sstable_level(), sstables::compaction_descriptor::default_max_sstable_bytes, sst->run_identifier());
55-
}));
53+
}) | std::ranges::to<std::vector>();
5654
}
5755

5856
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
246244
if (!size_tiered_compaction_strategy::is_bucket_interesting(bucket, threshold)) {
247245
continue;
248246
}
249-
contrib.value += boost::accumulate(bucket | boost::adaptors::transformed([] (const shared_sstable& sst) -> double {
247+
contrib.value += std::ranges::fold_left(bucket | std::views::transform([] (const shared_sstable& sst) -> double {
250248
return sst->data_size() * log4(sst->data_size());
251-
}), double(0.0f));
249+
}), double(0.0f), std::plus{});
252250
// Controller is disabled if exception is caught during add / remove calls, so not making any effort to make this exception safe
253251
contrib.sstables.insert(bucket.begin(), bucket.end());
254252
}
@@ -259,7 +257,7 @@ size_tiered_backlog_tracker::sstables_backlog_contribution size_tiered_backlog_t
259257
double size_tiered_backlog_tracker::backlog(const compaction_backlog_tracker::ongoing_writes& ow, const compaction_backlog_tracker::ongoing_compactions& oc) const {
260258
inflight_component compacted = compacted_backlog(oc);
261259

262-
auto total_backlog_bytes = boost::accumulate(_contrib.sstables | boost::adaptors::transformed(std::mem_fn(&sstables::sstable::data_size)), uint64_t(0));
260+
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{});
263261

264262
// Bail out if effective backlog is zero, which happens in a small window where ongoing compaction exhausted
265263
// input files but is still sealing output files or doing managerial stuff like updating history table

compaction/size_tiered_compaction_strategy.cc

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -11,7 +11,6 @@
1111
#include "size_tiered_compaction_strategy.hh"
1212
#include "cql3/statements/property_definitions.hh"
1313

14-
#include <boost/range/adaptor/transformed.hpp>
1514
#include <boost/range/adaptor/reversed.hpp>
1615
#include <boost/range/algorithm.hpp>
1716

compaction/task_manager_module.cc

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -723,7 +723,7 @@ future<> table_resharding_compaction_task_impl::run() {
723723
auto all_jobs = co_await collect_all_shared_sstables(_dir, _db, _status.keyspace, _status.table, _owned_ranges_ptr);
724724
auto destinations = co_await distribute_reshard_jobs(std::move(all_jobs));
725725

726-
uint64_t total_size = boost::accumulate(destinations | boost::adaptors::transformed(std::mem_fn(&replica::reshard_shard_descriptor::size)), uint64_t(0));
726+
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{});
727727
if (total_size == 0) {
728728
co_return;
729729
}

compaction/time_window_compaction_strategy.cc

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -16,7 +16,6 @@
1616
#include <boost/range/algorithm/find.hpp>
1717
#include <boost/range/algorithm/remove_if.hpp>
1818
#include <boost/range/algorithm/min_element.hpp>
19-
#include <boost/range/adaptor/transformed.hpp>
2019

2120
#include <ranges>
2221

@@ -267,7 +266,7 @@ time_window_compaction_strategy::get_reshaping_job(std::vector<shared_sstable> i
267266
single_window.size(), !single_window.empty() && sstable_set_overlapping_count(schema, single_window) == 0);
268267

269268
auto get_job_size = [] (const std::vector<shared_sstable>& ssts) {
270-
return boost::accumulate(ssts | boost::adaptors::transformed(std::mem_fn(&sstable::bytes_on_disk)), uint64_t(0));
269+
return std::ranges::fold_left(ssts | std::views::transform(std::mem_fn(&sstable::bytes_on_disk)), uint64_t(0), std::plus{});
271270
};
272271

273272
// Targets a space overhead of 10%. All disjoint sstables can be compacted together as long as they won't

cql3/restrictions/statement_restrictions.cc

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -9,6 +9,7 @@
99

1010
#include <algorithm>
1111
#include <boost/range/algorithm.hpp>
12+
#include <boost/range/numeric.hpp>
1213
#include <functional>
1314
#include <ranges>
1415
#include <stdexcept>

cql3/selection/selection.cc

Lines changed: 4 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -494,7 +494,7 @@ ::shared_ptr<selection> selection::from_selectors(data_dictionary::database db,
494494
auto metadata = collect_metadata(*schema, prepared_selectors);
495495
if (processes_selection(prepared_selectors) || prepared_selectors.size() != defs.size()) {
496496
return ::make_shared<selection_with_processing>(schema, std::move(defs), std::move(metadata),
497-
boost::copy_range<std::vector<expr::expression>>(prepared_selectors | boost::adaptors::transformed(std::mem_fn(&prepared_selector::expr))));
497+
prepared_selectors | std::views::transform(std::mem_fn(&prepared_selector::expr)) | std::ranges::to<std::vector>());
498498
} else {
499499
return ::make_shared<simple_selection>(schema, std::move(defs), std::move(metadata), false);
500500
}
@@ -583,11 +583,9 @@ bool result_set_builder::last_group_ended() const {
583583
if (_last_group.empty()) {
584584
return !_selectors->is_aggregate();
585585
}
586-
using boost::adaptors::reversed;
587-
using boost::adaptors::transformed;
588-
return !boost::equal(
589-
_last_group | reversed,
590-
_group_by_cell_indices | reversed | transformed([this](size_t i) { return current[i]; }));
586+
return !std::ranges::equal(
587+
_last_group | std::views::reverse,
588+
_group_by_cell_indices | std::views::reverse | std::views::transform([this](size_t i) { return current[i]; }));
591589
}
592590

593591
void result_set_builder::flush_selectors() {

0 commit comments

Comments
 (0)