From 0bbcbbae22e1da647e1780b20b01cd243e25853c Mon Sep 17 00:00:00 2001 From: Duarte Nunes Date: Sat, 11 Aug 2018 20:23:08 +0200 Subject: [PATCH] Add compatbility with C++17's library types This patch introduces seastar::optional, seastar::string_view, seastar::basic_string_view and seastar::variant, which alias to std::experimental::optional, std::experimental::string_view, std::experimental::basic_string_view and boost::variant, respectively, if compiling Sestar with C++14 support. If being compiled with C++17 support, they alias the std types. This enables projects on C++17 to not have to work with experimental or external features. Due to [1], C++17 mode requires GCC 7.5 or GCC 8.1.1-5. [1] https://gcc.gnu.org/bugzilla/show_bug.cgi?id=84769 Signed-off-by: Duarte Nunes Message-Id: <20180811182309.19727-1-duarte@scylladb.com> --- CMakeLists.txt | 8 ++ README.md | 12 +++ apps/iotune/iotune.cc | 3 +- apps/memcached/memcache.cc | 12 +-- configure.py | 5 + core/abort_source.hh | 8 +- core/dpdk_rte.cc | 4 +- core/execution_stage.hh | 4 +- core/expiring_fifo.hh | 4 +- core/fair_queue.hh | 1 - core/file.hh | 4 +- core/fstream.cc | 2 +- core/future-util.hh | 20 ++-- core/gate.hh | 8 +- core/iostream-impl.hh | 8 +- core/iostream.hh | 15 ++- core/memory.cc | 10 +- core/memory.hh | 2 +- core/metrics.cc | 4 +- core/metrics.hh | 12 +-- core/pipe.hh | 14 +-- core/queue.hh | 14 +-- core/ragel.hh | 4 +- core/reactor.cc | 28 +++--- core/reactor.hh | 20 ++-- core/report_exception.hh | 4 +- core/resource.hh | 4 +- core/scattered_message.hh | 4 +- core/sstring.hh | 10 +- core/thread.cc | 6 +- core/thread.hh | 10 +- core/thread_impl.hh | 4 +- core/timer.hh | 10 +- doc/rpc-streaming.md | 4 +- doc/tutorial.md | 12 +-- http/httpd.hh | 10 +- http/transformers.cc | 1 - net/api.hh | 4 +- net/arp.cc | 4 +- net/arp.hh | 2 +- net/config.hh | 7 +- net/dns.cc | 8 +- net/dns.hh | 14 +-- net/dpdk.cc | 28 +++--- net/ip.cc | 4 +- net/ip.hh | 6 +- net/net.cc | 4 +- net/net.hh | 6 +- net/packet.hh | 10 +- net/posix-stack.cc | 4 +- net/tcp.hh | 24 ++--- net/tls.cc | 11 +- net/tls.hh | 4 +- net/udp.cc | 2 +- rpc/lz4_compressor.cc | 8 +- rpc/rpc.cc | 62 ++++++------ rpc/rpc.hh | 30 +++--- rpc/rpc_impl.hh | 26 ++--- rpc/rpc_types.hh | 29 +++--- tests/exchanger.hh | 4 +- tests/fstream_test.cc | 4 +- tests/futures_test.cc | 7 +- tests/l3_test.cc | 2 +- tests/linecount.cc | 2 +- tests/perf/perf_tests.hh | 4 +- tests/sstring_test.cc | 2 +- tests/thread_context_switch.cc | 1 - util/optimized_optional.hh | 10 +- util/std-compat.hh | 178 +++++++++++++++++++++++++++++++++ util/variant_utils.hh | 8 +- 70 files changed, 508 insertions(+), 316 deletions(-) create mode 100644 util/std-compat.hh diff --git a/CMakeLists.txt b/CMakeLists.txt index 05297b751e2..4a3a9c3df6e 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -137,6 +137,10 @@ set (SEASTAR_JENKINS STRING "Configure the build and tests for execution within a Jenkins context, with the given identifier.") +option (SEASTAR_USE_STD_OPTIONAL_VARIANT_STRINGVIEW + "Use C++17 std types for optional, variant, and string_view. Requires C++17 dialect and GCC >= 8.1.1-5." + OFF) + # # Link pools (only supported for the Ninja generator). # @@ -577,6 +581,10 @@ if (${SEASTAR_ENABLE_ALLOC_FAILURE_INJECTOR}) SEASTAR_ENABLE_ALLOC_FAILURE_INJECTION) endif () +if ((${SEASTAR_USE_STD_OPTIONAL_VARIANT_STRINGVIEW} EQUAL 1) AND (${SEASTAR_CXX_DIALECT} STREQUAL "gnu++17")) + target_compile_definitions (seastar PUBLIC SEASTAR_USE_STD_OPTIONAL_VARIANT_STRINGVIEW) +endif () + ## ## Code tests. ## diff --git a/README.md b/README.md index 008d8846a90..5ffeccf98b7 100644 --- a/README.md +++ b/README.md @@ -17,6 +17,18 @@ There are also instructions for building on any host that supports [Docker](doc/ Use of the [DPDK](http://dpdk.org) is [optional](doc/building-dpdk.md). +#### Using C++17 + +Seastar can be built with the C++17 dialect by supporting compilers, conditional +on the `--c++-dialect` option being set to `gnu++17`. + +However, by default Seastar uses C++14-compatible types such as +`std::experimental::optional<>` or `boost::variant`, both internally and in its public +API, thus forcing them on C++17 projects. To fix this, Seastar provides the `--use-std-optional-variant-stringview 0|1` +option, which changes those types to their `stdlib` incarnation, and allows +seemless use of C++17. Usage of this option requires an updated compiler, such +as GCC 8.1.1-5 on Fedora. + Getting started --------------- diff --git a/apps/iotune/iotune.cc b/apps/iotune/iotune.cc index 81cc6285b55..1da2c3a466a 100644 --- a/apps/iotune/iotune.cc +++ b/apps/iotune/iotune.cc @@ -47,6 +47,7 @@ #include "core/fsqual.hh" #include "util/defer.hh" #include "util/log.hh" +#include "util/std-compat.hh" using namespace seastar; using namespace std::chrono_literals; @@ -561,7 +562,7 @@ void write_property_file(sstring conf_file, struct std::vector // (absolute, with symlinks resolved), until we find a point that crosses a device ID. fs::path mountpoint_of(sstring filename) { fs::path mnt_candidate = fs::canonical(fs::path(filename)); - std::experimental::optional candidate_id = {}; + compat::optional candidate_id = {}; auto current = mnt_candidate; do { auto f = open_directory(current.string()).get0(); diff --git a/apps/memcached/memcache.cc b/apps/memcached/memcache.cc index 4914739fb94..defe89773e3 100644 --- a/apps/memcached/memcache.cc +++ b/apps/memcached/memcache.cc @@ -178,19 +178,19 @@ class item : public slab_item_base { return _version; } - const std::experimental::string_view key() const { - return std::experimental::string_view(_data, _key_size); + const compat::string_view key() const { + return compat::string_view(_data, _key_size); } - const std::experimental::string_view ascii_prefix() const { + const compat::string_view ascii_prefix() const { const char *p = _data + align_up(_key_size, field_alignment); - return std::experimental::string_view(p, _ascii_prefix_size); + return compat::string_view(p, _ascii_prefix_size); } - const std::experimental::string_view value() const { + const compat::string_view value() const { const char *p = _data + align_up(_key_size, field_alignment) + align_up(_ascii_prefix_size, field_alignment); - return std::experimental::string_view(p, _value_size); + return compat::string_view(p, _value_size); } size_t key_size() const { diff --git a/configure.py b/configure.py index 22ec5bb802f..9f6c6c102e5 100755 --- a/configure.py +++ b/configure.py @@ -381,6 +381,8 @@ def configure_fmt(mode, cxx='g++', cc='gcc'): help='Use CMake as the underlying build-sytem') arg_parser.add_argument('--without-tests', dest='exclude_tests', action='store_true', help='Do not build tests by default (CMake only)') arg_parser.add_argument('--without-apps', dest='exclude_apps', action='store_true', help='Do not build applications by default (CMake only)') +arg_parser.add_argument('--use-std-optional-variant-stringview', dest='cpp17_goodies', action='store', type=int, default=0, + help='Use C++17 std types for optional, variant, and string_view. Requires C++17 dialect and GCC >= 8.1.1-5') args = arg_parser.parse_args() # Forwarding to CMake. @@ -411,6 +413,7 @@ def configure_mode(mode): tr(args.alloc_failure_injector, 'ENABLE_ALLOC_FAILURE_INJECTOR'), tr(args.exception_workaround, 'ENABLE_EXCEPTION_SCALABILITY_WORKAROUND'), tr(args.allocator_page_size, 'ALLOCATOR_PAGE_SIZE'), + tr(args.cpp17_goodies, 'USE_STD_OPTIONAL_VARIANT_STRINGVIEW'), ] # Generate a new build by pointing to the source directory. @@ -518,6 +521,8 @@ def maybe_static(flag, libs): boost_unit_test_lib = maybe_static(args.staticboost, '-lboost_unit_test_framework') +if args.cpp17_goodies is 1 and args.cpp_dialect == 'gnu++17': + defines.append('SEASTAR_USE_STD_OPTIONAL_VARIANT_STRINGVIEW') hwloc_libs = '-lhwloc -lnuma -lpciaccess -lxml2 -lz' diff --git a/core/abort_source.hh b/core/abort_source.hh index e8248b57db1..5929887e41b 100644 --- a/core/abort_source.hh +++ b/core/abort_source.hh @@ -23,18 +23,16 @@ #include "util/noncopyable_function.hh" #include "util/optimized_optional.hh" +#include "util/std-compat.hh" #include #include -#include namespace bi = boost::intrusive; namespace seastar { -namespace stdx = std::experimental; - /// \addtogroup fiber-module /// @{ @@ -102,7 +100,7 @@ public: private: using subscription_list_type = bi::list>; - stdx::optional _subscriptions = subscription_list_type(); + compat::optional _subscriptions = subscription_list_type(); public: /// Delays the invocation of the callback \c f until \ref request_abort() is called. @@ -139,4 +137,4 @@ public: /// @} -} \ No newline at end of file +} diff --git a/core/dpdk_rte.cc b/core/dpdk_rte.cc index fba0b5af8a4..ced5aec68e7 100644 --- a/core/dpdk_rte.cc +++ b/core/dpdk_rte.cc @@ -21,7 +21,7 @@ #include "net/dpdk.hh" #include "core/dpdk_rte.hh" #include "util/conversions.hh" -#include +#include "util/std-compat.hh" #include namespace seastar { @@ -46,7 +46,7 @@ void eal::init(cpuset cpus, boost::program_options::variables_map opts) string2vector("-n"), string2vector("1") }; - std::experimental::optional hugepages_path; + compat::optional hugepages_path; if (opts.count("hugepages")) { hugepages_path = opts["hugepages"].as(); } diff --git a/core/execution_stage.hh b/core/execution_stage.hh index eba024ee331..5897b176fd2 100644 --- a/core/execution_stage.hh +++ b/core/execution_stage.hh @@ -32,9 +32,9 @@ #include "util/noncopyable_function.hh" #include "../util/tuple_utils.hh" #include "../util/defer.hh" +#include "util/std-compat.hh" #include "../fmt/fmt/format.h" #include -#include #include #include @@ -313,7 +313,7 @@ class inheriting_concrete_execution_stage final { sstring _name; noncopyable_function _function; - std::vector> _stage_for_group{max_scheduling_groups()}; + std::vector> _stage_for_group{max_scheduling_groups()}; private: per_group_stage_type make_stage_for_group(scheduling_group sg) { // We can't use std::ref(function), because reference_wrapper decays to noncopyable_function& and diff --git a/core/expiring_fifo.hh b/core/expiring_fifo.hh index 63fe96259be..4f23eed5042 100644 --- a/core/expiring_fifo.hh +++ b/core/expiring_fifo.hh @@ -58,7 +58,7 @@ public: using time_point = typename Clock::time_point; private: struct entry { - std::experimental::optional payload; // disengaged means that it's expired + compat::optional payload; // disengaged means that it's expired timer tr; entry(T&& payload_) : payload(std::move(payload_)) {} entry(const T& payload_) : payload(payload_) {} @@ -66,7 +66,7 @@ private: : payload(std::move(payload_)) , tr([this, &ef] { ef._on_expiry(*payload); - payload = std::experimental::nullopt; + payload = compat::nullopt; --ef._size; ef.drop_expired_front(); }) diff --git a/core/fair_queue.hh b/core/fair_queue.hh index b5bfe4dc4c3..83224b80b09 100644 --- a/core/fair_queue.hh +++ b/core/fair_queue.hh @@ -29,7 +29,6 @@ #include "util/noncopyable_function.hh" #include #include -#include #include #include #include diff --git a/core/file.hh b/core/file.hh index 006468d5401..6a9a688ed64 100644 --- a/core/file.hh +++ b/core/file.hh @@ -27,7 +27,7 @@ #include "core/align.hh" #include "core/future-util.hh" #include "core/fair_queue.hh" -#include +#include "util/std-compat.hh" #include #include #include @@ -70,7 +70,7 @@ struct directory_entry { /// Name of the file in a directory entry. Will never be "." or "..". Only the last component is included. sstring name; /// Type of the directory entry, if known. - std::experimental::optional type; + compat::optional type; }; /// File open options diff --git a/core/fstream.cc b/core/fstream.cc index 71a7a0ad103..e5a846f473b 100644 --- a/core/fstream.cc +++ b/core/fstream.cc @@ -48,7 +48,7 @@ class file_data_source_impl : public data_source_impl { unsigned _reads_in_progress = 0; unsigned _current_read_ahead; future<> _dropped_reads = make_ready_future<>(); - std::experimental::optional> _done; + compat::optional> _done; size_t _current_buffer_size; bool _in_slow_start = false; using unused_ratio_target = std::ratio<25, 100>; diff --git a/core/future-util.hh b/core/future-util.hh index a10c4604690..987a5c63204 100644 --- a/core/future-util.hh +++ b/core/future-util.hh @@ -33,7 +33,7 @@ #include #include #include -#include +#include "util/std-compat.hh" #include "util/tuple_utils.hh" #include "util/noncopyable_function.hh" @@ -92,7 +92,7 @@ with_scheduling_group(scheduling_group sg, Func func, Args&&... args) { struct parallel_for_each_state { // use optional<> to avoid out-of-line constructor - std::experimental::optional ex; + compat::optional ex; promise<> pr; ~parallel_for_each_state() { if (ex) { @@ -312,11 +312,11 @@ struct repeat_until_value_type_helper; /// Type helper for repeat_until_value() template -struct repeat_until_value_type_helper>> { +struct repeat_until_value_type_helper>> { /// The type of the value we are computing using value_type = T; /// Type used by \c AsyncAction while looping - using optional_type = std::experimental::optional; + using optional_type = compat::optional; /// Return type of repeat_until_value() using future_type = future; /// Return type of \c AsyncAction @@ -331,12 +331,12 @@ using repeat_until_value_return_type namespace internal { template -class repeat_until_value_state final : public continuation_base> { +class repeat_until_value_state final : public continuation_base> { promise _promise; AsyncAction _action; public: explicit repeat_until_value_state(AsyncAction action) : _action(std::move(action)) {} - repeat_until_value_state(std::experimental::optional st, AsyncAction action) : repeat_until_value_state(std::move(action)) { + repeat_until_value_state(compat::optional st, AsyncAction action) : repeat_until_value_state(std::move(action)) { this->_state.set(std::make_tuple(std::move(st))); } future get_future() { return _promise.get_future(); } @@ -370,7 +370,7 @@ public: _promise.set_exception(std::current_exception()); return; } - this->_state.set(std::experimental::nullopt); + this->_state.set(compat::nullopt); schedule(std::move(zis)); } }; @@ -378,10 +378,10 @@ public: } /// Invokes given action until it fails or the function requests iteration to stop by returning -/// an engaged \c future>. The value is extracted from the +/// an engaged \c future>. The value is extracted from the /// \c optional, and returned, as a future, from repeat_until_value(). /// -/// \param action a callable taking no arguments, returning a future>. +/// \param action a callable taking no arguments, returning a future>. /// Will be called again as soon as the future resolves, unless the /// future fails, action throws, or it resolves with an engaged \c optional. /// If \c action is an r-value it can be moved in the middle of iteration. @@ -425,7 +425,7 @@ repeat_until_value(AsyncAction action) { } while (!need_preempt()); try { - auto state = std::make_unique>(std::experimental::nullopt, std::move(futurized_action)); + auto state = std::make_unique>(compat::nullopt, std::move(futurized_action)); auto f = state->get_future(); schedule(std::move(state)); return f; diff --git a/core/gate.hh b/core/gate.hh index d3d25e99b30..9991330cef4 100644 --- a/core/gate.hh +++ b/core/gate.hh @@ -22,7 +22,7 @@ #pragma once #include "future.hh" -#include +#include "util/std-compat.hh" #include namespace seastar { @@ -30,8 +30,6 @@ namespace seastar { /// \addtogroup fiber-module /// @{ -namespace stdx = std::experimental; - /// Exception thrown when a \ref gate object has been closed /// by the \ref gate::close() method. class gate_closed_exception : public std::exception { @@ -48,7 +46,7 @@ public: /// requests have completed. The \c gate class provides a solution. class gate { size_t _count = 0; - stdx::optional> _stopped; + compat::optional> _stopped; public: /// Registers an in-progress request. /// @@ -91,7 +89,7 @@ public: /// made ready. future<> close() { assert(!_stopped && "seastar::gate::close() cannot be called more than once"); - _stopped = stdx::make_optional(promise<>()); + _stopped = compat::make_optional(promise<>()); if (!_count) { _stopped->set_value(); } diff --git a/core/iostream-impl.hh b/core/iostream-impl.hh index 991f71b0d2f..55e0a5794cf 100644 --- a/core/iostream-impl.hh +++ b/core/iostream-impl.hh @@ -34,7 +34,7 @@ inline future> data_source_impl::skip(uint64_t n) { return do_with(uint64_t(n), [this] (uint64_t& n) { return repeat_until_value([&] { - return get().then([&] (temporary_buffer buffer) -> std::experimental::optional> { + return get().then([&] (temporary_buffer buffer) -> compat::optional> { if (buffer.size() >= n) { buffer.trim_front(n); return std::move(buffer); @@ -206,7 +206,7 @@ input_stream::consume(Consumer&& consumer) { }); } return consumer(std::move(_buf)).then([this] (consumption_result_type result) { - return visit(result.get(), [this] (const continue_consuming&) { + return seastar::visit(result.get(), [this] (const continue_consuming&) { // If we're here, consumer consumed entire buffer and is ready for // more now. So we do not return, and rather continue the loop. // @@ -429,7 +429,7 @@ output_stream::poll_flush() { // flush was canceled, do nothing _flushing = false; _in_batch.value().set_value(); - _in_batch = std::experimental::nullopt; + _in_batch = compat::nullopt; return; } @@ -495,7 +495,7 @@ template struct stream_copy_consumer { private: output_stream& _os; - using unconsumed_remainder = std::experimental::optional>; + using unconsumed_remainder = compat::optional>; public: stream_copy_consumer(output_stream& os) : _os(os) { } diff --git a/core/iostream.hh b/core/iostream.hh index 66565f25cba..3d2ce2fb05e 100644 --- a/core/iostream.hh +++ b/core/iostream.hh @@ -38,13 +38,10 @@ #include "future.hh" #include "temporary_buffer.hh" #include "scattered_message.hh" - -#include +#include "util/std-compat.hh" namespace seastar { -namespace stdx = std::experimental; - namespace net { class packet; } class data_source_impl { @@ -144,10 +141,10 @@ template class consumption_result { public: using stop_consuming_type = stop_consuming; - using consumption_variant = boost::variant; + using consumption_variant = compat::variant; using tmp_buf = typename stop_consuming_type::tmp_buf; - /*[[deprecated]]*/ consumption_result(stdx::optional opt_buf) { + /*[[deprecated]]*/ consumption_result(compat::optional opt_buf) { if (opt_buf) { _result = stop_consuming_type{std::move(opt_buf.value())}; } @@ -191,7 +188,7 @@ concept bool InputStreamConsumer = requires (Consumer c) { template concept bool ObsoleteInputStreamConsumer = requires (Consumer c) { - { c(temporary_buffer{}) } -> future>>; + { c(temporary_buffer{}) } -> future>>; }; ) @@ -210,7 +207,7 @@ protected: public: using consumption_result_type = consumption_result; // unconsumed_remainder is mapped for compatibility only; new code should use consumption_result_type - using unconsumed_remainder = stdx::optional; + using unconsumed_remainder = compat::optional; using char_type = CharType; input_stream() = default; explicit input_stream(data_source fd) : _fd(std::move(fd)), _buf(0) {} @@ -280,7 +277,7 @@ class output_stream final { size_t _end = 0; bool _trim_to_size = false; bool _batch_flushes = false; - std::experimental::optional> _in_batch; + compat::optional> _in_batch; bool _flush = false; bool _flushing = false; std::exception_ptr _ex; diff --git a/core/memory.cc b/core/memory.cc index fcd1a551fb7..6aa6639cc55 100644 --- a/core/memory.cc +++ b/core/memory.cc @@ -88,7 +88,7 @@ disable_abort_on_alloc_failure_temporarily::~disable_abort_on_alloc_failure_temp #include #include #include -#include +#include "util/std-compat.hh" #include #include #include @@ -158,7 +158,7 @@ static thread_local uint64_t g_cross_cpu_frees; static thread_local uint64_t g_reclaims; static thread_local uint64_t g_large_allocs; -using std::experimental::optional; +using compat::optional; using allocate_system_memory_fn = std::function where, size_t how_much)>; @@ -887,7 +887,7 @@ bool cpu_pages::initialize() { } mmap_area -allocate_anonymous_memory(std::experimental::optional where, size_t how_much) { +allocate_anonymous_memory(compat::optional where, size_t how_much) { return mmap_anonymous(where.value_or(nullptr), how_much, PROT_READ | PROT_WRITE, @@ -895,7 +895,7 @@ allocate_anonymous_memory(std::experimental::optional where, size_t how_m } mmap_area -allocate_hugetlbfs_memory(file_desc& fd, std::experimental::optional where, size_t how_much) { +allocate_hugetlbfs_memory(file_desc& fd, compat::optional where, size_t how_much) { auto pos = fd.size(); fd.truncate(pos + how_much); auto ret = fd.map( @@ -1853,7 +1853,7 @@ reclaimer::~reclaimer() { void set_reclaim_hook(std::function)> hook) { } -void configure(std::vector m, bool mbind, std::experimental::optional hugepages_path) { +void configure(std::vector m, bool mbind, compat::optional hugepages_path) { } statistics stats() { diff --git a/core/memory.hh b/core/memory.hh index abde7075db6..d9cf85250a1 100644 --- a/core/memory.hh +++ b/core/memory.hh @@ -74,7 +74,7 @@ static constexpr size_t huge_page_size = #endif void configure(std::vector m, bool mbind, - std::experimental::optional hugetlbfs_path = {}); + compat::optional hugetlbfs_path = {}); void enable_abort_on_allocation_failure(); diff --git a/core/metrics.cc b/core/metrics.cc index c56e480ad05..a2b5077bee8 100644 --- a/core/metrics.cc +++ b/core/metrics.cc @@ -125,10 +125,10 @@ metric_value metric_value::operator+(const metric_value& c) { metric_value res(*this); switch (_type) { case data_type::HISTOGRAM: - boost::get(res.u) += boost::get(c.u); + compat::get(res.u) += compat::get(c.u); break; default: - boost::get(res.u) += boost::get(c.u); + compat::get(res.u) += compat::get(c.u); break; } return res; diff --git a/core/metrics.hh b/core/metrics.hh index f8b4ec4c9a6..ccff71c190c 100644 --- a/core/metrics.hh +++ b/core/metrics.hh @@ -28,7 +28,7 @@ #include #include #include "core/metrics_types.hh" -#include +#include "util/std-compat.hh" /*! \file metrics.hh * \brief header for metrics creation. @@ -257,22 +257,22 @@ enum class data_type : uint8_t { * Do not use directly @see metrics_creation */ struct metric_value { - boost::variant u; + compat::variant u; data_type _type; data_type type() const { return _type; } double d() const { - return boost::get(u); + return compat::get(u); } uint64_t ui() const { - return boost::get(u); + return compat::get(u); } int64_t i() const { - return boost::get(u); + return compat::get(u); } metric_value() @@ -299,7 +299,7 @@ struct metric_value { metric_value operator+(const metric_value& c); const histogram& get_histogram() const { - return boost::get(u); + return compat::get(u); } }; diff --git a/core/pipe.hh b/core/pipe.hh index 606916e178b..2a82a2e24d2 100644 --- a/core/pipe.hh +++ b/core/pipe.hh @@ -24,7 +24,7 @@ #include "future.hh" #include "queue.hh" -#include +#include "util/std-compat.hh" /// \defgroup fiber-module Fibers /// @@ -82,12 +82,12 @@ namespace internal { template class pipe_buffer { private: - queue> _buf; + queue> _buf; bool _read_open = true; bool _write_open = true; public: pipe_buffer(size_t size) : _buf(size) {} - future> read() { + future> read() { return _buf.pop_eventually(); } future<> write(T&& data) { @@ -134,7 +134,7 @@ template class pipe_reader { private: internal::pipe_buffer *_bufp; - std::experimental::optional _unread; + compat::optional _unread; pipe_reader(internal::pipe_buffer *bufp) : _bufp(bufp) { } friend class pipe; public: @@ -144,16 +144,16 @@ public: /// becomes non-empty, or the write side is closed. The value returned /// is an optional, which is disengaged to mark and end of file /// (i.e., the write side was closed, and we've read everything it sent). - future> read() { + future> read() { if (_unread) { auto ret = std::move(*_unread); _unread = {}; - return make_ready_future>(std::move(ret)); + return make_ready_future>(std::move(ret)); } if (_bufp->readable()) { return _bufp->read(); } else { - return make_ready_future>(); + return make_ready_future>(); } } /// \brief Return an item to the front of the pipe diff --git a/core/queue.hh b/core/queue.hh index ca21405b060..2d024d3dfdf 100644 --- a/core/queue.hh +++ b/core/queue.hh @@ -24,7 +24,7 @@ #include "circular_buffer.hh" #include "future.hh" #include -#include +#include "util/std-compat.hh" namespace seastar { @@ -32,8 +32,8 @@ template class queue { std::queue> _q; size_t _max; - std::experimental::optional> _not_empty; - std::experimental::optional> _not_full; + compat::optional> _not_empty; + compat::optional> _not_full; std::exception_ptr _ex = nullptr; private: void notify_not_empty(); @@ -110,11 +110,11 @@ public: _ex = ex; if (_not_full) { _not_full->set_exception(ex); - _not_full= std::experimental::nullopt; + _not_full= compat::nullopt; } if (_not_empty) { _not_empty->set_exception(std::move(ex)); - _not_empty = std::experimental::nullopt; + _not_empty = compat::nullopt; } } }; @@ -130,7 +130,7 @@ inline void queue::notify_not_empty() { if (_not_empty) { _not_empty->set_value(); - _not_empty = std::experimental::optional>(); + _not_empty = compat::optional>(); } } @@ -139,7 +139,7 @@ inline void queue::notify_not_full() { if (_not_full) { _not_full->set_value(); - _not_full = std::experimental::optional>(); + _not_full = compat::optional>(); } } diff --git a/core/ragel.hh b/core/ragel.hh index 4f21237077a..dd9ab9e057f 100644 --- a/core/ragel.hh +++ b/core/ragel.hh @@ -27,7 +27,7 @@ #include #include #include -#include +#include "util/std-compat.hh" #include "future.hh" namespace seastar { @@ -124,7 +124,7 @@ protected: return std::move(s); } public: - using unconsumed_remainder = std::experimental::optional>; + using unconsumed_remainder = compat::optional>; future operator()(temporary_buffer buf) { char* p = buf.get_write(); char* pe = p + buf.size(); diff --git a/core/reactor.cc b/core/reactor.cc index 3c73321fba3..4b93012a17b 100644 --- a/core/reactor.cc +++ b/core/reactor.cc @@ -2016,7 +2016,7 @@ directory_entry_type stat_to_entry_type(__mode_t type) { } -future> +future> reactor::file_type(sstring name) { return _thread_pool.submit>([name] { struct stat st; @@ -2027,11 +2027,11 @@ reactor::file_type(sstring name) { if (sr.error != ENOENT && sr.error != ENOTDIR) { sr.throw_fs_exception_if_error("stat failed", name); } - return make_ready_future > - (std::experimental::optional() ); + return make_ready_future > + (compat::optional() ); } - return make_ready_future > - (std::experimental::optional(stat_to_entry_type(sr.extra.st_mode)) ); + return make_ready_future > + (compat::optional(stat_to_entry_type(sr.extra.st_mode)) ); }); } @@ -2389,7 +2389,7 @@ posix_file_impl::list_directory(std::function (directory_entry de)> nex } auto start = w->buffer + w->current; auto de = reinterpret_cast(start); - std::experimental::optional type; + compat::optional type; switch (de->d_type) { case DT_BLK: type = directory_entry_type::block_device; @@ -3118,9 +3118,9 @@ int reactor::run() { register_metrics(); - std::experimental::optional io_poller = {}; - std::experimental::optional aio_poller = {}; - std::experimental::optional smp_poller = {}; + compat::optional io_poller = {}; + compat::optional aio_poller = {}; + compat::optional smp_poller = {}; // I/O Performance greatly increases if the smp poller runs before the I/O poller. This is // because requests that were just added can be polled and processed by the I/O poller right @@ -3859,7 +3859,7 @@ thread_local std::unique_ptr reactor_holder; std::vector smp::_threads; std::vector> smp::_thread_loops; -std::experimental::optional smp::_all_event_loops_done; +compat::optional smp::_all_event_loops_done; std::vector smp::_reactors; std::unique_ptr smp::_qs; std::thread::id smp::_tmain; @@ -4022,7 +4022,7 @@ class disk_config_params { throw std::runtime_error("Both io-properties and io-properties-file specified. Don't know which to trust!"); } - std::experimental::optional doc; + compat::optional doc; if (configuration.count("io-properties-file")) { doc = YAML::LoadFile(configuration["io-properties-file"].as()); } else if (configuration.count("io-properties")) { @@ -4170,7 +4170,7 @@ void smp::configure(boost::program_options::variables_map configuration) if (configuration.count("reserve-memory")) { rc.reserve_memory = parse_memory_size(configuration["reserve-memory"].as()); } - std::experimental::optional hugepages_path; + compat::optional hugepages_path; if (configuration.count("hugepages")) { hugepages_path = configuration["hugepages"].as(); } @@ -4499,7 +4499,7 @@ reactor_backend_osv::enable_timer(steady_clock_type::time_point when) { #endif -void report_exception(std::experimental::string_view message, std::exception_ptr eptr) noexcept { +void report_exception(compat::string_view message, std::exception_ptr eptr) noexcept { seastar_logger.error("{}: {}", message, eptr); } @@ -4527,7 +4527,7 @@ future<> check_direct_io_support(sstring path) { open_flags flags; std::function()> cleanup; - static w parse(sstring path, std::experimental::optional type) { + static w parse(sstring path, compat::optional type) { if (!type) { throw std::invalid_argument(sprint("Could not open file at %s. Make sure it exists", path)); } diff --git a/core/reactor.hh b/core/reactor.hh index b0779ff10ac..f4b58f3544b 100644 --- a/core/reactor.hh +++ b/core/reactor.hh @@ -47,7 +47,7 @@ #include #include #include -#include +#include "util/std-compat.hh" #include #include #include @@ -259,7 +259,7 @@ class syscall_work_queue { struct work_item_returning : work_item { Func _func; promise _promise; - boost::optional _result; + compat::optional _result; work_item_returning(Func&& func) : _func(std::move(func)) {} virtual void process() override { _result = this->_func(); } virtual void complete() override { _promise.set_value(std::move(*_result)); } @@ -334,7 +334,7 @@ class smp_message_queue { using futurator = futurize>; using future_type = typename futurator::type; using value_type = typename future_type::value_type; - std::experimental::optional _result; + compat::optional _result; std::exception_ptr _ex; // if !_result typename futurator::promise_type _promise; // used on local side async_work_item(smp_message_queue& queue, Func&& func) : _queue(queue), _func(std::move(func)) {} @@ -830,8 +830,8 @@ private: // _lowres_clock_impl will only be created on cpu 0 std::unique_ptr _lowres_clock_impl; lowres_clock::time_point _lowres_next_timeout; - std::experimental::optional _epoll_poller; - std::experimental::optional _aio_eventfd; + compat::optional _epoll_poller; + compat::optional _aio_eventfd; const bool _reuseport; circular_buffer _loads; double _load = 0; @@ -973,7 +973,7 @@ public: future open_directory(sstring name); future<> make_directory(sstring name); future<> touch_directory(sstring name); - future> file_type(sstring name); + future> file_type(sstring name); future file_size(sstring pathname); future file_exists(sstring pathname); future file_system_at(sstring pathname); @@ -1189,7 +1189,7 @@ inline bool engine_is_ready() { class smp { static std::vector _threads; static std::vector> _thread_loops; // for dpdk - static std::experimental::optional _all_event_loops_done; + static compat::optional _all_event_loops_done; static std::vector _reactors; struct qs_deleter { void operator()(smp_message_queue** qs) const; @@ -1527,7 +1527,7 @@ void timer::set_callback(callback_t&& callback) { template inline -void timer::arm_state(time_point until, std::experimental::optional period) { +void timer::arm_state(time_point until, compat::optional period) { assert(!_armed); _period = period; _armed = true; @@ -1538,14 +1538,14 @@ void timer::arm_state(time_point until, std::experimental::optional inline -void timer::arm(time_point until, std::experimental::optional period) { +void timer::arm(time_point until, compat::optional period) { arm_state(until, period); engine().add_timer(this); } template inline -void timer::rearm(time_point until, std::experimental::optional period) { +void timer::rearm(time_point until, compat::optional period) { if (_armed) { cancel(); } diff --git a/core/report_exception.hh b/core/report_exception.hh index af2de3b6e79..677c5a53124 100644 --- a/core/report_exception.hh +++ b/core/report_exception.hh @@ -21,11 +21,11 @@ #pragma once -#include +#include "util/std-compat.hh" namespace seastar { -void report_exception(std::experimental::string_view message, std::exception_ptr) noexcept; +void report_exception(compat::string_view message, std::exception_ptr) noexcept; } diff --git a/core/resource.hh b/core/resource.hh index bebff2cbdc1..9010f037218 100644 --- a/core/resource.hh +++ b/core/resource.hh @@ -23,7 +23,7 @@ #include #include -#include +#include "util/std-compat.hh" #include #include #include @@ -35,7 +35,7 @@ cpu_set_t cpuid_to_cpuset(unsigned cpuid); namespace resource { -using std::experimental::optional; +using compat::optional; using cpuset = std::set; diff --git a/core/scattered_message.hh b/core/scattered_message.hh index 3eaaec403de..c4d21bf457a 100644 --- a/core/scattered_message.hh +++ b/core/scattered_message.hh @@ -27,7 +27,7 @@ #include "sstring.hh" #include #include -#include +#include "util/std-compat.hh" namespace seastar { @@ -63,7 +63,7 @@ public: append_static(s.begin(), s.size()); } - void append_static(const std::experimental::string_view& s) { + void append_static(const compat::string_view& s) { append_static(s.data(), s.size()); } diff --git a/core/sstring.hh b/core/sstring.hh index dfc46796b5e..67154d82c01 100644 --- a/core/sstring.hh +++ b/core/sstring.hh @@ -33,7 +33,7 @@ #include #include #include -#include +#include "util/std-compat.hh" #include "core/temporary_buffer.hh" namespace seastar { @@ -248,7 +248,7 @@ public: : basic_sstring(initialized_later(), std::distance(first, last)) { std::copy(first, last, begin()); } - explicit basic_sstring(std::experimental::basic_string_view v) + explicit basic_sstring(compat::basic_string_view v) : basic_sstring(v.data(), v.size()) { } ~basic_sstring() noexcept { @@ -595,8 +595,8 @@ public: return str()[pos]; } - operator std::experimental::basic_string_view() const { - return std::experimental::basic_string_view(str(), size()); + operator compat::basic_string_view() const { + return compat::basic_string_view(str(), size()); } template @@ -681,7 +681,7 @@ namespace std { template struct hash> { size_t operator()(const seastar::basic_sstring& s) const { - return std::hash>()(s); + return std::hash>()(s); } }; diff --git a/core/thread.cc b/core/thread.cc index e4ac13cf55d..21741f18bf1 100644 --- a/core/thread.cc +++ b/core/thread.cc @@ -258,7 +258,7 @@ thread_context::yield() { auto fut = _sched_promise->get_future(); _sched_timer.arm(*when); fut.get(); - _sched_promise = stdx::nullopt; + _sched_promise = compat::nullopt; } else if (need_preempt()) { later().get(); } @@ -376,12 +376,12 @@ thread_scheduling_group::account_stop() { _this_period_remain -= thread_clock::now() - _this_run_start; } -stdx::optional +compat::optional thread_scheduling_group::next_scheduling_point() const { auto now = thread_clock::now(); auto current_remain = _this_period_remain - (now - _this_run_start); if (current_remain > std::chrono::nanoseconds(0)) { - return stdx::nullopt; + return compat::nullopt; } return _this_period_ends - current_remain; diff --git a/core/thread.hh b/core/thread.hh index 3d163d434ac..6a462474b1e 100644 --- a/core/thread.hh +++ b/core/thread.hh @@ -33,7 +33,7 @@ #include #include #include -#include +#include "util/std-compat.hh" #include #include @@ -70,8 +70,6 @@ /// Seastar API namespace namespace seastar { -namespace stdx = std::experimental; - /// \addtogroup thread-module /// @{ @@ -83,7 +81,7 @@ class thread_scheduling_group; class thread_attributes { public: thread_scheduling_group* scheduling_group = nullptr; // FIXME: remove - stdx::optional sched_group; + compat::optional sched_group; }; @@ -113,7 +111,7 @@ class thread_context { promise<> _done; bool _joined = false; timer<> _sched_timer{[this] { reschedule(); }}; - stdx::optional> _sched_promise; + compat::optional> _sched_promise; boost::intrusive::list_member_hook<> _preempted_link; using preempted_thread_list = boost::intrusive::list next_scheduling_point() const; + compat::optional next_scheduling_point() const; friend class thread_context; }; diff --git a/core/thread_impl.hh b/core/thread_impl.hh index ecd662530ea..c1cd01e184f 100644 --- a/core/thread_impl.hh +++ b/core/thread_impl.hh @@ -25,7 +25,7 @@ #include #include #include -#include +#include "util/std-compat.hh" namespace seastar { /// Clock used for scheduling threads @@ -46,7 +46,7 @@ struct jmp_buf_link { #endif jmp_buf_link* link; thread_context* thread; - std::experimental::optional> yield_at = {}; + compat::optional> yield_at = {}; public: void initial_switch_in(ucontext_t* initial_context, const void* stack_bottom, size_t stack_size); void switch_in(); diff --git a/core/timer.hh b/core/timer.hh index 3fdb180b7d6..8c8bed51d89 100644 --- a/core/timer.hh +++ b/core/timer.hh @@ -22,7 +22,7 @@ #pragma once #include -#include +#include "util/std-compat.hh" #include #include #include "future.hh" @@ -43,12 +43,12 @@ private: boost::intrusive::list_member_hook<> _link; callback_t _callback; time_point _expiry; - std::experimental::optional _period; + compat::optional _period; bool _armed = false; bool _queued = false; bool _expired = false; void readd_periodic(); - void arm_state(time_point until, std::experimental::optional period); + void arm_state(time_point until, compat::optional period); public: timer() = default; timer(timer&& t) noexcept : _callback(std::move(t._callback)), _expiry(std::move(t._expiry)), _period(std::move(t._period)), @@ -61,8 +61,8 @@ public: ~timer(); future<> expired(); void set_callback(callback_t&& callback); - void arm(time_point until, std::experimental::optional period = {}); - void rearm(time_point until, std::experimental::optional period = {}); + void arm(time_point until, compat::optional period = {}); + void rearm(time_point until, compat::optional period = {}); void arm(duration delta); void arm_periodic(duration delta); bool armed() const { return _armed; } diff --git a/doc/rpc-streaming.md b/doc/rpc-streaming.md index ba35355e4a6..d09d43deb39 100644 --- a/doc/rpc-streaming.md +++ b/doc/rpc-streaming.md @@ -12,7 +12,7 @@ that is sent/received. For instance the sink that is used to send messages containing `int` and `long` will be of a type `rpc::sink`. The opposite end of the stream will have a source of the type `rpc::source` which will be used to receive those messages. Messages are received at a -source as `std::optional` containing an actual message as an `std::tuple`. Unengaged +source as `compat::optional` containing an actual message as an `std::tuple`. Unengaged optional means EOS (end of stream) - the stream was closed by a peer. If error happen before EOS is received a receiver cannot be sure it received all the data. @@ -32,7 +32,7 @@ To receive: ```c++ while (true) { - stdx:optional> data = source().get(); + seastar:optional> data = source().get(); if (!data) { // unengaged optional means EOS break; diff --git a/doc/tutorial.md b/doc/tutorial.md index d8dfaf76e1f..90d7a312c8c 100644 --- a/doc/tutorial.md +++ b/doc/tutorial.md @@ -895,12 +895,12 @@ seastar::future<> g() { `with_semaphore()`, like the earlier code snippets, waits for the given number of units from the semaphore, then runs the given lambda, and when the future returned by the lambda is resolved, `with_semaphore()` returns back the units to the semaphore. `with_semaphore()` returns a future which only resolves after all these steps are done. -The function `seastar::get_units()` is more general. It provides an exception-safe alternative to `seastar::semaphore`'s separate `wait()` and `signal()` methods, based on C++'s RAII philosophy: The function returns an opaque units object, which while held, keeps the semaphore's counter decreased - and as soon as this object is destructed, the counter is increased back. With this interface you cannot forget to increase the counter, or increase it twice, or increase without decreasing: The counter will always be decreased once when the units object is created, and if that succeeded, increased when the object is destructed. When the units object is moved into a continuation, no matter how this continuation ends, when the continuation is destructed, the units object is destructed and the units are returned to the semaphore's counter. The above examples, written with `get_units()`, looks like this: +The function `compat::get_units()` is more general. It provides an exception-safe alternative to `seastar::semaphore`'s separate `wait()` and `signal()` methods, based on C++'s RAII philosophy: The function returns an opaque units object, which while held, keeps the semaphore's counter decreased - and as soon as this object is destructed, the counter is increased back. With this interface you cannot forget to increase the counter, or increase it twice, or increase without decreasing: The counter will always be decreased once when the units object is created, and if that succeeded, increased when the object is destructed. When the units object is moved into a continuation, no matter how this continuation ends, when the continuation is destructed, the units object is destructed and the units are returned to the semaphore's counter. The above examples, written with `get_units()`, looks like this: ```cpp seastar::future<> g() { static thread_local semaphore limit(100); - return seastar::get_units(limit, 1).then([] (auto units) { + return compat::get_units(limit, 1).then([] (auto units) { return slow().finally([units = std::move(units)] {}); }); } @@ -979,14 +979,14 @@ Note how this code differs from the code we saw above for limiting the number of 1. Here we cannot use a single `thread_local` semaphore. Each call to `f()` has its loop with parallelism of 100, so needs its own semaphore "`limit`", kept alive during the loop with `do_with()`. 2. Here we do not wait for `slow()` to complete before continuing the loop, i.e., we do not `return` the future chain starting at `futurize_apply(slow)`. The loop continues to the next iteration when a semaphore unit becomes available, while (in our example) 99 other operations might be ongoing in the background and we do not wait for them. -In the examples in this section, we cannot use the `with_semaphore()` shortcut. `with_semaphore()` returns a future which only resolves after the lambda's returned future resolves. But in the above example, the loop needs to know when just the semaphore units are available, to start the next iteration --- and not wait for the previous iteration to complete. We could not achieve that with `with_semaphore()`. But the more general exception-safe idiom, `seastar::get_units()`, can be used in this case, and is recommended: +In the examples in this section, we cannot use the `with_semaphore()` shortcut. `with_semaphore()` returns a future which only resolves after the lambda's returned future resolves. But in the above example, the loop needs to know when just the semaphore units are available, to start the next iteration --- and not wait for the previous iteration to complete. We could not achieve that with `with_semaphore()`. But the more general exception-safe idiom, `compat::get_units()`, can be used in this case, and is recommended: ```cpp seastar::future<> f() { return seastar::do_with(seastar::semaphore(100), [] (auto& limit) { return seastar::repeat([&limit] { - return seastar::get_units(limit, 1).then([] (auto units) { + return compat::get_units(limit, 1).then([] (auto units) { slow().finally([units = std::move(units)] {}); return seastar::stop_iteration::no; }); @@ -1002,7 +1002,7 @@ seastar::future<> f() { return seastar::do_with(seastar::semaphore(100), [] (auto& limit) { return seastar::do_for_each(boost::counting_iterator(0), boost::counting_iterator(456), [&limit] (int i) { - return seastar::get_units(limit, 1).then([] (auto units) { + return compat::get_units(limit, 1).then([] (auto units) { slow().finally([units = std::move(units)] {}); }); }).finally([&limit] { @@ -1022,7 +1022,7 @@ seastar::future<> f() { return seastar::do_with(seastar::gate(), [] (auto& gate) { return seastar::do_for_each(boost::counting_iterator(0), boost::counting_iterator(456), [&gate] (int i) { - return seastar::get_units(limit, 1).then([&gate] (auto units) { + return compat::get_units(limit, 1).then([&gate] (auto units) { gate.enter(); seastar::futurize_apply(slow).finally([&gate, units = std::move(units)] { gate.leave(); diff --git a/http/httpd.hh b/http/httpd.hh index 753308d29bb..defdd393fdb 100644 --- a/http/httpd.hh +++ b/http/httpd.hh @@ -25,13 +25,13 @@ #include "http/request.hh" #include "core/reactor.hh" #include "core/sstring.hh" -#include #include "core/app-template.hh" #include "core/circular_buffer.hh" #include "core/distributed.hh" #include "core/queue.hh" #include "core/future-util.hh" #include "core/metrics_registration.hh" +#include "util/std-compat.hh" #include #include #include @@ -133,7 +133,7 @@ public: /** * Convert a hex encoded 2 bytes substring to char */ - static char hexstr_to_char(const std::experimental::string_view& in, size_t from) { + static char hexstr_to_char(const compat::string_view& in, size_t from) { return static_cast(hex_to_byte(in[from]) * 16 + hex_to_byte(in[from + 1])); } @@ -141,7 +141,7 @@ public: /** * URL_decode a substring and place it in the given out sstring */ - static bool url_decode(const std::experimental::string_view& in, sstring& out) { + static bool url_decode(const compat::string_view& in, sstring& out) { size_t pos = 0; char buff[in.length()]; for (size_t i = 0; i < in.length(); ++i) { @@ -165,7 +165,7 @@ public: /** * Add a single query parameter to the parameter list */ - static void add_param(request& req, const std::experimental::string_view& param) { + static void add_param(request& req, const compat::string_view& param) { size_t split = param.find('='); if (split >= param.length() - 1) { @@ -196,7 +196,7 @@ public: } size_t curr = pos + 1; size_t end_param; - std::experimental::string_view url = req._url; + compat::string_view url = req._url; while ((end_param = req._url.find('&', curr)) != sstring::npos) { add_param(req, url.substr(curr, end_param - curr) ); curr = end_param + 1; diff --git a/http/transformers.cc b/http/transformers.cc index 37af7c52b29..083d26fa361 100644 --- a/http/transformers.cc +++ b/http/transformers.cc @@ -21,7 +21,6 @@ #include #include "transformers.hh" -#include #include namespace seastar { diff --git a/net/api.hh b/net/api.hh index e12cc53b913..60fb186ff3d 100644 --- a/net/api.hh +++ b/net/api.hh @@ -31,8 +31,8 @@ #include "core/print.hh" #include "core/temporary_buffer.hh" #include "core/iostream.hh" +#include "util/std-compat.hh" #include -#include namespace seastar { @@ -89,7 +89,7 @@ struct sctp_keepalive_params { unsigned count; // spp_pathmaxrt }; -using keepalive_params = boost::variant; +using keepalive_params = compat::variant; /// \cond internal class connected_socket_impl; diff --git a/net/arp.cc b/net/arp.cc index 9b8dc86a234..239115fd818 100644 --- a/net/arp.cc +++ b/net/arp.cc @@ -43,8 +43,8 @@ arp::arp(interface* netif) : _netif(netif), _proto(netif, eth_protocol_num::arp, })) { } -std::experimental::optional arp::get_packet() { - std::experimental::optional p; +compat::optional arp::get_packet() { + compat::optional p; if (!_packetq.empty()) { p = std::move(_packetq.front()); _packetq.pop_front(); diff --git a/net/arp.hh b/net/arp.hh index 012778725e7..252f03409d5 100644 --- a/net/arp.hh +++ b/net/arp.hh @@ -76,7 +76,7 @@ private: ethernet_address l2self() { return _netif->hw_address(); } future<> process_packet(packet p, ethernet_address from); bool forward(forward_hash& out_hash_data, packet& p, size_t off); - std::experimental::optional get_packet(); + compat::optional get_packet(); template friend class arp_for; }; diff --git a/net/config.hh b/net/config.hh index 43c3676da31..a2d6709df64 100644 --- a/net/config.hh +++ b/net/config.hh @@ -21,14 +21,13 @@ #pragma once -#include +#include "util/std-compat.hh" #include #include #include namespace seastar { namespace net { - namespace stdx = std::experimental; struct ipv4_config { std::string ip; @@ -39,14 +38,14 @@ namespace net { struct hw_config { std::string pci_address; - stdx::optional port_index; + compat::optional port_index; bool lro{ true }; bool tso{ true }; bool ufo{ true }; bool hw_fc{ true }; bool event_index{ true }; bool csum_offload{ true }; - stdx::optional ring_size; + compat::optional ring_size; }; struct device_config { diff --git a/net/dns.cc b/net/dns.cc index 4fe23b9159a..3ff3eeb2738 100644 --- a/net/dns.cc +++ b/net/dns.cc @@ -20,7 +20,6 @@ */ #include -#include #include @@ -32,6 +31,7 @@ #include "core/reactor.hh" #include "core/gate.hh" #include "util/log.hh" +#include "util/std-compat.hh" namespace seastar { @@ -768,8 +768,8 @@ class net::dns_resolver::impl } ; connected_socket socket; - std::experimental::optional> in; - std::experimental::optional> out; + compat::optional> in; + compat::optional> out; temporary_buffer indata; }; struct udp_entry { @@ -777,7 +777,7 @@ class net::dns_resolver::impl : channel(std::move(c)) { } net::udp_channel channel; - std::experimental::optional in;; + compat::optional in;; socket_address dst; }; struct sock_entry { diff --git a/net/dns.hh b/net/dns.hh index 096b035e83f..ad522d6b169 100644 --- a/net/dns.hh +++ b/net/dns.hh @@ -24,7 +24,7 @@ #include #include #include -#include +#include "util/std-compat.hh" #include "../core/future.hh" #include "../core/sstring.hh" @@ -56,7 +56,7 @@ struct hostent { std::vector addr_list; }; -typedef std::experimental::optional opt_family; +typedef compat::optional opt_family; /** * A DNS resolver object. @@ -69,15 +69,15 @@ typedef std::experimental::optional opt_family; class dns_resolver { public: struct options { - std::experimental::optional + compat::optional use_tcp_query; - std::experimental::optional> + compat::optional> servers; - std::experimental::optional + compat::optional timeout; - std::experimental::optional + compat::optional tcp_port, udp_port; - std::experimental::optional> + compat::optional> domains; }; diff --git a/net/dpdk.cc b/net/dpdk.cc index b1af6f39d07..6f62204e7f4 100644 --- a/net/dpdk.cc +++ b/net/dpdk.cc @@ -36,7 +36,7 @@ #include #include #include -#include +#include "util/std-compat.hh" #include #include "ip.hh" #include "const.hh" @@ -1082,11 +1082,11 @@ class dpdk_qp : public net::qp { // if (_p) { // - // Reset the std::optional. This in particular is going + // Reset the compat::optional. This in particular is going // to call the "packet"'s destructor and reset the // "optional" state to "nonengaged". // - _p = std::experimental::nullopt; + _p = compat::nullopt; } else if (!_is_zc) { return; @@ -1118,7 +1118,7 @@ class dpdk_qp : public net::qp { private: struct rte_mbuf _mbuf; MARKER private_start; - std::experimental::optional _p; + compat::optional _p; phys_addr_t _buf_physaddr; uint16_t _data_off; // TRUE if underlying mbuf has been used in the zero-copy flow @@ -1443,7 +1443,7 @@ class dpdk_qp : public net::qp { * @return a "optional" object representing the newly received data if in an * "engaged" state or an error if in a "disengaged" state. */ - std::experimental::optional from_mbuf(rte_mbuf* m); + compat::optional from_mbuf(rte_mbuf* m); /** * Transform an LRO rte_mbuf cluster into the "packet" object. @@ -1452,7 +1452,7 @@ class dpdk_qp : public net::qp { * @return a "optional" object representing the newly received LRO packet if * in an "engaged" state or an error if in a "disengaged" state. */ - std::experimental::optional from_mbuf_lro(rte_mbuf* m); + compat::optional from_mbuf_lro(rte_mbuf* m); private: dpdk_device* _dev; @@ -1466,7 +1466,7 @@ class dpdk_qp : public net::qp { reactor::poller _rx_gc_poller; std::unique_ptr _rx_xmem; tx_buf_factory _tx_buf_factory; - std::experimental::optional _rx_poller; + compat::optional _rx_poller; reactor::poller _tx_gc_poller; std::vector _tx_burst; uint16_t _tx_burst_idx = 0; @@ -1975,7 +1975,7 @@ void dpdk_qp::rx_start() { } template<> -inline std::experimental::optional +inline compat::optional dpdk_qp::from_mbuf_lro(rte_mbuf* m) { // @@ -2005,11 +2005,11 @@ dpdk_qp::from_mbuf_lro(rte_mbuf* m) // Drop if allocation failed rte_pktmbuf_free(m); - return std::experimental::nullopt; + return compat::nullopt; } template<> -inline std::experimental::optional +inline compat::optional dpdk_qp::from_mbuf(rte_mbuf* m) { if (!_dev->hw_features_ref().rx_lro || rte_pktmbuf_is_contiguous(m)) { @@ -2026,7 +2026,7 @@ dpdk_qp::from_mbuf(rte_mbuf* m) // Drop if allocation failed rte_pktmbuf_free(m); - return std::experimental::nullopt; + return compat::nullopt; } else { rte_memcpy(buf, rte_pktmbuf_mtod(m, char*), len); rte_pktmbuf_free(m); @@ -2039,7 +2039,7 @@ dpdk_qp::from_mbuf(rte_mbuf* m) } template<> -inline std::experimental::optional +inline compat::optional dpdk_qp::from_mbuf_lro(rte_mbuf* m) { _frags.clear(); @@ -2062,7 +2062,7 @@ dpdk_qp::from_mbuf_lro(rte_mbuf* m) } template<> -inline std::experimental::optional dpdk_qp::from_mbuf(rte_mbuf* m) +inline compat::optional dpdk_qp::from_mbuf(rte_mbuf* m) { _rx_free_pkts.push_back(m); _num_rx_free_segs += m->nb_segs; @@ -2144,7 +2144,7 @@ void dpdk_qp::process_packets( struct rte_mbuf *m = bufs[i]; offload_info oi; - std::experimental::optional p = from_mbuf(m); + compat::optional p = from_mbuf(m); // Drop the packet if translation above has failed if (!p) { diff --git a/net/ip.cc b/net/ip.cc index e3d9e61fa34..7a58d44c013 100644 --- a/net/ip.cc +++ b/net/ip.cc @@ -294,7 +294,7 @@ void ipv4::send(ipv4_address to, ip_protocol_num proto_num, packet p, ethernet_a } } -std::experimental::optional ipv4::get_packet() { +compat::optional ipv4::get_packet() { // _packetq will be mostly empty here unless it hold remnants of previously // fragmented packet if (_packetq.empty()) { @@ -311,7 +311,7 @@ std::experimental::optional ipv4::get_packet() { } } - std::experimental::optional p; + compat::optional p; if (!_packetq.empty()) { p = std::move(_packetq.front()); _packetq.pop_front(); diff --git a/net/ip.hh b/net/ip.hh index 9d6a68e119c..c2bf3381074 100644 --- a/net/ip.hh +++ b/net/ip.hh @@ -133,7 +133,7 @@ struct ipv4_traits { ethernet_address e_dst; ip_protocol_num proto_num; }; - using packet_provider_type = std::function ()>; + using packet_provider_type = std::function ()>; static void tcp_pseudo_header_checksum(checksummer& csum, ipv4_address src, ipv4_address dst, uint16_t len) { csum.sum_many(src.ip.raw, dst.ip.raw, uint8_t(0), uint8_t(ip_protocol_num::tcp), len); } @@ -221,7 +221,7 @@ public: using inet_type = ipv4_l4; explicit icmp(inet_type& inet) : _inet(inet) { _inet.register_packet_provider([this] { - std::experimental::optional l4p; + compat::optional l4p; if (!_packetq.empty()) { l4p = std::move(_packetq.front()); _packetq.pop_front(); @@ -372,7 +372,7 @@ private: private: future<> handle_received_packet(packet p, ethernet_address from); bool forward(forward_hash& out_hash_data, packet& p, size_t off); - std::experimental::optional get_packet(); + compat::optional get_packet(); bool in_my_netmask(ipv4_address a) const; void frag_limit_mem(); void frag_timeout(); diff --git a/net/net.cc b/net/net.cc index 8ba2ca929ad..6ad86f9a2a8 100644 --- a/net/net.cc +++ b/net/net.cc @@ -181,7 +181,7 @@ void qp::configure_proxies(const std::map& cpu_weights) { return; } register_packet_provider([this] { - std::experimental::optional p; + compat::optional p; if (!_proxy_packetq.empty()) { p = std::move(_proxy_packetq.front()); _proxy_packetq.pop_front(); @@ -241,7 +241,7 @@ interface::interface(std::shared_ptr dev) , _hw_address(_dev->hw_address()) , _hw_features(_dev->hw_features()) { dev->local_queue().register_packet_provider([this, idx = 0u] () mutable { - std::experimental::optional p; + compat::optional p; for (size_t i = 0; i < _pkt_providers.size(); i++) { auto l3p = _pkt_providers[idx++](); if (idx == _pkt_providers.size()) diff --git a/net/net.hh b/net/net.hh index 313b74dd47c..ca5db267da1 100644 --- a/net/net.hh +++ b/net/net.hh @@ -92,7 +92,7 @@ public: ethernet_address to; packet p; }; - using packet_provider_type = std::function ()>; + using packet_provider_type = std::function ()>; private: interface* _netif; eth_protocol_num _proto_num; @@ -211,9 +211,9 @@ struct qp_stats { }; class qp { - using packet_provider_type = std::function ()>; + using packet_provider_type = std::function ()>; std::vector _pkt_providers; - std::experimental::optional> _sw_reta; + compat::optional> _sw_reta; circular_buffer _proxy_packetq; stream _rx_stream; reactor::poller _tx_poller; diff --git a/net/packet.hh b/net/packet.hh index 4741f5577aa..b7f8da08476 100644 --- a/net/packet.hh +++ b/net/packet.hh @@ -28,7 +28,7 @@ #include #include #include -#include +#include "util/std-compat.hh" #include namespace seastar { @@ -50,7 +50,7 @@ struct offload_info { bool reassembled = false; uint16_t tso_seg_size = 0; // HW stripped VLAN header (CPU order) - std::experimental::optional vlan_tci; + compat::optional vlan_tci; }; // Zero-copy friendly packet class @@ -98,7 +98,7 @@ class packet final { uint16_t _nr_frags = 0; uint16_t _allocated_frags; offload_info _offload_info; - std::experimental::optional _rss_hash; + compat::optional _rss_hash; char _data[internal_data_size]; // only _frags[0] may use unsigned _headroom = internal_data_size; // in _data // FIXME: share _data/_frags space @@ -273,10 +273,10 @@ public: _impl = impl::allocate_if_needed(std::move(_impl), extra); } } - std::experimental::optional rss_hash() { + compat::optional rss_hash() { return _impl->_rss_hash; } - std::experimental::optional set_rss_hash(uint32_t hash) { + compat::optional set_rss_hash(uint32_t hash) { return _impl->_rss_hash = hash; } // Call `func` for each fragment, avoiding data copies when possible diff --git a/net/posix-stack.cc b/net/posix-stack.cc index 0baad9fc762..36df944d33e 100644 --- a/net/posix-stack.cc +++ b/net/posix-stack.cc @@ -51,7 +51,7 @@ class posix_connected_socket_operations { return _fd.getsockopt(SOL_SOCKET, SO_KEEPALIVE); } void set_keepalive_parameters(file_desc& _fd, const keepalive_params& params) { - const tcp_keepalive_params& pms = boost::get(params); + const tcp_keepalive_params& pms = compat::get(params); _fd.setsockopt(IPPROTO_TCP, TCP_KEEPCNT, pms.count); _fd.setsockopt(IPPROTO_TCP, TCP_KEEPIDLE, int(pms.idle.count())); _fd.setsockopt(IPPROTO_TCP, TCP_KEEPINTVL, int(pms.interval.count())); @@ -87,7 +87,7 @@ class posix_connected_socket_operations { return _fd.getsockopt(SOL_SCTP, SCTP_PEER_ADDR_PARAMS).spp_flags & SPP_HB_ENABLE; } void set_keepalive_parameters(file_desc& _fd, const keepalive_params& kpms) { - const sctp_keepalive_params& pms = boost::get(kpms); + const sctp_keepalive_params& pms = compat::get(kpms); auto params = _fd.getsockopt(SOL_SCTP, SCTP_PEER_ADDR_PARAMS); params.spp_hbinterval = pms.interval.count() * 1000; // in milliseconds params.spp_pathmaxrxt = pms.count; diff --git a/net/tcp.hh b/net/tcp.hh index 926855e74e6..d080afaf05e 100644 --- a/net/tcp.hh +++ b/net/tcp.hh @@ -32,12 +32,12 @@ #include "ip.hh" #include "const.hh" #include "packet-util.hh" +#include "util/std-compat.hh" #include #include #include #include #include -#include #include #include #include @@ -343,12 +343,12 @@ private: bool closed = false; promise<> _window_opened; // Wait for all data are acked - std::experimental::optional> _all_data_acked_promise; + compat::optional> _all_data_acked_promise; // Limit number of data queued into send queue size_t max_queue_space = 212992; size_t current_queue_space = 0; // wait for there is at least one byte available in the queue - std::experimental::optional> _send_available_promise; + compat::optional> _send_available_promise; // Round-trip time variation std::chrono::milliseconds rttvar; // Smoothed round-trip time @@ -380,7 +380,7 @@ private: // The total size of data stored in std::deque data size_t data_size = 0; tcp_packet_merger out_of_order; - std::experimental::optional> _data_received_promise; + compat::optional> _data_received_promise; // The maximun memory buffer size allowed for receiving // Currently, it is the same as default receive window size when window scaling is enabled size_t max_receive_buf_size = 3737600; @@ -442,7 +442,7 @@ private: auto id = connid{_local_ip, _foreign_ip, _local_port, _foreign_port}; _tcp._tcbs.erase(id); } - std::experimental::optional get_packet(); + compat::optional get_packet(); void output() { if (!_poll_active) { _poll_active = true; @@ -595,15 +595,15 @@ private: cleanup(); if (_rcv._data_received_promise) { _rcv._data_received_promise->set_exception(tcp_reset_error()); - _rcv._data_received_promise = std::experimental::nullopt; + _rcv._data_received_promise = compat::nullopt; } if (_snd._all_data_acked_promise) { _snd._all_data_acked_promise->set_exception(tcp_reset_error()); - _snd._all_data_acked_promise = std::experimental::nullopt; + _snd._all_data_acked_promise = compat::nullopt; } if (_snd._send_available_promise) { _snd._send_available_promise->set_exception(tcp_reset_error()); - _snd._send_available_promise = std::experimental::nullopt; + _snd._send_available_promise = compat::nullopt; } } void do_time_wait() { @@ -770,7 +770,7 @@ tcp::tcp(inet_type& inet) }); _inet.register_packet_provider([this, tcb_polled = 0u] () mutable { - std::experimental::optional l4p; + compat::optional l4p; auto c = _poll_tcbs.size(); if (!_packetq.empty() && (!(tcb_polled % 128) || c == 0)) { l4p = std::move(_packetq.front()); @@ -1723,7 +1723,7 @@ tcp::tcb::abort_reader() { if (_rcv._data_received_promise) { _rcv._data_received_promise->set_exception( std::make_exception_ptr(std::system_error(ECONNABORTED, std::system_category()))); - _rcv._data_received_promise = std::experimental::nullopt; + _rcv._data_received_promise = compat::nullopt; } } @@ -2059,14 +2059,14 @@ tcp_seq tcp::tcb::get_isn() { } template -std::experimental::optional tcp::tcb::get_packet() { +compat::optional tcp::tcb::get_packet() { _poll_active = false; if (_packetq.empty()) { output_one(); } if (in_state(CLOSED)) { - return std::experimental::optional(); + return compat::optional(); } assert(!_packetq.empty()); diff --git a/net/tls.cc b/net/tls.cc index 927eef4a5ff..6282d0cb1fa 100644 --- a/net/tls.cc +++ b/net/tls.cc @@ -21,8 +21,6 @@ #include #include - -#include #include #include "core/reactor.hh" @@ -32,6 +30,7 @@ #include "core/timer.hh" #include "tls.hh" #include "stack.hh" +#include "util/std-compat.hh" namespace seastar { @@ -438,19 +437,19 @@ void tls::credentials_builder::set_dh_level(dh_params::level level) { } void tls::credentials_builder::set_x509_trust(const blob& b, x509_crt_format fmt) { - _blobs.emplace(x509_trust_key, std::make_pair(b.to_string(), fmt)); + _blobs.emplace(x509_trust_key, std::make_pair(compat::string_view_to_string(b), fmt)); } void tls::credentials_builder::set_x509_crl(const blob& b, x509_crt_format fmt) { - _blobs.emplace(x509_crl_key, std::make_pair(b.to_string(), fmt)); + _blobs.emplace(x509_crl_key, std::make_pair(compat::string_view_to_string(b), fmt)); } void tls::credentials_builder::set_x509_key(const blob& cert, const blob& key, x509_crt_format fmt) { - _blobs.emplace(x509_key_key, std::make_tuple(cert.to_string(), key.to_string(), fmt)); + _blobs.emplace(x509_key_key, std::make_tuple(compat::string_view_to_string(cert), compat::string_view_to_string(key), fmt)); } void tls::credentials_builder::set_simple_pkcs12(const blob& b, x509_crt_format fmt, const sstring& password) { - _blobs.emplace(pkcs12_key, std::make_tuple(b.to_string(), fmt, password)); + _blobs.emplace(pkcs12_key, std::make_tuple(compat::string_view_to_string(b), fmt, password)); } future<> tls::credentials_builder::set_system_trust() { diff --git a/net/tls.hh b/net/tls.hh index 994fb9a9c53..b5f81fc3318 100644 --- a/net/tls.hh +++ b/net/tls.hh @@ -20,7 +20,6 @@ */ #pragma once -#include #include #include @@ -30,6 +29,7 @@ #include "core/sstring.hh" #include "core/shared_ptr.hh" #include "socket_defs.hh" +#include "util/std-compat.hh" namespace seastar { @@ -54,7 +54,7 @@ namespace tls { PEM, }; - typedef std::experimental::basic_string_view blob; + typedef compat::basic_string_view blob; class session; class server_session; diff --git a/net/udp.cc b/net/udp.cc index 5e96d76efef..fb5bf4ad33b 100644 --- a/net/udp.cc +++ b/net/udp.cc @@ -127,7 +127,7 @@ ipv4_udp::ipv4_udp(ipv4& inet) : _inet(inet) { _inet.register_packet_provider([this] { - std::experimental::optional l4p; + compat::optional l4p; if (!_packetq.empty()) { l4p = std::move(_packetq.front()); _packetq.pop_front(); diff --git a/rpc/lz4_compressor.cc b/rpc/lz4_compressor.cc index 2cca3b9e022..9d9214aaecf 100644 --- a/rpc/lz4_compressor.cc +++ b/rpc/lz4_compressor.cc @@ -29,15 +29,15 @@ namespace rpc { const sstring lz4_compressor::factory::_name = "LZ4"; -static temporary_buffer linearize(boost::variant>, temporary_buffer>& v, uint32_t size) { - auto* one = boost::get>(&v); +static temporary_buffer linearize(compat::variant>, temporary_buffer>& v, uint32_t size) { + auto* one = compat::get_if>(&v); if (one) { // no need to linearize return std::move(*one); } else { temporary_buffer src(size); auto p = src.get_write(); - for (auto&& b : boost::get>>(v)) { + for (auto&& b : compat::get>>(v)) { p = std::copy_n(b.begin(), b.size(), p); } return src; @@ -75,7 +75,7 @@ rcv_buf lz4_compressor::decompress(rcv_buf data) { src.trim_front(4); rcv_buf rb(size); rb.bufs = temporary_buffer(size); - auto& dst = boost::get>(rb.bufs); + auto& dst = compat::get>(rb.bufs); if (LZ4_decompress_fast(src.begin(), dst.get_write(), dst.size()) < 0) { throw std::runtime_error("RPC frame LZ4 decompression failure"); } diff --git a/rpc/rpc.cc b/rpc/rpc.cc index f4a1de1e076..cf6d4c424e3 100644 --- a/rpc/rpc.cc +++ b/rpc/rpc.cc @@ -23,11 +23,11 @@ namespace rpc { } temporary_buffer& snd_buf::front() { - auto *one = boost::get>(&bufs); + auto* one = compat::get_if>(&bufs); if (one) { return *one; } else { - return boost::get>>(bufs).front(); + return compat::get>>(bufs).front(); } } @@ -39,12 +39,12 @@ namespace rpc { return std::move(*org); } T buf(org->size); - auto *one = boost::get>(&org->bufs); + auto* one = compat::get_if>(&org->bufs); if (one) { buf.bufs = temporary_buffer(one->get_write(), one->size(), make_object_deleter(std::move(org))); } else { - auto& orgbufs = boost::get>>(org->bufs); + auto& orgbufs = compat::get>>(org->bufs); std::vector> newbufs; newbufs.reserve(orgbufs.size()); deleter d = make_object_deleter(std::move(org)); @@ -71,11 +71,11 @@ namespace rpc { } future<> connection::send_buffer(snd_buf buf) { - auto* b = boost::get>(&buf.bufs); + auto* b = compat::get_if>(&buf.bufs); if (b) { return _write_buf.write(std::move(*b)); } else { - return do_with(std::move(boost::get>>(buf.bufs)), + return do_with(std::move(compat::get>>(buf.bufs)), [this] (std::vector>& ar) { return do_for_each(ar.begin(), ar.end(), [this] (auto& b) { return _write_buf.write(std::move(b)); @@ -175,7 +175,7 @@ namespace rpc { }); } - future<> connection::send(snd_buf buf, std::experimental::optional timeout, cancellable* cancel) { + future<> connection::send(snd_buf buf, compat::optional timeout, cancellable* cancel) { if (!_error) { if (timeout && *timeout <= rpc_clock_type::now()) { return make_ready_future<>(); @@ -291,7 +291,7 @@ namespace rpc { return stop_iteration::yes; } else { left -= data.size(); - boost::get>>(rb.bufs).push_back(std::move(data)); + compat::get>>(rb.bufs).push_back(std::move(data)); return left ? stop_iteration::no : stop_iteration::yes; } }); @@ -351,11 +351,11 @@ namespace rpc { } auto eb = compressor->decompress(std::move(compressed_data)); net::packet p; - auto* one = boost::get>(&eb.bufs); + auto* one = compat::get_if>(&eb.bufs); if (one) { p = net::packet(std::move(p), std::move(*one)); } else { - for (auto&& b : boost::get>>(eb.bufs)) { + for (auto&& b : compat::get>>(eb.bufs)) { p = net::packet(std::move(p), std::move(b)); } } @@ -370,7 +370,7 @@ namespace rpc { } struct stream_frame { - using opt_buf_type = std::experimental::optional; + using opt_buf_type = compat::optional; using return_type = future; struct header_type { uint32_t size; @@ -383,7 +383,7 @@ namespace rpc { return "stream"; } static future empty_value() { - return make_ready_future(std::experimental::nullopt); + return make_ready_future(compat::nullopt); } static header_type decode_header(const char* ptr) { header_type h{read_le(ptr), false}; @@ -404,7 +404,7 @@ namespace rpc { } }; - future> + future> connection::read_stream_frame_compressed(input_stream& in) { return read_frame_compressed(peer_address(), _compressor, in); } @@ -432,7 +432,7 @@ namespace rpc { } future<> connection::handle_stream_frame() { - return read_stream_frame_compressed(_read_buf).then([this] (std::experimental::optional data) { + return read_stream_frame_compressed(_read_buf).then([this] (compat::optional data) { if (!data) { _error = true; return make_ready_future<>(); @@ -521,7 +521,7 @@ namespace rpc { } struct response_frame { - using opt_buf_type = std::experimental::optional; + using opt_buf_type = compat::optional; using return_type = future; using header_type = std::tuple; static size_t header_size() { @@ -531,7 +531,7 @@ namespace rpc { return "client"; } static auto empty_value() { - return make_ready_future(0, std::experimental::nullopt); + return make_ready_future(0, compat::nullopt); } static header_type decode_header(const char* ptr) { auto msgid = read_le(ptr); @@ -547,12 +547,12 @@ namespace rpc { }; - future> + future> client::read_response_frame(input_stream& in) { return read_frame(_server_addr, in); } - future> + future> client::read_response_frame_compressed(input_stream& in) { return read_frame_compressed(_server_addr, _compressor, in); } @@ -564,7 +564,7 @@ namespace rpc { return res; } - void client::wait_for_reply(id_type id, std::unique_ptr&& h, std::experimental::optional timeout, cancellable* cancel) { + void client::wait_for_reply(id_type id, std::unique_ptr&& h, compat::optional timeout, cancellable* cancel) { if (timeout) { h->t.set_callback(std::bind(std::mem_fn(&client::wait_timed_out), this, id)); h->t.arm(timeout.value()); @@ -633,13 +633,13 @@ namespace rpc { return negotiate_protocol(_read_buf).then([this] () { _client_negotiated->set_value(); - _client_negotiated = stdx::nullopt; + _client_negotiated = compat::nullopt; send_loop(); return do_until([this] { return _read_buf.eof() || _error; }, [this] () mutable { if (is_stream()) { return handle_stream_frame(); } - return read_response_frame_compressed(_read_buf).then([this] (int64_t msg_id, std::experimental::optional data) { + return read_response_frame_compressed(_read_buf).then([this] (int64_t msg_id, compat::optional data) { auto it = _outstanding.find(std::abs(msg_id)); if (!data) { _error = true; @@ -774,9 +774,9 @@ namespace rpc { } struct request_frame { - using opt_buf_type = std::experimental::optional; - using return_type = future, uint64_t, int64_t, opt_buf_type>; - using header_type = std::tuple, uint64_t, int64_t, uint32_t>; + using opt_buf_type = compat::optional; + using return_type = future, uint64_t, int64_t, opt_buf_type>; + using header_type = std::tuple, uint64_t, int64_t, uint32_t>; static size_t header_size() { return 20; } @@ -784,19 +784,19 @@ namespace rpc { return "server"; } static auto empty_value() { - return make_ready_future, uint64_t, int64_t, opt_buf_type>(std::experimental::nullopt, uint64_t(0), 0, std::experimental::nullopt); + return make_ready_future, uint64_t, int64_t, opt_buf_type>(compat::nullopt, uint64_t(0), 0, compat::nullopt); } static header_type decode_header(const char* ptr) { auto type = read_le(ptr); auto msgid = read_le(ptr + 8); auto size = read_le(ptr + 16); - return std::make_tuple(std::experimental::nullopt, type, msgid, size); + return std::make_tuple(compat::nullopt, type, msgid, size); } static uint32_t get_size(const header_type& t) { return std::get<3>(t); } static auto make_value(const header_type& t, rcv_buf data) { - return make_ready_future, uint64_t, int64_t, opt_buf_type>(std::get<0>(t), std::get<1>(t), std::get<2>(t), std::move(data)); + return make_ready_future, uint64_t, int64_t, opt_buf_type>(std::get<0>(t), std::get<1>(t), std::get<2>(t), std::move(data)); } }; @@ -812,7 +812,7 @@ namespace rpc { } }; - future, uint64_t, int64_t, std::experimental::optional> + future, uint64_t, int64_t, compat::optional> server::connection::read_request_frame_compressed(input_stream& in) { if (_timeout_negotiated) { return read_frame_compressed(_info, _compressor, in); @@ -822,7 +822,7 @@ namespace rpc { } future<> - server::connection::respond(int64_t msg_id, snd_buf&& data, std::experimental::optional timeout) { + server::connection::respond(int64_t msg_id, snd_buf&& data, compat::optional timeout) { static_assert(snd_buf::chunk_size >= 12, "send buffer chunk size is too small"); auto p = data.front().get_write(); write_le(p, msg_id); @@ -837,12 +837,12 @@ namespace rpc { if (is_stream()) { return handle_stream_frame(); } - return read_request_frame_compressed(_read_buf).then([this] (std::experimental::optional expire, uint64_t type, int64_t msg_id, std::experimental::optional data) { + return read_request_frame_compressed(_read_buf).then([this] (compat::optional expire, uint64_t type, int64_t msg_id, compat::optional data) { if (!data) { _error = true; return make_ready_future<>(); } else { - std::experimental::optional timeout; + compat::optional timeout; if (expire && *expire) { timeout = rpc_clock_type::now() + std::chrono::milliseconds(*expire); } diff --git a/rpc/rpc.hh b/rpc/rpc.hh index 7251da69d11..46b670fea23 100644 --- a/rpc/rpc.hh +++ b/rpc/rpc.hh @@ -78,7 +78,7 @@ struct resource_limits { }; struct client_options { - std::experimental::optional keepalive; + compat::optional keepalive; bool tcp_nodelay = true; compressor::factory* compressor_factory = nullptr; bool send_timeout_data = true; @@ -110,7 +110,7 @@ public: struct server_options { compressor::factory* compressor_factory = nullptr; bool tcp_nodelay = true; - stdx::optional streaming_domain; + compat::optional streaming_domain; }; inline @@ -181,11 +181,11 @@ protected: struct outgoing_entry { timer t; snd_buf buf; - std::experimental::optional> p = promise<>(); + compat::optional> p = promise<>(); cancellable* pcancel = nullptr; outgoing_entry(snd_buf b) : buf(std::move(b)) {} outgoing_entry(outgoing_entry&& o) : t(std::move(o.t)), buf(std::move(o.buf)), p(std::move(o.p)), pcancel(o.pcancel) { - o.p = std::experimental::nullopt; + o.p = compat::nullopt; } ~outgoing_entry() { if (p) { @@ -231,7 +231,7 @@ protected: template void send_loop(); future<> stop_send_loop(); - future> read_stream_frame_compressed(input_stream& in); + future> read_stream_frame_compressed(input_stream& in); bool stream_check_twoway_closed() { return _sink_closed && _source_closed; } @@ -247,7 +247,7 @@ public: future<> send_negotiation_frame(feature_map features); // functions below are public because they are used by external heavily templated functions // and I am not smart enough to know how to define them as friends - future<> send(snd_buf buf, std::experimental::optional timeout = {}, cancellable* cancel = nullptr); + future<> send(snd_buf buf, compat::optional timeout = {}, cancellable* cancel = nullptr); bool error() { return _error; } void abort(); future<> stop(); @@ -307,7 +307,7 @@ template class source_impl : public source::impl { public: source_impl(xshard_connection_ptr con) : source::impl(std::move(con)) {} - future>> operator()() override; + future>> operator()() override; }; class client : public rpc::connection, public weakly_referencable { @@ -349,15 +349,15 @@ private: std::unordered_map> _outstanding; ipv4_addr _server_addr; client_options _options; - stdx::optional> _client_negotiated = shared_promise<>(); + compat::optional> _client_negotiated = shared_promise<>(); weak_ptr _parent; // for stream clients private: future<> negotiate_protocol(input_stream& in); void negotiate(feature_map server_features); - future> + future> read_response_frame(input_stream& in); - future> + future> read_response_frame_compressed(input_stream& in); void send_loop() { if (is_stream()) { @@ -392,7 +392,7 @@ public: return _stats; } auto next_message_id() { return _message_id++; } - void wait_for_reply(id_type id, std::unique_ptr&& h, std::experimental::optional timeout, cancellable* cancel); + void wait_for_reply(id_type id, std::unique_ptr&& h, compat::optional timeout, cancellable* cancel); void wait_timed_out(id_type id); future<> stop(); void abort_all_streams(); @@ -446,7 +446,7 @@ public: private: future<> negotiate_protocol(input_stream& in); - future, uint64_t, int64_t, std::experimental::optional> + future, uint64_t, int64_t, compat::optional> read_request_frame_compressed(input_stream& in); future negotiate(feature_map requested); void send_loop() { @@ -459,7 +459,7 @@ public: public: connection(server& s, connected_socket&& fd, socket_address&& addr, const logger& l, void* seralizer, connection_id id); future<> process(); - future<> respond(int64_t msg_id, snd_buf&& data, std::experimental::optional timeout); + future<> respond(int64_t msg_id, snd_buf&& data, compat::optional timeout); client_info& info() { return _info; } const client_info& info() const { return _info; } stats get_stats() const { @@ -475,7 +475,7 @@ public: return ipv4_addr(_info.addr); } // Resources will be released when this goes out of scope - future wait_for_resources(size_t memory_consumed, std::experimental::optional timeout) { + future wait_for_resources(size_t memory_consumed, compat::optional timeout) { if (timeout) { return get_units(_server._resources_available, memory_consumed, *timeout); } else { @@ -524,7 +524,7 @@ public: friend client; }; -using rpc_handler_func = std::function (shared_ptr, std::experimental::optional timeout, int64_t msgid, +using rpc_handler_func = std::function (shared_ptr, compat::optional timeout, int64_t msgid, rcv_buf data)>; struct rpc_handler { diff --git a/rpc/rpc_impl.hh b/rpc/rpc_impl.hh index fb44b939e9c..fe67ebfafec 100644 --- a/rpc/rpc_impl.hh +++ b/rpc/rpc_impl.hh @@ -243,11 +243,11 @@ inline void do_marshall(Serializer& serializer, Output& out, const T&... args) { } static inline memory_output_stream make_serializer_stream(snd_buf& output) { - auto* b = boost::get>(&output.bufs); + auto* b = compat::get_if>(&output.bufs); if (b) { return memory_output_stream(memory_output_stream::simple(b->get_write(), b->size())); } else { - auto& ar = boost::get>>(output.bufs); + auto& ar = compat::get>>(output.bufs); return memory_output_stream(memory_output_stream::fragmented(ar.begin(), output.size)); } } @@ -391,7 +391,7 @@ template struct rcv_reply> : rcv_reply {}; template -inline auto wait_for_reply(wait_type, std::experimental::optional timeout, cancellable* cancel, rpc::client& dst, id_type msg_id, +inline auto wait_for_reply(wait_type, compat::optional timeout, cancellable* cancel, rpc::client& dst, id_type msg_id, signature sig) { using reply_type = rcv_reply; auto lambda = [] (reply_type& r, rpc::client& dst, id_type msg_id, rcv_buf data) mutable { @@ -412,13 +412,13 @@ inline auto wait_for_reply(wait_type, std::experimental::optional -inline auto wait_for_reply(no_wait_type, std::experimental::optional, cancellable* cancel, rpc::client& dst, id_type msg_id, +inline auto wait_for_reply(no_wait_type, compat::optional, cancellable* cancel, rpc::client& dst, id_type msg_id, signature sig) { // no_wait overload return make_ready_future<>(); } template -inline auto wait_for_reply(no_wait_type, std::experimental::optional, cancellable* cancel, rpc::client& dst, id_type msg_id, +inline auto wait_for_reply(no_wait_type, compat::optional, cancellable* cancel, rpc::client& dst, id_type msg_id, signature (InArgs...)> sig) { // future overload return make_ready_future<>(); } @@ -432,7 +432,7 @@ auto send_helper(MsgType xt, signature xsig) { struct shelper { MsgType t; signature sig; - auto send(rpc::client& dst, std::experimental::optional timeout, cancellable* cancel, const InArgs&... args) { + auto send(rpc::client& dst, compat::optional timeout, cancellable* cancel, const InArgs&... args) { if (dst.error()) { using cleaned_ret_type = typename wait_signature::cleaned_type; return futurize::make_exception_future(closed_error()); @@ -472,7 +472,7 @@ auto send_helper(MsgType xt, signature xsig) { template inline future<> reply(wait_type, future&& ret, int64_t msg_id, shared_ptr client, - std::experimental::optional timeout) { + compat::optional timeout) { if (!client->error()) { snd_buf data; try { @@ -500,7 +500,7 @@ inline future<> reply(wait_type, future&& ret, int64_t msg_id, shar // specialization for no_wait_type which does not send a reply template -inline future<> reply(no_wait_type, future&& r, int64_t msgid, shared_ptr client, std::experimental::optional timeout) { +inline future<> reply(no_wait_type, future&& r, int64_t msgid, shared_ptr client, compat::optional timeout) { try { r.get(); } catch (std::exception& ex) { @@ -537,7 +537,7 @@ auto recv_helper(signature sig, Func&& func, WantClientInfo wci using signature = decltype(sig); using wait_style = wait_signature_t; return [func = lref_to_cref(std::forward(func))](shared_ptr client, - std::experimental::optional timeout, + compat::optional timeout, int64_t msg_id, rcv_buf data) mutable { auto memory_consumed = client->estimate_request_size(data.size); @@ -702,13 +702,13 @@ future<> sink_impl::close() { } template -future>> source_impl::operator()() { +future>> source_impl::operator()() { auto process_one_buffer = [this] { foreign_ptr> buf = std::move(this->_bufs.front()); this->_bufs.pop_front(); return seastar::apply([] (In&&... args) { - auto ret = std::experimental::make_optional(std::make_tuple(std::move(args)...)); - return make_ready_future>>(std::move(ret)); + auto ret = compat::make_optional(std::make_tuple(std::move(args)...)); + return make_ready_future>>(std::move(ret)); }, unmarshall(*this->_con->get(), make_shard_local_buffer_copy(std::move(buf)))); }; @@ -736,7 +736,7 @@ future>> source_impl_bufs.empty()) { - return make_ready_future>>(std::experimental::nullopt); + return make_ready_future>>(compat::nullopt); } else { return process_one_buffer(); } diff --git a/rpc/rpc_types.hh b/rpc/rpc_types.hh index 857572954dc..f062562d3bb 100644 --- a/rpc/rpc_types.hh +++ b/rpc/rpc_types.hh @@ -26,8 +26,7 @@ #include #include #include -#include -#include +#include "util/std-compat.hh" #include "core/timer.hh" #include "core/simple-stream.hh" #include "core/lowres_clock.hh" @@ -122,16 +121,16 @@ struct no_wait_type {}; extern no_wait_type no_wait; template -class optional : public std::experimental::optional { +class optional : public compat::optional { public: - using std::experimental::optional::optional; + using compat::optional::optional; }; -class opt_time_point : public std::experimental::optional { +class opt_time_point : public compat::optional { public: - using std::experimental::optional::optional; - opt_time_point(std::experimental::optional time_point) { - static_cast&>(*this) = time_point; + using compat::optional::optional; + opt_time_point(compat::optional time_point) { + static_cast&>(*this) = time_point; } }; @@ -173,8 +172,8 @@ struct cancellable { struct rcv_buf { uint32_t size = 0; - std::experimental::optional> su; - boost::variant>, temporary_buffer> bufs; + compat::optional> su; + compat::variant>, temporary_buffer> bufs; using iterator = std::vector>::iterator; rcv_buf() {} explicit rcv_buf(size_t size_) : size(size_) {} @@ -183,7 +182,7 @@ struct rcv_buf { struct snd_buf { static constexpr size_t chunk_size = 128*1024; uint32_t size = 0; - boost::variant>, temporary_buffer> bufs; + compat::variant>, temporary_buffer> bufs; using iterator = std::vector>::iterator; snd_buf() {} explicit snd_buf(size_t size_); @@ -192,11 +191,11 @@ struct snd_buf { }; static inline memory_input_stream make_deserializer_stream(rcv_buf& input) { - auto* b = boost::get>(&input.bufs); + auto* b = compat::get_if>(&input.bufs); if (b) { return memory_input_stream(memory_input_stream::simple(b->begin(), b->size())); } else { - auto& ar = boost::get>>(input.bufs); + auto& ar = compat::get>>(input.bufs); return memory_input_stream(memory_input_stream::fragmented(ar.begin(), input.size)); } } @@ -287,7 +286,7 @@ public: } public: virtual ~impl() {} - virtual future>> operator()() = 0; + virtual future>> operator()() = 0; friend source; }; private: @@ -295,7 +294,7 @@ private: public: source(shared_ptr impl) : _impl(std::move(impl)) {} - future>> operator()() { + future>> operator()() { return _impl->operator()(); }; connection_id get_id() const; diff --git a/tests/exchanger.hh b/tests/exchanger.hh index 13eedd2cbfa..0e4ed62ad0e 100644 --- a/tests/exchanger.hh +++ b/tests/exchanger.hh @@ -23,7 +23,7 @@ #include #include -#include +#include "util/std-compat.hh" // Single-element blocking queue template @@ -31,7 +31,7 @@ class exchanger { private: std::mutex _mutex; std::condition_variable _cv; - std::experimental::optional _element; + seastar::compat::optional _element; std::exception_ptr _exception; private: void interrupt_ptr(std::exception_ptr e) { diff --git a/tests/fstream_test.cc b/tests/fstream_test.cc index ad9d6d6532a..026c155731e 100644 --- a/tests/fstream_test.cc +++ b/tests/fstream_test.cc @@ -238,7 +238,7 @@ future<> test_consume_until_end(uint64_t size) { std::iota(expected.begin(), expected.end(), offset); offset += buf.size(); BOOST_REQUIRE(std::equal(buf.begin(), buf.end(), expected.begin())); - return make_ready_future::unconsumed_remainder>(std::experimental::nullopt); + return make_ready_future::unconsumed_remainder>(compat::nullopt); }; return do_with(make_file_input_stream(f), std::move(consumer), [] (input_stream& in, auto& consumer) { return in.consume(consumer).then([&in] { @@ -372,7 +372,7 @@ SEASTAR_TEST_CASE(test_fstream_slow_start) { static constexpr size_t requests_at_slow_start = 2; // 1 request + 1 read-ahead static constexpr size_t requests_at_full_speed = read_ahead + 1; // 1 request + read_ahead - std::experimental::optional initial_read_size; + compat::optional initial_read_size; auto read_whole_file_with_slow_start = [&] (auto fstr) { uint64_t total_read = 0; diff --git a/tests/futures_test.cc b/tests/futures_test.cc index 9e2f1a94cc3..eff9e92e65e 100644 --- a/tests/futures_test.cc +++ b/tests/futures_test.cc @@ -718,14 +718,13 @@ SEASTAR_TEST_CASE(test_futurize_from_tuple) { } SEASTAR_TEST_CASE(test_repeat_until_value) { - namespace stdx = std::experimental; return do_with(int(), [] (int& counter) { - return repeat_until_value([&counter] () -> future> { + return repeat_until_value([&counter] () -> future> { if (counter == 10000) { - return make_ready_future>(counter); + return make_ready_future>(counter); } else { ++counter; - return make_ready_future>(stdx::nullopt); + return make_ready_future>(compat::nullopt); } }).then([&counter] (int result) { BOOST_REQUIRE(counter == 10000); diff --git a/tests/l3_test.cc b/tests/l3_test.cc index 883ee51642e..7e406b02437 100644 --- a/tests/l3_test.cc +++ b/tests/l3_test.cc @@ -39,7 +39,7 @@ int main(int ac, char** av) { auto vnet = create_virtio_net_device(opts); interface netif(std::move(vnet)); - l3_protocol arp(&netif, eth_protocol_num::arp, []{ return std::experimental::optional(); }); + l3_protocol arp(&netif, eth_protocol_num::arp, []{ return compat::optional(); }); dump_arp_packets(arp); engine().run(); return 0; diff --git a/tests/linecount.cc b/tests/linecount.cc index 6332da89981..d1eaef84dec 100644 --- a/tests/linecount.cc +++ b/tests/linecount.cc @@ -40,7 +40,7 @@ struct reader { size_t count = 0; // for input_stream::consume(): - using unconsumed_remainder = std::experimental::optional>; + using unconsumed_remainder = compat::optional>; future operator()(temporary_buffer data) { if (data.empty()) { return make_ready_future(std::move(data)); diff --git a/tests/perf/perf_tests.hh b/tests/perf/perf_tests.hh index 729b78f6c33..e17fd2e9a4f 100644 --- a/tests/perf/perf_tests.hh +++ b/tests/perf/perf_tests.hh @@ -146,14 +146,14 @@ do_if_constexpr_ if_constexpr_(TrueFn&& true_fn, Fal template class concrete_performance_test final : public performance_test { - std::experimental::optional _test; + compat::optional _test; protected: virtual void set_up() override { _test.emplace(); } virtual void tear_down() noexcept override { - _test = std::experimental::nullopt; + _test = compat::nullopt; } [[gnu::hot]] diff --git a/tests/sstring_test.cc b/tests/sstring_test.cc index 4cfec4c8ec2..91e64b1be27 100644 --- a/tests/sstring_test.cc +++ b/tests/sstring_test.cc @@ -28,7 +28,7 @@ using namespace seastar; BOOST_AUTO_TEST_CASE(test_construction) { - BOOST_REQUIRE_EQUAL(sstring(std::experimental::string_view("abc")), sstring("abc")); + BOOST_REQUIRE_EQUAL(sstring(compat::string_view("abc")), sstring("abc")); } BOOST_AUTO_TEST_CASE(test_equality) { diff --git a/tests/thread_context_switch.cc b/tests/thread_context_switch.cc index 9f7866989c7..0bddcde9dd6 100644 --- a/tests/thread_context_switch.cc +++ b/tests/thread_context_switch.cc @@ -20,7 +20,6 @@ * Copyright (C) 2015 Cloudius Systems, Ltd. */ -#include #include "core/thread.hh" #include "core/semaphore.hh" #include "core/app-template.hh" diff --git a/util/optimized_optional.hh b/util/optimized_optional.hh index e410eb28f63..232b57e7bb2 100644 --- a/util/optimized_optional.hh +++ b/util/optimized_optional.hh @@ -22,8 +22,8 @@ #pragma once #include "util/gcc6-concepts.hh" +#include "util/std-compat.hh" -#include #include #include @@ -53,10 +53,10 @@ class optimized_optional { T _object; public: optimized_optional() = default; - optimized_optional(std::experimental::nullopt_t) noexcept { } + optimized_optional(compat::nullopt_t) noexcept { } optimized_optional(const T& obj) : _object(obj) { } optimized_optional(T&& obj) noexcept : _object(std::move(obj)) { } - optimized_optional(std::experimental::optional&& obj) noexcept { + optimized_optional(compat::optional&& obj) noexcept { if (obj) { _object = std::move(*obj); } @@ -64,7 +64,7 @@ public: optimized_optional(const optimized_optional&) = default; optimized_optional(optimized_optional&&) = default; - optimized_optional& operator=(std::experimental::nullopt_t) noexcept { + optimized_optional& operator=(compat::nullopt_t) noexcept { _object = T(); return *this; } @@ -101,4 +101,4 @@ public: } }; -} \ No newline at end of file +} diff --git a/util/std-compat.hh b/util/std-compat.hh new file mode 100644 index 00000000000..5c530fadab7 --- /dev/null +++ b/util/std-compat.hh @@ -0,0 +1,178 @@ +/* + * This file is open source software, licensed to you under the terms + * of the Apache License, Version 2.0 (the "License"). See the NOTICE file + * distributed with this work for additional information regarding copyright + * ownership. You may not use this file except in compliance with the License. + * + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +/* + * Copyright (C) 2018 ScyllaDB + */ + +#pragma once + +#ifdef SEASTAR_USE_STD_OPTIONAL_VARIANT_STRINGVIEW +#include +#include +#include +#else +#include +#include +#include +#endif + +namespace seastar { + +/// \cond internal + +namespace compat { + +#ifdef SEASTAR_USE_STD_OPTIONAL_VARIANT_STRINGVIEW + +template +using optional = std::optional; + +using nullopt_t = std::nullopt_t; + +inline constexpr auto nullopt = std::nullopt; + +template +inline constexpr optional> make_optional(T&& value) { + return std::make_optional(std::forward(value)); +} + +template > +using basic_string_view = std::basic_string_view; + +template > +std::string string_view_to_string(const basic_string_view& v) { + return std::string(v); +} + +template +using variant = std::variant; + +template +constexpr std::variant_alternative_t>& get(variant& v) { + return std::get(v); +} + +template +constexpr const std::variant_alternative_t>& get(const variant& v) { + return std::get(v); +} + +template +constexpr std::variant_alternative_t>&& get(variant&& v) { + return std::get(v); +} + +template +constexpr const std::variant_alternative_t>&& get(const variant&& v) { + return std::get(v); +} + +template +constexpr U& get(variant& v) { + return std::get(v); +} + +template +constexpr const U& get(const variant& v) { + return std::get(v); +} + +template +constexpr U&& get(variant&& v) { + return std::get(v); +} + +template +constexpr const U&& get(const variant&& v) { + return std::get(v); +} + +template +constexpr U* get_if(variant* v) { + return std::get_if(v); +} + +template +constexpr const U* get_if(const variant* v) { + return std::get_if(v); +} + +#else + +template +using optional = std::experimental::optional; + +using nullopt_t = std::experimental::nullopt_t; + +constexpr auto nullopt = std::experimental::nullopt; + +template +inline constexpr optional> make_optional(T&& value) { + return compat::make_optional(std::forward(value)); +} + +template > +using basic_string_view = std::experimental::basic_string_view; + +template > +std::string string_view_to_string(const basic_string_view& v) { + return v.to_string(); +} + +template +using variant = boost::variant; + +template +U& get(variant& v) { + return boost::get(v); +} + +template +U&& get(variant&& v) { + return boost::get(v); +} + +template +const U& get(const variant& v) { + return boost::get(v); +} + +template +const U&& get(const variant&& v) { + return boost::get(v); +} + +template +U* get_if(variant* v) { + return boost::get(v); +} + +template +const U* get_if(const variant* v) { + return boost::get(v); +} + +#endif + +using string_view = basic_string_view; + +} // namespace compat + +/// \endcond + +} // namespace seastar diff --git a/util/variant_utils.hh b/util/variant_utils.hh index fa1200e101e..1e786aa255b 100644 --- a/util/variant_utils.hh +++ b/util/variant_utils.hh @@ -21,7 +21,7 @@ #pragma once -#include +#include "util/std-compat.hh" #include #if (BOOST_VERSION < 105800) @@ -105,10 +105,14 @@ template inline auto visit(Variant&& variant, Args&&... args) { static_assert(sizeof...(Args) > 0, "At least one lambda must be provided for visitation"); +#ifdef SEASTAR_USE_STD_OPTIONAL_VARIANT_STRINGVIEW + return std::visit( +#else return boost::apply_visitor( +#endif make_visitor(std::forward(args)...), variant); -}; +} /// @}