Skip to content

Commit

Permalink
treewide: replace boost::algorithm::join() with fmt::join()
Browse files Browse the repository at this point in the history
Replace usages of `boost::algorithm::join()` with `fmt::join()` to improve
performance and reduce dependency on Boost. `fmt::join()` allows direct
formatting of ranges and tuples with custom separators without creating
intermediate strings.

When formatting comma-separated values into another string, fmt::join()
avoids the overhead of temporary string creation that
`boost::algorithm::join()` requires. This change also helps streamline
our dependencies by leveraging the existing fmt library instead of
Boost.Algorithm.

To avoid the ambiguity, some caller sites were updated to call
`seastar::format()` explicitly.

See also

- boost::algorithm::join():
  https://www.boost.org/doc/libs/1_87_0/doc/html/string_algo/reference.html#doxygen.join_8hpp
- fmt::join():
  https://fmt.dev/11.0/api/#ranges-api

Signed-off-by: Kefu Chai <[email protected]>

Closes scylladb#22082
  • Loading branch information
tchaikov authored and denesb committed Jan 7, 2025
1 parent a91e037 commit e4463b1
Show file tree
Hide file tree
Showing 22 changed files with 19 additions and 47 deletions.
1 change: 0 additions & 1 deletion auth/default_authorizer.cc
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ extern "C" {
#include <unistd.h>
}

#include <boost/algorithm/string/join.hpp>
#include <boost/range.hpp>
#include <seastar/core/seastar.hh>
#include <seastar/core/sleep.hh>
Expand Down
3 changes: 1 addition & 2 deletions auth/resource.cc
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@
#include <iterator>
#include <unordered_map>

#include <boost/algorithm/string/join.hpp>
#include <boost/algorithm/string/split.hpp>
#include <boost/algorithm/string/classification.hpp>

Expand Down Expand Up @@ -148,7 +147,7 @@ resource::resource(functions_resource_t, std::string_view keyspace, std::string_
}

sstring resource::name() const {
return boost::algorithm::join(_parts, "/");
return fmt::to_string(fmt::join(_parts, "/"));
}

