Skip to content

Commit

Permalink
Merge 'treewide: reduce dependencies on boost ranges and algorithms i…
Browse files Browse the repository at this point in the history
…n public headers' from Avi Kivity

The \<ranges> library can replace the use of boost algorithms and ranges. Since
most applications will have started using \<range> themselves, there's no need
to burden them with the double load of both boost and std. So here we reduce
the use of boost in public headers.

No attempt is made at reducing usage of boost in non-public headers or source,
and boost libraries that have no std replacement are kept.

Closes #2459

* github.com:scylladb/seastar:
  rpc: rpc_types: replace boost::any with std::any
  tls: drop dependency on boost::any
  rpc: drop unnecessaty includes to boost libraries
  rpc: compressor factory: deinline some boost-using functions
  sharded: replace boost ranges with <ranges>
  scheduling_specific: drop dependency on boost range adaptors
  prefetch: drop dependency on boost::mpl
  resource: drop unused dependency on boost::any
  smp: drop dependency on boost ranges
  reactor: remove unnecessary boost includes
  execution_stage: remove unnecessary boost includes
  • Loading branch information
nyh committed Sep 30, 2024
2 parents b571539 + 867a242 commit f322e76
Show file tree
Hide file tree
Showing 22 changed files with 87 additions and 65 deletions.
2 changes: 2 additions & 0 deletions demos/rpc_demo.cc
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,8 @@
#include <seastar/util/log.hh>
#include <seastar/core/loop.hh>

#include <boost/range/irange.hpp>

using namespace seastar;

