Skip to content

Commit

Permalink
Add compatbility with C++17's library types
Browse files Browse the repository at this point in the history
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 <[email protected]>
Message-Id: <[email protected]>
  • Loading branch information
duarten authored and avikivity committed Aug 12, 2018
1 parent 8ad870f commit 0bbcbba
Show file tree
Hide file tree
Showing 70 changed files with 508 additions and 316 deletions.
8 changes: 8 additions & 0 deletions CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -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).
#
Expand Down Expand Up @@ -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.
##
Expand Down
12 changes: 12 additions & 0 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -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
---------------

Expand Down
3 changes: 2 additions & 1 deletion apps/iotune/iotune.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -561,7 +562,7 @@ void write_property_file(sstring conf_file, struct std::vector<disk_descriptor>
// (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<dev_t> candidate_id = {};
compat::optional<dev_t> candidate_id = {};
auto current = mnt_candidate;
do {
auto f = open_directory(current.string()).get0();
Expand Down
12 changes: 6 additions & 6 deletions apps/memcached/memcache.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
5 changes: 5 additions & 0 deletions configure.py
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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.
Expand Down Expand Up @@ -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'

Expand Down
8 changes: 3 additions & 5 deletions core/abort_source.hh
Original file line number Diff line number Diff line change
Expand Up @@ -23,18 +23,16 @@

#include "util/noncopyable_function.hh"
#include "util/optimized_optional.hh"
#include "util/std-compat.hh"

#include <boost/intrusive/list.hpp>

#include <exception>
#include <experimental/optional>

namespace bi = boost::intrusive;

namespace seastar {

namespace stdx = std::experimental;

/// \addtogroup fiber-module
/// @{

Expand Down Expand Up @@ -102,7 +100,7 @@ public:

private:
using subscription_list_type = bi::list<subscription, bi::constant_time_size<false>>;
stdx::optional<subscription_list_type> _subscriptions = subscription_list_type();
compat::optional<subscription_list_type> _subscriptions = subscription_list_type();

public:
/// Delays the invocation of the callback \c f until \ref request_abort() is called.
Expand Down Expand Up @@ -139,4 +137,4 @@ public:

/// @}

}
}
4 changes: 2 additions & 2 deletions core/dpdk_rte.cc
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@
#include "net/dpdk.hh"
#include "core/dpdk_rte.hh"
#include "util/conversions.hh"
#include <experimental/optional>
#include "util/std-compat.hh"
#include <rte_pci.h>

namespace seastar {
Expand All @@ -46,7 +46,7 @@ void eal::init(cpuset cpus, boost::program_options::variables_map opts)
string2vector("-n"), string2vector("1")
};

std::experimental::optional<std::string> hugepages_path;
compat::optional<std::string> hugepages_path;
if (opts.count("hugepages")) {
hugepages_path = opts["hugepages"].as<std::string>();
}
Expand Down
4 changes: 2 additions & 2 deletions core/execution_stage.hh
Original file line number Diff line number Diff line change
Expand Up @@ -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 <vector>
#include <experimental/optional>
#include <boost/range/irange.hpp>
#include <boost/range/adaptor/transformed.hpp>

Expand Down Expand Up @@ -313,7 +313,7 @@ class inheriting_concrete_execution_stage final {

sstring _name;
noncopyable_function<ReturnType (Args...)> _function;
std::vector<std::experimental::optional<per_group_stage_type>> _stage_for_group{max_scheduling_groups()};
std::vector<compat::optional<per_group_stage_type>> _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
Expand Down
4 changes: 2 additions & 2 deletions core/expiring_fifo.hh
Original file line number Diff line number Diff line change
Expand Up @@ -58,15 +58,15 @@ public:
using time_point = typename Clock::time_point;
private:
struct entry {
std::experimental::optional<T> payload; // disengaged means that it's expired
compat::optional<T> payload; // disengaged means that it's expired
timer<Clock> tr;
entry(T&& payload_) : payload(std::move(payload_)) {}
entry(const T& payload_) : payload(payload_) {}
entry(T payload_, expiring_fifo& ef, time_point timeout)
: payload(std::move(payload_))
, tr([this, &ef] {
ef._on_expiry(*payload);
payload = std::experimental::nullopt;
payload = compat::nullopt;
--ef._size;
ef.drop_expired_front();
})
Expand Down
1 change: 0 additions & 1 deletion core/fair_queue.hh
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@
#include "util/noncopyable_function.hh"
#include <queue>
#include <type_traits>
#include <experimental/optional>
#include <chrono>
#include <unordered_set>
#include <cmath>
Expand Down
4 changes: 2 additions & 2 deletions core/file.hh
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@
#include "core/align.hh"
#include "core/future-util.hh"
#include "core/fair_queue.hh"
#include <experimental/optional>
#include "util/std-compat.hh"
#include <system_error>
#include <sys/stat.h>
#include <sys/ioctl.h>
Expand Down Expand Up @@ -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<directory_entry_type> type;
compat::optional<directory_entry_type> type;
};

/// File open options
Expand Down
2 changes: 1 addition & 1 deletion core/fstream.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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<promise<>> _done;
compat::optional<promise<>> _done;
size_t _current_buffer_size;
bool _in_slow_start = false;
using unused_ratio_target = std::ratio<25, 100>;
Expand Down
20 changes: 10 additions & 10 deletions core/future-util.hh
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@
#include <tuple>
#include <iterator>
#include <vector>
#include <experimental/optional>
#include "util/std-compat.hh"
#include "util/tuple_utils.hh"
#include "util/noncopyable_function.hh"

Expand Down Expand Up @@ -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<std::exception_ptr> ex;
compat::optional<std::exception_ptr> ex;
promise<> pr;
~parallel_for_each_state() {
if (ex) {
Expand Down Expand Up @@ -312,11 +312,11 @@ struct repeat_until_value_type_helper;

/// Type helper for repeat_until_value()
template <typename T>
struct repeat_until_value_type_helper<future<std::experimental::optional<T>>> {
struct repeat_until_value_type_helper<future<compat::optional<T>>> {
/// 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<T>;
using optional_type = compat::optional<T>;
/// Return type of repeat_until_value()
using future_type = future<value_type>;
/// Return type of \c AsyncAction
Expand All @@ -331,12 +331,12 @@ using repeat_until_value_return_type
namespace internal {

template <typename AsyncAction, typename T>
class repeat_until_value_state final : public continuation_base<std::experimental::optional<T>> {
class repeat_until_value_state final : public continuation_base<compat::optional<T>> {
promise<T> _promise;
AsyncAction _action;
public:
explicit repeat_until_value_state(AsyncAction action) : _action(std::move(action)) {}
repeat_until_value_state(std::experimental::optional<T> st, AsyncAction action) : repeat_until_value_state(std::move(action)) {
repeat_until_value_state(compat::optional<T> st, AsyncAction action) : repeat_until_value_state(std::move(action)) {
this->_state.set(std::make_tuple(std::move(st)));
}
future<T> get_future() { return _promise.get_future(); }
Expand Down Expand Up @@ -370,18 +370,18 @@ public:
_promise.set_exception(std::current_exception());
return;
}
this->_state.set(std::experimental::nullopt);
this->_state.set(compat::nullopt);
schedule(std::move(zis));
}
};

}

/// Invokes given action until it fails or the function requests iteration to stop by returning
/// an engaged \c future<std::experimental::optional<T>>. The value is extracted from the
/// an engaged \c future<compat::optional<T>>. 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<std::experimental::optional<T>>.
/// \param action a callable taking no arguments, returning a future<compat::optional<T>>.
/// 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.
Expand Down Expand Up @@ -425,7 +425,7 @@ repeat_until_value(AsyncAction action) {
} while (!need_preempt());

try {
auto state = std::make_unique<internal::repeat_until_value_state<futurized_action_type, value_type>>(std::experimental::nullopt, std::move(futurized_action));
auto state = std::make_unique<internal::repeat_until_value_state<futurized_action_type, value_type>>(compat::nullopt, std::move(futurized_action));
auto f = state->get_future();
schedule(std::move(state));
return f;
Expand Down
8 changes: 3 additions & 5 deletions core/gate.hh
Original file line number Diff line number Diff line change
Expand Up @@ -22,16 +22,14 @@
#pragma once

#include "future.hh"
#include <experimental/optional>
#include "util/std-compat.hh"
#include <exception>

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 {
Expand All @@ -48,7 +46,7 @@ public:
/// requests have completed. The \c gate class provides a solution.
class gate {
size_t _count = 0;
stdx::optional<promise<>> _stopped;
compat::optional<promise<>> _stopped;
public:
/// Registers an in-progress request.
///
Expand Down Expand Up @@ -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();
}
Expand Down
8 changes: 4 additions & 4 deletions core/iostream-impl.hh
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ inline future<temporary_buffer<char>> 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<char> buffer) -> std::experimental::optional<temporary_buffer<char>> {
return get().then([&] (temporary_buffer<char> buffer) -> compat::optional<temporary_buffer<char>> {
if (buffer.size() >= n) {
buffer.trim_front(n);
return std::move(buffer);
Expand Down Expand Up @@ -206,7 +206,7 @@ input_stream<CharType>::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.
//
Expand Down Expand Up @@ -429,7 +429,7 @@ output_stream<CharType>::poll_flush() {
// flush was canceled, do nothing
_flushing = false;
_in_batch.value().set_value();
_in_batch = std::experimental::nullopt;
_in_batch = compat::nullopt;
return;
}

Expand Down Expand Up @@ -495,7 +495,7 @@ template <typename CharType>
struct stream_copy_consumer {
private:
output_stream<CharType>& _os;
using unconsumed_remainder = std::experimental::optional<temporary_buffer<CharType>>;
using unconsumed_remainder = compat::optional<temporary_buffer<CharType>>;
public:
stream_copy_consumer(output_stream<CharType>& os) : _os(os) {
}
Expand Down
Loading

0 comments on commit 0bbcbba

Please sign in to comment.