std::optional<resource> resource::parent() const {
Expand Down
6 changes: 2 additions & 4 deletions auth/service.cc
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@
#include "auth/service.hh"

#include <algorithm>
#include <boost/algorithm/string/join.hpp>
#include <chrono>

#include <seastar/core/future-util.hh>
Expand Down Expand Up @@ -875,7 +874,6 @@ future<> migrate_to_auth_v2(db::system_keyspace& sys_ks, ::service::raft_group0_
for (const auto& col : schema->all_columns()) {
col_names.push_back(col.name_as_cql_string());
}
auto col_names_str = boost::algorithm::join(col_names, ", ");
sstring val_binders_str = "?";
for (size_t i = 1; i < col_names.size(); ++i) {
val_binders_str += ", ?";
Expand All @@ -891,10 +889,10 @@ future<> migrate_to_auth_v2(db::system_keyspace& sys_ks, ::service::raft_group0_
}
}
auto muts = co_await qp.get_mutations_internal(
format("INSERT INTO {}.{} ({}) VALUES ({})",
seastar::format("INSERT INTO {}.{} ({}) VALUES ({})",
db::system_keyspace::NAME,
cf_name,
col_names_str,
fmt::join(col_names, ", "),
val_binders_str),
internal_distributed_query_state(),
ts,
Expand Down
3 changes: 1 addition & 2 deletions auth/standard_role_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,6 @@
#include <unordered_set>
#include <vector>

#include <boost/algorithm/string/join.hpp>
#include <seastar/core/future-util.hh>
#include <seastar/core/on_internal_error.hh>
#include <seastar/core/format.hh>
Expand Down Expand Up @@ -330,7 +329,7 @@ standard_role_manager::alter(std::string_view role_name, const role_config_updat
assignments.push_back(sstring("can_login = ") + (*u.can_login ? "true" : "false"));
}

return boost::algorithm::join(assignments, ", ");
return fmt::to_string(fmt::join(assignments, ", "));
};

return require_record(_qp, role_name).then([this, role_name, &u, &mc](record) {
Expand Down
3 changes: 0 additions & 3 deletions cql3/statements/authorization_statement.cc
Original file line number Diff line number Diff line change
Expand Up @@ -13,9 +13,6 @@
#include "auth/resource.hh"
#include "cql3/query_processor.hh"
#include "exceptions/exceptions.hh"
#include <boost/algorithm/string/join.hpp>
#include <boost/algorithm/string/split.hpp>
#include <boost/algorithm/string/classification.hpp>
#include "db/cql_type_parser.hh"
#include "auth/common.hh"

Expand Down
2 changes: 0 additions & 2 deletions cql3/util.cc
Original file line number Diff line number Diff line change
Expand Up @@ -8,8 +8,6 @@
#include "util.hh"
#include "cql3/expr/expr-utils.hh"

#include <boost/algorithm/string/join.hpp>

#ifdef DEBUG

#include <ucontext.h>
Expand Down
5 changes: 2 additions & 3 deletions db/view/view.cc
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@
#include <vector>
#include <algorithm>

#include <boost/algorithm/string/join.hpp>
#include <boost/range/numeric.hpp>

#include <fmt/ranges.h>
Expand Down Expand Up @@ -2656,7 +2655,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 = schema->all_columns() | std::views::transform([] (const auto& col) {return col.name_as_cql_string(); }) | std::ranges::to<std::vector<sstring>>();
auto col_names_str = boost::algorithm::join(col_names, ", ");
auto col_names_str = fmt::to_string(fmt::join(col_names, ", "));
sstring val_binders_str = "?";
for (size_t i = 1; i < col_names.size(); ++i) {
val_binders_str += ", ?";
Expand Down Expand Up @@ -2698,7 +2697,7 @@ future<> view_builder::migrate_to_v2(locator::token_metadata_ptr tmptr, db::syst
auto row_ts = row.get_as<api::timestamp_type>("ts");

auto muts = co_await qp.get_mutations_internal(
format("INSERT INTO {}.{} ({}) VALUES ({})",
seastar::format("INSERT INTO {}.{} ({}) VALUES ({})",
db::system_keyspace::NAME,
db::system_keyspace::VIEW_BUILD_STATUS_V2,
col_names_str,
Expand Down
1 change: 0 additions & 1 deletion index/secondary_index.cc
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@
#include "cql3/statements/index_target.hh"

#include <boost/regex.hpp>
#include <boost/algorithm/string/join.hpp>
#include <seastar/util/log.hh>

#include "exceptions/exceptions.hh"
Expand Down
7 changes: 3 additions & 4 deletions main.cc
Original file line number Diff line number Diff line change
Expand Up @@ -118,8 +118,6 @@
#include "utils/shared_dict.hh"
#include "message/dictionary_service.hh"

#include <boost/algorithm/string/join.hpp>

seastar::metrics::metric_groups app_metrics;

using namespace std::chrono_literals;
Expand Down Expand Up @@ -533,8 +531,9 @@ std::string format_parsed_options(const std::vector<bpo::option>& opts) {
return opt.string_key;
}

return (opt.string_key.empty() ? "(positional) " : fmt::format("{}: ", opt.string_key)) +
boost::algorithm::join(opt.value, " ");
return fmt::format("{}{}",
opt.string_key.empty() ? "(positional) " : fmt::format("{}: ", opt.string_key),
fmt::join(opt.value, " "));
}), ", ")
);
}
Expand Down
5 changes: 2 additions & 3 deletions service/qos/service_level_controller.cc
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,6 @@

#include "cql3/util.hh"
#include "utils/assert.hh"
#include <boost/algorithm/string/join.hpp>
#include <boost/range/adaptor/reversed.hpp>
#include <chrono>

Expand Down Expand Up @@ -824,7 +823,7 @@ future<> service_level_controller::migrate_to_v2(size_t nodes_count, db::system_


auto col_names = schema->all_columns() | std::views::transform([] (const auto& col) {return col.name_as_cql_string(); }) | std::ranges::to<std::vector<sstring>>();
auto col_names_str = boost::algorithm::join(col_names, ", ");
auto col_names_str = fmt::to_string(fmt::join(col_names, ", "));
sstring val_binders_str = "?";
for (size_t i = 1; i < col_names.size(); ++i) {
val_binders_str += ", ?";
Expand All @@ -844,7 +843,7 @@ future<> service_level_controller::migrate_to_v2(size_t nodes_count, db::system_
}

auto muts = co_await qp.get_mutations_internal(
format("INSERT INTO {}.{} ({}) VALUES ({})",
seastar::format("INSERT INTO {}.{} ({}) VALUES ({})",
db::system_keyspace::NAME,
db::system_keyspace::SERVICE_LEVELS_V2,
col_names_str,
Expand Down
3 changes: 1 addition & 2 deletions service/raft/raft_group0_client.cc
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,6 @@
#include <seastar/core/coroutine.hh>
#include "raft_group0_client.hh"
#include "raft_group_registry.hh"
#include <boost/algorithm/string/join.hpp>

#include "frozen_schema.hh"
#include "schema_mutations.hh"
Expand Down Expand Up @@ -589,7 +588,7 @@ future<> group0_batch::commit(::service::raft_group0_client& group0_client, seas
if (!_guard) {
on_internal_error(logger, "group0_batch: trying to announce without guard");
}
auto description = boost::algorithm::join(_descriptions, "; ");
auto description = fmt::to_string(fmt::join(_descriptions, "; "));
// common case, don't bother with generators as we would have only 1-2 mutations,
// when producer expects substantial number or size of mutations it should use generator
if (_generators.size() == 0) {
Expand Down
1 change: 0 additions & 1 deletion service/storage_service.cc
Original file line number Diff line number Diff line change
Expand Up @@ -114,7 +114,6 @@

#include <boost/algorithm/string/split.hpp>
#include <boost/algorithm/string/classification.hpp>
#include <boost/algorithm/string/join.hpp>
#include <stdexcept>
#include <unistd.h>

Expand Down
1 change: 0 additions & 1 deletion test/boost/cql_query_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,6 @@
*/


#include <boost/algorithm/string/join.hpp>
#include <boost/range/irange.hpp>
#include <boost/range/algorithm.hpp>
#include <boost/range/adaptor/reversed.hpp>
Expand Down
1 change: 0 additions & 1 deletion test/boost/user_types_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@
#include "db/config.hh"

#include <fmt/ranges.h>
#include <boost/algorithm/string/join.hpp>

BOOST_AUTO_TEST_SUITE(user_types_test)

Expand Down
4 changes: 1 addition & 3 deletions test/lib/data_model.cc
Original file line number Diff line number Diff line change
Expand Up @@ -11,8 +11,6 @@
#include "utils/assert.hh"
#include "test/lib/data_model.hh"

#include <boost/algorithm/string/join.hpp>

#include "schema/schema_builder.hh"
#include "concrete_types.hh"

Expand Down Expand Up @@ -356,7 +354,7 @@ void table_description::rename_clustering_column(const sstring& from, const sstr

table_description::table table_description::build() const {
auto s = build_schema();
return { boost::algorithm::join(_change_log, "\n"), s, build_mutations(s) };
return { fmt::to_string(fmt::join(_change_log, "\n")), s, build_mutations(s) };
}

}
1 change: 0 additions & 1 deletion test/lib/mutation_source_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@
#include "test/lib/key_utils.hh"
#include "test/lib/log.hh"
#include "test/lib/reader_concurrency_semaphore.hh"
#include <boost/algorithm/string/join.hpp>
#include "types/user.hh"
#include "types/map.hh"
#include "types/list.hh"
Expand Down
11 changes: 5 additions & 6 deletions test/lib/random_schema.cc
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,6 @@

#include <algorithm>

#include <boost/algorithm/string/join.hpp>
#include <boost/range/algorithm/unique.hpp>

#include <seastar/coroutine/maybe_yield.hh>
Expand Down Expand Up @@ -1016,22 +1015,22 @@ sstring random_schema::cql() const {
std::move(cols.begin(), cols.end(), std::back_inserter(col_specs));
}

sstring primary_key;
std::string primary_key;
auto partition_column_names = column_names(_schema, column_kind::partition_key);
auto clustering_key_names = column_names(_schema, column_kind::clustering_key);
if (!clustering_key_names.empty()) {
primary_key = format("({}), {}", boost::algorithm::join(partition_column_names, ", "), boost::algorithm::join(clustering_key_names, ", "));
primary_key = fmt::format("({}), {}", fmt::join(partition_column_names, ", "), fmt::join(clustering_key_names, ", "));
} else {
primary_key = format("{}", boost::algorithm::join(partition_column_names, ", "));
primary_key = fmt::format("{}", fmt::join(partition_column_names, ", "));
}

// FIXME include the clustering column orderings
return format(
return seastar::format(
"{}\nCREATE TABLE {}.{} (\n\t{}\n\tPRIMARY KEY ({}))",
udts_str,
_schema->ks_name(),
_schema->cf_name(),
boost::algorithm::join(col_specs, ",\n\t"),
fmt::join(col_specs, ",\n\t"),
primary_key);
}

Expand Down
1 change: 0 additions & 1 deletion test/perf/perf_fast_forward.cc
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@
#include "utils/assert.hh"
#include <boost/algorithm/string/replace.hpp>
#include <boost/algorithm/string/split.hpp>
#include <boost/algorithm/string/join.hpp>
#include <boost/date_time/posix_time/posix_time.hpp>
#include <boost/range/irange.hpp>
#include <boost/range/algorithm_ext.hpp>
Expand Down
1 change: 0 additions & 1 deletion tools/lua_sstable_consumer.cc
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,6 @@
*/

#include "utils/assert.hh"
#include <boost/algorithm/string/join.hpp>
#include <fmt/chrono.h>
#include <lua.hpp>
#include <seastar/core/fstream.hh>
Expand Down
1 change: 0 additions & 1 deletion tools/scylla-nodetool.cc
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@
#include <numeric>
#include <boost/algorithm/string/case_conv.hpp>
#include <boost/algorithm/string/classification.hpp>
#include <boost/algorithm/string/join.hpp>
#include <boost/algorithm/string/split.hpp>
#include <boost/lexical_cast.hpp>
#include <boost/make_shared.hpp>
Expand Down
1 change: 0 additions & 1 deletion tools/scylla-sstable.cc
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,6 @@
*/

#include <boost/algorithm/string.hpp>
#include <boost/algorithm/string/join.hpp>
#include <boost/range/adaptor/map.hpp>
#include <filesystem>
#include <set>
Expand Down
4 changes: 1 addition & 3 deletions tools/scylla-types.cc
Original file line number Diff line number Diff line change
Expand Up @@ -6,8 +6,6 @@
* SPDX-License-Identifier: LicenseRef-ScyllaDB-Source-Available-1.0
*/

#include <boost/algorithm/string/join.hpp>
#include <boost/range/adaptor/map.hpp>
#include <seastar/core/coroutine.hh>

#include <fmt/ranges.h>
Expand Down Expand Up @@ -96,7 +94,7 @@ sstring to_printable_string(const compound_type<AllowPrefixes>& type, bytes_view
for (size_t i = 0; i != values.size(); ++i) {
printable_values.emplace_back(types.at(i)->to_string(values.at(i)));
}
return format("({})", boost::algorithm::join(printable_values, ", "));
return seastar::format("({})", fmt::join(printable_values, ", "));
}

struct printing_visitor {
Expand Down

0 comments on commit e4463b1

Please sign in to comment.