struct serializer {
Expand Down
2 changes: 2 additions & 0 deletions demos/tls_simple_client_demo.cc
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,8 @@
#include <seastar/net/dns.hh>
#include "tls_echo_server.hh"

#include <boost/range/irange.hpp>

using namespace seastar;
namespace bpo = boost::program_options;

Expand Down
2 changes: 0 additions & 2 deletions include/seastar/core/execution_stage.hh
Original file line number Diff line number Diff line change
Expand Up @@ -35,8 +35,6 @@
#ifndef SEASTAR_MODULE
#include <fmt/format.h>
#include <vector>
#include <boost/range/irange.hpp>
#include <boost/range/adaptor/transformed.hpp>
#include <boost/container/static_vector.hpp>
#endif

Expand Down
20 changes: 14 additions & 6 deletions include/seastar/core/prefetch.hh
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,8 @@
#ifndef SEASTAR_MODULE
#include <algorithm>
#include <atomic>
#include <boost/mpl/range_c.hpp>
#include <boost/mpl/for_each.hpp>
#include <utility>
#include <functional>
#include <seastar/core/align.hh>
#include <seastar/core/cacheline.hh>
#include <seastar/util/modules.hh>
Expand Down Expand Up @@ -70,7 +70,9 @@ void prefetch(Iterator begin, Iterator end) {

template<size_t C, typename T, int LOC = 3>
void prefetch_n(T** pptr) {
boost::mpl::for_each< boost::mpl::range_c<size_t,0,C> >( [pptr] (size_t x) { prefetch<T, LOC>(*(pptr + x)); } );
std::invoke([&] <size_t... x> (std::index_sequence<x...>) {
(..., prefetch<T, LOC>(*(pptr + x)));
}, std::make_index_sequence<C>{});
}

template<size_t L, int LOC = 3>
Expand All @@ -85,7 +87,9 @@ void prefetch_n(Iterator begin, Iterator end) {

template<size_t L, size_t C, typename T, int LOC = 3>
void prefetch_n(T** pptr) {
boost::mpl::for_each< boost::mpl::range_c<size_t,0,C> >( [pptr] (size_t x) { prefetch<L, LOC>(*(pptr + x)); } );
std::invoke([&] <size_t... x> (std::index_sequence<x...>) {
(..., prefetch<L, LOC>(*(pptr + x)));
}, std::make_index_sequence<C>{});
}

template<typename T, int LOC = 3>
Expand All @@ -100,7 +104,9 @@ void prefetchw_n(Iterator begin, Iterator end) {

template<size_t C, typename T, int LOC = 3>
void prefetchw_n(T** pptr) {
boost::mpl::for_each< boost::mpl::range_c<size_t,0,C> >( [pptr] (size_t x) { prefetchw<T, LOC>(*(pptr + x)); } );
std::invoke([&] <size_t... x> (std::index_sequence<x...>) {
(..., prefetchw<T, LOC>(*(pptr + x)));
}, std::make_index_sequence<C>{});
}

template<size_t L, int LOC = 3>
Expand All @@ -115,7 +121,9 @@ void prefetchw_n(Iterator begin, Iterator end) {

template<size_t L, size_t C, typename T, int LOC = 3>
void prefetchw_n(T** pptr) {
boost::mpl::for_each< boost::mpl::range_c<size_t,0,C> >( [pptr] (size_t x) { prefetchw<L, LOC>(*(pptr + x)); } );
std::invoke([&] <size_t... x> (std::index_sequence<x...>) {
(..., prefetchw<L, LOC>(*(pptr + x)));
}, std::make_index_sequence<C>{});
}
SEASTAR_MODULE_EXPORT_END

Expand Down
4 changes: 0 additions & 4 deletions include/seastar/core/reactor.hh
Original file line number Diff line number Diff line change
Expand Up @@ -62,10 +62,6 @@

#ifndef SEASTAR_MODULE
#include <boost/container/static_vector.hpp>
#include <boost/lockfree/spsc_queue.hpp>
#include <boost/next_prior.hpp>
#include <boost/range/irange.hpp>
#include <boost/thread/barrier.hpp>
#include <atomic>
#include <cassert>
#include <chrono>
Expand Down
1 change: 0 additions & 1 deletion include/seastar/core/resource.hh
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,6 @@
#include <vector>
#include <set>
#include <sched.h>
#include <boost/any.hpp>
#include <unordered_map>
#ifdef SEASTAR_HAVE_HWLOC
#include <hwloc.h>
Expand Down
6 changes: 3 additions & 3 deletions include/seastar/core/scheduling_specific.hh
Original file line number Diff line number Diff line change
Expand Up @@ -20,13 +20,13 @@
*/

#ifndef SEASTAR_MODULE
#include <boost/range/adaptor/filtered.hpp>
#include <seastar/core/scheduling.hh>
#include <seastar/core/map_reduce.hh>
#include <seastar/util/modules.hh>
#include <array>
#include <typeindex>
#include <vector>
#include <ranges>
#endif

#pragma once
Expand Down Expand Up @@ -159,7 +159,7 @@ map_reduce_scheduling_group_specific(Mapper mapper, Reducer reducer,

return map_reduce(
data.per_scheduling_group_data
| boost::adaptors::filtered(std::mem_fn(&per_scheduling_group::queue_is_initialized)),
| std::views::filter(std::mem_fn(&per_scheduling_group::queue_is_initialized)),
wrapped_mapper, std::move(initial_val), reducer);
}

Expand Down Expand Up @@ -192,7 +192,7 @@ reduce_scheduling_group_specific(Reducer reducer, Initial initial_val, schedulin

return map_reduce(
data.per_scheduling_group_data
| boost::adaptors::filtered(std::mem_fn(&per_scheduling_group::queue_is_initialized)),
| std::views::filter(std::mem_fn(&per_scheduling_group::queue_is_initialized)),
mapper, std::move(initial_val), reducer);
}

Expand Down
11 changes: 5 additions & 6 deletions include/seastar/core/sharded.hh
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,6 @@
#include <seastar/util/modules.hh>

#ifndef SEASTAR_MODULE
#include <boost/iterator/counting_iterator.hpp>
#include <concepts>
#include <functional>
#include <ranges>
Expand Down Expand Up @@ -388,8 +387,8 @@ public:
inline
auto map_reduce(Reducer&& r, Func&& func, Args&&... args) -> typename reducer_traits<Reducer>::future_type
{
return ::seastar::map_reduce(boost::make_counting_iterator<unsigned>(0),
boost::make_counting_iterator<unsigned>(_instances.size()),
auto rng = std::views::iota(size_t(0), _instances.size());
return ::seastar::map_reduce(rng.begin(), rng.end(),
[this, func = std::forward<Func>(func), args = std::make_tuple(std::forward<Args>(args)...)] (unsigned c) mutable {
return smp::submit_to(c, [this, &func, args] () mutable {
return std::apply([this, &func] (Args&&... args) mutable {
Expand All @@ -405,8 +404,8 @@ public:
inline
auto map_reduce(Reducer&& r, Func&& func, Args&&... args) const -> typename reducer_traits<Reducer>::future_type
{
return ::seastar::map_reduce(boost::make_counting_iterator<unsigned>(0),
boost::make_counting_iterator<unsigned>(_instances.size()),
auto rng = std::views::iota(size_t(0), _instances.size());
return ::seastar::map_reduce(rng.begin(), rng.end(),
[this, func = std::forward<Func>(func), args = std::make_tuple(std::forward<Args>(args)...)] (unsigned c) {
return smp::submit_to(c, [this, &func, args] () {
return std::apply([this, &func] (Args&&... args) {
Expand Down Expand Up @@ -480,7 +479,7 @@ public:
return do_with(std::vector<return_type>(), std::move(mapper),
[this] (std::vector<return_type>& vec, Mapper& mapper) mutable {
vec.resize(_instances.size());
return parallel_for_each(boost::irange<unsigned>(0, _instances.size()), [this, &vec, &mapper] (unsigned c) {
return parallel_for_each(std::views::iota(0u, _instances.size()), [this, &vec, &mapper] (unsigned c) {
return smp::submit_to(c, [this, &mapper] {
auto inst = get_local_service();
return mapper(*inst);
Expand Down
6 changes: 3 additions & 3 deletions include/seastar/core/smp.hh
Original file line number Diff line number Diff line change
Expand Up @@ -34,10 +34,10 @@
#ifndef SEASTAR_MODULE
#include <boost/lockfree/spsc_queue.hpp>
#include <boost/thread/barrier.hpp>
#include <boost/range/irange.hpp>
#include <deque>
#include <optional>
#include <thread>
#include <ranges>
#endif

/// \file
Expand Down Expand Up @@ -395,8 +395,8 @@ public:
}
static bool poll_queues();
static bool pure_poll_queues();
static boost::integer_range<unsigned> all_cpus() noexcept {
return boost::irange(0u, count);
static std::ranges::range auto all_cpus() noexcept {
return std::views::iota(0u, count);
}
/// Invokes func on all shards.
///
Expand Down
4 changes: 2 additions & 2 deletions include/seastar/net/tls.hh
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
#include <functional>
#include <unordered_set>
#include <map>
#include <boost/any.hpp>
#include <any>
#include <fmt/format.h>
#endif

Expand Down Expand Up @@ -322,7 +322,7 @@ namespace tls {
private:
friend class reloadable_credentials_base;

std::multimap<sstring, boost::any> _blobs;
std::multimap<sstring, std::any> _blobs;
client_auth _client_auth = client_auth::NONE;
session_resume_mode _session_resume_mode = session_resume_mode::NONE;
sstring _priority;
Expand Down
30 changes: 2 additions & 28 deletions include/seastar/rpc/multi_algo_compressor_factory.hh
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,6 @@

#pragma once

#include <boost/range/adaptor/transformed.hpp>
#include <boost/algorithm/string.hpp>
#include <seastar/core/sstring.hh>
#include <seastar/rpc/rpc_types.hh>

Expand All @@ -39,9 +37,7 @@ class multi_algo_compressor_factory : public rpc::compressor::factory {
sstring _features;

public:
multi_algo_compressor_factory(std::vector<const rpc::compressor::factory*> factories) : _factories(std::move(factories)) {
_features = boost::algorithm::join(_factories | boost::adaptors::transformed(std::mem_fn(&rpc::compressor::factory::supported)), sstring(","));
}
multi_algo_compressor_factory(std::vector<const rpc::compressor::factory*> factories);
multi_algo_compressor_factory(std::initializer_list<const rpc::compressor::factory*> factories) :
multi_algo_compressor_factory(std::vector<const rpc::compressor::factory*>(std::move(factories))) {}
multi_algo_compressor_factory(const rpc::compressor::factory* factory) : multi_algo_compressor_factory({factory}) {}
Expand All @@ -53,29 +49,7 @@ public:
std::unique_ptr<compressor> negotiate(sstring feature, bool is_server) const override {
return negotiate(feature, is_server, nullptr);
}
std::unique_ptr<compressor> negotiate(sstring feature, bool is_server, std::function<future<>()> send_empty_frame) const override {
std::vector<sstring> names;
boost::split(names, feature, boost::is_any_of(","));
std::unique_ptr<compressor> c;
if (is_server) {
for (auto&& n : names) {
for (auto&& f : _factories) {
if ((c = f->negotiate(n, is_server, send_empty_frame))) {
return c;
}
}
}
} else {
for (auto&& f : _factories) {
for (auto&& n : names) {
if ((c = f->negotiate(n, is_server, send_empty_frame))) {
return c;
}
}
}
}
return nullptr;
}
std::unique_ptr<compressor> negotiate(sstring feature, bool is_server, std::function<future<>()> send_empty_frame) const override;
};

}
Expand Down
2 changes: 0 additions & 2 deletions include/seastar/rpc/rpc_impl.hh
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,6 @@
#include <seastar/core/when_all.hh>
#include <seastar/util/is_smart_ptr.hh>
#include <seastar/core/simple-stream.hh>
#include <boost/range/numeric.hpp>
#include <boost/range/adaptor/transformed.hpp>
#include <seastar/net/packet-data-source.hh>
#include <seastar/core/print.hh>

Expand Down
12 changes: 6 additions & 6 deletions include/seastar/rpc/rpc_types.hh
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@
#include <seastar/net/api.hh>
#include <stdexcept>
#include <string>
#include <boost/any.hpp>
#include <any>
#include <boost/type.hpp>
#include <seastar/util/std-compat.hh>
#include <seastar/util/variant_utils.hh>
Expand Down Expand Up @@ -99,16 +99,16 @@ struct client_info {
socket_address addr;
rpc::server& server;
connection_id conn_id;
std::unordered_map<sstring, boost::any> user_data;
std::unordered_map<sstring, std::any> user_data;
template <typename T>
void attach_auxiliary(const sstring& key, T&& object) {
user_data.emplace(key, boost::any(std::forward<T>(object)));
user_data.emplace(key, std::any(std::forward<T>(object)));
}
template <typename T>
T& retrieve_auxiliary(const sstring& key) {
auto it = user_data.find(key);
assert(it != user_data.end());
return boost::any_cast<T&>(it->second);
return std::any_cast<T&>(it->second);
}
template <typename T>
std::add_const_t<T>& retrieve_auxiliary(const sstring& key) const {
Expand All @@ -120,15 +120,15 @@ struct client_info {
if (it == user_data.end()) {
return nullptr;
}
return &boost::any_cast<T&>(it->second);
return &std::any_cast<T&>(it->second);
}
template <typename T>
const T* retrieve_auxiliary_opt(const sstring& key) const noexcept {
auto it = user_data.find(key);
if (it == user_data.end()) {
return nullptr;
}
return &boost::any_cast<const T&>(it->second);
return &std::any_cast<const T&>(it->second);
}
};

Expand Down
1 change: 1 addition & 0 deletions src/core/prometheus.cc
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
#include <boost/algorithm/string.hpp>
#include <boost/range/algorithm.hpp>
#include <boost/range/combine.hpp>
#include <boost/range/irange.hpp>
#include <seastar/core/thread.hh>
#include <seastar/core/loop.hh>
#include <regex>
Expand Down
5 changes: 3 additions & 2 deletions src/net/tls.cc
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
module;
#endif

#include <any>
#include <filesystem>
#include <stdexcept>
#include <system_error>
Expand Down Expand Up @@ -703,7 +704,7 @@ static void visit_blobs(Blobs& blobs, Visitor&& visitor) {
auto visit = [&](const sstring& key, auto* vt) {
auto tr = blobs.equal_range(key);
for (auto& p : boost::make_iterator_range(tr.first, tr.second)) {
auto* v = boost::any_cast<std::decay_t<decltype(*vt)>>(&p.second);
auto* v = std::any_cast<std::decay_t<decltype(*vt)>>(&p.second);
visitor(key, *v);
}
};
Expand Down Expand Up @@ -766,7 +767,7 @@ shared_ptr<tls::server_credentials> tls::credentials_builder::build_server_crede
return creds;
#endif
}
auto creds = make_shared<server_credentials>(dh_params(boost::any_cast<dh_params::level>(i->second)));
auto creds = make_shared<server_credentials>(dh_params(std::any_cast<dh_params::level>(i->second)));
apply_to(*creds);
return creds;
}
Expand Down
Loading

0 comments on commit f322e76

Please sign in to comment.