diff options
Diffstat (limited to 'src/seastar/include')
238 files changed, 50346 insertions, 0 deletions
diff --git a/src/seastar/include/seastar/core/abort_on_ebadf.hh b/src/seastar/include/seastar/core/abort_on_ebadf.hh new file mode 100644 index 000000000..7cb8c05b1 --- /dev/null +++ b/src/seastar/include/seastar/core/abort_on_ebadf.hh @@ -0,0 +1,36 @@ +/* + * 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 2019 ScyllaDB + */ + +#pragma once + +namespace seastar { + +/// Determines whether seastar should throw or abort when operation made by +/// seastar fails because the target file descriptor is not valid. This is +/// detected when underlying system calls return EBADF or ENOTSOCK. +/// The default behavior is to throw std::system_error. +void set_abort_on_ebadf(bool do_abort); + +/// Queries the current setting for seastar's behavior on invalid file descriptor access. +/// See set_abort_on_ebadf(). +bool is_abort_on_ebadf_enabled(); + +} diff --git a/src/seastar/include/seastar/core/abort_on_expiry.hh b/src/seastar/include/seastar/core/abort_on_expiry.hh new file mode 100644 index 000000000..8d54ad0eb --- /dev/null +++ b/src/seastar/include/seastar/core/abort_on_expiry.hh @@ -0,0 +1,60 @@ +/* + * 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) 2022 ScyllaDB. + */ + +#pragma once + +#include <seastar/core/abort_source.hh> +#include <seastar/core/timer.hh> +#include <seastar/core/lowres_clock.hh> + +namespace seastar { + +/// \addtogroup fiber-module +/// @{ + +/// Facility to tie a timeout with an abort source +/// Can be used to make abortanle fibers also support timeouts +template<typename Clock = lowres_clock> +class abort_on_expiry { + timer<Clock> _tr; + seastar::abort_source _as; +public: + using clock = Clock; + using time_point = typename Clock::time_point; + /// Creates a timer and an abort source associated with it + /// When the timer reaches timeout point it triggers an + /// abort autimatically + abort_on_expiry(time_point timeout) : _tr([this] { + _as.request_abort(); + }) { + _tr.arm(timeout); + } + abort_on_expiry(abort_on_expiry&&) = delete; + + /// \returns abort source associated with the timeout + seastar::abort_source& abort_source() { + return _as; + } +}; + +/// @} + +} diff --git a/src/seastar/include/seastar/core/abort_source.hh b/src/seastar/include/seastar/core/abort_source.hh new file mode 100644 index 000000000..8e7d57ee0 --- /dev/null +++ b/src/seastar/include/seastar/core/abort_source.hh @@ -0,0 +1,192 @@ +/* + * 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) 2017 ScyllaDB. + */ + +#pragma once + +#include <seastar/util/noncopyable_function.hh> +#include <seastar/util/optimized_optional.hh> +#include <seastar/util/std-compat.hh> + +#include <boost/intrusive/list.hpp> + +#include <exception> + +namespace bi = boost::intrusive; + +namespace seastar { + +/// \addtogroup fiber-module +/// @{ + +/// Exception thrown when an \ref abort_source object has been +/// notified by the \ref abort_source::request_abort() method. +class abort_requested_exception : public std::exception { +public: + virtual const char* what() const noexcept override { + return "abort requested"; + } +}; + +/// Facility to communicate a cancellation request to a fiber. +/// Callbacks can be registered with the \c abort_source, which are called +/// atomically with a call to request_abort(). +class abort_source { + using subscription_callback_type = noncopyable_function<void (const std::optional<std::exception_ptr>&) noexcept>; + using naive_subscription_callback_type = noncopyable_function<void() noexcept>; + +public: + /// Represents a handle to the callback registered by a given fiber. Ending the + /// lifetime of the \c subscription will unregister the callback, if it hasn't + /// been invoked yet. + class subscription : public bi::list_base_hook<bi::link_mode<bi::auto_unlink>> { + friend class abort_source; + + subscription_callback_type _target; + + explicit subscription(abort_source& as, subscription_callback_type target) + : _target(std::move(target)) { + as._subscriptions->push_back(*this); + } + + struct naive_cb_tag {}; // to disambiguate constructors + explicit subscription(naive_cb_tag, abort_source& as, naive_subscription_callback_type naive_cb) + : _target([cb = std::move(naive_cb)] (const std::optional<std::exception_ptr>&) noexcept { cb(); }) { + as._subscriptions->push_back(*this); + } + + void on_abort(const std::optional<std::exception_ptr>& ex) noexcept { + _target(ex); + } + + public: + subscription() = default; + + subscription(subscription&& other) noexcept(std::is_nothrow_move_constructible<subscription_callback_type>::value) + : _target(std::move(other._target)) { + subscription_list_type::node_algorithms::swap_nodes(other.this_ptr(), this_ptr()); + } + + subscription& operator=(subscription&& other) noexcept(std::is_nothrow_move_assignable<subscription_callback_type>::value) { + if (this != &other) { + _target = std::move(other._target); + unlink(); + subscription_list_type::node_algorithms::swap_nodes(other.this_ptr(), this_ptr()); + } + return *this; + } + + explicit operator bool() const noexcept { + return is_linked(); + } + }; + +private: + using subscription_list_type = bi::list<subscription, bi::constant_time_size<false>>; + std::optional<subscription_list_type> _subscriptions = subscription_list_type(); + std::exception_ptr _ex; + + void do_request_abort(std::optional<std::exception_ptr> ex) noexcept { + assert(_subscriptions); + _ex = ex.value_or(get_default_exception()); + auto subs = std::exchange(_subscriptions, std::nullopt); + while (!subs->empty()) { + subscription& s = subs->front(); + s.unlink(); + s.on_abort(ex); + } + } + +public: + abort_source() = default; + virtual ~abort_source() = default; + + abort_source(abort_source&&) = default; + abort_source& operator=(abort_source&&) = default; + + /// Delays the invocation of the callback \c f until \ref request_abort() is called. + /// \returns an engaged \ref optimized_optional containing a \ref subscription that can be used to control + /// the lifetime of the callback \c f, if \ref abort_requested() is \c false. Otherwise, + /// returns a disengaged \ref optimized_optional. + template <typename Func> + SEASTAR_CONCEPT(requires + requires (Func f, const std::optional<std::exception_ptr>& opt_ex) { { f(opt_ex) } noexcept -> std::same_as<void>; } + || requires (Func f) { { f() } noexcept -> std::same_as<void>; } + ) + [[nodiscard]] + optimized_optional<subscription> subscribe(Func&& f) { + if (abort_requested()) { + return { }; + } + if constexpr (std::is_invocable_v<Func, std::exception_ptr>) { + return { subscription(*this, std::forward<Func>(f)) }; + } else { + return { subscription(subscription::naive_cb_tag{}, *this, std::forward<Func>(f)) }; + } + } + + /// Requests that the target operation be aborted. Current subscriptions + /// are invoked inline with this call with a disengaged optional<std::exception_ptr>, + /// and no new ones can be registered. + /// Must be called exactly once, otherwise the program will be aborted. + void request_abort() noexcept { + do_request_abort(std::nullopt); + } + + /// Requests that the target operation be aborted with a given \c exception_ptr. + /// Current subscriptions are invoked inline with this exception, + /// and no new ones can be registered. + /// Must be called exactly once, otherwise the program will be aborted. + void request_abort_ex(std::exception_ptr ex) noexcept { + do_request_abort(std::make_optional(std::move(ex))); + } + + /// Requests that the target operation be aborted with a given \c Exception object. + /// Current subscriptions are invoked inline with this exception, converted to std::exception_ptr, + /// and no new ones can be registered. + /// Must be called exactly once, otherwise the program will be aborted. + template <typename Exception> + void request_abort_ex(Exception&& e) noexcept { + do_request_abort(std::make_optional(std::make_exception_ptr(std::forward<Exception>(e)))); + } + + /// Returns whether an abort has been requested. + bool abort_requested() const noexcept { + return !_subscriptions; + } + + + /// Throws a \ref abort_requested_exception if cancellation has been requested. + void check() const { + if (abort_requested()) { + std::rethrow_exception(_ex); + } + } + + /// Returns the default exception type (\ref abort_requested_exception) for this abort source. + /// Overridable by derived classes. + virtual std::exception_ptr get_default_exception() const noexcept { + return make_exception_ptr(abort_requested_exception()); + } +}; + +/// @} + +} diff --git a/src/seastar/include/seastar/core/abortable_fifo.hh b/src/seastar/include/seastar/core/abortable_fifo.hh new file mode 100644 index 000000000..7af78f611 --- /dev/null +++ b/src/seastar/include/seastar/core/abortable_fifo.hh @@ -0,0 +1,256 @@ +/* + * 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) 2022 ScyllaDB + */ + +#pragma once + +#include <seastar/core/future.hh> +#include <seastar/core/chunked_fifo.hh> +#include <stdexcept> +#include <exception> +#include <memory> +#include <seastar/core/abort_source.hh> + +namespace seastar { + +namespace internal { + +SEASTAR_CONCEPT( + template <typename Aborter, typename T> + concept aborter = requires (Aborter abort, T& t) { + { abort(t) } noexcept -> std::same_as<void>; + }; +) + +// This class satisfies 'aborter' concept and is used by default +template<typename... T> +struct noop_aborter { + void operator()(T...) noexcept {}; +}; + + +/// Container for elements with support for cancelation of entries. +/// +/// OnAbort is a functor which will be called with a reference to T right before it expires. +/// T is removed and destroyed from the container immediately after OnAbort returns. +/// OnAbort callback must not modify the container, it can only modify its argument. +/// +/// The container can only be moved before any elements are pushed. +/// +template <typename T, typename OnAbort = noop_aborter<T>> +SEASTAR_CONCEPT( requires aborter<OnAbort, T> ) +class abortable_fifo { +private: + struct entry { + std::optional<T> payload; // disengaged means that it's expired + optimized_optional<abort_source::subscription> sub; + entry(T&& payload_) : payload(std::move(payload_)) {} + entry(const T& payload_) : payload(payload_) {} + entry(T payload_, abortable_fifo& ef, abort_source& as) + : payload(std::move(payload_)) + , sub(as.subscribe([this, &ef] () noexcept { + ef._on_abort(*payload); + payload = std::nullopt; + --ef._size; + ef.drop_expired_front(); + })) {} + entry() = default; + entry(entry&& x) = delete; + entry(const entry& x) = delete; + }; + + // If engaged, represents the first element. + // This is to avoid large allocations done by chunked_fifo for single-element cases. + // abortable_fifo is used to implement wait lists in synchronization primitives + // and in some uses it's common to have at most one waiter. + std::unique_ptr<entry> _front; + + // There is an invariant that the front element is never expired. + chunked_fifo<entry> _list; + OnAbort _on_abort; + size_t _size = 0; + + // Ensures that front() is not expired by dropping expired elements from the front. + void drop_expired_front() noexcept { + while (!_list.empty() && !_list.front().payload) { + _list.pop_front(); + } + if (_front && !_front->payload) { + _front.reset(); + } + } +public: + abortable_fifo() noexcept = default; + abortable_fifo(OnAbort on_abort) noexcept(std::is_nothrow_move_constructible_v<OnAbort>) : _on_abort(std::move(on_abort)) {} + + abortable_fifo(abortable_fifo&& o) noexcept + : abortable_fifo(std::move(o._on_abort)) { + // entry objects hold a reference to this so non-empty containers cannot be moved. + assert(o._size == 0); + } + + abortable_fifo& operator=(abortable_fifo&& o) noexcept { + if (this != &o) { + this->~abortable_fifo(); + new (this) abortable_fifo(std::move(o)); + } + return *this; + } + + /// Checks if container contains any elements + /// + /// \note Inside OnAbort callback, the expired element is still contained. + /// + /// \return true if and only if there are any elements contained. + bool empty() const noexcept { + return _size == 0; + } + + /// Equivalent to !empty() + explicit operator bool() const noexcept { + return !empty(); + } + + /// Returns a reference to the element in the front. + /// Valid only when !empty(). + T& front() noexcept { + if (_front) { + return *_front->payload; + } + return *_list.front().payload; + } + + /// Returns a reference to the element in the front. + /// Valid only when !empty(). + const T& front() const noexcept { + if (_front) { + return *_front->payload; + } + return *_list.front().payload; + } + + /// Returns the number of elements contained. + /// + /// \note Expired elements are not contained. Expiring element is still contained when OnAbort is called. + size_t size() const noexcept { + return _size; + } + + /// Reserves storage in the container for at least 'size' elements. + /// Note that expired elements may also take space when they are not in the front of the queue. + /// + /// Doesn't give any guarantees about exception safety of subsequent push_back(). + void reserve(size_t size) { + return _list.reserve(size); + } + + /// Adds element to the back of the queue. + /// The element will never expire. + void push_back(const T& payload) { + if (_size == 0) { + _front = std::make_unique<entry>(payload); + } else { + _list.emplace_back(payload); + } + ++_size; + } + + /// Adds element to the back of the queue. + /// The element will never expire. + void push_back(T&& payload) { + if (_size == 0) { + _front = std::make_unique<entry>(std::move(payload)); + } else { + _list.emplace_back(std::move(payload)); + } + ++_size; + } + + /// Adds element to the back of the queue. + /// The element will expire when abort source is triggered. + void push_back(T&& payload, abort_source& as) { + if (as.abort_requested()) { + _on_abort(payload); + return; + } + if (_size == 0) { + _front = std::make_unique<entry>(std::move(payload), *this, as); + } else { + _list.emplace_back(std::move(payload), *this, as); + } + ++_size; + } + + /// Create element in place at the back of the queue. + template<typename... U> + T& emplace_back(U&&... args) { + if (_size == 0) { + _front = std::make_unique<entry>(); + _front->payload.emplace(std::forward<U>(args)...); + _size = 1; + return *_front->payload; + } else { + _list.emplace_back(); + _list.back().payload.emplace(std::forward<U>(args)...); + ++_size; + return *_list.back().payload; + } + } + + /// Make an element at the back of the queue abortable + /// The element will expire when abort source is triggered. + /// Valid only when !empty(). + /// Cannot be called on an element that si already associated + /// with an abort source. + void make_back_abortable(abort_source& as) { + entry* e = _front.get(); + if (!_list.empty()) { + e = &_list.back(); + } + assert(!e->sub); + auto aborter = [this, e] () noexcept { + _on_abort(*e->payload); + e->payload = std::nullopt; + --_size; + drop_expired_front(); + }; + if (as.abort_requested()) { + aborter(); + return; + } + e->sub = as.subscribe(std::move(aborter)); + } + + /// Removes the element at the front. + /// Can be called only if !empty(). + void pop_front() noexcept { + if (_front) { + _front.reset(); + } else { + _list.pop_front(); + } + --_size; + drop_expired_front(); + } +}; + +} +} + diff --git a/src/seastar/include/seastar/core/alien.hh b/src/seastar/include/seastar/core/alien.hh new file mode 100644 index 000000000..055b9bd79 --- /dev/null +++ b/src/seastar/include/seastar/core/alien.hh @@ -0,0 +1,232 @@ +// -*- mode:C++; tab-width:4; c-basic-offset:4; indent-tabs-mode:nil -*- +/* + * 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 Red Hat + */ + +#pragma once + +#include <atomic> +#include <deque> +#include <future> +#include <memory> + +#include <boost/lockfree/queue.hpp> + +#include <seastar/core/future.hh> +#include <seastar/core/cacheline.hh> +#include <seastar/core/sstring.hh> +#include <seastar/core/metrics_registration.hh> + +/// \file + +namespace seastar { + +class reactor; + +/// \brief Integration with non-seastar applications. +namespace alien { + +class message_queue { + static constexpr size_t batch_size = 128; + static constexpr size_t prefetch_cnt = 2; + struct work_item; + struct lf_queue_remote { + reactor* remote; + }; + using lf_queue_base = boost::lockfree::queue<work_item*>; + // use inheritence to control placement order + struct lf_queue : lf_queue_remote, lf_queue_base { + lf_queue(reactor* remote) + : lf_queue_remote{remote}, lf_queue_base{batch_size} {} + void maybe_wakeup(); + } _pending; + struct alignas(seastar::cache_line_size) { + std::atomic<size_t> value{0}; + } _sent; + // keep this between two structures with statistics + // this makes sure that they have at least one cache line + // between them, so hw prefetcher will not accidentally prefetch + // cache line used by another cpu. + metrics::metric_groups _metrics; + struct alignas(seastar::cache_line_size) { + size_t _received = 0; + size_t _last_rcv_batch = 0; + }; + struct work_item { + virtual ~work_item() = default; + virtual void process() = 0; + }; + template <typename Func> + struct async_work_item : work_item { + Func _func; + async_work_item(Func&& func) : _func(std::move(func)) {} + void process() override { + _func(); + } + }; + template<typename Func> + size_t process_queue(lf_queue& q, Func process); + void submit_item(std::unique_ptr<work_item> wi); +public: + message_queue(reactor *to); + void start(); + void stop(); + template <typename Func> + void submit(Func&& func) { + auto wi = std::make_unique<async_work_item<Func>>(std::forward<Func>(func)); + submit_item(std::move(wi)); + } + size_t process_incoming(); + bool pure_poll_rx() const; +}; + +namespace internal { + +struct qs_deleter { + unsigned count; + qs_deleter(unsigned n = 0) : count(n) {} + qs_deleter(const qs_deleter& d) : count(d.count) {} + void operator()(message_queue* qs) const; +}; + +} + +/// Represents the Seastar system from alien's point of view. In a normal +/// system, there is just one instance, but for in-process clustering testing +/// there may be more than one. Function such as run_on() direct messages to +/// and (instance, shard) tuple. +class instance { + using qs = std::unique_ptr<message_queue[], internal::qs_deleter>; +public: + static qs create_qs(const std::vector<reactor*>& reactors); + qs _qs; + bool poll_queues(); + bool pure_poll_queues(); +}; + +namespace internal { + +extern instance* default_instance; + +} + +/// Runs a function on a remote shard from an alien thread where engine() is not available. +/// +/// \param instance designates the Seastar instance to process the message +/// \param shard designates the shard to run the function on +/// \param func a callable to run on shard \c t. If \c func is a temporary object, +/// its lifetime will be extended by moving it. If \c func is a reference, +/// the caller must guarantee that it will survive the call. +/// \note the func must not throw and should return \c void. as we cannot identify the +/// alien thread, hence we are not able to post the fulfilled promise to the +/// message queue managed by the shard executing the alien thread which is +/// interested to the return value. Please use \c submit_to() instead, if +/// \c func throws. +template <typename Func> +void run_on(instance& instance, unsigned shard, Func func) { + instance._qs[shard].submit(std::move(func)); +} + +/// Runs a function on a remote shard from an alien thread where engine() is not available. +/// +/// \param shard designates the shard to run the function on +/// \param func a callable to run on shard \c t. If \c func is a temporary object, +/// its lifetime will be extended by moving it. If \c func is a reference, +/// the caller must guarantee that it will survive the call. +/// \note the func must not throw and should return \c void. as we cannot identify the +/// alien thread, hence we are not able to post the fulfilled promise to the +/// message queue managed by the shard executing the alien thread which is +/// interested to the return value. Please use \c submit_to() instead, if +/// \c func throws. +template <typename Func> +[[deprecated("Use run_on(instance&, unsigned shard, Func) instead")]] +void run_on(unsigned shard, Func func) { + run_on(*internal::default_instance, shard, std::move(func)); +} + +namespace internal { +template<typename Func> +using return_value_t = typename futurize<std::invoke_result_t<Func>>::value_type; + +template<typename Func, + bool = std::is_empty_v<return_value_t<Func>>> +struct return_type_of { + using type = void; + static void set(std::promise<void>& p, return_value_t<Func>&&) { + p.set_value(); + } +}; +template<typename Func> +struct return_type_of<Func, false> { + using return_tuple_t = typename futurize<std::invoke_result_t<Func>>::tuple_type; + using type = std::tuple_element_t<0, return_tuple_t>; + static void set(std::promise<type>& p, return_value_t<Func>&& t) { +#if SEASTAR_API_LEVEL < 5 + p.set_value(std::get<0>(std::move(t))); +#else + p.set_value(std::move(t)); +#endif + } +}; +template <typename Func> using return_type_t = typename return_type_of<Func>::type; +} + +/// Runs a function on a remote shard from an alien thread where engine() is not available. +/// +/// \param instance designates the Seastar instance to process the message +/// \param shard designates the shard to run the function on +/// \param func a callable to run on \c shard. If \c func is a temporary object, +/// its lifetime will be extended by moving it. If \c func is a reference, +/// the caller must guarantee that it will survive the call. +/// \return whatever \c func returns, as a \c std::future<> +/// \note the caller must keep the returned future alive until \c func returns +template<typename Func, typename T = internal::return_type_t<Func>> +std::future<T> submit_to(instance& instance, unsigned shard, Func func) { + std::promise<T> pr; + auto fut = pr.get_future(); + run_on(instance, shard, [pr = std::move(pr), func = std::move(func)] () mutable { + // std::future returned via std::promise above. + (void)func().then_wrapped([pr = std::move(pr)] (auto&& result) mutable { + try { + internal::return_type_of<Func>::set(pr, result.get()); + } catch (...) { + pr.set_exception(std::current_exception()); + } + }); + }); + return fut; +} + +/// Runs a function on a remote shard from an alien thread where engine() is not available. +/// +/// \param shard designates the shard to run the function on +/// \param func a callable to run on \c shard. If \c func is a temporary object, +/// its lifetime will be extended by moving it. If \c func is a reference, +/// the caller must guarantee that it will survive the call. +/// \return whatever \c func returns, as a \c std::future<> +/// \note the caller must keep the returned future alive until \c func returns +template<typename Func, typename T = internal::return_type_t<Func>> +[[deprecated("Use submit_to(instance&, unsigned shard, Func) instead.")]] +std::future<T> submit_to(unsigned shard, Func func) { + return submit_to(*internal::default_instance, shard, std::move(func)); +} + +} +} diff --git a/src/seastar/include/seastar/core/align.hh b/src/seastar/include/seastar/core/align.hh new file mode 100644 index 000000000..5dd07c279 --- /dev/null +++ b/src/seastar/include/seastar/core/align.hh @@ -0,0 +1,55 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <cstdint> +#include <cstdlib> + +namespace seastar { + +template <typename T> +inline constexpr +T align_up(T v, T align) { + return (v + align - 1) & ~(align - 1); +} + +template <typename T> +inline constexpr +T* align_up(T* v, size_t align) { + static_assert(sizeof(T) == 1, "align byte pointers only"); + return reinterpret_cast<T*>(align_up(reinterpret_cast<uintptr_t>(v), align)); +} + +template <typename T> +inline constexpr +T align_down(T v, T align) { + return v & ~(align - 1); +} + +template <typename T> +inline constexpr +T* align_down(T* v, size_t align) { + static_assert(sizeof(T) == 1, "align byte pointers only"); + return reinterpret_cast<T*>(align_down(reinterpret_cast<uintptr_t>(v), align)); +} + +} diff --git a/src/seastar/include/seastar/core/aligned_buffer.hh b/src/seastar/include/seastar/core/aligned_buffer.hh new file mode 100644 index 000000000..2cc0f5a74 --- /dev/null +++ b/src/seastar/include/seastar/core/aligned_buffer.hh @@ -0,0 +1,45 @@ +/* + * 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) 2016 ScyllaDB. + */ +#pragma once +#include <stdlib.h> +#include <memory> +#include <stdexcept> + +namespace seastar { + +namespace internal { +void* allocate_aligned_buffer_impl(size_t size, size_t align); +} + +struct free_deleter { + void operator()(void* p) { ::free(p); } +}; + +template <typename CharType> +inline +std::unique_ptr<CharType[], free_deleter> allocate_aligned_buffer(size_t size, size_t align) { + static_assert(sizeof(CharType) == 1, "must allocate byte type"); + void* ret = internal::allocate_aligned_buffer_impl(size, align); + return std::unique_ptr<CharType[], free_deleter>(reinterpret_cast<CharType *>(ret)); +} + + +} diff --git a/src/seastar/include/seastar/core/app-template.hh b/src/seastar/include/seastar/core/app-template.hh new file mode 100644 index 000000000..00bc986c8 --- /dev/null +++ b/src/seastar/include/seastar/core/app-template.hh @@ -0,0 +1,173 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ +#pragma once + +#include <boost/program_options.hpp> +#include <functional> +#include <seastar/core/future.hh> +#include <seastar/core/smp.hh> +#include <seastar/core/smp_options.hh> +#include <seastar/core/sstring.hh> +#include <seastar/util/program-options.hh> +#include <seastar/core/metrics_api.hh> +#include <seastar/core/scollectd.hh> +#include <seastar/util/log-cli.hh> +#include <chrono> + +namespace seastar { + +namespace alien { + +class instance; + +} + +class app_template { +public: + struct config { + /// The name of the application. + /// + /// Will be used in the --help output to distinguish command line args + /// registered by the application, as opposed to those registered by + /// seastar and its subsystems. + sstring name = "App"; + /// The description of the application. + /// + /// Will be printed on the top of the --help output. Lines should be + /// hard-wrapped for 80 chars. + sstring description = ""; + std::chrono::duration<double> default_task_quota = std::chrono::microseconds(500); + /// \brief Handle SIGINT/SIGTERM by calling reactor::stop() + /// + /// When true, Seastar will set up signal handlers for SIGINT/SIGTERM that call + /// reactor::stop(). The reactor will then execute callbacks installed by + /// reactor::at_exit(). + /// + /// When false, Seastar will not set up signal handlers for SIGINT/SIGTERM + /// automatically. The default behavior (terminate the program) will be kept. + /// You can adjust the behavior of SIGINT/SIGTERM by installing signal handlers + /// via reactor::handle_signal(). + bool auto_handle_sigint_sigterm = true; + /// Specifies the default value for linux-aio I/O control blocks. This translates + /// to the maximum number of sockets the shard can handle. + unsigned max_networking_aio_io_control_blocks = 10000; + /// The amount of memory that should not be used by the seastar allocator, + /// additional to the amount of memory already reserved for the OS. + /// This can be used when the application allocates some of its memory using the + /// seastar allocator, and some using the system allocator, in particular when it + /// uses the mmap system call with MAP_ANONYMOUS which is not overridden in seastar. + size_t reserve_additional_memory = 0; + config() {} + }; + + /// Seastar configuration options + struct seastar_options : public program_options::option_group { + /// The name of the application. + /// + /// Will be used in the --help output to distinguish command line args + /// registered by the application, as opposed to those registered by + /// seastar and its subsystems. + sstring name = "App"; + /// The description of the application. + /// + /// Will be printed on the top of the --help output. Lines should be + /// hard-wrapped for 80 chars. + sstring description = ""; + /// \brief Handle SIGINT/SIGTERM by calling reactor::stop() + /// + /// When true, Seastar will set up signal handlers for SIGINT/SIGTERM that call + /// reactor::stop(). The reactor will then execute callbacks installed by + /// reactor::at_exit(). + /// + /// When false, Seastar will not set up signal handlers for SIGINT/SIGTERM + /// automatically. The default behavior (terminate the program) will be kept. + /// You can adjust the behavior of SIGINT/SIGTERM by installing signal handlers + /// via reactor::handle_signal(). + bool auto_handle_sigint_sigterm = true; + /// Configuration options for the reactor. + reactor_options reactor_opts; + /// Configuration for the metrics sub-system. + metrics::options metrics_opts; + /// Configuration options for the smp aspect of seastar. + smp_options smp_opts; + /// Configuration for the scollectd sub-system. + scollectd::options scollectd_opts; + /// Configuration for the logging sub-system. + log_cli::options log_opts; + + seastar_options(); + }; + + using configuration_reader = std::function<void (boost::program_options::variables_map&)>; +private: + // unique_ptr to avoid pulling in alien.hh. + std::unique_ptr<alien::instance> _alien; + // reactor destruction is asynchronous, so we must let the last reactor + // destroy the smp instance + std::shared_ptr<smp> _smp; + seastar_options _opts; + boost::program_options::options_description _app_opts; + boost::program_options::options_description _seastar_opts; + boost::program_options::options_description _opts_conf_file; + boost::program_options::positional_options_description _pos_opts; + std::optional<boost::program_options::variables_map> _configuration; + configuration_reader _conf_reader; + + configuration_reader get_default_configuration_reader(); +public: + struct positional_option { + const char* name; + const boost::program_options::value_semantic* value_semantic; + const char* help; + int max_count; + }; +public: + explicit app_template(seastar_options opts); + explicit app_template(config cfg = config()); + ~app_template(); + + const seastar_options& options() const; + + boost::program_options::options_description& get_options_description(); + boost::program_options::options_description& get_conf_file_options_description(); + boost::program_options::options_description_easy_init add_options(); + void add_positional_options(std::initializer_list<positional_option> options); + boost::program_options::variables_map& configuration(); + int run_deprecated(int ac, char ** av, std::function<void ()>&& func) noexcept; + + void set_configuration_reader(configuration_reader conf_reader); + + /// Obtains an alien::instance object that can be used to send messages + /// to Seastar shards from non-Seastar threads. + alien::instance& alien() { return *_alien; } + + // Runs given function and terminates the application when the future it + // returns resolves. The value with which the future resolves will be + // returned by this function. + int run(int ac, char ** av, std::function<future<int> ()>&& func) noexcept; + + // Like run() which takes std::function<future<int>()>, but returns + // with exit code 0 when the future returned by func resolves + // successfully. + int run(int ac, char ** av, std::function<future<> ()>&& func) noexcept; +}; + +} diff --git a/src/seastar/include/seastar/core/array_map.hh b/src/seastar/include/seastar/core/array_map.hh new file mode 100644 index 000000000..14ec4118a --- /dev/null +++ b/src/seastar/include/seastar/core/array_map.hh @@ -0,0 +1,50 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <array> + +namespace seastar { + +// unordered_map implemented as a simple array + +template <typename Value, size_t Max> +class array_map { + std::array<Value, Max> _a {}; +public: + array_map(std::initializer_list<std::pair<size_t, Value>> i) { + for (auto kv : i) { + _a[kv.first] = kv.second; + } + } + Value& operator[](size_t key) { return _a[key]; } + const Value& operator[](size_t key) const { return _a[key]; } + + Value& at(size_t key) { + if (key >= Max) { + throw std::out_of_range(std::to_string(key) + " >= " + std::to_string(Max)); + } + return _a[key]; + } +}; + +} diff --git a/src/seastar/include/seastar/core/bitops.hh b/src/seastar/include/seastar/core/bitops.hh new file mode 100644 index 000000000..2b7b28a54 --- /dev/null +++ b/src/seastar/include/seastar/core/bitops.hh @@ -0,0 +1,75 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <limits> +#include <type_traits> +#include <seastar/util/concepts.hh> + +namespace seastar { + +inline +constexpr unsigned count_leading_zeros(unsigned x) { + return __builtin_clz(x); +} + +inline +constexpr unsigned count_leading_zeros(unsigned long x) { + return __builtin_clzl(x); +} + +inline +constexpr unsigned count_leading_zeros(unsigned long long x) { + return __builtin_clzll(x); +} + +inline +constexpr unsigned count_trailing_zeros(unsigned x) { + return __builtin_ctz(x); +} + +inline +constexpr unsigned count_trailing_zeros(unsigned long x) { + return __builtin_ctzl(x); +} + +inline +constexpr unsigned count_trailing_zeros(unsigned long long x) { + return __builtin_ctzll(x); +} + +template<typename T> +SEASTAR_CONCEPT( requires std::is_integral_v<T> ) +inline constexpr unsigned log2ceil(T n) { + if (n == 1) { + return 0; + } + return std::numeric_limits<T>::digits - count_leading_zeros(n - 1); +} + +template<typename T> +SEASTAR_CONCEPT( requires std::is_integral_v<T> ) +inline constexpr unsigned log2floor(T n) { + return std::numeric_limits<T>::digits - count_leading_zeros(n) - 1; +} + +} diff --git a/src/seastar/include/seastar/core/bitset-iter.hh b/src/seastar/include/seastar/core/bitset-iter.hh new file mode 100644 index 000000000..7e9093ef2 --- /dev/null +++ b/src/seastar/include/seastar/core/bitset-iter.hh @@ -0,0 +1,212 @@ +/* + * Copyright (C) 2014 Cloudius Systems, Ltd. + */ + +/* + * Imported from OSv: + * + * Copyright (C) 2014 Cloudius Systems, Ltd. + * + * This work is open source software, licensed under the terms of the + * BSD license as described in the LICENSE file in the top-level directory. + */ + +#pragma once + +#include <bitset> +#include <limits> + +namespace seastar { + +namespace bitsets { + +static constexpr int ulong_bits = std::numeric_limits<unsigned long>::digits; + +/** + * Returns the number of leading zeros in value's binary representation. + * + * If value == 0 the result is undefied. If T is signed and value is negative + * the result is undefined. + * + * The highest value that can be returned is std::numeric_limits<T>::digits - 1, + * which is returned when value == 1. + */ +template<typename T> +inline size_t count_leading_zeros(T value) noexcept; + +/** + * Returns the number of trailing zeros in value's binary representation. + * + * If value == 0 the result is undefied. If T is signed and value is negative + * the result is undefined. + * + * The highest value that can be returned is std::numeric_limits<T>::digits - 1. + */ +template<typename T> +static inline size_t count_trailing_zeros(T value) noexcept; + +template<> +inline size_t count_leading_zeros<unsigned long>(unsigned long value) noexcept +{ + return __builtin_clzl(value); +} + +template<> +inline size_t count_leading_zeros<long>(long value) noexcept +{ + return __builtin_clzl((unsigned long)value) - 1; +} + +template<> +inline size_t count_leading_zeros<unsigned long long>(unsigned long long value) noexcept +{ + return __builtin_clzll(value); +} + +template<> +inline size_t count_leading_zeros<long long>(long long value) noexcept +{ + return __builtin_clzll((unsigned long long)value) - 1; +} + +template<> +inline +size_t count_trailing_zeros<unsigned long>(unsigned long value) noexcept +{ + return __builtin_ctzl(value); +} + +template<> +inline +size_t count_trailing_zeros<long>(long value) noexcept +{ + return __builtin_ctzl((unsigned long)value); +} + +template<> +inline +size_t count_trailing_zeros<unsigned long long>(unsigned long long value) noexcept +{ + return __builtin_ctzll(value); +} + +template<> +inline +size_t count_trailing_zeros<long long>(long long value) noexcept +{ + return __builtin_ctzll((unsigned long long)value); +} + +/** + * Returns the index of the first set bit. + * Result is undefined if bitset.any() == false. + */ +template<size_t N> +static inline size_t get_first_set(const std::bitset<N>& bitset) noexcept +{ + static_assert(N <= ulong_bits, "bitset too large"); + return count_trailing_zeros(bitset.to_ulong()); +} + +/** + * Returns the index of the last set bit in the bitset. + * Result is undefined if bitset.any() == false. + */ +template<size_t N> +static inline size_t get_last_set(const std::bitset<N>& bitset) noexcept +{ + static_assert(N <= ulong_bits, "bitset too large"); + return ulong_bits - 1 - count_leading_zeros(bitset.to_ulong()); +} + +template<size_t N> +class set_iterator +{ +private: + void advance() noexcept + { + if (_bitset.none()) { + _index = -1; + } else { + auto shift = get_first_set(_bitset) + 1; + _index += shift; + _bitset >>= shift; + } + } +public: + using iterator_category = std::input_iterator_tag; + using value_type = int; + using difference_type = std::ptrdiff_t; + using pointer = int*; + using reference = int&; + + set_iterator(std::bitset<N> bitset, int offset = 0) noexcept + : _bitset(bitset) + , _index(offset - 1) + { + static_assert(N <= ulong_bits, "This implementation is inefficient for large bitsets"); + _bitset >>= offset; + advance(); + } + + set_iterator& operator++() noexcept + { + advance(); + return *this; + } + + set_iterator operator++(int) noexcept + { + auto ret = *this; + advance(); + return ret; + } + + int operator*() const noexcept + { + return _index; + } + + bool operator==(const set_iterator& other) const noexcept + { + return _index == other._index; + } + + bool operator!=(const set_iterator& other) const noexcept + { + return !(*this == other); + } +private: + std::bitset<N> _bitset; + int _index; +}; + +template<size_t N> +class set_range +{ +public: + using iterator = set_iterator<N>; + using value_type = int; + + constexpr set_range(std::bitset<N> bitset, int offset = 0) noexcept + : _bitset(bitset) + , _offset(offset) + { + } + + iterator begin() const noexcept { return iterator(_bitset, _offset); } + iterator end() const noexcept { return iterator(0); } +private: + std::bitset<N> _bitset; + int _offset; +}; + +template<size_t N> +static inline set_range<N> for_each_set(std::bitset<N> bitset, int offset = 0) noexcept +{ + return set_range<N>(bitset, offset); +} + +} + +} diff --git a/src/seastar/include/seastar/core/byteorder.hh b/src/seastar/include/seastar/core/byteorder.hh new file mode 100644 index 000000000..5cbc565d6 --- /dev/null +++ b/src/seastar/include/seastar/core/byteorder.hh @@ -0,0 +1,109 @@ +/* + * 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) 2015 Scylladb, Ltd. + */ + +#pragma once + +#include <algorithm> +#include <boost/endian/conversion.hpp> +#include <seastar/core/unaligned.hh> + +namespace seastar { + +template <typename T> +inline T cpu_to_le(T x) noexcept { + return boost::endian::native_to_little(x); +} +template <typename T> +inline T le_to_cpu(T x) noexcept { + return boost::endian::little_to_native(x); +} + +template <typename T> +inline T cpu_to_be(T x) noexcept { + return boost::endian::native_to_big(x); +} +template <typename T> +inline T be_to_cpu(T x) noexcept { + return boost::endian::big_to_native(x); +} + +template <typename T> +inline T cpu_to_le(const unaligned<T>& v) noexcept { + return cpu_to_le(T(v)); +} + +template <typename T> +inline T le_to_cpu(const unaligned<T>& v) noexcept { + return le_to_cpu(T(v)); +} + +template <typename T> +inline +T +read_le(const char* p) noexcept { + T datum; + std::copy_n(p, sizeof(T), reinterpret_cast<char*>(&datum)); + return le_to_cpu(datum); +} + +template <typename T> +inline +void +write_le(char* p, T datum) noexcept { + datum = cpu_to_le(datum); + std::copy_n(reinterpret_cast<const char*>(&datum), sizeof(T), p); +} + +template <typename T> +inline +T +read_be(const char* p) noexcept { + T datum; + std::copy_n(p, sizeof(T), reinterpret_cast<char*>(&datum)); + return be_to_cpu(datum); +} + +template <typename T> +inline +void +write_be(char* p, T datum) noexcept { + datum = cpu_to_be(datum); + std::copy_n(reinterpret_cast<const char*>(&datum), sizeof(T), p); +} + +template <typename T> +inline +T +consume_be(const char*& p) noexcept { + auto ret = read_be<T>(p); + p += sizeof(T); + return ret; +} + +template <typename T> +inline +void +produce_be(char*& p, T datum) noexcept { + write_be<T>(p, datum); + p += sizeof(T); +} + +} diff --git a/src/seastar/include/seastar/core/cacheline.hh b/src/seastar/include/seastar/core/cacheline.hh new file mode 100644 index 000000000..89bb3846c --- /dev/null +++ b/src/seastar/include/seastar/core/cacheline.hh @@ -0,0 +1,42 @@ +/* + * 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) 2017 IBM. + */ + +#pragma once + +#include <cstddef> + +namespace seastar { + +// Platform-dependent cache line size for alignment and padding purposes. +static constexpr size_t cache_line_size = +#if defined(__x86_64__) || defined(__i386__) + 64; +#elif defined(__s390x__) || defined(__zarch__) + 256; +#elif defined(__PPC64__) + 128; +#elif defined(__aarch64__) + 128; // from Linux, may vary among different microarchitetures? +#else +#error "cache_line_size not defined for this architecture" +#endif + +} diff --git a/src/seastar/include/seastar/core/checked_ptr.hh b/src/seastar/include/seastar/core/checked_ptr.hh new file mode 100644 index 000000000..a4d10d26d --- /dev/null +++ b/src/seastar/include/seastar/core/checked_ptr.hh @@ -0,0 +1,199 @@ +/* + * 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) 2017 ScyllaDB + */ + +#pragma once + +/// \file +/// \brief Contains a seastar::checked_ptr class implementation. + +#include <exception> +#include <seastar/util/concepts.hh> + +/// \namespace seastar +namespace seastar { + +/// The exception thrown by a default_null_deref_action. +class checked_ptr_is_null_exception : public std::exception {}; + +/// \brief +/// Default not engaged seastar::checked_ptr dereferencing action (functor). +/// +/// Throws a seastar::checked_ptr_is_null_exception. +/// +struct default_null_deref_action { + /// \throw seastar::checked_ptr_is_null_exception + void operator()() const { + throw checked_ptr_is_null_exception(); + } +}; + +/// \cond internal +/// \namespace seastar::internal +namespace internal { + +/// \name seastar::checked_ptr::get() helpers +/// Helper functions that simplify the seastar::checked_ptr::get() implementation. +/// @{ + +/// Invokes the get() method of a smart pointer object. +/// \param ptr A smart pointer object +/// \return A pointer to the underlying object +template <typename T> +/// cond SEASTAR_CONCEPT_DOC - nested '\ cond' doesn't seem to work (bug 736553), so working it around +SEASTAR_CONCEPT( requires requires (T ptr) { + ptr.get(); +}) +/// endcond +inline typename std::pointer_traits<std::remove_const_t<T>>::element_type* checked_ptr_do_get(T& ptr) { + return ptr.get(); +} + +/// Return a pointer itself for a naked pointer argument. +/// \param ptr A naked pointer object +/// \return An input naked pointer object +template <typename T> +inline T* checked_ptr_do_get(T* ptr) noexcept { + return ptr; +} +/// @} +} +/// \endcond + +/// \class seastar::checked_ptr +/// \brief +/// seastar::checked_ptr class is a wrapper class that may be used with any pointer type +/// (smart like std::unique_ptr or raw pointers like int*). +/// +/// The seastar::checked_ptr object will invoke the NullDerefAction functor if +/// it is dereferenced when the underlying pointer is not engaged. +/// +/// It may still be assigned, compared to other seastar::checked_ptr objects or +/// moved without limitations. +/// +/// The default NullDerefAction will throw a seastar::default_null_deref_action exception. +/// +/// \tparam NullDerefAction a functor that is invoked when a user tries to dereference a not engaged pointer. +/// +template<typename Ptr, typename NullDerefAction = default_null_deref_action> +/// \cond SEASTAR_CONCEPT_DOC +SEASTAR_CONCEPT( requires std::is_default_constructible<NullDerefAction>::value && requires (NullDerefAction action) { + NullDerefAction(); +}) +/// \endcond +class checked_ptr { +public: + /// Underlying element type + using element_type = typename std::pointer_traits<Ptr>::element_type; + + /// Type of the pointer to the underlying element + using pointer = element_type*; + +private: + Ptr _ptr = nullptr; + +private: + /// Invokes a NullDerefAction functor if the underlying pointer is not engaged. + void check() const { + if (!_ptr) { + NullDerefAction()(); + } + } + +public: + checked_ptr() noexcept(noexcept(Ptr(nullptr))) = default; + checked_ptr(std::nullptr_t) noexcept(std::is_nothrow_default_constructible<checked_ptr<Ptr, NullDerefAction>>::value) : checked_ptr() {} + checked_ptr(Ptr&& ptr) noexcept(std::is_nothrow_move_constructible<Ptr>::value) : _ptr(std::move(ptr)) {} + checked_ptr(const Ptr& p) noexcept(std::is_nothrow_copy_constructible<Ptr>::value) : _ptr(p) {} + + /// \name Checked Methods + /// These methods start with invoking a NullDerefAction functor if the underlying pointer is not engaged. + /// @{ + + /// Invokes the get() method of the underlying smart pointer or returns the pointer itself for a raw pointer (const variant). + /// \return The pointer to the underlying object + pointer get() const { + check(); + return internal::checked_ptr_do_get(_ptr); + } + + /// Gets a reference to the underlying pointer object. + /// \return The underlying pointer object + const Ptr& operator->() const { + check(); + return _ptr; + } + + /// Gets a reference to the underlying pointer object (const variant). + /// \return The underlying pointer object + Ptr& operator->() { + check(); + return _ptr; + } + + /// Gets the reference to the underlying object (const variant). + /// \return The reference to the underlying object + const element_type& operator*() const { + check(); + return *_ptr; + } + + /// Gets the reference to the underlying object. + /// \return The reference to the underlying object + element_type& operator*() { + check(); + return *_ptr; + } + /// @} + + /// \name Unchecked methods + /// These methods may be invoked when the underlying pointer is not engaged. + /// @{ + + /// Checks if the underlying pointer is engaged. + /// \return TRUE if the underlying pointer is engaged + explicit operator bool() const { return bool(_ptr); } + + bool operator==(const checked_ptr& other) const { return _ptr == other._ptr; } + bool operator!=(const checked_ptr& other) const { return _ptr != other._ptr; } + + /// Gets the hash value for the underlying pointer object. + /// \return The hash value for the underlying pointer object + size_t hash() const { + return std::hash<Ptr>()(_ptr); + } + ///@} +}; + +} + +namespace std { +/// std::hash specialization for seastar::checked_ptr class +template<typename T> +struct hash<seastar::checked_ptr<T>> { + /// Get the hash value for the given seastar::checked_ptr object. + /// The hash will calculated using the seastar::checked_ptr::hash method. + /// \param p object for hash value calculation + /// \return The hash value for the given object + size_t operator()(const seastar::checked_ptr<T>& p) const { + return p.hash(); + } +}; +} diff --git a/src/seastar/include/seastar/core/chunked_fifo.hh b/src/seastar/include/seastar/core/chunked_fifo.hh new file mode 100644 index 000000000..b6c98c441 --- /dev/null +++ b/src/seastar/include/seastar/core/chunked_fifo.hh @@ -0,0 +1,626 @@ +/* + * 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) 2016 ScyllaDB Ltd. + */ + +#pragma once + +#include <memory> +#include <algorithm> + +namespace seastar { + +// An unbounded FIFO queue of objects of type T. +// +// It provides operations to push items in one end of the queue, and pop them +// from the other end of the queue - both operations are guaranteed O(1) +// (not just amortized O(1)). The size() operation is also O(1). +// chunked_fifo also guarantees that the largest contiguous memory allocation +// it does is O(1). The total memory used is, of course, O(N). +// +// How does chunked_fifo differ from std::list<>, circular_buffer<> and +// std::deque()? +// +// std::list<> can also make all the above guarantees, but is inefficient - +// both at run speed (every operation requires an allocation), and in memory +// use. Much more efficient than std::list<> is our circular_buffer<>, which +// allocates a contiguous array to hold the items and only reallocates it, +// exponentially, when the queue grows. On one test of several different +// push/pop scenarios, circular_buffer<> was between 5 and 20 times faster +// than std::list, and also used considerably less memory. +// The problem with circular_buffer<> is that gives up on the last guarantee +// we made above: circular_buffer<> allocates all the items in one large +// contiguous allocation - that might not be possible when the memory is +// highly fragmented. +// std::deque<> aims to solve the contiguous allocation problem by allocating +// smaller chunks of the queue, and keeping a list of them in an array. This +// array is necessary to allow for O(1) random access to any element, a +// feature which we do not need; But this array is itself contiguous so +// std::deque<> attempts larger contiguous allocations the larger the queue +// gets: std::deque<>'s contiguous allocation is still O(N) and in fact +// exactly 1/64 of the size of circular_buffer<>'s contiguous allocation. +// So it's an improvement over circular_buffer<>, but not a full solution. +// +// chunked_fifo<> is such a solution: it also allocates the queue in fixed- +// size chunks (just like std::deque) but holds them in a linked list, not +// a contiguous array, so there are no large contiguous allocations. +// +// Unlike std::deque<> or circular_buffer<>, chunked_fifo only provides the +// operations needed by std::queue, i.e.,: empty(), size(), front(), back(), +// push_back() and pop_front(). For simplicity, we do *not* implement other +// possible operations, like inserting or deleting elements from the "wrong" +// side of the queue or from the middle, nor random-access to items in the +// middle of the queue. However, chunked_fifo does allow iterating over all +// of the queue's elements without popping them, a feature which std::queue +// is missing. +// +// Another feature of chunked_fifo which std::deque is missing is the ability +// to control the chunk size, as a template parameter. In std::deque the +// chunk size is undocumented and fixed - in gcc, it is always 512 bytes. +// chunked_fifo, on the other hand, makes the chunk size (in number of items +// instead of bytes) a template parameter; In situations where the queue is +// expected to become very long, using a larger chunk size might make sense +// because it will result in fewer allocations. +// +// chunked_fifo uses uninitialized storage for unoccupied elements, and thus +// uses move/copy constructors instead of move/copy assignments, which are +// less efficient. + +template <typename T, size_t items_per_chunk = 128> +class chunked_fifo { + static_assert((items_per_chunk & (items_per_chunk - 1)) == 0, + "chunked_fifo chunk size must be power of two"); + union maybe_item { + maybe_item() noexcept {} + ~maybe_item() {} + T data; + }; + struct chunk { + maybe_item items[items_per_chunk]; + struct chunk* next; + // begin and end interpreted mod items_per_chunk + unsigned begin; + unsigned end; + }; + // We pop from the chunk at _front_chunk. This chunk is then linked to + // the following chunks via the "next" link. _back_chunk points to the + // last chunk in this list, and it is where we push. + chunk* _front_chunk = nullptr; // where we pop + chunk* _back_chunk = nullptr; // where we push + // We want an O(1) size but don't want to maintain a size() counter + // because this will slow down every push and pop operation just for + // the rare size() call. Instead, we just keep a count of chunks (which + // doesn't change on every push or pop), from which we can calculate + // size() when needed, and still be O(1). + // This assumes the invariant that all middle chunks (except the front + // and back) are always full. + size_t _nchunks = 0; + // A list of freed chunks, to support reserve() and to improve + // performance of repeated push and pop, especially on an empty queue. + // It is a performance/memory tradeoff how many freed chunks to keep + // here (see save_free_chunks constant below). + chunk* _free_chunks = nullptr; + size_t _nfree_chunks = 0; +public: + using value_type = T; + using size_type = size_t; + using reference = T&; + using pointer = T*; + using const_reference = const T&; + using const_pointer = const T*; + +private: + template <typename U> + class basic_iterator { + friend class chunked_fifo; + + public: + using iterator_category = std::forward_iterator_tag; + using difference_type = std::ptrdiff_t; + using value_type = U; + using pointer = U*; + using reference = U&; + + protected: + chunk* _chunk = nullptr; + size_t _item_index = 0; + + protected: + inline explicit basic_iterator(chunk* c) noexcept; + inline basic_iterator(chunk* c, size_t item_index) noexcept; + + public: + inline bool operator==(const basic_iterator& o) const noexcept; + inline bool operator!=(const basic_iterator& o) const noexcept; + inline pointer operator->() const noexcept; + inline reference operator*() const noexcept; + inline basic_iterator operator++(int) noexcept; + basic_iterator& operator++() noexcept; + }; + +public: + class iterator : public basic_iterator<T> { + using basic_iterator<T>::basic_iterator; + public: + iterator() noexcept = default; + }; + class const_iterator : public basic_iterator<const T> { + using basic_iterator<const T>::basic_iterator; + public: + const_iterator() noexcept = default; + inline const_iterator(iterator o) noexcept; + }; + +public: + chunked_fifo() noexcept = default; + chunked_fifo(chunked_fifo&& x) noexcept; + chunked_fifo(const chunked_fifo& X) = delete; + ~chunked_fifo(); + chunked_fifo& operator=(const chunked_fifo&) = delete; + chunked_fifo& operator=(chunked_fifo&&) noexcept; + inline void push_back(const T& data); + inline void push_back(T&& data); + T& back() noexcept; + const T& back() const noexcept; + template <typename... A> + inline void emplace_back(A&&... args); + inline T& front() const noexcept; + inline void pop_front() noexcept; + inline bool empty() const noexcept; + inline size_t size() const noexcept; + void clear() noexcept; + // reserve(n) ensures that at least (n - size()) further push() calls can + // be served without needing new memory allocation. + // Calling pop()s between these push()es is also allowed and does not + // alter this guarantee. + // Note that reserve() does not reduce the amount of memory already + // reserved - use shrink_to_fit() for that. + void reserve(size_t n); + // shrink_to_fit() frees memory held, but unused, by the queue. Such + // unused memory might exist after pops, or because of reserve(). + void shrink_to_fit() noexcept; + inline iterator begin() noexcept; + inline iterator end() noexcept; + inline const_iterator begin() const noexcept; + inline const_iterator end() const noexcept; + inline const_iterator cbegin() const noexcept; + inline const_iterator cend() const noexcept; +private: + void back_chunk_new(); + void front_chunk_delete() noexcept; + inline void ensure_room_back(); + void undo_room_back() noexcept; + static inline size_t mask(size_t idx) noexcept; + +}; + +template <typename T, size_t items_per_chunk> +template <typename U> +inline +chunked_fifo<T, items_per_chunk>::basic_iterator<U>::basic_iterator(chunk* c) noexcept : _chunk(c), _item_index(_chunk ? _chunk->begin : 0) { +} + +template <typename T, size_t items_per_chunk> +template <typename U> +inline +chunked_fifo<T, items_per_chunk>::basic_iterator<U>::basic_iterator(chunk* c, size_t item_index) noexcept : _chunk(c), _item_index(item_index) { +} + +template <typename T, size_t items_per_chunk> +template <typename U> +inline bool +chunked_fifo<T, items_per_chunk>::basic_iterator<U>::operator==(const basic_iterator& o) const noexcept { + return _chunk == o._chunk && _item_index == o._item_index; +} + +template <typename T, size_t items_per_chunk> +template <typename U> +inline bool +chunked_fifo<T, items_per_chunk>::basic_iterator<U>::operator!=(const basic_iterator& o) const noexcept { + return !(*this == o); +} + +template <typename T, size_t items_per_chunk> +template <typename U> +inline typename chunked_fifo<T, items_per_chunk>::template basic_iterator<U>::pointer +chunked_fifo<T, items_per_chunk>::basic_iterator<U>::operator->() const noexcept { + return &_chunk->items[chunked_fifo::mask(_item_index)].data; +} + +template <typename T, size_t items_per_chunk> +template <typename U> +inline typename chunked_fifo<T, items_per_chunk>::template basic_iterator<U>::reference +chunked_fifo<T, items_per_chunk>::basic_iterator<U>::operator*() const noexcept { + return _chunk->items[chunked_fifo::mask(_item_index)].data; +} + +template <typename T, size_t items_per_chunk> +template <typename U> +inline typename chunked_fifo<T, items_per_chunk>::template basic_iterator<U> +chunked_fifo<T, items_per_chunk>::basic_iterator<U>::operator++(int) noexcept { + auto it = *this; + ++(*this); + return it; +} + +template <typename T, size_t items_per_chunk> +template <typename U> +typename chunked_fifo<T, items_per_chunk>::template basic_iterator<U>& +chunked_fifo<T, items_per_chunk>::basic_iterator<U>::operator++() noexcept { + ++_item_index; + if (_item_index == _chunk->end) { + _chunk = _chunk->next; + _item_index = _chunk ? _chunk->begin : 0; + } + return *this; +} + +template <typename T, size_t items_per_chunk> +inline +chunked_fifo<T, items_per_chunk>::const_iterator::const_iterator(chunked_fifo<T, items_per_chunk>::iterator o) noexcept + : basic_iterator<const T>(o._chunk, o._item_index) { +} + +template <typename T, size_t items_per_chunk> +inline +chunked_fifo<T, items_per_chunk>::chunked_fifo(chunked_fifo&& x) noexcept + : _front_chunk(x._front_chunk) + , _back_chunk(x._back_chunk) + , _nchunks(x._nchunks) + , _free_chunks(x._free_chunks) + , _nfree_chunks(x._nfree_chunks) { + x._front_chunk = nullptr; + x._back_chunk = nullptr; + x._nchunks = 0; + x._free_chunks = nullptr; + x._nfree_chunks = 0; +} + +template <typename T, size_t items_per_chunk> +inline +chunked_fifo<T, items_per_chunk>& +chunked_fifo<T, items_per_chunk>::operator=(chunked_fifo&& x) noexcept { + if (&x != this) { + this->~chunked_fifo(); + new (this) chunked_fifo(std::move(x)); + } + return *this; +} + +template <typename T, size_t items_per_chunk> +inline size_t +chunked_fifo<T, items_per_chunk>::mask(size_t idx) noexcept { + return idx & (items_per_chunk - 1); +} + +template <typename T, size_t items_per_chunk> +inline bool +chunked_fifo<T, items_per_chunk>::empty() const noexcept { + return _front_chunk == nullptr; +} + +template <typename T, size_t items_per_chunk> +inline size_t +chunked_fifo<T, items_per_chunk>::size() const noexcept{ + if (_front_chunk == nullptr) { + return 0; + } else if (_back_chunk == _front_chunk) { + // Single chunk. + return _front_chunk->end - _front_chunk->begin; + } else { + return _front_chunk->end - _front_chunk->begin + +_back_chunk->end - _back_chunk->begin + + (_nchunks - 2) * items_per_chunk; + } +} + +template <typename T, size_t items_per_chunk> +void chunked_fifo<T, items_per_chunk>::clear() noexcept { +#if 1 + while (!empty()) { + pop_front(); + } +#else + // This is specialized code to free the contents of all the chunks and the + // chunks themselves. but since destroying a very full queue is not an + // important use case to optimize, the simple loop above is preferable. + if (!_front_chunk) { + // Empty, nothing to do + return; + } + // Delete front chunk (partially filled) + for (auto i = _front_chunk->begin; i != _front_chunk->end; ++i) { + _front_chunk->items[mask(i)].data.~T(); + } + chunk *p = _front_chunk->next; + delete _front_chunk; + // Delete all the middle chunks (all completely filled) + if (p) { + while (p != _back_chunk) { + // These are full chunks + chunk *nextp = p->next; + for (auto i = 0; i != items_per_chunk; ++i) { + // Note we delete out of order (we don't start with p->begin). + // That should be fine.. + p->items[i].data.~T(); + } + delete p; + p = nextp; + } + // Finally delete back chunk (partially filled) + for (auto i = _back_chunk->begin; i != _back_chunk->end; ++i) { + _back_chunk->items[mask(i)].data.~T(); + } + delete _back_chunk; + } + _front_chunk = nullptr; + _back_chunk = nullptr; + _nchunks = 0; +#endif +} + +template <typename T, size_t items_per_chunk> void +chunked_fifo<T, items_per_chunk>::shrink_to_fit() noexcept { + while (_free_chunks) { + auto next = _free_chunks->next; + delete _free_chunks; + _free_chunks = next; + } + _nfree_chunks = 0; +} + +template <typename T, size_t items_per_chunk> +chunked_fifo<T, items_per_chunk>::~chunked_fifo() { + clear(); + shrink_to_fit(); +} + +template <typename T, size_t items_per_chunk> +void +chunked_fifo<T, items_per_chunk>::back_chunk_new() { + chunk *old = _back_chunk; + if (_free_chunks) { + _back_chunk = _free_chunks; + _free_chunks = _free_chunks->next; + --_nfree_chunks; + } else { + _back_chunk = new chunk; + } + _back_chunk->next = nullptr; + _back_chunk->begin = 0; + _back_chunk->end = 0; + if (old) { + old->next = _back_chunk; + } + if (_front_chunk == nullptr) { + _front_chunk = _back_chunk; + } + _nchunks++; +} + + +template <typename T, size_t items_per_chunk> +inline void +chunked_fifo<T, items_per_chunk>::ensure_room_back() { + // If we don't have a back chunk or it's full, we need to create a new one + if (_back_chunk == nullptr || + (_back_chunk->end - _back_chunk->begin) == items_per_chunk) { + back_chunk_new(); + } +} + +template <typename T, size_t items_per_chunk> +void +chunked_fifo<T, items_per_chunk>::undo_room_back() noexcept { + // If we failed creating a new item after ensure_room_back() created a + // new empty chunk, we must remove it, or empty() will be incorrect + // (either immediately, if the fifo was empty, or when all the items are + // popped, if it already had items). + if (_back_chunk->begin == _back_chunk->end) { + delete _back_chunk; + --_nchunks; + if (_nchunks == 0) { + _back_chunk = nullptr; + _front_chunk = nullptr; + } else { + // Because we don't usually pop from the back, we don't have a "prev" + // pointer so we need to find the previous chunk the hard and slow + // way. B + chunk *old = _back_chunk; + _back_chunk = _front_chunk; + while (_back_chunk->next != old) { + _back_chunk = _back_chunk->next; + } + _back_chunk->next = nullptr; + } + } + +} + +template <typename T, size_t items_per_chunk> +template <typename... Args> +inline void +chunked_fifo<T, items_per_chunk>::emplace_back(Args&&... args) { + ensure_room_back(); + auto p = &_back_chunk->items[mask(_back_chunk->end)].data; + try { + new(p) T(std::forward<Args>(args)...); + } catch(...) { + undo_room_back(); + throw; + } + ++_back_chunk->end; +} + +template <typename T, size_t items_per_chunk> +inline void +chunked_fifo<T, items_per_chunk>::push_back(const T& data) { + ensure_room_back(); + auto p = &_back_chunk->items[mask(_back_chunk->end)].data; + try { + new(p) T(data); + } catch(...) { + undo_room_back(); + throw; + } + ++_back_chunk->end; +} + +template <typename T, size_t items_per_chunk> +inline void +chunked_fifo<T, items_per_chunk>::push_back(T&& data) { + ensure_room_back(); + auto p = &_back_chunk->items[mask(_back_chunk->end)].data; + try { + new(p) T(std::move(data)); + } catch(...) { + undo_room_back(); + throw; + } + ++_back_chunk->end; +} + +template <typename T, size_t items_per_chunk> +inline +T& +chunked_fifo<T, items_per_chunk>::back() noexcept { + return _back_chunk->items[mask(_back_chunk->end - 1)].data; +} + +template <typename T, size_t items_per_chunk> +inline +const T& +chunked_fifo<T, items_per_chunk>::back() const noexcept { + return _back_chunk->items[mask(_back_chunk->end - 1)].data; +} + +template <typename T, size_t items_per_chunk> +inline T& +chunked_fifo<T, items_per_chunk>::front() const noexcept { + return _front_chunk->items[mask(_front_chunk->begin)].data; +} + +template <typename T, size_t items_per_chunk> +inline void +chunked_fifo<T, items_per_chunk>::front_chunk_delete() noexcept { + chunk *next = _front_chunk->next; + // Certain use cases may need to repeatedly allocate and free a chunk - + // an obvious example is an empty queue to which we push, and then pop, + // repeatedly. Another example is pushing and popping to a non-empty queue + // we push and pop at different chunks so we need to free and allocate a + // chunk every items_per_chunk operations. + // The solution is to keep a list of freed chunks instead of freeing them + // immediately. There is a performance/memory tradeoff of how many freed + // chunks to save: If we save them all, the queue can never shrink from + // its maximum memory use (this is how circular_buffer behaves). + // The ad-hoc choice made here is to limit the number of saved chunks to 1, + // but this could easily be made a configuration option. + static constexpr int save_free_chunks = 1; + if (_nfree_chunks < save_free_chunks) { + _front_chunk->next = _free_chunks; + _free_chunks = _front_chunk; + ++_nfree_chunks; + } else { + delete _front_chunk; + } + // If we only had one chunk, _back_chunk is gone too. + if (_back_chunk == _front_chunk) { + _back_chunk = nullptr; + } + _front_chunk = next; + --_nchunks; +} + +template <typename T, size_t items_per_chunk> +inline void +chunked_fifo<T, items_per_chunk>::pop_front() noexcept { + front().~T(); + // If the front chunk has become empty, we need to free remove it and use + // the next one. + if (++_front_chunk->begin == _front_chunk->end) { + front_chunk_delete(); + } +} + +template <typename T, size_t items_per_chunk> +void chunked_fifo<T, items_per_chunk>::reserve(size_t n) { + // reserve() guarantees that (n - size()) additional push()es will + // succeed without reallocation: + if (n <= size()) { + return; + } + size_t need = n - size(); + // If we already have a back chunk, it might have room for some pushes + // before filling up, so decrease "need": + if (_back_chunk) { + size_t back_chunk_n = items_per_chunk - (_back_chunk->end - _back_chunk->begin); + need -= std::min(back_chunk_n, need); + } + size_t needed_chunks = (need + items_per_chunk - 1) / items_per_chunk; + // If we already have some freed chunks saved, we need to allocate fewer + // additional chunks, or none at all + if (needed_chunks <= _nfree_chunks) { + return; + } + needed_chunks -= _nfree_chunks; + while (needed_chunks--) { + chunk *c = new chunk; + c->next = _free_chunks; + _free_chunks = c; + ++_nfree_chunks; + } +} + +template <typename T, size_t items_per_chunk> +inline typename chunked_fifo<T, items_per_chunk>::iterator +chunked_fifo<T, items_per_chunk>::begin() noexcept { + return iterator(_front_chunk); +} + +template <typename T, size_t items_per_chunk> +inline typename chunked_fifo<T, items_per_chunk>::iterator +chunked_fifo<T, items_per_chunk>::end() noexcept { + return iterator(nullptr); +} + +template <typename T, size_t items_per_chunk> +inline typename chunked_fifo<T, items_per_chunk>::const_iterator +chunked_fifo<T, items_per_chunk>::begin() const noexcept { + return const_iterator(_front_chunk); +} + +template <typename T, size_t items_per_chunk> +inline typename chunked_fifo<T, items_per_chunk>::const_iterator +chunked_fifo<T, items_per_chunk>::end() const noexcept { + return const_iterator(nullptr); +} + +template <typename T, size_t items_per_chunk> +inline typename chunked_fifo<T, items_per_chunk>::const_iterator +chunked_fifo<T, items_per_chunk>::cbegin() const noexcept { + return const_iterator(_front_chunk); +} + +template <typename T, size_t items_per_chunk> +inline typename chunked_fifo<T, items_per_chunk>::const_iterator +chunked_fifo<T, items_per_chunk>::cend() const noexcept { + return const_iterator(nullptr); +} + +} diff --git a/src/seastar/include/seastar/core/circular_buffer.hh b/src/seastar/include/seastar/core/circular_buffer.hh new file mode 100644 index 000000000..646c71e63 --- /dev/null +++ b/src/seastar/include/seastar/core/circular_buffer.hh @@ -0,0 +1,512 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/core/transfer.hh> +#include <seastar/core/bitops.hh> +#include <seastar/util/concepts.hh> +#include <memory> +#include <algorithm> + +namespace seastar { + +/// A growable double-ended queue container that can be efficiently +/// extended (and shrunk) from both ends. Implementation is a single +/// storage vector. +/// +/// Similar to libstdc++'s std::deque, except that it uses a single +/// level store, and so is more efficient for simple stored items. +/// Similar to boost::circular_buffer_space_optimized, except it uses +/// uninitialized storage for unoccupied elements (and thus move/copy +/// constructors instead of move/copy assignments, which are less +/// efficient). +/// +/// The storage of the circular_buffer is expanded automatically in +/// exponential increments. +/// When adding new elements: +/// * if size + 1 > capacity: all iterators and references are +/// invalidated, +/// * otherwise only the begin() or end() iterator is invalidated: +/// * push_front() and emplace_front() will invalidate begin() and +/// * push_back() and emplace_back() will invalidate end(). +/// +/// Removing elements never invalidates any references and only +/// invalidates begin() or end() iterators: +/// * pop_front() will invalidate begin() and +/// * pop_back() will invalidate end(). +/// +/// reserve() may also invalidate all iterators and references. +template <typename T, typename Alloc = std::allocator<T>> +class circular_buffer { + struct impl : Alloc { + T* storage = nullptr; + // begin, end interpreted (mod capacity) + size_t begin = 0; + size_t end = 0; + size_t capacity = 0; + + impl(Alloc a) noexcept : Alloc(std::move(a)) { } + void reset() { + storage = {}; + begin = 0; + end = 0; + capacity = 0; + } + }; + static_assert(!std::is_default_constructible_v<Alloc> + || std::is_nothrow_default_constructible_v<Alloc>); + static_assert(std::is_nothrow_move_constructible_v<Alloc>); + impl _impl; +public: + using value_type = T; + using size_type = size_t; + using reference = T&; + using pointer = T*; + using const_reference = const T&; + using const_pointer = const T*; +public: + circular_buffer() noexcept SEASTAR_CONCEPT(requires std::default_initializable<Alloc>) : circular_buffer(Alloc()) {} + circular_buffer(Alloc alloc) noexcept; + circular_buffer(circular_buffer&& X) noexcept; + circular_buffer(const circular_buffer& X) = delete; + ~circular_buffer(); + circular_buffer& operator=(const circular_buffer&) = delete; + circular_buffer& operator=(circular_buffer&& b) noexcept; + void push_front(const T& data); + void push_front(T&& data); + template <typename... A> + void emplace_front(A&&... args); + void push_back(const T& data); + void push_back(T&& data); + template <typename... A> + void emplace_back(A&&... args); + T& front() noexcept; + const T& front() const noexcept; + T& back() noexcept; + const T& back() const noexcept; + void pop_front() noexcept; + void pop_back() noexcept; + bool empty() const noexcept; + size_t size() const noexcept; + size_t capacity() const noexcept; + void reserve(size_t); + void clear() noexcept; + T& operator[](size_t idx) noexcept; + const T& operator[](size_t idx) const noexcept; + template <typename Func> + void for_each(Func func); + // access an element, may return wrong or destroyed element + // only useful if you do not rely on data accuracy (e.g. prefetch) + T& access_element_unsafe(size_t idx) noexcept; +private: + void expand(); + void expand(size_t); + void maybe_expand(size_t nr = 1); + size_t mask(size_t idx) const; + + template<typename CB, typename ValueType> + struct cbiterator { + using iterator_category = std::random_access_iterator_tag; + using value_type = ValueType; + using difference_type = std::ptrdiff_t; + using pointer = ValueType*; + using reference = ValueType&; + + ValueType& operator*() const noexcept { return cb->_impl.storage[cb->mask(idx)]; } + ValueType* operator->() const noexcept { return &cb->_impl.storage[cb->mask(idx)]; } + // prefix + cbiterator<CB, ValueType>& operator++() noexcept { + idx++; + return *this; + } + // postfix + cbiterator<CB, ValueType> operator++(int) noexcept { + auto v = *this; + idx++; + return v; + } + // prefix + cbiterator<CB, ValueType>& operator--() noexcept { + idx--; + return *this; + } + // postfix + cbiterator<CB, ValueType> operator--(int) noexcept { + auto v = *this; + idx--; + return v; + } + cbiterator<CB, ValueType> operator+(difference_type n) const noexcept { + return cbiterator<CB, ValueType>(cb, idx + n); + } + cbiterator<CB, ValueType> operator-(difference_type n) const noexcept { + return cbiterator<CB, ValueType>(cb, idx - n); + } + cbiterator<CB, ValueType>& operator+=(difference_type n) noexcept { + idx += n; + return *this; + } + cbiterator<CB, ValueType>& operator-=(difference_type n) noexcept { + idx -= n; + return *this; + } + bool operator==(const cbiterator<CB, ValueType>& rhs) const noexcept { + return idx == rhs.idx; + } + bool operator!=(const cbiterator<CB, ValueType>& rhs) const noexcept { + return idx != rhs.idx; + } + bool operator<(const cbiterator<CB, ValueType>& rhs) const noexcept { + return *this - rhs < 0; + } + bool operator>(const cbiterator<CB, ValueType>& rhs) const noexcept { + return *this - rhs > 0; + } + bool operator>=(const cbiterator<CB, ValueType>& rhs) const noexcept { + return *this - rhs >= 0; + } + bool operator<=(const cbiterator<CB, ValueType>& rhs) const noexcept { + return *this - rhs <= 0; + } + difference_type operator-(const cbiterator<CB, ValueType>& rhs) const noexcept { + return idx - rhs.idx; + } + private: + CB* cb; + size_t idx; + cbiterator(CB* b, size_t i) noexcept : cb(b), idx(i) {} + friend class circular_buffer; + }; + friend class iterator; + +public: + typedef cbiterator<circular_buffer, T> iterator; + typedef cbiterator<const circular_buffer, const T> const_iterator; + + iterator begin() noexcept { + return iterator(this, _impl.begin); + } + const_iterator begin() const noexcept { + return const_iterator(this, _impl.begin); + } + iterator end() noexcept { + return iterator(this, _impl.end); + } + const_iterator end() const noexcept { + return const_iterator(this, _impl.end); + } + const_iterator cbegin() const noexcept { + return const_iterator(this, _impl.begin); + } + const_iterator cend() const noexcept { + return const_iterator(this, _impl.end); + } + iterator erase(iterator first, iterator last) noexcept; +}; + +template <typename T, typename Alloc> +inline +size_t +circular_buffer<T, Alloc>::mask(size_t idx) const { + return idx & (_impl.capacity - 1); +} + +template <typename T, typename Alloc> +inline +bool +circular_buffer<T, Alloc>::empty() const noexcept { + return _impl.begin == _impl.end; +} + +template <typename T, typename Alloc> +inline +size_t +circular_buffer<T, Alloc>::size() const noexcept { + return _impl.end - _impl.begin; +} + +template <typename T, typename Alloc> +inline +size_t +circular_buffer<T, Alloc>::capacity() const noexcept { + return _impl.capacity; +} + +template <typename T, typename Alloc> +inline +void +circular_buffer<T, Alloc>::reserve(size_t size) { + if (capacity() < size) { + // Make sure that the new capacity is a power of two. + expand(size_t(1) << log2ceil(size)); + } +} + +template <typename T, typename Alloc> +inline +void +circular_buffer<T, Alloc>::clear() noexcept { + erase(begin(), end()); +} + +template <typename T, typename Alloc> +inline +circular_buffer<T, Alloc>::circular_buffer(Alloc alloc) noexcept + : _impl(std::move(alloc)) { +} + +template <typename T, typename Alloc> +inline +circular_buffer<T, Alloc>::circular_buffer(circular_buffer&& x) noexcept + : _impl(std::move(x._impl)) { + x._impl.reset(); +} + +template <typename T, typename Alloc> +inline +circular_buffer<T, Alloc>& circular_buffer<T, Alloc>::operator=(circular_buffer&& x) noexcept { + if (this != &x) { + this->~circular_buffer(); + new (this) circular_buffer(std::move(x)); + } + return *this; +} + +template <typename T, typename Alloc> +template <typename Func> +inline +void +circular_buffer<T, Alloc>::for_each(Func func) { + auto s = _impl.storage; + auto m = _impl.capacity - 1; + for (auto i = _impl.begin; i != _impl.end; ++i) { + func(s[i & m]); + } +} + +template <typename T, typename Alloc> +inline +circular_buffer<T, Alloc>::~circular_buffer() { + for_each([this] (T& obj) { + std::allocator_traits<Alloc>::destroy(_impl, &obj); + }); + _impl.deallocate(_impl.storage, _impl.capacity); +} + +template <typename T, typename Alloc> +void +circular_buffer<T, Alloc>::expand() { + expand(std::max<size_t>(_impl.capacity * 2, 1)); +} + +template <typename T, typename Alloc> +void +circular_buffer<T, Alloc>::expand(size_t new_cap) { + auto new_storage = _impl.allocate(new_cap); + auto p = new_storage; + try { + for_each([this, &p] (T& obj) { + transfer_pass1(_impl, &obj, p); + p++; + }); + } catch (...) { + while (p != new_storage) { + std::allocator_traits<Alloc>::destroy(_impl, --p); + } + _impl.deallocate(new_storage, new_cap); + throw; + } + p = new_storage; + for_each([this, &p] (T& obj) { + transfer_pass2(_impl, &obj, p++); + }); + std::swap(_impl.storage, new_storage); + std::swap(_impl.capacity, new_cap); + _impl.begin = 0; + _impl.end = p - _impl.storage; + _impl.deallocate(new_storage, new_cap); +} + +template <typename T, typename Alloc> +inline +void +circular_buffer<T, Alloc>::maybe_expand(size_t nr) { + if (_impl.end - _impl.begin + nr > _impl.capacity) { + expand(); + } +} + +template <typename T, typename Alloc> +inline +void +circular_buffer<T, Alloc>::push_front(const T& data) { + maybe_expand(); + auto p = &_impl.storage[mask(_impl.begin - 1)]; + std::allocator_traits<Alloc>::construct(_impl, p, data); + --_impl.begin; +} + +template <typename T, typename Alloc> +inline +void +circular_buffer<T, Alloc>::push_front(T&& data) { + maybe_expand(); + auto p = &_impl.storage[mask(_impl.begin - 1)]; + std::allocator_traits<Alloc>::construct(_impl, p, std::move(data)); + --_impl.begin; +} + +template <typename T, typename Alloc> +template <typename... Args> +inline +void +circular_buffer<T, Alloc>::emplace_front(Args&&... args) { + maybe_expand(); + auto p = &_impl.storage[mask(_impl.begin - 1)]; + std::allocator_traits<Alloc>::construct(_impl, p, std::forward<Args>(args)...); + --_impl.begin; +} + +template <typename T, typename Alloc> +inline +void +circular_buffer<T, Alloc>::push_back(const T& data) { + maybe_expand(); + auto p = &_impl.storage[mask(_impl.end)]; + std::allocator_traits<Alloc>::construct(_impl, p, data); + ++_impl.end; +} + +template <typename T, typename Alloc> +inline +void +circular_buffer<T, Alloc>::push_back(T&& data) { + maybe_expand(); + auto p = &_impl.storage[mask(_impl.end)]; + std::allocator_traits<Alloc>::construct(_impl, p, std::move(data)); + ++_impl.end; +} + +template <typename T, typename Alloc> +template <typename... Args> +inline +void +circular_buffer<T, Alloc>::emplace_back(Args&&... args) { + maybe_expand(); + auto p = &_impl.storage[mask(_impl.end)]; + std::allocator_traits<Alloc>::construct(_impl, p, std::forward<Args>(args)...); + ++_impl.end; +} + +template <typename T, typename Alloc> +inline +T& +circular_buffer<T, Alloc>::front() noexcept { + return _impl.storage[mask(_impl.begin)]; +} + +template <typename T, typename Alloc> +inline +const T& +circular_buffer<T, Alloc>::front() const noexcept { + return _impl.storage[mask(_impl.begin)]; +} + +template <typename T, typename Alloc> +inline +T& +circular_buffer<T, Alloc>::back() noexcept { + return _impl.storage[mask(_impl.end - 1)]; +} + +template <typename T, typename Alloc> +inline +const T& +circular_buffer<T, Alloc>::back() const noexcept { + return _impl.storage[mask(_impl.end - 1)]; +} + +template <typename T, typename Alloc> +inline +void +circular_buffer<T, Alloc>::pop_front() noexcept { + std::allocator_traits<Alloc>::destroy(_impl, &front()); + ++_impl.begin; +} + +template <typename T, typename Alloc> +inline +void +circular_buffer<T, Alloc>::pop_back() noexcept { + std::allocator_traits<Alloc>::destroy(_impl, &back()); + --_impl.end; +} + +template <typename T, typename Alloc> +inline +T& +circular_buffer<T, Alloc>::operator[](size_t idx) noexcept { + return _impl.storage[mask(_impl.begin + idx)]; +} + +template <typename T, typename Alloc> +inline +const T& +circular_buffer<T, Alloc>::operator[](size_t idx) const noexcept { + return _impl.storage[mask(_impl.begin + idx)]; +} + +template <typename T, typename Alloc> +inline +T& +circular_buffer<T, Alloc>::access_element_unsafe(size_t idx) noexcept { + return _impl.storage[mask(_impl.begin + idx)]; +} + +template <typename T, typename Alloc> +inline +typename circular_buffer<T, Alloc>::iterator +circular_buffer<T, Alloc>::erase(iterator first, iterator last) noexcept { + static_assert(std::is_nothrow_move_assignable<T>::value, "erase() assumes move assignment does not throw"); + if (first == last) { + return last; + } + // Move to the left or right depending on which would result in least amount of moves. + // This also guarantees that iterators will be stable when removing from either front or back. + if (std::distance(begin(), first) < std::distance(last, end())) { + auto new_start = std::move_backward(begin(), first, last); + for (auto i = begin(); i < new_start; ++i) { + std::allocator_traits<Alloc>::destroy(_impl, &*i); + } + _impl.begin = new_start.idx; + return last; + } else { + auto new_end = std::move(last, end(), first); + for (auto i = new_end, e = end(); i < e; ++i) { + std::allocator_traits<Alloc>::destroy(_impl, &*i); + } + _impl.end = new_end.idx; + return first; + } +} + +} diff --git a/src/seastar/include/seastar/core/circular_buffer_fixed_capacity.hh b/src/seastar/include/seastar/core/circular_buffer_fixed_capacity.hh new file mode 100644 index 000000000..7e7c093e2 --- /dev/null +++ b/src/seastar/include/seastar/core/circular_buffer_fixed_capacity.hh @@ -0,0 +1,378 @@ +/* + * 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) 2017 ScyllaDB + */ + +#pragma once + +// A fixed capacity double-ended queue container that can be efficiently +// extended (and shrunk) from both ends. Implementation is a single +// storage vector. +// +// Similar to libstdc++'s std::deque, except that it uses a single level +// store, and so is more efficient for simple stored items. + +#include <type_traits> +#include <cstddef> +#include <iterator> +#include <utility> + + +/// \file + +namespace seastar { + +/// A fixed-capacity container (like boost::static_vector) that can insert +/// and remove at both ends (like std::deque). Does not allocate. +/// +/// Does not perform overflow checking when size exceeds capacity. +/// +/// \tparam T type of objects stored in the container; must be noexcept move enabled +/// \tparam Capacity maximum number of objects that can be stored in the container; must be a power of 2 +template <typename T, size_t Capacity> +class circular_buffer_fixed_capacity { + size_t _begin = 0; + size_t _end = 0; + union maybe_storage { + T data; + maybe_storage() noexcept {} + ~maybe_storage() {} + }; + maybe_storage _storage[Capacity]; +private: + static size_t mask(size_t idx) { return idx % Capacity; } + T* obj(size_t idx) { return &_storage[mask(idx)].data; } + const T* obj(size_t idx) const { return &_storage[mask(idx)].data; } +public: + static_assert((Capacity & (Capacity - 1)) == 0, "capacity must be a power of two"); + static_assert(std::is_nothrow_move_constructible<T>::value && std::is_nothrow_move_assignable<T>::value, + "circular_buffer_fixed_capacity only supports nothrow-move value types"); + using value_type = T; + using size_type = size_t; + using reference = T&; + using pointer = T*; + using const_reference = const T&; + using const_pointer = const T*; + using difference_type = ssize_t; +public: + template <typename ValueType> + class cbiterator { + using holder = std::conditional_t<std::is_const<ValueType>::value, const maybe_storage, maybe_storage>; + holder* _start; + size_t _idx; + private: + cbiterator(holder* start, size_t idx) noexcept : _start(start), _idx(idx) {} + public: + using iterator_category = std::random_access_iterator_tag; + using value_type = ValueType; + using difference_type = ssize_t; + using pointer = ValueType*; + using reference = ValueType&; + public: + cbiterator(); + ValueType& operator*() const { return _start[mask(_idx)].data; } + ValueType* operator->() const { return &operator*(); } + // prefix + cbiterator& operator++() { + ++_idx; + return *this; + } + // postfix + cbiterator operator++(int) { + auto v = *this; + ++_idx; + return v; + } + // prefix + cbiterator& operator--() { + --_idx; + return *this; + } + // postfix + cbiterator operator--(int) { + auto v = *this; + --_idx; + return v; + } + cbiterator operator+(difference_type n) const { + return cbiterator{_start, _idx + n}; + } + friend cbiterator operator+(difference_type n, cbiterator i) { + return i + n; + } + cbiterator operator-(difference_type n) const { + return cbiterator{_start, _idx - n}; + } + cbiterator& operator+=(difference_type n) { + _idx += n; + return *this; + } + cbiterator& operator-=(difference_type n) { + _idx -= n; + return *this; + } + bool operator==(const cbiterator& rhs) const { + return _idx == rhs._idx; + } + bool operator!=(const cbiterator& rhs) const { + return _idx != rhs._idx; + } + bool operator<(const cbiterator& rhs) const { + return ssize_t(_idx - rhs._idx) < 0; + } + bool operator>(const cbiterator& rhs) const { + return ssize_t(_idx - rhs._idx) > 0; + } + bool operator<=(const cbiterator& rhs) const { + return ssize_t(_idx - rhs._idx) <= 0; + } + bool operator>=(const cbiterator& rhs) const { + return ssize_t(_idx - rhs._idx) >= 0; + } + difference_type operator-(const cbiterator& rhs) const { + return _idx - rhs._idx; + } + friend class circular_buffer_fixed_capacity; + }; +public: + using iterator = cbiterator<T>; + using const_iterator = cbiterator<const T>; +public: + circular_buffer_fixed_capacity() = default; + circular_buffer_fixed_capacity(circular_buffer_fixed_capacity&& x) noexcept; + ~circular_buffer_fixed_capacity(); + circular_buffer_fixed_capacity& operator=(circular_buffer_fixed_capacity&& x) noexcept; + void push_front(const T& data); + void push_front(T&& data); + template <typename... A> + T& emplace_front(A&&... args); + void push_back(const T& data); + void push_back(T&& data); + template <typename... A> + T& emplace_back(A&&... args); + T& front(); + T& back(); + void pop_front(); + void pop_back(); + bool empty() const; + size_t size() const; + size_t capacity() const; + T& operator[](size_t idx); + void clear(); + iterator begin() { + return iterator(_storage, _begin); + } + const_iterator begin() const { + return const_iterator(_storage, _begin); + } + iterator end() { + return iterator(_storage, _end); + } + const_iterator end() const { + return const_iterator(_storage, _end); + } + const_iterator cbegin() const { + return const_iterator(_storage, _begin); + } + const_iterator cend() const { + return const_iterator(_storage, _end); + } + iterator erase(iterator first, iterator last); +}; + +template <typename T, size_t Capacity> +inline +bool +circular_buffer_fixed_capacity<T, Capacity>::empty() const { + return _begin == _end; +} + +template <typename T, size_t Capacity> +inline +size_t +circular_buffer_fixed_capacity<T, Capacity>::size() const { + return _end - _begin; +} + +template <typename T, size_t Capacity> +inline +size_t +circular_buffer_fixed_capacity<T, Capacity>::capacity() const { + return Capacity; +} + +template <typename T, size_t Capacity> +inline +circular_buffer_fixed_capacity<T, Capacity>::circular_buffer_fixed_capacity(circular_buffer_fixed_capacity&& x) noexcept + : _begin(x._begin), _end(x._end) { + // This is std::uninitialized_move, but that is c++17 only + auto dest = begin(); + for (auto& obj : x) { + new (&*dest++) T(std::move(obj)); + } +} + +template <typename T, size_t Capacity> +inline +circular_buffer_fixed_capacity<T, Capacity>& +circular_buffer_fixed_capacity<T, Capacity>::operator=(circular_buffer_fixed_capacity&& x) noexcept { + if (this != &x) { + this->~circular_buffer_fixed_capacity(); + new (this) circular_buffer_fixed_capacity(std::move(x)); + } + return *this; +} + +template <typename T, size_t Capacity> +inline +circular_buffer_fixed_capacity<T, Capacity>::~circular_buffer_fixed_capacity() { + clear(); +} + +template <typename T, size_t Capacity> +inline +void +circular_buffer_fixed_capacity<T, Capacity>::push_front(const T& data) { + new (obj(_begin - 1)) T(data); + --_begin; +} + +template <typename T, size_t Capacity> +inline +void +circular_buffer_fixed_capacity<T, Capacity>::push_front(T&& data) { + new (obj(_begin - 1)) T(std::move(data)); + --_begin; +} + +template <typename T, size_t Capacity> +template <typename... Args> +inline +T& +circular_buffer_fixed_capacity<T, Capacity>::emplace_front(Args&&... args) { + auto p = new (obj(_begin - 1)) T(std::forward<Args>(args)...); + --_begin; + return *p; +} + +template <typename T, size_t Capacity> +inline +void +circular_buffer_fixed_capacity<T, Capacity>::push_back(const T& data) { + new (obj(_end)) T(data); + ++_end; +} + +template <typename T, size_t Capacity> +inline +void +circular_buffer_fixed_capacity<T, Capacity>::push_back(T&& data) { + new (obj(_end)) T(std::move(data)); + ++_end; +} + +template <typename T, size_t Capacity> +template <typename... Args> +inline +T& +circular_buffer_fixed_capacity<T, Capacity>::emplace_back(Args&&... args) { + auto p = new (obj(_end)) T(std::forward<Args>(args)...); + ++_end; + return *p; +} + +template <typename T, size_t Capacity> +inline +T& +circular_buffer_fixed_capacity<T, Capacity>::front() { + return *obj(_begin); +} + +template <typename T, size_t Capacity> +inline +T& +circular_buffer_fixed_capacity<T, Capacity>::back() { + return *obj(_end - 1); +} + +template <typename T, size_t Capacity> +inline +void +circular_buffer_fixed_capacity<T, Capacity>::pop_front() { + obj(_begin)->~T(); + ++_begin; +} + +template <typename T, size_t Capacity> +inline +void +circular_buffer_fixed_capacity<T, Capacity>::pop_back() { + obj(_end - 1)->~T(); + --_end; +} + +template <typename T, size_t Capacity> +inline +T& +circular_buffer_fixed_capacity<T, Capacity>::operator[](size_t idx) { + return *obj(_begin + idx); +} + +template <typename T, size_t Capacity> +inline +typename circular_buffer_fixed_capacity<T, Capacity>::iterator +circular_buffer_fixed_capacity<T, Capacity>::erase(iterator first, iterator last) { + static_assert(std::is_nothrow_move_assignable<T>::value, "erase() assumes move assignment does not throw"); + if (first == last) { + return last; + } + // Move to the left or right depending on which would result in least amount of moves. + // This also guarantees that iterators will be stable when removing from either front or back. + if (std::distance(begin(), first) < std::distance(last, end())) { + auto new_start = std::move_backward(begin(), first, last); + auto i = begin(); + while (i < new_start) { + *i++.~T(); + } + _begin = new_start.idx; + return last; + } else { + auto new_end = std::move(last, end(), first); + auto i = new_end; + auto e = end(); + while (i < e) { + *i++.~T(); + } + _end = new_end.idx; + return first; + } +} + +template <typename T, size_t Capacity> +inline +void +circular_buffer_fixed_capacity<T, Capacity>::clear() { + for (auto& obj : *this) { + obj.~T(); + } + _begin = _end = 0; +} + +} + diff --git a/src/seastar/include/seastar/core/condition-variable.hh b/src/seastar/include/seastar/core/condition-variable.hh new file mode 100644 index 000000000..c9911c30d --- /dev/null +++ b/src/seastar/include/seastar/core/condition-variable.hh @@ -0,0 +1,411 @@ +/* + * 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) 2016 ScyllaDB, Ltd. + */ + +#pragma once + +#include <boost/intrusive/list.hpp> + +#include <seastar/core/timer.hh> +#ifdef SEASTAR_COROUTINES_ENABLED +# include <seastar/core/coroutine.hh> +#endif +#include <seastar/core/loop.hh> + +namespace seastar { + +/// \addtogroup fiber-module +/// @{ + +/// Exception thrown when a condition variable is broken by +/// \ref condition_variable::broken(). +class broken_condition_variable : public std::exception { +public: + /// Reports the exception reason. + virtual const char* what() const noexcept; +}; + +/// Exception thrown when wait() operation times out +/// \ref condition_variable::wait(time_point timeout). +class condition_variable_timed_out : public std::exception { +public: + /// Reports the exception reason. + virtual const char* what() const noexcept; +}; + +/// \brief Conditional variable. +/// +/// This is a standard computer science condition variable sans locking, +/// since in seastar access to variables is atomic anyway, adapted +/// for futures. You can wait for variable to be notified. +/// +/// To support exceptional conditions, a \ref broken() method +/// is provided, which causes all current waiters to stop waiting, +/// with an exceptional future returned. This allows causing all +/// fibers that are blocked on a condition variable to continue. +/// This issimilar to POSIX's `pthread_cancel()`, with \ref wait() +/// acting as a cancellation point. + +class condition_variable { +private: + // the base for queue waiters. looks complicated, but this is + // to make it transparent once we add non-promise based nodes + struct waiter : public boost::intrusive::list_base_hook<boost::intrusive::link_mode<boost::intrusive::auto_unlink>> { + virtual ~waiter() = default; + void timeout() noexcept; + + virtual void signal() noexcept = 0; + virtual void set_exception(std::exception_ptr) noexcept = 0; + }; + + struct promise_waiter : public waiter, public promise<> { + void signal() noexcept override { + set_value(); + // note: we self-delete in either case we are woken + // up. See usage below: only the resulting future + // state is required once we've left the wait queue + delete this; + } + void set_exception(std::exception_ptr ep) noexcept override { + promise<>::set_exception(std::move(ep)); + // see comment above + delete this; + } + }; + +#ifdef SEASTAR_COROUTINES_ENABLED + struct [[nodiscard("must co_await a when() call")]] awaiter : public waiter, private seastar::task { + using handle_type = std::coroutine_handle<void>; + + condition_variable* _cv; + handle_type _when_ready; + std::exception_ptr _ex; + task* _waiting_task = nullptr; + + awaiter(condition_variable* cv) + : _cv(cv) + {} + + bool await_ready() const { + return _cv->check_and_consume_signal(); + } + template<typename T> + void await_suspend(std::coroutine_handle<T> h) { + _when_ready = h; + _waiting_task = &h.promise(); + _cv->add_waiter(*this); + } + void run_and_dispose() noexcept override { + _when_ready.resume(); + } + task* waiting_task() noexcept override { + return _waiting_task; + } + void await_resume() { + if (_ex) { + std::rethrow_exception(std::move(_ex)); + } + } + void signal() noexcept override { + schedule(this); + } + void set_exception(std::exception_ptr ep) noexcept override { + _ex = std::move(ep); + schedule(this); + } + }; + + template<typename Clock, typename Duration> + struct [[nodiscard("must co_await a when() call")]] timeout_awaiter : public awaiter, public timer<Clock> { + using my_type = timeout_awaiter<Clock, Duration>; + using time_point = std::chrono::time_point<Clock, Duration>; + + time_point _timeout; + + timeout_awaiter(condition_variable* cv, time_point timeout) + : awaiter(cv) + , _timeout(timeout) + {} + template<typename T> + void await_suspend(std::coroutine_handle<T> h) { + awaiter::await_suspend(std::move(h)); + this->set_callback(std::bind(&waiter::timeout, this)); + this->arm(_timeout); + } + void signal() noexcept override { + this->cancel(); + awaiter::signal(); + } + void set_exception(std::exception_ptr ep) noexcept override { + this->cancel(); + awaiter::set_exception(std::move(ep)); + } + }; + + template<typename Func, typename Base> + struct [[nodiscard("must co_await a when() call")]] predicate_awaiter : public Base { + Func _func; + template<typename... Args> + predicate_awaiter(Func func, Args&& ...args) + : Base(std::forward<Args>(args)...) + , _func(std::move(func)) + {} + bool await_ready() const { + if (!_func()) { + Base::await_ready(); // clear out any signal state + return false; + } + return true; + } + void signal() noexcept override { + if (Base::_ex || _func()) { + Base::signal(); + } else { + // must re-enter waiter queue + // this maintains "wait" version + // semantics of moving to back of queue + // if predicate fails + Base::_cv->add_waiter(*this); + } + } + }; +#endif + + boost::intrusive::list<waiter, boost::intrusive::constant_time_size<false>> _waiters; + std::exception_ptr _ex; //"broken" exception + bool _signalled = false; // set to true if signalled while no waiters + + void add_waiter(waiter&) noexcept; + void timeout(waiter&) noexcept; + bool wakeup_first() noexcept; + bool check_and_consume_signal() noexcept; +public: + /// Constructs a condition_variable object. + /// Initialzie the semaphore with a default value of 0 to enusre + /// the first call to wait() before signal() won't be waken up immediately. + condition_variable() noexcept = default; + condition_variable(condition_variable&& rhs) noexcept = default; + ~condition_variable(); + + /// Waits until condition variable is signaled, may wake up without condition been met + /// + /// \return a future that becomes ready when \ref signal() is called + /// If the condition variable was \ref broken() will return \ref broken_condition_variable + /// exception. + future<> wait() noexcept { + if (check_and_consume_signal()) { + return make_ready_future(); + } + auto* w = new promise_waiter; + auto f = w->get_future(); + add_waiter(*w); + return f; + } + + /// Waits until condition variable is signaled or timeout is reached + /// + /// \param timeout time point at which wait will exit with a timeout + /// + /// \return a future that becomes ready when \ref signal() is called + /// If the condition variable was \ref broken() will return \ref broken_condition_variable + /// exception. If timepoint is reached will return \ref condition_variable_timed_out exception. + template<typename Clock = typename timer<>::clock, typename Duration = typename Clock::duration> + future<> wait(std::chrono::time_point<Clock, Duration> timeout) noexcept { + if (check_and_consume_signal()) { + return make_ready_future(); + } + struct timeout_waiter : public promise_waiter, public timer<Clock> {}; + + auto w = std::make_unique<timeout_waiter>(); + auto f = w->get_future(); + + w->set_callback(std::bind(&waiter::timeout, w.get())); + w->arm(timeout); + add_waiter(*w.release()); + return f; + } + + /// Waits until condition variable is signaled or timeout is reached + /// + /// \param timeout duration after which wait will exit with a timeout + /// + /// \return a future that becomes ready when \ref signal() is called + /// If the condition variable was \ref broken() will return \ref broken_condition_variable + /// exception. If timepoint is passed will return \ref condition_variable_timed_out exception. + template<typename Rep, typename Period> + future<> wait(std::chrono::duration<Rep, Period> timeout) noexcept { + return wait(timer<>::clock::now() + timeout); + } + + /// Waits until condition variable is notified and pred() == true, otherwise + /// wait again. + /// + /// \param pred predicate that checks that awaited condition is true + /// + /// \return a future that becomes ready when \ref signal() is called + /// If the condition variable was \ref broken(), may contain an exception. + template<typename Pred> + SEASTAR_CONCEPT( requires seastar::InvokeReturns<Pred, bool> ) + future<> wait(Pred&& pred) noexcept { + return do_until(std::forward<Pred>(pred), [this] { + return wait(); + }); + } + + /// Waits until condition variable is notified and pred() == true or timeout is reached, otherwise + /// wait again. + /// + /// \param timeout time point at which wait will exit with a timeout + /// \param pred predicate that checks that awaited condition is true + /// + /// \return a future that becomes ready when \ref signal() is called + /// If the condition variable was \ref broken() will return \ref broken_condition_variable + /// exception. If timepoint is reached will return \ref condition_variable_timed_out exception. + template<typename Clock = typename timer<>::clock, typename Duration = typename Clock::duration, typename Pred> + SEASTAR_CONCEPT( requires seastar::InvokeReturns<Pred, bool> ) + future<> wait(std::chrono::time_point<Clock, Duration> timeout, Pred&& pred) noexcept { + return do_until(std::forward<Pred>(pred), [this, timeout] { + return wait(timeout); + }); + } + + /// Waits until condition variable is notified and pred() == true or timeout is reached, otherwise + /// wait again. + /// + /// \param timeout duration after which wait will exit with a timeout + /// \param pred predicate that checks that awaited condition is true + /// + /// \return a future that becomes ready when \ref signal() is called + /// If the condition variable was \ref broken() will return \ref broken_condition_variable + /// exception. If timepoint is passed will return \ref condition_variable_timed_out exception. + template<typename Rep, typename Period, typename Pred> + SEASTAR_CONCEPT( requires seastar::InvokeReturns<Pred, bool> ) + future<> wait(std::chrono::duration<Rep, Period> timeout, Pred&& pred) noexcept { + return wait(timer<>::clock::now() + timeout, std::forward<Pred>(pred)); + } + +#ifdef SEASTAR_COROUTINES_ENABLED + /// Coroutine/co_await only waiter. + /// Waits until condition variable is signaled, may wake up without condition been met + /// + /// \return a future that becomes ready when \ref signal() is called + /// If the condition variable was \ref broken() will return \ref broken_condition_variable + /// exception. + awaiter when() noexcept { + return awaiter{this}; + } + + /// Coroutine/co_await only waiter. + /// Waits until condition variable is signaled or timeout is reached + /// + /// \param timeout time point at which wait will exit with a timeout + /// + /// \return a future that becomes ready when \ref signal() is called + /// If the condition variable was \ref broken() will return \ref broken_condition_variable + /// exception. If timepoint is reached will return \ref condition_variable_timed_out exception. + template<typename Clock = typename timer<>::clock, typename Duration = typename Clock::duration> + timeout_awaiter<Clock, Duration> when(std::chrono::time_point<Clock, Duration> timeout) noexcept { + return timeout_awaiter<Clock, Duration>{this, timeout}; + } + + /// Coroutine/co_await only waiter. + /// Waits until condition variable is signaled or timeout is reached + /// + /// \param timeout duration after which wait will exit with a timeout + /// + /// \return a future that becomes ready when \ref signal() is called + /// If the condition variable was \ref broken() will return \ref broken_condition_variable + /// exception. If timepoint is passed will return \ref condition_variable_timed_out exception. + template<typename Rep, typename Period> + auto when(std::chrono::duration<Rep, Period> timeout) noexcept { + return when(timer<>::clock::now() + timeout); + } + + /// Coroutine/co_await only waiter. + /// Waits until condition variable is notified and pred() == true, otherwise + /// wait again. + /// + /// \param pred predicate that checks that awaited condition is true + /// + /// \return a future that becomes ready when \ref signal() is called + /// If the condition variable was \ref broken(), may contain an exception. + template<typename Pred> + SEASTAR_CONCEPT( requires seastar::InvokeReturns<Pred, bool> ) + auto when(Pred&& pred) noexcept { + return predicate_awaiter<Pred, awaiter>{std::forward<Pred>(pred), when()}; + } + + /// Coroutine/co_await only waiter. + /// Waits until condition variable is notified and pred() == true or timeout is reached, otherwise + /// wait again. + /// + /// \param timeout time point at which wait will exit with a timeout + /// \param pred predicate that checks that awaited condition is true + /// + /// \return a future that becomes ready when \ref signal() is called + /// If the condition variable was \ref broken() will return \ref broken_condition_variable + /// exception. If timepoint is reached will return \ref condition_variable_timed_out exception. + template<typename Clock = typename timer<>::clock, typename Duration = typename Clock::duration, typename Pred> + SEASTAR_CONCEPT( requires seastar::InvokeReturns<Pred, bool> ) + auto when(std::chrono::time_point<Clock, Duration> timeout, Pred&& pred) noexcept { + return predicate_awaiter<Pred, timeout_awaiter<Clock, Duration>>{std::forward<Pred>(pred), when(timeout)}; + } + + /// Coroutine/co_await only waiter. + /// Waits until condition variable is notified and pred() == true or timeout is reached, otherwise + /// wait again. + /// + /// \param timeout duration after which wait will exit with a timeout + /// \param pred predicate that checks that awaited condition is true + /// + /// \return a future that becomes ready when \ref signal() is called + /// If the condition variable was \ref broken() will return \ref broken_condition_variable + /// exception. If timepoint is passed will return \ref condition_variable_timed_out exception. + template<typename Rep, typename Period, typename Pred> + SEASTAR_CONCEPT( requires seastar::InvokeReturns<Pred, bool> ) + auto when(std::chrono::duration<Rep, Period> timeout, Pred&& pred) noexcept { + return when(timer<>::clock::now() + timeout, std::forward<Pred>(pred)); + } + +#endif + + /// Whether or not the condition variable currently has pending waiter(s) + /// The returned answer is valid until next continuation/fiber switch. + bool has_waiters() const noexcept { + return !_waiters.empty(); + } + + /// Notify variable and wake up a waiter if there is one + void signal() noexcept; + + /// Notify variable and wake up all waiter + void broadcast() noexcept; + + /// Signal to waiters that an error occurred. \ref wait() will see + /// an exceptional future<> containing the provided exception parameter. + /// The future is made available immediately. + void broken() noexcept; + + void broken(std::exception_ptr) noexcept; +}; + +/// @} + +} diff --git a/src/seastar/include/seastar/core/coroutine.hh b/src/seastar/include/seastar/core/coroutine.hh new file mode 100644 index 000000000..f8fe0b9db --- /dev/null +++ b/src/seastar/include/seastar/core/coroutine.hh @@ -0,0 +1,283 @@ +/* + * 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) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include <seastar/core/future.hh> + +#ifndef SEASTAR_COROUTINES_ENABLED +#error Coroutines support disabled. +#endif + +#include <seastar/coroutine/exception.hh> +#include <coroutine> + +namespace seastar { + +namespace internal { + +template <typename T = void> +class coroutine_traits_base { +public: + class promise_type final : public seastar::task { + seastar::promise<T> _promise; + public: + promise_type() = default; + promise_type(promise_type&&) = delete; + promise_type(const promise_type&) = delete; + + template<typename... U> + void return_value(U&&... value) { + _promise.set_value(std::forward<U>(value)...); + } + + void return_value(coroutine::exception ce) noexcept { + _promise.set_exception(std::move(ce.eptr)); + } + + void set_exception(std::exception_ptr&& eptr) noexcept { + _promise.set_exception(std::move(eptr)); + } + + [[deprecated("Forwarding coroutine returns are deprecated as too dangerous. Use 'co_return co_await ...' until explicit syntax is available.")]] + void return_value(future<T>&& fut) noexcept { + fut.forward_to(std::move(_promise)); + } + + void unhandled_exception() noexcept { + _promise.set_exception(std::current_exception()); + } + + seastar::future<T> get_return_object() noexcept { + return _promise.get_future(); + } + + std::suspend_never initial_suspend() noexcept { return { }; } + std::suspend_never final_suspend() noexcept { return { }; } + + virtual void run_and_dispose() noexcept override { + auto handle = std::coroutine_handle<promise_type>::from_promise(*this); + handle.resume(); + } + + task* waiting_task() noexcept override { return _promise.waiting_task(); } + + scheduling_group set_scheduling_group(scheduling_group sg) noexcept { + return std::exchange(this->_sg, sg); + } + }; +}; + +template <> +class coroutine_traits_base<> { +public: + class promise_type final : public seastar::task { + seastar::promise<> _promise; + public: + promise_type() = default; + promise_type(promise_type&&) = delete; + promise_type(const promise_type&) = delete; + + void return_void() noexcept { + _promise.set_value(); + } + + void set_exception(std::exception_ptr&& eptr) noexcept { + _promise.set_exception(std::move(eptr)); + } + + void unhandled_exception() noexcept { + _promise.set_exception(std::current_exception()); + } + + seastar::future<> get_return_object() noexcept { + return _promise.get_future(); + } + + std::suspend_never initial_suspend() noexcept { return { }; } + std::suspend_never final_suspend() noexcept { return { }; } + + virtual void run_and_dispose() noexcept override { + auto handle = std::coroutine_handle<promise_type>::from_promise(*this); + handle.resume(); + } + + task* waiting_task() noexcept override { return _promise.waiting_task(); } + + scheduling_group set_scheduling_group(scheduling_group new_sg) noexcept { + return task::set_scheduling_group(new_sg); + } + }; +}; + +template<bool CheckPreempt, typename... T> +struct awaiter { + seastar::future<T...> _future; +public: + explicit awaiter(seastar::future<T...>&& f) noexcept : _future(std::move(f)) { } + + awaiter(const awaiter&) = delete; + awaiter(awaiter&&) = delete; + + bool await_ready() const noexcept { + return _future.available() && (!CheckPreempt || !need_preempt()); + } + + template<typename U> + void await_suspend(std::coroutine_handle<U> hndl) noexcept { + if (!CheckPreempt || !_future.available()) { + _future.set_coroutine(hndl.promise()); + } else { + schedule(&hndl.promise()); + } + } + + std::tuple<T...> await_resume() { return _future.get(); } +}; + +template<bool CheckPreempt, typename T> +struct awaiter<CheckPreempt, T> { + seastar::future<T> _future; +public: + explicit awaiter(seastar::future<T>&& f) noexcept : _future(std::move(f)) { } + + awaiter(const awaiter&) = delete; + awaiter(awaiter&&) = delete; + + bool await_ready() const noexcept { + return _future.available() && (!CheckPreempt || !need_preempt()); + } + + template<typename U> + void await_suspend(std::coroutine_handle<U> hndl) noexcept { + if (!CheckPreempt || !_future.available()) { + _future.set_coroutine(hndl.promise()); + } else { + schedule(&hndl.promise()); + } + } + + T await_resume() { return _future.get0(); } +}; + +template<bool CheckPreempt> +struct awaiter<CheckPreempt> { + seastar::future<> _future; +public: + explicit awaiter(seastar::future<>&& f) noexcept : _future(std::move(f)) { } + + awaiter(const awaiter&) = delete; + awaiter(awaiter&&) = delete; + + bool await_ready() const noexcept { + return _future.available() && (!CheckPreempt || !need_preempt()); + } + + template<typename U> + void await_suspend(std::coroutine_handle<U> hndl) noexcept { + if (!CheckPreempt || !_future.available()) { + _future.set_coroutine(hndl.promise()); + } else { + schedule(&hndl.promise()); + } + } + + void await_resume() { _future.get(); } +}; + +} // seastar::internal + +template<typename... T> +auto operator co_await(future<T...> f) noexcept { + return internal::awaiter<true, T...>(std::move(f)); +} + +namespace coroutine { +/// Wrapper for a future which turns off checking for preemption +/// when awaiting it in a coroutine. +/// If constructed from a future, co_await-ing it will bypass +/// checking if the task quota is depleted, which means that +/// a ready future will be handled immediately. +template<typename... T> struct SEASTAR_NODISCARD without_preemption_check : public seastar::future<T...> { + explicit without_preemption_check(seastar::future<T...>&& f) noexcept : seastar::future<T...>(std::move(f)) {} +}; +template<typename T> struct SEASTAR_NODISCARD without_preemption_check<T> : public seastar::future<T> { + explicit without_preemption_check(seastar::future<T>&& f) noexcept : seastar::future<T>(std::move(f)) {} +}; +template<> struct SEASTAR_NODISCARD without_preemption_check<> : public seastar::future<> { + explicit without_preemption_check(seastar::future<>&& f) noexcept : seastar::future<>(std::move(f)) {} +}; + +/// Make a lambda coroutine safe for use in an outer coroutine with +/// functions that accept continuations. +/// +/// A lambda coroutine is not a safe parameter to a function that expects +/// a regular Seastar continuation. +/// +/// To use, wrap the lambda coroutine in seastar::coroutine::lambda(). The +/// lambda coroutine must complete (co_await) in the same statement. +/// +/// Example:: +/// ``` +/// // `future::then()` expects a continuation, so not safe for lambda +/// // coroutines without seastar::coroutine::lambda. +/// co_await seastar::yield().then(seastar::coroutine::lambda([captures] () -> future<> { +/// co_await seastar::coroutine::maybe_yield(); +/// // use of `captures` here can break without seastar::coroutine::lambda. +/// })); +/// ``` +/// +/// \tparam Func type of function object (typically inferred) +template <typename Func> +class lambda { + Func* _func; +public: + /// Create a lambda coroutine wrapper from a function object, to be passed + /// to a Seastar function that accepts a continuation. + explicit lambda(Func&& func) : _func(&func) {} + /// Calls the lambda coroutine object. Normally invoked by Seastar. + template <typename... Args> + decltype(auto) operator()(Args&&... args) const { + return std::invoke(*_func, std::forward<Args>(args)...); + } +}; + +} + +/// Wait for a future without a preemption check +/// +/// \param f a \c future<> wrapped with \c without_preemption_check +template<typename... T> +auto operator co_await(coroutine::without_preemption_check<T...> f) noexcept { + return internal::awaiter<false, T...>(std::move(f)); +} + +} // seastar + + +namespace std { + +template<typename... T, typename... Args> +class coroutine_traits<seastar::future<T...>, Args...> : public seastar::internal::coroutine_traits_base<T...> { +}; + +} // std + diff --git a/src/seastar/include/seastar/core/deleter.hh b/src/seastar/include/seastar/core/deleter.hh new file mode 100644 index 000000000..991199dc9 --- /dev/null +++ b/src/seastar/include/seastar/core/deleter.hh @@ -0,0 +1,281 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <memory> +#include <cstdlib> +#include <assert.h> +#include <type_traits> + +namespace seastar { + +/// \addtogroup memory-module +/// @{ + +/// Provides a mechanism for managing the lifetime of a buffer. +/// +/// A \c deleter is an object that is used to inform the consumer +/// of some buffer (not referenced by the deleter itself) how to +/// delete the buffer. This can be by calling an arbitrary function +/// or destroying an object carried by the deleter. Examples of +/// a deleter's encapsulated actions are: +/// +/// - calling \c std::free(p) on some captured pointer, p +/// - calling \c delete \c p on some captured pointer, p +/// - decrementing a reference count somewhere +/// +/// A deleter performs its action from its destructor. +class deleter final { +public: + /// \cond internal + struct impl; + struct raw_object_tag {}; + /// \endcond +private: + // if bit 0 set, point to object to be freed directly. + impl* _impl = nullptr; +public: + /// Constructs an empty deleter that does nothing in its destructor. + deleter() noexcept = default; + deleter(const deleter&) = delete; + /// Moves a deleter. + deleter(deleter&& x) noexcept : _impl(x._impl) { x._impl = nullptr; } + /// \cond internal + explicit deleter(impl* i) noexcept : _impl(i) {} + deleter(raw_object_tag, void* object) noexcept + : _impl(from_raw_object(object)) {} + /// \endcond + /// Destroys the deleter and carries out the encapsulated action. + ~deleter(); + deleter& operator=(deleter&& x) noexcept; + deleter& operator=(deleter&) = delete; + /// Performs a sharing operation. The encapsulated action will only + /// be carried out after both the original deleter and the returned + /// deleter are both destroyed. + /// + /// \return a deleter with the same encapsulated action as this one. + deleter share(); + /// Checks whether the deleter has an associated action. + explicit operator bool() const noexcept { return bool(_impl); } + /// \cond internal + void reset(impl* i) { + this->~deleter(); + new (this) deleter(i); + } + /// \endcond + /// Appends another deleter to this deleter. When this deleter is + /// destroyed, both encapsulated actions will be carried out. + void append(deleter d); +private: + static bool is_raw_object(impl* i) noexcept { + auto x = reinterpret_cast<uintptr_t>(i); + return x & 1; + } + bool is_raw_object() const noexcept { + return is_raw_object(_impl); + } + static void* to_raw_object(impl* i) noexcept { + auto x = reinterpret_cast<uintptr_t>(i); + return reinterpret_cast<void*>(x & ~uintptr_t(1)); + } + void* to_raw_object() const noexcept { + return to_raw_object(_impl); + } + impl* from_raw_object(void* object) noexcept { + auto x = reinterpret_cast<uintptr_t>(object); + return reinterpret_cast<impl*>(x | 1); + } +}; + +/// \cond internal +struct deleter::impl { + unsigned refs = 1; + deleter next; + impl(deleter next) : next(std::move(next)) {} + virtual ~impl() {} +}; +/// \endcond + +inline +deleter::~deleter() { + if (is_raw_object()) { + std::free(to_raw_object()); + return; + } + if (_impl && --_impl->refs == 0) { + delete _impl; + } +} + +inline +deleter& deleter::operator=(deleter&& x) noexcept { + if (this != &x) { + this->~deleter(); + new (this) deleter(std::move(x)); + } + return *this; +} + +/// \cond internal +template <typename Deleter> +struct lambda_deleter_impl final : deleter::impl { + Deleter del; + lambda_deleter_impl(deleter next, Deleter&& del) + : impl(std::move(next)), del(std::move(del)) {} + virtual ~lambda_deleter_impl() override { del(); } +}; + +template <typename Object> +struct object_deleter_impl final : deleter::impl { + Object obj; + object_deleter_impl(deleter next, Object&& obj) + : impl(std::move(next)), obj(std::move(obj)) {} +}; + +template <typename Object> +inline +object_deleter_impl<Object>* make_object_deleter_impl(deleter next, Object obj) { + return new object_deleter_impl<Object>(std::move(next), std::move(obj)); +} +/// \endcond + +/// Makes a \ref deleter that encapsulates the action of +/// destroying an object, as well as running another deleter. The input +/// object is moved to the deleter, and destroyed when the deleter is destroyed. +/// +/// \param next deleter that will become part of the new deleter's encapsulated action +/// \param o object whose destructor becomes part of the new deleter's encapsulated action +/// \related deleter +template <typename Object> +deleter +make_deleter(deleter next, Object o) { + return deleter(new lambda_deleter_impl<Object>(std::move(next), std::move(o))); +} + +/// Makes a \ref deleter that encapsulates the action of destroying an object. The input +/// object is moved to the deleter, and destroyed when the deleter is destroyed. +/// +/// \param o object whose destructor becomes the new deleter's encapsulated action +/// \related deleter +template <typename Object> +deleter +make_deleter(Object o) { + return make_deleter(deleter(), std::move(o)); +} + +/// \cond internal +struct free_deleter_impl final : deleter::impl { + void* obj; + free_deleter_impl(void* obj) : impl(deleter()), obj(obj) {} + virtual ~free_deleter_impl() override { std::free(obj); } +}; +/// \endcond + +inline +deleter +deleter::share() { + if (!_impl) { + return deleter(); + } + if (is_raw_object()) { + _impl = new free_deleter_impl(to_raw_object()); + } + ++_impl->refs; + return deleter(_impl); +} + +// Appends 'd' to the chain of deleters. Avoids allocation if possible. For +// performance reasons the current chain should be shorter and 'd' should be +// longer. +inline +void deleter::append(deleter d) { + if (!d._impl) { + return; + } + impl* next_impl = _impl; + deleter* next_d = this; + while (next_impl) { + if (next_impl == d._impl) { + return; // Already appended + } + if (is_raw_object(next_impl)) { + next_d->_impl = next_impl = new free_deleter_impl(to_raw_object(next_impl)); + } + + if (next_impl->refs != 1) { + next_d->_impl = next_impl = make_object_deleter_impl(deleter(next_impl), std::move(d)); + return; + } + + next_d = &next_impl->next; + next_impl = next_d->_impl; + } + next_d->_impl = d._impl; + d._impl = nullptr; +} + +/// Makes a deleter that calls \c std::free() when it is destroyed. +/// +/// \param obj object to free. +/// \related deleter +inline +deleter +make_free_deleter(void* obj) { + if (!obj) { + return deleter(); + } + return deleter(deleter::raw_object_tag(), obj); +} + +/// Makes a deleter that calls \c std::free() when it is destroyed, as well +/// as invoking the encapsulated action of another deleter. +/// +/// \param next deleter to invoke. +/// \param obj object to free. +/// \related deleter +inline +deleter +make_free_deleter(deleter next, void* obj) { + return make_deleter(std::move(next), [obj] () mutable { std::free(obj); }); +} + +/// \see make_deleter(Object) +/// \related deleter +template <typename T> +inline +deleter +make_object_deleter(T&& obj) { + return deleter{make_object_deleter_impl(deleter(), std::move(obj))}; +} + +/// \see make_deleter(deleter, Object) +/// \related deleter +template <typename T> +inline +deleter +make_object_deleter(deleter d, T&& obj) { + return deleter{make_object_deleter_impl(std::move(d), std::move(obj))}; +} + +/// @} + +} diff --git a/src/seastar/include/seastar/core/distributed.hh b/src/seastar/include/seastar/core/distributed.hh new file mode 100644 index 000000000..c2641f612 --- /dev/null +++ b/src/seastar/include/seastar/core/distributed.hh @@ -0,0 +1,32 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/core/sharded.hh> + +namespace seastar { + + +template <typename Service> +using distributed = sharded<Service>; + +} diff --git a/src/seastar/include/seastar/core/do_with.hh b/src/seastar/include/seastar/core/do_with.hh new file mode 100644 index 000000000..6aa3a914d --- /dev/null +++ b/src/seastar/include/seastar/core/do_with.hh @@ -0,0 +1,153 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/core/future.hh> +#include <utility> +#include <memory> +#include <tuple> + +namespace seastar { + + +/// \cond internal + +namespace internal { + +template <typename HeldState, typename Future> +class do_with_state final : public continuation_base_from_future<Future>::type { + HeldState _held; + typename Future::promise_type _pr; +public: + template<typename... T> + explicit do_with_state(T&&... args) : _held(std::forward<T>(args)...) {} + virtual void run_and_dispose() noexcept override { + _pr.set_urgent_state(std::move(this->_state)); + delete this; + } + task* waiting_task() noexcept override { + return _pr.waiting_task(); + } + HeldState& data() { + return _held; + } + Future get_future() { + return _pr.get_future(); + } +}; + +} +/// \endcond + +namespace internal { +template <typename Tuple, size_t... Idx> +inline +auto +cherry_pick_tuple(std::index_sequence<Idx...>, Tuple&& tuple) { + return std::forward_as_tuple(std::get<Idx>(std::forward<Tuple>(tuple))...); +} + +template <typename Tuple, typename Seq> +struct subtuple; + +template <typename Tuple, size_t... Idx> +struct subtuple<Tuple, std::index_sequence<Idx...>> { + using type = std::tuple<std::decay_t<std::tuple_element_t<Idx, Tuple>>...>; +}; + +template <typename T1, typename T2, typename... More> +inline +auto +do_with_impl(T1&& rv1, T2&& rv2, More&&... more) { + auto all = std::forward_as_tuple( + std::forward<T1>(rv1), + std::forward<T2>(rv2), + std::forward<More>(more)...); + constexpr size_t nr = std::tuple_size<decltype(all)>::value - 1; + using idx = std::make_index_sequence<nr>; + auto&& just_values = cherry_pick_tuple(idx(), std::move(all)); + auto&& just_func = std::move(std::get<nr>(std::move(all))); + using value_tuple = typename subtuple<decltype(all), idx>::type; + using ret_type = decltype(std::apply(just_func, std::declval<value_tuple&>())); + auto task = std::apply( + [](auto&&... x) { + return std::make_unique<internal::do_with_state<value_tuple, ret_type>>(std::forward<decltype(x)>(x)...); + }, + std::move(just_values)); + auto fut = std::apply(just_func, task->data()); + if (fut.available()) { + return fut; + } + auto ret = task->get_future(); + internal::set_callback(std::move(fut), task.release()); + return ret; +} +} + +/// \addtogroup future-util +/// @{ + +/// do_with() holds a objects alive until a future completes, and +/// allow the code involved in making the future complete to have easy +/// access to this object. +/// +/// do_with() takes multiple arguments: The last is a function +/// returning a future. The other are temporary objects (rvalue). The +/// function is given (a moved copy of) these temporary object, by +/// reference, and it is ensured that the objects will not be +/// destructed until the completion of the future returned by the +/// function. +/// +/// do_with() returns a future which resolves to whatever value the given future +/// (returned by the given function) resolves to. This returned value must not +/// contain references to the temporary object, as at that point the temporary +/// is destructed. +/// +/// \return whatever the function returns +template <typename T1, typename T2, typename... More> +inline +auto +do_with(T1&& rv1, T2&& rv2, More&&... more) noexcept { + auto func = internal::do_with_impl<T1, T2, More...>; + return futurize_invoke(func, std::forward<T1>(rv1), std::forward<T2>(rv2), std::forward<More>(more)...); +} + +/// Executes the function \c func making sure the lock \c lock is taken, +/// and later on properly released. +/// +/// \param lock the lock, which is any object having providing a lock() / unlock() semantics. +/// Caller must make sure that it outlives \c func. +/// \param func function to be executed +/// \returns whatever \c func returns +template<typename Lock, typename Func> +inline +auto with_lock(Lock& lock, Func&& func) { + return lock.lock().then([&lock, func = std::forward<Func>(func)] () mutable { + return futurize_invoke(func).finally([&lock] { + lock.unlock(); + }); + }); +} + +/// @} + +} diff --git a/src/seastar/include/seastar/core/dpdk_rte.hh b/src/seastar/include/seastar/core/dpdk_rte.hh new file mode 100644 index 000000000..6588dc10f --- /dev/null +++ b/src/seastar/include/seastar/core/dpdk_rte.hh @@ -0,0 +1,63 @@ +/* + * 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. + */ +#pragma once + +#ifdef SEASTAR_HAVE_DPDK + +#include <bitset> +#include <rte_config.h> +#include <rte_ethdev.h> +#include <rte_version.h> + +/*********************** Compat section ***************************************/ +// We currently support only versions 2.0 and above. +#if (RTE_VERSION < RTE_VERSION_NUM(2,0,0,0)) +#error "DPDK version above 2.0.0 is required" +#endif + +#if defined(RTE_MBUF_REFCNT_ATOMIC) +#warning "CONFIG_RTE_MBUF_REFCNT_ATOMIC should be disabled in DPDK's " \ + "config/common_linuxapp" +#endif +/******************************************************************************/ + +namespace seastar { + +namespace dpdk { + +// DPDK Environment Abstraction Layer +class eal { +public: + using cpuset = std::bitset<RTE_MAX_LCORE>; + + static void init(cpuset cpus, const std::string& argv0, const std::optional<std::string>& hugepages_path, bool dpdk_pmd); + /** + * Returns the amount of memory needed for DPDK + * @param num_cpus Number of CPUs the application is going to use + * + * @return + */ + static size_t mem_size(int num_cpus, bool hugetlbfs_membackend = true); + static bool initialized; +}; + +} // namespace dpdk + +} + +#endif // SEASTAR_HAVE_DPDK diff --git a/src/seastar/include/seastar/core/enum.hh b/src/seastar/include/seastar/core/enum.hh new file mode 100644 index 000000000..1ea342322 --- /dev/null +++ b/src/seastar/include/seastar/core/enum.hh @@ -0,0 +1,46 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ + +#pragma once + +/* + * This header file defines a hash function for enum types, using the + * standard hash function of the underlying type (such as int). This makes + * it possible to inherit from this type to + */ + +#include <type_traits> +#include <functional> +#include <cstddef> + +namespace seastar { + +template <typename T> +class enum_hash { + static_assert(std::is_enum<T>::value, "must be an enum"); +public: + std::size_t operator()(const T& e) const { + using utype = typename std::underlying_type<T>::type; + return std::hash<utype>()(static_cast<utype>(e)); + } +}; + +} diff --git a/src/seastar/include/seastar/core/exception_hacks.hh b/src/seastar/include/seastar/core/exception_hacks.hh new file mode 100644 index 000000000..5a8d54035 --- /dev/null +++ b/src/seastar/include/seastar/core/exception_hacks.hh @@ -0,0 +1,26 @@ +/* + * 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) 2017 ScyllaDB + */ + +#pragma once + +namespace seastar { +void init_phdr_cache(); +} diff --git a/src/seastar/include/seastar/core/execution_stage.hh b/src/seastar/include/seastar/core/execution_stage.hh new file mode 100644 index 000000000..01430974e --- /dev/null +++ b/src/seastar/include/seastar/core/execution_stage.hh @@ -0,0 +1,542 @@ +/* + * 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) 2017 ScyllaDB Ltd. + */ + +#pragma once + +#include <seastar/core/future.hh> +#include <seastar/core/chunked_fifo.hh> +#include <seastar/core/function_traits.hh> +#include <seastar/core/sstring.hh> +#include <seastar/core/metrics.hh> +#include <seastar/core/scheduling.hh> +#include <seastar/util/reference_wrapper.hh> +#include <seastar/util/concepts.hh> +#include <seastar/util/noncopyable_function.hh> +#include <seastar/util/tuple_utils.hh> +#include <seastar/util/std-compat.hh> +#include <fmt/format.h> +#include <fmt/ostream.h> +#include <vector> +#include <boost/range/irange.hpp> +#include <boost/range/adaptor/transformed.hpp> +#include <boost/container/static_vector.hpp> + +namespace seastar { + +/// \defgroup execution-stages Execution Stages +/// +/// \brief +/// Execution stages provide an infrastructure for processing function calls in +/// batches in order to improve instruction cache locality. +/// +/// When the application logic becomes more and more complex and the length +/// of the data processing pipeline grows it may happen that the most +/// significant bottleneck are instruction cache misses. The solution for that +/// problem may be processing similar operations in batches so that instruction +/// cache locality is improved at the cost of potentially higher latencies and +/// worse data cache locality. +/// +/// Execution stages allow batching calls to the specified function object. +/// Every time concrete_execution_stage::operator()() is used the function call +/// is added to the queue and a future is returned. Once the number of queued +/// calls reaches certain threshold the stage is flushed and a task is which +/// would execute these function calls is scheduled. Execution stages are also +/// flushed when the reactor polls for events. +/// +/// When calling a function that is wrapped inside execution stage it is +/// important to remember that the actual function call will happen at some +/// later time and it has to be guaranteed the objects passed by lvalue +/// reference are still alive. In order to avoid accidental passing of a +/// temporary object by lvalue reference the interface of execution stages +/// accepts only lvalue references wrapped in reference_wrapper. It is safe to +/// pass rvalue references, they are decayed and the objects are moved. See +/// concrete_execution_stage::operator()() for more details. + +/// \addtogroup execution-stages +/// @{ + +/// \cond internal +namespace internal { + +// Execution wraps lreferences in reference_wrapper so that the caller is forced +// to use seastar::ref(). Then when the function is actually called the +// reference is unwrapped. However, we need to distinguish between functions +// which argument is lvalue reference and functions that take +// reference_wrapper<> as an argument and not unwrap the latter. To solve this +// issue reference_wrapper_for_es type is used for wrappings done automatically +// by execution stage. +template<typename T> +struct reference_wrapper_for_es : reference_wrapper<T> { + reference_wrapper_for_es(reference_wrapper <T> rw) noexcept + : reference_wrapper<T>(std::move(rw)) {} +}; + +template<typename T> +struct wrap_for_es { + using type = T; +}; + +template<typename T> +struct wrap_for_es<T&> { + using type = reference_wrapper_for_es<T>; +}; + +template<typename T> +struct wrap_for_es<T&&> { + using type = T; +}; + +template<typename T> +decltype(auto) unwrap_for_es(T&& object) { + return std::forward<T>(object); +} + +template<typename T> +std::reference_wrapper<T> unwrap_for_es(reference_wrapper_for_es<T> ref) { + return std::reference_wrapper<T>(ref.get()); +} + +} +/// \endcond + +/// Base execution stage class +class execution_stage { +public: + struct stats { + uint64_t tasks_scheduled = 0; + uint64_t tasks_preempted = 0; + uint64_t function_calls_enqueued = 0; + uint64_t function_calls_executed = 0; + }; +protected: + bool _empty = true; + bool _flush_scheduled = false; + scheduling_group _sg; + stats _stats; + sstring _name; + metrics::metric_group _metric_group; +protected: + virtual void do_flush() noexcept = 0; +public: + explicit execution_stage(const sstring& name, scheduling_group sg = {}); + virtual ~execution_stage(); + + execution_stage(const execution_stage&) = delete; + + /// Move constructor + /// + /// \warning It is illegal to move execution_stage after any operation has + /// been pushed to it. The only reason why the move constructor is not + /// deleted is the fact that C++14 does not guarantee return value + /// optimisation which is required by make_execution_stage(). + execution_stage(execution_stage&&); + + /// Returns execution stage name + const sstring& name() const noexcept { return _name; } + + /// Returns execution stage usage statistics + const stats& get_stats() const noexcept { return _stats; } + + /// Flushes execution stage + /// + /// Ensures that a task which would execute all queued operations is + /// scheduled. Does not schedule a new task if there is one already pending + /// or the queue is empty. + /// + /// \return true if a new task has been scheduled + bool flush() noexcept; + + /// Checks whether there are pending operations. + /// + /// \return true if there is at least one queued operation + bool poll() const noexcept { + return !_empty; + } +}; + +/// \cond internal +namespace internal { + +class execution_stage_manager { + std::vector<execution_stage*> _execution_stages; + std::unordered_map<sstring, execution_stage*> _stages_by_name; +private: + execution_stage_manager() = default; + execution_stage_manager(const execution_stage_manager&) = delete; + execution_stage_manager(execution_stage_manager&&) = delete; +public: + void register_execution_stage(execution_stage& stage); + void unregister_execution_stage(execution_stage& stage) noexcept; + void update_execution_stage_registration(execution_stage& old_es, execution_stage& new_es) noexcept; + execution_stage* get_stage(const sstring& name); + bool flush() noexcept; + bool poll() const noexcept; +public: + static execution_stage_manager& get() noexcept; +}; + +} +/// \endcond + +/// \brief Concrete execution stage class +/// +/// \note The recommended way of creating execution stages is to use +/// make_execution_stage(). +/// +/// \tparam ReturnType return type of the function object +/// \tparam Args argument pack containing arguments to the function object, needs +/// to have move constructor that doesn't throw +template<typename ReturnType, typename... Args> +SEASTAR_CONCEPT(requires std::is_nothrow_move_constructible<std::tuple<Args...>>::value) +class concrete_execution_stage final : public execution_stage { + using args_tuple = std::tuple<Args...>; + static_assert(std::is_nothrow_move_constructible<args_tuple>::value, + "Function arguments need to be nothrow move constructible"); + + static constexpr size_t flush_threshold = 128; + static constexpr size_t max_queue_length = 1024; + + using return_type = futurize_t<ReturnType>; + using promise_type = typename return_type::promise_type; + using input_type = typename tuple_map_types<internal::wrap_for_es, args_tuple>::type; + + struct work_item { + input_type _in; + promise_type _ready; + + work_item(typename internal::wrap_for_es<Args>::type... args) : _in(std::move(args)...) { } + + work_item(work_item&& other) = delete; + work_item(const work_item&) = delete; + work_item(work_item&) = delete; + }; + chunked_fifo<work_item, flush_threshold> _queue; + + noncopyable_function<ReturnType (Args...)> _function; +private: + auto unwrap(input_type&& in) { + return tuple_map(std::move(in), [] (auto&& obj) { + return internal::unwrap_for_es(std::forward<decltype(obj)>(obj)); + }); + } + + virtual void do_flush() noexcept override { + while (!_queue.empty()) { + auto& wi = _queue.front(); + auto wi_in = std::move(wi._in); + auto wi_ready = std::move(wi._ready); + _queue.pop_front(); + futurize<ReturnType>::apply(_function, unwrap(std::move(wi_in))).forward_to(std::move(wi_ready)); + _stats.function_calls_executed++; + + if (need_preempt()) { + _stats.tasks_preempted++; + break; + } + } + _empty = _queue.empty(); + } +public: + explicit concrete_execution_stage(const sstring& name, scheduling_group sg, noncopyable_function<ReturnType (Args...)> f) + : execution_stage(name, sg) + , _function(std::move(f)) + { + _queue.reserve(flush_threshold); + } + explicit concrete_execution_stage(const sstring& name, noncopyable_function<ReturnType (Args...)> f) + : concrete_execution_stage(name, scheduling_group(), std::move(f)) { + } + + /// Enqueues a call to the stage's function + /// + /// Adds a function call to the queue. Objects passed by value are moved, + /// rvalue references are decayed and the objects are moved, lvalue + /// references need to be explicitly wrapped using seastar::ref(). + /// + /// Usage example: + /// ``` + /// void do_something(int&, int, std::vector<int>&&); + /// thread_local auto stage = seastar::make_execution_stage("execution-stage", do_something); + /// + /// int global_value; + /// + /// future<> func(std::vector<int> vec) { + /// //return stage(global_value, 42, std::move(vec)); // fail: use seastar::ref to pass references + /// return stage(seastar::ref(global_value), 42, std::move(vec)); // ok + /// } + /// ``` + /// + /// \param args arguments passed to the stage's function + /// \return future containing the result of the call to the stage's function + return_type operator()(typename internal::wrap_for_es<Args>::type... args) { + if (_queue.size() >= max_queue_length) { + do_flush(); + } + _queue.emplace_back(std::move(args)...); + _empty = false; + _stats.function_calls_enqueued++; + auto f = _queue.back()._ready.get_future(); + flush(); + return f; + } +}; + +/// \brief Base class for execution stages with support for automatic \ref scheduling_group inheritance +class inheriting_execution_stage { +public: + struct per_scheduling_group_stats { + scheduling_group sg; + execution_stage::stats stats; + }; + using stats = boost::container::static_vector<per_scheduling_group_stats, max_scheduling_groups()>; +}; + +/// \brief Concrete execution stage class, with support for automatic \ref scheduling_group inheritance +/// +/// A variation of \ref concrete_execution_stage that inherits the \ref scheduling_group +/// from the caller. Each call (of `operator()`) can be in its own scheduling group. +/// +/// \tparam ReturnType return type of the function object +/// \tparam Args argument pack containing arguments to the function object, needs +/// to have move constructor that doesn't throw +template<typename ReturnType, typename... Args> +SEASTAR_CONCEPT(requires std::is_nothrow_move_constructible<std::tuple<Args...>>::value) +class inheriting_concrete_execution_stage final : public inheriting_execution_stage { + using return_type = futurize_t<ReturnType>; + using args_tuple = std::tuple<Args...>; + using per_group_stage_type = concrete_execution_stage<ReturnType, Args...>; + + static_assert(std::is_nothrow_move_constructible<args_tuple>::value, + "Function arguments need to be nothrow move constructible"); + + sstring _name; + noncopyable_function<ReturnType (Args...)> _function; + std::vector<std::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 + // that selects the noncopyable_function copy constructor. Use a lambda instead. + auto wrapped_function = [&_function = _function] (Args... args) { + return _function(std::forward<Args>(args)...); + }; + auto name = fmt::format("{}.{}", _name, sg.name()); + return per_group_stage_type(name, sg, wrapped_function); + } +public: + /// Construct an inheriting concrete execution stage. + /// + /// \param name A name for the execution stage; must be unique + /// \param f Function to be called in response to operator(). The function + /// call will be deferred and batched with similar calls to increase + /// instruction cache hit rate. + inheriting_concrete_execution_stage(const sstring& name, noncopyable_function<ReturnType (Args...)> f) + : _name(std::move(name)),_function(std::move(f)) { + } + + /// Enqueues a call to the stage's function + /// + /// Adds a function call to the queue. Objects passed by value are moved, + /// rvalue references are decayed and the objects are moved, lvalue + /// references need to be explicitly wrapped using seastar::ref(). + /// + /// The caller's \ref scheduling_group will be preserved across the call. + /// + /// Usage example: + /// ``` + /// void do_something(int); + /// thread_local auto stage = seastar::inheriting_concrete_execution_stage<int>("execution-stage", do_something); + /// + /// future<> func(int x) { + /// return stage(x); + /// } + /// ``` + /// + /// \param args arguments passed to the stage's function + /// \return future containing the result of the call to the stage's function + return_type operator()(typename internal::wrap_for_es<Args>::type... args) { + auto sg = current_scheduling_group(); + auto sg_id = internal::scheduling_group_index(sg); + auto& slot = _stage_for_group[sg_id]; + if (!slot) { + slot.emplace(make_stage_for_group(sg)); + } + return (*slot)(std::move(args)...); + } + + /// Returns summary of individual execution stage usage statistics + /// + /// \returns a vector of the stats of the individual per-scheduling group + /// executation stages. Each element in the vector is a pair composed of + /// the scheduling group and the stats for the respective execution + /// stage. Scheduling groups that have had no respective calls enqueued + /// yet are omitted. + inheriting_execution_stage::stats get_stats() const noexcept { + inheriting_execution_stage::stats summary; + for (unsigned sg_id = 0; sg_id != _stage_for_group.size(); ++sg_id) { + auto sg = internal::scheduling_group_from_index(sg_id); + if (_stage_for_group[sg_id]) { + summary.push_back({sg, _stage_for_group[sg_id]->get_stats()}); + } + } + return summary; + } +}; + + +/// \cond internal +namespace internal { + +template <typename Ret, typename ArgsTuple> +struct concrete_execution_stage_helper; + +template <typename Ret, typename... Args> +struct concrete_execution_stage_helper<Ret, std::tuple<Args...>> { + using type = concrete_execution_stage<Ret, Args...>; +}; + +} +/// \endcond + +/// Creates a new execution stage +/// +/// Wraps given function object in a concrete_execution_stage. All arguments +/// of the function object are required to have move constructors that do not +/// throw. Function object may return a future or an immediate object or void. +/// +/// Moving execution stages is discouraged and illegal after first function +/// call is enqueued. +/// +/// Usage example: +/// ``` +/// double do_something(int); +/// thread_local auto stage1 = seastar::make_execution_stage("execution-stage1", do_something); +/// +/// future<double> func1(int val) { +/// return stage1(val); +/// } +/// +/// future<double> do_some_io(int); +/// thread_local auto stage2 = seastar::make_execution_stage("execution-stage2", do_some_io); +/// +/// future<double> func2(int val) { +/// return stage2(val); +/// } +/// ``` +/// +/// \param name unique name of the execution stage +/// \param sg scheduling group to run under +/// \param fn function to be executed by the stage +/// \return concrete_execution_stage +/// +template<typename Function> +auto make_execution_stage(const sstring& name, scheduling_group sg, Function&& fn) { + using traits = function_traits<Function>; + using ret_type = typename traits::return_type; + using args_as_tuple = typename traits::args_as_tuple; + using concrete_execution_stage = typename internal::concrete_execution_stage_helper<ret_type, args_as_tuple>::type; + return concrete_execution_stage(name, sg, std::forward<Function>(fn)); +} + +/// Creates a new execution stage (variant taking \ref scheduling_group) +/// +/// Wraps given function object in a concrete_execution_stage. All arguments +/// of the function object are required to have move constructors that do not +/// throw. Function object may return a future or an immediate object or void. +/// +/// Moving execution stages is discouraged and illegal after first function +/// call is enqueued. +/// +/// Usage example: +/// ``` +/// double do_something(int); +/// thread_local auto stage1 = seastar::make_execution_stage("execution-stage1", do_something); +/// +/// future<double> func1(int val) { +/// return stage1(val); +/// } +/// +/// future<double> do_some_io(int); +/// thread_local auto stage2 = seastar::make_execution_stage("execution-stage2", do_some_io); +/// +/// future<double> func2(int val) { +/// return stage2(val); +/// } +/// ``` +/// +/// \param name unique name of the execution stage (variant not taking \ref scheduling_group) +/// \param fn function to be executed by the stage +/// \return concrete_execution_stage +/// +template<typename Function> +auto make_execution_stage(const sstring& name, Function&& fn) { + return make_execution_stage(name, scheduling_group(), std::forward<Function>(fn)); +} + +/// Creates a new execution stage from a member function +/// +/// Wraps a pointer to member function in a concrete_execution_stage. When +/// a function call is pushed to the stage the first argument should be a +/// pointer to the object the function is a member of. +/// +/// Usage example: +/// ``` +/// struct foo { +/// void do_something(int); +/// }; +/// +/// thread_local auto stage = seastar::make_execution_stage("execution-stage", &foo::do_something); +/// +/// future<> func(foo& obj, int val) { +/// return stage(&obj, val); +/// } +/// ``` +/// +/// \see make_execution_stage(const sstring&, Function&&) +/// \param name unique name of the execution stage +/// \param fn member function to be executed by the stage +/// \return concrete_execution_stage +template<typename Ret, typename Object, typename... Args> +concrete_execution_stage<Ret, Object*, Args...> +make_execution_stage(const sstring& name, scheduling_group sg, Ret (Object::*fn)(Args...)) { + return concrete_execution_stage<Ret, Object*, Args...>(name, sg, std::mem_fn(fn)); +} + +template<typename Ret, typename Object, typename... Args> +concrete_execution_stage<Ret, const Object*, Args...> +make_execution_stage(const sstring& name, scheduling_group sg, Ret (Object::*fn)(Args...) const) { + return concrete_execution_stage<Ret, const Object*, Args...>(name, sg, std::mem_fn(fn)); +} + +template<typename Ret, typename Object, typename... Args> +concrete_execution_stage<Ret, Object*, Args...> +make_execution_stage(const sstring& name, Ret (Object::*fn)(Args...)) { + return make_execution_stage(name, scheduling_group(), fn); +} + +template<typename Ret, typename Object, typename... Args> +concrete_execution_stage<Ret, const Object*, Args...> +make_execution_stage(const sstring& name, Ret (Object::*fn)(Args...) const) { + return make_execution_stage(name, scheduling_group(), fn); +} + +/// @} + +} diff --git a/src/seastar/include/seastar/core/expiring_fifo.hh b/src/seastar/include/seastar/core/expiring_fifo.hh new file mode 100644 index 000000000..acc747c29 --- /dev/null +++ b/src/seastar/include/seastar/core/expiring_fifo.hh @@ -0,0 +1,217 @@ +/* + * 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) 2016 ScyllaDB + */ + +#pragma once + +#include <seastar/core/future.hh> +#include <seastar/core/chunked_fifo.hh> +#include <stdexcept> +#include <exception> +#include <memory> +#include <seastar/core/timer.hh> +#include <seastar/core/lowres_clock.hh> +#include <seastar/core/timed_out_error.hh> + +namespace seastar { + +template<typename T> +struct dummy_expiry { + void operator()(T&) noexcept {}; +}; + +template<typename... T> +struct promise_expiry { + void operator()(promise<T...>& pr) noexcept { + pr.set_exception(std::make_exception_ptr(timed_out_error())); + }; +}; + +/// Container for elements with support for expiration of entries. +/// +/// OnExpiry is a functor which will be called with a reference to T right before it expires. +/// T is removed and destroyed from the container immediately after OnExpiry returns. +/// OnExpiry callback must not modify the container, it can only modify its argument. +/// +/// The container can only be moved before any elements are pushed. +/// +template <typename T, typename OnExpiry = dummy_expiry<T>, typename Clock = lowres_clock> +class expiring_fifo { +public: + using clock = Clock; + using time_point = typename Clock::time_point; +private: + struct entry { + std::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::nullopt; + --ef._size; + ef.drop_expired_front(); + }) + { + tr.arm(timeout); + } + entry(entry&& x) = delete; + entry(const entry& x) = delete; + }; + + // If engaged, represents the first element. + // This is to avoid large allocations done by chunked_fifo for single-element cases. + // expiring_fifo is used to implement wait lists in synchronization primitives + // and in some uses it's common to have at most one waiter. + std::unique_ptr<entry> _front; + + // There is an invariant that the front element is never expired. + chunked_fifo<entry> _list; + OnExpiry _on_expiry; + size_t _size = 0; + + // Ensures that front() is not expired by dropping expired elements from the front. + void drop_expired_front() noexcept { + while (!_list.empty() && !_list.front().payload) { + _list.pop_front(); + } + if (_front && !_front->payload) { + _front.reset(); + } + } +public: + expiring_fifo() noexcept = default; + expiring_fifo(OnExpiry on_expiry) noexcept(std::is_nothrow_move_constructible_v<OnExpiry>) : _on_expiry(std::move(on_expiry)) {} + + expiring_fifo(expiring_fifo&& o) noexcept + : expiring_fifo(std::move(o._on_expiry)) { + // entry objects hold a reference to this so non-empty containers cannot be moved. + assert(o._size == 0); + } + + expiring_fifo& operator=(expiring_fifo&& o) noexcept { + if (this != &o) { + this->~expiring_fifo(); + new (this) expiring_fifo(std::move(o)); + } + return *this; + } + + /// Checks if container contains any elements + /// + /// \note Inside OnExpiry callback, the expired element is still contained. + /// + /// \return true if and only if there are any elements contained. + bool empty() const noexcept { + return _size == 0; + } + + /// Equivalent to !empty() + explicit operator bool() const noexcept { + return !empty(); + } + + /// Returns a reference to the element in the front. + /// Valid only when !empty(). + T& front() noexcept { + if (_front) { + return *_front->payload; + } + return *_list.front().payload; + } + + /// Returns a reference to the element in the front. + /// Valid only when !empty(). + const T& front() const noexcept { + if (_front) { + return *_front->payload; + } + return *_list.front().payload; + } + + /// Returns the number of elements contained. + /// + /// \note Expired elements are not contained. Expiring element is still contained when OnExpiry is called. + size_t size() const noexcept { + return _size; + } + + /// Reserves storage in the container for at least 'size' elements. + /// Note that expired elements may also take space when they are not in the front of the queue. + /// + /// Doesn't give any guarantees about exception safety of subsequent push_back(). + void reserve(size_t size) { + return _list.reserve(size); + } + + /// Adds element to the back of the queue. + /// The element will never expire. + void push_back(const T& payload) { + if (_size == 0) { + _front = std::make_unique<entry>(payload); + } else { + _list.emplace_back(payload); + } + ++_size; + } + + /// Adds element to the back of the queue. + /// The element will never expire. + void push_back(T&& payload) { + if (_size == 0) { + _front = std::make_unique<entry>(std::move(payload)); + } else { + _list.emplace_back(std::move(payload)); + } + ++_size; + } + + /// Adds element to the back of the queue. + /// The element will expire when timeout is reached, unless it is time_point::max(), in which + /// case it never expires. + void push_back(T&& payload, time_point timeout) { + if (timeout == time_point::max()) { + push_back(std::move(payload)); + return; + } + if (_size == 0) { + _front = std::make_unique<entry>(std::move(payload), *this, timeout); + } else { + _list.emplace_back(std::move(payload), *this, timeout); + } + ++_size; + } + + /// Removes the element at the front. + /// Can be called only if !empty(). + void pop_front() noexcept { + if (_front) { + _front.reset(); + } else { + _list.pop_front(); + } + --_size; + drop_expired_front(); + } +}; + +} diff --git a/src/seastar/include/seastar/core/fair_queue.hh b/src/seastar/include/seastar/core/fair_queue.hh new file mode 100644 index 000000000..dc01c0f1c --- /dev/null +++ b/src/seastar/include/seastar/core/fair_queue.hh @@ -0,0 +1,420 @@ +/* + * 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) 2016 ScyllaDB + */ +#pragma once + +#include <boost/intrusive/slist.hpp> +#include <seastar/core/sstring.hh> +#include <seastar/core/shared_ptr.hh> +#include <seastar/core/circular_buffer.hh> +#include <seastar/core/metrics_registration.hh> +#include <seastar/util/shared_token_bucket.hh> +#include <functional> +#include <queue> +#include <chrono> +#include <unordered_set> +#include <optional> + +namespace bi = boost::intrusive; + +namespace seastar { + +/// \brief describes a request that passes through the \ref fair_queue. +/// +/// A ticket is specified by a \c weight and a \c size. For example, one can specify a request of \c weight +/// 1 and \c size 16kB. If the \ref fair_queue accepts one such request per second, it will sustain 1 IOPS +/// at 16kB/s bandwidth. +/// +/// \related fair_queue +class fair_queue_ticket { + uint32_t _weight = 0; ///< the total weight of these requests for capacity purposes (IOPS). + uint32_t _size = 0; ///< the total effective size of these requests +public: + /// Constructs a fair_queue_ticket with a given \c weight and a given \c size + /// + /// \param weight the weight of the request + /// \param size the size of the request + fair_queue_ticket(uint32_t weight, uint32_t size) noexcept; + fair_queue_ticket() noexcept {} + fair_queue_ticket operator+(fair_queue_ticket desc) const noexcept; + fair_queue_ticket operator-(fair_queue_ticket desc) const noexcept; + /// Increase the quantity represented in this ticket by the amount represented by \c desc + /// \param desc another \ref fair_queue_ticket whose \c weight \c and size will be added to this one + fair_queue_ticket& operator+=(fair_queue_ticket desc) noexcept; + /// Decreases the quantity represented in this ticket by the amount represented by \c desc + /// \param desc another \ref fair_queue_ticket whose \c weight \c and size will be decremented from this one + fair_queue_ticket& operator-=(fair_queue_ticket desc) noexcept; + /// Checks if the tickets fully equals to another one + /// \param desc another \ref fair_queue_ticket to compare with + bool operator==(const fair_queue_ticket& desc) const noexcept; + + /// \returns true if the fair_queue_ticket represents a non-zero quantity. + /// + /// For a fair_queue ticket to be non-zero, at least one of its represented quantities need to + /// be non-zero + explicit operator bool() const noexcept; + bool is_non_zero() const noexcept; + + friend std::ostream& operator<<(std::ostream& os, fair_queue_ticket t); + + /// \returns the normalized value of this \ref fair_queue_ticket along a base axis + /// + /// The normalization function itself is an implementation detail, but one can expect either weight or + /// size to have more or less relative importance depending on which of the dimensions in the + /// denominator is relatively higher. For example, given this request a, and two other requests + /// b and c, such that that c has the same \c weight but a higher \c size than b, one can expect + /// the \c size component of this request to play a larger role. + /// + /// It is legal for the numerator to have one of the quantities set to zero, in which case only + /// the other quantity is taken into consideration. + /// + /// It is however not legal for the axis to have any quantity set to zero. + /// \param axis another \ref fair_queue_ticket to be used as a a base vector against which to normalize this fair_queue_ticket. + float normalize(fair_queue_ticket axis) const noexcept; + + /* + * For both dimentions checks if the first rover is ahead of the + * second and returns the difference. If behind returns zero. + */ + friend fair_queue_ticket wrapping_difference(const fair_queue_ticket& a, const fair_queue_ticket& b) noexcept; +}; + +/// \addtogroup io-module +/// @{ + +class fair_queue_entry { + friend class fair_queue; + + fair_queue_ticket _ticket; + bi::slist_member_hook<> _hook; + +public: + fair_queue_entry(fair_queue_ticket t) noexcept + : _ticket(std::move(t)) {} + using container_list_t = bi::slist<fair_queue_entry, + bi::constant_time_size<false>, + bi::cache_last<true>, + bi::member_hook<fair_queue_entry, bi::slist_member_hook<>, &fair_queue_entry::_hook>>; + + fair_queue_ticket ticket() const noexcept { return _ticket; } +}; + +/// \brief Group of queues class +/// +/// This is a fair group. It's attached by one or mode fair queues. On machines having the +/// big* amount of shards, queues use the group to borrow/lend the needed capacity for +/// requests dispatching. +/// +/// * Big means that when all shards sumbit requests alltogether the disk is unable to +/// dispatch them efficiently. The inability can be of two kinds -- either disk cannot +/// cope with the number of arriving requests, or the total size of the data withing +/// the given time frame exceeds the disk throughput. +class fair_group { +public: + using capacity_t = uint64_t; + using clock_type = std::chrono::steady_clock; + + /* + * tldr; The math + * + * Bw, Br -- write/read bandwidth (bytes per second) + * Ow, Or -- write/read iops (ops per second) + * + * xx_max -- their maximum values (configured) + * + * Throttling formula: + * + * Bw/Bw_max + Br/Br_max + Ow/Ow_max + Or/Or_max <= K + * + * where K is the scalar value <= 1.0 (also configured) + * + * Bandwidth is bytes time derivatite, iops is ops time derivative, i.e. + * Bx = d(bx)/dt, Ox = d(ox)/dt. Then the formula turns into + * + * d(bw/Bw_max + br/Br_max + ow/Ow_max + or/Or_max)/dt <= K + * + * Fair queue tickets are {w, s} weight-size pairs that are + * + * s = read_base_count * br, for reads + * Br_max/Bw_max * read_base_count * bw, for writes + * + * w = read_base_count, for reads + * Or_max/Ow_max * read_base_count, for writes + * + * Thus the formula turns into + * + * d(sum(w/W + s/S))/dr <= K + * + * where {w, s} is the ticket value if a request and sum summarizes the + * ticket values from all the requests seen so far, {W, S} is the ticket + * value that corresonds to a virtual summary of Or_max requests of + * Br_max size total. + */ + + /* + * The normalization results in a float of the 2^-30 seconds order of + * magnitude. Not to invent float point atomic arithmetics, the result + * is converted to an integer by multiplying by a factor that's large + * enough to turn these values into a non-zero integer. + * + * Also, the rates in bytes/sec when adjusted by io-queue according to + * multipliers become too large to be stored in 32-bit ticket value. + * Thus the rate resolution is applied. The t.bucket is configured with a + * time period for which the speeds from F (in above formula) are taken. + */ + + static constexpr float fixed_point_factor = float(1 << 24); + using rate_resolution = std::milli; + using token_bucket_t = internal::shared_token_bucket<capacity_t, rate_resolution, internal::capped_release::yes>; + +private: + + /* + * The dF/dt <= K limitation is managed by the modified token bucket + * algo where tokens are ticket.normalize(cost_capacity), the refill + * rate is K. + * + * The token bucket algo must have the limit on the number of tokens + * accumulated. Here it's configured so that it accumulates for the + * latency_goal duration. + * + * The replenish threshold is the minimal number of tokens to put back. + * It's reserved for future use to reduce the load on the replenish + * timestamp. + * + * The timestamp, in turn, is the time when the bucket was replenished + * last. Every time a shard tries to get tokens from bucket it first + * tries to convert the time that had passed since this timestamp + * into more tokens in the bucket. + */ + + const fair_queue_ticket _cost_capacity; + token_bucket_t _token_bucket; + +public: + + // Convert internal capacity value back into the real token + static double capacity_tokens(capacity_t cap) noexcept { + return (double)cap / fixed_point_factor / token_bucket_t::rate_cast(std::chrono::seconds(1)).count(); + } + + auto capacity_duration(capacity_t cap) const noexcept { + return _token_bucket.duration_for(cap); + } + + struct config { + sstring label = ""; + /* + * There are two "min" values that can be configured. The former one + * is the minimal weight:size pair that the upper layer is going to + * submit. However, it can submit _larger_ values, and the fair queue + * must accept those as large as the latter pair (but it can accept + * even larger values, of course) + */ + unsigned min_weight = 0; + unsigned min_size = 0; + unsigned limit_min_weight = 0; + unsigned limit_min_size = 0; + unsigned long weight_rate; + unsigned long size_rate; + float rate_factor = 1.0; + std::chrono::duration<double> rate_limit_duration = std::chrono::milliseconds(1); + }; + + explicit fair_group(config cfg); + fair_group(fair_group&&) = delete; + + fair_queue_ticket cost_capacity() const noexcept { return _cost_capacity; } + capacity_t maximum_capacity() const noexcept { return _token_bucket.limit(); } + capacity_t grab_capacity(capacity_t cap) noexcept; + clock_type::time_point replenished_ts() const noexcept { return _token_bucket.replenished_ts(); } + void release_capacity(capacity_t cap) noexcept; + void replenish_capacity(clock_type::time_point now) noexcept; + void maybe_replenish_capacity(clock_type::time_point& local_ts) noexcept; + + capacity_t capacity_deficiency(capacity_t from) const noexcept; + capacity_t ticket_capacity(fair_queue_ticket ticket) const noexcept; + + std::chrono::duration<double> rate_limit_duration() const noexcept { + std::chrono::duration<double, rate_resolution> dur((double)_token_bucket.limit() / _token_bucket.rate()); + return std::chrono::duration_cast<std::chrono::duration<double>>(dur); + } +}; + +/// \brief Fair queuing class +/// +/// This is a fair queue, allowing multiple request producers to queue requests +/// that will then be served proportionally to their classes' shares. +/// +/// To each request, a weight can also be associated. A request of weight 1 will consume +/// 1 share. Higher weights for a request will consume a proportionally higher amount of +/// shares. +/// +/// The user of this interface is expected to register multiple `priority_class_data` +/// objects, which will each have a shares attribute. +/// +/// Internally, each priority class may keep a separate queue of requests. +/// Requests pertaining to a class can go through even if they are over its +/// share limit, provided that the other classes have empty queues. +/// +/// When the classes that lag behind start seeing requests, the fair queue will serve +/// them first, until balance is restored. This balancing is expected to happen within +/// a certain time window that obeys an exponential decay. +class fair_queue { +public: + /// \brief Fair Queue configuration structure. + /// + /// \sets the operation parameters of a \ref fair_queue + /// \related fair_queue + struct config { + sstring label = ""; + std::chrono::microseconds tau = std::chrono::milliseconds(5); + }; + + using class_id = unsigned int; + class priority_class_data; + using capacity_t = fair_group::capacity_t; + using signed_capacity_t = std::make_signed<capacity_t>::type; + +private: + using clock_type = std::chrono::steady_clock; + using priority_class_ptr = priority_class_data*; + struct class_compare { + bool operator() (const priority_class_ptr& lhs, const priority_class_ptr & rhs) const noexcept; + }; + + class priority_queue : public std::priority_queue<priority_class_ptr, std::vector<priority_class_ptr>, class_compare> { + using super = std::priority_queue<priority_class_ptr, std::vector<priority_class_ptr>, class_compare>; + public: + void reserve(size_t len) { + c.reserve(len); + } + + void assert_enough_capacity() const noexcept { + assert(c.size() < c.capacity()); + } + }; + + config _config; + fair_group& _group; + clock_type::time_point _group_replenish; + fair_queue_ticket _resources_executing; + fair_queue_ticket _resources_queued; + unsigned _requests_executing = 0; + unsigned _requests_queued = 0; + priority_queue _handles; + std::vector<std::unique_ptr<priority_class_data>> _priority_classes; + size_t _nr_classes = 0; + capacity_t _last_accumulated = 0; + + /* + * When the shared capacity os over the local queue delays + * further dispatching untill better times + * + * \head -- the value group head rover is expected to cross + * \cap -- the capacity that's accounted on the group + * + * The last field is needed to "rearm" the wait in case + * queue decides that it wants to dispatch another capacity + * in the middle of the waiting + */ + struct pending { + capacity_t head; + capacity_t cap; + + pending(capacity_t t, capacity_t c) noexcept : head(t), cap(c) {} + }; + + std::optional<pending> _pending; + + void push_priority_class(priority_class_data& pc) noexcept; + void push_priority_class_from_idle(priority_class_data& pc) noexcept; + void pop_priority_class(priority_class_data& pc) noexcept; + void plug_priority_class(priority_class_data& pc) noexcept; + void unplug_priority_class(priority_class_data& pc) noexcept; + + enum class grab_result { grabbed, cant_preempt, pending }; + grab_result grab_capacity(const fair_queue_entry& ent) noexcept; + grab_result grab_pending_capacity(const fair_queue_entry& ent) noexcept; +public: + /// Constructs a fair queue with configuration parameters \c cfg. + /// + /// \param cfg an instance of the class \ref config + explicit fair_queue(fair_group& shared, config cfg); + fair_queue(fair_queue&&); + ~fair_queue(); + + sstring label() const noexcept { return _config.label; } + + /// Registers a priority class against this fair queue. + /// + /// \param shares how many shares to create this class with + void register_priority_class(class_id c, uint32_t shares); + + /// Unregister a priority class. + /// + /// It is illegal to unregister a priority class that still have pending requests. + void unregister_priority_class(class_id c); + + void update_shares_for_class(class_id c, uint32_t new_shares); + + /// \return how many waiters are currently queued for all classes. + [[deprecated("fair_queue users should not track individual requests, but resources (weight, size) passing through the queue")]] + size_t waiters() const; + + /// \return the number of requests currently executing + [[deprecated("fair_queue users should not track individual requests, but resources (weight, size) passing through the queue")]] + size_t requests_currently_executing() const; + + /// \return how much resources (weight, size) are currently queued for all classes. + fair_queue_ticket resources_currently_waiting() const; + + /// \return the amount of resources (weight, size) currently executing + fair_queue_ticket resources_currently_executing() const; + + /// Queue the entry \c ent through this class' \ref fair_queue + /// + /// The user of this interface is supposed to call \ref notify_requests_finished when the + /// request finishes executing - regardless of success or failure. + void queue(class_id c, fair_queue_entry& ent) noexcept; + + void plug_class(class_id c) noexcept; + void unplug_class(class_id c) noexcept; + + /// Notifies that ont request finished + /// \param desc an instance of \c fair_queue_ticket structure describing the request that just finished. + void notify_request_finished(fair_queue_ticket desc) noexcept; + void notify_request_cancelled(fair_queue_entry& ent) noexcept; + + /// Try to execute new requests if there is capacity left in the queue. + void dispatch_requests(std::function<void(fair_queue_entry&)> cb); + + clock_type::time_point next_pending_aio() const noexcept; + + std::vector<seastar::metrics::impl::metric_definition_impl> metrics(class_id c); +}; +/// @} + +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<seastar::fair_queue_ticket> : fmt::ostream_formatter {}; +#endif diff --git a/src/seastar/include/seastar/core/file-types.hh b/src/seastar/include/seastar/core/file-types.hh new file mode 100644 index 000000000..38052c897 --- /dev/null +++ b/src/seastar/include/seastar/core/file-types.hh @@ -0,0 +1,140 @@ +/* + * 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 2015 Cloudius Systems + */ + +#pragma once + +#include <fcntl.h> +#include <sys/stat.h> +#include <type_traits> + +namespace seastar { + +/// \addtogroup fileio-module +/// @{ + +/// Enumeration describing how a file is to be opened. +/// +/// \see file::open_file_dma() +enum class open_flags { + rw = O_RDWR, + ro = O_RDONLY, + wo = O_WRONLY, + create = O_CREAT, + truncate = O_TRUNC, + exclusive = O_EXCL, + dsync = O_DSYNC, +}; + +inline open_flags operator|(open_flags a, open_flags b) { + return open_flags(std::underlying_type_t<open_flags>(a) | std::underlying_type_t<open_flags>(b)); +} + +inline void operator|=(open_flags& a, open_flags b) { + a = (a | b); +} + +inline open_flags operator&(open_flags a, open_flags b) { + return open_flags(std::underlying_type_t<open_flags>(a) & std::underlying_type_t<open_flags>(b)); +} + +inline void operator&=(open_flags& a, open_flags b) { + a = (a & b); +} + +/// Enumeration describing the type of a directory entry being listed. +/// +/// \see file::list_directory() +enum class directory_entry_type { + unknown, + block_device, + char_device, + directory, + fifo, + link, + regular, + socket, +}; + +/// Enumeration describing the type of a particular filesystem +enum class fs_type { + other, + xfs, + ext2, + ext3, + ext4, + btrfs, + hfs, + tmpfs, +}; + +// Access flags for files/directories +enum class access_flags { + exists = F_OK, + read = R_OK, + write = W_OK, + execute = X_OK, + + // alias for directory access + lookup = execute, +}; + +inline access_flags operator|(access_flags a, access_flags b) { + return access_flags(std::underlying_type_t<access_flags>(a) | std::underlying_type_t<access_flags>(b)); +} + +inline access_flags operator&(access_flags a, access_flags b) { + return access_flags(std::underlying_type_t<access_flags>(a) & std::underlying_type_t<access_flags>(b)); +} + +// Permissions for files/directories +enum class file_permissions { + user_read = S_IRUSR, // Read by owner + user_write = S_IWUSR, // Write by owner + user_execute = S_IXUSR, // Execute by owner + + group_read = S_IRGRP, // Read by group + group_write = S_IWGRP, // Write by group + group_execute = S_IXGRP, // Execute by group + + others_read = S_IROTH, // Read by others + others_write = S_IWOTH, // Write by others + others_execute = S_IXOTH, // Execute by others + + user_permissions = user_read | user_write | user_execute, + group_permissions = group_read | group_write | group_execute, + others_permissions = others_read | others_write | others_execute, + all_permissions = user_permissions | group_permissions | others_permissions, + + default_file_permissions = user_read | user_write | group_read | group_write | others_read | others_write, // 0666 + default_dir_permissions = all_permissions, // 0777 +}; + +inline constexpr file_permissions operator|(file_permissions a, file_permissions b) { + return file_permissions(std::underlying_type_t<file_permissions>(a) | std::underlying_type_t<file_permissions>(b)); +} + +inline constexpr file_permissions operator&(file_permissions a, file_permissions b) { + return file_permissions(std::underlying_type_t<file_permissions>(a) & std::underlying_type_t<file_permissions>(b)); +} + +/// @} + +} // namespace seastar diff --git a/src/seastar/include/seastar/core/file.hh b/src/seastar/include/seastar/core/file.hh new file mode 100644 index 000000000..28e0cae35 --- /dev/null +++ b/src/seastar/include/seastar/core/file.hh @@ -0,0 +1,677 @@ +/* + * 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 2015 Cloudius Systems + */ + +#pragma once + +#include <seastar/core/do_with.hh> +#include <seastar/core/stream.hh> +#include <seastar/core/sstring.hh> +#include <seastar/core/shared_ptr.hh> +#include <seastar/core/align.hh> +#include <seastar/core/io_priority_class.hh> +#include <seastar/core/file-types.hh> +#include <seastar/util/std-compat.hh> +#include <system_error> +#include <sys/statvfs.h> +#include <sys/ioctl.h> +#include <linux/fs.h> +#include <sys/uio.h> +#include <unistd.h> + +namespace seastar { + +/// \addtogroup fileio-module +/// @{ + +/// A directory entry being listed. +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::optional<directory_entry_type> type; +}; + +/// Filesystem object stat information +struct stat_data { + uint64_t device_id; // ID of device containing file + uint64_t inode_number; // Inode number + uint64_t mode; // File type and mode + directory_entry_type type; + uint64_t number_of_links;// Number of hard links + uint64_t uid; // User ID of owner + uint64_t gid; // Group ID of owner + uint64_t rdev; // Device ID (if special file) + uint64_t size; // Total size, in bytes + uint64_t block_size; // Block size for filesystem I/O + uint64_t allocated_size; // Total size of allocated storage, in bytes + + std::chrono::system_clock::time_point time_accessed; // Time of last content access + std::chrono::system_clock::time_point time_modified; // Time of last content modification + std::chrono::system_clock::time_point time_changed; // Time of last status change (either content or attributes) +}; + +/// File open options +/// +/// Options used to configure an open file. +/// +/// \ref file +struct file_open_options { + uint64_t extent_allocation_size_hint = 1 << 20; ///< Allocate this much disk space when extending the file + bool sloppy_size = false; ///< Allow the file size not to track the amount of data written until a flush + uint64_t sloppy_size_hint = 1 << 20; ///< Hint as to what the eventual file size will be + file_permissions create_permissions = file_permissions::default_file_permissions; ///< File permissions to use when creating a file + bool append_is_unlikely = false; ///< Hint that user promises (or at least tries hard) not to write behind file size + + // The fsxattr.fsx_extsize is 32-bit + static constexpr uint64_t max_extent_allocation_size_hint = 1 << 31; +}; + +class file; +class file_impl; +class io_intent; +class file_handle; + +// A handle that can be transported across shards and used to +// create a dup(2)-like `file` object referring to the same underlying file +class file_handle_impl { +public: + virtual ~file_handle_impl() = default; + virtual std::unique_ptr<file_handle_impl> clone() const = 0; + virtual shared_ptr<file_impl> to_file() && = 0; +}; + +class file_impl { + friend class file; +protected: + static file_impl* get_file_impl(file& f); + unsigned _memory_dma_alignment = 4096; + unsigned _disk_read_dma_alignment = 4096; + unsigned _disk_write_dma_alignment = 4096; + unsigned _disk_overwrite_dma_alignment = 4096; + unsigned _read_max_length = 1u << 30; + unsigned _write_max_length = 1u << 30; +public: + virtual ~file_impl() {} + + virtual future<size_t> write_dma(uint64_t pos, const void* buffer, size_t len, const io_priority_class& pc) = 0; + virtual future<size_t> write_dma(uint64_t pos, std::vector<iovec> iov, const io_priority_class& pc) = 0; + virtual future<size_t> read_dma(uint64_t pos, void* buffer, size_t len, const io_priority_class& pc) = 0; + virtual future<size_t> read_dma(uint64_t pos, std::vector<iovec> iov, const io_priority_class& pc) = 0; + + virtual future<size_t> write_dma(uint64_t pos, const void* buffer, size_t len, const io_priority_class& pc, io_intent*) { + return write_dma(pos, buffer, len, pc); + } + virtual future<size_t> write_dma(uint64_t pos, std::vector<iovec> iov, const io_priority_class& pc, io_intent*) { + return write_dma(pos, std::move(iov), pc); + } + virtual future<size_t> read_dma(uint64_t pos, void* buffer, size_t len, const io_priority_class& pc, io_intent*) { + return read_dma(pos, buffer, len, pc); + } + virtual future<size_t> read_dma(uint64_t pos, std::vector<iovec> iov, const io_priority_class& pc, io_intent*) { + return read_dma(pos, std::move(iov), pc); + } + + virtual future<> flush(void) = 0; + virtual future<struct stat> stat(void) = 0; + virtual future<> truncate(uint64_t length) = 0; + virtual future<> discard(uint64_t offset, uint64_t length) = 0; + virtual future<int> ioctl(uint64_t cmd, void* argp) noexcept; + virtual future<int> ioctl_short(uint64_t cmd, void* argp) noexcept; + virtual future<int> fcntl(int op, uintptr_t arg) noexcept; + virtual future<int> fcntl_short(int op, uintptr_t arg) noexcept; + virtual future<> allocate(uint64_t position, uint64_t length) = 0; + virtual future<uint64_t> size(void) = 0; + virtual future<> close() = 0; + virtual std::unique_ptr<file_handle_impl> dup(); + virtual subscription<directory_entry> list_directory(std::function<future<> (directory_entry de)> next) = 0; + virtual future<temporary_buffer<uint8_t>> dma_read_bulk(uint64_t offset, size_t range_size, const io_priority_class& pc) = 0; + virtual future<temporary_buffer<uint8_t>> dma_read_bulk(uint64_t offset, size_t range_size, const io_priority_class& pc, io_intent*) { + return dma_read_bulk(offset, range_size, pc); + } + + friend class reactor; +}; + +future<shared_ptr<file_impl>> make_file_impl(int fd, file_open_options options, int oflags) noexcept; + +/// \endcond + +/// A data file on persistent storage. +/// +/// File objects represent uncached, unbuffered files. As such great care +/// must be taken to cache data at the application layer; neither seastar +/// nor the OS will cache these file. +/// +/// Data is transferred using direct memory access (DMA). This imposes +/// restrictions on file offsets and data pointers. The former must be aligned +/// on a 4096 byte boundary, while a 512 byte boundary suffices for the latter. +class file { + shared_ptr<file_impl> _file_impl; +public: + /// Default constructor constructs an uninitialized file object. + /// + /// A default constructor is useful for the common practice of declaring + /// a variable, and only assigning to it later. The uninitialized file + /// must not be used, or undefined behavior will result (currently, a null + /// pointer dereference). + /// + /// One can check whether a file object is in uninitialized state with + /// \ref operator bool(); One can reset a file back to uninitialized state + /// by assigning file() to it. + file() noexcept : _file_impl(nullptr) {} + + file(shared_ptr<file_impl> impl) noexcept + : _file_impl(std::move(impl)) {} + + /// Constructs a file object from a \ref file_handle obtained from another shard + explicit file(file_handle&& handle) noexcept; + + /// Checks whether the file object was initialized. + /// + /// \return false if the file object is uninitialized (default + /// constructed), true if the file object refers to an actual file. + explicit operator bool() const noexcept { return bool(_file_impl); } + + /// Copies a file object. The new and old objects refer to the + /// same underlying file. + /// + /// \param x file object to be copied + file(const file& x) = default; + /// Moves a file object. + file(file&& x) noexcept : _file_impl(std::move(x._file_impl)) {} + /// Assigns a file object. After assignent, the destination and source refer + /// to the same underlying file. + /// + /// \param x file object to assign to `this`. + file& operator=(const file& x) noexcept = default; + /// Moves assigns a file object. + file& operator=(file&& x) noexcept = default; + + // O_DIRECT reading requires that buffer, offset, and read length, are + // all aligned. Alignment of 4096 was necessary in the past, but no longer + // is - 512 is usually enough; But we'll need to use BLKSSZGET ioctl to + // be sure it is really enough on this filesystem. 4096 is always safe. + // In addition, if we start reading in things outside page boundaries, + // we will end up with various pages around, some of them with + // overlapping ranges. Those would be very challenging to cache. + + /// Alignment requirement for file offsets (for reads) + uint64_t disk_read_dma_alignment() const noexcept { + return _file_impl->_disk_read_dma_alignment; + } + + /// Alignment requirement for file offsets (for writes) + uint64_t disk_write_dma_alignment() const noexcept { + return _file_impl->_disk_write_dma_alignment; + } + + /// Alignment requirement for file offsets (for overwrites). + /// + /// Specifies the minimum alignment for disk offsets for + /// overwrites (writes to a location that was previously written). + /// This can be smaller than \ref disk_write_dma_alignment(), allowing + /// a reduction in disk bandwidth used. + uint64_t disk_overwrite_dma_alignment() const noexcept { + return _file_impl->_disk_overwrite_dma_alignment; + } + + /// Alignment requirement for data buffers + uint64_t memory_dma_alignment() const noexcept { + return _file_impl->_memory_dma_alignment; + } + + /// Recommended limit for read request size. + /// Submitting a larger request will not cause any error, + /// but may result in poor latencies for this and any other + /// concurrent requests + size_t disk_read_max_length() const noexcept { + return _file_impl->_read_max_length; + } + + /// Recommended limit for write request size. + /// Submitting a larger request will not cause any error, + /// but may result in poor latencies for this and any other + /// concurrent requests + size_t disk_write_max_length() const noexcept { + return _file_impl->_write_max_length; + } + + /** + * Perform a single DMA read operation. + * + * @param aligned_pos offset to begin reading at (should be aligned) + * @param aligned_buffer output buffer (should be aligned) + * @param aligned_len number of bytes to read (should be aligned) + * @param pc the IO priority class under which to queue this operation + * @param intent the IO intention confirmation (\ref seastar::io_intent) + * + * Alignment is HW dependent but use 4KB alignment to be on the safe side as + * explained above. + * + * @return number of bytes actually read + * or exceptional future in case of I/O error + */ + template <typename CharType> + future<size_t> + dma_read(uint64_t aligned_pos, CharType* aligned_buffer, size_t aligned_len, const io_priority_class& pc = default_priority_class(), io_intent* intent = nullptr) noexcept { + return dma_read_impl(aligned_pos, reinterpret_cast<uint8_t*>(aligned_buffer), aligned_len, pc, intent); + } + + /** + * Read the requested amount of bytes starting from the given offset. + * + * @param pos offset to begin reading from + * @param len number of bytes to read + * @param pc the IO priority class under which to queue this operation + * @param intent the IO intention confirmation (\ref seastar::io_intent) + * + * @return temporary buffer containing the requested data. + * or exceptional future in case of I/O error + * + * This function doesn't require any alignment for both "pos" and "len" + * + * @note size of the returned buffer may be smaller than "len" if EOF is + * reached or in case of I/O error. + */ + template <typename CharType> + future<temporary_buffer<CharType>> dma_read(uint64_t pos, size_t len, const io_priority_class& pc = default_priority_class(), io_intent* intent = nullptr) noexcept { + return dma_read_impl(pos, len, pc, intent).then([] (temporary_buffer<uint8_t> t) { + return temporary_buffer<CharType>(reinterpret_cast<CharType*>(t.get_write()), t.size(), t.release()); + }); + } + + /// Error thrown when attempting to read past end-of-file + /// with \ref dma_read_exactly(). + class eof_error : public std::exception {}; + + /** + * Read the exact amount of bytes. + * + * @param pos offset in a file to begin reading from + * @param len number of bytes to read + * @param pc the IO priority class under which to queue this operation + * @param intent the IO intention confirmation (\ref seastar::io_intent) + * + * @return temporary buffer containing the read data + * or exceptional future in case an error, holding: + * end_of_file_error if EOF is reached, file_io_error or + * std::system_error in case of I/O error. + */ + template <typename CharType> + future<temporary_buffer<CharType>> + dma_read_exactly(uint64_t pos, size_t len, const io_priority_class& pc = default_priority_class(), io_intent* intent = nullptr) noexcept { + return dma_read_exactly_impl(pos, len, pc, intent).then([] (temporary_buffer<uint8_t> t) { + return temporary_buffer<CharType>(reinterpret_cast<CharType*>(t.get_write()), t.size(), t.release()); + }); + } + + /// Performs a DMA read into the specified iovec. + /// + /// \param pos offset to read from. Must be aligned to \ref disk_read_dma_alignment. + /// \param iov vector of address/size pairs to read into. Addresses must be + /// aligned. + /// \param pc the IO priority class under which to queue this operation + /// \param intent the IO intention confirmation (\ref seastar::io_intent) + /// + /// \return a future representing the number of bytes actually read. A short + /// read may happen due to end-of-file or an I/O error. + future<size_t> dma_read(uint64_t pos, std::vector<iovec> iov, const io_priority_class& pc = default_priority_class(), io_intent* intent = nullptr) noexcept; + + /// Performs a DMA write from the specified buffer. + /// + /// \param pos offset to write into. Must be aligned to \ref disk_write_dma_alignment. + /// \param buffer aligned address of buffer to read from. Buffer must exists + /// until the future is made ready. + /// \param len number of bytes to write. Must be aligned. + /// \param pc the IO priority class under which to queue this operation + /// \param intent the IO intention confirmation (\ref seastar::io_intent) + /// + /// \return a future representing the number of bytes actually written. A short + /// write may happen due to an I/O error. + template <typename CharType> + future<size_t> dma_write(uint64_t pos, const CharType* buffer, size_t len, const io_priority_class& pc = default_priority_class(), io_intent* intent = nullptr) noexcept { + return dma_write_impl(pos, reinterpret_cast<const uint8_t*>(buffer), len, pc, intent); + } + + /// Performs a DMA write to the specified iovec. + /// + /// \param pos offset to write into. Must be aligned to \ref disk_write_dma_alignment. + /// \param iov vector of address/size pairs to write from. Addresses must be + /// aligned. + /// \param pc the IO priority class under which to queue this operation + /// \param intent the IO intention confirmation (\ref seastar::io_intent) + /// + /// \return a future representing the number of bytes actually written. A short + /// write may happen due to an I/O error. + future<size_t> dma_write(uint64_t pos, std::vector<iovec> iov, const io_priority_class& pc = default_priority_class(), io_intent* intent = nullptr) noexcept; + + /// Causes any previously written data to be made stable on persistent storage. + /// + /// Prior to a flush, written data may or may not survive a power failure. After + /// a flush, data is guaranteed to be on disk. + future<> flush() noexcept; + + /// Returns \c stat information about the file. + future<struct stat> stat() noexcept; + + /// Truncates the file to a specified length. + future<> truncate(uint64_t length) noexcept; + + /// Preallocate disk blocks for a specified byte range. + /// + /// Requests the file system to allocate disk blocks to + /// back the specified range (\c length bytes starting at + /// \c position). The range may be outside the current file + /// size; the blocks can then be used when appending to the + /// file. + /// + /// \param position beginning of the range at which to allocate + /// blocks. + /// \param length length of range to allocate. + /// \return future that becomes ready when the operation completes. + future<> allocate(uint64_t position, uint64_t length) noexcept; + + /// Discard unneeded data from the file. + /// + /// The discard operation tells the file system that a range of offsets + /// (which be aligned) is no longer needed and can be reused. + future<> discard(uint64_t offset, uint64_t length) noexcept; + + /// Generic ioctl syscall support for special file handling. + /// + /// This interface is useful for many non-standard operations on seastar::file. + /// The examples can be - querying device or file system capabilities, + /// configuring special performance or access modes on devices etc. + /// Refer ioctl(2) man page for more details. + /// + /// \param cmd ioctl command to be executed + /// \param argp pointer to the buffer which holds the argument + /// + /// \return a future containing the return value if any, or an exceptional future + /// if the operation has failed. + future<int> ioctl(uint64_t cmd, void* argp) noexcept; + + /// Performs a short ioctl syscall on seastar::file + /// + /// This is similar to generic \c ioctl; the difference is, here user indicates + /// that this operation is a short one, and does not involve any i/o or locking. + /// The \c file module will process this differently from the normal \ref ioctl(). + /// Use this method only if the user is sure that the operation does not involve any + /// blocking operation. If unsure, use the default \ref ioctl() method. + /// Refer ioctl(2) man page for more details on ioctl operation. + /// + /// \param cmd ioctl command to be executed + /// \param argp pointer to the buffer which holds the argument + /// + /// \return a future containing the return value if any, or an exceptional future + /// if the operation has failed. + future<int> ioctl_short(uint64_t cmd, void* argp) noexcept; + + /// Generic fcntl syscall support for special file handling. + /// + /// fcntl performs the operation specified by 'op' field on the file. + /// Some of the use cases can be - setting file status flags, advisory record locking, + /// managing signals, managing file leases or write hints etc. + /// Refer fcntl(2) man page for more details. + /// + /// \param op the operation to be executed + /// \param arg the optional argument + /// \return a future containing the return value if any, or an exceptional future + /// if the operation has failed + future<int> fcntl(int op, uintptr_t arg = 0UL) noexcept; + + /// Performs a 'short' fcntl syscall on seastar::file + /// + /// This is similar to generic \c fcntl; the difference is, here user indicates + /// that this operation is a short one, and does not involve any i/o or locking. + /// The \c file module will process this differently from normal \ref fcntl(). + /// Use this only if the user is sure that the operation does not involve any + /// blocking operation. If unsure, use the default \ref fcntl() method. + /// Refer fcntl(2) man page for more details on fcntl operation. + /// + /// \param op the operation to be executed + /// \param arg the optional argument + /// \return a future containing the return value if any, or an exceptional future + /// if the operation has failed + future<int> fcntl_short(int op, uintptr_t arg = 0UL) noexcept; + + /// Set a lifetime hint for the open file descriptor corresponding to seastar::file + /// + /// Write lifetime hints can be used to inform the kernel about the relative + /// expected lifetime of writes on a given inode or via open file descriptor. + /// An application may use the different hint values to separate writes into different + /// write classes, so that multiple users or applications running on a single storage back-end + /// can aggregate their I/O patterns in a consistent manner. + /// Refer fcntl(2) man page for more details on write lifetime hints. + /// + /// \param hint the hint value of the stream + /// \return future indicating success or failure + [[deprecated("This API was removed from the kernel")]] + future<> set_file_lifetime_hint(uint64_t hint) noexcept; + + /// Set a lifetime hint for the inode corresponding to seastar::file + /// + /// Write lifetime hints can be used to inform the kernel about the relative + /// expected lifetime of writes on a given inode or via open file descriptor. + /// An application may use the different hint values to separate writes into different + /// write classes, so that multiple users or applications running on a single storage back-end + /// can aggregate their I/O patterns in a consistent manner. + /// Refer fcntl(2) man page for more details on write lifetime hints. + /// + /// \param hint the hint value of the stream + /// \return future indicating success or failure + future<> set_inode_lifetime_hint(uint64_t hint) noexcept; + + /// Get the lifetime hint of the open file descriptor of seastar::file which was set by + /// \ref set_file_lifetime_hint() + /// + /// Write lifetime hints can be used to inform the kernel about the relative + /// expected lifetime of writes on a given inode or via open file descriptor. + /// An application may use the different hint values to separate writes into different + /// write classes, so that multiple users or applications running on a single storage back-end + /// can aggregate their I/O patterns in a consistent manner. + /// Refer fcntl(2) man page for more details on write lifetime hints. + /// + /// \return the hint value of the open file descriptor + [[deprecated("This API was removed from the kernel")]] + future<uint64_t> get_file_lifetime_hint() noexcept; + + /// Get the lifetime hint of the inode of seastar::file which was set by + /// \ref set_inode_lifetime_hint() + /// + /// Write lifetime hints can be used to inform the kernel about the relative + /// expected lifetime of writes on a given inode or via open file descriptor. + /// An application may use the different hint values to separate writes into different + /// write classes, so that multiple users or applications running on a single storage back-end + /// can aggregate their I/O patterns in a consistent manner. + /// Refer fcntl(2) man page for more details on write lifetime hints. + /// + /// \return the hint value of the inode + future<uint64_t> get_inode_lifetime_hint() noexcept; + + /// Gets the file size. + future<uint64_t> size() const noexcept; + + /// Closes the file. + /// + /// Flushes any pending operations and release any resources associated with + /// the file (except for stable storage). + /// + /// \note + /// \c close() never fails. It just reports errors and swallows them. + /// To ensure file data reaches stable storage, you must call \ref flush() + /// before calling \c close(). + future<> close() noexcept; + + /// Returns a directory listing, given that this file object is a directory. + subscription<directory_entry> list_directory(std::function<future<> (directory_entry de)> next); + + /** + * Read a data bulk containing the provided addresses range that starts at + * the given offset and ends at either the address aligned to + * dma_alignment (4KB) or at the file end. + * + * @param offset starting address of the range the read bulk should contain + * @param range_size size of the addresses range + * @param pc the IO priority class under which to queue this operation + * @param intent the IO intention confirmation (\ref seastar::io_intent) + * + * @return temporary buffer containing the read data bulk. + * or exceptional future holding: + * system_error exception in case of I/O error or eof_error when + * "offset" is beyond EOF. + */ + template <typename CharType> + future<temporary_buffer<CharType>> + dma_read_bulk(uint64_t offset, size_t range_size, const io_priority_class& pc = default_priority_class(), io_intent* intent = nullptr) noexcept { + return dma_read_bulk_impl(offset, range_size, pc, intent).then([] (temporary_buffer<uint8_t> t) { + return temporary_buffer<CharType>(reinterpret_cast<CharType*>(t.get_write()), t.size(), t.release()); + }); + } + + /// \brief Creates a handle that can be transported across shards. + /// + /// Creates a handle that can be transported across shards, and then + /// used to create a new shard-local \ref file object that refers to + /// the same on-disk file. + /// + /// \note Use on read-only files. + /// + file_handle dup(); +private: + future<temporary_buffer<uint8_t>> + dma_read_bulk_impl(uint64_t offset, size_t range_size, const io_priority_class& pc, io_intent* intent) noexcept; + + future<size_t> + dma_write_impl(uint64_t pos, const uint8_t* buffer, size_t len, const io_priority_class& pc, io_intent* intent) noexcept; + + future<temporary_buffer<uint8_t>> + dma_read_impl(uint64_t pos, size_t len, const io_priority_class& pc, io_intent* intent) noexcept; + + future<size_t> + dma_read_impl(uint64_t aligned_pos, uint8_t* aligned_buffer, size_t aligned_len, const io_priority_class& pc, io_intent* intent) noexcept; + + future<temporary_buffer<uint8_t>> + dma_read_exactly_impl(uint64_t pos, size_t len, const io_priority_class& pc, io_intent* intent) noexcept; + + future<uint64_t> get_lifetime_hint_impl(int op) noexcept; + future<> set_lifetime_hint_impl(int op, uint64_t hint) noexcept; + + friend class reactor; + friend class file_impl; +}; + +/// \brief Helper for ensuring a file is closed after \c func is called. +/// +/// The file provided by the \c file_fut future is passed to \c func. +/// +/// \param file_fut A future that produces a file +/// \param func A function that uses a file +/// \returns the future returned by \c func, or an exceptional future if either \c file_fut or closing the file failed. +template <typename Func> +SEASTAR_CONCEPT( requires std::invocable<Func, file&> && std::is_nothrow_move_constructible_v<Func> ) +auto with_file(future<file> file_fut, Func func) noexcept { + static_assert(std::is_nothrow_move_constructible_v<Func>, "Func's move constructor must not throw"); + return file_fut.then([func = std::move(func)] (file f) mutable { + return do_with(std::move(f), [func = std::move(func)] (file& f) mutable { + return futurize_invoke(func, f).finally([&f] { + return f.close(); + }); + }); + }); +} + +/// \brief Helper for ensuring a file is closed if \c func fails. +/// +/// The file provided by the \c file_fut future is passed to \c func. +/// * If func throws an exception E, the file is closed and we return +/// a failed future with E. +/// * If func returns a value V, the file is not closed and we return +/// a future with V. +/// Note that when an exception is not thrown, it is the +/// responsibility of func to make sure the file will be closed. It +/// can close the file itself, return it, or store it somewhere. +/// +/// \param file_fut A future that produces a file +/// \param func A function that uses a file +/// \returns the future returned by \c func, or an exceptional future if \c file_fut failed or a nested exception if closing the file failed. +template <typename Func> +SEASTAR_CONCEPT( requires std::invocable<Func, file&> && std::is_nothrow_move_constructible_v<Func> ) +auto with_file_close_on_failure(future<file> file_fut, Func func) noexcept { + static_assert(std::is_nothrow_move_constructible_v<Func>, "Func's move constructor must not throw"); + return file_fut.then([func = std::move(func)] (file f) mutable { + return do_with(std::move(f), [func = std::move(func)] (file& f) mutable { + return futurize_invoke(std::move(func), f).then_wrapped([&f] (auto ret) mutable { + if (!ret.failed()) { + return ret; + } + return ret.finally([&f] { + // If f.close() fails, return that as nested exception. + return f.close(); + }); + }); + }); + }); +} + +/// \example file_demo.cc +/// A program demonstrating the use of \ref seastar::with_file +/// and \ref seastar::with_file_close_on_failure + +/// \brief A shard-transportable handle to a file +/// +/// If you need to access a file (for reads only) across multiple shards, +/// you can use the file::dup() method to create a `file_handle`, transport +/// this file handle to another shard, and use the handle to create \ref file +/// object on that shard. This is more efficient than calling open_file_dma() +/// again. +class file_handle { + std::unique_ptr<file_handle_impl> _impl; +private: + explicit file_handle(std::unique_ptr<file_handle_impl> impl) : _impl(std::move(impl)) {} +public: + /// Copies a file handle object + file_handle(const file_handle&); + /// Moves a file handle object + file_handle(file_handle&&) noexcept; + /// Assigns a file handle object + file_handle& operator=(const file_handle&); + /// Move-assigns a file handle object + file_handle& operator=(file_handle&&) noexcept; + /// Converts the file handle object to a \ref file. + file to_file() const &; + /// Converts the file handle object to a \ref file. + file to_file() &&; + + friend class file; +}; + +/// @} + +/// An exception Cancelled IOs resolve their future into (see \ref io_intent "io_intent") +class cancelled_error : public std::exception { +public: + virtual const char* what() const noexcept { + return "cancelled"; + } +}; + +} diff --git a/src/seastar/include/seastar/core/fsnotify.hh b/src/seastar/include/seastar/core/fsnotify.hh new file mode 100644 index 000000000..5f4f1a859 --- /dev/null +++ b/src/seastar/include/seastar/core/fsnotify.hh @@ -0,0 +1,201 @@ +/* + * 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 2020 ScyllaDB Ltd. + */ + +#pragma once + +#include <memory> +#include <sys/inotify.h> + +#include <seastar/core/future.hh> +#include <seastar/core/sstring.hh> +#include <seastar/core/shared_ptr.hh> + +namespace seastar::experimental { + +/// \defgroup fsnotifier FileSystem Notifier +/// +/// Seastar provides an API which can be used to monitor filesystem modifications. +/// +/// \addtogroup fsnotifier +/// @{ + +/// \brief Filesystem modification notifier. +/// +/// This is a thin wrapper around inotify(see http://man7.org/linux/man-pages/man7/inotify.7.html), +/// which is the de-facto light-weight filesystem modification watch +/// interface in Linux and can be used to log filesystem activities, +/// reload configurations, etc. +/// +/// The wrapper provides a buffered read of events, and RAII handling +/// of watches themselves. +/// +/// \note Note that this is an experimental feature, and thus any tweaks that +/// are backward incompatible can be made before finally freezing it. +/// Besides, the impl currently does not (yet) handle re-writing watches. +class fsnotifier { + class impl; + shared_ptr<impl> _impl; +public: + class watch; + friend class watch; + + /// \brief Flags of events supported by \ref fsnotifier. + /// + /// \note Note that the flags are bit-matched with inotify and they can + /// be calculated using the bitwise AND and bitwise OR operators + /// (including the assignment form) directly to take intersection + /// or union. + enum class flags : uint32_t { + access = IN_ACCESS, // File was accessed (e.g., read(2), execve(2)). + attrib = IN_ATTRIB, // Metadata changed—for example, permissions, timestamps, extended attributes + close_write = IN_CLOSE_WRITE, // File opened for writing was closed. + close_nowrite = IN_CLOSE_NOWRITE,// File or directory not opened for writing was closed. + create_child = IN_CREATE, // File/directory created in watched directory + delete_child = IN_DELETE, // File/directory deleted from watched directory. + delete_self = IN_DELETE_SELF, // Watched file/directory was itself deleted. (This event + // also occurs if an object is moved to another filesystem) + modify = IN_MODIFY, // File was modified (e.g., write(2), truncate(2)). + move_self = IN_MOVE_SELF, // Watched file/directory was itself moved. + move_from = IN_MOVED_FROM, // Generated for the directory containing the old filename + // when a file is renamed. + move_to = IN_MOVED_TO, // Generated for the directory containing the new filename + // when a file is renamed. + open = IN_OPEN, // File was opened + close = IN_CLOSE, // close_write|close_nowrite + move = IN_MOVE, // move_from|move_to + oneshot = IN_ONESHOT, // listen for only a single notification, after which the + // token will be invalid + ignored = IN_IGNORED, // generated when a token or the file being watched is deleted + onlydir = IN_ONLYDIR, // Watch pathname only if it is a directory; the error ENOT‐ + // DIR results if pathname is not a directory. Using this + // flag provides an application with a race-free way of + // ensuring that the monitored object is a directory. + }; + + /// \brief Token of a watch point. + using watch_token = int32_t; + /// \brief Unique sequence number of associating related events. + /// + /// \note The sequence number is used to connect related events. + /// Currently, it is used only for the rename events(i.e., + /// move_from and move_to), and is 0 for other types. + using sequence_no = uint32_t; + + /// \brief Simple RAII wrapper around a \ref fsnotifier::watch_token + /// + /// The events of the path will be unregistered automatically on + /// destruction of the \ref watch. + class watch { + public: + ~watch(); + watch(watch&&) noexcept; + watch& operator=(watch&&) noexcept; + + /// Reset the watch point. + /// + /// \note Note that this operation won't unregister the event for + /// the path, but simply releases resources used internally. + watch_token release(); + + /// Cast this watch point to a watch token. + operator watch_token() const { + return _token; + } + + /// Get the token of this watch point. + watch_token token() const { + return _token; + } + + private: + friend class fsnotifier; + watch(shared_ptr<impl>, watch_token); + watch_token _token; + shared_ptr<impl> _impl; + }; + + fsnotifier(); + ~fsnotifier(); + + fsnotifier(fsnotifier&&); + fsnotifier& operator=(fsnotifier&&); + + /// \brief Monitor events specified in mask for the give path. + /// + /// \param path path of the file or directory to monitor for. + /// \param mask events of interest. + /// \return a future that becomes ready when the underlying + /// inotify_add_watch(2) call completes. + future<watch> create_watch(const sstring& path, flags mask); + + /// \brief A wrapper around inotify_event. + struct event { + // matches source watch + watch_token id; + // event(s) generated + flags mask; + sequence_no seq; // event correlation -> move_from+move_to + sstring name; // optional file name, in case of move_from/to + }; + + /// Wait for events. + /// + /// \return a future that becomes ready when registered events occur. + future<std::vector<event>> wait() const; + + /// Shutdown the notifier and abort any waiting events. + /// + /// \note After shutdown, all watches are invalidated, + /// and no new ones can be created. + void shutdown(); + + /// Check if the notifier is activated. + bool active() const; + + /// Equivalent to \ref active(). + operator bool() const { + return active(); + } +}; + +/// Take the union of two events. +inline fsnotifier::flags operator|(fsnotifier::flags a, fsnotifier::flags b) { + return fsnotifier::flags(std::underlying_type_t<fsnotifier::flags>(a) | std::underlying_type_t<fsnotifier::flags>(b)); +} + +/// Take the union of two events, assignment form. +inline void operator|=(fsnotifier::flags& a, fsnotifier::flags b) { + a = (a | b); +} + +/// Take the intersection of two events. +inline fsnotifier::flags operator&(fsnotifier::flags a, fsnotifier::flags b) { + return fsnotifier::flags(std::underlying_type_t<fsnotifier::flags>(a) & std::underlying_type_t<fsnotifier::flags>(b)); +} + +/// Take the intersection of two events, assignment form. +inline void operator&=(fsnotifier::flags& a, fsnotifier::flags b) { + a = (a & b); +} + +/// @} + +}
\ No newline at end of file diff --git a/src/seastar/include/seastar/core/fsqual.hh b/src/seastar/include/seastar/core/fsqual.hh new file mode 100644 index 000000000..24a3d1d72 --- /dev/null +++ b/src/seastar/include/seastar/core/fsqual.hh @@ -0,0 +1,30 @@ +/* + * Copyright 2017 ScyllaDB + */ +/* + * 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. + */ + +#pragma once + +#include <seastar/core/sstring.hh> + +namespace seastar { + +bool filesystem_has_good_aio_support(sstring directory, bool verbose = false); + +} diff --git a/src/seastar/include/seastar/core/fstream.hh b/src/seastar/include/seastar/core/fstream.hh new file mode 100644 index 000000000..67d59abfd --- /dev/null +++ b/src/seastar/include/seastar/core/fstream.hh @@ -0,0 +1,151 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ + +#pragma once + +/// \file + +// File <-> streams adapters +// +// Seastar files are block-based due to the reliance on DMA - you must read +// on sector boundaries. The adapters in this file provide a byte stream +// interface to files, while retaining the zero-copy characteristics of +// seastar files. + +#include <seastar/core/file.hh> +#include <seastar/core/iostream.hh> +#include <seastar/core/shared_ptr.hh> +#include <seastar/core/internal/api-level.hh> + +namespace seastar { + +class file_input_stream_history { + static constexpr uint64_t window_size = 4 * 1024 * 1024; + struct window { + uint64_t total_read = 0; + uint64_t unused_read = 0; + }; + window current_window; + window previous_window; + unsigned read_ahead = 1; + + friend class file_data_source_impl; +}; + +/// Data structure describing options for opening a file input stream +struct file_input_stream_options { + size_t buffer_size = 8192; ///< I/O buffer size + unsigned read_ahead = 0; ///< Maximum number of extra read-ahead operations + ::seastar::io_priority_class io_priority_class = default_priority_class(); + lw_shared_ptr<file_input_stream_history> dynamic_adjustments = { }; ///< Input stream history, if null dynamic adjustments are disabled +}; + +/// \brief Creates an input_stream to read a portion of a file. +/// +/// \param file File to read; multiple streams for the same file may coexist +/// \param offset Starting offset to read from (no alignment restrictions) +/// \param len Maximum number of bytes to read; the stream will stop at end-of-file +/// even if `offset + len` is beyond end-of-file. +/// \param options A set of options controlling the stream. +/// +/// \note Multiple input streams may exist concurrently for the same file. +input_stream<char> make_file_input_stream( + file file, uint64_t offset, uint64_t len, file_input_stream_options options = {}); + +// Create an input_stream for a given file, with the specified options. +// Multiple fibers of execution (continuations) may safely open +// multiple input streams concurrently for the same file. +input_stream<char> make_file_input_stream( + file file, uint64_t offset, file_input_stream_options = {}); + +// Create an input_stream for reading starting at a given position of the +// given file. Multiple fibers of execution (continuations) may safely open +// multiple input streams concurrently for the same file. +input_stream<char> make_file_input_stream( + file file, file_input_stream_options = {}); + +struct file_output_stream_options { + // For small files, setting preallocation_size can make it impossible for XFS to find + // an aligned extent. On the other hand, without it, XFS will divide the file into + // file_size/buffer_size extents. To avoid fragmentation, we set the default buffer_size + // to 64k (so each extent will be a minimum of 64k) and preallocation_size to 0 (to avoid + // extent allocation problems). + // + // Large files should increase both buffer_size and preallocation_size. + unsigned buffer_size = 65536; + unsigned preallocation_size = 0; ///< Preallocate extents. For large files, set to a large number (a few megabytes) to reduce fragmentation + unsigned write_behind = 1; ///< Number of buffers to write in parallel + ::seastar::io_priority_class io_priority_class = default_priority_class(); +}; + +SEASTAR_INCLUDE_API_V2 namespace api_v2 { + +/// Create an output_stream for writing starting at the position zero of a +/// newly created file. +/// NOTE: flush() should be the last thing to be called on a file output stream. +[[deprecated("use Seastar_API_LEVEL=3 instead")]] +output_stream<char> make_file_output_stream( + file file, + uint64_t buffer_size = 8192); + +/// Create an output_stream for writing starting at the position zero of a +/// newly created file. +/// NOTE: flush() should be the last thing to be called on a file output stream. +[[deprecated("use Seastar_API_LEVEL=3 instead")]] +output_stream<char> make_file_output_stream( + file file, + file_output_stream_options options); + +/// Create a data_sink for writing starting at the position zero of a +/// newly created file. +[[deprecated("use Seastar_API_LEVEL=3 instead")]] +data_sink make_file_data_sink(file, file_output_stream_options); + +} + +SEASTAR_INCLUDE_API_V3 namespace api_v3 { +inline namespace and_newer { + +/// Create an output_stream for writing starting at the position zero of a +/// newly created file. +/// NOTE: flush() should be the last thing to be called on a file output stream. +/// Closes the file if the stream creation fails. +future<output_stream<char>> make_file_output_stream( + file file, + uint64_t buffer_size = 8192) noexcept; + +/// Create an output_stream for writing starting at the position zero of a +/// newly created file. +/// NOTE: flush() should be the last thing to be called on a file output stream. +/// Closes the file if the stream creation fails. +future<output_stream<char>> make_file_output_stream( + file file, + file_output_stream_options options) noexcept; + +/// Create a data_sink for writing starting at the position zero of a +/// newly created file. +/// Closes the file if the sink creation fails. +future<data_sink> make_file_data_sink(file, file_output_stream_options) noexcept; + +} +} + +} diff --git a/src/seastar/include/seastar/core/function_traits.hh b/src/seastar/include/seastar/core/function_traits.hh new file mode 100644 index 000000000..a3b9b9d31 --- /dev/null +++ b/src/seastar/include/seastar/core/function_traits.hh @@ -0,0 +1,68 @@ +/* + * 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 2015 Cloudius Systems + */ + +#pragma once + +#include <tuple> + +namespace seastar { + +template<typename T> +struct function_traits; + +template<typename Ret, typename... Args> +struct function_traits<Ret(Args...)> +{ + using return_type = Ret; + using args_as_tuple = std::tuple<Args...>; + using signature = Ret (Args...); + + static constexpr std::size_t arity = sizeof...(Args); + + template <std::size_t N> + struct arg + { + static_assert(N < arity, "no such parameter index."); + using type = typename std::tuple_element<N, std::tuple<Args...>>::type; + }; +}; + +template<typename Ret, typename... Args> +struct function_traits<Ret(*)(Args...)> : public function_traits<Ret(Args...)> +{}; + +template <typename T, typename Ret, typename... Args> +struct function_traits<Ret(T::*)(Args...)> : public function_traits<Ret(Args...)> +{}; + +template <typename T, typename Ret, typename... Args> +struct function_traits<Ret(T::*)(Args...) const> : public function_traits<Ret(Args...)> +{}; + +template <typename T> +struct function_traits : public function_traits<decltype(&T::operator())> +{}; + +template<typename T> +struct function_traits<T&> : public function_traits<std::remove_reference_t<T>> +{}; + +} diff --git a/src/seastar/include/seastar/core/future-util.hh b/src/seastar/include/seastar/core/future-util.hh new file mode 100644 index 000000000..3252accf5 --- /dev/null +++ b/src/seastar/include/seastar/core/future-util.hh @@ -0,0 +1,31 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/core/future.hh> +#include <seastar/core/do_with.hh> +#include <seastar/core/with_scheduling_group.hh> +#include <seastar/core/loop.hh> +#include <seastar/core/when_all.hh> +#include <seastar/core/map_reduce.hh> +#include <seastar/core/with_timeout.hh> +#include <seastar/util/later.hh> diff --git a/src/seastar/include/seastar/core/future.hh b/src/seastar/include/seastar/core/future.hh new file mode 100644 index 000000000..f5c6dfcab --- /dev/null +++ b/src/seastar/include/seastar/core/future.hh @@ -0,0 +1,2208 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/core/task.hh> +#include <seastar/core/thread_impl.hh> +#include <stdexcept> +#include <atomic> +#include <memory> +#include <type_traits> +#include <assert.h> +#include <cstdlib> +#include <seastar/core/function_traits.hh> +#include <seastar/util/critical_alloc_section.hh> +#include <seastar/util/attribute-compat.hh> +#include <seastar/util/concepts.hh> +#include <seastar/util/noncopyable_function.hh> +#include <seastar/util/backtrace.hh> +#include <seastar/util/std-compat.hh> + +#if __cplusplus > 201703L +#include <concepts> +#endif + +namespace seastar { + +struct nested_exception : public std::exception { + std::exception_ptr inner; + std::exception_ptr outer; + nested_exception(std::exception_ptr inner, std::exception_ptr outer) noexcept; + nested_exception(nested_exception&&) noexcept; + nested_exception(const nested_exception&) noexcept; + [[noreturn]] void rethrow_nested() const; + virtual const char* what() const noexcept override; +}; + +/// \defgroup future-module Futures and Promises +/// +/// \brief +/// Futures and promises are the basic tools for asynchronous +/// programming in seastar. A future represents a result that +/// may not have been computed yet, for example a buffer that +/// is being read from the disk, or the result of a function +/// that is executed on another cpu. A promise object allows +/// the future to be eventually resolved by assigning it a value. +/// +/// \brief +/// Another way to look at futures and promises are as the reader +/// and writer sides, respectively, of a single-item, single use +/// queue. You read from the future, and write to the promise, +/// and the system takes care that it works no matter what the +/// order of operations is. +/// +/// \brief +/// The normal way of working with futures is to chain continuations +/// to them. A continuation is a block of code (usually a lamdba) +/// that is called when the future is assigned a value (the future +/// is resolved); the continuation can then access the actual value. +/// + +/// \defgroup future-module-impl Implementation overview +/// \ingroup future-module +/// +/// A future has a stored value. Semantically, the value is a +/// std::optional<std::variant<T, std::exception_ptr>>. The actual +/// type of the value in the implementation is future_state<T>. +/// +/// A future without an initial value can be created by first creating +/// a promise and then calling promise::get_future. The promise also +/// stores a future_state<T> in case promise::set_value is called +/// before get_future. +/// +/// In addition to the future_state<T>, the promise and the future +/// point to each other and the pointers are updated when either is +/// moved. +/// +/// If a future is consumed by future::then before the future is +/// ready, a continuation is dynamically allocated. The continuation +/// also has a future_state<T>, but unlinke a future it is never +/// moved. +/// +/// After a future creates a continuation, the corresponding promise +/// points to the newly allocated continuation. When +/// promise::set_value is called, the continuation is ready and is +/// scheduled. +/// +/// A promise then consists of +/// * A future_state<T> for use when there is no corresponding future +/// or continuation (_local_state). +/// * A pointer to a future to allow updates when the promise is moved +/// (_future). +/// * A pointer to the continuation (_task). +/// * A pointer to future_state<T> (_state) that can point to +/// 1. The future_state<T> in the promise itself +/// 2. The future_state<T> in the future +/// 2. The future_state<T> in the continuation +/// +/// A special case is when a future blocks inside a thread. In that +/// case we still need a continuation, but that continuation doesn't +/// need a future_state<T> since the original future still exists on +/// the stack. +/// +/// So the valid states for a promise are: +/// +/// 1. A newly created promise. _state points to _local_state and +/// _task and _future are null. +/// 2. After get_future is called. _state points to the state in the +/// future, _future points to the future and _task is null. +/// 3. The future has been consumed by future::then. Now the _state +/// points to the state in the continuation, _future is null and +/// _task points to the continuation. +/// 4. A call to future::get is blocked in a thread. This is a mix of +/// cases 2 and 3. Like 2, there is a valid future and _future and +/// _state point to the future and its state. Like 3, there is a +/// valid continuation and _task points to it, but that +/// continuation has no state of its own. + +/// \defgroup future-util Future Utilities +/// \ingroup future-module +/// +/// \brief +/// These utilities are provided to help perform operations on futures. + + +/// \addtogroup future-module +/// @{ + +#if SEASTAR_API_LEVEL < 6 +template <class... T> +#else +template <class T = void> +#endif +class promise; + +template <class SEASTAR_ELLIPSIS T> +class future; + +template <typename... T> +class shared_future; + +struct future_state_base; + +/// \brief Creates a \ref future in an available, value state. +/// +/// Creates a \ref future object that is already resolved. This +/// is useful when it is determined that no I/O needs to be performed +/// to perform a computation (for example, because the data is cached +/// in some buffer). +template <typename... T, typename... A> +future<T...> make_ready_future(A&&... value) noexcept; + +/// \brief Creates a \ref future in an available, failed state. +/// +/// Creates a \ref future object that is already resolved in a failed +/// state. This is useful when no I/O needs to be performed to perform +/// a computation (for example, because the connection is closed and +/// we cannot read from it). +template <typename... T> +future<T...> make_exception_future(std::exception_ptr&& value) noexcept; + +template <typename... T> +future<T...> make_exception_future(const std::exception_ptr& ex) noexcept { + return make_exception_future<T...>(std::exception_ptr(ex)); +} + +template <typename... T> +future<T...> make_exception_future(std::exception_ptr& ex) noexcept { + return make_exception_future<T...>(static_cast<const std::exception_ptr&>(ex)); +} + +template <typename... T> +future<T...> make_exception_future(const std::exception_ptr&& ex) noexcept { + // as ex is const, we cannot move it, but can copy it. + return make_exception_future<T...>(std::exception_ptr(ex)); +} + +/// \cond internal +void engine_exit(std::exception_ptr eptr = {}); + +void report_failed_future(const std::exception_ptr& ex) noexcept; + +void report_failed_future(const future_state_base& state) noexcept; + +void with_allow_abandoned_failed_futures(unsigned count, noncopyable_function<void ()> func); + +/// \endcond + +/// \brief Exception type for broken promises +/// +/// When a promise is broken, i.e. a promise object with an attached +/// continuation is destroyed before setting any value or exception, an +/// exception of `broken_promise` type is propagated to that abandoned +/// continuation. +struct broken_promise : std::logic_error { + broken_promise(); +}; + +/// \brief Returns std::current_exception() wrapped in a future +/// +/// This is equivalent to +/// make_exception_future(std::current_exception()), but expands to +/// less code. +template <typename... T> +future<T...> current_exception_as_future() noexcept; + +extern template +future<> current_exception_as_future() noexcept; + +namespace internal { +#if SEASTAR_API_LEVEL < 6 +template <class... T> +#else +template <class T = void> +#endif +class promise_base_with_type; +class promise_base; + +struct monostate {}; + +template <typename... T> +struct future_stored_type; + +template <> +struct future_stored_type<> { +#if SEASTAR_API_LEVEL < 5 + using type = std::tuple<>; +#else + using type = monostate; +#endif +}; + +template <typename T> +struct future_stored_type<T> { +#if SEASTAR_API_LEVEL < 5 + using type = std::tuple<T>; +#else + using type = std::conditional_t<std::is_void_v<T>, internal::monostate, T>; +#endif +}; + +template <typename... T> +using future_stored_type_t = typename future_stored_type<T...>::type; + +template<typename T> +#if SEASTAR_API_LEVEL < 5 +using future_tuple_type_t = T; +#else +using future_tuple_type_t = std::conditional_t<std::is_same_v<T, monostate>, std::tuple<>, std::tuple<T>>; +#endif + +// It doesn't seem to be possible to use std::tuple_element_t with an empty tuple. There is an static_assert in it that +// fails the build even if it is in the non enabled side of std::conditional. +template <typename T> +struct get0_return_type; + +template <> +struct get0_return_type<std::tuple<>> { + using type = void; + static type get0(std::tuple<>) { } +}; + +template <typename T0, typename... T> +struct get0_return_type<std::tuple<T0, T...>> { + using type = T0; + static type get0(std::tuple<T0, T...> v) { return std::get<0>(std::move(v)); } +}; + +template<typename T> +using maybe_wrap_ref = std::conditional_t<std::is_reference_v<T>, std::reference_wrapper<std::remove_reference_t<T>>, T>; + +/// \brief Wrapper for keeping uninitialized values of non default constructible types. +/// +/// This is similar to a std::optional<T>, but it doesn't know if it is holding a value or not, so the user is +/// responsible for calling constructors and destructors. +/// +/// The advantage over just using a union directly is that this uses inheritance when possible and so benefits from the +/// empty base optimization. +template <typename T, bool is_trivial_class> +struct uninitialized_wrapper_base; + +template <typename T> +struct uninitialized_wrapper_base<T, false> { + using tuple_type = future_tuple_type_t<T>; + union any { + any() noexcept {} + ~any() {} + // T can be a reference, so wrap it. + maybe_wrap_ref<T> value; + } _v; + +public: + uninitialized_wrapper_base() noexcept = default; + template<typename... U> + std::enable_if_t<!std::is_same_v<std::tuple<std::remove_cv_t<U>...>, std::tuple<tuple_type>>, void> + uninitialized_set(U&&... vs) { + new (&_v.value) maybe_wrap_ref<T>{T(std::forward<U>(vs)...)}; + } + void uninitialized_set(tuple_type&& v) { + uninitialized_set(std::move(std::get<0>(v))); + } + void uninitialized_set(const tuple_type& v) { + uninitialized_set(std::get<0>(v)); + } + maybe_wrap_ref<T>& uninitialized_get() { + return _v.value; + } + const maybe_wrap_ref<T>& uninitialized_get() const { + return _v.value; + } +}; + +template <typename T> struct uninitialized_wrapper_base<T, true> : private T { + using tuple_type = future_tuple_type_t<T>; + uninitialized_wrapper_base() noexcept = default; + template<typename... U> + std::enable_if_t<!std::is_same_v<std::tuple<std::remove_cv_t<U>...>, std::tuple<tuple_type>>, void> + uninitialized_set(U&&... vs) { + new (this) T(std::forward<U>(vs)...); + } + void uninitialized_set(tuple_type&& v) { + if constexpr (std::tuple_size_v<tuple_type> != 0) { + uninitialized_set(std::move(std::get<0>(v))); + } + } + void uninitialized_set(const tuple_type& v) { + if constexpr (std::tuple_size_v<tuple_type> != 0) { + uninitialized_set(std::get<0>(v)); + } + } + T& uninitialized_get() { + return *this; + } + const T& uninitialized_get() const { + return *this; + } +}; + +template <typename T> +constexpr bool can_inherit = +#ifdef _LIBCPP_VERSION +// We expect std::tuple<> to be trivially constructible and +// destructible. That is not the case with libc++ +// (https://bugs.llvm.org/show_bug.cgi?id=41714). We could avoid this +// optimization when using libc++ and relax the asserts, but +// inspection suggests that std::tuple<> is trivial, it is just not +// marked as such. + std::is_same<std::tuple<>, T>::value || +#endif + (std::is_trivially_destructible<T>::value && std::is_trivially_constructible<T>::value && + std::is_class<T>::value && !std::is_final<T>::value); + +// The objective is to avoid extra space for empty types like std::tuple<>. We could use std::is_empty_v, but it is +// better to check that both the constructor and destructor can be skipped. +template <typename T> +struct uninitialized_wrapper + : public uninitialized_wrapper_base<T, can_inherit<T>> {}; + +template <typename T> +struct is_trivially_move_constructible_and_destructible { + static constexpr bool value = std::is_trivially_move_constructible<T>::value && std::is_trivially_destructible<T>::value; +}; + +template <bool... v> +struct all_true : std::false_type {}; + +template <> +struct all_true<> : std::true_type {}; + +template <bool... v> +struct all_true<true, v...> : public all_true<v...> {}; + +template<typename T> +struct is_tuple_effectively_trivially_move_constructible_and_destructible_helper; + +template <typename... T> +struct is_tuple_effectively_trivially_move_constructible_and_destructible_helper<std::tuple<T...>> { + static constexpr bool value = all_true<is_trivially_move_constructible_and_destructible<T>::value...>::value; +}; + +template <typename T> +static constexpr bool is_tuple_effectively_trivially_move_constructible_and_destructible = + is_tuple_effectively_trivially_move_constructible_and_destructible_helper<T>::value; + +} + +// +// A future/promise pair maintain one logical value (a future_state). +// There are up to three places that can store it, but only one is +// active at any time. +// +// - in the promise _local_state member variable +// +// This is necessary because a promise is created first and there +// would be nowhere else to put the value. +// +// - in the future _state variable +// +// This is used anytime a future exists and then has not been called +// yet. This guarantees a simple access to the value for any code +// that already has a future. +// +// - in the task associated with the .then() clause (after .then() is called, +// if a value was not set) +// +// +// The promise maintains a pointer to the state, which is modified as +// the state moves to a new location due to events (such as .then() or +// get_future being called) or due to the promise or future being +// moved around. +// + +// non templated base class to reduce code duplication +struct future_state_base { + static_assert(sizeof(std::exception_ptr) == sizeof(void*), "exception_ptr not a pointer"); + enum class state : uintptr_t { + invalid = 0, + future = 1, + // the substate is intended to decouple the run-time prevention + // for duplicative result extraction (calling e.g. then() twice + // ends up in abandoned()) from the wrapped object's destruction + // handling which is orchestrated by future_state. Instead of + // creating a temporary future_state just for the sake of setting + // the "invalid" in the source instance, result_unavailable can + // be set to ensure future_state_base::available() returns false. + result_unavailable = 2, + result = 3, + exception_min = 4, // or anything greater + }; + union any { + any() noexcept { st = state::future; } + any(state s) noexcept { st = s; } + void set_exception(std::exception_ptr&& e) noexcept { + new (&ex) std::exception_ptr(std::move(e)); + assert(st >= state::exception_min); + } + any(std::exception_ptr&& e) noexcept { + set_exception(std::move(e)); + } + // From a users' perspective, a result_unavailable is not valid + bool valid() const noexcept { return st != state::invalid && st != state::result_unavailable; } + bool available() const noexcept { return st == state::result || st >= state::exception_min; } + bool failed() const noexcept { return __builtin_expect(st >= state::exception_min, false); } + void check_failure() noexcept; + ~any() noexcept { } + std::exception_ptr take_exception() noexcept { + std::exception_ptr ret(std::move(ex)); + // Unfortunately in libstdc++ ~exception_ptr is defined out of line. We know that it does nothing for + // moved out values, so we omit calling it. This is critical for the code quality produced for this + // function. Without the out of line call, gcc can figure out that both sides of the if produce + // identical code and merges them.if + // We don't make any assumptions about other c++ libraries. + // There is request with gcc to define it inline: https://gcc.gnu.org/bugzilla/show_bug.cgi?id=90295 +#ifndef __GLIBCXX__ + ex.~exception_ptr(); +#endif + st = state::invalid; + return ret; + } + void move_it(any&& x) noexcept { +#ifdef __GLIBCXX__ + // Unfortunally gcc cannot fully optimize the regular + // implementation: + // https://gcc.gnu.org/bugzilla/show_bug.cgi?id=95014 + // Given what we know about the libstdc++ implementation + // (see the comment in take_exception), we can just + // memmove and zero x. We use memmove to guarantee + // vaild results if &x == this. + memmove(static_cast<void*>(this), &x, sizeof(any)); + x.st = state::invalid; +#else + if (x.st < state::exception_min) { + st = x.st; + x.st = state::invalid; + } else { + new (&ex) std::exception_ptr(x.take_exception()); + } +#endif + } + any(any&& x) noexcept { + move_it(std::move(x)); + } + any& operator=(any&& x) noexcept { + check_failure(); + // If this is a self move assignment, check_failure + // guarantees that we don't have an exception and calling + // move_it is safe. + move_it(std::move(x)); + return *this; + } + bool has_result() const noexcept { + return st == state::result || st == state::result_unavailable; + } + state st; + std::exception_ptr ex; + } _u; + + future_state_base() noexcept = default; + future_state_base(state st) noexcept : _u(st) { } + future_state_base(std::exception_ptr&& ex) noexcept : _u(std::move(ex)) { } + future_state_base(future_state_base&& x) noexcept : _u(std::move(x._u)) { } + + // We never need to destruct this polymorphicly, so we can make it + // protected instead of virtual. +protected: + struct current_exception_future_marker {}; + future_state_base(current_exception_future_marker) noexcept; + struct nested_exception_marker {}; + future_state_base(nested_exception_marker, future_state_base&& old) noexcept; + future_state_base(nested_exception_marker, future_state_base&& n, future_state_base&& old) noexcept; + ~future_state_base() noexcept = default; + + void rethrow_exception() &&; + void rethrow_exception() const&; + +public: + + bool valid() const noexcept { return _u.valid(); } + bool available() const noexcept { return _u.available(); } + bool failed() const noexcept { return _u.failed(); } + + void ignore() noexcept; + + void set_exception(std::exception_ptr&& ex) noexcept { + assert(_u.st == state::future); + _u.set_exception(std::move(ex)); + } + future_state_base& operator=(future_state_base&& x) noexcept = default; + void set_exception(future_state_base&& state) noexcept { + assert(_u.st == state::future); + *this = std::move(state); + } + std::exception_ptr get_exception() && noexcept { + assert(_u.st >= state::exception_min); + // Move ex out so future::~future() knows we've handled it + return _u.take_exception(); + } + const std::exception_ptr& get_exception() const& noexcept { + assert(_u.st >= state::exception_min); + return _u.ex; + } + template <typename U> + friend struct future_state; + template <typename... U> + friend future<U...> current_exception_as_future() noexcept; + template <typename SEASTAR_ELLIPSIS U> + friend class future; + template <typename T> + friend struct futurize; +}; + +void report_failed_future(future_state_base::any&& state) noexcept; + +inline void future_state_base::any::check_failure() noexcept { + if (failed()) { + report_failed_future(std::move(*this)); + } +} + +struct ready_future_marker {}; +struct exception_future_marker {}; +struct future_for_get_promise_marker {}; + +/// \cond internal +template <typename T> +struct future_state : public future_state_base, private internal::uninitialized_wrapper<T> { + static constexpr bool copy_noexcept = std::is_nothrow_copy_constructible<T>::value; +#if SEASTAR_API_LEVEL < 5 + static constexpr bool has_trivial_move_and_destroy = internal::is_tuple_effectively_trivially_move_constructible_and_destructible<T>; +#else + static constexpr bool has_trivial_move_and_destroy = internal::is_trivially_move_constructible_and_destructible<T>::value; +#endif + static_assert(std::is_nothrow_move_constructible<T>::value, + "Types must be no-throw move constructible"); + static_assert(std::is_nothrow_destructible<T>::value, + "Types must be no-throw destructible"); + future_state() noexcept = default; + void move_it(future_state&& x) noexcept { + if constexpr (has_trivial_move_and_destroy) { +#pragma GCC diagnostic push + // This function may copy uninitialized memory, such as when + // creating an uninitialized promise and calling get_future() + // on it. Gcc 12 started to catch some simple cases of this + // at compile time, so we need to tell it that it's fine. +#pragma GCC diagnostic ignored "-Wuninitialized" + memmove(reinterpret_cast<char*>(&this->uninitialized_get()), + &x.uninitialized_get(), + internal::used_size<internal::maybe_wrap_ref<T>>::value); +#pragma GCC diagnostic pop + } else if (_u.has_result()) { + this->uninitialized_set(std::move(x.uninitialized_get())); + std::destroy_at(&x.uninitialized_get()); + } + } + + [[gnu::always_inline]] + future_state(future_state&& x) noexcept : future_state_base(std::move(x)) { + move_it(std::move(x)); + } + + void clear() noexcept { + if (_u.has_result()) { + std::destroy_at(&this->uninitialized_get()); + } else { + _u.check_failure(); + } + } + __attribute__((always_inline)) + ~future_state() noexcept { + clear(); + } + future_state& operator=(future_state&& x) noexcept { + clear(); + future_state_base::operator=(std::move(x)); + // If &x == this, _u.st is now state::invalid and so it is + // safe to call move_it. + move_it(std::move(x)); + return *this; + } + template <typename... A> + future_state(ready_future_marker, A&&... a) noexcept : future_state_base(state::result) { + try { + this->uninitialized_set(std::forward<A>(a)...); + } catch (...) { + new (this) future_state(current_exception_future_marker()); + } + } + template <typename... A> + void set(A&&... a) noexcept { + assert(_u.st == state::future); + new (this) future_state(ready_future_marker(), std::forward<A>(a)...); + } + future_state(exception_future_marker, std::exception_ptr&& ex) noexcept : future_state_base(std::move(ex)) { } + future_state(exception_future_marker, future_state_base&& state) noexcept : future_state_base(std::move(state)) { } + future_state(current_exception_future_marker m) noexcept : future_state_base(m) { } + future_state(nested_exception_marker m, future_state_base&& old) noexcept : future_state_base(m, std::move(old)) { } + future_state(nested_exception_marker m, future_state_base&& n, future_state_base&& old) noexcept : future_state_base(m, std::move(n), std::move(old)) { } + T&& get_value() && noexcept { + assert(_u.st == state::result); + return static_cast<T&&>(this->uninitialized_get()); + } + T&& take_value() && noexcept { + assert(_u.st == state::result); + _u.st = state::result_unavailable; + return static_cast<T&&>(this->uninitialized_get()); + } + template<typename U = T> + const std::enable_if_t<std::is_copy_constructible<U>::value, U>& get_value() const& noexcept(copy_noexcept) { + assert(_u.st == state::result); + return this->uninitialized_get(); + } + T&& take() && { + assert(available()); + if (_u.st >= state::exception_min) { + std::move(*this).rethrow_exception(); + } + _u.st = state::result_unavailable; + return static_cast<T&&>(this->uninitialized_get()); + } + T&& get() && { + assert(available()); + if (_u.st >= state::exception_min) { + std::move(*this).rethrow_exception(); + } + return static_cast<T&&>(this->uninitialized_get()); + } + const T& get() const& { + assert(available()); + if (_u.st >= state::exception_min) { + rethrow_exception(); + } + return this->uninitialized_get(); + } + using get0_return_type = typename internal::get0_return_type<internal::future_tuple_type_t<T>>::type; + static get0_return_type get0(T&& x) { + return internal::get0_return_type<T>::get0(std::move(x)); + } + + get0_return_type get0() { +#if SEASTAR_API_LEVEL < 5 + return get0(std::move(*this).get()); +#else + return std::move(*this).get(); +#endif + } +}; + +#if SEASTAR_API_LEVEL < 6 +template <typename... T> +#else +template <typename T = void> +#endif +class continuation_base : public task { +protected: + using future_state = seastar::future_state<internal::future_stored_type_t<T SEASTAR_ELLIPSIS>>; + future_state _state; + using future_type = future<T SEASTAR_ELLIPSIS>; + using promise_type = promise<T SEASTAR_ELLIPSIS>; +public: + continuation_base() noexcept = default; + void set_state(future_state&& state) noexcept { + _state = std::move(state); + } + // This override of waiting_task() is needed here because there are cases + // when backtrace is obtained from the destructor of this class and objects + // of derived classes are already destroyed at that time. If we didn't + // have this override we would get a "pure virtual function call" exception. + virtual task* waiting_task() noexcept override { return nullptr; } + friend class internal::promise_base_with_type<T SEASTAR_ELLIPSIS>; + friend class promise<T SEASTAR_ELLIPSIS>; + friend class future<T SEASTAR_ELLIPSIS>; +}; + +// Given a future type, find the corresponding continuation_base. +template <typename Future> +struct continuation_base_from_future; + +template <typename... T> +struct continuation_base_from_future<future<T...>> { + using type = continuation_base<T...>; +}; + +template <typename Future> +using continuation_base_from_future_t = typename continuation_base_from_future<Future>::type; + +#if SEASTAR_API_LEVEL < 6 +template <typename Promise, typename... T> +#else +template <typename Promise, typename T = void> +#endif +class continuation_base_with_promise : public continuation_base<T SEASTAR_ELLIPSIS> { + friend class internal::promise_base_with_type<T SEASTAR_ELLIPSIS>; +protected: + continuation_base_with_promise(Promise&& pr) noexcept : _pr(std::move(pr)) { + task::make_backtrace(); + } + virtual task* waiting_task() noexcept override; + Promise _pr; +}; + +#if SEASTAR_API_LEVEL < 6 +template <typename Promise, typename Func, typename Wrapper, typename... T> +#else +template <typename Promise, typename Func, typename Wrapper, typename T = void> +#endif +struct continuation final : continuation_base_with_promise<Promise, T SEASTAR_ELLIPSIS> { + // Func is the original function passed to then/then_wrapped. The + // Wrapper is a helper function that implements the specific logic + // needed by then/then_wrapped. We call the wrapper passing it the + // original function, promise and state. + // Note that if Func's move constructor throws, this will call + // std::unexpected. We could try to require Func to be nothrow + // move constructible, but that will cause a lot of churn. Since + // we can't support a failure to create a continuation, calling + // std::unexpected as close to the failure as possible is the best + // we can do. + continuation(Promise&& pr, Func&& func, Wrapper&& wrapper) noexcept + : continuation_base_with_promise<Promise, T SEASTAR_ELLIPSIS>(std::move(pr)) + , _func(std::move(func)) + , _wrapper(std::move(wrapper)) {} + virtual void run_and_dispose() noexcept override { + try { + _wrapper(std::move(this->_pr), _func, std::move(this->_state)); + } catch (...) { + this->_pr.set_to_current_exception(); + } + delete this; + } + Func _func; + [[no_unique_address]] Wrapper _wrapper; +}; + +#if SEASTAR_API_LEVEL < 4 + +// This is an internal future<> payload for seastar::when_all_succeed(). It is used +// to return a variadic future (when two or more of its input futures were non-void), +// but with variadic futures deprecated and soon gone this is no longer possible. +// +// Instead, we use this tuple type, and future::then() knows to unpack it. +// +// The whole thing is temporary for a transition period. +template <typename... T> +struct when_all_succeed_tuple : std::tuple<T...> { + using std::tuple<T...>::tuple; + when_all_succeed_tuple(std::tuple<T...>&& t) + noexcept(std::is_nothrow_move_constructible<std::tuple<T...>>::value) + : std::tuple<T...>(std::move(t)) {} +}; + +#endif + +namespace internal { + +template <typename... T> +future<T...> make_exception_future(future_state_base&& state) noexcept; + +template <typename... T, typename U> +void set_callback(future<T...>&& fut, U* callback) noexcept; + +class future_base; + +class promise_base { +protected: + enum class urgent { no, yes }; + future_base* _future = nullptr; + + // This points to the future_state that is currently being + // used. See comment above the future_state struct definition for + // details. + future_state_base* _state; + + task* _task = nullptr; + + promise_base(const promise_base&) = delete; + promise_base(future_state_base* state) noexcept : _state(state) {} + promise_base(future_base* future, future_state_base* state) noexcept; + void move_it(promise_base&& x) noexcept; + promise_base(promise_base&& x) noexcept; + + void clear() noexcept; + + // We never need to destruct this polymorphicly, so we can make it + // protected instead of virtual + ~promise_base() noexcept { + clear(); + } + + void operator=(const promise_base&) = delete; + promise_base& operator=(promise_base&& x) noexcept; + + template<urgent Urgent> + void make_ready() noexcept; + + template<typename T> + void set_exception_impl(T&& val) noexcept { + if (_state) { + _state->set_exception(std::move(val)); + make_ready<urgent::no>(); + } else { + // We get here if promise::get_future is called and the + // returned future is destroyed without creating a + // continuation. + // In older versions of seastar we would store a local + // copy of ex and warn in the promise destructor. + // Since there isn't any way for the user to clear + // the exception, we issue the warning from here. + report_failed_future(val); + } + } + + void set_exception(future_state_base&& state) noexcept { + set_exception_impl(std::move(state)); + } + + void set_exception(std::exception_ptr&& ex) noexcept { + set_exception_impl(std::move(ex)); + } + + void set_exception(const std::exception_ptr& ex) noexcept { + set_exception(std::exception_ptr(ex)); + } + + template<typename Exception> + std::enable_if_t<!std::is_same<std::remove_reference_t<Exception>, std::exception_ptr>::value, void> set_exception(Exception&& e) noexcept { + set_exception(std::make_exception_ptr(std::forward<Exception>(e))); + } + + friend class future_base; + template <typename SEASTAR_ELLIPSIS U> friend class seastar::future; + +public: + /// Set this promise to the current exception. + /// + /// This is equivalent to set_exception(std::current_exception()), + /// but expands to less code. + void set_to_current_exception() noexcept; + + /// Returns the task which is waiting for this promise to resolve, or nullptr. + task* waiting_task() const noexcept { return _task; } +}; + +/// \brief A promise with type but no local data. +/// +/// This is a promise without any local data. We use this for when the +/// future is created first, so we know the promise always has an +/// external place to point to. We cannot just use promise_base +/// because we need to know the type that is being stored. +template <typename SEASTAR_ELLIPSIS T> +class promise_base_with_type : protected internal::promise_base { +protected: + using future_state = seastar::future_state<future_stored_type_t<T SEASTAR_ELLIPSIS>>; + future_state* get_state() noexcept { + return static_cast<future_state*>(_state); + } + static constexpr bool copy_noexcept = future_state::copy_noexcept; +public: + promise_base_with_type(future_state_base* state) noexcept : promise_base(state) { } + promise_base_with_type(future<T SEASTAR_ELLIPSIS>* future) noexcept : promise_base(future, &future->_state) { } + promise_base_with_type(promise_base_with_type&& x) noexcept = default; + promise_base_with_type(const promise_base_with_type&) = delete; + promise_base_with_type& operator=(promise_base_with_type&& x) noexcept = default; + void operator=(const promise_base_with_type&) = delete; + + void set_urgent_state(future_state&& state) noexcept { + auto* ptr = get_state(); + // The state can be null if the corresponding future has been + // destroyed without producing a continuation. + if (ptr) { + // FIXME: This is a fairly expensive assert. It would be a + // good candidate for being disabled in release builds if + // we had such an assert. + assert(ptr->_u.st == future_state_base::state::future); + new (ptr) future_state(std::move(state)); + make_ready<urgent::yes>(); + } + } + + template <typename... A> + void set_value(A&&... a) noexcept { + if (auto *s = get_state()) { + s->set(std::forward<A>(a)...); + make_ready<urgent::no>(); + } + } + + /// Set this promise to the current exception. + /// + /// This is equivalent to set_exception(std::current_exception()), + /// but expands to less code. + void set_to_current_exception() noexcept { + internal::promise_base::set_to_current_exception(); + } + + /// Returns the task which is waiting for this promise to resolve, or nullptr. + using internal::promise_base::waiting_task; + +private: + + template <typename SEASTAR_ELLIPSIS U> + friend class seastar::future; + + friend future_state; +}; +} +/// \endcond + +/// \brief promise - allows a future value to be made available at a later time. +/// +/// \tparam T A list of types to be carried as the result of the associated future. +/// A list with two or more types is deprecated; use +/// \c promise<std::tuple<T...>> instead. +template <typename SEASTAR_ELLIPSIS T> +class promise : private internal::promise_base_with_type<T SEASTAR_ELLIPSIS> { + using future_state = typename internal::promise_base_with_type<T SEASTAR_ELLIPSIS>::future_state; + future_state _local_state; + +public: + /// \brief Constructs an empty \c promise. + /// + /// Creates promise with no associated future yet (see get_future()). + promise() noexcept : internal::promise_base_with_type<T SEASTAR_ELLIPSIS>(&_local_state) {} + + /// \brief Moves a \c promise object. + void move_it(promise&& x) noexcept; + promise(promise&& x) noexcept : internal::promise_base_with_type<T SEASTAR_ELLIPSIS>(std::move(x)) { + move_it(std::move(x)); + } + promise(const promise&) = delete; + promise& operator=(promise&& x) noexcept { + internal::promise_base_with_type<T SEASTAR_ELLIPSIS>::operator=(std::move(x)); + // If this is a self-move, _state is now nullptr and it is + // safe to call move_it. + move_it(std::move(x)); + return *this; + } + void operator=(const promise&) = delete; + + /// Set this promise to the current exception. + /// + /// This is equivalent to set_exception(std::current_exception()), + /// but expands to less code. + void set_to_current_exception() noexcept { + internal::promise_base::set_to_current_exception(); + } + + /// Returns the task which is waiting for this promise to resolve, or nullptr. + using internal::promise_base::waiting_task; + + /// \brief Gets the promise's associated future. + /// + /// The future and promise will be remember each other, even if either or + /// both are moved. When \c set_value() or \c set_exception() are called + /// on the promise, the future will be become ready, and if a continuation + /// was attached to the future, it will run. + future<T SEASTAR_ELLIPSIS> get_future() noexcept; + + /// \brief Sets the promises value + /// + /// Forwards the arguments and makes them available to the associated + /// future. May be called either before or after \c get_future(). + /// + /// The arguments can have either the types the promise is + /// templated with, or a corresponding std::tuple. That is, given + /// a promise<int, double>, both calls are valid: + /// + /// pr.set_value(42, 43.0); + /// pr.set_value(std::tuple<int, double>(42, 43.0)) + template <typename... A> + void set_value(A&&... a) noexcept { + internal::promise_base_with_type<T SEASTAR_ELLIPSIS>::set_value(std::forward<A>(a)...); + } + + /// \brief Marks the promise as failed + /// + /// Forwards the exception argument to the future and makes it + /// available. May be called either before or after \c get_future(). + void set_exception(std::exception_ptr&& ex) noexcept { + internal::promise_base::set_exception(std::move(ex)); + } + + void set_exception(const std::exception_ptr& ex) noexcept { + internal::promise_base::set_exception(ex); + } + + /// \brief Marks the promise as failed + /// + /// Forwards the exception argument to the future and makes it + /// available. May be called either before or after \c get_future(). + template<typename Exception> + std::enable_if_t<!std::is_same<std::remove_reference_t<Exception>, std::exception_ptr>::value, void> set_exception(Exception&& e) noexcept { + internal::promise_base::set_exception(std::forward<Exception>(e)); + } + + using internal::promise_base_with_type<T SEASTAR_ELLIPSIS>::set_urgent_state; + + template <typename SEASTAR_ELLIPSIS U> + friend class future; +}; + +#if SEASTAR_API_LEVEL < 6 +/// \brief Specialization of \c promise<void> +/// +/// This is an alias for \c promise<>, for generic programming purposes. +/// For example, You may have a \c promise<T> where \c T can legally be +/// \c void. +template<> +class promise<void> : public promise<> {}; +#endif + +/// @} + +/// \addtogroup future-util +/// @{ + + +/// \brief Check whether a type is a future +/// +/// This is a type trait evaluating to \c true if the given type is a +/// future. +/// +template <typename... T> struct is_future : std::false_type {}; + +/// \cond internal +/// \addtogroup future-util +template <typename... T> struct is_future<future<T...>> : std::true_type {}; + +/// \endcond + + +/// \brief Converts a type to a future type, if it isn't already. +/// +/// \return Result in member type 'type'. +template <typename T> +struct futurize; + +SEASTAR_CONCEPT( + +template <typename T> +concept Future = is_future<T>::value; + +template <typename Func, typename... T> +concept CanInvoke = std::invocable<Func, T...>; + +// Deprecated alias +template <typename Func, typename... T> +concept CanApply = CanInvoke<Func, T...>; + +template <typename Func, typename... T> +concept CanApplyTuple + = sizeof...(T) == 1 + && requires (Func func, std::tuple<T...> wrapped_val) { + { std::apply(func, std::get<0>(std::move(wrapped_val))) }; + }; + +template <typename Func, typename Return, typename... T> +concept InvokeReturns = requires (Func f, T... args) { + { f(std::forward<T>(args)...) } -> std::same_as<Return>; +}; + +// Deprecated alias +template <typename Func, typename Return, typename... T> +concept ApplyReturns = InvokeReturns<Func, Return, T...>; + +template <typename Func, typename... T> +concept InvokeReturnsAnyFuture = Future<std::invoke_result_t<Func, T...>>; + +// Deprecated alias +template <typename Func, typename... T> +concept ApplyReturnsAnyFuture = InvokeReturnsAnyFuture<Func, T...>; + +) + +/// \endcond + +// Converts a type to a future type, if it isn't already. +template <typename T> +using futurize_t = typename futurize<T>::type; + +/// @} + +template<typename Func, typename... Args> +auto futurize_invoke(Func&& func, Args&&... args) noexcept; + +template<typename Func, typename... Args> +auto futurize_apply(Func&& func, std::tuple<Args...>&& args) noexcept; + +/// \addtogroup future-module +/// @{ +namespace internal { +class future_base { +protected: + promise_base* _promise; + future_base() noexcept : _promise(nullptr) {} + future_base(promise_base* promise, future_state_base* state) noexcept : _promise(promise) { + _promise->_future = this; + _promise->_state = state; + } + + void move_it(future_base&& x, future_state_base* state) noexcept { + _promise = x._promise; + if (auto* p = _promise) { + x.detach_promise(); + p->_future = this; + p->_state = state; + } + } + + future_base(future_base&& x, future_state_base* state) noexcept { + move_it(std::move(x), state); + } + + void clear() noexcept { + if (_promise) { + detach_promise(); + } + } + + ~future_base() noexcept { + clear(); + } + + promise_base* detach_promise() noexcept { + _promise->_state = nullptr; + _promise->_future = nullptr; + return std::exchange(_promise, nullptr); + } + + void schedule(task* tws, future_state_base* state) noexcept { + promise_base* p = detach_promise(); + p->_state = state; + p->_task = tws; + } + + void do_wait() noexcept; + +#ifdef SEASTAR_COROUTINES_ENABLED + void set_coroutine(task& coroutine) noexcept; +#endif + + friend class promise_base; +}; + +template <typename Func, typename... T> +struct future_result { + using type = std::invoke_result_t<Func, T...>; + using future_type = futurize_t<type>; + using func_type = future_type (T&&...); +}; + +template <typename Func> +struct future_result<Func, void> { + using type = std::invoke_result_t<Func>; + using future_type = futurize_t<type>; + using func_type = future_type (); +}; + +template <typename Func, typename SEASTAR_ELLIPSIS T> +using future_result_t = typename future_result<Func, T SEASTAR_ELLIPSIS>::type; + +template <typename Func, typename T> +auto future_invoke(Func&& func, T&& v) { + if constexpr (std::is_same_v<T, monostate>) { + return std::invoke(std::forward<Func>(func)); + } else { + return std::invoke(std::forward<Func>(func), std::forward<T>(v)); + } +} + +// This is a customization point for future::then()'s implementation. +// It behaves differently when the future value type is a when_all_succeed_tuple +// instantiation, indicating we need to unpack the tuple into multiple lambda +// arguments. +template <typename Future> +struct call_then_impl; + +// Generic case - the input is not a future<when_all_succeed_tuple<...>>, so +// we just forward everything to future::then_impl. +template <typename... T> +struct call_then_impl<future<T...>> { + template <typename Func> + using result_type = typename future_result<Func, T...>::future_type; + + template <typename Func> + using func_type = typename future_result<Func, T...>::func_type; + + template <typename Func> + static result_type<Func> run(future<T...>& fut, Func&& func) noexcept { + return fut.then_impl(std::forward<Func>(func)); + } +}; + +#if SEASTAR_API_LEVEL < 4 + +// Special case: we unpack the tuple before calling the function +template <typename... T> +struct call_then_impl<future<when_all_succeed_tuple<T...>>> { + template <typename Func> + using result_type = futurize_t<std::invoke_result_t<Func, T&&...>>; + + template <typename Func> + using func_type = result_type<Func> (T&&...); + + using was_tuple = when_all_succeed_tuple<T...>; + using std_tuple = std::tuple<T...>; + + template <typename Func> + static auto run(future<was_tuple>& fut, Func&& func) noexcept { + // constructing func in the lambda can throw, but there's nothing we can do + // about it, similar to #84. + return fut.then_impl([func = std::forward<Func>(func)] (was_tuple&& t) mutable { + return std::apply(func, static_cast<std_tuple&&>(std::move(t))); + }); + } +}; + +#endif + +template <typename Func, typename... Args> +using call_then_impl_result_type = typename call_then_impl<future<Args...>>::template result_type<Func>; + +SEASTAR_CONCEPT( +template <typename Func, typename... Args> +concept CanInvokeWhenAllSucceed = requires { + typename call_then_impl_result_type<Func, Args...>; +}; +) + +template <typename Func, typename... T> +struct result_of_apply { + // no "type" member if not a function call signature or not a tuple +}; + +template <typename Func, typename... T> +struct result_of_apply<Func, std::tuple<T...>> : std::invoke_result<Func, T...> { + // Let std::invoke_result_t determine the result if the input is a tuple +}; + +template <typename Func, typename... T> +using result_of_apply_t = typename result_of_apply<Func, T...>::type; + +} + +template <typename Promise, typename SEASTAR_ELLIPSIS T> +task* continuation_base_with_promise<Promise, T SEASTAR_ELLIPSIS>::waiting_task() noexcept { + return _pr.waiting_task(); +} + +/// \brief A representation of a possibly not-yet-computed value. +/// +/// A \c future represents a value that has not yet been computed +/// (an asynchronous computation). It can be in one of several +/// states: +/// - unavailable: the computation has not been completed yet +/// - value: the computation has been completed successfully and a +/// value is available. +/// - failed: the computation completed with an exception. +/// +/// methods in \c future allow querying the state and, most importantly, +/// scheduling a \c continuation to be executed when the future becomes +/// available. Only one such continuation may be scheduled. +/// +/// A \ref future should not be discarded before it is waited upon and +/// its result is extracted. Discarding a \ref future means that the +/// computed value becomes inaccessible, but more importantly, any +/// exceptions raised from the computation will disappear unchecked as +/// well. Another very important consequence is potentially unbounded +/// resource consumption due to the launcher of the deserted +/// continuation not being able track the amount of in-progress +/// continuations, nor their individual resource consumption. +/// To prevent accidental discarding of futures, \ref future is +/// declared `[[nodiscard]]` if the compiler supports it. Also, when a +/// discarded \ref future resolves with an error a warning is logged +/// (at runtime). +/// That said there can be legitimate cases where a \ref future is +/// discarded. The most prominent example is launching a new +/// [fiber](\ref fiber-module), or in other words, moving a continuation +/// chain to the background (off the current [fiber](\ref fiber-module)). +/// Even if a \ref future is discarded purposefully, it is still strongly +/// advisable to wait on it indirectly (via a \ref gate or +/// \ref semaphore), control their concurrency, their resource consumption +/// and handle any errors raised from them. +/// +/// \tparam T A list of types to be carried as the result of the future, +/// similar to \c std::tuple<T...>. An empty list (\c future<>) +/// means that there is no result, and an available future only +/// contains a success/failure indication (and in the case of a +/// failure, an exception). +/// A list with two or more types is deprecated; use +/// \c future<std::tuple<T...>> instead. +template <typename SEASTAR_ELLIPSIS T> +class SEASTAR_NODISCARD future : private internal::future_base { + using future_state = seastar::future_state<internal::future_stored_type_t<T SEASTAR_ELLIPSIS>>; + future_state _state; + static constexpr bool copy_noexcept = future_state::copy_noexcept; + using call_then_impl = internal::call_then_impl<future>; + +private: + // This constructor creates a future that is not ready but has no + // associated promise yet. The use case is to have a less flexible + // but more efficient future/promise pair where we know that + // promise::set_value cannot possibly be called without a matching + // future and so that promise doesn't need to store a + // future_state. + future(future_for_get_promise_marker) noexcept { } + + future(promise<T SEASTAR_ELLIPSIS>* pr) noexcept : future_base(pr, &_state), _state(std::move(pr->_local_state)) { } + template <typename... A> + future(ready_future_marker m, A&&... a) noexcept : _state(m, std::forward<A>(a)...) { } + future(future_state_base::current_exception_future_marker m) noexcept : _state(m) {} + future(future_state_base::nested_exception_marker m, future_state_base&& old) noexcept : _state(m, std::move(old)) {} + future(future_state_base::nested_exception_marker m, future_state_base&& n, future_state_base&& old) noexcept : _state(m, std::move(n), std::move(old)) {} + future(exception_future_marker m, std::exception_ptr&& ex) noexcept : _state(m, std::move(ex)) { } + future(exception_future_marker m, future_state_base&& state) noexcept : _state(m, std::move(state)) { } + [[gnu::always_inline]] + explicit future(future_state&& state) noexcept + : _state(std::move(state)) { + } + internal::promise_base_with_type<T SEASTAR_ELLIPSIS> get_promise() noexcept { + assert(!_promise); + return internal::promise_base_with_type<T SEASTAR_ELLIPSIS>(this); + } + internal::promise_base_with_type<T SEASTAR_ELLIPSIS>* detach_promise() noexcept { + return static_cast<internal::promise_base_with_type<T SEASTAR_ELLIPSIS>*>(future_base::detach_promise()); + } + void schedule(continuation_base<T SEASTAR_ELLIPSIS>* tws) noexcept { + future_base::schedule(tws, &tws->_state); + } + template <typename Pr, typename Func, typename Wrapper> + void schedule(Pr&& pr, Func&& func, Wrapper&& wrapper) noexcept { + // If this new throws a std::bad_alloc there is nothing that + // can be done about it. The corresponding future is not ready + // and we cannot break the chain. Since this function is + // noexcept, it will call std::terminate if new throws. + memory::scoped_critical_alloc_section _; + auto tws = new continuation<Pr, Func, Wrapper, T SEASTAR_ELLIPSIS>(std::move(pr), std::move(func), std::move(wrapper)); + // In a debug build we schedule ready futures, but not in + // other build modes. +#ifdef SEASTAR_DEBUG + if (_state.available()) { + tws->set_state(get_available_state_ref()); + ::seastar::schedule(tws); + return; + } +#endif + schedule(tws); + _state._u.st = future_state_base::state::invalid; + } + + [[gnu::always_inline]] + future_state&& get_available_state_ref() noexcept { + if (_promise) { + detach_promise(); + } + return std::move(_state); + } + + future<T SEASTAR_ELLIPSIS> rethrow_with_nested(future_state_base&& n) noexcept { + return future<T SEASTAR_ELLIPSIS>(future_state_base::nested_exception_marker(), std::move(n), std::move(_state)); + } + + future<T SEASTAR_ELLIPSIS> rethrow_with_nested() noexcept { + return future<T SEASTAR_ELLIPSIS>(future_state_base::nested_exception_marker(), std::move(_state)); + } + + template<typename... U> + friend class shared_future; +public: + /// \brief The data type carried by the future. + using value_type = internal::future_stored_type_t<T SEASTAR_ELLIPSIS>; + using tuple_type = internal::future_tuple_type_t<value_type>; + /// \brief The data type carried by the future. + using promise_type = promise<T SEASTAR_ELLIPSIS>; + /// \brief Moves the future into a new object. + [[gnu::always_inline]] + future(future&& x) noexcept : future_base(std::move(x), &_state), _state(std::move(x._state)) { } + future(const future&) = delete; + future& operator=(future&& x) noexcept { + clear(); + move_it(std::move(x), &_state); + _state = std::move(x._state); + return *this; + } + void operator=(const future&) = delete; + /// \brief gets the value returned by the computation + /// + /// Requires that the future be available. If the value + /// was computed successfully, it is returned (as an + /// \c std::tuple). Otherwise, an exception is thrown. + /// + /// If get() is called in a \ref seastar::thread context, + /// then it need not be available; instead, the thread will + /// be paused until the future becomes available. + [[gnu::always_inline]] + value_type&& get() { + wait(); + return get_available_state_ref().take(); + } + + [[gnu::always_inline]] + std::exception_ptr get_exception() noexcept { + return get_available_state_ref().get_exception(); + } + + /// Gets the value returned by the computation. + /// + /// Similar to \ref get(), but instead of returning a + /// tuple, returns the first value of the tuple. This is + /// useful for the common case of a \c future<T> with exactly + /// one type parameter. + /// + /// Equivalent to: \c std::get<0>(f.get()). + using get0_return_type = typename future_state::get0_return_type; + get0_return_type get0() { +#if SEASTAR_API_LEVEL < 5 + return future_state::get0(get()); +#else + return (get0_return_type)get(); +#endif + } + + /// Wait for the future to be available (in a seastar::thread) + /// + /// When called from a seastar::thread, this function blocks the + /// thread until the future is availble. Other threads and + /// continuations continue to execute; only the thread is blocked. + void wait() noexcept { + if (_state.available()) { + return; + } + do_wait(); + } + + /// \brief Checks whether the future is available. + /// + /// \return \c true if the future has a value, or has failed. + [[gnu::always_inline]] + bool available() const noexcept { + return _state.available(); + } + + /// \brief Checks whether the future has failed. + /// + /// \return \c true if the future is availble and has failed. + [[gnu::always_inline]] + bool failed() const noexcept { + return _state.failed(); + } + + /// \brief Schedule a block of code to run when the future is ready. + /// + /// Schedules a function (often a lambda) to run when the future becomes + /// available. The function is called with the result of this future's + /// computation as parameters. The return value of the function becomes + /// the return value of then(), itself as a future; this allows then() + /// calls to be chained. + /// + /// If the future failed, the function is not called, and the exception + /// is propagated into the return value of then(). + /// + /// \param func - function to be called when the future becomes available, + /// unless it has failed. + /// \return a \c future representing the return value of \c func, applied + /// to the eventual value of this future. + template <typename Func, typename Result = futurize_t<typename call_then_impl::template result_type<Func>>> + SEASTAR_CONCEPT( requires std::invocable<Func, T SEASTAR_ELLIPSIS> || internal::CanInvokeWhenAllSucceed<Func, T SEASTAR_ELLIPSIS>) + Result + then(Func&& func) noexcept { + // The implementation of then() is customized via the call_then_impl helper + // template, in order to special case the results of when_all_succeed(). + // when_all_succeed() used to return a variadic future, which is deprecated, so + // now it returns a when_all_succeed_tuple, which we intercept in call_then_impl, + // and treat it as a variadic future. +#ifndef SEASTAR_TYPE_ERASE_MORE + return call_then_impl::run(*this, std::move(func)); +#else + using func_type = typename call_then_impl::template func_type<Func>; + noncopyable_function<func_type> ncf; + { + memory::scoped_critical_alloc_section _; + ncf = noncopyable_function<func_type>([func = std::forward<Func>(func)](auto&&... args) mutable { + return futurize_invoke(func, std::forward<decltype(args)>(args)...); + }); + } + return call_then_impl::run(*this, std::move(ncf)); +#endif + } + + /// \brief Schedule a block of code to run when the future is ready, unpacking tuples. + /// + /// Schedules a function (often a lambda) to run when the future becomes + /// available. The function is called with the result of this future's + /// computation as parameters. The return value of the function becomes + /// the return value of then(), itself as a future; this allows then() + /// calls to be chained. + /// + /// This member function is only available when the payload is std::tuple; + /// The tuple elements are passed as individual arguments to `func`, which + /// must have the same arity as the tuple. + /// + /// If the future failed, the function is not called, and the exception + /// is propagated into the return value of then(). + /// + /// \param func - function to be called when the future becomes available, + /// unless it has failed. + /// \return a \c future representing the return value of \c func, applied + /// to the eventual value of this future. + template <typename Func, typename Result = futurize_t<internal::result_of_apply_t<Func, T SEASTAR_ELLIPSIS>>> + SEASTAR_CONCEPT( requires ::seastar::CanApplyTuple<Func, T SEASTAR_ELLIPSIS>) + Result + then_unpack(Func&& func) noexcept { + return then([func = std::forward<Func>(func)] (T&& SEASTAR_ELLIPSIS tuple) mutable { + // sizeof...(tuple) is required to be 1 + return std::apply(func, std::move(tuple) SEASTAR_ELLIPSIS); + }); + } + +private: + + // Keep this simple so that Named Return Value Optimization is used. + template <typename Func, typename Result> + Result then_impl_nrvo(Func&& func) noexcept { + using futurator = futurize<internal::future_result_t<Func, T SEASTAR_ELLIPSIS>>; + typename futurator::type fut(future_for_get_promise_marker{}); + using pr_type = decltype(fut.get_promise()); + schedule(fut.get_promise(), std::move(func), [](pr_type&& pr, Func& func, future_state&& state) { + if (state.failed()) { + pr.set_exception(static_cast<future_state_base&&>(std::move(state))); + } else { + futurator::satisfy_with_result_of(std::move(pr), [&func, &state] { +#if SEASTAR_API_LEVEL < 5 + return std::apply(func, std::move(state).get_value()); +#else + // clang thinks that "state" is not used, below, for future<>. + // Make it think it is used to avoid an unused-lambda-capture warning. + (void)state; + return internal::future_invoke(func, std::move(state).get_value()); +#endif + }); + } + }); + return fut; + } + + template <typename Func, typename Result = futurize_t<internal::future_result_t<Func, T SEASTAR_ELLIPSIS>>> + Result + then_impl(Func&& func) noexcept { +#ifndef SEASTAR_DEBUG + using futurator = futurize<internal::future_result_t<Func, T SEASTAR_ELLIPSIS>>; + if (failed()) { + return futurator::make_exception_future(static_cast<future_state_base&&>(get_available_state_ref())); + } else if (available()) { +#if SEASTAR_API_LEVEL < 5 + return futurator::apply(std::forward<Func>(func), get_available_state_ref().take_value()); +#else + return futurator::invoke(std::forward<Func>(func), get_available_state_ref().take_value()); +#endif + } +#endif + return then_impl_nrvo<Func, Result>(std::forward<Func>(func)); + } + +public: + /// \brief Schedule a block of code to run when the future is ready, allowing + /// for exception handling. + /// + /// Schedules a function (often a lambda) to run when the future becomes + /// available. The function is called with the this future as a parameter; + /// it will be in an available state. The return value of the function becomes + /// the return value of then_wrapped(), itself as a future; this allows + /// then_wrapped() calls to be chained. + /// + /// Unlike then(), the function will be called for both value and exceptional + /// futures. + /// + /// \param func - function to be called when the future becomes available, + /// \return a \c future representing the return value of \c func, applied + /// to the eventual value of this future. + template <typename Func, typename FuncResult = std::invoke_result_t<Func, future>> + SEASTAR_CONCEPT( requires std::invocable<Func, future> ) + futurize_t<FuncResult> + then_wrapped(Func&& func) & noexcept { + return then_wrapped_maybe_erase<false, FuncResult>(std::forward<Func>(func)); + } + + template <typename Func, typename FuncResult = std::invoke_result_t<Func, future&&>> + SEASTAR_CONCEPT( requires std::invocable<Func, future&&> ) + futurize_t<FuncResult> + then_wrapped(Func&& func) && noexcept { + return then_wrapped_maybe_erase<true, FuncResult>(std::forward<Func>(func)); + } + +private: + + template <bool AsSelf, typename FuncResult, typename Func> + futurize_t<FuncResult> + then_wrapped_maybe_erase(Func&& func) noexcept { +#ifndef SEASTAR_TYPE_ERASE_MORE + return then_wrapped_common<AsSelf, FuncResult>(std::forward<Func>(func)); +#else + using futurator = futurize<FuncResult>; + using WrapFuncResult = typename futurator::type; + noncopyable_function<WrapFuncResult (future&&)> ncf; + { + memory::scoped_critical_alloc_section _; + ncf = noncopyable_function<WrapFuncResult(future &&)>([func = std::forward<Func>(func)](future&& f) mutable { + return futurator::invoke(func, std::move(f)); + }); + } + return then_wrapped_common<AsSelf, WrapFuncResult>(std::move(ncf)); +#endif + } + + // Keep this simple so that Named Return Value Optimization is used. + template <typename FuncResult, typename Func> + futurize_t<FuncResult> + then_wrapped_nrvo(Func&& func) noexcept { + using futurator = futurize<FuncResult>; + typename futurator::type fut(future_for_get_promise_marker{}); + using pr_type = decltype(fut.get_promise()); + schedule(fut.get_promise(), std::move(func), [](pr_type&& pr, Func& func, future_state&& state) { + futurator::satisfy_with_result_of(std::move(pr), [&func, &state] { + return func(future(std::move(state))); + }); + }); + return fut; + } + + + template <bool AsSelf, typename FuncResult, typename Func> + futurize_t<FuncResult> + then_wrapped_common(Func&& func) noexcept { +#ifndef SEASTAR_DEBUG + using futurator = futurize<FuncResult>; + if (available()) { + if constexpr (AsSelf) { + if (_promise) { + detach_promise(); + } + return futurator::invoke(std::forward<Func>(func), std::move(*this)); + } else { + return futurator::invoke(std::forward<Func>(func), future(get_available_state_ref())); + } + } +#endif + return then_wrapped_nrvo<FuncResult, Func>(std::forward<Func>(func)); + } + + void forward_to(internal::promise_base_with_type<T SEASTAR_ELLIPSIS>&& pr) noexcept { + if (_state.available()) { + pr.set_urgent_state(std::move(_state)); + } else { + *detach_promise() = std::move(pr); + } + } + +public: + /// \brief Satisfy some \ref promise object with this future as a result. + /// + /// Arranges so that when this future is resolve, it will be used to + /// satisfy an unrelated promise. This is similar to scheduling a + /// continuation that moves the result of this future into the promise + /// (using promise::set_value() or promise::set_exception(), except + /// that it is more efficient. + /// + /// \param pr a promise that will be fulfilled with the results of this + /// future. + void forward_to(promise<T SEASTAR_ELLIPSIS>&& pr) noexcept { + if (_state.available()) { + pr.set_urgent_state(std::move(_state)); + } else if (&pr._local_state != pr._state) { + // The only case when _state points to _local_state is + // when get_future was never called. Given that pr will + // soon be destroyed, we know get_future will never be + // called and we can just ignore this request. + *detach_promise() = std::move(pr); + } + } + + + + /** + * Finally continuation for statements that require waiting for the result. + * I.e. you need to "finally" call a function that returns a possibly + * unavailable future. The returned future will be "waited for", any + * exception generated will be propagated, but the return value is ignored. + * I.e. the original return value (the future upon which you are making this + * call) will be preserved. + * + * If the original return value or the callback return value is an + * exceptional future it will be propagated. + * + * If both of them are exceptional - the std::nested_exception exception + * with the callback exception on top and the original future exception + * nested will be propagated. + */ + template <typename Func> + SEASTAR_CONCEPT( requires std::invocable<Func> ) + future<T SEASTAR_ELLIPSIS> finally(Func&& func) noexcept { + return then_wrapped(finally_body<Func, is_future<std::invoke_result_t<Func>>::value>(std::forward<Func>(func))); + } + + + template <typename Func, bool FuncReturnsFuture> + struct finally_body; + + template <typename Func> + struct finally_body<Func, true> { + Func _func; + + finally_body(Func&& func) noexcept : _func(std::forward<Func>(func)) + { } + + future<T SEASTAR_ELLIPSIS> operator()(future<T SEASTAR_ELLIPSIS>&& result) noexcept { + return futurize_invoke(_func).then_wrapped([result = std::move(result)](auto&& f_res) mutable { + if (!f_res.failed()) { + return std::move(result); + } else { + return result.rethrow_with_nested(std::move(f_res._state)); + } + }); + } + }; + + template <typename Func> + struct finally_body<Func, false> { + Func _func; + + finally_body(Func&& func) noexcept : _func(std::forward<Func>(func)) + { } + + future<T SEASTAR_ELLIPSIS> operator()(future<T SEASTAR_ELLIPSIS>&& result) noexcept { + try { + _func(); + return std::move(result); + } catch (...) { + return result.rethrow_with_nested(); + } + }; + }; + + /// \brief Terminate the program if this future fails. + /// + /// Terminates the entire program is this future resolves + /// to an exception. Use with caution. + future<> or_terminate() noexcept { + return then_wrapped([] (auto&& f) { + try { + f.get(); + } catch (...) { + engine_exit(std::current_exception()); + } + }); + } + + /// \brief Discards the value carried by this future. + /// + /// Converts the future into a no-value \c future<>, by + /// ignoring any result. Exceptions are propagated unchanged. + future<> discard_result() noexcept { + // We need the generic variadic lambda, below, because then() behaves differently + // when value_type is when_all_succeed_tuple + return then([] (auto&&...) {}); + } + + /// \brief Handle the exception carried by this future. + /// + /// When the future resolves, if it resolves with an exception, + /// handle_exception(func) replaces the exception with the value + /// returned by func. The exception is passed (as a std::exception_ptr) + /// as a parameter to func; func may return the replacement value + /// immediately (T or std::tuple<T...>) or in the future (future<T...>) + /// and is even allowed to return (or throw) its own exception. + /// + /// The idiom fut.discard_result().handle_exception(...) can be used + /// to handle an exception (if there is one) without caring about the + /// successful value; Because handle_exception() is used here on a + /// future<>, the handler function does not need to return anything. + template <typename Func> + /* Broken? + SEASTAR_CONCEPT( requires ::seastar::InvokeReturns<Func, future<T...>, std::exception_ptr> + || (sizeof...(T) == 0 && ::seastar::InvokeReturns<Func, void, std::exception_ptr>) + || (sizeof...(T) == 1 && ::seastar::InvokeReturns<Func, T..., std::exception_ptr>) + ) */ + future<T SEASTAR_ELLIPSIS> handle_exception(Func&& func) noexcept { + return then_wrapped([func = std::forward<Func>(func)] + (auto&& fut) mutable -> future<T SEASTAR_ELLIPSIS> { + if (!fut.failed()) { + return make_ready_future<T SEASTAR_ELLIPSIS>(fut.get()); + } else { + return futurize_invoke(func, fut.get_exception()); + } + }); + } + + /// \brief Handle the exception of a certain type carried by this future. + /// + /// When the future resolves, if it resolves with an exception of a type that + /// provided callback receives as a parameter, \c handle_exception_type(func) replaces + /// the exception with the value returned by func. The exception is passed (by + /// reference) as a parameter to func; func may return the replacement value + /// immediately (T or std::tuple<T...>) or in the future (future<T...>) + /// and is even allowed to return (or throw) its own exception. + /// If exception, that future holds, does not match func parameter type + /// it is propagated as is. + template <typename Func> + future<T SEASTAR_ELLIPSIS> handle_exception_type(Func&& func) noexcept { + using trait = function_traits<Func>; + static_assert(trait::arity == 1, "func can take only one parameter"); + using ex_type = typename trait::template arg<0>::type; + return then_wrapped([func = std::forward<Func>(func)] + (auto&& fut) mutable -> future<T SEASTAR_ELLIPSIS> { + try { + return make_ready_future<T SEASTAR_ELLIPSIS>(fut.get()); + } catch(ex_type& ex) { + return futurize_invoke(func, ex); + } + }); + } + + /// \brief Ignore any result hold by this future + /// + /// Ignore any result (value or exception) hold by this future. + /// Use with caution since usually ignoring exception is not what + /// you want + void ignore_ready_future() noexcept { + _state.ignore(); + } + +#ifdef SEASTAR_COROUTINES_ENABLED + using future_base::set_coroutine; +#endif +private: + void set_callback(continuation_base<T SEASTAR_ELLIPSIS>* callback) noexcept { + if (_state.available()) { + callback->set_state(get_available_state_ref()); + ::seastar::schedule(callback); + } else { + assert(_promise); + schedule(callback); + } + + } + + /// \cond internal + template <typename SEASTAR_ELLIPSIS U> + friend class future; + template <typename SEASTAR_ELLIPSIS U> + friend class promise; + template <typename U> + friend struct futurize; + template <typename SEASTAR_ELLIPSIS U> + friend class internal::promise_base_with_type; + template <typename... U, typename... A> + friend future<U...> make_ready_future(A&&... value) noexcept; + template <typename... U> + friend future<U...> make_exception_future(std::exception_ptr&& ex) noexcept; + template <typename... U, typename Exception> + friend future<U...> make_exception_future(Exception&& ex) noexcept; + template <typename... U> + friend future<U...> internal::make_exception_future(future_state_base&& state) noexcept; + template <typename... U> + friend future<U...> current_exception_as_future() noexcept; + template <typename... U, typename V> + friend void internal::set_callback(future<U...>&&, V*) noexcept; + template <typename Future> + friend struct internal::call_then_impl; + /// \endcond +}; + + +namespace internal { +template <typename T> +struct futurize_base { + /// If \c T is a future, \c T; otherwise \c future<T> + using type = future<T>; + /// The promise type associated with \c type. + using promise_type = promise<T>; + using promise_base_with_type = internal::promise_base_with_type<T>; + + /// Convert a value or a future to a future + static inline type convert(T&& value) { return make_ready_future<T>(std::move(value)); } + static inline type convert(type&& value) { return std::move(value); } + + /// Makes an exceptional future of type \ref type. + template <typename Arg> + static inline type make_exception_future(Arg&& arg) noexcept; +}; + +template <> +struct futurize_base<void> { + using type = future<>; + using promise_type = promise<>; + using promise_base_with_type = internal::promise_base_with_type<>; + + static inline type convert(type&& value) { + return std::move(value); + } + template <typename Arg> + static inline type make_exception_future(Arg&& arg) noexcept; +}; + +template <typename T> +struct futurize_base<future<T>> : public futurize_base<T> {}; + +template <> +struct futurize_base<future<>> : public futurize_base<void> {}; +} + +template <typename T> +struct futurize : public internal::futurize_base<T> { + using base = internal::futurize_base<T>; + using type = typename base::type; + using promise_type = typename base::promise_type; + using promise_base_with_type = typename base::promise_base_with_type; + /// The value tuple type associated with \c type + using value_type = typename type::value_type; + using tuple_type = typename type::tuple_type; + using base::convert; + using base::make_exception_future; + + /// Apply a function to an argument list (expressed as a tuple) + /// and return the result, as a future (if it wasn't already). + template<typename Func, typename... FuncArgs> + static inline type apply(Func&& func, std::tuple<FuncArgs...>&& args) noexcept; + + /// Invoke a function to an argument list + /// and return the result, as a future (if it wasn't already). + template<typename Func, typename... FuncArgs> + static inline type invoke(Func&& func, FuncArgs&&... args) noexcept; + + template<typename Func> + static inline type invoke(Func&& func, internal::monostate) noexcept { + return invoke(std::forward<Func>(func)); + } + + /// Deprecated alias of invoke + template<typename Func, typename... FuncArgs> + [[deprecated("Use invoke for varargs")]] + static inline type apply(Func&& func, FuncArgs&&... args) noexcept { + return invoke(std::forward<Func>(func), std::forward<FuncArgs>(args)...); + } + + static type current_exception_as_future() noexcept { + return type(future_state_base::current_exception_future_marker()); + } + + /// Convert the tuple representation into a future + static type from_tuple(tuple_type&& value) { + return type(ready_future_marker(), std::move(value)); + } + /// Convert the tuple representation into a future + static type from_tuple(const tuple_type& value) { + return type(ready_future_marker(), value); + } + +#if SEASTAR_API_LEVEL >= 5 + /// Convert the tuple representation into a future + static type from_tuple(value_type&& value) { + return type(ready_future_marker(), std::move(value)); + } + /// Convert the tuple representation into a future + static type from_tuple(const value_type& value) { + return type(ready_future_marker(), value); + } +#endif +private: + /// Forwards the result of, or exception thrown by, func() to the + /// promise. This avoids creating a future if func() doesn't + /// return one. + template<typename Func> + SEASTAR_CONCEPT( requires std::invocable<Func> ) + static void satisfy_with_result_of(promise_base_with_type&&, Func&& func); + + template <typename SEASTAR_ELLIPSIS U> + friend class future; +}; + +inline internal::promise_base::promise_base(future_base* future, future_state_base* state) noexcept + : _future(future), _state(state) { + _future->_promise = this; +} + +template <typename SEASTAR_ELLIPSIS T> +inline +future<T SEASTAR_ELLIPSIS> +promise<T SEASTAR_ELLIPSIS>::get_future() noexcept { + assert(!this->_future && this->_state && !this->_task); + return future<T SEASTAR_ELLIPSIS>(this); +} + +template <typename SEASTAR_ELLIPSIS T> +inline +void promise<T SEASTAR_ELLIPSIS>::move_it(promise&& x) noexcept { + if (this->_state == &x._local_state) { + this->_state = &_local_state; + new (&_local_state) future_state(std::move(x._local_state)); + } +} + +template <typename... T, typename... A> +inline +future<T...> make_ready_future(A&&... value) noexcept { + return future<T...>(ready_future_marker(), std::forward<A>(value)...); +} + +template <typename... T> +inline +future<T...> make_exception_future(std::exception_ptr&& ex) noexcept { + return future<T...>(exception_future_marker(), std::move(ex)); +} + +template <typename... T> +inline +future<T...> internal::make_exception_future(future_state_base&& state) noexcept { + return future<T...>(exception_future_marker(), std::move(state)); +} + +template <typename... T> +future<T...> current_exception_as_future() noexcept { + return future<T...>(future_state_base::current_exception_future_marker()); +} + +void log_exception_trace() noexcept; + +/// \brief Creates a \ref future in an available, failed state. +/// +/// Creates a \ref future object that is already resolved in a failed +/// state. This no I/O needs to be performed to perform a computation +/// (for example, because the connection is closed and we cannot read +/// from it). +template <typename... T, typename Exception> +inline +future<T...> make_exception_future(Exception&& ex) noexcept { + log_exception_trace(); + return make_exception_future<T...>(std::make_exception_ptr(std::forward<Exception>(ex))); +} + +template <typename... T, typename Exception> +future<T...> make_exception_future_with_backtrace(Exception&& ex) noexcept { + return make_exception_future<T...>(make_backtraced_exception_ptr<Exception>(std::forward<Exception>(ex))); +} + +/// @} + +/// \cond internal + +template<typename T> +template<typename Func, typename... FuncArgs> +typename futurize<T>::type futurize<T>::apply(Func&& func, std::tuple<FuncArgs...>&& args) noexcept { + try { + using ret_t = decltype(std::apply(std::forward<Func>(func), std::move(args))); + if constexpr (std::is_void_v<ret_t>) { + std::apply(std::forward<Func>(func), std::move(args)); + return make_ready_future<>(); + } else if constexpr (is_future<ret_t>::value){ + return std::apply(std::forward<Func>(func), std::move(args)); + } else { + return convert(std::apply(std::forward<Func>(func), std::move(args))); + } + } catch (...) { + return current_exception_as_future(); + } +} + +template<typename T> +template<typename Func> +SEASTAR_CONCEPT( requires std::invocable<Func> ) +void futurize<T>::satisfy_with_result_of(promise_base_with_type&& pr, Func&& func) { + using ret_t = decltype(func()); + if constexpr (std::is_void_v<ret_t>) { + func(); + pr.set_value(); + } else if constexpr (is_future<ret_t>::value) { + func().forward_to(std::move(pr)); + } else { + pr.set_value(func()); + } +} + +template<typename T> +template<typename Func, typename... FuncArgs> +typename futurize<T>::type futurize<T>::invoke(Func&& func, FuncArgs&&... args) noexcept { + try { + using ret_t = decltype(func(std::forward<FuncArgs>(args)...)); + if constexpr (std::is_void_v<ret_t>) { + func(std::forward<FuncArgs>(args)...); + return make_ready_future<>(); + } else if constexpr (is_future<ret_t>::value) { + return func(std::forward<FuncArgs>(args)...); + } else { + return convert(func(std::forward<FuncArgs>(args)...)); + } + } catch (...) { + return current_exception_as_future(); + } +} + +template <typename T> +template <typename Arg> +inline +future<T> +internal::futurize_base<T>::make_exception_future(Arg&& arg) noexcept { + using ::seastar::make_exception_future; + using ::seastar::internal::make_exception_future; + return make_exception_future<T>(std::forward<Arg>(arg)); +} + +template <typename Arg> +inline +future<> +internal::futurize_base<void>::make_exception_future(Arg&& arg) noexcept { + using ::seastar::make_exception_future; + using ::seastar::internal::make_exception_future; + return make_exception_future<>(std::forward<Arg>(arg)); +} + +template<typename Func, typename... Args> +auto futurize_invoke(Func&& func, Args&&... args) noexcept { + using futurator = futurize<std::invoke_result_t<Func, Args&&...>>; + return futurator::invoke(std::forward<Func>(func), std::forward<Args>(args)...); +} + +template<typename Func, typename... Args> +[[deprecated("Use futurize_invoke for varargs")]] +auto futurize_apply(Func&& func, Args&&... args) noexcept { + return futurize_invoke(std::forward<Func>(func), std::forward<Args>(args)...); +} + +template<typename Func, typename... Args> +auto futurize_apply(Func&& func, std::tuple<Args...>&& args) noexcept { + using futurator = futurize<std::invoke_result_t<Func, Args&&...>>; + return futurator::apply(std::forward<Func>(func), std::move(args)); +} + +namespace internal { + +template <typename... T, typename U> +inline +void set_callback(future<T...>&& fut, U* callback) noexcept { + // It would be better to use continuation_base<T...> for U, but + // then a derived class of continuation_base<T...> won't be matched + return std::move(fut).set_callback(callback); +} + +} + + +/// \endcond + +} diff --git a/src/seastar/include/seastar/core/gate.hh b/src/seastar/include/seastar/core/gate.hh new file mode 100644 index 000000000..5b436ecc7 --- /dev/null +++ b/src/seastar/include/seastar/core/gate.hh @@ -0,0 +1,314 @@ +/* + * 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 2014 Cloudius Systems + */ + +#pragma once + +#include <seastar/core/future.hh> +#include <seastar/util/std-compat.hh> +#include <exception> + +#ifdef SEASTAR_DEBUG +#define SEASTAR_GATE_HOLDER_DEBUG +#endif + +namespace seastar { + +/// \addtogroup fiber-module +/// @{ + +/// Exception thrown when a \ref gate object has been closed +/// by the \ref gate::close() method. +class gate_closed_exception : public std::exception { +public: + virtual const char* what() const noexcept override { + return "gate closed"; + } +}; + +/// Facility to stop new requests, and to tell when existing requests are done. +/// +/// When stopping a service that serves asynchronous requests, we are faced with +/// two problems: preventing new requests from coming in, and knowing when existing +/// requests have completed. The \c gate class provides a solution. +class gate { + size_t _count = 0; + std::optional<promise<>> _stopped; + +#ifdef SEASTAR_GATE_HOLDER_DEBUG + size_t _holders = 0; + + void assert_not_held_when_moved() const noexcept { + assert(!_holders && "gate moved with outstanding holders"); + } + void assert_not_held_when_destroyed() const noexcept { + assert(!_holders && "gate destroyed with outstanding holders"); + } +#else // SEASTAR_GATE_HOLDER_DEBUG + void assert_not_held_when_moved() const noexcept {} + void assert_not_held_when_destroyed() const noexcept {} +#endif // SEASTAR_GATE_HOLDER_DEBUG + +public: + gate() = default; + gate(const gate&) = delete; + gate(gate&& x) noexcept + : _count(std::exchange(x._count, 0)), _stopped(std::exchange(x._stopped, std::nullopt)) { + x.assert_not_held_when_moved(); + } + gate& operator=(gate&& x) noexcept { + if (this != &x) { + assert(!_count && "gate reassigned with outstanding requests"); + x.assert_not_held_when_moved(); + _count = std::exchange(x._count, 0); + _stopped = std::exchange(x._stopped, std::nullopt); + } + return *this; + } + ~gate() { + assert(!_count && "gate destroyed with outstanding requests"); + assert_not_held_when_destroyed(); + } + /// Tries to register an in-progress request. + /// + /// If the gate is not closed, the request is registered and the function returns `true`, + /// Otherwise the function just returns `false` and has no other effect. + bool try_enter() noexcept { + bool opened = !_stopped; + if (opened) { + ++_count; + } + return opened; + } + /// Registers an in-progress request. + /// + /// If the gate is not closed, the request is registered. Otherwise, + /// a \ref gate_closed_exception is thrown. + void enter() { + if (!try_enter()) { + throw gate_closed_exception(); + } + } + /// Unregisters an in-progress request. + /// + /// If the gate is closed, and there are no more in-progress requests, + /// the `_stopped` promise will be fulfilled. + void leave() noexcept { + --_count; + if (!_count && _stopped) { + _stopped->set_value(); + } + } + /// Potentially stop an in-progress request. + /// + /// If the gate is already closed, a \ref gate_closed_exception is thrown. + /// By using \ref enter() and \ref leave(), the program can ensure that + /// no further requests are serviced. However, long-running requests may + /// continue to run. The check() method allows such a long operation to + /// voluntarily stop itself after the gate is closed, by making calls to + /// check() in appropriate places. check() with throw an exception and + /// bail out of the long-running code if the gate is closed. + void check() { + if (_stopped) { + throw gate_closed_exception(); + } + } + /// Closes the gate. + /// + /// Future calls to \ref enter() will fail with an exception, and when + /// all current requests call \ref leave(), the returned future will be + /// made ready. + future<> close() noexcept { + assert(!_stopped && "seastar::gate::close() cannot be called more than once"); + _stopped = std::make_optional(promise<>()); + if (!_count) { + _stopped->set_value(); + } + return _stopped->get_future(); + } + + /// Returns a current number of registered in-progress requests. + size_t get_count() const noexcept { + return _count; + } + + /// Returns whether the gate is closed. + bool is_closed() const noexcept { + return bool(_stopped); + } + + /// Facility to hold a gate opened using RAII. + /// + /// A \ref gate::holder is usually obtained using \ref gate::get_holder. + /// + /// The \c gate is entered when the \ref gate::holder is constructed, + /// And the \c gate is left when the \ref gate::holder is destroyed. + /// + /// Copying the \ref gate::holder reenters the \c gate to keep an extra reference on it. + /// Moving the \ref gate::holder is supported and has no effect on the \c gate itself. + class holder { + gate* _g; + + public: + /// Construct a default \ref holder, referencing no \ref gate. + /// Never throws. + holder() noexcept : _g(nullptr) { } + + /// Construct a \ref holder by entering the \c gate. + /// May throw \ref gate_closed_exception if the gate is already closed. + explicit holder(gate& g) : _g(&g) { + _g->enter(); + debug_hold_gate(); + } + + /// Construct a \ref holder by copying another \c holder. + /// Copying a holder never throws: The original holder has already entered the gate, + /// so even if later the gate was \ref close "close()d", the copy of the holder is also allowed to enter too. + /// Note that the fiber waiting for the close(), which until now was waiting for the one holder to leave, + /// will now wait for both copies to leave. + holder(const holder& x) noexcept : _g(x._g) { + if (_g) { + _g->_count++; + debug_hold_gate(); + } + } + + /// Construct a \ref holder by moving another \c holder. + /// The referenced \ref gate is unaffected, and so the + /// move-constructor must never throw. + holder(holder&& x) noexcept : _g(std::exchange(x._g, nullptr)) { } + + /// Destroy a \ref holder and leave the referenced \ref gate. + ~holder() { + release(); + } + + /// Copy-assign another \ref holder. + /// \ref leave "Leave()" the current \ref gate before assigning the other one, if they are different. + /// Copying a holder never throws: The original holder has already entered the gate, + /// so even if later the gate was \ref close "close()d", the copy of the holder is also allowed to enter too. + /// Note that the fiber waiting for the close(), which until now was waiting for the one holder to leave, + /// will now wait for both copies to leave. + holder& operator=(const holder& x) noexcept { + if (x._g != _g) { + release(); + _g = x._g; + if (_g) { + _g->_count++; + debug_hold_gate(); + } + } + return *this; + } + + /// Move-assign another \ref holder. + /// The other \ref gate is unaffected, + /// and so the move-assign operator must always succeed. + /// Leave the current \ref gate before assigning the other one. + holder& operator=(holder&& x) noexcept { + if (&x != this) { + release(); + _g = std::exchange(x._g, nullptr); + } + return *this; + } + + /// Leave the held \c gate + void release() noexcept { + if (_g) { + _g->leave(); + debug_release_gate(); + _g = nullptr; + } + } + + private: +#ifdef SEASTAR_GATE_HOLDER_DEBUG + void debug_hold_gate() noexcept { + ++_g->_holders; + } + + void debug_release_gate() noexcept { + --_g->_holders; + } +#else // SEASTAR_GATE_HOLDER_DEBUG + void debug_hold_gate() noexcept {} + void debug_release_gate() noexcept {} +#endif // SEASTAR_GATE_HOLDER_DEBUG + }; + + /// Get a RAII-based gate::holder object that \ref enter "enter()s" + /// the gate when constructed and \ref leave "leave()s" it when destroyed. + holder hold() { + return holder(*this); + } +}; + +namespace internal { + +template <typename Func> +inline +auto +invoke_func_with_gate(gate& g, Func&& func) noexcept { + return futurize_invoke(std::forward<Func>(func)).finally([&g] { g.leave(); }); +} + +} // namespace intgernal + +/// Executes the function \c func making sure the gate \c g is properly entered +/// and later on, properly left. +/// +/// \param func function to be executed +/// \param g the gate. Caller must make sure that it outlives this function. +/// \returns whatever \c func returns +/// +/// \relates gate +template <typename Func> +inline +auto +with_gate(gate& g, Func&& func) { + g.enter(); + return internal::invoke_func_with_gate(g, std::forward<Func>(func)); +} + +/// Executes the function \c func if the gate \c g can be entered +/// and later on, properly left. +/// +/// \param func function to be executed +/// \param g the gate. Caller must make sure that it outlives this function. +/// +/// If the gate is already closed, an exception future holding +/// \ref gate_closed_exception is returned, otherwise +/// \returns whatever \c func returns. +/// +/// \relates gate +template <typename Func> +inline +auto +try_with_gate(gate& g, Func&& func) noexcept { + if (!g.try_enter()) { + using futurator = futurize<std::invoke_result_t<Func>>; + return futurator::make_exception_future(gate_closed_exception()); + } + return internal::invoke_func_with_gate(g, std::forward<Func>(func)); +} +/// @} + +} diff --git a/src/seastar/include/seastar/core/idle_cpu_handler.hh b/src/seastar/include/seastar/core/idle_cpu_handler.hh new file mode 100644 index 000000000..793fc422a --- /dev/null +++ b/src/seastar/include/seastar/core/idle_cpu_handler.hh @@ -0,0 +1,59 @@ +/* + * 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 2020 ScyllaDB + */ + +#pragma once + +#include <seastar/util/noncopyable_function.hh> + +/// \file + +namespace seastar { + +/// Indicates the outcome of a user callback installed to take advantage of +/// idle CPU cycles. +enum class idle_cpu_handler_result { + no_more_work, //!< The user callback has no more work to perform + interrupted_by_higher_priority_task //!< A call to the work_waiting_on_reactor parameter to idle_cpu_handler returned `true` +}; + +/// Signature of a callback provided by the reactor to a user callback installed to take +/// advantage of idle cpu cycles, used to periodically check if the CPU is still idle. +/// +/// \return true if the reactor has new work to do +using work_waiting_on_reactor = const noncopyable_function<bool()>&; + +/// Signature of a callback provided by the user, that the reactor calls when it has idle cycles. +/// +/// The `poll` parameter is a work_waiting_on_reactor function that should be periodically called +/// to check if the idle callback should return with idle_cpu_handler_result::interrupted_by_higher_priority_task +using idle_cpu_handler = noncopyable_function<idle_cpu_handler_result(work_waiting_on_reactor poll)>; + +/// Set a handler that will be called when there is no task to execute on cpu. +/// Handler should do a low priority work. +/// +/// Handler's return value determines whether handler did any actual work. If no work was done then reactor will go +/// into sleep. +/// +/// Handler's argument is a function that returns true if a task which should be executed on cpu appears or false +/// otherwise. This function should be used by a handler to return early if a task appears. +void set_idle_cpu_handler(idle_cpu_handler&& handler); + +} diff --git a/src/seastar/include/seastar/core/internal/api-level.hh b/src/seastar/include/seastar/core/internal/api-level.hh new file mode 100644 index 000000000..c18e03f82 --- /dev/null +++ b/src/seastar/include/seastar/core/internal/api-level.hh @@ -0,0 +1,82 @@ +/* + * 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) 2019 ScyllaDB + */ + +#pragma once + +// For IDEs that don't see SEASTAR_API_LEVEL, generate a nice default +#ifndef SEASTAR_API_LEVEL +#define SEASTAR_API_LEVEL 3 +#endif + +#if SEASTAR_API_LEVEL == 6 +#define SEASTAR_INCLUDE_API_V6 inline +#else +#define SEASTAR_INCLUDE_API_V6 +#endif + +#if SEASTAR_API_LEVEL == 5 +#define SEASTAR_INCLUDE_API_V5 inline +#else +#define SEASTAR_INCLUDE_API_V5 +#endif + +#if SEASTAR_API_LEVEL == 4 +#define SEASTAR_INCLUDE_API_V4 inline +#else +#define SEASTAR_INCLUDE_API_V4 +#endif + +#if SEASTAR_API_LEVEL == 3 +#define SEASTAR_INCLUDE_API_V3 inline +#else +#define SEASTAR_INCLUDE_API_V3 +#endif + +#if SEASTAR_API_LEVEL == 2 +#define SEASTAR_INCLUDE_API_V2 inline +#else +#define SEASTAR_INCLUDE_API_V2 +#endif + +// Declare them here so we don't have to use the macros everywhere +namespace seastar { + SEASTAR_INCLUDE_API_V2 namespace api_v2 { + } + SEASTAR_INCLUDE_API_V3 namespace api_v3 { + inline namespace and_newer { + } + } + SEASTAR_INCLUDE_API_V4 namespace api_v4 { + inline namespace and_newer { + using namespace api_v3::and_newer; + } + } + SEASTAR_INCLUDE_API_V5 namespace api_v5 { + inline namespace and_newer { + using namespace api_v4::and_newer; + } + } + SEASTAR_INCLUDE_API_V6 namespace api_v6 { + inline namespace and_newer { + using namespace api_v5::and_newer; + } + } +} diff --git a/src/seastar/include/seastar/core/internal/buffer_allocator.hh b/src/seastar/include/seastar/core/internal/buffer_allocator.hh new file mode 100644 index 000000000..0cdd39e75 --- /dev/null +++ b/src/seastar/include/seastar/core/internal/buffer_allocator.hh @@ -0,0 +1,43 @@ +/* + * 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) 2020 ScyllaDB + */ + +#pragma once + +namespace seastar { + +template <typename CharType> +class temporary_buffer; + +namespace internal { + +// Internal interface for allocating buffers for reads. Used to decouple +// allocation strategies (where to allocate from, and what sizes) from the +// point where allocation happens, to make it as late as possible. +class buffer_allocator { +public: + virtual ~buffer_allocator() = default; + virtual temporary_buffer<char> allocate_buffer() = 0; +}; + + +} + +} diff --git a/src/seastar/include/seastar/core/internal/io_desc.hh b/src/seastar/include/seastar/core/internal/io_desc.hh new file mode 100644 index 000000000..74d1cf90f --- /dev/null +++ b/src/seastar/include/seastar/core/internal/io_desc.hh @@ -0,0 +1,35 @@ +/* + * 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) 2019 ScyllaDB Ltd. + */ + +#pragma once + +#include <seastar/core/linux-aio.hh> +#include <exception> + +namespace seastar { + +class kernel_completion { +protected: + ~kernel_completion() = default; +public: + virtual void complete_with(ssize_t res) = 0; +}; +} diff --git a/src/seastar/include/seastar/core/internal/io_intent.hh b/src/seastar/include/seastar/core/internal/io_intent.hh new file mode 100644 index 000000000..a8cbd071d --- /dev/null +++ b/src/seastar/include/seastar/core/internal/io_intent.hh @@ -0,0 +1,129 @@ +/* + * 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 2021 ScyllaDB + */ + +#pragma once + +#include <utility> +#include <boost/intrusive/list.hpp> +#include <boost/intrusive/slist.hpp> + +namespace bi = boost::intrusive; + +namespace seastar { + +class io_intent; + +namespace internal { + +/* + * The tracker of cancellable sub-queue of requests. + * + * This queue is stuffed with requests that sit in the + * same IO queue for dispatching (there can be other requests + * as well) and ties them together for cancellation. + * This IO queue is the fair_queue's priority_class's one. + * Beware, if requests from different IO queues happen + * in the same cancellable queue the whole thing blows up. + */ +class cancellable_queue { +public: + class link { + friend class cancellable_queue; + + union { + cancellable_queue* _ref; + bi::slist_member_hook<> _hook; + }; + + public: + link() noexcept : _ref(nullptr) {} + ~link() { assert(_ref == nullptr); } + + void enqueue(cancellable_queue& cq) noexcept { + cq.push_back(*this); + } + + void maybe_dequeue() noexcept { + if (_ref != nullptr) { + _ref->pop_front(); + } + } + }; + +private: + static_assert(sizeof(link) == sizeof(void*), "cancellable_queue::link size is too big"); + using list_of_links_t = bi::slist<link, + bi::constant_time_size<false>, + bi::cache_last<true>, + bi::member_hook<link, bi::slist_member_hook<>, &link::_hook>>; + + link* _first; + list_of_links_t _rest; + + void push_back(link& il) noexcept; + void pop_front() noexcept; + +public: + cancellable_queue() noexcept : _first(nullptr) {} + cancellable_queue(const cancellable_queue&) = delete; + cancellable_queue(cancellable_queue&& o) noexcept; + cancellable_queue& operator=(cancellable_queue&& o) noexcept; + ~cancellable_queue(); +}; + +/* + * A "safe" reference on a intent. Safe here means that the original + * intent can be destroyed at any time and this reference will be + * updated not to point at it any longer. + * The retrieve() method brings the original intent back or throws + * and exception if the intent was cancelled. + */ +class intent_reference : public bi::list_base_hook<bi::link_mode<bi::auto_unlink>> { + friend class seastar::io_intent; + using container_type = bi::list<intent_reference, bi::constant_time_size<false>>; + static constexpr uintptr_t _cancelled_intent = 1; + io_intent* _intent; + + void on_cancel() noexcept { _intent = reinterpret_cast<io_intent*>(_cancelled_intent); } + bool is_cancelled() const noexcept { return _intent == reinterpret_cast<io_intent*>(_cancelled_intent); } + +public: + intent_reference(io_intent* intent) noexcept; + + intent_reference(intent_reference&& o) noexcept : _intent(std::exchange(o._intent, nullptr)) { + container_type::node_algorithms::swap_nodes(o.this_ptr(), this_ptr()); + } + + intent_reference& operator=(intent_reference&& o) noexcept { + if (this != &o) { + _intent = std::exchange(o._intent, nullptr); + unlink(); + container_type::node_algorithms::swap_nodes(o.this_ptr(), this_ptr()); + } + return *this; + } + + io_intent* retrieve() const; +}; + +} // namespace internal + +} // namespace seastar diff --git a/src/seastar/include/seastar/core/internal/io_request.hh b/src/seastar/include/seastar/core/internal/io_request.hh new file mode 100644 index 000000000..277b338c5 --- /dev/null +++ b/src/seastar/include/seastar/core/internal/io_request.hh @@ -0,0 +1,420 @@ +/* + * 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 2020 ScyllaDB + */ + +#pragma once + +#include <seastar/core/sstring.hh> +#include <seastar/core/linux-aio.hh> +#include <seastar/core/internal/io_desc.hh> +#include <seastar/core/on_internal_error.hh> +#include <sys/types.h> +#include <sys/socket.h> + +namespace seastar { +extern logger io_log; + +namespace internal { + +class io_request { +public: + enum class operation { read, readv, write, writev, fdatasync, recv, recvmsg, send, sendmsg, accept, connect, poll_add, poll_remove, cancel }; +private: + operation _op; + // the upper layers give us void pointers, but storing void pointers here is just + // dangerous. The constructors seem to be happy to convert other pointers to void*, + // even if they are marked as explicit, and then you end up losing approximately 3 hours + // and 15 minutes (hypothetically, of course), trying to chase the weirdest bug. + // Let's store a char* for safety, and cast it back to void* in the accessor. + struct read_op { + int fd; + uint64_t pos; + char* addr; + size_t size; + bool nowait_works; + }; + struct readv_op { + int fd; + uint64_t pos; + ::iovec* iovec; + size_t iov_len; + bool nowait_works; + }; + struct recv_op { + int fd; + char* addr; + size_t size; + int flags; + }; + struct recvmsg_op { + int fd; + ::msghdr* msghdr; + int flags; + }; + using send_op = recv_op; + using sendmsg_op = recvmsg_op; + using write_op = read_op; + using writev_op = readv_op; + struct fdatasync_op { + int fd; + }; + struct accept_op { + int fd; + ::sockaddr* sockaddr; + socklen_t* socklen_ptr; + int flags; + }; + struct connect_op { + int fd; + ::sockaddr* sockaddr; + socklen_t socklen; + }; + struct poll_add_op { + int fd; + int events; + }; + struct poll_remove_op { + int fd; + char* addr; + }; + struct cancel_op { + int fd; + char* addr; + }; + union { + read_op _read; + readv_op _readv; + recv_op _recv; + recvmsg_op _recvmsg; + send_op _send; + sendmsg_op _sendmsg; + write_op _write; + writev_op _writev; + fdatasync_op _fdatasync; + accept_op _accept; + connect_op _connect; + poll_add_op _poll_add; + poll_remove_op _poll_remove; + cancel_op _cancel; + }; + +public: + static io_request make_read(int fd, uint64_t pos, void* address, size_t size, bool nowait_works) { + io_request req; + req._op = operation::read; + req._read = { + .fd = fd, + .pos = pos, + .addr = reinterpret_cast<char*>(address), + .size = size, + .nowait_works = nowait_works, + }; + return req; + } + + static io_request make_readv(int fd, uint64_t pos, std::vector<iovec>& iov, bool nowait_works) { + io_request req; + req._op = operation::readv; + req._readv = { + .fd = fd, + .pos = pos, + .iovec = iov.data(), + .iov_len = iov.size(), + .nowait_works = nowait_works, + }; + return req; + } + + static io_request make_recv(int fd, void* address, size_t size, int flags) { + io_request req; + req._op = operation::recv; + req._recv = { + .fd = fd, + .addr = reinterpret_cast<char*>(address), + .size = size, + .flags = flags, + }; + return req; + } + + static io_request make_recvmsg(int fd, ::msghdr* msg, int flags) { + io_request req; + req._op = operation::recvmsg; + req._recvmsg = { + .fd = fd, + .msghdr = msg, + .flags = flags, + }; + return req; + } + + static io_request make_send(int fd, const void* address, size_t size, int flags) { + io_request req; + req._op = operation::send; + req._send = { + .fd = fd, + .addr = const_cast<char*>(reinterpret_cast<const char*>(address)), + .size = size, + .flags = flags, + }; + return req; + } + + static io_request make_sendmsg(int fd, ::msghdr* msg, int flags) { + io_request req; + req._op = operation::sendmsg; + req._sendmsg = { + .fd = fd, + .msghdr = msg, + .flags = flags, + }; + return req; + } + + static io_request make_write(int fd, uint64_t pos, const void* address, size_t size, bool nowait_works) { + io_request req; + req._op = operation::write; + req._write = { + .fd = fd, + .pos = pos, + .addr = const_cast<char*>(reinterpret_cast<const char*>(address)), + .size = size, + .nowait_works = nowait_works, + }; + return req; + } + + static io_request make_writev(int fd, uint64_t pos, std::vector<iovec>& iov, bool nowait_works) { + io_request req; + req._op = operation::writev; + req._writev = { + .fd = fd, + .pos = pos, + .iovec = iov.data(), + .iov_len = iov.size(), + .nowait_works = nowait_works, + }; + return req; + } + + static io_request make_fdatasync(int fd) { + io_request req; + req._op = operation::fdatasync; + req._fdatasync = { + .fd = fd, + }; + return req; + } + + static io_request make_accept(int fd, struct sockaddr* addr, socklen_t* addrlen, int flags) { + io_request req; + req._op = operation::accept; + req._accept = { + .fd = fd, + .sockaddr = addr, + .socklen_ptr = addrlen, + .flags = flags, + }; + return req; + } + + static io_request make_connect(int fd, struct sockaddr* addr, socklen_t addrlen) { + io_request req; + req._op = operation::connect; + req._connect = { + .fd = fd, + .sockaddr = addr, + .socklen = addrlen, + }; + return req; + } + + static io_request make_poll_add(int fd, int events) { + io_request req; + req._op = operation::poll_add; + req._poll_add = { + .fd = fd, + .events = events, + }; + return req; + } + + static io_request make_poll_remove(int fd, void *addr) { + io_request req; + req._op = operation::poll_remove; + req._poll_remove = { + .fd = fd, + .addr = reinterpret_cast<char*>(addr), + }; + return req; + } + + static io_request make_cancel(int fd, void *addr) { + io_request req; + req._op = operation::cancel; + req._cancel = { + .fd = fd, + .addr = reinterpret_cast<char*>(addr), + }; + return req; + } + + bool is_read() const { + switch (_op) { + case operation::read: + case operation::readv: + case operation::recvmsg: + case operation::recv: + return true; + default: + return false; + } + } + + bool is_write() const { + switch (_op) { + case operation::write: + case operation::writev: + case operation::send: + case operation::sendmsg: + return true; + default: + return false; + } + } + + sstring opname() const; + + operation opcode() const { + return _op; + } + + template <operation Op> + auto& as() const { + if constexpr (Op == operation::read) { + return _read; + } + if constexpr (Op == operation::readv) { + return _readv; + } + if constexpr (Op == operation::recv) { + return _recv; + } + if constexpr (Op == operation::recvmsg) { + return _recvmsg; + } + if constexpr (Op == operation::send) { + return _send; + } + if constexpr (Op == operation::sendmsg) { + return _sendmsg; + } + if constexpr (Op == operation::write) { + return _write; + } + if constexpr (Op == operation::writev) { + return _writev; + } + if constexpr (Op == operation::fdatasync) { + return _fdatasync; + } + if constexpr (Op == operation::accept) { + return _accept; + } + if constexpr (Op == operation::connect) { + return _connect; + } + if constexpr (Op == operation::poll_add) { + return _poll_add; + } + if constexpr (Op == operation::poll_remove) { + return _poll_remove; + } + if constexpr (Op == operation::cancel) { + return _cancel; + } + } + + struct part; + std::vector<part> split(size_t max_length); + +private: + io_request sub_req_buffer(size_t pos, size_t len) const { + io_request sub_req; + sub_req._op = _op; + // read_op and write_op share the same layout, so we don't handle + // them separately + auto& op = _read; + auto& sub_op = sub_req._read; + sub_op = { + .fd = op.fd, + .pos = op.pos + pos, + .addr = op.addr + pos, + .size = len, + .nowait_works = op.nowait_works, + }; + return sub_req; + } + std::vector<part> split_buffer(size_t max_length); + + io_request sub_req_iovec(size_t pos, std::vector<iovec>& iov) const { + io_request sub_req; + sub_req._op = _op; + // readv_op and writev_op share the same layout, so we don't handle + // them separately + auto& op = _readv; + auto& sub_op = sub_req._readv; + sub_op = { + .fd = op.fd, + .pos = op.pos + pos, + .iovec = iov.data(), + .iov_len = iov.size(), + .nowait_works = op.nowait_works, + }; + return sub_req; + } + std::vector<part> split_iovec(size_t max_length); +}; + +struct io_request::part { + io_request req; + size_t size; + std::vector<::iovec> iovecs; +}; + +// Helper pair of IO direction and length +struct io_direction_and_length { + size_t _directed_length; // bit 0 is R/W flag + +public: + size_t length() const noexcept { return _directed_length >> 1; } + int rw_idx() const noexcept { return _directed_length & 0x1; } + static constexpr int read_idx = 1; + static constexpr int write_idx = 0; + + io_direction_and_length(int idx, size_t val) noexcept + : _directed_length((val << 1) | idx) + { + assert(idx == read_idx || idx == write_idx); + } +}; + +} +} diff --git a/src/seastar/include/seastar/core/internal/io_sink.hh b/src/seastar/include/seastar/core/internal/io_sink.hh new file mode 100644 index 000000000..c91cd3307 --- /dev/null +++ b/src/seastar/include/seastar/core/internal/io_sink.hh @@ -0,0 +1,76 @@ +/* + * 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 2021 ScyllaDB + */ + +#pragma once + +#include <seastar/core/circular_buffer.hh> +#include <seastar/core/internal/io_request.hh> +#include <seastar/util/concepts.hh> + +namespace seastar { + +class io_completion; + +namespace internal { + +class io_sink; + +class pending_io_request : private internal::io_request { + friend class io_sink; + io_completion* _completion; + +public: + pending_io_request(internal::io_request req, io_completion* desc) noexcept + : io_request(std::move(req)) + , _completion(desc) + { } +}; + +class io_sink { + circular_buffer<pending_io_request> _pending_io; +public: + void submit(io_completion* desc, internal::io_request req) noexcept; + + template <typename Fn> + // Fn should return whether the request was consumed and + // draining should try to drain more + SEASTAR_CONCEPT( requires std::is_invocable_r<bool, Fn, internal::io_request&, io_completion*>::value ) + size_t drain(Fn&& consume) { + size_t pending = _pending_io.size(); + size_t drained = 0; + + while (pending > drained) { + pending_io_request& req = _pending_io[drained]; + + if (!consume(req, req._completion)) { + break; + } + drained++; + } + + _pending_io.erase(_pending_io.begin(), _pending_io.begin() + drained); + return drained; + } +}; + +} // namespace internal + +} // namespace seastar diff --git a/src/seastar/include/seastar/core/internal/poll.hh b/src/seastar/include/seastar/core/internal/poll.hh new file mode 100644 index 000000000..1816a60de --- /dev/null +++ b/src/seastar/include/seastar/core/internal/poll.hh @@ -0,0 +1,59 @@ +/* + * 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 2019 ScyllaDB + */ + +#pragma once + +namespace seastar { + +struct pollfn { + virtual ~pollfn() {} + // Returns true if work was done (false = idle) + virtual bool poll() = 0; + // Checks if work needs to be done, but without actually doing any + // returns true if works needs to be done (false = idle) + virtual bool pure_poll() = 0; + // Tries to enter interrupt mode. + // + // If it returns true, then events from this poller will wake + // a sleeping idle loop, and exit_interrupt_mode() must be called + // to return to normal polling. + // + // If it returns false, the sleeping idle loop may not be entered. + virtual bool try_enter_interrupt_mode() = 0; + virtual void exit_interrupt_mode() = 0; +}; + +// The common case for poller -- do not make any difference between +// poll() and pure_poll(), always/never agree to go to sleep and do +// nothing on wakeup. +template <bool Passive> +struct simple_pollfn : public pollfn { + virtual bool pure_poll() override final { + return poll(); + } + virtual bool try_enter_interrupt_mode() override final { + return Passive; + } + virtual void exit_interrupt_mode() override final { + } +}; + +} diff --git a/src/seastar/include/seastar/core/internal/pollable_fd.hh b/src/seastar/include/seastar/core/internal/pollable_fd.hh new file mode 100644 index 000000000..9b88b86e3 --- /dev/null +++ b/src/seastar/include/seastar/core/internal/pollable_fd.hh @@ -0,0 +1,240 @@ +/* + * 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 2019 ScyllaDB + */ + +#pragma once + +#include <seastar/core/future.hh> +#include <seastar/core/posix.hh> +#include <vector> +#include <tuple> +#include <seastar/core/internal/io_desc.hh> +#include <seastar/util/bool_class.hh> +#include <boost/intrusive_ptr.hpp> + +namespace seastar { + +class reactor; +class pollable_fd; +class pollable_fd_state; +class socket_address; + +namespace internal { + +class buffer_allocator; + +} + +namespace net { + +class packet; + +} + +class pollable_fd_state; + +using pollable_fd_state_ptr = boost::intrusive_ptr<pollable_fd_state>; + +class pollable_fd_state { + unsigned _refs = 0; +public: + virtual ~pollable_fd_state() {} + struct speculation { + int events = 0; + explicit speculation(int epoll_events_guessed = 0) : events(epoll_events_guessed) {} + }; + pollable_fd_state(const pollable_fd_state&) = delete; + void operator=(const pollable_fd_state&) = delete; + /// Set the speculation of specified I/O events + /// + /// We try to speculate. If an I/O is completed successfully without being + /// blocked and it didn't return the short read/write. We anticipate that + /// the next I/O will also be non-blocking and will not return EAGAIN. + /// But the speculation is invalidated once it is "used" by + /// \c take_speculation() + void speculate_epoll(int events) { events_known |= events; } + /// Check whether we speculate specified I/O is possible on the fd, + /// invalidate the speculation if it matches with all specified \c events. + /// + /// \return true if the current speculation includes all specified events + bool take_speculation(int events) { + // invalidate the speculation set by the last speculate_epoll() call, + if (events_known & events) { + events_known &= ~events; + return true; + } + return false; + } + file_desc fd; + bool events_rw = false; // single consumer for both read and write (accept()) + unsigned shutdown_mask = 0; // For udp, there is no shutdown indication from the kernel + int events_requested = 0; // wanted by pollin/pollout promises + int events_epoll = 0; // installed in epoll + int events_known = 0; // returned from epoll + + friend class reactor; + friend class pollable_fd; + friend class reactor_backend_uring; + + future<size_t> read_some(char* buffer, size_t size); + future<size_t> read_some(uint8_t* buffer, size_t size); + future<size_t> read_some(const std::vector<iovec>& iov); + future<temporary_buffer<char>> read_some(internal::buffer_allocator* ba); + future<> write_all(const char* buffer, size_t size); + future<> write_all(const uint8_t* buffer, size_t size); + future<size_t> write_some(net::packet& p); + future<> write_all(net::packet& p); + future<> readable(); + future<> writeable(); + future<> readable_or_writeable(); + future<std::tuple<pollable_fd, socket_address>> accept(); + future<> connect(socket_address& sa); + future<temporary_buffer<char>> recv_some(internal::buffer_allocator* ba); + future<size_t> sendmsg(struct msghdr *msg); + future<size_t> recvmsg(struct msghdr *msg); + future<size_t> sendto(socket_address addr, const void* buf, size_t len); + future<> poll_rdhup(); + +protected: + explicit pollable_fd_state(file_desc fd, speculation speculate = speculation()) + : fd(std::move(fd)), events_known(speculate.events) {} +private: + void maybe_no_more_recv(); + void maybe_no_more_send(); + void forget(); // called on end-of-life + + friend void intrusive_ptr_add_ref(pollable_fd_state* fd) { + ++fd->_refs; + } + friend void intrusive_ptr_release(pollable_fd_state* fd); +}; + +class pollable_fd { +public: + using speculation = pollable_fd_state::speculation; + pollable_fd() = default; + pollable_fd(file_desc fd, speculation speculate = speculation()); +public: + future<size_t> read_some(char* buffer, size_t size) { + return _s->read_some(buffer, size); + } + future<size_t> read_some(uint8_t* buffer, size_t size) { + return _s->read_some(buffer, size); + } + future<size_t> read_some(const std::vector<iovec>& iov) { + return _s->read_some(iov); + } + future<temporary_buffer<char>> read_some(internal::buffer_allocator* ba) { + return _s->read_some(ba); + } + future<> write_all(const char* buffer, size_t size) { + return _s->write_all(buffer, size); + } + future<> write_all(const uint8_t* buffer, size_t size) { + return _s->write_all(buffer, size); + } + future<size_t> write_some(net::packet& p) { + return _s->write_some(p); + } + future<> write_all(net::packet& p) { + return _s->write_all(p); + } + future<> readable() { + return _s->readable(); + } + future<> writeable() { + return _s->writeable(); + } + future<> readable_or_writeable() { + return _s->readable_or_writeable(); + } + future<std::tuple<pollable_fd, socket_address>> accept() { + return _s->accept(); + } + future<> connect(socket_address& sa) { + return _s->connect(sa); + } + future<temporary_buffer<char>> recv_some(internal::buffer_allocator* ba) { + return _s->recv_some(ba); + } + future<size_t> sendmsg(struct msghdr *msg) { + return _s->sendmsg(msg); + } + future<size_t> recvmsg(struct msghdr *msg) { + return _s->recvmsg(msg); + } + future<size_t> sendto(socket_address addr, const void* buf, size_t len) { + return _s->sendto(addr, buf, len); + } + file_desc& get_file_desc() const { return _s->fd; } + using shutdown_kernel_only = bool_class<struct shutdown_kernel_only_tag>; + void shutdown(int how, shutdown_kernel_only kernel_only = shutdown_kernel_only::yes); + void close() { _s.reset(); } + explicit operator bool() const noexcept { + return bool(_s); + } + future<> poll_rdhup() { + return _s->poll_rdhup(); + } +protected: + int get_fd() const { return _s->fd.get(); } + void maybe_no_more_recv() { return _s->maybe_no_more_recv(); } + void maybe_no_more_send() { return _s->maybe_no_more_send(); } + friend class reactor; + friend class readable_eventfd; + friend class writeable_eventfd; + friend class aio_storage_context; +private: + pollable_fd_state_ptr _s; +}; + +class writeable_eventfd; + +class readable_eventfd { + pollable_fd _fd; +public: + explicit readable_eventfd(size_t initial = 0) : _fd(try_create_eventfd(initial)) {} + readable_eventfd(readable_eventfd&&) = default; + writeable_eventfd write_side(); + future<size_t> wait(); + int get_write_fd() { return _fd.get_fd(); } +private: + explicit readable_eventfd(file_desc&& fd) : _fd(std::move(fd)) {} + static file_desc try_create_eventfd(size_t initial); + + friend class writeable_eventfd; +}; + +class writeable_eventfd { + file_desc _fd; +public: + explicit writeable_eventfd(size_t initial = 0) : _fd(try_create_eventfd(initial)) {} + writeable_eventfd(writeable_eventfd&&) = default; + readable_eventfd read_side(); + void signal(size_t nr); + int get_read_fd() { return _fd.get(); } +private: + explicit writeable_eventfd(file_desc&& fd) : _fd(std::move(fd)) {} + static file_desc try_create_eventfd(size_t initial); + + friend class readable_eventfd; +}; + +} diff --git a/src/seastar/include/seastar/core/internal/read_state.hh b/src/seastar/include/seastar/core/internal/read_state.hh new file mode 100644 index 000000000..65282869c --- /dev/null +++ b/src/seastar/include/seastar/core/internal/read_state.hh @@ -0,0 +1,103 @@ +/* + * 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 2021 ScyllaDB + */ + +#include <cstring> +#include <seastar/core/align.hh> +#include <seastar/core/internal/io_intent.hh> +#include <seastar/core/temporary_buffer.hh> + +namespace seastar { +namespace internal { + +template <typename CharType> +struct file_read_state { + typedef temporary_buffer<CharType> tmp_buf_type; + + file_read_state(uint64_t offset, uint64_t front, size_t to_read, + size_t memory_alignment, size_t disk_alignment, io_intent* intent) + : buf(tmp_buf_type::aligned(memory_alignment, + align_up(to_read, disk_alignment))) + , _offset(offset) + , _to_read(to_read) + , _front(front) + , _iref(intent) {} + + bool done() const { + return eof || pos >= _to_read; + } + + /** + * Trim the buffer to the actual number of read bytes and cut the + * bytes from offset 0 till "_front". + * + * @note this function has to be called only if we read bytes beyond + * "_front". + */ + void trim_buf_before_ret() { + if (have_good_bytes()) { + buf.trim(pos); + buf.trim_front(_front); + } else { + buf.trim(0); + } + } + + uint64_t cur_offset() const { + return _offset + pos; + } + + size_t left_space() const { + return buf.size() - pos; + } + + size_t left_to_read() const { + // positive as long as (done() == false) + return _to_read - pos; + } + + void append_new_data(tmp_buf_type& new_data) { + auto to_copy = std::min(left_space(), new_data.size()); + + std::memcpy(buf.get_write() + pos, new_data.get(), to_copy); + pos += to_copy; + } + + bool have_good_bytes() const { + return pos > _front; + } + + io_intent* get_intent() { + return _iref.retrieve(); + } + +public: + bool eof = false; + tmp_buf_type buf; + size_t pos = 0; +private: + uint64_t _offset; + size_t _to_read; + uint64_t _front; + internal::intent_reference _iref; +}; + +} // namespace internal +} // namespace seastar diff --git a/src/seastar/include/seastar/core/internal/stall_detector.hh b/src/seastar/include/seastar/core/internal/stall_detector.hh new file mode 100644 index 000000000..326dc08b8 --- /dev/null +++ b/src/seastar/include/seastar/core/internal/stall_detector.hh @@ -0,0 +1,180 @@ + +/* + * 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 + +#include <signal.h> +#include <atomic> +#include <limits> +#include <chrono> +#include <functional> +#include <memory> +#include <seastar/core/posix.hh> +#include <seastar/core/metrics_registration.hh> +#include <seastar/core/scheduling.hh> +#include <linux/perf_event.h> + +namespace seastar { + +class reactor; +class thread_cputime_clock; + +namespace internal { + +struct cpu_stall_detector_config { + std::chrono::duration<double> threshold = std::chrono::seconds(2); + unsigned stall_detector_reports_per_minute = 1; + float slack = 0.3; // fraction of threshold that we're allowed to overshoot + bool oneline = true; // print a simplified backtrace on a single line + std::function<void ()> report; // alternative reporting function for tests +}; + +// Detects stalls in continuations that run for too long +class cpu_stall_detector { +protected: + std::atomic<uint64_t> _last_tasks_processed_seen{}; + unsigned _stall_detector_reports_per_minute; + std::atomic<uint64_t> _stall_detector_missed_ticks = { 0 }; + unsigned _reported = 0; + unsigned _total_reported = 0; + unsigned _max_reports_per_minute; + unsigned _shard_id; + unsigned _thread_id; + unsigned _report_at{}; + sched_clock::time_point _minute_mark{}; + sched_clock::time_point _rearm_timer_at{}; + sched_clock::time_point _run_started_at{}; + sched_clock::duration _threshold; + sched_clock::duration _slack; + cpu_stall_detector_config _config; + seastar::metrics::metric_groups _metrics; + friend reactor; + virtual bool is_spurious_signal() { + return false; + } + virtual void maybe_report_kernel_trace() {} +private: + void maybe_report(); + virtual void arm_timer() = 0; + void report_suppressions(sched_clock::time_point now); + void reset_suppression_state(sched_clock::time_point now); +public: + using clock_type = thread_cputime_clock; +public: + explicit cpu_stall_detector(cpu_stall_detector_config cfg = {}); + virtual ~cpu_stall_detector() = default; + static int signal_number() { return SIGRTMIN + 1; } + void start_task_run(sched_clock::time_point now); + void end_task_run(sched_clock::time_point now); + void generate_trace(); + void update_config(cpu_stall_detector_config cfg); + cpu_stall_detector_config get_config() const; + void on_signal(); + virtual void start_sleep() = 0; + void end_sleep(); +}; + +class cpu_stall_detector_posix_timer : public cpu_stall_detector { + timer_t _timer; +public: + explicit cpu_stall_detector_posix_timer(cpu_stall_detector_config cfg = {}); + virtual ~cpu_stall_detector_posix_timer() override; +private: + virtual void arm_timer() override; + virtual void start_sleep() override; +}; + +class cpu_stall_detector_linux_perf_event : public cpu_stall_detector { + file_desc _fd; + bool _enabled = false; + uint64_t _current_period = 0; + struct ::perf_event_mmap_page* _mmap; + char* _data_area; + size_t _data_area_mask; + // after the detector has been armed (i.e., _enabled is true), this + // is the moment at or after which the next signal is expected to occur + // and can be used for detecting spurious signals + sched_clock::time_point _next_signal_time{}; +private: + class data_area_reader { + cpu_stall_detector_linux_perf_event& _p; + const char* _data_area; + size_t _data_area_mask; + uint64_t _head; + uint64_t _tail; + public: + explicit data_area_reader(cpu_stall_detector_linux_perf_event& p) + : _p(p) + , _data_area(p._data_area) + , _data_area_mask(p._data_area_mask) { + _head = _p._mmap->data_head; + _tail = _p._mmap->data_tail; + std::atomic_thread_fence(std::memory_order_acquire); // required after reading data_head + } + ~data_area_reader() { + std::atomic_thread_fence(std::memory_order_release); // not documented, but probably required before writing data_tail + _p._mmap->data_tail = _tail; + } + uint64_t read_u64() { + uint64_t ret; + // We cannot wrap around if the 8-byte unit is aligned + std::copy_n(_data_area + (_tail & _data_area_mask), 8, reinterpret_cast<char*>(&ret)); + _tail += 8; + return ret; + } + template <typename S> + S read_struct() { + static_assert(sizeof(S) % 8 == 0); + S ret; + char* p = reinterpret_cast<char*>(&ret); + for (size_t i = 0; i != sizeof(S); i += 8) { + uint64_t w = read_u64(); + std::copy_n(reinterpret_cast<const char*>(&w), 8, p + i); + } + return ret; + } + void skip(uint64_t bytes_to_skip) { + _tail += bytes_to_skip; + } + // skip all the remaining data in the buffer, as-if calling read until + // have_data returns false (but much faster) + void skip_all() { + _tail = _head; + } + bool have_data() const { + return _head != _tail; + } + }; +public: + static std::unique_ptr<cpu_stall_detector_linux_perf_event> try_make(cpu_stall_detector_config cfg = {}); + explicit cpu_stall_detector_linux_perf_event(file_desc fd, cpu_stall_detector_config cfg = {}); + ~cpu_stall_detector_linux_perf_event(); + virtual void arm_timer() override; + virtual void start_sleep() override; + virtual bool is_spurious_signal() override; + virtual void maybe_report_kernel_trace() override; +}; + +std::unique_ptr<cpu_stall_detector> make_cpu_stall_detector(cpu_stall_detector_config cfg = {}); + +} +} diff --git a/src/seastar/include/seastar/core/internal/uname.hh b/src/seastar/include/seastar/core/internal/uname.hh new file mode 100644 index 000000000..66b3452e6 --- /dev/null +++ b/src/seastar/include/seastar/core/internal/uname.hh @@ -0,0 +1,60 @@ +/* + * 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) 2019 ScyllaDB + */ + +#pragma once + +#include <string> +#include <initializer_list> +#include <seastar/util/std-compat.hh> +#include <iosfwd> + +namespace seastar { + +namespace internal { + +// Representation of a Linux kernel version number +struct uname_t { + int version; // 4 in "4.5" + int patchlevel; // 5 in "4.5" + std::optional<int> sublevel; // 1 in "4.5.1" + std::optional<int> subsublevel; // 33 in "2.6.44.33" + std::optional<int> distro_patch; // 957 in "3.10.0-957.5.1.el7.x86_64" + std::string distro_extra; // .5.1.el7.x86_64 + + bool same_as_or_descendant_of(const uname_t& x) const; + bool same_as_or_descendant_of(const char* x) const; + bool whitelisted(std::initializer_list<const char*>) const; + + // 3 for "4.5.0", 5 for "5.1.3-33.3.el7"; "el7" doesn't count as a component + int component_count() const; + + // The "el7" that wasn't counted in components() + bool has_distro_extra(std::string extra) const; + friend std::ostream& operator<<(std::ostream& os, const uname_t& u); +}; + +uname_t kernel_uname(); + +uname_t parse_uname(const char* u); + +} +} diff --git a/src/seastar/include/seastar/core/io_intent.hh b/src/seastar/include/seastar/core/io_intent.hh new file mode 100644 index 000000000..a4bb78ed5 --- /dev/null +++ b/src/seastar/include/seastar/core/io_intent.hh @@ -0,0 +1,107 @@ +/* + * 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 2021 ScyllaDB + */ + +#pragma once + +#include <boost/container/small_vector.hpp> +#include <seastar/core/internal/io_intent.hh> +#include <seastar/core/io_priority_class.hh> + +namespace seastar { + +/// \example file_demo.cc +/// A handle confirming the caller's intent to do the IO +/// +/// When a pointer to an intent is passed to the \ref io_queue +/// "io_queue"::queue_request() method, the issued request is pinned +/// to the intent and is only processed as long as the intent object +/// is alive and the **cancel()** method is not called. +/// +/// If no intent is provided, then the request is processed till its +/// completion be it success or error +class io_intent { + struct intents_for_queue { + dev_t dev; + io_priority_class_id qid; + internal::cancellable_queue cq; + + intents_for_queue(dev_t dev_, io_priority_class_id qid_) noexcept + : dev(dev_), qid(qid_), cq() {} + + intents_for_queue(intents_for_queue&&) noexcept = default; + intents_for_queue& operator=(intents_for_queue&&) noexcept = default; + }; + + struct references { + internal::intent_reference::container_type list; + + references(references&& o) noexcept : list(std::move(o.list)) {} + references() noexcept : list() {} + ~references() { clear(); } + + void clear() { + list.clear_and_dispose([] (internal::intent_reference* r) { r->on_cancel(); }); + } + + void bind(internal::intent_reference& iref) noexcept { + list.push_back(iref); + } + }; + + boost::container::small_vector<intents_for_queue, 1> _intents; + references _refs; + friend internal::intent_reference::intent_reference(io_intent*) noexcept; + +public: + io_intent() = default; + ~io_intent() = default; + + io_intent(const io_intent&) = delete; + io_intent& operator=(const io_intent&) = delete; + io_intent& operator=(io_intent&&) = delete; + io_intent(io_intent&& o) noexcept : _intents(std::move(o._intents)), _refs(std::move(o._refs)) { + for (auto&& r : _refs.list) { + r._intent = this; + } + } + + /// Explicitly cancels all the requests attached to this intent + /// so far. The respective futures are resolved into the \ref + /// cancelled_error "cancelled_error" + void cancel() noexcept { + _refs.clear(); + _intents.clear(); + } + + /// @private + internal::cancellable_queue& find_or_create_cancellable_queue(dev_t dev, io_priority_class_id qid) { + for (auto&& i : _intents) { + if (i.dev == dev && i.qid == qid) { + return i.cq; + } + } + + _intents.emplace_back(dev, qid); + return _intents.back().cq; + } +}; + +} // namespace seastar diff --git a/src/seastar/include/seastar/core/io_priority_class.hh b/src/seastar/include/seastar/core/io_priority_class.hh new file mode 100644 index 000000000..66ce00e2f --- /dev/null +++ b/src/seastar/include/seastar/core/io_priority_class.hh @@ -0,0 +1,99 @@ +/* + * 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 2021 ScyllaDB + */ + +#pragma once + +#include <seastar/core/sstring.hh> +#include <seastar/core/future.hh> + +#include <array> +#include <mutex> + +namespace seastar { + +class io_queue; +using io_priority_class_id = unsigned; +// We could very well just add the name to the io_priority_class. However, because that +// structure is passed along all the time - and sometimes we can't help but copy it, better keep +// it lean. The name won't really be used for anything other than monitoring. +class io_priority_class { + io_priority_class_id _id; + + io_priority_class() = delete; + explicit io_priority_class(io_priority_class_id id) noexcept + : _id(id) + { } + + bool rename_registered(sstring name); + +public: + io_priority_class_id id() const noexcept { + return _id; + } + + static io_priority_class register_one(sstring name, uint32_t shares); + + /// \brief Updates the current amount of shares for a given priority class + /// + /// \param pc the priority class handle + /// \param shares the new shares value + /// \return a future that is ready when the share update is applied + future<> update_shares(uint32_t shares) const; + + /// \brief Updates the current bandwidth for a given priority class + /// + /// The bandwidth applied is NOT shard-local, instead it is applied so that + /// all shards cannot consume more bytes-per-second altogether + /// + /// \param bandwidth the new bandwidth value in bytes/second + /// \return a future that is ready when the bandwidth update is applied + future<> update_bandwidth(uint64_t bandwidth) const; + + /// Renames an io priority class + /// + /// Renames an `io_priority_class` previously created with register_one_priority_class(). + /// + /// The operation is global and affects all shards. + /// The operation affects the exported statistics labels. + /// + /// \param pc The io priority class to be renamed + /// \param new_name The new name for the io priority class + /// \return a future that is ready when the io priority class have been renamed + future<> rename(sstring new_name) noexcept; + + unsigned get_shares() const; + sstring get_name() const; + +private: + struct class_info { + unsigned shares = 0; + sstring name; + bool registered() const noexcept { return shares != 0; } + }; + + static constexpr unsigned _max_classes = 2048; + static std::mutex _register_lock; + static std::array<class_info, _max_classes> _infos; +}; + +const io_priority_class& default_priority_class(); + +} // namespace seastar diff --git a/src/seastar/include/seastar/core/io_queue.hh b/src/seastar/include/seastar/core/io_queue.hh new file mode 100644 index 000000000..e608cc10e --- /dev/null +++ b/src/seastar/include/seastar/core/io_queue.hh @@ -0,0 +1,201 @@ +/* + * 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 2019 ScyllaDB + */ + +#pragma once + +#include <boost/container/small_vector.hpp> +#include <seastar/core/sstring.hh> +#include <seastar/core/fair_queue.hh> +#include <seastar/core/metrics_registration.hh> +#include <seastar/core/future.hh> +#include <seastar/core/internal/io_request.hh> +#include <seastar/util/spinlock.hh> + +struct io_queue_for_tests; + +namespace seastar { + +class io_priority_class; + +[[deprecated("Use io_priority_class.rename")]] +future<> +rename_priority_class(io_priority_class pc, sstring new_name); + +class io_intent; + +namespace internal { +class io_sink; +namespace linux_abi { + +struct io_event; +struct iocb; + +} +} + +using shard_id = unsigned; +using stream_id = unsigned; + +class io_desc_read_write; +class queued_io_request; +class io_group; + +using io_group_ptr = std::shared_ptr<io_group>; +using iovec_keeper = std::vector<::iovec>; + +class io_queue { +public: + class priority_class_data; + +private: + std::vector<std::unique_ptr<priority_class_data>> _priority_classes; + io_group_ptr _group; + boost::container::small_vector<fair_queue, 2> _streams; + internal::io_sink& _sink; + + priority_class_data& find_or_create_class(const io_priority_class& pc); + + // The fields below are going away, they are just here so we can implement deprecated + // functions that used to be provided by the fair_queue and are going away (from both + // the fair_queue and the io_queue). Double-accounting for now will allow for easier + // decoupling and is temporary + size_t _queued_requests = 0; + size_t _requests_executing = 0; +public: + + using clock_type = std::chrono::steady_clock; + + // We want to represent the fact that write requests are (maybe) more expensive + // than read requests. To avoid dealing with floating point math we will scale one + // read request to be counted by this amount. + // + // A write request that is 30% more expensive than a read will be accounted as + // (read_request_base_count * 130) / 100. + // It is also technically possible for reads to be the expensive ones, in which case + // writes will have an integer value lower than read_request_base_count. + static constexpr unsigned read_request_base_count = 128; + static constexpr unsigned block_size_shift = 9; + + struct config { + dev_t devid; + unsigned long req_count_rate = std::numeric_limits<int>::max(); + unsigned long blocks_count_rate = std::numeric_limits<int>::max(); + unsigned disk_req_write_to_read_multiplier = read_request_base_count; + unsigned disk_blocks_write_to_read_multiplier = read_request_base_count; + size_t disk_read_saturation_length = std::numeric_limits<size_t>::max(); + size_t disk_write_saturation_length = std::numeric_limits<size_t>::max(); + sstring mountpoint = "undefined"; + bool duplex = false; + float rate_factor = 1.0; + std::chrono::duration<double> rate_limit_duration = std::chrono::milliseconds(1); + size_t block_count_limit_min = 1; + }; + + io_queue(io_group_ptr group, internal::io_sink& sink); + ~io_queue(); + + stream_id request_stream(internal::io_direction_and_length dnl) const noexcept; + + future<size_t> submit_io_read(const io_priority_class& priority_class, + size_t len, internal::io_request req, io_intent* intent, iovec_keeper iovs = {}) noexcept; + future<size_t> submit_io_write(const io_priority_class& priority_class, + size_t len, internal::io_request req, io_intent* intent, iovec_keeper iovs = {}) noexcept; + + future<size_t> queue_request(const io_priority_class& pc, internal::io_direction_and_length dnl, internal::io_request req, io_intent* intent, iovec_keeper iovs) noexcept; + future<size_t> queue_one_request(const io_priority_class& pc, internal::io_direction_and_length dnl, internal::io_request req, io_intent* intent, iovec_keeper iovs) noexcept; + void submit_request(io_desc_read_write* desc, internal::io_request req) noexcept; + void cancel_request(queued_io_request& req) noexcept; + void complete_cancelled_request(queued_io_request& req) noexcept; + void complete_request(io_desc_read_write& desc) noexcept; + + + [[deprecated("I/O queue users should not track individual requests, but resources (weight, size) passing through the queue")]] + size_t queued_requests() const { + return _queued_requests; + } + + // How many requests are sent to disk but not yet returned. + [[deprecated("I/O queue users should not track individual requests, but resources (weight, size) passing through the queue")]] + size_t requests_currently_executing() const { + return _requests_executing; + } + + // Dispatch requests that are pending in the I/O queue + void poll_io_queue(); + + clock_type::time_point next_pending_aio() const noexcept; + + sstring mountpoint() const; + dev_t dev_id() const noexcept; + + void update_shares_for_class(io_priority_class pc, size_t new_shares); + future<> update_bandwidth_for_class(io_priority_class pc, uint64_t new_bandwidth); + void rename_priority_class(io_priority_class pc, sstring new_name); + void throttle_priority_class(const priority_class_data& pc) noexcept; + void unthrottle_priority_class(const priority_class_data& pc) noexcept; + + struct request_limits { + size_t max_read; + size_t max_write; + }; + + request_limits get_request_limits() const noexcept; + const config& get_config() const noexcept; + +private: + static fair_queue::config make_fair_queue_config(const config& cfg, sstring label); + void register_stats(sstring name, priority_class_data& pc); +}; + +class io_group { +public: + explicit io_group(io_queue::config io_cfg); + ~io_group(); + struct priority_class_data; + +private: + friend class io_queue; + friend struct ::io_queue_for_tests; + + const io_queue::config _config; + size_t _max_request_length[2]; + std::vector<std::unique_ptr<fair_group>> _fgs; + std::vector<std::unique_ptr<priority_class_data>> _priority_classes; + util::spinlock _lock; + const shard_id _allocated_on; + + static fair_group::config make_fair_group_config(const io_queue::config& qcfg) noexcept; + priority_class_data& find_or_create_class(io_priority_class pc); +}; + +inline const io_queue::config& io_queue::get_config() const noexcept { + return _group->_config; +} + +inline sstring io_queue::mountpoint() const { + return get_config().mountpoint; +} + +inline dev_t io_queue::dev_id() const noexcept { + return get_config().devid; +} + +} diff --git a/src/seastar/include/seastar/core/iostream-impl.hh b/src/seastar/include/seastar/core/iostream-impl.hh new file mode 100644 index 000000000..1c2a9ef7a --- /dev/null +++ b/src/seastar/include/seastar/core/iostream-impl.hh @@ -0,0 +1,552 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ + + +#pragma once + +#include <seastar/core/do_with.hh> +#include <seastar/core/loop.hh> +#include <seastar/net/packet.hh> +#include <seastar/util/variant_utils.hh> + +namespace seastar { + +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::optional<temporary_buffer<char>> { + if (buffer.empty()) { + return buffer; + } + if (buffer.size() >= n) { + buffer.trim_front(n); + return buffer; + } + n -= buffer.size(); + return { }; + }); + }); + }); +} + +template<typename CharType> +inline +future<> output_stream<CharType>::write(const char_type* buf) noexcept { + return write(buf, strlen(buf)); +} + +template<typename CharType> +template<typename StringChar, typename SizeType, SizeType MaxSize, bool NulTerminate> +inline +future<> output_stream<CharType>::write(const basic_sstring<StringChar, SizeType, MaxSize, NulTerminate>& s) noexcept { + return write(reinterpret_cast<const CharType *>(s.c_str()), s.size()); +} + +template<typename CharType> +inline +future<> output_stream<CharType>::write(const std::basic_string<CharType>& s) noexcept { + return write(s.c_str(), s.size()); +} + +template<typename CharType> +future<> output_stream<CharType>::write(scattered_message<CharType> msg) noexcept { + return write(std::move(msg).release()); +} + +template<typename CharType> +future<> +output_stream<CharType>::zero_copy_put(net::packet p) noexcept { + // if flush is scheduled, disable it, so it will not try to write in parallel + _flush = false; + if (_flushing) { + // flush in progress, wait for it to end before continuing + return _in_batch.value().get_future().then([this, p = std::move(p)] () mutable { + return _fd.put(std::move(p)); + }); + } else { + return _fd.put(std::move(p)); + } +} + +// Writes @p in chunks of _size length. The last chunk is buffered if smaller. +template <typename CharType> +future<> +output_stream<CharType>::zero_copy_split_and_put(net::packet p) noexcept { + return repeat([this, p = std::move(p)] () mutable { + if (p.len() < _size) { + if (p.len()) { + _zc_bufs = std::move(p); + } else { + _zc_bufs = net::packet::make_null_packet(); + } + return make_ready_future<stop_iteration>(stop_iteration::yes); + } + auto chunk = p.share(0, _size); + p.trim_front(_size); + return zero_copy_put(std::move(chunk)).then([] { + return stop_iteration::no; + }); + }); +} + +template<typename CharType> +future<> output_stream<CharType>::write(net::packet p) noexcept { + static_assert(std::is_same<CharType, char>::value, "packet works on char"); + try { + if (p.len() != 0) { + assert(!_end && "Mixing buffered writes and zero-copy writes not supported yet"); + + if (_zc_bufs) { + _zc_bufs.append(std::move(p)); + } else { + _zc_bufs = std::move(p); + } + + if (_zc_bufs.len() >= _size) { + if (_trim_to_size) { + return zero_copy_split_and_put(std::move(_zc_bufs)); + } else { + return zero_copy_put(std::move(_zc_bufs)); + } + } + } + return make_ready_future<>(); + } catch (...) { + return current_exception_as_future(); + } +} + +template<typename CharType> +future<> output_stream<CharType>::write(temporary_buffer<CharType> p) noexcept { + try { + if (p.empty()) { + return make_ready_future<>(); + } + assert(!_end && "Mixing buffered writes and zero-copy writes not supported yet"); + return write(net::packet(std::move(p))); + } catch (...) { + return current_exception_as_future(); + } +} + +template <typename CharType> +future<temporary_buffer<CharType>> +input_stream<CharType>::read_exactly_part(size_t n, tmp_buf out, size_t completed) noexcept { + if (available()) { + auto now = std::min(n - completed, available()); + std::copy(_buf.get(), _buf.get() + now, out.get_write() + completed); + _buf.trim_front(now); + completed += now; + } + if (completed == n) { + return make_ready_future<tmp_buf>(std::move(out)); + } + + // _buf is now empty + return _fd.get().then([this, n, out = std::move(out), completed] (auto buf) mutable { + if (buf.size() == 0) { + _eof = true; + out.trim(completed); + return make_ready_future<tmp_buf>(std::move(out)); + } + _buf = std::move(buf); + return this->read_exactly_part(n, std::move(out), completed); + }); +} + +template <typename CharType> +future<temporary_buffer<CharType>> +input_stream<CharType>::read_exactly(size_t n) noexcept { + if (_buf.size() == n) { + // easy case: steal buffer, return to caller + return make_ready_future<tmp_buf>(std::move(_buf)); + } else if (_buf.size() > n) { + // buffer large enough, share it with caller + auto front = _buf.share(0, n); + _buf.trim_front(n); + return make_ready_future<tmp_buf>(std::move(front)); + } else if (_buf.size() == 0) { + // buffer is empty: grab one and retry + return _fd.get().then([this, n] (auto buf) mutable { + if (buf.size() == 0) { + _eof = true; + return make_ready_future<tmp_buf>(std::move(buf)); + } + _buf = std::move(buf); + return this->read_exactly(n); + }); + } else { + try { + // buffer too small: start copy/read loop + tmp_buf b(n); + return read_exactly_part(n, std::move(b), 0); + } catch (...) { + return current_exception_as_future<tmp_buf>(); + } + } +} + +template <typename CharType> +template <typename Consumer> +SEASTAR_CONCEPT(requires InputStreamConsumer<Consumer, CharType> || ObsoleteInputStreamConsumer<Consumer, CharType>) +future<> +input_stream<CharType>::consume(Consumer&& consumer) noexcept(std::is_nothrow_move_constructible_v<Consumer>) { + return repeat([consumer = std::move(consumer), this] () mutable { + if (_buf.empty() && !_eof) { + return _fd.get().then([this] (tmp_buf buf) { + _buf = std::move(buf); + _eof = _buf.empty(); + return make_ready_future<stop_iteration>(stop_iteration::no); + }); + } + return consumer(std::move(_buf)).then([this] (consumption_result_type result) { + 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. + // + // If we're at eof, we should stop. + return make_ready_future<stop_iteration>(stop_iteration(this->_eof)); + }, [this] (stop_consuming<CharType>& stop) { + // consumer is done + this->_buf = std::move(stop.get_buffer()); + return make_ready_future<stop_iteration>(stop_iteration::yes); + }, [this] (const skip_bytes& skip) { + return this->_fd.skip(skip.get_value()).then([this](tmp_buf buf) { + if (!buf.empty()) { + this->_buf = std::move(buf); + } + return make_ready_future<stop_iteration>(stop_iteration::no); + }); + }); + }); + }); +} + +template <typename CharType> +template <typename Consumer> +SEASTAR_CONCEPT(requires InputStreamConsumer<Consumer, CharType> || ObsoleteInputStreamConsumer<Consumer, CharType>) +future<> +input_stream<CharType>::consume(Consumer& consumer) noexcept(std::is_nothrow_move_constructible_v<Consumer>) { + return consume(std::ref(consumer)); +} + +template <typename CharType> +future<temporary_buffer<CharType>> +input_stream<CharType>::read_up_to(size_t n) noexcept { + using tmp_buf = temporary_buffer<CharType>; + if (_buf.empty()) { + if (_eof) { + return make_ready_future<tmp_buf>(); + } else { + return _fd.get().then([this, n] (tmp_buf buf) { + _eof = buf.empty(); + _buf = std::move(buf); + return read_up_to(n); + }); + } + } else if (_buf.size() <= n) { + // easy case: steal buffer, return to caller + return make_ready_future<tmp_buf>(std::move(_buf)); + } else { + try { + // buffer is larger than n, so share its head with a caller + auto front = _buf.share(0, n); + _buf.trim_front(n); + return make_ready_future<tmp_buf>(std::move(front)); + } catch (...) { + return current_exception_as_future<tmp_buf>(); + } + } +} + +template <typename CharType> +future<temporary_buffer<CharType>> +input_stream<CharType>::read() noexcept { + using tmp_buf = temporary_buffer<CharType>; + if (_eof) { + return make_ready_future<tmp_buf>(); + } + if (_buf.empty()) { + return _fd.get().then([this] (tmp_buf buf) { + _eof = buf.empty(); + return make_ready_future<tmp_buf>(std::move(buf)); + }); + } else { + return make_ready_future<tmp_buf>(std::move(_buf)); + } +} + +template <typename CharType> +future<> +input_stream<CharType>::skip(uint64_t n) noexcept { + auto skip_buf = std::min(n, _buf.size()); + _buf.trim_front(skip_buf); + n -= skip_buf; + if (!n) { + return make_ready_future<>(); + } + return _fd.skip(n).then([this] (temporary_buffer<CharType> buffer) { + _buf = std::move(buffer); + }); +} + +template <typename CharType> +data_source +input_stream<CharType>::detach() && { + if (_buf) { + throw std::logic_error("detach() called on a used input_stream"); + } + + return std::move(_fd); +} + +// Writes @buf in chunks of _size length. The last chunk is buffered if smaller. +template <typename CharType> +future<> +output_stream<CharType>::split_and_put(temporary_buffer<CharType> buf) noexcept { + assert(_end == 0); + + return repeat([this, buf = std::move(buf)] () mutable { + if (buf.size() < _size) { + if (!_buf) { + _buf = _fd.allocate_buffer(_size); + } + std::copy(buf.get(), buf.get() + buf.size(), _buf.get_write()); + _end = buf.size(); + return make_ready_future<stop_iteration>(stop_iteration::yes); + } + auto chunk = buf.share(0, _size); + buf.trim_front(_size); + return put(std::move(chunk)).then([] { + return stop_iteration::no; + }); + }); +} + +template <typename CharType> +future<> +output_stream<CharType>::write(const char_type* buf, size_t n) noexcept { + if (__builtin_expect(!_buf || n > _size - _end, false)) { + return slow_write(buf, n); + } + std::copy_n(buf, n, _buf.get_write() + _end); + _end += n; + return make_ready_future<>(); +} + +template <typename CharType> +future<> +output_stream<CharType>::slow_write(const char_type* buf, size_t n) noexcept { + try { + assert(!_zc_bufs && "Mixing buffered writes and zero-copy writes not supported yet"); + auto bulk_threshold = _end ? (2 * _size - _end) : _size; + if (n >= bulk_threshold) { + if (_end) { + auto now = _size - _end; + std::copy(buf, buf + now, _buf.get_write() + _end); + _end = _size; + temporary_buffer<char> tmp = _fd.allocate_buffer(n - now); + std::copy(buf + now, buf + n, tmp.get_write()); + _buf.trim(_end); + _end = 0; + return put(std::move(_buf)).then([this, tmp = std::move(tmp)]() mutable { + if (_trim_to_size) { + return split_and_put(std::move(tmp)); + } else { + return put(std::move(tmp)); + } + }); + } else { + temporary_buffer<char> tmp = _fd.allocate_buffer(n); + std::copy(buf, buf + n, tmp.get_write()); + if (_trim_to_size) { + return split_and_put(std::move(tmp)); + } else { + return put(std::move(tmp)); + } + } + } + + if (!_buf) { + _buf = _fd.allocate_buffer(_size); + } + + auto now = std::min(n, _size - _end); + std::copy(buf, buf + now, _buf.get_write() + _end); + _end += now; + if (now == n) { + return make_ready_future<>(); + } else { + temporary_buffer<char> next = _fd.allocate_buffer(_size); + std::copy(buf + now, buf + n, next.get_write()); + _end = n - now; + std::swap(next, _buf); + return put(std::move(next)); + } + } catch (...) { + return current_exception_as_future(); + } +} + +void add_to_flush_poller(output_stream<char>& x) noexcept; + +template <typename CharType> +future<> output_stream<CharType>::do_flush() noexcept { + if (_end) { + _buf.trim(_end); + _end = 0; + return _fd.put(std::move(_buf)).then([this] { + return _fd.flush(); + }); + } else if (_zc_bufs) { + return _fd.put(std::move(_zc_bufs)).then([this] { + return _fd.flush(); + }); + } else { + return make_ready_future<>(); + } +} + +template <typename CharType> +future<> +output_stream<CharType>::flush() noexcept { + if (!_batch_flushes) { + return do_flush(); + } else { + if (_ex) { + // flush is a good time to deliver outstanding errors + return make_exception_future<>(std::move(_ex)); + } else { + _flush = true; + if (!_in_batch) { + add_to_flush_poller(*this); + _in_batch = promise<>(); + } + } + } + return make_ready_future<>(); +} + +template <typename CharType> +future<> +output_stream<CharType>::put(temporary_buffer<CharType> buf) noexcept { + // if flush is scheduled, disable it, so it will not try to write in parallel + _flush = false; + if (_flushing) { + // flush in progress, wait for it to end before continuing + return _in_batch.value().get_future().then([this, buf = std::move(buf)] () mutable { + return _fd.put(std::move(buf)); + }); + } else { + return _fd.put(std::move(buf)); + } +} + +template <typename CharType> +void +output_stream<CharType>::poll_flush() noexcept { + if (!_flush) { + // flush was canceled, do nothing + _flushing = false; + _in_batch.value().set_value(); + _in_batch = std::nullopt; + return; + } + + _flush = false; + _flushing = true; // make whoever wants to write into the fd to wait for flush to complete + + // FIXME: future is discarded + (void)do_flush().then_wrapped([this] (future<> f) { + try { + f.get(); + } catch (...) { + _ex = std::current_exception(); + } + // if flush() was called while flushing flush once more + poll_flush(); + }); +} + +template <typename CharType> +future<> +output_stream<CharType>::close() noexcept { + return flush().finally([this] { + if (_in_batch) { + return _in_batch.value().get_future(); + } else { + return make_ready_future(); + } + }).then([this] { + // report final exception as close error + if (_ex) { + std::rethrow_exception(_ex); + } + }).finally([this] { + return _fd.close(); + }); +} + +template <typename CharType> +data_sink +output_stream<CharType>::detach() && { + if (_buf) { + throw std::logic_error("detach() called on a used output_stream"); + } + + return std::move(_fd); +} + +namespace internal { + +/// \cond internal +template <typename CharType> +struct stream_copy_consumer { +private: + output_stream<CharType>& _os; + using unconsumed_remainder = std::optional<temporary_buffer<CharType>>; +public: + stream_copy_consumer(output_stream<CharType>& os) : _os(os) { + } + future<unconsumed_remainder> operator()(temporary_buffer<CharType> data) { + if (data.empty()) { + return make_ready_future<unconsumed_remainder>(std::move(data)); + } + return _os.write(data.get(), data.size()).then([] () { + return make_ready_future<unconsumed_remainder>(); + }); + } +}; +/// \endcond + +} + +extern template struct internal::stream_copy_consumer<char>; + +template <typename CharType> +future<> copy(input_stream<CharType>& in, output_stream<CharType>& out) { + return in.consume(internal::stream_copy_consumer<CharType>(out)); +} + +extern template future<> copy<char>(input_stream<char>&, output_stream<char>&); +} diff --git a/src/seastar/include/seastar/core/iostream.hh b/src/seastar/include/seastar/core/iostream.hh new file mode 100644 index 000000000..55f761968 --- /dev/null +++ b/src/seastar/include/seastar/core/iostream.hh @@ -0,0 +1,445 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ + +// +// Buffered input and output streams +// +// Two abstract classes (data_source and data_sink) provide means +// to acquire bulk data from, or push bulk data to, some provider. +// These could be tied to a TCP connection, a disk file, or a memory +// buffer. +// +// Two concrete classes (input_stream and output_stream) buffer data +// from data_source and data_sink and provide easier means to process +// it. +// + +#pragma once + +#include <boost/intrusive/slist.hpp> +#include <seastar/core/future.hh> +#include <seastar/core/temporary_buffer.hh> +#include <seastar/core/scattered_message.hh> +#include <seastar/util/std-compat.hh> + +namespace bi = boost::intrusive; + +namespace seastar { + +namespace net { class packet; } + +class data_source_impl { +public: + virtual ~data_source_impl() {} + virtual future<temporary_buffer<char>> get() = 0; + virtual future<temporary_buffer<char>> skip(uint64_t n); + virtual future<> close() { return make_ready_future<>(); } +}; + +class data_source { + std::unique_ptr<data_source_impl> _dsi; +protected: + data_source_impl* impl() const { return _dsi.get(); } +public: + using tmp_buf = temporary_buffer<char>; + + data_source() noexcept = default; + explicit data_source(std::unique_ptr<data_source_impl> dsi) noexcept : _dsi(std::move(dsi)) {} + data_source(data_source&& x) noexcept = default; + data_source& operator=(data_source&& x) noexcept = default; + + future<tmp_buf> get() noexcept { + try { + return _dsi->get(); + } catch (...) { + return current_exception_as_future<tmp_buf>(); + } + } + future<tmp_buf> skip(uint64_t n) noexcept { + try { + return _dsi->skip(n); + } catch (...) { + return current_exception_as_future<tmp_buf>(); + } + } + future<> close() noexcept { + try { + return _dsi->close(); + } catch (...) { + return current_exception_as_future<>(); + } + } +}; + +class data_sink_impl { +public: + virtual ~data_sink_impl() {} + virtual temporary_buffer<char> allocate_buffer(size_t size) { + return temporary_buffer<char>(size); + } + virtual future<> put(net::packet data) = 0; + virtual future<> put(std::vector<temporary_buffer<char>> data) { + net::packet p; + p.reserve(data.size()); + for (auto& buf : data) { + p = net::packet(std::move(p), net::fragment{buf.get_write(), buf.size()}, buf.release()); + } + return put(std::move(p)); + } + virtual future<> put(temporary_buffer<char> buf) { + return put(net::packet(net::fragment{buf.get_write(), buf.size()}, buf.release())); + } + virtual future<> flush() { + return make_ready_future<>(); + } + virtual future<> close() = 0; + + // The method should return the maximum buffer size that's acceptable by + // the sink. It's used when the output stream is constructed without any + // specific buffer size. In this case the stream accepts this value as its + // buffer size and doesn't put larger buffers (see trim_to_size). + virtual size_t buffer_size() const noexcept { + assert(false && "Data sink must have the buffer_size() method overload"); + return 0; + } +}; + +class data_sink { + std::unique_ptr<data_sink_impl> _dsi; +public: + data_sink() noexcept = default; + explicit data_sink(std::unique_ptr<data_sink_impl> dsi) noexcept : _dsi(std::move(dsi)) {} + data_sink(data_sink&& x) noexcept = default; + data_sink& operator=(data_sink&& x) noexcept = default; + temporary_buffer<char> allocate_buffer(size_t size) { + return _dsi->allocate_buffer(size); + } + future<> put(std::vector<temporary_buffer<char>> data) noexcept { + try { + return _dsi->put(std::move(data)); + } catch (...) { + return current_exception_as_future(); + } + } + future<> put(temporary_buffer<char> data) noexcept { + try { + return _dsi->put(std::move(data)); + } catch (...) { + return current_exception_as_future(); + } + } + future<> put(net::packet p) noexcept { + try { + return _dsi->put(std::move(p)); + } catch (...) { + return current_exception_as_future(); + } + } + future<> flush() noexcept { + try { + return _dsi->flush(); + } catch (...) { + return current_exception_as_future(); + } + } + future<> close() noexcept { + try { + return _dsi->close(); + } catch (...) { + return current_exception_as_future(); + } + } + + size_t buffer_size() const noexcept { return _dsi->buffer_size(); } +}; + +struct continue_consuming {}; + +template <typename CharType> +class stop_consuming { +public: + using tmp_buf = temporary_buffer<CharType>; + explicit stop_consuming(tmp_buf buf) : _buf(std::move(buf)) {} + + tmp_buf& get_buffer() { return _buf; } + const tmp_buf& get_buffer() const { return _buf; } +private: + tmp_buf _buf; +}; + +class skip_bytes { +public: + explicit skip_bytes(uint64_t v) : _value(v) {} + uint64_t get_value() const { return _value; } +private: + uint64_t _value; +}; + +template <typename CharType> +class consumption_result { +public: + using stop_consuming_type = stop_consuming<CharType>; + using consumption_variant = std::variant<continue_consuming, stop_consuming_type, skip_bytes>; + using tmp_buf = typename stop_consuming_type::tmp_buf; + + /*[[deprecated]]*/ consumption_result(std::optional<tmp_buf> opt_buf) { + if (opt_buf) { + _result = stop_consuming_type{std::move(opt_buf.value())}; + } + } + + consumption_result(const continue_consuming&) {} + consumption_result(stop_consuming_type&& stop) : _result(std::move(stop)) {} + consumption_result(skip_bytes&& skip) : _result(std::move(skip)) {} + + consumption_variant& get() { return _result; } + const consumption_variant& get() const { return _result; } + +private: + consumption_variant _result; +}; + +// Consumer concept, for consume() method +SEASTAR_CONCEPT( +// The consumer should operate on the data given to it, and +// return a future "consumption result", which can be +// - continue_consuming, if the consumer has consumed all the input given +// to it and is ready for more +// - stop_consuming, when the consumer is done (and in that case +// the contained buffer is the unconsumed part of the last data buffer - this +// can also happen to be empty). +// - skip_bytes, when the consumer has consumed all the input given to it +// and wants to skip before processing the next chunk +// +// For backward compatibility reasons, we also support the deprecated return value +// of type "unconsumed remainder" which can be +// - empty optional, if the consumer consumed all the input given to it +// and is ready for more +// - non-empty optional, when the consumer is done (and in that case +// the value is the unconsumed part of the last data buffer - this +// can also happen to be empty). + +template <typename Consumer, typename CharType> +concept InputStreamConsumer = requires (Consumer c) { + { c(temporary_buffer<CharType>{}) } -> std::same_as<future<consumption_result<CharType>>>; +}; + +template <typename Consumer, typename CharType> +concept ObsoleteInputStreamConsumer = requires (Consumer c) { + { c(temporary_buffer<CharType>{}) } -> std::same_as<future<std::optional<temporary_buffer<CharType>>>>; +}; +) + +/// Buffers data from a data_source and provides a stream interface to the user. +/// +/// \note All methods must be called sequentially. That is, no method may be +/// invoked before the previous method's returned future is resolved. +template <typename CharType> +class input_stream final { + static_assert(sizeof(CharType) == 1, "must buffer stream of bytes"); + data_source _fd; + temporary_buffer<CharType> _buf; + bool _eof = false; +private: + using tmp_buf = temporary_buffer<CharType>; + size_t available() const noexcept { return _buf.size(); } +protected: + void reset() noexcept { _buf = {}; } + data_source* fd() noexcept { return &_fd; } +public: + using consumption_result_type = consumption_result<CharType>; + // unconsumed_remainder is mapped for compatibility only; new code should use consumption_result_type + using unconsumed_remainder = std::optional<tmp_buf>; + using char_type = CharType; + input_stream() noexcept = default; + explicit input_stream(data_source fd) noexcept : _fd(std::move(fd)), _buf() {} + input_stream(input_stream&&) = default; + input_stream& operator=(input_stream&&) = default; + /// Reads n bytes from the stream, or fewer if reached the end of stream. + /// + /// \returns a future that waits until n bytes are available in the + /// stream and returns them. If the end of stream is reached before n + /// bytes were read, fewer than n bytes will be returned - so despite + /// the method's name, the caller must not assume the returned buffer + /// will always contain exactly n bytes. + /// + /// \throws if an I/O error occurs during the read. As explained above, + /// prematurely reaching the end of stream is *not* an I/O error. + future<temporary_buffer<CharType>> read_exactly(size_t n) noexcept; + template <typename Consumer> + SEASTAR_CONCEPT(requires InputStreamConsumer<Consumer, CharType> || ObsoleteInputStreamConsumer<Consumer, CharType>) + future<> consume(Consumer&& c) noexcept(std::is_nothrow_move_constructible_v<Consumer>); + template <typename Consumer> + SEASTAR_CONCEPT(requires InputStreamConsumer<Consumer, CharType> || ObsoleteInputStreamConsumer<Consumer, CharType>) + future<> consume(Consumer& c) noexcept(std::is_nothrow_move_constructible_v<Consumer>); + /// Returns true if the end-of-file flag is set on the stream. + /// Note that the eof flag is only set after a previous attempt to read + /// from the stream noticed the end of the stream. In other words, it is + /// possible that eof() returns false but read() will return an empty + /// buffer. Checking eof() again after the read will return true. + bool eof() const noexcept { return _eof; } + /// Returns some data from the stream, or an empty buffer on end of + /// stream. + future<tmp_buf> read() noexcept; + /// Returns up to n bytes from the stream, or an empty buffer on end of + /// stream. + future<tmp_buf> read_up_to(size_t n) noexcept; + /// Detaches the \c input_stream from the underlying data source. + /// + /// Waits for any background operations (for example, read-ahead) to + /// complete, so that the any resources the stream is using can be + /// safely destroyed. An example is a \ref file resource used by + /// the stream returned by make_file_input_stream(). + /// + /// \return a future that becomes ready when this stream no longer + /// needs the data source. + future<> close() noexcept { + return _fd.close(); + } + /// Ignores n next bytes from the stream. + future<> skip(uint64_t n) noexcept; + + /// Detaches the underlying \c data_source from the \c input_stream. + /// + /// The intended usage is custom \c data_source_impl implementations + /// wrapping an existing \c input_stream, therefore it shouldn't be + /// called on an \c input_stream that was already used. + /// After calling \c detach() the \c input_stream is in an unusable, + /// moved-from state. + /// + /// \throws std::logic_error if called on a used stream + /// + /// \returns the data_source + data_source detach() &&; +private: + future<temporary_buffer<CharType>> read_exactly_part(size_t n, tmp_buf buf, size_t completed) noexcept; +}; + +struct output_stream_options { + bool trim_to_size = false; ///< Make sure that buffers put into sink haven't + ///< grown larger than the configured size + bool batch_flushes = false; ///< Try to merge flushes with each other +}; + +/// Facilitates data buffering before it's handed over to data_sink. +/// +/// When trim_to_size is true it's guaranteed that data sink will not receive +/// chunks larger than the configured size, which could be the case when a +/// single write call is made with data larger than the configured size. +/// +/// The data sink will not receive empty chunks. +/// +/// \note All methods must be called sequentially. That is, no method +/// may be invoked before the previous method's returned future is +/// resolved. +template <typename CharType> +class output_stream final { + static_assert(sizeof(CharType) == 1, "must buffer stream of bytes"); + data_sink _fd; + temporary_buffer<CharType> _buf; + net::packet _zc_bufs = net::packet::make_null_packet(); //zero copy buffers + size_t _size = 0; + size_t _begin = 0; + size_t _end = 0; + bool _trim_to_size = false; + bool _batch_flushes = false; + std::optional<promise<>> _in_batch; + bool _flush = false; + bool _flushing = false; + std::exception_ptr _ex; + bi::slist_member_hook<> _in_poller; + +private: + size_t available() const noexcept { return _end - _begin; } + future<> split_and_put(temporary_buffer<CharType> buf) noexcept; + future<> put(temporary_buffer<CharType> buf) noexcept; + void poll_flush() noexcept; + future<> do_flush() noexcept; + future<> zero_copy_put(net::packet p) noexcept; + future<> zero_copy_split_and_put(net::packet p) noexcept; + [[gnu::noinline]] + future<> slow_write(const CharType* buf, size_t n) noexcept; +public: + using char_type = CharType; + output_stream() noexcept = default; + output_stream(data_sink fd, size_t size, output_stream_options opts = {}) noexcept + : _fd(std::move(fd)), _size(size), _trim_to_size(opts.trim_to_size), _batch_flushes(opts.batch_flushes) {} + [[deprecated("use output_stream_options instead of booleans")]] + output_stream(data_sink fd, size_t size, bool trim_to_size, bool batch_flushes = false) noexcept + : _fd(std::move(fd)), _size(size), _trim_to_size(trim_to_size), _batch_flushes(batch_flushes) {} + output_stream(data_sink fd) noexcept + : _fd(std::move(fd)), _size(_fd.buffer_size()), _trim_to_size(true) {} + output_stream(output_stream&&) noexcept = default; + output_stream& operator=(output_stream&&) noexcept = default; + ~output_stream() { + if (_batch_flushes) { + assert(!_in_batch && "Was this stream properly closed?"); + } else { + assert(!_end && !_zc_bufs && "Was this stream properly closed?"); + } + } + future<> write(const char_type* buf, size_t n) noexcept; + future<> write(const char_type* buf) noexcept; + + template <typename StringChar, typename SizeType, SizeType MaxSize, bool NulTerminate> + future<> write(const basic_sstring<StringChar, SizeType, MaxSize, NulTerminate>& s) noexcept; + future<> write(const std::basic_string<char_type>& s) noexcept; + + future<> write(net::packet p) noexcept; + future<> write(scattered_message<char_type> msg) noexcept; + future<> write(temporary_buffer<char_type>) noexcept; + future<> flush() noexcept; + + /// Flushes the stream before closing it (and the underlying data sink) to + /// any further writes. The resulting future must be waited on before + /// destroying this object. + future<> close() noexcept; + + /// Detaches the underlying \c data_sink from the \c output_stream. + /// + /// The intended usage is custom \c data_sink_impl implementations + /// wrapping an existing \c output_stream, therefore it shouldn't be + /// called on an \c output_stream that was already used. + /// After calling \c detach() the \c output_stream is in an unusable, + /// moved-from state. + /// + /// \throws std::logic_error if called on a used stream + /// + /// \returns the data_sink + data_sink detach() &&; + + using batch_flush_list_t = bi::slist<output_stream, + bi::constant_time_size<false>, bi::cache_last<true>, + bi::member_hook<output_stream, bi::slist_member_hook<>, &output_stream::_in_poller>>; +private: + friend class reactor; +}; + +/*! + * \brief copy all the content from the input stream to the output stream + */ +template <typename CharType> +future<> copy(input_stream<CharType>&, output_stream<CharType>&); + +} + +#include "iostream-impl.hh" diff --git a/src/seastar/include/seastar/core/layered_file.hh b/src/seastar/include/seastar/core/layered_file.hh new file mode 100644 index 000000000..f59465aec --- /dev/null +++ b/src/seastar/include/seastar/core/layered_file.hh @@ -0,0 +1,68 @@ +/* + * 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 2020 ScyllaDB + */ + +#pragma once + +#include <seastar/core/file.hh> + +namespace seastar { + +/// \addtogroup fileio-module +/// @{ + +/// Base class for layered file implementations. +/// +/// A layered file implementation implements `file_impl` virtual +/// functions such as dma_read() by forwarding them to another, existing +/// file called the underlying file. This base class simplifies construction +/// of layered files by performing standard tasks such as setting up the +/// file alignment. Actual implementation of the I/O methods is left for the +/// derived class. +class layered_file_impl : public file_impl { +protected: + file _underlying_file; +public: + /// Constructs a layered file. This sets up the underlying_file() method + /// and initializes alignment constants to be the same as the underlying file. + explicit layered_file_impl(file underlying_file) noexcept + : _underlying_file(std::move(underlying_file)) { + _memory_dma_alignment = _underlying_file.memory_dma_alignment(); + _disk_read_dma_alignment = _underlying_file.disk_read_dma_alignment(); + _disk_write_dma_alignment = _underlying_file.disk_write_dma_alignment(); + _disk_overwrite_dma_alignment = _underlying_file.disk_overwrite_dma_alignment(); + } + + /// The underlying file which can be used to back I/O methods. + file& underlying_file() noexcept { + return _underlying_file; + } + + /// The underlying file which can be used to back I/O methods. + const file& underlying_file() const noexcept { + return _underlying_file; + } +}; + + +/// @} + + +} diff --git a/src/seastar/include/seastar/core/linux-aio.hh b/src/seastar/include/seastar/core/linux-aio.hh new file mode 100644 index 000000000..88d90ef9a --- /dev/null +++ b/src/seastar/include/seastar/core/linux-aio.hh @@ -0,0 +1,237 @@ +/* + * 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) 2017 ScyllaDB + */ + +#pragma once + +#include <endian.h> +#include <sys/time.h> +#include <sys/uio.h> +#include <signal.h> +#include <cstdint> + +namespace seastar { + +namespace internal { + +namespace linux_abi { + +using aio_context_t = unsigned long; + +enum class iocb_cmd : uint16_t { + PREAD = 0, + PWRITE = 1, + FSYNC = 2, + FDSYNC = 3, + POLL = 5, + NOOP = 6, + PREADV = 7, + PWRITEV = 8, +}; + +struct io_event { + uint64_t data; + uint64_t obj; + int64_t res; + int64_t res2; +}; + +constexpr int IOCB_FLAG_RESFD = 1; + +struct iocb { + uint64_t aio_data; + +#if __BYTE_ORDER == __LITTLE_ENDIAN + uint32_t aio_key; + int32_t aio_rw_flags; +#elif __BYTE_ORDER == __BIG_ENDIAN + int32_t aio_rw_flags; + uint32_t aio_key; +#else +#error bad byteorder +#endif + + iocb_cmd aio_lio_opcode; + int16_t aio_reqprio; + uint32_t aio_fildes; + + uint64_t aio_buf; + uint64_t aio_nbytes; + int64_t aio_offset; + + uint64_t aio_reserved2; + + uint32_t aio_flags; + + uint32_t aio_resfd; +}; + +struct aio_sigset { + const sigset_t *sigmask; + size_t sigsetsize; +}; + +} + +linux_abi::iocb make_read_iocb(int fd, uint64_t offset, void* buffer, size_t len); +linux_abi::iocb make_write_iocb(int fd, uint64_t offset, const void* buffer, size_t len); +linux_abi::iocb make_readv_iocb(int fd, uint64_t offset, const ::iovec* iov, size_t niov); +linux_abi::iocb make_writev_iocb(int fd, uint64_t offset, const ::iovec* iov, size_t niov); +linux_abi::iocb make_poll_iocb(int fd, uint32_t events); + +void set_user_data(linux_abi::iocb& iocb, void* data); +void set_nowait(linux_abi::iocb& iocb, bool nowait); + +void set_eventfd_notification(linux_abi::iocb& iocb, int eventfd); + +linux_abi::iocb* get_iocb(const linux_abi::io_event& ioev); + +int io_setup(int nr_events, linux_abi::aio_context_t* io_context); +int io_destroy(linux_abi::aio_context_t io_context) noexcept; +int io_submit(linux_abi::aio_context_t io_context, long nr, linux_abi::iocb** iocbs); +int io_cancel(linux_abi::aio_context_t io_context, linux_abi::iocb* iocb, linux_abi::io_event* result); +int io_getevents(linux_abi::aio_context_t io_context, long min_nr, long nr, linux_abi::io_event* events, const ::timespec* timeout, + bool force_syscall = false); +int io_pgetevents(linux_abi::aio_context_t io_context, long min_nr, long nr, linux_abi::io_event* events, const ::timespec* timeout, const sigset_t* sigmask, + bool force_syscall = false); + +void setup_aio_context(size_t nr, linux_abi::aio_context_t* io_context); + +} + +extern bool aio_nowait_supported; + +namespace internal { + +inline +linux_abi::iocb +make_read_iocb(int fd, uint64_t offset, void* buffer, size_t len) { + linux_abi::iocb iocb{}; + iocb.aio_lio_opcode = linux_abi::iocb_cmd::PREAD; + iocb.aio_fildes = fd; + iocb.aio_offset = offset; + iocb.aio_buf = reinterpret_cast<uintptr_t>(buffer); + iocb.aio_nbytes = len; + return iocb; +} + +inline +linux_abi::iocb +make_write_iocb(int fd, uint64_t offset, const void* buffer, size_t len) { + linux_abi::iocb iocb{}; + iocb.aio_lio_opcode = linux_abi::iocb_cmd::PWRITE; + iocb.aio_fildes = fd; + iocb.aio_offset = offset; + iocb.aio_buf = reinterpret_cast<uintptr_t>(buffer); + iocb.aio_nbytes = len; + return iocb; +} + +inline +linux_abi::iocb +make_readv_iocb(int fd, uint64_t offset, const ::iovec* iov, size_t niov) { + linux_abi::iocb iocb{}; + iocb.aio_lio_opcode = linux_abi::iocb_cmd::PREADV; + iocb.aio_fildes = fd; + iocb.aio_offset = offset; + iocb.aio_buf = reinterpret_cast<uintptr_t>(iov); + iocb.aio_nbytes = niov; + return iocb; +} + +inline +linux_abi::iocb +make_writev_iocb(int fd, uint64_t offset, const ::iovec* iov, size_t niov) { + linux_abi::iocb iocb{}; + iocb.aio_lio_opcode = linux_abi::iocb_cmd::PWRITEV; + iocb.aio_fildes = fd; + iocb.aio_offset = offset; + iocb.aio_buf = reinterpret_cast<uintptr_t>(iov); + iocb.aio_nbytes = niov; + return iocb; +} + +inline +linux_abi::iocb +make_poll_iocb(int fd, uint32_t events) { + linux_abi::iocb iocb{}; + iocb.aio_lio_opcode = linux_abi::iocb_cmd::POLL; + iocb.aio_fildes = fd; + iocb.aio_buf = events; + return iocb; +} + +inline +linux_abi::iocb +make_fdsync_iocb(int fd) { + linux_abi::iocb iocb{}; + iocb.aio_lio_opcode = linux_abi::iocb_cmd::FDSYNC; + iocb.aio_fildes = fd; + return iocb; +} + +inline +void +set_user_data(linux_abi::iocb& iocb, void* data) { + iocb.aio_data = reinterpret_cast<uintptr_t>(data); +} + +template <typename T> +inline T* get_user_data(const linux_abi::iocb& iocb) noexcept { + return reinterpret_cast<T*>(uintptr_t(iocb.aio_data)); +} + +template <typename T> +inline T* get_user_data(const linux_abi::io_event& ev) noexcept { + return reinterpret_cast<T*>(uintptr_t(ev.data)); +} + +inline +void +set_eventfd_notification(linux_abi::iocb& iocb, int eventfd) { + iocb.aio_flags |= linux_abi::IOCB_FLAG_RESFD; + iocb.aio_resfd = eventfd; +} + +inline +linux_abi::iocb* +get_iocb(const linux_abi::io_event& ev) { + return reinterpret_cast<linux_abi::iocb*>(uintptr_t(ev.obj)); +} + +inline +void +set_nowait(linux_abi::iocb& iocb, bool nowait) { +#ifdef RWF_NOWAIT + if (aio_nowait_supported) { + if (nowait) { + iocb.aio_rw_flags |= RWF_NOWAIT; + } else { + iocb.aio_rw_flags &= ~RWF_NOWAIT; + } + } +#endif +} + +} + + +} + diff --git a/src/seastar/include/seastar/core/loop.hh b/src/seastar/include/seastar/core/loop.hh new file mode 100644 index 000000000..35d808b95 --- /dev/null +++ b/src/seastar/include/seastar/core/loop.hh @@ -0,0 +1,759 @@ +/* + * 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) 2020 ScyllaDB. + */ + +#pragma once + +#include <cstddef> +#include <iterator> +#include <memory> +#include <type_traits> +#include <vector> + +#include <seastar/core/future.hh> +#include <seastar/core/task.hh> +#include <seastar/util/bool_class.hh> +#include <seastar/core/semaphore.hh> + +namespace seastar { + +/// \addtogroup future-util +/// @{ + +// The AsyncAction concept represents an action which can complete later than +// the actual function invocation. It is represented by a function which +// returns a future which resolves when the action is done. + +struct stop_iteration_tag { }; +using stop_iteration = bool_class<stop_iteration_tag>; + +namespace internal { + +template <typename AsyncAction> +class repeater final : public continuation_base<stop_iteration> { + promise<> _promise; + AsyncAction _action; +public: + explicit repeater(AsyncAction&& action) : _action(std::move(action)) {} + future<> get_future() { return _promise.get_future(); } + task* waiting_task() noexcept override { return _promise.waiting_task(); } + virtual void run_and_dispose() noexcept override { + if (_state.failed()) { + _promise.set_exception(std::move(_state).get_exception()); + delete this; + return; + } else { + if (_state.get0() == stop_iteration::yes) { + _promise.set_value(); + delete this; + return; + } + _state = {}; + } + try { + do { + auto f = futurize_invoke(_action); + if (!f.available()) { + internal::set_callback(std::move(f), this); + return; + } + if (f.get0() == stop_iteration::yes) { + _promise.set_value(); + delete this; + return; + } + } while (!need_preempt()); + } catch (...) { + _promise.set_exception(std::current_exception()); + delete this; + return; + } + _state.set(stop_iteration::no); + schedule(this); + } +}; + +} // namespace internal + +// Delete these overloads so that the actual implementation can use a +// universal reference but still reject lvalue references. +template<typename AsyncAction> +future<> repeat(const AsyncAction& action) noexcept = delete; +template<typename AsyncAction> +future<> repeat(AsyncAction& action) noexcept = delete; + +/// Invokes given action until it fails or the function requests iteration to stop by returning +/// \c stop_iteration::yes. +/// +/// \param action a callable taking no arguments, returning a future<stop_iteration>. Will +/// be called again as soon as the future resolves, unless the +/// future fails, action throws, or it resolves with \c stop_iteration::yes. +/// If \c action is an r-value it can be moved in the middle of iteration. +/// \return a ready future if we stopped successfully, or a failed future if +/// a call to to \c action failed. +template<typename AsyncAction> +SEASTAR_CONCEPT( requires seastar::InvokeReturns<AsyncAction, stop_iteration> || seastar::InvokeReturns<AsyncAction, future<stop_iteration>> ) +inline +future<> repeat(AsyncAction&& action) noexcept { + using futurator = futurize<std::invoke_result_t<AsyncAction>>; + static_assert(std::is_same<future<stop_iteration>, typename futurator::type>::value, "bad AsyncAction signature"); + for (;;) { + // Do not type-erase here in case this is a short repeat() + auto f = futurator::invoke(action); + + if (!f.available() || f.failed() || need_preempt()) { + return [&] () noexcept { + memory::scoped_critical_alloc_section _; + auto repeater = new internal::repeater<AsyncAction>(std::move(action)); + auto ret = repeater->get_future(); + internal::set_callback(std::move(f), repeater); + return ret; + }(); + } + + if (f.get0() == stop_iteration::yes) { + return make_ready_future<>(); + } + } +} + +/// \cond internal + +template <typename T> +struct repeat_until_value_type_helper; + +/// Type helper for repeat_until_value() +template <typename T> +struct repeat_until_value_type_helper<future<std::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::optional<T>; + /// Return type of repeat_until_value() + using future_type = future<value_type>; +}; + +/// Return value of repeat_until_value() +template <typename AsyncAction> +using repeat_until_value_return_type + = typename repeat_until_value_type_helper<typename futurize<std::invoke_result_t<AsyncAction>>::type>::future_type; + +/// \endcond + +namespace internal { + +template <typename AsyncAction, typename T> +class repeat_until_value_state final : public continuation_base<std::optional<T>> { + promise<T> _promise; + AsyncAction _action; +public: + explicit repeat_until_value_state(AsyncAction action) : _action(std::move(action)) {} + repeat_until_value_state(std::optional<T> st, AsyncAction action) : repeat_until_value_state(std::move(action)) { + this->_state.set(std::move(st)); + } + future<T> get_future() { return _promise.get_future(); } + task* waiting_task() noexcept override { return _promise.waiting_task(); } + virtual void run_and_dispose() noexcept override { + if (this->_state.failed()) { + _promise.set_exception(std::move(this->_state).get_exception()); + delete this; + return; + } else { + auto v = std::move(this->_state).get0(); + if (v) { + _promise.set_value(std::move(*v)); + delete this; + return; + } + this->_state = {}; + } + try { + do { + auto f = futurize_invoke(_action); + if (!f.available()) { + internal::set_callback(std::move(f), this); + return; + } + auto ret = f.get0(); + if (ret) { + _promise.set_value(std::move(*ret)); + delete this; + return; + } + } while (!need_preempt()); + } catch (...) { + _promise.set_exception(std::current_exception()); + delete this; + return; + } + this->_state.set(std::nullopt); + schedule(this); + } +}; + +} // namespace internal + +/// Invokes given action until it fails or the function requests iteration to stop by returning +/// an engaged \c future<std::optional<T>> or std::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::optional<T>> +/// or std::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. +/// \return a ready future if we stopped successfully, or a failed future if +/// a call to to \c action failed. The \c optional's value is returned. +template<typename AsyncAction> +SEASTAR_CONCEPT( requires requires (AsyncAction aa) { + bool(futurize_invoke(aa).get0()); + futurize_invoke(aa).get0().value(); +} ) +repeat_until_value_return_type<AsyncAction> +repeat_until_value(AsyncAction action) noexcept { + using futurator = futurize<std::invoke_result_t<AsyncAction>>; + using type_helper = repeat_until_value_type_helper<typename futurator::type>; + // the "T" in the documentation + using value_type = typename type_helper::value_type; + using optional_type = typename type_helper::optional_type; + do { + auto f = futurator::invoke(action); + + if (!f.available()) { + return [&] () noexcept { + memory::scoped_critical_alloc_section _; + auto state = new internal::repeat_until_value_state<AsyncAction, value_type>(std::move(action)); + auto ret = state->get_future(); + internal::set_callback(std::move(f), state); + return ret; + }(); + } + + if (f.failed()) { + return make_exception_future<value_type>(f.get_exception()); + } + + optional_type&& optional = std::move(f).get0(); + if (optional) { + return make_ready_future<value_type>(std::move(optional.value())); + } + } while (!need_preempt()); + + try { + auto state = new internal::repeat_until_value_state<AsyncAction, value_type>(std::nullopt, std::move(action)); + auto f = state->get_future(); + schedule(state); + return f; + } catch (...) { + return make_exception_future<value_type>(std::current_exception()); + } +} + +namespace internal { + +template <typename StopCondition, typename AsyncAction> +class do_until_state final : public continuation_base<> { + promise<> _promise; + StopCondition _stop; + AsyncAction _action; +public: + explicit do_until_state(StopCondition stop, AsyncAction action) : _stop(std::move(stop)), _action(std::move(action)) {} + future<> get_future() { return _promise.get_future(); } + task* waiting_task() noexcept override { return _promise.waiting_task(); } + virtual void run_and_dispose() noexcept override { + if (_state.available()) { + if (_state.failed()) { + _promise.set_urgent_state(std::move(_state)); + delete this; + return; + } + _state = {}; // allow next cycle to overrun state + } + try { + do { + if (_stop()) { + _promise.set_value(); + delete this; + return; + } + auto f = _action(); + if (!f.available()) { + internal::set_callback(std::move(f), this); + return; + } + if (f.failed()) { + f.forward_to(std::move(_promise)); + delete this; + return; + } + } while (!need_preempt()); + } catch (...) { + _promise.set_exception(std::current_exception()); + delete this; + return; + } + schedule(this); + } +}; + +} // namespace internal + +/// Invokes given action until it fails or given condition evaluates to true or fails. +/// +/// \param stop_cond a callable taking no arguments, returning a boolean that +/// evalutes to true when you don't want to call \c action +/// any longer. If \c stop_cond fails, the exception is propagated +// in the returned 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, or \c stop_cond returns \c true or fails. +/// \return a ready future if we stopped successfully, or a failed future if +/// a call to to \c action or a call to \c stop_cond failed. +template<typename AsyncAction, typename StopCondition> +SEASTAR_CONCEPT( requires seastar::InvokeReturns<StopCondition, bool> && seastar::InvokeReturns<AsyncAction, future<>> ) +inline +future<> do_until(StopCondition stop_cond, AsyncAction action) noexcept { + using namespace internal; + for (;;) { + try { + if (stop_cond()) { + return make_ready_future<>(); + } + } catch (...) { + return current_exception_as_future(); + } + auto f = futurize_invoke(action); + if (f.failed()) { + return f; + } + if (!f.available() || need_preempt()) { + return [&] () noexcept { + memory::scoped_critical_alloc_section _; + auto task = new do_until_state<StopCondition, AsyncAction>(std::move(stop_cond), std::move(action)); + auto ret = task->get_future(); + internal::set_callback(std::move(f), task); + return ret; + }(); + } + } +} + +/// Invoke given action until it fails. +/// +/// Calls \c action repeatedly until it returns a failed future. +/// +/// \param action a callable taking no arguments, returning a \c future<> +/// that becomes ready when you wish it to be called again. +/// \return a future<> that will resolve to the first failure of \c action +template<typename AsyncAction> +SEASTAR_CONCEPT( requires seastar::InvokeReturns<AsyncAction, future<>> ) +inline +future<> keep_doing(AsyncAction action) noexcept { + return repeat([action = std::move(action)] () mutable { + return action().then([] { + return stop_iteration::no; + }); + }); +} + +namespace internal { +template <typename Iterator, typename AsyncAction> +class do_for_each_state final : public continuation_base<> { + Iterator _begin; + Iterator _end; + AsyncAction _action; + promise<> _pr; + +public: + do_for_each_state(Iterator begin, Iterator end, AsyncAction action, future<>&& first_unavailable) + : _begin(std::move(begin)), _end(std::move(end)), _action(std::move(action)) { + internal::set_callback(std::move(first_unavailable), this); + } + virtual void run_and_dispose() noexcept override { + std::unique_ptr<do_for_each_state> zis(this); + if (_state.failed()) { + _pr.set_urgent_state(std::move(_state)); + return; + } + while (_begin != _end) { + auto f = futurize_invoke(_action, *_begin++); + if (f.failed()) { + f.forward_to(std::move(_pr)); + return; + } + if (!f.available() || need_preempt()) { + _state = {}; + internal::set_callback(std::move(f), this); + zis.release(); + return; + } + } + _pr.set_value(); + } + task* waiting_task() noexcept override { + return _pr.waiting_task(); + } + future<> get_future() { + return _pr.get_future(); + } +}; + +template<typename Iterator, typename AsyncAction> +inline +future<> do_for_each_impl(Iterator begin, Iterator end, AsyncAction action) { + while (begin != end) { + auto f = futurize_invoke(action, *begin++); + if (f.failed()) { + return f; + } + if (!f.available() || need_preempt()) { + auto* s = new internal::do_for_each_state<Iterator, AsyncAction>{ + std::move(begin), std::move(end), std::move(action), std::move(f)}; + return s->get_future(); + } + } + return make_ready_future<>(); +} +} // namespace internal + +/// \addtogroup future-util + +/// \brief Call a function for each item in a range, sequentially (iterator version). +/// +/// For each item in a range, call a function, waiting for the previous +/// invocation to complete before calling the next one. +/// +/// \param begin an \c InputIterator designating the beginning of the range +/// \param end an \c InputIterator designating the endof the range +/// \param action a callable, taking a reference to objects from the range +/// as a parameter, and returning a \c future<> that resolves +/// when it is acceptable to process the next item. +/// \return a ready future on success, or the first failed future if +/// \c action failed. +template<typename Iterator, typename AsyncAction> +SEASTAR_CONCEPT( requires requires (Iterator i, AsyncAction aa) { + { futurize_invoke(aa, *i) } -> std::same_as<future<>>; +} ) +inline +future<> do_for_each(Iterator begin, Iterator end, AsyncAction action) noexcept { + try { + return internal::do_for_each_impl(std::move(begin), std::move(end), std::move(action)); + } catch (...) { + return current_exception_as_future(); + } +} + +/// \brief Call a function for each item in a range, sequentially (range version). +/// +/// For each item in a range, call a function, waiting for the previous +/// invocation to complete before calling the next one. +/// +/// \param c an \c Container object designating input range +/// \param action a callable, taking a reference to objects from the range +/// as a parameter, and returning a \c future<> that resolves +/// when it is acceptable to process the next item. +/// \return a ready future on success, or the first failed future if +/// \c action failed. +template<typename Container, typename AsyncAction> +SEASTAR_CONCEPT( requires requires (Container c, AsyncAction aa) { + { futurize_invoke(aa, *std::begin(c)) } -> std::same_as<future<>>; + std::end(c); +} ) +inline +future<> do_for_each(Container& c, AsyncAction action) noexcept { + try { + return internal::do_for_each_impl(std::begin(c), std::end(c), std::move(action)); + } catch (...) { + return current_exception_as_future(); + } +} + +namespace internal { + +template <typename T, typename = void> +struct has_iterator_category : std::false_type {}; + +template <typename T> +struct has_iterator_category<T, std::void_t<typename std::iterator_traits<T>::iterator_category >> : std::true_type {}; + +template <typename Iterator, typename Sentinel, typename IteratorCategory> +inline +size_t +iterator_range_estimate_vector_capacity(Iterator const&, Sentinel const&, IteratorCategory) { + // For InputIterators we can't estimate needed capacity + return 0; +} + +template <typename Iterator, typename Sentinel> +inline +size_t +iterator_range_estimate_vector_capacity(Iterator begin, Sentinel end, std::forward_iterator_tag) { + // May be linear time below random_access_iterator_tag, but still better than reallocation + return std::distance(begin, end); +} + +} // namespace internal + +/// \cond internal + +class parallel_for_each_state final : private continuation_base<> { + std::vector<future<>> _incomplete; + promise<> _result; + std::exception_ptr _ex; +private: + // Wait for one of the futures in _incomplete to complete, and then + // decide what to do: wait for another one, or deliver _result if all + // are complete. + void wait_for_one() noexcept; + virtual void run_and_dispose() noexcept override; + task* waiting_task() noexcept override { return _result.waiting_task(); } +public: + parallel_for_each_state(size_t n); + void add_future(future<>&& f); + future<> get_future(); +}; + +/// \endcond + +/// \brief Run tasks in parallel (iterator version). +/// +/// Given a range [\c begin, \c end) of objects, run \c func on each \c *i in +/// the range, and return a future<> that resolves when all the functions +/// complete. \c func should return a future<> that indicates when it is +/// complete. All invocations are performed in parallel. This allows the range +/// to refer to stack objects, but means that unlike other loops this cannot +/// check need_preempt and can only be used with small ranges. +/// +/// \param begin an \c InputIterator designating the beginning of the range +/// \param end an \c InputIterator designating the end of the range +/// \param func Function to invoke with each element in the range (returning +/// a \c future<>) +/// \return a \c future<> that resolves when all the function invocations +/// complete. If one or more return an exception, the return value +/// contains one of the exceptions. +/// \note parallel_for_each() schedules all invocations of \c func on the +/// current shard. If you want to run a function on all shards in parallel, +/// have a look at \ref smp::invoke_on_all() instead. +template <typename Iterator, typename Sentinel, typename Func> +SEASTAR_CONCEPT( requires (requires (Func f, Iterator i) { { f(*i) } -> std::same_as<future<>>; { i++ }; } && (std::same_as<Sentinel, Iterator> || std::sentinel_for<Sentinel, Iterator>))) +// We use a conjunction with std::same_as<Sentinel, Iterator> because std::sentinel_for requires Sentinel to be semiregular, +// which implies that it requires Sentinel to be default-constructible, which is unnecessarily strict in below's context and could +// break legacy code, for which it holds that Sentinel equals Iterator. +inline +future<> +parallel_for_each(Iterator begin, Sentinel end, Func&& func) noexcept { + parallel_for_each_state* s = nullptr; + // Process all elements, giving each future the following treatment: + // - available, not failed: do nothing + // - available, failed: collect exception in ex + // - not available: collect in s (allocating it if needed) + while (begin != end) { + auto f = futurize_invoke(std::forward<Func>(func), *begin); + ++begin; + memory::scoped_critical_alloc_section _; + if (!f.available() || f.failed()) { + if (!s) { + using itraits = std::iterator_traits<Iterator>; + size_t n{0U}; + if constexpr (internal::has_iterator_category<Iterator>::value) { + // We need if-constexpr here because there exist iterators for which std::iterator_traits + // does not have 'iterator_category' as member type + n = (internal::iterator_range_estimate_vector_capacity(begin, end, typename itraits::iterator_category{}) + 1); + } + s = new parallel_for_each_state(n); + } + s->add_future(std::move(f)); + } + } + // If any futures were not available, hand off to parallel_for_each_state::start(). + // Otherwise we can return a result immediately. + if (s) { + // s->get_future() takes ownership of s (and chains it to one of the futures it contains) + // so this isn't a leak + return s->get_future(); + } + return make_ready_future<>(); +} + +/// \brief Run tasks in parallel (range version). +/// +/// Given a \c range of objects, invoke \c func with each object +/// in the range, and return a future<> that resolves when all +/// the functions complete. \c func should return a future<> that indicates +/// when it is complete. All invocations are performed in parallel. This allows +/// the range to refer to stack objects, but means that unlike other loops this +/// cannot check need_preempt and can only be used with small ranges. +/// +/// \param range A range of objects to iterate run \c func on +/// \param func A callable, accepting reference to the range's +/// \c value_type, and returning a \c future<>. +/// \return a \c future<> that becomes ready when the entire range +/// was processed. If one or more of the invocations of +/// \c func returned an exceptional future, then the return +/// value will contain one of those exceptions. +/// \note parallel_for_each() schedules all invocations of \c func on the +/// current shard. If you want to run a function on all shards in parallel, +/// have a look at \ref smp::invoke_on_all() instead. + +namespace internal { + +template <typename Range, typename Func> +inline +future<> +parallel_for_each_impl(Range&& range, Func&& func) { + return parallel_for_each(std::begin(range), std::end(range), + std::forward<Func>(func)); +} + +} // namespace internal + +template <typename Range, typename Func> +SEASTAR_CONCEPT( requires requires (Func f, Range r) { + { f(*std::begin(r)) } -> std::same_as<future<>>; + std::end(r); +} ) +inline +future<> +parallel_for_each(Range&& range, Func&& func) noexcept { + auto impl = internal::parallel_for_each_impl<Range, Func>; + return futurize_invoke(impl, std::forward<Range>(range), std::forward<Func>(func)); +} + +/// Run a maximum of \c max_concurrent tasks in parallel (iterator version). +/// +/// Given a range [\c begin, \c end) of objects, run \c func on each \c *i in +/// the range, and return a future<> that resolves when all the functions +/// complete. \c func should return a future<> that indicates when it is +/// complete. Up to \c max_concurrent invocations are performed in parallel. +/// This does not allow the range to refer to stack objects. The caller +/// must ensure that the range outlives the call to max_concurrent_for_each +/// so it can be iterated in the background. +/// +/// \param begin an \c InputIterator designating the beginning of the range +/// \param end an \c InputIterator designating the end of the range +/// \param max_concurrent maximum number of concurrent invocations of \c func, must be greater than zero. +/// \param func Function to invoke with each element in the range (returning +/// a \c future<>) +/// \return a \c future<> that resolves when all the function invocations +/// complete. If one or more return an exception, the return value +/// contains one of the exceptions. +/// \note max_concurrent_for_each() schedules all invocations of \c func on the +/// current shard. If you want to run a function on all shards in parallel, +/// have a look at \ref smp::invoke_on_all() instead. +template <typename Iterator, typename Sentinel, typename Func> +SEASTAR_CONCEPT( requires (requires (Func f, Iterator i) { { f(*i) } -> std::same_as<future<>>; { ++i }; } && (std::same_as<Sentinel, Iterator> || std::sentinel_for<Sentinel, Iterator>) ) ) +// We use a conjunction with std::same_as<Sentinel, Iterator> because std::sentinel_for requires Sentinel to be semiregular, +// which implies that it requires Sentinel to be default-constructible, which is unnecessarily strict in below's context and could +// break legacy code, for which it holds that Sentinel equals Iterator. +inline +future<> +max_concurrent_for_each(Iterator begin, Sentinel end, size_t max_concurrent, Func&& func) noexcept { + struct state { + Iterator begin; + Sentinel end; + Func func; + size_t max_concurrent; + semaphore sem; + std::exception_ptr err; + + state(Iterator begin_, Sentinel end_, size_t max_concurrent_, Func func_) + : begin(std::move(begin_)) + , end(std::move(end_)) + , func(std::move(func_)) + , max_concurrent(max_concurrent_) + , sem(max_concurrent_) + , err() + { } + }; + + assert(max_concurrent > 0); + + try { + return do_with(state(std::move(begin), std::move(end), max_concurrent, std::forward<Func>(func)), [] (state& s) { + return do_until([&s] { return s.begin == s.end; }, [&s] { + return s.sem.wait().then([&s] () mutable noexcept { + // Possibly run in background and signal _sem when the task is done. + // The background tasks are waited on using _sem. + (void)futurize_invoke(s.func, *s.begin).then_wrapped([&s] (future<> fut) { + if (fut.failed()) { + auto e = fut.get_exception();; + if (!s.err) { + s.err = std::move(e); + } + } + s.sem.signal(); + }); + ++s.begin; + }); + }).then([&s] { + // Wait for any background task to finish + // and signal and semaphore + return s.sem.wait(s.max_concurrent); + }).then([&s] { + if (!s.err) { + return make_ready_future<>(); + } + return seastar::make_exception_future<>(std::move(s.err)); + }); + }); + } catch (...) { + return current_exception_as_future(); + } +} + +/// Run a maximum of \c max_concurrent tasks in parallel (range version). +/// +/// Given a range of objects, run \c func on each \c *i in +/// the range, and return a future<> that resolves when all the functions +/// complete. \c func should return a future<> that indicates when it is +/// complete. Up to \c max_concurrent invocations are performed in parallel. +/// This does not allow the range to refer to stack objects. The caller +/// must ensure that the range outlives the call to max_concurrent_for_each +/// so it can be iterated in the background. +/// +/// \param range a \c Range to be processed +/// \param max_concurrent maximum number of concurrent invocations of \c func, must be greater than zero. +/// \param func Function to invoke with each element in the range (returning +/// a \c future<>) +/// \return a \c future<> that resolves when all the function invocations +/// complete. If one or more return an exception, the return value +/// contains one of the exceptions. +/// \note max_concurrent_for_each() schedules all invocations of \c func on the +/// current shard. If you want to run a function on all shards in parallel, +/// have a look at \ref smp::invoke_on_all() instead. +template <typename Range, typename Func> +SEASTAR_CONCEPT( requires requires (Func f, Range r) { + { f(*std::begin(r)) } -> std::same_as<future<>>; + std::end(r); +} ) +inline +future<> +max_concurrent_for_each(Range&& range, size_t max_concurrent, Func&& func) noexcept { + try { + return max_concurrent_for_each(std::begin(range), std::end(range), max_concurrent, std::forward<Func>(func)); + } catch (...) { + return current_exception_as_future(); + } +} + +/// @} + +} // namespace seastar diff --git a/src/seastar/include/seastar/core/lowres_clock.hh b/src/seastar/include/seastar/core/lowres_clock.hh new file mode 100644 index 000000000..27358f906 --- /dev/null +++ b/src/seastar/include/seastar/core/lowres_clock.hh @@ -0,0 +1,115 @@ +/* + * 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) 2016 ScyllaDB + */ + +#pragma once + +#include <seastar/core/cacheline.hh> +#include <seastar/core/timer.hh> + +#include <cstdint> + +#include <atomic> +#include <chrono> + +namespace seastar { + +// +// Forward declarations. +// + +class lowres_clock; +class lowres_system_clock; + +/// \endcond + +// +/// \brief Low-resolution and efficient steady clock. +/// +/// This is a monotonic clock with a granularity of ~task_quota. Time points from this clock do not correspond to system +/// time. +/// +/// The primary benefit of this clock is that invoking \c now() is inexpensive compared to +/// \c std::chrono::steady_clock::now(). +/// +/// \see \c lowres_system_clock for a low-resolution clock which produces time points corresponding to system time. +/// +class lowres_clock final { +public: + using rep = std::chrono::steady_clock::rep; + using period = std::chrono::steady_clock::period; + using duration = std::chrono::steady_clock::duration; + using time_point = std::chrono::time_point<lowres_clock, duration>; + static constexpr bool is_steady = true; +private: + // Use inline variable to prevent the compiler from introducing an initialization guard + inline static thread_local time_point _now; +public: + /// + /// \note Outside of a Seastar application, the result is undefined. + /// + static time_point now() noexcept { + return _now; + } + + static void update() noexcept; +}; + +/// +/// \brief Low-resolution and efficient system clock. +/// +/// This clock has the same granularity as \c lowres_clock, but it is not required to be monotonic and its time points +/// correspond to system time. +/// +/// The primary benefit of this clock is that invoking \c now() is inexpensive compared to +/// \c std::chrono::system_clock::now(). +/// +class lowres_system_clock final { +public: + using rep = std::chrono::system_clock::rep; + using period = std::chrono::system_clock::period; + using duration = std::chrono::system_clock::duration; + using time_point = std::chrono::time_point<lowres_system_clock, duration>; + static constexpr bool is_steady = false; +private: + // Use inline variable to prevent the compiler from introducing an initialization guard + inline static thread_local time_point _now; + friend class lowres_clock; // for updates +public: + /// + /// \note Outside of a Seastar application, the result is undefined. + /// + static time_point now() noexcept { + return _now; + } + + static std::time_t to_time_t(time_point t) noexcept { + return std::chrono::duration_cast<std::chrono::seconds>(t.time_since_epoch()).count(); + } + + static time_point from_time_t(std::time_t t) noexcept { + return time_point(std::chrono::duration_cast<duration>(std::chrono::seconds(t))); + } +}; + +extern template class timer<lowres_clock>; + +} + diff --git a/src/seastar/include/seastar/core/make_task.hh b/src/seastar/include/seastar/core/make_task.hh new file mode 100644 index 000000000..9c2f7400f --- /dev/null +++ b/src/seastar/include/seastar/core/make_task.hh @@ -0,0 +1,62 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <memory> +#include <seastar/core/task.hh> +#include <seastar/core/future.hh> + +namespace seastar { + +template <typename Func> +class lambda_task final : public task { + Func _func; + using futurator = futurize<std::invoke_result_t<Func>>; + typename futurator::promise_type _result; +public: + lambda_task(scheduling_group sg, const Func& func) : task(sg), _func(func) {} + lambda_task(scheduling_group sg, Func&& func) : task(sg), _func(std::move(func)) {} + typename futurator::type get_future() noexcept { return _result.get_future(); } + virtual void run_and_dispose() noexcept override { + futurator::invoke(_func).forward_to(std::move(_result)); + delete this; + } + virtual task* waiting_task() noexcept override { + return _result.waiting_task(); + } +}; + +template <typename Func> +inline +lambda_task<Func>* +make_task(Func&& func) noexcept { + return new lambda_task<Func>(current_scheduling_group(), std::forward<Func>(func)); +} + +template <typename Func> +inline +lambda_task<Func>* +make_task(scheduling_group sg, Func&& func) noexcept { + return new lambda_task<Func>(sg, std::forward<Func>(func)); +} + +} diff --git a/src/seastar/include/seastar/core/manual_clock.hh b/src/seastar/include/seastar/core/manual_clock.hh new file mode 100644 index 000000000..34bc5bec1 --- /dev/null +++ b/src/seastar/include/seastar/core/manual_clock.hh @@ -0,0 +1,51 @@ +/* + * 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) 2016 ScyllaDB + */ + +#pragma once + +#include <seastar/core/timer.hh> + +#include <atomic> +#include <chrono> + +namespace seastar { + +class manual_clock { +public: + using rep = int64_t; + using period = std::chrono::nanoseconds::period; + using duration = std::chrono::duration<rep, period>; + using time_point = std::chrono::time_point<manual_clock, duration>; +private: + static std::atomic<rep> _now; + static void expire_timers() noexcept; +public: + manual_clock() noexcept; + static time_point now() noexcept { + return time_point(duration(_now.load(std::memory_order_relaxed))); + } + static void advance(duration d) noexcept; +}; + +extern template class timer<manual_clock>; + +} + diff --git a/src/seastar/include/seastar/core/map_reduce.hh b/src/seastar/include/seastar/core/map_reduce.hh new file mode 100644 index 000000000..96de11fe1 --- /dev/null +++ b/src/seastar/include/seastar/core/map_reduce.hh @@ -0,0 +1,279 @@ +/* + * 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) 2020 ScyllaDB. + */ + +#pragma once + +#include <iterator> + +#include <seastar/core/future.hh> +#include <seastar/core/shared_ptr.hh> + +namespace seastar { + +/// \addtogroup future-util +/// @{ + +/// \cond internal + +template <typename T, typename Ptr, bool IsFuture> +struct reducer_with_get_traits; + +template <typename T, typename Ptr> +struct reducer_with_get_traits<T, Ptr, false> { + using result_type = decltype(std::declval<T>().get()); + using future_type = future<result_type>; + static future_type maybe_call_get(future<> f, Ptr r) { + return f.then([r = std::move(r)] () mutable { + return make_ready_future<result_type>(std::move(r->reducer).get()); + }); + } +}; + +template <typename T, typename Ptr> +struct reducer_with_get_traits<T, Ptr, true> { + using future_type = decltype(std::declval<T>().get()); + static future_type maybe_call_get(future<> f, Ptr r) { + return f.then([r = std::move(r)] () mutable { + return r->reducer.get(); + }).then_wrapped([r] (future_type f) { + return f; + }); + } +}; + +template <typename T, typename Ptr = lw_shared_ptr<T>, typename V = void> +struct reducer_traits { + using future_type = future<>; + static future_type maybe_call_get(future<> f, Ptr r) { + return f.then([r = std::move(r)] {}); + } +}; + +template <typename T, typename Ptr> +struct reducer_traits<T, Ptr, decltype(std::declval<T>().get(), void())> : public reducer_with_get_traits<T, Ptr, is_future<std::invoke_result_t<decltype(&T::get),T>>::value> {}; + +/// \endcond + +/// Map a function over a range and reduce the result. +/// +/// \param begin an \c InputIterator designating the beginning of the range +/// \param end an \c InputIterator designating the end of the range +/// \param mapper is a callable which transforms values from the iterator range into a future<T> +/// \param r is an object which can be called with T as parameter and yields a future<> +/// It may have a get() method which returns a value of type U which holds the result of reduction. +/// \return Th reduced value wrapped in a future. +/// If the reducer has no get() method then this function returns future<>. +/// \note map-reduce() schedules all invocations of both \c mapper and \c reduce +/// on the current shard. If you want to run a function on all shards in +/// parallel, have a look at \ref smp::invoke_on_all() instead, or combine +/// map_reduce() with \ref smp::submit_to(). +/// Sharded services have their own \ref sharded::map_reduce() which +/// map-reduces across all shards. + +// TODO: specialize for non-deferring reducer +template <typename Iterator, typename Mapper, typename Reducer> +SEASTAR_CONCEPT( requires requires (Iterator i, Mapper mapper, Reducer reduce) { + *i++; + { i != i } -> std::convertible_to<bool>; + mapper(*i); + reduce(futurize_invoke(mapper, *i).get0()); +} ) +inline +auto +map_reduce(Iterator begin, Iterator end, Mapper&& mapper, Reducer&& r) + -> typename reducer_traits<Reducer>::future_type +{ + struct state { + Mapper mapper; + Reducer reducer; + }; + auto s = make_lw_shared(state{std::forward<Mapper>(mapper), std::forward<Reducer>(r)}); + future<> ret = make_ready_future<>(); + while (begin != end) { + ret = futurize_invoke(s->mapper, *begin++).then_wrapped([ret = std::move(ret), s] (auto f) mutable { + return ret.then_wrapped([f = std::move(f), s] (auto rf) mutable { + if (rf.failed()) { + f.ignore_ready_future(); + return rf; + } else { + return futurize_invoke(s->reducer, std::move(f.get0())); + } + }); + }); + } + return reducer_traits<Reducer, lw_shared_ptr<state>>::maybe_call_get(std::move(ret), s); +} + +/// Asynchronous map/reduce transformation. +/// +/// Given a range of objects, an asynchronous unary function +/// operating on these objects, an initial value, and a +/// binary function for reducing, map_reduce() will +/// transform each object in the range, then invoke +/// the the reducing function with the result. +/// +/// Example: +/// +/// Calculate the total size of several files: +/// +/// \code +/// map_reduce(files.begin(), files.end(), +/// std::mem_fn(file::size), +/// size_t(0), +/// std::plus<size_t>()) +/// \endcode +/// +/// Requirements: +/// - Iterator: an InputIterator. +/// - Mapper: unary function taking Iterator::value_type and producing a future<...>. +/// - Initial: any value type +/// - Reduce: a binary function taking two Initial values and returning an Initial +/// +/// Return type: +/// - future<Initial> +/// +/// \param begin beginning of object range to operate on +/// \param end end of object range to operate on +/// \param mapper map function to call on each object, returning a future +/// \param initial initial input value to reduce function +/// \param reduce binary function for merging two result values from \c mapper +/// +/// \return equivalent to \c reduce(reduce(initial, mapper(obj0)), mapper(obj1)) ... +/// +/// \note map-reduce() schedules all invocations of both \c mapper and \c reduce +/// on the current shard. If you want to run a function on all shards in +/// parallel, have a look at \ref smp::invoke_on_all() instead, or combine +/// map_reduce() with \ref smp::submit_to(). +/// Sharded services have their own \ref sharded::map_reduce() which +/// map-reduces across all shards. +template <typename Iterator, typename Mapper, typename Initial, typename Reduce> +SEASTAR_CONCEPT( requires requires (Iterator i, Mapper mapper, Initial initial, Reduce reduce) { + *i++; + { i != i} -> std::convertible_to<bool>; + mapper(*i); + requires is_future<decltype(mapper(*i))>::value; + { reduce(std::move(initial), mapper(*i).get0()) } -> std::convertible_to<Initial>; +} ) +inline +future<Initial> +map_reduce(Iterator begin, Iterator end, Mapper&& mapper, Initial initial, Reduce reduce) { + struct state { + Mapper mapper; + Initial result; + Reduce reduce; + }; + auto s = make_lw_shared(state{std::forward<Mapper>(mapper), std::move(initial), std::move(reduce)}); + future<> ret = make_ready_future<>(); + while (begin != end) { + ret = futurize_invoke(s->mapper, *begin++).then_wrapped([s = s.get(), ret = std::move(ret)] (auto f) mutable { + try { + s->result = s->reduce(std::move(s->result), std::move(f.get0())); + return std::move(ret); + } catch (...) { + return std::move(ret).then_wrapped([ex = std::current_exception()] (auto f) { + f.ignore_ready_future(); + return make_exception_future<>(ex); + }); + } + }); + } + return ret.then([s] { + return make_ready_future<Initial>(std::move(s->result)); + }); +} + +/// Asynchronous map/reduce transformation (range version). +/// +/// Given a range of objects, an asynchronous unary function +/// operating on these objects, an initial value, and a +/// binary function for reducing, map_reduce() will +/// transform each object in the range, then invoke +/// the the reducing function with the result. +/// +/// Example: +/// +/// Calculate the total size of several files: +/// +/// \code +/// std::vector<file> files = ...; +/// map_reduce(files, +/// std::mem_fn(file::size), +/// size_t(0), +/// std::plus<size_t>()) +/// \endcode +/// +/// Requirements: +/// - Iterator: an InputIterator. +/// - Mapper: unary function taking Iterator::value_type and producing a future<...>. +/// - Initial: any value type +/// - Reduce: a binary function taking two Initial values and returning an Initial +/// +/// Return type: +/// - future<Initial> +/// +/// \param range object range to operate on +/// \param mapper map function to call on each object, returning a future +/// \param initial initial input value to reduce function +/// \param reduce binary function for merging two result values from \c mapper +/// +/// \return equivalent to \c reduce(reduce(initial, mapper(obj0)), mapper(obj1)) ... +/// +/// \note map-reduce() schedules all invocations of both \c mapper and \c reduce +/// on the current shard. If you want to run a function on all shards in +/// parallel, have a look at \ref smp::invoke_on_all() instead, or combine +/// map_reduce() with \ref smp::submit_to(). +/// Sharded services have their own \ref sharded::map_reduce() which +/// map-reduces across all shards. +template <typename Range, typename Mapper, typename Initial, typename Reduce> +SEASTAR_CONCEPT( requires requires (Range range, Mapper mapper, Initial initial, Reduce reduce) { + std::begin(range); + std::end(range); + mapper(*std::begin(range)); + requires is_future<std::remove_reference_t<decltype(mapper(*std::begin(range)))>>::value; + { reduce(std::move(initial), mapper(*std::begin(range)).get0()) } -> std::convertible_to<Initial>; +} ) +inline +future<Initial> +map_reduce(Range&& range, Mapper&& mapper, Initial initial, Reduce reduce) { + return map_reduce(std::begin(range), std::end(range), std::forward<Mapper>(mapper), + std::move(initial), std::move(reduce)); +} + +/// Implements @Reducer concept. Calculates the result by +/// adding elements to the accumulator. +template <typename Result, typename Addend = Result> +class adder { +private: + Result _result; +public: + future<> operator()(const Addend& value) { + _result += value; + return make_ready_future<>(); + } + Result get() && { + return std::move(_result); + } +}; + +/// @} + +} // namespace seastar diff --git a/src/seastar/include/seastar/core/memory.hh b/src/seastar/include/seastar/core/memory.hh new file mode 100644 index 000000000..0206bae97 --- /dev/null +++ b/src/seastar/include/seastar/core/memory.hh @@ -0,0 +1,379 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/core/resource.hh> +#include <seastar/core/bitops.hh> +#include <new> +#include <functional> +#include <vector> + +namespace seastar { + +/// \defgroup memory-module Memory management +/// +/// Functions and classes for managing memory. +/// +/// Memory management in seastar consists of the following: +/// +/// - Low-level memory management in the \ref memory namespace. +/// - Various smart pointers: \ref shared_ptr, \ref lw_shared_ptr, +/// and \ref foreign_ptr. +/// - zero-copy support: \ref temporary_buffer and \ref deleter. + +/// Low-level memory management support +/// +/// The \c memory namespace provides functions and classes for interfacing +/// with the seastar memory allocator. +/// +/// The seastar memory allocator splits system memory into a pool per +/// logical core (lcore). Memory allocated one an lcore should be freed +/// on the same lcore; failing to do so carries a severe performance +/// penalty. It is possible to share memory with another core, but this +/// should be limited to avoid cache coherency traffic. +/// You can obtain the memory layout of the current shard with +/// \ref get_memory_layout(). +/// +/// ## Critical allocation scopes +/// +/// Seastar supports marking scopes as critical allocation scopes for the purpose +/// of special treatment from various memory related utilities. +/// See \ref scoped_critical_alloc_section. +/// +/// ## Diagnostics and debugging features +/// +/// ### Allocation failure injector +/// +/// Allows injecting allocation failures for testing resiliency against +/// allocation failures, or exceptions in general. See: +/// * \ref alloc_failure_injector +/// * \ref with_allocation_failures() +/// +/// ### Large allocation warning +/// +/// Large allocations put great pressure on the allocator which might be unable +/// to serve them even if there is enough memory available, due to memory +/// fragmentation. This is especially relevant for long-running applications, +/// the kind of applications that are typically built with seastar. This feature +/// allows finding these large by logging a warning on large allocations, with +/// the stacktrace of the. See: +/// * \ref set_large_allocation_warning_threshold() +/// * \ref get_large_allocation_warning_threshold() +/// * \ref scoped_large_allocation_warning_threshold +/// * \ref scoped_large_allocation_warning_disable +/// +/// ### Heap profiling +/// +/// Heap profiling allows finding out how memory is used by your application, by +/// recording the stacktrace of all allocations. See: +/// * \ref set_heap_profiling_enabled() +/// * \ref scoped_heap_profiling +/// +/// ### Abort on allocation failure +/// +/// Often, the best way to debug an allocation failure is a coredump. This +/// feature allows dumping core on allocation failures, containing the stack of +/// the failed allocation, by means of aborting. To enable set the +/// `abort_on_seastar_bad_alloc` configuration option or the respective command +/// line flag. +/// +/// ### Dump diagnostics report +/// +/// Dump a diagnostic report of the state of the seastar allocator upon allocation +/// failure. The report is dumped with the `seastar_memory` logger, with debug +/// level. +/// You can configure a report to be dumped with error level on certain allocation +/// kinds, see: +/// * set_dump_memory_diagnostics_on_alloc_failure_kind() +/// * set_additional_diagnostics_producer() +/// * generate_memory_diagnostics_report() +/// +/// The diagnostics report dump can be configured with the command +/// line/configuration file via the \p dump-memory-diagnostics-on-alloc-failure-kind +/// command-line flag/configuration item. +namespace memory { + +/// \cond internal + +#ifdef SEASTAR_OVERRIDE_ALLOCATOR_PAGE_SIZE +#define SEASTAR_INTERNAL_ALLOCATOR_PAGE_SIZE (SEASTAR_OVERRIDE_ALLOCATOR_PAGE_SIZE) +#else +#define SEASTAR_INTERNAL_ALLOCATOR_PAGE_SIZE 4096 +#endif + +static constexpr size_t page_size = SEASTAR_INTERNAL_ALLOCATOR_PAGE_SIZE; +static constexpr size_t page_bits = log2ceil(page_size); +static constexpr size_t huge_page_size = +#if defined(__x86_64__) || defined(__i386__) || defined(__s390x__) || defined(__zarch__) + 1 << 21; // 2M +#elif defined(__aarch64__) + 1 << 21; // 2M +#elif defined(__PPC__) + 1 << 24; // 16M +#else +#error "Huge page size is not defined for this architecture" +#endif + +void configure(std::vector<resource::memory> m, bool mbind, + std::optional<std::string> hugetlbfs_path = {}); + +void enable_abort_on_allocation_failure(); + +class disable_abort_on_alloc_failure_temporarily { +public: + disable_abort_on_alloc_failure_temporarily(); + ~disable_abort_on_alloc_failure_temporarily() noexcept; +}; + +// Disables heap profiling as long as this object is alive. +// Can be nested, in which case the profiling is re-enabled when all +// the objects go out of scope. +class disable_backtrace_temporarily { + bool _old; +public: + disable_backtrace_temporarily(); + ~disable_backtrace_temporarily(); +}; + +enum class reclaiming_result { + reclaimed_nothing, + reclaimed_something +}; + +// Determines when reclaimer can be invoked +enum class reclaimer_scope { + // + // Reclaimer is only invoked in its own fiber. That fiber will be + // given higher priority than regular application fibers. + // + async, + + // + // Reclaimer may be invoked synchronously with allocation. + // It may also be invoked in async scope. + // + // Reclaimer may invoke allocation, though it is discouraged because + // the system may be low on memory and such allocations may fail. + // Reclaimers which allocate should be prepared for re-entry. + // + sync +}; + +class reclaimer { +public: + struct request { + // The number of bytes which is needed to be released. + // The reclaimer can release a different amount. + // If less is released then the reclaimer may be invoked again. + size_t bytes_to_reclaim; + }; + using reclaim_fn = std::function<reclaiming_result ()>; +private: + std::function<reclaiming_result (request)> _reclaim; + reclaimer_scope _scope; +public: + // Installs new reclaimer which will be invoked when system is falling + // low on memory. 'scope' determines when reclaimer can be executed. + reclaimer(std::function<reclaiming_result ()> reclaim, reclaimer_scope scope = reclaimer_scope::async); + reclaimer(std::function<reclaiming_result (request)> reclaim, reclaimer_scope scope = reclaimer_scope::async); + ~reclaimer(); + reclaiming_result do_reclaim(size_t bytes_to_reclaim) { return _reclaim(request{bytes_to_reclaim}); } + reclaimer_scope scope() const { return _scope; } +}; + +extern std::pmr::polymorphic_allocator<char>* malloc_allocator; + +// Call periodically to recycle objects that were freed +// on cpu other than the one they were allocated on. +// +// Returns @true if any work was actually performed. +bool drain_cross_cpu_freelist(); + + +// We don't want the memory code calling back into the rest of +// the system, so allow the rest of the system to tell the memory +// code how to initiate reclaim. +// +// When memory is low, calling \c hook(fn) will result in fn being called +// in a safe place wrt. allocations. +void set_reclaim_hook( + std::function<void (std::function<void ()>)> hook); + +/// \endcond + +class statistics; + +/// Capture a snapshot of memory allocation statistics for this lcore. +statistics stats(); + +/// Memory allocation statistics. +class statistics { + uint64_t _mallocs; + uint64_t _frees; + uint64_t _cross_cpu_frees; + size_t _total_memory; + size_t _free_memory; + uint64_t _reclaims; + uint64_t _large_allocs; + uint64_t _failed_allocs; + + uint64_t _foreign_mallocs; + uint64_t _foreign_frees; + uint64_t _foreign_cross_frees; +private: + statistics(uint64_t mallocs, uint64_t frees, uint64_t cross_cpu_frees, + uint64_t total_memory, uint64_t free_memory, uint64_t reclaims, + uint64_t large_allocs, uint64_t failed_allocs, + uint64_t foreign_mallocs, uint64_t foreign_frees, uint64_t foreign_cross_frees) + : _mallocs(mallocs), _frees(frees), _cross_cpu_frees(cross_cpu_frees) + , _total_memory(total_memory), _free_memory(free_memory), _reclaims(reclaims) + , _large_allocs(large_allocs), _failed_allocs(failed_allocs) + , _foreign_mallocs(foreign_mallocs), _foreign_frees(foreign_frees) + , _foreign_cross_frees(foreign_cross_frees) {} +public: + /// Total number of memory allocations calls since the system was started. + uint64_t mallocs() const { return _mallocs; } + /// Total number of memory deallocations calls since the system was started. + uint64_t frees() const { return _frees; } + /// Total number of memory deallocations that occured on a different lcore + /// than the one on which they were allocated. + uint64_t cross_cpu_frees() const { return _cross_cpu_frees; } + /// Total number of objects which were allocated but not freed. + size_t live_objects() const { return mallocs() - frees(); } + /// Total free memory (in bytes) + size_t free_memory() const { return _free_memory; } + /// Total allocated memory (in bytes) + size_t allocated_memory() const { return _total_memory - _free_memory; } + /// Total memory (in bytes) + size_t total_memory() const { return _total_memory; } + /// Number of reclaims performed due to low memory + uint64_t reclaims() const { return _reclaims; } + /// Number of allocations which violated the large allocation threshold + uint64_t large_allocations() const { return _large_allocs; } + /// Number of allocations which failed, i.e., where the required memory could not be obtained + /// even after reclaim was attempted + uint64_t failed_allocations() const { return _failed_allocs; } + /// Number of foreign allocations + uint64_t foreign_mallocs() const { return _foreign_mallocs; } + /// Number of foreign frees + uint64_t foreign_frees() const { return _foreign_frees; } + /// Number of foreign frees on reactor threads + uint64_t foreign_cross_frees() const { return _foreign_cross_frees; } + friend statistics stats(); +}; + +struct memory_layout { + uintptr_t start; + uintptr_t end; +}; + +// Discover virtual address range used by the allocator on current shard. +// Supported only when seastar allocator is enabled. +memory::memory_layout get_memory_layout(); + +/// Returns the size of free memory in bytes. +size_t free_memory(); + +/// Returns the value of free memory low water mark in bytes. +/// When free memory is below this value, reclaimers are invoked until it goes above again. +size_t min_free_memory(); + +/// Sets the value of free memory low water mark in memory::page_size units. +void set_min_free_pages(size_t pages); + +/// Enable the large allocation warning threshold. +/// +/// Warn when allocation above a given threshold are performed. +/// +/// \param threshold size (in bytes) above which an allocation will be logged +void set_large_allocation_warning_threshold(size_t threshold); + +/// Gets the current large allocation warning threshold. +size_t get_large_allocation_warning_threshold(); + +/// Disable large allocation warnings. +void disable_large_allocation_warning(); + +/// Set a different large allocation warning threshold for a scope. +class scoped_large_allocation_warning_threshold { + size_t _old_threshold; +public: + explicit scoped_large_allocation_warning_threshold(size_t threshold) + : _old_threshold(get_large_allocation_warning_threshold()) { + set_large_allocation_warning_threshold(threshold); + } + scoped_large_allocation_warning_threshold(const scoped_large_allocation_warning_threshold&) = delete; + scoped_large_allocation_warning_threshold(scoped_large_allocation_warning_threshold&& x) = delete; + ~scoped_large_allocation_warning_threshold() { + if (_old_threshold) { + set_large_allocation_warning_threshold(_old_threshold); + } + } + void operator=(const scoped_large_allocation_warning_threshold&) const = delete; + void operator=(scoped_large_allocation_warning_threshold&&) = delete; +}; + +/// Disable large allocation warnings for a scope. +class scoped_large_allocation_warning_disable { + size_t _old_threshold; +public: + scoped_large_allocation_warning_disable() + : _old_threshold(get_large_allocation_warning_threshold()) { + disable_large_allocation_warning(); + } + scoped_large_allocation_warning_disable(const scoped_large_allocation_warning_disable&) = delete; + scoped_large_allocation_warning_disable(scoped_large_allocation_warning_disable&& x) = delete; + ~scoped_large_allocation_warning_disable() { + if (_old_threshold) { + set_large_allocation_warning_threshold(_old_threshold); + } + } + void operator=(const scoped_large_allocation_warning_disable&) const = delete; + void operator=(scoped_large_allocation_warning_disable&&) = delete; +}; + +/// Enable/disable heap profiling. +/// +/// In order to use heap profiling you have to define +/// `SEASTAR_HEAPPROF`. +/// Heap profiling data is not currently exposed via an API for +/// inspection, instead it was designed to be inspected from a +/// debugger. +/// For an example script that makes use of the heap profiling data +/// see [scylla-gdb.py] (https://github.com/scylladb/scylla/blob/e1b22b6a4c56b4f1d0adf65d1a11db4bcb51fe7d/scylla-gdb.py#L1439) +/// This script can generate either textual representation of the data, +/// or a zoomable flame graph ([flame graph generation instructions](https://github.com/scylladb/scylla/wiki/Seastar-heap-profiler), +/// [example flame graph](https://user-images.githubusercontent.com/1389273/72920437-f0cf8a80-3d51-11ea-92f0-f3dbeb698871.png)). +void set_heap_profiling_enabled(bool); + +/// Enable heap profiling for the duration of the scope. +/// +/// For more information about heap profiling see +/// \ref set_heap_profiling_enabled(). +class scoped_heap_profiling { +public: + scoped_heap_profiling() noexcept; + ~scoped_heap_profiling(); +}; + +} +} diff --git a/src/seastar/include/seastar/core/metrics.hh b/src/seastar/include/seastar/core/metrics.hh new file mode 100644 index 000000000..ad0b234f0 --- /dev/null +++ b/src/seastar/include/seastar/core/metrics.hh @@ -0,0 +1,678 @@ +/* + * 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) 2016 ScyllaDB. + */ + +#pragma once + +#include <functional> +#include <limits> +#include <map> +#include <type_traits> +#include <seastar/core/sstring.hh> +#include <seastar/core/shared_ptr.hh> +#include <seastar/core/metrics_registration.hh> +#include <boost/lexical_cast.hpp> +#include <seastar/core/metrics_types.hh> +#include <seastar/util/std-compat.hh> +#include <seastar/util/bool_class.hh> + +/*! \file metrics.hh + * \brief header for metrics creation. + * + * This header file contains the metrics creation method with their helper function. + * Include this file when need to create metrics. + * Typically this will be in your source file. + * + * Code that is under the impl namespace should not be used directly. + * + */ + +namespace seastar { + +/*! + * \addtogroup metrics + * @{ + * + * \namespace seastar::metrics + * \brief metrics creation and registration + * + * the metrics namespace holds the relevant method and classes to generate metrics. + * + * The metrics layer support registering metrics, that later will be + * exported via different API protocols. + * + * To be able to support multiple protocols the following simplifications where made: + * 1. The id of the metrics is based on the collectd id + * 2. A metric could be a single value either a reference or a function + * + * To add metrics definition to class A do the following: + * * Add a metrics_group memeber to A + * * Add a a set_metrics() method that would be called in the constructor. + * + * + * In A header file + * \code + * #include "core/metrics_registration.hh" + * class A { + * metric_groups _metrics + * + * void setup_metrics(); + * + * }; + * \endcode + * + * In A source file: + * + * \code + * include "core/metrics.hh" + * + * void A::setup_metrics() { + * namespace sm = seastar::metrics; + * _metrics = sm::create_metric_group(); + * _metrics->add_group("cache", {sm::make_gauge("bytes", "used", [this] { return _region.occupancy().used_space(); })}); + * } + * \endcode + */ + +namespace metrics { + +class double_registration : public std::runtime_error { +public: + double_registration(std::string what); +}; + +/*! + * \defgroup metrics_types metrics type definitions + * The following are for the metric layer use, do not use them directly + * Instead use the make_counter, make_gauge + * + */ +using metric_type_def = sstring; /*!< Used to hold an inherit type (like bytes)*/ +using metric_name_type = sstring; /*!< The metric name'*/ +using instance_id_type = sstring; /*!< typically used for the shard id*/ +using skip_when_empty = bool_class<class skip_when_empty_tag>; + +/*! + * \brief Human-readable description of a metric/group. + * + * + * Uses a separate class to deal with type resolution + * + * Add this to metric creation: + * + * \code + * _metrics->add_group("groupname", { + * sm::make_gauge("metric_name", value, description("A documentation about the return value")) + * }); + * \endcode + * + */ +class description { +public: + description(sstring s = sstring()) : _s(std::move(s)) + {} + const sstring& str() const { + return _s; + } +private: + sstring _s; +}; + +/*! + * \brief Label a metrics + * + * Label are useful for adding information about a metric that + * later you would need to aggregate by. + * For example, if you have multiple queues on a shard. + * Adding the queue id as a Label will allow you to use the same name + * of the metrics with multiple id instances. + * + * label_instance holds an instance of label consist of a key and value. + * + * Typically you will not generate a label_instance yourself, but use a label + * object for that. + * @see label for more information + * + * + */ +class label_instance { + sstring _key; + sstring _value; +public: + /*! + * \brief create a label_instance + * label instance consists of key and value. + * The key is an sstring. + * T - the value type can be any type that can be lexical_cast to string + * (ie. if it support the redirection operator for stringstream). + * + * All primitive types are supported so all the following examples are valid: + * label_instance a("smp_queue", 1) + * label_instance a("my_key", "my_value") + * label_instance a("internal_id", -1) + */ + template<typename T> + label_instance(const sstring& key, T v) : _key(key), _value(boost::lexical_cast<std::string>(v)){} + + /*! + * \brief returns the label key + */ + const sstring key() const { + return _key; + } + + /*! + * \brief returns the label value + */ + const sstring value() const { + return _value; + } + bool operator<(const label_instance&) const; + bool operator==(const label_instance&) const; + bool operator!=(const label_instance&) const; +}; + + +/*! + * \brief Class that creates label instances + * + * A factory class to create label instance + * Typically, the same Label name is used in multiple places. + * label is a label factory, you create it once, and use it to create the label_instance. + * + * In the example we would like to label the smp_queue with with the queue owner + * + * seastar::metrics::label smp_owner("smp_owner"); + * + * now, when creating a new smp metric we can add a label to it: + * + * sm::make_queue_length("send_batch_queue_length", _last_snt_batch, {smp_owner(cpuid)}) + * + * where cpuid in this case is unsiged. + */ +class label { + sstring key; +public: + using instance = label_instance; + /*! + * \brief creating a label + * key is the label name, it will be the key for all label_instance + * that will be created from this label. + */ + explicit label(const sstring& key) : key(key) { + } + + /*! + * \brief creating a label instance + * + * Use the function operator to create a new label instance. + * T - the value type can be any type that can be lexical_cast to string + * (ie. if it support the redirection operator for stringstream). + * + * All primitive types are supported so if lab is a label, all the following examples are valid: + * lab(1) + * lab("my_value") + * lab(-1) + */ + template<typename T> + instance operator()(T value) const { + return label_instance(key, std::forward<T>(value)); + } + + /*! + * \brief returns the label name + */ + const sstring& name() const { + return key; + } +}; + +/*! + * \namespace impl + * \brief holds the implementation parts of the metrics layer, do not use directly. + * + * The metrics layer define a thin API for adding metrics. + * Some of the implementation details need to be in the header file, they should not be use directly. + */ +namespace impl { + +// The value binding data types +enum class data_type : uint8_t { + COUNTER, + REAL_COUNTER, + GAUGE, + HISTOGRAM, + SUMMARY, +}; + +template <bool callable, typename T> +struct real_counter_type_traits { + using type = T; +}; + +template <typename T> +struct real_counter_type_traits<true, T> { + using type = typename std::invoke_result<T>::type; +}; + +template <typename T> +struct counter_type_traits { + using real_traits = real_counter_type_traits<std::is_invocable<T>::value, T>; + static constexpr bool is_integral = std::is_integral<typename real_traits::type>::value; + static constexpr data_type type = is_integral ? data_type::COUNTER : data_type::REAL_COUNTER; +}; + +/*! + * \brief A helper class that used to return metrics value. + * + * Do not use directly @see metrics_creation + */ +class metric_value { +public: + std::variant<double, histogram> u; + data_type _type; + data_type type() const { + return _type; + } + + double d() const { + return std::get<double>(u); + } + + uint64_t ui() const { + auto d = std::get<double>(u); + if (d >= 0 && d <= double(std::numeric_limits<long>::max())) { + return lround(d); + } else { + // double value is out of range or NaN or Inf + ulong_conversion_error(d); + return 0; + } + } + + int64_t i() const { + auto d = std::get<double>(u); + if (d >= double(std::numeric_limits<long>::min()) && d <= double(std::numeric_limits<long>::max())) { + return lround(d); + } else { + // double value is out of range or NaN or Inf + ulong_conversion_error(d); + return 0; + } + } + + metric_value() + : _type(data_type::GAUGE) { + } + + metric_value(histogram&& h, data_type t = data_type::HISTOGRAM) : + u(std::move(h)), _type(t) { + } + metric_value(const histogram& h, data_type t = data_type::HISTOGRAM) : + u(h), _type(t) { + } + + metric_value(double d, data_type t) + : u(d), _type(t) { + } + + metric_value& operator=(const metric_value& c) = default; + + metric_value& operator+=(const metric_value& c) { + *this = *this + c; + return *this; + } + + metric_value operator+(const metric_value& c); + const histogram& get_histogram() const { + return std::get<histogram>(u); + } + + /*! + * \brief return true if this metric was never used + * + * Histograms, Summaries and counters are ever growing by nature, so + * it is possible to check if they have been used or not. + */ + bool is_empty() const noexcept { + return ((_type == data_type::HISTOGRAM || _type == data_type::SUMMARY) && get_histogram().sample_count == 0) || + ((_type == data_type::COUNTER || _type == data_type::REAL_COUNTER) && d() == 0); + } +private: + static void ulong_conversion_error(double d); +}; + +using metric_function = std::function<metric_value()>; + +struct metric_type { + data_type base_type; + metric_type_def type_name; +}; + +struct metric_definition_impl { + metric_name_type name; + metric_type type; + metric_function f; + description d; + bool enabled = true; + skip_when_empty _skip_when_empty = skip_when_empty::no; + std::vector<std::string> aggregate_labels; + std::map<sstring, sstring> labels; + metric_definition_impl& operator ()(bool enabled); + metric_definition_impl& operator ()(const label_instance& label); + metric_definition_impl& operator ()(skip_when_empty skip) noexcept; + metric_definition_impl& aggregate(const std::vector<label>& labels) noexcept; + metric_definition_impl& set_skip_when_empty(bool skip=true) noexcept; + metric_definition_impl& set_type(const sstring& type_name); + metric_definition_impl( + metric_name_type name, + metric_type type, + metric_function f, + description d, + std::vector<label_instance> labels, + std::vector<label> aggregate_labels = {}); +}; + +class metric_groups_def { +public: + metric_groups_def() = default; + virtual ~metric_groups_def() = default; + metric_groups_def(const metric_groups_def&) = delete; + metric_groups_def(metric_groups_def&&) = default; + virtual metric_groups_def& add_metric(group_name_type name, const metric_definition& md) = 0; + virtual metric_groups_def& add_group(group_name_type name, const std::initializer_list<metric_definition>& l) = 0; + virtual metric_groups_def& add_group(group_name_type name, const std::vector<metric_definition>& l) = 0; +}; + +instance_id_type shard(); + +template<typename T, typename = std::enable_if_t<std::is_invocable_v<T>>> +metric_function make_function(T val, data_type dt) { + return [dt, val = std::move(val)] { + return metric_value(val(), dt); + }; +} + +template<typename T, typename = std::enable_if_t<!std::is_invocable_v<T>>> +metric_function make_function(T& val, data_type dt) { + return [dt, &val] { + return metric_value(val, dt); + }; +} +} + +extern const bool metric_disabled; + +extern label shard_label; + +/* + * The metrics definition are defined to be compatible with collectd metrics defintion. + * Typically you should used gauge or derived. + */ + + +/*! + * \brief Gauge are a general purpose metric. + * + * They can support floating point and can increase or decrease + */ +template<typename T> +impl::metric_definition_impl make_gauge(metric_name_type name, + T&& val, description d = description(), std::vector<label_instance> labels = {}) { + return {name, {impl::data_type::GAUGE, "gauge"}, make_function(std::forward<T>(val), impl::data_type::GAUGE), d, labels}; +} + +/*! + * \brief Gauge are a general purpose metric. + * + * They can support floating point and can increase or decrease + */ +template<typename T> +impl::metric_definition_impl make_gauge(metric_name_type name, + description d, T&& val) { + return {name, {impl::data_type::GAUGE, "gauge"}, make_function(std::forward<T>(val), impl::data_type::GAUGE), d, {}}; +} + +/*! + * \brief Gauge are a general purpose metric. + * + * They can support floating point and can increase or decrease + */ +template<typename T> +impl::metric_definition_impl make_gauge(metric_name_type name, + description d, std::vector<label_instance> labels, T&& val) { + return {name, {impl::data_type::GAUGE, "gauge"}, make_function(std::forward<T>(val), impl::data_type::GAUGE), d, labels}; +} + + +/*! + * \brief Derive are used when a rate is more interesting than the value. + * + * Derive is an integer value that can increase or decrease, typically it is used when looking at the + * derivation of the value. + * + * It is OK to use it when counting things and if no wrap-around is expected (it shouldn't) it's prefer over counter metric. + */ +template<typename T> +[[deprecated("Use make_counter()")]] +impl::metric_definition_impl make_derive(metric_name_type name, + T&& val, description d = description(), std::vector<label_instance> labels = {}) { + return make_counter(std::move(name), std::forward<T>(val), std::move(d), std::move(labels)); +} + + +/*! + * \brief Derive are used when a rate is more interesting than the value. + * + * Derive is an integer value that can increase or decrease, typically it is used when looking at the + * derivation of the value. + * + * It is OK to use it when counting things and if no wrap-around is expected (it shouldn't) it's prefer over counter metric. + */ +template<typename T> +[[deprecated("Use make_counter()")]] +impl::metric_definition_impl make_derive(metric_name_type name, description d, + T&& val) { + return make_counter(std::move(name), std::forward<T>(val), std::move(d), {}); +} + + +/*! + * \brief Derive are used when a rate is more interesting than the value. + * + * Derive is an integer value that can increase or decrease, typically it is used when looking at the + * derivation of the value. + * + * It is OK to use it when counting things and if no wrap-around is expected (it shouldn't) it's prefer over counter metric. + */ +template<typename T> +[[deprecated("Use make_counter()")]] +impl::metric_definition_impl make_derive(metric_name_type name, description d, std::vector<label_instance> labels, + T&& val) { + return make_counter(std::move(name), std::forward<T>(val), std::move(d), std::move(labels)); +} + + +/*! + * \brief create a counter metric + * + * Counters are used when a rate is more interesting than the value, monitoring systems take + * derivation from it to display. + * + * It's an integer or floating point value that can increase or decrease. + * + */ +template<typename T> +impl::metric_definition_impl make_counter(metric_name_type name, + T&& val, description d = description(), std::vector<label_instance> labels = {}) { + auto type = impl::counter_type_traits<std::remove_reference_t<T>>::type; + return {name, {type, "counter"}, make_function(std::forward<T>(val), type), d, labels}; +} + +/*! + * \brief create a counter metric + * + * Counters are used when a rate is more interesting than the value, monitoring systems take + * derivation from it to display. + * + * It's an integer or floating point value that can increase or decrease. + * + */ +template<typename T> +impl::metric_definition_impl make_counter(metric_name_type name, description d, T&& val) { + return make_counter(std::move(name), std::forward<T>(val), std::move(d), {}); +} + +/*! + * \brief create a counter metric + * + * Counters are used when a rate is more interesting than the value, monitoring systems take + * derivation from it to display. + * + * It's an integer or floating point value that can increase or decrease. + * + */ +template<typename T> +impl::metric_definition_impl make_counter(metric_name_type name, description d, std::vector<label_instance> labels, T&& val) { + return make_counter(std::move(name), std::forward<T>(val), std::move(d), std::move(labels)); +} + +/*! + * \brief create an absolute metric. + * + * Absolute are used for metric that are being erased after each time they are read. + * They are here for compatibility reasons and should general be avoided in most applications. + */ +template<typename T> +[[deprecated("Use make_counter()")]] +impl::metric_definition_impl make_absolute(metric_name_type name, + T&& val, description d = description(), std::vector<label_instance> labels = {}) { + return make_counter(std::move(name), std::forward<T>(val), std::move(d), std::move(labels)); +} + +/*! + * \brief create a histogram metric. + * + * Histograms are a list o buckets with upper values and counter for the number + * of entries in each bucket. + */ +template<typename T> +impl::metric_definition_impl make_histogram(metric_name_type name, + T&& val, description d = description(), std::vector<label_instance> labels = {}) { + return {name, {impl::data_type::HISTOGRAM, "histogram"}, make_function(std::forward<T>(val), impl::data_type::HISTOGRAM), d, labels}; +} + +/*! + * \brief create a histogram metric. + * + * Histograms are a list o buckets with upper values and counter for the number + * of entries in each bucket. + */ +template<typename T> +impl::metric_definition_impl make_histogram(metric_name_type name, + description d, std::vector<label_instance> labels, T&& val) { + return {name, {impl::data_type::HISTOGRAM, "histogram"}, make_function(std::forward<T>(val), impl::data_type::HISTOGRAM), d, labels}; +} + + +/*! + * \brief create a histogram metric. + * + * Histograms are a list o buckets with upper values and counter for the number + * of entries in each bucket. + */ +template<typename T> +impl::metric_definition_impl make_histogram(metric_name_type name, + description d, T&& val) { + return {name, {impl::data_type::HISTOGRAM, "histogram"}, make_function(std::forward<T>(val), impl::data_type::HISTOGRAM), d, {}}; +} + +/*! + * \brief create a summary metric. + * + * Summaries are a different kind of histograms. It reports in quantiles. + * For example, the p99 and p95 latencies. + */ +template<typename T> +impl::metric_definition_impl make_summary(metric_name_type name, + description d, T&& val) { + return {name, {impl::data_type::SUMMARY, "summary"}, make_function(std::forward<T>(val), impl::data_type::SUMMARY), d, {}}; +} + + +/*! + * \brief create a total_bytes metric. + * + * total_bytes are used for an ever growing counters, like the total bytes + * passed on a network. + */ + +template<typename T> +impl::metric_definition_impl make_total_bytes(metric_name_type name, + T&& val, description d = description(), std::vector<label_instance> labels = {}, + instance_id_type = impl::shard()) { + return make_counter(name, std::forward<T>(val), d, labels).set_type("total_bytes"); +} + +/*! + * \brief create a current_bytes metric. + * + * current_bytes are used to report on current status in bytes. + * For example the current free memory. + */ + +template<typename T> +impl::metric_definition_impl make_current_bytes(metric_name_type name, + T&& val, description d = description(), std::vector<label_instance> labels = {}, + instance_id_type = impl::shard()) { + return make_gauge(name, std::forward<T>(val), d, labels).set_type("bytes"); +} + + +/*! + * \brief create a queue_length metric. + * + * queue_length are used to report on queue length + */ + +template<typename T> +impl::metric_definition_impl make_queue_length(metric_name_type name, + T&& val, description d = description(), std::vector<label_instance> labels = {}, + instance_id_type = impl::shard()) { + return make_gauge(name, std::forward<T>(val), d, labels).set_type("queue_length"); +} + + +/*! + * \brief create a total operation metric. + * + * total_operations are used for ever growing operation counter. + */ + +template<typename T> +impl::metric_definition_impl make_total_operations(metric_name_type name, + T&& val, description d = description(), std::vector<label_instance> labels = {}, + instance_id_type = impl::shard()) { + return make_counter(name, std::forward<T>(val), d, labels).set_type("total_operations"); +} + +/*! @} */ +} +} diff --git a/src/seastar/include/seastar/core/metrics_api.hh b/src/seastar/include/seastar/core/metrics_api.hh new file mode 100644 index 000000000..77a254793 --- /dev/null +++ b/src/seastar/include/seastar/core/metrics_api.hh @@ -0,0 +1,484 @@ +/* + * 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) 2016 ScyllaDB. + */ + +#pragma once + +#include <seastar/core/metrics.hh> +#include <unordered_map> +#include <seastar/core/sharded.hh> +#include <boost/functional/hash.hpp> + +/*! + * \file metrics_api.hh + * \brief header file for metric API layer (like prometheus or collectd) + * + * + * + */ +namespace seastar { +namespace metrics { +namespace impl { + +using labels_type = std::map<sstring, sstring>; +} +} +} + +namespace std { + +template<> +struct hash<seastar::metrics::impl::labels_type> { + using argument_type = seastar::metrics::impl::labels_type; + using result_type = ::std::size_t; + result_type operator()(argument_type const& s) const { + result_type h = 0; + for (auto&& i : s) { + boost::hash_combine(h, std::hash<seastar::sstring>{}(i.second)); + } + return h; + } +}; + +} + +namespace seastar { +namespace metrics { +struct relabel_config; + +/*! + * \brief result of metric relabeling + * + * The result of calling set_relabel_configs. + * + * metrics_relabeled_due_to_collision the number of metrics that caused conflict + * and were relabeled to avoid name collision. + * + * Non zero value indicates there were name collisions. + * + */ +struct metric_relabeling_result { + size_t metrics_relabeled_due_to_collision; +}; + +namespace impl { + +/** + * Metrics are collected in groups that belongs to some logical entity. + * For example, different measurements of the cpu, will belong to group "cpu". + * + * Name is the metric name like used_objects or used_bytes + * + * Inherit type allows customizing one of the basic types (gauge, counter, derive). + * + * Instance_id is used to differentiate multiple instance of the metrics. + * In the seastar environment it is typical to have a metric per shard. + * + */ + +class metric_id { +public: + metric_id() = default; + metric_id(group_name_type group, metric_name_type name, + labels_type labels = {}) + : _group(std::move(group)), _name( + std::move(name)), _labels(labels) { + } + metric_id(metric_id &&) = default; + metric_id(const metric_id &) = default; + + metric_id & operator=(metric_id &&) = default; + metric_id & operator=(const metric_id &) = default; + + const group_name_type & group_name() const { + return _group; + } + void group_name(const group_name_type & name) { + _group = name; + } + const instance_id_type & instance_id() const { + return _labels.at(shard_label.name()); + } + const metric_name_type & name() const { + return _name; + } + const labels_type& labels() const { + return _labels; + } + labels_type& labels() { + return _labels; + } + sstring full_name() const; + + bool operator<(const metric_id&) const; + bool operator==(const metric_id&) const; +private: + auto as_tuple() const { + return std::tie(group_name(), instance_id(), name(), labels()); + } + group_name_type _group; + metric_name_type _name; + labels_type _labels; +}; +} +} +} + +namespace std { + +template<> +struct hash<seastar::metrics::impl::metric_id> +{ + typedef seastar::metrics::impl::metric_id argument_type; + typedef ::std::size_t result_type; + result_type operator()(argument_type const& s) const + { + result_type const h1 ( std::hash<seastar::sstring>{}(s.group_name()) ); + result_type const h2 ( std::hash<seastar::sstring>{}(s.instance_id()) ); + return h1 ^ (h2 << 1); // or use boost::hash_combine + } +}; + +} + +namespace seastar { +namespace metrics { +namespace impl { + +/*! + * \brief holds metadata information of a metric family + * + * Holds the information that is shared between all metrics + * that belongs to the same metric_family + */ +struct metric_family_info { + data_type type; + metric_type_def inherit_type; + description d; + sstring name; + std::vector<std::string> aggregate_labels; +}; + + +/*! + * \brief holds metric metadata + */ +struct metric_info { + metric_id id; + labels_type original_labels; + bool enabled; + skip_when_empty should_skip_when_empty; +}; + + +using metrics_registration = std::vector<metric_id>; + +class metric_groups_impl : public metric_groups_def { + metrics_registration _registration; +public: + metric_groups_impl() = default; + ~metric_groups_impl(); + metric_groups_impl(const metric_groups_impl&) = delete; + metric_groups_impl(metric_groups_impl&&) = default; + metric_groups_impl& add_metric(group_name_type name, const metric_definition& md); + metric_groups_impl& add_group(group_name_type name, const std::initializer_list<metric_definition>& l); + metric_groups_impl& add_group(group_name_type name, const std::vector<metric_definition>& l); +}; + +class impl; + +class registered_metric { + metric_info _info; + metric_function _f; + shared_ptr<impl> _impl; +public: + registered_metric(metric_id id, metric_function f, bool enabled=true, skip_when_empty skip=skip_when_empty::no); + virtual ~registered_metric() {} + virtual metric_value operator()() const { + return _f(); + } + + bool is_enabled() const { + return _info.enabled; + } + + void set_enabled(bool b) { + _info.enabled = b; + } + void set_skip_when_empty(skip_when_empty skip) noexcept { + _info.should_skip_when_empty = skip; + } + const metric_id& get_id() const { + return _info.id; + } + + const metric_info& info() const { + return _info; + } + metric_info& info() { + return _info; + } + metric_function& get_function() { + return _f; + } +}; + +using register_ref = shared_ptr<registered_metric>; +using metric_instances = std::map<labels_type, register_ref>; + +class metric_family { + metric_instances _instances; + metric_family_info _info; +public: + using iterator = metric_instances::iterator; + using const_iterator = metric_instances::const_iterator; + + metric_family() = default; + metric_family(const metric_family&) = default; + metric_family(const metric_instances& instances) : _instances(instances) { + } + metric_family(const metric_instances& instances, const metric_family_info& info) : _instances(instances), _info(info) { + } + metric_family(metric_instances&& instances, metric_family_info&& info) : _instances(std::move(instances)), _info(std::move(info)) { + } + metric_family(metric_instances&& instances) : _instances(std::move(instances)) { + } + + register_ref& operator[](const labels_type& l) { + return _instances[l]; + } + + const register_ref& at(const labels_type& l) const { + return _instances.at(l); + } + + metric_family_info& info() { + return _info; + } + + const metric_family_info& info() const { + return _info; + } + + iterator find(const labels_type& l) { + return _instances.find(l); + } + + const_iterator find(const labels_type& l) const { + return _instances.find(l); + } + + iterator begin() { + return _instances.begin(); + } + + const_iterator begin() const { + return _instances.cbegin(); + } + + iterator end() { + return _instances.end(); + } + + bool empty() const { + return _instances.empty(); + } + + iterator erase(const_iterator position) { + return _instances.erase(position); + } + + const_iterator end() const { + return _instances.cend(); + } + + uint32_t size() const { + return _instances.size(); + } + +}; + +using value_map = std::map<sstring, metric_family>; + +using metric_metadata_vector = std::vector<metric_info>; + +/*! + * \brief holds a metric family metadata + * + * The meta data of a metric family compose of the + * metadata of the family, and a vector of the metadata for + * each of the metric. + */ +struct metric_family_metadata { + metric_family_info mf; + metric_metadata_vector metrics; +}; + +using value_vector = std::vector<metric_value>; +using metric_metadata = std::vector<metric_family_metadata>; +using metric_values = std::vector<value_vector>; + +struct values_copy { + shared_ptr<metric_metadata> metadata; + metric_values values; +}; + +struct config { + sstring hostname; +}; + +class impl { + value_map _value_map; + config _config; + bool _dirty = true; + shared_ptr<metric_metadata> _metadata; + std::set<sstring> _labels; + std::vector<std::vector<metric_function>> _current_metrics; + std::vector<relabel_config> _relabel_configs; +public: + value_map& get_value_map() { + return _value_map; + } + + const value_map& get_value_map() const { + return _value_map; + } + + void add_registration(const metric_id& id, const metric_type& type, metric_function f, const description& d, bool enabled, skip_when_empty skip, const std::vector<std::string>& aggregate_labels); + void remove_registration(const metric_id& id); + future<> stop() { + return make_ready_future<>(); + } + const config& get_config() const { + return _config; + } + void set_config(const config& c) { + _config = c; + } + + shared_ptr<metric_metadata> metadata(); + + std::vector<std::vector<metric_function>>& functions(); + + void update_metrics_if_needed(); + + void dirty() { + _dirty = true; + } + + const std::set<sstring>& get_labels() const noexcept { + return _labels; + } + + future<metric_relabeling_result> set_relabel_configs(const std::vector<relabel_config>& relabel_configs); + + const std::vector<relabel_config>& get_relabel_configs() const noexcept { + return _relabel_configs; + } +}; + +const value_map& get_value_map(); +using values_reference = shared_ptr<values_copy>; + +foreign_ptr<values_reference> get_values(); + +shared_ptr<impl> get_local_impl(); + +void unregister_metric(const metric_id & id); + +/*! + * \brief initialize metric group + * + * Create a metric_group_def. + * No need to use it directly. + */ +std::unique_ptr<metric_groups_def> create_metric_groups(); + +} + +/// Metrics configuration options. +struct options : public program_options::option_group { + /// \brief The hostname used by the metrics. + /// + /// If not set, the local hostname will be used. + program_options::value<std::string> metrics_hostname; + + options(program_options::option_group* parent_group); +}; + +/*! + * \brief set the metrics configuration + */ +future<> configure(const options& opts); + +/*! + * \brief Perform relabeling and operation on metrics dynamically. + * + * The function would return true if the changes were applied with no conflict + * or false, if there was a conflict in the registration. + * + * The general logic follows Prometheus metrics_relabel_config configuration. + * The relabel rules are applied one after the other. + * You can add or change a label. you can enable or disable a metric, + * in that case the metrics will not be reported at all. + * You can turn on and off the skip_when_empty flag. + * + * Using the Prometheus convention, the metric name is __name__. + * Names cannot be changed. + * + * Import notes: + * - The relabeling always starts from the original set of labels the metric + * was created with. + * - calling with an empty set will remove the relabel config and will + * return all metrics to their original labels + * - To prevent a situation that calling this function would crash the system. + * in a situation where a conflicting metrics name are entered, an additional label + * will be added to the labels with a unique ID. + * + * A few examples: + * To add a level label with a value 1, to the reactor_utilization metric: + * std::vector<sm::relabel_config> rl(1); + rl[0].source_labels = {"__name__"}; + rl[0].target_label = "level"; + rl[0].replacement = "1"; + rl[0].expr = "reactor_utilization"; + set_relabel_configs(rl); + * + * To report only the metrics with the level label equals 1 + * + std::vector<sm::relabel_config> rl(2); + rl[0].source_labels = {"__name__"}; + rl[0].action = sm::relabel_config::relabel_action::drop; + + rl[1].source_labels = {"level"}; + rl[1].expr = "1"; + rl[1].action = sm::relabel_config::relabel_action::keep; + set_relabel_configs(rl); + + */ +future<metric_relabeling_result> set_relabel_configs(const std::vector<relabel_config>& relabel_configs); +/* + * \brief return the current relabel_configs + * This function returns a vector of the current relabel configs + */ +const std::vector<relabel_config>& get_relabel_configs(); + +} +} diff --git a/src/seastar/include/seastar/core/metrics_registration.hh b/src/seastar/include/seastar/core/metrics_registration.hh new file mode 100644 index 000000000..6f57b708b --- /dev/null +++ b/src/seastar/include/seastar/core/metrics_registration.hh @@ -0,0 +1,173 @@ +/* + * 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) 2016 ScyllaDB. + */ + +#pragma once + +#include <memory> +#include <vector> + +#include <seastar/core/sstring.hh> + +/*! + * \file metrics_registration.hh + * \brief holds the metric_groups definition needed by class that reports metrics + * + * If class A needs to report metrics, + * typically you include metrics_registration.hh, in A header file and add to A: + * * metric_groups _metrics as a member + * * set_metrics() method that would be called in the constructor. + * \code + * class A { + * metric_groups _metrics + * + * void setup_metrics(); + * + * }; + * \endcode + * To define the metrics, include in your source file metircs.hh + * @see metrics.hh for the definition for adding a metric. + */ + +namespace seastar { + +namespace metrics { + +namespace impl { +class metric_groups_def; +struct metric_definition_impl; +class metric_groups_impl; +} + +using group_name_type = sstring; /*!< A group of logically related metrics */ +class metric_groups; + +class metric_definition { + std::unique_ptr<impl::metric_definition_impl> _impl; +public: + metric_definition(const impl::metric_definition_impl& impl) noexcept; + metric_definition(metric_definition&& m) noexcept; + ~metric_definition(); + friend metric_groups; + friend impl::metric_groups_impl; +}; + +class metric_group_definition { +public: + group_name_type name; + std::initializer_list<metric_definition> metrics; + metric_group_definition(const group_name_type& name, std::initializer_list<metric_definition> l); + metric_group_definition(const metric_group_definition&) = delete; + ~metric_group_definition(); +}; + +/*! + * metric_groups + * \brief holds the metric definition. + * + * Add multiple metric groups definitions. + * Initialization can be done in the constructor or with a call to add_group + * @see metrics.hh for example and supported metrics + */ +class metric_groups { + std::unique_ptr<impl::metric_groups_def> _impl; +public: + metric_groups() noexcept; + metric_groups(metric_groups&&) = default; + virtual ~metric_groups(); + metric_groups& operator=(metric_groups&&) = default; + /*! + * \brief add metrics belong to the same group in the constructor. + * + * combine the constructor with the add_group functionality. + */ + metric_groups(std::initializer_list<metric_group_definition> mg); + + /*! + * \brief Add metrics belonging to the same group. + * + * Use the metrics creation functions to add metrics. + * + * For example: + * _metrics.add_group("my_group", { + * make_counter("my_counter_name1", counter, description("my counter description")), + * make_counter("my_counter_name2", counter, description("my second counter description")), + * make_gauge("my_gauge_name1", gauge, description("my gauge description")), + * }); + * + * Metric name should be unique inside the group. + * You can chain add_group calls like: + * _metrics.add_group("my group1", {...}).add_group("my group2", {...}); + * + * This overload (with initializer_list) is needed because metric_definition + * has no copy constructor, so the other overload (with vector) cannot be + * invoked on a braced-init-list. + */ + metric_groups& add_group(const group_name_type& name, const std::initializer_list<metric_definition>& l); + + /*! + * \brief Add metrics belonging to the same group. + * + * Use the metrics creation functions to add metrics. + * + * For example: + * vector<metric_definition> v; + * v.push_back(make_counter("my_counter_name1", counter, description("my counter description"))); + * v.push_back(make_counter("my_counter_name2", counter, description("my second counter description"))); + * v.push_back(make_gauge("my_gauge_name1", gauge, description("my gauge description"))); + * _metrics.add_group("my_group", v); + * + * Metric name should be unique inside the group. + * You can chain add_group calls like: + * _metrics.add_group("my group1", vec1).add_group("my group2", vec2); + */ + metric_groups& add_group(const group_name_type& name, const std::vector<metric_definition>& l); + + /*! + * \brief clear all metrics groups registrations. + */ + void clear(); +}; + + +/*! + * \brief hold a single metric group + * Initialization is done in the constructor or + * with a call to add_group + */ +class metric_group : public metric_groups { +public: + metric_group() noexcept; + metric_group(const metric_group&) = delete; + metric_group(metric_group&&) = default; + virtual ~metric_group(); + metric_group& operator=(metric_group&&) = default; + + /*! + * \brief add metrics belong to the same group in the constructor. + * + * + */ + metric_group(const group_name_type& name, std::initializer_list<metric_definition> l); +}; + + +} +} diff --git a/src/seastar/include/seastar/core/metrics_types.hh b/src/seastar/include/seastar/core/metrics_types.hh new file mode 100644 index 000000000..13d79cb21 --- /dev/null +++ b/src/seastar/include/seastar/core/metrics_types.hh @@ -0,0 +1,83 @@ +/* + * 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) 2016 ScyllaDB + */ + +#pragma once +#include <vector> + +namespace seastar { +namespace metrics { + + +/*! + * \brief Histogram bucket type + * + * A histogram bucket contains an upper bound and the number + * of events in the buckets. + */ +struct histogram_bucket { + uint64_t count = 0; // number of events. + double upper_bound = 0; // Inclusive. +}; + + +/*! + * \brief Histogram data type + * + * The histogram struct is a container for histogram values. + * It is not a histogram implementation but it will be used by histogram + * implementation to return its internal values. + */ +struct histogram { + uint64_t sample_count = 0; + double sample_sum = 0; + std::vector<histogram_bucket> buckets; // Ordered in increasing order of upper_bound, +Inf bucket is optional. + + /*! + * \brief Addition assigning a historgram + * + * The histogram must match the buckets upper bounds + * or an exception will be thrown + */ + histogram& operator+=(const histogram& h); + + /*! + * \brief Addition historgrams + * + * Add two histograms and return the result as a new histogram + * The histogram must match the buckets upper bounds + * or an exception will be thrown + */ + histogram operator+(const histogram& h) const; + + /*! + * \brief Addition historgrams + * + * Add two histograms and return the result as a new histogram + * The histogram must match the buckets upper bounds + * or an exception will be thrown + */ + histogram operator+(histogram&& h) const; + +}; + +} + +} diff --git a/src/seastar/include/seastar/core/on_internal_error.hh b/src/seastar/include/seastar/core/on_internal_error.hh new file mode 100644 index 000000000..94275f205 --- /dev/null +++ b/src/seastar/include/seastar/core/on_internal_error.hh @@ -0,0 +1,64 @@ +/* + * 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 2020 ScyllaDB + */ + +#pragma once + +#include <seastar/util/std-compat.hh> + +namespace seastar { + +class logger; + +/// Controls whether on_internal_error() aborts or throws. The default +/// is to throw. +/// \returns the current abort_on_internal_error state. +bool set_abort_on_internal_error(bool do_abort) noexcept; + +/// Report an internal error +/// +/// Depending on the value passed to set_abort_on_internal_error, this +/// will either log to \p logger and abort or throw a std::runtime_error. +[[noreturn]] void on_internal_error(logger& logger, std::string_view reason); + +/// Report an internal error +/// +/// Depending on the value passed to set_abort_on_internal_error, this +/// will either log to \p logger and abort or throw the passed-in +/// \p ex. +/// This overload cannot attach a backtrace to the exception, so if the +/// caller wishes to have one attached they have to do it themselves. +[[noreturn]] void on_internal_error(logger& logger, std::exception_ptr ex); + +/// Report an internal error in a noexcept context +/// +/// The error will be logged to \logger and if set_abort_on_internal_error, +/// was set to true, the program will be aborted. This overload can be used +/// in noexcept contexts like destructors or noexcept functions. +void on_internal_error_noexcept(logger& logger, std::string_view reason) noexcept; + +/// Report an internal error and abort unconditionally +/// +/// The error will be logged to \logger and the program will be aborted, +/// regardless of the abort_on_internal_error setting. +/// This overload can be used to replace assert(). +[[noreturn]] void on_fatal_internal_error(logger& logger, std::string_view reason) noexcept; + +} diff --git a/src/seastar/include/seastar/core/pipe.hh b/src/seastar/include/seastar/core/pipe.hh new file mode 100644 index 000000000..291e068d8 --- /dev/null +++ b/src/seastar/include/seastar/core/pipe.hh @@ -0,0 +1,269 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/core/future.hh> +#include <seastar/core/queue.hh> + +#include <seastar/util/std-compat.hh> + +/// \defgroup fiber-module Fibers +/// +/// \brief Fibers of execution +/// +/// Seastar continuations are normally short, but often chained to one +/// another, so that one continuation does a bit of work and then schedules +/// another continuation for later. Such chains can be long, and often even +/// involve loopings - see for example \ref repeat. We call such chains +/// "fibers" of execution. +/// +/// These fibers are not threads - each is just a string of continuations - +/// but they share some common requirements with traditional threads. +/// For example, we want to avoid one fiber getting starved while a second +/// fiber continuously runs its continuations one after another. +/// As another example, fibers may want to communicate - e.g., one fiber +/// produces data that a second fiber consumes, and we wish to ensure that +/// both fibers get a chance to run, and that if one stops prematurely, +/// the other doesn't hang forever. +/// +/// Consult the following table to see which APIs are useful for fiber tasks: +/// +/// Task | APIs +/// -----------------------------------------------|------------------- +/// Repeat a blocking task indefinitely | \ref keep_doing() +/// Repeat a blocking task, then exit | \ref repeat(), \ref do_until() +/// Provide mutual exclusion between two tasks | \ref semaphore, \ref shared_mutex +/// Pass a stream of data between two fibers | \ref seastar::pipe +/// Safely shut down a resource | \ref seastar::gate +/// Hold on to an object while a fiber is running | \ref do_with() +/// + +/// Seastar API namespace +namespace seastar { + +/// \addtogroup fiber-module +/// @{ + +class broken_pipe_exception : public std::exception { +public: + virtual const char* what() const noexcept { + return "Broken pipe"; + } +}; + +class unread_overflow_exception : public std::exception { +public: + virtual const char* what() const noexcept { + return "pipe_reader::unread() overflow"; + } +}; + +/// \cond internal +namespace internal { +template <typename T> +class pipe_buffer { +private: + queue<std::optional<T>> _buf; + bool _read_open = true; + bool _write_open = true; +public: + pipe_buffer(size_t size) : _buf(size) {} + future<std::optional<T>> read() { + return _buf.pop_eventually(); + } + future<> write(T&& data) { + return _buf.push_eventually(std::move(data)); + } + bool readable() const { + return _write_open || !_buf.empty(); + } + bool writeable() const { + return _read_open; + } + bool close_read() { + // If a writer blocking (on a full queue), need to stop it. + if (_buf.full()) { + _buf.abort(std::make_exception_ptr(broken_pipe_exception())); + } + _read_open = false; + return !_write_open; + } + bool close_write() { + // If the queue is empty, write the EOF (disengaged optional) to the + // queue to wake a blocked reader. If the queue is not empty, there is + // no need to write the EOF to the queue - the reader will return an + // EOF when it sees that _write_open == false. + if (_buf.empty()) { + _buf.push({}); + } + _write_open = false; + return !_read_open; + } +}; +} // namespace internal +/// \endcond + +template <typename T> +class pipe; + +/// \brief Read side of a \ref seastar::pipe +/// +/// The read side of a pipe, which allows only reading from the pipe. +/// A pipe_reader object cannot be created separately, but only as part of a +/// reader/writer pair through \ref seastar::pipe. +template <typename T> +class pipe_reader { +private: + internal::pipe_buffer<T> *_bufp; + std::optional<T> _unread; + pipe_reader(internal::pipe_buffer<T> *bufp) noexcept : _bufp(bufp) { } + friend class pipe<T>; +public: + /// \brief Read next item from the pipe + /// + /// Returns a future value, which is fulfilled when the pipe's buffer + /// becomes non-empty, or the write side is closed. The value returned + /// is an optional<T>, which is disengaged to mark and end of file + /// (i.e., the write side was closed, and we've read everything it sent). + future<std::optional<T>> read() { + if (_unread) { + auto ret = std::move(*_unread); + _unread = {}; + return make_ready_future<std::optional<T>>(std::move(ret)); + } + if (_bufp->readable()) { + return _bufp->read(); + } else { + return make_ready_future<std::optional<T>>(); + } + } + /// \brief Return an item to the front of the pipe + /// + /// Pushes the given item to the front of the pipe, so it will be + /// returned by the next read() call. The typical use case is to + /// unread() the last item returned by read(). + /// More generally, it is legal to unread() any item, not just one + /// previously returned by read(), but note that the unread() is limited + /// to just one item - two calls to unread() without an intervening call + /// to read() will cause an exception. + void unread(T&& item) { + if (_unread) { + throw unread_overflow_exception(); + } + _unread = std::move(item); + } + ~pipe_reader() { + if (_bufp && _bufp->close_read()) { + delete _bufp; + } + } + // Allow move, but not copy, of pipe_reader + pipe_reader(pipe_reader&& other) noexcept : _bufp(other._bufp) { + other._bufp = nullptr; + } + pipe_reader& operator=(pipe_reader&& other) noexcept { + std::swap(_bufp, other._bufp); + return *this; + } +}; + +/// \brief Write side of a \ref seastar::pipe +/// +/// The write side of a pipe, which allows only writing to the pipe. +/// A pipe_writer object cannot be created separately, but only as part of a +/// reader/writer pair through \ref seastar::pipe. +template <typename T> +class pipe_writer { +private: + internal::pipe_buffer<T> *_bufp; + pipe_writer(internal::pipe_buffer<T> *bufp) noexcept : _bufp(bufp) { } + friend class pipe<T>; +public: + /// \brief Write an item to the pipe + /// + /// Returns a future value, which is fulfilled when the data was written + /// to the buffer (when it become non-full). If the data could not be + /// written because the read side was closed, an exception + /// \ref broken_pipe_exception is returned in the future. + future<> write(T&& data) { + if (_bufp->writeable()) { + return _bufp->write(std::move(data)); + } else { + return make_exception_future<>(broken_pipe_exception()); + } + } + ~pipe_writer() { + if (_bufp && _bufp->close_write()) { + delete _bufp; + } + } + // Allow move, but not copy, of pipe_writer + pipe_writer(pipe_writer&& other) noexcept : _bufp(other._bufp) { + other._bufp = nullptr; + } + pipe_writer& operator=(pipe_writer&& other) noexcept { + std::swap(_bufp, other._bufp); + return *this; + } +}; + +/// \brief A fixed-size pipe for communicating between two fibers. +/// +/// A pipe<T> is a mechanism to transfer data between two fibers, one +/// producing data, and the other consuming it. The fixed-size buffer also +/// ensures a balanced execution of the two fibers, because the producer +/// fiber blocks when it writes to a full pipe, until the consumer fiber gets +/// to run and read from the pipe. +/// +/// A pipe<T> resembles a Unix pipe, in that it has a read side, a write side, +/// and a fixed-sized buffer between them, and supports either end to be closed +/// independently (and EOF or broken pipe when using the other side). +/// A pipe<T> object holds the reader and write sides of the pipe as two +/// separate objects. These objects can be moved into two different fibers. +/// Importantly, if one of the pipe ends is destroyed (i.e., the continuations +/// capturing it end), the other end of the pipe will stop blocking, so the +/// other fiber will not hang. +/// +/// The pipe's read and write interfaces are future-based blocking. I.e., the +/// write() and read() methods return a future which is fulfilled when the +/// operation is complete. The pipe is single-reader single-writer, meaning +/// that until the future returned by read() is fulfilled, read() must not be +/// called again (and same for write). +/// +/// Note: The pipe reader and writer are movable, but *not* copyable. It is +/// often convenient to wrap each end in a shared pointer, so it can be +/// copied (e.g., used in an std::function which needs to be copyable) or +/// easily captured into multiple continuations. +template <typename T> +class pipe { +public: + pipe_reader<T> reader; + pipe_writer<T> writer; + explicit pipe(size_t size) : pipe(new internal::pipe_buffer<T>(size)) { } +private: + pipe(internal::pipe_buffer<T> *bufp) noexcept : reader(bufp), writer(bufp) { } +}; + + +/// @} + +} // namespace seastar diff --git a/src/seastar/include/seastar/core/polymorphic_temporary_buffer.hh b/src/seastar/include/seastar/core/polymorphic_temporary_buffer.hh new file mode 100644 index 000000000..7fe09625a --- /dev/null +++ b/src/seastar/include/seastar/core/polymorphic_temporary_buffer.hh @@ -0,0 +1,44 @@ +/* + * 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) 2019 Elazar Leibovich + */ + +#pragma once + +#include <seastar/core/memory.hh> +#include <seastar/core/temporary_buffer.hh> +#include <seastar/util/std-compat.hh> + +namespace seastar { + +/// Creates a `temporary_buffer` allocated by a custom allocator +/// +/// \param allocator allocator to use when allocating the temporary_buffer +/// \param size size of the temporary buffer +template <typename CharType> +temporary_buffer<CharType> make_temporary_buffer(std::pmr::polymorphic_allocator<CharType>* allocator, std::size_t size) { + if (allocator == memory::malloc_allocator) { + return temporary_buffer<CharType>(size); + } + CharType *buffer = allocator->allocate(size); + return temporary_buffer<CharType>(buffer, size, + make_deleter(deleter(), [allocator, buffer, size] () mutable { allocator->deallocate(buffer, size); })); +} + +} diff --git a/src/seastar/include/seastar/core/posix.hh b/src/seastar/include/seastar/core/posix.hh new file mode 100644 index 000000000..67132c967 --- /dev/null +++ b/src/seastar/include/seastar/core/posix.hh @@ -0,0 +1,514 @@ +/* + * 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) 2016 ScyllaDB + */ + +#pragma once + +#include <set> +#include <seastar/core/sstring.hh> +#include "abort_on_ebadf.hh" +#include <sys/types.h> +#include <sys/stat.h> +#include <unistd.h> +#include <assert.h> +#include <utility> +#include <fcntl.h> +#include <sys/ioctl.h> +#include <sys/eventfd.h> +#include <sys/timerfd.h> +#include <sys/socket.h> +#include <sys/epoll.h> +#include <sys/mman.h> +#include <signal.h> +#include <system_error> +#include <pthread.h> +#include <signal.h> +#include <spawn.h> +#include <memory> +#include <chrono> +#include <sys/uio.h> + +#include <seastar/net/socket_defs.hh> +#include <seastar/util/std-compat.hh> + +namespace seastar { + +/// \file +/// \defgroup posix-support POSIX Support +/// +/// Mostly-internal APIs to provide C++ glue for the underlying POSIX platform; +/// but can be used by the application when they don't block. +/// +/// \addtogroup posix-support +/// @{ + +inline void throw_system_error_on(bool condition, const char* what_arg = ""); + +template <typename T> +inline void throw_kernel_error(T r); + +template <typename T> +inline void throw_pthread_error(T r); + +struct mmap_deleter { + size_t _size; + void operator()(void* ptr) const; +}; + +using mmap_area = std::unique_ptr<char[], mmap_deleter>; + +mmap_area mmap_anonymous(void* addr, size_t length, int prot, int flags); + +class file_desc { + int _fd; +public: + file_desc() = delete; + file_desc(const file_desc&) = delete; + file_desc(file_desc&& x) noexcept : _fd(x._fd) { x._fd = -1; } + ~file_desc() { if (_fd != -1) { ::close(_fd); } } + void operator=(const file_desc&) = delete; + file_desc& operator=(file_desc&& x) { + if (this != &x) { + std::swap(_fd, x._fd); + if (x._fd != -1) { + x.close(); + } + } + return *this; + } + void close() { + assert(_fd != -1); + auto r = ::close(_fd); + throw_system_error_on(r == -1, "close"); + _fd = -1; + } + int get() const { return _fd; } + + sstring fdinfo() const noexcept; + + static file_desc from_fd(int fd) { + return file_desc(fd); + } + + static file_desc open(sstring name, int flags, mode_t mode = 0) { + int fd = ::open(name.c_str(), flags, mode); + throw_system_error_on(fd == -1, "open"); + return file_desc(fd); + } + static file_desc socket(int family, int type, int protocol = 0) { + int fd = ::socket(family, type, protocol); + throw_system_error_on(fd == -1, "socket"); + return file_desc(fd); + } + static file_desc eventfd(unsigned initval, int flags) { + int fd = ::eventfd(initval, flags); + throw_system_error_on(fd == -1, "eventfd"); + return file_desc(fd); + } + static file_desc epoll_create(int flags = 0) { + int fd = ::epoll_create1(flags); + throw_system_error_on(fd == -1, "epoll_create1"); + return file_desc(fd); + } + static file_desc timerfd_create(int clockid, int flags) { + int fd = ::timerfd_create(clockid, flags); + throw_system_error_on(fd == -1, "timerfd_create"); + return file_desc(fd); + } + static file_desc temporary(sstring directory); + file_desc dup() const { + int fd = ::dup(get()); + throw_system_error_on(fd == -1, "dup"); + return file_desc(fd); + } + file_desc accept(socket_address& sa, int flags = 0) { + auto ret = ::accept4(_fd, &sa.as_posix_sockaddr(), &sa.addr_length, flags); + throw_system_error_on(ret == -1, "accept4"); + return file_desc(ret); + } + static file_desc inotify_init(int flags); + // return nullopt if no connection is availbale to be accepted + std::optional<file_desc> try_accept(socket_address& sa, int flags = 0) { + auto ret = ::accept4(_fd, &sa.as_posix_sockaddr(), &sa.addr_length, flags); + if (ret == -1 && errno == EAGAIN) { + return {}; + } + throw_system_error_on(ret == -1, "accept4"); + return file_desc(ret); + } + void shutdown(int how) { + auto ret = ::shutdown(_fd, how); + if (ret == -1 && errno != ENOTCONN) { + throw_system_error_on(ret == -1, "shutdown"); + } + } + void truncate(size_t size) { + auto ret = ::ftruncate(_fd, size); + throw_system_error_on(ret, "ftruncate"); + } + int ioctl(int request) { + return ioctl(request, 0); + } + int ioctl(int request, int value) { + int r = ::ioctl(_fd, request, value); + throw_system_error_on(r == -1, "ioctl"); + return r; + } + int ioctl(int request, unsigned int value) { + int r = ::ioctl(_fd, request, value); + throw_system_error_on(r == -1, "ioctl"); + return r; + } + template <class X> + int ioctl(int request, X& data) { + int r = ::ioctl(_fd, request, &data); + throw_system_error_on(r == -1, "ioctl"); + return r; + } + template <class X> + int ioctl(int request, X&& data) { + int r = ::ioctl(_fd, request, &data); + throw_system_error_on(r == -1, "ioctl"); + return r; + } + template <class X> + int setsockopt(int level, int optname, X&& data) { + int r = ::setsockopt(_fd, level, optname, &data, sizeof(data)); + throw_system_error_on(r == -1, "setsockopt"); + return r; + } + int setsockopt(int level, int optname, const char* data) { + int r = ::setsockopt(_fd, level, optname, data, strlen(data) + 1); + throw_system_error_on(r == -1, "setsockopt"); + return r; + } + int setsockopt(int level, int optname, const void* data, socklen_t len) { + int r = ::setsockopt(_fd, level, optname, data, len); + throw_system_error_on(r == -1, "setsockopt"); + return r; + } + template <typename Data> + Data getsockopt(int level, int optname) { + Data data; + socklen_t len = sizeof(data); + memset(&data, 0, len); + int r = ::getsockopt(_fd, level, optname, &data, &len); + throw_system_error_on(r == -1, "getsockopt"); + return data; + } + int getsockopt(int level, int optname, char* data, socklen_t len) { + int r = ::getsockopt(_fd, level, optname, data, &len); + throw_system_error_on(r == -1, "getsockopt"); + return r; + } + size_t size() { + struct stat buf; + auto r = ::fstat(_fd, &buf); + throw_system_error_on(r == -1, "fstat"); + return buf.st_size; + } + std::optional<size_t> read(void* buffer, size_t len) { + auto r = ::read(_fd, buffer, len); + if (r == -1 && errno == EAGAIN) { + return {}; + } + throw_system_error_on(r == -1, "read"); + return { size_t(r) }; + } + std::optional<ssize_t> recv(void* buffer, size_t len, int flags) { + auto r = ::recv(_fd, buffer, len, flags); + if (r == -1 && errno == EAGAIN) { + return {}; + } + throw_system_error_on(r == -1, "recv"); + return { ssize_t(r) }; + } + std::optional<size_t> recvmsg(msghdr* mh, int flags) { + auto r = ::recvmsg(_fd, mh, flags); + if (r == -1 && errno == EAGAIN) { + return {}; + } + throw_system_error_on(r == -1, "recvmsg"); + return { size_t(r) }; + } + std::optional<size_t> send(const void* buffer, size_t len, int flags) { + auto r = ::send(_fd, buffer, len, flags); + if (r == -1 && errno == EAGAIN) { + return {}; + } + throw_system_error_on(r == -1, "send"); + return { size_t(r) }; + } + std::optional<size_t> sendto(socket_address& addr, const void* buf, size_t len, int flags) { + auto r = ::sendto(_fd, buf, len, flags, &addr.u.sa, addr.length()); + if (r == -1 && errno == EAGAIN) { + return {}; + } + throw_system_error_on(r == -1, "sendto"); + return { size_t(r) }; + } + std::optional<size_t> sendmsg(const msghdr* msg, int flags) { + auto r = ::sendmsg(_fd, msg, flags); + if (r == -1 && errno == EAGAIN) { + return {}; + } + throw_system_error_on(r == -1, "sendmsg"); + return { size_t(r) }; + } + void bind(sockaddr& sa, socklen_t sl) { + auto r = ::bind(_fd, &sa, sl); + throw_system_error_on(r == -1, "bind"); + } + void connect(sockaddr& sa, socklen_t sl) { + auto r = ::connect(_fd, &sa, sl); + if (r == -1 && errno == EINPROGRESS) { + return; + } + throw_system_error_on(r == -1, "connect"); + } + socket_address get_address() { + socket_address addr; + auto r = ::getsockname(_fd, &addr.u.sa, &addr.addr_length); + throw_system_error_on(r == -1, "getsockname"); + return addr; + } + void listen(int backlog) { + auto fd = ::listen(_fd, backlog); + throw_system_error_on(fd == -1, "listen"); + } + std::optional<size_t> write(const void* buf, size_t len) { + auto r = ::write(_fd, buf, len); + if (r == -1 && errno == EAGAIN) { + return {}; + } + throw_system_error_on(r == -1, "write"); + return { size_t(r) }; + } + std::optional<size_t> writev(const iovec *iov, int iovcnt) { + auto r = ::writev(_fd, iov, iovcnt); + if (r == -1 && errno == EAGAIN) { + return {}; + } + throw_system_error_on(r == -1, "writev"); + return { size_t(r) }; + } + size_t pread(void* buf, size_t len, off_t off) { + auto r = ::pread(_fd, buf, len, off); + throw_system_error_on(r == -1, "pread"); + return size_t(r); + } + void timerfd_settime(int flags, const itimerspec& its) { + auto r = ::timerfd_settime(_fd, flags, &its, NULL); + throw_system_error_on(r == -1, "timerfd_settime"); + } + + mmap_area map(size_t size, unsigned prot, unsigned flags, size_t offset, + void* addr = nullptr) { + void *x = mmap(addr, size, prot, flags, _fd, offset); + throw_system_error_on(x == MAP_FAILED, "mmap"); + return mmap_area(static_cast<char*>(x), mmap_deleter{size}); + } + + mmap_area map_shared_rw(size_t size, size_t offset) { + return map(size, PROT_READ | PROT_WRITE, MAP_SHARED, offset); + } + + mmap_area map_shared_ro(size_t size, size_t offset) { + return map(size, PROT_READ, MAP_SHARED, offset); + } + + mmap_area map_private_rw(size_t size, size_t offset) { + return map(size, PROT_READ | PROT_WRITE, MAP_PRIVATE, offset); + } + + mmap_area map_private_ro(size_t size, size_t offset) { + return map(size, PROT_READ, MAP_PRIVATE, offset); + } + + void spawn_actions_add_close(posix_spawn_file_actions_t* actions) { + auto r = ::posix_spawn_file_actions_addclose(actions, _fd); + throw_pthread_error(r); + } + + void spawn_actions_add_dup2(posix_spawn_file_actions_t* actions, int newfd) { + auto r = ::posix_spawn_file_actions_adddup2(actions, _fd, newfd); + throw_pthread_error(r); + } +private: + file_desc(int fd) : _fd(fd) {} + }; + + +namespace posix { + +static constexpr unsigned rcv_shutdown = 0x1; +static constexpr unsigned snd_shutdown = 0x2; +static inline constexpr unsigned shutdown_mask(int how) { return how + 1; } + +/// Converts a duration value to a `timespec` +/// +/// \param d a duration value to convert to the POSIX `timespec` format +/// \return `d` as a `timespec` value +template <typename Rep, typename Period> +struct timespec +to_timespec(std::chrono::duration<Rep, Period> d) { + auto ns = std::chrono::duration_cast<std::chrono::nanoseconds>(d).count(); + struct timespec ts {}; + ts.tv_sec = ns / 1000000000; + ts.tv_nsec = ns % 1000000000; + return ts; +} + +/// Converts a relative start time and an interval to an `itimerspec` +/// +/// \param base First expiration of the timer, relative to the current time +/// \param interval period for re-arming the timer +/// \return `base` and `interval` converted to an `itimerspec` +template <typename Rep1, typename Period1, typename Rep2, typename Period2> +struct itimerspec +to_relative_itimerspec(std::chrono::duration<Rep1, Period1> base, std::chrono::duration<Rep2, Period2> interval) { + struct itimerspec its {}; + its.it_interval = to_timespec(interval); + its.it_value = to_timespec(base); + return its; +} + + +/// Converts a time_point and a duration to an `itimerspec` +/// +/// \param base base time for the timer; must use the same clock as the timer +/// \param interval period for re-arming the timer +/// \return `base` and `interval` converted to an `itimerspec` +template <typename Clock, class Duration, class Rep, class Period> +struct itimerspec +to_absolute_itimerspec(std::chrono::time_point<Clock, Duration> base, std::chrono::duration<Rep, Period> interval) { + return to_relative_itimerspec(base.time_since_epoch(), interval); +} + +} + +class posix_thread { +public: + class attr; +private: + // must allocate, since this class is moveable + std::unique_ptr<std::function<void ()>> _func; + pthread_t _pthread; + bool _valid = true; + mmap_area _stack; +private: + static void* start_routine(void* arg) noexcept; +public: + posix_thread(std::function<void ()> func); + posix_thread(attr a, std::function<void ()> func); + posix_thread(posix_thread&& x); + ~posix_thread(); + void join(); +public: + class attr { + public: + struct stack_size { size_t size = 0; }; + attr() = default; + template <typename... A> + attr(A... a) { + set(std::forward<A>(a)...); + } + void set() {} + template <typename A, typename... Rest> + void set(A a, Rest... rest) { + set(std::forward<A>(a)); + set(std::forward<Rest>(rest)...); + } + void set(stack_size ss) { _stack_size = ss; } + private: + stack_size _stack_size; + friend class posix_thread; + }; +}; + + +inline +void throw_system_error_on(bool condition, const char* what_arg) { + if (condition) { + if ((errno == EBADF || errno == ENOTSOCK) && is_abort_on_ebadf_enabled()) { + abort(); + } + throw std::system_error(errno, std::system_category(), what_arg); + } +} + +template <typename T> +inline +void throw_kernel_error(T r) { + static_assert(std::is_signed<T>::value, "kernel error variables must be signed"); + if (r < 0) { + auto ec = -r; + if ((ec == EBADF || ec == ENOTSOCK) && is_abort_on_ebadf_enabled()) { + abort(); + } + throw std::system_error(-r, std::system_category()); + } +} + +template <typename T> +inline +void throw_pthread_error(T r) { + if (r != 0) { + throw std::system_error(r, std::system_category()); + } +} + +inline +sigset_t make_sigset_mask(int signo) { + sigset_t set; + sigemptyset(&set); + sigaddset(&set, signo); + return set; +} + +inline +sigset_t make_full_sigset_mask() { + sigset_t set; + sigfillset(&set); + return set; +} + +inline +sigset_t make_empty_sigset_mask() { + sigset_t set; + sigemptyset(&set); + return set; +} + +inline +void pin_this_thread(unsigned cpu_id) { + cpu_set_t cs; + CPU_ZERO(&cs); + CPU_SET(cpu_id, &cs); + auto r = pthread_setaffinity_np(pthread_self(), sizeof(cs), &cs); + assert(r == 0); + (void)r; +} + +std::set<unsigned> get_current_cpuset(); + +/// @} + +} diff --git a/src/seastar/include/seastar/core/preempt.hh b/src/seastar/include/seastar/core/preempt.hh new file mode 100644 index 000000000..a1c7bde05 --- /dev/null +++ b/src/seastar/include/seastar/core/preempt.hh @@ -0,0 +1,64 @@ +/* + * 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) 2016 ScyllaDB. + */ + +#pragma once +#include <atomic> + +namespace seastar { + +namespace internal { + +struct preemption_monitor { + // We preempt when head != tail + // This happens to match the Linux aio completion ring, so we can have the + // kernel preempt a task by queuing a completion event to an io_context. + std::atomic<uint32_t> head; + std::atomic<uint32_t> tail; +}; + +inline const preemption_monitor*& get_need_preempt_var() { + static preemption_monitor bootstrap_preemption_monitor; + static thread_local const preemption_monitor* g_need_preempt = &bootstrap_preemption_monitor; + return g_need_preempt; +} + +void set_need_preempt_var(const preemption_monitor* pm); + +} + +inline bool need_preempt() noexcept { +#ifndef SEASTAR_DEBUG + // prevent compiler from eliminating loads in a loop + std::atomic_signal_fence(std::memory_order_seq_cst); + auto np = internal::get_need_preempt_var(); + // We aren't reading anything from the ring, so we don't need + // any barriers. + auto head = np->head.load(std::memory_order_relaxed); + auto tail = np->tail.load(std::memory_order_relaxed); + // Possible optimization: read head and tail in a single 64-bit load, + // and find a funky way to compare the two 32-bit halves. + return __builtin_expect(head != tail, false); +#else + return true; +#endif +} + +} diff --git a/src/seastar/include/seastar/core/prefetch.hh b/src/seastar/include/seastar/core/prefetch.hh new file mode 100644 index 000000000..73f9abd3f --- /dev/null +++ b/src/seastar/include/seastar/core/prefetch.hh @@ -0,0 +1,115 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <atomic> +#include <boost/mpl/range_c.hpp> +#include <boost/mpl/for_each.hpp> +#include <seastar/core/align.hh> +#include <seastar/core/cacheline.hh> + +namespace seastar { + +template <size_t N, int RW, int LOC> +struct prefetcher; + +template<int RW, int LOC> +struct prefetcher<0, RW, LOC> { + prefetcher(uintptr_t ptr) {} +}; + +template <size_t N, int RW, int LOC> +struct prefetcher { + prefetcher(uintptr_t ptr) { + __builtin_prefetch(reinterpret_cast<void*>(ptr), RW, LOC); + std::atomic_signal_fence(std::memory_order_seq_cst); + prefetcher<N-cache_line_size, RW, LOC>(ptr + cache_line_size); + } +}; + +// LOC is a locality from __buitin_prefetch() gcc documentation: +// "The value locality must be a compile-time constant integer between zero and three. A value of +// zero means that the data has no temporal locality, so it need not be left in the cache after +// the access. A value of three means that the data has a high degree of temporal locality and +// should be left in all levels of cache possible. Values of one and two mean, respectively, a +// low or moderate degree of temporal locality. The default is three." +template<typename T, int LOC = 3> +void prefetch(T* ptr) { + prefetcher<align_up(sizeof(T), cache_line_size), 0, LOC>(reinterpret_cast<uintptr_t>(ptr)); +} + +template<typename Iterator, int LOC = 3> +void prefetch(Iterator begin, Iterator end) { + std::for_each(begin, end, [] (auto v) { prefetch<decltype(*v), LOC>(v); }); +} + +template<size_t C, typename T, int LOC = 3> +void prefetch_n(T** pptr) { + boost::mpl::for_each< boost::mpl::range_c<size_t,0,C> >( [pptr] (size_t x) { prefetch<T, LOC>(*(pptr + x)); } ); +} + +template<size_t L, int LOC = 3> +void prefetch(void* ptr) { + prefetcher<L*cache_line_size, 0, LOC>(reinterpret_cast<uintptr_t>(ptr)); +} + +template<size_t L, typename Iterator, int LOC = 3> +void prefetch_n(Iterator begin, Iterator end) { + std::for_each(begin, end, [] (auto v) { prefetch<L, LOC>(v); }); +} + +template<size_t L, size_t C, typename T, int LOC = 3> +void prefetch_n(T** pptr) { + boost::mpl::for_each< boost::mpl::range_c<size_t,0,C> >( [pptr] (size_t x) { prefetch<L, LOC>(*(pptr + x)); } ); +} + +template<typename T, int LOC = 3> +void prefetchw(T* ptr) { + prefetcher<align_up(sizeof(T), cache_line_size), 1, LOC>(reinterpret_cast<uintptr_t>(ptr)); +} + +template<typename Iterator, int LOC = 3> +void prefetchw_n(Iterator begin, Iterator end) { + std::for_each(begin, end, [] (auto v) { prefetchw<decltype(*v), LOC>(v); }); +} + +template<size_t C, typename T, int LOC = 3> +void prefetchw_n(T** pptr) { + boost::mpl::for_each< boost::mpl::range_c<size_t,0,C> >( [pptr] (size_t x) { prefetchw<T, LOC>(*(pptr + x)); } ); +} + +template<size_t L, int LOC = 3> +void prefetchw(void* ptr) { + prefetcher<L*cache_line_size, 1, LOC>(reinterpret_cast<uintptr_t>(ptr)); +} + +template<size_t L, typename Iterator, int LOC = 3> +void prefetchw_n(Iterator begin, Iterator end) { + std::for_each(begin, end, [] (auto v) { prefetchw<L, LOC>(v); }); +} + +template<size_t L, size_t C, typename T, int LOC = 3> +void prefetchw_n(T** pptr) { + boost::mpl::for_each< boost::mpl::range_c<size_t,0,C> >( [pptr] (size_t x) { prefetchw<L, LOC>(*(pptr + x)); } ); +} + +} diff --git a/src/seastar/include/seastar/core/print.hh b/src/seastar/include/seastar/core/print.hh new file mode 100644 index 000000000..0b123c60e --- /dev/null +++ b/src/seastar/include/seastar/core/print.hh @@ -0,0 +1,157 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <fmt/ostream.h> +#include <fmt/printf.h> +#include <iostream> +#include <iomanip> +#include <chrono> +#include <sstream> +#include <seastar/core/sstring.hh> + +#if 0 +inline +std::ostream& +operator<<(std::ostream& os, const void* ptr) { + auto flags = os.flags(); + os << "0x" << std::hex << reinterpret_cast<uintptr_t>(ptr); + os.flags(flags); + return os; +} +#endif + +inline +std::ostream& +operator<<(std::ostream&& os, const void* ptr) { + return os << ptr; // selects non-rvalue version +} + +namespace seastar { + +template <typename... A> +[[deprecated("use std::format_to() or fmt::print()")]] +std::ostream& +fprint(std::ostream& os, const char* fmt, A&&... a) { + ::fmt::fprintf(os, fmt, std::forward<A>(a)...); + return os; +} + +template <typename... A> +[[deprecated("use std::format_to() or fmt::print()")]] +void +print(const char* fmt, A&&... a) { + ::fmt::printf(fmt, std::forward<A>(a)...); +} + +template <typename... A> +[[deprecated("use std::format() or fmt::format()")]] +std::string +sprint(const char* fmt, A&&... a) { + std::ostringstream os; + ::fmt::fprintf(os, fmt, std::forward<A>(a)...); + return os.str(); +} + +template <typename... A> +[[deprecated("use std::format() or fmt::format()")]] +std::string +sprint(const sstring& fmt, A&&... a) { + std::ostringstream os; + ::fmt::fprintf(os, fmt.c_str(), std::forward<A>(a)...); + return os.str(); +} + +template <typename Iterator> +std::string +format_separated(Iterator b, Iterator e, const char* sep = ", ") { + std::string ret; + if (b == e) { + return ret; + } + ret += *b++; + while (b != e) { + ret += sep; + ret += *b++; + } + return ret; +} + +template <typename TimePoint> +struct usecfmt_wrapper { + TimePoint val; +}; + +template <typename TimePoint> +inline +usecfmt_wrapper<TimePoint> +usecfmt(TimePoint tp) { + return { tp }; +}; + +template <typename Clock, typename Rep, typename Period> +std::ostream& +operator<<(std::ostream& os, usecfmt_wrapper<std::chrono::time_point<Clock, std::chrono::duration<Rep, Period>>> tp) { + auto usec = std::chrono::duration_cast<std::chrono::microseconds>(tp.val.time_since_epoch()).count(); + std::ostream tmp(os.rdbuf()); + tmp << std::setw(12) << (usec / 1000000) << "." << std::setw(6) << std::setfill('0') << (usec % 1000000); + return os; +} + +template <typename... A> +void +log(A&&... a) { + std::cout << usecfmt(std::chrono::high_resolution_clock::now()) << " "; + print(std::forward<A>(a)...); +} + +/** + * Evaluate the formatted string in a native fmt library format + * + * @param fmt format string with the native fmt library syntax + * @param a positional parameters + * + * @return sstring object with the result of applying the given positional + * parameters on a given format string. + */ +template <typename... A> +sstring +format(const char* fmt, A&&... a) { + fmt::memory_buffer out; +#if FMT_VERSION >= 80000 + fmt::format_to(fmt::appender(out), fmt::runtime(fmt), std::forward<A>(a)...); +#else + fmt::format_to(out, fmt, std::forward<A>(a)...); +#endif + return sstring{out.data(), out.size()}; +} + +// temporary, use fmt::print() instead +template <typename... A> +[[deprecated("use std::format() or fmt::print()")]] +std::ostream& +fmt_print(std::ostream& os, const char* format, A&&... a) { + fmt::print(os, format, std::forward<A>(a)...); + return os; +} + +} diff --git a/src/seastar/include/seastar/core/prometheus.hh b/src/seastar/include/seastar/core/prometheus.hh new file mode 100644 index 000000000..187af568f --- /dev/null +++ b/src/seastar/include/seastar/core/prometheus.hh @@ -0,0 +1,51 @@ +/* + * 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) 2016 ScyllaDB + */ + +#pragma once + +#include <seastar/http/httpd.hh> +#include <seastar/core/metrics.hh> +#include <seastar/util/std-compat.hh> + +namespace seastar { + +namespace prometheus { + +/*! + * Holds prometheus related configuration + */ +struct config { + sstring metric_help; //!< Default help message for the returned metrics + sstring hostname; //!< hostname is deprecated, use label instead + std::optional<metrics::label_instance> label; //!< A label that will be added to all metrics, we advice not to use it and set it on the prometheus server + sstring prefix = "seastar"; //!< a prefix that will be added to metric names +}; + +future<> start(httpd::http_server_control& http_server, config ctx); + +/// \defgroup add_prometheus_routes adds a /metrics endpoint that returns prometheus metrics +/// in txt format +/// @{ +future<> add_prometheus_routes(distributed<http_server>& server, config ctx); +future<> add_prometheus_routes(http_server& server, config ctx); +/// @} +} +} diff --git a/src/seastar/include/seastar/core/queue.hh b/src/seastar/include/seastar/core/queue.hh new file mode 100644 index 000000000..b341307df --- /dev/null +++ b/src/seastar/include/seastar/core/queue.hh @@ -0,0 +1,332 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/core/circular_buffer.hh> +#include <seastar/core/future.hh> +#include <queue> +#include <seastar/util/std-compat.hh> + +namespace seastar { + +/// Asynchronous single-producer single-consumer queue with limited capacity. +/// There can be at most one producer-side and at most one consumer-side operation active at any time. +/// Operations returning a future are considered to be active until the future resolves. +/// +/// Note: queue requires the data type T to be nothrow move constructible as it's +/// returned as future<T> by \ref pop_eventually and seastar futurized data type +/// are required to be nothrow move-constructible. +template <typename T> +SEASTAR_CONCEPT(requires std::is_nothrow_move_constructible_v<T>) +class queue { + std::queue<T, circular_buffer<T>> _q; + size_t _max; + std::optional<promise<>> _not_empty; + std::optional<promise<>> _not_full; + std::exception_ptr _ex = nullptr; +private: + void notify_not_empty() noexcept; + void notify_not_full() noexcept; +public: + explicit queue(size_t size); + + /// \brief Push an item. + /// + /// Returns false if the queue was full and the item was not pushed. + bool push(T&& a); + + /// \brief Pop an item. + /// + /// Popping from an empty queue will result in undefined behavior. + T pop() noexcept; + + /// \brief access the front element in the queue + /// + /// Accessing the front of an empty or aborted queue will result in undefined + /// behaviour. + T& front() noexcept; + + /// Consumes items from the queue, passing them to \c func, until \c func + /// returns false or the queue it empty + /// + /// Returns false if func returned false. + template <typename Func> + bool consume(Func&& func); + + /// Returns true when the queue is empty. + bool empty() const noexcept; + + /// Returns true when the queue is full. + bool full() const noexcept; + + /// Returns a future<> that becomes available when pop() or consume() + /// can be called. + /// A consumer-side operation. Cannot be called concurrently with other consumer-side operations. + future<> not_empty() noexcept; + + /// Returns a future<> that becomes available when push() can be called. + /// A producer-side operation. Cannot be called concurrently with other producer-side operations. + future<> not_full() noexcept; + + /// Pops element now or when there is some. Returns a future that becomes + /// available when some element is available. + /// If the queue is, or already was, abort()ed, the future resolves with + /// the exception provided to abort(). + /// A consumer-side operation. Cannot be called concurrently with other consumer-side operations. + future<T> pop_eventually() noexcept; + + /// Pushes the element now or when there is room. Returns a future<> which + /// resolves when data was pushed. + /// If the queue is, or already was, abort()ed, the future resolves with + /// the exception provided to abort(). + /// A producer-side operation. Cannot be called concurrently with other producer-side operations. + future<> push_eventually(T&& data) noexcept; + + /// Returns the number of items currently in the queue. + size_t size() const noexcept { + // std::queue::size() has no reason to throw + return _q.size(); + } + + /// Returns the size limit imposed on the queue during its construction + /// or by a call to set_max_size(). If the queue contains max_size() + /// items (or more), further items cannot be pushed until some are popped. + size_t max_size() const noexcept { return _max; } + + /// Set the maximum size to a new value. If the queue's max size is reduced, + /// items already in the queue will not be expunged and the queue will be temporarily + /// bigger than its max_size. + void set_max_size(size_t max) noexcept { + _max = max; + if (!full()) { + notify_not_full(); + } + } + + /// Destroy any items in the queue, and pass the provided exception to any + /// waiting readers or writers - or to any later read or write attempts. + void abort(std::exception_ptr ex) noexcept { + // std::queue::empty() and pop() doesn't throw + // since it just calls seastar::circular_buffer::pop_front + // that is specified as noexcept. + while (!_q.empty()) { + _q.pop(); + } + _ex = ex; + if (_not_full) { + _not_full->set_exception(ex); + _not_full= std::nullopt; + } + if (_not_empty) { + _not_empty->set_exception(std::move(ex)); + _not_empty = std::nullopt; + } + } + + /// \brief Check if there is an active consumer + /// + /// Returns true if another fiber waits for an item to be pushed into the queue + bool has_blocked_consumer() const noexcept { + return bool(_not_empty); + } +}; + +template <typename T> +SEASTAR_CONCEPT(requires std::is_nothrow_move_constructible_v<T>) +inline +queue<T>::queue(size_t size) + : _max(size) { +} + +template <typename T> +SEASTAR_CONCEPT(requires std::is_nothrow_move_constructible_v<T>) +inline +void queue<T>::notify_not_empty() noexcept { + if (_not_empty) { + _not_empty->set_value(); + _not_empty = std::optional<promise<>>(); + } +} + +template <typename T> +SEASTAR_CONCEPT(requires std::is_nothrow_move_constructible_v<T>) +inline +void queue<T>::notify_not_full() noexcept { + if (_not_full) { + _not_full->set_value(); + _not_full = std::optional<promise<>>(); + } +} + +template <typename T> +SEASTAR_CONCEPT(requires std::is_nothrow_move_constructible_v<T>) +inline +bool queue<T>::push(T&& data) { + if (_q.size() < _max) { + _q.push(std::move(data)); + notify_not_empty(); + return true; + } else { + return false; + } +} + +template <typename T> +SEASTAR_CONCEPT(requires std::is_nothrow_move_constructible_v<T>) +inline +T& queue<T>::front() noexcept { + // std::queue::front() has no reason to throw + return _q.front(); +} + +template <typename T> +SEASTAR_CONCEPT(requires std::is_nothrow_move_constructible_v<T>) +inline +T queue<T>::pop() noexcept { + if (_q.size() == _max) { + notify_not_full(); + } + // popping the front element must not throw + // as T is required to be nothrow_move_constructible + // and std::queue::pop won't throw since it uses + // seastar::circular_beffer::pop_front. + assert(!_q.empty()); + T data = std::move(_q.front()); + _q.pop(); + return data; +} + +template <typename T> +SEASTAR_CONCEPT(requires std::is_nothrow_move_constructible_v<T>) +inline +future<T> queue<T>::pop_eventually() noexcept { + // seastar allows only nothrow_move_constructible types + // to be returned as future<T> + static_assert(std::is_nothrow_move_constructible_v<T>, + "Queue element type must be no-throw move constructible"); + + if (_ex) { + return make_exception_future<T>(_ex); + } + if (empty()) { + return not_empty().then([this] { + if (_ex) { + return make_exception_future<T>(_ex); + } else { + return make_ready_future<T>(pop()); + } + }); + } else { + return make_ready_future<T>(pop()); + } +} + +template <typename T> +SEASTAR_CONCEPT(requires std::is_nothrow_move_constructible_v<T>) +inline +future<> queue<T>::push_eventually(T&& data) noexcept { + if (_ex) { + return make_exception_future<>(_ex); + } + if (full()) { + return not_full().then([this, data = std::move(data)] () mutable { + _q.push(std::move(data)); + notify_not_empty(); + }); + } else { + try { + _q.push(std::move(data)); + notify_not_empty(); + return make_ready_future<>(); + } catch (...) { + return current_exception_as_future(); + } + } +} + +template <typename T> +SEASTAR_CONCEPT(requires std::is_nothrow_move_constructible_v<T>) +template <typename Func> +inline +bool queue<T>::consume(Func&& func) { + if (_ex) { + std::rethrow_exception(_ex); + } + bool running = true; + while (!_q.empty() && running) { + running = func(std::move(_q.front())); + _q.pop(); + } + if (!full()) { + notify_not_full(); + } + return running; +} + +template <typename T> +SEASTAR_CONCEPT(requires std::is_nothrow_move_constructible_v<T>) +inline +bool queue<T>::empty() const noexcept { + // std::queue::empty() has no reason to throw + return _q.empty(); +} + +template <typename T> +SEASTAR_CONCEPT(requires std::is_nothrow_move_constructible_v<T>) +inline +bool queue<T>::full() const noexcept { + // std::queue::size() has no reason to throw + return _q.size() >= _max; +} + +template <typename T> +SEASTAR_CONCEPT(requires std::is_nothrow_move_constructible_v<T>) +inline +future<> queue<T>::not_empty() noexcept { + if (_ex) { + return make_exception_future<>(_ex); + } + if (!empty()) { + return make_ready_future<>(); + } else { + _not_empty = promise<>(); + return _not_empty->get_future(); + } +} + +template <typename T> +SEASTAR_CONCEPT(requires std::is_nothrow_move_constructible_v<T>) +inline +future<> queue<T>::not_full() noexcept { + if (_ex) { + return make_exception_future<>(_ex); + } + if (!full()) { + return make_ready_future<>(); + } else { + _not_full = promise<>(); + return _not_full->get_future(); + } +} + +} + diff --git a/src/seastar/include/seastar/core/ragel.hh b/src/seastar/include/seastar/core/ragel.hh new file mode 100644 index 000000000..e6b47e743 --- /dev/null +++ b/src/seastar/include/seastar/core/ragel.hh @@ -0,0 +1,152 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/core/sstring.hh> +#include <seastar/core/temporary_buffer.hh> +#include <seastar/util/eclipse.hh> +#include <algorithm> +#include <memory> +#include <cassert> +#include <seastar/util/std-compat.hh> +#include <seastar/core/future.hh> + +namespace seastar { + +// Support classes for Ragel parsers + +// Builds an sstring that can be scattered across multiple packets. +// +// Use a sstring_build::guard variable to designate each scattered +// char array, and call mark_start() and mark_end() at the start +// and end points, respectively. sstring_builder will collect data +// from intervening segments, if needed. +// +// After mark_end() has been called, use the get() method to obtain +// the built string. +// +// FIXME: switch to string_view. +// +class sstring_builder { + sstring _value; + const char* _start = nullptr; +public: + class guard; +public: + sstring get() && { + return std::move(_value); + } + void reset() { + _value = {}; + _start = nullptr; + } + friend class guard; +}; + +class sstring_builder::guard { + sstring_builder& _builder; + const char* _block_end; +public: + guard(sstring_builder& builder, const char* block_start, const char* block_end) + : _builder(builder), _block_end(block_end) { + if (!_builder._value.empty()) { + mark_start(block_start); + } + } + ~guard() { + if (_builder._start) { + mark_end(_block_end); + } + } + void mark_start(const char* p) { + _builder._start = p; + } + void mark_end(const char* p) { + if (_builder._value.empty()) { + // avoid an allocation in the common case + _builder._value = sstring(_builder._start, p); + } else { + _builder._value += sstring(_builder._start, p); + } + _builder._start = nullptr; + } +}; + + +// CRTP +template <typename ConcreteParser> +class ragel_parser_base { +protected: + int _fsm_cs; + std::unique_ptr<int[]> _fsm_stack = nullptr; + int _fsm_stack_size = 0; + int _fsm_top; + int _fsm_act; + char* _fsm_ts; + char* _fsm_te; + sstring_builder _builder; +protected: + void init_base() { + _builder.reset(); + } + void prepush() { + if (_fsm_top == _fsm_stack_size) { + auto old = _fsm_stack_size; + _fsm_stack_size = std::max(_fsm_stack_size * 2, 16); + assert(_fsm_stack_size > old); + std::unique_ptr<int[]> new_stack{new int[_fsm_stack_size]}; + std::copy(_fsm_stack.get(), _fsm_stack.get() + _fsm_top, new_stack.get()); + std::swap(_fsm_stack, new_stack); + } + } + void postpop() {} + sstring get_str() { + return std::move(_builder).get(); + } +public: + using unconsumed_remainder = std::optional<temporary_buffer<char>>; + future<unconsumed_remainder> operator()(temporary_buffer<char> buf) { + char* p = buf.get_write(); + char* pe = p + buf.size(); + char* eof = buf.empty() ? pe : nullptr; + char* parsed = static_cast<ConcreteParser*>(this)->parse(p, pe, eof); + if (parsed) { + buf.trim_front(parsed - p); + return make_ready_future<unconsumed_remainder>(std::move(buf)); + } + return make_ready_future<unconsumed_remainder>(); + } +}; + +inline void trim_trailing_spaces_and_tabs(sstring& str) { + auto data = str.data(); + size_t i; + for (i = str.size(); i > 0; --i) { + auto c = data[i-1]; + if (!(c == ' ' || c == '\t')) { + break; + } + } + str.resize(i); +} + +} diff --git a/src/seastar/include/seastar/core/reactor.hh b/src/seastar/include/seastar/core/reactor.hh new file mode 100644 index 000000000..559798024 --- /dev/null +++ b/src/seastar/include/seastar/core/reactor.hh @@ -0,0 +1,733 @@ +/* + * 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 2014 Cloudius Systems + */ + +#pragma once + +#include <seastar/core/seastar.hh> +#include <seastar/core/iostream.hh> +#include <seastar/core/aligned_buffer.hh> +#include <seastar/core/cacheline.hh> +#include <seastar/core/circular_buffer_fixed_capacity.hh> +#include <seastar/core/idle_cpu_handler.hh> +#include <memory> +#include <type_traits> +#include <sys/epoll.h> +#include <sys/types.h> +#include <sys/socket.h> +#include <unordered_map> +#include <netinet/ip.h> +#include <cstring> +#include <cassert> +#include <stdexcept> +#include <unistd.h> +#include <vector> +#include <queue> +#include <algorithm> +#include <thread> +#include <system_error> +#include <chrono> +#include <ratio> +#include <atomic> +#include <stack> +#include <seastar/util/std-compat.hh> +#include <boost/next_prior.hpp> +#include <boost/lockfree/spsc_queue.hpp> +#include <boost/thread/barrier.hpp> +#include <boost/container/static_vector.hpp> +#include <set> +#include <seastar/core/reactor_config.hh> +#include <seastar/core/linux-aio.hh> +#include <seastar/util/eclipse.hh> +#include <seastar/core/future.hh> +#include <seastar/core/posix.hh> +#include <seastar/core/sstring.hh> +#include <seastar/net/api.hh> +#include <seastar/core/temporary_buffer.hh> +#include <seastar/core/circular_buffer.hh> +#include <seastar/core/file.hh> +#include <seastar/core/semaphore.hh> +#include <seastar/core/fair_queue.hh> +#include <seastar/core/scattered_message.hh> +#include <seastar/core/enum.hh> +#include <seastar/core/memory.hh> +#include <seastar/core/thread_cputime_clock.hh> +#include <boost/range/irange.hpp> +#include <seastar/core/timer.hh> +#include <seastar/core/condition-variable.hh> +#include <seastar/util/log.hh> +#include <seastar/core/lowres_clock.hh> +#include <seastar/core/manual_clock.hh> +#include <seastar/core/metrics_registration.hh> +#include <seastar/core/scheduling.hh> +#include <seastar/core/scheduling_specific.hh> +#include <seastar/core/smp.hh> +#include <seastar/core/internal/io_request.hh> +#include <seastar/core/internal/io_sink.hh> +#include <seastar/core/make_task.hh> +#include "internal/pollable_fd.hh" +#include "internal/poll.hh" + +#ifdef HAVE_OSV +#include <osv/sched.hh> +#include <osv/mutex.h> +#include <osv/condvar.h> +#include <osv/newpoll.hh> +#endif + +struct _Unwind_Exception; + +namespace seastar { + +using shard_id = unsigned; + +namespace alien { +class message_queue; +class instance; +} +class reactor; + +} + +namespace std { + +template <> +struct hash<::sockaddr_in> { + size_t operator()(::sockaddr_in a) const { + return a.sin_port ^ a.sin_addr.s_addr; + } +}; + +} + +bool operator==(const ::sockaddr_in a, const ::sockaddr_in b); + +namespace seastar { + +class thread_pool; +class smp; + +class reactor_backend_selector; + +class reactor_backend; + +namespace internal { + +class reactor_stall_sampler; +class cpu_stall_detector; +class buffer_allocator; + +template <typename Func> // signature: bool () +std::unique_ptr<pollfn> make_pollfn(Func&& func); + +class poller { + std::unique_ptr<pollfn> _pollfn; + class registration_task; + class deregistration_task; + registration_task* _registration_task = nullptr; +public: + template <typename Func> // signature: bool () + static poller simple(Func&& poll) { + return poller(make_pollfn(std::forward<Func>(poll))); + } + poller(std::unique_ptr<pollfn> fn) + : _pollfn(std::move(fn)) { + do_register(); + } + ~poller(); + poller(poller&& x) noexcept; + poller& operator=(poller&& x) noexcept; + void do_register() noexcept; + friend class reactor; +}; + +size_t scheduling_group_count(); + +void increase_thrown_exceptions_counter() noexcept; + +} + +class kernel_completion; +class io_queue; +class io_intent; +class disk_config_params; + +class io_completion : public kernel_completion { +public: + virtual void complete_with(ssize_t res) final override; + + virtual void complete(size_t res) noexcept = 0; + virtual void set_exception(std::exception_ptr eptr) noexcept = 0; +}; + +class reactor { +private: + struct task_queue; + using task_queue_list = circular_buffer_fixed_capacity<task_queue*, 1 << log2ceil(max_scheduling_groups())>; + using pollfn = seastar::pollfn; + + class signal_pollfn; + class batch_flush_pollfn; + class smp_pollfn; + class drain_cross_cpu_freelist_pollfn; + class lowres_timer_pollfn; + class manual_timer_pollfn; + class epoll_pollfn; + class reap_kernel_completions_pollfn; + class kernel_submit_work_pollfn; + class io_queue_submission_pollfn; + class syscall_pollfn; + class execution_stage_pollfn; + friend class manual_clock; + friend class file_data_source_impl; // for fstream statistics + friend class internal::reactor_stall_sampler; + friend class preempt_io_context; + friend struct hrtimer_aio_completion; + friend class reactor_backend_epoll; + friend class reactor_backend_aio; + friend class reactor_backend_uring; + friend class reactor_backend_selector; + friend class io_queue; // for aio statistics + friend struct reactor_options; + friend class aio_storage_context; +public: + using poller = internal::poller; + using idle_cpu_handler_result = seastar::idle_cpu_handler_result; + using work_waiting_on_reactor = seastar::work_waiting_on_reactor; + using idle_cpu_handler = seastar::idle_cpu_handler; + + struct io_stats { + uint64_t aio_reads = 0; + uint64_t aio_read_bytes = 0; + uint64_t aio_writes = 0; + uint64_t aio_write_bytes = 0; + uint64_t aio_outsizes = 0; + uint64_t aio_errors = 0; + uint64_t fstream_reads = 0; + uint64_t fstream_read_bytes = 0; + uint64_t fstream_reads_blocked = 0; + uint64_t fstream_read_bytes_blocked = 0; + uint64_t fstream_read_aheads_discarded = 0; + uint64_t fstream_read_ahead_discarded_bytes = 0; + }; + /// Scheduling statistics. + struct sched_stats { + /// Total number of tasks processed by this shard's reactor until this point. + /// Note that tasks can be tiny, running for a few nanoseconds, or can take an + /// entire task quota. + uint64_t tasks_processed = 0; + }; + friend void io_completion::complete_with(ssize_t); + + /// Obtains an alien::instance object that can be used to send messages + /// to Seastar shards from non-Seastar threads. + alien::instance& alien() { return _alien; } + +private: + std::shared_ptr<smp> _smp; + alien::instance& _alien; + reactor_config _cfg; + file_desc _notify_eventfd; + file_desc _task_quota_timer; +#ifdef HAVE_OSV + reactor_backend_osv _backend; + sched::thread _timer_thread; + sched::thread *_engine_thread; + mutable mutex _timer_mutex; + condvar _timer_cond; + s64 _timer_due = 0; +#else + std::unique_ptr<reactor_backend> _backend; +#endif + sigset_t _active_sigmask; // holds sigmask while sleeping with sig disabled + std::vector<pollfn*> _pollers; + + static constexpr unsigned max_aio_per_queue = 128; + static constexpr unsigned max_queues = 8; + static constexpr unsigned max_aio = max_aio_per_queue * max_queues; + friend disk_config_params; + + // Each mountpouint is controlled by its own io_queue, but ... + std::unordered_map<dev_t, std::unique_ptr<io_queue>> _io_queues; + // ... when dispatched all requests get into this single sink + internal::io_sink _io_sink; + unsigned _num_io_groups = 0; + + std::vector<noncopyable_function<future<> ()>> _exit_funcs; + unsigned _id = 0; + bool _stopping = false; + bool _stopped = false; + bool _finished_running_tasks = false; + condition_variable _stop_requested; + bool _handle_sigint = true; + std::optional<future<std::unique_ptr<network_stack>>> _network_stack_ready; + int _return = 0; + promise<> _start_promise; + semaphore _cpu_started; + internal::preemption_monitor _preemption_monitor{}; + uint64_t _global_tasks_processed = 0; + uint64_t _polls = 0; + std::unique_ptr<internal::cpu_stall_detector> _cpu_stall_detector; + + unsigned _max_task_backlog = 1000; + timer_set<timer<>, &timer<>::_link> _timers; + timer_set<timer<>, &timer<>::_link>::timer_list_t _expired_timers; + timer_set<timer<lowres_clock>, &timer<lowres_clock>::_link> _lowres_timers; + timer_set<timer<lowres_clock>, &timer<lowres_clock>::_link>::timer_list_t _expired_lowres_timers; + timer_set<timer<manual_clock>, &timer<manual_clock>::_link> _manual_timers; + timer_set<timer<manual_clock>, &timer<manual_clock>::_link>::timer_list_t _expired_manual_timers; + io_stats _io_stats; + uint64_t _fsyncs = 0; + uint64_t _cxx_exceptions = 0; + uint64_t _abandoned_failed_futures = 0; + struct task_queue { + explicit task_queue(unsigned id, sstring name, float shares); + int64_t _vruntime = 0; + float _shares; + int64_t _reciprocal_shares_times_2_power_32; + bool _current = false; + bool _active = false; + uint8_t _id; + sched_clock::time_point _ts; // to help calculating wait/starve-times + sched_clock::duration _runtime = {}; + sched_clock::duration _waittime = {}; + sched_clock::duration _starvetime = {}; + uint64_t _tasks_processed = 0; + circular_buffer<task*> _q; + sstring _name; + int64_t to_vruntime(sched_clock::duration runtime) const; + void set_shares(float shares) noexcept; + struct indirect_compare; + sched_clock::duration _time_spent_on_task_quota_violations = {}; + seastar::metrics::metric_groups _metrics; + void rename(sstring new_name); + private: + void register_stats(); + }; + + boost::container::static_vector<std::unique_ptr<task_queue>, max_scheduling_groups()> _task_queues; + internal::scheduling_group_specific_thread_local_data _scheduling_group_specific_data; + int64_t _last_vruntime = 0; + task_queue_list _active_task_queues; + task_queue_list _activating_task_queues; + task_queue* _at_destroy_tasks; + sched_clock::duration _task_quota; + task* _current_task = nullptr; + /// Handler that will be called when there is no task to execute on cpu. + /// It represents a low priority work. + /// + /// Handler's return value determines whether handler did any actual work. If no work was done then reactor will go + /// into sleep. + /// + /// Handler's argument is a function that returns true if a task which should be executed on cpu appears or false + /// otherwise. This function should be used by a handler to return early if a task appears. + idle_cpu_handler _idle_cpu_handler{ [] (work_waiting_on_reactor) {return idle_cpu_handler_result::no_more_work;} }; + std::unique_ptr<network_stack> _network_stack; + lowres_clock::time_point _lowres_next_timeout = lowres_clock::time_point::max(); + std::optional<pollable_fd> _aio_eventfd; + const bool _reuseport; + circular_buffer<double> _loads; + double _load = 0; + sched_clock::duration _total_idle{0}; + sched_clock::duration _total_sleep; + sched_clock::time_point _start_time = now(); + std::chrono::nanoseconds _max_poll_time = calculate_poll_time(); + output_stream<char>::batch_flush_list_t _flush_batching; + std::atomic<bool> _sleeping alignas(seastar::cache_line_size){0}; + pthread_t _thread_id alignas(seastar::cache_line_size) = pthread_self(); + bool _strict_o_direct = true; + bool _force_io_getevents_syscall = false; + bool _bypass_fsync = false; + bool _have_aio_fsync = false; + bool _kernel_page_cache = false; + std::atomic<bool> _dying{false}; +private: + static std::chrono::nanoseconds calculate_poll_time(); + static void block_notifier(int); + bool flush_pending_aio(); + steady_clock_type::time_point next_pending_aio() const noexcept; + bool reap_kernel_completions(); + bool flush_tcp_batches(); + void update_lowres_clocks() noexcept; + bool do_expire_lowres_timers() noexcept; + bool do_check_lowres_timers() const noexcept; + void expire_manual_timers() noexcept; + void start_aio_eventfd_loop(); + void stop_aio_eventfd_loop(); + template <typename T, typename E, typename EnableFunc> + void complete_timers(T&, E&, EnableFunc&& enable_fn) noexcept(noexcept(enable_fn())); + + /** + * Returns TRUE if all pollers allow blocking. + * + * @return FALSE if at least one of the blockers requires a non-blocking + * execution. + */ + bool poll_once(); + bool pure_poll_once(); +public: + /// Register a user-defined signal handler + void handle_signal(int signo, noncopyable_function<void ()>&& handler); + void wakeup(); + +private: + class signals { + public: + signals(); + ~signals(); + + bool poll_signal(); + bool pure_poll_signal() const; + void handle_signal(int signo, noncopyable_function<void ()>&& handler); + void handle_signal_once(int signo, noncopyable_function<void ()>&& handler); + static void action(int signo, siginfo_t* siginfo, void* ignore); + static void failed_to_handle(int signo); + private: + struct signal_handler { + signal_handler(int signo, noncopyable_function<void ()>&& handler); + noncopyable_function<void ()> _handler; + }; + std::atomic<uint64_t> _pending_signals; + std::unordered_map<int, signal_handler> _signal_handlers; + + friend void reactor::handle_signal(int, noncopyable_function<void ()>&&); + }; + + signals _signals; + std::unique_ptr<thread_pool> _thread_pool; + friend class thread_pool; + friend class thread_context; + friend class internal::cpu_stall_detector; + + uint64_t pending_task_count() const; + void run_tasks(task_queue& tq); + bool have_more_tasks() const; + bool posix_reuseport_detect(); + void run_some_tasks(); + void activate(task_queue& tq); + void insert_active_task_queue(task_queue* tq); + task_queue* pop_active_task_queue(sched_clock::time_point now); + void insert_activating_task_queues(); + void account_runtime(task_queue& tq, sched_clock::duration runtime); + void account_idle(sched_clock::duration idletime); + void allocate_scheduling_group_specific_data(scheduling_group sg, scheduling_group_key key); + future<> init_scheduling_group(scheduling_group sg, sstring name, float shares); + future<> init_new_scheduling_group_key(scheduling_group_key key, scheduling_group_key_config cfg); + future<> destroy_scheduling_group(scheduling_group sg) noexcept; + uint64_t tasks_processed() const; + uint64_t min_vruntime() const; + void request_preemption(); + void start_handling_signal(); + void reset_preemption_monitor(); + void service_highres_timer() noexcept; + + future<std::tuple<pollable_fd, socket_address>> + do_accept(pollable_fd_state& listen_fd); + future<> do_connect(pollable_fd_state& pfd, socket_address& sa); + + future<size_t> + do_read(pollable_fd_state& fd, void* buffer, size_t size); + future<size_t> + do_recvmsg(pollable_fd_state& fd, const std::vector<iovec>& iov); + future<temporary_buffer<char>> + do_read_some(pollable_fd_state& fd, internal::buffer_allocator* ba); + + future<size_t> + do_send(pollable_fd_state& fd, const void* buffer, size_t size); + future<size_t> + do_sendmsg(pollable_fd_state& fd, net::packet& p); + + future<temporary_buffer<char>> + do_recv_some(pollable_fd_state& fd, internal::buffer_allocator* ba); + + int do_run(); +public: + explicit reactor(std::shared_ptr<smp> smp, alien::instance& alien, unsigned id, reactor_backend_selector rbs, reactor_config cfg); + reactor(const reactor&) = delete; + ~reactor(); + void operator=(const reactor&) = delete; + + static sched_clock::time_point now() noexcept { + return sched_clock::now(); + } + sched_clock::duration uptime() { + return now() - _start_time; + } + + io_queue& get_io_queue(dev_t devid = 0) { + auto queue = _io_queues.find(devid); + if (queue == _io_queues.end()) { + return *_io_queues.at(0); + } else { + return *(queue->second); + } + } + + [[deprecated("Use io_priority_class::register_one")]] + io_priority_class register_one_priority_class(sstring name, uint32_t shares); + + [[deprecated("Use io_priority_class.update_shares")]] + future<> update_shares_for_class(io_priority_class pc, uint32_t shares); + /// @private + void update_shares_for_queues(io_priority_class pc, uint32_t shares); + /// @private + future<> update_bandwidth_for_queues(io_priority_class pc, uint64_t bandwidth); + + [[deprecated("Use io_priority_class.rename")]] + static future<> rename_priority_class(io_priority_class pc, sstring new_name) noexcept; + /// @private + void rename_queues(io_priority_class pc, sstring new_name); + + void configure(const reactor_options& opts); + + server_socket listen(socket_address sa, listen_options opts = {}); + + future<connected_socket> connect(socket_address sa); + future<connected_socket> connect(socket_address, socket_address, transport proto = transport::TCP); + + pollable_fd posix_listen(socket_address sa, listen_options opts = {}); + + bool posix_reuseport_available() const { return _reuseport; } + + pollable_fd make_pollable_fd(socket_address sa, int proto); + + future<> posix_connect(pollable_fd pfd, socket_address sa, socket_address local); + + future<> send_all(pollable_fd_state& fd, const void* buffer, size_t size); + + future<file> open_file_dma(std::string_view name, open_flags flags, file_open_options options = {}) noexcept; + future<file> open_directory(std::string_view name) noexcept; + future<> make_directory(std::string_view name, file_permissions permissions = file_permissions::default_dir_permissions) noexcept; + future<> touch_directory(std::string_view name, file_permissions permissions = file_permissions::default_dir_permissions) noexcept; + future<std::optional<directory_entry_type>> file_type(std::string_view name, follow_symlink = follow_symlink::yes) noexcept; + future<stat_data> file_stat(std::string_view pathname, follow_symlink) noexcept; + future<uint64_t> file_size(std::string_view pathname) noexcept; + future<bool> file_accessible(std::string_view pathname, access_flags flags) noexcept; + future<bool> file_exists(std::string_view pathname) noexcept { + return file_accessible(pathname, access_flags::exists); + } + future<fs_type> file_system_at(std::string_view pathname) noexcept; + future<struct statvfs> statvfs(std::string_view pathname) noexcept; + future<> remove_file(std::string_view pathname) noexcept; + future<> rename_file(std::string_view old_pathname, std::string_view new_pathname) noexcept; + future<> link_file(std::string_view oldpath, std::string_view newpath) noexcept; + future<> chmod(std::string_view name, file_permissions permissions) noexcept; + + future<int> inotify_add_watch(int fd, std::string_view path, uint32_t flags); + + future<std::tuple<file_desc, file_desc>> make_pipe(); + future<std::tuple<pid_t, file_desc, file_desc, file_desc>> + spawn(std::string_view pathname, + std::vector<sstring> argv, + std::vector<sstring> env = {}); + future<int> waitpid(pid_t pid); + void kill(pid_t pid, int sig); + + int run() noexcept; + void exit(int ret); + future<> when_started() { return _start_promise.get_future(); } + // The function waits for timeout period for reactor stop notification + // which happens on termination signals or call for exit(). + template <typename Rep, typename Period> + future<> wait_for_stop(std::chrono::duration<Rep, Period> timeout) { + return _stop_requested.wait(timeout, [this] { return _stopping; }); + } + + void at_exit(noncopyable_function<future<> ()> func); + + template <typename Func> + void at_destroy(Func&& func) { + _at_destroy_tasks->_q.push_back(make_task(default_scheduling_group(), std::forward<Func>(func))); + } + + task* current_task() const { return _current_task; } + + void add_task(task* t) noexcept; + void add_urgent_task(task* t) noexcept; + + /// Set a handler that will be called when there is no task to execute on cpu. + /// Handler should do a low priority work. + /// + /// Handler's return value determines whether handler did any actual work. If no work was done then reactor will go + /// into sleep. + /// + /// Handler's argument is a function that returns true if a task which should be executed on cpu appears or false + /// otherwise. This function should be used by a handler to return early if a task appears. + void set_idle_cpu_handler(idle_cpu_handler&& handler) { + _idle_cpu_handler = std::move(handler); + } + void force_poll(); + + void add_high_priority_task(task*) noexcept; + + network_stack& net() { return *_network_stack; } + + [[deprecated("Use this_shard_id")]] + shard_id cpu_id() const; + + void sleep(); + + steady_clock_type::duration total_idle_time(); + steady_clock_type::duration total_busy_time(); + std::chrono::nanoseconds total_steal_time(); + + const io_stats& get_io_stats() const { return _io_stats; } + /// Returns statistics related to scheduling. The statistics are + /// local to this shard. + /// + /// See \ref sched_stats for a description of individual statistics. + /// \return An object containing a snapshot of the statistics at this point in time. + sched_stats get_sched_stats() const; + uint64_t abandoned_failed_futures() const { return _abandoned_failed_futures; } +#ifdef HAVE_OSV + void timer_thread_func(); + void set_timer(sched::timer &tmr, s64 t); +#endif +private: + /** + * Add a new "poller" - a non-blocking function returning a boolean, that + * will be called every iteration of a main loop. + * If it returns FALSE then reactor's main loop is forbidden to block in the + * current iteration. + * + * @param fn a new "poller" function to register + */ + void register_poller(pollfn* p); + void unregister_poller(pollfn* p); + void replace_poller(pollfn* old, pollfn* neww); + void register_metrics(); + future<> send_all_part(pollable_fd_state& fd, const void* buffer, size_t size, size_t completed); + + future<> fdatasync(int fd) noexcept; + + void add_timer(timer<steady_clock_type>*) noexcept; + bool queue_timer(timer<steady_clock_type>*) noexcept; + void del_timer(timer<steady_clock_type>*) noexcept; + void add_timer(timer<lowres_clock>*) noexcept; + bool queue_timer(timer<lowres_clock>*) noexcept; + void del_timer(timer<lowres_clock>*) noexcept; + void add_timer(timer<manual_clock>*) noexcept; + bool queue_timer(timer<manual_clock>*) noexcept; + void del_timer(timer<manual_clock>*) noexcept; + + future<> run_exit_tasks(); + void stop(); + friend class alien::message_queue; + friend class pollable_fd; + friend class pollable_fd_state; + friend struct pollable_fd_state_deleter; + friend class posix_file_impl; + friend class blockdev_file_impl; + friend class timer<>; + friend class timer<lowres_clock>; + friend class timer<manual_clock>; + friend class smp; + friend class smp_message_queue; + friend class internal::poller; + friend class scheduling_group; + friend void add_to_flush_poller(output_stream<char>& os) noexcept; + friend void seastar::internal::increase_thrown_exceptions_counter() noexcept; + friend void report_failed_future(const std::exception_ptr& eptr) noexcept; + friend void with_allow_abandoned_failed_futures(unsigned count, noncopyable_function<void ()> func); + metrics::metric_groups _metric_groups; + friend future<scheduling_group> create_scheduling_group(sstring name, float shares) noexcept; + friend future<> seastar::destroy_scheduling_group(scheduling_group) noexcept; + friend future<> seastar::rename_scheduling_group(scheduling_group sg, sstring new_name) noexcept; + friend future<scheduling_group_key> scheduling_group_key_create(scheduling_group_key_config cfg) noexcept; + + template<typename T> + friend T* internal::scheduling_group_get_specific_ptr(scheduling_group sg, scheduling_group_key key) noexcept; + template<typename SpecificValType, typename Mapper, typename Reducer, typename Initial> + SEASTAR_CONCEPT( requires requires(SpecificValType specific_val, Mapper mapper, Reducer reducer, Initial initial) { + {reducer(initial, mapper(specific_val))} -> std::convertible_to<Initial>; + }) + friend future<typename function_traits<Reducer>::return_type> + map_reduce_scheduling_group_specific(Mapper mapper, Reducer reducer, Initial initial_val, scheduling_group_key key); + template<typename SpecificValType, typename Reducer, typename Initial> + SEASTAR_CONCEPT( requires requires(SpecificValType specific_val, Reducer reducer, Initial initial) { + {reducer(initial, specific_val)} -> std::convertible_to<Initial>; + }) + friend future<typename function_traits<Reducer>::return_type> + reduce_scheduling_group_specific(Reducer reducer, Initial initial_val, scheduling_group_key key); + + future<struct stat> fstat(int fd) noexcept; + future<struct statfs> fstatfs(int fd) noexcept; + friend future<shared_ptr<file_impl>> make_file_impl(int fd, file_open_options options, int flags) noexcept; +public: + future<> readable(pollable_fd_state& fd); + future<> writeable(pollable_fd_state& fd); + future<> readable_or_writeable(pollable_fd_state& fd); + future<> poll_rdhup(pollable_fd_state& fd); + void enable_timer(steady_clock_type::time_point when) noexcept; + /// Sets the "Strict DMA" flag. + /// + /// When true (default), file I/O operations must use DMA. This is + /// the most performant option, but does not work on some file systems + /// such as tmpfs or aufs (used in some Docker setups). + /// + /// When false, file I/O operations can fall back to buffered I/O if + /// DMA is not available. This can result in dramatic reducation in + /// performance and an increase in memory consumption. + void set_strict_dma(bool value); + void set_bypass_fsync(bool value); + void update_blocked_reactor_notify_ms(std::chrono::milliseconds ms); + std::chrono::milliseconds get_blocked_reactor_notify_ms() const; + /// For testing, sets the stall reporting function which is called when + /// a stall is detected (and not suppressed). Setting the function also + /// resets the supression state. + void set_stall_detector_report_function(std::function<void ()> report); + std::function<void ()> get_stall_detector_report_function() const; +}; + +template <typename Func> // signature: bool () +inline +std::unique_ptr<seastar::pollfn> +internal::make_pollfn(Func&& func) { + struct the_pollfn : simple_pollfn<false> { + the_pollfn(Func&& func) : func(std::forward<Func>(func)) {} + Func func; + virtual bool poll() override final { + return func(); + } + }; + return std::make_unique<the_pollfn>(std::forward<Func>(func)); +} + +extern __thread reactor* local_engine; +extern __thread size_t task_quota; + +inline reactor& engine() { + return *local_engine; +} + +inline bool engine_is_ready() { + return local_engine != nullptr; +} + +inline int hrtimer_signal() { + // We don't want to use SIGALRM, because the boost unit test library + // also plays with it. + return SIGRTMIN; +} + + +extern logger seastar_logger; + +} diff --git a/src/seastar/include/seastar/core/reactor_config.hh b/src/seastar/include/seastar/core/reactor_config.hh new file mode 100644 index 000000000..2689efafb --- /dev/null +++ b/src/seastar/include/seastar/core/reactor_config.hh @@ -0,0 +1,166 @@ +/* + * 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) 2019 ScyllaDB + */ + +#pragma once + +#include <chrono> +#include <seastar/util/program-options.hh> +#include <seastar/util/memory_diagnostics.hh> + +namespace seastar { + +/// \cond internal +struct reactor_config { + bool auto_handle_sigint_sigterm = true; + unsigned max_networking_aio_io_control_blocks = 10000; +}; +/// \endcond + +class reactor_backend_selector; +class network_stack_factory; + +/// Configuration for the reactor. +struct reactor_options : public program_options::option_group { + /// \brief Select network stack to use. + /// + /// Each network stack has it corresponding + /// \ref program_options::option_group to further tune it. The available + /// stacks are: + /// * Posix stack (default) - no tunable options; + /// * Native stack - \ref net::native_stack_options; + program_options::selection_value<network_stack_factory> network_stack; + /// Poll continuously (100% cpu use). + program_options::value<> poll_mode; + /// \brief Idle polling time in microseconds. + /// + /// Reduce for overprovisioned environments or laptops. + program_options::value<unsigned> idle_poll_time_us; + /// \brief Busy-poll for disk I/O. + /// + /// Reduces latency and increases throughput. + program_options::value<bool> poll_aio; + /// \brief Max time (ms) between polls. + /// + /// Default: 0.5. + program_options::value<double> task_quota_ms; + /// \brief Max time (ms) IO operations must take. + /// + /// Default: 1.5 * task_quota_ms value + program_options::value<double> io_latency_goal_ms; + /// \brief Maximum number of task backlog to allow. + /// + /// When the number of tasks grow above this, we stop polling (e.g. I/O) + /// until it goes back below the limit. + /// Default: 1000. + program_options::value<unsigned> max_task_backlog; + /// \brief Threshold in milliseconds over which the reactor is considered + /// blocked if no progress is made. + /// + /// Default: 25. + program_options::value<unsigned> blocked_reactor_notify_ms; + /// \brief Maximum number of backtraces reported by stall detector per minute. + /// + /// Default: 5. + program_options::value<unsigned> blocked_reactor_reports_per_minute; + /// \brief Print a simplified backtrace on a single line. + /// + /// Default: \p true. + program_options::value<bool> blocked_reactor_report_format_oneline; + /// \brief Allow using buffered I/O if DMA is not available (reduces performance). + program_options::value<> relaxed_dma; + /// \brief Use the Linux NOWAIT AIO feature, which reduces reactor stalls due + /// to aio (autodetected). + program_options::value<bool> linux_aio_nowait; + /// \brief Bypass fsync(), may result in data loss. + /// + /// Use for testing on consumer drives. + /// Default: \p false. + program_options::value<bool> unsafe_bypass_fsync; + /// \brief Use the kernel page cache. + /// + /// This disables DMA (O_DIRECT). Useful for short-lived functional tests + /// with a small data set. + /// Default: \p false. + program_options::value<bool> kernel_page_cache; + /// \brief Run in an overprovisioned environment (such as docker or a laptop). + /// + /// Equivalent to: + /// * \ref idle_poll_time_us = 0 + /// * \ref smp_options::thread_affinity = 0 + /// * \ref poll_aio = 0 + program_options::value<> overprovisioned; + /// \brief Abort when seastar allocator cannot allocate memory. + program_options::value<> abort_on_seastar_bad_alloc; + /// \brief Force \p io_getevents(2) to issue a system call, instead of + /// bypassing the kernel when possible. + /// + /// This makes strace output more useful, but slows down the application. + /// Default: \p false. + program_options::value<bool> force_aio_syscalls; + /// \brief Dump diagnostics of the seastar allocator state on allocation + /// failure. + /// + /// See \ref memory::alloc_failure_kind for allowed values. The diagnostics + /// will be written to the \p seastar_memory logger, with error level. + /// Default: \ref memory::alloc_failure_kind::critical. + /// \note Even if the \p seastar_memory logger is set to debug or trace + /// level, the diagnostics will be logged irrespective of this setting. + program_options::value<memory::alloc_failure_kind> dump_memory_diagnostics_on_alloc_failure_kind; + /// \brief Internal reactor implementation. + /// + /// Available backends: + /// * \p linux-aio + /// * \p epoll + /// * \p io_uring + /// + /// Default: \p linux-aio (if available). + program_options::selection_value<reactor_backend_selector> reactor_backend; + /// \brief Use Linux aio for fsync() calls. + /// + /// This reduces latency. Requires Linux 4.18 or later. + program_options::value<bool> aio_fsync; + /// \brief Maximum number of I/O control blocks (IOCBs) to allocate per shard. + /// + /// This translates to the number of sockets supported per shard. Requires + /// tuning \p /proc/sys/fs/aio-max-nr. Only valid for the \p linux-aio + /// reactor backend (see \ref reactor_backend). + /// + /// Default: 10000. + program_options::value<unsigned> max_networking_io_control_blocks; + /// \brief Enable seastar heap profiling. + /// + /// \note Unused when seastar was compiled without heap profiling support. + program_options::value<> heapprof; + /// Ignore SIGINT (for gdb). + program_options::value<> no_handle_interrupt; + + /// \cond internal + std::string _argv0; + bool _auto_handle_sigint_sigterm = true; + /// \endcond + +public: + /// \cond internal + reactor_options(program_options::option_group* parent_group); + /// \endcond +}; + +} diff --git a/src/seastar/include/seastar/core/relabel_config.hh b/src/seastar/include/seastar/core/relabel_config.hh new file mode 100644 index 000000000..4c1ba11de --- /dev/null +++ b/src/seastar/include/seastar/core/relabel_config.hh @@ -0,0 +1,101 @@ +/* + * 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 2022 ScyllaDB + */ +#include <regex> + +namespace seastar { +namespace metrics { + +/*! + * \brief a wrapper class around regex with the original expr + * + * regex does not contain the original expression, this wrapper class + * acts both as a string and as a regex. + */ +class relabel_config_regex { + std::string _regex_str; + std::regex _regex; +public: + relabel_config_regex() = default; + relabel_config_regex(const std::string& expr) : _regex_str(expr), _regex(std::regex(expr)) {} + relabel_config_regex(const char* expr) : _regex_str(expr), _regex(std::regex(expr)) {} + const std::string& str() const noexcept { + return _regex_str; + } + const std::regex& regex() const noexcept { + return _regex; + } + + relabel_config_regex& operator=(const char* expr) { + std::string str(expr); + return operator=(str); + } + + relabel_config_regex& operator=(const std::string& expr) { + _regex_str = expr; + _regex = std::regex(_regex_str); + return *this; + } +}; + +/*! + * \brief a relabel_config allows changing metrics labels dynamically + * + * The logic is similar to Prometheus configuration + * This is how Prometheus entry looks like: + * - source_labels: [version] + regex: '([0-9]+\.[0-9]+)(\.?[0-9]*).*' + replacement: '$1$2' + target_label: svr + * relabel_action values: + * skip_when_empty - when set supported metrics (histogram, summary and counters) + * will not be reported if they were never used. + * report_when_empty - revert the skip_when_empty flag + * replace - replace the value of the target_label + * keep - enable the metrics + * drop - disable the metrics + * drop_label - remove the target label + * + * source_labels - a list of source labels, the labels are concatenated + * with the separator and and the combine value is match to the regex. + * target_label - the labels to perform the action on when replacing a value or when dropping a label. + * replacement - the string to use when replacing a label value, regex group can be used. + * expr - a regular expression in a string format. Action would be taken if the regex + * match the concatenated labels. + * action - The action to perform when there is a match. + * separator - separator to use when concatenating the labels. + * + */ +struct relabel_config { + enum class relabel_action {skip_when_empty, report_when_empty, replace, keep, drop, drop_label}; + std::vector<std::string> source_labels; + std::string target_label; + std::string replacement = "${1}"; + relabel_config_regex expr = "(.*)"; + relabel_action action = relabel_action::replace; + std::string separator = ";"; +}; + +/*! + * \brief a helper function to translate a string to relabel_config::relabel_action enum values + */ +relabel_config::relabel_action relabel_config_action(const std::string& action); +} +} diff --git a/src/seastar/include/seastar/core/report_exception.hh b/src/seastar/include/seastar/core/report_exception.hh new file mode 100644 index 000000000..7738d9bae --- /dev/null +++ b/src/seastar/include/seastar/core/report_exception.hh @@ -0,0 +1,31 @@ +/* + * 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 2014 Cloudius Systems + */ + +#pragma once + +#include <seastar/util/std-compat.hh> + +namespace seastar { + +void report_exception(std::string_view message, std::exception_ptr) noexcept; + +} + diff --git a/src/seastar/include/seastar/core/resource.hh b/src/seastar/include/seastar/core/resource.hh new file mode 100644 index 000000000..b03dd9133 --- /dev/null +++ b/src/seastar/include/seastar/core/resource.hh @@ -0,0 +1,134 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <cassert> +#include <cstdlib> +#include <string> +#include <seastar/util/std-compat.hh> +#include <seastar/util/spinlock.hh> +#include <vector> +#include <set> +#include <sched.h> +#include <boost/any.hpp> +#include <unordered_map> +#ifdef SEASTAR_HAVE_HWLOC +#include <hwloc.h> +#endif + +namespace seastar { + +cpu_set_t cpuid_to_cpuset(unsigned cpuid); +class io_queue; +class io_group; + +namespace resource { + +using std::optional; + +using cpuset = std::set<unsigned>; + +/// \cond internal +std::optional<cpuset> parse_cpuset(std::string value); +/// \endcond + +namespace hwloc::internal { + +#ifdef SEASTAR_HAVE_HWLOC +class topology_holder { + hwloc_topology_t _topology; + +public: + topology_holder() noexcept + : _topology(nullptr) + { } + + topology_holder(topology_holder&& o) noexcept; + + ~topology_holder(); + + topology_holder& operator=(topology_holder&& o) noexcept; + + operator bool() const noexcept { + return _topology != nullptr; + } + + void init_and_load(); + hwloc_topology_t get(); +}; + +#else // SEASTAR_HAVE_HWLOC + +struct topology_holder {}; + +#endif // SEASTAR_HAVE_HWLOC + +} // namespace hwloc::internal + +struct configuration { + optional<size_t> total_memory; + optional<size_t> reserve_memory; // if total_memory not specified + size_t reserve_additional_memory; + size_t cpus; + cpuset cpu_set; + bool assign_orphan_cpus = false; + std::vector<dev_t> devices; + unsigned num_io_groups; + hwloc::internal::topology_holder topology; +}; + +struct memory { + size_t bytes; + unsigned nodeid; + +}; + +struct io_queue_topology { + std::vector<std::unique_ptr<io_queue>> queues; + std::vector<unsigned> shard_to_group; + std::vector<std::shared_ptr<io_group>> groups; + + util::spinlock lock; + + io_queue_topology(); + io_queue_topology(const io_queue_topology&) = delete; + io_queue_topology(io_queue_topology&&); + ~io_queue_topology(); +}; + +struct cpu { + unsigned cpu_id; + std::vector<memory> mem; +}; + +struct resources { + std::vector<cpu> cpus; + std::unordered_map<dev_t, io_queue_topology> ioq_topology; +}; + +resources allocate(configuration& c); +unsigned nr_processing_units(configuration& c); + +std::optional<resource::cpuset> parse_cpuset(std::string value); + +} +} diff --git a/src/seastar/include/seastar/core/rwlock.hh b/src/seastar/include/seastar/core/rwlock.hh new file mode 100644 index 000000000..5d8feb82b --- /dev/null +++ b/src/seastar/include/seastar/core/rwlock.hh @@ -0,0 +1,196 @@ +/* +* 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) 2015 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/core/semaphore.hh> + +namespace seastar { + +/// \cond internal +// lock / unlock semantics for rwlock, so it can be used with with_lock() +template<typename Clock> +class basic_rwlock; + +template<typename Clock = typename timer<>::clock> +class rwlock_for_read { +public: + future<> lock() { + return static_cast<basic_rwlock<Clock>*>(this)->read_lock(); + } + void unlock() { + static_cast<basic_rwlock<Clock>*>(this)->read_unlock(); + } + friend class basic_rwlock<Clock>; +}; + +template<typename Clock = typename timer<>::clock> +class rwlock_for_write { +public: + future<> lock() { + return static_cast<basic_rwlock<Clock>*>(this)->write_lock(); + } + void unlock() { + static_cast<basic_rwlock<Clock>*>(this)->write_unlock(); + } + friend class basic_rwlock<Clock>; +}; +/// \endcond + + +/// \addtogroup fiber-module +/// @{ + +/// Implements a read-write lock mechanism. Beware: this is not a cross-CPU +/// lock, due to seastar's sharded architecture. +/// Instead, it can be used to achieve rwlock semantics between two (or more) +/// fibers running in the same CPU that may use the same resource. +/// Acquiring the write lock will effectively cause all readers not to be executed +/// until the write part is done. +template<typename Clock = typename timer<>::clock> +class basic_rwlock : private rwlock_for_read<Clock>, rwlock_for_write<Clock> { + using semaphore_type = basic_semaphore<semaphore_default_exception_factory, Clock>; + + static constexpr size_t max_ops = semaphore_type::max_counter(); + + semaphore_type _sem; +public: + basic_rwlock() + : _sem(max_ops) { + } + + /// Cast this rwlock into read lock object with lock semantics appropriate to be used + /// by "with_lock". The resulting object will have lock / unlock calls that, when called, + /// will acquire / release the lock in read mode. + rwlock_for_read<Clock>& for_read() { + return *this; + } + + /// Cast this rwlock into write lock object with lock semantics appropriate to be used + /// by "with_lock". The resulting object will have lock / unlock calls that, when called, + /// will acquire / release the lock in write mode. + rwlock_for_write<Clock>& for_write() { + return *this; + } + + /// Acquires this lock in read mode. Many readers are allowed, but when + /// this future returns, and until \ref read_unlock is called, all fibers + /// waiting on \ref write_lock are guaranteed not to execute. + future<> read_lock(typename semaphore_type::time_point timeout = semaphore_type::time_point::max()) { + return _sem.wait(timeout); + } + + future<> read_lock(abort_source& as) { + return _sem.wait(as); + } + + /// Releases the lock, which must have been taken in read mode. After this + /// is called, one of the fibers waiting on \ref write_lock will be allowed + /// to proceed. + void read_unlock() { + assert(_sem.current() < max_ops); + _sem.signal(); + } + + /// Acquires this lock in write mode. Only one writer is allowed. When + /// this future returns, and until \ref write_unlock is called, all other + /// fibers waiting on either \ref read_lock or \ref write_lock are guaranteed + /// not to execute. + future<> write_lock(typename semaphore_type::time_point timeout = semaphore_type::time_point::max()) { + return _sem.wait(timeout, max_ops); + } + + future<> write_lock(abort_source& as) { + return _sem.wait(as, max_ops); + } + + /// Releases the lock, which must have been taken in write mode. After this + /// is called, one of the other fibers waiting on \ref write_lock or the fibers + /// waiting on \ref read_lock will be allowed to proceed. + void write_unlock() { + assert(_sem.current() == 0); + _sem.signal(max_ops); + } + + /// Tries to acquire the lock in read mode iff this can be done without waiting. + bool try_read_lock() { + return _sem.try_wait(); + } + + /// Tries to acquire the lock in write mode iff this can be done without waiting. + bool try_write_lock() { + return _sem.try_wait(max_ops); + } + + using holder = semaphore_units<semaphore_default_exception_factory, Clock>; + + /// hold_read_lock() waits for a read lock and returns an object which, + /// when destroyed, releases the lock. This makes it easy to ensure that + /// the lock is eventually undone, at any circumstance (even including + /// exceptions). The release() method can be used on the returned object + /// to release its ownership of the lock and avoid the automatic unlock. + /// Note that both hold_read_lock() and hold_write_lock() return an object + /// of the same type, rwlock::holder. + /// + /// hold_read_lock() may throw an exception (or, in other implementations, + /// return an exceptional future) when it failed to obtain the lock - + /// e.g., on allocation failure. + future<holder> hold_read_lock(typename semaphore_type::time_point timeout = semaphore_type::time_point::max()) { + return get_units(_sem, 1, timeout); + } + + future<holder> hold_read_lock(abort_source& as) { + return get_units(_sem, 1, as); + } + + /// hold_write_lock() waits for a write lock and returns an object which, + /// when destroyed, releases the lock. This makes it easy to ensure that + /// the lock is eventually undone, at any circumstance (even including + /// exceptions). The release() method can be used on the returned object + /// to release its ownership of the lock and avoid the automatic unlock. + /// Note that both hold_read_lock() and hold_write_lock() return an object + /// of the same type, rwlock::holder. + /// + /// hold_read_lock() may throw an exception (or, in other implementations, + /// return an exceptional future) when it failed to obtain the lock - + /// e.g., on allocation failure. + future<holder> hold_write_lock(typename semaphore_type::time_point timeout = semaphore_type::time_point::max()) { + return get_units(_sem, max_ops, timeout); + } + + future<holder> hold_write_lock(abort_source& as) { + return get_units(_sem, max_ops, as); + } + + /// Checks if any read or write locks are currently held. + bool locked() const { + return _sem.available_units() != max_ops; + } + + friend class rwlock_for_read<Clock>; + friend class rwlock_for_write<Clock>; +}; + +using rwlock = basic_rwlock<>; + +/// @} + +} diff --git a/src/seastar/include/seastar/core/scattered_message.hh b/src/seastar/include/seastar/core/scattered_message.hh new file mode 100644 index 000000000..145016155 --- /dev/null +++ b/src/seastar/include/seastar/core/scattered_message.hh @@ -0,0 +1,118 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/core/deleter.hh> +#include <seastar/core/temporary_buffer.hh> +#include <seastar/net/packet.hh> +#include <seastar/core/sstring.hh> +#include <memory> +#include <vector> +#include <seastar/util/std-compat.hh> + +namespace seastar { + +template <typename CharType> +class scattered_message { +private: + using fragment = net::fragment; + using packet = net::packet; + using char_type = CharType; + packet _p; +public: + scattered_message() {} + scattered_message(scattered_message&&) = default; + scattered_message(const scattered_message&) = delete; + + void append_static(const char_type* buf, size_t size) { + if (size) { + _p = packet(std::move(_p), fragment{(char_type*)buf, size}, deleter()); + } + } + + template <size_t N> + void append_static(const char_type(&s)[N]) { + append_static(s, N - 1); + } + + void append_static(const char_type* s) { + append_static(s, strlen(s)); + } + + template <typename size_type, size_type max_size> + void append_static(const basic_sstring<char_type, size_type, max_size>& s) { + append_static(s.begin(), s.size()); + } + + void append_static(const std::string_view& s) { + append_static(s.data(), s.size()); + } + + void append(std::string_view v) { + if (v.size()) { + _p = packet(std::move(_p), temporary_buffer<char>::copy_of(v)); + } + } + + void append(temporary_buffer<CharType> buff) { + if (buff.size()) { + _p = packet(std::move(_p), std::move(buff)); + } + } + + template <typename size_type, size_type max_size> + void append(basic_sstring<char_type, size_type, max_size> s) { + if (s.size()) { + _p = packet(std::move(_p), std::move(s).release()); + } + } + + template <typename size_type, size_type max_size, typename Callback> + void append(const basic_sstring<char_type, size_type, max_size>& s, Callback callback) { + if (s.size()) { + _p = packet(std::move(_p), fragment{s.begin(), s.size()}, make_deleter(std::move(callback))); + } + } + + void reserve(int n_frags) { + _p.reserve(n_frags); + } + + packet release() && { + return std::move(_p); + } + + template <typename Callback> + void on_delete(Callback callback) { + _p = packet(std::move(_p), make_deleter(std::move(callback))); + } + + operator bool() const { + return _p.len(); + } + + size_t size() { + return _p.len(); + } +}; + +} diff --git a/src/seastar/include/seastar/core/scheduling.hh b/src/seastar/include/seastar/core/scheduling.hh new file mode 100644 index 000000000..d652fecca --- /dev/null +++ b/src/seastar/include/seastar/core/scheduling.hh @@ -0,0 +1,369 @@ +/* + * 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) 2016 Scylla DB Ltd + */ + +#pragma once + +#include <chrono> +#include <typeindex> +#include <seastar/core/sstring.hh> +#include <seastar/core/function_traits.hh> +#include <seastar/util/concepts.hh> + +/// \file + +namespace seastar { + +constexpr unsigned max_scheduling_groups() { return SEASTAR_SCHEDULING_GROUPS_COUNT; } + +#if SEASTAR_API_LEVEL < 6 +#define SEASTAR_ELLIPSIS ... +template <typename SEASTAR_ELLIPSIS T> +#else +#define SEASTAR_ELLIPSIS +template <typename T = void> +#endif +class future; + +class reactor; + +class scheduling_group; +class scheduling_group_key; + +using sched_clock = std::chrono::steady_clock; + +namespace internal { + +// Returns an index between 0 and max_scheduling_groups() +unsigned scheduling_group_index(scheduling_group sg) noexcept; +scheduling_group scheduling_group_from_index(unsigned index) noexcept; + +unsigned long scheduling_group_key_id(scheduling_group_key) noexcept; + +template<typename T> +T* scheduling_group_get_specific_ptr(scheduling_group sg, scheduling_group_key key) noexcept; + +} + + +/// Creates a scheduling group with a specified number of shares. +/// +/// The operation is global and affects all shards. The returned scheduling +/// group can then be used in any shard. +/// +/// \param name A name that identifiers the group; will be used as a label +/// in the group's metrics +/// \param shares number of shares of the CPU time allotted to the group; +/// Use numbers in the 1-1000 range (but can go above). +/// \return a scheduling group that can be used on any shard +future<scheduling_group> create_scheduling_group(sstring name, float shares) noexcept; + +/// Destroys a scheduling group. +/// +/// Destroys a \ref scheduling_group previously created with create_scheduling_group(). +/// The destroyed group must not be currently in use and must not be used later. +/// +/// The operation is global and affects all shards. +/// +/// \param sg The scheduling group to be destroyed +/// \return a future that is ready when the scheduling group has been torn down +future<> destroy_scheduling_group(scheduling_group sg) noexcept; + +/// Rename scheduling group. +/// +/// Renames a \ref scheduling_group previously created with create_scheduling_group(). +/// +/// The operation is global and affects all shards. +/// The operation affects the exported statistics labels. +/// +/// \param sg The scheduling group to be renamed +/// \param new_name The new name for the scheduling group. +/// \return a future that is ready when the scheduling group has been renamed +future<> rename_scheduling_group(scheduling_group sg, sstring new_name) noexcept; + + +/** + * Represents a configuration for a specific scheduling group value, + * it contains all that is needed to maintain a scheduling group specific + * value when it needs to be created, due to, for example, a new + * \ref scheduling_group being created. + * + * @note is is recomended to use @ref make_scheduling_group_key_config in order to + * create and configure this syructure. The only reason that one might want to not use + * this method is because of a need for specific intervention in the construction or + * destruction of the value. Even then, it is recommended to first create the configuration + * with @ref make_scheduling_group_key_config and only the change it. + * + */ +struct scheduling_group_key_config { + /** + * Constructs a default configuration + */ + scheduling_group_key_config() : + scheduling_group_key_config(typeid(void)) {} + /** + * Creates a configuration that is made for a specific type. + * It does not contain the right alignment and allocation sizes + * neither the correct construction or destruction logic, but only + * the indication for the intended type which is used in debug mode + * to make sure that the correct type is reffered to when accessing + * the value. + * @param type_info - the type information class (create with typeid(T)). + */ + scheduling_group_key_config(const std::type_info& type_info) : + type_index(type_info) {} + /// The allocation size for the value (usually: sizeof(T)) + size_t allocation_size; + /// The required alignment of the value (usually: alignof(T)) + size_t alignment; + /// Holds the type information for debug mode runtime validation + std::type_index type_index; + /// A function that will be called for each newly allocated value + std::function<void (void*)> constructor; + /// A function that will be called for each element that is about + /// to be dealocated. + std::function<void (void*)> destructor; + +}; + + +/** + * A class that is intended to encapsulate the scheduling group specific + * key and "hide" it implementation concerns and details. + * + * @note this object can be copied accross shards and scheduling groups. + */ +class scheduling_group_key { +public: + /// The only user allowed operation on a key is copying. + scheduling_group_key(const scheduling_group_key&) noexcept = default; + scheduling_group_key(scheduling_group_key&&) noexcept = default; +private: + scheduling_group_key(unsigned long id) noexcept : + _id(id) {} + unsigned long _id; + unsigned long id() const noexcept { + return _id; + } + friend class reactor; + friend future<scheduling_group_key> scheduling_group_key_create(scheduling_group_key_config cfg) noexcept; + template<typename T> + friend T* internal::scheduling_group_get_specific_ptr(scheduling_group sg, scheduling_group_key key) noexcept; + template<typename T> + friend T& scheduling_group_get_specific(scheduling_group_key key) noexcept; + + friend unsigned long internal::scheduling_group_key_id(scheduling_group_key key) noexcept; +}; + +namespace internal { + +inline unsigned long scheduling_group_key_id(scheduling_group_key key) noexcept { + return key.id(); +} + +/** + * @brief A function in the spirit of Cpp17 apply, but specifically for constructors. + * This function is used in order to preserve support in Cpp14. + + * @tparam ConstructorType - the constructor type or in other words the type to be constructed + * @tparam Tuple - T params tuple type (should be deduced) + * @tparam size_t...Idx - a sequence of indexes in order to access the typpels members in compile time. + * (should be deduced) + * + * @param pre_alocated_mem - a pointer to the pre allocated memory chunk that will hold the + * the initialized object. + * @param args - A tupple that holds the prarameters for the constructor + * @param idx_seq - An index sequence that will be used to access the members of the tuple in compile + * time. + * + * @note this function was not intended to be called by users and it is only a utility function + * for suporting \ref make_scheduling_group_key_config + */ +template<typename ConstructorType, typename Tuple, size_t...Idx> +void apply_constructor(void* pre_alocated_mem, Tuple args, std::index_sequence<Idx...>) { + new (pre_alocated_mem) ConstructorType(std::get<Idx>(args)...); +} +} + +/** + * A template function that builds a scheduling group specific value configuration. + * This configuration is used by the infrastructure to allocate memory for the values + * and initialize or deinitialize them when they are created or destroyed. + * + * @tparam T - the type for the newly created value. + * @tparam ...ConstructorArgs - the types for the constructor parameters (should be deduced) + * @param args - The parameters for the constructor. + * @return a fully initialized \ref scheduling_group_key_config object. + */ +template <typename T, typename... ConstructorArgs> +scheduling_group_key_config +make_scheduling_group_key_config(ConstructorArgs... args) { + scheduling_group_key_config sgkc(typeid(T)); + sgkc.allocation_size = sizeof(T); + sgkc.alignment = alignof(T); + sgkc.constructor = [args = std::make_tuple(args...)] (void* p) { + internal::apply_constructor<T>(p, args, std::make_index_sequence<sizeof...(ConstructorArgs)>()); + }; + sgkc.destructor = [] (void* p) { + static_cast<T*>(p)->~T(); + }; + return sgkc; +} + +/** + * Returns a future that holds a scheduling key and resolves when this key can be used + * to access the scheduling group specific value it represents. + * @param cfg - A \ref scheduling_group_key_config object (by recomendation: initialized with + * \ref make_scheduling_group_key_config ) + * @return A future containing \ref scheduling_group_key for the newly created specific value. + */ +future<scheduling_group_key> scheduling_group_key_create(scheduling_group_key_config cfg) noexcept; + +/** + * Returnes a reference to the given scheduling group specific value + * @tparam T - the type of the scheduling specific type (cannot be deduced) + * @param sg - the scheduling group which it's specific value to retrieve + * @param key - the key of the value to retrieve. + * @return A reference to the scheduling specific value. + */ +template<typename T> +T& scheduling_group_get_specific(scheduling_group sg, scheduling_group_key key); + + +/// \brief Identifies function calls that are accounted as a group +/// +/// A `scheduling_group` is a tag that can be used to mark a function call. +/// Executions of such tagged calls are accounted as a group. +class scheduling_group { + unsigned _id; +private: + explicit scheduling_group(unsigned id) noexcept : _id(id) {} +public: + /// Creates a `scheduling_group` object denoting the default group + constexpr scheduling_group() noexcept : _id(0) {} // must be constexpr for current_scheduling_group_holder + bool active() const noexcept; + const sstring& name() const noexcept; + bool operator==(scheduling_group x) const noexcept { return _id == x._id; } + bool operator!=(scheduling_group x) const noexcept { return _id != x._id; } + bool is_main() const noexcept { return _id == 0; } + template<typename T> + /** + * Returnes a reference to this scheduling group specific value + * @tparam T - the type of the scheduling specific type (cannot be deduced) + * @param key - the key of the value to retrieve. + * @return A reference to this scheduling specific value. + */ + T& get_specific(scheduling_group_key key) noexcept { + return *internal::scheduling_group_get_specific_ptr<T>(*this, key); + } + /// Adjusts the number of shares allotted to the group. + /// + /// Dynamically adjust the number of shares allotted to the group, increasing or + /// decreasing the amount of CPU bandwidth it gets. The adjustment is local to + /// the shard. + /// + /// This can be used to reduce a background job's interference with a foreground + /// load: the shares can be started at a low value, increased when the background + /// job's backlog increases, and reduced again when the backlog decreases. + /// + /// \param shares number of shares allotted to the group. Use numbers + /// in the 1-1000 range. + void set_shares(float shares) noexcept; + friend future<scheduling_group> create_scheduling_group(sstring name, float shares) noexcept; + friend future<> destroy_scheduling_group(scheduling_group sg) noexcept; + friend future<> rename_scheduling_group(scheduling_group sg, sstring new_name) noexcept; + friend class reactor; + friend unsigned internal::scheduling_group_index(scheduling_group sg) noexcept; + friend scheduling_group internal::scheduling_group_from_index(unsigned index) noexcept; + + template<typename SpecificValType, typename Mapper, typename Reducer, typename Initial> + SEASTAR_CONCEPT( requires requires(SpecificValType specific_val, Mapper mapper, Reducer reducer, Initial initial) { + {reducer(initial, mapper(specific_val))} -> std::convertible_to<Initial>; + }) + friend future<typename function_traits<Reducer>::return_type> + map_reduce_scheduling_group_specific(Mapper mapper, Reducer reducer, Initial initial_val, scheduling_group_key key); + + template<typename SpecificValType, typename Reducer, typename Initial> + SEASTAR_CONCEPT( requires requires(SpecificValType specific_val, Reducer reducer, Initial initial) { + {reducer(initial, specific_val)} -> std::convertible_to<Initial>; + }) + friend future<typename function_traits<Reducer>::return_type> + reduce_scheduling_group_specific(Reducer reducer, Initial initial_val, scheduling_group_key key); + + +}; + +/// \cond internal +namespace internal { + +inline +unsigned +scheduling_group_index(scheduling_group sg) noexcept { + return sg._id; +} + +inline +scheduling_group +scheduling_group_from_index(unsigned index) noexcept { + return scheduling_group(index); +} + +inline +scheduling_group* +current_scheduling_group_ptr() noexcept { + // Slow unless constructor is constexpr + static thread_local scheduling_group sg; + return &sg; +} + +} +/// \endcond + +/// Returns the current scheduling group +inline +scheduling_group +current_scheduling_group() noexcept { + return *internal::current_scheduling_group_ptr(); +} + +inline +scheduling_group +default_scheduling_group() noexcept { + return scheduling_group(); +} + +inline +bool +scheduling_group::active() const noexcept { + return *this == current_scheduling_group(); +} + +} + +namespace std { + +template <> +struct hash<seastar::scheduling_group> { + size_t operator()(seastar::scheduling_group sg) const noexcept { + return seastar::internal::scheduling_group_index(sg); + } +}; + +} diff --git a/src/seastar/include/seastar/core/scheduling_specific.hh b/src/seastar/include/seastar/core/scheduling_specific.hh new file mode 100644 index 000000000..d2e3158f5 --- /dev/null +++ b/src/seastar/include/seastar/core/scheduling_specific.hh @@ -0,0 +1,189 @@ +/* + * 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) 2019 Scylla DB Ltd + */ + +#include <boost/range/adaptor/filtered.hpp> +#include <seastar/core/scheduling.hh> +#include <seastar/core/map_reduce.hh> +#include <array> +#include <vector> + +#pragma once + +namespace seastar { + +namespace internal { + +struct scheduling_group_specific_thread_local_data { + struct per_scheduling_group { + bool queue_is_initialized = false; + /** + * This array holds pointers to the scheduling group specific + * data. The pointer is not use as is but is cast to a reference + * to the appropriate type that is actually pointed to. + */ + std::vector<void*> specific_vals; + }; + std::array<per_scheduling_group, max_scheduling_groups()> per_scheduling_group_data; + std::vector<scheduling_group_key_config> scheduling_group_key_configs; +}; + +inline +scheduling_group_specific_thread_local_data** get_scheduling_group_specific_thread_local_data_ptr() noexcept { + static thread_local scheduling_group_specific_thread_local_data* data; + return &data; +} +inline +scheduling_group_specific_thread_local_data& get_scheduling_group_specific_thread_local_data() noexcept { + return **get_scheduling_group_specific_thread_local_data_ptr(); +} + +[[noreturn]] void no_such_scheduling_group(scheduling_group sg); + +/** + * Returns a pointer to the given scheduling group specific data. + * @param sg - The scheduling group which it's data needs to be accessed + * @param key - The scheduling group key that for the data to access + * @return A pointer of type T* to the data, if sg is valid initialized. + * + * @note The parameter T has to be given since there is no way to deduce it. + */ +template<typename T> +T* scheduling_group_get_specific_ptr(scheduling_group sg, scheduling_group_key key) noexcept { + auto& data = internal::get_scheduling_group_specific_thread_local_data(); +#ifdef SEASTAR_DEBUG + assert(std::type_index(typeid(T)) == data.scheduling_group_key_configs[key.id()].type_index); +#endif + auto sg_id = internal::scheduling_group_index(sg); + if (__builtin_expect(sg_id < data.per_scheduling_group_data.size() && + data.per_scheduling_group_data[sg_id].queue_is_initialized, true)) { + return reinterpret_cast<T*>(data.per_scheduling_group_data[sg_id].specific_vals[key.id()]); + } + return nullptr; +} + +} + +/** + * Returns a reference to the given scheduling group specific data. + * @param sg - The scheduling group which it's data needs to be accessed + * @param key - The scheduling group key that for the data to access + * @return A reference of type T& to the data. + * + * @note The parameter T has to be given since there is no way to deduce it. + * May throw std::invalid_argument if sg does not exist or is uninitialized. + */ +template<typename T> +T& scheduling_group_get_specific(scheduling_group sg, scheduling_group_key key) { + T* p = internal::scheduling_group_get_specific_ptr<T>(sg, std::move(key)); + if (!p) { + internal::no_such_scheduling_group(sg); + } + return *p; +} + +/** + * Returns a reference to the current specific data. + * @param key - The scheduling group key that for the data to access + * @return A reference of type T& to the data. + * + * @note The parameter T has to be given since there is no way to deduce it. + */ +template<typename T> +T& scheduling_group_get_specific(scheduling_group_key key) noexcept { + // Unlike internal::scheduling_group_get_specific_ptr, this can + // return a reference to an element whose queue_is_initialized is + // false. + auto& data = internal::get_scheduling_group_specific_thread_local_data(); + assert(std::type_index(typeid(T)) == data.scheduling_group_key_configs[key.id()].type_index); + auto sg_id = internal::scheduling_group_index(current_scheduling_group()); + return *reinterpret_cast<T*>(data.per_scheduling_group_data[sg_id].specific_vals[key.id()]); +} + +/** + * A map reduce over all values of a specific scheduling group data. + * @param mapper - A functor SomeType(SpecificValType&) or SomeType(SpecificValType) that maps + * the specific data to a value of any type. + * @param reducer - A functor of of type ConvetibleToInitial(Initial, MapperReurnType) that reduces + * a value of type Initial and of the mapper return type to a value of type convertible to Initial. + * @param initial_val - the initial value to pass in the first call to the reducer. + * @param key - the key to the specific data that the mapper should act upon. + * @return A future that resolves when the result of the map reduce is ready. + * @note The type of SpecificValType must be given because there is no way to deduce it in a *consistent* + * manner. + * @note Theoretically the parameter type of Mapper can be deduced to be the type (function_traits<Mapper>::arg<0>) + * but then there is a danger when the Mapper accepts a parameter type T where SpecificValType is convertible to + * SpecificValType. + */ +template<typename SpecificValType, typename Mapper, typename Reducer, typename Initial> +SEASTAR_CONCEPT( requires requires(SpecificValType specific_val, Mapper mapper, Reducer reducer, Initial initial) { + {reducer(initial, mapper(specific_val))} -> std::convertible_to<Initial>; +}) +future<typename function_traits<Reducer>::return_type> +map_reduce_scheduling_group_specific(Mapper mapper, Reducer reducer, + Initial initial_val, scheduling_group_key key) { + using per_scheduling_group = internal::scheduling_group_specific_thread_local_data::per_scheduling_group; + auto& data = internal::get_scheduling_group_specific_thread_local_data(); + auto wrapped_mapper = [key, mapper] (per_scheduling_group& psg) { + auto id = internal::scheduling_group_key_id(key); + return make_ready_future<typename function_traits<Mapper>::return_type> + (mapper(*reinterpret_cast<SpecificValType*>(psg.specific_vals[id]))); + }; + + return map_reduce( + data.per_scheduling_group_data + | boost::adaptors::filtered(std::mem_fn(&per_scheduling_group::queue_is_initialized)), + wrapped_mapper, std::move(initial_val), reducer); +} + +/** + * A reduce over all values of a specific scheduling group data. + * @param reducer - A functor of of type ConvetibleToInitial(Initial, SpecificValType) that reduces + * a value of type Initial and of the sg specific data type to a value of type convertible to Initial. + * @param initial_val - the initial value to pass in the first call to the reducer. + * @param key - the key to the specific data that the mapper should act upon. + * @return A future that resolves when the result of the reduce is ready. + * * @note The type of SpecificValType must be given because there is no way to deduce it in a *consistent* + * manner. + * @note Theoretically the parameter type of Reducer can be deduced to be the type (function_traits<Reducer>::arg<0>) + * but then there is a danger when the Reducer accepts a parameter type T where SpecificValType is convertible to + * SpecificValType. + */ +template<typename SpecificValType, typename Reducer, typename Initial> +SEASTAR_CONCEPT( requires requires(SpecificValType specific_val, Reducer reducer, Initial initial) { + {reducer(initial, specific_val)} -> std::convertible_to<Initial>; +}) +future<typename function_traits<Reducer>::return_type> +reduce_scheduling_group_specific(Reducer reducer, Initial initial_val, scheduling_group_key key) { + using per_scheduling_group = internal::scheduling_group_specific_thread_local_data::per_scheduling_group; + auto& data = internal::get_scheduling_group_specific_thread_local_data(); + + auto mapper = [key] (per_scheduling_group& psg) { + auto id = internal::scheduling_group_key_id(key); + return make_ready_future<SpecificValType>(*reinterpret_cast<SpecificValType*>(psg.specific_vals[id])); + }; + + return map_reduce( + data.per_scheduling_group_data + | boost::adaptors::filtered(std::mem_fn(&per_scheduling_group::queue_is_initialized)), + mapper, std::move(initial_val), reducer); +} + +} diff --git a/src/seastar/include/seastar/core/scollectd.hh b/src/seastar/include/seastar/core/scollectd.hh new file mode 100644 index 000000000..27c076690 --- /dev/null +++ b/src/seastar/include/seastar/core/scollectd.hh @@ -0,0 +1,864 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <type_traits> +#include <utility> +#include <functional> +#include <array> +#include <iterator> +#include <stdint.h> +#include <memory> +#include <string> +#include <tuple> +#include <chrono> + +#include <seastar/core/future.hh> +#include <seastar/net/byteorder.hh> +#include <seastar/core/shared_ptr.hh> +#include <seastar/core/sstring.hh> +#include <seastar/util/log.hh> +#include <seastar/util/program-options.hh> + +#include <seastar/core/metrics_api.hh> + +namespace seastar { + +/** + * Implementation of rudimentary collectd data gathering. + * + * Usage is hopefully straight forward. Though, feel free to read + * https://collectd.org/wiki/index.php/Naming_schema + * for an explanation on the naming model. + * + * Typically, you'll add values something like: + * + * scollectd::type_instance_id typ("<pluginname>", "<instance_name>", "<type_name>", "<instance_name>"); + * scollectd::add_polled_metric(typ, [<metric var> | scollectd::make_typed(<data_type>, <metric_var>) [, ...]); + * + * Where + * `<pluginname>` would be the overall 'module', e.g. "cpu" + * `<instance_name>` -> optional distinguisher between plugin instances. For cpu, the built-in + * scollectd::per_cpu_plugin_instance constant is a good choice, i.e. 0->N cpu. + * If there are no instances (e.g. only one), empty constant is appropriate (none) + * `<type_name>` is the 'type' of metric collected, for ex. "usage" (cpu/0/usage) + * `<type_instance>` is a distinguisher for metric parts of the type, e.g. "idle", "user", "kernel" + * -> cpu/0/usage/idle | cpu/0/usage/user | cpu/0/usage/kernel + * + * Each type instance can bind an arbitrary number of values, ech representing some aspect in turn of the instance. + * The structure and interpretation is up to the producer/consumer + * + * There is a single "scollectd" instance per cpu, and values should be bound locally + * to this cpu. Polling is done at a frequency set in the seastar config (def once per s), + * and all registered values will be sent via UDP packages to the destination host(s) + * + * Note that the tuple { plugin, plugin_instance, type, type_instance } is considered a + * unique ID for a value registration, so using the same tuple twice will remove the previously + * registered values. + * + * Values can be unregistered at any time, though they must be so on the same thread/cpu + * as they we're registered. The "registration" achor type provides RAII style value unregistration + * semantics. + * + */ + +namespace scollectd { + +extern seastar::logger logger; + +using data_type = seastar::metrics::impl::data_type; + +enum class known_type { + // from types.db. Defined collectd types (type_id) selection. + // This enum omits the very application specific types, such + // as mysql_* etc, since if you really are re-writing mysql + // in seastar, you probably know how to look the type up manually... + + absolute, + backends, + bitrate, + blocked_clients, + bytes, + cache_eviction, + cache_operation, + cache_ratio, + cache_result, + cache_size, + capacity, + changes_since_last_save, + charge, + clock_last_meas, + clock_last_update, + clock_mode, + clock_reachability, + clock_skew_ppm, + clock_state, + clock_stratum, + compression, + compression_ratio, + connections, + conntrack, + contextswitch, + count, + counter, + cpu, + cpufreq, + current, + current_connections, + current_sessions, + delay, + derive, + df, + df_complex, + df_inodes, + disk_io_time, + disk_latency, + disk_merged, + disk_octets, + disk_ops, + disk_ops_complex, + disk_time, + dns_answer, + dns_notify, + dns_octets, + dns_opcode, + dns_qtype, + dns_qtype_cached, + dns_query, + dns_question, + dns_rcode, + dns_reject, + dns_request, + dns_resolver, + dns_response, + dns_transfer, + dns_update, + dns_zops, + drbd_resource, + duration, + email_check, + email_count, + email_size, + entropy, + evicted_keys, + expired_keys, + fanspeed, + file_handles, + file_size, + files, + flow, + fork_rate, + frequency, + frequency_error, + frequency_offset, + fscache_stat, + gauge, + hash_collisions, + http_request_methods, + http_requests, + http_response_codes, + humidity, + if_collisions, + if_dropped, + if_errors, + if_multicast, + if_octets, + if_packets, + if_rx_errors, + if_rx_octets, + if_tx_errors, + if_tx_octets, + invocations, + io_octets, + io_packets, + ipt_bytes, + ipt_packets, + irq, + latency, + links, + load, + md_disks, + memory, + memory_lua, + memory_throttle_count, + multimeter, + mutex_operations, + objects, + operations, + packets, + pending_operations, + percent, + percent_bytes, + percent_inodes, + ping, + ping_droprate, + ping_stddev, + players, + power, + pressure, + protocol_counter, + pubsub, + queue_length, + records, + requests, + response_code, + response_time, + root_delay, + root_dispersion, + route_etx, + route_metric, + routes, + segments, + serial_octets, + signal_noise, + signal_power, + signal_quality, + snr, + spl, + swap, + swap_io, + tcp_connections, + temperature, + threads, + time_dispersion, + time_offset, + time_offset_ntp, + time_offset_rms, + time_ref, + timeleft, + total_bytes, + total_connections, + total_objects, + total_operations, + total_requests, + total_sessions, + total_threads, + total_time_in_ms, + total_values, + uptime, + users, + vcl, + vcpu, + virt_cpu_total, + virt_vcpu, + vmpage_action, + vmpage_faults, + vmpage_io, + vmpage_number, + volatile_changes, + voltage, + voltage_threshold, + vs_memory, + vs_processes, + vs_threads, +}; + +// don't use directly. use make_typed. +template<typename T> +struct typed { + typed(data_type t, T && v) + : type(t), value(std::forward<T>(v)) { + } + data_type type; + T value; +}; + +template<typename T> +static inline typed<T> make_typed(data_type type, T&& t) { + return typed<T>(type, std::forward<T>(t)); +} + +using plugin_id = seastar::metrics::group_name_type; +using plugin_instance_id = seastar::metrics::instance_id_type; +using type_id = seastar::metrics::metric_type_def; +using type_instance = seastar::metrics::metric_name_type; + +type_id type_id_for(known_type); + +using description = seastar::metrics::description; + +static constexpr unsigned max_collectd_field_text_len = 63; + +class type_instance_id { + static thread_local unsigned _next_truncated_idx; + + /// truncate a given field to the maximum allowed length + void truncate(sstring& field, const char* field_desc); +public: + type_instance_id() = default; + type_instance_id(plugin_id p, plugin_instance_id pi, type_id t, + scollectd::type_instance ti = std::string()) + : _plugin(std::move(p)), _plugin_instance(std::move(pi)), _type( + std::move(t)), _type_instance(std::move(ti)) { + // truncate strings to the maximum allowed length + truncate(_plugin, "plugin"); + truncate(_plugin_instance, "plugin_instance"); + truncate(_type, "type"); + truncate(_type_instance, "type_instance"); + } + type_instance_id(const seastar::metrics::impl::metric_id &id, const type_id& inherit_type) : _plugin(id.group_name()), + _plugin_instance(id.instance_id()), _type(inherit_type), + _type_instance(id.name()) { + } + type_instance_id(type_instance_id &&) = default; + type_instance_id(const type_instance_id &) = default; + + type_instance_id & operator=(type_instance_id &&) = default; + type_instance_id & operator=(const type_instance_id &) = default; + + const plugin_id & plugin() const { + return _plugin; + } + const plugin_instance_id & plugin_instance() const { + return _plugin_instance; + } + const type_id & type() const { + return _type; + } + const scollectd::type_instance & type_instance() const { + return _type_instance; + } + bool operator<(const type_instance_id&) const; + bool operator==(const type_instance_id&) const; +private: + plugin_id _plugin; + plugin_instance_id _plugin_instance; + type_id _type; + scollectd::type_instance _type_instance; +}; + +extern const plugin_instance_id per_cpu_plugin_instance; + +// Scollectd configuration options. +struct options : public program_options::option_group { + /// \brief Enable collectd daemon. + /// + /// Default: \p false. + program_options::value<bool> collectd; + /// \brief Address to send/broadcast metrics to. + /// + /// Default: \p 239.192.74.66:25826. + program_options::value<std::string> collectd_address; + /// \brief Poll period (ms). + /// + /// Frequency of sending counter metrics (0 disables). + /// Default: \p 1000. + program_options::value<unsigned> collectd_poll_period; + /// \deprecated use \ref metrics::options::metrics_hostname instead + program_options::value<std::string> collectd_hostname; + + /// \cond internal + options(program_options::option_group* parent_group); + /// \endcond +}; + +void configure(const options&); +void remove_polled_metric(const type_instance_id &); + +class plugin_instance_metrics; + +/** + * Anchor for polled registration. + * Iff the registered type is in some way none-persistent, + * use this as receiver of the reg and ensure it dies before the + * added value(s). + * + * Use: + * uint64_t v = 0; + * registration r = add_polled_metric(v); + * ++r; + * <scope end, above dies> + */ +struct registration { + registration() = default; + registration(const type_instance_id& id); + registration(type_instance_id&& id); + registration(const registration&) = delete; + registration(registration&&) = default; + ~registration(); + registration & operator=(const registration&) = delete; + registration & operator=(registration&&) = default; + + void unregister() { + remove_polled_metric(_id); + _id = type_instance_id(); + } +private: + friend class plugin_instance_metrics; + type_instance_id _id; + shared_ptr<seastar::metrics::impl::impl> _impl; +}; + +/** + * Helper type to make generating vectors of registration objects + * easier, since it constructs from an initializer list of + * type_instance_id:s, avoiding early conversion to registration objs, + * which in case of init lists, are copy semantics, not move... + */ +class registrations + : public std::vector<registration> +{ +public: + typedef std::vector<registration> vector_type; + + registrations() + {} + registrations(vector_type&& v) : vector_type(std::move(v)) + {} + registrations(const std::initializer_list<type_instance_id>& l) + : vector_type(l.begin(),l.end()) + {} + registrations& operator=(vector_type&& v) { + vector_type::operator=(std::move(v)); + return *this; + } + registrations& operator=(const std::initializer_list<type_instance_id>& l) { + return registrations::operator=(registrations(l)); + } +}; + +class value_list; + +struct typed_value { + /** + * Wraps N values of a given type (type_id). + * Used to group types into a plugin_instance_metrics + */ + template<typename... Args> + typed_value(const type_id& tid, const scollectd::type_instance& ti, description, Args&&... args); + + template<typename... Args> + typed_value(const type_id& tid, const scollectd::type_instance& ti, Args&&... args) + : typed_value(tid, ti, description(), std::forward<Args>(args)...) + {} + + const scollectd::type_instance& type_instance() const { + return _type_instance; + } + const shared_ptr<value_list>& values() const { + return _values; + } + const type_id & type() const { + return _type_id; + } +private: + type_id _type_id; + scollectd::type_instance _type_instance; + shared_ptr<value_list> _values; +}; + +class plugin_instance_metrics { +public: + template<typename... TypedValues> + plugin_instance_metrics(const plugin_id& p, const plugin_instance_id& pi, TypedValues&&... values) + : _plugin_id(p) + , _plugin_instance(pi) + , _registrations({ add_impl(values)... }) + {} + std::vector<type_instance_id> bound_ids() const; + void add(const typed_value&); +private: + type_instance_id add_impl(const typed_value&); + + plugin_id _plugin_id; + plugin_instance_id _plugin_instance; + registrations _registrations; +}; + +/** + * Simplified wrapper for the common case of per-cpu plugin instances + * (i.e. distributed objects) + */ +class percpu_plugin_instance_metrics : public plugin_instance_metrics { +public: + template<typename... TypedValues> + percpu_plugin_instance_metrics(const plugin_id& p, TypedValues&&... values) + : plugin_instance_metrics(p, per_cpu_plugin_instance, std::forward<TypedValues>(values)...) + {} +}; + +/** + * Template wrapper for type_id values, deriving type_id string + * from the known_types enum, for auto-completetion joy. + */ +template<known_type Type> +struct typed_value_impl: public typed_value { + template<typename ... Args> + typed_value_impl(const scollectd::type_instance& ti, Args&& ... args) + : typed_value(type_id_for(Type), ti, std::forward<Args>(args)...) + {} + + template<typename ... Args> + typed_value_impl(scollectd::type_instance ti, description d, Args&& ... args) + : typed_value(type_id_for(Type), std::move(ti), std::move(d), std::forward<Args>(args)...) + {} + template<typename ... Args> + typed_value_impl(description d, Args&& ... args) + : typed_value(type_id_for(Type), scollectd::type_instance(), std::move(d), std::forward<Args>(args)...) + {} +}; + +/*! + * \deprecated metrics registration should be done using the metrics layer + * + * Some typedefs for common used types. Feel free to add. + */ +typedef typed_value_impl<known_type::total_bytes> total_bytes; +typedef typed_value_impl<known_type::total_connections> total_connections; +typedef typed_value_impl<known_type::total_objects> total_objects; +typedef typed_value_impl<known_type::total_operations> total_operations; +typedef typed_value_impl<known_type::total_requests> total_requests; +typedef typed_value_impl<known_type::total_sessions> total_sessions; +typedef typed_value_impl<known_type::total_threads> total_threads; +typedef typed_value_impl<known_type::total_time_in_ms> total_time_in_ms; +typedef typed_value_impl<known_type::total_values> total_values; +typedef typed_value_impl<known_type::queue_length> queue_length; +typedef typed_value_impl<known_type::counter> counter; +typedef typed_value_impl<known_type::count> count; +typedef typed_value_impl<known_type::gauge> gauge; + +// lots of template junk to build typed value list tuples +// for registered values. +template<typename T, typename En = void> +struct data_type_for; + +template<typename T, typename En = void> +struct is_callable; + +template<typename T> +struct is_callable<T, +typename std::enable_if< +!std::is_void<std::invoke_result_t<T>>::value, +void>::type> : public std::true_type { +}; + +template<typename T> +struct is_callable<T, +typename std::enable_if<std::is_fundamental<T>::value, void>::type> : public std::false_type { +}; + +template<typename T> +struct data_type_for<T, +typename std::enable_if< +std::is_integral<T>::value && std::is_unsigned<T>::value, +void>::type> : public std::integral_constant<data_type, +data_type::COUNTER> { +}; +template<typename T> +struct data_type_for<T, +typename std::enable_if<std::is_floating_point<T>::value, void>::type> : public std::integral_constant< +data_type, data_type::GAUGE> { +}; +template<typename T> +struct data_type_for<T, +typename std::enable_if<is_callable<T>::value, void>::type> : public data_type_for< +std::invoke_result_t<T>> { +}; +template<typename T> +struct data_type_for<typed<T>> : public data_type_for<T> { +}; + +template<typename T> +class value { +public: + template<typename W> + struct wrap { + wrap(const W & v) + : _v(v) { + } + const W & operator()() const { + return _v; + } + const W & _v; + }; + + typedef typename std::remove_reference<T>::type value_type; + typedef typename std::conditional< + is_callable<typename std::remove_reference<T>::type>::value, + value_type, wrap<value_type> >::type stored_type; + + value(const value_type & t) + : value<T>(data_type_for<value_type>::value, t) { + } + value(data_type type, const value_type & t) + : _type(type), _t(t) { + } + uint64_t operator()() const { + auto v = _t(); + if (_type == data_type::GAUGE) { + return convert(double(v)); + } else { + uint64_t u = v; + return convert(u); + } + } + operator uint64_t() const { + return (*this)(); + } + operator data_type() const { + return _type; + } + data_type type() const { + return _type; + } +private: + // not super quick value -> protocol endian 64-bit values. + template<typename _Iter> + void bpack(_Iter s, _Iter e, uint64_t v) const { + while (s != e) { + *s++ = (v & 0xff); + v >>= 8; + } + } + template<typename V> + typename std::enable_if<std::is_integral<V>::value, uint64_t>::type convert( + V v) const { + uint64_t i = v; + // network byte order + return ntohq(i); + } + template<typename V> + typename std::enable_if<std::is_floating_point<V>::value, uint64_t>::type convert( + V t) const { + union { + uint64_t i; + double v; + } v; + union { + uint64_t i; + uint8_t b[8]; + } u; + v.v = t; + // intel byte order. could also obviously be faster. + // could be ignored if we just assume we're le (for now), + // but this is ok me thinks. + bpack(std::begin(u.b), std::end(u.b), v.i); + return u.i; + } + ; + + const data_type _type; + const stored_type _t; +}; + +template<typename T> +class value<typed<T>> : public value<T> { +public: + value(const typed<T> & args) +: value<T>(args.type, args.value) { + } +}; + +class value_list { + bool _enabled = true; +public: + value_list(description d) : _description(std::move(d)) + {} + value_list(value_list&&) = default; + virtual ~value_list() {} + + virtual size_t size() const = 0; + + virtual void types(data_type *) const = 0; + virtual void values(net::packed<uint64_t> *) const = 0; + + const description& desc() const { + return _description; + } + + bool empty() const { + return size() == 0; + } + + bool is_enabled() const { + return _enabled; + } + + void set_enabled(bool b) { + _enabled = b; + } +private: + description _description; +}; + +template<typename ... Args> +class values_impl: public value_list { +public: + static const size_t num_values = sizeof...(Args); + + values_impl(description d, Args&& ...args) + : value_list(std::move(d)) + , _values(std::forward<Args>(args)...) + {} + + values_impl(values_impl<Args...>&& a) = default; + values_impl(const values_impl<Args...>& a) = default; + + size_t size() const override { + return num_values; + } + void types(data_type * p) const override { + unpack(_values, [p](Args... args) { + std::initializer_list<data_type> tmp = { args... }; + std::copy(tmp.begin(), tmp.end(), p); + }); + } + void values(net::packed<uint64_t> * p) const override { + unpack(_values, [p](Args... args) { + std::initializer_list<uint64_t> tmp = { args... }; + std::copy(tmp.begin(), tmp.end(), p); + }); + } +private: + template<typename _Op> + void unpack(const std::tuple<Args...>& t, _Op&& op) const { + do_unpack(t, std::index_sequence_for<Args...> {}, std::forward<_Op>(op)); + } + + template<size_t ...S, typename _Op> + void do_unpack(const std::tuple<Args...>& t, const std::index_sequence<S...> &, _Op&& op) const { + op(std::get<S>(t)...); + } + + std::tuple < Args... > _values; +}; + +void add_polled(const type_instance_id &, const shared_ptr<value_list> &, bool enabled = true); + +typedef std::function<void()> notify_function; +template<typename... _Args> +static auto make_type_instance(description d, _Args && ... args) -> values_impl < decltype(value<_Args>(std::forward<_Args>(args)))... > +{ + return values_impl<decltype(value<_Args>(std::forward<_Args>(args)))...>( + std::move(d), value<_Args>(std::forward<_Args>(args))...); +} +/*! + * \deprecated metrics registration should be done using the metrics layer + * + */ +template<typename ... _Args> +[[deprecated("Use the metrics layer")]] static type_instance_id add_polled_metric(const plugin_id & plugin, + const plugin_instance_id & plugin_instance, const type_id & type, + const scollectd::type_instance & type_instance, _Args&& ... args) { + return add_polled_metric(plugin, plugin_instance, type, type_instance, description(), + std::forward<_Args>(args)...); +} +/*! + * \deprecated metrics registration should be done using the metrics layer + * + */ +template<typename ... _Args> +[[deprecated("Use the metrics layer")]] static type_instance_id add_polled_metric(const plugin_id & plugin, + const plugin_instance_id & plugin_instance, const type_id & type, + const scollectd::type_instance & type_instance, description d, _Args&& ... args) { + return add_polled_metric( + type_instance_id(plugin, plugin_instance, type, type_instance), std::move(d), + std::forward<_Args>(args)...); +} +template<typename ... _Args> +static future<> send_explicit_metric(const plugin_id & plugin, + const plugin_instance_id & plugin_instance, const type_id & type, + const scollectd::type_instance & type_instance, _Args&& ... args) { + return send_explicit_metric( + type_instance_id(plugin, plugin_instance, type, type_instance), + std::forward<_Args>(args)...); +} +template<typename ... _Args> +static notify_function create_explicit_metric(const plugin_id & plugin, + const plugin_instance_id & plugin_instance, const type_id & type, + const scollectd::type_instance & type_instance, _Args&& ... args) { + return create_explicit_metric( + type_instance_id(plugin, plugin_instance, type, type_instance), + std::forward<_Args>(args)...); +} + +seastar::metrics::impl::metric_id to_metrics_id(const type_instance_id & id); +/*! + * \deprecated metrics registration should be done using the metrics layer + * + */ +template<typename Arg> +[[deprecated("Use the metrics layer")]] static type_instance_id add_polled_metric(const type_instance_id & id, description d, + Arg&& arg, bool enabled = true) { + seastar::metrics::impl::get_local_impl()->add_registration(to_metrics_id(id), arg.type, seastar::metrics::impl::make_function(arg.value, arg.type), d, enabled); + return id; +} +/*! + * \deprecated metrics registration should be done using the metrics layer + * + */ +template<typename Arg> +[[deprecated("Use the metrics layer")]] static type_instance_id add_polled_metric(const type_instance_id & id, + Arg&& arg) { + return std::move(add_polled_metric(id, description(), std::forward<Arg>(arg))); +} + +/*! + * \deprecated metrics registration should be done using the metrics layer + * + */ +template<typename Args> +[[deprecated("Use the metrics layer")]] static type_instance_id add_disabled_polled_metric(const type_instance_id & id, description d, + Args&& arg) { + return add_polled_metric(id, d, std::forward<Args>(arg), false); +} + +template<typename Args> +static type_instance_id add_disabled_polled_metric(const type_instance_id & id, + Args&& args) { + return add_disabled_polled_metric(id, description(), std::forward<Args>(args)); +} + +template<typename ... Args> +static type_instance_id add_disabled_polled_metric(const type_instance_id & id, + Args&& ... args) { + return add_disabled_polled_metric(id, description(), std::forward<Args>(args)...); +} + +// "Explicit" metric sends. Sends a single value list as a message. +// Obviously not super efficient either. But maybe someone needs it sometime. +template<typename ... _Args> +static future<> send_explicit_metric(const type_instance_id & id, + _Args&& ... args) { + return send_metric(id, make_type_instance(std::forward<_Args>(args)...)); +} +template<typename ... _Args> +static notify_function create_explicit_metric(const type_instance_id & id, + _Args&& ... args) { + auto list = make_type_instance(std::forward<_Args>(args)...); + return [id, list=std::move(list)]() { + send_metric(id, list); + }; +} + +template<typename... Args> +typed_value::typed_value(const type_id& tid, const scollectd::type_instance& ti, description d, Args&&... args) + : _type_id(tid) + , _type_instance(ti) + , _values(::seastar::make_shared<decltype(make_type_instance(std::move(d), std::forward<Args>(args)...))>(make_type_instance(std::move(d), std::forward<Args>(args)...))) +{} + +// Send a message packet (string) +future<> send_notification(const type_instance_id & id, const sstring & msg); +}; + +} diff --git a/src/seastar/include/seastar/core/scollectd_api.hh b/src/seastar/include/seastar/core/scollectd_api.hh new file mode 100644 index 000000000..b4ca7eae9 --- /dev/null +++ b/src/seastar/include/seastar/core/scollectd_api.hh @@ -0,0 +1,35 @@ +/* + * Copyright 2015 Cloudius Systems + */ + +#pragma once + +#include <seastar/core/scollectd.hh> +#include <seastar/core/metrics_api.hh> + +namespace seastar { + +namespace scollectd { + +using collectd_value = seastar::metrics::impl::metric_value; + +std::vector<collectd_value> get_collectd_value( + const scollectd::type_instance_id& id); + +std::vector<scollectd::type_instance_id> get_collectd_ids(); + +sstring get_collectd_description_str(const scollectd::type_instance_id&); + +bool is_enabled(const scollectd::type_instance_id& id); +/** + * Enable or disable collectd metrics on local instance + * @param id - the metric to enable or disable + * @param enable - should the collectd metrics be enable or disable + */ +void enable(const scollectd::type_instance_id& id, bool enable); + + +metrics::impl::value_map get_value_map(); +} + +} diff --git a/src/seastar/include/seastar/core/seastar.hh b/src/seastar/include/seastar/core/seastar.hh new file mode 100644 index 000000000..401ef8fa0 --- /dev/null +++ b/src/seastar/include/seastar/core/seastar.hh @@ -0,0 +1,431 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ + +#pragma once + +/// \mainpage +/// +/// Seastar is a high performance C++ application framework for high +/// concurrency server applications. +/// +/// A good place to start is the [Tutorial](tutorial.html) or [Multi-page version](split/). +/// +/// Please see: +/// - \ref future-module Documentation on futures and promises, which are +/// the seastar building blocks. +/// - \ref future-util Utililty functions for working with futures +/// - \ref memory-module Memory management +/// - \ref networking-module TCP/IP networking +/// - \ref fileio-module File Input/Output +/// - \ref smp-module Multicore support +/// - \ref fiber-module Utilities for managing loosely coupled chains of +/// continuations, also known as fibers +/// - \ref thread-module Support for traditional threaded execution +/// - \ref rpc Build high-level communication protocols +/// - \ref websocket (experimental) Implement a WebSocket-based server +/// - \ref fsnotifier (experimental) Implement a filesystem modification notifier. +/// +/// View the [Seastar compatibility statement](./md_compatibility.html) for +/// information about library evolution. + +#include <seastar/core/sstring.hh> +#include <seastar/core/future.hh> +#include <seastar/core/file-types.hh> +#include <seastar/core/posix.hh> +#include <seastar/util/bool_class.hh> +#include <seastar/util/std-compat.hh> +#include "./internal/api-level.hh" + +namespace seastar { + +// iostream.hh +template <class CharType> class input_stream; +template <class CharType> class output_stream; + +class server_socket; +class socket; +class connected_socket; +class socket_address; +struct listen_options; +enum class transport; + +// file.hh +class file; +struct file_open_options; +struct stat_data; + +namespace net { + +class udp_channel; + +} + +namespace experimental { +// process.hh +class process; +struct spawn_parameters; +} + +// Networking API + +/// \defgroup networking-module Networking +/// +/// Seastar provides a simple networking API, backed by two +/// TCP/IP stacks: the POSIX stack, utilizing the kernel's +/// BSD socket APIs, and the native stack, implement fully +/// within seastar and able to drive network cards directly. +/// The native stack supports zero-copy on both transmit +/// and receive, and is implemented using seastar's high +/// performance, lockless sharded design. The network stack +/// can be selected with the \c \--network-stack command-line +/// parameter. + +/// \addtogroup networking-module +/// @{ + +/// Listen for connections on a given port +/// +/// Starts listening on a given address for incoming connections. +/// +/// \param sa socket address to listen on +/// +/// \return \ref server_socket object ready to accept connections. +/// +/// \see listen(socket_address sa, listen_options opts) +server_socket listen(socket_address sa); + +/// Listen for connections on a given port +/// +/// Starts listening on a given address for incoming connections. +/// +/// \param sa socket address to listen on +/// \param opts options controlling the listen operation +/// +/// \return \ref server_socket object ready to accept connections. +/// +/// \see listen(socket_address sa) +server_socket listen(socket_address sa, listen_options opts); + +/// Establishes a connection to a given address +/// +/// Attempts to connect to the given address. +/// +/// \param sa socket address to connect to +/// +/// \return a \ref connected_socket object, or an exception +future<connected_socket> connect(socket_address sa); + +/// Establishes a connection to a given address +/// +/// Attempts to connect to the given address with a defined local endpoint +/// +/// \param sa socket address to connect to +/// \param local socket address for local endpoint +/// \param proto transport protocol (TCP or SCTP) +/// +/// \return a \ref connected_socket object, or an exception +future<connected_socket> connect(socket_address sa, socket_address local, transport proto); + + +/// Creates a socket object suitable for establishing stream-oriented connections +/// +/// \return a \ref socket object that can be used for establishing connections +socket make_socket(); + +/// Creates a udp_channel object suitable for sending UDP packets +/// +/// The channel is not bound to a local address, and thus can only be used +/// for sending. +/// +/// \return a \ref net::udp_channel object that can be used for UDP transfers. +net::udp_channel make_udp_channel(); + + +/// Creates a udp_channel object suitable for sending and receiving UDP packets +/// +/// \param local local address to bind to +/// +/// \return a \ref net::udp_channel object that can be used for UDP transfers. +net::udp_channel make_udp_channel(const socket_address& local); + +/// @} + +/// \defgroup fileio-module File Input/Output +/// +/// Seastar provides a file API to deal with persistent storage. +/// Unlike most file APIs, seastar offers unbuffered file I/O +/// (similar to, and based on, \c O_DIRECT). Unbuffered I/O means +/// that the application is required to do its own caching, but +/// delivers better performance if this caching is done correctly. +/// +/// For random I/O or sequential unbuffered I/O, the \ref file +/// class provides a set of methods for reading, writing, discarding, +/// or otherwise manipulating a file. For buffered sequential I/O, +/// see \ref make_file_input_stream() and \ref make_file_output_stream(). + +/// \addtogroup fileio-module +/// @{ + +/// Opens or creates a file. The "dma" in the name refers to the fact +/// that data transfers are unbuffered and uncached. +/// +/// \param name the name of the file to open or create +/// \param flags various flags controlling the open process +/// \return a \ref file object, as a future +/// +/// \note +/// The file name is not guaranteed to be stable on disk, unless the +/// containing directory is sync'ed. +/// +/// \relates file +future<file> open_file_dma(std::string_view name, open_flags flags) noexcept; + +/// Opens or creates a file. The "dma" in the name refers to the fact +/// that data transfers are unbuffered and uncached. +/// +/// \param name the name of the file to open or create +/// \param flags various flags controlling the open process +/// \param options options for opening the file +/// \return a \ref file object, as a future +/// +/// \note +/// The file name is not guaranteed to be stable on disk, unless the +/// containing directory is sync'ed. +/// +/// \relates file +future<file> open_file_dma(std::string_view name, open_flags flags, file_open_options options) noexcept; + +/// Checks if a given directory supports direct io +/// +/// Seastar bypasses the Operating System caches and issues direct io to the +/// underlying block devices. Projects using seastar should check if the directory +/// lies in a filesystem that support such operations. This function can be used +/// to do that. +/// +/// It will return if direct io can be used, or throw an std::system_error +/// exception, with the EINVAL error code. +/// +/// A std::system_error with the respective error code is also thrown if \c path is +/// not a directory. +/// +/// \param path the directory we need to verify. +future<> check_direct_io_support(std::string_view path) noexcept; + +/// Opens a directory. +/// +/// \param name name of the directory to open +/// +/// \return a \ref file object representing a directory. The only +/// legal operations are \ref file::list_directory(), +/// \ref file::flush(), and \ref file::close(). +/// +/// \relates file +future<file> open_directory(std::string_view name) noexcept; + +/// Creates a new directory. +/// +/// \param name name of the directory to create +/// \param permissions optional file permissions of the directory to create. +/// +/// \note +/// The directory is not guaranteed to be stable on disk, unless the +/// containing directory is sync'ed. +future<> make_directory(std::string_view name, file_permissions permissions = file_permissions::default_dir_permissions) noexcept; + +/// Ensures a directory exists +/// +/// Checks whether a directory exists, and if not, creates it. Only +/// the last component of the directory name is created. +/// +/// \param name name of the directory to potentially create +/// \param permissions optional file permissions of the directory to create. +/// +/// \note +/// The directory is not guaranteed to be stable on disk, unless the +/// containing directory is sync'ed. +/// If the directory exists, the provided permissions are not applied. +future<> touch_directory(std::string_view name, file_permissions permissions = file_permissions::default_dir_permissions) noexcept; + +/// Recursively ensures a directory exists +/// +/// Checks whether each component of a directory exists, and if not, creates it. +/// +/// \param name name of the directory to potentially create +/// \param permissions optional file permissions of the directory to create. +/// +/// \note +/// This function fsyncs each component created, and is therefore guaranteed to be stable on disk. +/// The provided permissions are applied only on the last component in the path, if it needs to be created, +/// if intermediate directories do not exist, they are created with the default_dir_permissions. +/// If any directory exists, the provided permissions are not applied. +future<> recursive_touch_directory(std::string_view name, file_permissions permissions = file_permissions::default_dir_permissions) noexcept; + +/// Synchronizes a directory to disk +/// +/// Makes sure the modifications in a directory are synchronized in disk. +/// This is useful, for instance, after creating or removing a file inside the +/// directory. +/// +/// \param name name of the directory to potentially create +future<> sync_directory(std::string_view name) noexcept; + + +/// Removes (unlinks) a file or an empty directory +/// +/// \param name name of the file or the directory to remove +/// +/// \note +/// The removal is not guaranteed to be stable on disk, unless the +/// containing directory is sync'ed. +future<> remove_file(std::string_view name) noexcept; + +/// Renames (moves) a file. +/// +/// \param old_name existing file name +/// \param new_name new file name +/// +/// \note +/// The rename is not guaranteed to be stable on disk, unless the +/// both containing directories are sync'ed. +future<> rename_file(std::string_view old_name, std::string_view new_name) noexcept; + +struct follow_symlink_tag { }; +using follow_symlink = bool_class<follow_symlink_tag>; + +/// Return stat information about a file. +/// +/// \param name name of the file to return its stat information +/// \param fs a follow_symlink flag to follow symbolic links. +/// +/// \return stat_data of the file identified by name. +/// If name identifies a symbolic link then stat_data is returned either for the target of the link, +/// with follow_symlink::yes, or for the link itself, with follow_symlink::no. +future<stat_data> file_stat(std::string_view name, follow_symlink fs = follow_symlink::yes) noexcept; + +/// Return the size of a file. +/// +/// \param name name of the file to return the size +/// +/// Note that file_size of a symlink is NOT the size of the symlink - +/// which is the length of the pathname it contains - +/// but rather the size of the file to which it points. +future<uint64_t> file_size(std::string_view name) noexcept; + +/// Check file access. +/// +/// \param name name of the file to check +/// \param flags bit pattern containing type of access to check (read/write/execute or exists). +/// +/// If only access_flags::exists is queried, returns true if the file exists, or false otherwise. +/// Throws a std::filesystem::filesystem_error exception if any error other than ENOENT is encountered. +/// +/// If any of the access_flags (read/write/execute) is set, returns true if the file exists and is +/// accessible with the requested flags, or false if the file exists and is not accessible +/// as queried. +/// Throws a std::filesystem::filesystem_error exception if any error other than EACCES is encountered. +/// Note that if any path component leading to the file is not searchable, the file is considered inaccessible +/// with the requested mode and false will be returned. +future<bool> file_accessible(std::string_view name, access_flags flags) noexcept; + +/// check if a file exists. +/// +/// \param name name of the file to check +future<bool> file_exists(std::string_view name) noexcept; + +/// Determine the type of a file (regular file, directory, etc.) +/// +/// \param name name of the file for which type information is requested +/// \param follow a follow_symlink flag that determines whether a trailing symbolic link should be followed or not +/// +/// \return a engaged optional with the file type if lookup was successful; a disengaged optional +/// if the file (or one of its parent directories) does not exist; an exceptional future on +/// other errors. +future<std::optional<directory_entry_type>> file_type(std::string_view name, follow_symlink follow = follow_symlink::yes) noexcept; + + +/// Creates a hard link for a file +/// +/// \param oldpath existing file name +/// \param newpath name of link +/// +future<> link_file(std::string_view oldpath, std::string_view newpath) noexcept; + +/// Changes the permissions mode of a file or directory +/// +/// \param name name of the file ot directory to change +/// \param permissions permissions to set +/// +future<> chmod(std::string_view name, file_permissions permissions) noexcept; + +/// Return information about the filesystem where a file is located. +/// +/// \param name name of the file to inspect +future<fs_type> file_system_at(std::string_view name) noexcept; + +/// Return space available to unprivileged users in filesystem where a file is located, in bytes. +/// +/// \param name name of the file to inspect +future<uint64_t> fs_avail(std::string_view name) noexcept; + +/// Return free space in filesystem where a file is located, in bytes. +/// +/// \param name name of the file to inspect +future<uint64_t> fs_free(std::string_view name) noexcept; +/// @} + +namespace experimental { +/// \defgroup interprocess-module Interprocess Communication +/// +/// Seastar provides a set of APIs for interprocess communicate + +/// \addtogroup interprocess-module +/// @{ + +/// Create a pipe using \c pipe2 +/// +/// \return a tuple of \c file_desc, the first one for reading from the pipe, the second +/// for writing to it. +future<std::tuple<file_desc, file_desc>> make_pipe(); + +/// Spawn a subprocess +/// +/// \param pathname the path to the executable +/// \param params parameters for spawning the subprocess +/// +/// \return a process representing the spawned subprocess +/// \note +/// the subprocess is spawned with \c posix_spawn() system call, so the +/// pathname should be relative or absolute path of the executable. +future<process> spawn_process(const std::filesystem::path& pathname, + spawn_parameters params); +/// Spawn a subprocess +/// +/// \param pathname the path to the executable +/// +/// \return a process representing the spawned subprocess +/// \note +/// the this overload does not specify a \c params parameters for spawning the +/// subprocess. Instead, it uses the pathname for the \c argv[0] in the params. +future<process> spawn_process(const std::filesystem::path& pathname); +/// @} +} +} diff --git a/src/seastar/include/seastar/core/semaphore.hh b/src/seastar/include/seastar/core/semaphore.hh new file mode 100644 index 000000000..23f3b7984 --- /dev/null +++ b/src/seastar/include/seastar/core/semaphore.hh @@ -0,0 +1,787 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/core/future.hh> +#include <seastar/core/chunked_fifo.hh> +#include <stdexcept> +#include <exception> +#include <optional> +#include <seastar/core/timer.hh> +#include <seastar/core/abortable_fifo.hh> +#include <seastar/core/timed_out_error.hh> +#include <seastar/core/abort_on_expiry.hh> + +#ifdef SEASTAR_DEBUG +#define SEASTAR_SEMAPHORE_DEBUG +#endif + +namespace seastar { + +namespace internal { +// Test if a class T has member function broken() +template <typename T> +class has_broken { + template <typename U> constexpr static bool check(decltype(&U::broken)) { return true; } + template <typename U> constexpr static bool check(...) { return false; } + +public: + constexpr static bool value = check<T>(nullptr); +}; +// Test if a class T has member function aborted() +template <typename T> +class has_aborted { + template <typename U> constexpr static bool check(decltype(&U::aborted)) { return true; } + template <typename U> constexpr static bool check(...) { return false; } + +public: + constexpr static bool value = check<T>(nullptr); +}; +} + +/// \addtogroup fiber-module +/// @{ + +/// Exception thrown when a semaphore is broken by +/// \ref semaphore::broken(). +class broken_semaphore : public std::exception { +public: + /// Reports the exception reason. + virtual const char* what() const noexcept; +}; + +/// Exception thrown when a semaphore wait operation +/// times out. +/// +/// \see semaphore::wait(typename timer<>::duration timeout, size_t nr) +class semaphore_timed_out : public timed_out_error { +public: + /// Reports the exception reason. + virtual const char* what() const noexcept; +}; + +/// Exception thrown when a semaphore wait operation +/// was aborted. +/// +/// \see semaphore::wait(abort_source&, size_t nr) +class semaphore_aborted : public abort_requested_exception { +public: + /// Reports the exception reason. + virtual const char* what() const noexcept; +}; + +/// Exception Factory for standard semaphore +/// +/// constructs standard semaphore exceptions +/// \see semaphore_timed_out and broken_semaphore +struct semaphore_default_exception_factory { + static semaphore_timed_out timeout() noexcept; + static broken_semaphore broken() noexcept; + static semaphore_aborted aborted() noexcept; +}; + +class named_semaphore_timed_out : public semaphore_timed_out { + sstring _msg; +public: + named_semaphore_timed_out(std::string_view msg) noexcept; + virtual const char* what() const noexcept; +}; + +class broken_named_semaphore : public broken_semaphore { + sstring _msg; +public: + broken_named_semaphore(std::string_view msg) noexcept; + virtual const char* what() const noexcept; +}; + +class named_semaphore_aborted : public semaphore_aborted { + sstring _msg; +public: + named_semaphore_aborted(std::string_view msg) noexcept; + virtual const char* what() const noexcept; +}; + +// A factory of semaphore exceptions that contain additional context: the semaphore name +// auto sem = named_semaphore(0, named_semaphore_exception_factory{"file_opening_limit_semaphore"}); +struct named_semaphore_exception_factory { + sstring name; + named_semaphore_timed_out timeout() const noexcept; + broken_named_semaphore broken() const noexcept; + named_semaphore_aborted aborted() const noexcept; +}; + +template<typename ExceptionFactory, typename Clock> +class semaphore_units; + +/// \brief Counted resource guard. +/// +/// This is a standard computer science semaphore, adapted +/// for futures. You can deposit units into a counter, +/// or take them away. Taking units from the counter may wait +/// if not enough units are available. +/// +/// To support exceptional conditions, a \ref broken() method +/// is provided, which causes all current waiters to stop waiting, +/// with an exceptional future returned. This allows causing all +/// fibers that are blocked on a semaphore to continue. This is +/// similar to POSIX's `pthread_cancel()`, with \ref wait() acting +/// as a cancellation point. +/// +/// \tparam ExceptionFactory template parameter allows modifying a semaphore to throw +/// customized exceptions on timeout/broken/aborted. It has to provide three functions: +/// ExceptionFactory::timeout() - that returns a \ref semaphore_timed_out exception by default, +/// ExceptionFactory::broken() - that returns a \ref broken_semaphore exception by default, and +/// ExceptionFactory::aborted() - that returns a \ref semaphore_aborted exception by default. +template<typename ExceptionFactory, typename Clock = typename timer<>::clock> +class basic_semaphore : private ExceptionFactory { +public: + using duration = typename timer<Clock>::duration; + using clock = typename timer<Clock>::clock; + using time_point = typename timer<Clock>::time_point; + using exception_factory = ExceptionFactory; +private: + ssize_t _count; + std::exception_ptr _ex; +#ifdef SEASTAR_SEMAPHORE_DEBUG + size_t _outstanding_units = 0; + + void assert_not_held() const noexcept { + assert(!_outstanding_units && "semaphore moved with outstanding units"); + } + void add_outstanding_units(size_t n) { + _outstanding_units += n; + } + void sub_outstanding_units(size_t n) { + _outstanding_units -= n; + } +#else // SEASTAR_SEMAPHORE_DEBUG + void assert_not_held() const noexcept {} + void add_outstanding_units(size_t) {} + void sub_outstanding_units(size_t) {} +#endif // SEASTAR_SEMAPHORE_DEBUG + + struct entry { + promise<> pr; + size_t nr; + std::optional<abort_on_expiry<clock>> timer; + entry(promise<>&& pr_, size_t nr_) noexcept : pr(std::move(pr_)), nr(nr_) {} + }; + struct expiry_handler { + basic_semaphore& sem; + void operator()(entry& e) noexcept { + if (e.timer) { + try { + e.pr.set_exception(sem.timeout()); + } catch (...) { + e.pr.set_exception(semaphore_timed_out()); + } + } else if (sem._ex) { + e.pr.set_exception(sem._ex); + } else { + if constexpr (internal::has_aborted<exception_factory>::value) { + try { + e.pr.set_exception(static_cast<exception_factory>(sem).aborted()); + } catch (...) { + e.pr.set_exception(semaphore_aborted()); + } + } else { + e.pr.set_exception(semaphore_aborted()); + } + } + } + }; + internal::abortable_fifo<entry, expiry_handler> _wait_list; + bool has_available_units(size_t nr) const noexcept { + return _count >= 0 && (static_cast<size_t>(_count) >= nr); + } + bool may_proceed(size_t nr) const noexcept { + return has_available_units(nr) && _wait_list.empty(); + } + + friend class semaphore_units<ExceptionFactory, Clock>; +public: + /// Returns the maximum number of units the semaphore counter can hold + static constexpr size_t max_counter() noexcept { + return std::numeric_limits<decltype(_count)>::max(); + } + + /// Constructs a semaphore object with a specific number of units + /// in its internal counter. E.g., starting it at 1 is suitable for use as + /// an unlocked mutex. + /// + /// \param count number of initial units present in the counter. + basic_semaphore(size_t count) noexcept(std::is_nothrow_default_constructible_v<exception_factory>) + : exception_factory() + , _count(count), + _wait_list(expiry_handler{*this}) + {} + basic_semaphore(size_t count, exception_factory&& factory) noexcept(std::is_nothrow_move_constructible_v<exception_factory>) + : exception_factory(std::move(factory)) + , _count(count) + , _wait_list(expiry_handler{*this}) + { + static_assert(std::is_nothrow_move_constructible_v<expiry_handler>); + } + + basic_semaphore(const basic_semaphore&) = delete; + basic_semaphore(basic_semaphore&& o) noexcept + : _count(std::exchange(o._count, 0)) + , _ex(std::move(o._ex)) + , _wait_list(std::move(o._wait_list)) + { + o.assert_not_held(); + } + + basic_semaphore& operator=(basic_semaphore&& o) noexcept { + if (this != &o) { + o.assert_not_held(); + assert_not_held(); + _count = std::exchange(o._count, 0); + _ex = std::move(o._ex); + _wait_list = std::move(o._wait_list); + } + return *this; + } + + /// Waits until at least a specific number of units are available in the + /// counter, and reduces the counter by that amount of units. + /// + /// \note Waits are serviced in FIFO order, though if several are awakened + /// at once, they may be reordered by the scheduler. + /// + /// \param nr Amount of units to wait for (default 1). + /// \return a future that becomes ready when sufficient units are available + /// to satisfy the request. If the semaphore was \ref broken(), may + /// contain an exception. + future<> wait(size_t nr = 1) noexcept { + return wait(time_point::max(), nr); + } + /// Waits until at least a specific number of units are available in the + /// counter, and reduces the counter by that amount of units. If the request + /// cannot be satisfied in time, the request is aborted. + /// + /// \note Waits are serviced in FIFO order, though if several are awakened + /// at once, they may be reordered by the scheduler. + /// + /// \param timeout expiration time. + /// \param nr Amount of units to wait for (default 1). + /// \return a future that becomes ready when sufficient units are available + /// to satisfy the request. On timeout, the future contains a + /// \ref semaphore_timed_out exception. If the semaphore was + /// \ref broken(), may contain a \ref broken_semaphore exception. + future<> wait(time_point timeout, size_t nr = 1) noexcept { + if (may_proceed(nr)) { + _count -= nr; + return make_ready_future<>(); + } + if (_ex) { + return make_exception_future(_ex); + } + try { + entry& e = _wait_list.emplace_back(promise<>(), nr); + auto f = e.pr.get_future(); + if (timeout != time_point::max()) { + e.timer.emplace(timeout); + abort_source& as = e.timer->abort_source(); + _wait_list.make_back_abortable(as); + } + return f; + } catch (...) { + return make_exception_future(std::current_exception()); + } + } + + /// Waits until at least a specific number of units are available in the + /// counter, and reduces the counter by that amount of units. The request + /// can be aborted using an \ref abort_source. + /// + /// \note Waits are serviced in FIFO order, though if several are awakened + /// at once, they may be reordered by the scheduler. + /// + /// \param as abort source. + /// \param nr Amount of units to wait for (default 1). + /// \return a future that becomes ready when sufficient units are available + /// to satisfy the request. On abort, the future contains a + /// \ref semaphore_aborted exception. If the semaphore was + /// \ref broken(), may contain a \ref broken_semaphore exception. + future<> wait(abort_source& as, size_t nr = 1) noexcept { + if (may_proceed(nr)) { + _count -= nr; + return make_ready_future<>(); + } + if (_ex) { + return make_exception_future(_ex); + } + try { + entry& e = _wait_list.emplace_back(promise<>(), nr); + // taking future here since make_back_abortable may expire the entry + auto f = e.pr.get_future(); + _wait_list.make_back_abortable(as); + return f; + } catch (...) { + return make_exception_future(std::current_exception()); + } + } + + /// Waits until at least a specific number of units are available in the + /// counter, and reduces the counter by that amount of units. If the request + /// cannot be satisfied in time, the request is aborted. + /// + /// \note Waits are serviced in FIFO order, though if several are awakened + /// at once, they may be reordered by the scheduler. + /// + /// \param timeout how long to wait. + /// \param nr Amount of units to wait for (default 1). + /// \return a future that becomes ready when sufficient units are available + /// to satisfy the request. On timeout, the future contains a + /// \ref semaphore_timed_out exception. If the semaphore was + /// \ref broken(), may contain a \ref broken_semaphore exception. + future<> wait(duration timeout, size_t nr = 1) noexcept { + return wait(clock::now() + timeout, nr); + } + /// Deposits a specified number of units into the counter. + /// + /// The counter is incremented by the specified number of units. + /// If the new counter value is sufficient to satisfy the request + /// of one or more waiters, their futures (in FIFO order) become + /// ready, and the value of the counter is reduced according to + /// the amount requested. + /// + /// \param nr Number of units to deposit (default 1). + void signal(size_t nr = 1) noexcept { + if (_ex) { + return; + } + _count += nr; + while (!_wait_list.empty() && has_available_units(_wait_list.front().nr)) { + auto& x = _wait_list.front(); + _count -= x.nr; + x.pr.set_value(); + _wait_list.pop_front(); + } + } + + /// Consume the specific number of units without blocking + // + /// Consume the specific number of units now, regardless of how many units are available + /// in the counter, and reduces the counter by that amount of units. This operation may + /// cause the counter to go negative. + /// + /// \param nr Amount of units to consume (default 1). + void consume(size_t nr = 1) noexcept { + if (_ex) { + return; + } + _count -= nr; + } + + /// Attempts to reduce the counter value by a specified number of units. + /// + /// If sufficient units are available in the counter, and if no + /// other fiber is waiting, then the counter is reduced. Otherwise, + /// nothing happens. This is useful for "opportunistic" waits where + /// useful work can happen if the counter happens to be ready, but + /// when it is not worthwhile to wait. + /// + /// \param nr number of units to reduce the counter by (default 1). + /// \return `true` if the counter had sufficient units, and was decremented. + bool try_wait(size_t nr = 1) noexcept { + if (may_proceed(nr)) { + _count -= nr; + return true; + } else { + return false; + } + } + /// Returns the number of units available in the counter. + /// + /// Does not take into account any waiters. + size_t current() const noexcept { return std::max(_count, ssize_t(0)); } + + /// Returns the number of available units. + /// + /// Takes into account units consumed using \ref consume() and therefore + /// may return a negative value. + ssize_t available_units() const noexcept { return _count; } + + /// Returns the current number of waiters + size_t waiters() const noexcept { return _wait_list.size(); } + + /// Signal to waiters that an error occurred. \ref wait() will see + /// an exceptional future<> containing a \ref broken_semaphore exception. + /// The future is made available immediately. + void broken() noexcept { + std::exception_ptr ep; + if constexpr (internal::has_broken<exception_factory>::value) { + try { + ep = std::make_exception_ptr(exception_factory::broken()); + } catch (...) { + ep = std::make_exception_ptr(broken_semaphore()); + } + } else { + ep = std::make_exception_ptr(broken_semaphore()); + } + broken(std::move(ep)); + } + + /// Signal to waiters that an error occurred. \ref wait() will see + /// an exceptional future<> containing the provided exception parameter. + /// The future is made available immediately. + template <typename Exception> + void broken(const Exception& ex) noexcept { + broken(std::make_exception_ptr(ex)); + } + + /// Signal to waiters that an error occurred. \ref wait() will see + /// an exceptional future<> containing the provided exception parameter. + /// The future is made available immediately. + void broken(std::exception_ptr ex) noexcept; + + /// Reserve memory for waiters so that wait() will not throw. + void ensure_space_for_waiters(size_t n) { + _wait_list.reserve(n); + } +}; + +template<typename ExceptionFactory, typename Clock> +inline +void +basic_semaphore<ExceptionFactory, Clock>::broken(std::exception_ptr xp) noexcept { + static_assert(std::is_nothrow_copy_constructible_v<std::exception_ptr>); + _ex = xp; + _count = 0; + while (!_wait_list.empty()) { + auto& x = _wait_list.front(); + x.pr.set_exception(xp); + _wait_list.pop_front(); + } +} + +template<typename ExceptionFactory = semaphore_default_exception_factory, typename Clock = typename timer<>::clock> +class semaphore_units { + basic_semaphore<ExceptionFactory, Clock>* _sem; + size_t _n; + + semaphore_units(basic_semaphore<ExceptionFactory, Clock>* sem, size_t n) noexcept : _sem(sem), _n(n) { + _sem->add_outstanding_units(n); + } +public: + semaphore_units() noexcept : _sem(nullptr), _n(0) {} + semaphore_units(basic_semaphore<ExceptionFactory, Clock>& sem, size_t n) noexcept : semaphore_units(&sem, n) {} + semaphore_units(semaphore_units&& o) noexcept : _sem(o._sem), _n(std::exchange(o._n, 0)) { + } + semaphore_units& operator=(semaphore_units&& o) noexcept { + _sem = o._sem; + _n = std::exchange(o._n, 0); + return *this; + } + semaphore_units(const semaphore_units&) = delete; + ~semaphore_units() noexcept { + return_all(); + } + /// Return ownership of some units to the semaphore. The semaphore will be signaled by the number of units returned. + /// + /// \param units number of units to subtract. + /// + /// \note throws exception if \c units is more than those protected by the semaphore + /// + /// \return the number of remaining units + size_t return_units(size_t units) { + if (units > _n) { + throw std::invalid_argument("Cannot take more units than those protected by the semaphore"); + } + _n -= units; + _sem->sub_outstanding_units(units); + _sem->signal(units); + return _n; + } + /// Return ownership of all units. The semaphore will be signaled by the number of units returned. + void return_all() noexcept { + if (_n) { + _sem->sub_outstanding_units(_n); + _sem->signal(_n); + _n = 0; + } + } + /// Releases ownership of the units. The semaphore will not be signalled. + /// + /// \return the number of units held + size_t release() noexcept { + _sem->sub_outstanding_units(_n); + return std::exchange(_n, 0); + } + /// Splits this instance into a \ref semaphore_units object holding the specified amount of units. + /// This object will continue holding the remaining units. + /// + /// \param units number of units to subtract. + /// + /// \note throws exception if \c units is more than those protected by the semaphore + /// + /// \return semaphore_units holding the specified number of units + semaphore_units split(size_t units) { + if (units > _n) { + throw std::invalid_argument("Cannot take more units than those protected by the semaphore"); + } + _n -= units; + // `units` are split into a new `semaphore_units` object. + // since none are returned to the semaphore we subtract the + // outstanding units here to balance their eventual addition by the + // `semaphore_units` ctor. + _sem->sub_outstanding_units(units); + return semaphore_units(_sem, units); + } + /// The inverse of split(), in which the units held by the specified \ref semaphore_units + /// object are merged into the current one. The function assumes (and asserts) that both + /// are associated with the same \ref semaphore. + /// + /// \return the updated semaphore_units object + void adopt(semaphore_units&& other) noexcept { + assert(other._sem == _sem); + auto o = other.release(); + _sem->add_outstanding_units(o); + _n += o; + } + + /// Returns the number of units held + size_t count() const noexcept { + return _n; + } + + /// Returns true iff there any units held + explicit operator bool() const noexcept { + return _n != 0; + } +}; + +/// \brief Take units from semaphore temporarily +/// +/// Takes units from the semaphore and returns them when the \ref semaphore_units object goes out of scope. +/// This provides a safe way to temporarily take units from a semaphore and ensure +/// that they are eventually returned under all circumstances (exceptions, premature scope exits, etc). +/// +/// Unlike with_semaphore(), the scope of unit holding is not limited to the scope of a single async lambda. +/// +/// \param sem The semaphore to take units from +/// \param units Number of units to take +/// \return a \ref future<> holding \ref semaphore_units object. When the object goes out of scope +/// the units are returned to the semaphore. +/// +/// \note The caller must guarantee that \c sem is valid as long as +/// \ref seaphore_units object is alive. +/// +/// \related semaphore +template<typename ExceptionFactory, typename Clock = typename timer<>::clock> +future<semaphore_units<ExceptionFactory, Clock>> +get_units(basic_semaphore<ExceptionFactory, Clock>& sem, size_t units) noexcept { + return sem.wait(units).then([&sem, units] { + return semaphore_units<ExceptionFactory, Clock>{ sem, units }; + }); +} + +/// \brief Take units from semaphore temporarily with time bound on wait +/// +/// Like \ref get_units(basic_semaphore<ExceptionFactory>&, size_t) but when +/// timeout is reached before units are granted, returns an exceptional future holding \ref semaphore_timed_out. +/// +/// \param sem The semaphore to take units from +/// \param units Number of units to take +/// \return a \ref future<> holding \ref semaphore_units object. When the object goes out of scope +/// the units are returned to the semaphore. +/// If a timeout is reached before units are granted, returns an exceptional future holding \ref semaphore_timed_out. +/// +/// \note The caller must guarantee that \c sem is valid as long as +/// \ref seaphore_units object is alive. +/// +/// \related semaphore +template<typename ExceptionFactory, typename Clock = typename timer<>::clock> +future<semaphore_units<ExceptionFactory, Clock>> +get_units(basic_semaphore<ExceptionFactory, Clock>& sem, size_t units, typename basic_semaphore<ExceptionFactory, Clock>::time_point timeout) noexcept { + return sem.wait(timeout, units).then([&sem, units] { + return semaphore_units<ExceptionFactory, Clock>{ sem, units }; + }); +} + +/// \brief Take units from semaphore temporarily with time bound on wait +/// +/// Like \ref get_units(basic_semaphore<ExceptionFactory>&, size_t, basic_semaphore<ExceptionFactory>::time_point) but +/// allow the timeout to be specified as a duration. +/// +/// \param sem The semaphore to take units from +/// \param units Number of units to take +/// \param timeout a duration specifying when to timeout the current request +/// \return a \ref future<> holding \ref semaphore_units object. When the object goes out of scope +/// the units are returned to the semaphore. +/// If a timeout is reached before units are granted, returns an exceptional future holding \ref semaphore_timed_out. +/// +/// \note The caller must guarantee that \c sem is valid as long as +/// \ref seaphore_units object is alive. +/// +/// \related semaphore +template<typename ExceptionFactory, typename Clock> +future<semaphore_units<ExceptionFactory, Clock>> +get_units(basic_semaphore<ExceptionFactory, Clock>& sem, size_t units, typename basic_semaphore<ExceptionFactory, Clock>::duration timeout) noexcept { + return sem.wait(timeout, units).then([&sem, units] { + return semaphore_units<ExceptionFactory, Clock>{ sem, units }; + }); +} + +/// \brief Take units from semaphore temporarily with an \ref abort_source +/// +/// Like \ref get_units(basic_semaphore<ExceptionFactory>&, size_t, basic_semaphore<ExceptionFactory>::time_point) but +/// allow the function to be aborted using an \ref abort_source. +/// +/// \param sem The semaphore to take units from +/// \param units Number of units to take +/// \param as abort source. +/// \return a \ref future<> holding \ref semaphore_units object. When the object goes out of scope +/// the units are returned to the semaphore. +/// If get_units is aborted before units are granted, returns an exceptional future holding \ref semaphore_aborted. +/// +/// \note The caller must guarantee that \c sem is valid as long as +/// \ref seaphore_units object is alive. +/// +/// \related semaphore +template<typename ExceptionFactory, typename Clock> +future<semaphore_units<ExceptionFactory, Clock>> +get_units(basic_semaphore<ExceptionFactory, Clock>& sem, size_t units, abort_source& as) noexcept { + return sem.wait(as, units).then([&sem, units] { + return semaphore_units<ExceptionFactory, Clock>{ sem, units }; + }); +} + +/// \brief Try to take units from semaphore temporarily +/// +/// Takes units from the semaphore, if available, and returns them when the \ref semaphore_units object goes out of scope. +/// This provides a safe way to temporarily take units from a semaphore and ensure +/// that they are eventually returned under all circumstances (exceptions, premature scope exits, etc). +/// +/// Unlike with_semaphore(), the scope of unit holding is not limited to the scope of a single async lambda. +/// +/// \param sem The semaphore to take units from +/// \param units Number of units to take +/// \return an optional \ref semaphore_units object. If engaged, when the object goes out of scope or is reset +/// the units are returned to the semaphore. +/// +/// \note The caller must guarantee that \c sem is valid as long as +/// \ref seaphore_units object is alive. +/// +/// \related semaphore +template<typename ExceptionFactory, typename Clock = typename timer<>::clock> +std::optional<semaphore_units<ExceptionFactory, Clock>> +try_get_units(basic_semaphore<ExceptionFactory, Clock>& sem, size_t units) noexcept { + if (!sem.try_wait(units)) { + return std::nullopt; + } + return std::make_optional<semaphore_units<ExceptionFactory, Clock>>(sem, units); +} + +/// \brief Consume units from semaphore temporarily +/// +/// Consume units from the semaphore and returns them when the \ref semaphore_units object goes out of scope. +/// This provides a safe way to temporarily take units from a semaphore and ensure +/// that they are eventually returned under all circumstances (exceptions, premature scope exits, etc). +/// +/// Unlike get_units(), this calls the non-blocking consume() API. +/// +/// Unlike with_semaphore(), the scope of unit holding is not limited to the scope of a single async lambda. +/// +/// \param sem The semaphore to take units from +/// \param units Number of units to consume +template<typename ExceptionFactory, typename Clock = typename timer<>::clock> +semaphore_units<ExceptionFactory, Clock> +consume_units(basic_semaphore<ExceptionFactory, Clock>& sem, size_t units) noexcept { + sem.consume(units); + return semaphore_units<ExceptionFactory, Clock>{ sem, units }; +} + +/// \brief Runs a function protected by a semaphore +/// +/// Acquires a \ref semaphore, runs a function, and releases +/// the semaphore, returning the the return value of the function, +/// as a \ref future. +/// +/// \param sem The semaphore to be held while the \c func is +/// running. +/// \param units Number of units to acquire from \c sem (as +/// with semaphore::wait()) +/// \param func The function to run; signature \c void() or +/// \c future<>(). +/// \return a \ref future<> holding the function's return value +/// or exception thrown; or a \ref future<> containing +/// an exception from one of the semaphore::broken() +/// variants. +/// +/// \note The caller must guarantee that \c sem is valid until +/// the future returned by with_semaphore() resolves. +/// +/// \related semaphore +template <typename ExceptionFactory, typename Func, typename Clock = typename timer<>::clock> +inline +futurize_t<std::invoke_result_t<Func>> +with_semaphore(basic_semaphore<ExceptionFactory, Clock>& sem, size_t units, Func&& func) noexcept { + return get_units(sem, units).then([func = std::forward<Func>(func)] (auto units) mutable { + return futurize_invoke(std::forward<Func>(func)).finally([units = std::move(units)] {}); + }); +} + +/// \brief Runs a function protected by a semaphore with time bound on wait +/// +/// If possible, acquires a \ref semaphore, runs a function, and releases +/// the semaphore, returning the the return value of the function, +/// as a \ref future. +/// +/// If the semaphore can't be acquired within the specified timeout, returns +/// a semaphore_timed_out exception +/// +/// \param sem The semaphore to be held while the \c func is +/// running. +/// \param units Number of units to acquire from \c sem (as +/// with semaphore::wait()) +/// \param timeout a duration specifying when to timeout the current request +/// \param func The function to run; signature \c void() or +/// \c future<>(). +/// \return a \ref future<> holding the function's return value +/// or exception thrown; or a \ref future<> containing +/// an exception from one of the semaphore::broken() +/// variants. +/// +/// \note The caller must guarantee that \c sem is valid until +/// the future returned by with_semaphore() resolves. +/// +/// \related semaphore +template <typename ExceptionFactory, typename Clock, typename Func> +inline +futurize_t<std::invoke_result_t<Func>> +with_semaphore(basic_semaphore<ExceptionFactory, Clock>& sem, size_t units, typename basic_semaphore<ExceptionFactory, Clock>::duration timeout, Func&& func) noexcept { + return get_units(sem, units, timeout).then([func = std::forward<Func>(func)] (auto units) mutable { + return futurize_invoke(std::forward<Func>(func)).finally([units = std::move(units)] {}); + }); +} + +/// default basic_semaphore specialization that throws semaphore specific exceptions +/// on error conditions. +using semaphore = basic_semaphore<semaphore_default_exception_factory>; +using named_semaphore = basic_semaphore<named_semaphore_exception_factory>; + +/// @} + +} diff --git a/src/seastar/include/seastar/core/sharded.hh b/src/seastar/include/seastar/core/sharded.hh new file mode 100644 index 000000000..d8b006dd9 --- /dev/null +++ b/src/seastar/include/seastar/core/sharded.hh @@ -0,0 +1,966 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/core/smp.hh> +#include <seastar/core/loop.hh> +#include <seastar/core/map_reduce.hh> +#include <seastar/util/is_smart_ptr.hh> +#include <seastar/util/tuple_utils.hh> +#include <seastar/core/do_with.hh> +#include <seastar/util/concepts.hh> +#include <seastar/util/log.hh> +#include <boost/iterator/counting_iterator.hpp> +#include <functional> +#if __has_include(<concepts>) +#include <concepts> +#endif + +/// \defgroup smp-module Multicore +/// +/// \brief Support for exploiting multiple cores on a server. +/// +/// Seastar supports multicore servers by using *sharding*. Each logical +/// core (lcore) runs a separate event loop, with its own memory allocator, +/// TCP/IP stack, and other services. Shards communicate by explicit message +/// passing, rather than using locks and condition variables as with traditional +/// threaded programming. + +namespace seastar { + +template <typename Func, typename... Param> +class sharded_parameter; + +template <typename Service> +class sharded; + +namespace internal { + +template <typename Func, typename... Param> +auto unwrap_sharded_arg(sharded_parameter<Func, Param...> sp); + +using on_each_shard_func = std::function<future<> (unsigned shard)>; + +future<> sharded_parallel_for_each(unsigned nr_shards, on_each_shard_func on_each_shard) noexcept(std::is_nothrow_move_constructible_v<on_each_shard_func>); + +template <typename Service> +class either_sharded_or_local { + sharded<Service>& _sharded; +public: + either_sharded_or_local(sharded<Service>& s) : _sharded(s) {} + operator sharded<Service>& (); + operator Service& (); +}; + +template <typename T> +struct sharded_unwrap { + using evaluated_type = T; + using type = T; +}; + +template <typename T> +struct sharded_unwrap<std::reference_wrapper<sharded<T>>> { + using evaluated_type = T&; + using type = either_sharded_or_local<T>; +}; + +template <typename Func, typename... Param> +struct sharded_unwrap<sharded_parameter<Func, Param...>> { + using type = std::invoke_result_t<Func, Param...>; +}; + +template <typename T> +using sharded_unwrap_evaluated_t = typename sharded_unwrap<T>::evaluated_type; + +template <typename T> +using sharded_unwrap_t = typename sharded_unwrap<T>::type; + +} // internal + + +/// \addtogroup smp-module +/// @{ + +template <typename T> +class sharded; + +/// If a sharded service inherits from this class, sharded::stop() will wait +/// until all references to this service on each shard are released before +/// returning. It is still service's own responsibility to track its references +/// in asynchronous code by calling shared_from_this() and keeping returned smart +/// pointer as long as object is in use. +template<typename T> +class async_sharded_service : public enable_shared_from_this<T> { +protected: + std::function<void()> _delete_cb; + async_sharded_service() noexcept = default; + virtual ~async_sharded_service() { + if (_delete_cb) { + _delete_cb(); + } + } + template <typename Service> friend class sharded; +}; + + +/// \brief Provide a sharded service with access to its peers +/// +/// If a service class inherits from this, it will gain a \code container() +/// \endcode method that provides access to the \ref sharded object, with which +/// it can call its peers. +template <typename Service> +class peering_sharded_service { + sharded<Service>* _container = nullptr; +private: + template <typename T> friend class sharded; + void set_container(sharded<Service>* container) noexcept { _container = container; } +public: + peering_sharded_service() noexcept = default; + peering_sharded_service(peering_sharded_service<Service>&&) noexcept = default; + peering_sharded_service(const peering_sharded_service<Service>&) = delete; + peering_sharded_service& operator=(const peering_sharded_service<Service>&) = delete; + sharded<Service>& container() noexcept { return *_container; } + const sharded<Service>& container() const noexcept { return *_container; } +}; + + +/// Exception thrown when a \ref sharded object does not exist +class no_sharded_instance_exception : public std::exception { + sstring _msg; +public: + no_sharded_instance_exception() : _msg("sharded instance does not exist") {} + explicit no_sharded_instance_exception(sstring type_info) + : _msg("sharded instance does not exist: " + type_info) {} + virtual const char* what() const noexcept override { + return _msg.c_str(); + } +}; + +/// Template helper to distribute a service across all logical cores. +/// +/// The \c sharded template manages a sharded service, by creating +/// a copy of the service on each logical core, providing mechanisms to communicate +/// with each shard's copy, and a way to stop the service. +/// +/// \tparam Service a class to be instantiated on each core. Must expose +/// a \c stop() method that returns a \c future<>, to be called when +/// the service is stopped. +template <typename Service> +class sharded { + struct entry { + shared_ptr<Service> service; + promise<> freed; + }; + std::vector<entry> _instances; +private: + using invoke_on_all_func_type = std::function<future<> (Service&)>; +private: + void service_deleted() noexcept { + _instances[this_shard_id()].freed.set_value(); + } + template <typename U, bool async> + friend struct shared_ptr_make_helper; + + template <typename T> + std::enable_if_t<std::is_base_of<peering_sharded_service<T>, T>::value> + set_container(T& service) noexcept { + service.set_container(this); + } + + template <typename T> + std::enable_if_t<!std::is_base_of<peering_sharded_service<T>, T>::value> + set_container(T&) noexcept { + } + + future<> + sharded_parallel_for_each(internal::on_each_shard_func func) noexcept(std::is_nothrow_move_constructible_v<internal::on_each_shard_func>) { + return internal::sharded_parallel_for_each(_instances.size(), std::move(func)); + } +public: + /// Constructs an empty \c sharded object. No instances of the service are + /// created. + sharded() noexcept {} + sharded(const sharded& other) = delete; + sharded& operator=(const sharded& other) = delete; + /// Sharded object with T that inherits from peering_sharded_service + /// cannot be moved safely, so disable move operations. + sharded(sharded&& other) = delete; + sharded& operator=(sharded&& other) = delete; + /// Destroyes a \c sharded object. Must not be in a started state. + ~sharded(); + + /// Starts \c Service by constructing an instance on every logical core + /// with a copy of \c args passed to the constructor. + /// + /// \param args Arguments to be forwarded to \c Service constructor + /// \return a \ref seastar::future<> that becomes ready when all instances have been + /// constructed. + template <typename... Args> + future<> start(Args&&... args) noexcept; + + /// Starts \c Service by constructing an instance on a single logical core + /// with a copy of \c args passed to the constructor. + /// + /// \param args Arguments to be forwarded to \c Service constructor + /// \return a \ref seastar::future<> that becomes ready when the instance has been + /// constructed. + template <typename... Args> + future<> start_single(Args&&... args) noexcept; + + /// Stops all started instances and destroys them. + /// + /// For every started instance, its \c stop() method is called, and then + /// it is destroyed. + future<> stop() noexcept; + + /// Invoke a type-erased function on all instances of `Service`. + /// The return value becomes ready when all instances have processed + /// the message. + /// + /// \param options the options to forward to the \ref smp::submit_to() + /// called behind the scenes. + /// \param func Function to be invoked on all shards + /// \return Future that becomes ready once all calls have completed + future<> invoke_on_all(smp_submit_to_options options, std::function<future<> (Service&)> func) noexcept; + + /// Invoke a type-erased function on all instances of `Service`. + /// The return value becomes ready when all instances have processed + /// the message. + /// Passes the default \ref smp_submit_to_options to the + /// \ref smp::submit_to() called behind the scenes. + future<> invoke_on_all(std::function<future<> (Service&)> func) noexcept { + try { + return invoke_on_all(smp_submit_to_options{}, std::move(func)); + } catch (...) { + return current_exception_as_future(); + } + } + + /// Invoke a function on all instances of `Service`. + /// The return value becomes ready when all instances have processed + /// the message. The function can be a member pointer to function, + /// a free function, or a functor. The first argument of the function + /// will be a reference to the local service on the shard. + /// + /// For a non-static pointer-to-member-function, the first argument + /// becomes `this`, not the first declared parameter. + /// + /// \param options the options to forward to the \ref smp::submit_to() + /// called behind the scenes. + /// \param func invocable accepting a `Service&` as the first parameter + /// to be invoked on all shards + /// \return Future that becomes ready once all calls have completed + template <typename Func, typename... Args> + SEASTAR_CONCEPT(requires std::invocable<Func, Service&, internal::sharded_unwrap_t<Args>...>) + future<> invoke_on_all(smp_submit_to_options options, Func func, Args... args) noexcept; + + /// Invoke a function on all instances of `Service`. + /// The return value becomes ready when all instances have processed + /// the message. + /// Passes the default \ref smp_submit_to_options to the + /// \ref smp::submit_to() called behind the scenes. + template <typename Func, typename... Args> + SEASTAR_CONCEPT(requires std::invocable<Func, Service&, internal::sharded_unwrap_t<Args>...>) + future<> invoke_on_all(Func func, Args... args) noexcept { + try { + return invoke_on_all(smp_submit_to_options{}, std::move(func), std::move(args)...); + } catch (...) { + return current_exception_as_future(); + } + } + + /// Invoke a callable on all instances of \c Service except the instance + /// which is allocated on current shard. + /// + /// \param options the options to forward to the \ref smp::submit_to() + /// called behind the scenes. + /// \param func a callable with the signature `void (Service&)` + /// or `future<> (Service&)`, to be called on each core + /// with the local instance as an argument. + /// \return a `future<>` that becomes ready when all cores but the current one have + /// processed the message. + template <typename Func, typename... Args> + SEASTAR_CONCEPT(requires std::invocable<Func, Service&, Args...>) + future<> invoke_on_others(smp_submit_to_options options, Func func, Args... args) noexcept; + + /// Invoke a callable on all instances of \c Service except the instance + /// which is allocated on current shard. + /// + /// \param func a callable with the signature `void (Service&)` + /// or `future<> (Service&)`, to be called on each core + /// with the local instance as an argument. + /// \return a `future<>` that becomes ready when all cores but the current one have + /// processed the message. + /// + /// Passes the default \ref smp_submit_to_options to the + /// \ref smp::submit_to() called behind the scenes. + template <typename Func, typename... Args> + SEASTAR_CONCEPT(requires std::invocable<Func, Service&, Args...>) + future<> invoke_on_others(Func func, Args... args) noexcept { + try { + return invoke_on_others(smp_submit_to_options{}, std::move(func), std::move(args)...); + } catch (...) { + return current_exception_as_future(); + } + } + + /// Invoke a callable on all instances of `Service` and reduce the results using + /// `Reducer`. + /// + /// \see map_reduce(Iterator begin, Iterator end, Mapper&& mapper, Reducer&& r) + template <typename Reducer, typename Func, typename... Args> + inline + auto map_reduce(Reducer&& r, Func&& func, Args&&... args) -> typename reducer_traits<Reducer>::future_type + { + return ::seastar::map_reduce(boost::make_counting_iterator<unsigned>(0), + boost::make_counting_iterator<unsigned>(_instances.size()), + [this, func = std::forward<Func>(func), args = std::make_tuple(std::forward<Args>(args)...)] (unsigned c) mutable { + return smp::submit_to(c, [this, &func, args] () mutable { + return std::apply([this, &func] (Args&&... args) mutable { + auto inst = get_local_service(); + return std::invoke(func, *inst, std::forward<Args>(args)...); + }, std::move(args)); + }); + }, std::forward<Reducer>(r)); + } + + /// The const version of \ref map_reduce(Reducer&& r, Func&& func) + template <typename Reducer, typename Func, typename... Args> + inline + auto map_reduce(Reducer&& r, Func&& func, Args&&... args) const -> typename reducer_traits<Reducer>::future_type + { + return ::seastar::map_reduce(boost::make_counting_iterator<unsigned>(0), + boost::make_counting_iterator<unsigned>(_instances.size()), + [this, func = std::forward<Func>(func), args = std::make_tuple(std::forward<Args>(args)...)] (unsigned c) { + return smp::submit_to(c, [this, &func, args] () { + return std::apply([this, &func] (Args&&... args) { + auto inst = get_local_service(); + return std::invoke(func, *inst, std::forward<Args>(args)...); + }, std::move(args)); + }); + }, std::forward<Reducer>(r)); + } + + /// Applies a map function to all shards, then reduces the output by calling a reducer function. + /// + /// \param map callable with the signature `Value (Service&)` or + /// `future<Value> (Service&)` (for some `Value` type). + /// used as the second input to \c reduce + /// \param initial initial value used as the first input to \c reduce. + /// \param reduce binary function used to left-fold the return values of \c map + /// into \c initial . + /// + /// Each \c map invocation runs on the shard associated with the service. + /// + /// \tparam Mapper unary function taking `Service&` and producing some result. + /// \tparam Initial any value type + /// \tparam Reduce a binary function taking two Initial values and returning an Initial + /// \return Result of invoking `map` with each instance in parallel, reduced by calling + /// `reduce()` on each adjacent pair of results. + template <typename Mapper, typename Initial, typename Reduce> + inline + future<Initial> + map_reduce0(Mapper map, Initial initial, Reduce reduce) { + auto wrapped_map = [this, map] (unsigned c) { + return smp::submit_to(c, [this, map] { + auto inst = get_local_service(); + return std::invoke(map, *inst); + }); + }; + return ::seastar::map_reduce(smp::all_cpus().begin(), smp::all_cpus().end(), + std::move(wrapped_map), + std::move(initial), + std::move(reduce)); + } + + /// The const version of \ref map_reduce0(Mapper map, Initial initial, Reduce reduce) + template <typename Mapper, typename Initial, typename Reduce> + inline + future<Initial> + map_reduce0(Mapper map, Initial initial, Reduce reduce) const { + auto wrapped_map = [this, map] (unsigned c) { + return smp::submit_to(c, [this, map] { + auto inst = get_local_service(); + return std::invoke(map, *inst); + }); + }; + return ::seastar::map_reduce(smp::all_cpus().begin(), smp::all_cpus().end(), + std::move(wrapped_map), + std::move(initial), + std::move(reduce)); + } + + /// Applies a map function to all shards, and return a vector of the result. + /// + /// \param mapper callable with the signature `Value (Service&)` or + /// `future<Value> (Service&)` (for some `Value` type). + /// + /// Each \c map invocation runs on the shard associated with the service. + /// + /// \tparam Mapper unary function taking `Service&` and producing some result. + /// \return Result vector of invoking `map` with each instance in parallel + template <typename Mapper, typename Future = futurize_t<std::invoke_result_t<Mapper,Service&>>, typename return_type = decltype(internal::untuple(std::declval<typename Future::tuple_type>()))> + inline future<std::vector<return_type>> map(Mapper mapper) { + return do_with(std::vector<return_type>(), std::move(mapper), + [this] (std::vector<return_type>& vec, Mapper& mapper) mutable { + vec.resize(_instances.size()); + return parallel_for_each(boost::irange<unsigned>(0, _instances.size()), [this, &vec, &mapper] (unsigned c) { + return smp::submit_to(c, [this, &mapper] { + auto inst = get_local_service(); + return mapper(*inst); + }).then([&vec, c] (auto&& res) { + vec[c] = std::move(res); + }); + }).then([&vec] { + return make_ready_future<std::vector<return_type>>(std::move(vec)); + }); + }); + } + + /// Invoke a callable on a specific instance of `Service`. + /// + /// \param id shard id to call + /// \param options the options to forward to the \ref smp::submit_to() + /// called behind the scenes. + /// \param func a callable with signature `Value (Service&, Args...)` or + /// `future<Value> (Service&, Args...)` (for some `Value` type), or a pointer + /// to a member function of Service + /// \param args parameters to the callable; will be copied or moved. To pass by reference, + /// use std::ref(). + /// + /// \return result of calling `func(instance)` on the designated instance + template <typename Func, typename... Args, typename Ret = futurize_t<std::invoke_result_t<Func, Service&, Args...>>> + SEASTAR_CONCEPT(requires std::invocable<Func, Service&, Args&&...>) + Ret + invoke_on(unsigned id, smp_submit_to_options options, Func&& func, Args&&... args) { + return smp::submit_to(id, options, [this, func = std::forward<Func>(func), args = std::tuple(std::move(args)...)] () mutable { + auto inst = get_local_service(); + return std::apply(std::forward<Func>(func), std::tuple_cat(std::forward_as_tuple(*inst), std::move(args))); + }); + } + + /// Invoke a callable on a specific instance of `Service`. + /// + /// \param id shard id to call + /// \param func a callable with signature `Value (Service&)` or + /// `future<Value> (Service&)` (for some `Value` type), or a pointer + /// to a member function of Service + /// \param args parameters to the callable + /// \return result of calling `func(instance)` on the designated instance + template <typename Func, typename... Args, typename Ret = futurize_t<std::invoke_result_t<Func, Service&, Args&&...>>> + SEASTAR_CONCEPT(requires std::invocable<Func, Service&, Args&&...>) + Ret + invoke_on(unsigned id, Func&& func, Args&&... args) { + return invoke_on(id, smp_submit_to_options(), std::forward<Func>(func), std::forward<Args>(args)...); + } + + /// Gets a reference to the local instance. + const Service& local() const noexcept; + + /// Gets a reference to the local instance. + Service& local() noexcept; + + /// Gets a shared pointer to the local instance. + shared_ptr<Service> local_shared() noexcept; + + /// Checks whether the local instance has been initialized. + bool local_is_initialized() const noexcept; + +private: + void track_deletion(shared_ptr<Service>&, std::false_type) noexcept { + // do not wait for instance to be deleted since it is not going to notify us + service_deleted(); + } + + void track_deletion(shared_ptr<Service>& s, std::true_type) { + s->_delete_cb = std::bind(std::mem_fn(&sharded<Service>::service_deleted), this); + } + + template <typename... Args> + shared_ptr<Service> create_local_service(Args&&... args) { + auto s = ::seastar::make_shared<Service>(std::forward<Args>(args)...); + set_container(*s); + track_deletion(s, std::is_base_of<async_sharded_service<Service>, Service>()); + return s; + } + + shared_ptr<Service> get_local_service() { + auto inst = _instances[this_shard_id()].service; + if (!inst) { + throw no_sharded_instance_exception(pretty_type_name(typeid(Service))); + } + return inst; + } + + shared_ptr<const Service> get_local_service() const { + auto inst = _instances[this_shard_id()].service; + if (!inst) { + throw no_sharded_instance_exception(pretty_type_name(typeid(Service))); + } + return inst; + } +}; + + +/// \brief Helper to pass a parameter to a `sharded<>` object that depends +/// on the shard. It is evaluated on the shard, just before being +/// passed to the local instance. It is useful when passing +/// parameters to sharded::start(). +template <typename Func, typename... Params> +class sharded_parameter { + Func _func; + std::tuple<Params...> _params; +public: + /// Creates a sharded parameter, which evaluates differently based on + /// the shard it is executed on. + /// + /// \param func Function to be executed + /// \param params optional parameters to be passed to the function. Can + /// be std::ref(sharded<whatever>), in which case the local + /// instance will be passed. Anything else + /// will be passed by value unchanged. + explicit sharded_parameter(Func func, Params... params) + SEASTAR_CONCEPT(requires std::invocable<Func, internal::sharded_unwrap_evaluated_t<Params>...>) + : _func(std::move(func)), _params(std::make_tuple(std::move(params)...)) { + } +private: + auto evaluate() const; + + template <typename Func_, typename... Param_> + friend auto internal::unwrap_sharded_arg(sharded_parameter<Func_, Param_...> sp); +}; + +/// \example sharded_parameter_demo.cc +/// +/// Example use of \ref sharded_parameter. + +/// @} + +template <typename Service> +sharded<Service>::~sharded() { + assert(_instances.empty()); +} + +namespace internal { + +template <typename T> +inline +T&& +unwrap_sharded_arg(T&& arg) { + return std::forward<T>(arg); +} + +template <typename Service> +either_sharded_or_local<Service> +unwrap_sharded_arg(std::reference_wrapper<sharded<Service>> arg) { + return either_sharded_or_local<Service>(arg); +} + +template <typename Func, typename... Param> +auto +unwrap_sharded_arg(sharded_parameter<Func, Param...> sp) { + return sp.evaluate(); +} + +template <typename Service> +either_sharded_or_local<Service>::operator sharded<Service>& () { return _sharded; } + +template <typename Service> +either_sharded_or_local<Service>::operator Service& () { return _sharded.local(); } + +} + +template <typename Func, typename... Param> +auto +sharded_parameter<Func, Param...>::evaluate() const { + auto unwrap_params_and_invoke = [this] (const auto&... params) { + return std::invoke(_func, internal::unwrap_sharded_arg(params)...); + }; + return std::apply(unwrap_params_and_invoke, _params); +} + +template <typename Service> +template <typename... Args> +future<> +sharded<Service>::start(Args&&... args) noexcept { + try { + _instances.resize(smp::count); + return sharded_parallel_for_each( + [this, args = std::make_tuple(std::forward<Args>(args)...)] (unsigned c) mutable { + return smp::submit_to(c, [this, args] () mutable { + _instances[this_shard_id()].service = std::apply([this] (Args... args) { + return create_local_service(internal::unwrap_sharded_arg(std::forward<Args>(args))...); + }, args); + }); + }).then_wrapped([this] (future<> f) { + try { + f.get(); + return make_ready_future<>(); + } catch (...) { + return this->stop().then([e = std::current_exception()] () mutable { + std::rethrow_exception(e); + }); + } + }); + } catch (...) { + return current_exception_as_future(); + } +} + +template <typename Service> +template <typename... Args> +future<> +sharded<Service>::start_single(Args&&... args) noexcept { + try { + assert(_instances.empty()); + _instances.resize(1); + return smp::submit_to(0, [this, args = std::make_tuple(std::forward<Args>(args)...)] () mutable { + _instances[0].service = std::apply([this] (Args... args) { + return create_local_service(internal::unwrap_sharded_arg(std::forward<Args>(args))...); + }, args); + }).then_wrapped([this] (future<> f) { + try { + f.get(); + return make_ready_future<>(); + } catch (...) { + return this->stop().then([e = std::current_exception()] () mutable { + std::rethrow_exception(e); + }); + } + }); + } catch (...) { + return current_exception_as_future(); + } +} + +namespace internal { + +// Helper check if Service::stop exists + +struct sharded_has_stop { + // If a member names "stop" exists, try to call it, even if it doesn't + // have the correct signature. This is so that we don't ignore a function + // named stop() just because the signature is incorrect, and instead + // force the user to resolve the ambiguity. + template <typename Service> + constexpr static auto check(int) -> std::enable_if_t<(sizeof(&Service::stop) >= 0), bool> { + return true; + } + + // Fallback in case Service::stop doesn't exist. + template<typename> + static constexpr auto check(...) -> bool { + return false; + } +}; + +template <bool stop_exists> +struct sharded_call_stop { + template <typename Service> + static future<> call(Service& instance); +}; + +template <> +template <typename Service> +inline +future<> sharded_call_stop<true>::call(Service& instance) { + return instance.stop(); +} + +template <> +template <typename Service> +inline +future<> sharded_call_stop<false>::call(Service&) { + return make_ready_future<>(); +} + +template <typename Service> +inline +future<> +stop_sharded_instance(Service& instance) { + constexpr bool has_stop = internal::sharded_has_stop::check<Service>(0); + return internal::sharded_call_stop<has_stop>::call(instance); +} + +} + +template <typename Service> +future<> +sharded<Service>::stop() noexcept { + try { + return sharded_parallel_for_each([this] (unsigned c) mutable { + return smp::submit_to(c, [this] () mutable { + auto inst = _instances[this_shard_id()].service; + if (!inst) { + return make_ready_future<>(); + } + return internal::stop_sharded_instance(*inst); + }); + }).then_wrapped([this] (future<> fut) { + return sharded_parallel_for_each([this] (unsigned c) { + return smp::submit_to(c, [this] { + if (_instances[this_shard_id()].service == nullptr) { + return make_ready_future<>(); + } + _instances[this_shard_id()].service = nullptr; + return _instances[this_shard_id()].freed.get_future(); + }); + }).finally([this, fut = std::move(fut)] () mutable { + _instances.clear(); + _instances = std::vector<sharded<Service>::entry>(); + return std::move(fut); + }); + }); + } catch (...) { + return current_exception_as_future(); + } +} + +template <typename Service> +future<> +sharded<Service>::invoke_on_all(smp_submit_to_options options, std::function<future<> (Service&)> func) noexcept { + try { + return sharded_parallel_for_each([this, options, func = std::move(func)] (unsigned c) { + return smp::submit_to(c, options, [this, func] { + return func(*get_local_service()); + }); + }); + } catch (...) { + return current_exception_as_future(); + } +} + +template <typename Service> +template <typename Func, typename... Args> +SEASTAR_CONCEPT(requires std::invocable<Func, Service&, internal::sharded_unwrap_t<Args>...>) +inline +future<> +sharded<Service>::invoke_on_all(smp_submit_to_options options, Func func, Args... args) noexcept { + static_assert(std::is_same_v<futurize_t<std::invoke_result_t<Func, Service&, internal::sharded_unwrap_t<Args>...>>, future<>>, + "invoke_on_all()'s func must return void or future<>"); + try { + return invoke_on_all(options, invoke_on_all_func_type([func = std::move(func), args = std::tuple(std::move(args)...)] (Service& service) mutable { + return std::apply([&service, &func] (Args&&... args) mutable { + return futurize_apply(func, std::tuple_cat(std::forward_as_tuple(service), std::tuple(internal::unwrap_sharded_arg(std::forward<Args>(args))...))); + }, std::move(args)); + })); + } catch (...) { + return current_exception_as_future(); + } +} + +template <typename Service> +template <typename Func, typename... Args> +SEASTAR_CONCEPT(requires std::invocable<Func, Service&, Args...>) +inline +future<> +sharded<Service>::invoke_on_others(smp_submit_to_options options, Func func, Args... args) noexcept { + static_assert(std::is_same_v<futurize_t<std::invoke_result_t<Func, Service&, Args...>>, future<>>, + "invoke_on_others()'s func must return void or future<>"); + try { + return invoke_on_all(options, [orig = this_shard_id(), func = std::move(func), args = std::tuple(std::move(args)...)] (Service& s) -> future<> { + return this_shard_id() == orig ? make_ready_future<>() : futurize_apply(func, std::tuple_cat(std::forward_as_tuple(s), args));; + }); + } catch (...) { + return current_exception_as_future(); + } +} + +template <typename Service> +const Service& sharded<Service>::local() const noexcept { + assert(local_is_initialized()); + return *_instances[this_shard_id()].service; +} + +template <typename Service> +Service& sharded<Service>::local() noexcept { + assert(local_is_initialized()); + return *_instances[this_shard_id()].service; +} + +template <typename Service> +shared_ptr<Service> sharded<Service>::local_shared() noexcept { + assert(local_is_initialized()); + return _instances[this_shard_id()].service; +} + +template <typename Service> +inline bool sharded<Service>::local_is_initialized() const noexcept { + return _instances.size() > this_shard_id() && + _instances[this_shard_id()].service; +} + +/// \addtogroup smp-module +/// @{ + +/// Smart pointer wrapper which makes it safe to move across CPUs. +/// +/// \c foreign_ptr<> is a smart pointer wrapper which, unlike +/// \ref shared_ptr and \ref lw_shared_ptr, is safe to move to a +/// different core. +/// +/// As seastar avoids locking, any but the most trivial objects must +/// be destroyed on the same core they were created on, so that, +/// for example, their destructors can unlink references to the +/// object from various containers. In addition, for performance +/// reasons, the shared pointer types do not use atomic operations +/// to manage their reference counts. As a result they cannot be +/// used on multiple cores in parallel. +/// +/// \c foreign_ptr<> provides a solution to that problem. +/// \c foreign_ptr<> wraps smart pointers -- \ref seastar::shared_ptr<>, +/// or similar, and remembers on what core this happened. +/// When the \c foreign_ptr<> object is destroyed, it sends a message to +/// the original core so that the wrapped object can be safely destroyed. +/// +/// \c foreign_ptr<> is a move-only object; it cannot be copied. +/// +template <typename PtrType> +SEASTAR_CONCEPT( requires (!std::is_pointer<PtrType>::value) ) +class foreign_ptr { +private: + PtrType _value; + unsigned _cpu; +private: + void destroy(PtrType p, unsigned cpu) noexcept { + // `destroy()` is called from the destructor and other + // synchronous methods (like `reset()`), that have no way to + // wait for this future. + (void)destroy_on(std::move(p), cpu); + } + + static future<> destroy_on(PtrType p, unsigned cpu) noexcept { + if (p) { + if (cpu != this_shard_id()) { + return smp::submit_to(cpu, [v = std::move(p)] () mutable { + // Destroy the contained pointer. We do this explicitly + // in the current shard, because the lambda is destroyed + // in the shard that submitted the task. + v = {}; + }); + } else { + p = {}; + } + } + return make_ready_future<>(); + } +public: + using element_type = typename std::pointer_traits<PtrType>::element_type; + using pointer = element_type*; + + /// Constructs a null \c foreign_ptr<>. + foreign_ptr() noexcept(std::is_nothrow_default_constructible_v<PtrType>) + : _value(PtrType()) + , _cpu(this_shard_id()) { + } + /// Constructs a null \c foreign_ptr<>. + foreign_ptr(std::nullptr_t) noexcept(std::is_nothrow_default_constructible_v<foreign_ptr>) : foreign_ptr() {} + /// Wraps a pointer object and remembers the current core. + foreign_ptr(PtrType value) noexcept(std::is_nothrow_move_constructible_v<PtrType>) + : _value(std::move(value)) + , _cpu(this_shard_id()) { + } + // The type is intentionally non-copyable because copies + // are expensive because each copy requires across-CPU call. + foreign_ptr(const foreign_ptr&) = delete; + /// Moves a \c foreign_ptr<> to another object. + foreign_ptr(foreign_ptr&& other) noexcept(std::is_nothrow_move_constructible_v<PtrType>) = default; + /// Destroys the wrapped object on its original cpu. + ~foreign_ptr() { + destroy(std::move(_value), _cpu); + } + /// Creates a copy of this foreign ptr. Only works if the stored ptr is copyable. + future<foreign_ptr> copy() const noexcept { + return smp::submit_to(_cpu, [this] () mutable { + auto v = _value; + return make_foreign(std::move(v)); + }); + } + /// Accesses the wrapped object. + element_type& operator*() const noexcept(noexcept(*_value)) { return *_value; } + /// Accesses the wrapped object. + element_type* operator->() const noexcept(noexcept(&*_value)) { return &*_value; } + /// Access the raw pointer to the wrapped object. + pointer get() const noexcept(noexcept(&*_value)) { return &*_value; } + /// Return the owner-shard of this pointer. + /// + /// The owner shard of the pointer can change as a result of + /// move-assigment or a call to reset(). + unsigned get_owner_shard() const noexcept { return _cpu; } + /// Checks whether the wrapped pointer is non-null. + operator bool() const noexcept(noexcept(static_cast<bool>(_value))) { return static_cast<bool>(_value); } + /// Move-assigns a \c foreign_ptr<>. + foreign_ptr& operator=(foreign_ptr&& other) noexcept(std::is_nothrow_move_constructible<PtrType>::value) { + destroy(std::move(_value), _cpu); + _value = std::move(other._value); + _cpu = other._cpu; + return *this; + } + /// Releases the owned pointer + /// + /// Warning: the caller is now responsible for destroying the + /// pointer on its owner shard. This method is best called on the + /// owner shard to avoid accidents. + PtrType release() noexcept(std::is_nothrow_default_constructible_v<PtrType>) { + return std::exchange(_value, {}); + } + /// Replace the managed pointer with new_ptr. + /// + /// The previous managed pointer is destroyed on its owner shard. + void reset(PtrType new_ptr) noexcept(std::is_nothrow_move_constructible_v<PtrType>) { + auto old_ptr = std::move(_value); + auto old_cpu = _cpu; + + _value = std::move(new_ptr); + _cpu = this_shard_id(); + + destroy(std::move(old_ptr), old_cpu); + } + /// Replace the managed pointer with a null value. + /// + /// The previous managed pointer is destroyed on its owner shard. + void reset(std::nullptr_t = nullptr) noexcept(std::is_nothrow_default_constructible_v<PtrType>) { + reset(PtrType()); + } + + /// Destroy the managed pointer. + /// + /// \returns a future that is resolved when managed pointer is destroyed on its owner shard. + future<> destroy() noexcept { + return destroy_on(std::move(_value), _cpu); + } +}; + +/// Wraps a raw or smart pointer object in a \ref foreign_ptr<>. +/// +/// \relates foreign_ptr +template <typename T> +foreign_ptr<T> make_foreign(T ptr) { + return foreign_ptr<T>(std::move(ptr)); +} + +/// @} + +template<typename T> +struct is_smart_ptr<foreign_ptr<T>> : std::true_type {}; + +} diff --git a/src/seastar/include/seastar/core/shared_future.hh b/src/seastar/include/seastar/core/shared_future.hh new file mode 100644 index 000000000..32ccb829d --- /dev/null +++ b/src/seastar/include/seastar/core/shared_future.hh @@ -0,0 +1,364 @@ +/* + * 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) 2015 ScyllaDB + */ + +#pragma once + +#include <seastar/core/future.hh> +#include <seastar/core/abortable_fifo.hh> +#include <seastar/core/abort_on_expiry.hh> +#include <seastar/core/timed_out_error.hh> + +namespace seastar { + +/// \addtogroup future-module +/// @{ + +/// Changes the clock used by shared_future<> and shared_promise<> when passed as the first template parameter. +template<typename Clock> +struct with_clock {}; + +/// \cond internal + +template <typename... T> +struct future_option_traits; + +template <typename Clock, typename... T> +struct future_option_traits<with_clock<Clock>, T...> { + using clock_type = Clock; + + template<template <typename...> class Class> + struct parametrize { + using type = Class<T...>; + }; +}; + +template <typename... T> +struct future_option_traits { + using clock_type = lowres_clock; + + template<template <typename...> class Class> + struct parametrize { + using type = Class<T...>; + }; +}; + +/// \endcond + +/// \brief Like \ref future except the result can be waited for by many fibers. +/// +/// Represents a value which may not yet be ready. A fiber can wait for the value using +/// the \ref future obtained by calling \ref get_future() or casting to \ref future type. +/// Multiple fibers are allowed to obtain a \ref future for the result using the same +/// instance of \ref shared_future. +/// +/// All futures obtained from shared_future should end up in the same state. However, +/// if the value's copy constructor throws, some of the futures may end up in a failed state +/// with an exception thrown from the copy constructor and end up with a state +/// different than other futures. +/// +/// The scope of shared_future instance doesn't have to include scopes of the futures +/// obtained from that instance. In that sense the returned futures are independent. +/// +/// shared_future can be copied at any time and all copies will resolve with the same value. +/// +/// shared_future can be in a disengaged state when it's default-constructed or moved-from. +/// When it's in such a state we say it's invalid and obtaining futures must not be attempted. +/// +/// The types in the parameter pack T must all be copy-constructible. +/// +/// When the first type in the parameter pack is \ref with_clock then it has the effect +/// of changing the clock used for timeouts by this instance. This type is omitted from +/// the parameter of the future<> objects. +/// +/// Example: +/// +/// future<int> f; +/// shared_future<with_clock<manual_clock>, int> sf(std::move(f)); +/// future<int> f2 = sf; +/// +template<typename... T> +class shared_future { + template <typename... U> friend class shared_promise; + using options = future_option_traits<T...>; +public: + using clock = typename options::clock_type; + using time_point = typename clock::time_point; + using future_type = typename future_option_traits<T...>::template parametrize<future>::type; + using promise_type = typename future_option_traits<T...>::template parametrize<promise>::type; + using value_tuple_type = typename future_option_traits<T...>::template parametrize<std::tuple>::type; +private: + /// \cond internal + class shared_state : public enable_lw_shared_from_this<shared_state> { + future_type _original_future; + struct entry { + promise_type pr; + std::optional<abort_on_expiry<clock>> timer; + }; + + struct entry_expiry { + void operator()(entry& e) noexcept { + if (e.timer) { + e.pr.set_exception(std::make_exception_ptr(timed_out_error())); + } else { + e.pr.set_exception(std::make_exception_ptr(abort_requested_exception())); + } + }; + }; + + internal::abortable_fifo<entry, entry_expiry> _peers; + + public: + ~shared_state() { + // Don't warn if the shared future is exceptional. Any + // warnings will be reported by the futures returned by + // get_future. + if (_original_future.failed()) { + _original_future.ignore_ready_future(); + } + } + explicit shared_state(future_type f) noexcept : _original_future(std::move(f)) { } + void resolve(future_type&& f) noexcept { + _original_future = std::move(f); + auto& state = _original_future._state; + if (_original_future.failed()) { + while (_peers) { + _peers.front().pr.set_exception(state.get_exception()); + _peers.pop_front(); + } + } else { + while (_peers) { + auto& p = _peers.front().pr; + try { + p.set_value(state.get_value()); + } catch (...) { + p.set_exception(std::current_exception()); + } + _peers.pop_front(); + } + } + } + + future_type get_future(time_point timeout = time_point::max()) noexcept { + // Note that some functions called below may throw, + // like pushing to _peers or copying _original_future's ready value. + // We'd rather terminate than propagate these errors similar to + // .then()'s failure to allocate a continuation as the caller cannot + // distinguish between an error returned by the original future to + // failing to perform `get_future` itself. + memory::scoped_critical_alloc_section _; + if (!_original_future.available()) { + entry& e = _peers.emplace_back(); + + auto f = e.pr.get_future(); + if (timeout != time_point::max()) { + e.timer.emplace(timeout); + abort_source& as = e.timer->abort_source(); + _peers.make_back_abortable(as); + } + if (_original_future._state.valid()) { + // _original_future's result is forwarded to each peer. + (void)_original_future.then_wrapped([s = this->shared_from_this()] (future_type&& f) mutable { + s->resolve(std::move(f)); + }); + } + return f; + } else if (_original_future.failed()) { + return future_type(exception_future_marker(), std::exception_ptr(_original_future._state.get_exception())); + } else { + return future_type(ready_future_marker(), _original_future._state.get_value()); + } + } + + future_type get_future(abort_source& as) noexcept { + // Note that some functions called below may throw, + // like pushing to _peers or copying _original_future's ready value. + // We'd rather terminate than propagate these errors similar to + // .then()'s failure to allocate a continuation as the caller cannot + // distinguish between an error returned by the original future to + // failing to perform `get_future` itself. + memory::scoped_critical_alloc_section _; + if (!_original_future.available()) { + entry& e = _peers.emplace_back(); + + auto f = e.pr.get_future(); + _peers.make_back_abortable(as); + if (_original_future._state.valid()) { + // _original_future's result is forwarded to each peer. + (void)_original_future.then_wrapped([s = this->shared_from_this()] (future_type&& f) mutable { + s->resolve(std::move(f)); + }); + } + return f; + } else if (_original_future.failed()) { + return future_type(exception_future_marker(), std::exception_ptr(_original_future._state.get_exception())); + } else { + return future_type(ready_future_marker(), _original_future._state.get_value()); + } + } + + bool available() const noexcept { + return _original_future.available(); + } + + bool failed() const noexcept { + return _original_future.failed(); + } + }; + /// \endcond + lw_shared_ptr<shared_state> _state; +public: + /// \brief Forwards the result of future \c f into this shared_future. + shared_future(future_type f) + : _state(make_lw_shared<shared_state>(std::move(f))) { } + + shared_future() = default; // noexcept, based on the respective lw_shared_ptr constructor + shared_future(const shared_future&) = default; // noexcept, based on the respective lw_shared_ptr constructor + shared_future& operator=(const shared_future&) = default; // noexcept, based on respective constructor + shared_future(shared_future&&) = default; // noexcept, based on the respective lw_shared_ptr constructor + shared_future& operator=(shared_future&&) = default; // noexcept, based on the respective constructor + + /// \brief Creates a new \c future which will resolve with the result of this shared_future + /// + /// \param timeout When engaged, the returned future will resolve with \ref timed_out_error + /// if this shared_future doesn't resolve before timeout is reached. + /// + /// This object must be in a valid state. + future_type get_future(time_point timeout = time_point::max()) const noexcept { + return _state->get_future(timeout); + } + + /// \brief Creates a new \c future which will resolve with the result of this shared_future + /// + /// \param as abort source. The returned future will resolve with \ref abort_requested_exception + /// if this shared_future doesn't resolve before aborted. + /// + /// This object must be in a valid state. + future_type get_future(abort_source& as) const noexcept { + return _state->get_future(as); + } + + /// \brief Returns true if the future is available (ready or failed) + /// + /// \note This object must be in a valid state. + bool available() const noexcept { + return _state->available(); + } + + /// \brief Returns true if the future is failed + /// + /// \note This object must be in a valid state. + bool failed() const noexcept { + return _state->failed(); + } + + /// \brief Equivalent to \ref get_future() + operator future_type() const noexcept { + return get_future(); + } + + /// \brief Returns true if the instance is in valid state + bool valid() const noexcept { + return bool(_state); + } +}; + +/// \brief Like \ref promise except that its counterpart is \ref shared_future instead of \ref future +/// +/// When the shared_promise is made ready, every waiter is also made ready. +/// +/// Like the shared_future, the types in the parameter pack T must all be copy-constructible. +template <typename... T> +class shared_promise { +public: + using shared_future_type = shared_future<T...>; + using future_type = typename shared_future_type::future_type; + using promise_type = typename shared_future_type::promise_type; + using clock = typename shared_future_type::clock; + using time_point = typename shared_future_type::time_point; + using value_tuple_type = typename shared_future_type::value_tuple_type; +private: + promise_type _promise; + shared_future_type _shared_future; + static constexpr bool copy_noexcept = future_type::copy_noexcept; +public: + shared_promise(const shared_promise&) = delete; + shared_promise(shared_promise&&) = default; // noexcept, based on the respective promise and shared_future constructors + shared_promise& operator=(shared_promise&&) = default; // noexcept, based on the respective promise and shared_future constructors + shared_promise() : _promise(), _shared_future(_promise.get_future()) { + } + + /// \brief Gets new future associated with this promise. + /// If the promise is not resolved before timeout the returned future will resolve with \ref timed_out_error. + /// This instance doesn't have to be kept alive until the returned future resolves. + future_type get_shared_future(time_point timeout = time_point::max()) const noexcept { + return _shared_future.get_future(timeout); + } + + /// \brief Gets new future associated with this promise. + /// If the promise is not resolved before abort source is triggered the returned future will + /// resolve with \ref abort_requests_exception. + /// This instance doesn't have to be kept alive until the returned future resolves. + future_type get_shared_future(abort_source& as) const noexcept { + return _shared_future.get_future(as); + } + + /// \brief Sets the shared_promise's value (as tuple; by copying), same as normal promise + void set_value(const value_tuple_type& result) noexcept(copy_noexcept) { + _promise.set_value(result); + } + + /// \brief Sets the shared_promise's value (as tuple; by moving), same as normal promise + void set_value(value_tuple_type&& result) noexcept { + _promise.set_value(std::move(result)); + } + + /// \brief Sets the shared_promise's value (variadic), same as normal promise + template <typename... A> + void set_value(A&&... a) noexcept { + _promise.set_value(std::forward<A>(a)...); + } + + /// \brief Marks the shared_promise as failed, same as normal promise + void set_exception(std::exception_ptr ex) noexcept { + _promise.set_exception(std::move(ex)); + } + + /// \brief Marks the shared_promise as failed, same as normal promise + template<typename Exception> + void set_exception(Exception&& e) noexcept { + set_exception(make_exception_ptr(std::forward<Exception>(e))); + } + + /// \brief Returns true if the underlying future is available (ready or failed) + bool available() const noexcept { + return _shared_future.available(); + } + + /// \brief Returns true if the underlying future is failed + bool failed() const noexcept { + return _shared_future.failed(); + } +}; + +/// @} + +} diff --git a/src/seastar/include/seastar/core/shared_mutex.hh b/src/seastar/include/seastar/core/shared_mutex.hh new file mode 100644 index 000000000..601a8a37c --- /dev/null +++ b/src/seastar/include/seastar/core/shared_mutex.hh @@ -0,0 +1,256 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/core/future.hh> +#include <seastar/core/chunked_fifo.hh> + +namespace seastar { + +/// \addtogroup fiber-module +/// @{ + +/// \brief Shared/exclusive mutual exclusion. +/// +/// Similar to \c std::shared_mutex, this class provides protection +/// for a shared resource, with two levels of access protection: shared +/// and exclusive. Shared access allows multiple tasks to access the +/// shared resource concurrently, while exclusive access allows just +/// one task to access the resource at a time. +/// +/// Note that many seastar tasks do not require protection at all, +/// since the seastar scheduler is not preemptive; however tasks that do +/// (by waiting on a future) may require explicit locking. +/// +/// The \ref with_shared(shared_mutex&, Func&&) and +/// \ref with_lock(shared_mutex&, Func&&) provide exception-safe +/// wrappers for use with \c shared_mutex. +/// +/// \see semaphore simpler mutual exclusion +class shared_mutex { + unsigned _readers = 0; + bool _writer = false; + struct waiter { + waiter(promise<>&& pr, bool for_write) : pr(std::move(pr)), for_write(for_write) {} + promise<> pr; + bool for_write; + }; + chunked_fifo<waiter> _waiters; +public: + shared_mutex() = default; + shared_mutex(shared_mutex&&) = default; + shared_mutex& operator=(shared_mutex&&) = default; + shared_mutex(const shared_mutex&) = delete; + void operator=(const shared_mutex&) = delete; + /// Lock the \c shared_mutex for shared access + /// + /// \return a future that becomes ready when no exclusive access + /// is granted to anyone. + future<> lock_shared() noexcept { + if (try_lock_shared()) { + return make_ready_future<>(); + } + try { + _waiters.emplace_back(promise<>(), false); + return _waiters.back().pr.get_future(); + } catch (...) { + return current_exception_as_future(); + } + } + /// Try to lock the \c shared_mutex for shared access + /// + /// \return true iff could acquire the lock for shared access. + bool try_lock_shared() noexcept { + if (!_writer && _waiters.empty()) { + ++_readers; + return true; + } + return false; + } + /// Unlocks a \c shared_mutex after a previous call to \ref lock_shared(). + void unlock_shared() noexcept { + assert(_readers > 0); + --_readers; + wake(); + } + /// Lock the \c shared_mutex for exclusive access + /// + /// \return a future that becomes ready when no access, shared or exclusive + /// is granted to anyone. + future<> lock() noexcept { + if (try_lock()) { + return make_ready_future<>(); + } + try { + _waiters.emplace_back(promise<>(), true); + return _waiters.back().pr.get_future(); + } catch (...) { + return current_exception_as_future(); + } + } + /// Try to lock the \c shared_mutex for exclusive access + /// + /// \return true iff could acquire the lock for exclusive access. + bool try_lock() noexcept { + if (!_readers && !_writer) { + _writer = true; + return true; + } + return false; + } + /// Unlocks a \c shared_mutex after a previous call to \ref lock(). + void unlock() noexcept { + assert(_writer); + _writer = false; + wake(); + } +private: + void wake() noexcept { + while (!_waiters.empty()) { + auto& w = _waiters.front(); + // note: _writer == false in wake() + if (w.for_write) { + if (!_readers) { + _writer = true; + w.pr.set_value(); + _waiters.pop_front(); + } + break; + } else { // for read + ++_readers; + w.pr.set_value(); + _waiters.pop_front(); + } + } + } +}; + +/// Executes a function while holding shared access to a resource. +/// +/// Executes a function while holding shared access to a resource. When +/// the function returns, the mutex is automatically unlocked. +/// +/// \param sm a \ref shared_mutex guarding access to the shared resource +/// \param func callable object to invoke while the mutex is held for shared access +/// \return whatever \c func returns, as a future +/// +/// \relates shared_mutex +template <typename Func> +SEASTAR_CONCEPT( + requires (std::invocable<Func> && std::is_nothrow_move_constructible_v<Func>) + inline + futurize_t<std::invoke_result_t<Func>> +) +SEASTAR_NO_CONCEPT( + inline + std::enable_if_t<std::is_nothrow_move_constructible_v<Func>, futurize_t<std::result_of_t<Func ()>>> +) +with_shared(shared_mutex& sm, Func&& func) noexcept { + return sm.lock_shared().then([&sm, func = std::forward<Func>(func)] () mutable { + return futurize_invoke(func).finally([&sm] { + sm.unlock_shared(); + }); + }); +} + +template <typename Func> +SEASTAR_CONCEPT( + requires (std::invocable<Func> && !std::is_nothrow_move_constructible_v<Func>) + inline + futurize_t<std::invoke_result_t<Func>> +) +SEASTAR_NO_CONCEPT( + inline + std::enable_if_t<!std::is_nothrow_move_constructible_v<Func>, futurize_t<std::result_of_t<Func ()>>> +) +with_shared(shared_mutex& sm, Func&& func) noexcept { + // FIXME: use a coroutine when c++17 support is dropped + try { + return do_with(std::forward<Func>(func), [&sm] (Func& func) { + return sm.lock_shared().then([&func] { + return func(); + }).finally([&sm] { + sm.unlock_shared(); + }); + }); + } catch (...) { + return futurize<std::invoke_result_t<Func>>::current_exception_as_future(); + } +} + +/// Executes a function while holding exclusive access to a resource. +/// +/// Executes a function while holding exclusive access to a resource. When +/// the function returns, the mutex is automatically unlocked. +/// +/// \param sm a \ref shared_mutex guarding access to the shared resource +/// \param func callable object to invoke while the mutex is held for shared access +/// \return whatever \c func returns, as a future +/// +/// \relates shared_mutex +template <typename Func> +SEASTAR_CONCEPT( + requires (std::invocable<Func> && std::is_nothrow_move_constructible_v<Func>) + inline + futurize_t<std::invoke_result_t<Func>> +) +SEASTAR_NO_CONCEPT( + inline + std::enable_if_t<std::is_nothrow_move_constructible_v<Func>, futurize_t<std::result_of_t<Func ()>>> +) +with_lock(shared_mutex& sm, Func&& func) noexcept { + return sm.lock().then([&sm, func = std::forward<Func>(func)] () mutable { + return futurize_invoke(func).finally([&sm] { + sm.unlock(); + }); + }); +} + + +template <typename Func> +SEASTAR_CONCEPT( + requires (std::invocable<Func> && !std::is_nothrow_move_constructible_v<Func>) + inline + futurize_t<std::invoke_result_t<Func>> +) +SEASTAR_NO_CONCEPT( + inline + std::enable_if_t<!std::is_nothrow_move_constructible_v<Func>, futurize_t<std::result_of_t<Func ()>>> +) +with_lock(shared_mutex& sm, Func&& func) noexcept { + // FIXME: use a coroutine when c++17 support is dropped + try { + return do_with(std::forward<Func>(func), [&sm] (Func& func) { + return sm.lock().then([&func] { + return func(); + }).finally([&sm] { + sm.unlock(); + }); + }); + } catch (...) { + return futurize<std::invoke_result_t<Func>>::current_exception_as_future(); + } +} + +/// @} + +} diff --git a/src/seastar/include/seastar/core/shared_ptr.hh b/src/seastar/include/seastar/core/shared_ptr.hh new file mode 100644 index 000000000..0f9e47e68 --- /dev/null +++ b/src/seastar/include/seastar/core/shared_ptr.hh @@ -0,0 +1,932 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/core/shared_ptr_debug_helper.hh> +#include <utility> +#include <type_traits> +#include <functional> +#include <ostream> +#include <seastar/util/is_smart_ptr.hh> +#include <seastar/util/indirect.hh> + +#include <boost/intrusive/parent_from_member.hpp> + +#if defined(__GNUC__) && !defined(__clang__) && (__GNUC__ >= 12) +// to silence the false alarm from GCC 12, see +// https://gcc.gnu.org/bugzilla/show_bug.cgi?id=105204 +#define SEASTAR_IGNORE_USE_AFTER_FREE +#endif + +namespace seastar { + +// This header defines two shared pointer facilities, lw_shared_ptr<> and +// shared_ptr<>, both modeled after std::shared_ptr<>. +// +// Unlike std::shared_ptr<>, neither of these implementations are thread +// safe, and two pointers sharing the same object must not be used in +// different threads. +// +// lw_shared_ptr<> is the more lightweight variant, with a lw_shared_ptr<> +// occupying just one machine word, and adding just one word to the shared +// object. However, it does not support polymorphism. +// +// shared_ptr<> is more expensive, with a pointer occupying two machine +// words, and with two words of overhead in the shared object. In return, +// it does support polymorphism. +// +// Both variants support shared_from_this() via enable_shared_from_this<> +// and lw_enable_shared_from_this<>(). +// + +#ifndef SEASTAR_DEBUG_SHARED_PTR +using shared_ptr_counter_type = long; +#else +using shared_ptr_counter_type = debug_shared_ptr_counter_type; +#endif + +template <typename T> +class lw_shared_ptr; + +template <typename T> +class shared_ptr; + +template <typename T> +class enable_lw_shared_from_this; + +template <typename T> +class enable_shared_from_this; + +template <typename T, typename... A> +lw_shared_ptr<T> make_lw_shared(A&&... a); + +template <typename T> +lw_shared_ptr<T> make_lw_shared(T&& a); + +template <typename T> +lw_shared_ptr<T> make_lw_shared(T& a); + +template <typename T, typename... A> +shared_ptr<T> make_shared(A&&... a); + +template <typename T> +shared_ptr<T> make_shared(T&& a); + +template <typename T, typename U> +shared_ptr<T> static_pointer_cast(const shared_ptr<U>& p); + +template <typename T, typename U> +shared_ptr<T> dynamic_pointer_cast(const shared_ptr<U>& p); + +template <typename T, typename U> +shared_ptr<T> const_pointer_cast(const shared_ptr<U>& p); + +struct lw_shared_ptr_counter_base { + shared_ptr_counter_type _count = 0; +}; + + +namespace internal { + +template <class T, class U> +struct lw_shared_ptr_accessors; + +template <class T> +struct lw_shared_ptr_accessors_esft; + +template <class T> +struct lw_shared_ptr_accessors_no_esft; + +} + + +// We want to support two use cases for shared_ptr<T>: +// +// 1. T is any type (primitive or class type) +// +// 2. T is a class type that inherits from enable_shared_from_this<T>. +// +// In the first case, we must wrap T in an object containing the counter, +// since T may be a primitive type and cannot be a base class. +// +// In the second case, we want T to reach the counter through its +// enable_shared_from_this<> base class, so that we can implement +// shared_from_this(). +// +// To implement those two conflicting requirements (T alongside its counter; +// T inherits from an object containing the counter) we use std::conditional<> +// and some accessor functions to select between two implementations. + + +// CRTP from this to enable shared_from_this: +template <typename T> +class enable_lw_shared_from_this : private lw_shared_ptr_counter_base { + using ctor = T; +protected: + enable_lw_shared_from_this() noexcept {} + enable_lw_shared_from_this(enable_lw_shared_from_this&&) noexcept {} + enable_lw_shared_from_this(const enable_lw_shared_from_this&) noexcept {} + enable_lw_shared_from_this& operator=(const enable_lw_shared_from_this&) noexcept { return *this; } + enable_lw_shared_from_this& operator=(enable_lw_shared_from_this&&) noexcept { return *this; } +public: + lw_shared_ptr<T> shared_from_this() noexcept; + lw_shared_ptr<const T> shared_from_this() const noexcept; + long use_count() const noexcept { return _count; } + + template <typename X> + friend class lw_shared_ptr; + template <typename X> + friend struct internal::lw_shared_ptr_accessors_esft; + template <typename X, class Y> + friend struct internal::lw_shared_ptr_accessors; +}; + +template <typename T> +struct lw_shared_ptr_no_esft : private lw_shared_ptr_counter_base { + T _value; + + lw_shared_ptr_no_esft() = default; + lw_shared_ptr_no_esft(const T& x) : _value(x) {} + lw_shared_ptr_no_esft(T&& x) : _value(std::move(x)) {} + template <typename... A> + lw_shared_ptr_no_esft(A&&... a) : _value(std::forward<A>(a)...) {} + + template <typename X> + friend class lw_shared_ptr; + template <typename X> + friend struct internal::lw_shared_ptr_accessors_no_esft; + template <typename X, class Y> + friend struct internal::lw_shared_ptr_accessors; +}; + + +/// Extension point: the user may override this to change how \ref lw_shared_ptr objects are destroyed, +/// primarily so that incomplete classes can be used. +/// +/// Customizing the deleter requires that \c T be derived from \c enable_lw_shared_from_this<T>. +/// The specialization must be visible for all uses of \c lw_shared_ptr<T>. +/// +/// To customize, the template must have a `static void dispose(T*)` operator that disposes of +/// the object. +template <typename T> +struct lw_shared_ptr_deleter; // No generic implementation + +namespace internal { + +template <typename T> +struct lw_shared_ptr_accessors_esft { + using concrete_type = std::remove_const_t<T>; + static T* to_value(lw_shared_ptr_counter_base* counter) { + return static_cast<T*>(counter); + } + static void dispose(lw_shared_ptr_counter_base* counter) { + dispose(static_cast<T*>(counter)); + } + static void dispose(T* value_ptr) { + delete value_ptr; + } + static void instantiate_to_value(lw_shared_ptr_counter_base*) { + // since to_value() is defined above, we don't need to do anything special + // to force-instantiate it + } +}; + +template <typename T> +struct lw_shared_ptr_accessors_no_esft { + using concrete_type = lw_shared_ptr_no_esft<T>; + static T* to_value(lw_shared_ptr_counter_base* counter) { + return &static_cast<concrete_type*>(counter)->_value; + } + static void dispose(lw_shared_ptr_counter_base* counter) { + delete static_cast<concrete_type*>(counter); + } + static void dispose(T* value_ptr) { + delete boost::intrusive::get_parent_from_member(value_ptr, &concrete_type::_value); + } + static void instantiate_to_value(lw_shared_ptr_counter_base*) { + // since to_value() is defined above, we don't need to do anything special + // to force-instantiate it + } +}; + +// Generic case: lw_shared_ptr_deleter<T> is not specialized, select +// implementation based on whether T inherits from enable_lw_shared_from_this<T>. +template <typename T, typename U = void> +struct lw_shared_ptr_accessors : std::conditional_t< + std::is_base_of<enable_lw_shared_from_this<T>, T>::value, + lw_shared_ptr_accessors_esft<T>, + lw_shared_ptr_accessors_no_esft<T>> { +}; + +// void_t is C++17, use this temporarily +template <typename... T> +using void_t = void; + +// Overload when lw_shared_ptr_deleter<T> specialized +template <typename T> +struct lw_shared_ptr_accessors<T, void_t<decltype(lw_shared_ptr_deleter<T>{})>> { + using concrete_type = T; + static T* to_value(lw_shared_ptr_counter_base* counter); + static void dispose(lw_shared_ptr_counter_base* counter) { + lw_shared_ptr_deleter<T>::dispose(to_value(counter)); + } + static void instantiate_to_value(lw_shared_ptr_counter_base* p) { + // instantiate to_value(); must be defined by shared_ptr_incomplete.hh + to_value(p); + } +}; + +} + +template <typename T> +class lw_shared_ptr { + template <typename U> + using accessors = internal::lw_shared_ptr_accessors<std::remove_const_t<U>>; + + mutable lw_shared_ptr_counter_base* _p = nullptr; +private: + lw_shared_ptr(lw_shared_ptr_counter_base* p) noexcept : _p(p) { + if (_p) { + ++_p->_count; + } + } + template <typename... A> + static lw_shared_ptr make(A&&... a) { + auto p = new typename accessors<T>::concrete_type(std::forward<A>(a)...); + accessors<T>::instantiate_to_value(p); + return lw_shared_ptr(p); + } +public: + using element_type = T; + + // Destroys the object pointed to by p and disposes of its storage. + // The pointer to the object must have been obtained through release(). + static void dispose(T* p) noexcept { + accessors<T>::dispose(const_cast<std::remove_const_t<T>*>(p)); + } + + // A functor which calls dispose(). + class disposer { + public: + void operator()(T* p) const noexcept { + dispose(p); + } + }; + + lw_shared_ptr() noexcept = default; + lw_shared_ptr(std::nullptr_t) noexcept : lw_shared_ptr() {} + lw_shared_ptr(const lw_shared_ptr& x) noexcept : _p(x._p) { + if (_p) { +#pragma GCC diagnostic push +#ifdef SEASTAR_IGNORE_USE_AFTER_FREE +#pragma GCC diagnostic ignored "-Wuse-after-free" +#endif + ++_p->_count; +#pragma GCC diagnostic pop + } + } + lw_shared_ptr(lw_shared_ptr&& x) noexcept : _p(x._p) { + x._p = nullptr; + } + [[gnu::always_inline]] + ~lw_shared_ptr() { +#pragma GCC diagnostic push +#ifdef SEASTAR_IGNORE_USE_AFTER_FREE +#pragma GCC diagnostic ignored "-Wuse-after-free" +#endif + if (_p && !--_p->_count) { + accessors<T>::dispose(_p); + } +#pragma GCC diagnostic pop + } + lw_shared_ptr& operator=(const lw_shared_ptr& x) noexcept { + if (_p != x._p) { + this->~lw_shared_ptr(); + new (this) lw_shared_ptr(x); + } + return *this; + } + lw_shared_ptr& operator=(lw_shared_ptr&& x) noexcept { + if (_p != x._p) { + this->~lw_shared_ptr(); + new (this) lw_shared_ptr(std::move(x)); + } + return *this; + } + lw_shared_ptr& operator=(std::nullptr_t) noexcept { + return *this = lw_shared_ptr(); + } + lw_shared_ptr& operator=(T&& x) noexcept { + this->~lw_shared_ptr(); + new (this) lw_shared_ptr(make_lw_shared<T>(std::move(x))); + return *this; + } + + T& operator*() const noexcept { return *accessors<T>::to_value(_p); } + T* operator->() const noexcept { return accessors<T>::to_value(_p); } + T* get() const noexcept { + if (_p) { + return accessors<T>::to_value(_p); + } else { + return nullptr; + } + } + + // Releases ownership of the object without destroying it. + // If this was the last owner then returns an engaged unique_ptr + // which is now the sole owner of the object. + // Returns a disengaged pointer if there are still some owners. + // + // Note that in case the raw pointer is extracted from the unique_ptr + // using unique_ptr::release(), it must be still destroyed using + // lw_shared_ptr::disposer or lw_shared_ptr::dispose(). + std::unique_ptr<T, disposer> release() noexcept { + auto p = std::exchange(_p, nullptr); + if (--p->_count) { + return nullptr; + } else { + return std::unique_ptr<T, disposer>(accessors<T>::to_value(p)); + } + } + + long int use_count() const noexcept { + if (_p) { + return _p->_count; + } else { + return 0; + } + } + + operator lw_shared_ptr<const T>() const noexcept { + return lw_shared_ptr<const T>(_p); + } + + explicit operator bool() const noexcept { + return _p; + } + + bool owned() const noexcept { + return _p->_count == 1; + } + + bool operator==(const lw_shared_ptr<const T>& x) const { + return _p == x._p; + } + + bool operator!=(const lw_shared_ptr<const T>& x) const { + return !operator==(x); + } + + bool operator==(const lw_shared_ptr<std::remove_const_t<T>>& x) const { + return _p == x._p; + } + + bool operator!=(const lw_shared_ptr<std::remove_const_t<T>>& x) const { + return !operator==(x); + } + + bool operator<(const lw_shared_ptr<const T>& x) const { + return _p < x._p; + } + + bool operator<(const lw_shared_ptr<std::remove_const_t<T>>& x) const { + return _p < x._p; + } + + template <typename U> + friend class lw_shared_ptr; + + template <typename X, typename... A> + friend lw_shared_ptr<X> make_lw_shared(A&&...); + + template <typename U> + friend lw_shared_ptr<U> make_lw_shared(U&&); + + template <typename U> + friend lw_shared_ptr<U> make_lw_shared(U&); + + template <typename U> + friend class enable_lw_shared_from_this; +}; + +template <typename T, typename... A> +inline +lw_shared_ptr<T> make_lw_shared(A&&... a) { + return lw_shared_ptr<T>::make(std::forward<A>(a)...); +} + +template <typename T> +inline +lw_shared_ptr<T> make_lw_shared(T&& a) { + return lw_shared_ptr<T>::make(std::move(a)); +} + +template <typename T> +inline +lw_shared_ptr<T> make_lw_shared(T& a) { + return lw_shared_ptr<T>::make(a); +} + +template <typename T> +inline +lw_shared_ptr<T> +enable_lw_shared_from_this<T>::shared_from_this() noexcept { + return lw_shared_ptr<T>(this); +} + +template <typename T> +inline +lw_shared_ptr<const T> +enable_lw_shared_from_this<T>::shared_from_this() const noexcept { + return lw_shared_ptr<const T>(const_cast<enable_lw_shared_from_this*>(this)); +} + +template <typename T> +static inline +std::ostream& operator<<(std::ostream& out, const lw_shared_ptr<T>& p) { + if (!p) { + return out << "null"; + } + return out << *p; +} + +// Polymorphic shared pointer class + +struct shared_ptr_count_base { + // destructor is responsible for fully-typed deletion + virtual ~shared_ptr_count_base() {} + shared_ptr_counter_type count = 0; +}; + +template <typename T> +struct shared_ptr_count_for : shared_ptr_count_base { + T data; + template <typename... A> + shared_ptr_count_for(A&&... a) : data(std::forward<A>(a)...) {} +}; + +template <typename T> +class enable_shared_from_this : private shared_ptr_count_base { +public: + shared_ptr<T> shared_from_this() noexcept; + shared_ptr<const T> shared_from_this() const noexcept; + long use_count() const noexcept { return count; } + + template <typename U> + friend class shared_ptr; + + template <typename U, bool esft> + friend struct shared_ptr_make_helper; +}; + +template <typename T> +class shared_ptr { + mutable shared_ptr_count_base* _b = nullptr; + mutable T* _p = nullptr; +private: + explicit shared_ptr(shared_ptr_count_for<T>* b) noexcept : _b(b), _p(&b->data) { + ++_b->count; + } + shared_ptr(shared_ptr_count_base* b, T* p) noexcept : _b(b), _p(p) { + if (_b) { + ++_b->count; + } + } + explicit shared_ptr(enable_shared_from_this<std::remove_const_t<T>>* p) noexcept : _b(p), _p(static_cast<T*>(p)) { + if (_b) { + ++_b->count; + } + } +public: + using element_type = T; + + shared_ptr() noexcept = default; + shared_ptr(std::nullptr_t) noexcept : shared_ptr() {} + shared_ptr(const shared_ptr& x) noexcept + : _b(x._b) + , _p(x._p) { + if (_b) { + ++_b->count; + } + } + shared_ptr(shared_ptr&& x) noexcept + : _b(x._b) + , _p(x._p) { + x._b = nullptr; + x._p = nullptr; + } + template <typename U, typename = std::enable_if_t<std::is_base_of<T, U>::value>> + shared_ptr(const shared_ptr<U>& x) noexcept + : _b(x._b) + , _p(x._p) { + if (_b) { + ++_b->count; + } + } + template <typename U, typename = std::enable_if_t<std::is_base_of<T, U>::value>> + shared_ptr(shared_ptr<U>&& x) noexcept + : _b(x._b) + , _p(x._p) { + x._b = nullptr; + x._p = nullptr; + } + ~shared_ptr() { +#pragma GCC diagnostic push +#ifdef SEASTAR_IGNORE_USE_AFTER_FREE +#pragma GCC diagnostic ignored "-Wuse-after-free" +#endif + if (_b && !--_b->count) { + delete _b; + } +#pragma GCC diagnostic pop + } + shared_ptr& operator=(const shared_ptr& x) noexcept { + if (this != &x) { + this->~shared_ptr(); + new (this) shared_ptr(x); + } + return *this; + } + shared_ptr& operator=(shared_ptr&& x) noexcept { + if (this != &x) { + this->~shared_ptr(); + new (this) shared_ptr(std::move(x)); + } + return *this; + } + shared_ptr& operator=(std::nullptr_t) noexcept { + return *this = shared_ptr(); + } + template <typename U, typename = std::enable_if_t<std::is_base_of<T, U>::value>> + shared_ptr& operator=(const shared_ptr<U>& x) noexcept { + if (*this != x) { + this->~shared_ptr(); + new (this) shared_ptr(x); + } + return *this; + } + template <typename U, typename = std::enable_if_t<std::is_base_of<T, U>::value>> + shared_ptr& operator=(shared_ptr<U>&& x) noexcept { + if (*this != x) { + this->~shared_ptr(); + new (this) shared_ptr(std::move(x)); + } + return *this; + } + explicit operator bool() const noexcept { + return _p; + } + T& operator*() const noexcept { + return *_p; + } + T* operator->() const noexcept { + return _p; + } + T* get() const noexcept { + return _p; + } + long use_count() const noexcept { + if (_b) { + return _b->count; + } else { + return 0; + } + } + + template <bool esft> + struct make_helper; + + template <typename U, typename... A> + friend shared_ptr<U> make_shared(A&&... a); + + template <typename U> + friend shared_ptr<U> make_shared(U&& a); + + template <typename V, typename U> + friend shared_ptr<V> static_pointer_cast(const shared_ptr<U>& p); + + template <typename V, typename U> + friend shared_ptr<V> dynamic_pointer_cast(const shared_ptr<U>& p); + + template <typename V, typename U> + friend shared_ptr<V> const_pointer_cast(const shared_ptr<U>& p); + + template <bool esft, typename... A> + static shared_ptr make(A&&... a); + + template <typename U> + friend class enable_shared_from_this; + + template <typename U, bool esft> + friend struct shared_ptr_make_helper; + + template <typename U> + friend class shared_ptr; +}; + +template <typename U, bool esft> +struct shared_ptr_make_helper; + +template <typename T> +struct shared_ptr_make_helper<T, false> { + template <typename... A> + static shared_ptr<T> make(A&&... a) { + return shared_ptr<T>(new shared_ptr_count_for<T>(std::forward<A>(a)...)); + } +}; + +template <typename T> +struct shared_ptr_make_helper<T, true> { + template <typename... A> + static shared_ptr<T> make(A&&... a) { + auto p = new T(std::forward<A>(a)...); + return shared_ptr<T>(p, p); + } +}; + +template <typename T, typename... A> +inline +shared_ptr<T> +make_shared(A&&... a) { + using helper = shared_ptr_make_helper<T, std::is_base_of<shared_ptr_count_base, T>::value>; + return helper::make(std::forward<A>(a)...); +} + +template <typename T> +inline +shared_ptr<T> +make_shared(T&& a) { + using helper = shared_ptr_make_helper<T, std::is_base_of<shared_ptr_count_base, T>::value>; + return helper::make(std::forward<T>(a)); +} + +template <typename T, typename U> +inline +shared_ptr<T> +static_pointer_cast(const shared_ptr<U>& p) { + return shared_ptr<T>(p._b, static_cast<T*>(p._p)); +} + +template <typename T, typename U> +inline +shared_ptr<T> +dynamic_pointer_cast(const shared_ptr<U>& p) { + auto q = dynamic_cast<T*>(p._p); + return shared_ptr<T>(q ? p._b : nullptr, q); +} + +template <typename T, typename U> +inline +shared_ptr<T> +const_pointer_cast(const shared_ptr<U>& p) { + return shared_ptr<T>(p._b, const_cast<T*>(p._p)); +} + +template <typename T> +inline +shared_ptr<T> +enable_shared_from_this<T>::shared_from_this() noexcept { + auto unconst = reinterpret_cast<enable_shared_from_this<std::remove_const_t<T>>*>(this); + return shared_ptr<T>(unconst); +} + +template <typename T> +inline +shared_ptr<const T> +enable_shared_from_this<T>::shared_from_this() const noexcept { + auto esft = const_cast<enable_shared_from_this*>(this); + auto unconst = reinterpret_cast<enable_shared_from_this<std::remove_const_t<T>>*>(esft); + return shared_ptr<const T>(unconst); +} + +template <typename T, typename U> +inline +bool +operator==(const shared_ptr<T>& x, const shared_ptr<U>& y) { + return x.get() == y.get(); +} + +template <typename T> +inline +bool +operator==(const shared_ptr<T>& x, std::nullptr_t) { + return x.get() == nullptr; +} + +template <typename T> +inline +bool +operator==(std::nullptr_t, const shared_ptr<T>& y) { + return nullptr == y.get(); +} + +template <typename T> +inline +bool +operator==(const lw_shared_ptr<T>& x, std::nullptr_t) { + return x.get() == nullptr; +} + +template <typename T> +inline +bool +operator==(std::nullptr_t, const lw_shared_ptr<T>& y) { + return nullptr == y.get(); +} + +template <typename T, typename U> +inline +bool +operator!=(const shared_ptr<T>& x, const shared_ptr<U>& y) { + return x.get() != y.get(); +} + +template <typename T> +inline +bool +operator!=(const shared_ptr<T>& x, std::nullptr_t) { + return x.get() != nullptr; +} + +template <typename T> +inline +bool +operator!=(std::nullptr_t, const shared_ptr<T>& y) { + return nullptr != y.get(); +} + +template <typename T> +inline +bool +operator!=(const lw_shared_ptr<T>& x, std::nullptr_t) { + return x.get() != nullptr; +} + +template <typename T> +inline +bool +operator!=(std::nullptr_t, const lw_shared_ptr<T>& y) { + return nullptr != y.get(); +} + +template <typename T, typename U> +inline +bool +operator<(const shared_ptr<T>& x, const shared_ptr<U>& y) { + return x.get() < y.get(); +} + +template <typename T> +inline +bool +operator<(const shared_ptr<T>& x, std::nullptr_t) { + return x.get() < nullptr; +} + +template <typename T> +inline +bool +operator<(std::nullptr_t, const shared_ptr<T>& y) { + return nullptr < y.get(); +} + +template <typename T, typename U> +inline +bool +operator<=(const shared_ptr<T>& x, const shared_ptr<U>& y) { + return x.get() <= y.get(); +} + +template <typename T> +inline +bool +operator<=(const shared_ptr<T>& x, std::nullptr_t) { + return x.get() <= nullptr; +} + +template <typename T> +inline +bool +operator<=(std::nullptr_t, const shared_ptr<T>& y) { + return nullptr <= y.get(); +} + +template <typename T, typename U> +inline +bool +operator>(const shared_ptr<T>& x, const shared_ptr<U>& y) { + return x.get() > y.get(); +} + +template <typename T> +inline +bool +operator>(const shared_ptr<T>& x, std::nullptr_t) { + return x.get() > nullptr; +} + +template <typename T> +inline +bool +operator>(std::nullptr_t, const shared_ptr<T>& y) { + return nullptr > y.get(); +} + +template <typename T, typename U> +inline +bool +operator>=(const shared_ptr<T>& x, const shared_ptr<U>& y) { + return x.get() >= y.get(); +} + +template <typename T> +inline +bool +operator>=(const shared_ptr<T>& x, std::nullptr_t) { + return x.get() >= nullptr; +} + +template <typename T> +inline +bool +operator>=(std::nullptr_t, const shared_ptr<T>& y) { + return nullptr >= y.get(); +} + +template <typename T> +static inline +std::ostream& operator<<(std::ostream& out, const shared_ptr<T>& p) { + if (!p) { + return out << "null"; + } + return out << *p; +} + +template<typename T> +using shared_ptr_equal_by_value = indirect_equal_to<shared_ptr<T>>; + +template<typename T> +using shared_ptr_value_hash = indirect_hash<shared_ptr<T>>; + +} + +namespace std { + +template <typename T> +struct hash<seastar::lw_shared_ptr<T>> : private hash<T*> { + size_t operator()(const seastar::lw_shared_ptr<T>& p) const { + return hash<T*>::operator()(p.get()); + } +}; + +template <typename T> +struct hash<seastar::shared_ptr<T>> : private hash<T*> { + size_t operator()(const seastar::shared_ptr<T>& p) const { + return hash<T*>::operator()(p.get()); + } +}; + +} + +namespace fmt { + +template<typename T> +const void* ptr(const seastar::lw_shared_ptr<T>& p) { + return p.get(); +} + +template<typename T> +const void* ptr(const seastar::shared_ptr<T>& p) { + return p.get(); +} + +} + +namespace seastar { + +template<typename T> +struct is_smart_ptr<shared_ptr<T>> : std::true_type {}; + +template<typename T> +struct is_smart_ptr<lw_shared_ptr<T>> : std::true_type {}; + +} diff --git a/src/seastar/include/seastar/core/shared_ptr_debug_helper.hh b/src/seastar/include/seastar/core/shared_ptr_debug_helper.hh new file mode 100644 index 000000000..592f8125e --- /dev/null +++ b/src/seastar/include/seastar/core/shared_ptr_debug_helper.hh @@ -0,0 +1,76 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ + +#pragma once + +#ifdef SEASTAR_DEBUG_SHARED_PTR + +#include <thread> +#include <cassert> + +#include <seastar/core/on_internal_error.hh> + +namespace seastar { + +extern logger seastar_logger; + +// A counter that is only comfortable being incremented on the cpu +// it was created on. Useful for verifying that a shared_ptr +// or lw_shared_ptr isn't misued across cores. +class debug_shared_ptr_counter_type { + long _counter = 0; + std::thread::id _cpu = std::this_thread::get_id(); +public: + debug_shared_ptr_counter_type(long x) noexcept : _counter(x) {} + operator long() const { + check(); + return _counter; + } + debug_shared_ptr_counter_type& operator++() { + check(); + ++_counter; + return *this; + } + long operator++(int) { + check(); + return _counter++; + } + debug_shared_ptr_counter_type& operator--() { + check(); + --_counter; + return *this; + } + long operator--(int) { + check(); + return _counter--; + } +private: + void check() const { + if (__builtin_expect(_cpu != std::this_thread::get_id(), false)) { + on_fatal_internal_error(seastar_logger, "shared_ptr accessed on non-owner cpu"); + } + } +}; + +} + +#endif + diff --git a/src/seastar/include/seastar/core/shared_ptr_incomplete.hh b/src/seastar/include/seastar/core/shared_ptr_incomplete.hh new file mode 100644 index 000000000..84f2b44a7 --- /dev/null +++ b/src/seastar/include/seastar/core/shared_ptr_incomplete.hh @@ -0,0 +1,44 @@ +/* + * 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) 2017 ScyllaDB + */ + +#pragma once + +#include <seastar/core/shared_ptr.hh> + + +/// \file +/// \brief Include this header files when using \c lw_shared_ptr<some_incomplete_type>, at the point +/// where \c some_incomplete_type is defined. + +namespace seastar { + +namespace internal { + +// Overload when lw_shared_ptr_deleter<T> specialized +template <typename T> +T* +lw_shared_ptr_accessors<T, void_t<decltype(lw_shared_ptr_deleter<T>{})>>::to_value(lw_shared_ptr_counter_base* counter) { + return static_cast<T*>(counter); +} + +} + +} diff --git a/src/seastar/include/seastar/core/simple-stream.hh b/src/seastar/include/seastar/core/simple-stream.hh new file mode 100644 index 000000000..6c1072865 --- /dev/null +++ b/src/seastar/include/seastar/core/simple-stream.hh @@ -0,0 +1,639 @@ +/* + * 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) 2016 Scylladb, Ltd. + */ + +#pragma once +#include <seastar/core/sstring.hh> +#include <seastar/util/variant_utils.hh> + +namespace seastar { + +class measuring_output_stream { + size_t _size = 0; +public: + void write(const char*, size_t size) { + _size += size; + } + + size_t size() const { + return _size; + } +}; + +template<typename> +class memory_output_stream; + +class simple_memory_input_stream; + +template<typename Iterator> +class fragmented_memory_input_stream; + +template<typename Iterator> +class memory_input_stream; + +class simple_memory_output_stream { + char* _p = nullptr; + size_t _size = 0; +public: + using has_with_stream = std::false_type; + simple_memory_output_stream() {} + simple_memory_output_stream(char* p, size_t size, size_t start = 0) : _p(p + start), _size(size) {} + char* begin() { return _p; } + + [[gnu::always_inline]] + void skip(size_t size) { + if (size > _size) { + throw std::out_of_range("serialization buffer overflow"); + } + _p += size; + _size -= size; + } + + [[gnu::always_inline]] + simple_memory_output_stream write_substream(size_t size) { + if (size > _size) { + throw std::out_of_range("serialization buffer overflow"); + } + simple_memory_output_stream substream(_p, size); + skip(size); + return substream; + } + + [[gnu::always_inline]] + void write(const char* p, size_t size) { + if (size > _size) { + throw std::out_of_range("serialization buffer overflow"); + } + std::copy_n(p, size, _p); + skip(size); + } + + [[gnu::always_inline]] + void fill(char c, size_t size) { + if (size > _size) { + throw std::out_of_range("serialization buffer overflow"); + } + std::fill_n(_p, size, c); + skip(size); + } + + [[gnu::always_inline]] + size_t size() const { + return _size; + } + + // simple_memory_output_stream is a write cursor that keeps a mutable view of some + // underlying buffer and provides write interface. to_input_stream() converts it + // to a read cursor that points to the same part of the buffer but provides + // read interface. + simple_memory_input_stream to_input_stream() const; +}; + +template<typename Iterator> +class fragmented_memory_output_stream { + using simple = simple_memory_output_stream ; + + Iterator _it; + simple _current; + size_t _size = 0; + + friend class memory_input_stream<Iterator>; +private: + template<typename Func> + //requires requires(Func f, view bv) { { f(bv) } -> void; } + void for_each_fragment(size_t size, Func&& func) { + if (size > _size) { + throw std::out_of_range("serialization buffer overflow"); + } + _size -= size; + while (size) { + if (!_current.size()) { + _current = simple(reinterpret_cast<char*>((*_it).get_write()), (*_it).size()); + _it++; + } + auto this_size = std::min(_current.size(), size); + func(_current.write_substream(this_size)); + size -= this_size; + } + } + fragmented_memory_output_stream(Iterator it, simple_memory_output_stream bv, size_t size) + : _it(it), _current(bv), _size(size) { } +public: + using has_with_stream = std::false_type; + using iterator_type = Iterator; + + fragmented_memory_output_stream() = default; + + fragmented_memory_output_stream(Iterator it, size_t size) + : _it(it), _size(size) { + } + + void skip(size_t size) { + for_each_fragment(size, [] (auto) { }); + } + memory_output_stream<Iterator> write_substream(size_t size) { + if (size > _size) { + throw std::out_of_range("serialization buffer overflow"); + } + if (_current.size() >= size) { + _size -= size; + return _current.write_substream(size); + } + fragmented_memory_output_stream substream(_it, _current, size); + skip(size); + return substream; + } + void write(const char* p, size_t size) { + for_each_fragment(size, [&p] (auto bv) { + std::copy_n(p, bv.size(), bv.begin()); + p += bv.size(); + }); + } + void fill(char c, size_t size) { + for_each_fragment(size, [c] (simple fragment) { + std::fill_n(fragment.begin(), fragment.size(), c); + }); + } + size_t size() const { + return _size; + } + + // fragmented_memory_input_stream is a write cursor that keeps a mutable view of some + // underlying fragmented buffer and provides write interface. to_input_stream() converts + // it to a read cursor that points to the same part of the buffer but provides read interface. + fragmented_memory_input_stream<Iterator> to_input_stream() const; +}; + +template<typename Iterator> +class memory_output_stream { +public: + using simple = simple_memory_output_stream; + using fragmented = fragmented_memory_output_stream<Iterator>; + +private: + const bool _is_simple; + using fragmented_type = fragmented; + union { + simple _simple; + fragmented_type _fragmented; + }; +public: + template<typename StreamVisitor> + [[gnu::always_inline]] + decltype(auto) with_stream(StreamVisitor&& visitor) { + if (__builtin_expect(_is_simple, true)) { + return visitor(_simple); + } + return visitor(_fragmented); + } + + template<typename StreamVisitor> + [[gnu::always_inline]] + decltype(auto) with_stream(StreamVisitor&& visitor) const { + if (__builtin_expect(_is_simple, true)) { + return visitor(_simple); + } + return visitor(_fragmented); + } +public: + using has_with_stream = std::true_type; + using iterator_type = Iterator; + memory_output_stream() + : _is_simple(true), _simple() {} + memory_output_stream(simple stream) + : _is_simple(true), _simple(std::move(stream)) {} + memory_output_stream(fragmented stream) + : _is_simple(false), _fragmented(std::move(stream)) {} + + [[gnu::always_inline]] + memory_output_stream(const memory_output_stream& other) noexcept : _is_simple(other._is_simple) { + // Making this copy constructor noexcept makes copy assignment simpler. + // Besides, performance of memory_output_stream relies on the fact that both + // fragmented and simple input stream are PODs and the branch below + // is optimized away, so throwable copy constructors aren't something + // we want. + static_assert(std::is_nothrow_copy_constructible<fragmented>::value, + "seastar::memory_output_stream::fragmented should be copy constructible"); + static_assert(std::is_nothrow_copy_constructible<simple>::value, + "seastar::memory_output_stream::simple should be copy constructible"); + if (_is_simple) { + new (&_simple) simple(other._simple); + } else { + new (&_fragmented) fragmented_type(other._fragmented); + } + } + + [[gnu::always_inline]] + memory_output_stream(memory_output_stream&& other) noexcept : _is_simple(other._is_simple) { + if (_is_simple) { + new (&_simple) simple(std::move(other._simple)); + } else { + new (&_fragmented) fragmented_type(std::move(other._fragmented)); + } + } + + [[gnu::always_inline]] + memory_output_stream& operator=(const memory_output_stream& other) noexcept { + // Copy constructor being noexcept makes copy assignment simpler. + static_assert(std::is_nothrow_copy_constructible<memory_output_stream>::value, + "memory_output_stream copy constructor shouldn't throw"); + if (this != &other) { + this->~memory_output_stream(); + new (this) memory_output_stream(other); + } + return *this; + } + + [[gnu::always_inline]] + memory_output_stream& operator=(memory_output_stream&& other) noexcept { + if (this != &other) { + this->~memory_output_stream(); + new (this) memory_output_stream(std::move(other)); + } + return *this; + } + + [[gnu::always_inline]] + ~memory_output_stream() { + if (_is_simple) { + _simple.~simple(); + } else { + _fragmented.~fragmented_type(); + } + } + + [[gnu::always_inline]] + void skip(size_t size) { + with_stream([size] (auto& stream) { + stream.skip(size); + }); + } + + [[gnu::always_inline]] + memory_output_stream write_substream(size_t size) { + return with_stream([size] (auto& stream) -> memory_output_stream { + return stream.write_substream(size); + }); + } + + [[gnu::always_inline]] + void write(const char* p, size_t size) { + with_stream([p, size] (auto& stream) { + stream.write(p, size); + }); + } + + [[gnu::always_inline]] + void fill(char c, size_t size) { + with_stream([c, size] (auto& stream) { + stream.fill(c, size); + }); + } + + [[gnu::always_inline]] + size_t size() const { + return with_stream([] (auto& stream) { + return stream.size(); + }); + } + + memory_input_stream<Iterator> to_input_stream() const; +}; + +class simple_memory_input_stream { + using simple = simple_memory_input_stream; + + const char* _p = nullptr; + size_t _size = 0; +public: + using has_with_stream = std::false_type; + simple_memory_input_stream() = default; + simple_memory_input_stream(const char* p, size_t size) : _p(p), _size(size) {} + + const char* begin() const { return _p; } + + [[gnu::always_inline]] + void skip(size_t size) { + if (size > _size) { + throw std::out_of_range("deserialization buffer underflow"); + } + _p += size; + _size -= size; + } + + [[gnu::always_inline]] + simple read_substream(size_t size) { + if (size > _size) { + throw std::out_of_range("deserialization buffer underflow"); + } + simple substream(_p, size); + skip(size); + return substream; + } + + [[gnu::always_inline]] + void read(char* p, size_t size) { + if (size > _size) { + throw std::out_of_range("deserialization buffer underflow"); + } + std::copy_n(_p, size, p); + skip(size); + } + + template<typename Output> + [[gnu::always_inline]] + void copy_to(Output& out) const { + out.write(_p, _size); + } + + [[gnu::always_inline]] + size_t size() const { + return _size; + } +}; + +template<typename Iterator> +class fragmented_memory_input_stream { + using simple = simple_memory_input_stream; + using fragmented = fragmented_memory_input_stream; + + Iterator _it; + simple _current; + size_t _size; +private: + template<typename Func> + //requires requires(Func f, view bv) { { f(bv) } -> void; } + void for_each_fragment(size_t size, Func&& func) { + if (size > _size) { + throw std::out_of_range("deserialization buffer underflow"); + } + _size -= size; + while (size) { + if (!_current.size()) { + _current = simple(reinterpret_cast<const char*>((*_it).begin()), (*_it).size()); + _it++; + } + auto this_size = std::min(_current.size(), size); + func(_current.read_substream(this_size)); + size -= this_size; + } + } + fragmented_memory_input_stream(Iterator it, simple bv, size_t size) + : _it(it), _current(bv), _size(size) { } + friend class fragmented_memory_output_stream<Iterator>; +public: + using has_with_stream = std::false_type; + using iterator_type = Iterator; + fragmented_memory_input_stream(Iterator it, size_t size) + : _it(it), _size(size) { + } + + void skip(size_t size) { + for_each_fragment(size, [] (auto) { }); + } + fragmented read_substream(size_t size) { + if (size > _size) { + throw std::out_of_range("deserialization buffer underflow"); + } + fragmented substream(_it, _current, size); + skip(size); + return substream; + } + void read(char* p, size_t size) { + for_each_fragment(size, [&p] (auto bv) { + p = std::copy_n(bv.begin(), bv.size(), p); + }); + } + template<typename Output> + void copy_to(Output& out) { + for_each_fragment(_size, [&out] (auto bv) { + bv.copy_to(out); + }); + } + size_t size() const { + return _size; + } + + const char* first_fragment_data() const { return _current.begin(); } + size_t first_fragment_size() const { return _current.size(); } + Iterator fragment_iterator() const { return _it; } +}; + +/* +template<typename Visitor> +concept bool StreamVisitor() { + return requires(Visitor visitor, simple& simple, fragmented& fragmented) { + visitor(simple); + visitor(fragmented); + }; +} +*/ +// memory_input_stream performs type erasure optimized for cases where +// simple is used. +// By using a lot of [[gnu::always_inline]] attributes this class attempts to +// make the compiler generate code with simple functions inlined +// directly in the user of the intput_stream. +template<typename Iterator> +class memory_input_stream { +public: + using simple = simple_memory_input_stream; + using fragmented = fragmented_memory_input_stream<Iterator>; +private: + const bool _is_simple; + using fragmented_type = fragmented; + union { + simple _simple; + fragmented_type _fragmented; + }; +public: + template<typename StreamVisitor> + [[gnu::always_inline]] + decltype(auto) with_stream(StreamVisitor&& visitor) { + if (__builtin_expect(_is_simple, true)) { + return visitor(_simple); + } + return visitor(_fragmented); + } + + template<typename StreamVisitor> + [[gnu::always_inline]] + decltype(auto) with_stream(StreamVisitor&& visitor) const { + if (__builtin_expect(_is_simple, true)) { + return visitor(_simple); + } + return visitor(_fragmented); + } +public: + using has_with_stream = std::true_type; + using iterator_type = Iterator; + memory_input_stream(simple stream) + : _is_simple(true), _simple(std::move(stream)) {} + memory_input_stream(fragmented stream) + : _is_simple(false), _fragmented(std::move(stream)) {} + + [[gnu::always_inline]] + memory_input_stream(const memory_input_stream& other) noexcept : _is_simple(other._is_simple) { + // Making this copy constructor noexcept makes copy assignment simpler. + // Besides, performance of memory_input_stream relies on the fact that both + // fragmented and simple input stream are PODs and the branch below + // is optimized away, so throwable copy constructors aren't something + // we want. + static_assert(std::is_nothrow_copy_constructible<fragmented>::value, + "seastar::memory_input_stream::fragmented should be copy constructible"); + static_assert(std::is_nothrow_copy_constructible<simple>::value, + "seastar::memory_input_stream::simple should be copy constructible"); + if (_is_simple) { + new (&_simple) simple(other._simple); + } else { + new (&_fragmented) fragmented_type(other._fragmented); + } + } + + [[gnu::always_inline]] + memory_input_stream(memory_input_stream&& other) noexcept : _is_simple(other._is_simple) { + if (_is_simple) { + new (&_simple) simple(std::move(other._simple)); + } else { + new (&_fragmented) fragmented_type(std::move(other._fragmented)); + } + } + + [[gnu::always_inline]] + memory_input_stream& operator=(const memory_input_stream& other) noexcept { + // Copy constructor being noexcept makes copy assignment simpler. + static_assert(std::is_nothrow_copy_constructible<memory_input_stream>::value, + "memory_input_stream copy constructor shouldn't throw"); + if (this != &other) { + this->~memory_input_stream(); + new (this) memory_input_stream(other); + } + return *this; + } + + [[gnu::always_inline]] + memory_input_stream& operator=(memory_input_stream&& other) noexcept { + if (this != &other) { + this->~memory_input_stream(); + new (this) memory_input_stream(std::move(other)); + } + return *this; + } + + [[gnu::always_inline]] + ~memory_input_stream() { + if (_is_simple) { + _simple.~simple_memory_input_stream(); + } else { + _fragmented.~fragmented_type(); + } + } + + [[gnu::always_inline]] + void skip(size_t size) { + with_stream([size] (auto& stream) { + stream.skip(size); + }); + } + + [[gnu::always_inline]] + memory_input_stream read_substream(size_t size) { + return with_stream([size] (auto& stream) -> memory_input_stream { + return stream.read_substream(size); + }); + } + + [[gnu::always_inline]] + void read(char* p, size_t size) { + with_stream([p, size] (auto& stream) { + stream.read(p, size); + }); + } + + template<typename Output> + [[gnu::always_inline]] + void copy_to(Output& out) { + with_stream([&out] (auto& stream) { + stream.copy_to(out); + }); + } + + [[gnu::always_inline]] + size_t size() const { + return with_stream([] (auto& stream) { + return stream.size(); + }); + } + + template<typename Stream, typename StreamVisitor> + friend decltype(auto) with_serialized_stream(Stream& stream, StreamVisitor&& visitor); +}; + +inline simple_memory_input_stream simple_memory_output_stream::to_input_stream() const { + return simple_memory_input_stream(_p, _size); +} + +template<typename Iterator> +inline fragmented_memory_input_stream<Iterator> fragmented_memory_output_stream<Iterator>::to_input_stream() const { + return fragmented_memory_input_stream<Iterator>(_it, _current.to_input_stream(), _size); +} + +template<typename Iterator> +inline memory_input_stream<Iterator> memory_output_stream<Iterator>::to_input_stream() const { + return with_stream(make_visitor( + [] (const simple_memory_output_stream& ostream) -> memory_input_stream<Iterator> { + return ostream.to_input_stream(); + }, + [] (const fragmented_memory_output_stream<Iterator>& ostream) -> memory_input_stream<Iterator> { + return ostream.to_input_stream(); + } + )); +} + +// The purpose of the with_serialized_stream() is to minimize number of dynamic +// dispatches. For example, a lot of IDL-generated code looks like this: +// auto some_value() const { +// return seastar::with_serialized_stream(v, [] (auto& v) { +// auto in = v; +// ser::skip(in, boost::type<type1>()); +// ser::skip(in, boost::type<type2>()); +// return deserialize(in, boost::type<type3>()); +// }); +// } +// Using with_stream() there is at most one dynamic dispatch per such +// function, instead of one per each skip() and deserialize() call. + +template<typename Stream, typename StreamVisitor, typename = std::enable_if_t<Stream::has_with_stream::value>> +[[gnu::always_inline]] + static inline decltype(auto) + with_serialized_stream(Stream& stream, StreamVisitor&& visitor) { + return stream.with_stream(std::forward<StreamVisitor>(visitor)); +} + +template<typename Stream, typename StreamVisitor, typename = std::enable_if_t<!Stream::has_with_stream::value>, typename = void> +[[gnu::always_inline]] + static inline decltype(auto) + with_serialized_stream(Stream& stream, StreamVisitor&& visitor) { + return visitor(stream); +} + +using simple_input_stream = simple_memory_input_stream; +using simple_output_stream = simple_memory_output_stream; + +} diff --git a/src/seastar/include/seastar/core/slab.hh b/src/seastar/include/seastar/core/slab.hh new file mode 100644 index 000000000..a1944d4d1 --- /dev/null +++ b/src/seastar/include/seastar/core/slab.hh @@ -0,0 +1,568 @@ +/* + * 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 2015 Cloudius Systems + */ +#pragma once + +#include <boost/intrusive/unordered_set.hpp> +#include <boost/intrusive/list.hpp> +#include <stdlib.h> +#include <stdio.h> +#include <stdint.h> +#include <assert.h> +#include <memory> +#include <vector> +#include <algorithm> +#include <seastar/core/metrics.hh> +#include <seastar/core/align.hh> +#include <seastar/core/memory.hh> + +namespace seastar { + +static constexpr uint16_t SLAB_MAGIC_NUMBER = 0x51AB; // meant to be 'SLAB' :-) + +/* + * Item requirements + * - Extend it to slab_item_base. + * - First parameter of constructor must be uint32_t _slab_page_index. + * - Implement get_slab_page_index() to return _slab_page_index. + * - Implement is_unlocked() to check if Item can be evicted. + */ + +/* + * slab_page_desc is 1:1 mapped to slab page. + * footprint: 80b for each slab page. + */ +struct slab_page_desc { +private: + boost::intrusive::list_member_hook<> _lru_link; + boost::intrusive::list_member_hook<> _free_pages_link; + void *_slab_page; + std::vector<uintptr_t> _free_objects; + uint32_t _refcnt; + uint32_t _index; // index into slab page vector + uint16_t _magic; + uint8_t _slab_class_id; +public: + slab_page_desc(void *slab_page, size_t objects, size_t object_size, uint8_t slab_class_id, uint32_t index) + : _slab_page(slab_page) + , _refcnt(0U) + , _index(index) + , _magic(SLAB_MAGIC_NUMBER) + , _slab_class_id(slab_class_id) + { + auto object = reinterpret_cast<uintptr_t>(slab_page); + _free_objects.reserve(objects - 1); + for (auto i = 1u; i < objects; i++) { + object += object_size; + _free_objects.push_back(object); + } + } + + bool empty() const { + return _free_objects.empty(); + } + + size_t size() const { + return _free_objects.size(); + } + + uint32_t& refcnt() { + return _refcnt; + } + + uint32_t index() const { + return _index; + } + + uint16_t magic() const { + return _magic; + } + + uint8_t slab_class_id() const { + return _slab_class_id; + } + + void* slab_page() const { + return _slab_page; + } + + std::vector<uintptr_t>& free_objects() { + return _free_objects; + } + + void* allocate_object() { + assert(!_free_objects.empty()); + auto object = reinterpret_cast<void*>(_free_objects.back()); + _free_objects.pop_back(); + return object; + } + + void free_object(void *object) { + _free_objects.push_back(reinterpret_cast<uintptr_t>(object)); + } + + template<typename Item> + friend class slab_class; + template<typename Item> + friend class slab_allocator; +}; + +class slab_item_base { + boost::intrusive::list_member_hook<> _lru_link; + + template<typename Item> + friend class slab_class; +}; + +template<typename Item> +class slab_class { +private: + boost::intrusive::list<slab_page_desc, + boost::intrusive::member_hook<slab_page_desc, boost::intrusive::list_member_hook<>, + &slab_page_desc::_free_pages_link>> _free_slab_pages; + boost::intrusive::list<slab_item_base, + boost::intrusive::member_hook<slab_item_base, boost::intrusive::list_member_hook<>, + &slab_item_base::_lru_link>> _lru; + size_t _size; // size of objects + uint8_t _slab_class_id; +private: + template<typename... Args> + inline + Item* create_item(void *object, uint32_t slab_page_index, Args&&... args) { + Item *new_item = new(object) Item(slab_page_index, std::forward<Args>(args)...); + _lru.push_front(reinterpret_cast<slab_item_base&>(*new_item)); + return new_item; + } + + inline + std::pair<void *, uint32_t> evict_lru_item(std::function<void (Item& item_ref)>& erase_func) { + if (_lru.empty()) { + return { nullptr, 0U }; + } + + Item& victim = reinterpret_cast<Item&>(_lru.back()); + uint32_t index = victim.get_slab_page_index(); + assert(victim.is_unlocked()); + _lru.erase(_lru.iterator_to(reinterpret_cast<slab_item_base&>(victim))); + // WARNING: You need to make sure that erase_func will not release victim back to slab. + erase_func(victim); + + return { reinterpret_cast<void*>(&victim), index }; + } +public: + slab_class(size_t size, uint8_t slab_class_id) + : _size(size) + , _slab_class_id(slab_class_id) + { + } + slab_class(slab_class&&) = default; + ~slab_class() { + _free_slab_pages.clear(); + _lru.clear(); + } + + size_t size() const { + return _size; + } + + bool empty() const { + return _free_slab_pages.empty(); + } + + bool has_no_slab_pages() const { + return _lru.empty(); + } + + template<typename... Args> + Item *create(Args&&... args) { + assert(!_free_slab_pages.empty()); + auto& desc = _free_slab_pages.back(); + auto object = desc.allocate_object(); + if (desc.empty()) { + // if empty, remove desc from the list of slab pages with free objects. + _free_slab_pages.erase(_free_slab_pages.iterator_to(desc)); + } + + return create_item(object, desc.index(), std::forward<Args>(args)...); + } + + template<typename... Args> + Item *create_from_new_page(uint64_t max_object_size, uint32_t slab_page_index, + std::function<void (slab_page_desc& desc)> insert_slab_page_desc, + Args&&... args) { + // allocate slab page. + constexpr size_t alignment = std::alignment_of<Item>::value; + void *slab_page = aligned_alloc(alignment, max_object_size); + if (!slab_page) { + throw std::bad_alloc{}; + } + // allocate descriptor to slab page. + slab_page_desc *desc = nullptr; + assert(_size % alignment == 0); + try { + auto objects = max_object_size / _size; + desc = new slab_page_desc(slab_page, objects, _size, _slab_class_id, slab_page_index); + } catch (const std::bad_alloc& e) { + ::free(slab_page); + throw std::bad_alloc{}; + } + + _free_slab_pages.push_front(*desc); + insert_slab_page_desc(*desc); + + // first object from the allocated slab page is returned. + return create_item(slab_page, slab_page_index, std::forward<Args>(args)...); + } + + template<typename... Args> + Item *create_from_lru(std::function<void (Item& item_ref)>& erase_func, Args&&... args) { + auto ret = evict_lru_item(erase_func); + if (!ret.first) { + throw std::bad_alloc{}; + } + return create_item(ret.first, ret.second, std::forward<Args>(args)...); + } + + void free_item(Item *item, slab_page_desc& desc) { + void *object = item; + _lru.erase(_lru.iterator_to(reinterpret_cast<slab_item_base&>(*item))); + desc.free_object(object); + if (desc.size() == 1) { + // push back desc into the list of slab pages with free objects. + _free_slab_pages.push_back(desc); + } + } + + void touch_item(Item *item) { + auto& item_ref = reinterpret_cast<slab_item_base&>(*item); + _lru.erase(_lru.iterator_to(item_ref)); + _lru.push_front(item_ref); + } + + void remove_item_from_lru(Item *item) { + auto& item_ref = reinterpret_cast<slab_item_base&>(*item); + _lru.erase(_lru.iterator_to(item_ref)); + } + + void insert_item_into_lru(Item *item) { + auto& item_ref = reinterpret_cast<slab_item_base&>(*item); + _lru.push_front(item_ref); + } + + void remove_desc_from_free_list(slab_page_desc& desc) { + assert(desc.slab_class_id() == _slab_class_id); + _free_slab_pages.erase(_free_slab_pages.iterator_to(desc)); + } +}; + +template<typename Item> +class slab_allocator { +private: + std::vector<size_t> _slab_class_sizes; + std::vector<slab_class<Item>> _slab_classes; + seastar::metrics::metric_groups _metrics; + // erase_func() is used to remove the item from the cache using slab. + std::function<void (Item& item_ref)> _erase_func; + std::vector<slab_page_desc*> _slab_pages_vector; + boost::intrusive::list<slab_page_desc, + boost::intrusive::member_hook<slab_page_desc, boost::intrusive::list_member_hook<>, + &slab_page_desc::_lru_link>> _slab_page_desc_lru; + uint64_t _max_object_size; + uint64_t _available_slab_pages; + struct collectd_stats { + uint64_t allocs; + uint64_t frees; + } _stats; + memory::reclaimer *_reclaimer = nullptr; + bool _reclaimed = false; +private: + memory::reclaiming_result evict_lru_slab_page() { + if (_slab_page_desc_lru.empty()) { + // NOTE: Nothing to evict. If this happens, it implies that all + // slab pages in the slab are being used at the same time. + // That being said, this event is very unlikely to happen. + return memory::reclaiming_result::reclaimed_nothing; + } + // get descriptor of the least-recently-used slab page and related info. + auto& desc = _slab_page_desc_lru.back(); + assert(desc.refcnt() == 0); + uint8_t slab_class_id = desc.slab_class_id(); + auto slab_class = get_slab_class(slab_class_id); + void *slab_page = desc.slab_page(); + + auto& free_objects = desc.free_objects(); + if (!desc.empty()) { + // if not empty, remove desc from the list of slab pages with free objects. + slab_class->remove_desc_from_free_list(desc); + // and sort the array of free objects for binary search later on. + std::sort(free_objects.begin(), free_objects.end()); + } + // remove desc from the list of slab page descriptors. + _slab_page_desc_lru.erase(_slab_page_desc_lru.iterator_to(desc)); + // remove desc from the slab page vector. + _slab_pages_vector[desc.index()] = nullptr; + + // Iterate through objects in the slab page and if the object is an allocated + // item, the item should be removed from LRU and then erased. + uintptr_t object = reinterpret_cast<uintptr_t>(slab_page); + auto object_size = slab_class->size(); + auto objects = _max_object_size / object_size; + for (auto i = 0u; i < objects; i++, object += object_size) { + if (!desc.empty()) { + // if binary_search returns true, it means that object at the current + // offset isn't an item. + if (std::binary_search(free_objects.begin(), free_objects.end(), object)) { + continue; + } + } + Item* item = reinterpret_cast<Item*>(object); + assert(item->is_unlocked()); + slab_class->remove_item_from_lru(item); + _erase_func(*item); + _stats.frees++; + } +#ifdef SEASTAR_DEBUG + printf("lru slab page eviction succeeded! desc_empty?=%d\n", desc.empty()); +#endif + ::free(slab_page); // free slab page object + delete &desc; // free its descriptor + return memory::reclaiming_result::reclaimed_something; + } + + /* + * Reclaim the least recently used slab page that is unused. + */ + memory::reclaiming_result reclaim() { + // once reclaimer was called, slab pages should no longer be allocated, as the + // memory used by slab is supposed to be calibrated. + _reclaimed = true; + // FIXME: Should reclaim() only evict a single slab page at a time? + return evict_lru_slab_page(); + } + + void initialize_slab_allocator(double growth_factor, uint64_t limit) { + constexpr size_t alignment = std::alignment_of<Item>::value; + constexpr size_t initial_size = 96; + size_t size = initial_size; // initial object size + uint8_t slab_class_id = 0U; + + while (_max_object_size / size > 1) { + size = align_up(size, alignment); + _slab_class_sizes.push_back(size); + _slab_classes.emplace_back(size, slab_class_id); + size *= growth_factor; + assert(slab_class_id < std::numeric_limits<uint8_t>::max()); + slab_class_id++; + } + _slab_class_sizes.push_back(_max_object_size); + _slab_classes.emplace_back(_max_object_size, slab_class_id); + + // If slab limit is zero, enable reclaimer. + if (!limit) { + _reclaimer = new memory::reclaimer([this] { return reclaim(); }); + } else { + _slab_pages_vector.reserve(_available_slab_pages); + } + } + + slab_class<Item>* get_slab_class(const size_t size) { + // given a size, find slab class with binary search. + auto i = std::lower_bound(_slab_class_sizes.begin(), _slab_class_sizes.end(), size); + if (i == _slab_class_sizes.end()) { + return nullptr; + } + auto dist = std::distance(_slab_class_sizes.begin(), i); + return &_slab_classes[dist]; + } + + slab_class<Item>* get_slab_class(const uint8_t slab_class_id) { + assert(slab_class_id >= 0 && slab_class_id < _slab_classes.size()); + return &_slab_classes[slab_class_id]; + } + + void register_metrics() { + namespace sm = seastar::metrics; + _metrics.add_group("slab", { + sm::make_counter("malloc_total_operations", sm::description("Total number of slab malloc operations"), _stats.allocs), + sm::make_counter("free_total_operations", sm::description("Total number of slab free operations"), _stats.frees), + sm::make_gauge("malloc_objects", sm::description("Number of slab created objects currently in memory"), [this] { + return _stats.allocs - _stats.frees; + }) + }); + } + + inline slab_page_desc& get_slab_page_desc(Item *item) + { + auto desc = _slab_pages_vector[item->get_slab_page_index()]; + assert(desc != nullptr); + assert(desc->magic() == SLAB_MAGIC_NUMBER); + return *desc; + } + + inline bool can_allocate_page(slab_class<Item>& sc) { + return (_reclaimer && !_reclaimed) || + (_available_slab_pages > 0 || sc.has_no_slab_pages()); + } +public: + slab_allocator(double growth_factor, uint64_t limit, uint64_t max_object_size) + : _max_object_size(max_object_size) + , _available_slab_pages(limit / max_object_size) + { + initialize_slab_allocator(growth_factor, limit); + register_metrics(); + } + + slab_allocator(double growth_factor, uint64_t limit, uint64_t max_object_size, + std::function<void (Item& item_ref)> erase_func) + : _erase_func(std::move(erase_func)) + , _max_object_size(max_object_size) + , _available_slab_pages(limit / max_object_size) + { + initialize_slab_allocator(growth_factor, limit); + register_metrics(); + } + + ~slab_allocator() + { + _slab_classes.clear(); + _slab_page_desc_lru.clear(); + for (auto desc : _slab_pages_vector) { + if (!desc) { + continue; + } + ::free(desc->slab_page()); + delete desc; + } + delete _reclaimer; + } + + /** + * Create an item from a given slab class based on requested size. + */ + template<typename... Args> + Item* create(const size_t size, Args&&... args) { + auto slab_class = get_slab_class(size); + if (!slab_class) { + throw std::bad_alloc{}; + } + + Item *item = nullptr; + if (!slab_class->empty()) { + item = slab_class->create(std::forward<Args>(args)...); + _stats.allocs++; + } else { + if (can_allocate_page(*slab_class)) { + auto index_to_insert = _slab_pages_vector.size(); + item = slab_class->create_from_new_page(_max_object_size, index_to_insert, + [this](slab_page_desc& desc) { + if (_reclaimer) { + // insert desc into the LRU list of slab page descriptors. + _slab_page_desc_lru.push_front(desc); + } + // insert desc into the slab page vector. + _slab_pages_vector.push_back(&desc); + }, + std::forward<Args>(args)...); + if (_available_slab_pages > 0) { + _available_slab_pages--; + } + _stats.allocs++; + } else if (_erase_func) { + item = slab_class->create_from_lru(_erase_func, std::forward<Args>(args)...); + } + } + return item; + } + + void lock_item(Item *item) { + auto& desc = get_slab_page_desc(item); + if (_reclaimer) { + auto& refcnt = desc.refcnt(); + + if (++refcnt == 1) { + // remove slab page descriptor from list of slab page descriptors. + _slab_page_desc_lru.erase(_slab_page_desc_lru.iterator_to(desc)); + } + } + // remove item from the lru of its slab class. + auto slab_class = get_slab_class(desc.slab_class_id()); + slab_class->remove_item_from_lru(item); + } + + void unlock_item(Item *item) { + auto& desc = get_slab_page_desc(item); + if (_reclaimer) { + auto& refcnt = desc.refcnt(); + + if (--refcnt == 0) { + // insert slab page descriptor back into list of slab page descriptors. + _slab_page_desc_lru.push_front(desc); + } + } + // insert item into the lru of its slab class. + auto slab_class = get_slab_class(desc.slab_class_id()); + slab_class->insert_item_into_lru(item); + } + + /** + * Free an item back to its original slab class. + */ + void free(Item *item) { + if (item) { + auto& desc = get_slab_page_desc(item); + auto slab_class = get_slab_class(desc.slab_class_id()); + slab_class->free_item(item, desc); + _stats.frees++; + } + } + + /** + * Update item position in the LRU of its slab class. + */ + void touch(Item *item) { + if (item) { + auto& desc = get_slab_page_desc(item); + auto slab_class = get_slab_class(desc.slab_class_id()); + slab_class->touch_item(item); + } + } + + /** + * Helper function: Print all available slab classes and their respective properties. + */ + void print_slab_classes() { + auto class_id = 0; + for (auto& slab_class : _slab_classes) { + size_t size = slab_class.size(); + printf("slab[%3d]\tsize: %10lu\tper-slab-page: %5lu\n", class_id, size, _max_object_size / size); + class_id++; + } + } + + /** + * Helper function: Useful for getting a slab class' chunk size from a size parameter. + */ + size_t class_size(const size_t size) { + auto slab_class = get_slab_class(size); + return (slab_class) ? slab_class->size() : 0; + } +}; + +} diff --git a/src/seastar/include/seastar/core/sleep.hh b/src/seastar/include/seastar/core/sleep.hh new file mode 100644 index 000000000..88bc185dc --- /dev/null +++ b/src/seastar/include/seastar/core/sleep.hh @@ -0,0 +1,93 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <chrono> +#include <functional> + +#include <seastar/core/abort_source.hh> +#include <seastar/core/future.hh> +#include <seastar/core/lowres_clock.hh> +#include <seastar/core/timer.hh> + +namespace seastar { + +/// \file + +/// Returns a future which completes after a specified time has elapsed. +/// +/// \param dur minimum amount of time before the returned future becomes +/// ready. +/// \return A \ref future which becomes ready when the sleep duration elapses. +template <typename Clock = steady_clock_type, typename Rep, typename Period> +future<> sleep(std::chrono::duration<Rep, Period> dur) { + struct sleeper { + promise<> done; + timer<Clock> tmr; + sleeper(std::chrono::duration<Rep, Period> dur) + : tmr([this] { done.set_value(); }) + { + tmr.arm(dur); + } + }; + sleeper *s = new sleeper(dur); + future<> fut = s->done.get_future(); + return fut.then([s] { delete s; }); +} + +/// exception that is thrown when application is in process of been stopped +class sleep_aborted : public abort_requested_exception { +public: + /// Reports the exception reason. + virtual const char* what() const noexcept { + return "Sleep is aborted"; + } +}; + +/// Returns a future which completes after a specified time has elapsed +/// or throws \ref sleep_aborted exception if application is aborted +/// +/// \param dur minimum amount of time before the returned future becomes +/// ready. +/// \return A \ref future which becomes ready when the sleep duration elapses. +template <typename Clock = steady_clock_type> +future<> sleep_abortable(typename Clock::duration dur); + +extern template future<> sleep_abortable<steady_clock_type>(typename steady_clock_type::duration); +extern template future<> sleep_abortable<lowres_clock>(typename lowres_clock::duration); + +/// Returns a future which completes after a specified time has elapsed +/// or throws \ref sleep_aborted exception if the sleep is aborted. +/// +/// \param dur minimum amount of time before the returned future becomes +/// ready. +/// \param as the \ref abort_source that eventually notifies that the sleep +/// should be aborted. +/// \return A \ref future which becomes ready when the sleep duration elapses. +template <typename Clock = steady_clock_type> +future<> sleep_abortable(typename Clock::duration dur, abort_source& as); + +extern template future<> sleep_abortable<steady_clock_type>(typename steady_clock_type::duration, abort_source&); +extern template future<> sleep_abortable<lowres_clock>(typename lowres_clock::duration, abort_source&); + +} diff --git a/src/seastar/include/seastar/core/smp.hh b/src/seastar/include/seastar/core/smp.hh new file mode 100644 index 000000000..57c7b779f --- /dev/null +++ b/src/seastar/include/seastar/core/smp.hh @@ -0,0 +1,472 @@ +/* + * 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 2019 ScyllaDB + */ + +#pragma once + +#include <seastar/core/future.hh> +#include <seastar/core/loop.hh> +#include <seastar/core/semaphore.hh> +#include <seastar/core/metrics_registration.hh> +#include <seastar/core/posix.hh> +#include <seastar/core/reactor_config.hh> +#include <seastar/core/resource.hh> +#include <boost/lockfree/spsc_queue.hpp> +#include <boost/thread/barrier.hpp> +#include <boost/range/irange.hpp> +#include <deque> +#include <thread> + +/// \file + +namespace seastar { + +using shard_id = unsigned; + +class smp_service_group; +class reactor_backend_selector; + +namespace alien { + +class instance; + +} + +namespace internal { + +unsigned smp_service_group_id(smp_service_group ssg) noexcept; + +inline shard_id* this_shard_id_ptr() noexcept { + static thread_local shard_id g_this_shard_id; + return &g_this_shard_id; +} + +} + +/// Returns shard_id of the of the current shard. +inline shard_id this_shard_id() noexcept { + return *internal::this_shard_id_ptr(); +} + +/// Configuration for smp_service_group objects. +/// +/// \see create_smp_service_group() +struct smp_service_group_config { + /// The maximum number of non-local requests that execute on a shard concurrently + /// + /// Will be adjusted upwards to allow at least one request per non-local shard. + unsigned max_nonlocal_requests = 0; + /// An optional name for this smp group + /// + /// If this optional is engaged, timeout exception messages of the group's + /// semaphores will indicate the group's name. + std::optional<sstring> group_name; +}; + +/// A resource controller for cross-shard calls. +/// +/// An smp_service_group allows you to limit the concurrency of +/// smp::submit_to() and similar calls. While it's easy to limit +/// the caller's concurrency (for example, by using a semaphore), +/// the concurrency at the remote end can be multiplied by a factor +/// of smp::count-1, which can be large. +/// +/// The class is called a service _group_ because it can be used +/// to group similar calls that share resource usage characteristics, +/// need not be isolated from each other, but do need to be isolated +/// from other groups. Calls in a group should not nest; doing so +/// can result in ABA deadlocks. +/// +/// Nested submit_to() calls must form a directed acyclic graph +/// when considering their smp_service_groups as nodes. For example, +/// if a call using ssg1 then invokes another call using ssg2, the +/// internal call may not call again via either ssg1 or ssg2, or it +/// may form a cycle (and risking an ABBA deadlock). Create a +/// new smp_service_group_instead. +class smp_service_group { + unsigned _id; +#ifdef SEASTAR_DEBUG + unsigned _version = 0; +#endif +private: + explicit smp_service_group(unsigned id) noexcept : _id(id) {} + + friend unsigned internal::smp_service_group_id(smp_service_group ssg) noexcept; + friend smp_service_group default_smp_service_group() noexcept; + friend future<smp_service_group> create_smp_service_group(smp_service_group_config ssgc) noexcept; + friend future<> destroy_smp_service_group(smp_service_group) noexcept; +}; + +inline +unsigned +internal::smp_service_group_id(smp_service_group ssg) noexcept { + return ssg._id; +} + +/// Returns the default smp_service_group. This smp_service_group +/// does not impose any limits on concurrency in the target shard. +/// This makes is deadlock-safe, but can consume unbounded resources, +/// and should therefore only be used when initiator concurrency is +/// very low (e.g. administrative tasks). +smp_service_group default_smp_service_group() noexcept; + +/// Creates an smp_service_group with the specified configuration. +/// +/// The smp_service_group is global, and after this call completes, +/// the returned value can be used on any shard. +future<smp_service_group> create_smp_service_group(smp_service_group_config ssgc) noexcept; + +/// Destroy an smp_service_group. +/// +/// Frees all resources used by an smp_service_group. It must not +/// be used again once this function is called. +future<> destroy_smp_service_group(smp_service_group ssg) noexcept; + +inline +smp_service_group default_smp_service_group() noexcept { + return smp_service_group(0); +} + +using smp_timeout_clock = lowres_clock; +using smp_service_group_semaphore = basic_semaphore<named_semaphore_exception_factory, smp_timeout_clock>; +using smp_service_group_semaphore_units = semaphore_units<named_semaphore_exception_factory, smp_timeout_clock>; + +static constexpr smp_timeout_clock::time_point smp_no_timeout = smp_timeout_clock::time_point::max(); + +/// Options controlling the behaviour of \ref smp::submit_to(). +struct smp_submit_to_options { + /// Controls resource allocation. + smp_service_group service_group = default_smp_service_group(); + /// The timeout is relevant only to the time the call spends waiting to be + /// processed by the remote shard, and *not* to the time it takes to be + /// executed there. + smp_timeout_clock::time_point timeout = smp_no_timeout; + + smp_submit_to_options(smp_service_group service_group = default_smp_service_group(), smp_timeout_clock::time_point timeout = smp_no_timeout) noexcept + : service_group(service_group) + , timeout(timeout) { + } +}; + +void init_default_smp_service_group(shard_id cpu); + +smp_service_group_semaphore& get_smp_service_groups_semaphore(unsigned ssg_id, shard_id t) noexcept; + +class smp_message_queue { + static constexpr size_t queue_length = 128; + static constexpr size_t batch_size = 16; + static constexpr size_t prefetch_cnt = 2; + struct work_item; + struct lf_queue_remote { + reactor* remote; + }; + using lf_queue_base = boost::lockfree::spsc_queue<work_item*, + boost::lockfree::capacity<queue_length>>; + // use inheritence to control placement order + struct lf_queue : lf_queue_remote, lf_queue_base { + lf_queue(reactor* remote) : lf_queue_remote{remote} {} + void maybe_wakeup(); + ~lf_queue(); + }; + lf_queue _pending; + lf_queue _completed; + struct alignas(seastar::cache_line_size) { + size_t _sent = 0; + size_t _compl = 0; + size_t _last_snt_batch = 0; + size_t _last_cmpl_batch = 0; + size_t _current_queue_length = 0; + }; + // keep this between two structures with statistics + // this makes sure that they have at least one cache line + // between them, so hw prefetcher will not accidentally prefetch + // cache line used by another cpu. + metrics::metric_groups _metrics; + struct alignas(seastar::cache_line_size) { + size_t _received = 0; + size_t _last_rcv_batch = 0; + }; + struct work_item : public task { + explicit work_item(smp_service_group ssg) : task(current_scheduling_group()), ssg(ssg) {} + smp_service_group ssg; + virtual ~work_item() {} + virtual void fail_with(std::exception_ptr) = 0; + void process(); + virtual void complete() = 0; + }; + template <typename Func> + struct async_work_item : work_item { + smp_message_queue& _queue; + Func _func; + using futurator = futurize<std::invoke_result_t<Func>>; + using future_type = typename futurator::type; + using value_type = typename future_type::value_type; + std::optional<value_type> _result; + std::exception_ptr _ex; // if !_result + typename futurator::promise_type _promise; // used on local side + async_work_item(smp_message_queue& queue, smp_service_group ssg, Func&& func) : work_item(ssg), _queue(queue), _func(std::move(func)) {} + virtual void fail_with(std::exception_ptr ex) override { + _promise.set_exception(std::move(ex)); + } + virtual task* waiting_task() noexcept override { + // FIXME: waiting_tasking across shards is not implemented. Unsynchronized task access is unsafe. + return nullptr; + } + virtual void run_and_dispose() noexcept override { + // _queue.respond() below forwards the continuation chain back to the + // calling shard. + (void)futurator::invoke(this->_func).then_wrapped([this] (auto f) { + if (f.failed()) { + _ex = f.get_exception(); + } else { + _result = f.get(); + } + _queue.respond(this); + }); + // We don't delete the task here as the creator of the work item will + // delete it on the origin shard. + } + virtual void complete() override { + if (_result) { + _promise.set_value(std::move(*_result)); + } else { + // FIXME: _ex was allocated on another cpu + _promise.set_exception(std::move(_ex)); + } + } + future_type get_future() { return _promise.get_future(); } + }; + union tx_side { + tx_side() {} + ~tx_side() {} + void init() { new (&a) aa; } + struct aa { + std::deque<work_item*> pending_fifo; + } a; + } _tx; + std::vector<work_item*> _completed_fifo; +public: + smp_message_queue(reactor* from, reactor* to); + ~smp_message_queue(); + template <typename Func> + futurize_t<std::invoke_result_t<Func>> submit(shard_id t, smp_submit_to_options options, Func&& func) noexcept { + memory::scoped_critical_alloc_section _; + auto wi = std::make_unique<async_work_item<Func>>(*this, options.service_group, std::forward<Func>(func)); + auto fut = wi->get_future(); + submit_item(t, options.timeout, std::move(wi)); + return fut; + } + void start(unsigned cpuid); + template<size_t PrefetchCnt, typename Func> + size_t process_queue(lf_queue& q, Func process); + size_t process_incoming(); + size_t process_completions(shard_id t); + void stop(); +private: + void work(); + void submit_item(shard_id t, smp_timeout_clock::time_point timeout, std::unique_ptr<work_item> wi); + void respond(work_item* wi); + void move_pending(); + void flush_request_batch(); + void flush_response_batch(); + bool has_unflushed_responses() const; + bool pure_poll_rx() const; + bool pure_poll_tx() const; + + friend class smp; +}; + +class smp_message_queue; +struct reactor_options; +struct smp_options; + +class smp : public std::enable_shared_from_this<smp> { + alien::instance& _alien; + std::vector<posix_thread> _threads; + std::vector<std::function<void ()>> _thread_loops; // for dpdk + std::optional<boost::barrier> _all_event_loops_done; + struct qs_deleter { + void operator()(smp_message_queue** qs) const; + }; + std::unique_ptr<smp_message_queue*[], qs_deleter> _qs_owner; + static thread_local smp_message_queue**_qs; + static thread_local std::thread::id _tmain; + bool _using_dpdk = false; + + template <typename Func> + using returns_future = is_future<std::invoke_result_t<Func>>; + template <typename Func> + using returns_void = std::is_same<std::invoke_result_t<Func>, void>; +public: + explicit smp(alien::instance& alien) : _alien(alien) {} + void configure(const smp_options& smp_opts, const reactor_options& reactor_opts); + void cleanup() noexcept; + void cleanup_cpu(); + void arrive_at_event_loop_end(); + void join_all(); + static bool main_thread() { return std::this_thread::get_id() == _tmain; } + + /// Runs a function on a remote core. + /// + /// \param t designates the core to run the function on (may be a remote + /// core or the local core). + /// \param options an \ref smp_submit_to_options that contains options for this call. + /// \param func a callable to run on core \c t. + /// If \c func is a temporary object, its lifetime will be + /// extended by moving. This movement and the eventual + /// destruction of func are both done in the _calling_ core. + /// If \c func is a reference, the caller must guarantee that + /// it will survive the call. + /// \return whatever \c func returns, as a future<> (if \c func does not return a future, + /// submit_to() will wrap it in a future<>). + template <typename Func> + static futurize_t<std::invoke_result_t<Func>> submit_to(unsigned t, smp_submit_to_options options, Func&& func) noexcept { + using ret_type = std::invoke_result_t<Func>; + if (t == this_shard_id()) { + try { + if (!is_future<ret_type>::value) { + // Non-deferring function, so don't worry about func lifetime + return futurize<ret_type>::invoke(std::forward<Func>(func)); + } else if (std::is_lvalue_reference<Func>::value) { + // func is an lvalue, so caller worries about its lifetime + return futurize<ret_type>::invoke(func); + } else { + // Deferring call on rvalue function, make sure to preserve it across call + auto w = std::make_unique<std::decay_t<Func>>(std::move(func)); + auto ret = futurize<ret_type>::invoke(*w); + return ret.finally([w = std::move(w)] {}); + } + } catch (...) { + // Consistently return a failed future rather than throwing, to simplify callers + return futurize<std::invoke_result_t<Func>>::make_exception_future(std::current_exception()); + } + } else { + return _qs[t][this_shard_id()].submit(t, options, std::forward<Func>(func)); + } + } + /// Runs a function on a remote core. + /// + /// Uses default_smp_service_group() to control resource allocation. + /// + /// \param t designates the core to run the function on (may be a remote + /// core or the local core). + /// \param func a callable to run on core \c t. + /// If \c func is a temporary object, its lifetime will be + /// extended by moving. This movement and the eventual + /// destruction of func are both done in the _calling_ core. + /// If \c func is a reference, the caller must guarantee that + /// it will survive the call. + /// \return whatever \c func returns, as a future<> (if \c func does not return a future, + /// submit_to() will wrap it in a future<>). + template <typename Func> + static futurize_t<std::invoke_result_t<Func>> submit_to(unsigned t, Func&& func) noexcept { + return submit_to(t, default_smp_service_group(), std::forward<Func>(func)); + } + static bool poll_queues(); + static bool pure_poll_queues(); + static boost::integer_range<unsigned> all_cpus() noexcept { + return boost::irange(0u, count); + } + /// Invokes func on all shards. + /// + /// \param options the options to forward to the \ref smp::submit_to() + /// called behind the scenes. + /// \param func the function to be invoked on each shard. May return void or + /// future<>. Each async invocation will work with a separate copy + /// of \c func. + /// \returns a future that resolves when all async invocations finish. + template<typename Func> + SEASTAR_CONCEPT( requires std::is_nothrow_move_constructible_v<Func> ) + static future<> invoke_on_all(smp_submit_to_options options, Func&& func) noexcept { + static_assert(std::is_same<future<>, typename futurize<std::invoke_result_t<Func>>::type>::value, "bad Func signature"); + static_assert(std::is_nothrow_move_constructible_v<Func>); + return parallel_for_each(all_cpus(), [options, &func] (unsigned id) { + return smp::submit_to(id, options, Func(func)); + }); + } + /// Invokes func on all shards. + /// + /// \param func the function to be invoked on each shard. May return void or + /// future<>. Each async invocation will work with a separate copy + /// of \c func. + /// \returns a future that resolves when all async invocations finish. + /// + /// Passes the default \ref smp_submit_to_options to the + /// \ref smp::submit_to() called behind the scenes. + template<typename Func> + static future<> invoke_on_all(Func&& func) noexcept { + return invoke_on_all(smp_submit_to_options{}, std::forward<Func>(func)); + } + /// Invokes func on all other shards. + /// + /// \param cpu_id the cpu on which **not** to run the function. + /// \param options the options to forward to the \ref smp::submit_to() + /// called behind the scenes. + /// \param func the function to be invoked on each shard. May return void or + /// future<>. Each async invocation will work with a separate copy + /// of \c func. + /// \returns a future that resolves when all async invocations finish. + template<typename Func> + SEASTAR_CONCEPT( requires std::is_nothrow_move_constructible_v<Func> && + std::is_nothrow_copy_constructible_v<Func> ) + static future<> invoke_on_others(unsigned cpu_id, smp_submit_to_options options, Func func) noexcept { + static_assert(std::is_same<future<>, typename futurize<std::invoke_result_t<Func>>::type>::value, "bad Func signature"); + static_assert(std::is_nothrow_move_constructible_v<Func>); + return parallel_for_each(all_cpus(), [cpu_id, options, func = std::move(func)] (unsigned id) { + return id != cpu_id ? smp::submit_to(id, options, Func(func)) : make_ready_future<>(); + }); + } + /// Invokes func on all other shards. + /// + /// \param cpu_id the cpu on which **not** to run the function. + /// \param func the function to be invoked on each shard. May return void or + /// future<>. Each async invocation will work with a separate copy + /// of \c func. + /// \returns a future that resolves when all async invocations finish. + /// + /// Passes the default \ref smp_submit_to_options to the + /// \ref smp::submit_to() called behind the scenes. + template<typename Func> + SEASTAR_CONCEPT( requires std::is_nothrow_move_constructible_v<Func> ) + static future<> invoke_on_others(unsigned cpu_id, Func func) noexcept { + return invoke_on_others(cpu_id, smp_submit_to_options{}, std::move(func)); + } + /// Invokes func on all shards but the current one + /// + /// \param func the function to be invoked on each shard. May return void or + /// future<>. Each async invocation will work with a separate copy + /// of \c func. + /// \returns a future that resolves when all async invocations finish. + template<typename Func> + SEASTAR_CONCEPT( requires std::is_nothrow_move_constructible_v<Func> ) + static future<> invoke_on_others(Func func) noexcept { + return invoke_on_others(this_shard_id(), std::move(func)); + } +private: + void start_all_queues(); + void pin(unsigned cpu_id); + void allocate_reactor(unsigned id, reactor_backend_selector rbs, reactor_config cfg); + void create_thread(std::function<void ()> thread_loop); + unsigned adjust_max_networking_aio_io_control_blocks(unsigned network_iocbs); +public: + static unsigned count; +}; + +} diff --git a/src/seastar/include/seastar/core/smp_options.hh b/src/seastar/include/seastar/core/smp_options.hh new file mode 100644 index 000000000..f0ca9f1cb --- /dev/null +++ b/src/seastar/include/seastar/core/smp_options.hh @@ -0,0 +1,108 @@ +/* + * 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 2022 ScyllaDB + */ + +#pragma once + +#include <seastar/util/program-options.hh> + +/// \file + +namespace seastar { + +enum class memory_allocator { + /// Seastar's own allocator, optimized for its shard-per core design. + /// Strongly recommended for most seastar apps. + seastar, + /// The standard allocator from libc. + /// Useful when writing seastar-based tool apps that want to + /// minimize their footprint on the system they run on. + standard, +}; + +/// Configuration for the multicore aspect of seastar. +struct smp_options : public program_options::option_group { + /// Number of threads (default: one per CPU). + program_options::value<unsigned> smp; + /// CPUs to use (in cpuset(7) format; default: all)). + program_options::value<resource::cpuset> cpuset; + /// Memory to use, in bytes (ex: 4G) (default: all). + program_options::value<std::string> memory; + /// Memory reserved to OS (if \ref memory not specified). + program_options::value<std::string> reserve_memory; + /// Path to accessible hugetlbfs mount (typically /dev/hugepages/something). + program_options::value<std::string> hugepages; + /// Lock all memory (prevents swapping). + program_options::value<bool> lock_memory; + /// Pin threads to their cpus (disable for overprovisioning). + /// + /// Default: \p true. + program_options::value<bool> thread_affinity; + /// \brief Number of IO queues. + /// + /// Each IO unit will be responsible for a fraction of the IO requests. + /// Defaults to the number of threads + /// \note Unused when seastar is compiled without \p HWLOC support. + program_options::value<unsigned> num_io_queues; + /// \brief Number of IO groups. + /// + /// Each IO group will be responsible for a fraction of the IO requests. + /// Defaults to the number of NUMA nodes + /// \note Unused when seastar is compiled without \p HWLOC support. + program_options::value<unsigned> num_io_groups; + /// Path to a YAML file describing the characteristics of the I/O Subsystem. + program_options::value<std::string> io_properties_file; + /// A YAML string describing the characteristics of the I/O Subsystem. + program_options::value<std::string> io_properties; + /// Enable mbind. + /// + /// Default: \p true. + program_options::value<bool> mbind; + /// Enable workaround for glibc/gcc c++ exception scalablity problem. + /// + /// Default: \p true. + /// \note Unused when seastar is compiled without the exception scaling support. + program_options::value<bool> enable_glibc_exception_scaling_workaround; + /// If some CPUs are found not to have any local NUMA nodes, allow assigning + /// them to remote ones. + /// \note Unused when seastar is compiled without \p HWLOC support. + program_options::value<bool> allow_cpus_in_remote_numa_nodes; + + /// Memory allocator to use. + /// + /// The following options only have effect if the \ref memory_allocator::seastar is used: + /// * \ref smp_options::memory + /// * \ref smp_options::reserve_memory + /// * \ref smp_options::hugepages + /// * \ref smp_options::mbind + /// * \ref reactor_options::heapprof + /// * \ref reactor_options::abort_on_seastar_bad_alloc + /// * \ref reactor_options::dump_memory_diagnostics_on_alloc_failure_kind + seastar::memory_allocator memory_allocator = memory_allocator::seastar; + + /// \cond internal + /// Additional memory reserved to OS (added to the default value or the value specified by \ref reserve_memory). + size_t reserve_additional_memory = 0; + /// \endcond +public: + smp_options(program_options::option_group* parent_group); +}; + +} diff --git a/src/seastar/include/seastar/core/sstring.hh b/src/seastar/include/seastar/core/sstring.hh new file mode 100644 index 000000000..908c47cfb --- /dev/null +++ b/src/seastar/include/seastar/core/sstring.hh @@ -0,0 +1,780 @@ +/* + * 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 2014 Cloudius Systems + */ + +#pragma once + +#include <stdint.h> +#include <algorithm> +#include <cassert> +#include <string> +#include <vector> +#include <unordered_map> +#include <cstring> +#include <stdexcept> +#include <initializer_list> +#include <istream> +#include <ostream> +#include <functional> +#include <type_traits> +#include <fmt/ostream.h> +#include <seastar/util/concepts.hh> +#include <seastar/util/std-compat.hh> +#include <seastar/core/temporary_buffer.hh> + +namespace seastar { + +template <typename char_type, typename Size, Size max_size, bool NulTerminate = true> +class basic_sstring; + +#ifdef SEASTAR_SSTRING +// Older std::string used atomic reference counting and had no small-buffer-optimization. +// At some point the new std::string ABI improved -- no reference counting plus the small +// buffer optimization. However, aliasing seastar::sstring to std::string still ends up +// with a small performance degradation. (FIXME?) +using sstring = basic_sstring<char, uint32_t, 15>; +#else +using sstring = std::string; +#endif + +namespace internal { +[[noreturn]] void throw_bad_alloc(); +[[noreturn]] void throw_sstring_overflow(); +[[noreturn]] void throw_sstring_out_of_range(); +} + +template <typename char_type, typename Size, Size max_size, bool NulTerminate> +class basic_sstring { + static_assert( + (std::is_same<char_type, char>::value + || std::is_same<char_type, signed char>::value + || std::is_same<char_type, unsigned char>::value), + "basic_sstring only supports single byte char types"); + union contents { + struct external_type { + char_type* str; + Size size; + int8_t pad; + } external; + struct internal_type { + char_type str[max_size]; + int8_t size; + } internal; + static_assert(sizeof(external_type) <= sizeof(internal_type), "max_size too small"); + static_assert(max_size <= 127, "max_size too large"); + } u; + bool is_internal() const noexcept { + return u.internal.size >= 0; + } + bool is_external() const noexcept { + return !is_internal(); + } + const char_type* str() const noexcept { + return is_internal() ? u.internal.str : u.external.str; + } + char_type* str() noexcept { + return is_internal() ? u.internal.str : u.external.str; + } + +public: + using value_type = char_type; + using traits_type = std::char_traits<char_type>; + using allocator_type = std::allocator<char_type>; + using reference = char_type&; + using const_reference = const char_type&; + using pointer = char_type*; + using const_pointer = const char_type*; + using iterator = char_type*; + using const_iterator = const char_type*; + // FIXME: add reverse_iterator and friend + using difference_type = ssize_t; // std::make_signed_t<Size> can be too small + using size_type = Size; + static constexpr size_type npos = static_cast<size_type>(-1); + static constexpr unsigned padding() { return unsigned(NulTerminate); } +public: + struct initialized_later {}; + + basic_sstring() noexcept { + u.internal.size = 0; + if (NulTerminate) { + u.internal.str[0] = '\0'; + } + } + basic_sstring(const basic_sstring& x) { + if (x.is_internal()) { + u.internal = x.u.internal; + } else { + u.internal.size = -1; + u.external.str = reinterpret_cast<char_type*>(std::malloc(x.u.external.size + padding())); + if (!u.external.str) { + internal::throw_bad_alloc(); + } + std::copy(x.u.external.str, x.u.external.str + x.u.external.size + padding(), u.external.str); + u.external.size = x.u.external.size; + } + } + basic_sstring(basic_sstring&& x) noexcept { +#pragma GCC diagnostic push + // Is a small-string construction is followed by this move constructor, then the trailing bytes + // of x.u are not initialized, but copied. gcc complains, but it is both legitimate to copy + // these bytes, and more efficient than a variable-size copy +#pragma GCC diagnostic ignored "-Wuninitialized" + u = x.u; +#pragma GCC diagnostic pop + x.u.internal.size = 0; + x.u.internal.str[0] = '\0'; + } + basic_sstring(initialized_later, size_t size) { + if (size_type(size) != size) { + internal::throw_sstring_overflow(); + } + if (size + padding() <= sizeof(u.internal.str)) { + if (NulTerminate) { + u.internal.str[size] = '\0'; + } + u.internal.size = size; + } else { + u.internal.size = -1; + u.external.str = reinterpret_cast<char_type*>(std::malloc(size + padding())); + if (!u.external.str) { + internal::throw_bad_alloc(); + } + u.external.size = size; + if (NulTerminate) { + u.external.str[size] = '\0'; + } + } + } + basic_sstring(const char_type* x, size_t size) { + if (size_type(size) != size) { + internal::throw_sstring_overflow(); + } + if (size + padding() <= sizeof(u.internal.str)) { + std::copy(x, x + size, u.internal.str); + if (NulTerminate) { + u.internal.str[size] = '\0'; + } + u.internal.size = size; + } else { + u.internal.size = -1; + u.external.str = reinterpret_cast<char_type*>(std::malloc(size + padding())); + if (!u.external.str) { + internal::throw_bad_alloc(); + } + u.external.size = size; + std::copy(x, x + size, u.external.str); + if (NulTerminate) { + u.external.str[size] = '\0'; + } + } + } + + basic_sstring(size_t size, char_type x) : basic_sstring(initialized_later(), size) { + memset(begin(), x, size); + } + + basic_sstring(const char* x) : basic_sstring(reinterpret_cast<const char_type*>(x), std::strlen(x)) {} + basic_sstring(std::basic_string<char_type>& x) : basic_sstring(x.c_str(), x.size()) {} + basic_sstring(std::initializer_list<char_type> x) : basic_sstring(x.begin(), x.end() - x.begin()) {} + basic_sstring(const char_type* b, const char_type* e) : basic_sstring(b, e - b) {} + basic_sstring(const std::basic_string<char_type>& s) + : basic_sstring(s.data(), s.size()) {} + template <typename InputIterator> + basic_sstring(InputIterator first, InputIterator last) + : basic_sstring(initialized_later(), std::distance(first, last)) { + std::copy(first, last, begin()); + } + explicit basic_sstring(std::basic_string_view<char_type, traits_type> v) + : basic_sstring(v.data(), v.size()) { + } + ~basic_sstring() noexcept { + if (is_external()) { + std::free(u.external.str); + } + } + basic_sstring& operator=(const basic_sstring& x) { + basic_sstring tmp(x); + swap(tmp); + return *this; + } + basic_sstring& operator=(basic_sstring&& x) noexcept { + if (this != &x) { + this->~basic_sstring(); + new (this) basic_sstring(std::move(x)); + } + return *this; + } + operator std::basic_string<char_type>() const { + return { str(), size() }; + } + + size_t size() const noexcept { + return is_internal() ? u.internal.size : u.external.size; + } + + size_t length() const noexcept { + return size(); + } + + size_t find(char_type t, size_t pos = 0) const noexcept { + const char_type* it = str() + pos; + const char_type* end = str() + size(); + while (it < end) { + if (*it == t) { + return it - str(); + } + it++; + } + return npos; + } + + size_t find(const basic_sstring& s, size_t pos = 0) const noexcept { + const char_type* it = str() + pos; + const char_type* end = str() + size(); + const char_type* c_str = s.str(); + + if (pos > size()) { + return npos; + } + + const size_t len2 = s.size(); + if (len2 == 0) { + return pos; + } + + size_t len1 = end - it; + if (len1 < len2) { + return npos; + } + + char_type f2 = *c_str; + while (true) { + len1 = end - it; + if (len1 < len2) { + return npos; + } + + // find the first byte of pattern string matching in source string + it = traits_type::find(it, len1 - len2 + 1, f2); + if (it == nullptr) { + return npos; + } + + if (traits_type::compare(it, c_str, len2) == 0) { + return it - str(); + } + + ++it; + } + } + + /** + * find_last_of find the last occurrence of c in the string. + * When pos is specified, the search only includes characters + * at or before position pos. + * + */ + size_t find_last_of (char_type c, size_t pos = npos) const noexcept { + const char_type* str_start = str(); + if (size()) { + if (pos >= size()) { + pos = size() - 1; + } + const char_type* p = str_start + pos + 1; + do { + p--; + if (*p == c) { + return (p - str_start); + } + } while (p != str_start); + } + return npos; + } + + /** + * Append a C substring. + * @param s The C string to append. + * @param n The number of characters to append. + * @return Reference to this string. + */ + basic_sstring& append (const char_type* s, size_t n) { + basic_sstring ret(initialized_later(), size() + n); + std::copy(begin(), end(), ret.begin()); + std::copy(s, s + n, ret.begin() + size()); + *this = std::move(ret); + return *this; + } + + /** + * Resize string and use the specified @c op to modify the content and the length + * @param n new size + * @param op the function object used for setting the new content of the string + */ + template <class Operation> + SEASTAR_CONCEPT( requires std::is_invocable_r<size_t, Operation, char_type*, size_t>::value ) + void resize_and_overwrite(size_t n, Operation op) { + if (n > size()) { + *this = basic_sstring(initialized_later(), n); + } + size_t r = std::move(op)(data(), n); + assert(r <= n); + resize(r); + } + + /** + * Resize string. + * @param n new size. + * @param c if n greater than current size character to fill newly allocated space with. + */ + void resize(size_t n, const char_type c = '\0') { + if (n > size()) { + *this += basic_sstring(n - size(), c); + } else if (n < size()) { + if (is_internal()) { + u.internal.size = n; + if (NulTerminate) { + u.internal.str[n] = '\0'; + } + } else if (n + padding() <= sizeof(u.internal.str)) { + *this = basic_sstring(u.external.str, n); + } else { + u.external.size = n; + if (NulTerminate) { + u.external.str[n] = '\0'; + } + } + } + } + + /** + * Replace characters with a value of a C style substring. + * + */ + basic_sstring& replace(size_type pos, size_type n1, const char_type* s, + size_type n2) { + if (pos > size()) { + internal::throw_sstring_out_of_range(); + } + + if (n1 > size() - pos) { + n1 = size() - pos; + } + + if (n1 == n2) { + if (n2) { + std::copy(s, s + n2, begin() + pos); + } + return *this; + } + basic_sstring ret(initialized_later(), size() + n2 - n1); + char_type* p= ret.begin(); + std::copy(begin(), begin() + pos, p); + p += pos; + if (n2) { + std::copy(s, s + n2, p); + } + p += n2; + std::copy(begin() + pos + n1, end(), p); + *this = std::move(ret); + return *this; + } + + template <class InputIterator> + basic_sstring& replace (const_iterator i1, const_iterator i2, + InputIterator first, InputIterator last) { + if (i1 < begin() || i1 > end() || i2 < begin()) { + internal::throw_sstring_out_of_range(); + } + if (i2 > end()) { + i2 = end(); + } + + if (i2 - i1 == last - first) { + //in place replacement + std::copy(first, last, const_cast<char_type*>(i1)); + return *this; + } + basic_sstring ret(initialized_later(), size() + (last - first) - (i2 - i1)); + char_type* p = ret.begin(); + p = std::copy(cbegin(), i1, p); + p = std::copy(first, last, p); + std::copy(i2, cend(), p); + *this = std::move(ret); + return *this; + } + + iterator erase(iterator first, iterator last) { + size_t pos = first - begin(); + replace(pos, last - first, nullptr, 0); + return begin() + pos; + } + + /** + * Inserts additional characters into the string right before + * the character indicated by p. + */ + template <class InputIterator> + void insert(const_iterator p, InputIterator beg, InputIterator end) { + replace(p, p, beg, end); + } + + /** + * Returns a read/write reference to the data at the last + * element of the string. + * This function shall not be called on empty strings. + */ + reference + back() noexcept { + return operator[](size() - 1); + } + + /** + * Returns a read-only (constant) reference to the data at the last + * element of the string. + * This function shall not be called on empty strings. + */ + const_reference + back() const noexcept { + return operator[](size() - 1); + } + + basic_sstring substr(size_t from, size_t len = npos) const { + if (from > size()) { + internal::throw_sstring_out_of_range(); + } + if (len > size() - from) { + len = size() - from; + } + if (len == 0) { + return ""; + } + return { str() + from , len }; + } + + const char_type& at(size_t pos) const { + if (pos >= size()) { + internal::throw_sstring_out_of_range(); + } + return *(str() + pos); + } + + char_type& at(size_t pos) { + if (pos >= size()) { + internal::throw_sstring_out_of_range(); + } + return *(str() + pos); + } + + bool empty() const noexcept { + return u.internal.size == 0; + } + + // Deprecated March 2020. + [[deprecated("Use = {}")]] + void reset() noexcept { + if (is_external()) { + std::free(u.external.str); + } + u.internal.size = 0; + if (NulTerminate) { + u.internal.str[0] = '\0'; + } + } + temporary_buffer<char_type> release() && { + if (is_external()) { + auto ptr = u.external.str; + auto size = u.external.size; + u.external.str = nullptr; + u.external.size = 0; + return temporary_buffer<char_type>(ptr, size, make_free_deleter(ptr)); + } else { + auto buf = temporary_buffer<char_type>(u.internal.size); + std::copy(u.internal.str, u.internal.str + u.internal.size, buf.get_write()); + u.internal.size = 0; + if (NulTerminate) { + u.internal.str[0] = '\0'; + } + return buf; + } + } + int compare(std::basic_string_view<char_type, traits_type> x) const noexcept { + auto n = traits_type::compare(begin(), x.begin(), std::min(size(), x.size())); + if (n != 0) { + return n; + } + if (size() < x.size()) { + return -1; + } else if (size() > x.size()) { + return 1; + } else { + return 0; + } + } + + int compare(size_t pos, size_t sz, std::basic_string_view<char_type, traits_type> x) const { + if (pos > size()) { + internal::throw_sstring_out_of_range(); + } + + sz = std::min(size() - pos, sz); + auto n = traits_type::compare(begin() + pos, x.begin(), std::min(sz, x.size())); + if (n != 0) { + return n; + } + if (sz < x.size()) { + return -1; + } else if (sz > x.size()) { + return 1; + } else { + return 0; + } + } + + void swap(basic_sstring& x) noexcept { + contents tmp; + tmp = x.u; + x.u = u; + u = tmp; + } + char_type* data() noexcept { + return str(); + } + const char_type* data() const noexcept { + return str(); + } + const char_type* c_str() const noexcept { + return str(); + } + const char_type* begin() const noexcept { return str(); } + const char_type* end() const noexcept { return str() + size(); } + const char_type* cbegin() const noexcept { return str(); } + const char_type* cend() const noexcept { return str() + size(); } + char_type* begin() noexcept { return str(); } + char_type* end() noexcept { return str() + size(); } + bool operator==(const basic_sstring& x) const noexcept { + return size() == x.size() && std::equal(begin(), end(), x.begin()); + } + bool operator!=(const basic_sstring& x) const noexcept { + return !operator==(x); + } + bool operator<(const basic_sstring& x) const noexcept { + return compare(x) < 0; + } + basic_sstring operator+(const basic_sstring& x) const { + basic_sstring ret(initialized_later(), size() + x.size()); + std::copy(begin(), end(), ret.begin()); + std::copy(x.begin(), x.end(), ret.begin() + size()); + return ret; + } + basic_sstring& operator+=(const basic_sstring& x) { + return *this = *this + x; + } + char_type& operator[](size_type pos) noexcept { + return str()[pos]; + } + const char_type& operator[](size_type pos) const noexcept { + return str()[pos]; + } + + operator std::basic_string_view<char_type>() const noexcept { + // we assume that std::basic_string_view<char_type>(str(), size()) + // won't throw, although it is not specified as noexcept in + // https://en.cppreference.com/w/cpp/string/basic_string_view/basic_string_view + // at this time (C++20). + // + // This is similar to std::string operator std::basic_string_view: + // https://en.cppreference.com/w/cpp/string/basic_string/operator_basic_string_view + // that is specified as noexcept too. + static_assert(noexcept(std::basic_string_view<char_type>(str(), size()))); + return std::basic_string_view<char_type>(str(), size()); + } +}; +template <typename char_type, typename Size, Size max_size, bool NulTerminate> +constexpr Size basic_sstring<char_type, Size, max_size, NulTerminate>::npos; + +namespace internal { +template <class T> struct is_sstring : std::false_type {}; +template <typename char_type, typename Size, Size max_size, bool NulTerminate> +struct is_sstring<basic_sstring<char_type, Size, max_size, NulTerminate>> : std::true_type {}; +} + +template <typename string_type = sstring> +string_type uninitialized_string(size_t size) { + if constexpr (internal::is_sstring<string_type>::value) { + return string_type(typename string_type::initialized_later(), size); + } else { + string_type ret; +#ifdef __cpp_lib_string_resize_and_overwrite + ret.resize_and_overwrite(size, [](string_type::value_type*, string_type::size_type n) { return n; }); +#else + ret.resize(size); +#endif + return ret; + } +} + +template <typename char_type, typename size_type, size_type Max, size_type N, bool NulTerminate> +inline +basic_sstring<char_type, size_type, Max, NulTerminate> +operator+(const char(&s)[N], const basic_sstring<char_type, size_type, Max, NulTerminate>& t) { + using sstring = basic_sstring<char_type, size_type, Max, NulTerminate>; + // don't copy the terminating NUL character + sstring ret(typename sstring::initialized_later(), N-1 + t.size()); + auto p = std::copy(std::begin(s), std::end(s)-1, ret.begin()); + std::copy(t.begin(), t.end(), p); + return ret; +} + +template <typename T> +static inline +size_t constexpr str_len(const T& s) { + return std::string_view(s).size(); +} + +template <typename char_type, typename size_type, size_type max_size> +inline +void swap(basic_sstring<char_type, size_type, max_size>& x, + basic_sstring<char_type, size_type, max_size>& y) noexcept +{ + return x.swap(y); +} + +template <typename char_type, typename size_type, size_type max_size, bool NulTerminate, typename char_traits> +inline +std::basic_ostream<char_type, char_traits>& +operator<<(std::basic_ostream<char_type, char_traits>& os, + const basic_sstring<char_type, size_type, max_size, NulTerminate>& s) { + return os.write(s.begin(), s.size()); +} + +template <typename char_type, typename size_type, size_type max_size, bool NulTerminate, typename char_traits> +inline +std::basic_istream<char_type, char_traits>& +operator>>(std::basic_istream<char_type, char_traits>& is, + basic_sstring<char_type, size_type, max_size, NulTerminate>& s) { + std::string tmp; + is >> tmp; + s = tmp; + return is; +} + +} + +namespace std { + +template <typename char_type, typename size_type, size_type max_size, bool NulTerminate> +struct hash<seastar::basic_sstring<char_type, size_type, max_size, NulTerminate>> { + size_t operator()(const seastar::basic_sstring<char_type, size_type, max_size, NulTerminate>& s) const { + return std::hash<std::basic_string_view<char_type>>()(s); + } +}; + +} + +namespace seastar { + +template <typename T> +static inline +void copy_str_to(char*& dst, const T& s) { + std::string_view v(s); + dst = std::copy(v.begin(), v.end(), dst); +} + +template <typename String = sstring, typename... Args> +static String make_sstring(Args&&... args) +{ + String ret = uninitialized_string<String>((str_len(args) + ...)); + auto dst = ret.data(); + (copy_str_to(dst, args), ...); + return ret; +} + +namespace internal { +template <typename string_type, typename T> +string_type to_sstring(T value) { + auto size = fmt::formatted_size("{}", value); + auto formatted = uninitialized_string<string_type>(size); + fmt::format_to(formatted.data(), "{}", value); + return formatted; +} + +template <typename string_type> +string_type to_sstring(const char* value) { + return string_type(value); +} + +template <typename string_type> +string_type to_sstring(sstring value) { + return value; +} + +template <typename string_type> +string_type to_sstring(const temporary_buffer<char>& buf) { + return string_type(buf.get(), buf.size()); +} +} + +template <typename string_type = sstring, typename T> +string_type to_sstring(T value) { + return internal::to_sstring<string_type>(value); +} +} + +namespace std { +template <typename T> +inline +std::ostream& operator<<(std::ostream& os, const std::vector<T>& v) { + bool first = true; + os << "{"; + for (auto&& elem : v) { + if (!first) { + os << ", "; + } else { + first = false; + } + os << elem; + } + os << "}"; + return os; +} + +template <typename Key, typename T, typename Hash, typename KeyEqual, typename Allocator> +std::ostream& operator<<(std::ostream& os, const std::unordered_map<Key, T, Hash, KeyEqual, Allocator>& v) { + bool first = true; + os << "{"; + for (auto&& elem : v) { + if (!first) { + os << ", "; + } else { + first = false; + } + os << "{" << elem.first << " -> " << elem.second << "}"; + } + os << "}"; + return os; +} +} + +#if FMT_VERSION >= 90000 + +template <typename char_type, typename Size, Size max_size, bool NulTerminate> +struct fmt::formatter<seastar::basic_sstring<char_type, Size, max_size, NulTerminate>> : fmt::ostream_formatter {}; + +#endif diff --git a/src/seastar/include/seastar/core/stall_sampler.hh b/src/seastar/include/seastar/core/stall_sampler.hh new file mode 100644 index 000000000..9a2db6546 --- /dev/null +++ b/src/seastar/include/seastar/core/stall_sampler.hh @@ -0,0 +1,53 @@ +/* + * Copyright (C) 2018 ScyllaDB + */ + +/* + * 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. + */ + +#pragma once + +#include <seastar/core/future.hh> +#include <seastar/util/noncopyable_function.hh> + +#include <chrono> +#include <iosfwd> + +// Instrumentation to detect context switches during reactor execution +// and associated stall time, intended for use in tests + +namespace seastar { + +namespace internal { + +struct stall_report { + uint64_t kernel_stalls; + sched_clock::duration run_wall_time; // excludes sleeps + sched_clock::duration stall_time; +}; + +/// Run the unit-under-test (uut) function until completion, and report on any +/// reactor stalls it generated. +future<stall_report> report_reactor_stalls(noncopyable_function<future<> ()> uut); + +std::ostream& operator<<(std::ostream& os, const stall_report& sr); + +} + +} + diff --git a/src/seastar/include/seastar/core/stream.hh b/src/seastar/include/seastar/core/stream.hh new file mode 100644 index 000000000..b23106108 --- /dev/null +++ b/src/seastar/include/seastar/core/stream.hh @@ -0,0 +1,174 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/core/future.hh> +#include <exception> +#include <functional> +#include <cassert> + +namespace seastar { + +// A stream/subscription pair is similar to a promise/future pair, +// but apply to a sequence of values instead of a single value. +// +// A stream<> is the producer side. It may call produce() as long +// as the future<> returned from the previous invocation is ready. +// To signify no more data is available, call close(). +// +// A subscription<> is the consumer side. It is created by a call +// to stream::listen(). Calling subscription::start(), +// which registers the data processing callback, starts processing +// events. It may register for end-of-stream notifications by +// chaining the when_done() future, which also delivers error +// events (as exceptions). +// +// The consumer can pause generation of new data by returning +// a non-ready future; when the future becomes ready, the producer +// will resume processing. + +template <typename... T> +class stream; + +template <typename... T> +class subscription; + +template <typename... T> +class stream { +public: + using next_fn = noncopyable_function<future<> (T...)>; + +private: + promise<> _done; + promise<> _ready; + next_fn _next; + + /// \brief Start receiving events from the stream. + /// + /// \param next Callback to call for each event + void start(next_fn next) { + _next = std::move(next); + _ready.set_value(); + } + +public: + stream() = default; + stream(const stream&) = delete; + stream(stream&&) = delete; + void operator=(const stream&) = delete; + void operator=(stream&&) = delete; + + // Returns a subscription that reads value from this + // stream. + subscription<T...> listen() { + return subscription<T...>(this); + } + + // Returns a subscription that reads value from this + // stream, and also sets up the listen function. + subscription<T...> listen(next_fn next) { + start(std::move(next)); + return subscription<T...>(this); + } + + // Becomes ready when the listener is ready to accept + // values. Call only once, when beginning to produce + // values. + future<> started() { + return _ready.get_future(); + } + + // Produce a value. Call only after started(), and after + // a previous produce() is ready. + future<> produce(T... data); + + // End the stream. Call only after started(), and after + // a previous produce() is ready. No functions may be called + // after this. + void close() { + _done.set_value(); + } + + // Signal an error. Call only after started(), and after + // a previous produce() is ready. No functions may be called + // after this. + template <typename E> + void set_exception(E ex) { + _done.set_exception(ex); + } + + friend class subscription<T...>; +}; + +template <typename... T> +class subscription { + stream<T...>* _stream; + future<> _done; + explicit subscription(stream<T...>* s) : _stream(s), _done(s->_done.get_future()) { + } + +public: + using next_fn = typename stream<T...>::next_fn; + subscription(subscription&& x) : _stream(x._stream), _done(std::move(x._done)) { + x._stream = nullptr; + } + + /// \brief Start receiving events from the stream. + /// + /// \param next Callback to call for each event + void start(next_fn next) { + return _stream->start(std::move(next)); + } + + // Becomes ready when the stream is empty, or when an error + // happens (in that case, an exception is held). + future<> done() { + return std::move(_done); + } + + friend class stream<T...>; +}; + +template <typename... T> +inline +future<> +stream<T...>::produce(T... data) { + auto ret = futurize_invoke(_next, std::move(data)...); + if (ret.available() && !ret.failed()) { + // Native network stack depends on stream::produce() returning + // a ready future to push packets along without dropping. As + // a temporary workaround, special case a ready, unfailed future + // and return it immediately, so that then_wrapped(), below, + // doesn't convert a ready future to an unready one. + return ret; + } + return ret.then_wrapped([this] (auto&& f) { + try { + f.get(); + } catch (...) { + _done.set_exception(std::current_exception()); + // FIXME: tell the producer to stop producing + throw; + } + }); +} +} diff --git a/src/seastar/include/seastar/core/systemwide_memory_barrier.hh b/src/seastar/include/seastar/core/systemwide_memory_barrier.hh new file mode 100644 index 000000000..879d1bfc6 --- /dev/null +++ b/src/seastar/include/seastar/core/systemwide_memory_barrier.hh @@ -0,0 +1,37 @@ +/* + * 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 2015 Scylla DB + */ + +#pragma once + +namespace seastar { + +/// \cond internal + +// cause all threads to invoke a full memory barrier +void systemwide_memory_barrier(); +// attempt to invoke a systemwide memory barrier; return false +// if doing so would cause lock contention in the kernel +bool try_systemwide_memory_barrier(); + +/// \endcond + +} + diff --git a/src/seastar/include/seastar/core/task.hh b/src/seastar/include/seastar/core/task.hh new file mode 100644 index 000000000..f78fa9a83 --- /dev/null +++ b/src/seastar/include/seastar/core/task.hh @@ -0,0 +1,73 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <memory> +#include <seastar/core/scheduling.hh> +#include <seastar/util/backtrace.hh> + +namespace seastar { + +class task { +protected: + scheduling_group _sg; +private: +#ifdef SEASTAR_TASK_BACKTRACE + shared_backtrace _bt; +#endif +protected: + // Task destruction is performed by run_and_dispose() via a concrete type, + // so no need for a virtual destructor here. Derived classes that implement + // run_and_dispose() should be declared final to avoid losing concrete type + // information via inheritance. + ~task() = default; + + scheduling_group set_scheduling_group(scheduling_group new_sg) noexcept{ + return std::exchange(_sg, new_sg); + } +public: + explicit task(scheduling_group sg = current_scheduling_group()) noexcept : _sg(sg) {} + virtual void run_and_dispose() noexcept = 0; + /// Returns the next task which is waiting for this task to complete execution, or nullptr. + virtual task* waiting_task() noexcept = 0; + scheduling_group group() const { return _sg; } + shared_backtrace get_backtrace() const; +#ifdef SEASTAR_TASK_BACKTRACE + void make_backtrace() noexcept; +#else + void make_backtrace() noexcept {} +#endif +}; + +inline +shared_backtrace task::get_backtrace() const { +#ifdef SEASTAR_TASK_BACKTRACE + return _bt; +#else + return {}; +#endif +} + +void schedule(task* t) noexcept; +void schedule_urgent(task* t) noexcept; + +} diff --git a/src/seastar/include/seastar/core/temporary_buffer.hh b/src/seastar/include/seastar/core/temporary_buffer.hh new file mode 100644 index 000000000..6ae4100f2 --- /dev/null +++ b/src/seastar/include/seastar/core/temporary_buffer.hh @@ -0,0 +1,246 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/core/deleter.hh> +#include <seastar/util/eclipse.hh> +#include <seastar/util/std-compat.hh> +#include <malloc.h> +#include <algorithm> +#include <cstddef> + +namespace seastar { + +/// \addtogroup memory-module +/// @{ + +/// Temporary, self-managed byte buffer. +/// +/// A \c temporary_buffer is similar to an \c std::string or a \c std::unique_ptr<char[]>, +/// but provides more flexible memory management. A \c temporary_buffer can own the memory +/// it points to, or it can be shared with another \c temporary_buffer, or point at a substring +/// of a buffer. It uses a \ref deleter to manage its memory. +/// +/// A \c temporary_buffer should not be held indefinitely. It can be held while a request +/// is processed, or for a similar duration, but not longer, as it can tie up more memory +/// that its size indicates. +/// +/// A buffer can be shared: two \c temporary_buffer objects will point to the same data, +/// or a subset of it. See the \ref temporary_buffer::share() method. +/// +/// Unless you created a \c temporary_buffer yourself, do not modify its contents, as they +/// may be shared with another user that does not expect the data to change. +/// +/// Use cases for a \c temporary_buffer include: +/// - passing a substring of a tcp packet for the user to consume (zero-copy +/// tcp input) +/// - passing a refcounted blob held in memory to tcp, ensuring that when the TCP ACK +/// is received, the blob is released (by decrementing its reference count) (zero-copy +/// tcp output) +/// +/// \tparam CharType underlying character type (must be a variant of \c char). +template <typename CharType> +class temporary_buffer { + static_assert(sizeof(CharType) == 1, "must buffer stream of bytes"); + CharType* _buffer; + size_t _size; + deleter _deleter; +public: + /// Creates a \c temporary_buffer of a specified size. The buffer is not shared + /// with anyone, and is not initialized. + /// + /// \param size buffer size, in bytes + explicit temporary_buffer(size_t size) + : _buffer(static_cast<CharType*>(malloc(size * sizeof(CharType)))), _size(size) + , _deleter(make_free_deleter(_buffer)) { + if (size && !_buffer) { + throw std::bad_alloc(); + } + } + //explicit temporary_buffer(CharType* borrow, size_t size) : _buffer(borrow), _size(size) {} + /// Creates an empty \c temporary_buffer that does not point at anything. + temporary_buffer() noexcept + : _buffer(nullptr) + , _size(0) {} + temporary_buffer(const temporary_buffer&) = delete; + + /// Moves a \c temporary_buffer. + temporary_buffer(temporary_buffer&& x) noexcept : _buffer(x._buffer), _size(x._size), _deleter(std::move(x._deleter)) { + x._buffer = nullptr; + x._size = 0; + } + + /// Creates a \c temporary_buffer with a specific deleter. + /// + /// \param buf beginning of the buffer held by this \c temporary_buffer + /// \param size size of the buffer + /// \param d deleter controlling destruction of the buffer. The deleter + /// will be destroyed when there are no longer any users for the buffer. + temporary_buffer(CharType* buf, size_t size, deleter d) noexcept + : _buffer(buf), _size(size), _deleter(std::move(d)) {} + /// Creates a `temporary_buffer` containing a copy of the provided data + /// + /// \param src data buffer to be copied + /// \param size size of data buffer in `src` + temporary_buffer(const CharType* src, size_t size) : temporary_buffer(size) { + std::copy_n(src, size, _buffer); + } + void operator=(const temporary_buffer&) = delete; + /// Moves a \c temporary_buffer. + temporary_buffer& operator=(temporary_buffer&& x) noexcept { + if (this != &x) { + _buffer = x._buffer; + _size = x._size; + _deleter = std::move(x._deleter); + x._buffer = nullptr; + x._size = 0; + } + return *this; + } + /// Gets a pointer to the beginning of the buffer. + const CharType* get() const noexcept { return _buffer; } + /// Gets a writable pointer to the beginning of the buffer. Use only + /// when you are certain no user expects the buffer data not to change. + CharType* get_write() noexcept { return _buffer; } + /// Gets the buffer size. + size_t size() const noexcept { return _size; } + /// Gets a pointer to the beginning of the buffer. + const CharType* begin() const noexcept { return _buffer; } + /// Gets a pointer to the end of the buffer. + const CharType* end() const noexcept { return _buffer + _size; } + /// Returns the buffer, but with a reduced size. The original + /// buffer is consumed by this call and can no longer be used. + /// + /// \param size New size; must be smaller than current size. + /// \return the same buffer, with a prefix removed. + temporary_buffer prefix(size_t size) && noexcept { + auto ret = std::move(*this); + ret._size = size; + return ret; + } + /// Reads a character from a specific position in the buffer. + /// + /// \param pos position to read character from; must be less than size. + CharType operator[](size_t pos) const noexcept { + return _buffer[pos]; + } + /// Checks whether the buffer is empty. + bool empty() const noexcept { return !size(); } + /// Checks whether the buffer is not empty. + explicit operator bool() const noexcept { return size(); } + /// Create a new \c temporary_buffer object referring to the same + /// underlying data. The underlying \ref deleter will not be destroyed + /// until both the original and the clone have been destroyed. + /// + /// \return a clone of the buffer object. + temporary_buffer share() { + return temporary_buffer(_buffer, _size, _deleter.share()); + } + /// Create a new \c temporary_buffer object referring to a substring of the + /// same underlying data. The underlying \ref deleter will not be destroyed + /// until both the original and the clone have been destroyed. + /// + /// \param pos Position of the first character to share. + /// \param len Length of substring to share. + /// \return a clone of the buffer object, referring to a substring. + temporary_buffer share(size_t pos, size_t len) { + auto ret = share(); + ret._buffer += pos; + ret._size = len; + return ret; + } + /// Clone the current \c temporary_buffer object into a new one. + /// This creates a temporary buffer with the same length and data but not + /// pointing to the memory of the original object. + temporary_buffer clone() const { + return {_buffer, _size}; + } + /// Remove a prefix from the buffer. The underlying data + /// is not modified. + /// + /// \param pos Position of first character to retain. + void trim_front(size_t pos) noexcept { + _buffer += pos; + _size -= pos; + } + /// Remove a suffix from the buffer. The underlying data + /// is not modified. + /// + /// \param pos Position of first character to drop. + void trim(size_t pos) noexcept { + _size = pos; + } + /// Stops automatic memory management. When the \c temporary_buffer + /// object is destroyed, the underlying \ref deleter will not be called. + /// Instead, it is the caller's responsibility to destroy the deleter object + /// when the data is no longer needed. + /// + /// \return \ref deleter object managing the data's lifetime. + deleter release() noexcept { + return std::move(_deleter); + } + /// Creates a \c temporary_buffer object with a specified size, with + /// memory aligned to a specific boundary. + /// + /// \param alignment Required alignment; must be a power of two and a multiple of sizeof(void *). + /// \param size Required size; must be a multiple of alignment. + /// \return a new \c temporary_buffer object. + static temporary_buffer aligned(size_t alignment, size_t size) { + void *ptr = nullptr; + auto ret = ::posix_memalign(&ptr, alignment, size * sizeof(CharType)); + auto buf = static_cast<CharType*>(ptr); + if (ret) { + throw std::bad_alloc(); + } + return temporary_buffer(buf, size, make_free_deleter(buf)); + } + + static temporary_buffer copy_of(std::string_view view) { + void* ptr = ::malloc(view.size()); + if (!ptr) { + throw std::bad_alloc(); + } + auto buf = static_cast<CharType*>(ptr); + memcpy(buf, view.data(), view.size()); + return temporary_buffer(buf, view.size(), make_free_deleter(buf)); + } + + /// Compare contents of this buffer with another buffer for equality + /// + /// \param o buffer to compare with + /// \return true if and only if contents are the same + bool operator==(const temporary_buffer& o) const noexcept { + return size() == o.size() && std::equal(begin(), end(), o.begin()); + } + + /// Compare contents of this buffer with another buffer for inequality + /// + /// \param o buffer to compare with + /// \return true if and only if contents are not the same + bool operator!=(const temporary_buffer& o) const noexcept { + return !(*this == o); + } +}; + +/// @} + +} diff --git a/src/seastar/include/seastar/core/thread.hh b/src/seastar/include/seastar/core/thread.hh new file mode 100644 index 000000000..c503f2b25 --- /dev/null +++ b/src/seastar/include/seastar/core/thread.hh @@ -0,0 +1,285 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/core/thread_impl.hh> +#include <seastar/core/future.hh> +#include <seastar/core/do_with.hh> +#include <seastar/core/timer.hh> +#include <seastar/core/scheduling.hh> +#include <memory> +#include <setjmp.h> +#include <type_traits> +#include <chrono> +#include <seastar/util/std-compat.hh> +#include <ucontext.h> +#include <boost/intrusive/list.hpp> + +/// \defgroup thread-module Seastar threads +/// +/// Seastar threads provide an execution environment where blocking +/// is tolerated; you can issue I/O, and wait for it in the same function, +/// rather then establishing a callback to be called with \ref future<>::then(). +/// +/// Seastar threads are not the same as operating system threads: +/// - seastar threads are cooperative; they are never preempted except +/// at blocking points (see below) +/// - seastar threads always run on the same core they were launched on +/// +/// Like other seastar code, seastar threads may not issue blocking system calls. +/// +/// A seastar thread blocking point is any function that returns a \ref future. +/// you block by calling \ref future<>::get(); this waits for the future to become +/// available, and in the meanwhile, other seastar threads and seastar non-threaded +/// code may execute. +/// +/// Example: +/// \code +/// seastar::thread th([] { +/// sleep(5s).get(); // blocking point +/// }); +/// \endcode +/// +/// An easy way to launch a thread and carry out some computation, and return a +/// result from this execution is by using the \ref seastar::async() function. +/// The result is returned as a future, so that non-threaded code can wait for +/// the thread to terminate and yield a result. + +/// Seastar API namespace +namespace seastar { + +/// \addtogroup thread-module +/// @{ + +class thread; +class thread_attributes; + +/// Class that holds attributes controling the behavior of a thread. +class thread_attributes { +public: + std::optional<seastar::scheduling_group> sched_group; + // For stack_size 0, a default value will be used (128KiB when writing this comment) + size_t stack_size = 0; +}; + + +/// \cond internal +extern thread_local jmp_buf_link g_unthreaded_context; + +// Internal class holding thread state. We can't hold this in +// \c thread itself because \c thread is movable, and we want pointers +// to this state to be captured. +class thread_context final : private task { + struct stack_deleter { + void operator()(char *ptr) const noexcept; + int valgrind_id; + stack_deleter(int valgrind_id); + }; + using stack_holder = std::unique_ptr<char[], stack_deleter>; + + stack_holder _stack; + noncopyable_function<void ()> _func; + jmp_buf_link _context; + promise<> _done; + bool _joined = false; + + boost::intrusive::list_member_hook<> _all_link; + using all_thread_list = boost::intrusive::list<thread_context, + boost::intrusive::member_hook<thread_context, boost::intrusive::list_member_hook<>, + &thread_context::_all_link>, + boost::intrusive::constant_time_size<false>>; + + static thread_local all_thread_list _all_threads; +private: + static void s_main(int lo, int hi); // all parameters MUST be 'int' for makecontext + void setup(size_t stack_size); + void main(); + stack_holder make_stack(size_t stack_size); + virtual void run_and_dispose() noexcept override; // from task class +public: + thread_context(thread_attributes attr, noncopyable_function<void ()> func); + ~thread_context(); + void switch_in(); + void switch_out(); + bool should_yield() const; + void reschedule(); + void yield(); + task* waiting_task() noexcept override { return _done.waiting_task(); } + friend class thread; + friend void thread_impl::switch_in(thread_context*); + friend void thread_impl::switch_out(thread_context*); + friend scheduling_group thread_impl::sched_group(const thread_context*); +}; + +/// \endcond + + +/// \brief thread - stateful thread of execution +/// +/// Threads allow using seastar APIs in a blocking manner, +/// by calling future::get() on a non-ready future. When +/// this happens, the thread is put to sleep until the future +/// becomes ready. +class thread { + std::unique_ptr<thread_context> _context; + static thread_local thread* _current; +public: + /// \brief Constructs a \c thread object that does not represent a thread + /// of execution. + thread() = default; + /// \brief Constructs a \c thread object that represents a thread of execution + /// + /// \param func Callable object to execute in thread. The callable is + /// called immediately. + template <typename Func> + thread(Func func); + /// \brief Constructs a \c thread object that represents a thread of execution + /// + /// \param attr Attributes describing the new thread. + /// \param func Callable object to execute in thread. The callable is + /// called immediately. + template <typename Func> + thread(thread_attributes attr, Func func); + /// \brief Moves a thread object. + thread(thread&& x) noexcept = default; + /// \brief Move-assigns a thread object. + thread& operator=(thread&& x) noexcept = default; + /// \brief Destroys a \c thread object. + /// + /// The thread must not represent a running thread of execution (see join()). + ~thread() { assert(!_context || _context->_joined); } + /// \brief Waits for thread execution to terminate. + /// + /// Waits for thread execution to terminate, and marks the thread object as not + /// representing a running thread of execution. + future<> join(); + /// \brief Voluntarily defer execution of current thread. + /// + /// Gives other threads/fibers a chance to run on current CPU. + /// The current thread will resume execution promptly. + static void yield(); + /// \brief Checks whether this thread ought to call yield() now. + /// + /// Useful where we cannot call yield() immediately because we + /// Need to take some cleanup action first. + static bool should_yield(); + + /// \brief Yield if this thread ought to call yield() now. + /// + /// Useful where a code does long running computation and does + /// not want to hog cpu for more then its share + static void maybe_yield(); + + static bool running_in_thread() { + return thread_impl::get() != nullptr; + } +}; + +template <typename Func> +inline +thread::thread(thread_attributes attr, Func func) + : _context(std::make_unique<thread_context>(std::move(attr), std::move(func))) { +} + +template <typename Func> +inline +thread::thread(Func func) + : thread(thread_attributes(), std::move(func)) { +} + +inline +future<> +thread::join() { + _context->_joined = true; + return _context->_done.get_future(); +} + +/// Executes a callable in a seastar thread. +/// +/// Runs a block of code in a threaded context, +/// which allows it to block (using \ref future::get()). The +/// result of the callable is returned as a future. +/// +/// \param attr a \ref thread_attributes instance +/// \param func a callable to be executed in a thread +/// \param args a parameter pack to be forwarded to \c func. +/// \return whatever \c func returns, as a future. +/// +/// Example: +/// \code +/// future<int> compute_sum(int a, int b) { +/// thread_attributes attr = {}; +/// attr.sched_group = some_scheduling_group_ptr; +/// return seastar::async(attr, [a, b] { +/// // some blocking code: +/// sleep(1s).get(); +/// return a + b; +/// }); +/// } +/// \endcode +template <typename Func, typename... Args> +inline +futurize_t<std::invoke_result_t<Func, Args...>> +async(thread_attributes attr, Func&& func, Args&&... args) noexcept { + using return_type = std::invoke_result_t<Func, Args...>; + struct work { + thread_attributes attr; + Func func; + std::tuple<Args...> args; + promise<return_type> pr{}; + thread th{}; + }; + + try { + auto wp = std::make_unique<work>(work{std::move(attr), std::forward<Func>(func), std::forward_as_tuple(std::forward<Args>(args)...)}); + auto& w = *wp; + auto ret = w.pr.get_future(); + w.th = thread(std::move(w.attr), [&w] { + futurize<return_type>::apply(std::move(w.func), std::move(w.args)).forward_to(std::move(w.pr)); + }); + return w.th.join().then([ret = std::move(ret)] () mutable { + return std::move(ret); + }).finally([wp = std::move(wp)] {}); + } catch (...) { + return futurize<return_type>::make_exception_future(std::current_exception()); + } +} + +/// Executes a callable in a seastar thread. +/// +/// Runs a block of code in a threaded context, +/// which allows it to block (using \ref future::get()). The +/// result of the callable is returned as a future. +/// +/// \param func a callable to be executed in a thread +/// \param args a parameter pack to be forwarded to \c func. +/// \return whatever \c func returns, as a future. +template <typename Func, typename... Args> +inline +futurize_t<std::invoke_result_t<Func, Args...>> +async(Func&& func, Args&&... args) noexcept { + return async(thread_attributes{}, std::forward<Func>(func), std::forward<Args>(args)...); +} +/// @} + +} diff --git a/src/seastar/include/seastar/core/thread_cputime_clock.hh b/src/seastar/include/seastar/core/thread_cputime_clock.hh new file mode 100644 index 000000000..9e905397b --- /dev/null +++ b/src/seastar/include/seastar/core/thread_cputime_clock.hh @@ -0,0 +1,48 @@ +/* + * 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 + +#include <chrono> +#include <time.h> +#include <cassert> + +namespace seastar { + +class thread_cputime_clock { +public: + using rep = int64_t; + using period = std::chrono::nanoseconds::period; + using duration = std::chrono::duration<rep, period>; + using time_point = std::chrono::time_point<thread_cputime_clock, duration>; +public: + static time_point now() { + using namespace std::chrono_literals; + + struct timespec tp; + [[gnu::unused]] auto ret = clock_gettime(CLOCK_THREAD_CPUTIME_ID, &tp); + assert(ret == 0); + return time_point(tp.tv_nsec * 1ns + tp.tv_sec * 1s); + } +}; + +} + diff --git a/src/seastar/include/seastar/core/thread_impl.hh b/src/seastar/include/seastar/core/thread_impl.hh new file mode 100644 index 000000000..214872406 --- /dev/null +++ b/src/seastar/include/seastar/core/thread_impl.hh @@ -0,0 +1,84 @@ +/* + * 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) 2016 ScyllaDB. + */ + +#pragma once +#include <seastar/core/preempt.hh> +#include <setjmp.h> +#include <ucontext.h> +#include <chrono> +#include <seastar/util/std-compat.hh> + +namespace seastar { +/// Clock used for scheduling threads +using thread_clock = std::chrono::steady_clock; + +/// \cond internal +class thread_context; +class scheduling_group; + +struct jmp_buf_link { +#ifdef SEASTAR_ASAN_ENABLED + ucontext_t context; + void* fake_stack = nullptr; + const void* stack_bottom; + size_t stack_size; +#else + jmp_buf jmpbuf; +#endif + jmp_buf_link* link; + thread_context* thread; +public: + void initial_switch_in(ucontext_t* initial_context, const void* stack_bottom, size_t stack_size); + void switch_in(); + void switch_out(); + void initial_switch_in_completed(); + void final_switch_out(); +}; + +extern thread_local jmp_buf_link* g_current_context; + +namespace thread_impl { + +inline thread_context* get() { + return g_current_context->thread; +} + +inline bool should_yield() { + if (need_preempt()) { + return true; + } else { + return false; + } +} + +scheduling_group sched_group(const thread_context*); + +void yield(); +void switch_in(thread_context* to); +void switch_out(thread_context* from); +void init(); + +} +} +/// \endcond + + diff --git a/src/seastar/include/seastar/core/timed_out_error.hh b/src/seastar/include/seastar/core/timed_out_error.hh new file mode 100644 index 000000000..48d878b92 --- /dev/null +++ b/src/seastar/include/seastar/core/timed_out_error.hh @@ -0,0 +1,42 @@ +/* + * 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 2020 ScyllaDB + */ + +#pragma once + +#include <exception> + +namespace seastar { + +class timed_out_error : public std::exception { +public: + virtual const char* what() const noexcept { + return "timedout"; + } +}; + +struct default_timeout_exception_factory { + static auto timeout() { + return timed_out_error(); + } +}; + +} // namespace seastar diff --git a/src/seastar/include/seastar/core/timer-set.hh b/src/seastar/include/seastar/core/timer-set.hh new file mode 100644 index 000000000..5032eb686 --- /dev/null +++ b/src/seastar/include/seastar/core/timer-set.hh @@ -0,0 +1,253 @@ +/* + * Copyright (C) 2014 Cloudius Systems, Ltd. + */ + +/* + * Imported from OSv: + * + * Copyright (C) 2014 Cloudius Systems, Ltd. + * + * This work is open source software, licensed under the terms of the + * BSD license as described in the LICENSE file in the top-level directory. + */ + +#pragma once + +#include <chrono> +#include <limits> +#include <bitset> +#include <array> +#include <boost/intrusive/list.hpp> +#include <seastar/core/bitset-iter.hh> + +namespace seastar { + +/** + * A data structure designed for holding and expiring timers. It's + * optimized for timer non-delivery by deferring sorting cost until + * expiry time. The optimization is based on the observation that in + * many workloads timers are cancelled or rescheduled before they + * expire. That's especially the case for TCP timers. + * + * The template type "Timer" should have a method named + * get_timeout() which returns Timer::time_point which denotes + * timer's expiration. + */ +template<typename Timer, boost::intrusive::list_member_hook<> Timer::*link> +class timer_set { +public: + using time_point = typename Timer::time_point; + using timer_list_t = boost::intrusive::list<Timer, boost::intrusive::member_hook<Timer, boost::intrusive::list_member_hook<>, link>>; +private: + using duration = typename Timer::duration; + using timestamp_t = typename Timer::duration::rep; + + static constexpr timestamp_t max_timestamp = std::numeric_limits<timestamp_t>::max(); + static constexpr int timestamp_bits = std::numeric_limits<timestamp_t>::digits; + + // The last bucket is reserved for active timers with timeout <= _last. + static constexpr int n_buckets = timestamp_bits + 1; + + std::array<timer_list_t, n_buckets> _buckets; + timestamp_t _last; + timestamp_t _next; + + std::bitset<n_buckets> _non_empty_buckets; +private: + static timestamp_t get_timestamp(time_point _time_point) noexcept + { + return _time_point.time_since_epoch().count(); + } + + static timestamp_t get_timestamp(Timer& timer) noexcept + { + return get_timestamp(timer.get_timeout()); + } + + int get_index(timestamp_t timestamp) const noexcept + { + if (timestamp <= _last) { + return n_buckets - 1; + } + + auto index = bitsets::count_leading_zeros(timestamp ^ _last); + assert(index < n_buckets - 1); + return index; + } + + int get_index(Timer& timer) const noexcept + { + return get_index(get_timestamp(timer)); + } + + int get_last_non_empty_bucket() const noexcept + { + return bitsets::get_last_set(_non_empty_buckets); + } +public: + timer_set() noexcept + : _last(0) + , _next(max_timestamp) + , _non_empty_buckets(0) + { + } + + ~timer_set() { + for (auto&& list : _buckets) { + while (!list.empty()) { + auto& timer = *list.begin(); + timer.cancel(); + } + } + } + + /** + * Adds timer to the active set. + * + * The value returned by timer.get_timeout() is used as timer's expiry. The result + * of timer.get_timeout() must not change while the timer is in the active set. + * + * Preconditions: + * - this timer must not be currently in the active set or in the expired set. + * + * Postconditions: + * - this timer will be added to the active set until it is expired + * by a call to expire() or removed by a call to remove(). + * + * Returns true if and only if this timer's timeout is less than get_next_timeout(). + * When this function returns true the caller should reschedule expire() to be + * called at timer.get_timeout() to ensure timers are expired in a timely manner. + */ + bool insert(Timer& timer) noexcept + { + auto timestamp = get_timestamp(timer); + auto index = get_index(timestamp); + + _buckets[index].push_back(timer); + _non_empty_buckets[index] = true; + + if (timestamp < _next) { + _next = timestamp; + return true; + } + return false; + } + + /** + * Removes timer from the active set. + * + * Preconditions: + * - timer must be currently in the active set. Note: it must not be in + * the expired set. + * + * Postconditions: + * - timer is no longer in the active set. + * - this object will no longer hold any references to this timer. + */ + void remove(Timer& timer) noexcept + { + auto index = get_index(timer); + auto& list = _buckets[index]; + list.erase(list.iterator_to(timer)); + if (list.empty()) { + _non_empty_buckets[index] = false; + } + } + + /** + * Expires active timers. + * + * The time points passed to this function must be monotonically increasing. + * Use get_next_timeout() to query for the next time point. + * + * Preconditions: + * - the time_point passed to this function must not be lesser than + * the previous one passed to this function. + * + * Postconditons: + * - all timers from the active set with Timer::get_timeout() <= now are moved + * to the expired set. + */ + timer_list_t expire(time_point now) noexcept + { + timer_list_t exp; + auto timestamp = get_timestamp(now); + + if (timestamp < _last) { + abort(); + } + + auto index = get_index(timestamp); + + for (int i : bitsets::for_each_set(_non_empty_buckets, index + 1)) { + exp.splice(exp.end(), _buckets[i]); + _non_empty_buckets[i] = false; + } + + _last = timestamp; + _next = max_timestamp; + + auto& list = _buckets[index]; + while (!list.empty()) { + auto& timer = *list.begin(); + list.pop_front(); + if (timer.get_timeout() <= now) { + exp.push_back(timer); + } else { + insert(timer); + } + } + + _non_empty_buckets[index] = !list.empty(); + + if (_next == max_timestamp && _non_empty_buckets.any()) { + for (auto& timer : _buckets[get_last_non_empty_bucket()]) { + _next = std::min(_next, get_timestamp(timer)); + } + } + return exp; + } + + /** + * Returns a time point at which expire() should be called + * in order to ensure timers are expired in a timely manner. + * + * Returned values are monotonically increasing. + */ + time_point get_next_timeout() const noexcept + { + return time_point(duration(std::max(_last, _next))); + } + + /** + * Clears both active and expired timer sets. + */ + void clear() noexcept + { + for (int i : bitsets::for_each_set(_non_empty_buckets)) { + _buckets[i].clear(); + } + } + + size_t size() const noexcept + { + size_t res = 0; + for (int i : bitsets::for_each_set(_non_empty_buckets)) { + res += _buckets[i].size(); + } + return res; + } + + /** + * Returns true if and only if there are no timers in the active set. + */ + bool empty() const noexcept + { + return _non_empty_buckets.none(); + } + + time_point now() noexcept { + return Timer::clock::now(); + } +}; +}; diff --git a/src/seastar/include/seastar/core/timer.hh b/src/seastar/include/seastar/core/timer.hh new file mode 100644 index 000000000..39bf0c272 --- /dev/null +++ b/src/seastar/include/seastar/core/timer.hh @@ -0,0 +1,225 @@ +/* + * 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 2015 Cloudius Systems + */ + +#pragma once + +#include <chrono> +#include <seastar/util/std-compat.hh> +#include <atomic> +#include <functional> +#include <seastar/core/future.hh> +#include <seastar/core/timer-set.hh> +#include <seastar/core/scheduling.hh> + +/// \file + +/// \defgroup timers Timers +/// +/// Seastar provides timers that can be defined to run a callback at a certain +/// time point in the future; timers are provided for \ref lowres_clock (10ms +/// resolution, efficient), for std::chrono::steady_clock (accurate but less +/// efficient) and for \ref manual_clock (for testing purposes). +/// +/// Timers are optimized for cancellation; that is, adding a timer and cancelling +/// it is very efficient. This means that attaching a timer per object for +/// a timeout that rarely happens is reasonable; one does not have to maintain +/// a single timer and a sorted list for this use case. +/// +/// Timer callbacks should be short and execute quickly. If involved processing +/// is required, a timer can launch a continuation. + +namespace seastar { + +using steady_clock_type = std::chrono::steady_clock; + + +/// \addtogroup timers +/// @{ + +/// Timer - run a callback at a certain time point in the future. +/// +/// Timer callbacks should execute quickly. If more involved computation +/// is required, the timer should launch it as a fiber (or signal an +/// existing fiber to continue execution). Fibers launched from a timer +/// callback are executed under the scheduling group that was current +/// when the timer was created (see current_scheduling_group()), or the +/// scheduling that was given explicitly by the caller when the callback +/// was specified. +/// +/// Expiration of a `timer<std::chrono::steady_clock>` is independent of +/// task_quota, so it has relatively high accuracy, but as a result this +/// is a relatively expensive timer. It is recommended to use `timer<lowres_clock>` +/// instead, which has very coarse resolution (~10ms) but is quite efficient. +/// It is suitable for most user timeouts. +/// +/// \tparam Clock type of clock used to denote time points; can be +/// std::chrono::steady_clock_type (default), lowres_clock (more efficient +/// but with less resolution) and manual_clock_type (fine-grained control +/// for testing. +template <typename Clock = steady_clock_type> +class timer { +public: + typedef typename Clock::time_point time_point; + typedef typename Clock::duration duration; + typedef Clock clock; +private: + using callback_t = noncopyable_function<void()>; + boost::intrusive::list_member_hook<> _link; + scheduling_group _sg; + callback_t _callback; + time_point _expiry; + std::optional<duration> _period; + bool _armed = false; + bool _queued = false; + bool _expired = false; + void readd_periodic() noexcept; + void arm_state(time_point until, std::optional<duration> period) noexcept { + assert(!_armed); + _period = period; + _armed = true; + _expired = false; + _expiry = until; + _queued = true; + } +public: + /// Constructs a timer with no callback set and no expiration time. + timer() noexcept {}; // implementation is required (instead of = default) for noexcept due to a bug in gcc 9.3.1, + // since boost::intrusive::list_member_hook default constructor is not specified as noexcept. + /// Constructs a timer from another timer that is moved from. + /// + /// \note care should be taken when moving a timer whose callback captures `this`, + /// since the object pointed to by `this` may have been moved as well. + timer(timer&& t) noexcept : _sg(t._sg), _callback(std::move(t._callback)), _expiry(std::move(t._expiry)), _period(std::move(t._period)), + _armed(t._armed), _queued(t._queued), _expired(t._expired) { + _link.swap_nodes(t._link); + t._queued = false; + t._armed = false; + } + /// Constructs a timer with a callback. The timer is not armed. + /// + /// \param sg Scheduling group to run the callback under. + /// \param callback function (with signature `void ()`) to execute after the timer is armed and expired. + timer(scheduling_group sg, noncopyable_function<void ()>&& callback) noexcept : _sg(sg), _callback{std::move(callback)} { + } + /// Constructs a timer with a callback. The timer is not armed. + /// + /// \param callback function (with signature `void ()`) to execute after the timer is armed and expired. + explicit timer(noncopyable_function<void ()>&& callback) noexcept : timer(current_scheduling_group(), std::move(callback)) { + } + /// Destroys the timer. The timer is cancelled if armed. + ~timer(); + /// Sets the callback function to be called when the timer expires. + /// + /// \param sg the scheduling group under which the callback will be executed. + /// \param callback the callback to be executed when the timer expires. + void set_callback(scheduling_group sg, noncopyable_function<void ()>&& callback) noexcept { + _sg = sg; + _callback = std::move(callback); + } + /// Sets the callback function to be called when the timer expires. + /// + /// \param callback the callback to be executed when the timer expires. + void set_callback(noncopyable_function<void ()>&& callback) noexcept { + set_callback(current_scheduling_group(), std::move(callback)); + } + /// Sets the timer expiration time. + /// + /// It is illegal to arm a timer that has already been armed (and + /// not disarmed by expiration or cancel()). In the current + /// implementation, this will result in an assertion failure. See + /// rearm(). + /// + /// \param until the time when the timer expires + /// \param period optional automatic rearm duration; if given the timer + /// will automatically rearm itself when it expires, using the period + /// to calculate the next expiration time. + void arm(time_point until, std::optional<duration> period = {}) noexcept; + /// Sets the timer expiration time. If the timer was already armed, it is + /// canceled first. + /// + /// \param until the time when the timer expires + /// \param period optional automatic rearm duration; if given the timer + /// will automatically rearm itself when it expires, using the period + /// to calculate the next expiration time. + void rearm(time_point until, std::optional<duration> period = {}) noexcept { + if (_armed) { + cancel(); + } + arm(until, period); + } + /// Sets the timer expiration time. + /// + /// It is illegal to arm a timer that has already been armed (and + /// not disarmed by expiration or cancel()). In the current + /// implementation, this will result in an assertion failure. See + /// rearm(). + /// + /// \param delta the time when the timer expires, relative to now + void arm(duration delta) noexcept { + return arm(Clock::now() + delta); + } + /// Sets the timer expiration time, with automatic rearming + /// + /// \param delta the time when the timer expires, relative to now. The timer + /// will also rearm automatically using the same delta time. + void arm_periodic(duration delta) noexcept { + arm(Clock::now() + delta, {delta}); + } + /// Sets the timer expiration time, with automatic rearming. + /// If the timer was already armed, it is canceled first. + /// + /// \param delta the time when the timer expires, relative to now. The timer + /// will also rearm automatically using the same delta time. + void rearm_periodic(duration delta) noexcept { + if (_armed) { + cancel(); + } + arm_periodic(delta); + } + /// Returns whether the timer is armed + /// + /// \return `true` if the timer is armed and has not expired yet. + bool armed() const noexcept { return _armed; } + /// Cancels an armed timer. + /// + /// If the timer was armed, it is disarmed. If the timer was not + /// armed, does nothing. + /// + /// \return `true` if the timer was armed before the call. + bool cancel() noexcept; + /// Gets the expiration time of an armed timer. + /// + /// \return the time at which the timer is scheduled to expire (undefined if the + /// timer is not armed). + time_point get_timeout() const noexcept { + return _expiry; + } + friend class reactor; + friend class timer_set<timer, &timer::_link>; +}; + +extern template class timer<steady_clock_type>; + + +/// @} + +} + diff --git a/src/seastar/include/seastar/core/transfer.hh b/src/seastar/include/seastar/core/transfer.hh new file mode 100644 index 000000000..eb24db406 --- /dev/null +++ b/src/seastar/include/seastar/core/transfer.hh @@ -0,0 +1,75 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +// Helper functions for copying or moving multiple objects in an exception +// safe manner, then destroying the sources. +// +// To transfer, call transfer_pass1(allocator, &from, &to) on all object pairs, +// (this copies the object from @from to @to). If no exceptions are encountered, +// call transfer_pass2(allocator, &from, &to). This destroys the object at the +// origin. If exceptions were encountered, simply destroy all copied objects. +// +// As an optimization, if the objects are moveable without throwing (noexcept) +// transfer_pass1() simply moves the objects and destroys the source, and +// transfer_pass2() does nothing. + +#include <memory> +#include <type_traits> +#include <utility> + +namespace seastar { + +template <typename T, typename Alloc> +inline +void +transfer_pass1(Alloc& a, T* from, T* to, + typename std::enable_if<std::is_nothrow_move_constructible<T>::value>::type* = nullptr) { + std::allocator_traits<Alloc>::construct(a, to, std::move(*from)); + std::allocator_traits<Alloc>::destroy(a, from); +} + +template <typename T, typename Alloc> +inline +void +transfer_pass2(Alloc&, T*, T*, + typename std::enable_if<std::is_nothrow_move_constructible<T>::value>::type* = nullptr) { +} + +template <typename T, typename Alloc> +inline +void +transfer_pass1(Alloc& a, T* from, T* to, + typename std::enable_if<!std::is_nothrow_move_constructible<T>::value>::type* = nullptr) { + std::allocator_traits<Alloc>::construct(a, to, *from); +} + +template <typename T, typename Alloc> +inline +void +transfer_pass2(Alloc& a, T* from, T*, + typename std::enable_if<!std::is_nothrow_move_constructible<T>::value>::type* = nullptr) { + std::allocator_traits<Alloc>::destroy(a, from); +} + +} + diff --git a/src/seastar/include/seastar/core/unaligned.hh b/src/seastar/include/seastar/core/unaligned.hh new file mode 100644 index 000000000..39868ab89 --- /dev/null +++ b/src/seastar/include/seastar/core/unaligned.hh @@ -0,0 +1,78 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ + +#pragma once + +// The following unaligned_cast<T*>(p) is a portable replacement for +// reinterpret_cast<T*>(p) which should be used every time address p +// is not guaranteed to be properly aligned to alignof(T). +// +// On architectures like x86 and ARM, where unaligned access is allowed, +// unaligned_cast will behave the same as reinterpret_cast and will generate +// the same code. +// +// Certain architectures (e.g., MIPS) make it extremely slow or outright +// forbidden to use ordinary machine instructions on a primitive type at an +// unaligned addresses - e.g., access a uint32_t at an address which is not +// a multiple of 4. Gcc's "undefined behavior sanitizer" (enabled in our debug +// build) also catches such unaligned accesses and reports them as errors, +// even when running on x86. +// +// Therefore, reinterpret_cast<int32_t*> on an address which is not guaranteed +// to be a multiple of 4 may generate extremely slow code or runtime errors, +// and must be avoided. The compiler needs to be told about the unaligned +// access, so it can generate reasonably-efficient code for the access +// (in MIPS, this means generating two instructions "lwl" and "lwr", instead +// of the one instruction "lw" which faults on unaligned/ access). The way to +// tell the compiler this is with __attribute__((packed)). This will also +// cause the sanitizer not to generate runtime alignment checks for this +// access. + +#include <type_traits> + +namespace seastar { + +template <typename T> +struct unaligned { + // This is made to support only simple types, so it is fine to + // require them to be trivially copy constructible. + static_assert(std::is_trivially_copy_constructible_v<T>); + T raw; + unaligned() noexcept = default; + unaligned(T x) noexcept : raw(x) {} + unaligned& operator=(const T& x) noexcept { raw = x; return *this; } + operator T() const noexcept { return raw; } +} __attribute__((packed)); + + +template <typename T, typename F> +[[deprecated("violates strict aliasing rules. See issue #165.")]] +inline auto unaligned_cast(F* p) noexcept { + return reinterpret_cast<unaligned<std::remove_pointer_t<T>>*>(p); +} + +template <typename T, typename F> +[[deprecated("violates strict aliasing rules. See issue #165.")]] +inline auto unaligned_cast(const F* p) noexcept { + return reinterpret_cast<const unaligned<std::remove_pointer_t<T>>*>(p); +} + +} diff --git a/src/seastar/include/seastar/core/units.hh b/src/seastar/include/seastar/core/units.hh new file mode 100644 index 000000000..00d410628 --- /dev/null +++ b/src/seastar/include/seastar/core/units.hh @@ -0,0 +1,37 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <cstddef> + +namespace seastar { + +static constexpr size_t KB = 1 << 10; +static constexpr size_t MB = 1 << 20; +static constexpr size_t GB = 1 << 30; + +constexpr size_t operator"" _KiB(unsigned long long n) { return n << 10; } +constexpr size_t operator"" _MiB(unsigned long long n) { return n << 20; } +constexpr size_t operator"" _GiB(unsigned long long n) { return n << 30; } +constexpr size_t operator"" _TiB(unsigned long long n) { return n << 40; } + +} diff --git a/src/seastar/include/seastar/core/vector-data-sink.hh b/src/seastar/include/seastar/core/vector-data-sink.hh new file mode 100644 index 000000000..4a39103ff --- /dev/null +++ b/src/seastar/include/seastar/core/vector-data-sink.hh @@ -0,0 +1,47 @@ +/* + * 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 2014 Cloudius Systems + */ + +#pragma once + +#include <seastar/core/iostream.hh> + +namespace seastar { + +class vector_data_sink final : public data_sink_impl { +public: + using vector_type = std::vector<net::packet>; +private: + vector_type& _v; +public: + vector_data_sink(vector_type& v) : _v(v) {} + + virtual future<> put(net::packet p) override { + _v.push_back(std::move(p)); + return make_ready_future<>(); + } + + virtual future<> close() override { + // TODO: close on local side + return make_ready_future<>(); + } +}; + +} diff --git a/src/seastar/include/seastar/core/weak_ptr.hh b/src/seastar/include/seastar/core/weak_ptr.hh new file mode 100644 index 000000000..08b4aa77e --- /dev/null +++ b/src/seastar/include/seastar/core/weak_ptr.hh @@ -0,0 +1,140 @@ +/* + * 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) 2016 ScyllaDB + */ + +#pragma once + +#include <boost/intrusive/list.hpp> + +namespace seastar { + +/// A non-owning reference to an object. +/// +/// weak_ptr allows one to keep a non-owning reference to an object. When the +/// object is destroyed, it notifies all weak_ptr instances pointing to it. +/// A weak_ptr instance pointing to a destroyed object is equivalent to a +/// `nullptr`. +/// +/// The referenced object must inherit from weakly_referencable. +/// weak_ptr instances can only be obtained by calling weak_from_this() on +/// the to-be-referenced object. +/// +/// \see weakly_referencable +template<typename T> +class weak_ptr { + template<typename U> + friend class weakly_referencable; +private: + using hook_type = boost::intrusive::list_member_hook<boost::intrusive::link_mode<boost::intrusive::auto_unlink>>; + hook_type _hook; + T* _ptr = nullptr; + weak_ptr(T* p) noexcept : _ptr(p) {} + void clear() noexcept { + _hook = {}; + _ptr = nullptr; + } + void swap(weak_ptr&& o) noexcept { + _hook.swap_nodes(o._hook); + std::swap(_ptr, o._ptr); + } +public: + // Note: The default constructor's body is implemented as no-op + // rather than `noexcept = default` due to a bug with gcc 9.3.1 + // that deletes the constructor since boost::intrusive::list_member_hook + // is not default_nothrow_constructible. + weak_ptr() noexcept {} + weak_ptr(std::nullptr_t) noexcept : weak_ptr() {} + weak_ptr(weak_ptr&& o) noexcept + { + swap(std::move(o)); + } + weak_ptr(const weak_ptr& o) noexcept { + if (o._ptr) { + swap(o._ptr->weak_from_this()); + } + } + weak_ptr& operator=(weak_ptr&& o) noexcept { + if (this != &o) { + clear(); + swap(std::move(o)); + } + return *this; + } + weak_ptr& operator=(const weak_ptr& o) noexcept { + if (this != &o) { + clear(); + if (o._ptr) { + swap(o._ptr->weak_from_this()); + } + } + return *this; + } + explicit operator bool() const noexcept { return _ptr != nullptr; } + T* operator->() const noexcept { return _ptr; } + T& operator*() const noexcept { return *_ptr; } + T* get() const noexcept { return _ptr; } + bool operator==(const weak_ptr& o) const noexcept { return _ptr == o._ptr; } + bool operator!=(const weak_ptr& o) const noexcept { return _ptr != o._ptr; } +}; + +/// Allows obtaining a non-owning reference (weak_ptr) to the object. +/// +/// A live weak_ptr object doesn't prevent the referenced object form being destroyed. +/// +/// The underlying pointer held by weak_ptr is valid as long as the referenced object is alive. +/// When the object dies, all weak_ptr objects associated with it are emptied. +/// +/// A weak reference is obtained like this: +/// +/// class X : public weakly_referencable<X> {}; +/// auto x = std::make_unique<X>(); +/// weak_ptr<X> ptr = x->weak_from_this(); +/// +/// The user of weak_ptr can check if it still holds a valid pointer like this: +/// +/// if (ptr) ptr->do_something(); +/// +template<typename T> +class weakly_referencable { + boost::intrusive::list<weak_ptr<T>, + boost::intrusive::member_hook<weak_ptr<T>, typename weak_ptr<T>::hook_type, &weak_ptr<T>::_hook>, + boost::intrusive::constant_time_size<false>> _ptr_list; +public: + // Note: The default constructor's body is implemented as no-op + // rather than `noexcept = default` due to a bug with gcc 9.3.1 + // that deletes the constructor since boost::intrusive::member_hook + // is not default_nothrow_constructible. + weakly_referencable() noexcept {} + weakly_referencable(weakly_referencable&&) = delete; // pointer to this is captured and passed to weak_ptr + weakly_referencable(const weakly_referencable&) = delete; + ~weakly_referencable() noexcept { + _ptr_list.clear_and_dispose([] (weak_ptr<T>* wp) noexcept { + wp->_ptr = nullptr; + }); + } + weak_ptr<T> weak_from_this() noexcept { + weak_ptr<T> ptr(static_cast<T*>(this)); + _ptr_list.push_back(ptr); + return ptr; + } +}; + +} + diff --git a/src/seastar/include/seastar/core/when_all.hh b/src/seastar/include/seastar/core/when_all.hh new file mode 100644 index 000000000..97eb7e5d5 --- /dev/null +++ b/src/seastar/include/seastar/core/when_all.hh @@ -0,0 +1,563 @@ +/* + * 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) 2020 ScyllaDB. + */ + +#pragma once + +#include <tuple> +#include <utility> +#include <type_traits> + +#include <seastar/core/future.hh> +#include <seastar/core/loop.hh> +#include <seastar/util/tuple_utils.hh> +#include <seastar/util/critical_alloc_section.hh> + +namespace seastar { + +/// \addtogroup future-util +/// @{ + +namespace internal { + +template<typename... Futures> +struct identity_futures_tuple { + using future_type = future<std::tuple<Futures...>>; + using promise_type = typename future_type::promise_type; + + static void set_promise(promise_type& p, std::tuple<Futures...> futures) { + p.set_value(std::move(futures)); + } + + static future_type make_ready_future(std::tuple<Futures...> futures) noexcept { + return seastar::make_ready_future<std::tuple<Futures...>>(std::move(futures)); + } + + static future_type current_exception_as_future() noexcept { + return seastar::current_exception_as_future<std::tuple<Futures...>>(); + } +}; + +class when_all_state_base; + +// If the future is ready, return true +// if the future is not ready, chain a continuation to it, and return false +using when_all_process_element_func = bool (*)(void* future, void* continuation, when_all_state_base* wasb) noexcept; + +struct when_all_process_element { + when_all_process_element_func func; + void* future; +}; + +class when_all_state_base { + size_t _nr_remain; + const when_all_process_element* _processors; + void* _continuation; +public: + virtual ~when_all_state_base() {} + when_all_state_base(size_t nr_remain, const when_all_process_element* processors, void* continuation) + : _nr_remain(nr_remain), _processors(processors), _continuation(continuation) { + } + virtual task* waiting_task() = 0; + void complete_one() noexcept { + // We complete in reverse order; if the futures happen to complete + // in order, then waiting for the last one will find the rest ready + --_nr_remain; + while (_nr_remain) { + bool ready = process_one(_nr_remain - 1); + if (!ready) { + return; + } + --_nr_remain; + } + if (!_nr_remain) { + delete this; + } + } + void do_wait_all() noexcept { + ++_nr_remain; // fake pending completion for complete_one() + complete_one(); + } + bool process_one(size_t idx) noexcept { + auto p = _processors[idx]; + return p.func(p.future, _continuation, this); + } +}; + +template <typename Future> +class when_all_state_component final : public continuation_base_from_future_t<Future> { + when_all_state_base* _base; + Future* _final_resting_place; +public: + static bool process_element_func(void* future, void* continuation, when_all_state_base* wasb) noexcept { + auto f = reinterpret_cast<Future*>(future); + if (f->available()) { + return true; + } else { + auto c = new (continuation) when_all_state_component(wasb, f); + set_callback(std::move(*f), c); + return false; + } + } + when_all_state_component(when_all_state_base *base, Future* future) noexcept : _base(base), _final_resting_place(future) {} + task* waiting_task() noexcept override { return _base->waiting_task(); } + virtual void run_and_dispose() noexcept override { + using futurator = futurize<Future>; + if (__builtin_expect(this->_state.failed(), false)) { + *_final_resting_place = futurator::make_exception_future(std::move(this->_state).get_exception()); + } else { + *_final_resting_place = futurator::from_tuple(std::move(this->_state).get_value()); + } + auto base = _base; + this->~when_all_state_component(); + base->complete_one(); + } +}; + +template<typename ResolvedTupleTransform, typename... Futures> +class when_all_state : public when_all_state_base { + static constexpr size_t nr = sizeof...(Futures); + using type = std::tuple<Futures...>; + type tuple; + // We only schedule one continuation at a time, and store it in _cont. + // This way, if while the future we wait for completes, some other futures + // also complete, we won't need to schedule continuations for them. + std::aligned_union_t<1, when_all_state_component<Futures>...> _cont; + when_all_process_element _processors[nr]; +public: + typename ResolvedTupleTransform::promise_type p; + when_all_state(Futures&&... t) : when_all_state_base(nr, _processors, &_cont), tuple(std::make_tuple(std::move(t)...)) { + init_element_processors(std::make_index_sequence<nr>()); + } + virtual ~when_all_state() { + ResolvedTupleTransform::set_promise(p, std::move(tuple)); + } + task* waiting_task() noexcept override { + return p.waiting_task(); + } +private: + template <size_t... Idx> + void init_element_processors(std::index_sequence<Idx...>) { + auto ignore = { + 0, + (_processors[Idx] = when_all_process_element{ + when_all_state_component<std::tuple_element_t<Idx, type>>::process_element_func, + &std::get<Idx>(tuple) + }, 0)... + }; + (void)ignore; + } +public: + static typename ResolvedTupleTransform::future_type wait_all(Futures&&... futures) noexcept { + if ((futures.available() && ...)) { + return ResolvedTupleTransform::make_ready_future(std::make_tuple(std::move(futures)...)); + } + auto state = [&] () noexcept { + memory::scoped_critical_alloc_section _; + return new when_all_state(std::move(futures)...); + }(); + auto ret = state->p.get_future(); + state->do_wait_all(); + return ret; + } +}; + +} // namespace internal + +/// \cond internal +SEASTAR_CONCEPT( + +namespace impl { + +// Want: folds + +template <typename T> +struct is_tuple_of_futures : std::false_type { +}; + +template <> +struct is_tuple_of_futures<std::tuple<>> : std::true_type { +}; + +template <typename... T, typename... Rest> +struct is_tuple_of_futures<std::tuple<future<T...>, Rest...>> : is_tuple_of_futures<std::tuple<Rest...>> { +}; + +} + +template <typename... Futs> +concept AllAreFutures = impl::is_tuple_of_futures<std::tuple<Futs...>>::value; + +) + +template<typename Fut, std::enable_if_t<is_future<Fut>::value, int> = 0> +auto futurize_invoke_if_func(Fut&& fut) noexcept { + return std::forward<Fut>(fut); +} + +template<typename Func, std::enable_if_t<!is_future<Func>::value, int> = 0> +auto futurize_invoke_if_func(Func&& func) noexcept { + return futurize_invoke(std::forward<Func>(func)); +} +/// \endcond + +namespace internal { + +template <typename... Futs> +SEASTAR_CONCEPT( requires seastar::AllAreFutures<Futs...> ) +inline +future<std::tuple<Futs...>> +when_all_impl(Futs&&... futs) noexcept { + using state = when_all_state<identity_futures_tuple<Futs...>, Futs...>; + return state::wait_all(std::forward<Futs>(futs)...); +} + +} // namespace internal + +/// Wait for many futures to complete, capturing possible errors (variadic version). +/// +/// Each future can be passed directly, or a function that returns a +/// future can be given instead. +/// +/// If any function throws, an exceptional future is created for it. +/// +/// Returns a tuple of futures so individual values or exceptions can be +/// examined. +/// +/// \param fut_or_funcs futures or functions that return futures +/// \return an \c std::tuple<> of all futures returned; when ready, +/// all contained futures will be ready as well. +template <typename... FutOrFuncs> +inline auto when_all(FutOrFuncs&&... fut_or_funcs) noexcept { + return internal::when_all_impl(futurize_invoke_if_func(std::forward<FutOrFuncs>(fut_or_funcs))...); +} + +namespace internal { + +template<typename Future> +struct identity_futures_vector { + using future_type = future<std::vector<Future>>; + static future_type run(std::vector<Future> futures) noexcept { + return make_ready_future<std::vector<Future>>(std::move(futures)); + } + static future_type current_exception_as_future() noexcept { + return seastar::current_exception_as_future<std::vector<Future>>(); + } +}; + +// Internal function for when_all(). +template <typename ResolvedVectorTransform, typename Future> +inline +typename ResolvedVectorTransform::future_type +complete_when_all(std::vector<Future>&& futures, typename std::vector<Future>::iterator pos) noexcept { + // If any futures are already ready, skip them. + while (pos != futures.end() && pos->available()) { + ++pos; + } + // Done? + if (pos == futures.end()) { + return ResolvedVectorTransform::run(std::move(futures)); + } + // Wait for unready future, store, and continue. + return pos->then_wrapped([futures = std::move(futures), pos] (auto fut) mutable { + *pos++ = std::move(fut); + return complete_when_all<ResolvedVectorTransform>(std::move(futures), pos); + }); +} + +template<typename ResolvedVectorTransform, typename FutureIterator> +inline auto +do_when_all(FutureIterator begin, FutureIterator end) noexcept { + using itraits = std::iterator_traits<FutureIterator>; + auto make_values_vector = [] (size_t size) noexcept { + memory::scoped_critical_alloc_section _; + std::vector<typename itraits::value_type> ret; + ret.reserve(size); + return ret; + }; + std::vector<typename itraits::value_type> ret = + make_values_vector(iterator_range_estimate_vector_capacity(begin, end, typename itraits::iterator_category())); + // Important to invoke the *begin here, in case it's a function iterator, + // so we launch all computation in parallel. + std::move(begin, end, std::back_inserter(ret)); + return complete_when_all<ResolvedVectorTransform>(std::move(ret), ret.begin()); +} + +} // namespace internal + +/// Wait for many futures to complete, capturing possible errors (iterator version). +/// +/// Given a range of futures as input, wait for all of them +/// to resolve (either successfully or with an exception), and return +/// them as a \c std::vector so individual values or exceptions can be examined. +/// +/// \param begin an \c InputIterator designating the beginning of the range of futures +/// \param end an \c InputIterator designating the end of the range of futures +/// \return an \c std::vector<> of all the futures in the input; when +/// ready, all contained futures will be ready as well. +template <typename FutureIterator> +SEASTAR_CONCEPT( requires requires (FutureIterator i) { { *i++ }; requires is_future<std::remove_reference_t<decltype(*i)>>::value; } ) +inline +future<std::vector<typename std::iterator_traits<FutureIterator>::value_type>> +when_all(FutureIterator begin, FutureIterator end) noexcept { + namespace si = internal; + using itraits = std::iterator_traits<FutureIterator>; + using result_transform = si::identity_futures_vector<typename itraits::value_type>; + try { + return si::do_when_all<result_transform>(std::move(begin), std::move(end)); + } catch (...) { + return result_transform::current_exception_as_future(); + } +} + +namespace internal { + +template<typename Future> +struct future_has_value { + enum { + value = !std::is_same<std::decay_t<Future>, future<>>::value + }; +}; + +template<typename Tuple> +struct tuple_to_future; + +template<typename... Elements> +struct tuple_to_future<std::tuple<Elements...>> { +#if SEASTAR_API_LEVEL < 4 + using value_type = when_all_succeed_tuple<Elements...>; +#else + using value_type = std::tuple<Elements...>; +#endif + using type = future<value_type>; + using promise_type = promise<value_type>; + + // Elements... all come from futures, so we know they are nothrow move + // constructible. `future` also has a static assertion to that effect. + + static auto make_ready(std::tuple<Elements...> t) noexcept { + return make_ready_future<value_type>(value_type(std::move(t))); + } + + static auto make_failed(std::exception_ptr excp) noexcept { + return seastar::make_exception_future<value_type>(std::move(excp)); + } +}; + +#if SEASTAR_API_LEVEL < 4 + +template<typename Element> +struct tuple_to_future<std::tuple<Element>> { + using type = future<Element>; + using promise_type = promise<Element>; + + // Element comes from a future, so we know it is nothrow move + // constructible. `future` also has a static assertion to that effect. + + static auto make_ready(std::tuple<Element> t) noexcept { + return make_ready_future<Element>(std::get<0>(std::move(t))); + } + + static auto make_failed(std::exception_ptr excp) noexcept { + return seastar::make_exception_future<Element>(std::move(excp)); + } +}; + +template<> +struct tuple_to_future<std::tuple<>> { + using type = future<>; + using promise_type = promise<>; + + static auto make_ready(std::tuple<> t) noexcept { + return make_ready_future<>(); + } + + static auto make_failed(std::exception_ptr excp) noexcept { + return seastar::make_exception_future<>(std::move(excp)); + } +}; + +#endif + +template<typename... Futures> +class extract_values_from_futures_tuple { + static auto transform(std::tuple<Futures...> futures) noexcept { + auto prepare_result = [] (auto futures) noexcept { + auto fs = tuple_filter_by_type<internal::future_has_value>(std::move(futures)); + return tuple_map(std::move(fs), [] (auto&& e) { + return e.get0(); + }); + }; + + using tuple_futurizer = internal::tuple_to_future<decltype(prepare_result(std::move(futures)))>; + + std::exception_ptr excp; + tuple_for_each(futures, [&excp] (auto& f) { + if (!excp) { + if (f.failed()) { + excp = f.get_exception(); + } + } else { + f.ignore_ready_future(); + } + }); + if (excp) { + return tuple_futurizer::make_failed(std::move(excp)); + } + + return tuple_futurizer::make_ready(prepare_result(std::move(futures))); + } +public: + using future_type = decltype(transform(std::declval<std::tuple<Futures...>>())); + using promise_type = typename future_type::promise_type; + + static void set_promise(promise_type& p, std::tuple<Futures...> tuple) { + transform(std::move(tuple)).forward_to(std::move(p)); + } + + static future_type make_ready_future(std::tuple<Futures...> tuple) noexcept { + return transform(std::move(tuple)); + } + + static future_type current_exception_as_future() noexcept { + future_type (*type_deduct)() = current_exception_as_future; + return type_deduct(); + } +}; + +template<typename Future> +struct extract_values_from_futures_vector { + using value_type = decltype(untuple(std::declval<typename Future::tuple_type>())); + + using future_type = future<std::vector<value_type>>; + + static future_type run(std::vector<Future> futures) noexcept { + auto make_values_vector = [] (size_t size) noexcept { + memory::scoped_critical_alloc_section _; + std::vector<value_type> values; + values.reserve(size); + return values; + }; + std::vector<value_type> values = make_values_vector(futures.size()); + + std::exception_ptr excp; + for (auto&& f : futures) { + if (!excp) { + if (f.failed()) { + excp = f.get_exception(); + } else { + values.emplace_back(f.get0()); + } + } else { + f.ignore_ready_future(); + } + } + if (excp) { + return seastar::make_exception_future<std::vector<value_type>>(std::move(excp)); + } + return make_ready_future<std::vector<value_type>>(std::move(values)); + } + + static future_type current_exception_as_future() noexcept { + return seastar::current_exception_as_future<std::vector<value_type>>(); + } +}; + +template<> +struct extract_values_from_futures_vector<future<>> { + using future_type = future<>; + + static future_type run(std::vector<future<>> futures) noexcept { + std::exception_ptr excp; + for (auto&& f : futures) { + if (!excp) { + if (f.failed()) { + excp = f.get_exception(); + } + } else { + f.ignore_ready_future(); + } + } + if (excp) { + return seastar::make_exception_future<>(std::move(excp)); + } + return make_ready_future<>(); + } + + static future_type current_exception_as_future() noexcept { + return seastar::current_exception_as_future<>(); + } +}; + +template<typename... Futures> +SEASTAR_CONCEPT( requires seastar::AllAreFutures<Futures...> ) +inline auto when_all_succeed_impl(Futures&&... futures) noexcept { + using state = when_all_state<extract_values_from_futures_tuple<Futures...>, Futures...>; + return state::wait_all(std::forward<Futures>(futures)...); +} + +} // namespace internal + +/// Wait for many futures to complete (variadic version). +/// +/// Each future can be passed directly, or a function that returns a +/// future can be given instead. +/// +/// If any function throws, or if the returned future fails, one of +/// the exceptions is returned by this function as a failed future. +/// +/// \param fut_or_funcs futures or functions that return futures +/// \return future containing values of futures returned by funcs +template <typename... FutOrFuncs> +inline auto when_all_succeed(FutOrFuncs&&... fut_or_funcs) noexcept { + return internal::when_all_succeed_impl(futurize_invoke_if_func(std::forward<FutOrFuncs>(fut_or_funcs))...); +} + +/// Wait for many futures to complete (iterator version). +/// +/// Given a range of futures as input, wait for all of them +/// to resolve, and return a future containing a vector of values of the +/// original futures. +/// In case any of the given futures fails one of the exceptions is returned +/// by this function as a failed future. +/// \param begin an \c InputIterator designating the beginning of the range of futures +/// \param end an \c InputIterator designating the end of the range of futures +/// \return an \c std::vector<> of all the valus in the input +template <typename FutureIterator, typename = typename std::iterator_traits<FutureIterator>::value_type> +SEASTAR_CONCEPT( requires requires (FutureIterator i) { + *i++; + { i != i } -> std::convertible_to<bool>; + requires is_future<std::remove_reference_t<decltype(*i)>>::value; +} ) +inline auto +when_all_succeed(FutureIterator begin, FutureIterator end) noexcept { + using itraits = std::iterator_traits<FutureIterator>; + using result_transform = internal::extract_values_from_futures_vector<typename itraits::value_type>; + try { + return internal::do_when_all<result_transform>(std::move(begin), std::move(end)); + } catch (...) { + return result_transform::current_exception_as_future(); + } +} + +/// @} + +} // namespace seastar diff --git a/src/seastar/include/seastar/core/when_any.hh b/src/seastar/include/seastar/core/when_any.hh new file mode 100644 index 000000000..354abd86c --- /dev/null +++ b/src/seastar/include/seastar/core/when_any.hh @@ -0,0 +1,152 @@ +/* + * 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. + */ + +/* + * author: Niek J Bouman + * reviewers: Avi Kivity, Benny Halevy + * November 2021 + */ + +#pragma once + +#include <iterator> +#include <cstddef> +#include <type_traits> +#include <vector> +#include <tuple> +#include <utility> +#include <seastar/core/future.hh> +#include <seastar/core/shared_ptr.hh> + +namespace seastar { + +template <class Sequence> +struct when_any_result { + std::size_t index; + Sequence futures; +}; + +namespace internal { +class waiter { + bool _done = false; + promise<std::size_t> _promise; + +public: + void done(std::size_t index) { + if (!_done) { + _done = true; + _promise.set_value(index); + } + } + auto get_future() { return _promise.get_future(); } +}; + +} // namespace internal + +/// Wait for the first of multiple futures to complete (iterator version). +/// +/// Given a range of futures as input, wait for the first of them +/// to resolve (either successfully or with an exception), and return +/// all of them in a \c when_any_result (following the concurrency TS from +/// the standard library), containing a std::vector to all futures +/// and the index (into the vector) of the future that resolved. +/// +/// \param begin an \c InputIterator designating the beginning of the range of futures +/// \param end an \c InputIterator designating the end of the range of futures +/// \return a \c when_any_result of all the futures in the input; when +/// ready, at least one of the contained futures (the one indicated by index) will be ready. +template <class FutureIterator> +SEASTAR_CONCEPT( requires requires (FutureIterator i) { { *i++ }; requires is_future<std::remove_reference_t<decltype(*i)>>::value; } ) +auto when_any(FutureIterator begin, FutureIterator end) noexcept + -> future<when_any_result<std::vector<std::decay_t<typename std::iterator_traits<FutureIterator>::value_type>>>> +{ + using ReturnType = when_any_result<std::vector<typename std::iterator_traits<FutureIterator>::value_type>>; + if (begin == end) { + return make_ready_future<ReturnType>(); + } + ReturnType result; + result.futures.reserve(std::distance(begin, end)); + auto waiter_obj = make_lw_shared<internal::waiter>(); + std::size_t index{0}; + for (auto it = begin; it != end; ++it) { + if (it->available()) { + result.futures.push_back(std::move(*it)); + waiter_obj->done(index); + } else { + result.futures.push_back(it->finally([waiter_obj, index] { + waiter_obj->done(index); + })); + } + index++; + } + return waiter_obj->get_future().then( + [result = std::move(result)](std::size_t index) mutable { + result.index = index; + return std::move(result); + } + ); +} + +namespace internal { + +template <class... Futures, std::size_t... I> +future<when_any_result<std::tuple<Futures...>>> +when_any_impl(std::index_sequence<I...>, Futures&&... futs) noexcept +{ + auto waiter_obj = make_lw_shared<waiter>(); + auto attach_notifier = [&](auto&& fut, size_t index) { + if (fut.available()) { + waiter_obj->done(index); + return std::move(fut); + } + else { + return fut.finally([waiter_obj, index] { waiter_obj->done(index); }); + } + }; + + auto result = + when_any_result<std::tuple<Futures...>>{0, std::make_tuple(attach_notifier(std::forward<Futures>(futs), I)...)}; + return waiter_obj->get_future().then([result = std::move(result)](std::size_t index) mutable { + result.index = index; + return std::move(result); + }); +} + +} // namespace internal + +/// Wait for the first of multiple futures to complete (variadic version). +/// +/// Each future can be passed directly, or a function that returns a +/// future can be given instead. +/// +/// Returns a \c when_any_result (following the concurrency TS from +/// the standard library), containing a std::tuple to all futures +/// and the index (into the vector) of the future that resolved. +/// +/// \param fut_or_funcs futures or functions that return futures +/// \return a \c when_any_result containing a tuple of all futures +/// and and index; when ready, at least one of the contained futures +/// (the one indicated by index) will be ready. +template <class... FutOrFuncs> +auto when_any(FutOrFuncs&&... fut_or_funcs) noexcept +{ + return internal::when_any_impl(std::make_index_sequence<sizeof...(FutOrFuncs)>{}, + futurize_invoke_if_func(std::forward<FutOrFuncs>(fut_or_funcs))...); +} + +} // namespace seastar diff --git a/src/seastar/include/seastar/core/with_scheduling_group.hh b/src/seastar/include/seastar/core/with_scheduling_group.hh new file mode 100644 index 000000000..6b652f41f --- /dev/null +++ b/src/seastar/include/seastar/core/with_scheduling_group.hh @@ -0,0 +1,77 @@ +/* + * 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) 2020 ScyllaDB. + */ + +#pragma once + +#include <seastar/core/future.hh> +#include <seastar/core/make_task.hh> + +namespace seastar { + +/// \addtogroup future-util +/// @{ + +namespace internal { + +template <typename Func> +SEASTAR_CONCEPT( requires std::is_nothrow_move_constructible_v<Func> ) +auto +schedule_in_group(scheduling_group sg, Func func) noexcept { + static_assert(std::is_nothrow_move_constructible_v<Func>); + auto tsk = make_task(sg, std::move(func)); + schedule(tsk); + return tsk->get_future(); +} + + +} + +/// \brief run a callable (with some arbitrary arguments) in a scheduling group +/// +/// If the conditions are suitable (see scheduling_group::may_run_immediately()), +/// then the function is run immediately. Otherwise, the function is queued to run +/// when its scheduling group next runs. +/// +/// \param sg scheduling group that controls execution time for the function +/// \param func function to run; must be movable or copyable +/// \param args arguments to the function; may be copied or moved, so use \c std::ref() +/// to force passing references +template <typename Func, typename... Args> +SEASTAR_CONCEPT( requires std::is_nothrow_move_constructible_v<Func> ) +inline +auto +with_scheduling_group(scheduling_group sg, Func func, Args&&... args) noexcept { + static_assert(std::is_nothrow_move_constructible_v<Func>); + using return_type = decltype(func(std::forward<Args>(args)...)); + using futurator = futurize<return_type>; + if (sg.active()) { + return futurator::invoke(func, std::forward<Args>(args)...); + } else { + return internal::schedule_in_group(sg, [func = std::move(func), args = std::make_tuple(std::forward<Args>(args)...)] () mutable { + return futurator::apply(func, std::move(args)); + }); + } +} + +/// @} + +} // namespace seastar diff --git a/src/seastar/include/seastar/core/with_timeout.hh b/src/seastar/include/seastar/core/with_timeout.hh new file mode 100644 index 000000000..cbe4758dd --- /dev/null +++ b/src/seastar/include/seastar/core/with_timeout.hh @@ -0,0 +1,73 @@ +/* + * 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) 2020 ScyllaDB. + */ + + +#pragma once + +#include <chrono> + +#include <seastar/core/future.hh> +#include <seastar/core/timed_out_error.hh> +#include <seastar/core/timer.hh> + +namespace seastar { + +/// \addtogroup future-util +/// @{ + +/// \brief Wait for either a future, or a timeout, whichever comes first +/// +/// When timeout is reached the returned future resolves with an exception +/// produced by ExceptionFactory::timeout(). By default it is \ref timed_out_error exception. +/// +/// Note that timing out doesn't cancel any tasks associated with the original future. +/// It also doesn't cancel the callback registerred on it. +/// +/// \param f future to wait for +/// \param timeout time point after which the returned future should be failed +/// +/// \return a future which will be either resolved with f or a timeout exception +template<typename ExceptionFactory = default_timeout_exception_factory, typename Clock, typename Duration, typename... T> +future<T...> with_timeout(std::chrono::time_point<Clock, Duration> timeout, future<T...> f) { + if (f.available()) { + return f; + } + auto pr = std::make_unique<promise<T...>>(); + auto result = pr->get_future(); + timer<Clock> timer([&pr = *pr] { + pr.set_exception(std::make_exception_ptr(ExceptionFactory::timeout())); + }); + timer.arm(timeout); + // Future is returned indirectly. + (void)f.then_wrapped([pr = std::move(pr), timer = std::move(timer)] (auto&& f) mutable { + if (timer.cancel()) { + f.forward_to(std::move(*pr)); + } else { + f.ignore_ready_future(); + } + }); + return result; +} + +/// @} + +} // namespace seastar diff --git a/src/seastar/include/seastar/coroutine/all.hh b/src/seastar/include/seastar/coroutine/all.hh new file mode 100644 index 000000000..0e58cb3eb --- /dev/null +++ b/src/seastar/include/seastar/coroutine/all.hh @@ -0,0 +1,210 @@ +/* + * 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) 2021-present ScyllaDB + */ + +#pragma once + +#include <cstddef> +#include <concepts> +#include <tuple> +#include <seastar/core/coroutine.hh> + +namespace seastar::coroutine { + +template <typename Future> +constexpr inline bool is_future_v = is_future<Future>::value; + +template <typename Future> +concept future_type = is_future_v<Future>; + +namespace internal { + +// Given a bunch of futures, find the indexes of the ones that are not avoid +// and store them in member type `type` as an std::integer_sequence. +// +// `IndexSequence` and `current` are intermediates used for recursion. +template <typename IndexSequence, size_t current, typename... Futures> +struct index_sequence_for_non_void_futures_helper; + +// Terminate recursion be returning the accumulated `IndexSequence` +template <typename IndexSequence, size_t current> +struct index_sequence_for_non_void_futures_helper<IndexSequence, current> { + using type = IndexSequence; +}; + +// Process a future<T> by adding it to the current IndexSequence and recursing +template <size_t... Existing, size_t current, typename T, typename... Futures> +struct index_sequence_for_non_void_futures_helper<std::integer_sequence<size_t, Existing...>, current, future<T>, Futures...> { + using type = typename index_sequence_for_non_void_futures_helper<std::integer_sequence<size_t, Existing..., current>, current + 1, Futures...>::type; +}; + +// Process a future<void> by ignoring it and recursing +template <size_t... Existing, size_t current, typename... Futures> +struct index_sequence_for_non_void_futures_helper<std::integer_sequence<size_t, Existing...>, current, future<>, Futures...> { + using type = typename index_sequence_for_non_void_futures_helper<std::integer_sequence<size_t, Existing...>, current + 1, Futures...>::type; +}; + +// Simple interface for the above. +template <typename... Futures> +using index_sequence_for_non_void_futures = typename index_sequence_for_non_void_futures_helper<std::integer_sequence<size_t>, 0, Futures...>::type; + +// Given a tuple of futures, return a tuple of the value types, excluding future<void>. +template <typename IndexSequence, typename FutureTuple> +struct value_tuple_for_non_void_futures_helper; + +template <size_t... Idx, typename FutureTuple> +struct value_tuple_for_non_void_futures_helper<std::integer_sequence<size_t, Idx...>, FutureTuple> { + using type = std::tuple<typename std::tuple_element_t<Idx, FutureTuple>::value_type...>; +}; + +// Simple interface for the above +template <typename... Futures> +using value_tuple_for_non_void_futures = typename value_tuple_for_non_void_futures_helper<index_sequence_for_non_void_futures<Futures...>, std::tuple<Futures...>>::type; + +} + +/// Wait for serveral futures to complete in a coroutine. +/// +/// `all` can be used to launch several computations concurrently +/// and wait for all of them to complete. Computations are provided +/// as callable objects (typically lambda coroutines) that are invoked +/// by `all`. Waiting is performend by `co_await` and returns a tuple +/// of values, one for each non-void future. +/// +/// If one or more of the function objects throws an exception, or if one +/// or more of the futures resolves to an exception, then the exception is +/// thrown. All of the futures are waited for, even in the case of exceptions. +/// If more than one exception is present, an arbitrary one is thrown. +/// +/// Example +/// +/// ``` +/// future<int> add() { +/// auto [a, b] = co_await all( +/// [] () -> future<int> { +/// co_await sleep(1ms); +/// co_return 2; +/// }, +/// [] () -> future<int> { +/// co_await sleep(1ms); +/// co_return 3; +/// } +/// ); +/// co_return a + b; +/// }; +/// ``` +/// +/// Safe for use with lambda coroutines. +template <typename... Futures> +requires (sizeof ...(Futures) > 0) +class [[nodiscard("must co_await an all() object")]] all { + using tuple = std::tuple<Futures...>; + using value_tuple = typename internal::value_tuple_for_non_void_futures<Futures...>; + struct awaiter; + template <size_t idx> + struct intermediate_task final : continuation_base_from_future_t<std::tuple_element_t<idx, tuple>> { + awaiter& container; + explicit intermediate_task(awaiter& container) : container(container) {} + virtual void run_and_dispose() noexcept { + using value_type = typename std::tuple_element_t<idx, tuple>::value_type; + if (__builtin_expect(this->_state.failed(), false)) { + using futurator = futurize<std::tuple_element_t<idx, tuple>>; + std::get<idx>(container.state._futures) = futurator::make_exception_future(std::move(this->_state).get_exception()); + } else { + if constexpr (std::same_as<std::tuple_element_t<idx, tuple>, future<>>) { + std::get<idx>(container.state._futures) = make_ready_future<>(); + } else { + std::get<idx>(container.state._futures) = make_ready_future<value_type>(std::move(this->_state).get0()); + } + } + this->~intermediate_task(); + container.template process<idx+1>(); + } + }; + template <typename IndexSequence> + struct generate_aligned_union; + template <size_t... idx> + struct generate_aligned_union<std::integer_sequence<size_t, idx...>> { + static constexpr std::size_t alignment_value = std::max({alignof(intermediate_task<idx>)...}); + using type = std::byte[std::max({sizeof(intermediate_task<idx>)...})]; + }; + using continuation_storage = generate_aligned_union<std::make_index_sequence<std::tuple_size_v<tuple>>>; + using coroutine_handle_t = std::coroutine_handle<void>; +private: + tuple _futures; +private: + struct awaiter { + all& state; + alignas(continuation_storage::alignment_value) typename continuation_storage::type _continuation_storage; + coroutine_handle_t when_ready; + awaiter(all& state) : state(state) {} + bool await_ready() const { + return std::apply([] (const Futures&... futures) { + return (... && futures.available()); + }, state._futures); + } + void await_suspend(coroutine_handle_t h) { + when_ready = h; + process<0>(); + } + value_tuple await_resume() { + std::apply([] (Futures&... futures) { + std::exception_ptr e; + // Call get_exception for every failed future, to avoid exceptional future + // ignored warnings. + (void)(..., (futures.failed() ? (e = futures.get_exception(), 0) : 0)); + if (e) { + std::rethrow_exception(std::move(e)); + } + }, state._futures); + // This immediately-invoked lambda is used to materialize the indexes + // of non-void futures in the tuple. + return [&] <size_t... Idx> (std::integer_sequence<size_t, Idx...>) { + return value_tuple(std::get<Idx>(state._futures).get0()...); + } (internal::index_sequence_for_non_void_futures<Futures...>()); + } + template <unsigned idx> + void process() { + if constexpr (idx == sizeof...(Futures)) { + when_ready.resume(); + } else { + if (!std::get<idx>(state._futures).available()) { + auto task = new (&_continuation_storage) intermediate_task<idx>(*this); + seastar::internal::set_callback(std::move(std::get<idx>(state._futures)), task); + } else { + process<idx + 1>(); + } + } + } + }; +public: + template <typename... Func> + requires (... && std::invocable<Func>) && (... && future_type<std::invoke_result_t<Func>>) + explicit all(Func&&... funcs) + : _futures(futurize_invoke(funcs)...) { + } + awaiter operator co_await() { return awaiter{*this}; } +}; + +template <typename FirstFunc, typename... MoreFuncs> +explicit all(FirstFunc&&, MoreFuncs&&...) -> all<std::invoke_result_t<FirstFunc>, + std::invoke_result_t<MoreFuncs>...>; + +} diff --git a/src/seastar/include/seastar/coroutine/as_future.hh b/src/seastar/include/seastar/coroutine/as_future.hh new file mode 100644 index 000000000..6ce52d9aa --- /dev/null +++ b/src/seastar/include/seastar/coroutine/as_future.hh @@ -0,0 +1,106 @@ +/* + * 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) 2022-present ScyllaDB + */ + +#pragma once + +#include <seastar/core/coroutine.hh> + +namespace seastar { + +namespace internal { + +template <bool CheckPreempt, typename T> +class SEASTAR_NODISCARD as_future_awaiter { + seastar::future<T> _future; + +public: + explicit as_future_awaiter(seastar::future<T>&& f) noexcept : _future(std::move(f)) {} + + as_future_awaiter(const as_future_awaiter&) = delete; + as_future_awaiter(as_future_awaiter&&) = delete; + + bool await_ready() const noexcept { + return _future.available() && (!CheckPreempt || !need_preempt()); + } + + template<typename U> + void await_suspend(std::coroutine_handle<U> hndl) noexcept { + if (!CheckPreempt || !_future.available()) { + _future.set_coroutine(hndl.promise()); + } else { + schedule(&hndl.promise()); + } + } + + seastar::future<T> await_resume() { + return std::move(_future); + } +}; + +} // namespace seastar::internal + +namespace coroutine { + +/// \brief co_await:s a \ref future, returning it as result. +/// +/// Similar to \ref seastar::future::then_wrapped, `coroutine::as_future` +/// waits for the \ref future to resolve either to a ready future +/// or to an exceptional one. It then returns it as the co_await result. +/// +/// For example: +/// ``` +/// static future<bool> did_future_fail(future<> fut) { +/// auto f = co_await coroutine::as_future(std::move(fut)); +/// if (f.failed()) { +/// mylog.warn("Future failed: {}", f.get_exception()); +/// co_return true; +/// } else { +/// co_return false; +/// } +/// } +/// ``` +/// +/// Note that by default, `as_future` checks for if the task quota is depleted, +/// which means that it will yield if the future is ready and \ref seastar::need_preempt() +/// returns true. Use \ref coroutine::as_future_without_preemption_check +/// to disable preemption checking. +template<typename T = void> +class SEASTAR_NODISCARD as_future : public seastar::internal::as_future_awaiter<true, T> { +public: + explicit as_future(seastar::future<T>&& f) noexcept : seastar::internal::as_future_awaiter<true, T>(std::move(f)) {} +}; + +/// \brief co_await:s a \ref future, returning it as result, without +/// checking if preemption is needed. +/// +/// Like \ref coroutine::as_future, co_await-ing as_future_without_preemption_check +/// returns the input `future` as the co_await result. +/// However, it bypasses checking if the task quota is depleted, which means that +/// a ready `future` will be handled immediately. +template<typename T = void> +class SEASTAR_NODISCARD as_future_without_preemption_check : public seastar::internal::as_future_awaiter<false, T> { +public: + explicit as_future_without_preemption_check(seastar::future<T>&& f) noexcept : seastar::internal::as_future_awaiter<false, T>(std::move(f)) {} +}; + +} // namespace seastar::coroutine + +} // namespace seastar diff --git a/src/seastar/include/seastar/coroutine/exception.hh b/src/seastar/include/seastar/coroutine/exception.hh new file mode 100644 index 000000000..b449fb0d9 --- /dev/null +++ b/src/seastar/include/seastar/coroutine/exception.hh @@ -0,0 +1,141 @@ +/* + * 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) 2021-present ScyllaDB + */ + +#pragma once + +#include <seastar/core/future.hh> +#include <coroutine> +#include <exception> + +namespace seastar { + +namespace internal { + +struct exception_awaiter { + std::exception_ptr eptr; + + explicit exception_awaiter(std::exception_ptr&& eptr) noexcept : eptr(std::move(eptr)) {} + + exception_awaiter(const exception_awaiter&) = delete; + exception_awaiter(exception_awaiter&&) = delete; + + bool await_ready() const noexcept { + return false; + } + + template<typename U> + void await_suspend(std::coroutine_handle<U> hndl) noexcept { + hndl.promise().set_exception(std::move(eptr)); + hndl.destroy(); + } + + void await_resume() noexcept {} +}; + +} // internal + +namespace coroutine { + +/// Wrapper for propagating an exception directly rather than +/// throwing it. The wrapper can be used with both co_await and co_return. +/// +/// \note It is not possible to co_return the wrapper in coroutines which +/// return future<> due to language limitations (it's not possible to specify +/// both return_value and return_void in the promise_type). You can use co_await +/// instead which works in coroutines which return either future<> or future<T>. +/// +/// Example usage: +/// +/// ``` +/// co_await coroutine::exception(std::make_exception_ptr(std::runtime_error("something failed miserably"))); +/// co_return coroutine::exception(std::make_exception_ptr(std::runtime_error("something failed miserably"))); +/// ``` +struct exception { + std::exception_ptr eptr; + explicit exception(std::exception_ptr eptr) noexcept : eptr(std::move(eptr)) {} +}; + +/// Allows propagating an exception from a coroutine directly rather than +/// throwing it. +/// +/// `make_exception()` returns an object which must be co_returned. +/// Co_returning the object will immediately resolve the current coroutine +/// to the given exception. +/// +/// \note Due to language limitations, this function doesn't work in coroutines +/// which return future<>. Consider using return_exception instead. +/// +/// Example usage: +/// +/// ``` +/// co_return coroutine::make_exception(std::runtime_error("something failed miserably")); +/// ``` +[[deprecated("Use co_await coroutine::return_exception or co_return coroutine::exception instead")]] +[[nodiscard]] +inline exception make_exception(std::exception_ptr ex) noexcept { + return exception(std::move(ex)); +} + +template<typename T> +[[deprecated("Use co_await coroutine::return_exception or co_return coroutine::exception instead")]] +[[nodiscard]] +exception make_exception(T&& t) noexcept { + log_exception_trace(); + return exception(std::make_exception_ptr(std::forward<T>(t))); +} + +/// Allows propagating an exception from a coroutine directly rather than +/// throwing it. +/// +/// `return_exception()` returns an object which must be co_awaited. +/// Co_awaiting the object will immediately resolve the current coroutine +/// to the given exception. +/// +/// Example usage: +/// +/// ``` +/// co_await coroutine::return_exception(std::runtime_error("something failed miserably")); +/// ``` +[[nodiscard]] +inline exception return_exception_ptr(std::exception_ptr ex) noexcept { + return exception(std::move(ex)); +} + +[[deprecated("Use co_await coroutine::return_exception_ptr instead")]] +[[nodiscard]] +inline exception return_exception(std::exception_ptr ex) noexcept { + return exception(std::move(ex)); +} + +template<typename T> +[[nodiscard]] +exception return_exception(T&& t) noexcept { + log_exception_trace(); + return exception(std::make_exception_ptr(std::forward<T>(t))); +} + +} // coroutine + +inline auto operator co_await(coroutine::exception ex) noexcept { + return internal::exception_awaiter(std::move(ex.eptr)); +} + +} // seastar diff --git a/src/seastar/include/seastar/coroutine/generator.hh b/src/seastar/include/seastar/coroutine/generator.hh new file mode 100644 index 000000000..1bdb6ac81 --- /dev/null +++ b/src/seastar/include/seastar/coroutine/generator.hh @@ -0,0 +1,546 @@ +/* + * 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) 2022 Kefu Chai ( tchaikov@gmail.com ) + */ + +#pragma once + +#include <cassert> +#include <optional> +#include <utility> +#include <seastar/core/future.hh> +#include <seastar/util/attribute-compat.hh> + +namespace seastar::coroutine::experimental { + +template<typename T, template <typename> class Container> +class generator; + +/// `seastar::coroutine::experimental` is used as the type of the first +/// parameter of a buffered generator coroutine. +/// +/// the value of a `buffer_size_t` specifies the size of the buffer holding the +/// values produced by the generator coroutine. Unlike its unbuffered variant, +/// the bufferred generator does not wait for its caller to consume every single +/// produced values. Instead, it puts the produced values into an internal +/// buffer, before the buffer is full or the generator is suspended. This helps +/// to alleviate the problem of pingpong between the generator coroutine and +/// its caller. +enum class buffer_size_t : size_t; + +namespace internal { + +using std::coroutine_handle; +using std::suspend_never; +using std::suspend_always; +using std::suspend_never; +using std::noop_coroutine; + +template<typename T> +using next_value_t = std::optional<T>; + +template <template <typename> class Container, typename T> +concept Fifo = requires(Container<T>&& c, T&& value) { + // better off returning a reference though, so we can move away from it + { c.front() } -> std::same_as<T&>; + c.pop_front(); + c.push_back(value); + bool(c.empty()); + { c.size() } -> std::convertible_to<size_t>; +}; + +template<typename T> +concept NothrowMoveConstructible = std::is_nothrow_move_constructible_v<T>; + +template<NothrowMoveConstructible T> +class generator_unbuffered_promise final : public seastar::task { + using generator_type = seastar::coroutine::experimental::generator<T, std::optional>; + std::optional<seastar::promise<>> _wait_for_next_value; + generator_type* _generator = nullptr; + +public: + generator_unbuffered_promise() = default; + generator_unbuffered_promise(generator_unbuffered_promise&&) = delete; + generator_unbuffered_promise(const generator_unbuffered_promise&) = delete; + + void return_void() noexcept; + void unhandled_exception() noexcept; + + template<std::convertible_to<T> U> + suspend_always yield_value(U&& value) noexcept { + assert(_generator); + _generator->put_next_value(std::forward<U>(value)); + assert(_wait_for_next_value); + _wait_for_next_value->set_value(); + _wait_for_next_value = {}; + return {}; + } + + generator_type get_return_object() noexcept; + void set_generator(generator_type* g) noexcept { + assert(!_generator); + _generator = g; + } + + suspend_always initial_suspend() const noexcept { return {}; } + suspend_never final_suspend() const noexcept { + assert(_generator); + _generator->on_finished(); + return {}; + } + + seastar::future<> wait_for_next_value() noexcept { + assert(!_wait_for_next_value); + return _wait_for_next_value.emplace().get_future(); + } + + void run_and_dispose() noexcept final { + using handle_type = coroutine_handle<generator_unbuffered_promise>; + handle_type::from_promise(*this).resume(); + } + + seastar::task* waiting_task() noexcept final { + if (_wait_for_next_value) { + return _wait_for_next_value->waiting_task(); + } else { + return nullptr; + } + } +}; + +template <NothrowMoveConstructible T, template <typename> class Container> +requires Fifo<Container, T> +class generator_buffered_promise; + +template<typename T, template <typename> class Container> +struct yield_awaiter final { + using promise_type = generator_buffered_promise<T, Container>; + seastar::future<> _future; + +public: + yield_awaiter(seastar::future<>&& f) noexcept + : _future{std::move(f)} {} + + bool await_ready() noexcept { + return _future.available(); + } + + coroutine_handle<> await_suspend(coroutine_handle<promise_type> coro) noexcept; + void await_resume() noexcept { } +}; + + +template<NothrowMoveConstructible T, template <typename> class Container> +requires Fifo<Container, T> +class generator_buffered_promise final : public seastar::task { + using generator_type = seastar::coroutine::experimental::generator<T, Container>; + + std::optional<seastar::promise<>> _wait_for_next_value; + std::optional<seastar::promise<>> _wait_for_free_space; + generator_type* _generator = nullptr; + const size_t _buffer_capacity; + +public: + template<typename... Args> + generator_buffered_promise(buffer_size_t buffer_capacity, Args&&... args) + : _buffer_capacity{static_cast<size_t>(buffer_capacity)} {} + generator_buffered_promise(generator_buffered_promise&&) = delete; + generator_buffered_promise(const generator_buffered_promise&) = delete; + ~generator_buffered_promise() = default; + void return_void() noexcept { + if (_wait_for_next_value) { + _wait_for_next_value->set_value(); + _wait_for_next_value = {}; + } + } + void unhandled_exception() noexcept; + + template<std::convertible_to<T> U> + yield_awaiter<T, Container> yield_value(U&& value) noexcept { + bool ready = _generator->put_next_value(std::forward<U>(value)); + + if (_wait_for_next_value) { + _wait_for_next_value->set_value(); + _wait_for_next_value = {}; + return {make_ready_future()}; + } + if (ready) { + return {make_ready_future()}; + } else { + assert(!_wait_for_free_space); + return {_wait_for_free_space.emplace().get_future()}; + } + } + + auto get_return_object() noexcept -> generator_type; + void set_generator(generator_type* g) noexcept { + assert(!_generator); + _generator = g; + } + + suspend_always initial_suspend() const noexcept { return {}; } + suspend_never final_suspend() const noexcept { + assert(_generator); + _generator->on_finished(); + return {}; + } + + bool is_awaiting() const noexcept { + return _wait_for_next_value.has_value(); + } + + coroutine_handle<> coroutine() const noexcept { + return coroutine_handle<>::from_address(_wait_for_next_value->waiting_task()); + } + + seastar::future<> wait_for_next_value() noexcept { + assert(!_wait_for_next_value); + return _wait_for_next_value.emplace().get_future(); + } + + void on_reclaim_free_space() noexcept { + assert(_wait_for_free_space); + _wait_for_free_space->set_value(); + _wait_for_free_space = {}; + } + +private: + void run_and_dispose() noexcept final { + using handle_type = coroutine_handle<generator_buffered_promise>; + handle_type::from_promise(*this).resume(); + } + + seastar::task* waiting_task() noexcept final { + if (_wait_for_next_value) { + return _wait_for_next_value->waiting_task(); + } else if (_wait_for_free_space) { + return _wait_for_free_space->waiting_task(); + } else { + return nullptr; + } + } +}; + +template<typename T, typename Generator> +struct next_awaiter final { + using next_value_type = next_value_t<T>; + Generator* const _generator; + seastar::task* const _task; + seastar::future<> _next_value_future; + +public: + next_awaiter(Generator* generator, + seastar::task* task, + seastar::future<>&& f) noexcept + : _generator{generator} + , _task(task) + , _next_value_future(std::move(f)) {} + + next_awaiter(const next_awaiter&) = delete; + next_awaiter(next_awaiter&&) = delete; + + constexpr bool await_ready() const noexcept { + return _next_value_future.available() && !seastar::need_preempt(); + } + + template<typename Promise> + void await_suspend(coroutine_handle<Promise> coro) noexcept { + auto& current_task = coro.promise(); + if (_next_value_future.available()) { + seastar::schedule(¤t_task); + } else { + _next_value_future.set_coroutine(current_task); + seastar::schedule(_task); + } + } + + next_value_type await_resume() { + assert(_next_value_future.available()); + assert(_generator); + return _generator->take_next_value(); + } +}; + +} // namespace internal + +/// `seastar::coroutine::experimental::generator<T>` can be used to model a +/// generator which yields a sequence of values asynchronously. +/// +/// Typically, it is used as the return type of a coroutine which is only +/// allowed to either `co_yield` the next element in the sequence, or +/// `co_return` nothing to indicate the end of this sequence. Please note, +/// despite that `tl::generator` defined in `tests/unit/tl-generator.hh` also +/// represents a generator, it can only produce the elements in a synchronous +/// coroutine which does not suspend itself because of any asynchronous +/// operation. +/// +/// Example +/// +/// ``` +/// auto generate_request = [&input_stream](coroutine::experimental::buffer_size_t) +/// -> seastar::coroutine::generator<Request> { +/// while (!input_stream.eof()) { +/// co_yield co_await input_stream.read_exactly(42); +/// } +/// } +/// +/// const coroutine::experimental::buffer_size_t backlog = 42; +/// while (true) { +/// auto request = co_await generate_request(backlog); +/// if (!request) { +/// break; +/// } +/// co_await process(*std::move(request)); +/// } +/// ```` +template <typename T, template <typename> class Container> +class generator { +public: + using promise_type = internal::generator_buffered_promise<T, Container>; + +private: + using handle_type = internal::coroutine_handle<promise_type>; + handle_type _coro; + promise_type* _promise; + Container<T> _values; + const size_t _buffer_capacity; + std::exception_ptr _exception; + +public: + generator(size_t buffer_capacity, + handle_type coro, + promise_type* promise) noexcept + : _coro{coro} + , _promise{promise} + , _buffer_capacity{buffer_capacity} { + assert(_promise); + _promise->set_generator(this); + } + generator(const generator&) = delete; + generator(generator&& other) noexcept + : _coro{std::exchange(other._coro, {})} {} + generator& operator=(generator&& other) noexcept { + if (std::addressof(other) != this) { + auto old_coro = std::exchange(_coro, std::exchange(other._coro, {})); + if (old_coro) { + old_coro.destroy(); + } + } + return *this; + } + ~generator() { + if (_coro) { + _coro.destroy(); + } + } + + void swap(generator& other) noexcept { + std::swap(_coro, other._coro); + } + + internal::next_awaiter<T, generator> operator()() noexcept { + if (!_values.empty()) { + return {this, nullptr, make_ready_future<>()}; + } else if (_exception) [[unlikely]] { + return {this, nullptr, make_ready_future<>()}; + } else if (_promise) { + return {this, _promise, _promise->wait_for_next_value()}; + } else { + return {this, nullptr, make_ready_future<>()}; + } + } + + template<typename U> + bool put_next_value(U&& value) { + _values.push_back(std::forward<U>(value)); + return _values.size() < _buffer_capacity; + } + + internal::next_value_t<T> take_next_value() { + if (!_values.empty()) [[likely]] { + auto value = std::move(_values.front()); + bool maybe_reclaim = _values.size() == _buffer_capacity; + _values.pop_front(); + if (maybe_reclaim) { + if (_promise) [[likely]] { + _promise->on_reclaim_free_space(); + } + } + return internal::next_value_t<T>(std::move(value)); + } else if (_exception) [[unlikely]] { + std::rethrow_exception(std::exchange(_exception, nullptr)); + } else { + return std::nullopt; + } + } + + void on_finished() { + _promise = nullptr; + _coro = nullptr; + } + + void unhandled_exception() noexcept { + // called by promise's unhandled_exception() + assert(!_exception); + _exception = std::current_exception(); + } +}; + +template <typename T> +class generator<T, std::optional> { +public: + using promise_type = internal::generator_unbuffered_promise<T>; + +private: + using handle_type = internal::coroutine_handle<promise_type>; + handle_type _coro; + promise_type* _promise; + std::optional<T> _maybe_value; + std::exception_ptr _exception; + +public: + generator(handle_type coro, + promise_type* promise) noexcept + : _coro{coro} + , _promise{promise} { + assert(_promise); + _promise->set_generator(this); + } + generator(const generator&) = delete; + generator(generator&& other) noexcept + : _coro{std::exchange(other._coro, {})} {} + generator& operator=(generator&& other) noexcept { + if (std::addressof(other) != this) { + auto old_coro = std::exchange(_coro, std::exchange(other._coro, {})); + if (old_coro) { + old_coro.destroy(); + } + } + return *this; + } + ~generator() { + if (_coro) { + _coro.destroy(); + } + } + + void swap(generator& other) noexcept { + std::swap(_coro, other._coro); + } + + internal::next_awaiter<T, generator> operator()() noexcept { + if (_promise) [[likely]] { + return {this, _promise, _promise->wait_for_next_value()}; + } else { + return {this, nullptr, make_ready_future<>()}; + } + } + + template<typename U> + void put_next_value(U&& value) noexcept { + _maybe_value.emplace(std::forward<U>(value)); + } + + internal::next_value_t<T> take_next_value() { + if (_maybe_value.has_value()) [[likely]] { + return std::exchange(_maybe_value, std::nullopt); + } else if (_exception) [[unlikely]] { + std::rethrow_exception(std::exchange(_exception, nullptr)); + } else { + return std::nullopt; + } + } + + void on_finished() { + _promise = nullptr; + _coro = nullptr; + } + + void unhandled_exception() noexcept { + assert(!_exception); + _exception = std::current_exception(); + } +}; + +namespace internal { + +template<NothrowMoveConstructible T> +void generator_unbuffered_promise<T>::return_void() noexcept { + assert(_wait_for_next_value); + _wait_for_next_value->set_value(); + _wait_for_next_value = {}; +} + +template<NothrowMoveConstructible T> +void generator_unbuffered_promise<T>::unhandled_exception() noexcept { + // instead of storing the current exception into promise, in order to be + // more consistent, we let generator preserve all the output of produced + // value, including the values and the exception if any. so we just signal + // _wait_for_next_value, and delegate generator's unhandled_exception() to + // store the exception. + _generator->unhandled_exception(); + if (_wait_for_next_value.has_value()) { + _wait_for_next_value->set_value(); + _wait_for_next_value = {}; + } +} + +template<NothrowMoveConstructible T> +auto generator_unbuffered_promise<T>::get_return_object() noexcept -> generator_type { + using handle_type = coroutine_handle<generator_unbuffered_promise<T>>; + return generator_type{handle_type::from_promise(*this), this}; +} + +template<NothrowMoveConstructible T, template <typename> class Container> +requires Fifo<Container, T> +void generator_buffered_promise<T, Container>::unhandled_exception() noexcept { + _generator->unhandled_exception(); + if (_wait_for_next_value.has_value()) { + _wait_for_next_value->set_value(); + _wait_for_next_value = {}; + } +} + +template<NothrowMoveConstructible T, template <typename> class Container> +requires Fifo<Container, T> +auto generator_buffered_promise<T, Container>::get_return_object() noexcept -> generator_type { + using handle_type = coroutine_handle<generator_buffered_promise<T, Container>>; + return generator_type{_buffer_capacity, handle_type::from_promise(*this), this}; +} + +template<typename T, template <typename> class Container> +coroutine_handle<> yield_awaiter<T, Container>::await_suspend( + coroutine_handle<generator_buffered_promise<T, Container>> coro) noexcept { + if (_future.available()) { + auto& current_task = coro.promise(); + seastar::schedule(¤t_task); + return coro; + } else { + // we cannot do something like `task.set_coroutine(consumer_task)`. + // because, instead of waiting for a subcoroutine, we are pending on + // the caller of current coroutine to consume the produced values to + // free up at least a free slot in the buffer, if we set the `_task` + // of the of the awaiting task, we would have an infinite loop of + // "promise->_task". + return noop_coroutine(); + } +} + +} // namespace internal +} // namespace seastar::coroutine::experimental diff --git a/src/seastar/include/seastar/coroutine/maybe_yield.hh b/src/seastar/include/seastar/coroutine/maybe_yield.hh new file mode 100644 index 000000000..2ecfd793c --- /dev/null +++ b/src/seastar/include/seastar/coroutine/maybe_yield.hh @@ -0,0 +1,91 @@ +/* + * 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) 2021-present ScyllaDB + */ + +#pragma once + +#include <concepts> +#include <type_traits> +#include <seastar/core/coroutine.hh> + +namespace seastar::coroutine { + +namespace internal { + +struct maybe_yield_awaiter final : task { + using coroutine_handle_t = std::coroutine_handle<void>; + + coroutine_handle_t when_ready; + task* main_coroutine_task; + + bool await_ready() const { + return !need_preempt(); + } + + template <typename T> + void await_suspend(std::coroutine_handle<T> h) { + when_ready = h; + main_coroutine_task = &h.promise(); // for waiting_task() + schedule(this); + } + + void await_resume() { + } + + virtual void run_and_dispose() noexcept override { + when_ready.resume(); + // No need to delete, this is allocated on the coroutine frame + } + + virtual task* waiting_task() noexcept override { + return main_coroutine_task; + } +}; + +} + +/// Preempt if the current task quota expired. +/// +/// `maybe_yield()` can be used to break a long computation in a +/// coroutine and allow the reactor to preempt its execution. This +/// allows other tasks to gain access to the CPU. If the task quota +/// did not expire, the coroutine continues execution. +/// +/// It should be used in long loops that do not contain other `co_await` +/// calls. +/// +/// Example +/// +/// ``` +/// seastar::future<int> long_loop(int n) { +/// float acc = 0; +/// for (int i = 0; i < n; ++i) { +/// acc += std::sin(float(i)); +/// co_await seastar::coroutine::maybe_yield(); +/// } +/// co_return acc; +/// } +/// ``` +class [[nodiscard("must co_await an maybe_yield() object")]] maybe_yield { +public: + auto operator co_await() { return internal::maybe_yield_awaiter(); } +}; + +} diff --git a/src/seastar/include/seastar/coroutine/parallel_for_each.hh b/src/seastar/include/seastar/coroutine/parallel_for_each.hh new file mode 100644 index 000000000..f1a3b8020 --- /dev/null +++ b/src/seastar/include/seastar/coroutine/parallel_for_each.hh @@ -0,0 +1,189 @@ +/* + * 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) 2022-present ScyllaDB + */ + +#pragma once + +#include <ranges> + +#include <boost/container/small_vector.hpp> + +#include <seastar/core/loop.hh> +#include <seastar/core/coroutine.hh> + +namespace seastar::coroutine { + +/// Invoke a function on all elements in a range in parallel and wait for all futures to complete in a coroutine. +/// +/// `parallel_for_each` can be used to launch a function concurrently +/// on all elements in a given range and wait for all of them to complete. +/// Waiting is performend by `co_await` and returns a future. +/// +/// If one or more of the function invocations resolve to an exception +/// then the one of the exceptions is re-thrown. +/// All of the futures are waited for, even in the case of exceptions. +/// +/// Example +/// +/// ``` +/// future<int> sum_of_squares(std::vector<int> v) { +/// int sum = 0; +/// return co_await parallel_for_each(v, [&sum] (int& x) { +/// sum += x * x; +/// }); +/// co_return sum; +/// }; +/// ``` +/// +/// Safe for use with lambda coroutines. +/// +/// \note parallel_for_each() schedules all invocations of \c func on the +/// current shard. If you want to run a function on all shards in parallel, +/// have a look at \ref smp::invoke_on_all() instead. +template <typename Func> +// constaints for Func are defined at the parallel_for_each constructor +class [[nodiscard("must co_await an parallel_for_each() object")]] parallel_for_each final : continuation_base<> { + using coroutine_handle_t = std::coroutine_handle<void>; + + Func _func; + boost::container::small_vector<future<>, 5> _futures; + std::exception_ptr _ex; + coroutine_handle_t _when_ready; + task* _waiting_task = nullptr; + + // Consume futures in reverse order. + // Since futures at the front are expected + // to become ready before futures at the back, + // therefore it is less likely we will have + // to wait on them, after the back futures + // become available. + // + // Return true iff all futures were consumed. + bool consume_next() noexcept { + while (!_futures.empty()) { + auto& fut = _futures.back(); + if (!fut.available()) { + return false; + } + if (fut.failed()) { + _ex = fut.get_exception(); + } + _futures.pop_back(); + } + return true; + } + + void set_callback() noexcept { + // To reuse `this` as continuation_base<> + // we must reset _state, to allow setting + // it again. + this->_state = {}; + seastar::internal::set_callback(std::move(_futures.back()), reinterpret_cast<continuation_base<>*>(this)); + _futures.pop_back(); + } + + void resume_or_set_callback() noexcept { + if (consume_next()) { + _when_ready.resume(); + } else { + set_callback(); + } + } + +public: + // clang 13.0.1 doesn't support subrange + // so provide also a Iterator/Sentinel based constructor. + // See https://github.com/llvm/llvm-project/issues/46091 + template <typename Iterator, typename Sentinel, typename Func1> + requires (std::same_as<Sentinel, Iterator> || std::sentinel_for<Sentinel, Iterator>) + && std::same_as<future<>, futurize_t<std::invoke_result_t<Func, typename std::iterator_traits<Iterator>::reference>>> + explicit parallel_for_each(Iterator begin, Sentinel end, Func1&& func) noexcept + : _func(std::forward<Func1>(func)) + { + for (auto it = begin; it != end; ++it) { + auto fut = futurize_invoke(_func, *it); + if (fut.available()) { + if (fut.failed()) { + _ex = fut.get_exception(); + } + } else { + memory::scoped_critical_alloc_section _; + if (_futures.empty()) { + using itraits = std::iterator_traits<Iterator>; + if constexpr (seastar::internal::has_iterator_category<Iterator>::value) { + auto n = seastar::internal::iterator_range_estimate_vector_capacity(it, end, typename itraits::iterator_category{}); + _futures.reserve(n); + } + } + _futures.push_back(std::move(fut)); + } + } + } + + template <std::ranges::range Range, typename Func1> + requires std::invocable<Func, std::ranges::range_reference_t<Range>> + explicit parallel_for_each(Range&& range, Func1&& func) noexcept + : parallel_for_each(std::ranges::begin(range), std::ranges::end(range), std::forward<Func1>(func)) + { } + + bool await_ready() const noexcept { + if (_futures.empty()) { + return !_ex; + } + return false; + } + + template<typename T> + void await_suspend(std::coroutine_handle<T> h) { + _when_ready = h; + _waiting_task = &h.promise(); + resume_or_set_callback(); + } + + void await_resume() const { + if (_ex) [[unlikely]] { + std::rethrow_exception(std::move(_ex)); + } + } + + virtual void run_and_dispose() noexcept override { + if (this->_state.failed()) { + _ex = std::move(this->_state).get_exception(); + } + resume_or_set_callback(); + } + + virtual task* waiting_task() noexcept override { + return _waiting_task; + } +}; + +template <typename Iterator, typename Sentinel, typename Func> +requires (std::same_as<Sentinel, Iterator> || std::sentinel_for<Sentinel, Iterator>) + && std::same_as<future<>, futurize_t<std::invoke_result_t<Func, typename std::iterator_traits<Iterator>::reference>>> +parallel_for_each(Iterator begin, Sentinel end, Func&& func) -> parallel_for_each<Func>; + +template <std::ranges::range Range, typename Func> +requires std::invocable<Func, std::ranges::range_reference_t<Range>> +parallel_for_each(Range&& range, Func&& func) -> parallel_for_each<Func>; + + + +} diff --git a/src/seastar/include/seastar/coroutine/switch_to.hh b/src/seastar/include/seastar/coroutine/switch_to.hh new file mode 100644 index 000000000..09c6ef912 --- /dev/null +++ b/src/seastar/include/seastar/coroutine/switch_to.hh @@ -0,0 +1,91 @@ +/* + * 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) 2022-present ScyllaDB + */ + +#pragma once + +#include <seastar/core/coroutine.hh> +#include <seastar/core/scheduling.hh> + +namespace seastar::coroutine { + +/// Switch the current task scheduling group. +/// +/// `switch_to(new_scheduling_group)` can be used to change +/// the \ref scheduling_group of the currently running coroutine. +/// +/// If the new scheduling group is different than the current scheduling_group, +/// the coroutine is re-scheduled using the new scheduling group. +/// Otherwise, the coroutine just continues to run with +/// the current scheduling group. +/// +/// `switch_to` returns the current scheduling group +/// to make it easy to switch back to it if needed. +/// +/// Example +/// +/// ``` +/// seastar::future<> cor() { +/// ... // do some preliminary work +/// auto prev_sg = co_await coroutine::switch_to(other_sg); +/// ... // do some work using another scheduling group +/// co_await coroutine::switch_to(prev_sg); +/// ... // do some more work +/// co_return; +/// } +/// ``` + +struct [[nodiscard("must co_await a switch_to object")]] switch_to final : task { + scheduling_group _prev_sg; + scheduling_group _switch_to_sg; + task* _task = nullptr; +public: + explicit switch_to(scheduling_group new_sg) noexcept + : _prev_sg(current_scheduling_group()) + , _switch_to_sg(std::move(new_sg)) + { } + + switch_to(const switch_to&) = delete; + switch_to(switch_to&&) = delete; + + bool await_ready() const noexcept { + return current_scheduling_group() == _switch_to_sg; + } + + template<typename T> + void await_suspend(std::coroutine_handle<T> hndl) noexcept { + auto& t = hndl.promise(); + t.set_scheduling_group(_switch_to_sg); + _task = &t; + schedule(_task); + } + + scheduling_group await_resume() { + return _prev_sg; + } + + virtual void run_and_dispose() noexcept override { } + + virtual task* waiting_task() noexcept override { + return _task; + } +}; + +} // namespace seastar::coroutine diff --git a/src/seastar/include/seastar/http/api_docs.hh b/src/seastar/include/seastar/http/api_docs.hh new file mode 100644 index 000000000..325dd5e15 --- /dev/null +++ b/src/seastar/include/seastar/http/api_docs.hh @@ -0,0 +1,340 @@ +/* + * 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 2015 Cloudius Systems + */ + +#pragma once +#include <seastar/json/json_elements.hh> +#include <seastar/json/formatter.hh> +#include <seastar/http/routes.hh> +#include <seastar/http/transformers.hh> +#include <string> +#include <seastar/util/noncopyable_function.hh> + +namespace seastar { + +namespace httpd { + +struct api_doc : public json::json_base { + json::json_element<std::string> path; + json::json_element<std::string> description; + + void register_params() { + add(&path, "path"); + add(&description, "description"); + + } + api_doc() { + register_params(); + } + api_doc(const api_doc & e) + : json::json_base() + { + register_params(); + path = e.path; + description = e.description; + } + template<class T> + api_doc& operator=(const T& e) { + path = e.path; + description = e.description; + return *this; + } + api_doc& operator=(const api_doc& e) { + path = e.path; + description = e.description; + return *this; + } +}; + +struct api_docs : public json::json_base { + json::json_element<std::string> apiVersion; + json::json_element<std::string> swaggerVersion; + json::json_list<api_doc> apis; + + void register_params() { + add(&apiVersion, "apiVersion"); + add(&swaggerVersion, "swaggerVersion"); + add(&apis, "apis"); + + } + api_docs() { + apiVersion = "0.0.1"; + swaggerVersion = "1.2"; + register_params(); + } + api_docs(const api_docs & e) + : json::json_base() + { + apiVersion = "0.0.1"; + swaggerVersion = "1.2"; + register_params(); + } + template<class T> + api_docs& operator=(const T& e) { + apis = e.apis; + return *this; + } + api_docs& operator=(const api_docs& e) { + apis = e.apis; + return *this; + } +}; + +class api_registry_base : public handler_base { +protected: + sstring _base_path; + sstring _file_directory; + routes& _routes; + +public: + api_registry_base(routes& routes, const sstring& file_directory, + const sstring& base_path) + : _base_path(base_path), _file_directory(file_directory), _routes( + routes) { + } + + void set_route(handler_base* h) { + _routes.put(GET, _base_path, h); + } + virtual ~api_registry_base() = default; +}; + +class api_registry : public api_registry_base { + api_docs _docs; +public: + api_registry(routes& routes, const sstring& file_directory, + const sstring& base_path) + : api_registry_base(routes, file_directory, base_path) { + set_route(this); + } + + future<std::unique_ptr<http::reply>> handle(const sstring& path, + std::unique_ptr<http::request> req, std::unique_ptr<http::reply> rep) override { + rep->_content = json::formatter::to_json(_docs); + rep->done("json"); + return make_ready_future<std::unique_ptr<http::reply>>(std::move(rep)); + } + + void reg(const sstring& api, const sstring& description, + const sstring& alternative_path = "") { + api_doc doc; + doc.description = description; + doc.path = "/" + api; + _docs.apis.push(doc); + sstring path = + (alternative_path == "") ? + _file_directory + api + ".json" : alternative_path; + file_handler* index = new file_handler(path, + new content_replace("json")); + _routes.put(GET, _base_path + "/" + api, index); + } +}; + +class api_registry_builder_base { +protected: + sstring _file_directory; + sstring _base_path; + static const sstring DEFAULT_DIR; + static const sstring DEFAULT_PATH; +public: + api_registry_builder_base(const sstring& file_directory = DEFAULT_DIR, + const sstring& base_path = DEFAULT_PATH) + : _file_directory(file_directory), _base_path(base_path) { + } +}; + +class api_registry_builder : public api_registry_builder_base { +public: + api_registry_builder(const sstring& file_directory = DEFAULT_DIR, + const sstring& base_path = DEFAULT_PATH) + : api_registry_builder_base(file_directory, base_path) { + } + + void set_api_doc(routes& r) { + new api_registry(r, _file_directory, _base_path); + } + + void register_function(routes& r, const sstring& api, + const sstring& description, const sstring& alternative_path = "") { + auto h = r.get_exact_match(GET, _base_path); + if (h) { + // if a handler is found, it was added there by the api_registry_builder + // with the set_api_doc method, so we know it's the type + static_cast<api_registry*>(h)->reg(api, description, alternative_path); + }; + } +}; + +using doc_entry = noncopyable_function<future<>(output_stream<char>&)>; + +/*! + * \brief a helper function that creates a reader from a file + */ + +doc_entry get_file_reader(sstring file_name); + +/*! + * \brief An api doc that support swagger version 2.0 + * + * The result is a unified JSON file with the swagger definitions. + * + * The file content is a concatenation of the doc_entry by the order of + * their entry. + * + * Definitions will be added under the definition section + * + * typical usage: + * + * First entry: + * + { + "swagger": "2.0", + "host": "localhost:10000", + "basePath": "/v2", + "paths": { + + * entry: + "/config/{id}": { + "get": { + "description": "Return a config value", + "operationId": "findConfigId", + "produces": [ + "application/json" + ], + } + } + * + * Closing the entries: + }, + + "definitions": { + ..... + + ..... + } +} + * + */ +class api_docs_20 { + std::vector<doc_entry> _apis; + content_replace _transform; + std::vector<doc_entry> _definitions; + +public: + future<> write(output_stream<char>&&, std::unique_ptr<http::request> req); + + void add_api(doc_entry&& f) { + _apis.emplace_back(std::move(f)); + } + + void add_definition(doc_entry&& f) { + _definitions.emplace_back(std::move(f)); + } +}; + +class api_registry_20 : public api_registry_base { + api_docs_20 _docs; +public: + api_registry_20(routes& routes, const sstring& file_directory, + const sstring& base_path) + : api_registry_base(routes, file_directory, base_path) { + set_route(this); + } + + future<std::unique_ptr<http::reply>> handle(const sstring& path, + std::unique_ptr<http::request> req, std::unique_ptr<http::reply> rep) override { + rep->write_body("json", [this, req = std::move(req)] (output_stream<char>&& os) mutable { + return _docs.write(std::move(os), std::move(req)); + }); + return make_ready_future<std::unique_ptr<http::reply>>(std::move(rep)); + } + + virtual void reg(doc_entry&& f) { + _docs.add_api(std::move(f)); + } + + virtual void add_definition(doc_entry&& f) { + _docs.add_definition(std::move(f)); + } +}; + +class api_registry_builder20 : public api_registry_builder_base { + api_registry_20* get_register_base(routes& r) { + auto h = r.get_exact_match(GET, _base_path); + if (h) { + // if a handler is found, it was added there by the api_registry_builder + // with the set_api_doc method, so we know it's the type + return static_cast<api_registry_20*>(h); + } + return nullptr; + } + +public: + api_registry_builder20(const sstring& file_directory = DEFAULT_DIR, + const sstring& base_path = DEFAULT_PATH) + : api_registry_builder_base(file_directory, base_path) { + } + + void set_api_doc(routes& r) { + new api_registry_20(r, _file_directory, _base_path); + } + + /*! + * \brief register a doc_entry + * This doc_entry can be used to either take the definition from a file + * or generate them dynamically. + */ + void register_function(routes& r, doc_entry&& f) { + auto h = get_register_base(r); + if (h) { + h->reg(std::move(f)); + } + } + /*! + * \brief register an API + */ + void register_api_file(routes& r, const sstring& api) { + register_function(r, get_file_reader(_file_directory + "/" + api + ".json")); + } + + + /*! + * Add a footer doc_entry + */ + void add_definition(routes& r, doc_entry&& f) { + auto h = get_register_base(r); + if (h) { + h->add_definition(std::move(f)); + } + + } + + /*! + * Add a definition file + */ + void add_definitions_file(routes& r, const sstring& file) { + add_definition(r, get_file_reader(_file_directory + file + ".def.json" )); + } + +}; + +} + +} diff --git a/src/seastar/include/seastar/http/client.hh b/src/seastar/include/seastar/http/client.hh new file mode 100644 index 000000000..dbddcede7 --- /dev/null +++ b/src/seastar/include/seastar/http/client.hh @@ -0,0 +1,100 @@ +/* + * 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) 2022 Scylladb, Ltd. + */ + +#include <seastar/net/api.hh> +#include <seastar/core/iostream.hh> + +namespace seastar { + +namespace http { + +struct request; +struct reply; + +namespace experimental { + +/** + * \brief Class connection represents an HTTP connection over a given transport + * + * Check the demos/http_client_demo.cc for usage example + */ + +class connection { + connected_socket _fd; + input_stream<char> _read_buf; + output_stream<char> _write_buf; + +public: + /** + * \brief Create an http connection + * + * Construct the connection that will work over the provided \fd transport socket + * + */ + connection(connected_socket&& fd); + + /** + * \brief Send the request and wait for response + * + * Sends the provided request to the server, and returns a future that will resolve + * into the server response. + * + * If the request was configured with the set_expects_continue() and the server replied + * early with some error code, this early reply will be returned back. + * + * The returned reply only contains the status and headers. To get the reply body the + * caller should read it via the input_stream provided by the connection.in() method. + * + * \param rq -- request to be sent + * + */ + future<reply> make_request(request rq); + + /** + * \brief Get a reference on the connection input stream + * + * The stream can be used to get back the server response body. After the stream is + * finished, the reply can be additionally updated with trailing headers and chunk + * extentions + * + */ + input_stream<char> in(reply& rep); + + /** + * \brief Closes the connection + * + * Connection must be closed regardless of whether there was an exception making the + * request or not + */ + future<> close(); + +private: + future<> send_request_head(request& rq); + future<std::optional<reply>> maybe_wait_for_continue(request& req); + future<> write_body(request& rq); + future<reply> recv_reply(); +}; + +} // experimental namespace + +} // http namespace + +} // seastar namespace diff --git a/src/seastar/include/seastar/http/common.hh b/src/seastar/include/seastar/http/common.hh new file mode 100644 index 000000000..595377605 --- /dev/null +++ b/src/seastar/include/seastar/http/common.hh @@ -0,0 +1,91 @@ +/* + * 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 2015 Cloudius Systems + */ + +#pragma once + +#include <unordered_map> +#include <seastar/core/sstring.hh> +#include <seastar/core/iostream.hh> + +namespace seastar { + +namespace http { +namespace internal { +output_stream<char> make_http_chunked_output_stream(output_stream<char>& out); +// The len parameter defines the maximum number of bytes to be written. After the +// stream is closed, the len is updated with the actual number of bytes written. +output_stream<char> make_http_content_length_output_stream(output_stream<char>& out, size_t& len); +} // internal namespace +} // http namespace + +namespace httpd { + + +class parameters { + std::unordered_map<sstring, sstring> params; +public: + const sstring& path(const sstring& key) const { + return params.at(key); + } + + sstring operator[](const sstring& key) const { + return params.at(key).substr(1); + } + + const sstring& at(const sstring& key) const { + return path(key); + } + + bool exists(const sstring& key) const { + return params.find(key) != params.end(); + } + + void set(const sstring& key, const sstring& value) { + params[key] = value; + } + + void clear() { + params.clear(); + } + +}; + +enum operation_type { + GET, POST, PUT, DELETE, HEAD, OPTIONS, TRACE, CONNECT, NUM_OPERATION +}; + +/** + * Translate the string command to operation type + * @param type the string "GET" or "POST" + * @return the operation_type + */ +operation_type str2type(const sstring& type); + +/** + * Translate the operation type to command string + * @param type the string GET or POST + * @return the command string "GET" or "POST" + */ +sstring type2str(operation_type type); + +} + +} diff --git a/src/seastar/include/seastar/http/exception.hh b/src/seastar/include/seastar/http/exception.hh new file mode 100644 index 000000000..6893df62c --- /dev/null +++ b/src/seastar/include/seastar/http/exception.hh @@ -0,0 +1,150 @@ +/* + * 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 2015 Cloudius Systems + */ + +#pragma once +#include <seastar/util/log.hh> +#include <seastar/http/reply.hh> +#include <seastar/json/json_elements.hh> + +namespace seastar { + +namespace httpd { + +/** + * The base_exception is a base for all http exception. + * It contains a message that will be return as the message content + * and a status that will be return as a status code. + */ +class base_exception : public std::exception { +public: + base_exception(const std::string& msg, http::reply::status_type status) + : _msg(msg), _status(status) { + } + + virtual const char* what() const throw () { + return _msg.c_str(); + } + + http::reply::status_type status() const { + return _status; + } + + virtual const std::string& str() const { + return _msg; + } +private: + std::string _msg; + http::reply::status_type _status; + +}; + +/** + * Throwing this exception will result in a redirect to the given url + */ +class redirect_exception : public base_exception { +public: + redirect_exception(const std::string& url) + : base_exception("", http::reply::status_type::moved_permanently), url( + url) { + } + std::string url; +}; + +/** + * Throwing this exception will result in a 404 not found result + */ +class not_found_exception : public base_exception { +public: + not_found_exception(const std::string& msg = "Not found") + : base_exception(msg, http::reply::status_type::not_found) { + } +}; + +/** + * Throwing this exception will result in a 400 bad request result + */ + +class bad_request_exception : public base_exception { +public: + bad_request_exception(const std::string& msg) + : base_exception(msg, http::reply::status_type::bad_request) { + } +}; + +class bad_param_exception : public bad_request_exception { +public: + bad_param_exception(const std::string& msg) + : bad_request_exception(msg) { + } +}; + +class missing_param_exception : public bad_request_exception { +public: + missing_param_exception(const std::string& param) + : bad_request_exception( + std::string("Missing mandatory parameter '") + param + "'") { + } +}; + +class bad_chunk_exception : public bad_request_exception { +public: + bad_chunk_exception(const std::string& msg) + : bad_request_exception( + std::string("Can't read body chunk in a 'chunked' request '") + msg + "'") { + } +}; + +class server_error_exception : public base_exception { +public: + server_error_exception(const std::string& msg) + : base_exception(msg, http::reply::status_type::internal_server_error) { + } +}; + +class json_exception : public json::json_base { +public: + json::json_element<std::string> _msg; + json::json_element<int> _code; + void register_params() { + add(&_msg, "message"); + add(&_code, "code"); + } + + json_exception(const base_exception & e) { + set(e.str(), e.status()); + } + + json_exception(std::exception_ptr e) { + std::ostringstream exception_description; + exception_description << e; + set(exception_description.str(), http::reply::status_type::internal_server_error); + } +private: + void set(const std::string& msg, http::reply::status_type code) { + register_params(); + _msg = msg; + _code = (int) code; + } +}; + +} + +} diff --git a/src/seastar/include/seastar/http/file_handler.hh b/src/seastar/include/seastar/http/file_handler.hh new file mode 100644 index 000000000..ab6d6675e --- /dev/null +++ b/src/seastar/include/seastar/http/file_handler.hh @@ -0,0 +1,172 @@ +/* + * 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 2015 Cloudius Systems + */ + +#pragma once + +#include <seastar/http/handlers.hh> +#include <seastar/core/iostream.hh> + +namespace seastar { + +namespace httpd { + +/** + * This is a base class for file transformer. + * + * File transformer adds the ability to modify a file content before returning + * the results, it acts as a factory class for output_stream. + * + * The transformer decides according to the file extension if transforming is + * needed. + * + * If a transformation is needed it would create a new output stream from the given stream. + */ +class file_transformer { +public: + /** + * Any file transformer should implement this method. + * @param req the request + * @param extension the file extension originating the content + * returns a new output stream to be used when writing the file to the reply + */ + virtual output_stream<char> transform(std::unique_ptr<http::request> req, + const sstring& extension, output_stream<char>&& s) = 0; + + virtual ~file_transformer() = default; +}; + +/** + * A base class for handlers that interact with files. + * directory and file handlers both share some common logic + * with regards to file handling. + * they both needs to read a file from the disk, optionally transform it, + * and return the result or page not found on error + */ +class file_interaction_handler : public handler_base { +public: + file_interaction_handler(file_transformer* p = nullptr) + : transformer(p) { + + } + + ~file_interaction_handler(); + + /** + * Allows setting a transformer to be used with the files returned. + * @param t the file transformer to use + * @return this + */ + file_interaction_handler* set_transformer(file_transformer* t) { + transformer = t; + return this; + } + + /** + * if the url ends without a slash redirect + * @param req the request + * @param rep the reply + * @return true on redirect + */ + bool redirect_if_needed(const http::request& req, http::reply& rep) const; + + /** + * A helper method that returns the file extension. + * @param file the file to check + * @return the file extension + */ + static sstring get_extension(const sstring& file); + +protected: + + /** + * read a file from the disk and return it in the replay. + * @param file the full path to a file on the disk + * @param req the reuest + * @param rep the reply + */ + future<std::unique_ptr<http::reply> > read(sstring file, + std::unique_ptr<http::request> req, std::unique_ptr<http::reply> rep); + file_transformer* transformer; + + output_stream<char> get_stream(std::unique_ptr<http::request> req, + const sstring& extension, output_stream<char>&& s); +}; + +/** + * The directory handler get a disk path in the + * constructor. + * and expect a path parameter in the handle method. + * it would concatenate the two and return the file + * e.g. if the path is /usr/mgmt/public in the path + * parameter is index.html + * handle will return the content of /usr/mgmt/public/index.html + */ +class directory_handler : public file_interaction_handler { +public: + + /** + * The directory handler map a base path and a path parameter to a file + * @param doc_root the root directory to search the file from. + * @param transformer an optional file transformer + * For example if the root is '/usr/mgmt/public' and the path parameter + * will be '/css/style.css' the file wil be /usr/mgmt/public/css/style.css' + */ + explicit directory_handler(const sstring& doc_root, + file_transformer* transformer = nullptr); + + future<std::unique_ptr<http::reply>> handle(const sstring& path, + std::unique_ptr<http::request> req, std::unique_ptr<http::reply> rep) override; + +private: + sstring doc_root; +}; + +/** + * The file handler get a path to a file on the disk + * in the constructor. + * it will always return the content of the file. + */ +class file_handler : public file_interaction_handler { +public: + + /** + * The file handler map a file to a url + * @param file the full path to the file on the disk + * @param transformer an optional file transformer + * @param force_path check if redirect is needed upon `handle` + */ + explicit file_handler(const sstring& file, file_transformer* transformer = + nullptr, bool force_path = true) + : file_interaction_handler(transformer), file(file), force_path( + force_path) { + } + + future<std::unique_ptr<http::reply>> handle(const sstring& path, + std::unique_ptr<http::request> req, std::unique_ptr<http::reply> rep) override; + +private: + sstring file; + bool force_path; +}; + +} + +} diff --git a/src/seastar/include/seastar/http/function_handlers.hh b/src/seastar/include/seastar/http/function_handlers.hh new file mode 100644 index 000000000..9fd81e63f --- /dev/null +++ b/src/seastar/include/seastar/http/function_handlers.hh @@ -0,0 +1,132 @@ +/* + * 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 2015 Cloudius Systems + */ + +#pragma once + +#include <seastar/http/handlers.hh> +#include <functional> +#include <seastar/json/json_elements.hh> + +namespace seastar { + +namespace httpd { + +/** + * A request function is a lambda expression that gets only the request + * as its parameter + */ +typedef std::function<sstring(const_req req)> request_function; + +/** + * A handle function is a lambda expression that gets request and reply + */ +typedef std::function<sstring(const_req req, http::reply&)> handle_function; + +/** + * A json request function is a lambda expression that gets only the request + * as its parameter and return a json response. + * Using the json response is done implicitly. + */ +typedef std::function<json::json_return_type(const_req req)> json_request_function; + +/** + * A future_json_function is a function that returns a future json reponse. + * Similiar to the json_request_function, using the json reponse is done + * implicitly. + */ +typedef std::function< + future<json::json_return_type>(std::unique_ptr<http::request> req)> future_json_function; + +typedef std::function< + future<std::unique_ptr<http::reply>>(std::unique_ptr<http::request> req, + std::unique_ptr<http::reply> rep)> future_handler_function; +/** + * The function handler get a lambda expression in the constructor. + * it will call that expression to get the result + * This is suited for very simple handlers + * + */ +class function_handler : public handler_base { +public: + + function_handler(const handle_function & f_handle, const sstring& type) + : _f_handle( + [f_handle](std::unique_ptr<http::request> req, std::unique_ptr<http::reply> rep) { + rep->_content += f_handle(*req.get(), *rep.get()); + return make_ready_future<std::unique_ptr<http::reply>>(std::move(rep)); + }), _type(type) { + } + + function_handler(const future_handler_function& f_handle, const sstring& type) + : _f_handle(f_handle), _type(type) { + } + + function_handler(const request_function & _handle, const sstring& type) + : _f_handle( + [_handle](std::unique_ptr<http::request> req, std::unique_ptr<http::reply> rep) { + rep->_content += _handle(*req.get()); + return make_ready_future<std::unique_ptr<http::reply>>(std::move(rep)); + }), _type(type) { + } + + function_handler(const json_request_function& _handle) + : _f_handle( + [_handle](std::unique_ptr<http::request> req, std::unique_ptr<http::reply> rep) { + json::json_return_type res = _handle(*req.get()); + rep->_content += res._res; + return make_ready_future<std::unique_ptr<http::reply>>(std::move(rep)); + }), _type("json") { + } + + function_handler(const future_json_function& _handle) + : _f_handle( + [_handle](std::unique_ptr<http::request> req, std::unique_ptr<http::reply> rep) { + return _handle(std::move(req)).then([rep = std::move(rep)](json::json_return_type&& res) mutable { + if (res._body_writer) { + rep->write_body("json", std::move(res._body_writer)); + } else { + rep->_content += res._res; + + } + return make_ready_future<std::unique_ptr<http::reply>>(std::move(rep)); + }); + }), _type("json") { + } + + function_handler(const function_handler&) = default; + + future<std::unique_ptr<http::reply>> handle(const sstring& path, + std::unique_ptr<http::request> req, std::unique_ptr<http::reply> rep) override { + return _f_handle(std::move(req), std::move(rep)).then( + [this](std::unique_ptr<http::reply> rep) { + rep->done(_type); + return make_ready_future<std::unique_ptr<http::reply>>(std::move(rep)); + }); + } + +protected: + future_handler_function _f_handle; + sstring _type; +}; + +} + +} diff --git a/src/seastar/include/seastar/http/handlers.hh b/src/seastar/include/seastar/http/handlers.hh new file mode 100644 index 000000000..a91d5f7ba --- /dev/null +++ b/src/seastar/include/seastar/http/handlers.hh @@ -0,0 +1,73 @@ +/* + * 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 2015 Cloudius Systems + */ + +#pragma once + +#include <seastar/http/request.hh> +#include <seastar/http/common.hh> +#include <seastar/http/reply.hh> + +#include <unordered_map> + +namespace seastar { + +namespace httpd { + +typedef const http::request& const_req; + +/** + * handlers holds the logic for serving an incoming request. + * All handlers inherit from the base httpserver_handler and + * implement the handle method. + * + */ +class handler_base { +public: + /** + * All handlers should implement this method. + * It fill the reply according to the request. + * @param path the url path used in this call + * @param req the original request + * @param rep the reply + */ + virtual future<std::unique_ptr<http::reply> > handle(const sstring& path, + std::unique_ptr<http::request> req, std::unique_ptr<http::reply> rep) = 0; + + virtual ~handler_base() = default; + + /** + * Add a mandatory parameter + * @param param a parameter name + * @return a reference to the handler + */ + handler_base& mandatory(const sstring& param) { + _mandatory_param.push_back(param); + return *this; + } + + std::vector<sstring> _mandatory_param; + +}; + +} + +} + diff --git a/src/seastar/include/seastar/http/httpd.hh b/src/seastar/include/seastar/http/httpd.hh new file mode 100644 index 000000000..e9f9606d4 --- /dev/null +++ b/src/seastar/include/seastar/http/httpd.hh @@ -0,0 +1,228 @@ +/* + * 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 2015 Cloudius Systems + */ + +#pragma once + +#include <seastar/http/request_parser.hh> +#include <seastar/http/request.hh> +#include <seastar/core/seastar.hh> +#include <seastar/core/sstring.hh> +#include <seastar/core/app-template.hh> +#include <seastar/core/circular_buffer.hh> +#include <seastar/core/distributed.hh> +#include <seastar/core/queue.hh> +#include <seastar/core/gate.hh> +#include <seastar/core/metrics_registration.hh> +#include <seastar/util/std-compat.hh> +#include <iostream> +#include <algorithm> +#include <unordered_map> +#include <queue> +#include <bitset> +#include <limits> +#include <cctype> +#include <vector> +#include <boost/intrusive/list.hpp> +#include <seastar/http/routes.hh> +#include <seastar/net/tls.hh> +#include <seastar/core/shared_ptr.hh> + +namespace seastar { + +namespace http { +struct reply; +} + +namespace httpd { + +class http_server; +class http_stats; + +using namespace std::chrono_literals; + +class http_stats { + metrics::metric_groups _metric_groups; +public: + http_stats(http_server& server, const sstring& name); +}; + +class connection : public boost::intrusive::list_base_hook<> { + http_server& _server; + connected_socket _fd; + input_stream<char> _read_buf; + output_stream<char> _write_buf; + static constexpr size_t limit = 4096; + using tmp_buf = temporary_buffer<char>; + http_request_parser _parser; + std::unique_ptr<http::request> _req; + std::unique_ptr<http::reply> _resp; + // null element marks eof + queue<std::unique_ptr<http::reply>> _replies { 10 }; + bool _done = false; +public: + [[deprecated("use connection(http_server&, connected_socket&&)")]] + connection(http_server& server, connected_socket&& fd, + socket_address) : connection(server, std::move(fd)) {} + connection(http_server& server, connected_socket&& fd) + : _server(server), _fd(std::move(fd)), _read_buf(_fd.input()), _write_buf( + _fd.output()) { + on_new_connection(); + } + ~connection(); + void on_new_connection(); + + future<> process(); + void shutdown(); + future<> read(); + future<> read_one(); + future<> respond(); + future<> do_response_loop(); + + void set_headers(http::reply& resp); + + future<> start_response(); + + future<bool> generate_reply(std::unique_ptr<http::request> req); + void generate_error_reply_and_close(std::unique_ptr<http::request> req, http::reply::status_type status, const sstring& msg); + + future<> write_body(); + + output_stream<char>& out(); +}; + +class http_server_tester; + +class http_server { + std::vector<server_socket> _listeners; + http_stats _stats; + uint64_t _total_connections = 0; + uint64_t _current_connections = 0; + uint64_t _requests_served = 0; + uint64_t _read_errors = 0; + uint64_t _respond_errors = 0; + shared_ptr<seastar::tls::server_credentials> _credentials; + sstring _date = http_date(); + timer<> _date_format_timer { [this] {_date = http_date();} }; + size_t _content_length_limit = std::numeric_limits<size_t>::max(); + bool _content_streaming = false; + gate _task_gate; +public: + routes _routes; + using connection = seastar::httpd::connection; + explicit http_server(const sstring& name) : _stats(*this, name) { + _date_format_timer.arm_periodic(1s); + } + /*! + * \brief set tls credentials for the server + * Setting the tls credentials will set the http-server to work in https mode. + * + * To use the https, create server credentials and pass it to the server before it starts. + * + * Use case example using seastar threads for clarity: + + distributed<http_server> server; // typical server + + seastar::shared_ptr<seastar::tls::credentials_builder> creds = seastar::make_shared<seastar::tls::credentials_builder>(); + sstring ms_cert = "MyCertificate.crt"; + sstring ms_key = "MyKey.key"; + + creds->set_dh_level(seastar::tls::dh_params::level::MEDIUM); + + creds->set_x509_key_file(ms_cert, ms_key, seastar::tls::x509_crt_format::PEM).get(); + creds->set_system_trust().get(); + + + server.invoke_on_all([creds](http_server& server) { + server.set_tls_credentials(creds->build_server_credentials()); + return make_ready_future<>(); + }).get(); + * + */ + void set_tls_credentials(shared_ptr<seastar::tls::server_credentials> credentials); + + size_t get_content_length_limit() const; + + void set_content_length_limit(size_t limit); + + bool get_content_streaming() const; + + void set_content_streaming(bool b); + + future<> listen(socket_address addr, listen_options lo); + future<> listen(socket_address addr); + future<> stop(); + + future<> do_accepts(int which); + + uint64_t total_connections() const; + uint64_t current_connections() const; + uint64_t requests_served() const; + uint64_t read_errors() const; + uint64_t reply_errors() const; + // Write the current date in the specific "preferred format" defined in + // RFC 7231, Section 7.1.1.1. + static sstring http_date(); +private: + future<> do_accept_one(int which); + boost::intrusive::list<connection> _connections; + friend class seastar::httpd::connection; + friend class http_server_tester; +}; + +class http_server_tester { +public: + static std::vector<server_socket>& listeners(http_server& server) { + return server._listeners; + } +}; + +/* + * A helper class to start, set and listen an http server + * typical use would be: + * + * auto server = new http_server_control(); + * server->start().then([server] { + * server->set_routes(set_routes); + * }).then([server, port] { + * server->listen(port); + * }).then([port] { + * std::cout << "Seastar HTTP server listening on port " << port << " ...\n"; + * }); + */ +class http_server_control { + std::unique_ptr<distributed<http_server>> _server_dist; +private: + static sstring generate_server_name(); +public: + http_server_control() : _server_dist(new distributed<http_server>) { + } + + future<> start(const sstring& name = generate_server_name()); + future<> stop(); + future<> set_routes(std::function<void(routes& r)> fun); + future<> listen(socket_address addr); + future<> listen(socket_address addr, listen_options lo); + distributed<http_server>& server(); +}; + +} + +} diff --git a/src/seastar/include/seastar/http/internal/content_source.hh b/src/seastar/include/seastar/http/internal/content_source.hh new file mode 100644 index 000000000..7da5f5c22 --- /dev/null +++ b/src/seastar/include/seastar/http/internal/content_source.hh @@ -0,0 +1,228 @@ +/* + * 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 2020 ScyllaDB + */ + +#pragma once + +#include <seastar/http/chunk_parsers.hh> +#include <seastar/core/iostream.hh> +#include <seastar/core/temporary_buffer.hh> +#include <seastar/http/common.hh> +#include <seastar/util/log.hh> +#include <seastar/http/exception.hh> + +namespace seastar { + +namespace httpd { + +namespace internal { + +/* + * An input_stream wrapper that allows to read only "length" bytes + * from it, used to handle requests with large bodies. + * */ +class content_length_source_impl : public data_source_impl { + input_stream<char>& _inp; + size_t _remaining_bytes = 0; +public: + content_length_source_impl(input_stream<char>& inp, size_t length) + : _inp(inp), _remaining_bytes(length) { + } + + virtual future<temporary_buffer<char>> get() override { + if (_remaining_bytes == 0) { + return make_ready_future<temporary_buffer<char>>(); + } + return _inp.read_up_to(_remaining_bytes).then([this] (temporary_buffer<char> tmp_buf) { + _remaining_bytes -= tmp_buf.size(); + return tmp_buf; + }); + } + + virtual future<temporary_buffer<char>> skip(uint64_t n) override { + uint64_t skip_bytes = std::min(n, _remaining_bytes); + _remaining_bytes -= skip_bytes; + return _inp.skip(skip_bytes).then([] { + return temporary_buffer<char>(); + }); + } + + virtual future<> close() override { + return make_ready_future<>(); + } +}; + +/* + * An input_stream wrapper that decodes a request body + * with "chunked" encoding. + * */ +class chunked_source_impl : public data_source_impl { + class chunk_parser { + enum class parsing_state + : uint8_t { + size_and_ext, + body, + trailer_part + }; + http_chunk_size_and_ext_parser _size_and_ext_parser; + http_chunk_trailer_parser _trailer_parser; + + temporary_buffer<char> _buf; + size_t _current_chunk_bytes_read = 0; + size_t _current_chunk_length; + parsing_state _ps = parsing_state::size_and_ext; + bool _end_of_request = false; + // references to fields in the request structure + std::unordered_map<sstring, sstring>& _chunk_extensions; + std::unordered_map<sstring, sstring>& _trailing_headers; + using consumption_result_type = consumption_result<char>; + public: + chunk_parser(std::unordered_map<sstring, sstring>& chunk_extensions, std::unordered_map<sstring, sstring>& trailing_headers) + : _chunk_extensions(chunk_extensions), _trailing_headers(trailing_headers) { + _size_and_ext_parser.init(); + } + temporary_buffer<char> buf() { + _current_chunk_bytes_read += _buf.size(); + return std::move(_buf); + } + + future<consumption_result_type> operator()(temporary_buffer<char> data) { + if (_buf.size() || _end_of_request || data.empty()) { + // return if we have already read some content (_buf.size()), we have already reached the end of the chunked request (_end_of_request), + // or the underlying stream reached eof (data.empty()) + return make_ready_future<consumption_result_type>(stop_consuming(std::move(data))); + } + switch (_ps) { + // "data" buffer is non-empty + case parsing_state::size_and_ext: + return _size_and_ext_parser(std::move(data)).then([this] (std::optional<temporary_buffer<char>> res) { + if (res.has_value()) { + if (_size_and_ext_parser.failed()) { + return make_exception_future<consumption_result_type>(bad_request_exception("Can't parse chunk size and extensions")); + } + // save extensions + auto parsed_extensions = _size_and_ext_parser.get_parsed_extensions(); + _chunk_extensions.merge(parsed_extensions); + for (auto& key_val : parsed_extensions) { + _chunk_extensions[key_val.first] += sstring(",") + key_val.second; + } + + // save size + auto size_string = _size_and_ext_parser.get_size(); + if (size_string.size() > 16) { + return make_exception_future<consumption_result_type>(bad_chunk_exception("Chunk length too big")); + } + _current_chunk_bytes_read = 0; + _current_chunk_length = strtol(size_string.c_str(), nullptr, 16); + + if (_current_chunk_length == 0) { + _ps = parsing_state::trailer_part; + _trailer_parser.init(); + } else { + _ps = parsing_state::body; + } + if (res->empty()) { + return make_ready_future<consumption_result_type>(continue_consuming{}); + } + return this->operator()(std::move(res.value())); + } else { + return make_ready_future<consumption_result_type>(continue_consuming{}); + } + }); + case parsing_state::body: + // read the new data into _buf + if (_current_chunk_bytes_read < _current_chunk_length) { + size_t to_read = std::min(_current_chunk_length - _current_chunk_bytes_read, data.size()); + if (_buf.empty()) { + _buf = data.share(0, to_read); + } + data.trim_front(to_read); + return make_ready_future<consumption_result_type>(stop_consuming(std::move(data))); + } + + // chunk body is finished, we haven't entered the previous if, so "data" is still non-empty + if (_current_chunk_bytes_read == _current_chunk_length) { + // we haven't read \r yet + if (data.get()[0] != '\r') { + return make_exception_future<consumption_result_type>(bad_chunk_exception("The actual chunk length exceeds the specified length")); + } else { + _current_chunk_bytes_read++; + data.trim_front(1); + if (data.empty()) { + return make_ready_future<consumption_result_type>(continue_consuming{}); + } + } + } + if (_current_chunk_bytes_read == _current_chunk_length + 1) { + // we haven't read \n but have \r + if (data.get()[0] != '\n') { + return make_exception_future<consumption_result_type>(bad_chunk_exception("The actual chunk length exceeds the specified length")); + } else { + _ps = parsing_state::size_and_ext; + _size_and_ext_parser.init(); + data.trim_front(1); + if (data.empty()) { + return make_ready_future<consumption_result_type>(continue_consuming{}); + } + } + } + return this->operator()(std::move(data)); + case parsing_state::trailer_part: + return _trailer_parser(std::move(data)).then([this] (std::optional<temporary_buffer<char>> res) { + if (res.has_value()) { + if (_trailer_parser.failed()) { + return make_exception_future<consumption_result_type>(bad_request_exception("Can't parse chunked request trailer")); + } + // save trailing headers + _trailing_headers = _trailer_parser.get_parsed_headers(); + _end_of_request = true; + return make_ready_future<consumption_result_type>(stop_consuming(std::move(*res))); + } else { + return make_ready_future<consumption_result_type>(continue_consuming{}); + } + }); + } + __builtin_unreachable(); + } + }; + input_stream<char>& _inp; + chunk_parser _chunk; + +public: + chunked_source_impl(input_stream<char>& inp, std::unordered_map<sstring, sstring>& chunk_extensions, std::unordered_map<sstring, sstring>& trailing_headers) + : _inp(inp), _chunk(chunk_extensions, trailing_headers) { + } + + virtual future<temporary_buffer<char>> get() override { + return _inp.consume(_chunk).then([this] () mutable { + return _chunk.buf(); + }); + } + + virtual future<> close() override { + return make_ready_future<>(); + } +}; + +} // namespace internal + +} // namespace httpd + +} diff --git a/src/seastar/include/seastar/http/json_path.hh b/src/seastar/include/seastar/http/json_path.hh new file mode 100644 index 000000000..a9bc22ca5 --- /dev/null +++ b/src/seastar/include/seastar/http/json_path.hh @@ -0,0 +1,190 @@ +/* + * 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 2015 Cloudius Systems + */ + +#pragma once + +#include <vector> +#include <unordered_map> +#include <tuple> +#include <seastar/http/common.hh> +#include <seastar/core/sstring.hh> +#include <seastar/http/routes.hh> +#include <seastar/http/function_handlers.hh> + +namespace seastar { + +namespace httpd { + +/** + * A json_operation contain a method and a nickname. + * operation are associated to a path, that can + * have multiple methods + */ +struct json_operation { + /** + * default constructor + */ + json_operation() + : method(GET) { + } + + /** + * Construct with assignment + * @param method the http method type + * @param nickname the http nickname + */ + json_operation(operation_type method, const sstring& nickname) + : method(method), nickname(nickname) { + } + + operation_type method; + sstring nickname; + +}; + +/** + * path description holds the path in the system. + * It maps a nickname to an operation, which allows + * defining the operation (path and method) by its + * nickname. + * + * A path_description has a type, a base path and a list of + * url components. + * Each component can be a regular path parameter, a path parameter that + * contains everything until the end of the path or a fixed string. + * + * the description are taken from the json swagger + * definition file, during auto code generation in the + * compilation. + */ +struct path_description { + // + enum class url_component_type { + PARAM, // a normal path parameter (starts with / and end with / or end of path) + PARAM_UNTIL_END_OF_PATH, // a parameter that contains all the path entil its end + FIXED_STRING, // a fixed string inside the path, must be a full match and does not count + // as a parameter + }; + + // path_part is either a parameter or a fixed string + struct path_part { + sstring name; + url_component_type type = url_component_type::PARAM; + }; + + /** + * default empty constructor + */ + path_description() = default; + + /** + * constructor for path with parameters + * The constructor is used by + * @param path the url path + * @param method the http method + * @param nickname the nickname + * @param path_parameters path parameters and url parts of the path + * @param mandatory_params the names of the mandatory query parameters + */ + path_description(const sstring& path, operation_type method, + const sstring& nickname, + const std::vector<std::pair<sstring, bool>>& path_parameters, + const std::vector<sstring>& mandatory_params); + + /** + * constructor for path with parameters + * The constructor is used by + * @param path the url path + * @param method the http method + * @param nickname the method nickname + * @param path_parameters path parameters and url parts of the path + * @param mandatory_params the names of the mandatory query parameters + */ + path_description(const sstring& path, operation_type method, + const sstring& nickname, + const std::initializer_list<path_part>& path_parameters, + const std::vector<sstring>& mandatory_params); + + /** + * Add a parameter to the path definition + * for example, if the url should match /file/{path} + * The constructor would be followed by a call to + * pushparam("path") + * + * @param param the name of the parameters, this name will + * be used by the handler to identify the parameters. + * A name can appear at most once in a description + * @param all_path when set to true the parameter will assume to match + * until the end of the url. + * This is useful for situation like file path with + * a rule like /file/{path} and a url /file/etc/hosts. + * path should be equal to /ets/hosts and not only /etc + * @return the current path description + */ + path_description* pushparam(const sstring& param, + bool all_path = false) { + params.push_back( { param, (all_path) ? url_component_type::PARAM_UNTIL_END_OF_PATH : url_component_type::PARAM}); + return this; + } + + /*! + * \brief adds a fixed string as part of the path + * This will allow to combine fixed string URL parts and path parameters. + * + * For example to map a path like: + * /mypath/{param1}/morepath/{param2} + * path_description p("/mypath", operation_type::GET); + * p.pushparam("param1)->pushurl("morepath")->pushparam("param2"); + */ + path_description* push_static_path_part(const sstring& url) { + params.push_back( { url, url_component_type::FIXED_STRING}); + return this; + } + /** + * adds a mandatory query parameter to the path + * this parameter will be check before calling a handler + * @param param the parameter to head + * @return a pointer to the current path description + */ + path_description* pushmandatory_param(const sstring& param) { + mandatory_queryparams.push_back(param); + return this; + } + + std::vector<path_part> params; + sstring path; + json_operation operations; + mutable routes::rule_cookie _cookie; + + std::vector<sstring> mandatory_queryparams; + + void set(routes& _routes, handler_base* handler) const; + + void set(routes& _routes, const json_request_function& f) const; + + void set(routes& _routes, const future_json_function& f) const; + + void unset(routes& _routes) const; +}; + +} + +} diff --git a/src/seastar/include/seastar/http/matcher.hh b/src/seastar/include/seastar/http/matcher.hh new file mode 100644 index 000000000..3921cab8e --- /dev/null +++ b/src/seastar/include/seastar/http/matcher.hh @@ -0,0 +1,111 @@ +/* + * 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 2015 Cloudius Systems + */ + +#pragma once + +#include <seastar/http/common.hh> + +#include <seastar/core/sstring.hh> + +namespace seastar { + +namespace httpd { + +/** + * a base class for the url matching. + * Each implementation check if the given url matches a criteria + */ +class matcher { +public: + + virtual ~matcher() = default; + + /** + * check if the given url matches the rule + * @param url the url to check + * @param ind the position to start from + * @param param fill the parameters hash + * @return the end of of the matched part, or sstring::npos if not matched + */ + virtual size_t match(const sstring& url, size_t ind, parameters& param) = 0; +}; + +/** + * Check if the url match a parameter and fill the parameters object + * + * Note that a non empty url will always return true with the parameters + * object filled + * + * Assume that the rule is /file/{path}/ and the param_matcher identify + * the /{path} + * + * For all non empty values, match will return true. + * If the entire url is /file/etc/hosts, and the part that is passed to + * param_matcher is /etc/hosts, if entire_path is true, the match will be + * '/etc/hosts' If entire_path is false, the match will be '/etc' + */ +class param_matcher : public matcher { +public: + /** + * Constructor + * @param name the name of the parameter, will be used as the key + * in the parameters object + * @param entire_path when set to true, the matched parameters will + * include all the remaining url until the end of it. + * when set to false the match will terminate at the next slash + */ + explicit param_matcher(const sstring& name, bool entire_path = false) + : _name(name), _entire_path(entire_path) { + } + + virtual size_t match(const sstring& url, size_t ind, parameters& param) + override; +private: + sstring _name; + bool _entire_path; +}; + +/** + * Check if the url match a predefine string. + * + * When parsing a match rule such as '/file/{path}' the str_match would parse + * the '/file' part + */ +class str_matcher : public matcher { +public: + /** + * Constructor + * @param cmp the string to match + */ + explicit str_matcher(const sstring& cmp) + : _cmp(cmp), _len(cmp.size()) { + } + + virtual size_t match(const sstring& url, size_t ind, parameters& param) + override; +private: + sstring _cmp; + unsigned _len; +}; + +} + +} diff --git a/src/seastar/include/seastar/http/matchrules.hh b/src/seastar/include/seastar/http/matchrules.hh new file mode 100644 index 000000000..032c60134 --- /dev/null +++ b/src/seastar/include/seastar/http/matchrules.hh @@ -0,0 +1,122 @@ +/* + * 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 2015 Cloudius Systems + */ + +#pragma once + +#include <seastar/http/handlers.hh> +#include <seastar/http/matcher.hh> +#include <seastar/http/common.hh> + +#include <seastar/core/sstring.hh> +#include <vector> + +namespace seastar { + +namespace httpd { + +/** + * match_rule check if a url matches criteria, that can contains + * parameters. + * the routes object would call the get method with a url and if + * it matches, the method will return a handler + * during the matching process, the method fill the parameters object. + */ +class match_rule { +public: + /** + * The destructor deletes matchers. + */ + ~match_rule() { + for (auto m : _match_list) { + delete m; + } + delete _handler; + } + + /** + * Constructor with a handler + * @param handler the handler to return when this match rule is met + */ + explicit match_rule(handler_base* handler) + : _handler(handler) { + } + + /** + * Check if url match the rule and return a handler if it does + * @param url a url to compare against the rule + * @param params the parameters object, matches parameters will fill + * the object during the matching process + * @return a handler if there is a full match or nullptr if not + */ + handler_base* get(const sstring& url, parameters& params) { + size_t ind = 0; + if (_match_list.empty()) { + return _handler; + } + for (unsigned int i = 0; i < _match_list.size(); i++) { + ind = _match_list.at(i)->match(url, ind, params); + if (ind == sstring::npos) { + return nullptr; + } + } + return (ind + 1 >= url.length()) ? _handler : nullptr; + } + + /** + * Add a matcher to the rule + * @param match the matcher to add + * @return this + */ + match_rule& add_matcher(matcher* match) { + _match_list.push_back(match); + return *this; + } + + /** + * Add a static url matcher + * @param str the string to search for + * @return this + */ + match_rule& add_str(const sstring& str) { + add_matcher(new str_matcher(str)); + return *this; + } + + /** + * add a parameter matcher to the rule + * @param str the parameter name + * @param fullpath when set to true, parameter will included all the + * remaining url until its end + * @return this + */ + match_rule& add_param(const sstring& str, bool fullpath = false) { + add_matcher(new param_matcher(str, fullpath)); + return *this; + } + +private: + std::vector<matcher*> _match_list; + handler_base* _handler; +}; + +} + +} diff --git a/src/seastar/include/seastar/http/mime_types.hh b/src/seastar/include/seastar/http/mime_types.hh new file mode 100644 index 000000000..cdeeacb13 --- /dev/null +++ b/src/seastar/include/seastar/http/mime_types.hh @@ -0,0 +1,42 @@ +// +// mime_types.hpp +// ~~~~~~~~~~~~~~ +// +// Copyright (c) 2003-2013 Christopher M. Kohlhoff (chris at kohlhoff dot com) +// +// Distributed under the Boost Software License, Version 1.0. (See accompanying +// file LICENSE_1_0.txt or copy at http://www.boost.org/LICENSE_1_0.txt) +// + +#pragma once + +#include <seastar/core/sstring.hh> + +namespace seastar { + +namespace http { + +namespace mime_types { + +/** + * Convert a file extension into a MIME type. + * + * @param extension the file extension + * @return the mime type as a string + */ +const char* extension_to_type(const sstring& extension); + +} // namespace mime_types + +} // namespace httpd + +namespace httpd { +namespace mime_types { +[[deprecated("Use http::mime_types::extension_to_type instead")]] +inline const char* extension_to_type(const sstring& extension) { + return http::mime_types::extension_to_type(extension); +} +} +} + +} diff --git a/src/seastar/include/seastar/http/reply.hh b/src/seastar/include/seastar/http/reply.hh new file mode 100644 index 000000000..1b6f87451 --- /dev/null +++ b/src/seastar/include/seastar/http/reply.hh @@ -0,0 +1,239 @@ +/* + * 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 2015 Cloudius Systems + */ + +// This file was modified from boost http example +// +// reply.hpp +// ~~~~~~~~~ +// +// Copyright (c) 2003-2013 Christopher M. Kohlhoff (chris at kohlhoff dot com) +// +// Distributed under the Boost Software License, Version 1.0. (See accompanying +// file LICENSE_1_0.txt or copy at http://www.boost.org/LICENSE_1_0.txt) +// +#pragma once + +#include <seastar/core/sstring.hh> +#include <unordered_map> +#include <seastar/http/mime_types.hh> +#include <seastar/core/iostream.hh> +#include <seastar/util/noncopyable_function.hh> + +namespace seastar { + +struct http_response; + +namespace httpd { + +class connection; +class routes; + +} + +namespace http { + +/** + * A reply to be sent to a client. + */ +struct reply { + /** + * The status of the reply. + */ + enum class status_type { + continue_ = 100, //!< continue + switching_protocols = 101, //!< switching_protocols + ok = 200, //!< ok + created = 201, //!< created + accepted = 202, //!< accepted + nonauthoritative_information = 203, //!< nonauthoritative_information + no_content = 204, //!< no_content + reset_content = 205, //!< reset_content + partial_content = 206, //! partial_content + multiple_choices = 300, //!< multiple_choices + moved_permanently = 301, //!< moved_permanently + moved_temporarily = 302, //!< moved_temporarily + see_other = 303, //!< see_other + not_modified = 304, //!< not_modified + use_proxy = 305, //!< use_proxy + temporary_redirect = 307, //!< temporary_redirect + bad_request = 400, //!< bad_request + unauthorized = 401, //!< unauthorized + payment_required = 402, //!< payment_required + forbidden = 403, //!< forbidden + not_found = 404, //!< not_found + method_not_allowed = 405, //!< method_not_allowed + not_acceptable = 406, //!< not_acceptable + request_timeout = 408, //!< request_timeout + conflict = 409, //!< conflict + gone = 410, //!< gone + length_required = 411, //!< length_required + payload_too_large = 413, //!< payload_too_large + uri_too_long = 414, //!< uri_too_long + unsupported_media_type = 415, //!< unsupported_media_type + expectation_failed = 417, //!< expectation_failed + unprocessable_entity = 422, //!< unprocessable_entity + upgrade_required = 426, //!< upgrade_required + too_many_requests = 429, //!< too_many_requests + internal_server_error = 500, //!< internal_server_error + not_implemented = 501, //!< not_implemented + bad_gateway = 502, //!< bad_gateway + service_unavailable = 503, //!< service_unavailable + gateway_timeout = 504, //!< gateway_timeout + http_version_not_supported = 505, //!< http_version_not_supported + insufficient_storage = 507 //!< insufficient_storage + } _status; + + /** + * The headers to be included in the reply. + */ + std::unordered_map<sstring, sstring> _headers; + + sstring _version; + /** + * The content to be sent in the reply. + */ + sstring _content; + size_t content_length = 0; // valid when received via client connection + + sstring _response_line; + std::unordered_map<sstring, sstring> trailing_headers; + std::unordered_map<sstring, sstring> chunk_extensions; + + reply() + : _status(status_type::ok) { + } + + explicit reply(http_response&&); + + reply& add_header(const sstring& h, const sstring& value) { + _headers[h] = value; + return *this; + } + + /** + * Search for the first header of a given name + * @param name the header name + * @return a pointer to the header value, if it exists or empty string + */ + sstring get_header(const sstring& name) const { + auto res = _headers.find(name); + if (res == _headers.end()) { + return ""; + } + return res->second; + } + + reply& set_version(const sstring& version) { + _version = version; + return *this; + } + + reply& set_status(status_type status, sstring content = "") { + _status = status; + if (content != "") { + _content = std::move(content); + } + return *this; + } + + /** + * Set the content type mime type. + * Used when the mime type is known. + * For most cases, use the set_content_type + */ + reply& set_mime_type(const sstring& mime) { + _headers["Content-Type"] = mime; + return *this; + } + + /** + * Set the content type mime type according to the file extension + * that would have been used if it was a file: e.g. html, txt, json etc' + */ + reply& set_content_type(const sstring& content_type = "html") { + set_mime_type(http::mime_types::extension_to_type(content_type)); + return *this; + } + + reply& done(const sstring& content_type) { + return set_content_type(content_type).done(); + } + /** + * Done should be called before using the reply. + * It would set the response line + */ + reply& done() { + _response_line = response_line(); + return *this; + } + sstring response_line(); + + /*! + * \brief use an output stream to write the message body + * + * When a handler needs to use an output stream it should call this method + * with a function. + * + * \param content_type - is used to choose the content type of the body. Use the file extension + * you would have used for such a content, (i.e. "txt", "html", "json", etc') + * \param body_writer - a function that accept an output stream and use that stream to write the body. + * The function should take ownership of the stream while using it and must close the stream when it + * is done. + * + * Message would use chunked transfer encoding in the reply. + * + */ + + void write_body(const sstring& content_type, noncopyable_function<future<>(output_stream<char>&&)>&& body_writer); + + /*! + * \brief Write a string as the reply + * + * \param content_type - is used to choose the content type of the body. Use the file extension + * you would have used for such a content, (i.e. "txt", "html", "json", etc') + * \param content - the message content. + * This would set the the content and content type of the message along + * with any additional information that is needed to send the message. + */ + void write_body(const sstring& content_type, sstring content); + +private: + future<> write_reply_to_connection(httpd::connection& con); + future<> write_reply_headers(httpd::connection& connection); + + noncopyable_function<future<>(output_stream<char>&&)> _body_writer; + friend class httpd::routes; + friend class httpd::connection; +}; + +std::ostream& operator<<(std::ostream& os, reply::status_type st); + +} // namespace http + +namespace httpd { +using reply [[deprecated("Use http::reply instead")]] = http::reply; +} + +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<seastar::http::reply::status_type> : fmt::ostream_formatter {}; +#endif diff --git a/src/seastar/include/seastar/http/request.hh b/src/seastar/include/seastar/http/request.hh new file mode 100644 index 000000000..3abec36af --- /dev/null +++ b/src/seastar/include/seastar/http/request.hh @@ -0,0 +1,285 @@ +/* + * 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 2015 Cloudius Systems + */ + +// +// request.hpp +// ~~~~~~~~~~~ +// +// Copyright (c) 2003-2013 Christopher M. Kohlhoff (chris at kohlhoff dot com) +// +// Distributed under the Boost Software License, Version 1.0. (See accompanying +// file LICENSE_1_0.txt or copy at http://www.boost.org/LICENSE_1_0.txt) +// +#pragma once + +#include <seastar/core/iostream.hh> +#include <seastar/core/sstring.hh> +#include <string> +#include <vector> +#include <strings.h> +#include <seastar/http/common.hh> +#include <seastar/http/mime_types.hh> +#include <seastar/core/iostream.hh> + +namespace seastar { + +namespace http { + +namespace experimental { class connection; } + +/** + * A request received from a client. + */ +struct request { + enum class ctclass + : char { + other, multipart, app_x_www_urlencoded, + }; + + struct case_insensitive_cmp { + bool operator()(const sstring& s1, const sstring& s2) const { + return std::equal(s1.begin(), s1.end(), s2.begin(), s2.end(), + [](char a, char b) { return ::tolower(a) == ::tolower(b); }); + } + }; + + struct case_insensitive_hash { + size_t operator()(sstring s) const { + std::transform(s.begin(), s.end(), s.begin(), ::tolower); + return std::hash<sstring>()(s); + } + }; + + sstring _method; + sstring _url; + sstring _version; + ctclass content_type_class; + size_t content_length = 0; + std::unordered_map<sstring, sstring, case_insensitive_hash, case_insensitive_cmp> _headers; + std::unordered_map<sstring, sstring> query_parameters; + httpd::parameters param; + sstring content; // server-side deprecated: use content_stream instead + /* + * The handler should read the contents of this stream till reaching eof (i.e., the end of this request's content). Failing to do so + * will force the server to close this connection, and the client will not be able to reuse this connection for the next request. + * The stream should not be closed by the handler, the server will close it for the handler. + * */ + input_stream<char>* content_stream; + std::unordered_map<sstring, sstring> trailing_headers; + std::unordered_map<sstring, sstring> chunk_extensions; + sstring protocol_name = "http"; + noncopyable_function<future<>(output_stream<char>&&)> body_writer; // for client + + /** + * Search for the first header of a given name + * @param name the header name + * @return a pointer to the header value, if it exists or empty string + */ + sstring get_header(const sstring& name) const { + auto res = _headers.find(name); + if (res == _headers.end()) { + return ""; + } + return res->second; + } + + /** + * Search for the first header of a given name + * @param name the header name + * @return a pointer to the header value, if it exists or empty string + */ + sstring get_query_param(const sstring& name) const { + auto res = query_parameters.find(name); + if (res == query_parameters.end()) { + return ""; + } + return res->second; + } + + /** + * Get the request protocol name. Can be either "http" or "https". + */ + sstring get_protocol_name() const { + return protocol_name; + } + + /** + * Get the request url. + * @return the request url + */ + sstring get_url() const { + return get_protocol_name() + "://" + get_header("Host") + _url; + } + + bool is_multi_part() const { + return content_type_class == ctclass::multipart; + } + + bool is_form_post() const { + return content_type_class == ctclass::app_x_www_urlencoded; + } + + bool should_keep_alive() const { + if (_version == "0.9") { + return false; + } + + // TODO: handle HTTP/2.0 when it releases + + auto it = _headers.find("Connection"); + if (_version == "1.0") { + return it != _headers.end() + && case_insensitive_cmp()(it->second, "keep-alive"); + } else { // HTTP/1.1 + return it == _headers.end() || !case_insensitive_cmp()(it->second, "close"); + } + } + + /** + * Set the query parameters in the request objects. + * Returns the URL path part, i.e. -- without the query paremters + * query param appear after the question mark and are separated + * by the ampersand sign + */ + sstring parse_query_param(); + + /** + * Generates the URL string from the _url and query_parameters + * values in a form parseable by the above method + */ + sstring format_url() const; + + /** + * Set the content type mime type. + * Used when the mime type is known. + * For most cases, use the set_content_type + */ + void set_mime_type(const sstring& mime) { + _headers["Content-Type"] = mime; + } + + /** + * Set the content type mime type according to the file extension + * that would have been used if it was a file: e.g. html, txt, json etc' + */ + void set_content_type(const sstring& content_type = "html") { + set_mime_type(http::mime_types::extension_to_type(content_type)); + } + + /** + * \brief Write a string as the body + * + * \param content_type - is used to choose the content type of the body. Use the file extension + * you would have used for such a content, (i.e. "txt", "html", "json", etc') + * \param content - the message content. + * This would set the the content, conent length and content type of the message along + * with any additional information that is needed to send the message. + * + * This method is good to be used if the body is available as a contiguous buffer. + */ + void write_body(const sstring& content_type, sstring content); + + /** + * \brief Use an output stream to write the message body + * + * When a handler needs to use an output stream it should call this method + * with a function. + * + * \param content_type - is used to choose the content type of the body. Use the file extension + * you would have used for such a content, (i.e. "txt", "html", "json", etc') + * \param body_writer - a function that accept an output stream and use that stream to write the body. + * The function should take ownership of the stream while using it and must close the stream when it + * is done. + * + * This method can be used to write body of unknown or hard to evaluate length. For example, + * when sending the contents of some other input_stream or when the body is available as a + * collection of memory buffers. Message would use chunked transfer encoding. + * + */ + void write_body(const sstring& content_type, noncopyable_function<future<>(output_stream<char>&&)>&& body_writer); + + /** + * \brief Use an output stream to write the message body + * + * When a handler needs to use an output stream it should call this method + * with a function. + * + * \param content_type - is used to choose the content type of the body. Use the file extension + * you would have used for such a content, (i.e. "txt", "html", "json", etc') + * \param len - known in advance content length + * \param body_writer - a function that accept an output stream and use that stream to write the body. + * The function should take ownership of the stream while using it and must close the stream when it + * is done. + * + * This method is to be used when the body is not available of a single contiguous buffer, but the + * size of it is known and it's desirable to provide it to the server, or when the server strongly + * requires the content-length header for any reason. + * + * Message would use plain encoding in the the reply with Content-Length header set accordingly. + * If the body_writer doesn't generate enough bytes into the stream or tries to put more data into + * the stream, sending the request would resolve with exceptional future. + * + */ + void write_body(const sstring& content_type, size_t len, noncopyable_function<future<>(output_stream<char>&&)>&& body_writer); + + /** + * \brief Make request send Expect header + * + * When set, the connection::make_request will send the Expect header and will wait for the + * server resply before tranferring the body + * + */ + void set_expects_continue(); + + /** + * \brief Make simple request + * + * \param method - method to use, e.g. "GET" or "POST" + * \param host - host to contact. This value will be used as the "Host" header + * \path - the URL to send the request to + * + */ + static request make(sstring method, sstring host, sstring path); + + /** + * \brief Make simple request + * + * \param method - method to use, e.g. operation_type::GET + * \param host - host to contact. This value will be used as the "Host" header + * \path - the URL to send the request to + * + */ + static request make(httpd::operation_type type, sstring host, sstring path); + +private: + void add_param(const std::string_view& param); + sstring request_line() const; + future<> write_request_headers(output_stream<char>& out); + friend class experimental::connection; +}; + +} // namespace httpd + +namespace httpd { +using request [[deprecated("Use http::request instead")]] = http::request; +} + +} diff --git a/src/seastar/include/seastar/http/routes.hh b/src/seastar/include/seastar/http/routes.hh new file mode 100644 index 000000000..3b801b86d --- /dev/null +++ b/src/seastar/include/seastar/http/routes.hh @@ -0,0 +1,299 @@ +/* + * 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 2015 Cloudius Systems + */ + +#pragma once + +#include <seastar/http/matchrules.hh> +#include <seastar/http/handlers.hh> +#include <seastar/http/common.hh> +#include <seastar/http/reply.hh> + +#include <boost/program_options/variables_map.hpp> +#include <unordered_map> + +namespace seastar { + +namespace httpd { + +/** + * The url helps defining a route. + */ +class url { +public: + /** + * Move constructor + */ + url(url&&) = default; + + /** + * Construct with a url path as it's parameter + * @param path the url path to be used + */ + url(const sstring& path) + : _path(path) { + } + + /** + * Adds a parameter that matches untill the end of the URL. + * @param param the parmaeter name + * @return the current url + */ + url& remainder(const sstring& param) { + this->_param = param; + return *this; + } + + sstring _path; + sstring _param; +}; + +struct path_description; + +/** + * routes object do the request dispatching according to the url. + * It uses two decision mechanism exact match, if a url matches exactly + * (an optional leading slash is permitted) it is choosen + * If not, the matching rules are used. + * matching rules are evaluated by their insertion order + */ +class routes { +public: + /** + * The destructor deletes the match rules and handlers + */ + ~routes(); + + /** + * adding a handler as an exact match + * @param url the url to match (note that url should start with /) + * @param handler the desire handler + * @return it self + */ + routes& put(operation_type type, const sstring& url, handler_base* handler); + + /** + * removing a handler from exact match + * @param url the url to match (note that url should start with /) + * @return the current handler (to be removed by caller) + */ + handler_base* drop(operation_type type, const sstring& url); + + /** + * add a rule to be used. + * rules are search only if an exact match was not found. + * rules are search by the order they were added. + * First in higher priority + * @param rule a rule to add + * @param type the operation type + * @return it self + */ + routes& add(match_rule* rule, operation_type type = GET) { + _rules[type][_rover++] = rule; + return *this; + } + + /** + * Add a url match to a handler: + * Example routes.add(GET, url("/api").remainder("path"), handler); + * @param type + * @param url + * @param handler + * @return + */ + routes& add(operation_type type, const url& url, handler_base* handler); + + /** + * Add a default handler - handles any HTTP Method and Path (/\*) combination: + * Example routes.add_default_handler(handler); + * @param handler + * @return + */ + routes& add_default_handler(handler_base* handler); + + /** + * the main entry point. + * the general handler calls this method with the request + * the method takes the headers from the request and find the + * right handler. + * It then call the handler with the parameters (if they exists) found in the url + * @param path the url path found + * @param req the http request + * @param rep the http reply + */ + future<std::unique_ptr<http::reply> > handle(const sstring& path, std::unique_ptr<http::request> req, std::unique_ptr<http::reply> rep); + + /** + * Search and return an exact match + * @param url the request url + * @return the handler if exists or nullptr if it does not + */ + handler_base* get_exact_match(operation_type type, const sstring& url) const { + auto i = _map[type].find(url); + return (i == _map[type].end()) ? nullptr : i->second; + } + + /** + * Search and return a handler by the operation type and url + * @param type the http operation type + * @param url the request url + * @param params a parameter object that will be filled during the match + * @return a handler based on the type/url match + */ + handler_base* get_handler(operation_type type, const sstring& url, + parameters& params); + +private: + /** + * Normalize the url to remove the last / if exists + * and get the parameter part + * @param url the full url path + * @param param_part will hold the string with the parameters + * @return the url from the request without the last / + */ + sstring normalize_url(const sstring& url); + + std::unordered_map<sstring, handler_base*> _map[NUM_OPERATION]; +public: + using rule_cookie = uint64_t; +private: + rule_cookie _rover = 0; + std::map<rule_cookie, match_rule*> _rules[NUM_OPERATION]; + //default Handler -- for any HTTP Method and Path (/*) + handler_base* _default_handler = nullptr; +public: + using exception_handler_fun = std::function<std::unique_ptr<http::reply>(std::exception_ptr eptr)>; + using exception_handler_id = size_t; +private: + std::map<exception_handler_id, exception_handler_fun> _exceptions; + exception_handler_id _exception_id = 0; + // for optimization reason, the lambda function + // that calls the exception_reply of the current object + // is stored + exception_handler_fun _general_handler; +public: + /** + * The exception_handler_fun expect to call + * std::rethrow_exception(eptr); + * and catch only the exception it handles + */ + exception_handler_id register_exeption_handler(exception_handler_fun fun) { + auto current = _exception_id++; + _exceptions[current] = fun; + return current; + } + + void remove_exception_handler(exception_handler_id id) { + _exceptions.erase(id); + } + + std::unique_ptr<http::reply> exception_reply(std::exception_ptr eptr); + + routes(); + + /*! + * \brief add an alias to an already registered path. + * After registering a handler to a path, use this method + * to add an alias to that handler. + * + */ + void add_alias(const path_description& old_path, const path_description& new_path); + + /** + * Add a rule to be used. + * @param rule a rule to add + * @param type the operation type + * @return a cookie using which the rule can be removed + */ + rule_cookie add_cookie(match_rule* rule, operation_type type) { + auto pos = _rover++; + _rules[type][pos] = rule; + return pos; + } + + /** + * Del a rule by cookie + * @param cookie a cookie returned previously by add_cookie + * @param type the operation type + * @return the pointer to the rule + */ + match_rule* del_cookie(rule_cookie cookie, operation_type type); +}; + +/** + * A helper function that check if a parameter is found in the params object + * if it does not the function would throw a parameter not found exception + * @param params the parameters object + * @param param the parameter to look for + */ +void verify_param(const http::request& req, const sstring& param); + +/** + * The handler_registration object facilitates registration and auto + * unregistration of an exact-match handler_base into \ref routes "routes" + */ +class handler_registration { + routes& _routes; + const sstring _url; + operation_type _op; + +public: + /** + * Registers the handler_base into routes with routes::put + * @param rs the routes object reference + * @param h the desire handler + * @param url the url to match + * @param op the operation type (`GET` by default) + */ + handler_registration(routes& rs, handler_base& h, const sstring& url, operation_type op = GET); + + /** + * Unregisters the handler from routes with routes::drop + */ + ~handler_registration(); +}; + +/** + * The rule_registration object facilitates registration and auto + * unregistration of a match_rule handler into \ref routes "routes" + */ +class rule_registration { + routes& _routes; + operation_type _op; + routes::rule_cookie _cookie; + +public: + /** + * Registers the match_rule into routes with routes::add_cookie + * @param rs the routes object reference + * @param rule a rule to add + * @param op the operation type (`GET` by default) + */ + rule_registration(routes& rs, match_rule& rule, operation_type op = GET); + + /** + * Unregisters the rule from routes with routes::del_cookie + */ + ~rule_registration(); +}; + +} + +} diff --git a/src/seastar/include/seastar/http/short_streams.hh b/src/seastar/include/seastar/http/short_streams.hh new file mode 100644 index 000000000..77c65bada --- /dev/null +++ b/src/seastar/include/seastar/http/short_streams.hh @@ -0,0 +1,54 @@ +/* + * 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) 2021 ScyllaDB + */ + +#include <seastar/core/future.hh> +#include <seastar/core/iostream.hh> +#include <seastar/core/temporary_buffer.hh> +#include <seastar/util/short_streams.hh> + +namespace seastar { + +namespace httpd { + +/// Returns all bytes from the stream until eof, accessible in chunks +[[deprecated("seastar::httpd::read_entire_stream was moved to seastar::util, #include <seastar/util/short_streams.hh> instead")]] +inline +future<std::vector<temporary_buffer<char>>> read_entire_stream(input_stream<char>& inp) { + return util::read_entire_stream(inp); +} + +/// Returns all bytes from the stream until eof as a single buffer, use only on short streams +[[deprecated("seastar::httpd::read_entire_stream_contiguous was moved to seastar::util, #include <seastar/util/short_streams.hh> instead")]] +inline +future<sstring> read_entire_stream_contiguous(input_stream<char>& inp) { + return util::read_entire_stream_contiguous(inp); +} + +/// Ignores all bytes until eof +[[deprecated("seastar::httpd::skip_entire_stream was moved to seastar::util, #include <seastar/util/short_streams.hh> instead")]] +inline +future<> skip_entire_stream(input_stream<char>& inp) { + return util::skip_entire_stream(inp); +} + +} + +} diff --git a/src/seastar/include/seastar/http/transformers.hh b/src/seastar/include/seastar/http/transformers.hh new file mode 100644 index 000000000..8e854b368 --- /dev/null +++ b/src/seastar/include/seastar/http/transformers.hh @@ -0,0 +1,59 @@ +/* + * 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 2015 Cloudius Systems + */ + +#pragma once + +#include <seastar/http/handlers.hh> +#include <seastar/http/file_handler.hh> + +namespace seastar { + +namespace httpd { + +/** + * content_replace replaces variable in a file with a dynamic value. + * It would take the host from request and will replace the variable + * in a file + * + * The replacement can be restricted to an extension. + * + * We are currently support only one file type for replacement. + * It could be extend if we will need it + * + */ +class content_replace : public file_transformer { +public: + virtual output_stream<char> transform(std::unique_ptr<http::request> req, + const sstring& extension, output_stream<char>&& s); + /** + * the constructor get the file extension the replace would work on. + * @param extension file extension, when not set all files extension + */ + explicit content_replace(const sstring& extension = "") + : extension(extension) { + } +private: + sstring extension; +}; + +} + +} diff --git a/src/seastar/include/seastar/http/url.hh b/src/seastar/include/seastar/http/url.hh new file mode 100644 index 000000000..24e4fbb82 --- /dev/null +++ b/src/seastar/include/seastar/http/url.hh @@ -0,0 +1,44 @@ +/* + * 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) 2022 Scylladb, Ltd. + */ + +#include <seastar/core/sstring.hh> + +namespace seastar { + +namespace http { +namespace internal { + +bool url_decode(const std::string_view& in, sstring& out); + +/** + * Makes a percent-encoded string out of the given parameter + * + * Note, that it does NOT parse and encode a URL correctly handling + * all the delimeters in arguments. It's up to the caller to split + * the URL into path and arguments (both names and values) and use + * this helper to encode individual strings + */ +sstring url_encode(const std::string_view& in); + +} // internal namespace +} // http namespace + +} // seastar namespace diff --git a/src/seastar/include/seastar/json/formatter.hh b/src/seastar/include/seastar/json/formatter.hh new file mode 100644 index 000000000..6ff2b231a --- /dev/null +++ b/src/seastar/include/seastar/json/formatter.hh @@ -0,0 +1,343 @@ +/* + * 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 2015 Cloudius Systems + */ + +#pragma once + +#include <string> +#include <vector> +#include <unordered_map> +#include <map> +#include <time.h> +#include <sstream> +#include <seastar/core/loop.hh> +#include <seastar/core/sstring.hh> +#include <seastar/core/iostream.hh> + +namespace seastar { + +namespace json { + +class jsonable; + +typedef struct tm date_time; + +/** + * The formatter prints json values in a json format + * it overload to_json method for each of the supported format + * all to_json parameters are passed as a pointer + */ +class formatter { + enum class state { + none, array, map + }; + static sstring begin(state); + static sstring end(state); + + template<typename K, typename V> + static sstring to_json(state s, const std::pair<K, V>& p) { + return s == state::array ? + "{" + to_json(state::none, p) + "}" : + to_json(p.first) + ":" + to_json(p.second); + } + + template<typename Iter> + static sstring to_json(state s, Iter i, Iter e) { + std::stringstream res; + res << begin(s); + size_t n = 0; + while (i != e) { + if (n++ != 0) { + res << ","; + } + res << to_json(s, *i++); + } + res << end(s); + return res.str(); + } + + // fallback template + template<typename T> + static sstring to_json(state, const T& t) { + return to_json(t); + } + + template<typename K, typename V> + static future<> write(output_stream<char>& stream, state s, const std::pair<K, V>& p) { + if (s == state::array) { + return stream.write("{").then([&stream, &p] { + return write(stream, state::none, p).then([&stream] { + return stream.write("}"); + }); + }); + } else { + return stream.write(to_json(p.first) + ":").then([&p, &stream] { + return write(stream, p.second); + }); + } + } + + template<typename Iter> + static future<> write(output_stream<char>& stream, state s, Iter i, Iter e) { + return do_with(true, [&stream, s, i, e] (bool& first) { + return stream.write(begin(s)).then([&first, &stream, s, i, e] { + return do_for_each(i, e, [&first, &stream] (auto& m) { + auto f = (first) ? make_ready_future<>() : stream.write(","); + first = false; + return f.then([&m, &stream] { + return write(stream, m); + }); + }).then([&stream, s] { + return stream.write(end(s)); + }); + }); + }); + } + + // fallback template + template<typename T> + static future<> write(output_stream<char>& stream, state, const T& t) { + return stream.write(to_json(t)); + } + +public: + + /** + * return a json formatted string + * @param str the string to format + * @return the given string in a json format + */ + static sstring to_json(const sstring& str); + + /** + * return a json formatted int + * @param n the int to format + * @return the given int in a json format + */ + static sstring to_json(int n); + + /** + * return a json formatted unsigned + * @param n the unsigned to format + * @return the given unsigned in a json format + */ + static sstring to_json(unsigned n); + + /** + * return a json formatted long + * @param n the long to format + * @return the given long in a json format + */ + static sstring to_json(long n); + + /** + * return a json formatted float + * @param f the float to format + * @return the given float in a json format + */ + static sstring to_json(float f); + + /** + * return a json formatted double + * @param d the double to format + * @return the given double in a json format + */ + static sstring to_json(double d); + + /** + * return a json formatted char* (treated as string), possibly with zero-chars in the middle + * @param str the char* to format + * @param len number of bytes to read from the \p str + * @return the given char* in a json format + */ + static sstring to_json(const char* str, size_t len); + + /** + * return a json formatted char* (treated as string), assuming there are no zero-chars in the middle + * @param str the char* to format + * @return the given char* in a json format + * @deprecated A more robust overload should be preferred: \ref to_json(const char*, size_t) + */ + static sstring to_json(const char* str); + + /** + * return a json formatted bool + * @param d the bool to format + * @return the given bool in a json format + */ + static sstring to_json(bool d); + + /** + * return a json formatted list of a given vector of params + * @param vec the vector to format + * @return the given vector in a json format + */ + template<typename... Args> + static sstring to_json(const std::vector<Args...>& vec) { + return to_json(state::array, vec.begin(), vec.end()); + } + + template<typename... Args> + static sstring to_json(const std::map<Args...>& map) { + return to_json(state::map, map.begin(), map.end()); + } + + template<typename... Args> + static sstring to_json(const std::unordered_map<Args...>& map) { + return to_json(state::map, map.begin(), map.end()); + } + + /** + * return a json formatted date_time + * @param d the date_time to format + * @return the given date_time in a json format + */ + static sstring to_json(const date_time& d); + + /** + * return a json formatted json object + * @param obj the date_time to format + * @return the given json object in a json format + */ + static sstring to_json(const jsonable& obj); + + /** + * return a json formatted unsigned long + * @param l unsigned long to format + * @return the given unsigned long in a json format + */ + static sstring to_json(unsigned long l); + + + + /** + * return a json formatted string + * @param str the string to format + * @return the given string in a json format + */ + static future<> write(output_stream<char>& s, const sstring& str) { + return s.write(to_json(str)); + } + + /** + * return a json formatted int + * @param n the int to format + * @return the given int in a json format + */ + static future<> write(output_stream<char>& s, int n) { + return s.write(to_json(n)); + } + + /** + * return a json formatted long + * @param n the long to format + * @return the given long in a json format + */ + static future<> write(output_stream<char>& s, long n) { + return s.write(to_json(n)); + } + + /** + * return a json formatted float + * @param f the float to format + * @return the given float in a json format + */ + static future<> write(output_stream<char>& s, float f) { + return s.write(to_json(f)); + } + + /** + * return a json formatted double + * @param d the double to format + * @return the given double in a json format + */ + static future<> write(output_stream<char>& s, double d) { + return s.write(to_json(d)); + } + + /** + * return a json formatted char* (treated as string) + * @param str the char* to format + * @return the given char* in a json format + */ + static future<> write(output_stream<char>& s, const char* str) { + return s.write(to_json(str)); + } + + /** + * return a json formatted bool + * @param d the bool to format + * @return the given bool in a json format + */ + static future<> write(output_stream<char>& s, bool d) { + return s.write(to_json(d)); + } + + /** + * return a json formatted list of a given vector of params + * @param vec the vector to format + * @return the given vector in a json format + */ + template<typename... Args> + static future<> write(output_stream<char>& s, const std::vector<Args...>& vec) { + return write(s, state::array, vec.begin(), vec.end()); + } + + template<typename... Args> + static future<> write(output_stream<char>& s, const std::map<Args...>& map) { + return write(s, state::map, map.begin(), map.end()); + } + + template<typename... Args> + static future<> write(output_stream<char>& s, const std::unordered_map<Args...>& map) { + return write(s, state::map, map.begin(), map.end()); + } + + /** + * return a json formatted date_time + * @param d the date_time to format + * @return the given date_time in a json format + */ + static future<> write(output_stream<char>& s, const date_time& d) { + return s.write(to_json(d)); + } + + /** + * return a json formatted json object + * @param obj the date_time to format + * @return the given json object in a json format + */ + static future<> write(output_stream<char>& s, const jsonable& obj) { + return s.write(to_json(obj)); + } + + /** + * return a json formatted unsigned long + * @param l unsigned long to format + * @return the given unsigned long in a json format + */ + static future<> write(output_stream<char>& s, unsigned long l) { + return s.write(to_json(l)); + } +}; + +} + +} diff --git a/src/seastar/include/seastar/json/json_elements.hh b/src/seastar/include/seastar/json/json_elements.hh new file mode 100644 index 000000000..00cb0b5e6 --- /dev/null +++ b/src/seastar/include/seastar/json/json_elements.hh @@ -0,0 +1,360 @@ +/* + * 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 2015 Cloudius Systems + */ + +#pragma once + +#include <string> +#include <vector> +#include <time.h> +#include <sstream> +#include <seastar/core/do_with.hh> +#include <seastar/core/loop.hh> +#include <seastar/json/formatter.hh> +#include <seastar/core/sstring.hh> +#include <seastar/core/iostream.hh> + +namespace seastar { + +namespace json { + +/** + * The base class for all json element. + * Every json element has a name + * An indication if it was set or not + * And is this element is mandatory. + * When a mandatory element is not set + * this is not a valid object + */ +class json_base_element { +public: + /** + * The constructors + */ + json_base_element() noexcept + : _mandatory(false), _set(false) { + } + + virtual ~json_base_element() = default; + + /** + * Check if it's a mandatory parameter + * and if it's set. + * @return true if this is not a mandatory parameter + * or if it is and it's value is set + */ + virtual bool is_verify() noexcept { + return !(_mandatory && !_set); + } + + json_base_element& operator=(const json_base_element& o) noexcept { + // Names and mandatory are never changed after creation + _set = o._set; + return *this; + } + + /** + * returns the internal value in a json format + * Each inherit class must implement this method + * @return formated internal value + */ + virtual std::string to_string() = 0; + + virtual future<> write(output_stream<char>& s) const = 0; + std::string _name; + bool _mandatory; + bool _set; +}; + +/** + * Basic json element instantiate + * the json_element template. + * it adds a value to the base definition + * and the to_string implementation using the formatter + */ +template<class T> +class json_element : public json_base_element { +public: + + /** + * the assignment operator also set + * the set value to true. + * @param new_value the new value + * @return the value itself + */ + json_element &operator=(const T& new_value) { + _value = new_value; + _set = true; + return *this; + } + /** + * the assignment operator also set + * the set value to true. + * @param new_value the new value + * @return the value itself + */ + template<class C> + json_element &operator=(const C& new_value) { + _value = new_value; + _set = true; + return *this; + } + /** + * The brackets operator + * @return the value + */ + const T& operator()() const noexcept { + return _value; + } + + /** + * The to_string return the value + * formated as a json value + * @return the value foramted for json + */ + virtual std::string to_string() override + { + return formatter::to_json(_value); + } + + virtual future<> write(output_stream<char>& s) const override { + return formatter::write(s, _value); + } +private: + T _value; +}; + +/** + * json_list is based on std vector implementation. + * + * When values are added with push it is set the "set" flag to true + * hence will be included in the parsed object + */ +template<class T> +class json_list : public json_base_element { +public: + + /** + * Add an element to the list. + * @param element a new element that will be added to the list + */ + void push(const T& element) { + _set = true; + _elements.push_back(element); + } + + virtual std::string to_string() override + { + return formatter::to_json(_elements); + } + + /** + * Assignment can be done from any object that support const range + * iteration and that it's elements can be assigned to the list elements + */ + template<class C> + json_list& operator=(const C& list) { + _elements.clear(); + for (auto i : list) { + push(i); + } + return *this; + } + virtual future<> write(output_stream<char>& s) const override { + return formatter::write(s, _elements); + } + std::vector<T> _elements; +}; + +class jsonable { +public: + virtual ~jsonable() = default; + /** + * create a foramted string of the object. + * @return the object formated. + */ + virtual std::string to_json() const = 0; + + /*! + * \brief write an object to the output stream + * + * The defult implementation uses the to_json + * Object implementation override it. + */ + virtual future<> write(output_stream<char>& s) const { + return s.write(to_json()); + } +}; + +/** + * The base class for all json objects + * It holds a list of all the element in it, + * allowing it implement the to_json method. + * + * It also allows iterating over the element + * in the object, even if not all the member + * are known in advance and in practice mimic + * reflection + */ +struct json_base : public jsonable { + + virtual ~json_base() = default; + + json_base() = default; + + json_base(const json_base&) = delete; + + json_base operator=(const json_base&) = delete; + + /** + * create a foramted string of the object. + * @return the object formated. + */ + virtual std::string to_json() const; + + /*! + * \brief write to an output stream + */ + virtual future<> write(output_stream<char>&) const; + + /** + * Check that all mandatory elements are set + * @return true if all mandatory parameters are set + */ + virtual bool is_verify() const; + + /** + * Register an element in an object + * @param element the element to be added + * @param name the element name + * @param mandatory is this element mandatory. + */ + virtual void add(json_base_element* element, std::string name, + bool mandatory = false); + + std::vector<json_base_element*> _elements; +}; + +/** + * There are cases where a json request needs to return a successful + * empty reply. + * The json_void class will be used to mark that the reply should be empty. + * + */ +struct json_void : public jsonable{ + virtual std::string to_json() const { + return ""; + } + + /*! + * \brief write to an output stream + */ + virtual future<> write(output_stream<char>& s) const { + return s.close(); + } +}; + + +/** + * The json return type, is a helper class to return a json + * formatted string. + * It uses autoboxing in its constructor so when a function return + * type is json_return_type, it could return a type that would be converted + * ie. + * json_return_type foo() { + * return "hello"; + * } + * + * would return a json formatted string: "hello" (rather then hello) + */ +struct json_return_type { + sstring _res; + std::function<future<>(output_stream<char>&&)> _body_writer; + json_return_type(std::function<future<>(output_stream<char>&&)>&& body_writer) : _body_writer(std::move(body_writer)) { + } + template<class T> + json_return_type(const T& res) { + _res = formatter::to_json(res); + } + + json_return_type(json_return_type&& o) noexcept : _res(std::move(o._res)), _body_writer(std::move(o._body_writer)) { + } + json_return_type& operator=(json_return_type&& o) noexcept { + if (this != &o) { + _res = std::move(o._res); + _body_writer = std::move(o._body_writer); + } + return *this; + } + + json_return_type(const json_return_type&) = default; + json_return_type& operator=(const json_return_type&) = default; +}; + +/*! + * \brief capture a range and return a serialize function for it as a json array. + * + * To use it, pass a range and a mapping function. + * For example, if res is a map: + * + * return make_ready_future<json::json_return_type>(stream_range_as_array(res, [](const auto&i) {return i.first})); + */ +template<typename Container, typename Func> +SEASTAR_CONCEPT( requires requires (Container c, Func aa, output_stream<char> s) { { formatter::write(s, aa(*c.begin())) } -> std::same_as<future<>>; } ) +std::function<future<>(output_stream<char>&&)> stream_range_as_array(Container val, Func fun) { + return [val = std::move(val), fun = std::move(fun)](output_stream<char>&& s) mutable { + return do_with(output_stream<char>(std::move(s)), Container(std::move(val)), Func(std::move(fun)), true, [](output_stream<char>& s, const Container& val, const Func& f, bool& first){ + return s.write("[").then([&val, &s, &first, &f] () { + return do_for_each(val, [&s, &first, &f](const typename Container::value_type& v){ + auto fut = first ? make_ready_future<>() : s.write(", "); + first = false; + return fut.then([&s, &f, &v]() { + return formatter::write(s, f(v)); + }); + }); + }).then([&s](){ + return s.write("]"); + }).finally([&s] { + return s.close(); + }); + }); + }; +} + +/*! + * \brief capture an object and return a serialize function for it. + * + * To use it: + * return make_ready_future<json::json_return_type>(stream_object(res)); + */ +template<class T> +std::function<future<>(output_stream<char>&&)> stream_object(T val) { + return [val = std::move(val)](output_stream<char>&& s) mutable { + return do_with(output_stream<char>(std::move(s)), T(std::move(val)), [](output_stream<char>& s, const T& val){ + return formatter::write(s, val).finally([&s] { + return s.close(); + }); + }); + }; +} + +} + +} diff --git a/src/seastar/include/seastar/net/api.hh b/src/seastar/include/seastar/net/api.hh new file mode 100644 index 000000000..cf1e339d5 --- /dev/null +++ b/src/seastar/include/seastar/net/api.hh @@ -0,0 +1,450 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <memory> +#include <vector> +#include <cstring> +#include <seastar/core/future.hh> +#include <seastar/net/byteorder.hh> +#include <seastar/net/socket_defs.hh> +#include <seastar/net/packet.hh> +#include <seastar/core/internal/api-level.hh> +#include <seastar/core/temporary_buffer.hh> +#include <seastar/core/iostream.hh> +#include <seastar/util/std-compat.hh> +#include <seastar/util/program-options.hh> +#include <sys/types.h> + +namespace seastar { + +inline +bool is_ip_unspecified(const ipv4_addr& addr) noexcept { + return addr.is_ip_unspecified(); +} + +inline +bool is_port_unspecified(const ipv4_addr& addr) noexcept { + return addr.is_port_unspecified(); +} + +inline +socket_address make_ipv4_address(const ipv4_addr& addr) noexcept { + return socket_address(addr); +} + +inline +socket_address make_ipv4_address(uint32_t ip, uint16_t port) noexcept { + return make_ipv4_address(ipv4_addr(ip, port)); +} + +namespace net { + +// see linux tcp(7) for parameter explanation +struct tcp_keepalive_params { + std::chrono::seconds idle; // TCP_KEEPIDLE + std::chrono::seconds interval; // TCP_KEEPINTVL + unsigned count; // TCP_KEEPCNT +}; + +// see linux sctp(7) for parameter explanation +struct sctp_keepalive_params { + std::chrono::seconds interval; // spp_hbinterval + unsigned count; // spp_pathmaxrt +}; + +using keepalive_params = std::variant<tcp_keepalive_params, sctp_keepalive_params>; + +/// \cond internal +class connected_socket_impl; +class socket_impl; + +class server_socket_impl; +class udp_channel_impl; +class get_impl; +/// \endcond + +class udp_datagram_impl { +public: + virtual ~udp_datagram_impl() {}; + virtual socket_address get_src() = 0; + virtual socket_address get_dst() = 0; + virtual uint16_t get_dst_port() = 0; + virtual packet& get_data() = 0; +}; + +class udp_datagram final { +private: + std::unique_ptr<udp_datagram_impl> _impl; +public: + udp_datagram(std::unique_ptr<udp_datagram_impl>&& impl) noexcept : _impl(std::move(impl)) {}; + socket_address get_src() { return _impl->get_src(); } + socket_address get_dst() { return _impl->get_dst(); } + uint16_t get_dst_port() { return _impl->get_dst_port(); } + packet& get_data() { return _impl->get_data(); } +}; + +class udp_channel { +private: + std::unique_ptr<udp_channel_impl> _impl; +public: + udp_channel() noexcept; + udp_channel(std::unique_ptr<udp_channel_impl>) noexcept; + ~udp_channel(); + + udp_channel(udp_channel&&) noexcept; + udp_channel& operator=(udp_channel&&) noexcept; + + socket_address local_address() const; + + future<udp_datagram> receive(); + future<> send(const socket_address& dst, const char* msg); + future<> send(const socket_address& dst, packet p); + bool is_closed() const; + /// Causes a pending receive() to complete (possibly with an exception) + void shutdown_input(); + /// Causes a pending send() to complete (possibly with an exception) + void shutdown_output(); + /// Close the channel and releases all resources. + /// + /// Must be called only when there are no unfinished send() or receive() calls. You + /// can force pending calls to complete soon by calling shutdown_input() and + /// shutdown_output(). + void close(); +}; + +class network_interface_impl; + +} /* namespace net */ + +/// \addtogroup networking-module +/// @{ + +/// Configuration for buffered connected_socket input operations +/// +/// This structure allows tuning of buffered input operations done via +/// connected_socket. It is a hint to the implementation and may be +/// ignored (e.g. the zero-copy native stack does not allocate buffers, +/// so it ignores buffer-size parameters). +struct connected_socket_input_stream_config final { + /// Initial buffer size to use for input buffering + unsigned buffer_size = 8192; + /// Minimum buffer size to use for input buffering. The system will decrease + /// buffer sizes if it sees a tendency towards small requests, but will not go + /// below this buffer size. + unsigned min_buffer_size = 512; + /// Maximum buffer size to use for input buffering. The system will increase + /// buffer sizes if it sees a tendency towards large requests, but will not go + /// above this buffer size. + unsigned max_buffer_size = 128 * 1024; +}; + +/// Distinguished name +struct session_dn { + sstring subject; + sstring issuer; +}; + +/// A TCP (or other stream-based protocol) connection. +/// +/// A \c connected_socket represents a full-duplex stream between +/// two endpoints, a local endpoint and a remote endpoint. +class connected_socket { + friend class net::get_impl; + std::unique_ptr<net::connected_socket_impl> _csi; +public: + /// Constructs a \c connected_socket not corresponding to a connection + connected_socket() noexcept; + ~connected_socket(); + + /// \cond internal + explicit connected_socket(std::unique_ptr<net::connected_socket_impl> csi) noexcept; + /// \endcond + /// Moves a \c connected_socket object. + connected_socket(connected_socket&& cs) noexcept; + /// Move-assigns a \c connected_socket object. + connected_socket& operator=(connected_socket&& cs) noexcept; + /// Gets the input stream. + /// + /// \param csisc Configuration for the input_stream returned + /// + /// Gets an object returning data sent from the remote endpoint. + input_stream<char> input(connected_socket_input_stream_config csisc = {}); + /// Gets the output stream. + /// + /// Gets an object that sends data to the remote endpoint. + /// \param buffer_size how much data to buffer + output_stream<char> output(size_t buffer_size = 8192); + /// Sets the TCP_NODELAY option (disabling Nagle's algorithm) + void set_nodelay(bool nodelay); + /// Gets the TCP_NODELAY option (Nagle's algorithm) + /// + /// \return whether the nodelay option is enabled or not + bool get_nodelay() const; + /// Sets SO_KEEPALIVE option (enable keepalive timer on a socket) + void set_keepalive(bool keepalive); + /// Gets O_KEEPALIVE option + /// \return whether the keepalive option is enabled or not + bool get_keepalive() const; + /// Sets TCP keepalive parameters + void set_keepalive_parameters(const net::keepalive_params& p); + /// Get TCP keepalive parameters + net::keepalive_params get_keepalive_parameters() const; + /// Sets custom socket options. Based on setsockopt function. + /// Linux users should refer to protocol-specific manuals + /// to see available options, e.g. tcp(7), ip(7), etc. + void set_sockopt(int level, int optname, const void* data, size_t len); + /// Gets custom socket options. Based on getsockopt function. + /// Linux users should refer to protocol-specific manuals + /// to see available options, e.g. tcp(7), ip(7), etc. + int get_sockopt(int level, int optname, void* data, size_t len) const; + /// Local address of the socket + socket_address local_address() const noexcept; + + /// Disables output to the socket. + /// + /// Current or future writes that have not been successfully flushed + /// will immediately fail with an error. This is useful to abort + /// operations on a socket that is not making progress due to a + /// peer failure. + void shutdown_output(); + /// Disables input from the socket. + /// + /// Current or future reads will immediately fail with an error. + /// This is useful to abort operations on a socket that is not making + /// progress due to a peer failure. + void shutdown_input(); + /// Check whether the \c connected_socket is initialized. + /// + /// \return true if this \c connected_socket socket_address is bound initialized + /// false otherwise. + /// + /// \see connect(socket_address sa) + /// \see connect(socket_address sa, socket_address local, transport proto) + explicit operator bool() const noexcept { + return static_cast<bool>(_csi); + } + /// Waits for the peer of this socket to disconnect + /// + /// \return future that resolves when the peer closes connection or shuts it down + /// for writing or when local socket is called \ref shutdown_input(). + /// + /// Note, that when the returned future is resolved for whatever reason socket + /// may still be readable from, so the caller may want to wait for both events + /// -- this one and EOF from read. + /// + /// Calling it several times per socket is not allowed (undefined behavior) + /// + /// \see poll(2) about POLLRDHUP for more details + future<> wait_input_shutdown(); +}; +/// @} + +/// \addtogroup networking-module +/// @{ + +/// The seastar socket. +/// +/// A \c socket that allows a connection to be established between +/// two endpoints. +class socket { + std::unique_ptr<net::socket_impl> _si; +public: + socket() noexcept = default; + ~socket(); + + /// \cond internal + explicit socket(std::unique_ptr<net::socket_impl> si) noexcept; + /// \endcond + /// Moves a \c seastar::socket object. + socket(socket&&) noexcept; + /// Move-assigns a \c seastar::socket object. + socket& operator=(socket&&) noexcept; + + /// Attempts to establish the connection. + /// + /// \return a \ref connected_socket representing the connection. + future<connected_socket> connect(socket_address sa, socket_address local = {}, transport proto = transport::TCP); + + /// Sets SO_REUSEADDR option (enable reuseaddr option on a socket) + void set_reuseaddr(bool reuseaddr); + /// Gets O_REUSEADDR option + /// \return whether the reuseaddr option is enabled or not + bool get_reuseaddr() const; + /// Stops any in-flight connection attempt. + /// + /// Cancels the connection attempt if it's still in progress, and + /// terminates the connection if it has already been established. + void shutdown(); +}; + +/// @} + +/// \addtogroup networking-module +/// @{ + +/// The result of an server_socket::accept() call +struct accept_result { + connected_socket connection; ///< The newly-accepted connection + socket_address remote_address; ///< The address of the peer that connected to us +}; + +/// A listening socket, waiting to accept incoming network connections. +class server_socket { + std::unique_ptr<net::server_socket_impl> _ssi; + bool _aborted = false; +public: + enum class load_balancing_algorithm { + // This algorithm tries to distribute all connections equally between all shards. + // It does this by sending new connections to a shard with smallest amount of connections. + connection_distribution, + // This algorithm distributes new connection based on peer's tcp port. Destination shard + // is calculated as a port number modulo number of shards. This allows a client to connect + // to a specific shard in a server given it knows how many shards server has by choosing + // src port number accordingly. + port, + // This algorithm distributes all new connections to listen_options::fixed_cpu shard only. + fixed, + default_ = connection_distribution + }; + /// Constructs a \c server_socket without being bound to any address + server_socket() noexcept; + /// \cond internal + explicit server_socket(std::unique_ptr<net::server_socket_impl> ssi) noexcept; + /// \endcond + /// Moves a \c server_socket object. + server_socket(server_socket&& ss) noexcept; + ~server_socket(); + /// Move-assigns a \c server_socket object. + server_socket& operator=(server_socket&& cs) noexcept; + + /// Accepts the next connection to successfully connect to this socket. + /// + /// \return an accept_result representing the connection and + /// the socket_address of the remote endpoint. + /// + /// \see listen(socket_address sa) + /// \see listen(socket_address sa, listen_options opts) + future<accept_result> accept(); + + /// Stops any \ref accept() in progress. + /// + /// Current and future \ref accept() calls will terminate immediately + /// with an error. + void abort_accept(); + + /// Local bound address + /// + /// \return the local bound address if the \c server_socket is listening, + /// an empty address constructed with \c socket_address() otherwise. + /// + /// \see listen(socket_address sa) + /// \see listen(socket_address sa, listen_options opts) + socket_address local_address() const noexcept; + + /// Check whether the \c server_socket is listening on any address. + /// + /// \return true if this \c socket_address is bound to an address, + /// false if it is just created with the default constructor. + /// + /// \see listen(socket_address sa) + /// \see listen(socket_address sa, listen_options opts) + explicit operator bool() const noexcept { + return static_cast<bool>(_ssi); + } +}; + +/// @} + +struct listen_options { + bool reuse_address = false; + server_socket::load_balancing_algorithm lba = server_socket::load_balancing_algorithm::default_; + transport proto = transport::TCP; + int listen_backlog = 100; + unsigned fixed_cpu = 0u; + void set_fixed_cpu(unsigned cpu) { + lba = server_socket::load_balancing_algorithm::fixed; + fixed_cpu = cpu; + } +}; + +class network_interface { +private: + shared_ptr<net::network_interface_impl> _impl; +public: + network_interface() = delete; + network_interface(shared_ptr<net::network_interface_impl>) noexcept; + network_interface(network_interface&&) noexcept; + + network_interface& operator=(network_interface&&) noexcept; + + uint32_t index() const; + uint32_t mtu() const; + + const sstring& name() const; + const sstring& display_name() const; + const std::vector<net::inet_address>& addresses() const; + const std::vector<uint8_t> hardware_address() const; + + bool is_loopback() const; + bool is_virtual() const; + bool is_up() const; + bool supports_ipv6() const; +}; + +class network_stack { +public: + virtual ~network_stack() {} + virtual server_socket listen(socket_address sa, listen_options opts) = 0; + // FIXME: local parameter assumes ipv4 for now, fix when adding other AF + future<connected_socket> connect(socket_address sa, socket_address = {}, transport proto = transport::TCP); + virtual ::seastar::socket socket() = 0; + virtual net::udp_channel make_udp_channel(const socket_address& = {}) = 0; + virtual future<> initialize() { + return make_ready_future(); + } + virtual bool has_per_core_namespace() = 0; + // NOTE: this is not a correct query approach. + // This question should be per NIC, but we have no such + // abstraction, so for now this is "stack-wide" + virtual bool supports_ipv6() const { + return false; + } + + /** + * Returns available network interfaces. This represents a + * snapshot of interfaces available at call time, hence the + * return by value. + */ + virtual std::vector<network_interface> network_interfaces(); +}; + +struct network_stack_entry { + using factory_func = noncopyable_function<future<std::unique_ptr<network_stack>> (const program_options::option_group&)>; + + sstring name; + std::unique_ptr<program_options::option_group> opts; + factory_func factory; + bool is_default; +}; + +} diff --git a/src/seastar/include/seastar/net/arp.hh b/src/seastar/include/seastar/net/arp.hh new file mode 100644 index 000000000..563e1e2fd --- /dev/null +++ b/src/seastar/include/seastar/net/arp.hh @@ -0,0 +1,296 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + * + */ + +#pragma once + +#include <seastar/net/net.hh> +#include <seastar/core/byteorder.hh> +#include <seastar/net/ethernet.hh> +#include <unordered_map> + +namespace seastar { + +namespace net { + +class arp; +class arp_for_protocol; +template <typename L3> +class arp_for; + +class arp_for_protocol { +protected: + arp& _arp; + uint16_t _proto_num; +public: + arp_for_protocol(arp& a, uint16_t proto_num); + virtual ~arp_for_protocol(); + virtual future<> received(packet p) = 0; + virtual bool forward(forward_hash& out_hash_data, packet& p, size_t off) { return false; } +}; + +class arp { + interface* _netif; + l3_protocol _proto; + std::unordered_map<uint16_t, arp_for_protocol*> _arp_for_protocol; + circular_buffer<l3_protocol::l3packet> _packetq; +private: + struct arp_hdr { + uint16_t htype; + uint16_t ptype; + + static arp_hdr read(const char* p) { + arp_hdr ah; + ah.htype = consume_be<uint16_t>(p); + ah.ptype = consume_be<uint16_t>(p); + return ah; + } + static constexpr size_t size() { return 4; } + }; +public: + explicit arp(interface* netif); + void add(uint16_t proto_num, arp_for_protocol* afp); + void del(uint16_t proto_num); +private: + ethernet_address l2self() const noexcept { 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::optional<l3_protocol::l3packet> get_packet(); + template <class l3_proto> + friend class arp_for; +}; + +template <typename L3> +class arp_for : public arp_for_protocol { +public: + using l2addr = ethernet_address; + using l3addr = typename L3::address_type; +private: + static constexpr auto max_waiters = 512; + enum oper { + op_request = 1, + op_reply = 2, + }; + struct arp_hdr { + uint16_t htype; + uint16_t ptype; + uint8_t hlen; + uint8_t plen; + uint16_t oper; + l2addr sender_hwaddr; + l3addr sender_paddr; + l2addr target_hwaddr; + l3addr target_paddr; + + static arp_hdr read(const char* p) { + arp_hdr ah; + ah.htype = consume_be<uint16_t>(p); + ah.ptype = consume_be<uint16_t>(p); + ah.hlen = consume_be<uint8_t>(p); + ah.plen = consume_be<uint8_t>(p); + ah.oper = consume_be<uint16_t>(p); + ah.sender_hwaddr = l2addr::consume(p); + ah.sender_paddr = l3addr::consume(p); + ah.target_hwaddr = l2addr::consume(p); + ah.target_paddr = l3addr::consume(p); + return ah; + } + void write(char* p) const { + produce_be<uint16_t>(p, htype); + produce_be<uint16_t>(p, ptype); + produce_be<uint8_t>(p, hlen); + produce_be<uint8_t>(p, plen); + produce_be<uint16_t>(p, oper); + sender_hwaddr.produce(p); + sender_paddr.produce(p); + target_hwaddr.produce(p); + target_paddr.produce(p); + } + static constexpr size_t size() { + return 8 + 2 * (l2addr::size() + l3addr::size()); + } + }; + struct resolution { + std::vector<promise<l2addr>> _waiters; + timer<> _timeout_timer; + }; +private: + l3addr _l3self = L3::broadcast_address(); + std::unordered_map<l3addr, l2addr> _table; + std::unordered_map<l3addr, resolution> _in_progress; +private: + packet make_query_packet(l3addr paddr); + virtual future<> received(packet p) override; + future<> handle_request(arp_hdr* ah); + l2addr l2self() const noexcept { return _arp.l2self(); } + void send(l2addr to, packet p); +public: + future<> send_query(const l3addr& paddr); + explicit arp_for(arp& a) : arp_for_protocol(a, L3::arp_protocol_type()) { + _table[L3::broadcast_address()] = ethernet::broadcast_address(); + } + future<ethernet_address> lookup(const l3addr& addr); + void learn(l2addr l2, l3addr l3); + void run(); + void set_self_addr(l3addr addr) { + _table.erase(_l3self); + _table[addr] = l2self(); + _l3self = addr; + } + friend class arp; +}; + +template <typename L3> +packet +arp_for<L3>::make_query_packet(l3addr paddr) { + arp_hdr hdr; + hdr.htype = ethernet::arp_hardware_type(); + hdr.ptype = L3::arp_protocol_type(); + hdr.hlen = sizeof(l2addr); + hdr.plen = sizeof(l3addr); + hdr.oper = op_request; + hdr.sender_hwaddr = l2self(); + hdr.sender_paddr = _l3self; + hdr.target_hwaddr = ethernet::broadcast_address(); + hdr.target_paddr = paddr; + auto p = packet(); + p.prepend_uninitialized_header(hdr.size()); + hdr.write(p.get_header(0, hdr.size())); + return p; +} + +template <typename L3> +void arp_for<L3>::send(l2addr to, packet p) { + _arp._packetq.push_back(l3_protocol::l3packet{eth_protocol_num::arp, to, std::move(p)}); +} + +template <typename L3> +future<> +arp_for<L3>::send_query(const l3addr& paddr) { + send(ethernet::broadcast_address(), make_query_packet(paddr)); + return make_ready_future<>(); +} + +class arp_error : public std::runtime_error { +public: + arp_error(const std::string& msg) : std::runtime_error(msg) {} +}; + +class arp_timeout_error : public arp_error { +public: + arp_timeout_error() : arp_error("ARP timeout") {} +}; + +class arp_queue_full_error : public arp_error { +public: + arp_queue_full_error() : arp_error("ARP waiter's queue is full") {} +}; + +template <typename L3> +future<ethernet_address> +arp_for<L3>::lookup(const l3addr& paddr) { + auto i = _table.find(paddr); + if (i != _table.end()) { + return make_ready_future<ethernet_address>(i->second); + } + auto j = _in_progress.find(paddr); + auto first_request = j == _in_progress.end(); + auto& res = first_request ? _in_progress[paddr] : j->second; + + if (first_request) { + res._timeout_timer.set_callback([paddr, this, &res] { + // FIXME: future is discarded + (void)send_query(paddr); + for (auto& w : res._waiters) { + w.set_exception(arp_timeout_error()); + } + res._waiters.clear(); + }); + res._timeout_timer.arm_periodic(std::chrono::seconds(1)); + // FIXME: future is discarded + (void)send_query(paddr); + } + + if (res._waiters.size() >= max_waiters) { + return make_exception_future<ethernet_address>(arp_queue_full_error()); + } + + res._waiters.emplace_back(); + return res._waiters.back().get_future(); +} + +template <typename L3> +void +arp_for<L3>::learn(l2addr hwaddr, l3addr paddr) { + _table[paddr] = hwaddr; + auto i = _in_progress.find(paddr); + if (i != _in_progress.end()) { + auto& res = i->second; + res._timeout_timer.cancel(); + for (auto &&pr : res._waiters) { + pr.set_value(hwaddr); + } + _in_progress.erase(i); + } +} + +template <typename L3> +future<> +arp_for<L3>::received(packet p) { + auto ah = p.get_header(0, arp_hdr::size()); + if (!ah) { + return make_ready_future<>(); + } + auto h = arp_hdr::read(ah); + if (h.hlen != sizeof(l2addr) || h.plen != sizeof(l3addr)) { + return make_ready_future<>(); + } + switch (h.oper) { + case op_request: + return handle_request(&h); + case op_reply: + arp_learn(h.sender_hwaddr, h.sender_paddr); + return make_ready_future<>(); + default: + return make_ready_future<>(); + } +} + +template <typename L3> +future<> +arp_for<L3>::handle_request(arp_hdr* ah) { + if (ah->target_paddr == _l3self + && _l3self != L3::broadcast_address()) { + ah->oper = op_reply; + ah->target_hwaddr = ah->sender_hwaddr; + ah->target_paddr = ah->sender_paddr; + ah->sender_hwaddr = l2self(); + ah->sender_paddr = _l3self; + auto p = packet(); + ah->write(p.prepend_uninitialized_header(ah->size())); + send(ah->target_hwaddr, std::move(p)); + } + return make_ready_future<>(); +} + +} + +} diff --git a/src/seastar/include/seastar/net/byteorder.hh b/src/seastar/include/seastar/net/byteorder.hh new file mode 100644 index 000000000..20342c3d0 --- /dev/null +++ b/src/seastar/include/seastar/net/byteorder.hh @@ -0,0 +1,128 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <arpa/inet.h> // for ntohs() and friends +#include <iosfwd> +#include <utility> + +#include <seastar/core/unaligned.hh> + +namespace seastar { + +inline uint64_t ntohq(uint64_t v) { +#if defined(__BYTE_ORDER__) && defined(__ORDER_BIG_ENDIAN__) && __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + // big endian, nothing to do + return v; +#else + // little endian, reverse bytes + return __builtin_bswap64(v); +#endif +} +inline uint64_t htonq(uint64_t v) { + // htonq and ntohq have identical implementations + return ntohq(v); +} + +namespace net { + +inline void ntoh() {} +inline void hton() {} + +inline uint8_t ntoh(uint8_t x) { return x; } +inline uint8_t hton(uint8_t x) { return x; } +inline uint16_t ntoh(uint16_t x) { return ntohs(x); } +inline uint16_t hton(uint16_t x) { return htons(x); } +inline uint32_t ntoh(uint32_t x) { return ntohl(x); } +inline uint32_t hton(uint32_t x) { return htonl(x); } +inline uint64_t ntoh(uint64_t x) { return ntohq(x); } +inline uint64_t hton(uint64_t x) { return htonq(x); } + +inline int8_t ntoh(int8_t x) { return x; } +inline int8_t hton(int8_t x) { return x; } +inline int16_t ntoh(int16_t x) { return ntohs(x); } +inline int16_t hton(int16_t x) { return htons(x); } +inline int32_t ntoh(int32_t x) { return ntohl(x); } +inline int32_t hton(int32_t x) { return htonl(x); } +inline int64_t ntoh(int64_t x) { return ntohq(x); } +inline int64_t hton(int64_t x) { return htonq(x); } + +// Deprecated alias net::packed<> for unaligned<> from unaligned.hh. +// TODO: get rid of this alias. +template <typename T> using packed = unaligned<T>; + +template <typename T> +inline T ntoh(const packed<T>& x) { + T v = x; + return ntoh(v); +} + +template <typename T> +inline T hton(const packed<T>& x) { + T v = x; + return hton(v); +} + +template <typename T> +inline std::ostream& operator<<(std::ostream& os, const packed<T>& v) { + auto x = v.raw; + return os << x; +} + +inline +void ntoh_inplace() {} +inline +void hton_inplace() {}; + +template <typename First, typename... Rest> +inline +void ntoh_inplace(First& first, Rest&... rest) { + first = ntoh(first); + ntoh_inplace(std::forward<Rest&>(rest)...); +} + +template <typename First, typename... Rest> +inline +void hton_inplace(First& first, Rest&... rest) { + first = hton(first); + hton_inplace(std::forward<Rest&>(rest)...); +} + +template <class T> +inline +T ntoh(const T& x) { + T tmp = x; + tmp.adjust_endianness([] (auto&&... what) { ntoh_inplace(std::forward<decltype(what)&>(what)...); }); + return tmp; +} + +template <class T> +inline +T hton(const T& x) { + T tmp = x; + tmp.adjust_endianness([] (auto&&... what) { hton_inplace(std::forward<decltype(what)&>(what)...); }); + return tmp; +} + +} + +} diff --git a/src/seastar/include/seastar/net/config.hh b/src/seastar/include/seastar/net/config.hh new file mode 100644 index 000000000..6432d3c45 --- /dev/null +++ b/src/seastar/include/seastar/net/config.hh @@ -0,0 +1,65 @@ +/* + * 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 2017 Marek Waszkiewicz ( marek.waszkiewicz77@gmail.com ) + */ + +#pragma once + +#include <seastar/util/std-compat.hh> +#include <istream> +#include <string> +#include <unordered_map> + +namespace seastar { +namespace net { + + struct ipv4_config { + std::string ip; + std::string netmask; + std::string gateway; + bool dhcp{ false }; + }; + + struct hw_config { + std::string pci_address; + std::optional<unsigned> port_index; + bool lro{ true }; + bool tso{ true }; + bool ufo{ true }; + bool hw_fc{ true }; + bool event_index{ true }; + bool csum_offload{ true }; + std::optional<unsigned> ring_size; + }; + + struct device_config { + ipv4_config ip_cfg; + hw_config hw_cfg; + }; + + std::unordered_map<std::string, device_config> parse_config(std::istream& input); + + class config_exception : public std::runtime_error { + public: + config_exception(const std::string& msg) + : std::runtime_error(msg) { + } + }; +} +} diff --git a/src/seastar/include/seastar/net/const.hh b/src/seastar/include/seastar/net/const.hh new file mode 100644 index 000000000..7a79b85d3 --- /dev/null +++ b/src/seastar/include/seastar/net/const.hh @@ -0,0 +1,44 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +namespace seastar { + +namespace net { + +enum class ip_protocol_num : uint8_t { + icmp = 1, tcp = 6, udp = 17, unused = 255 +}; + +enum class eth_protocol_num : uint16_t { + ipv4 = 0x0800, arp = 0x0806, ipv6 = 0x86dd +}; + +const uint8_t eth_hdr_len = 14; +const uint8_t tcp_hdr_len_min = 20; +const uint8_t ipv4_hdr_len_min = 20; +const uint8_t ipv6_hdr_len_min = 40; +const uint16_t ip_packet_len_max = 65535; + +} + +} diff --git a/src/seastar/include/seastar/net/dhcp.hh b/src/seastar/include/seastar/net/dhcp.hh new file mode 100644 index 000000000..09d209e83 --- /dev/null +++ b/src/seastar/include/seastar/net/dhcp.hh @@ -0,0 +1,84 @@ +/* + * 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 2014 Cloudius Systems + */ + +#pragma once + +#include <seastar/net/ip.hh> +#include <seastar/util/std-compat.hh> + +namespace seastar { + +namespace net { + +/* + * Simplistic DHCP query class. + * Due to the nature of the native stack, + * it operates on an "ipv4" object instead of, + * for example, an interface. + */ +class dhcp { +public: + dhcp(ipv4 &); + dhcp(dhcp &&) noexcept; + ~dhcp(); + + static const steady_clock_type::duration default_timeout; + + struct lease { + ipv4_address ip; + ipv4_address netmask; + ipv4_address broadcast; + + ipv4_address gateway; + ipv4_address dhcp_server; + + std::vector<ipv4_address> name_servers; + + std::chrono::seconds lease_time; + std::chrono::seconds renew_time; + std::chrono::seconds rebind_time; + + uint16_t mtu = 0; + }; + + typedef future<std::optional<lease>> result_type; + + /** + * Runs a discover/request sequence on the ipv4 "stack". + * During this execution the ipv4 will be "hijacked" + * more or less (through packet filter), and while not + * inoperable, most likely quite less efficient. + * + * Please note that this does _not_ modify the ipv4 object bound. + * It only makes queries and records replys for the related NIC. + * It is up to caller to use the returned information as he se fit. + */ + result_type discover(const steady_clock_type::duration & = default_timeout); + result_type renew(const lease &, const steady_clock_type::duration & = default_timeout); + ip_packet_filter* get_ipv4_filter(); +private: + class impl; + std::unique_ptr<impl> _impl; +}; + +} + +} diff --git a/src/seastar/include/seastar/net/dns.hh b/src/seastar/include/seastar/net/dns.hh new file mode 100644 index 000000000..3ece937d2 --- /dev/null +++ b/src/seastar/include/seastar/net/dns.hh @@ -0,0 +1,156 @@ +/* + * 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 2016 Cloudius Systems + */ + +#pragma once + +#include <vector> +#include <unordered_map> +#include <memory> +#include <seastar/util/std-compat.hh> + +#include <seastar/core/future.hh> +#include <seastar/core/sstring.hh> +#include <seastar/core/shared_ptr.hh> +#include <seastar/net/inet_address.hh> + +namespace seastar { + +struct ipv4_addr; + +class socket_address; +class network_stack; + +/** + * C-ares based dns query support. + * Handles name- and ip-based resolution. + * + */ + +namespace net { + +/** + * A c++-esque version of a hostent + */ +struct hostent { + // Primary name is always first + std::vector<sstring> names; + // Primary address is also always first. + std::vector<inet_address> addr_list; +}; + +typedef std::optional<inet_address::family> opt_family; + +struct srv_record { + unsigned short priority; + unsigned short weight; + unsigned short port; + sstring target; +}; + +/** + * A DNS resolver object. + * Wraps the query logic & networking. + * Can be instantiated with options and your network + * stack of choice, though for "normal" non-test + * querying, you are probably better of with the + * global calls further down. + */ +class dns_resolver { +public: + struct options { + std::optional<bool> + use_tcp_query; + std::optional<std::vector<inet_address>> + servers; + std::optional<std::chrono::milliseconds> + timeout; + std::optional<uint16_t> + tcp_port, udp_port; + std::optional<std::vector<sstring>> + domains; + }; + + enum class srv_proto { + tcp, udp + }; + using srv_records = std::vector<srv_record>; + + dns_resolver(); + dns_resolver(dns_resolver&&) noexcept; + explicit dns_resolver(const options&); + explicit dns_resolver(network_stack&, const options& = {}); + ~dns_resolver(); + + dns_resolver& operator=(dns_resolver&&) noexcept; + + /** + * Resolves a hostname to one or more addresses and aliases + */ + future<hostent> get_host_by_name(const sstring&, opt_family = {}); + /** + * Resolves an address to one or more addresses and aliases + */ + future<hostent> get_host_by_addr(const inet_address&); + + /** + * Resolves a hostname to one (primary) address + */ + future<inet_address> resolve_name(const sstring&, opt_family = {}); + /** + * Resolves an address to one (primary) name + */ + future<sstring> resolve_addr(const inet_address&); + + /** + * Resolve a service in given domain to one or more SRV records + */ + future<srv_records> get_srv_records(srv_proto proto, + const sstring& service, + const sstring& domain); + + /** + * Shuts the object down. Great for tests. + */ + future<> close(); +private: + class impl; + shared_ptr<impl> _impl; +}; + +namespace dns { + +// See above. These functions simply queries using a shard-local +// default-stack, default-opts resolver +future<hostent> get_host_by_name(const sstring&, opt_family = {}); +future<hostent> get_host_by_addr(const inet_address&); + +future<inet_address> resolve_name(const sstring&, opt_family = {}); +future<sstring> resolve_addr(const inet_address&); + +future<std::vector<srv_record>> get_srv_records(dns_resolver::srv_proto proto, + const sstring& service, + const sstring& domain); + +} + +} + +} diff --git a/src/seastar/include/seastar/net/dpdk.hh b/src/seastar/include/seastar/net/dpdk.hh new file mode 100644 index 000000000..bd07bb0dd --- /dev/null +++ b/src/seastar/include/seastar/net/dpdk.hh @@ -0,0 +1,76 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <memory> +#include <seastar/net/config.hh> +#include <seastar/net/net.hh> +#include <seastar/core/sstring.hh> +#include <seastar/util/program-options.hh> + +namespace seastar { + +namespace net { + +/// DPDK configuration. +struct dpdk_options : public program_options::option_group { + /// DPDK Port Index. + /// + /// Default: 0. + program_options::value<unsigned> dpdk_port_index; + /// \brief Enable HW Flow Control (on / off). + /// + /// Default: \p on. + program_options::value<std::string> hw_fc; + + /// \cond internal + dpdk_options(program_options::option_group* parent_group); + /// \endcond +}; + +} + +/// \cond internal + +#ifdef SEASTAR_HAVE_DPDK + +std::unique_ptr<net::device> create_dpdk_net_device( + uint16_t port_idx = 0, + uint16_t num_queues = 1, + bool use_lro = true, + bool enable_fc = true); + +std::unique_ptr<net::device> create_dpdk_net_device( + const net::hw_config& hw_cfg); + +namespace dpdk { +/** + * @return Number of bytes needed for mempool objects of each QP. + */ +uint32_t qp_mempool_obj_size(bool hugetlbfs_membackend); +} + +/// \endcond + +#endif // SEASTAR_HAVE_DPDK + +} diff --git a/src/seastar/include/seastar/net/ethernet.hh b/src/seastar/include/seastar/net/ethernet.hh new file mode 100644 index 000000000..041015537 --- /dev/null +++ b/src/seastar/include/seastar/net/ethernet.hh @@ -0,0 +1,100 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <array> +#include <assert.h> +#include <algorithm> +#include <seastar/net/byteorder.hh> + +namespace seastar { + +namespace net { + +struct ethernet_address { + ethernet_address() noexcept + : mac{} {} + + ethernet_address(const uint8_t *eaddr) noexcept { + std::copy(eaddr, eaddr + 6, mac.begin()); + } + + ethernet_address(std::initializer_list<uint8_t> eaddr) noexcept { + assert(eaddr.size() == mac.size()); + std::copy(eaddr.begin(), eaddr.end(), mac.begin()); + } + + std::array<uint8_t, 6> mac; + + template <typename Adjuster> + void adjust_endianness(Adjuster a) noexcept {} + + static ethernet_address read(const char* p) noexcept { + ethernet_address ea; + std::copy_n(p, size(), reinterpret_cast<char*>(ea.mac.data()));\ + return ea; + } + static ethernet_address consume(const char*& p) noexcept { + auto ea = read(p); + p += size(); + return ea; + } + void write(char* p) const noexcept { + std::copy_n(reinterpret_cast<const char*>(mac.data()), size(), p); + } + void produce(char*& p) const noexcept { + write(p); + p += size(); + } + static constexpr size_t size() noexcept { + return 6; + } +} __attribute__((packed)); + +std::ostream& operator<<(std::ostream& os, ethernet_address ea); + +struct ethernet { + using address = ethernet_address; + static address broadcast_address() { + return {0xff, 0xff, 0xff, 0xff, 0xff, 0xff}; + } + static constexpr uint16_t arp_hardware_type() { return 1; } +}; + +struct eth_hdr { + ethernet_address dst_mac; + ethernet_address src_mac; + packed<uint16_t> eth_proto; + template <typename Adjuster> + auto adjust_endianness(Adjuster a) { + return a(eth_proto); + } +} __attribute__((packed)); + +ethernet_address parse_ethernet_address(std::string addr); +} + +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<seastar::net::ethernet_address> : fmt::ostream_formatter {}; +#endif diff --git a/src/seastar/include/seastar/net/inet_address.hh b/src/seastar/include/seastar/net/inet_address.hh new file mode 100644 index 000000000..687ce8e6c --- /dev/null +++ b/src/seastar/include/seastar/net/inet_address.hh @@ -0,0 +1,136 @@ +/* + * 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) 2016 ScyllaDB. + */ + +#pragma once + +#include <iosfwd> +#include <sys/types.h> +#include <netinet/in.h> +#include <stdexcept> +#include <vector> + +#include <seastar/core/future.hh> +#include <seastar/core/sstring.hh> + +namespace seastar { +namespace net { + +struct ipv4_address; +struct ipv6_address; + +class unknown_host : public std::invalid_argument { +public: + using invalid_argument::invalid_argument; +}; + +class inet_address { +public: + enum class family : sa_family_t { + INET = AF_INET, INET6 = AF_INET6 + }; +private: + family _in_family; + + union { + ::in_addr _in; + ::in6_addr _in6; + }; + + uint32_t _scope = invalid_scope; +public: + static constexpr uint32_t invalid_scope = std::numeric_limits<uint32_t>::max(); + + inet_address() noexcept; + inet_address(family) noexcept; + inet_address(::in_addr i) noexcept; + inet_address(::in6_addr i, uint32_t scope = invalid_scope) noexcept; + // NOTE: does _not_ resolve the address. Only parses + // ipv4/ipv6 numerical address + // throws std::invalid_argument if sstring is invalid + inet_address(const sstring&); + inet_address(inet_address&&) noexcept = default; + inet_address(const inet_address&) noexcept = default; + + inet_address(const ipv4_address&) noexcept; + inet_address(const ipv6_address&, uint32_t scope = invalid_scope) noexcept; + + // throws iff ipv6 + ipv4_address as_ipv4_address() const; + ipv6_address as_ipv6_address() const noexcept; + + inet_address& operator=(const inet_address&) noexcept = default; + bool operator==(const inet_address&) const noexcept; + + family in_family() const noexcept { + return _in_family; + } + + bool is_ipv6() const noexcept { + return _in_family == family::INET6; + } + bool is_ipv4() const noexcept { + return _in_family == family::INET; + } + + size_t size() const noexcept; + const void * data() const noexcept; + + uint32_t scope() const noexcept { + return _scope; + } + + // throws iff ipv6 + operator ::in_addr() const; + operator ::in6_addr() const noexcept; + + operator ipv6_address() const noexcept; + + future<sstring> hostname() const; + future<std::vector<sstring>> aliases() const; + + static future<inet_address> find(const sstring&); + static future<inet_address> find(const sstring&, family); + static future<std::vector<inet_address>> find_all(const sstring&); + static future<std::vector<inet_address>> find_all(const sstring&, family); + + static std::optional<inet_address> parse_numerical(const sstring&); + + bool is_loopback() const noexcept; + bool is_addr_any() const noexcept; +}; + +std::ostream& operator<<(std::ostream&, const inet_address&); +std::ostream& operator<<(std::ostream&, const inet_address::family&); + +} +} + +namespace std { +template<> +struct hash<seastar::net::inet_address> { + size_t operator()(const seastar::net::inet_address&) const; +}; +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<seastar::net::inet_address> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<seastar::net::inet_address::family> : fmt::ostream_formatter {}; +#endif diff --git a/src/seastar/include/seastar/net/ip.hh b/src/seastar/include/seastar/net/ip.hh new file mode 100644 index 000000000..2b67df54c --- /dev/null +++ b/src/seastar/include/seastar/net/ip.hh @@ -0,0 +1,415 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + * + */ + +#pragma once + +#include <boost/asio/ip/address_v4.hpp> +#include <arpa/inet.h> +#include <unordered_map> +#include <cstdint> +#include <array> +#include <map> +#include <list> +#include <chrono> +#include <seastar/core/array_map.hh> +#include <seastar/net/byteorder.hh> +#include <seastar/core/byteorder.hh> +#include <seastar/net/arp.hh> +#include <seastar/net/ip_checksum.hh> +#include <seastar/net/const.hh> +#include <seastar/net/packet-util.hh> +#include <seastar/core/shared_ptr.hh> +#include <seastar/net/toeplitz.hh> +#include <seastar/net/udp.hh> +#include <seastar/core/metrics_registration.hh> + +#include "ipv4_address.hh" +#include "ipv6_address.hh" + +namespace seastar { + +namespace net { + +class ipv4; +template <ip_protocol_num ProtoNum> +class ipv4_l4; + +template <typename InetTraits> +class tcp; + +struct ipv4_traits { + using address_type = ipv4_address; + using inet_type = ipv4_l4<ip_protocol_num::tcp>; + struct l4packet { + ipv4_address to; + packet p; + ethernet_address e_dst; + ip_protocol_num proto_num; + }; + using packet_provider_type = std::function<std::optional<l4packet> ()>; + 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); + } + static void udp_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::udp), len); + } + static constexpr uint8_t ip_hdr_len_min = ipv4_hdr_len_min; +}; + +template <ip_protocol_num ProtoNum> +class ipv4_l4 { +public: + ipv4& _inet; +public: + ipv4_l4(ipv4& inet) : _inet(inet) {} + void register_packet_provider(ipv4_traits::packet_provider_type func); + future<ethernet_address> get_l2_dst_address(ipv4_address to); + const ipv4& inet() const { + return _inet; + } +}; + +class ip_protocol { +public: + virtual ~ip_protocol() {} + virtual void received(packet p, ipv4_address from, ipv4_address to) = 0; + virtual bool forward(forward_hash& out_hash_data, packet& p, size_t off) { return true; } +}; + +template <typename InetTraits> +struct l4connid { + using ipaddr = typename InetTraits::address_type; + using inet_type = typename InetTraits::inet_type; + struct connid_hash; + + ipaddr local_ip; + ipaddr foreign_ip; + uint16_t local_port; + uint16_t foreign_port; + + bool operator==(const l4connid& x) const { + return local_ip == x.local_ip + && foreign_ip == x.foreign_ip + && local_port == x.local_port + && foreign_port == x.foreign_port; + } + + uint32_t hash(rss_key_type rss_key) { + forward_hash hash_data; + hash_data.push_back(hton(foreign_ip.ip)); + hash_data.push_back(hton(local_ip.ip)); + hash_data.push_back(hton(foreign_port)); + hash_data.push_back(hton(local_port)); + return toeplitz_hash(rss_key, hash_data); + } +}; + +class ipv4_tcp final : public ip_protocol { + ipv4_l4<ip_protocol_num::tcp> _inet_l4; + std::unique_ptr<tcp<ipv4_traits>> _tcp; +public: + ipv4_tcp(ipv4& inet); + ~ipv4_tcp(); + virtual void received(packet p, ipv4_address from, ipv4_address to) override; + virtual bool forward(forward_hash& out_hash_data, packet& p, size_t off) override; + friend class ipv4; +}; + +struct icmp_hdr { + enum class msg_type : uint8_t { + echo_reply = 0, + echo_request = 8, + }; + msg_type type; + uint8_t code; + packed<uint16_t> csum; + packed<uint32_t> rest; + template <typename Adjuster> + auto adjust_endianness(Adjuster a) { + return a(csum); + } +} __attribute__((packed)); + + +class icmp { +public: + using ipaddr = ipv4_address; + using inet_type = ipv4_l4<ip_protocol_num::icmp>; + explicit icmp(inet_type& inet) : _inet(inet) { + _inet.register_packet_provider([this] { + std::optional<ipv4_traits::l4packet> l4p; + if (!_packetq.empty()) { + l4p = std::move(_packetq.front()); + _packetq.pop_front(); + _queue_space.signal(l4p.value().p.len()); + } + return l4p; + }); + } + void received(packet p, ipaddr from, ipaddr to); +private: + inet_type& _inet; + circular_buffer<ipv4_traits::l4packet> _packetq; + semaphore _queue_space = {212992}; +}; + +class ipv4_icmp final : public ip_protocol { + ipv4_l4<ip_protocol_num::icmp> _inet_l4; + icmp _icmp; +public: + ipv4_icmp(ipv4& inet) : _inet_l4(inet), _icmp(_inet_l4) {} + virtual void received(packet p, ipv4_address from, ipv4_address to) { + _icmp.received(std::move(p), from, to); + } + friend class ipv4; +}; + +class ipv4_udp : public ip_protocol { + using connid = l4connid<ipv4_traits>; + using connid_hash = typename connid::connid_hash; + +public: + static const int default_queue_size; +private: + static const uint16_t min_anonymous_port = 32768; + ipv4 &_inet; + std::unordered_map<uint16_t, lw_shared_ptr<udp_channel_state>> _channels; + int _queue_size = default_queue_size; + uint16_t _next_anonymous_port = min_anonymous_port; + circular_buffer<ipv4_traits::l4packet> _packetq; +private: + uint16_t next_port(uint16_t port); +public: + class registration { + private: + ipv4_udp &_proto; + uint16_t _port; + public: + registration(ipv4_udp &proto, uint16_t port) : _proto(proto), _port(port) {}; + + void unregister() { + _proto._channels.erase(_proto._channels.find(_port)); + } + + uint16_t port() const { + return _port; + } + }; + + ipv4_udp(ipv4& inet); + udp_channel make_channel(ipv4_addr addr); + virtual void received(packet p, ipv4_address from, ipv4_address to) override; + void send(uint16_t src_port, ipv4_addr dst, packet &&p); + bool forward(forward_hash& out_hash_data, packet& p, size_t off) override; + void set_queue_size(int size) { _queue_size = size; } + + const ipv4& inet() const { + return _inet; + } +}; + +struct ip_hdr; + +struct ip_packet_filter { + virtual ~ip_packet_filter() {}; + virtual future<> handle(packet& p, ip_hdr* iph, ethernet_address from, bool & handled) = 0; +}; + +struct ipv4_frag_id { + struct hash; + ipv4_address src_ip; + ipv4_address dst_ip; + uint16_t identification; + uint8_t protocol; + bool operator==(const ipv4_frag_id& x) const { + return src_ip == x.src_ip && + dst_ip == x.dst_ip && + identification == x.identification && + protocol == x.protocol; + } +}; + +struct ipv4_frag_id::hash : private std::hash<ipv4_address>, + private std::hash<uint16_t>, private std::hash<uint8_t> { + size_t operator()(const ipv4_frag_id& id) const noexcept { + using h1 = std::hash<ipv4_address>; + using h2 = std::hash<uint16_t>; + using h3 = std::hash<uint8_t>; + return h1::operator()(id.src_ip) ^ + h1::operator()(id.dst_ip) ^ + h2::operator()(id.identification) ^ + h3::operator()(id.protocol); + } +}; + +struct ipv4_tag {}; +using ipv4_packet_merger = packet_merger<uint32_t, ipv4_tag>; + +class ipv4 { +public: + using clock_type = lowres_clock; + using address_type = ipv4_address; + using proto_type = uint16_t; + static address_type broadcast_address() { return ipv4_address(0xffffffff); } + static proto_type arp_protocol_type() { return proto_type(eth_protocol_num::ipv4); } +private: + interface* _netif; + std::vector<ipv4_traits::packet_provider_type> _pkt_providers; + arp _global_arp; + arp_for<ipv4> _arp; + ipv4_address _host_address; + ipv4_address _gw_address; + ipv4_address _netmask; + l3_protocol _l3; + ipv4_tcp _tcp; + ipv4_icmp _icmp; + ipv4_udp _udp; + array_map<ip_protocol*, 256> _l4; + ip_packet_filter * _packet_filter = nullptr; + struct frag { + packet header; + ipv4_packet_merger data; + clock_type::time_point rx_time; + uint32_t mem_size = 0; + // fragment with MF == 0 inidates it is the last fragment + bool last_frag_received = false; + + packet get_assembled_packet(ethernet_address from, ethernet_address to); + int32_t merge(ip_hdr &h, uint16_t offset, packet p); + bool is_complete(); + }; + std::unordered_map<ipv4_frag_id, frag, ipv4_frag_id::hash> _frags; + std::list<ipv4_frag_id> _frags_age; + static constexpr std::chrono::seconds _frag_timeout{30}; + static constexpr uint32_t _frag_low_thresh{3 * 1024 * 1024}; + static constexpr uint32_t _frag_high_thresh{4 * 1024 * 1024}; + uint32_t _frag_mem{0}; + timer<lowres_clock> _frag_timer; + circular_buffer<l3_protocol::l3packet> _packetq; + unsigned _pkt_provider_idx = 0; + metrics::metric_groups _metrics; +private: + future<> handle_received_packet(packet p, ethernet_address from); + bool forward(forward_hash& out_hash_data, packet& p, size_t off); + std::optional<l3_protocol::l3packet> get_packet(); + bool in_my_netmask(ipv4_address a) const; + void frag_limit_mem(); + void frag_timeout(); + void frag_drop(ipv4_frag_id frag_id, uint32_t dropped_size); + void frag_arm(clock_type::time_point now) { + auto tp = now + _frag_timeout; + _frag_timer.arm(tp); + } + void frag_arm() { + auto now = clock_type::now(); + frag_arm(now); + } +public: + explicit ipv4(interface* netif); + void set_host_address(ipv4_address ip); + ipv4_address host_address() const; + void set_gw_address(ipv4_address ip); + ipv4_address gw_address() const; + void set_netmask_address(ipv4_address ip); + ipv4_address netmask_address() const; + interface * netif() const { + return _netif; + } + // TODO or something. Should perhaps truly be a list + // of filters. With ordering. And blackjack. Etc. + // But for now, a simple single raw pointer suffices + void set_packet_filter(ip_packet_filter *); + ip_packet_filter * packet_filter() const; + void send(ipv4_address to, ip_protocol_num proto_num, packet p, ethernet_address e_dst); + tcp<ipv4_traits>& get_tcp() { return *_tcp._tcp; } + ipv4_udp& get_udp() { return _udp; } + void register_l4(proto_type id, ip_protocol* handler); + const net::hw_features& hw_features() const { return _netif->hw_features(); } + static bool needs_frag(packet& p, ip_protocol_num proto_num, net::hw_features hw_features); + void learn(ethernet_address l2, ipv4_address l3) { + _arp.learn(l2, l3); + } + void register_packet_provider(ipv4_traits::packet_provider_type&& func) { + _pkt_providers.push_back(std::move(func)); + } + future<ethernet_address> get_l2_dst_address(ipv4_address to); +}; + +template <ip_protocol_num ProtoNum> +inline +void ipv4_l4<ProtoNum>::register_packet_provider(ipv4_traits::packet_provider_type func) { + _inet.register_packet_provider([func = std::move(func)] { + auto l4p = func(); + if (l4p) { + l4p.value().proto_num = ProtoNum; + } + return l4p; + }); +} + +template <ip_protocol_num ProtoNum> +inline +future<ethernet_address> ipv4_l4<ProtoNum>::get_l2_dst_address(ipv4_address to) { + return _inet.get_l2_dst_address(to); +} + +struct ip_hdr { + uint8_t ihl : 4; + uint8_t ver : 4; + uint8_t dscp : 6; + uint8_t ecn : 2; + packed<uint16_t> len; + packed<uint16_t> id; + packed<uint16_t> frag; + enum class frag_bits : uint8_t { mf = 13, df = 14, reserved = 15, offset_shift = 3 }; + uint8_t ttl; + uint8_t ip_proto; + packed<uint16_t> csum; + ipv4_address src_ip; + ipv4_address dst_ip; + uint8_t options[0]; + template <typename Adjuster> + auto adjust_endianness(Adjuster a) { + return a(len, id, frag, csum, src_ip, dst_ip); + } + bool mf() { return frag & (1 << uint8_t(frag_bits::mf)); } + bool df() { return frag & (1 << uint8_t(frag_bits::df)); } + uint16_t offset() { return frag << uint8_t(frag_bits::offset_shift); } +} __attribute__((packed)); + +template <typename InetTraits> +struct l4connid<InetTraits>::connid_hash : private std::hash<ipaddr>, private std::hash<uint16_t> { + size_t operator()(const l4connid<InetTraits>& id) const noexcept { + using h1 = std::hash<ipaddr>; + using h2 = std::hash<uint16_t>; + return h1::operator()(id.local_ip) + ^ h1::operator()(id.foreign_ip) + ^ h2::operator()(id.local_port) + ^ h2::operator()(id.foreign_port); + } +}; + +void arp_learn(ethernet_address l2, ipv4_address l3); + +} + +} diff --git a/src/seastar/include/seastar/net/ip_checksum.hh b/src/seastar/include/seastar/net/ip_checksum.hh new file mode 100644 index 000000000..cb3539149 --- /dev/null +++ b/src/seastar/include/seastar/net/ip_checksum.hh @@ -0,0 +1,76 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/net/packet.hh> +#include <cstdint> +#include <cstddef> +#include <arpa/inet.h> + +namespace seastar { + +namespace net { + +uint16_t ip_checksum(const void* data, size_t len); + +struct checksummer { + __int128 csum = 0; + bool odd = false; + void sum(const char* data, size_t len); + void sum(const packet& p); + void sum(uint8_t data) { + if (!odd) { + csum += data << 8; + } else { + csum += data; + } + odd = !odd; + } + void sum(uint16_t data) { + if (odd) { + sum(uint8_t(data >> 8)); + sum(uint8_t(data)); + } else { + csum += data; + } + } + void sum(uint32_t data) { + if (odd) { + sum(uint16_t(data)); + sum(uint16_t(data >> 16)); + } else { + csum += data; + } + } + void sum_many() {} + template <typename T0, typename... T> + void sum_many(T0 data, T... rest) { + sum(data); + sum_many(rest...); + } + uint16_t get() const; +}; + +} + +} + diff --git a/src/seastar/include/seastar/net/ipv4_address.hh b/src/seastar/include/seastar/net/ipv4_address.hh new file mode 100644 index 000000000..6056e23e2 --- /dev/null +++ b/src/seastar/include/seastar/net/ipv4_address.hh @@ -0,0 +1,89 @@ +/* + * 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) 2022 ScyllaDB + * + */ + +#pragma once + +#include <seastar/net/socket_defs.hh> +#include <seastar/core/byteorder.hh> + +namespace seastar { + +namespace net { + +struct ipv4_address { + ipv4_address() noexcept : ip(0) {} + explicit ipv4_address(uint32_t ip) noexcept : ip(ip) {} + // throws if addr is not a valid ipv4 address + explicit ipv4_address(const std::string& addr); + ipv4_address(ipv4_addr addr) noexcept { + ip = addr.ip; + } + + packed<uint32_t> ip; + + template <typename Adjuster> + auto adjust_endianness(Adjuster a) { return a(ip); } + + friend bool operator==(ipv4_address x, ipv4_address y) noexcept { + return x.ip == y.ip; + } + friend bool operator!=(ipv4_address x, ipv4_address y) noexcept { + return x.ip != y.ip; + } + + static ipv4_address read(const char* p) noexcept { + ipv4_address ia; + ia.ip = read_be<uint32_t>(p); + return ia; + } + static ipv4_address consume(const char*& p) noexcept { + auto ia = read(p); + p += 4; + return ia; + } + void write(char* p) const noexcept { + write_be<uint32_t>(p, ip); + } + void produce(char*& p) const noexcept { + produce_be<uint32_t>(p, ip); + } + static constexpr size_t size() { + return 4; + } +} __attribute__((packed)); + +static inline bool is_unspecified(ipv4_address addr) noexcept { return addr.ip == 0; } + +std::ostream& operator<<(std::ostream& os, const ipv4_address& a); + +} + +} + +namespace std { + +template <> +struct hash<seastar::net::ipv4_address> { + size_t operator()(seastar::net::ipv4_address a) const { return a.ip; } +}; + +} diff --git a/src/seastar/include/seastar/net/ipv6_address.hh b/src/seastar/include/seastar/net/ipv6_address.hh new file mode 100644 index 000000000..4906f2bfa --- /dev/null +++ b/src/seastar/include/seastar/net/ipv6_address.hh @@ -0,0 +1,90 @@ +/* + * 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) 2022 ScyllaDB + * + */ + +#pragma once + +#include <seastar/net/socket_defs.hh> +#include <seastar/core/byteorder.hh> + +#include <array> + +namespace seastar { + +namespace net { + +struct ipv6_address { + using ipv6_bytes = std::array<uint8_t, 16>; + + static_assert(alignof(ipv6_bytes) == 1, "ipv6_bytes should be byte-aligned"); + static_assert(sizeof(ipv6_bytes) == 16, "ipv6_bytes should be 16 bytes"); + + ipv6_address() noexcept; + explicit ipv6_address(const ::in6_addr&) noexcept; + explicit ipv6_address(const ipv6_bytes&) noexcept; + // throws if addr is not a valid ipv6 address + explicit ipv6_address(const std::string&); + ipv6_address(const ipv6_addr& addr) noexcept; + + // No need to use packed - we only store + // as byte array. If we want to read as + // uints or whatnot, we must copy + ipv6_bytes ip; + + template <typename Adjuster> + auto adjust_endianness(Adjuster a) { return a(ip); } + + bool operator==(const ipv6_address& y) const noexcept { + return bytes() == y.bytes(); + } + bool operator!=(const ipv6_address& y) const noexcept { + return !(*this == y); + } + + const ipv6_bytes& bytes() const noexcept { + return ip; + } + + bool is_unspecified() const noexcept; + + static ipv6_address read(const char*) noexcept; + static ipv6_address consume(const char*& p) noexcept; + void write(char* p) const noexcept; + void produce(char*& p) const noexcept; + static constexpr size_t size() { + return sizeof(ipv6_bytes); + } +} __attribute__((packed)); + +std::ostream& operator<<(std::ostream&, const ipv6_address&); + +} + +} + +namespace std { + +template <> +struct hash<seastar::net::ipv6_address> { + size_t operator()(const seastar::net::ipv6_address&) const; +}; + +} diff --git a/src/seastar/include/seastar/net/native-stack.hh b/src/seastar/include/seastar/net/native-stack.hh new file mode 100644 index 000000000..5c869a16f --- /dev/null +++ b/src/seastar/include/seastar/net/native-stack.hh @@ -0,0 +1,99 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/net/net.hh> +#include <seastar/net/virtio.hh> +#include <seastar/net/dpdk.hh> +#include <seastar/util/program-options.hh> + +namespace seastar { + +struct network_stack_entry; + +namespace net { + +/// Native stack configuration. +struct native_stack_options : public program_options::option_group { + program_options::value<std::string> net_config; + program_options::value<std::string> net_config_file; + /// \brief Tap device to connect to. + /// + /// Default: \p tap0. + program_options::value<std::string> tap_device; + /// \brief Static IPv4 address to use. + /// + /// Default: \p 192.168.122.2. + program_options::value<std::string> host_ipv4_addr; + /// \brief Static IPv4 gateway to use. + /// + /// Default: \p 192.168.122.1. + program_options::value<std::string> gw_ipv4_addr; + /// \brief Static IPv4 netmask to use. + /// + /// Default: \p 255.255.255.0. + program_options::value<std::string> netmask_ipv4_addr; + /// \brief Default size of the UDPv4 per-channel packet queue. + /// + /// Default: \ref ipv4_udp::default_queue_size. + program_options::value<int> udpv4_queue_size; + /// \brief Use DHCP discovery. + /// + /// Default: \p true. + program_options::value<bool> dhcp; + /// \brief Weighing of a hardware network queue relative to a software queue. + /// + /// Values: + /// * 0.0: no work + /// * 1.0: equal share + /// + /// Default: 1.0. + program_options::value<float> hw_queue_weight; + /// \brief Use DPDK PMD drivers. + /// + /// \note Unused when seastar is compiled without DPDK support. + program_options::value<> dpdk_pmd; + /// \brief Enable LRO (on/off). + /// + /// Default: \p on. + program_options::value<std::string> lro; + + /// Virtio configuration. + virtio_options virtio_opts; + /// DPDK configuration. + /// + /// \note Unused when seastar is compiled without DPDK support. + dpdk_options dpdk_opts; + + /// \cond internal + bool _hugepages; + + native_stack_options(); + /// \endcond +}; + +void create_native_stack(const native_stack_options& opts, std::shared_ptr<device> dev); +network_stack_entry register_native_stack(); + +} + +} diff --git a/src/seastar/include/seastar/net/net.hh b/src/seastar/include/seastar/net/net.hh new file mode 100644 index 000000000..f35d8ca97 --- /dev/null +++ b/src/seastar/include/seastar/net/net.hh @@ -0,0 +1,308 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/core/smp.hh> +#include <seastar/core/deleter.hh> +#include <seastar/core/queue.hh> +#include <seastar/core/stream.hh> +#include <seastar/core/metrics_registration.hh> +#include <seastar/net/toeplitz.hh> +#include <seastar/net/ethernet.hh> +#include <seastar/net/packet.hh> +#include <seastar/net/const.hh> +#include <unordered_map> + +namespace seastar { + +namespace internal { + +class poller; + +} + +namespace net { + +class packet; +class interface; +class device; +class qp; +class l3_protocol; + +class forward_hash { + uint8_t data[64]; + size_t end_idx = 0; +public: + size_t size() const { + return end_idx; + } + void push_back(uint8_t b) { + assert(end_idx < sizeof(data)); + data[end_idx++] = b; + } + void push_back(uint16_t b) { + push_back(uint8_t(b)); + push_back(uint8_t(b >> 8)); + } + void push_back(uint32_t b) { + push_back(uint16_t(b)); + push_back(uint16_t(b >> 16)); + } + const uint8_t& operator[](size_t idx) const { + return data[idx]; + } +}; + +struct hw_features { + // Enable tx ip header checksum offload + bool tx_csum_ip_offload = false; + // Enable tx l4 (TCP or UDP) checksum offload + bool tx_csum_l4_offload = false; + // Enable rx checksum offload + bool rx_csum_offload = false; + // LRO is enabled + bool rx_lro = false; + // Enable tx TCP segment offload + bool tx_tso = false; + // Enable tx UDP fragmentation offload + bool tx_ufo = false; + // Maximum Transmission Unit + uint16_t mtu = 1500; + // Maximun packet len when TCP/UDP offload is enabled + uint16_t max_packet_len = ip_packet_len_max - eth_hdr_len; +}; + +class l3_protocol { +public: + struct l3packet { + eth_protocol_num proto_num; + ethernet_address to; + packet p; + }; + using packet_provider_type = std::function<std::optional<l3packet> ()>; +private: + interface* _netif; + eth_protocol_num _proto_num; +public: + explicit l3_protocol(interface* netif, eth_protocol_num proto_num, packet_provider_type func); + future<> receive( + std::function<future<> (packet, ethernet_address)> rx_fn, + std::function<bool (forward_hash&, packet&, size_t)> forward); +private: + friend class interface; +}; + +class interface { + struct l3_rx_stream { + stream<packet, ethernet_address> packet_stream; + future<> ready; + std::function<bool (forward_hash&, packet&, size_t)> forward; + l3_rx_stream(std::function<bool (forward_hash&, packet&, size_t)>&& fw) : ready(packet_stream.started()), forward(fw) {} + }; + std::unordered_map<uint16_t, l3_rx_stream> _proto_map; + std::shared_ptr<device> _dev; + ethernet_address _hw_address; + net::hw_features _hw_features; + std::vector<l3_protocol::packet_provider_type> _pkt_providers; +private: + future<> dispatch_packet(packet p); +public: + explicit interface(std::shared_ptr<device> dev); + ethernet_address hw_address() const noexcept { return _hw_address; } + const net::hw_features& hw_features() const { return _hw_features; } + future<> register_l3(eth_protocol_num proto_num, + std::function<future<> (packet p, ethernet_address from)> next, + std::function<bool (forward_hash&, packet&, size_t)> forward); + void forward(unsigned cpuid, packet p); + unsigned hash2cpu(uint32_t hash); + void register_packet_provider(l3_protocol::packet_provider_type func) { + _pkt_providers.push_back(std::move(func)); + } + uint16_t hw_queues_count(); + rss_key_type rss_key() const; + friend class l3_protocol; +}; + +struct qp_stats_good { + /** + * Update the packets bunch related statistics. + * + * Update the last packets bunch size and the total packets counter. + * + * @param count Number of packets in the last packets bunch. + */ + void update_pkts_bunch(uint64_t count) { + last_bunch = count; + packets += count; + } + + /** + * Increment the appropriate counters when a few fragments have been + * processed in a copy-way. + * + * @param nr_frags Number of copied fragments + * @param bytes Number of copied bytes + */ + void update_copy_stats(uint64_t nr_frags, uint64_t bytes) { + copy_frags += nr_frags; + copy_bytes += bytes; + } + + /** + * Increment total fragments and bytes statistics + * + * @param nfrags Number of processed fragments + * @param nbytes Number of bytes in the processed fragments + */ + void update_frags_stats(uint64_t nfrags, uint64_t nbytes) { + nr_frags += nfrags; + bytes += nbytes; + } + + uint64_t bytes; // total number of bytes + uint64_t nr_frags; // total number of fragments + uint64_t copy_frags; // fragments that were copied on L2 level + uint64_t copy_bytes; // bytes that were copied on L2 level + uint64_t packets; // total number of packets + uint64_t last_bunch; // number of packets in the last sent/received bunch +}; + +struct qp_stats { + qp_stats() : rx{}, tx{} {} + + struct { + struct qp_stats_good good; + + struct { + void inc_csum_err() { + ++csum; + ++total; + } + + void inc_no_mem() { + ++no_mem; + ++total; + } + + uint64_t no_mem; // Packets dropped due to allocation failure + uint64_t total; // total number of erroneous packets + uint64_t csum; // packets with bad checksum + } bad; + } rx; + + struct { + struct qp_stats_good good; + uint64_t linearized; // number of packets that were linearized + } tx; +}; + +class qp { + using packet_provider_type = std::function<std::optional<packet> ()>; + std::vector<packet_provider_type> _pkt_providers; + std::optional<std::array<uint8_t, 128>> _sw_reta; + circular_buffer<packet> _proxy_packetq; + stream<packet> _rx_stream; + std::unique_ptr<internal::poller> _tx_poller; + circular_buffer<packet> _tx_packetq; + +protected: + const std::string _stats_plugin_name; + const std::string _queue_name; + metrics::metric_groups _metrics; + qp_stats _stats; + +public: + qp(bool register_copy_stats = false, + const std::string stats_plugin_name = std::string("network"), + uint8_t qid = 0); + virtual ~qp(); + virtual future<> send(packet p) = 0; + virtual uint32_t send(circular_buffer<packet>& p) { + uint32_t sent = 0; + while (!p.empty()) { + // FIXME: future is discarded + (void)send(std::move(p.front())); + p.pop_front(); + sent++; + } + return sent; + } + virtual void rx_start() {}; + void configure_proxies(const std::map<unsigned, float>& cpu_weights); + // build REdirection TAble for cpu_weights map: target cpu -> weight + void build_sw_reta(const std::map<unsigned, float>& cpu_weights); + void proxy_send(packet p) { + _proxy_packetq.push_back(std::move(p)); + } + void register_packet_provider(packet_provider_type func) { + _pkt_providers.push_back(std::move(func)); + } + bool poll_tx(); + friend class device; +}; + +class device { +protected: + std::unique_ptr<qp*[]> _queues; + size_t _rss_table_bits = 0; +public: + device() { + _queues = std::make_unique<qp*[]>(smp::count); + } + virtual ~device() {}; + qp& queue_for_cpu(unsigned cpu) { return *_queues[cpu]; } + qp& local_queue() { return queue_for_cpu(this_shard_id()); } + void l2receive(packet p) { + // FIXME: future is discarded + (void)_queues[this_shard_id()]->_rx_stream.produce(std::move(p)); + } + future<> receive(std::function<future<> (packet)> next_packet); + virtual ethernet_address hw_address() = 0; + virtual net::hw_features hw_features() = 0; + virtual rss_key_type rss_key() const { return default_rsskey_40bytes; } + virtual uint16_t hw_queues_count() { return 1; } + virtual future<> link_ready() { return make_ready_future<>(); } + virtual std::unique_ptr<qp> init_local_queue(const program_options::option_group& opts, uint16_t qid) = 0; + virtual unsigned hash2qid(uint32_t hash) { + return hash % hw_queues_count(); + } + void set_local_queue(std::unique_ptr<qp> dev); + template <typename Func> + unsigned forward_dst(unsigned src_cpuid, Func&& hashfn) { + auto& qp = queue_for_cpu(src_cpuid); + if (!qp._sw_reta) { + return src_cpuid; + } + auto hash = hashfn() >> _rss_table_bits; + auto& reta = *qp._sw_reta; + return reta[hash % reta.size()]; + } + virtual unsigned hash2cpu(uint32_t hash) { + // there is an assumption here that qid == cpu_id which will + // not necessary be true in the future + return forward_dst(hash2qid(hash), [hash] { return hash; }); + } +}; + +} + +} diff --git a/src/seastar/include/seastar/net/packet-data-source.hh b/src/seastar/include/seastar/net/packet-data-source.hh new file mode 100644 index 000000000..5c12d5f5c --- /dev/null +++ b/src/seastar/include/seastar/net/packet-data-source.hh @@ -0,0 +1,53 @@ +/* + * 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. + */ +#pragma once + +#include <seastar/net/packet.hh> +#include <seastar/core/iostream.hh> + +namespace seastar { + +namespace net { + +class packet_data_source final : public data_source_impl { + size_t _cur_frag = 0; + packet _p; +public: + explicit packet_data_source(net::packet&& p) + : _p(std::move(p)) + {} + + virtual future<temporary_buffer<char>> get() override { + if (_cur_frag != _p.nr_frags()) { + auto& f = _p.fragments()[_cur_frag++]; + return make_ready_future<temporary_buffer<char>>( + temporary_buffer<char>(f.base, f.size, + make_deleter(deleter(), [p = _p.share()] () mutable {}))); + } + return make_ready_future<temporary_buffer<char>>(temporary_buffer<char>()); + } +}; + +static inline +input_stream<char> as_input_stream(packet&& p) { + return input_stream<char>(data_source(std::make_unique<packet_data_source>(std::move(p)))); +} + +} + +} diff --git a/src/seastar/include/seastar/net/packet-util.hh b/src/seastar/include/seastar/net/packet-util.hh new file mode 100644 index 000000000..3e2718992 --- /dev/null +++ b/src/seastar/include/seastar/net/packet-util.hh @@ -0,0 +1,158 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/net/packet.hh> +#include <map> +#include <iostream> + +namespace seastar { + +namespace net { + +template <typename Offset, typename Tag> +class packet_merger { +private: + static uint64_t& linearizations_ref() { + static thread_local uint64_t linearization_count; + return linearization_count; + } +public: + std::map<Offset, packet> map; + + static uint64_t linearizations() { + return linearizations_ref(); + } + + void merge(Offset offset, packet p) { + bool insert = true; + auto beg = offset; + auto end = beg + p.len(); + // Fisrt, try to merge the packet with existing segment + for (auto it = map.begin(); it != map.end();) { + auto& seg_pkt = it->second; + auto seg_beg = it->first; + auto seg_end = seg_beg + seg_pkt.len(); + // There are 6 cases: + if (seg_beg <= beg && end <= seg_end) { + // 1) seg_beg beg end seg_end + // We already have data in this packet + return; + } else if (beg <= seg_beg && seg_end <= end) { + // 2) beg seg_beg seg_end end + // The new segment contains more data than this old segment + // Delete the old one, insert the new one + it = map.erase(it); + insert = true; + break; + } else if (beg < seg_beg && seg_beg <= end && end <= seg_end) { + // 3) beg seg_beg end seg_end + // Merge two segments, trim front of old segment + auto trim = end - seg_beg; + seg_pkt.trim_front(trim); + p.append(std::move(seg_pkt)); + // Delete the old one, insert the new one + it = map.erase(it); + insert = true; + break; + } else if (seg_beg <= beg && beg <= seg_end && seg_end < end) { + // 4) seg_beg beg seg_end end + // Merge two segments, trim front of new segment + auto trim = seg_end - beg; + p.trim_front(trim); + // Append new data to the old segment, keep the old segment + seg_pkt.append(std::move(p)); + seg_pkt.linearize(); + ++linearizations_ref(); + insert = false; + break; + } else { + // 5) beg end < seg_beg seg_end + // or + // 6) seg_beg seg_end < beg end + // Can not merge with this segment, keep looking + it++; + insert = true; + } + } + + if (insert) { + p.linearize(); + ++linearizations_ref(); + map.emplace(beg, std::move(p)); + } + + // Second, merge adjacent segments after this packet has been merged, + // becasue this packet might fill a "whole" and make two adjacent + // segments mergable + for (auto it = map.begin(); it != map.end();) { + // The first segment + auto& seg_pkt = it->second; + auto seg_beg = it->first; + auto seg_end = seg_beg + seg_pkt.len(); + + // The second segment + auto it_next = it; + it_next++; + if (it_next == map.end()) { + break; + } + auto& p = it_next->second; + auto beg = it_next->first; + auto end = beg + p.len(); + + // Merge the the second segment into first segment if possible + if (seg_beg <= beg && beg <= seg_end && seg_end < end) { + // Merge two segments, trim front of second segment + auto trim = seg_end - beg; + p.trim_front(trim); + // Append new data to the first segment, keep the first segment + seg_pkt.append(std::move(p)); + + // Delete the second segment + map.erase(it_next); + + // Keep merging this first segment with its new next packet + // So we do not update the iterator: it + continue; + } else if (end <= seg_end) { + // The first segment has all the data in the second segment + // Delete the second segment + map.erase(it_next); + continue; + } else if (seg_end < beg) { + // Can not merge first segment with second segment + it = it_next; + continue; + } else { + // If we reach here, we have a bug with merge. + std::cerr << "packet_merger: merge error\n"; + abort(); + break; + } + } + } +}; + +} + +} diff --git a/src/seastar/include/seastar/net/packet.hh b/src/seastar/include/seastar/net/packet.hh new file mode 100644 index 000000000..9049e6865 --- /dev/null +++ b/src/seastar/include/seastar/net/packet.hh @@ -0,0 +1,622 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/core/deleter.hh> +#include <seastar/core/temporary_buffer.hh> +#include <seastar/net/const.hh> +#include <vector> +#include <cassert> +#include <algorithm> +#include <iosfwd> +#include <seastar/util/std-compat.hh> +#include <functional> + +namespace seastar { + +namespace net { + +struct fragment { + char* base; + size_t size; +}; + +struct offload_info { + ip_protocol_num protocol = ip_protocol_num::unused; + bool needs_csum = false; + uint8_t ip_hdr_len = 20; + uint8_t tcp_hdr_len = 20; + uint8_t udp_hdr_len = 8; + bool needs_ip_csum = false; + bool reassembled = false; + uint16_t tso_seg_size = 0; + // HW stripped VLAN header (CPU order) + std::optional<uint16_t> vlan_tci; +}; + +// Zero-copy friendly packet class +// +// For implementing zero-copy, we need a flexible destructor that can +// destroy packet data in different ways: decrementing a reference count, +// or calling a free()-like function. +// +// Moreover, we need different destructors for each set of fragments within +// a single fragment. For example, a header and trailer might need delete[] +// to be called, while the internal data needs a reference count to be +// released. Matters are complicated in that fragments can be split +// (due to virtual/physical translation). +// +// To implement this, we associate each packet with a single destructor, +// but allow composing a packet from another packet plus a fragment to +// be added, with its own destructor, causing the destructors to be chained. +// +// The downside is that the data needed for the destructor is duplicated, +// if it is already available in the fragment itself. +// +// As an optimization, when we allocate small fragments, we allocate some +// extra space, so prepending to the packet does not require extra +// allocations. This is useful when adding headers. +// +class packet final { + // enough for lots of headers, not quite two cache lines: + static constexpr size_t internal_data_size = 128 - 16; + static constexpr size_t default_nr_frags = 4; + + struct pseudo_vector { + fragment* _start; + fragment* _finish; + pseudo_vector(fragment* start, size_t nr) noexcept + : _start(start), _finish(_start + nr) {} + fragment* begin() noexcept { return _start; } + fragment* end() noexcept { return _finish; } + fragment& operator[](size_t idx) noexcept { return _start[idx]; } + }; + + struct impl { + // when destroyed, virtual destructor will reclaim resources + deleter _deleter; + unsigned _len = 0; + uint16_t _nr_frags = 0; + uint16_t _allocated_frags; + offload_info _offload_info; + std::optional<uint32_t> _rss_hash; + char _data[internal_data_size]; // only _frags[0] may use + unsigned _headroom = internal_data_size; // in _data + // FIXME: share _data/_frags space + + fragment _frags[]; + + impl(size_t nr_frags = default_nr_frags) noexcept; + impl(const impl&) = delete; + impl(fragment frag, size_t nr_frags = default_nr_frags); + + pseudo_vector fragments() noexcept { return { _frags, _nr_frags }; } + + static std::unique_ptr<impl> allocate(size_t nr_frags) { + nr_frags = std::max(nr_frags, default_nr_frags); + return std::unique_ptr<impl>(new (nr_frags) impl(nr_frags)); + } + + static std::unique_ptr<impl> copy(impl* old, size_t nr) { + auto n = allocate(nr); + n->_deleter = std::move(old->_deleter); + n->_len = old->_len; + n->_nr_frags = old->_nr_frags; + n->_headroom = old->_headroom; + n->_offload_info = old->_offload_info; + n->_rss_hash = old->_rss_hash; + std::copy(old->_frags, old->_frags + old->_nr_frags, n->_frags); + old->copy_internal_fragment_to(n.get()); + return n; + } + + static std::unique_ptr<impl> copy(impl* old) { + return copy(old, old->_nr_frags); + } + + static std::unique_ptr<impl> allocate_if_needed(std::unique_ptr<impl> old, size_t extra_frags) { + if (old->_allocated_frags >= old->_nr_frags + extra_frags) { + return old; + } + return copy(old.get(), std::max<size_t>(old->_nr_frags + extra_frags, 2 * old->_nr_frags)); + } + void* operator new(size_t size, size_t nr_frags = default_nr_frags) { + assert(nr_frags == uint16_t(nr_frags)); + return ::operator new(size + nr_frags * sizeof(fragment)); + } + // Matching the operator new above + void operator delete(void* ptr, size_t) { + return ::operator delete(ptr); + } + // Since the above "placement delete" hides the global one, expose it + void operator delete(void* ptr) { + return ::operator delete(ptr); + } + + bool using_internal_data() const noexcept { + return _nr_frags + && _frags[0].base >= _data + && _frags[0].base < _data + internal_data_size; + } + + void unuse_internal_data() { + if (!using_internal_data()) { + return; + } + auto buf = static_cast<char*>(::malloc(_frags[0].size)); + if (!buf) { + throw std::bad_alloc(); + } + deleter d = make_free_deleter(buf); + std::copy(_frags[0].base, _frags[0].base + _frags[0].size, buf); + _frags[0].base = buf; + d.append(std::move(_deleter)); + _deleter = std::move(d); + _headroom = internal_data_size; + } + void copy_internal_fragment_to(impl* to) noexcept { + if (!using_internal_data()) { + return; + } + to->_frags[0].base = to->_data + _headroom; + std::copy(_frags[0].base, _frags[0].base + _frags[0].size, + to->_frags[0].base); + } + }; + packet(std::unique_ptr<impl>&& impl) noexcept : _impl(std::move(impl)) {} + std::unique_ptr<impl> _impl; +public: + static packet from_static_data(const char* data, size_t len) noexcept { + return {fragment{const_cast<char*>(data), len}, deleter()}; + } + + // build empty packet + packet(); + // build empty packet with nr_frags allocated + packet(size_t nr_frags); + // move existing packet + packet(packet&& x) noexcept; + // copy data into packet + packet(const char* data, size_t len); + // copy data into packet + packet(fragment frag); + // zero-copy single fragment + packet(fragment frag, deleter del); + // zero-copy multiple fragments + packet(std::vector<fragment> frag, deleter del); + // build packet with iterator + template <typename Iterator> + packet(Iterator begin, Iterator end, deleter del); + // append fragment (copying new fragment) + packet(packet&& x, fragment frag); + // prepend fragment (copying new fragment, with header optimization) + packet(fragment frag, packet&& x); + // prepend fragment (zero-copy) + packet(fragment frag, deleter del, packet&& x); + // append fragment (zero-copy) + packet(packet&& x, fragment frag, deleter d); + // append temporary_buffer (zero-copy) + packet(packet&& x, temporary_buffer<char> buf); + // create from temporary_buffer (zero-copy) + packet(temporary_buffer<char> buf); + // append deleter + packet(packet&& x, deleter d); + + packet& operator=(packet&& x) noexcept { + if (this != &x) { + this->~packet(); + new (this) packet(std::move(x)); + } + return *this; + } + + unsigned len() const noexcept { return _impl->_len; } + unsigned memory() const noexcept { return len() + sizeof(packet::impl); } + + fragment frag(unsigned idx) const noexcept { return _impl->_frags[idx]; } + fragment& frag(unsigned idx) noexcept { return _impl->_frags[idx]; } + + unsigned nr_frags() const noexcept { return _impl->_nr_frags; } + pseudo_vector fragments() const noexcept { return { _impl->_frags, _impl->_nr_frags }; } + fragment* fragment_array() const noexcept { return _impl->_frags; } + + // share packet data (reference counted, non COW) + packet share(); + packet share(size_t offset, size_t len); + + void append(packet&& p); + + void trim_front(size_t how_much) noexcept; + void trim_back(size_t how_much) noexcept; + + // get a header pointer, linearizing if necessary + template <typename Header> + Header* get_header(size_t offset = 0); + + // get a header pointer, linearizing if necessary + char* get_header(size_t offset, size_t size); + + // prepend a header (default-initializing it) + template <typename Header> + Header* prepend_header(size_t extra_size = 0); + + // prepend a header (uninitialized!) + char* prepend_uninitialized_header(size_t size); + + packet free_on_cpu(unsigned cpu, std::function<void()> cb = []{}); + + void linearize() { return linearize(0, len()); } + + void reset() noexcept { _impl.reset(); } + + void reserve(int n_frags) { + if (n_frags > _impl->_nr_frags) { + auto extra = n_frags - _impl->_nr_frags; + _impl = impl::allocate_if_needed(std::move(_impl), extra); + } + } + std::optional<uint32_t> rss_hash() const noexcept { + return _impl->_rss_hash; + } + std::optional<uint32_t> set_rss_hash(uint32_t hash) noexcept { + return _impl->_rss_hash = hash; + } + // Call `func` for each fragment, avoiding data copies when possible + // `func` is called with a temporary_buffer<char> parameter + template <typename Func> + void release_into(Func&& func) { + unsigned idx = 0; + if (_impl->using_internal_data()) { + auto&& f = frag(idx++); + func(temporary_buffer<char>(f.base, f.size)); + } + while (idx < nr_frags()) { + auto&& f = frag(idx++); + func(temporary_buffer<char>(f.base, f.size, _impl->_deleter.share())); + } + } + std::vector<temporary_buffer<char>> release() { + std::vector<temporary_buffer<char>> ret; + ret.reserve(_impl->_nr_frags); + release_into([&ret] (temporary_buffer<char>&& frag) { + ret.push_back(std::move(frag)); + }); + return ret; + } + explicit operator bool() noexcept { + return bool(_impl); + } + static packet make_null_packet() noexcept { + return net::packet(nullptr); + } +private: + void linearize(size_t at_frag, size_t desired_size); + bool allocate_headroom(size_t size); +public: + struct offload_info offload_info() const noexcept { return _impl->_offload_info; } + struct offload_info& offload_info_ref() noexcept { return _impl->_offload_info; } + void set_offload_info(struct offload_info oi) noexcept { _impl->_offload_info = oi; } +}; + +std::ostream& operator<<(std::ostream& os, const packet& p); + +inline +packet::packet(packet&& x) noexcept + : _impl(std::move(x._impl)) { +} + +inline +packet::impl::impl(size_t nr_frags) noexcept + : _len(0), _allocated_frags(nr_frags) { +} + +inline +packet::impl::impl(fragment frag, size_t nr_frags) + : _len(frag.size), _allocated_frags(nr_frags) { + assert(_allocated_frags > _nr_frags); + if (frag.size <= internal_data_size) { + _headroom -= frag.size; + _frags[0] = { _data + _headroom, frag.size }; + } else { + auto buf = static_cast<char*>(::malloc(frag.size)); + if (!buf) { + throw std::bad_alloc(); + } + deleter d = make_free_deleter(buf); + _frags[0] = { buf, frag.size }; + _deleter.append(std::move(d)); + } + std::copy(frag.base, frag.base + frag.size, _frags[0].base); + ++_nr_frags; +} + +inline +packet::packet() + : _impl(impl::allocate(1)) { +} + +inline +packet::packet(size_t nr_frags) + : _impl(impl::allocate(nr_frags)) { +} + +inline +packet::packet(fragment frag) : _impl(new impl(frag)) { +} + +inline +packet::packet(const char* data, size_t size) : packet(fragment{const_cast<char*>(data), size}) { +} + +inline +packet::packet(fragment frag, deleter d) + : _impl(impl::allocate(1)) { + _impl->_deleter = std::move(d); + _impl->_frags[_impl->_nr_frags++] = frag; + _impl->_len = frag.size; +} + +inline +packet::packet(std::vector<fragment> frag, deleter d) + : _impl(impl::allocate(frag.size())) { + _impl->_deleter = std::move(d); + std::copy(frag.begin(), frag.end(), _impl->_frags); + _impl->_nr_frags = frag.size(); + _impl->_len = 0; + for (auto&& f : _impl->fragments()) { + _impl->_len += f.size; + } +} + +template <typename Iterator> +inline +packet::packet(Iterator begin, Iterator end, deleter del) { + unsigned nr_frags = 0, len = 0; + nr_frags = std::distance(begin, end); + std::for_each(begin, end, [&] (const fragment& frag) { len += frag.size; }); + _impl = impl::allocate(nr_frags); + _impl->_deleter = std::move(del); + _impl->_len = len; + _impl->_nr_frags = nr_frags; + std::copy(begin, end, _impl->_frags); +} + +inline +packet::packet(packet&& x, fragment frag) + : _impl(impl::allocate_if_needed(std::move(x._impl), 1)) { + _impl->_len += frag.size; + std::unique_ptr<char[]> buf(new char[frag.size]); + std::copy(frag.base, frag.base + frag.size, buf.get()); + _impl->_frags[_impl->_nr_frags++] = {buf.get(), frag.size}; + _impl->_deleter = make_deleter(std::move(_impl->_deleter), [buf = buf.release()] { + delete[] buf; + }); +} + +inline +bool +packet::allocate_headroom(size_t size) { + if (_impl->_headroom >= size) { + _impl->_len += size; + if (!_impl->using_internal_data()) { + _impl = impl::allocate_if_needed(std::move(_impl), 1); + std::copy_backward(_impl->_frags, _impl->_frags + _impl->_nr_frags, + _impl->_frags + _impl->_nr_frags + 1); + _impl->_frags[0] = { _impl->_data + internal_data_size, 0 }; + ++_impl->_nr_frags; + } + _impl->_headroom -= size; + _impl->_frags[0].base -= size; + _impl->_frags[0].size += size; + return true; + } else { + return false; + } +} + + +inline +packet::packet(fragment frag, packet&& x) + : _impl(std::move(x._impl)) { + // try to prepend into existing internal fragment + if (allocate_headroom(frag.size)) { + std::copy(frag.base, frag.base + frag.size, _impl->_frags[0].base); + return; + } else { + // didn't work out, allocate and copy + _impl->unuse_internal_data(); + _impl = impl::allocate_if_needed(std::move(_impl), 1); + _impl->_len += frag.size; + std::unique_ptr<char[]> buf(new char[frag.size]); + std::copy(frag.base, frag.base + frag.size, buf.get()); + std::copy_backward(_impl->_frags, _impl->_frags + _impl->_nr_frags, + _impl->_frags + _impl->_nr_frags + 1); + ++_impl->_nr_frags; + _impl->_frags[0] = {buf.get(), frag.size}; + _impl->_deleter = make_deleter(std::move(_impl->_deleter), + [buf = std::move(buf)] {}); + } +} + +inline +packet::packet(packet&& x, fragment frag, deleter d) + : _impl(impl::allocate_if_needed(std::move(x._impl), 1)) { + _impl->_len += frag.size; + _impl->_frags[_impl->_nr_frags++] = frag; + d.append(std::move(_impl->_deleter)); + _impl->_deleter = std::move(d); +} + +inline +packet::packet(packet&& x, deleter d) + : _impl(std::move(x._impl)) { + _impl->_deleter.append(std::move(d)); +} + +inline +packet::packet(packet&& x, temporary_buffer<char> buf) + : packet(std::move(x), fragment{buf.get_write(), buf.size()}, buf.release()) { +} + +inline +packet::packet(temporary_buffer<char> buf) + : packet(fragment{buf.get_write(), buf.size()}, buf.release()) {} + +inline +void packet::append(packet&& p) { + if (!_impl->_len) { + *this = std::move(p); + return; + } + _impl = impl::allocate_if_needed(std::move(_impl), p._impl->_nr_frags); + _impl->_len += p._impl->_len; + p._impl->unuse_internal_data(); + std::copy(p._impl->_frags, p._impl->_frags + p._impl->_nr_frags, + _impl->_frags + _impl->_nr_frags); + _impl->_nr_frags += p._impl->_nr_frags; + p._impl->_deleter.append(std::move(_impl->_deleter)); + _impl->_deleter = std::move(p._impl->_deleter); +} + +inline +char* packet::get_header(size_t offset, size_t size) { + if (offset + size > _impl->_len) { + return nullptr; + } + size_t i = 0; + while (i != _impl->_nr_frags && offset >= _impl->_frags[i].size) { + offset -= _impl->_frags[i++].size; + } + if (i == _impl->_nr_frags) { + return nullptr; + } + if (offset + size > _impl->_frags[i].size) { + linearize(i, offset + size); + } + return _impl->_frags[i].base + offset; +} + +template <typename Header> +inline +Header* packet::get_header(size_t offset) { + return reinterpret_cast<Header*>(get_header(offset, sizeof(Header))); +} + +inline +void packet::trim_front(size_t how_much) noexcept { + assert(how_much <= _impl->_len); + _impl->_len -= how_much; + size_t i = 0; + while (how_much && how_much >= _impl->_frags[i].size) { + how_much -= _impl->_frags[i++].size; + } + std::copy(_impl->_frags + i, _impl->_frags + _impl->_nr_frags, _impl->_frags); + _impl->_nr_frags -= i; + if (!_impl->using_internal_data()) { + _impl->_headroom = internal_data_size; + } + if (how_much) { + if (_impl->using_internal_data()) { + _impl->_headroom += how_much; + } + _impl->_frags[0].base += how_much; + _impl->_frags[0].size -= how_much; + } +} + +inline +void packet::trim_back(size_t how_much) noexcept { + assert(how_much <= _impl->_len); + _impl->_len -= how_much; + size_t i = _impl->_nr_frags - 1; + while (how_much && how_much >= _impl->_frags[i].size) { + how_much -= _impl->_frags[i--].size; + } + _impl->_nr_frags = i + 1; + if (how_much) { + _impl->_frags[i].size -= how_much; + if (i == 0 && _impl->using_internal_data()) { + _impl->_headroom += how_much; + } + } +} + +template <typename Header> +Header* +packet::prepend_header(size_t extra_size) { + auto h = prepend_uninitialized_header(sizeof(Header) + extra_size); + return new (h) Header{}; +} + +// prepend a header (uninitialized!) +inline +char* packet::prepend_uninitialized_header(size_t size) { + if (!allocate_headroom(size)) { + // didn't work out, allocate and copy + _impl->unuse_internal_data(); + // try again, after unuse_internal_data we may have space after all + if (!allocate_headroom(size)) { + // failed + _impl->_len += size; + _impl = impl::allocate_if_needed(std::move(_impl), 1); + std::unique_ptr<char[]> buf(new char[size]); + std::copy_backward(_impl->_frags, _impl->_frags + _impl->_nr_frags, + _impl->_frags + _impl->_nr_frags + 1); + ++_impl->_nr_frags; + _impl->_frags[0] = {buf.get(), size}; + _impl->_deleter = make_deleter(std::move(_impl->_deleter), + [buf = std::move(buf)] {}); + } + } + return _impl->_frags[0].base; +} + +inline +packet packet::share() { + return share(0, _impl->_len); +} + +inline +packet packet::share(size_t offset, size_t len) { + _impl->unuse_internal_data(); // FIXME: eliminate? + packet n; + n._impl = impl::allocate_if_needed(std::move(n._impl), _impl->_nr_frags); + size_t idx = 0; + while (offset > 0 && offset >= _impl->_frags[idx].size) { + offset -= _impl->_frags[idx++].size; + } + while (n._impl->_len < len) { + auto& f = _impl->_frags[idx++]; + auto fsize = std::min(len - n._impl->_len, f.size - offset); + n._impl->_frags[n._impl->_nr_frags++] = { f.base + offset, fsize }; + n._impl->_len += fsize; + offset = 0; + } + n._impl->_offload_info = _impl->_offload_info; + assert(!n._impl->_deleter); + n._impl->_deleter = _impl->_deleter.share(); + return n; +} + +} + +} diff --git a/src/seastar/include/seastar/net/posix-stack.hh b/src/seastar/include/seastar/net/posix-stack.hh new file mode 100644 index 000000000..e76d7affa --- /dev/null +++ b/src/seastar/include/seastar/net/posix-stack.hh @@ -0,0 +1,223 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/core/sharded.hh> +#include <seastar/core/internal/pollable_fd.hh> +#include <seastar/net/stack.hh> +#include <seastar/core/polymorphic_temporary_buffer.hh> +#include <seastar/core/internal/buffer_allocator.hh> +#include <seastar/util/program-options.hh> + +namespace seastar { + +namespace net { + +using namespace seastar; + +// We can't keep this in any of the socket servers as instance members, because a connection can +// outlive the socket server. To avoid having the whole socket_server tracked as a shared pointer, +// we will have a conntrack structure. +// +// Right now this class is used by the posix_server_socket_impl, but it could be used by any other. +class conntrack { + class load_balancer { + std::vector<unsigned> _cpu_load; + public: + load_balancer() : _cpu_load(size_t(smp::count), 0) {} + void closed_cpu(shard_id cpu) { + _cpu_load[cpu]--; + } + shard_id next_cpu() { + // FIXME: The naive algorithm will just round robin the connections around the shards. + // A more complex version can keep track of the amount of activity in each connection, + // and use that information. + auto min_el = std::min_element(_cpu_load.begin(), _cpu_load.end()); + auto cpu = shard_id(std::distance(_cpu_load.begin(), min_el)); + _cpu_load[cpu]++; + return cpu; + } + shard_id force_cpu(shard_id cpu) { + _cpu_load[cpu]++; + return cpu; + } + }; + + lw_shared_ptr<load_balancer> _lb; + void closed_cpu(shard_id cpu) { + _lb->closed_cpu(cpu); + } +public: + class handle { + shard_id _host_cpu; + shard_id _target_cpu; + foreign_ptr<lw_shared_ptr<load_balancer>> _lb; + public: + handle() : _lb(nullptr) {} + handle(shard_id cpu, lw_shared_ptr<load_balancer> lb) + : _host_cpu(this_shard_id()) + , _target_cpu(cpu) + , _lb(make_foreign(std::move(lb))) {} + + handle(const handle&) = delete; + handle(handle&&) = default; + ~handle() { + if (!_lb) { + return; + } + // FIXME: future is discarded + (void)smp::submit_to(_host_cpu, [cpu = _target_cpu, lb = std::move(_lb)] { + lb->closed_cpu(cpu); + }); + } + shard_id cpu() { + return _target_cpu; + } + }; + friend class handle; + + conntrack() : _lb(make_lw_shared<load_balancer>()) {} + handle get_handle() { + return handle(_lb->next_cpu(), _lb); + } + handle get_handle(shard_id cpu) { + return handle(_lb->force_cpu(cpu), _lb); + } +}; + +class posix_data_source_impl final : public data_source_impl, private internal::buffer_allocator { + std::pmr::polymorphic_allocator<char>* _buffer_allocator; + pollable_fd _fd; + connected_socket_input_stream_config _config; +private: + virtual temporary_buffer<char> allocate_buffer() override; +public: + explicit posix_data_source_impl(pollable_fd fd, connected_socket_input_stream_config config, + std::pmr::polymorphic_allocator<char>* allocator=memory::malloc_allocator) + : _buffer_allocator(allocator), _fd(std::move(fd)), _config(config) { + } + future<temporary_buffer<char>> get() override; + future<> close() override; +}; + +class posix_data_sink_impl : public data_sink_impl { + pollable_fd _fd; + packet _p; +public: + explicit posix_data_sink_impl(pollable_fd fd) : _fd(std::move(fd)) {} + using data_sink_impl::put; + future<> put(packet p) override; + future<> put(temporary_buffer<char> buf) override; + future<> close() override; +}; + +class posix_ap_server_socket_impl : public server_socket_impl { + using protocol_and_socket_address = std::tuple<int, socket_address>; + struct connection { + pollable_fd fd; + socket_address addr; + conntrack::handle connection_tracking_handle; + connection(pollable_fd xfd, socket_address xaddr, conntrack::handle cth) : fd(std::move(xfd)), addr(xaddr), connection_tracking_handle(std::move(cth)) {} + }; + using sockets_map_t = std::unordered_map<protocol_and_socket_address, promise<accept_result>>; + using conn_map_t = std::unordered_multimap<protocol_and_socket_address, connection>; + static thread_local sockets_map_t sockets; + static thread_local conn_map_t conn_q; + int _protocol; + socket_address _sa; + std::pmr::polymorphic_allocator<char>* _allocator; +public: + explicit posix_ap_server_socket_impl(int protocol, socket_address sa, std::pmr::polymorphic_allocator<char>* allocator = memory::malloc_allocator) : _protocol(protocol), _sa(sa), _allocator(allocator) {} + virtual future<accept_result> accept() override; + virtual void abort_accept() override; + socket_address local_address() const override { + return _sa; + } + static void move_connected_socket(int protocol, socket_address sa, pollable_fd fd, socket_address addr, conntrack::handle handle, std::pmr::polymorphic_allocator<char>* allocator); + + template <typename T> + friend class std::hash; +}; + +class posix_server_socket_impl : public server_socket_impl { + socket_address _sa; + int _protocol; + pollable_fd _lfd; + conntrack _conntrack; + server_socket::load_balancing_algorithm _lba; + shard_id _fixed_cpu; + std::pmr::polymorphic_allocator<char>* _allocator; +public: + explicit posix_server_socket_impl(int protocol, socket_address sa, pollable_fd lfd, + server_socket::load_balancing_algorithm lba, shard_id fixed_cpu, + std::pmr::polymorphic_allocator<char>* allocator=memory::malloc_allocator) : _sa(sa), _protocol(protocol), _lfd(std::move(lfd)), _lba(lba), _fixed_cpu(fixed_cpu), _allocator(allocator) {} + virtual future<accept_result> accept() override; + virtual void abort_accept() override; + virtual socket_address local_address() const override; +}; + +class posix_reuseport_server_socket_impl : public server_socket_impl { + socket_address _sa; + int _protocol; + pollable_fd _lfd; + std::pmr::polymorphic_allocator<char>* _allocator; +public: + explicit posix_reuseport_server_socket_impl(int protocol, socket_address sa, pollable_fd lfd, + std::pmr::polymorphic_allocator<char>* allocator=memory::malloc_allocator) : _sa(sa), _protocol(protocol), _lfd(std::move(lfd)), _allocator(allocator) {} + virtual future<accept_result> accept() override; + virtual void abort_accept() override; + virtual socket_address local_address() const override; +}; + +class posix_network_stack : public network_stack { +private: + const bool _reuseport; +protected: + std::pmr::polymorphic_allocator<char>* _allocator; +public: + explicit posix_network_stack(const program_options::option_group& opts, std::pmr::polymorphic_allocator<char>* allocator=memory::malloc_allocator); + virtual server_socket listen(socket_address sa, listen_options opts) override; + virtual ::seastar::socket socket() override; + virtual net::udp_channel make_udp_channel(const socket_address&) override; + static future<std::unique_ptr<network_stack>> create(const program_options::option_group& opts, std::pmr::polymorphic_allocator<char>* allocator=memory::malloc_allocator) { + return make_ready_future<std::unique_ptr<network_stack>>(std::unique_ptr<network_stack>(new posix_network_stack(opts, allocator))); + } + virtual bool has_per_core_namespace() override { return _reuseport; }; + bool supports_ipv6() const override; + std::vector<network_interface> network_interfaces() override; +}; + +class posix_ap_network_stack : public posix_network_stack { +private: + const bool _reuseport; +public: + posix_ap_network_stack(const program_options::option_group& opts, std::pmr::polymorphic_allocator<char>* allocator=memory::malloc_allocator); + virtual server_socket listen(socket_address sa, listen_options opts) override; + static future<std::unique_ptr<network_stack>> create(const program_options::option_group& opts, std::pmr::polymorphic_allocator<char>* allocator=memory::malloc_allocator) { + return make_ready_future<std::unique_ptr<network_stack>>(std::unique_ptr<network_stack>(new posix_ap_network_stack(opts, allocator))); + } +}; + +network_stack_entry register_posix_stack(); +} + +} diff --git a/src/seastar/include/seastar/net/proxy.hh b/src/seastar/include/seastar/net/proxy.hh new file mode 100644 index 000000000..cd2632424 --- /dev/null +++ b/src/seastar/include/seastar/net/proxy.hh @@ -0,0 +1,32 @@ +/* + * 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. + */ +#pragma once + +#include <memory> +#include <seastar/net/net.hh> +#include <seastar/net/packet.hh> + +namespace seastar { + +namespace net { + +std::unique_ptr<qp> create_proxy_net_device(unsigned master_cpu, device* dev); + +} + +} diff --git a/src/seastar/include/seastar/net/socket_defs.hh b/src/seastar/include/seastar/net/socket_defs.hh new file mode 100644 index 000000000..698a4b685 --- /dev/null +++ b/src/seastar/include/seastar/net/socket_defs.hh @@ -0,0 +1,192 @@ +/* + * 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) 2016 ScyllaDB. + */ +#pragma once + +#include <iosfwd> +#include <array> +#include <sys/socket.h> +#include <sys/un.h> +#include <netinet/ip.h> +#include <seastar/net/byteorder.hh> +#include <seastar/net/unix_address.hh> +#include <cassert> + +namespace seastar { + +namespace net { +class inet_address; +} + +struct ipv4_addr; +struct ipv6_addr; + +class socket_address { +public: + socklen_t addr_length; ///!< actual size of the relevant 'u' member + union { + ::sockaddr_storage sas; + ::sockaddr sa; + ::sockaddr_in in; + ::sockaddr_in6 in6; + ::sockaddr_un un; + } u; + socket_address(const sockaddr_in& sa) noexcept : addr_length{sizeof(::sockaddr_in)} { + u.in = sa; + } + socket_address(const sockaddr_in6& sa) noexcept : addr_length{sizeof(::sockaddr_in6)} { + u.in6 = sa; + } + socket_address(uint16_t) noexcept; + socket_address(ipv4_addr) noexcept; + socket_address(const ipv6_addr&) noexcept; + socket_address(const ipv6_addr&, uint32_t scope) noexcept; + socket_address(const net::inet_address&, uint16_t p = 0) noexcept; + explicit socket_address(const unix_domain_addr&) noexcept; + /** creates an uninitialized socket_address. this can be written into, or used as + * "unspecified" for such addresses as bind(addr) or local address in socket::connect + * (i.e. system picks) + */ + socket_address() noexcept; + + ::sockaddr& as_posix_sockaddr() noexcept { return u.sa; } + ::sockaddr_in& as_posix_sockaddr_in() noexcept { return u.in; } + ::sockaddr_in6& as_posix_sockaddr_in6() noexcept { return u.in6; } + const ::sockaddr& as_posix_sockaddr() const noexcept { return u.sa; } + const ::sockaddr_in& as_posix_sockaddr_in() const noexcept { return u.in; } + const ::sockaddr_in6& as_posix_sockaddr_in6() const noexcept { return u.in6; } + + socket_address(uint32_t, uint16_t p = 0) noexcept; + + socklen_t length() const noexcept { return addr_length; }; + + bool is_af_unix() const noexcept { + return u.sa.sa_family == AF_UNIX; + } + + bool is_unspecified() const noexcept; + + sa_family_t family() const noexcept { + return u.sa.sa_family; + } + + net::inet_address addr() const noexcept; + ::in_port_t port() const noexcept; + bool is_wildcard() const noexcept; + + bool operator==(const socket_address&) const noexcept; + bool operator!=(const socket_address& a) const noexcept { + return !(*this == a); + } +}; + +std::ostream& operator<<(std::ostream&, const socket_address&); + +enum class transport { + TCP = IPPROTO_TCP, + SCTP = IPPROTO_SCTP +}; + +struct ipv4_addr { + uint32_t ip; + uint16_t port; + + ipv4_addr() noexcept : ip(0), port(0) {} + ipv4_addr(uint32_t ip, uint16_t port) noexcept : ip(ip), port(port) {} + ipv4_addr(uint16_t port) noexcept : ip(0), port(port) {} + // throws if not a valid ipv4 addr + ipv4_addr(const std::string &addr); + ipv4_addr(const std::string &addr, uint16_t port); + // throws if not an ipv4 addr + ipv4_addr(const net::inet_address&, uint16_t); + ipv4_addr(const socket_address &) noexcept; + ipv4_addr(const ::in_addr&, uint16_t = 0) noexcept; + + bool is_ip_unspecified() const noexcept { + return ip == 0; + } + bool is_port_unspecified() const noexcept { + return port == 0; + } +}; + +struct ipv6_addr { + using ipv6_bytes = std::array<uint8_t, 16>; + + ipv6_bytes ip; + uint16_t port; + + ipv6_addr(const ipv6_bytes&, uint16_t port = 0) noexcept; + ipv6_addr(uint16_t port = 0) noexcept; + // throws if not a valid ipv6 addr + ipv6_addr(const std::string&); + ipv6_addr(const std::string&, uint16_t port); + ipv6_addr(const net::inet_address&, uint16_t = 0) noexcept; + ipv6_addr(const ::in6_addr&, uint16_t = 0) noexcept; + ipv6_addr(const ::sockaddr_in6&) noexcept; + ipv6_addr(const socket_address&) noexcept; + + bool is_ip_unspecified() const noexcept; + bool is_port_unspecified() const noexcept { + return port == 0; + } +}; + +std::ostream& operator<<(std::ostream&, const ipv4_addr&); +std::ostream& operator<<(std::ostream&, const ipv6_addr&); + +inline bool operator==(const ipv4_addr &lhs, const ipv4_addr& rhs) noexcept { + return lhs.ip == rhs.ip && lhs.port == rhs.port; +} + +} + +namespace std { +template<> +struct hash<seastar::socket_address> { + size_t operator()(const seastar::socket_address&) const; +}; +template<> +struct hash<seastar::ipv4_addr> { + size_t operator()(const seastar::ipv4_addr&) const; +}; +template<> +struct hash<seastar::unix_domain_addr> { + size_t operator()(const seastar::unix_domain_addr&) const; +}; +template<> +struct hash<::sockaddr_un> { + size_t operator()(const ::sockaddr_un&) const; +}; + +template <> +struct hash<seastar::transport> { + size_t operator()(seastar::transport tr) const { + return static_cast<size_t>(tr); + } +}; + +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<seastar::socket_address> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<seastar::ipv4_addr> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<seastar::ipv6_addr> : fmt::ostream_formatter {}; +#endif diff --git a/src/seastar/include/seastar/net/stack.hh b/src/seastar/include/seastar/net/stack.hh new file mode 100644 index 000000000..27118cd4e --- /dev/null +++ b/src/seastar/include/seastar/net/stack.hh @@ -0,0 +1,105 @@ +/* + * 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 2015 Cloudius Systems + */ +#pragma once + +#include <chrono> +#include <seastar/net/api.hh> +#include <seastar/core/internal/api-level.hh> +#include <seastar/core/memory.hh> + +namespace seastar { + +namespace net { + +/// \cond internal +class connected_socket_impl { +public: + virtual ~connected_socket_impl() {} + virtual data_source source() = 0; + virtual data_source source(connected_socket_input_stream_config csisc); + virtual data_sink sink() = 0; + virtual void shutdown_input() = 0; + virtual void shutdown_output() = 0; + virtual void set_nodelay(bool nodelay) = 0; + virtual bool get_nodelay() const = 0; + virtual void set_keepalive(bool keepalive) = 0; + virtual bool get_keepalive() const = 0; + virtual void set_keepalive_parameters(const keepalive_params&) = 0; + virtual keepalive_params get_keepalive_parameters() const = 0; + virtual void set_sockopt(int level, int optname, const void* data, size_t len) = 0; + virtual int get_sockopt(int level, int optname, void* data, size_t len) const = 0; + virtual socket_address local_address() const noexcept = 0; + virtual future<> wait_input_shutdown() = 0; +}; + +class socket_impl { +public: + virtual ~socket_impl() {} + virtual future<connected_socket> connect(socket_address sa, socket_address local, transport proto = transport::TCP) = 0; + virtual void set_reuseaddr(bool reuseaddr) = 0; + virtual bool get_reuseaddr() const = 0; + virtual void shutdown() = 0; +}; + + +class server_socket_impl { +public: + virtual ~server_socket_impl() {} + virtual future<accept_result> accept() = 0; + virtual void abort_accept() = 0; + virtual socket_address local_address() const = 0; +}; + +class udp_channel_impl { +public: + virtual ~udp_channel_impl() {} + virtual socket_address local_address() const = 0; + virtual future<udp_datagram> receive() = 0; + virtual future<> send(const socket_address& dst, const char* msg) = 0; + virtual future<> send(const socket_address& dst, packet p) = 0; + virtual void shutdown_input() = 0; + virtual void shutdown_output() = 0; + virtual bool is_closed() const = 0; + virtual void close() = 0; +}; + +class network_interface_impl { +public: + virtual ~network_interface_impl() {} + virtual uint32_t index() const = 0; + virtual uint32_t mtu() const = 0; + + virtual const sstring& name() const = 0; + virtual const sstring& display_name() const = 0; + virtual const std::vector<net::inet_address>& addresses() const = 0; + virtual const std::vector<uint8_t> hardware_address() const = 0; + + virtual bool is_loopback() const = 0; + virtual bool is_virtual() const = 0; + virtual bool is_up() const = 0; + virtual bool supports_ipv6() const = 0; +}; + +/// \endcond + +} + +} diff --git a/src/seastar/include/seastar/net/tcp-stack.hh b/src/seastar/include/seastar/net/tcp-stack.hh new file mode 100644 index 000000000..83dd80e8c --- /dev/null +++ b/src/seastar/include/seastar/net/tcp-stack.hh @@ -0,0 +1,50 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +// tcp/network-stack integration + +#pragma once + +#include <seastar/core/future.hh> +#include <seastar/core/internal/api-level.hh> + +namespace seastar { + +struct listen_options; + +class server_socket; +class connected_socket; + +namespace net { + +struct ipv4_traits; +template <typename InetTraits> +class tcp; + +server_socket +tcpv4_listen(tcp<ipv4_traits>& tcpv4, uint16_t port, listen_options opts); + +seastar::socket +tcpv4_socket(tcp<ipv4_traits>& tcpv4); + +} + +} diff --git a/src/seastar/include/seastar/net/tcp.hh b/src/seastar/include/seastar/net/tcp.hh new file mode 100644 index 000000000..5fc4ddc80 --- /dev/null +++ b/src/seastar/include/seastar/net/tcp.hh @@ -0,0 +1,2163 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/core/shared_ptr.hh> +#include <seastar/core/queue.hh> +#include <seastar/core/semaphore.hh> +#include <seastar/core/byteorder.hh> +#include <seastar/core/metrics.hh> +#include <seastar/net/net.hh> +#include <seastar/net/ip_checksum.hh> +#include <seastar/net/ip.hh> +#include <seastar/net/const.hh> +#include <seastar/net/packet-util.hh> +#include <seastar/util/std-compat.hh> +#include <unordered_map> +#include <map> +#include <functional> +#include <deque> +#include <chrono> +#include <random> +#include <stdexcept> +#include <system_error> + +#define CRYPTOPP_ENABLE_NAMESPACE_WEAK 1 +#include <cryptopp/md5.h> + +namespace seastar { + +using namespace std::chrono_literals; + +namespace net { + +struct tcp_hdr; + +inline auto tcp_error(int err) { + return std::system_error(err, std::system_category()); +} + +inline auto tcp_reset_error() { + return tcp_error(ECONNRESET); +}; + +inline auto tcp_connect_error() { + return tcp_error(ECONNABORTED); +} + +inline auto tcp_refused_error() { + return tcp_error(ECONNREFUSED); +}; + +enum class tcp_state : uint16_t { + CLOSED = (1 << 0), + LISTEN = (1 << 1), + SYN_SENT = (1 << 2), + SYN_RECEIVED = (1 << 3), + ESTABLISHED = (1 << 4), + FIN_WAIT_1 = (1 << 5), + FIN_WAIT_2 = (1 << 6), + CLOSE_WAIT = (1 << 7), + CLOSING = (1 << 8), + LAST_ACK = (1 << 9), + TIME_WAIT = (1 << 10) +}; + +inline tcp_state operator|(tcp_state s1, tcp_state s2) { + return tcp_state(uint16_t(s1) | uint16_t(s2)); +} + +template <typename... Args> +void tcp_debug(const char* fmt, Args&&... args) { +#if TCP_DEBUG + print(fmt, std::forward<Args>(args)...); +#endif +} + +struct tcp_option { + // The kind and len field are fixed and defined in TCP protocol + enum class option_kind: uint8_t { mss = 2, win_scale = 3, sack = 4, timestamps = 8, nop = 1, eol = 0 }; + enum class option_len: uint8_t { mss = 4, win_scale = 3, sack = 2, timestamps = 10, nop = 1, eol = 1 }; + static void write(char* p, option_kind kind, option_len len) { + p[0] = static_cast<uint8_t>(kind); + if (static_cast<uint8_t>(len) > 1) { + p[1] = static_cast<uint8_t>(len); + } + } + struct mss { + static constexpr option_kind kind = option_kind::mss; + static constexpr option_len len = option_len::mss; + uint16_t mss; + static tcp_option::mss read(const char* p) { + tcp_option::mss x; + x.mss = read_be<uint16_t>(p + 2); + return x; + } + void write(char* p) const { + tcp_option::write(p, kind, len); + write_be<uint16_t>(p + 2, mss); + } + }; + struct win_scale { + static constexpr option_kind kind = option_kind::win_scale; + static constexpr option_len len = option_len::win_scale; + uint8_t shift; + static tcp_option::win_scale read(const char* p) { + tcp_option::win_scale x; + x.shift = p[2]; + return x; + } + void write(char* p) const { + tcp_option::write(p, kind, len); + p[2] = shift; + } + }; + struct sack { + static constexpr option_kind kind = option_kind::sack; + static constexpr option_len len = option_len::sack; + static tcp_option::sack read(const char* p) { + return {}; + } + void write(char* p) const { + tcp_option::write(p, kind, len); + } + }; + struct timestamps { + static constexpr option_kind kind = option_kind::timestamps; + static constexpr option_len len = option_len::timestamps; + uint32_t t1; + uint32_t t2; + static tcp_option::timestamps read(const char* p) { + tcp_option::timestamps ts; + ts.t1 = read_be<uint32_t>(p + 2); + ts.t2 = read_be<uint32_t>(p + 6); + return ts; + } + void write(char* p) const { + tcp_option::write(p, kind, len); + write_be<uint32_t>(p + 2, t1); + write_be<uint32_t>(p + 6, t2); + } + }; + struct nop { + static constexpr option_kind kind = option_kind::nop; + static constexpr option_len len = option_len::nop; + void write(char* p) const { + tcp_option::write(p, kind, len); + } + }; + struct eol { + static constexpr option_kind kind = option_kind::eol; + static constexpr option_len len = option_len::eol; + void write(char* p) const { + tcp_option::write(p, kind, len); + } + }; + static const uint8_t align = 4; + + void parse(uint8_t* beg, uint8_t* end); + uint8_t fill(void* h, const tcp_hdr* th, uint8_t option_size); + uint8_t get_size(bool syn_on, bool ack_on); + + // For option negotiattion + bool _mss_received = false; + bool _win_scale_received = false; + bool _timestamps_received = false; + bool _sack_received = false; + + // Option data + uint16_t _remote_mss = 536; + uint16_t _local_mss; + uint8_t _remote_win_scale = 0; + uint8_t _local_win_scale = 0; +}; +inline char*& operator+=(char*& x, tcp_option::option_len len) { x += uint8_t(len); return x; } +inline const char*& operator+=(const char*& x, tcp_option::option_len len) { x += uint8_t(len); return x; } +inline uint8_t& operator+=(uint8_t& x, tcp_option::option_len len) { x += uint8_t(len); return x; } + +struct tcp_seq { + uint32_t raw; +}; + +inline tcp_seq ntoh(tcp_seq s) { + return tcp_seq { ntoh(s.raw) }; +} + +inline tcp_seq hton(tcp_seq s) { + return tcp_seq { hton(s.raw) }; +} + +inline +std::ostream& operator<<(std::ostream& os, tcp_seq s) { + return os << s.raw; +} + +inline tcp_seq make_seq(uint32_t raw) { return tcp_seq{raw}; } +inline tcp_seq& operator+=(tcp_seq& s, int32_t n) { s.raw += n; return s; } +inline tcp_seq& operator-=(tcp_seq& s, int32_t n) { s.raw -= n; return s; } +inline tcp_seq operator+(tcp_seq s, int32_t n) { return s += n; } +inline tcp_seq operator-(tcp_seq s, int32_t n) { return s -= n; } +inline int32_t operator-(tcp_seq s, tcp_seq q) { return s.raw - q.raw; } +inline bool operator==(tcp_seq s, tcp_seq q) { return s.raw == q.raw; } +inline bool operator!=(tcp_seq s, tcp_seq q) { return !(s == q); } +inline bool operator<(tcp_seq s, tcp_seq q) { return s - q < 0; } +inline bool operator>(tcp_seq s, tcp_seq q) { return q < s; } +inline bool operator<=(tcp_seq s, tcp_seq q) { return !(s > q); } +inline bool operator>=(tcp_seq s, tcp_seq q) { return !(s < q); } + +struct tcp_hdr { + static constexpr size_t len = 20; + uint16_t src_port; + uint16_t dst_port; + tcp_seq seq; + tcp_seq ack; + uint8_t rsvd1 : 4; + uint8_t data_offset : 4; + uint8_t f_fin : 1; + uint8_t f_syn : 1; + uint8_t f_rst : 1; + uint8_t f_psh : 1; + uint8_t f_ack : 1; + uint8_t f_urg : 1; + uint8_t rsvd2 : 2; + uint16_t window; + uint16_t checksum; + uint16_t urgent; + static tcp_hdr read(const char* p) { + tcp_hdr h; + h.src_port = read_be<uint16_t>(p + 0); + h.dst_port = read_be<uint16_t>(p + 2); + h.seq = tcp_seq{read_be<uint32_t>(p + 4)}; + h.ack = tcp_seq{read_be<uint32_t>(p + 8)}; + h.rsvd1 = p[12] & 15; + h.data_offset = uint8_t(p[12]) >> 4; + h.f_fin = (uint8_t(p[13]) >> 0) & 1; + h.f_syn = (uint8_t(p[13]) >> 1) & 1; + h.f_rst = (uint8_t(p[13]) >> 2) & 1; + h.f_psh = (uint8_t(p[13]) >> 3) & 1; + h.f_ack = (uint8_t(p[13]) >> 4) & 1; + h.f_urg = (uint8_t(p[13]) >> 5) & 1; + h.rsvd2 = (uint8_t(p[13]) >> 6) & 3; + h.window = read_be<uint16_t>(p + 14); + h.checksum = read_be<uint16_t>(p + 16); + h.urgent = read_be<uint16_t>(p + 18); + return h; + } + void write(char* p) const { + write_be<uint16_t>(p + 0, src_port); + write_be<uint16_t>(p + 2, dst_port); + write_be<uint32_t>(p + 4, seq.raw); + write_be<uint32_t>(p + 8, ack.raw); + p[12] = rsvd1 | (data_offset << 4); + p[13] = (f_fin << 0) + | (f_syn << 1) + | (f_rst << 2) + | (f_psh << 3) + | (f_ack << 4) + | (f_urg << 5) + | (rsvd2 << 6); + write_be<uint16_t>(p + 14, window); + write_be<uint16_t>(p + 16, checksum); + write_be<uint16_t>(p + 18, urgent); + } + static void write_nbo_checksum(char* p, uint16_t checksum_in_network_byte_order) { + std::copy_n(reinterpret_cast<const char*>(&checksum_in_network_byte_order), 2, p + 16); + } +}; + +struct tcp_tag {}; +using tcp_packet_merger = packet_merger<tcp_seq, tcp_tag>; + +template <typename InetTraits> +class tcp { +public: + using ipaddr = typename InetTraits::address_type; + using inet_type = typename InetTraits::inet_type; + using connid = l4connid<InetTraits>; + using connid_hash = typename connid::connid_hash; + class connection; + class listener; +private: + class tcb; + + class tcb : public enable_lw_shared_from_this<tcb> { + using clock_type = lowres_clock; + static constexpr tcp_state CLOSED = tcp_state::CLOSED; + static constexpr tcp_state LISTEN = tcp_state::LISTEN; + static constexpr tcp_state SYN_SENT = tcp_state::SYN_SENT; + static constexpr tcp_state SYN_RECEIVED = tcp_state::SYN_RECEIVED; + static constexpr tcp_state ESTABLISHED = tcp_state::ESTABLISHED; + static constexpr tcp_state FIN_WAIT_1 = tcp_state::FIN_WAIT_1; + static constexpr tcp_state FIN_WAIT_2 = tcp_state::FIN_WAIT_2; + static constexpr tcp_state CLOSE_WAIT = tcp_state::CLOSE_WAIT; + static constexpr tcp_state CLOSING = tcp_state::CLOSING; + static constexpr tcp_state LAST_ACK = tcp_state::LAST_ACK; + static constexpr tcp_state TIME_WAIT = tcp_state::TIME_WAIT; + tcp_state _state = CLOSED; + tcp& _tcp; + connection* _conn = nullptr; + promise<> _connect_done; + std::optional<promise<>> _fin_recvd_promise = promise<>(); + ipaddr _local_ip; + ipaddr _foreign_ip; + uint16_t _local_port; + uint16_t _foreign_port; + struct unacked_segment { + packet p; + uint16_t data_len; + unsigned nr_transmits; + clock_type::time_point tx_time; + }; + struct send { + tcp_seq unacknowledged; + tcp_seq next; + uint32_t window; + uint8_t window_scale; + uint16_t mss; + tcp_seq urgent; + tcp_seq wl1; + tcp_seq wl2; + tcp_seq initial; + std::deque<unacked_segment> data; + std::deque<packet> unsent; + uint32_t unsent_len = 0; + bool closed = false; + promise<> _window_opened; + // Wait for all data are acked + std::optional<promise<>> _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::optional<promise<>> _send_available_promise; + // Round-trip time variation + std::chrono::milliseconds rttvar; + // Smoothed round-trip time + std::chrono::milliseconds srtt; + bool first_rto_sample = true; + clock_type::time_point syn_tx_time; + // Congestion window + uint32_t cwnd; + // Slow start threshold + uint32_t ssthresh; + // Duplicated ACKs + uint16_t dupacks = 0; + unsigned syn_retransmit = 0; + unsigned fin_retransmit = 0; + uint32_t limited_transfer = 0; + uint32_t partial_ack = 0; + tcp_seq recover; + bool window_probe = false; + uint8_t zero_window_probing_out = 0; + } _snd; + struct receive { + tcp_seq next; + uint32_t window; + uint8_t window_scale; + uint16_t mss; + tcp_seq urgent; + tcp_seq initial; + std::deque<packet> data; + // The total size of data stored in std::deque<packet> data + size_t data_size = 0; + tcp_packet_merger out_of_order; + std::optional<promise<>> _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; + } _rcv; + tcp_option _option; + timer<lowres_clock> _delayed_ack; + // Retransmission timeout + std::chrono::milliseconds _rto{1000}; + std::chrono::milliseconds _persist_time_out{1000}; + static constexpr std::chrono::milliseconds _rto_min{1000}; + static constexpr std::chrono::milliseconds _rto_max{60000}; + // Clock granularity + static constexpr std::chrono::milliseconds _rto_clk_granularity{1}; + static constexpr uint16_t _max_nr_retransmit{5}; + timer<lowres_clock> _retransmit; + timer<lowres_clock> _persist; + uint16_t _nr_full_seg_received = 0; + struct isn_secret { + // 512 bits secretkey for ISN generating + uint32_t key[16]; + isn_secret () { + std::random_device rd; + std::default_random_engine e(rd()); + std::uniform_int_distribution<uint32_t> dist{}; + for (auto& k : key) { + k = dist(e); + } + } + }; + static isn_secret _isn_secret; + tcp_seq get_isn(); + circular_buffer<typename InetTraits::l4packet> _packetq; + bool _poll_active = false; + uint32_t get_default_receive_window_size() { + // Linux's default window size + constexpr uint32_t size = 29200; + return size << _rcv.window_scale; + } + // Returns the current receive window according to available receiving buffer size + uint32_t get_modified_receive_window_size() { + uint32_t left = _rcv.data_size > _rcv.max_receive_buf_size ? 0 : _rcv.max_receive_buf_size - _rcv.data_size; + return std::min(left, get_default_receive_window_size()); + } + public: + tcb(tcp& t, connid id); + void input_handle_listen_state(tcp_hdr* th, packet p); + void input_handle_syn_sent_state(tcp_hdr* th, packet p); + void input_handle_other_state(tcp_hdr* th, packet p); + void output_one(bool data_retransmit = false); + future<> wait_for_data(); + future<> wait_input_shutdown(); + void abort_reader() noexcept; + future<> wait_for_all_data_acked(); + future<> wait_send_available(); + future<> send(packet p); + void connect(); + packet read(); + void close() noexcept; + void remove_from_tcbs() { + auto id = connid{_local_ip, _foreign_ip, _local_port, _foreign_port}; + _tcp._tcbs.erase(id); + } + std::optional<typename InetTraits::l4packet> get_packet(); + void output() { + if (!_poll_active) { + _poll_active = true; + // FIXME: future is discarded + (void)_tcp.poll_tcb(_foreign_ip, this->shared_from_this()).then_wrapped([this] (auto&& f) { + try { + f.get(); + } catch(arp_queue_full_error& ex) { + // retry later + _poll_active = false; + this->start_retransmit_timer(); + } catch(arp_timeout_error& ex) { + if (this->in_state(SYN_SENT)) { + _connect_done.set_exception(ex); + this->cleanup(); + } + // in other states connection should time out + } + }); + } + } + future<> connect_done() { + return _connect_done.get_future(); + } + tcp_state& state() { + return _state; + } + private: + void respond_with_reset(tcp_hdr* th); + bool merge_out_of_order(); + void insert_out_of_order(tcp_seq seq, packet p); + void trim_receive_data_after_window(); + bool should_send_ack(uint16_t seg_len); + void clear_delayed_ack() noexcept; + packet get_transmit_packet(); + void retransmit_one() { + bool data_retransmit = true; + output_one(data_retransmit); + } + void start_retransmit_timer() { + auto now = clock_type::now(); + start_retransmit_timer(now); + }; + void start_retransmit_timer(clock_type::time_point now) { + auto tp = now + _rto; + _retransmit.rearm(tp); + }; + void stop_retransmit_timer() noexcept { + _retransmit.cancel(); + }; + void start_persist_timer() { + auto now = clock_type::now(); + start_persist_timer(now); + }; + void start_persist_timer(clock_type::time_point now) { + auto tp = now + _persist_time_out; + _persist.rearm(tp); + }; + void stop_persist_timer() { + _persist.cancel(); + }; + void persist(); + void retransmit(); + void fast_retransmit(); + void update_rto(clock_type::time_point tx_time); + void update_cwnd(uint32_t acked_bytes); + void cleanup(); + uint32_t can_send() { + if (_snd.window_probe) { + return 1; + } + + // Can not send if send window is zero + if (_snd.window == 0) { + return 0; + } + + // Can not send if send window is less than unacknowledged data size + auto window_used = uint32_t(_snd.next - _snd.unacknowledged); + if (window_used > _snd.window) { + return 0; + } + + // Can not send more than advertised window allows or unsent data size + auto x = std::min(_snd.window - window_used, _snd.unsent_len); + + // Can not send more than congestion window allows + x = std::min(_snd.cwnd, x); + if (_snd.dupacks == 1 || _snd.dupacks == 2) { + // RFC5681 Step 3.1 + // Send cwnd + 2 * smss per RFC3042 + auto flight = flight_size(); + auto max = _snd.cwnd + 2 * _snd.mss; + x = flight <= max ? std::min(x, max - flight) : 0; + _snd.limited_transfer += x; + } else if (_snd.dupacks >= 3) { + // RFC5681 Step 3.5 + // Sent 1 full-sized segment at most + x = std::min(uint32_t(_snd.mss), x); + } + return x; + } + uint32_t flight_size() { + uint32_t size = 0; + std::for_each(_snd.data.begin(), _snd.data.end(), [&] (unacked_segment& seg) { size += seg.p.len(); }); + return size; + } + uint16_t local_mss() { + return _tcp.hw_features().mtu - net::tcp_hdr_len_min - InetTraits::ip_hdr_len_min; + } + void queue_packet(packet p) { + _packetq.emplace_back(typename InetTraits::l4packet{_foreign_ip, std::move(p)}); + } + void signal_data_received() { + if (_rcv._data_received_promise) { + _rcv._data_received_promise->set_value(); + _rcv._data_received_promise = {}; + } + } + void signal_all_data_acked() { + if (_snd._all_data_acked_promise && _snd.unsent_len == 0) { + _snd._all_data_acked_promise->set_value(); + _snd._all_data_acked_promise = {}; + } + } + void signal_send_available() { + if (_snd._send_available_promise && _snd.max_queue_space > _snd.current_queue_space) { + _snd._send_available_promise->set_value(); + _snd._send_available_promise = {}; + } + } + void do_syn_sent() { + _state = SYN_SENT; + _snd.syn_tx_time = clock_type::now(); + // Send <SYN> to remote + output(); + } + void do_syn_received() { + _state = SYN_RECEIVED; + _snd.syn_tx_time = clock_type::now(); + // Send <SYN,ACK> to remote + output(); + } + void do_established() { + _state = ESTABLISHED; + update_rto(_snd.syn_tx_time); + _connect_done.set_value(); + } + void do_reset() { + _state = CLOSED; + cleanup(); + if (_rcv._data_received_promise) { + _rcv._data_received_promise->set_exception(tcp_reset_error()); + _rcv._data_received_promise = std::nullopt; + } + if (_snd._all_data_acked_promise) { + _snd._all_data_acked_promise->set_exception(tcp_reset_error()); + _snd._all_data_acked_promise = std::nullopt; + } + if (_snd._send_available_promise) { + _snd._send_available_promise->set_exception(tcp_reset_error()); + _snd._send_available_promise = std::nullopt; + } + } + void do_time_wait() { + // FIXME: Implement TIME_WAIT state timer + _state = TIME_WAIT; + cleanup(); + } + void do_closed() { + _state = CLOSED; + cleanup(); + } + void do_setup_isn() { + _snd.initial = get_isn(); + _snd.unacknowledged = _snd.initial; + _snd.next = _snd.initial + 1; + _snd.recover = _snd.initial; + } + void do_local_fin_acked() { + _snd.unacknowledged += 1; + _snd.next += 1; + } + bool syn_needs_on() const noexcept { + return in_state(SYN_SENT | SYN_RECEIVED); + } + bool fin_needs_on() const noexcept { + return in_state(FIN_WAIT_1 | CLOSING | LAST_ACK) && _snd.closed && + _snd.unsent_len == 0; + } + bool ack_needs_on() const noexcept { + return !in_state(CLOSED | LISTEN | SYN_SENT); + } + bool foreign_will_not_send() const noexcept { + return in_state(CLOSING | TIME_WAIT | CLOSE_WAIT | LAST_ACK | CLOSED); + } + bool in_state(tcp_state state) const noexcept { + return uint16_t(_state) & uint16_t(state); + } + void exit_fast_recovery() { + _snd.dupacks = 0; + _snd.limited_transfer = 0; + _snd.partial_ack = 0; + } + uint32_t data_segment_acked(tcp_seq seg_ack); + bool segment_acceptable(tcp_seq seg_seq, unsigned seg_len); + void init_from_options(tcp_hdr* th, uint8_t* opt_start, uint8_t* opt_end); + friend class connection; + }; + inet_type& _inet; + std::unordered_map<connid, lw_shared_ptr<tcb>, connid_hash> _tcbs; + std::unordered_map<uint16_t, listener*> _listening; + std::random_device _rd; + std::default_random_engine _e; + std::uniform_int_distribution<uint16_t> _port_dist{41952, 65535}; + circular_buffer<std::pair<lw_shared_ptr<tcb>, ethernet_address>> _poll_tcbs; + // queue for packets that do not belong to any tcb + circular_buffer<ipv4_traits::l4packet> _packetq; + semaphore _queue_space = {212992}; + metrics::metric_groups _metrics; +public: + const inet_type& inet() const { + return _inet; + } + class connection { + lw_shared_ptr<tcb> _tcb; + public: + explicit connection(lw_shared_ptr<tcb> tcbp) : _tcb(std::move(tcbp)) { _tcb->_conn = this; } + connection(const connection&) = delete; + connection(connection&& x) noexcept : _tcb(std::move(x._tcb)) { + _tcb->_conn = this; + } + ~connection(); + void operator=(const connection&) = delete; + connection& operator=(connection&& x) { + if (this != &x) { + this->~connection(); + new (this) connection(std::move(x)); + } + return *this; + } + future<> connected() { + return _tcb->connect_done(); + } + future<> send(packet p) { + return _tcb->send(std::move(p)); + } + future<> wait_for_data() { + return _tcb->wait_for_data(); + } + future<> wait_input_shutdown() { + return _tcb->wait_input_shutdown(); + } + packet read() { + return _tcb->read(); + } + ipaddr foreign_ip() { + return _tcb->_foreign_ip; + } + uint16_t foreign_port() { + return _tcb->_foreign_port; + } + ipaddr local_ip() { + return _tcb->_local_ip; + } + uint16_t local_port() { + return _tcb->_local_port; + } + void shutdown_connect(); + void close_read() noexcept; + void close_write() noexcept; + }; + class listener { + tcp& _tcp; + uint16_t _port; + queue<connection> _q; + size_t _pending = 0; + private: + listener(tcp& t, uint16_t port, size_t queue_length) + : _tcp(t), _port(port), _q(queue_length) { + _tcp._listening.emplace(_port, this); + } + public: + listener(listener&& x) + : _tcp(x._tcp), _port(x._port), _q(std::move(x._q)) { + _tcp._listening[_port] = this; + x._port = 0; + } + ~listener() { + if (_port) { + _tcp._listening.erase(_port); + } + } + future<connection> accept() { + return _q.not_empty().then([this] { + return make_ready_future<connection>(_q.pop()); + }); + } + void abort_accept() { + _q.abort(std::make_exception_ptr(std::system_error(ECONNABORTED, std::system_category()))); + } + bool full() { return _pending + _q.size() >= _q.max_size(); } + void inc_pending() { _pending++; } + void dec_pending() { _pending--; } + + const tcp& get_tcp() const { + return _tcp; + } + uint16_t port() const { + return _port; + } + friend class tcp; + }; +public: + explicit tcp(inet_type& inet); + void received(packet p, ipaddr from, ipaddr to); + bool forward(forward_hash& out_hash_data, packet& p, size_t off); + listener listen(uint16_t port, size_t queue_length = 100); + connection connect(socket_address sa); + const net::hw_features& hw_features() const { return _inet._inet.hw_features(); } + future<> poll_tcb(ipaddr to, lw_shared_ptr<tcb> tcb); + void add_connected_tcb(lw_shared_ptr<tcb> tcbp, uint16_t local_port) { + auto it = _listening.find(local_port); + if (it != _listening.end()) { + it->second->_q.push(connection(tcbp)); + it->second->dec_pending(); + } + } +private: + void send_packet_without_tcb(ipaddr from, ipaddr to, packet p); + void respond_with_reset(tcp_hdr* rth, ipaddr local_ip, ipaddr foreign_ip); + friend class listener; +}; + +template <typename InetTraits> +tcp<InetTraits>::tcp(inet_type& inet) + : _inet(inet) + , _e(_rd()) { + namespace sm = metrics; + + _metrics.add_group("tcp", { + sm::make_counter("linearizations", [] { return tcp_packet_merger::linearizations(); }, + sm::description("Counts a number of times a buffer linearization was invoked during the buffers merge process. " + "Divide it by a total TCP receive packet rate to get an everage number of lineraizations per TCP packet.")) + }); + + _inet.register_packet_provider([this, tcb_polled = 0u] () mutable { + std::optional<typename InetTraits::l4packet> l4p; + auto c = _poll_tcbs.size(); + if (!_packetq.empty() && (!(tcb_polled % 128) || c == 0)) { + l4p = std::move(_packetq.front()); + _packetq.pop_front(); + _queue_space.signal(l4p.value().p.len()); + } else { + while (c--) { + tcb_polled++; + lw_shared_ptr<tcb> tcb; + ethernet_address dst; + std::tie(tcb, dst) = std::move(_poll_tcbs.front()); + _poll_tcbs.pop_front(); + l4p = tcb->get_packet(); + if (l4p) { + l4p.value().e_dst = dst; + break; + } + } + } + return l4p; + }); +} + +template <typename InetTraits> +future<> tcp<InetTraits>::poll_tcb(ipaddr to, lw_shared_ptr<tcb> tcb) { + return _inet.get_l2_dst_address(to).then([this, tcb = std::move(tcb)] (ethernet_address dst) { + _poll_tcbs.emplace_back(std::move(tcb), dst); + }); +} + +template <typename InetTraits> +auto tcp<InetTraits>::listen(uint16_t port, size_t queue_length) -> listener { + return listener(*this, port, queue_length); +} + +template <typename InetTraits> +auto tcp<InetTraits>::connect(socket_address sa) -> connection { + uint16_t src_port; + connid id; + auto src_ip = _inet._inet.host_address(); + auto dst_ip = ipv4_address(sa); + auto dst_port = net::ntoh(sa.u.in.sin_port); + + do { + src_port = _port_dist(_e); + id = connid{src_ip, dst_ip, src_port, dst_port}; + } while (_inet._inet.netif()->hw_queues_count() > 1 && + (_inet._inet.netif()->hash2cpu(id.hash(_inet._inet.netif()->rss_key())) != this_shard_id() + || _tcbs.find(id) != _tcbs.end())); + + auto tcbp = make_lw_shared<tcb>(*this, id); + _tcbs.insert({id, tcbp}); + tcbp->connect(); + return connection(tcbp); +} + +template <typename InetTraits> +bool tcp<InetTraits>::forward(forward_hash& out_hash_data, packet& p, size_t off) { + auto th = p.get_header(off, tcp_hdr::len); + if (th) { + // src_port, dst_port in network byte order + out_hash_data.push_back(uint8_t(th[0])); + out_hash_data.push_back(uint8_t(th[1])); + out_hash_data.push_back(uint8_t(th[2])); + out_hash_data.push_back(uint8_t(th[3])); + } + return true; +} + +template <typename InetTraits> +void tcp<InetTraits>::received(packet p, ipaddr from, ipaddr to) { + auto th = p.get_header(0, tcp_hdr::len); + if (!th) { + return; + } + // data_offset is correct even before ntoh() + auto data_offset = uint8_t(th[12]) >> 4; + if (size_t(data_offset * 4) < tcp_hdr::len) { + return; + } + + if (!hw_features().rx_csum_offload) { + checksummer csum; + InetTraits::tcp_pseudo_header_checksum(csum, from, to, p.len()); + csum.sum(p); + if (csum.get() != 0) { + return; + } + } + auto h = tcp_hdr::read(th); + auto id = connid{to, from, h.dst_port, h.src_port}; + auto tcbi = _tcbs.find(id); + lw_shared_ptr<tcb> tcbp; + if (tcbi == _tcbs.end()) { + auto listener = _listening.find(id.local_port); + if (listener == _listening.end() || listener->second->full()) { + // 1) In CLOSE state + // 1.1 all data in the incoming segment is discarded. An incoming + // segment containing a RST is discarded. An incoming segment not + // containing a RST causes a RST to be sent in response. + // FIXME: + // if ACK off: <SEQ=0><ACK=SEG.SEQ+SEG.LEN><CTL=RST,ACK> + // if ACK on: <SEQ=SEG.ACK><CTL=RST> + return respond_with_reset(&h, id.local_ip, id.foreign_ip); + } else { + // 2) In LISTEN state + // 2.1 first check for an RST + if (h.f_rst) { + // An incoming RST should be ignored + return; + } + // 2.2 second check for an ACK + if (h.f_ack) { + // Any acknowledgment is bad if it arrives on a connection + // still in the LISTEN state. + // <SEQ=SEG.ACK><CTL=RST> + return respond_with_reset(&h, id.local_ip, id.foreign_ip); + } + // 2.3 third check for a SYN + if (h.f_syn) { + // check the security + // NOTE: Ignored for now + tcbp = make_lw_shared<tcb>(*this, id); + _tcbs.insert({id, tcbp}); + // TODO: we need to remove the tcb and decrease the pending if + // it stays SYN_RECEIVED state forever. + listener->second->inc_pending(); + + return tcbp->input_handle_listen_state(&h, std::move(p)); + } + // 2.4 fourth other text or control + // So you are unlikely to get here, but if you do, drop the + // segment, and return. + return; + } + } else { + tcbp = tcbi->second; + if (tcbp->state() == tcp_state::SYN_SENT) { + // 3) In SYN_SENT State + return tcbp->input_handle_syn_sent_state(&h, std::move(p)); + } else { + // 4) In other state, can be one of the following: + // SYN_RECEIVED, ESTABLISHED, FIN_WAIT_1, FIN_WAIT_2 + // CLOSE_WAIT, CLOSING, LAST_ACK, TIME_WAIT + return tcbp->input_handle_other_state(&h, std::move(p)); + } + } +} + +// Send packet does not belong to any tcb +template <typename InetTraits> +void tcp<InetTraits>::send_packet_without_tcb(ipaddr from, ipaddr to, packet p) { + if (_queue_space.try_wait(p.len())) { // drop packets that do not fit the queue + // FIXME: future is discarded + (void)_inet.get_l2_dst_address(to).then([this, to, p = std::move(p)] (ethernet_address e_dst) mutable { + _packetq.emplace_back(ipv4_traits::l4packet{to, std::move(p), e_dst, ip_protocol_num::tcp}); + }); + } +} + +template <typename InetTraits> +tcp<InetTraits>::connection::~connection() { + if (_tcb) { + _tcb->_conn = nullptr; + close_read(); + close_write(); + } +} + +template <typename InetTraits> +tcp<InetTraits>::tcb::tcb(tcp& t, connid id) + : _tcp(t) + , _local_ip(id.local_ip) + , _foreign_ip(id.foreign_ip) + , _local_port(id.local_port) + , _foreign_port(id.foreign_port) + , _delayed_ack([this] { _nr_full_seg_received = 0; output(); }) + , _retransmit([this] { retransmit(); }) + , _persist([this] { persist(); }) { +} + +template <typename InetTraits> +void tcp<InetTraits>::tcb::respond_with_reset(tcp_hdr* rth) { + _tcp.respond_with_reset(rth, _local_ip, _foreign_ip); +} + +template <typename InetTraits> +void tcp<InetTraits>::respond_with_reset(tcp_hdr* rth, ipaddr local_ip, ipaddr foreign_ip) { + if (rth->f_rst) { + return; + } + packet p; + auto th = p.prepend_uninitialized_header(tcp_hdr::len); + auto h = tcp_hdr{}; + h.src_port = rth->dst_port; + h.dst_port = rth->src_port; + if (rth->f_ack) { + h.seq = rth->ack; + } + // If this RST packet is in response to a SYN packet. We ACK the ISN. + if (rth->f_syn) { + h.ack = rth->seq + 1; + h.f_ack = true; + } + h.f_rst = true; + h.data_offset = tcp_hdr::len / 4; + h.checksum = 0; + h.write(th); + + checksummer csum; + offload_info oi; + InetTraits::tcp_pseudo_header_checksum(csum, local_ip, foreign_ip, tcp_hdr::len); + uint16_t checksum; + if (hw_features().tx_csum_l4_offload) { + checksum = ~csum.get(); + oi.needs_csum = true; + } else { + csum.sum(p); + checksum = csum.get(); + oi.needs_csum = false; + } + tcp_hdr::write_nbo_checksum(th, checksum); + + oi.protocol = ip_protocol_num::tcp; + oi.tcp_hdr_len = tcp_hdr::len; + p.set_offload_info(oi); + + send_packet_without_tcb(local_ip, foreign_ip, std::move(p)); +} + +template <typename InetTraits> +uint32_t tcp<InetTraits>::tcb::data_segment_acked(tcp_seq seg_ack) { + uint32_t total_acked_bytes = 0; + // Full ACK of segment + while (!_snd.data.empty() + && (_snd.unacknowledged + _snd.data.front().p.len() <= seg_ack)) { + auto acked_bytes = _snd.data.front().p.len(); + _snd.unacknowledged += acked_bytes; + // Ignore retransmitted segments when setting the RTO + if (_snd.data.front().nr_transmits == 0) { + update_rto(_snd.data.front().tx_time); + } + update_cwnd(acked_bytes); + total_acked_bytes += acked_bytes; + _snd.current_queue_space -= _snd.data.front().data_len; + signal_send_available(); + _snd.data.pop_front(); + } + // Partial ACK of segment + if (_snd.unacknowledged < seg_ack) { + auto acked_bytes = seg_ack - _snd.unacknowledged; + if (!_snd.data.empty()) { + auto& unacked_seg = _snd.data.front(); + unacked_seg.p.trim_front(acked_bytes); + } + _snd.unacknowledged = seg_ack; + update_cwnd(acked_bytes); + total_acked_bytes += acked_bytes; + } + return total_acked_bytes; +} + +template <typename InetTraits> +bool tcp<InetTraits>::tcb::segment_acceptable(tcp_seq seg_seq, unsigned seg_len) { + if (seg_len == 0 && _rcv.window == 0) { + // SEG.SEQ = RCV.NXT + return seg_seq == _rcv.next; + } else if (seg_len == 0 && _rcv.window > 0) { + // RCV.NXT =< SEG.SEQ < RCV.NXT+RCV.WND + return (_rcv.next <= seg_seq) && (seg_seq < _rcv.next + _rcv.window); + } else if (seg_len > 0 && _rcv.window > 0) { + // RCV.NXT =< SEG.SEQ < RCV.NXT+RCV.WND + // or + // RCV.NXT =< SEG.SEQ+SEG.LEN-1 < RCV.NXT+RCV.WND + bool x = (_rcv.next <= seg_seq) && seg_seq < (_rcv.next + _rcv.window); + bool y = (_rcv.next <= seg_seq + seg_len - 1) && (seg_seq + seg_len - 1 < _rcv.next + _rcv.window); + return x || y; + } else { + // SEG.LEN > 0 RCV.WND = 0, not acceptable + return false; + } +} + +template <typename InetTraits> +void tcp<InetTraits>::tcb::init_from_options(tcp_hdr* th, uint8_t* opt_start, uint8_t* opt_end) { + // Handle tcp options + _option.parse(opt_start, opt_end); + + // Remote receive window scale factor + _snd.window_scale = _option._remote_win_scale; + // Local receive window scale factor + _rcv.window_scale = _option._local_win_scale; + + // Maximum segment size remote can receive + _snd.mss = _option._remote_mss; + // Maximum segment size local can receive + _rcv.mss = _option._local_mss = local_mss(); + + _rcv.window = get_default_receive_window_size(); + _snd.window = th->window << _snd.window_scale; + + // Segment sequence number used for last window update + _snd.wl1 = th->seq; + // Segment acknowledgment number used for last window update + _snd.wl2 = th->ack; + + // Setup initial congestion window + if (2190 < _snd.mss) { + _snd.cwnd = 2 * _snd.mss; + } else if (1095 < _snd.mss && _snd.mss <= 2190) { + _snd.cwnd = 3 * _snd.mss; + } else { + _snd.cwnd = 4 * _snd.mss; + } + + // Setup initial slow start threshold + _snd.ssthresh = th->window << _snd.window_scale; +} + +template <typename InetTraits> +void tcp<InetTraits>::tcb::input_handle_listen_state(tcp_hdr* th, packet p) { + auto opt_len = th->data_offset * 4 - tcp_hdr::len; + auto opt_start = reinterpret_cast<uint8_t*>(p.get_header(0, th->data_offset * 4)) + tcp_hdr::len; + auto opt_end = opt_start + opt_len; + p.trim_front(th->data_offset * 4); + tcp_seq seg_seq = th->seq; + + // Set RCV.NXT to SEG.SEQ+1, IRS is set to SEG.SEQ + _rcv.next = seg_seq + 1; + _rcv.initial = seg_seq; + + // ISS should be selected and a SYN segment sent of the form: + // <SEQ=ISS><ACK=RCV.NXT><CTL=SYN,ACK> + // SND.NXT is set to ISS+1 and SND.UNA to ISS + // NOTE: In previous code, _snd.next is set to ISS + 1 only when SYN is + // ACKed. Now, we set _snd.next to ISS + 1 here, so in output_one(): we + // have + // th->seq = syn_on ? _snd.initial : _snd.next + // to make sure retransmitted SYN has correct SEQ number. + do_setup_isn(); + + _rcv.urgent = _rcv.next; + + tcp_debug("listen: LISTEN -> SYN_RECEIVED\n"); + init_from_options(th, opt_start, opt_end); + do_syn_received(); +} + +template <typename InetTraits> +void tcp<InetTraits>::tcb::input_handle_syn_sent_state(tcp_hdr* th, packet p) { + auto opt_len = th->data_offset * 4 - tcp_hdr::len; + auto opt_start = reinterpret_cast<uint8_t*>(p.get_header(0, th->data_offset * 4)) + tcp_hdr::len; + auto opt_end = opt_start + opt_len; + p.trim_front(th->data_offset * 4); + tcp_seq seg_seq = th->seq; + auto seg_ack = th->ack; + + bool acceptable = false; + // 3.1 first check the ACK bit + if (th->f_ack) { + // If SEG.ACK =< ISS, or SEG.ACK > SND.NXT, send a reset (unless the + // RST bit is set, if so drop the segment and return) + if (seg_ack <= _snd.initial || seg_ack > _snd.next) { + return respond_with_reset(th); + } + + // If SND.UNA =< SEG.ACK =< SND.NXT then the ACK is acceptable. + acceptable = _snd.unacknowledged <= seg_ack && seg_ack <= _snd.next; + } + + // 3.2 second check the RST bit + if (th->f_rst) { + // If the ACK was acceptable then signal the user "error: connection + // reset", drop the segment, enter CLOSED state, delete TCB, and + // return. Otherwise (no ACK) drop the segment and return. + if (acceptable) { + _connect_done.set_exception(tcp_refused_error()); + return do_reset(); + } else { + return; + } + } + + // 3.3 third check the security and precedence + // NOTE: Ignored for now + + // 3.4 fourth check the SYN bit + if (th->f_syn) { + // RCV.NXT is set to SEG.SEQ+1, IRS is set to SEG.SEQ. SND.UNA should + // be advanced to equal SEG.ACK (if there is an ACK), and any segments + // on the retransmission queue which are thereby acknowledged should be + // removed. + _rcv.next = seg_seq + 1; + _rcv.initial = seg_seq; + if (th->f_ack) { + // TODO: clean retransmission queue + _snd.unacknowledged = seg_ack; + } + if (_snd.unacknowledged > _snd.initial) { + // If SND.UNA > ISS (our SYN has been ACKed), change the connection + // state to ESTABLISHED, form an ACK segment + // <SEQ=SND.NXT><ACK=RCV.NXT><CTL=ACK> + tcp_debug("syn: SYN_SENT -> ESTABLISHED\n"); + init_from_options(th, opt_start, opt_end); + do_established(); + output(); + } else { + // Otherwise enter SYN_RECEIVED, form a SYN,ACK segment + // <SEQ=ISS><ACK=RCV.NXT><CTL=SYN,ACK> + tcp_debug("syn: SYN_SENT -> SYN_RECEIVED\n"); + do_syn_received(); + } + } + + // 3.5 fifth, if neither of the SYN or RST bits is set then drop the + // segment and return. + return; +} + +template <typename InetTraits> +void tcp<InetTraits>::tcb::input_handle_other_state(tcp_hdr* th, packet p) { + p.trim_front(th->data_offset * 4); + bool do_output = false; + bool do_output_data = false; + tcp_seq seg_seq = th->seq; + auto seg_ack = th->ack; + auto seg_len = p.len(); + + // 4.1 first check sequence number + if (!segment_acceptable(seg_seq, seg_len)) { + //<SEQ=SND.NXT><ACK=RCV.NXT><CTL=ACK> + return output(); + } + + // In the following it is assumed that the segment is the idealized + // segment that begins at RCV.NXT and does not exceed the window. + if (seg_seq < _rcv.next) { + // ignore already acknowledged data + auto dup = std::min(uint32_t(_rcv.next - seg_seq), seg_len); + p.trim_front(dup); + seg_len -= dup; + seg_seq += dup; + } + // FIXME: We should trim data outside the right edge of the receive window as well + + if (seg_seq != _rcv.next) { + insert_out_of_order(seg_seq, std::move(p)); + // A TCP receiver SHOULD send an immediate duplicate ACK + // when an out-of-order segment arrives. + return output(); + } + + // 4.2 second check the RST bit + if (th->f_rst) { + if (in_state(SYN_RECEIVED)) { + // If this connection was initiated with a passive OPEN (i.e., + // came from the LISTEN state), then return this connection to + // LISTEN state and return. The user need not be informed. If + // this connection was initiated with an active OPEN (i.e., came + // from SYN_SENT state) then the connection was refused, signal + // the user "connection refused". In either case, all segments + // on the retransmission queue should be removed. And in the + // active OPEN case, enter the CLOSED state and delete the TCB, + // and return. + _connect_done.set_exception(tcp_refused_error()); + return do_reset(); + } + if (in_state(ESTABLISHED | FIN_WAIT_1 | FIN_WAIT_2 | CLOSE_WAIT)) { + // If the RST bit is set then, any outstanding RECEIVEs and SEND + // should receive "reset" responses. All segment queues should be + // flushed. Users should also receive an unsolicited general + // "connection reset" signal. Enter the CLOSED state, delete the + // TCB, and return. + return do_reset(); + } + if (in_state(CLOSING | LAST_ACK | TIME_WAIT)) { + // If the RST bit is set then, enter the CLOSED state, delete the + // TCB, and return. + return do_closed(); + } + } + + // 4.3 third check security and precedence + // NOTE: Ignored for now + + // 4.4 fourth, check the SYN bit + if (th->f_syn) { + // SYN_RECEIVED, ESTABLISHED, FIN_WAIT_1, FIN_WAIT_2 + // CLOSE_WAIT, CLOSING, LAST_ACK, TIME_WAIT + + // If the SYN is in the window it is an error, send a reset, any + // outstanding RECEIVEs and SEND should receive "reset" responses, + // all segment queues should be flushed, the user should also + // receive an unsolicited general "connection reset" signal, enter + // the CLOSED state, delete the TCB, and return. + respond_with_reset(th); + return do_reset(); + + // If the SYN is not in the window this step would not be reached + // and an ack would have been sent in the first step (sequence + // number check). + } + + // 4.5 fifth check the ACK field + if (!th->f_ack) { + // if the ACK bit is off drop the segment and return + return; + } else { + // SYN_RECEIVED STATE + if (in_state(SYN_RECEIVED)) { + // If SND.UNA =< SEG.ACK =< SND.NXT then enter ESTABLISHED state + // and continue processing. + if (_snd.unacknowledged <= seg_ack && seg_ack <= _snd.next) { + tcp_debug("SYN_RECEIVED -> ESTABLISHED\n"); + do_established(); + _tcp.add_connected_tcb(this->shared_from_this(), _local_port); + } else { + // <SEQ=SEG.ACK><CTL=RST> + return respond_with_reset(th); + } + } + auto update_window = [this, th, seg_seq, seg_ack] { + tcp_debug("window update seg_seq=%d, seg_ack=%d, old window=%d new window=%d\n", + seg_seq, seg_ack, _snd.window, th->window << _snd.window_scale); + _snd.window = th->window << _snd.window_scale; + _snd.wl1 = seg_seq; + _snd.wl2 = seg_ack; + _snd.zero_window_probing_out = 0; + if (_snd.window == 0) { + _persist_time_out = _rto; + start_persist_timer(); + } else { + stop_persist_timer(); + } + }; + // ESTABLISHED STATE or + // CLOSE_WAIT STATE: Do the same processing as for the ESTABLISHED state. + if (in_state(ESTABLISHED | CLOSE_WAIT)){ + // When we are in zero window probing phase and packets_out = 0 we bypass "duplicated ack" check + auto packets_out = _snd.next - _snd.unacknowledged - _snd.zero_window_probing_out; + // If SND.UNA < SEG.ACK =< SND.NXT then, set SND.UNA <- SEG.ACK. + if (_snd.unacknowledged < seg_ack && seg_ack <= _snd.next) { + // Remote ACKed data we sent + auto acked_bytes = data_segment_acked(seg_ack); + + // If SND.UNA < SEG.ACK =< SND.NXT, the send window should be updated. + if (_snd.wl1 < seg_seq || (_snd.wl1 == seg_seq && _snd.wl2 <= seg_ack)) { + update_window(); + } + + // some data is acked, try send more data + do_output_data = true; + + auto set_retransmit_timer = [this] { + if (_snd.data.empty()) { + // All outstanding segments are acked, turn off the timer. + stop_retransmit_timer(); + // Signal the waiter of this event + signal_all_data_acked(); + } else { + // Restart the timer becasue new data is acked. + start_retransmit_timer(); + } + }; + + if (_snd.dupacks >= 3) { + // We are in fast retransmit / fast recovery phase + uint32_t smss = _snd.mss; + if (seg_ack > _snd.recover) { + tcp_debug("ack: full_ack\n"); + // Set cwnd to min (ssthresh, max(FlightSize, SMSS) + SMSS) + _snd.cwnd = std::min(_snd.ssthresh, std::max(flight_size(), smss) + smss); + // Exit the fast recovery procedure + exit_fast_recovery(); + set_retransmit_timer(); + } else { + tcp_debug("ack: partial_ack\n"); + // Retransmit the first unacknowledged segment + fast_retransmit(); + // Deflate the congestion window by the amount of new data + // acknowledged by the Cumulative Acknowledgment field + _snd.cwnd -= acked_bytes; + // If the partial ACK acknowledges at least one SMSS of new + // data, then add back SMSS bytes to the congestion window + if (acked_bytes >= smss) { + _snd.cwnd += smss; + } + // Send a new segment if permitted by the new value of + // cwnd. Do not exit the fast recovery procedure For + // the first partial ACK that arrives during fast + // recovery, also reset the retransmit timer. + if (++_snd.partial_ack == 1) { + start_retransmit_timer(); + } + } + } else { + // RFC5681: The fast retransmit algorithm uses the arrival + // of 3 duplicate ACKs (as defined in section 2, without + // any intervening ACKs which move SND.UNA) as an + // indication that a segment has been lost. + // + // So, here we reset dupacks to zero becasue this ACK moves + // SND.UNA. + exit_fast_recovery(); + set_retransmit_timer(); + } + } else if ((packets_out > 0) && !_snd.data.empty() && seg_len == 0 && + th->f_fin == 0 && th->f_syn == 0 && + th->ack == _snd.unacknowledged && + uint32_t(th->window << _snd.window_scale) == _snd.window) { + // Note: + // RFC793 states: + // If the ACK is a duplicate (SEG.ACK < SND.UNA), it can be ignored + // RFC5681 states: + // The TCP sender SHOULD use the "fast retransmit" algorithm to detect + // and repair loss, based on incoming duplicate ACKs. + // Here, We follow RFC5681. + _snd.dupacks++; + uint32_t smss = _snd.mss; + // 3 duplicated ACKs trigger a fast retransmit + if (_snd.dupacks == 1 || _snd.dupacks == 2) { + // RFC5681 Step 3.1 + // Send cwnd + 2 * smss per RFC3042 + do_output_data = true; + } else if (_snd.dupacks == 3) { + // RFC6582 Step 3.2 + if (seg_ack - 1 > _snd.recover) { + _snd.recover = _snd.next - 1; + // RFC5681 Step 3.2 + _snd.ssthresh = std::max((flight_size() - _snd.limited_transfer) / 2, 2 * smss); + fast_retransmit(); + } else { + // Do not enter fast retransmit and do not reset ssthresh + } + // RFC5681 Step 3.3 + _snd.cwnd = _snd.ssthresh + 3 * smss; + } else if (_snd.dupacks > 3) { + // RFC5681 Step 3.4 + _snd.cwnd += smss; + // RFC5681 Step 3.5 + do_output_data = true; + } + } else if (seg_ack > _snd.next) { + // If the ACK acks something not yet sent (SEG.ACK > SND.NXT) + // then send an ACK, drop the segment, and return + return output(); + } else if (_snd.window == 0 && th->window > 0) { + update_window(); + do_output_data = true; + } + } + // FIN_WAIT_1 STATE + if (in_state(FIN_WAIT_1)) { + // In addition to the processing for the ESTABLISHED state, if + // our FIN is now acknowledged then enter FIN-WAIT-2 and continue + // processing in that state. + if (seg_ack == _snd.next + 1) { + tcp_debug("ack: FIN_WAIT_1 -> FIN_WAIT_2\n"); + _state = FIN_WAIT_2; + do_local_fin_acked(); + } + } + // FIN_WAIT_2 STATE + if (in_state(FIN_WAIT_2)) { + // In addition to the processing for the ESTABLISHED state, if + // the retransmission queue is empty, the user’s CLOSE can be + // acknowledged ("ok") but do not delete the TCB. + // TODO + } + // CLOSING STATE + if (in_state(CLOSING)) { + if (seg_ack == _snd.next + 1) { + tcp_debug("ack: CLOSING -> TIME_WAIT\n"); + do_local_fin_acked(); + return do_time_wait(); + } else { + return; + } + } + // LAST_ACK STATE + if (in_state(LAST_ACK)) { + if (seg_ack == _snd.next + 1) { + tcp_debug("ack: LAST_ACK -> CLOSED\n"); + do_local_fin_acked(); + return do_closed(); + } + } + // TIME_WAIT STATE + if (in_state(TIME_WAIT)) { + // The only thing that can arrive in this state is a + // retransmission of the remote FIN. Acknowledge it, and restart + // the 2 MSL timeout. + // TODO + } + } + + // 4.6 sixth, check the URG bit + if (th->f_urg) { + // TODO + } + + // 4.7 seventh, process the segment text + if (in_state(ESTABLISHED | FIN_WAIT_1 | FIN_WAIT_2)) { + if (p.len()) { + // Once the TCP takes responsibility for the data it advances + // RCV.NXT over the data accepted, and adjusts RCV.WND as + // apporopriate to the current buffer availability. The total of + // RCV.NXT and RCV.WND should not be reduced. + _rcv.data_size += p.len(); + _rcv.data.push_back(std::move(p)); + _rcv.next += seg_len; + auto merged = merge_out_of_order(); + _rcv.window = get_modified_receive_window_size(); + signal_data_received(); + // Send an acknowledgment of the form: + // <SEQ=SND.NXT><ACK=RCV.NXT><CTL=ACK> + // This acknowledgment should be piggybacked on a segment being + // transmitted if possible without incurring undue delay. + if (merged) { + // TCP receiver SHOULD send an immediate ACK when the + // incoming segment fills in all or part of a gap in the + // sequence space. + do_output = true; + } else { + do_output = should_send_ack(seg_len); + } + } + } else if (in_state(CLOSE_WAIT | CLOSING | LAST_ACK | TIME_WAIT)) { + // This should not occur, since a FIN has been received from the + // remote side. Ignore the segment text. + return; + } + + // 4.8 eighth, check the FIN bit + if (th->f_fin) { + if (_fin_recvd_promise) { + _fin_recvd_promise->set_value(); + _fin_recvd_promise.reset(); + } + if (in_state(CLOSED | LISTEN | SYN_SENT)) { + // Do not process the FIN if the state is CLOSED, LISTEN or SYN-SENT + // since the SEG.SEQ cannot be validated; drop the segment and return. + return; + } + auto fin_seq = seg_seq + seg_len; + if (fin_seq == _rcv.next) { + _rcv.next = fin_seq + 1; + signal_data_received(); + + // If this <FIN> packet contains data as well, we can ACK both data + // and <FIN> in a single packet, so canncel the previous ACK. + clear_delayed_ack(); + do_output = false; + // Send ACK for the FIN! + output(); + + if (in_state(SYN_RECEIVED | ESTABLISHED)) { + tcp_debug("fin: SYN_RECEIVED or ESTABLISHED -> CLOSE_WAIT\n"); + _state = CLOSE_WAIT; + } + if (in_state(FIN_WAIT_1)) { + // If our FIN has been ACKed (perhaps in this segment), then + // enter TIME-WAIT, start the time-wait timer, turn off the other + // timers; otherwise enter the CLOSING state. + // Note: If our FIN has been ACKed, we should be in FIN_WAIT_2 + // not FIN_WAIT_1 if we reach here. + tcp_debug("fin: FIN_WAIT_1 -> CLOSING\n"); + _state = CLOSING; + } + if (in_state(FIN_WAIT_2)) { + tcp_debug("fin: FIN_WAIT_2 -> TIME_WAIT\n"); + return do_time_wait(); + } + } + } + if (do_output || (do_output_data && can_send())) { + // Since we will do output, we can canncel scheduled delayed ACK. + clear_delayed_ack(); + output(); + } +} + +template <typename InetTraits> +packet tcp<InetTraits>::tcb::get_transmit_packet() { + // easy case: empty queue + if (_snd.unsent.empty()) { + return packet(); + } + auto can_send = this->can_send(); + // Max number of TCP payloads we can pass to NIC + uint32_t len; + if (_tcp.hw_features().tx_tso) { + // FIXME: Info tap device the size of the splitted packet + len = _tcp.hw_features().max_packet_len - net::tcp_hdr_len_min - InetTraits::ip_hdr_len_min; + } else { + len = std::min(uint16_t(_tcp.hw_features().mtu - net::tcp_hdr_len_min - InetTraits::ip_hdr_len_min), _snd.mss); + } + can_send = std::min(can_send, len); + // easy case: one small packet + if (_snd.unsent.size() == 1 && _snd.unsent.front().len() <= can_send) { + auto p = std::move(_snd.unsent.front()); + _snd.unsent.pop_front(); + _snd.unsent_len -= p.len(); + return p; + } + // moderate case: need to split one packet + if (_snd.unsent.front().len() > can_send) { + auto p = _snd.unsent.front().share(0, can_send); + _snd.unsent.front().trim_front(can_send); + _snd.unsent_len -= p.len(); + return p; + } + // hard case: merge some packets, possibly split last + auto p = std::move(_snd.unsent.front()); + _snd.unsent.pop_front(); + can_send -= p.len(); + while (!_snd.unsent.empty() + && _snd.unsent.front().len() <= can_send) { + can_send -= _snd.unsent.front().len(); + p.append(std::move(_snd.unsent.front())); + _snd.unsent.pop_front(); + } + if (!_snd.unsent.empty() && can_send) { + auto& q = _snd.unsent.front(); + p.append(q.share(0, can_send)); + q.trim_front(can_send); + } + _snd.unsent_len -= p.len(); + return p; +} + +template <typename InetTraits> +void tcp<InetTraits>::tcb::output_one(bool data_retransmit) { + if (in_state(CLOSED)) { + return; + } + + packet p = data_retransmit ? _snd.data.front().p.share() : get_transmit_packet(); + packet clone = p.share(); // early clone to prevent share() from calling packet::unuse_internal_data() on header. + uint16_t len = p.len(); + bool syn_on = syn_needs_on(); + bool ack_on = ack_needs_on(); + + auto options_size = _option.get_size(syn_on, ack_on); + auto th = p.prepend_uninitialized_header(tcp_hdr::len + options_size); + auto h = tcp_hdr{}; + + h.src_port = _local_port; + h.dst_port = _foreign_port; + + h.f_syn = syn_on; + h.f_ack = ack_on; + if (ack_on) { + clear_delayed_ack(); + } + h.f_urg = false; + h.f_psh = false; + + tcp_seq seq; + if (data_retransmit) { + seq = _snd.unacknowledged; + } else { + seq = syn_on ? _snd.initial : _snd.next; + _snd.next += len; + } + h.seq = seq; + h.ack = _rcv.next; + h.data_offset = (tcp_hdr::len + options_size) / 4; + h.window = _rcv.window >> _rcv.window_scale; + h.checksum = 0; + + // FIXME: does the FIN have to fit in the window? + bool fin_on = fin_needs_on(); + h.f_fin = fin_on; + + // Add tcp options + _option.fill(th, &h, options_size); + h.write(th); + + offload_info oi; + checksummer csum; + uint16_t pseudo_hdr_seg_len = 0; + + oi.tcp_hdr_len = tcp_hdr::len + options_size; + + if (_tcp.hw_features().tx_csum_l4_offload) { + oi.needs_csum = true; + + // + // tx checksum offloading: both virtio-net's VIRTIO_NET_F_CSUM dpdk's + // PKT_TX_TCP_CKSUM - requires th->checksum to be initialized to ones' + // complement sum of the pseudo header. + // + // For TSO the csum should be calculated for a pseudo header with + // segment length set to 0. All the rest is the same as for a TCP Tx + // CSUM offload case. + // + if (_tcp.hw_features().tx_tso && len > _snd.mss) { + oi.tso_seg_size = _snd.mss; + } else { + pseudo_hdr_seg_len = tcp_hdr::len + options_size + len; + } + } else { + pseudo_hdr_seg_len = tcp_hdr::len + options_size + len; + oi.needs_csum = false; + } + + InetTraits::tcp_pseudo_header_checksum(csum, _local_ip, _foreign_ip, + pseudo_hdr_seg_len); + + uint16_t checksum; + if (_tcp.hw_features().tx_csum_l4_offload) { + checksum = ~csum.get(); + } else { + csum.sum(p); + checksum = csum.get(); + } + tcp_hdr::write_nbo_checksum(th, checksum); + + oi.protocol = ip_protocol_num::tcp; + + p.set_offload_info(oi); + + if (!data_retransmit && (len || syn_on || fin_on)) { + auto now = clock_type::now(); + if (len) { + unsigned nr_transmits = 0; + _snd.data.emplace_back(unacked_segment{std::move(clone), + len, nr_transmits, now}); + } + if (!_retransmit.armed()) { + start_retransmit_timer(now); + } + } + + + // if advertised TCP receive window is 0 we may only transmit zero window probing segment. + // Payload size of this segment is 1. Queueing anything bigger when _snd.window == 0 is bug + // and violation of RFC + assert((_snd.window > 0) || ((_snd.window == 0) && (len <= 1))); + queue_packet(std::move(p)); +} + +template <typename InetTraits> +future<> tcp<InetTraits>::tcb::wait_for_data() { + if (!_rcv.data.empty() || foreign_will_not_send()) { + return make_ready_future<>(); + } + _rcv._data_received_promise = promise<>(); + return _rcv._data_received_promise->get_future(); +} + +template <typename InetTraits> +future<> tcp<InetTraits>::tcb::wait_input_shutdown() { + if (!_fin_recvd_promise) { + return make_ready_future<>(); + } + return _fin_recvd_promise->get_future(); +} + +template <typename InetTraits> +void +tcp<InetTraits>::tcb::abort_reader() noexcept { + 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::nullopt; + } + if (_fin_recvd_promise) { + _fin_recvd_promise->set_value(); + _fin_recvd_promise.reset(); + } +} + +template <typename InetTraits> +future<> tcp<InetTraits>::tcb::wait_for_all_data_acked() { + if (_snd.data.empty() && _snd.unsent_len == 0) { + return make_ready_future<>(); + } + _snd._all_data_acked_promise = promise<>(); + return _snd._all_data_acked_promise->get_future(); +} + +template <typename InetTraits> +void tcp<InetTraits>::tcb::connect() { + // An initial send sequence number (ISS) is selected. A SYN segment of the + // form <SEQ=ISS><CTL=SYN> is sent. Set SND.UNA to ISS, SND.NXT to ISS+1, + // enter SYN-SENT state, and return. + do_setup_isn(); + + // Local receive window scale factor + _rcv.window_scale = _option._local_win_scale = 7; + // Maximum segment size local can receive + _rcv.mss = _option._local_mss = local_mss(); + _rcv.window = get_default_receive_window_size(); + + do_syn_sent(); +} + +template <typename InetTraits> +packet tcp<InetTraits>::tcb::read() { + packet p; + for (auto&& q : _rcv.data) { + p.append(std::move(q)); + } + _rcv.data_size = 0; + _rcv.data.clear(); + _rcv.window = get_default_receive_window_size(); + return p; +} + +template <typename InetTraits> +future<> tcp<InetTraits>::tcb::wait_send_available() { + if (_snd.max_queue_space > _snd.current_queue_space) { + return make_ready_future<>(); + } + _snd._send_available_promise = promise<>(); + return _snd._send_available_promise->get_future(); +} + +template <typename InetTraits> +future<> tcp<InetTraits>::tcb::send(packet p) { + // We can not send after the connection is closed + if (_snd.closed || in_state(CLOSED)) { + return make_exception_future<>(tcp_reset_error()); + } + + auto len = p.len(); + _snd.current_queue_space += len; + _snd.unsent_len += len; + _snd.unsent.push_back(std::move(p)); + + if (can_send() > 0) { + output(); + } + + return wait_send_available(); +} + +template <typename InetTraits> +void tcp<InetTraits>::tcb::close() noexcept { + if (in_state(CLOSED) || _snd.closed) { + return; + } + // TODO: We should return a future to upper layer + (void)wait_for_all_data_acked().then([this, zis = this->shared_from_this()] () mutable { + _snd.closed = true; + tcp_debug("close: unsent_len=%d\n", _snd.unsent_len); + if (in_state(CLOSE_WAIT)) { + tcp_debug("close: CLOSE_WAIT -> LAST_ACK\n"); + _state = LAST_ACK; + } else if (in_state(ESTABLISHED)) { + tcp_debug("close: ESTABLISHED -> FIN_WAIT_1\n"); + _state = FIN_WAIT_1; + } + // Send <FIN> to remote + // Note: we call output_one to make sure a packet with FIN actually + // sent out. If we only call output() and _packetq is not empty, + // tcp::tcb::get_packet(), packet with FIN will not be generated. + output_one(); + output(); + }); +} + +template <typename InetTraits> +bool tcp<InetTraits>::tcb::should_send_ack(uint16_t seg_len) { + // We've received a TSO packet, do ack immediately + if (seg_len > _rcv.mss) { + _nr_full_seg_received = 0; + _delayed_ack.cancel(); + return true; + } + + // We've received a full sized segment, ack for every second full sized segment + if (seg_len == _rcv.mss) { + if (_nr_full_seg_received++ >= 1) { + _nr_full_seg_received = 0; + _delayed_ack.cancel(); + return true; + } + } + + // If the timer is armed and its callback hasn't been run. + if (_delayed_ack.armed()) { + return false; + } + + // If the timer is not armed, schedule a delayed ACK. + // The maximum delayed ack timer allowed by RFC1122 is 500ms, most + // implementations use 200ms. + _delayed_ack.arm(200ms); + return false; +} + +template <typename InetTraits> +void tcp<InetTraits>::tcb::clear_delayed_ack() noexcept { + _delayed_ack.cancel(); +} + +template <typename InetTraits> +bool tcp<InetTraits>::tcb::merge_out_of_order() { + bool merged = false; + if (_rcv.out_of_order.map.empty()) { + return merged; + } + for (auto it = _rcv.out_of_order.map.begin(); it != _rcv.out_of_order.map.end();) { + auto& p = it->second; + auto seg_beg = it->first; + auto seg_len = p.len(); + auto seg_end = seg_beg + seg_len; + if (seg_beg <= _rcv.next && _rcv.next < seg_end) { + // This segment has been received out of order and its previous + // segment has been received now + auto trim = _rcv.next - seg_beg; + if (trim) { + p.trim_front(trim); + seg_len -= trim; + } + _rcv.next += seg_len; + _rcv.data_size += p.len(); + _rcv.data.push_back(std::move(p)); + // Since c++11, erase() always returns the value of the following element + it = _rcv.out_of_order.map.erase(it); + merged = true; + } else if (_rcv.next >= seg_end) { + // This segment has been receive already, drop it + it = _rcv.out_of_order.map.erase(it); + } else { + // seg_beg > _rcv.need, can not merge. Note, seg_beg can grow only, + // so we can stop looking here. + it++; + break; + } + } + return merged; +} + +template <typename InetTraits> +void tcp<InetTraits>::tcb::insert_out_of_order(tcp_seq seg, packet p) { + _rcv.out_of_order.merge(seg, std::move(p)); +} + +template <typename InetTraits> +void tcp<InetTraits>::tcb::trim_receive_data_after_window() { + abort(); +} + +template <typename InetTraits> +void tcp<InetTraits>::tcb::persist() { + tcp_debug("persist timer fired\n"); + // Send 1 byte packet to probe peer's window size + _snd.window_probe = true; + _snd.zero_window_probing_out++; + output_one(); + _snd.window_probe = false; + + output(); + // Perform binary exponential back-off per RFC1122 + _persist_time_out = std::min(_persist_time_out * 2, _rto_max); + start_persist_timer(); +} + +template <typename InetTraits> +void tcp<InetTraits>::tcb::retransmit() { + auto output_update_rto = [this] { + output(); + // According to RFC6298, Update RTO <- RTO * 2 to perform binary exponential back-off + this->_rto = std::min(this->_rto * 2, this->_rto_max); + start_retransmit_timer(); + }; + + // Retransmit SYN + if (syn_needs_on()) { + if (_snd.syn_retransmit++ < _max_nr_retransmit) { + output_update_rto(); + } else { + _connect_done.set_exception(tcp_connect_error()); + cleanup(); + return; + } + } + + // Retransmit FIN + if (fin_needs_on()) { + if (_snd.fin_retransmit++ < _max_nr_retransmit) { + output_update_rto(); + } else { + cleanup(); + return; + } + } + + // Retransmit Data + if (_snd.data.empty()) { + return; + } + + // If there are unacked data, retransmit the earliest segment + auto& unacked_seg = _snd.data.front(); + + // According to RFC5681 + // Update ssthresh only for the first retransmit + uint32_t smss = _snd.mss; + if (unacked_seg.nr_transmits == 0) { + _snd.ssthresh = std::max(flight_size() / 2, 2 * smss); + } + // RFC6582 Step 4 + _snd.recover = _snd.next - 1; + // Start the slow start process + _snd.cwnd = smss; + // End fast recovery + exit_fast_recovery(); + + if (unacked_seg.nr_transmits < _max_nr_retransmit) { + unacked_seg.nr_transmits++; + } else { + // Delete connection when max num of retransmission is reached + do_reset(); + return; + } + retransmit_one(); + + output_update_rto(); +} + +template <typename InetTraits> +void tcp<InetTraits>::tcb::fast_retransmit() { + if (!_snd.data.empty()) { + auto& unacked_seg = _snd.data.front(); + unacked_seg.nr_transmits++; + retransmit_one(); + output(); + } +} + +template <typename InetTraits> +void tcp<InetTraits>::tcb::update_rto(clock_type::time_point tx_time) { + // Update RTO according to RFC6298 + auto R = std::chrono::duration_cast<std::chrono::milliseconds>(clock_type::now() - tx_time); + if (_snd.first_rto_sample) { + _snd.first_rto_sample = false; + // RTTVAR <- R/2 + // SRTT <- R + _snd.rttvar = R / 2; + _snd.srtt = R; + } else { + // RTTVAR <- (1 - beta) * RTTVAR + beta * |SRTT - R'| + // SRTT <- (1 - alpha) * SRTT + alpha * R' + // where alpha = 1/8 and beta = 1/4 + auto delta = _snd.srtt > R ? (_snd.srtt - R) : (R - _snd.srtt); + _snd.rttvar = _snd.rttvar * 3 / 4 + delta / 4; + _snd.srtt = _snd.srtt * 7 / 8 + R / 8; + } + // RTO <- SRTT + max(G, K * RTTVAR) + _rto = _snd.srtt + std::max(_rto_clk_granularity, 4 * _snd.rttvar); + + // Make sure 1 sec << _rto << 60 sec + _rto = std::max(_rto, _rto_min); + _rto = std::min(_rto, _rto_max); +} + +template <typename InetTraits> +void tcp<InetTraits>::tcb::update_cwnd(uint32_t acked_bytes) { + uint32_t smss = _snd.mss; + if (_snd.cwnd < _snd.ssthresh) { + // In slow start phase + _snd.cwnd += std::min(acked_bytes, smss); + } else { + // In congestion avoidance phase + uint32_t round_up = 1; + _snd.cwnd += std::max(round_up, smss * smss / _snd.cwnd); + } +} + +template <typename InetTraits> +void tcp<InetTraits>::tcb::cleanup() { + _snd.unsent.clear(); + _snd.data.clear(); + _rcv.out_of_order.map.clear(); + _rcv.data_size = 0; + _rcv.data.clear(); + stop_retransmit_timer(); + clear_delayed_ack(); + remove_from_tcbs(); +} + +template <typename InetTraits> +tcp_seq tcp<InetTraits>::tcb::get_isn() { + // Per RFC6528, TCP SHOULD generate its Initial Sequence Numbers + // with the expression: + // ISN = M + F(localip, localport, remoteip, remoteport, secretkey) + // M is the 4 microsecond timer + using namespace std::chrono; + uint32_t hash[4]; + hash[0] = _local_ip.ip; + hash[1] = _foreign_ip.ip; + hash[2] = (_local_port << 16) + _foreign_port; + hash[3] = _isn_secret.key[15]; + CryptoPP::Weak::MD5::Transform(hash, _isn_secret.key); + auto seq = hash[0]; + auto m = duration_cast<microseconds>(clock_type::now().time_since_epoch()); + seq += m.count() / 4; + return make_seq(seq); +} + +template <typename InetTraits> +std::optional<typename InetTraits::l4packet> tcp<InetTraits>::tcb::get_packet() { + _poll_active = false; + if (_packetq.empty()) { + output_one(); + } + + if (in_state(CLOSED)) { + return std::optional<typename InetTraits::l4packet>(); + } + + assert(!_packetq.empty()); + + auto p = std::move(_packetq.front()); + _packetq.pop_front(); + if (!_packetq.empty() || (_snd.dupacks < 3 && can_send() > 0 && (_snd.window > 0))) { + // If there are packets to send in the queue or tcb is allowed to send + // more add tcp back to polling set to keep sending. In addition, dupacks >= 3 + // is an indication that an segment is lost, stop sending more in this case. + // Finally - we can't send more until window is opened again. + output(); + } + return p; +} + +template <typename InetTraits> +void tcp<InetTraits>::connection::close_read() noexcept { + _tcb->abort_reader(); +} + +template <typename InetTraits> +void tcp<InetTraits>::connection::close_write() noexcept { + _tcb->close(); +} + +template <typename InetTraits> +void tcp<InetTraits>::connection::shutdown_connect() { + if (_tcb->syn_needs_on()) { + _tcb->_connect_done.set_exception(tcp_refused_error()); + _tcb->cleanup(); + } else { + close_read(); + close_write(); + } +} + +template <typename InetTraits> +constexpr uint16_t tcp<InetTraits>::tcb::_max_nr_retransmit; + +template <typename InetTraits> +constexpr std::chrono::milliseconds tcp<InetTraits>::tcb::_rto_min; + +template <typename InetTraits> +constexpr std::chrono::milliseconds tcp<InetTraits>::tcb::_rto_max; + +template <typename InetTraits> +constexpr std::chrono::milliseconds tcp<InetTraits>::tcb::_rto_clk_granularity; + +template <typename InetTraits> +typename tcp<InetTraits>::tcb::isn_secret tcp<InetTraits>::tcb::_isn_secret; + +} + +} diff --git a/src/seastar/include/seastar/net/tls.hh b/src/seastar/include/seastar/net/tls.hh new file mode 100644 index 000000000..4e178d027 --- /dev/null +++ b/src/seastar/include/seastar/net/tls.hh @@ -0,0 +1,359 @@ +/* + * 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 2015 Cloudius Systems + */ +#pragma once + +#include <functional> +#include <unordered_set> +#include <map> + +#include <boost/any.hpp> + +#include <seastar/core/future.hh> +#include <seastar/core/internal/api-level.hh> +#include <seastar/core/sstring.hh> +#include <seastar/core/shared_ptr.hh> +#include <seastar/net/socket_defs.hh> +#include <seastar/util/std-compat.hh> +#include <seastar/net/api.hh> + +namespace seastar { + +class socket; + +class server_socket; +class connected_socket; +class socket_address; + +/** + * Relatively thin SSL wrapper for socket IO. + * (Can be expanded to other IO forms). + * + * The current underlying mechanism is + * gnutls, however, all interfaces are kept + * agnostic, so in theory it could be replaced + * with OpenSSL or similar. + * + */ +namespace tls { + enum class x509_crt_format { + DER, + PEM, + }; + + typedef std::basic_string_view<char> blob; + + class session; + class server_session; + class server_credentials; + class certificate_credentials; + class credentials_builder; + + /** + * Diffie-Hellman parameters for + * wire encryption. + */ + class dh_params { + public: + // Key strength + enum class level { + LEGACY = 2, + MEDIUM = 3, + HIGH = 4, + ULTRA = 5 + }; + dh_params(level = level::LEGACY); + // loads a key from data + dh_params(const blob&, x509_crt_format); + ~dh_params(); + + dh_params(dh_params&&) noexcept; + dh_params& operator=(dh_params&&) noexcept; + + dh_params(const dh_params&) = delete; + dh_params& operator=(const dh_params&) = delete; + + /** loads a key from file */ + static future<dh_params> from_file(const sstring&, x509_crt_format); + private: + class impl; + friend class server_credentials; + friend class certificate_credentials; + std::unique_ptr<impl> _impl; + }; + + class x509_cert { + x509_cert(const blob&, x509_crt_format); + + static future<x509_cert> from_file(const sstring&, x509_crt_format); + private: + class impl; + x509_cert(shared_ptr<impl>); + shared_ptr<impl> _impl; + }; + + class abstract_credentials { + public: + virtual ~abstract_credentials() {}; + + virtual void set_x509_trust(const blob&, x509_crt_format) = 0; + virtual void set_x509_crl(const blob&, x509_crt_format) = 0; + virtual void set_x509_key(const blob& cert, const blob& key, x509_crt_format) = 0; + + virtual void set_simple_pkcs12(const blob&, x509_crt_format, const sstring& password) = 0; + + virtual future<> set_x509_trust_file(const sstring& cafile, x509_crt_format); + virtual future<> set_x509_crl_file(const sstring& crlfile, x509_crt_format); + virtual future<> set_x509_key_file(const sstring& cf, const sstring& kf, x509_crt_format); + + virtual future<> set_simple_pkcs12_file(const sstring& pkcs12file, x509_crt_format, const sstring& password); + }; + + template<typename Base> + class reloadable_credentials; + + /** + * Enum like tls::session::type but independent of gnutls headers + * + * \warning Uses a different internal encoding than tls::session::type + */ + enum class session_type { + CLIENT, SERVER, + }; + + /** + * Callback prototype for receiving Distinguished Name (DN) information + * + * \param type Our own role in the TLS handshake (client vs. server) + * \param subject The subject DN string + * \param issuer The issuer DN string + */ + using dn_callback = noncopyable_function<void(session_type type, sstring subject, sstring issuer)>; + + /** + * Holds certificates and keys. + * + * Typically, credentials are shared for multiple client/server + * sessions. Changes to the credentials object will affect all + * sessions instantiated with it. + * You should probably set it up once, before starting client/server + * connections. + */ + class certificate_credentials : public abstract_credentials { + public: + certificate_credentials(); + ~certificate_credentials(); + + certificate_credentials(certificate_credentials&&) noexcept; + certificate_credentials& operator=(certificate_credentials&&) noexcept; + + certificate_credentials(const certificate_credentials&) = delete; + certificate_credentials& operator=(const certificate_credentials&) = delete; + + void set_x509_trust(const blob&, x509_crt_format) override; + void set_x509_crl(const blob&, x509_crt_format) override; + void set_x509_key(const blob& cert, const blob& key, x509_crt_format) override; + void set_simple_pkcs12(const blob&, x509_crt_format, const sstring& password) override; + + /** + * Loads default system cert trust file + * into this object. + */ + future<> set_system_trust(); + + // TODO add methods for certificate verification + + /** + * TLS handshake priority string. See gnutls docs and syntax at + * https://gnutls.org/manual/html_node/Priority-Strings.html + * + * Allows specifying order and allowance for handshake alg. + */ + void set_priority_string(const sstring&); + + /** + * Register a callback for receiving Distinguished Name (DN) information + * during the TLS handshake, extracted from the certificate as sent by the peer. + * + * The callback is not invoked in case the peer did not send a certificate. + * (This could e.g. happen when we are the server, and a client connects while + * client_auth is not set to REQUIRE.) + * + * If, based upon the extracted DN information, you want to abort the handshake, + * then simply throw an exception (e.g., from the callback) like verification_error. + * + * Registering this callback does not bypass the 'standard' certificate verification + * procedure; instead it merely extracts the DN information from the peer certificate + * (i.e., the 'leaf' certificate from the chain of certificates sent by the peer) + * and allows for extra checks. + * + * To keep the API simple, you can unregister the callback by means of registering + * an empty callback, i.e. dn_callback{} + * + * The callback prototype is documented in the dn_callback typedef. + */ + void set_dn_verification_callback(dn_callback); + + private: + class impl; + friend class session; + friend class server_session; + friend class server_credentials; + friend class credentials_builder; + template<typename Base> + friend class reloadable_credentials; + shared_ptr<impl> _impl; + }; + + /** Exception thrown on certificate validation error */ + class verification_error : public std::runtime_error { + public: + using runtime_error::runtime_error; + }; + + enum class client_auth { + NONE, REQUEST, REQUIRE + }; + + /** + * Extending certificates and keys for server usage. + * More probably goes in here... + */ + class server_credentials : public certificate_credentials { + public: + server_credentials(); + server_credentials(shared_ptr<dh_params>); + server_credentials(const dh_params&); + + server_credentials(server_credentials&&) noexcept; + server_credentials& operator=(server_credentials&&) noexcept; + + server_credentials(const server_credentials&) = delete; + server_credentials& operator=(const server_credentials&) = delete; + + void set_client_auth(client_auth); + }; + + class reloadable_credentials_base; + + using reload_callback = std::function<void(const std::unordered_set<sstring>&, std::exception_ptr)>; + + /** + * Intentionally "primitive", and more importantly, copyable + * container for certificate credentials options. + * The intendend use case is to be able to use across shards, + * at, say, initialization of tls objects + * + * Note that loading invalid objects (malformed certs etc) will + * _not_ generate exceptions until, earliest, the build functions + * are called. + */ + class credentials_builder : public abstract_credentials { + public: + void set_dh_level(dh_params::level = dh_params::level::LEGACY); + + void set_x509_trust(const blob&, x509_crt_format) override ; + void set_x509_crl(const blob&, x509_crt_format) override; + void set_x509_key(const blob& cert, const blob& key, x509_crt_format) override; + void set_simple_pkcs12(const blob&, x509_crt_format, const sstring& password) override; + + future<> set_x509_trust_file(const sstring& cafile, x509_crt_format) override; + future<> set_x509_crl_file(const sstring& crlfile, x509_crt_format) override; + future<> set_x509_key_file(const sstring& cf, const sstring& kf, x509_crt_format) override; + future<> set_simple_pkcs12_file(const sstring& pkcs12file, x509_crt_format, const sstring& password) override; + + future<> set_system_trust(); + void set_client_auth(client_auth); + void set_priority_string(const sstring&); + + void apply_to(certificate_credentials&) const; + + shared_ptr<certificate_credentials> build_certificate_credentials() const; + shared_ptr<server_credentials> build_server_credentials() const; + + // same as above, but any files used for certs/keys etc will be watched + // for modification and reloaded if changed + future<shared_ptr<certificate_credentials>> build_reloadable_certificate_credentials(reload_callback = {}, std::optional<std::chrono::milliseconds> tolerance = {}) const; + future<shared_ptr<server_credentials>> build_reloadable_server_credentials(reload_callback = {}, std::optional<std::chrono::milliseconds> tolerance = {}) const; + private: + friend class reloadable_credentials_base; + + std::multimap<sstring, boost::any> _blobs; + client_auth _client_auth = client_auth::NONE; + sstring _priority; + }; + + /** + * Creates a TLS client connection using the default network stack and the + * supplied credentials. + * Typically these should contain enough information + * to validate the remote certificate (i.e. trust info). + * + * \param name An optional expected server name for the remote end point + */ + /// @{ + future<connected_socket> connect(shared_ptr<certificate_credentials>, socket_address, sstring name = {}); + future<connected_socket> connect(shared_ptr<certificate_credentials>, socket_address, socket_address local, sstring name = {}); + /// @} + + /** + * Creates a socket through which a TLS client connection can be created, + * using the default network stack and the supplied credentials. + * Typically these should contain enough information + * to validate the remote certificate (i.e. trust info). + * + * \param name An optional expected server name for the remote end point + */ + /// @{ + ::seastar::socket socket(shared_ptr<certificate_credentials>, sstring name = {}); + /// @} + + /** Wraps an existing connection in SSL/TLS. */ + /// @{ + future<connected_socket> wrap_client(shared_ptr<certificate_credentials>, connected_socket&&, sstring name = {}); + future<connected_socket> wrap_server(shared_ptr<server_credentials>, connected_socket&&); + /// @} + + /** + * Creates a server socket that accepts SSL/TLS clients using default network stack + * and the supplied credentials. + * The credentials object should contain certificate info + * for the server and optionally trust/crl data. + */ + /// @{ + server_socket listen(shared_ptr<server_credentials>, socket_address sa, listen_options opts = listen_options()); + // Wraps an existing server socket in SSL + server_socket listen(shared_ptr<server_credentials>, server_socket); + /// @} + + /** + * Get distinguished name from the leaf certificate in the certificate chain that + * the connected peer is using. + * This function forces the TLS handshake. If the handshake didn't happen before the + * call to 'get_dn_information' it will be completed when the returned future will become + * ready. + * The function returns DN information on success. If the peer didn't send the certificate + * during the handshake the function returns nullopt. If the socket is not connected the + * system_error exception will be thrown. + */ + future<std::optional<session_dn>> get_dn_information(connected_socket& socket); +} +} + diff --git a/src/seastar/include/seastar/net/toeplitz.hh b/src/seastar/include/seastar/net/toeplitz.hh new file mode 100644 index 000000000..01e1e8370 --- /dev/null +++ b/src/seastar/include/seastar/net/toeplitz.hh @@ -0,0 +1,99 @@ +/* + * 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) 2010 David Malone <dwmalone@FreeBSD.org> + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions + * are met: + * 1. Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * 2. Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in the + * documentation and/or other materials provided with the distribution. + * + * THIS SOFTWARE IS PROVIDED BY THE AUTHOR AND CONTRIBUTORS ``AS IS'' AND + * ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + * ARE DISCLAIMED. IN NO EVENT SHALL THE AUTHOR OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS + * OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) + * HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT + * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY + * OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF + * SUCH DAMAGE. + */ + +#pragma once + +#include <vector> + +namespace seastar { + +using rss_key_type = std::basic_string_view<uint8_t>; + +// Mellanox Linux's driver key +static constexpr uint8_t default_rsskey_40bytes_v[] = { + 0xd1, 0x81, 0xc6, 0x2c, 0xf7, 0xf4, 0xdb, 0x5b, + 0x19, 0x83, 0xa2, 0xfc, 0x94, 0x3e, 0x1a, 0xdb, + 0xd9, 0x38, 0x9e, 0x6b, 0xd1, 0x03, 0x9c, 0x2c, + 0xa7, 0x44, 0x99, 0xad, 0x59, 0x3d, 0x56, 0xd9, + 0xf3, 0x25, 0x3c, 0x06, 0x2a, 0xdc, 0x1f, 0xfc +}; + +static constexpr rss_key_type default_rsskey_40bytes{default_rsskey_40bytes_v, sizeof(default_rsskey_40bytes_v)}; + +// Intel's i40e PMD default RSS key +static constexpr uint8_t default_rsskey_52bytes_v[] = { + 0x44, 0x39, 0x79, 0x6b, 0xb5, 0x4c, 0x50, 0x23, + 0xb6, 0x75, 0xea, 0x5b, 0x12, 0x4f, 0x9f, 0x30, + 0xb8, 0xa2, 0xc0, 0x3d, 0xdf, 0xdc, 0x4d, 0x02, + 0xa0, 0x8c, 0x9b, 0x33, 0x4a, 0xf6, 0x4a, 0x4c, + 0x05, 0xc6, 0xfa, 0x34, 0x39, 0x58, 0xd8, 0x55, + 0x7d, 0x99, 0x58, 0x3a, 0xe1, 0x38, 0xc9, 0x2e, + 0x81, 0x15, 0x03, 0x66 +}; + +static constexpr rss_key_type default_rsskey_52bytes{default_rsskey_52bytes_v, sizeof(default_rsskey_52bytes_v)}; + +template<typename T> +static inline uint32_t +toeplitz_hash(rss_key_type key, const T& data) +{ + uint32_t hash = 0, v; + u_int i, b; + + /* XXXRW: Perhaps an assertion about key length vs. data length? */ + + v = (key[0]<<24) + (key[1]<<16) + (key[2] <<8) + key[3]; + for (i = 0; i < data.size(); i++) { + for (b = 0; b < 8; b++) { + if (data[i] & (1<<(7-b))) + hash ^= v; + v <<= 1; + if ((i + 4) < key.size() && + (key[i+4] & (1<<(7-b)))) + v |= 1; + } + } + return (hash); +} + +} diff --git a/src/seastar/include/seastar/net/udp.hh b/src/seastar/include/seastar/net/udp.hh new file mode 100644 index 000000000..28f447970 --- /dev/null +++ b/src/seastar/include/seastar/net/udp.hh @@ -0,0 +1,59 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + * + */ + +#pragma once + +#include <unordered_map> +#include <assert.h> +#include <seastar/core/shared_ptr.hh> +#include <seastar/net/api.hh> +#include <seastar/net/const.hh> +#include <seastar/net/net.hh> + +namespace seastar { + +namespace net { + +struct udp_hdr { + packed<uint16_t> src_port; + packed<uint16_t> dst_port; + packed<uint16_t> len; + packed<uint16_t> cksum; + + template<typename Adjuster> + auto adjust_endianness(Adjuster a) { + return a(src_port, dst_port, len, cksum); + } +} __attribute__((packed)); + +struct udp_channel_state { + queue<udp_datagram> _queue; + // Limit number of data queued into send queue + semaphore _user_queue_space = {212992}; + udp_channel_state(size_t queue_size) : _queue(queue_size) {} + future<> wait_for_send_buffer(size_t len) { return _user_queue_space.wait(len); } + void complete_send(size_t len) { _user_queue_space.signal(len); } +}; + +} + +} diff --git a/src/seastar/include/seastar/net/unix_address.hh b/src/seastar/include/seastar/net/unix_address.hh new file mode 100644 index 000000000..b177cfbfc --- /dev/null +++ b/src/seastar/include/seastar/net/unix_address.hh @@ -0,0 +1,75 @@ +/* + * 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) 2019 Red Hat, Inc. + */ +#pragma once + +#include <iosfwd> +#include <sys/types.h> +#include <sys/un.h> +#include <string> + +namespace seastar { + +/*! + A helper struct for creating/manipulating UNIX-domain sockets. + + A UNIX-domain socket is either named or unnamed. If named, the name is either + a path in the filesystem namespace, or an abstract-domain identifier. Abstract-domain + names start with a null byte, and may contain non-printable characters. + + std::string() can hold a sequence of arbitrary bytes, and has a length() attribute + that does not rely on using strlen(). Thus it is used here to hold the address. + */ +struct unix_domain_addr { + const std::string name; + const int path_count; // either name.length() or name.length()+1. See path_length_aux() below. + + explicit unix_domain_addr(const std::string& fn) : name{fn}, path_count{path_length_aux()} {} + + explicit unix_domain_addr(const char* fn) : name{fn}, path_count{path_length_aux()} {} + + int path_length() const { return path_count; } + + // the following holds: + // for abstract name: name.length() == number of meaningful bytes, including the null in name[0]. + // for filesystem path: name.length() does not count the implicit terminating null. + // Here we tweak the outside-visible length of the address. + int path_length_aux() const { + auto pl = (int)name.length(); + if (!pl || (name[0] == '\0')) { + // unnamed, or abstract-namespace + return pl; + } + return 1 + pl; + } + + const char* path_bytes() const { return name.c_str(); } + + bool operator==(const unix_domain_addr& a) const { + return name == a.name; + } + bool operator!=(const unix_domain_addr& a) const { + return !(*this == a); + } +}; + +std::ostream& operator<<(std::ostream&, const unix_domain_addr&); + +} // namespace seastar diff --git a/src/seastar/include/seastar/net/virtio-interface.hh b/src/seastar/include/seastar/net/virtio-interface.hh new file mode 100644 index 000000000..367806f01 --- /dev/null +++ b/src/seastar/include/seastar/net/virtio-interface.hh @@ -0,0 +1,131 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + * + */ + +#pragma once + +#ifndef _LINUX_VIRTIO_RING_H +#define _LINUX_VIRTIO_RING_H + +/* An interface for efficient virtio implementation, currently for use by KVM + * and lguest, but hopefully others soon. Do NOT change this since it will + * break existing servers and clients. + * + * This header is BSD licensed so anyone can use the definitions to implement + * compatible drivers/servers. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions + * are met: + * 1. Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * 2. Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in the + * documentation and/or other materials provided with the distribution. + * 3. Neither the name of IBM nor the names of its contributors + * may be used to endorse or promote products derived from this software + * without specific prior written permission. + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS ``AS IS'' AND + * ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + * ARE DISCLAIMED. IN NO EVENT SHALL IBM OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS + * OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) + * HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT + * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY + * OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF + * SUCH DAMAGE. + * + * Copyright Rusty Russell IBM Corporation 2007. */ + +/* This marks a buffer as continuing via the next field. */ +#define VRING_DESC_F_NEXT 1 +/* This marks a buffer as write-only (otherwise read-only). */ +#define VRING_DESC_F_WRITE 2 +/* This means the buffer contains a list of buffer descriptors. */ +#define VRING_DESC_F_INDIRECT 4 + +/* The Host uses this in used->flags to advise the Guest: don't kick me when + * you add a buffer. It's unreliable, so it's simply an optimization. Guest + * will still kick if it's out of buffers. */ +#define VRING_USED_F_NO_NOTIFY 1 +/* The Guest uses this in avail->flags to advise the Host: don't interrupt me + * when you consume a buffer. It's unreliable, so it's simply an + * optimization. */ +#define VRING_AVAIL_F_NO_INTERRUPT 1 + +/* We support indirect buffer descriptors */ +#define VIRTIO_RING_F_INDIRECT_DESC (1 << 28) + +/* The Guest publishes the used index for which it expects an interrupt + * at the end of the avail ring. Host should ignore the avail->flags field. */ +/* The Host publishes the avail index for which it expects a kick + * at the end of the used ring. Guest should ignore the used->flags field. */ +#define VIRTIO_RING_F_EVENT_IDX (1 << 29) + +/* The standard layout for the ring is a continuous chunk of memory which looks + * like this. We assume num is a power of 2. + * + * struct vring + * { + * // The actual descriptors (16 bytes each) + * struct vring_desc desc[num]; + * + * // A ring of available descriptor heads with free-running index. + * uint16_t avail_flags; + * uint16_t avail_idx; + * uint16_t available[num]; + * uint16_t used_event_idx; + * + * // Padding to the next align boundary. + * char pad[]; + * + * // A ring of used descriptor heads with free-running index. + * uint16_t used_flags; + * uint16_t used_idx; + * struct vring_used_elem used[num]; + * uint16_t avail_event_idx; + * }; + */ + +#endif + +#define VIRTIO_NET_F_CSUM (1 << 0) +#define VIRTIO_NET_F_GUEST_CSUM (1 << 1) +#define VIRTIO_NET_F_CTRL_GUEST_OFFLOADS (1 << 2) +#define VIRTIO_NET_F_MAC (1 << 5) +#define VIRTIO_NET_F_GUEST_TSO4 (1 << 7) +#define VIRTIO_NET_F_GUEST_TSO6 (1 << 8) +#define VIRTIO_NET_F_GUEST_ECN (1 << 9) +#define VIRTIO_NET_F_GUEST_UFO (1 << 10) +#define VIRTIO_NET_F_HOST_TSO4 (1 << 11) +#define VIRTIO_NET_F_HOST_TSO6 (1 << 12) +#define VIRTIO_NET_F_HOST_ECN (1 << 13) +#define VIRTIO_NET_F_HOST_UFO (1 << 14) +#define VIRTIO_NET_F_MRG_RXBUF (1 << 15) +#define VIRTIO_NET_F_STATUS (1 << 16) +#define VIRTIO_NET_F_CTRL_VQ (1 << 17) +#define VIRTIO_NET_F_CTRL_RX (1 << 18) +#define VIRTIO_NET_F_CTRL_VLAN (1 << 19) +#define VIRTIO_NET_F_GUEST_ANNOUNCE (1 << 21) +#define VIRTIO_NET_F_MQ (1 << 22) +#define VIRTIO_NET_F_CTRL_MAC_ADDR (1 << 23) diff --git a/src/seastar/include/seastar/net/virtio.hh b/src/seastar/include/seastar/net/virtio.hh new file mode 100644 index 000000000..3e3aa95ce --- /dev/null +++ b/src/seastar/include/seastar/net/virtio.hh @@ -0,0 +1,67 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <memory> +#include <seastar/net/net.hh> +#include <seastar/core/sstring.hh> +#include <seastar/util/program-options.hh> + +namespace seastar { + +namespace net { + +/// Virtio configuration. +struct virtio_options : public program_options::option_group { + /// \brief Enable event-index feature (on / off). + /// + /// Default: \p on. + program_options::value<std::string> event_index; + /// \brief Enable checksum offload feature (on / off). + /// + /// Default: \p on. + program_options::value<std::string> csum_offload; + /// \brief Enable TCP segment offload feature (on / off). + /// + /// Default: \p on. + program_options::value<std::string> tso; + /// \brief Enable UDP fragmentation offload feature (on / off). + /// + /// Default: \p on. + program_options::value<std::string> ufo; + /// \brief Virtio ring size (must be power-of-two). + /// + /// Default: 256. + program_options::value<unsigned> virtio_ring_size; + + /// \cond internal + virtio_options(program_options::option_group* parent_group); + /// \endcond +}; + +} + +/// \cond internal +std::unique_ptr<net::device> create_virtio_net_device(const net::virtio_options& opts, const program_options::value<std::string>& lro); +/// \endcond + +} diff --git a/src/seastar/include/seastar/rpc/lz4_compressor.hh b/src/seastar/include/seastar/rpc/lz4_compressor.hh new file mode 100644 index 000000000..3a4523623 --- /dev/null +++ b/src/seastar/include/seastar/rpc/lz4_compressor.hh @@ -0,0 +1,48 @@ +/* + * 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) 2016 Scylladb, Ltd. + */ + +#pragma once + +#include <seastar/core/sstring.hh> +#include <seastar/rpc/rpc_types.hh> +#include <lz4.h> + +namespace seastar { + +namespace rpc { + class lz4_compressor : public compressor { + public: + class factory: public rpc::compressor::factory { + public: + virtual const sstring& supported() const override; + virtual std::unique_ptr<rpc::compressor> negotiate(sstring feature, bool is_server) const override; + }; + public: + ~lz4_compressor() {} + // compress data, leaving head_space empty in returned buffer + snd_buf compress(size_t head_space, snd_buf data) override; + // decompress data + rcv_buf decompress(rcv_buf data) override; + sstring name() const override; + }; +} + +} diff --git a/src/seastar/include/seastar/rpc/lz4_fragmented_compressor.hh b/src/seastar/include/seastar/rpc/lz4_fragmented_compressor.hh new file mode 100644 index 000000000..a3ad29705 --- /dev/null +++ b/src/seastar/include/seastar/rpc/lz4_fragmented_compressor.hh @@ -0,0 +1,44 @@ +/* + * 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) 2019 Scylladb, Ltd. + */ + +#pragma once + +#include <seastar/core/sstring.hh> +#include <seastar/rpc/rpc_types.hh> + +namespace seastar { +namespace rpc { + +class lz4_fragmented_compressor final : public compressor { +public: + class factory final : public rpc::compressor::factory { + public: + virtual const sstring& supported() const override; + virtual std::unique_ptr<rpc::compressor> negotiate(sstring feature, bool is_server) const override; + }; +public: + virtual snd_buf compress(size_t head_space, snd_buf data) override; + virtual rcv_buf decompress(rcv_buf data) override; + sstring name() const override; +}; + +} +} diff --git a/src/seastar/include/seastar/rpc/multi_algo_compressor_factory.hh b/src/seastar/include/seastar/rpc/multi_algo_compressor_factory.hh new file mode 100644 index 000000000..7a4380dc5 --- /dev/null +++ b/src/seastar/include/seastar/rpc/multi_algo_compressor_factory.hh @@ -0,0 +1,80 @@ +/* + * 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) 2016 Scylladb, Ltd. + */ + +#pragma once + +#include <boost/range/adaptor/transformed.hpp> +#include <boost/algorithm/string.hpp> +#include <seastar/core/sstring.hh> +#include <seastar/rpc/rpc_types.hh> + +namespace seastar { + +namespace rpc { + +// This is meta compressor factory. It gets an array of regular factories that +// support one compression algorithm each and negotiates common compression algorithm +// that is supported both by a client and a server. The order of algorithm preferences +// is the order they appear in clien's list +class multi_algo_compressor_factory : public rpc::compressor::factory { + std::vector<const rpc::compressor::factory*> _factories; + sstring _features; + +public: + multi_algo_compressor_factory(std::vector<const rpc::compressor::factory*> factories) : _factories(std::move(factories)) { + _features = boost::algorithm::join(_factories | boost::adaptors::transformed(std::mem_fn(&rpc::compressor::factory::supported)), sstring(",")); + } + multi_algo_compressor_factory(std::initializer_list<const rpc::compressor::factory*> factories) : + multi_algo_compressor_factory(std::vector<const rpc::compressor::factory*>(std::move(factories))) {} + multi_algo_compressor_factory(const rpc::compressor::factory* factory) : multi_algo_compressor_factory({factory}) {} + // return feature string that will be sent as part of protocol negotiation + virtual const sstring& supported() const { + return _features; + } + // negotiate compress algorithm + virtual std::unique_ptr<compressor> negotiate(sstring feature, bool is_server) const { + std::vector<sstring> names; + boost::split(names, feature, boost::is_any_of(",")); + std::unique_ptr<compressor> c; + if (is_server) { + for (auto&& n : names) { + for (auto&& f : _factories) { + if ((c = f->negotiate(n, is_server))) { + return c; + } + } + } + } else { + for (auto&& f : _factories) { + for (auto&& n : names) { + if ((c = f->negotiate(n, is_server))) { + return c; + } + } + } + } + return nullptr; + } +}; + +} + +} diff --git a/src/seastar/include/seastar/rpc/rpc.hh b/src/seastar/include/seastar/rpc/rpc.hh new file mode 100644 index 000000000..f3fff61da --- /dev/null +++ b/src/seastar/include/seastar/rpc/rpc.hh @@ -0,0 +1,905 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <unordered_map> +#include <unordered_set> +#include <list> +#include <variant> +#include <boost/intrusive/list.hpp> +#include <seastar/core/future.hh> +#include <seastar/core/seastar.hh> +#include <seastar/net/api.hh> +#include <seastar/core/iostream.hh> +#include <seastar/core/shared_ptr.hh> +#include <seastar/core/condition-variable.hh> +#include <seastar/core/gate.hh> +#include <seastar/rpc/rpc_types.hh> +#include <seastar/core/byteorder.hh> +#include <seastar/core/shared_future.hh> +#include <seastar/core/queue.hh> +#include <seastar/core/weak_ptr.hh> +#include <seastar/core/scheduling.hh> +#include <seastar/util/backtrace.hh> +#include <seastar/util/log.hh> + +namespace bi = boost::intrusive; + +namespace seastar { + +namespace rpc { + +/// \defgroup rpc rpc - remote procedure call framework +/// +/// \brief +/// rpc is a framework that can be used to define client-server communication +/// protocols. +/// For a high-level description of the RPC features see +/// [doc/rpc.md](./md_rpc.html), +/// [doc/rpc-streaming.md](./md_rpc-streaming.html) and +/// [doc/rpc-compression.md](./md_rpc-compression.html) +/// +/// The entry point for setting up an rpc protocol is +/// seastar::rpc::protocol. + +using id_type = int64_t; + +using rpc_semaphore = basic_semaphore<semaphore_default_exception_factory, rpc_clock_type>; +using resource_permit = semaphore_units<semaphore_default_exception_factory, rpc_clock_type>; + +static constexpr char rpc_magic[] = "SSTARRPC"; + +/// \addtogroup rpc +/// @{ + +/// Specifies resource isolation for a connection. +struct isolation_config { + /// Specifies a scheduling group under which the connection (and all its + /// verb handlers) will execute. + scheduling_group sched_group = current_scheduling_group(); +}; + +/// Default isolation configuration - run everything in the default scheduling group. +/// +/// In the scheduling_group that the protocol::server was created in. +isolation_config default_isolate_connection(sstring isolation_cookie); + +/// \brief Resource limits for an RPC server +/// +/// A request's memory use will be estimated as +/// +/// req_mem = basic_request_size * sizeof(serialized_request) * bloat_factor +/// +/// Concurrent requests will be limited so that +/// +/// sum(req_mem) <= max_memory +/// +/// \see server +struct resource_limits { + size_t basic_request_size = 0; ///< Minimum request footprint in memory + unsigned bloat_factor = 1; ///< Serialized size multiplied by this to estimate memory used by request + size_t max_memory = rpc_semaphore::max_counter(); ///< Maximum amount of memory that may be consumed by all requests + /// Configures isolation for a connection based on its isolation cookie. May throw, + /// in which case the connection will be terminated. + using syncronous_isolation_function = std::function<isolation_config (sstring isolation_cookie)>; + using asyncronous_isolation_function = std::function<future<isolation_config> (sstring isolation_cookie)>; + using isolation_function_alternatives = std::variant<syncronous_isolation_function, asyncronous_isolation_function>; + isolation_function_alternatives isolate_connection = default_isolate_connection; +}; + +struct client_options { + std::optional<net::tcp_keepalive_params> keepalive; + bool tcp_nodelay = true; + bool reuseaddr = false; + compressor::factory* compressor_factory = nullptr; + bool send_timeout_data = true; + connection_id stream_parent = invalid_connection_id; + /// Configures how this connection is isolated from other connection on the same server. + /// + /// \see resource_limits::isolate_connection + sstring isolation_cookie; +}; + +/// @} + +// RPC call that passes stream connection id as a parameter +// may arrive to a different shard from where the stream connection +// was opened, so the connection id is not known to a server that handles +// the RPC call. The shard that the stream connection belong to is know +// since it is a part of connection id, but this is not enough to locate +// a server instance the connection belongs to if there are more than one +// server on the shard. Stream domain parameter is here to help with that. +// Different servers on all shards logically belonging to the same service should +// belong to the same streaming domain. Only one server on each shard can belong to +// a particulr streaming domain. +class streaming_domain_type { + uint64_t _id; +public: + explicit streaming_domain_type(uint64_t id) : _id(id) {} + bool operator==(const streaming_domain_type& o) const { + return _id == o._id; + } + friend struct std::hash<streaming_domain_type>; + friend std::ostream& operator<<(std::ostream&, const streaming_domain_type&); +}; + +/// \addtogroup rpc +/// @{ + +class server; + +struct server_options { + compressor::factory* compressor_factory = nullptr; + bool tcp_nodelay = true; + std::optional<streaming_domain_type> streaming_domain; + server_socket::load_balancing_algorithm load_balancing_algorithm = server_socket::load_balancing_algorithm::default_; + // optional filter function. If set, will be called with remote + // (connecting) address. + // Returning false will refuse the incoming connection. + // Returning true will allow the mechanism to proceed. + std::function<bool(const socket_address&)> filter_connection = {}; +}; + +/// @} + +inline +size_t +estimate_request_size(const resource_limits& lim, size_t serialized_size) { + return lim.basic_request_size + serialized_size * lim.bloat_factor; +} + +struct negotiation_frame { + char magic[sizeof(rpc_magic) - 1]; + uint32_t len; // additional negotiation data length; multiple negotiation_frame_feature_record structs +}; + +enum class protocol_features : uint32_t { + COMPRESS = 0, + TIMEOUT = 1, + CONNECTION_ID = 2, + STREAM_PARENT = 3, + ISOLATION = 4, +}; + +// internal representation of feature data +using feature_map = std::map<protocol_features, sstring>; + +// An rpc signature, in the form signature<Ret (In0, In1, In2)>. +template <typename Function> +struct signature; + +class logger { + std::function<void(const sstring&)> _logger; + ::seastar::logger* _seastar_logger = nullptr; + + // _seastar_logger will always be used first if it's available + void log(const sstring& str) const { + if (_seastar_logger) { + // default level for log messages is `info` + _seastar_logger->info("{}", str); + } else if (_logger) { + _logger(str); + } + } + + // _seastar_logger will always be used first if it's available + template <typename... Args> + void log(log_level level, const char* fmt, Args&&... args) const { + if (_seastar_logger) { + _seastar_logger->log(level, fmt, std::forward<Args>(args)...); + // If the log level is at least `info`, fall back to legacy logging without explicit level. + // Ignore less severe levels in order not to spam user's log with messages during transition, + // i.e. when the user still only defines a level-less logger. + } else if (_logger && level <= log_level::info) { + _logger(format(fmt, std::forward<Args>(args)...)); + } + } + +public: + void set(std::function<void(const sstring&)> l) { + _logger = std::move(l); + } + + void set(::seastar::logger* logger) { + _seastar_logger = logger; + } + + void operator()(const client_info& info, id_type msg_id, const sstring& str) const; + void operator()(const client_info& info, id_type msg_id, log_level level, std::string_view str) const; + + void operator()(const client_info& info, const sstring& str) const; + void operator()(const client_info& info, log_level level, std::string_view str) const; + + void operator()(const socket_address& addr, const sstring& str) const; + void operator()(const socket_address& addr, log_level level, std::string_view str) const; +}; + +class connection { +protected: + connected_socket _fd; + input_stream<char> _read_buf; + output_stream<char> _write_buf; + bool _error = false; + bool _connected = false; + std::optional<shared_promise<>> _negotiated = shared_promise<>(); + promise<> _stopped; + stats _stats; + const logger& _logger; + // The owner of the pointer below is an instance of rpc::protocol<typename Serializer> class. + // The type of the pointer is erased here, but the original type is Serializer + void* _serializer; + struct outgoing_entry : public bi::list_base_hook<bi::link_mode<bi::auto_unlink>> { + timer<rpc_clock_type> t; + snd_buf buf; + promise<> done; + cancellable* pcancel = nullptr; + outgoing_entry(snd_buf b) : buf(std::move(b)) {} + + outgoing_entry(outgoing_entry&&) = delete; + outgoing_entry(const outgoing_entry&) = delete; + + void uncancellable() { + t.cancel(); + if (pcancel) { + pcancel->cancel_send = std::function<void()>(); + } + } + + ~outgoing_entry() { + if (pcancel) { + pcancel->cancel_send = std::function<void()>(); + pcancel->send_back_pointer = nullptr; + } + } + + using container_t = bi::list<outgoing_entry, bi::constant_time_size<false>>; + }; + void withdraw(outgoing_entry::container_t::iterator it, std::exception_ptr ex = nullptr); + future<> _outgoing_queue_ready = _negotiated->get_shared_future(); + outgoing_entry::container_t _outgoing_queue; + size_t _outgoing_queue_size = 0; + std::unique_ptr<compressor> _compressor; + bool _propagate_timeout = false; + bool _timeout_negotiated = false; + // stream related fields + bool _is_stream = false; + connection_id _id = invalid_connection_id; + + std::unordered_map<connection_id, xshard_connection_ptr> _streams; + queue<rcv_buf> _stream_queue = queue<rcv_buf>(max_queued_stream_buffers); + semaphore _stream_sem = semaphore(max_stream_buffers_memory); + bool _sink_closed = true; + bool _source_closed = true; + // the future holds if sink is already closed + // if it is not ready it means the sink is been closed + future<bool> _sink_closed_future = make_ready_future<bool>(false); + + size_t outgoing_queue_length() const noexcept { + return _outgoing_queue_size; + } + + void set_negotiated() noexcept; + + bool is_stream() const noexcept { + return _is_stream; + } + + snd_buf compress(snd_buf buf); + future<> send_buffer(snd_buf buf); + + future<> send_entry(outgoing_entry& d); + future<> stop_send_loop(std::exception_ptr ex); + future<std::optional<rcv_buf>> read_stream_frame_compressed(input_stream<char>& in); + bool stream_check_twoway_closed() const noexcept { + return _sink_closed && _source_closed; + } + future<> stream_close(); + future<> stream_process_incoming(rcv_buf&&); + future<> handle_stream_frame(); + +public: + connection(connected_socket&& fd, const logger& l, void* s, connection_id id = invalid_connection_id) : connection(l, s, id) { + set_socket(std::move(fd)); + } + connection(const logger& l, void* s, connection_id id = invalid_connection_id) : _logger(l), _serializer(s), _id(id) {} + virtual ~connection() {} + void set_socket(connected_socket&& fd); + 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::optional<rpc_clock_type::time_point> timeout = {}, cancellable* cancel = nullptr); + bool error() const noexcept { return _error; } + void abort(); + future<> stop() noexcept; + future<> stream_receive(circular_buffer<foreign_ptr<std::unique_ptr<rcv_buf>>>& bufs); + future<> close_sink() { + _sink_closed = true; + if (stream_check_twoway_closed()) { + return stream_close(); + } + return make_ready_future(); + } + bool sink_closed() const noexcept { + return _sink_closed; + } + future<> close_source() { + _source_closed = true; + if (stream_check_twoway_closed()) { + return stream_close(); + } + return make_ready_future(); + } + connection_id get_connection_id() const noexcept { + return _id; + } + xshard_connection_ptr get_stream(connection_id id) const; + void register_stream(connection_id id, xshard_connection_ptr c); + virtual socket_address peer_address() const = 0; + + const logger& get_logger() const noexcept { + return _logger; + } + + template<typename Serializer> + Serializer& serializer() { + return *static_cast<Serializer*>(_serializer); + } + + template <typename FrameType> + typename FrameType::return_type read_frame(socket_address info, input_stream<char>& in); + + template <typename FrameType> + typename FrameType::return_type read_frame_compressed(socket_address info, std::unique_ptr<compressor>& compressor, input_stream<char>& in); + friend class client; + template<typename Serializer, typename... Out> + friend class sink_impl; + template<typename Serializer, typename... In> + friend class source_impl; + + void suspend_for_testing(promise<>& p) { + _outgoing_queue_ready.get(); + auto dummy = std::make_unique<outgoing_entry>(snd_buf()); + _outgoing_queue.push_back(*dummy); + _outgoing_queue_ready = dummy->done.get_future(); + (void)p.get_future().then([dummy = std::move(dummy)] { dummy->done.set_value(); }); + } +}; + +struct deferred_snd_buf { + promise<> pr; + snd_buf data; +}; + +// send data Out... +template<typename Serializer, typename... Out> +class sink_impl : public sink<Out...>::impl { + // Used on the shard *this lives on. + alignas (cache_line_size) uint64_t _next_seq_num = 1; + + // Used on the shard the _conn lives on. + struct alignas (cache_line_size) { + uint64_t last_seq_num = 0; + std::map<uint64_t, deferred_snd_buf> out_of_order_bufs; + } _remote_state; +public: + sink_impl(xshard_connection_ptr con) : sink<Out...>::impl(std::move(con)) { this->_con->get()->_sink_closed = false; } + future<> operator()(const Out&... args) override; + future<> close() override; + future<> flush() override; + ~sink_impl() override; +}; + +// receive data In... +template<typename Serializer, typename... In> +class source_impl : public source<In...>::impl { +public: + source_impl(xshard_connection_ptr con) : source<In...>::impl(std::move(con)) { this->_con->get()->_source_closed = false; } + future<std::optional<std::tuple<In...>>> operator()() override; +}; + +class client : public rpc::connection, public weakly_referencable<client> { + socket _socket; + id_type _message_id = 1; + struct reply_handler_base { + timer<rpc_clock_type> t; + cancellable* pcancel = nullptr; + virtual void operator()(client&, id_type, rcv_buf data) = 0; + virtual void timeout() {} + virtual void cancel() {} + virtual ~reply_handler_base() { + if (pcancel) { + pcancel->cancel_wait = std::function<void()>(); + pcancel->wait_back_pointer = nullptr; + } + }; + }; +public: + template<typename Reply, typename Func> + struct reply_handler final : reply_handler_base { + Func func; + Reply reply; + reply_handler(Func&& f) : func(std::move(f)) {} + virtual void operator()(client& client, id_type msg_id, rcv_buf data) override { + return func(reply, client, msg_id, std::move(data)); + } + virtual void timeout() override { + reply.done = true; + reply.p.set_exception(timeout_error()); + } + virtual void cancel() override { + reply.done = true; + reply.p.set_exception(canceled_error()); + } + virtual ~reply_handler() {} + }; +private: + std::unordered_map<id_type, std::unique_ptr<reply_handler_base>> _outstanding; + socket_address _server_addr, _local_addr; + client_options _options; + weak_ptr<client> _parent; // for stream clients + +private: + future<> negotiate_protocol(input_stream<char>& in); + void negotiate(feature_map server_features); + future<std::tuple<int64_t, std::optional<rcv_buf>>> + read_response_frame(input_stream<char>& in); + future<std::tuple<int64_t, std::optional<rcv_buf>>> + read_response_frame_compressed(input_stream<char>& in); +public: + /** + * Create client object which will attempt to connect to the remote address. + * + * @param l \ref seastar::logger to use for logging error messages + * @param s an optional connection serializer + * @param addr the remote address identifying this client + * @param local the local address of this client + */ + client(const logger& l, void* s, const socket_address& addr, const socket_address& local = {}); + client(const logger& l, void* s, client_options options, const socket_address& addr, const socket_address& local = {}); + + /** + * Create client object which will attempt to connect to the remote address using the + * specified seastar::socket. + * + * @param l \ref seastar::logger to use for logging error messages + * @param s an optional connection serializer + * @param addr the remote address identifying this client + * @param local the local address of this client + * @param socket the socket object use to connect to the remote address + */ + client(const logger& l, void* s, socket socket, const socket_address& addr, const socket_address& local = {}); + client(const logger& l, void* s, client_options options, socket socket, const socket_address& addr, const socket_address& local = {}); + + stats get_stats() const; + stats& get_stats_internal() { + return _stats; + } + auto next_message_id() { return _message_id++; } + void wait_for_reply(id_type id, std::unique_ptr<reply_handler_base>&& h, std::optional<rpc_clock_type::time_point> timeout, cancellable* cancel); + void wait_timed_out(id_type id); + future<> stop() noexcept; + void abort_all_streams(); + void deregister_this_stream(); + socket_address peer_address() const override { + return _server_addr; + } + future<> await_connection() { + if (!_negotiated) { + return make_ready_future<>(); + } else { + return _negotiated->get_shared_future(); + } + } + template<typename Serializer, typename... Out> + future<sink<Out...>> make_stream_sink(socket socket) { + return await_connection().then([this, socket = std::move(socket)] () mutable { + if (!this->get_connection_id()) { + return make_exception_future<sink<Out...>>(std::runtime_error("Streaming is not supported by the server")); + } + client_options o = _options; + o.stream_parent = this->get_connection_id(); + o.send_timeout_data = false; + auto c = make_shared<client>(_logger, _serializer, o, std::move(socket), _server_addr, _local_addr); + c->_parent = this->weak_from_this(); + c->_is_stream = true; + return c->await_connection().then([c, this] { + xshard_connection_ptr s = make_lw_shared(make_foreign(static_pointer_cast<rpc::connection>(c))); + this->register_stream(c->get_connection_id(), s); + return sink<Out...>(make_shared<sink_impl<Serializer, Out...>>(std::move(s))); + }).handle_exception([c] (std::exception_ptr eptr) { + // If await_connection fails we need to stop the client + // before destroying it. + return c->stop().then([eptr, c] { + return make_exception_future<sink<Out...>>(eptr); + }); + }); + }); + } + template<typename Serializer, typename... Out> + future<sink<Out...>> make_stream_sink() { + return make_stream_sink<Serializer, Out...>(make_socket()); + } +}; + +class protocol_base; + +class server { +private: + static thread_local std::unordered_map<streaming_domain_type, server*> _servers; + +public: + class connection : public rpc::connection, public enable_shared_from_this<connection> { + server& _server; + client_info _info; + connection_id _parent_id = invalid_connection_id; + std::optional<isolation_config> _isolation_config; + private: + future<> negotiate_protocol(input_stream<char>& in); + future<std::tuple<std::optional<uint64_t>, uint64_t, int64_t, std::optional<rcv_buf>>> + read_request_frame_compressed(input_stream<char>& in); + future<feature_map> negotiate(feature_map requested); + future<> send_unknown_verb_reply(std::optional<rpc_clock_type::time_point> timeout, int64_t msg_id, uint64_t type); + 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::optional<rpc_clock_type::time_point> timeout); + client_info& info() { return _info; } + const client_info& info() const { return _info; } + stats get_stats() const { + stats res = _stats; + res.pending = outgoing_queue_length(); + return res; + } + + stats& get_stats_internal() { + return _stats; + } + socket_address peer_address() const override { + return _info.addr; + } + // Resources will be released when this goes out of scope + future<resource_permit> wait_for_resources(size_t memory_consumed, std::optional<rpc_clock_type::time_point> timeout) { + if (timeout) { + return get_units(_server._resources_available, memory_consumed, *timeout); + } else { + return get_units(_server._resources_available, memory_consumed); + } + } + size_t estimate_request_size(size_t serialized_size) { + return rpc::estimate_request_size(_server._limits, serialized_size); + } + size_t max_request_size() const { + return _server._limits.max_memory; + } + server& get_server() { + return _server; + } + future<> deregister_this_stream(); + }; +private: + protocol_base* _proto; + server_socket _ss; + resource_limits _limits; + rpc_semaphore _resources_available; + std::unordered_map<connection_id, shared_ptr<connection>> _conns; + promise<> _ss_stopped; + gate _reply_gate; + server_options _options; + uint64_t _next_client_id = 1; + +public: + server(protocol_base* proto, const socket_address& addr, resource_limits memory_limit = resource_limits()); + server(protocol_base* proto, server_options opts, const socket_address& addr, resource_limits memory_limit = resource_limits()); + server(protocol_base* proto, server_socket, resource_limits memory_limit = resource_limits(), server_options opts = server_options{}); + server(protocol_base* proto, server_options opts, server_socket, resource_limits memory_limit = resource_limits()); + void accept(); + future<> stop(); + template<typename Func> + void foreach_connection(Func&& f) { + for (auto c : _conns) { + f(*c.second); + } + } + gate& reply_gate() { + return _reply_gate; + } + friend connection; + friend client; +}; + +using rpc_handler_func = std::function<future<> (shared_ptr<server::connection>, std::optional<rpc_clock_type::time_point> timeout, int64_t msgid, + rcv_buf data)>; + +struct rpc_handler { + scheduling_group sg; + rpc_handler_func func; + gate use_gate; +}; + +class protocol_base { +public: + virtual ~protocol_base() {}; + virtual shared_ptr<server::connection> make_server_connection(rpc::server& server, connected_socket fd, socket_address addr, connection_id id) = 0; +protected: + friend class server; + + virtual rpc_handler* get_handler(uint64_t msg_id) = 0; + virtual void put_handler(rpc_handler*) = 0; +}; + +/// \addtogroup rpc +/// @{ + +/// Defines a protocol for communication between a server and a client. +/// +/// A protocol is defined by a `Serializer` and a `MsgType`. The `Serializer` is +/// responsible for serializing and unserializing all types used as arguments and +/// return types used in the protocol. The `Serializer` is expected to define a +/// `read()` and `write()` method for each such type `T` as follows: +/// +/// template <typename Output> +/// void write(const serializer&, Output& output, const T& data); +/// +/// template <typename Input> +/// T read(const serializer&, Input& input, type<T> type_tag); // type_tag used to disambiguate +/// +/// Where `Input` and `Output` have a `void read(char*, size_t)` and +/// `write(const char*, size_t)` respectively. +/// `MsgType` defines the type to be used as the message id, the id which is +/// used to identify different messages used in the protocol. These are also +/// often referred to as "verbs". The client will use the message id, to +/// specify the remote method (verb) to invoke on the server. The server uses +/// the message id to dispatch the incoming call to the right handler. +/// `MsgType` should be hashable and serializable. It is preferable to use enum +/// for message types, but do not forget to provide hash function for it. +/// +/// Use register_handler() on the server to define the available verbs and the +/// code to be executed when they are invoked by clients. Use make_client() on +/// the client to create a matching callable that can be used to invoke the +/// verb on the server and wait for its result. Note that register_handler() +/// also returns a client, that can be used to invoke the registered verb on +/// another node (given that the other node has the same verb). This is useful +/// for symmetric protocols, where two or more nodes all have servers as well as +/// connect to the other nodes as clients. +/// +/// Use protocol::server to listen for and accept incoming connections on the +/// server and protocol::client to establish connections to the server. +/// Note that registering the available verbs can be done before/after +/// listening for connections, but best to ensure that by the time incoming +/// requests are to be expected, all the verbs are set-up. +/// +/// ## Configuration +/// +/// TODO +/// +/// ## Isolation +/// +/// RPC supports isolating verb handlers from each other. There are two ways to +/// achieve this: per-handler isolation (the old way) and per-connection +/// isolation (the new way). If no isolation is configured, all handlers will be +/// executed in the context of the scheduling_group in which the +/// protocol::server was created. +/// +/// Per-handler isolation (the old way) can be configured by using the +/// register_handler() overload which takes a scheduling_group. When invoked, +/// the body of the handler will be executed from the context of the configured +/// scheduling_group. +/// +/// Per-connection isolation (the new way) is a more flexible mechanism that +/// requires user application provided logic to determine how connections are +/// isolated. This mechanism has two parts, the server and the client part. +/// The client configures isolation by setting client_options::isolation_cookie. +/// This cookie is an opaque (to the RPC layer) string that is to be interpreted +/// on the server using user application provided logic. The application +/// provides this logic to the server by setting +/// resource_limits::isolate_connection to an appropriate handler function, that +/// interprets the opaque cookie and resolves it to an isolation_config. The +/// scheduling_group in the former will be used not just to execute all verb +/// handlers, but also the connection loop itself, hence providing better +/// isolation. +/// +/// There a few gotchas related to mixing the two isolation mechanisms. This can +/// happen when the application is updated and one of the client/server is +/// still using the old/new mechanism. In general per-connection isolation +/// overrides the per-handler one. If both are set up, the former will determine +/// the scheduling_group context for the handlers. If the client is not +/// configured to send an isolation cookie, the server's +/// resource_limits::isolate_connection will not be invoked and the server will +/// fall back to per-handler isolation if configured. If the client is +/// configured to send an isolation cookie but the server doesn't have a +/// resource_limits::isolate_connection configured, it will use +/// default_isolate_connection() to interpret the cookie. Note that this still +/// overrides the per-handler isolation if any is configured. If the server is +/// so old that it doesn't have the per-connection isolation feature at all, it +/// will of course just use the per-handler one, if configured. +/// +/// ## Compatibility +/// +/// TODO +/// +/// \tparam Serializer the serializer for the protocol. +/// \tparam MsgType the type to be used as the message id or verb id. +template<typename Serializer, typename MsgType = uint32_t> +class protocol final : public protocol_base { +public: + /// Represents the listening port and all accepted connections. + class server : public rpc::server { + public: + server(protocol& proto, const socket_address& addr, resource_limits memory_limit = resource_limits()) : + rpc::server(&proto, addr, memory_limit) {} + server(protocol& proto, server_options opts, const socket_address& addr, resource_limits memory_limit = resource_limits()) : + rpc::server(&proto, opts, addr, memory_limit) {} + server(protocol& proto, server_socket socket, resource_limits memory_limit = resource_limits(), server_options = server_options{}) : + rpc::server(&proto, std::move(socket), memory_limit) {} + server(protocol& proto, server_options opts, server_socket socket, resource_limits memory_limit = resource_limits()) : + rpc::server(&proto, opts, std::move(socket), memory_limit) {} + }; + /// Represents a client side connection. + class client : public rpc::client { + public: + /* + * Create client object which will attempt to connect to the remote address. + * + * @param addr the remote address identifying this client + * @param local the local address of this client + */ + client(protocol& p, const socket_address& addr, const socket_address& local = {}) : + rpc::client(p.get_logger(), &p._serializer, addr, local) {} + client(protocol& p, client_options options, const socket_address& addr, const socket_address& local = {}) : + rpc::client(p.get_logger(), &p._serializer, options, addr, local) {} + + /** + * Create client object which will attempt to connect to the remote address using the + * specified seastar::socket. + * + * @param addr the remote address identifying this client + * @param local the local address of this client + * @param socket the socket object use to connect to the remote address + */ + client(protocol& p, socket socket, const socket_address& addr, const socket_address& local = {}) : + rpc::client(p.get_logger(), &p._serializer, std::move(socket), addr, local) {} + client(protocol& p, client_options options, socket socket, const socket_address& addr, const socket_address& local = {}) : + rpc::client(p.get_logger(), &p._serializer, options, std::move(socket), addr, local) {} + }; + + friend server; +private: + std::unordered_map<MsgType, rpc_handler> _handlers; + Serializer _serializer; + logger _logger; + +public: + protocol(Serializer&& serializer) : _serializer(std::forward<Serializer>(serializer)) {} + + /// Creates a callable that can be used to invoke the verb on the remote. + /// + /// \tparam Func The signature of the verb. Has to be either the same or + /// compatible with the one passed to register_handler on the server. + /// \param t the verb to invoke on the remote. + /// + /// \returns a callable whose signature is derived from Func as follows: + /// given `Func == Ret(Args...)` the returned callable has the following + /// signature: `future<Ret>(protocol::client&, Args...)`. + template<typename Func> + auto make_client(MsgType t); + + /// Register a handler to be called when this verb is invoked. + /// + /// \tparam Func the type of the handler for the verb. This determines the + /// signature of the verb. + /// \param t the verb to register the handler for. + /// \param func the callable to be called when the verb is invoked by the + /// remote. + /// + /// \returns a client, a callable that can be used to invoke the verb. See + /// make_client(). The client can be discarded, in fact this is what + /// most callers will do as real clients will live on a remote node, not + /// on the one where handlers are registered. + template<typename Func> + auto register_handler(MsgType t, Func&& func); + + /// Register a handler to be called when this verb is invoked. + /// + /// \tparam Func the type of the handler for the verb. This determines the + /// signature of the verb. + /// \param t the verb to register the handler for. + /// \param sg the scheduling group that will be used to invoke the handler + /// in. This can be used to execute different verbs in different + /// scheduling groups. Note that there is a newer mechanism to determine + /// the scheduling groups a handler will run it per invocation, see + /// isolation_config. + /// \param func the callable to be called when the verb is invoked by the + /// remote. + /// + /// \returns a client, a callable that can be used to invoke the verb. See + /// make_client(). The client can be discarded, in fact this is what + /// most callers will do as real clients will live on a remote node, not + /// on the one where handlers are registered. + template <typename Func> + auto register_handler(MsgType t, scheduling_group sg, Func&& func); + + /// Unregister the handler for the verb. + /// + /// Waits for all currently running handlers, then unregisters the handler. + /// Future attempts to invoke the verb will fail. This becomes effective + /// immediately after calling this function. + /// + /// \param t the verb to unregister the handler for. + /// + /// \returns a future that becomes available once all currently running + /// handlers finished. + future<> unregister_handler(MsgType t); + + /// Set a logger function to be used to log messages. + /// + /// \deprecated use the logger overload set_logger(::seastar::logger*) + /// instead. + [[deprecated("Use set_logger(::seastar::logger*) instead")]] + void set_logger(std::function<void(const sstring&)> logger) { + _logger.set(std::move(logger)); + } + + /// Set a logger to be used to log messages. + void set_logger(::seastar::logger* logger) { + _logger.set(logger); + } + + const logger& get_logger() const { + return _logger; + } + + shared_ptr<rpc::server::connection> make_server_connection(rpc::server& server, connected_socket fd, socket_address addr, connection_id id) override { + return make_shared<rpc::server::connection>(server, std::move(fd), std::move(addr), _logger, &_serializer, id); + } + + bool has_handler(MsgType msg_id); + + /// Checks if any there are handlers registered. + /// Debugging helper, should only be used for debugging and not relied on. + /// + /// \returns true if there are, false if there are no registered handlers. + bool has_handlers() const noexcept { + return !_handlers.empty(); + } + +private: + rpc_handler* get_handler(uint64_t msg_id) override; + void put_handler(rpc_handler*) override; + + template<typename Ret, typename... In> + auto make_client(signature<Ret(In...)> sig, MsgType t); + + void register_receiver(MsgType t, rpc_handler&& handler) { + auto r = _handlers.emplace(t, std::move(handler)); + if (!r.second) { + throw_with_backtrace<std::runtime_error>("registered handler already exists"); + } + } +}; + +/// @} + +} + +} + +#include "rpc_impl.hh" diff --git a/src/seastar/include/seastar/rpc/rpc_impl.hh b/src/seastar/include/seastar/rpc/rpc_impl.hh new file mode 100644 index 000000000..5e27ce3c3 --- /dev/null +++ b/src/seastar/include/seastar/rpc/rpc_impl.hh @@ -0,0 +1,903 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ +#pragma once + +#include <seastar/core/function_traits.hh> +#include <seastar/core/shared_ptr.hh> +#include <seastar/core/sstring.hh> +#include <seastar/core/when_all.hh> +#include <seastar/util/is_smart_ptr.hh> +#include <seastar/core/simple-stream.hh> +#include <boost/range/numeric.hpp> +#include <boost/range/adaptor/transformed.hpp> +#include <seastar/net/packet-data-source.hh> +#include <seastar/core/print.hh> + +namespace seastar { + +namespace rpc { + +enum class exception_type : uint32_t { + USER = 0, + UNKNOWN_VERB = 1, +}; + +template<typename T> +struct remove_optional { + using type = T; +}; + +template<typename T> +struct remove_optional<optional<T>> { + using type = T; +}; + +struct wait_type {}; // opposite of no_wait_type + +// tags to tell whether we want a const client_info& parameter +struct do_want_client_info {}; +struct dont_want_client_info {}; + +// tags to tell whether we want a opt_time_point parameter +struct do_want_time_point {}; +struct dont_want_time_point {}; + +// General case +template <typename Ret, typename... In> +struct signature<Ret (In...)> { + using ret_type = Ret; + using arg_types = std::tuple<In...>; + using clean = signature; + using want_client_info = dont_want_client_info; + using want_time_point = dont_want_time_point; +}; + +// Specialize 'clean' for handlers that receive client_info +template <typename Ret, typename... In> +struct signature<Ret (const client_info&, In...)> { + using ret_type = Ret; + using arg_types = std::tuple<In...>; + using clean = signature<Ret (In...)>; + using want_client_info = do_want_client_info; + using want_time_point = dont_want_time_point; +}; + +template <typename Ret, typename... In> +struct signature<Ret (client_info&, In...)> { + using ret_type = Ret; + using arg_types = std::tuple<In...>; + using clean = signature<Ret (In...)>; + using want_client_info = do_want_client_info; + using want_time_point = dont_want_time_point; +}; + +// Specialize 'clean' for handlers that receive client_info and opt_time_point +template <typename Ret, typename... In> +struct signature<Ret (const client_info&, opt_time_point, In...)> { + using ret_type = Ret; + using arg_types = std::tuple<In...>; + using clean = signature<Ret (In...)>; + using want_client_info = do_want_client_info; + using want_time_point = do_want_time_point; +}; + +template <typename Ret, typename... In> +struct signature<Ret (client_info&, opt_time_point, In...)> { + using ret_type = Ret; + using arg_types = std::tuple<In...>; + using clean = signature<Ret (In...)>; + using want_client_info = do_want_client_info; + using want_time_point = do_want_time_point; +}; + +// Specialize 'clean' for handlers that receive opt_time_point +template <typename Ret, typename... In> +struct signature<Ret (opt_time_point, In...)> { + using ret_type = Ret; + using arg_types = std::tuple<In...>; + using clean = signature<Ret (In...)>; + using want_client_info = dont_want_client_info; + using want_time_point = do_want_time_point; +}; + +template <typename T> +struct wait_signature { + using type = wait_type; + using cleaned_type = T; +}; + +template <typename... T> +struct wait_signature<future<T...>> { + using type = wait_type; + using cleaned_type = future<T...>; +}; + +template <> +struct wait_signature<no_wait_type> { + using type = no_wait_type; + using cleaned_type = void; +}; + +template <> +struct wait_signature<future<no_wait_type>> { + using type = no_wait_type; + using cleaned_type = future<>; +}; + +template <typename T> +using wait_signature_t = typename wait_signature<T>::type; + +template <typename... In> +inline +std::tuple<In...> +maybe_add_client_info(dont_want_client_info, client_info&, std::tuple<In...>&& args) { + return std::move(args); +} + +template <typename... In> +inline +std::tuple<std::reference_wrapper<client_info>, In...> +maybe_add_client_info(do_want_client_info, client_info& ci, std::tuple<In...>&& args) { + return std::tuple_cat(std::make_tuple(std::ref(ci)), std::move(args)); +} + +template <typename... In> +inline +std::tuple<In...> +maybe_add_time_point(dont_want_time_point, opt_time_point&, std::tuple<In...>&& args) { + return std::move(args); +} + +template <typename... In> +inline +std::tuple<opt_time_point, In...> +maybe_add_time_point(do_want_time_point, opt_time_point& otp, std::tuple<In...>&& args) { + return std::tuple_cat(std::make_tuple(otp), std::move(args)); +} + +inline sstring serialize_connection_id(const connection_id& id) { + sstring p = uninitialized_string(sizeof(id)); + auto c = p.data(); + write_le(c, id.id); + return p; +} + +inline connection_id deserialize_connection_id(const sstring& s) { + connection_id id; + auto p = s.c_str(); + id.id = read_le<decltype(id.id)>(p); + return id; +} + +template <bool IsSmartPtr> +struct serialize_helper; + +template <> +struct serialize_helper<false> { + template <typename Serializer, typename Output, typename T> + static inline void serialize(Serializer& serializer, Output& out, const T& t) { + return write(serializer, out, t); + } +}; + +template <> +struct serialize_helper<true> { + template <typename Serializer, typename Output, typename T> + static inline void serialize(Serializer& serializer, Output& out, const T& t) { + return write(serializer, out, *t); + } +}; + +template <typename Serializer, typename Output, typename... T> +inline void do_marshall(Serializer& serializer, Output& out, const T&... args); + +template <typename Serializer, typename Output> +struct marshall_one { + template <typename T> struct helper { + static void doit(Serializer& serializer, Output& out, const T& arg) { + using serialize_helper_type = serialize_helper<is_smart_ptr<typename std::remove_reference<T>::type>::value>; + serialize_helper_type::serialize(serializer, out, arg); + } + }; + template<typename T> struct helper<std::reference_wrapper<const T>> { + static void doit(Serializer& serializer, Output& out, const std::reference_wrapper<const T>& arg) { + helper<T>::doit(serializer, out, arg.get()); + } + }; + static void put_connection_id(const connection_id& cid, Output& out) { + sstring id = serialize_connection_id(cid); + out.write(id.c_str(), id.size()); + } + template <typename... T> struct helper<sink<T...>> { + static void doit(Serializer&, Output& out, const sink<T...>& arg) { + put_connection_id(arg.get_id(), out); + } + }; + template <typename... T> struct helper<source<T...>> { + static void doit(Serializer&, Output& out, const source<T...>& arg) { + put_connection_id(arg.get_id(), out); + } + }; + template <typename... T> struct helper<tuple<T...>> { + static void doit(Serializer& serializer, Output& out, const tuple<T...>& arg) { + auto do_do_marshall = [&serializer, &out] (const auto&... args) { + do_marshall(serializer, out, args...); + }; + std::apply(do_do_marshall, arg); + } + }; +}; + +template <typename Serializer, typename Output, typename... T> +inline void do_marshall(Serializer& serializer, Output& out, const T&... args) { + // C++ guarantees that brace-initialization expressions are evaluted in order + (void)std::initializer_list<int>{(marshall_one<Serializer, Output>::template helper<T>::doit(serializer, out, args), 1)...}; +} + +static inline memory_output_stream<snd_buf::iterator> make_serializer_stream(snd_buf& output) { + auto* b = std::get_if<temporary_buffer<char>>(&output.bufs); + if (b) { + return memory_output_stream<snd_buf::iterator>(memory_output_stream<snd_buf::iterator>::simple(b->get_write(), b->size())); + } else { + auto& ar = std::get<std::vector<temporary_buffer<char>>>(output.bufs); + return memory_output_stream<snd_buf::iterator>(memory_output_stream<snd_buf::iterator>::fragmented(ar.begin(), output.size)); + } +} + +template <typename Serializer, typename... T> +inline snd_buf marshall(Serializer& serializer, size_t head_space, const T&... args) { + measuring_output_stream measure; + do_marshall(serializer, measure, args...); + snd_buf ret(measure.size() + head_space); + auto out = make_serializer_stream(ret); + out.skip(head_space); + do_marshall(serializer, out, args...); + return ret; +} + +template <typename Serializer, typename Input, typename... T> +std::tuple<T...> do_unmarshall(connection& c, Input& in); + +template<typename Serializer, typename Input> +struct unmarshal_one { + template<typename T> struct helper { + static T doit(connection& c, Input& in) { + return read(c.serializer<Serializer>(), in, type<T>()); + } + }; + template<typename T> struct helper<optional<T>> { + static optional<T> doit(connection& c, Input& in) { + if (in.size()) { + return optional<T>(read(c.serializer<Serializer>(), in, type<typename remove_optional<T>::type>())); + } else { + return optional<T>(); + } + } + }; + template<typename T> struct helper<std::reference_wrapper<const T>> { + static T doit(connection& c, Input& in) { + return helper<T>::doit(c, in); + } + }; + static connection_id get_connection_id(Input& in) { + sstring id = uninitialized_string(sizeof(connection_id)); + in.read(id.data(), sizeof(connection_id)); + return deserialize_connection_id(id); + } + template<typename... T> struct helper<sink<T...>> { + static sink<T...> doit(connection& c, Input& in) { + return sink<T...>(make_shared<sink_impl<Serializer, T...>>(c.get_stream(get_connection_id(in)))); + } + }; + template<typename... T> struct helper<source<T...>> { + static source<T...> doit(connection& c, Input& in) { + return source<T...>(make_shared<source_impl<Serializer, T...>>(c.get_stream(get_connection_id(in)))); + } + }; + template <typename... T> struct helper<tuple<T...>> { + static tuple<T...> doit(connection& c, Input& in) { + return do_unmarshall<Serializer, Input, T...>(c, in); + } + }; +}; + +template <typename Serializer, typename Input, typename... T> +inline std::tuple<T...> do_unmarshall(connection& c, Input& in) { + // Argument order processing is unspecified, but we need to deserialize + // left-to-right. So we deserialize into something that can be lazily + // constructed (and can conditionally destroy itself if we only constructed some + // of the arguments). + std::tuple<std::optional<T>...> temporary; + return std::apply([&] (auto&... args) { + // Comma-expression preserves left-to-right order + (..., (args = unmarshal_one<Serializer, Input>::template helper<typename std::remove_reference_t<decltype(args)>::value_type>::doit(c, in))); + return std::tuple(std::move(*args)...); + }, temporary); +} + +template <typename Serializer, typename... T> +inline std::tuple<T...> unmarshall(connection& c, rcv_buf input) { + auto in = make_deserializer_stream(input); + return do_unmarshall<Serializer, decltype(in), T...>(c, in); +} + +inline std::exception_ptr unmarshal_exception(rcv_buf& d) { + std::exception_ptr ex; + auto data = make_deserializer_stream(d); + + uint32_t v32; + data.read(reinterpret_cast<char*>(&v32), 4); + exception_type ex_type = exception_type(le_to_cpu(v32)); + data.read(reinterpret_cast<char*>(&v32), 4); + uint32_t ex_len = le_to_cpu(v32); + + switch (ex_type) { + case exception_type::USER: { + std::string s(ex_len, '\0'); + data.read(&*s.begin(), ex_len); + ex = std::make_exception_ptr(remote_verb_error(std::move(s))); + break; + } + case exception_type::UNKNOWN_VERB: { + uint64_t v64; + data.read(reinterpret_cast<char*>(&v64), 8); + ex = std::make_exception_ptr(unknown_verb_error(le_to_cpu(v64))); + break; + } + default: + ex = std::make_exception_ptr(unknown_exception_error()); + break; + } + return ex; +} + +template <typename Payload, typename... T> +struct rcv_reply_base { + bool done = false; + promise<T...> p; + template<typename... V> + void set_value(V&&... v) { + done = true; + p.set_value(internal::untuple(std::forward<V>(v))...); + } + ~rcv_reply_base() { + if (!done) { + p.set_exception(closed_error()); + } + } +}; + +template<typename Serializer, typename T> +struct rcv_reply : rcv_reply_base<T, T> { + inline void get_reply(rpc::client& dst, rcv_buf input) { + this->set_value(unmarshall<Serializer, T>(dst, std::move(input))); + } +}; + +template<typename Serializer, typename... T> +struct rcv_reply<Serializer, future<T...>> : rcv_reply_base<std::tuple<T...>, T...> { + inline void get_reply(rpc::client& dst, rcv_buf input) { + this->set_value(unmarshall<Serializer, T...>(dst, std::move(input))); + } +}; + +template<typename Serializer> +struct rcv_reply<Serializer, void> : rcv_reply_base<void, void> { + inline void get_reply(rpc::client&, rcv_buf) { + this->set_value(); + } +}; + +template<typename Serializer> +struct rcv_reply<Serializer, future<>> : rcv_reply<Serializer, void> {}; + +template <typename Serializer, typename Ret, typename... InArgs> +inline auto wait_for_reply(wait_type, std::optional<rpc_clock_type::time_point> timeout, cancellable* cancel, rpc::client& dst, id_type msg_id, + signature<Ret (InArgs...)>) { + using reply_type = rcv_reply<Serializer, Ret>; + auto lambda = [] (reply_type& r, rpc::client& dst, id_type msg_id, rcv_buf data) mutable { + if (msg_id >= 0) { + dst.get_stats_internal().replied++; + return r.get_reply(dst, std::move(data)); + } else { + dst.get_stats_internal().exception_received++; + r.done = true; + r.p.set_exception(unmarshal_exception(data)); + } + }; + using handler_type = typename rpc::client::template reply_handler<reply_type, decltype(lambda)>; + auto r = std::make_unique<handler_type>(std::move(lambda)); + auto fut = r->reply.p.get_future(); + dst.wait_for_reply(msg_id, std::move(r), timeout, cancel); + return fut; +} + +template<typename Serializer, typename... InArgs> +inline auto wait_for_reply(no_wait_type, std::optional<rpc_clock_type::time_point>, cancellable*, rpc::client&, id_type, + signature<no_wait_type (InArgs...)>) { // no_wait overload + return make_ready_future<>(); +} + +template<typename Serializer, typename... InArgs> +inline auto wait_for_reply(no_wait_type, std::optional<rpc_clock_type::time_point>, cancellable*, rpc::client&, id_type, + signature<future<no_wait_type> (InArgs...)>) { // future<no_wait> overload + return make_ready_future<>(); +} + +// Convert a relative timeout (a duration) to an absolute one (time_point). +// Do the calculation safely so that a very large duration will be capped by +// time_point::max, instead of wrapping around to ancient history. +inline rpc_clock_type::time_point +relative_timeout_to_absolute(rpc_clock_type::duration relative) { + rpc_clock_type::time_point now = rpc_clock_type::now(); + return now + std::min(relative, rpc_clock_type::time_point::max() - now); +} + +// Returns lambda that can be used to send rpc messages. +// The lambda gets client connection and rpc parameters as arguments, marshalls them sends +// to a server and waits for a reply. After receiving reply it unmarshalls it and signal completion +// to a caller. +template<typename Serializer, typename MsgType, typename Ret, typename... InArgs> +auto send_helper(MsgType xt, signature<Ret (InArgs...)> xsig) { + struct shelper { + MsgType t; + signature<Ret (InArgs...)> sig; + auto send(rpc::client& dst, std::optional<rpc_clock_type::time_point> timeout, cancellable* cancel, const InArgs&... args) { + if (dst.error()) { + using cleaned_ret_type = typename wait_signature<Ret>::cleaned_type; + return futurize<cleaned_ret_type>::make_exception_future(closed_error()); + } + + // send message + auto msg_id = dst.next_message_id(); + snd_buf data = marshall(dst.template serializer<Serializer>(), 28, args...); + static_assert(snd_buf::chunk_size >= 28, "send buffer chunk size is too small"); + auto p = data.front().get_write() + 8; // 8 extra bytes for expiration timer + write_le<uint64_t>(p, uint64_t(t)); + write_le<int64_t>(p + 8, msg_id); + write_le<uint32_t>(p + 16, data.size - 28); + + // prepare reply handler, if return type is now_wait_type this does nothing, since no reply will be sent + using wait = wait_signature_t<Ret>; + return when_all(dst.send(std::move(data), timeout, cancel), wait_for_reply<Serializer>(wait(), timeout, cancel, dst, msg_id, sig)).then([] (auto r) { + std::get<0>(r).ignore_ready_future(); + return std::move(std::get<1>(r)); // return future of wait_for_reply + }); + } + auto operator()(rpc::client& dst, const InArgs&... args) { + return send(dst, {}, nullptr, args...); + } + auto operator()(rpc::client& dst, rpc_clock_type::time_point timeout, const InArgs&... args) { + return send(dst, timeout, nullptr, args...); + } + auto operator()(rpc::client& dst, rpc_clock_type::duration timeout, const InArgs&... args) { + return send(dst, relative_timeout_to_absolute(timeout), nullptr, args...); + } + auto operator()(rpc::client& dst, cancellable& cancel, const InArgs&... args) { + return send(dst, {}, &cancel, args...); + } + + }; + return shelper{xt, xsig}; +} + +template<typename Serializer, typename SEASTAR_ELLIPSIS RetTypes> +inline future<> reply(wait_type, future<RetTypes SEASTAR_ELLIPSIS>&& ret, int64_t msg_id, shared_ptr<server::connection> client, + std::optional<rpc_clock_type::time_point> timeout) { + if (!client->error()) { + snd_buf data; + try { +#if SEASTAR_API_LEVEL < 6 + if constexpr (sizeof...(RetTypes) == 0) { +#else + if constexpr (std::is_void_v<RetTypes>) { +#endif + ret.get(); + data = std::invoke(marshall<Serializer>, std::ref(client->template serializer<Serializer>()), 12); + } else { + data = std::invoke(marshall<Serializer, const RetTypes& SEASTAR_ELLIPSIS>, std::ref(client->template serializer<Serializer>()), 12, std::move(ret.get0())); + } + } catch (std::exception& ex) { + uint32_t len = std::strlen(ex.what()); + data = snd_buf(20 + len); + auto os = make_serializer_stream(data); + os.skip(12); + uint32_t v32 = cpu_to_le(uint32_t(exception_type::USER)); + os.write(reinterpret_cast<char*>(&v32), sizeof(v32)); + v32 = cpu_to_le(len); + os.write(reinterpret_cast<char*>(&v32), sizeof(v32)); + os.write(ex.what(), len); + msg_id = -msg_id; + } + + return client->respond(msg_id, std::move(data), timeout); + } else { + ret.ignore_ready_future(); + return make_ready_future<>(); + } +} + +// specialization for no_wait_type which does not send a reply +template<typename Serializer> +inline future<> reply(no_wait_type, future<no_wait_type>&& r, int64_t msgid, shared_ptr<server::connection> client, std::optional<rpc_clock_type::time_point>) { + try { + r.get(); + } catch (std::exception& ex) { + client->get_logger()(client->info(), msgid, to_sstring("exception \"") + ex.what() + "\" in no_wait handler ignored"); + } + return make_ready_future<>(); +} + +template<typename Ret, typename... InArgs, typename WantClientInfo, typename WantTimePoint, typename Func, typename ArgsTuple> +inline futurize_t<Ret> apply(Func& func, client_info& info, opt_time_point time_point, WantClientInfo wci, WantTimePoint wtp, signature<Ret (InArgs...)>, ArgsTuple&& args) { + using futurator = futurize<Ret>; + return futurator::apply(func, maybe_add_client_info(wci, info, maybe_add_time_point(wtp, time_point, std::forward<ArgsTuple>(args)))); +} + +// lref_to_cref is a helper that encapsulates lvalue reference in std::ref() or does nothing otherwise +template<typename T> +auto lref_to_cref(T&& x) { + return std::move(x); +} + +template<typename T> +auto lref_to_cref(T& x) { + return std::ref(x); +} + +// Creates lambda to handle RPC message on a server. +// The lambda unmarshalls all parameters, calls a handler, marshall return values and sends them back to a client +template <typename Serializer, typename Func, typename Ret, typename... InArgs, typename WantClientInfo, typename WantTimePoint> +auto recv_helper(signature<Ret (InArgs...)> sig, Func&& func, WantClientInfo, WantTimePoint) { + using signature = decltype(sig); + using wait_style = wait_signature_t<Ret>; + return [func = lref_to_cref(std::forward<Func>(func))](shared_ptr<server::connection> client, + std::optional<rpc_clock_type::time_point> timeout, + int64_t msg_id, + rcv_buf data) mutable { + auto memory_consumed = client->estimate_request_size(data.size); + if (memory_consumed > client->max_request_size()) { + auto err = format("request size {:d} large than memory limit {:d}", memory_consumed, client->max_request_size()); + client->get_logger()(client->peer_address(), err); + // FIXME: future is discarded + (void)try_with_gate(client->get_server().reply_gate(), [client, timeout, msg_id, err = std::move(err)] { + return reply<Serializer>(wait_style(), futurize<Ret>::make_exception_future(std::runtime_error(err.c_str())), msg_id, client, timeout).handle_exception([client, msg_id] (std::exception_ptr eptr) { + client->get_logger()(client->info(), msg_id, format("got exception while processing an oversized message: {}", eptr)); + }); + }).handle_exception_type([] (gate_closed_exception&) {/* ignore */}); + return make_ready_future(); + } + // note: apply is executed asynchronously with regards to networking so we cannot chain futures here by doing "return apply()" + auto f = client->wait_for_resources(memory_consumed, timeout).then([client, timeout, msg_id, data = std::move(data), &func] (auto permit) mutable { + // FIXME: future is discarded + (void)try_with_gate(client->get_server().reply_gate(), [client, timeout, msg_id, data = std::move(data), permit = std::move(permit), &func] () mutable { + try { + auto args = unmarshall<Serializer, InArgs...>(*client, std::move(data)); + return apply(func, client->info(), timeout, WantClientInfo(), WantTimePoint(), signature(), std::move(args)).then_wrapped([client, timeout, msg_id, permit = std::move(permit)] (futurize_t<Ret> ret) mutable { + return reply<Serializer>(wait_style(), std::move(ret), msg_id, client, timeout).handle_exception([permit = std::move(permit), client, msg_id] (std::exception_ptr eptr) { + client->get_logger()(client->info(), msg_id, format("got exception while processing a message: {}", eptr)); + }); + }); + } catch (...) { + client->get_logger()(client->info(), msg_id, format("caught exception while processing a message: {}", std::current_exception())); + return make_ready_future(); + } + }).handle_exception_type([] (gate_closed_exception&) {/* ignore */}); + }); + + if (timeout) { + f = f.handle_exception_type([] (semaphore_timed_out&) { /* ignore */ }); + } + + return f; + }; +} + +// helper to create copy constructible lambda from non copy constructible one. std::function<> works only with former kind. +template<typename Func> +auto make_copyable_function(Func&& func, std::enable_if_t<!std::is_copy_constructible<std::decay_t<Func>>::value, void*> = nullptr) { + auto p = make_lw_shared<typename std::decay_t<Func>>(std::forward<Func>(func)); + return [p] (auto&&... args) { return (*p)( std::forward<decltype(args)>(args)... ); }; +} + +template<typename Func> +auto make_copyable_function(Func&& func, std::enable_if_t<std::is_copy_constructible<std::decay_t<Func>>::value, void*> = nullptr) { + return std::forward<Func>(func); +} + +// This class is used to calculate client side rpc function signature. +// Return type is converted from a smart pointer to a type it points to. +// rpc::optional are converted to non optional type. +// +// Examples: +// std::unique_ptr<int>(int, rpc::optional<long>) -> int(int, long) +// double(float) -> double(float) +template<typename Ret, typename... In> +class client_function_type { + template<typename T, bool IsSmartPtr> + struct drop_smart_ptr_impl; + template<typename T> + struct drop_smart_ptr_impl<T, true> { + using type = typename T::element_type; + }; + template<typename T> + struct drop_smart_ptr_impl<T, false> { + using type = T; + }; + template<typename T> + using drop_smart_ptr = drop_smart_ptr_impl<T, is_smart_ptr<T>::value>; + + // if return type is smart ptr take a type it points to instead + using return_type = typename drop_smart_ptr<Ret>::type; +public: + using type = return_type(typename remove_optional<In>::type...); +}; + +template<typename Serializer, typename MsgType> +template<typename Ret, typename... In> +auto protocol<Serializer, MsgType>::make_client(signature<Ret(In...)>, MsgType t) { + using sig_type = signature<typename client_function_type<Ret, In...>::type>; + return send_helper<Serializer>(t, sig_type()); +} + +template<typename Serializer, typename MsgType> +template<typename Func> +auto protocol<Serializer, MsgType>::make_client(MsgType t) { + return make_client(typename signature<typename function_traits<Func>::signature>::clean(), t); +} + +template<typename Serializer, typename MsgType> +template<typename Func> +auto protocol<Serializer, MsgType>::register_handler(MsgType t, scheduling_group sg, Func&& func) { + using sig_type = signature<typename function_traits<Func>::signature>; + using clean_sig_type = typename sig_type::clean; + using want_client_info = typename sig_type::want_client_info; + using want_time_point = typename sig_type::want_time_point; + auto recv = recv_helper<Serializer>(clean_sig_type(), std::forward<Func>(func), + want_client_info(), want_time_point()); + register_receiver(t, rpc_handler{sg, make_copyable_function(std::move(recv)), {}}); + return make_client(clean_sig_type(), t); +} + +template<typename Serializer, typename MsgType> +template<typename Func> +auto protocol<Serializer, MsgType>::register_handler(MsgType t, Func&& func) { + return register_handler(t, scheduling_group(), std::forward<Func>(func)); +} + +template<typename Serializer, typename MsgType> +future<> protocol<Serializer, MsgType>::unregister_handler(MsgType t) { + auto it = _handlers.find(t); + if (it != _handlers.end()) { + return it->second.use_gate.close().finally([this, t] { + _handlers.erase(t); + }); + } + return make_ready_future<>(); +} + +template<typename Serializer, typename MsgType> +bool protocol<Serializer, MsgType>::has_handler(MsgType msg_id) { + auto it = _handlers.find(msg_id); + if (it == _handlers.end()) { + return false; + } + return !it->second.use_gate.is_closed(); +} + +template<typename Serializer, typename MsgType> +rpc_handler* protocol<Serializer, MsgType>::get_handler(uint64_t msg_id) { + rpc_handler* h = nullptr; + auto it = _handlers.find(MsgType(msg_id)); + if (it != _handlers.end()) { + try { + it->second.use_gate.enter(); + h = &it->second; + } catch (gate_closed_exception&) { + // unregistered, just ignore + } + } + return h; +} + +template<typename Serializer, typename MsgType> +void protocol<Serializer, MsgType>::put_handler(rpc_handler* h) { + h->use_gate.leave(); +} + +template<typename T> T make_shard_local_buffer_copy(foreign_ptr<std::unique_ptr<T>> org); + +template<typename Serializer, typename... Out> +future<> sink_impl<Serializer, Out...>::operator()(const Out&... args) { + // note that we use remote serializer pointer, so if serailizer needs a state + // it should have per-cpu one + snd_buf data = marshall(this->_con->get()->template serializer<Serializer>(), 4, args...); + static_assert(snd_buf::chunk_size >= 4, "send buffer chunk size is too small"); + auto p = data.front().get_write(); + write_le<uint32_t>(p, data.size - 4); + // we do not want to dead lock on huge packets, so let them in + // but only one at a time + auto size = std::min(size_t(data.size), max_stream_buffers_memory); + const auto seq_num = _next_seq_num++; + return get_units(this->_sem, size).then([this, data = make_foreign(std::make_unique<snd_buf>(std::move(data))), seq_num] (semaphore_units<> su) mutable { + if (this->_ex) { + return make_exception_future(this->_ex); + } + // It is OK to discard this future. The user is required to + // wait for it when closing. + (void)smp::submit_to(this->_con->get_owner_shard(), [this, data = std::move(data), seq_num] () mutable { + connection* con = this->_con->get(); + if (con->error()) { + return make_exception_future(closed_error()); + } + if(con->sink_closed()) { + return make_exception_future(stream_closed()); + } + + auto& last_seq_num = _remote_state.last_seq_num; + auto& out_of_order_bufs = _remote_state.out_of_order_bufs; + + auto local_data = make_shard_local_buffer_copy(std::move(data)); + const auto seq_num_diff = seq_num - last_seq_num; + if (seq_num_diff > 1) { + auto [it, _] = out_of_order_bufs.emplace(seq_num, deferred_snd_buf{promise<>{}, std::move(local_data)}); + return it->second.pr.get_future(); + } + + last_seq_num = seq_num; + auto ret_fut = con->send(std::move(local_data), {}, nullptr); + while (!out_of_order_bufs.empty() && out_of_order_bufs.begin()->first == (last_seq_num + 1)) { + auto it = out_of_order_bufs.begin(); + last_seq_num = it->first; + auto fut = con->send(std::move(it->second.data), {}, nullptr); + fut.forward_to(std::move(it->second.pr)); + out_of_order_bufs.erase(it); + } + return ret_fut; + }).then_wrapped([su = std::move(su), this] (future<> f) { + if (f.failed() && !this->_ex) { // first error is the interesting one + this->_ex = f.get_exception(); + } else { + f.ignore_ready_future(); + } + }); + return make_ready_future<>(); + }); +} + +template<typename Serializer, typename... Out> +future<> sink_impl<Serializer, Out...>::flush() { + // wait until everything is sent out before returning. + return with_semaphore(this->_sem, max_stream_buffers_memory, [this] { + if (this->_ex) { + return make_exception_future(this->_ex); + } + return make_ready_future(); + }); +} + +template<typename Serializer, typename... Out> +future<> sink_impl<Serializer, Out...>::close() { + return with_semaphore(this->_sem, max_stream_buffers_memory, [this] { + return smp::submit_to(this->_con->get_owner_shard(), [this] { + connection* con = this->_con->get(); + if (con->sink_closed()) { // double close, should not happen! + return make_exception_future(stream_closed()); + } + future<> f = make_ready_future<>(); + if (!con->error() && !this->_ex) { + snd_buf data = marshall(con->template serializer<Serializer>(), 4); + static_assert(snd_buf::chunk_size >= 4, "send buffer chunk size is too small"); + auto p = data.front().get_write(); + write_le<uint32_t>(p, -1U); // max len fragment marks an end of a stream + f = con->send(std::move(data), {}, nullptr); + } else { + f = this->_ex ? make_exception_future(this->_ex) : make_exception_future(closed_error()); + } + return f.finally([con] { return con->close_sink(); }); + }); + }); +} + +template<typename Serializer, typename... Out> +sink_impl<Serializer, Out...>::~sink_impl() { + // A failure to close might leave some continuations running after + // this is destroyed, leading to use-after-free bugs. + assert(this->_con->get()->sink_closed()); +} + +template<typename Serializer, typename... In> +future<std::optional<std::tuple<In...>>> source_impl<Serializer, In...>::operator()() { + auto process_one_buffer = [this] { + foreign_ptr<std::unique_ptr<rcv_buf>> buf = std::move(this->_bufs.front()); + this->_bufs.pop_front(); + return std::apply([] (In&&... args) { + auto ret = std::make_optional(std::make_tuple(std::move(args)...)); + return make_ready_future<std::optional<std::tuple<In...>>>(std::move(ret)); + }, unmarshall<Serializer, In...>(*this->_con->get(), make_shard_local_buffer_copy(std::move(buf)))); + }; + + if (!this->_bufs.empty()) { + return process_one_buffer(); + } + + // refill buffers from remote cpu + return smp::submit_to(this->_con->get_owner_shard(), [this] () -> future<> { + connection* con = this->_con->get(); + if (con->_source_closed) { + return make_exception_future<>(stream_closed()); + } + return con->stream_receive(this->_bufs).then_wrapped([this, con] (future<>&& f) { + if (f.failed()) { + return con->close_source().then_wrapped([ex = f.get_exception()] (future<> f){ + f.ignore_ready_future(); + return make_exception_future<>(ex); + }); + } + if (this->_bufs.empty()) { // nothing to read -> eof + return con->close_source().then_wrapped([] (future<> f) { + f.ignore_ready_future(); + return make_ready_future<>(); + }); + } + return make_ready_future<>(); + }); + }).then([this, process_one_buffer] () { + if (this->_bufs.empty()) { + return make_ready_future<std::optional<std::tuple<In...>>>(std::nullopt); + } else { + return process_one_buffer(); + } + }); +} + +template<typename... Out> +connection_id sink<Out...>::get_id() const { + return _impl->_con->get()->get_connection_id(); +} + +template<typename... In> +connection_id source<In...>::get_id() const { + return _impl->_con->get()->get_connection_id(); +} + +template<typename... In> +template<typename Serializer, typename... Out> +sink<Out...> source<In...>::make_sink() { + return sink<Out...>(make_shared<sink_impl<Serializer, Out...>>(_impl->_con)); +} + +} + +} + +namespace std { +template<> +struct hash<seastar::rpc::streaming_domain_type> { + size_t operator()(const seastar::rpc::streaming_domain_type& domain) const { + size_t h = 0; + boost::hash_combine(h, std::hash<uint64_t>{}(domain._id)); + return h; + } +}; +} + + diff --git a/src/seastar/include/seastar/rpc/rpc_types.hh b/src/seastar/include/seastar/rpc/rpc_types.hh new file mode 100644 index 000000000..485e6c8fa --- /dev/null +++ b/src/seastar/include/seastar/rpc/rpc_types.hh @@ -0,0 +1,410 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <seastar/net/api.hh> +#include <stdexcept> +#include <string> +#include <boost/any.hpp> +#include <boost/type.hpp> +#include <seastar/util/std-compat.hh> +#include <seastar/util/variant_utils.hh> +#include <seastar/core/timer.hh> +#include <seastar/core/circular_buffer.hh> +#include <seastar/core/simple-stream.hh> +#include <seastar/core/lowres_clock.hh> +#include <boost/functional/hash.hpp> +#include <seastar/core/sharded.hh> + +namespace seastar { + +namespace rpc { + +using rpc_clock_type = lowres_clock; + +// used to tag a type for serializers +template<typename T> +using type = boost::type<T>; + +struct stats { + using counter_type = uint64_t; + counter_type replied = 0; + counter_type pending = 0; + counter_type exception_received = 0; + counter_type sent_messages = 0; + counter_type wait_reply = 0; + counter_type timeout = 0; +}; + + +struct client_info { + socket_address addr; + std::unordered_map<sstring, boost::any> user_data; + template <typename T> + void attach_auxiliary(const sstring& key, T&& object) { + user_data.emplace(key, boost::any(std::forward<T>(object))); + } + template <typename T> + T& retrieve_auxiliary(const sstring& key) { + auto it = user_data.find(key); + assert(it != user_data.end()); + return boost::any_cast<T&>(it->second); + } + template <typename T> + typename std::add_const<T>::type& retrieve_auxiliary(const sstring& key) const { + return const_cast<client_info*>(this)->retrieve_auxiliary<typename std::add_const<T>::type>(key); + } + template <typename T> + T* retrieve_auxiliary_opt(const sstring& key) noexcept { + auto it = user_data.find(key); + if (it == user_data.end()) { + return nullptr; + } + return &boost::any_cast<T&>(it->second); + } + template <typename T> + const T* retrieve_auxiliary_opt(const sstring& key) const noexcept { + auto it = user_data.find(key); + if (it == user_data.end()) { + return nullptr; + } + return &boost::any_cast<const T&>(it->second); + } +}; + +class error : public std::runtime_error { +public: + error(const std::string& msg) : std::runtime_error(msg) {} +}; + +class closed_error : public error { +public: + closed_error() : error("connection is closed") {} +}; + +class timeout_error : public error { +public: + timeout_error() : error("rpc call timed out") {} +}; + +class unknown_verb_error : public error { +public: + uint64_t type; + unknown_verb_error(uint64_t type_) : error("unknown verb"), type(type_) {} +}; + +class unknown_exception_error : public error { +public: + unknown_exception_error() : error("unknown exception") {} +}; + +class rpc_protocol_error : public error { +public: + rpc_protocol_error() : error("rpc protocol exception") {} +}; + +class canceled_error : public error { +public: + canceled_error() : error("rpc call was canceled") {} +}; + +class stream_closed : public error { +public: + stream_closed() : error("rpc stream was closed by peer") {} +}; + +class remote_verb_error : public error { + using error::error; +}; + +struct no_wait_type {}; + +// return this from a callback if client does not want to waiting for a reply +extern no_wait_type no_wait; + +/// \addtogroup rpc +/// @{ + +template <typename T> +class optional : public std::optional<T> { +public: + using std::optional<T>::optional; +}; + +class opt_time_point : public std::optional<rpc_clock_type::time_point> { +public: + using std::optional<rpc_clock_type::time_point>::optional; + opt_time_point(std::optional<rpc_clock_type::time_point> time_point) { + static_cast<std::optional<rpc_clock_type::time_point>&>(*this) = time_point; + } +}; + +/// @} + +struct cancellable { + std::function<void()> cancel_send; + std::function<void()> cancel_wait; + cancellable** send_back_pointer = nullptr; + cancellable** wait_back_pointer = nullptr; + cancellable() = default; + cancellable(cancellable&& x) : cancel_send(std::move(x.cancel_send)), cancel_wait(std::move(x.cancel_wait)), send_back_pointer(x.send_back_pointer), wait_back_pointer(x.wait_back_pointer) { + if (send_back_pointer) { + *send_back_pointer = this; + x.send_back_pointer = nullptr; + } + if (wait_back_pointer) { + *wait_back_pointer = this; + x.wait_back_pointer = nullptr; + } + } + cancellable& operator=(cancellable&& x) { + if (&x != this) { + this->~cancellable(); + new (this) cancellable(std::move(x)); + } + return *this; + } + void cancel() { + if (cancel_send) { + cancel_send(); + } + if (cancel_wait) { + cancel_wait(); + } + } + ~cancellable() { + cancel(); + } +}; + +struct rcv_buf { + uint32_t size = 0; + std::optional<semaphore_units<>> su; + std::variant<std::vector<temporary_buffer<char>>, temporary_buffer<char>> bufs; + using iterator = std::vector<temporary_buffer<char>>::iterator; + rcv_buf() {} + explicit rcv_buf(size_t size_) : size(size_) {} + explicit rcv_buf(temporary_buffer<char> b) : size(b.size()), bufs(std::move(b)) {}; + explicit rcv_buf(std::vector<temporary_buffer<char>> bufs, size_t size) + : size(size), bufs(std::move(bufs)) {}; +}; + +struct snd_buf { + // Preferred, but not required, chunk size. + static constexpr size_t chunk_size = 128*1024; + uint32_t size = 0; + std::variant<std::vector<temporary_buffer<char>>, temporary_buffer<char>> bufs; + using iterator = std::vector<temporary_buffer<char>>::iterator; + snd_buf() {} + snd_buf(snd_buf&&) noexcept; + snd_buf& operator=(snd_buf&&) noexcept; + explicit snd_buf(size_t size_); + explicit snd_buf(temporary_buffer<char> b) : size(b.size()), bufs(std::move(b)) {}; + + explicit snd_buf(std::vector<temporary_buffer<char>> bufs, size_t size) + : size(size), bufs(std::move(bufs)) {}; + + temporary_buffer<char>& front(); +}; + +static inline memory_input_stream<rcv_buf::iterator> make_deserializer_stream(rcv_buf& input) { + auto* b = std::get_if<temporary_buffer<char>>(&input.bufs); + if (b) { + return memory_input_stream<rcv_buf::iterator>(memory_input_stream<rcv_buf::iterator>::simple(b->begin(), b->size())); + } else { + auto& ar = std::get<std::vector<temporary_buffer<char>>>(input.bufs); + return memory_input_stream<rcv_buf::iterator>(memory_input_stream<rcv_buf::iterator>::fragmented(ar.begin(), input.size)); + } +} + +class compressor { +public: + virtual ~compressor() {} + // compress data and leave head_space bytes at the beginning of returned buffer + virtual snd_buf compress(size_t head_space, snd_buf data) = 0; + // decompress data + virtual rcv_buf decompress(rcv_buf data) = 0; + virtual sstring name() const = 0; + + // factory to create compressor for a connection + class factory { + public: + virtual ~factory() {} + // return feature string that will be sent as part of protocol negotiation + virtual const sstring& supported() const = 0; + // negotiate compress algorithm + virtual std::unique_ptr<compressor> negotiate(sstring feature, bool is_server) const = 0; + }; +}; + +class connection; + +struct connection_id { + uint64_t id; + bool operator==(const connection_id& o) const { + return id == o.id; + } + explicit operator bool() const { + return shard() != 0xffff; + } + size_t shard() const { + return size_t(id & 0xffff); + } + constexpr static connection_id make_invalid_id(uint64_t id = 0) { + return make_id(id, 0xffff); + } + constexpr static connection_id make_id(uint64_t id, uint16_t shard) { + return {id << 16 | shard}; + } +}; + +constexpr connection_id invalid_connection_id = connection_id::make_invalid_id(); + +std::ostream& operator<<(std::ostream&, const connection_id&); + +using xshard_connection_ptr = lw_shared_ptr<foreign_ptr<shared_ptr<connection>>>; +constexpr size_t max_queued_stream_buffers = 50; +constexpr size_t max_stream_buffers_memory = 100 * 1024; + +/// \addtogroup rpc +/// @{ + +// send data Out... +template<typename... Out> +class sink { +public: + class impl { + protected: + xshard_connection_ptr _con; + semaphore _sem; + std::exception_ptr _ex; + impl(xshard_connection_ptr con) : _con(std::move(con)), _sem(max_stream_buffers_memory) {} + public: + virtual ~impl() {}; + virtual future<> operator()(const Out&... args) = 0; + virtual future<> close() = 0; + virtual future<> flush() = 0; + friend sink; + }; + +private: + shared_ptr<impl> _impl; + +public: + sink(shared_ptr<impl> impl) : _impl(std::move(impl)) {} + future<> operator()(const Out&... args) { + return _impl->operator()(args...); + } + future<> close() { + return _impl->close(); + } + // Calling this function makes sure that any data buffered + // by the stream sink will be flushed to the network. + // It does not mean the data was received by the corresponding + // source. + future<> flush() { + return _impl->flush(); + } + connection_id get_id() const; +}; + +// receive data In... +template<typename... In> +class source { +public: + class impl { + protected: + xshard_connection_ptr _con; + circular_buffer<foreign_ptr<std::unique_ptr<rcv_buf>>> _bufs; + impl(xshard_connection_ptr con) : _con(std::move(con)) { + _bufs.reserve(max_queued_stream_buffers); + } + public: + virtual ~impl() {} + virtual future<std::optional<std::tuple<In...>>> operator()() = 0; + friend source; + }; +private: + shared_ptr<impl> _impl; + +public: + source(shared_ptr<impl> impl) : _impl(std::move(impl)) {} + future<std::optional<std::tuple<In...>>> operator()() { + return _impl->operator()(); + }; + connection_id get_id() const; + template<typename Serializer, typename... Out> sink<Out...> make_sink(); +}; + +/// Used to return multiple values in rpc without variadic futures +/// +/// If you wish to return multiple values from an rpc procedure, use a +/// signature `future<rpc::tuple<return type list> (argument list)>>`. This +/// will be marshalled by rpc, so you do not need to have your Serializer +/// serialize/deserialize this tuple type. The serialization format is +/// compatible with the deprecated variadic future support, and is compatible +/// with adding new return types in a backwards compatible way provided new +/// parameters are appended only, and wrapped with rpc::optional: +/// `future<rpc::tuple<existing return type list, rpc::optional<new_return_type>>> (argument list)` +/// +/// You may also use another tuple type, such as std::tuple. In this case, +/// your Serializer type must recognize your tuple type and provide serialization +/// and deserialization for it. +template <typename... T> +class tuple : public std::tuple<T...> { +public: + using std::tuple<T...>::tuple; + tuple(std::tuple<T...>&& x) : std::tuple<T...>(std::move(x)) {} +}; + +/// @} + +template <typename... T> +tuple(T&&...) -> tuple<T...>; + +} // namespace rpc + +} + +namespace std { +template<> +struct hash<seastar::rpc::connection_id> { + size_t operator()(const seastar::rpc::connection_id& id) const { + size_t h = 0; + boost::hash_combine(h, std::hash<uint64_t>{}(id.id)); + return h; + } +}; + +template <typename... T> +struct tuple_size<seastar::rpc::tuple<T...>> : tuple_size<tuple<T...>> { +}; + +template <size_t I, typename... T> +struct tuple_element<I, seastar::rpc::tuple<T...>> : tuple_element<I, tuple<T...>> { +}; + +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<seastar::rpc::connection_id> : fmt::ostream_formatter {}; +#endif diff --git a/src/seastar/include/seastar/testing/entry_point.hh b/src/seastar/include/seastar/testing/entry_point.hh new file mode 100644 index 000000000..f352db11d --- /dev/null +++ b/src/seastar/include/seastar/testing/entry_point.hh @@ -0,0 +1,33 @@ +/* + * 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 Ltd. + */ + +#pragma once + +namespace seastar { + +namespace testing { + +int entry_point(int argc, char **argv); + +} + +}
\ No newline at end of file diff --git a/src/seastar/include/seastar/testing/exchanger.hh b/src/seastar/include/seastar/testing/exchanger.hh new file mode 100644 index 000000000..6b91170ee --- /dev/null +++ b/src/seastar/include/seastar/testing/exchanger.hh @@ -0,0 +1,88 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <mutex> +#include <condition_variable> +#include <seastar/util/std-compat.hh> + +namespace seastar { + +namespace testing { + +class exchanger_base { +protected: + exchanger_base(); + ~exchanger_base(); + std::mutex _mutex; + std::condition_variable _cv; + std::exception_ptr _exception; + void interrupt_ptr(std::exception_ptr e) { + std::unique_lock<std::mutex> lock(_mutex); + if (!_exception) { + _exception = e; + _cv.notify_all(); + } + // FIXME: log if already interrupted + } +}; + +// Single-element blocking queue +template <typename T> +class exchanger : public exchanger_base { +private: + std::optional<T> _element; + +public: + template <typename Exception> + void interrupt(Exception e) { + try { + throw e; + } catch (...) { + interrupt_ptr(std::current_exception()); + } + } + void give(T value) { + std::unique_lock<std::mutex> lock(_mutex); + _cv.wait(lock, [this] { return !_element || _exception; }); + if (_exception) { + std::rethrow_exception(_exception); + } + _element = value; + _cv.notify_one(); + } + T take() { + std::unique_lock<std::mutex> lock(_mutex); + _cv.wait(lock, [this] { return bool(_element) || _exception; }); + if (_exception) { + std::rethrow_exception(_exception); + } + auto v = *_element; + _element = {}; + _cv.notify_one(); + return v; + } +}; + +} + +} diff --git a/src/seastar/include/seastar/testing/linux_perf_event.hh b/src/seastar/include/seastar/testing/linux_perf_event.hh new file mode 100644 index 000000000..00cc6fd28 --- /dev/null +++ b/src/seastar/include/seastar/testing/linux_perf_event.hh @@ -0,0 +1,49 @@ +/* + * Copyright (C) 2021-present ScyllaDB + */ + +/* + * 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. + */ + +/* + * This file was copied from Scylla (https://github.com/scylladb/scylla) + */ + +#pragma once + + +#include <cstdint> +#include <utility> +#include <unistd.h> + +struct perf_event_attr; // from <linux/perf_event.h> + +class linux_perf_event { + int _fd = -1; +public: + linux_perf_event(const struct ::perf_event_attr& attr, pid_t pid, int cpu, int group_fd, unsigned long flags); + linux_perf_event(linux_perf_event&& x) noexcept : _fd(std::exchange(x._fd, -1)) {} + linux_perf_event& operator=(linux_perf_event&& x) noexcept; + ~linux_perf_event(); + uint64_t read(); + void enable(); + void disable(); +public: + static linux_perf_event user_instructions_retired(); +}; + diff --git a/src/seastar/include/seastar/testing/on_internal_error.hh b/src/seastar/include/seastar/testing/on_internal_error.hh new file mode 100644 index 000000000..c9beaccc3 --- /dev/null +++ b/src/seastar/include/seastar/testing/on_internal_error.hh @@ -0,0 +1,41 @@ +/* + * 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 2020 ScyllaDB + */ + +#pragma once + +namespace seastar { +namespace testing { + +// Disables aborting in on_internal_error() for a scope. +// +// Intended for tests, which want to test error paths that invoke +// on_internal_error() without aborting, at the same time, having it enabled +// for other, indirectly affected code paths, that are not a direct target of +// the test. +class scoped_no_abort_on_internal_error { + bool _prev; +public: + scoped_no_abort_on_internal_error() noexcept; + ~scoped_no_abort_on_internal_error(); +}; + +} +} diff --git a/src/seastar/include/seastar/testing/perf_tests.hh b/src/seastar/include/seastar/testing/perf_tests.hh new file mode 100644 index 000000000..60b6fc1e0 --- /dev/null +++ b/src/seastar/include/seastar/testing/perf_tests.hh @@ -0,0 +1,383 @@ +/* + * 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 Ltd. + */ + +#pragma once + +#include <atomic> +#include <memory> + +#include <fmt/format.h> + +#include <seastar/core/future.hh> +#include <seastar/core/loop.hh> +#include <seastar/testing/linux_perf_event.hh> + +#ifdef SEASTAR_COROUTINES_ENABLED +#include <seastar/core/coroutine.hh> +#endif + +using namespace seastar; + +namespace perf_tests { +namespace internal { + +struct config; + +using clock_type = std::chrono::steady_clock; + +class perf_stats { +public: + uint64_t allocations = 0; + uint64_t tasks_executed = 0; + uint64_t instructions_retired = 0; + +private: + static uint64_t perf_mallocs(); + static uint64_t perf_tasks_processed(); + +public: + perf_stats() = default; + perf_stats(uint64_t allocations_, uint64_t tasks_executed_, uint64_t instructions_retired_ = 0) + : allocations(allocations_) + , tasks_executed(tasks_executed_) + , instructions_retired(instructions_retired_) + {} + perf_stats(perf_stats&& o) noexcept + : allocations(std::exchange(o.allocations, 0)) + , tasks_executed(std::exchange(o.tasks_executed, 0)) + , instructions_retired(std::exchange(o.instructions_retired, 0)) + {} + perf_stats(const perf_stats& o) = default; + + perf_stats& operator=(perf_stats&& o) = default; + perf_stats& operator=(const perf_stats& o) = default; + + perf_stats& operator+=(perf_stats b); + perf_stats& operator-=(perf_stats b); + + static perf_stats snapshot(linux_perf_event* instructions_retired_counter = nullptr); +}; + +inline +perf_stats +operator+(perf_stats a, perf_stats b) { + a.allocations += b.allocations; + a.tasks_executed += b.tasks_executed; + a.instructions_retired += b.instructions_retired; + return a; +} + +inline +perf_stats +operator-(perf_stats a, perf_stats b) { + a.allocations -= b.allocations; + a.tasks_executed -= b.tasks_executed; + a.instructions_retired -= b.instructions_retired; + return a; +} + +inline perf_stats& perf_stats::operator+=(perf_stats b) { + allocations += b.allocations; + tasks_executed += b.tasks_executed; + instructions_retired += b.instructions_retired; + return *this; +} + +inline perf_stats& perf_stats::operator-=(perf_stats b) { + allocations -= b.allocations; + tasks_executed -= b.tasks_executed; + instructions_retired -= b.instructions_retired; + return *this; +} + +class performance_test { + std::string _test_case; + std::string _test_group; + + uint64_t _single_run_iterations = 0; + std::atomic<uint64_t> _max_single_run_iterations; +protected: + linux_perf_event _instructions_retired_counter = linux_perf_event::user_instructions_retired(); +private: + void do_run(const config&); +public: + struct run_result { + clock_type::duration duration; + perf_stats stats; + }; +protected: + [[gnu::always_inline]] [[gnu::hot]] + bool stop_iteration() const { + return _single_run_iterations >= _max_single_run_iterations.load(std::memory_order_relaxed); + } + + [[gnu::always_inline]] [[gnu::hot]] + void next_iteration(size_t n) { + _single_run_iterations += n; + } + + virtual void set_up() = 0; + virtual void tear_down() noexcept = 0; + virtual future<run_result> do_single_run() = 0; +public: + performance_test(const std::string& test_case, const std::string& test_group) + : _test_case(test_case) + , _test_group(test_group) + { } + + virtual ~performance_test() = default; + + const std::string& test_case() const { return _test_case; } + const std::string& test_group() const { return _test_group; } + std::string name() const { return fmt::format("{}.{}", test_group(), test_case()); } + + void run(const config&); +public: + static void register_test(std::unique_ptr<performance_test>); +}; + +// Helper for measuring time. +// Each microbenchmark can either use the default behaviour which measures +// only the start and stop time of the whole run or manually invoke +// start_measuring_time() and stop_measuring_time() in order to measure +// only parts of each iteration. +class time_measurement { + clock_type::time_point _run_start_time; + clock_type::time_point _start_time; + clock_type::duration _total_time; + + perf_stats _start_stats; + perf_stats _total_stats; + + linux_perf_event* _instructions_retired_counter = nullptr; + +public: + [[gnu::always_inline]] [[gnu::hot]] + void start_run(linux_perf_event* instructions_retired_counter = nullptr) { + _instructions_retired_counter = instructions_retired_counter; + _total_time = { }; + _total_stats = {}; + auto t = clock_type::now(); + _run_start_time = t; + _start_time = t; + _start_stats = perf_stats::snapshot(_instructions_retired_counter); + } + + [[gnu::always_inline]] [[gnu::hot]] + performance_test::run_result stop_run() { + auto t = clock_type::now(); + performance_test::run_result ret; + if (_start_time == _run_start_time) { + ret.duration = t - _start_time; + auto stats = perf_stats::snapshot(_instructions_retired_counter); + ret.stats = stats - _start_stats; + } else { + ret.duration = _total_time; + ret.stats = _total_stats; + } + _instructions_retired_counter = nullptr; + return ret; + } + + [[gnu::always_inline]] [[gnu::hot]] + void start_iteration() { + _start_time = clock_type::now(); + _start_stats = perf_stats::snapshot(_instructions_retired_counter); + } + + [[gnu::always_inline]] [[gnu::hot]] + void stop_iteration() { + auto t = clock_type::now(); + _total_time += t - _start_time; + perf_stats stats; + stats = perf_stats::snapshot(_instructions_retired_counter); + _total_stats += stats - _start_stats; + } +}; + +extern time_measurement measure_time; + +namespace { + +template<bool Condition, typename TrueFn, typename FalseFn> +struct do_if_constexpr_ : FalseFn { + do_if_constexpr_(TrueFn, FalseFn false_fn) : FalseFn(std::move(false_fn)) { } + decltype(auto) operator()() const { + // https://gcc.gnu.org/bugzilla/show_bug.cgi?id=64095 + return FalseFn::operator()(0); + } +}; +template<typename TrueFn, typename FalseFn> +struct do_if_constexpr_<true, TrueFn, FalseFn> : TrueFn { + do_if_constexpr_(TrueFn true_fn, FalseFn) : TrueFn(std::move(true_fn)) { } + decltype(auto) operator()() const { return TrueFn::operator()(0); } +}; + +template<bool Condition, typename TrueFn, typename FalseFn> +do_if_constexpr_<Condition, TrueFn, FalseFn> if_constexpr_(TrueFn&& true_fn, FalseFn&& false_fn) +{ + return do_if_constexpr_<Condition, TrueFn, FalseFn>(std::forward<TrueFn>(true_fn), + std::forward<FalseFn>(false_fn)); +} + +} + +template<typename Test> +class concrete_performance_test final : public performance_test { + std::optional<Test> _test; +private: + template<typename... Args> + auto run_test(Args&&...) { + return _test->run(); + } + +protected: + virtual void set_up() override { + _test.emplace(); + } + + virtual void tear_down() noexcept override { + _test = std::nullopt; + } + + [[gnu::hot]] + virtual future<run_result> do_single_run() override { + // Redundant 'this->'s courtesy of https://gcc.gnu.org/bugzilla/show_bug.cgi?id=61636 + _instructions_retired_counter.enable(); + return if_constexpr_<is_future<decltype(_test->run())>::value>([&] (auto&&...) { + measure_time.start_run(&_instructions_retired_counter); + return do_until([this] { return this->stop_iteration(); }, [this] { + return if_constexpr_<std::is_same<decltype(_test->run()), future<>>::value>([&] (auto&&...) { + this->next_iteration(1); + return _test->run(); + }, [&] (auto&&... dependency) { + // We need `dependency` to make sure the compiler won't be able to instantiate anything + // (and notice that the code does not compile) if this part of if_constexpr_ is not active. + return run_test(dependency...).then([&] (size_t n) { + this->next_iteration(n); + }); + })(); + }).then([] { + return measure_time.stop_run(); + }).finally([this] { + _instructions_retired_counter.disable(); + }); + }, [&] (auto&&...) { + measure_time.start_run(&_instructions_retired_counter); + while (!stop_iteration()) { + if_constexpr_<std::is_void<decltype(_test->run())>::value>([&] (auto&&...) { + (void)_test->run(); + this->next_iteration(1); + }, [&] (auto&&... dependency) { + // We need `dependency` to make sure the compiler won't be able to instantiate anything + // (and notice that the code does not compile) if this part of if_constexpr_ is not active. + this->next_iteration(run_test(dependency...)); + })(); + } + auto ret = measure_time.stop_run(); + _instructions_retired_counter.disable(); + return make_ready_future<run_result>(std::move(ret)); + })(); + } +public: + using performance_test::performance_test; +}; + +void register_test(std::unique_ptr<performance_test>); + +template<typename Test> +struct test_registrar { + test_registrar(const std::string& test_group, const std::string& test_case) { + auto test = std::make_unique<concrete_performance_test<Test>>(test_case, test_group); + performance_test::register_test(std::move(test)); + } +}; + +} + +[[gnu::always_inline]] +inline void start_measuring_time() +{ + internal::measure_time.start_iteration(); +} + +[[gnu::always_inline]] +inline void stop_measuring_time() +{ + internal::measure_time.stop_iteration(); +} + + +template<typename T> +void do_not_optimize(const T& v) +{ + asm volatile("" : : "r,m" (v)); +} + +} + +// PERF_TEST and PERF_TEST_F support both synchronous and asynchronous functions. +// The former should return `void`, the latter `future<>`. +// PERF_TEST_C executes a coroutine function, if enabled. +// PERF_TEST_CN executes a coroutine function, if enabled, returning the number of inner-loops. +// +// Test cases may perform multiple operations in a single run, this may be desirable +// if the cost of an individual operation is very small. This allows measuring either +// the latency of throughput depending on how the test in written. In such cases, +// the test function shall return either size_t or future<size_t> for synchronous and +// asynchronous cases respectively. The returned value shall be the number of iterations +// done in a single test run. + +#define PERF_TEST_F(test_group, test_case) \ + struct test_##test_group##_##test_case : test_group { \ + [[gnu::always_inline]] inline auto run(); \ + }; \ + static ::perf_tests::internal::test_registrar<test_##test_group##_##test_case> \ + test_##test_group##_##test_case##_registrar(#test_group, #test_case); \ + [[gnu::always_inline]] auto test_##test_group##_##test_case::run() + +#define PERF_TEST(test_group, test_case) \ + struct test_##test_group##_##test_case { \ + [[gnu::always_inline]] inline auto run(); \ + }; \ + static ::perf_tests::internal::test_registrar<test_##test_group##_##test_case> \ + test_##test_group##_##test_case##_registrar(#test_group, #test_case); \ + [[gnu::always_inline]] auto test_##test_group##_##test_case::run() + +#ifdef SEASTAR_COROUTINES_ENABLED + +#define PERF_TEST_C(test_group, test_case) \ + struct test_##test_group##_##test_case : test_group { \ + inline future<> run(); \ + }; \ + static ::perf_tests::internal::test_registrar<test_##test_group##_##test_case> \ + test_##test_group##_##test_case##_registrar(#test_group, #test_case); \ + future<> test_##test_group##_##test_case::run() + +#define PERF_TEST_CN(test_group, test_case) \ + struct test_##test_group##_##test_case : test_group { \ + inline future<size_t> run(); \ + }; \ + static ::perf_tests::internal::test_registrar<test_##test_group##_##test_case> \ + test_##test_group##_##test_case##_registrar(#test_group, #test_case); \ + future<size_t> test_##test_group##_##test_case::run() + +#endif // SEASTAR_COROUTINES_ENABLED diff --git a/src/seastar/include/seastar/testing/random.hh b/src/seastar/include/seastar/testing/random.hh new file mode 100644 index 000000000..2259f1f23 --- /dev/null +++ b/src/seastar/include/seastar/testing/random.hh @@ -0,0 +1,34 @@ +/* + * 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) 2020 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <random> + +namespace seastar { + +namespace testing { + +extern thread_local std::default_random_engine local_random_engine; + +} // namespace testing + +} // namespace seastar diff --git a/src/seastar/include/seastar/testing/seastar_test.hh b/src/seastar/include/seastar/testing/seastar_test.hh new file mode 100644 index 000000000..7772a8016 --- /dev/null +++ b/src/seastar/include/seastar/testing/seastar_test.hh @@ -0,0 +1,68 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <vector> + +#include <boost/test/unit_test.hpp> + +#include <seastar/core/future.hh> +#include <seastar/util/std-compat.hh> +#include <seastar/testing/entry_point.hh> + +namespace seastar { + +namespace testing { + +class seastar_test { +public: + seastar_test(); + virtual ~seastar_test() {} + virtual const char* get_test_file() const = 0; + virtual const char* get_name() const = 0; + virtual int get_expected_failures() const { return 0; } + virtual future<> run_test_case() const = 0; + void run(); +}; + +const std::vector<seastar_test*>& known_tests(); + +// BOOST_REQUIRE_EXCEPTION predicates +namespace exception_predicate { + +std::function<bool(const std::exception&)> message_equals(std::string_view expected_message); +std::function<bool(const std::exception&)> message_contains(std::string_view expected_message); + +} // exception_predicate + +} + +} + +#ifdef SEASTAR_TESTING_MAIN + +int main(int argc, char** argv) { + return seastar::testing::entry_point(argc, argv); +} + +#endif // SEASTAR_TESTING_MAIN diff --git a/src/seastar/include/seastar/testing/test_case.hh b/src/seastar/include/seastar/testing/test_case.hh new file mode 100644 index 000000000..b8a23ab12 --- /dev/null +++ b/src/seastar/include/seastar/testing/test_case.hh @@ -0,0 +1,36 @@ +/* + * 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 Ltd. + */ + +#pragma once + +#include <seastar/core/future.hh> + +#include <seastar/testing/seastar_test.hh> + +#define SEASTAR_TEST_CASE(name) \ + struct name : public seastar::testing::seastar_test { \ + const char* get_test_file() const override { return __FILE__; } \ + const char* get_name() const override { return #name; } \ + seastar::future<> run_test_case() const override; \ + }; \ + static const name name ## _instance; /* NOLINT(cert-err58-cpp) */ \ + seastar::future<> name::run_test_case() const diff --git a/src/seastar/include/seastar/testing/test_runner.hh b/src/seastar/include/seastar/testing/test_runner.hh new file mode 100644 index 000000000..7b9e10767 --- /dev/null +++ b/src/seastar/include/seastar/testing/test_runner.hh @@ -0,0 +1,66 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <memory> +#include <functional> +#include <atomic> +#include <random> +#include <seastar/core/future.hh> +#include <seastar/core/posix.hh> +#include <seastar/testing/exchanger.hh> +#include <seastar/testing/random.hh> + +namespace seastar { + +namespace testing { + +class test_runner { +private: + std::unique_ptr<posix_thread> _thread; + std::atomic<bool> _started{false}; + exchanger<std::function<future<>()>> _task; + bool _done = false; + int _exit_code{0}; + + struct start_thread_args { + int ac; + char** av; + start_thread_args(int ac_, char** av_) noexcept : ac(ac_), av(av_) {} + }; + std::unique_ptr<start_thread_args> _st_args; + void start_thread(int ac, char** av); +public: + // Returns whether initialization was successful. + // Will return as soon as the seastar::app was started. + bool start(int argc, char** argv); + ~test_runner(); + void run_sync(std::function<future<>()> task); + // Returns the return value of the underlying `seastar::app::run()`. + int finalize(); +}; + +test_runner& global_test_runner(); + +} + +} diff --git a/src/seastar/include/seastar/testing/thread_test_case.hh b/src/seastar/include/seastar/testing/thread_test_case.hh new file mode 100644 index 000000000..6367243ae --- /dev/null +++ b/src/seastar/include/seastar/testing/thread_test_case.hh @@ -0,0 +1,46 @@ +/* + * 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 Ltd. + */ + +#pragma once + +#include <seastar/core/future.hh> +#include <seastar/core/thread.hh> + +#include <seastar/testing/seastar_test.hh> + +#define SEASTAR_THREAD_TEST_CASE_EXPECTED_FAILURES(name, failures) \ + struct name : public seastar::testing::seastar_test { \ + const char* get_test_file() const override { return __FILE__; } \ + const char* get_name() const override { return #name; } \ + int get_expected_failures() const override { return failures; } \ + seastar::future<> run_test_case() const override { \ + return seastar::async([this] { \ + do_run_test_case(); \ + }); \ + } \ + void do_run_test_case() const; \ + }; \ + static const name name ## _instance; /* NOLINT(cert-err58-cpp) */ \ + void name::do_run_test_case() const + +#define SEASTAR_THREAD_TEST_CASE(name) \ + SEASTAR_THREAD_TEST_CASE_EXPECTED_FAILURES(name, 0) diff --git a/src/seastar/include/seastar/util/alloc_failure_injector.hh b/src/seastar/include/seastar/util/alloc_failure_injector.hh new file mode 100644 index 000000000..163d2d9fc --- /dev/null +++ b/src/seastar/include/seastar/util/alloc_failure_injector.hh @@ -0,0 +1,132 @@ +/* + * 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 2017 ScyllaDB + */ + +#pragma once + +#include <limits> +#include <cstdint> +#include <functional> +#include <seastar/util/noncopyable_function.hh> +#include <seastar/util/critical_alloc_section.hh> + +namespace seastar { +namespace memory { + +/// +/// Allocation failure injection framework. Allows testing for exception safety. +/// +/// To exhaustively inject failure at every allocation point: +/// +/// uint64_t i = 0; +/// while (true) { +/// try { +/// local_failure_injector().fail_after(i++); +/// code_under_test(); +/// local_failure_injector().cancel(); +/// break; +/// } catch (const std::bad_alloc&) { +/// // expected +/// } +/// } +class alloc_failure_injector { + uint64_t _alloc_count = 0; + uint64_t _fail_at = std::numeric_limits<uint64_t>::max(); + noncopyable_function<void()> _on_alloc_failure = [] { throw std::bad_alloc(); }; + bool _failed = false; +private: + void fail(); +public: + /// \brief Marks a point in code which should be considered for failure injection. + void on_alloc_point() { + if (is_critical_alloc_section()) { + return; + } + if (_alloc_count >= _fail_at) { + fail(); + } + ++_alloc_count; + } + + /// Counts encountered allocation points which didn't fail and didn't have failure suppressed. + uint64_t alloc_count() const { + return _alloc_count; + } + + /// Will cause count-th allocation point from now to fail, counting from 0. + void fail_after(uint64_t count) { + _fail_at = _alloc_count + count; + _failed = false; + } + + /// Cancels the failure scheduled by fail_after(). + void cancel() { + _fail_at = std::numeric_limits<uint64_t>::max(); + } + + /// Returns true iff allocation was failed since last fail_after(). + bool failed() const { + return _failed; + } + + /// Runs given function with a custom failure action instead of the default std::bad_alloc throw. + void run_with_callback(noncopyable_function<void()> callback, noncopyable_function<void()> to_run); +}; + +/// \cond internal +extern thread_local alloc_failure_injector the_alloc_failure_injector; +/// \endcond + +/// \brief Return the shard-local \ref alloc_failure_injector instance. +inline +alloc_failure_injector& local_failure_injector() { + return the_alloc_failure_injector; +} + +#ifdef SEASTAR_ENABLE_ALLOC_FAILURE_INJECTION + +#ifdef SEASTAR_DEFAULT_ALLOCATOR +#error SEASTAR_ENABLE_ALLOC_FAILURE_INJECTION is not supported when using SEASTAR_DEFAULT_ALLOCATOR +#endif + +#endif + + +struct [[deprecated("Use scoped_critical_section instead")]] disable_failure_guard { + scoped_critical_alloc_section cs; +}; + +/// \brief Marks a point in code which should be considered for failure injection. +inline +void on_alloc_point() { +#ifdef SEASTAR_ENABLE_ALLOC_FAILURE_INJECTION + local_failure_injector().on_alloc_point(); +#endif +} + +/// Repeatedly run func with allocation failures +/// +/// Initially, allocations start to fail immediately. In each +/// subsequent run the failures start one allocation later. This +/// returns when func is run and no allocation failures are detected. +void with_allocation_failures(noncopyable_function<void()> func); + +} +} diff --git a/src/seastar/include/seastar/util/attribute-compat.hh b/src/seastar/include/seastar/util/attribute-compat.hh new file mode 100644 index 000000000..243d9b870 --- /dev/null +++ b/src/seastar/include/seastar/util/attribute-compat.hh @@ -0,0 +1,27 @@ +/* + * 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) 2019 ScyllaDB + */ +#pragma once + +#if defined(__has_cpp_attribute) && __has_cpp_attribute(nodiscard) + #define SEASTAR_NODISCARD [[nodiscard]] +#else + #define SEASTAR_NODISCARD +#endif diff --git a/src/seastar/include/seastar/util/backtrace.hh b/src/seastar/include/seastar/util/backtrace.hh new file mode 100644 index 000000000..2c7a640b4 --- /dev/null +++ b/src/seastar/include/seastar/util/backtrace.hh @@ -0,0 +1,238 @@ +/* + * 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 2016 ScyllaDB + */ + +#pragma once + +#include <execinfo.h> +#include <iosfwd> +#include <variant> +#include <boost/container/static_vector.hpp> + +#include <seastar/core/sstring.hh> +#include <seastar/core/print.hh> +#include <seastar/core/scheduling.hh> +#include <seastar/core/shared_ptr.hh> + +namespace seastar { + +struct shared_object { + sstring name; + uintptr_t begin; + uintptr_t end; // C++-style, last addr + 1 +}; + +struct frame { + const shared_object* so; + uintptr_t addr; +}; + +bool operator==(const frame& a, const frame& b) noexcept; + + +// If addr doesn't seem to belong to any of the provided shared objects, it +// will be considered as part of the executable. +frame decorate(uintptr_t addr) noexcept; + +// Invokes func for each frame passing it as argument. +template<typename Func> +void backtrace(Func&& func) noexcept(noexcept(func(frame()))) { + constexpr size_t max_backtrace = 100; + void* buffer[max_backtrace]; + int n = ::backtrace(buffer, max_backtrace); + for (int i = 0; i < n; ++i) { + auto ip = reinterpret_cast<uintptr_t>(buffer[i]); + func(decorate(ip - 1)); + } +} + +// Represents a call stack of a single thread. +class simple_backtrace { +public: + using vector_type = boost::container::static_vector<frame, 64>; +private: + vector_type _frames; + size_t _hash; + char _delimeter; +private: + size_t calculate_hash() const noexcept; +public: + simple_backtrace(char delimeter = ' ') noexcept : _delimeter(delimeter) {} + simple_backtrace(vector_type f, char delimeter = ' ') noexcept : _frames(std::move(f)), _delimeter(delimeter) {} + + size_t hash() const noexcept { return _hash; } + char delimeter() const noexcept { return _delimeter; } + + friend std::ostream& operator<<(std::ostream& out, const simple_backtrace&); + + bool operator==(const simple_backtrace& o) const noexcept { + return _hash == o._hash && _frames == o._frames; + } + + bool operator!=(const simple_backtrace& o) const noexcept { + return !(*this == o); + } +}; + +using shared_backtrace = seastar::lw_shared_ptr<simple_backtrace>; + +// Represents a task object inside a tasktrace. +class task_entry { + const std::type_info* _task_type; +public: + task_entry(const std::type_info& ti) noexcept + : _task_type(&ti) + { } + + friend std::ostream& operator<<(std::ostream& out, const task_entry&); + + bool operator==(const task_entry& o) const noexcept { + return *_task_type == *o._task_type; + } + + bool operator!=(const task_entry& o) const noexcept { + return !(*this == o); + } + + size_t hash() const noexcept { return _task_type->hash_code(); } +}; + +// Extended backtrace which consists of a backtrace of the currently running task +// and information about the chain of tasks waiting for the current operation to complete. +class tasktrace { +public: + using entry = std::variant<shared_backtrace, task_entry>; + using vector_type = boost::container::static_vector<entry, 16>; +private: + simple_backtrace _main; + vector_type _prev; + scheduling_group _sg; + size_t _hash; +public: + tasktrace() = default; + tasktrace(simple_backtrace main, vector_type prev, size_t prev_hash, scheduling_group sg) noexcept; + ~tasktrace(); + + size_t hash() const noexcept { return _hash; } + char delimeter() const noexcept { return _main.delimeter(); } + + friend std::ostream& operator<<(std::ostream& out, const tasktrace&); + + bool operator==(const tasktrace& o) const noexcept; + + bool operator!=(const tasktrace& o) const noexcept { + return !(*this == o); + } +}; + +} + +namespace std { + +template<> +struct hash<seastar::simple_backtrace> { + size_t operator()(const seastar::simple_backtrace& b) const { + return b.hash(); + } +}; + +template<> +struct hash<seastar::tasktrace> { + size_t operator()(const seastar::tasktrace& b) const { + return b.hash(); + } +}; + +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<seastar::tasktrace> : fmt::ostream_formatter {}; +#endif + +namespace seastar { + +using saved_backtrace = tasktrace; + +saved_backtrace current_backtrace() noexcept; + +tasktrace current_tasktrace() noexcept; + +// Collects backtrace only within the currently executing task. +simple_backtrace current_backtrace_tasklocal() noexcept; + +std::ostream& operator<<(std::ostream& out, const tasktrace& b); + +namespace internal { + +template<class Exc> +class backtraced : public Exc { + std::shared_ptr<sstring> _backtrace; +public: + template<typename... Args> + backtraced(Args&&... args) + : Exc(std::forward<Args>(args)...) + , _backtrace(std::make_shared<sstring>(format("{} Backtrace: {}", Exc::what(), current_backtrace()))) {} + + /** + * Returns the original exception message with a backtrace appended to it + * + * @return original exception message followed by a backtrace + */ + virtual const char* what() const noexcept override { + assert(_backtrace); + return _backtrace->c_str(); + } +}; + +} + + +/// Create an exception pointer of unspecified type that is derived from Exc type +/// with a backtrace attached to its message. +/// +/// \tparam Exc exception type to be caught at the receiving side +/// \tparam Args types of arguments forwarded to the constructor of Exc +/// \param args arguments forwarded to the constructor of Exc +/// \return std::exception_ptr containing the exception with the backtrace. +template <class Exc, typename... Args> +std::exception_ptr make_backtraced_exception_ptr(Args&&... args) { + using exc_type = std::decay_t<Exc>; + static_assert(std::is_base_of<std::exception, exc_type>::value, + "throw_with_backtrace only works with exception types"); + return std::make_exception_ptr<internal::backtraced<exc_type>>(Exc(std::forward<Args>(args)...)); +} + + /** + * Throws an exception of unspecified type that is derived from the Exc type + * with a backtrace attached to its message + * + * @tparam Exc exception type to be caught at the receiving side + * @tparam Args types of arguments forwarded to the constructor of Exc + * @param args arguments forwarded to the constructor of Exc + * @return never returns (throws an exception) + */ +template <class Exc, typename... Args> +[[noreturn]] +void +throw_with_backtrace(Args&&... args) { + std::rethrow_exception(make_backtraced_exception_ptr<Exc>(std::forward<Args>(args)...)); +}; + +} diff --git a/src/seastar/include/seastar/util/bool_class.hh b/src/seastar/include/seastar/util/bool_class.hh new file mode 100644 index 000000000..6acadb0fd --- /dev/null +++ b/src/seastar/include/seastar/util/bool_class.hh @@ -0,0 +1,110 @@ +/* + * 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) 2016 ScyllaDB. + */ + +#pragma once + +#include <ostream> + +namespace seastar { + +/// \addtogroup utilities +/// @{ + +/// \brief Type-safe boolean +/// +/// bool_class objects are type-safe boolean values that cannot be implicitly +/// casted to untyped bools, integers or different bool_class types while still +/// provides all relevant logical and comparison operators. +/// +/// bool_class template parameter is a tag type that is going to be used to +/// distinguish booleans of different types. +/// +/// Usage examples: +/// \code +/// struct foo_tag { }; +/// using foo = bool_class<foo_tag>; +/// +/// struct bar_tag { }; +/// using bar = bool_class<bar_tag>; +/// +/// foo v1 = foo::yes; // OK +/// bar v2 = foo::yes; // ERROR, no implicit cast +/// foo v4 = v1 || foo::no; // OK +/// bar v5 = bar::yes && bar(true); // OK +/// bool v6 = v5; // ERROR, no implicit cast +/// \endcode +/// +/// \tparam Tag type used as a tag +template<typename Tag> +class bool_class { + bool _value; +public: + static const bool_class yes; + static const bool_class no; + + /// Constructs a bool_class object initialised to \c false. + constexpr bool_class() noexcept : _value(false) { } + + /// Constructs a bool_class object initialised to \c v. + constexpr explicit bool_class(bool v) noexcept : _value(v) { } + + /// Casts a bool_class object to an untyped \c bool. + explicit operator bool() const noexcept { return _value; } + + /// Logical OR. + friend bool_class operator||(bool_class x, bool_class y) noexcept { + return bool_class(x._value || y._value); + } + + /// Logical AND. + friend bool_class operator&&(bool_class x, bool_class y) noexcept { + return bool_class(x._value && y._value); + } + + /// Logical NOT. + friend bool_class operator!(bool_class x) noexcept { + return bool_class(!x._value); + } + + /// Equal-to operator. + friend bool operator==(bool_class x, bool_class y) noexcept { + return x._value == y._value; + } + + /// Not-equal-to operator. + friend bool operator!=(bool_class x, bool_class y) noexcept { + return x._value != y._value; + } + + /// Prints bool_class value to an output stream. + friend std::ostream& operator<<(std::ostream& os, bool_class v) { + return os << (v._value ? "true" : "false"); + } +}; + +template<typename Tag> +const bool_class<Tag> bool_class<Tag>::yes { true }; +template<typename Tag> +const bool_class<Tag> bool_class<Tag>::no { false }; + +/// @} + +} diff --git a/src/seastar/include/seastar/util/closeable.hh b/src/seastar/include/seastar/util/closeable.hh new file mode 100644 index 000000000..8c87c6cd6 --- /dev/null +++ b/src/seastar/include/seastar/util/closeable.hh @@ -0,0 +1,183 @@ +/* + * 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) 2021 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <functional> +#include <seastar/core/future.hh> +#include <seastar/util/concepts.hh> +#include <seastar/util/defer.hh> + +/// \file + +/// \example closeable_test.cc + +namespace seastar { + +SEASTAR_CONCEPT( +template <typename Object> +concept closeable = requires (Object o) { + { o.close() } SEASTAR_DEFERRED_ACTION_NOEXCEPT -> std::same_as<future<>>; +}; +) + +/// Template helper to auto-close \c obj when destroyed. +/// +/// \tparam Object a class exposing a \c close() method that returns a \c future<> +/// that is called when the controller is destroyed. +/// +/// Must be used in a seastar thread as the destructor +/// needs to wait on the \c obj close() future. +template <typename Object> +SEASTAR_CONCEPT( requires closeable<Object> ) +class [[nodiscard("unassigned deferred_close")]] deferred_close { + std::reference_wrapper<Object> _obj; + bool _closed = false; + + void do_close() noexcept { + if (!_closed) { + _closed = true; + _obj.get().close().get(); + } + } +public: + /// Construct an object that will auto-close \c obj when destroyed. + /// \tparam obj the object to auto-close. + deferred_close(Object& obj) noexcept : _obj(obj) {} + /// Moves the \c deferred_close into a new one, and + /// the old one is canceled. + deferred_close(deferred_close&& x) noexcept : _obj(x._obj), _closed(std::exchange(x._closed, true)) {} + deferred_close(const deferred_close&) = delete; + /// Move-assign another \ref deferred_close. + /// The current \ref deferred_close is closed before being assigned. + /// And the other one's state is transferred to the current one. + deferred_close& operator=(deferred_close&& x) noexcept { + do_close(); + _obj = x._obj; + _closed = std::exchange(x._closed, true); + return *this; + } + /// Destruct the deferred_close object and auto-close \c obj. + ~deferred_close() { + do_close(); + } + /// Close \c obj once now. + void close_now() noexcept { + assert(!_closed); + do_close(); + } + + /// Prevents close() from being called when this object is destroyed. + /// Cannot call close_now() any more after this. + void cancel() noexcept { + _closed = true; + } +}; + +template <typename Closeable, typename Func> +SEASTAR_CONCEPT( +requires closeable<Closeable> && std::invocable<Func, Closeable&> && + std::is_nothrow_move_constructible_v<Closeable> && std::is_nothrow_move_constructible_v<Func> +) +inline futurize_t<std::invoke_result_t<Func, Closeable&>> +with_closeable(Closeable&& obj, Func func) noexcept { + return do_with(std::move(obj), [func = std::move(func)] (Closeable& obj) mutable { + return futurize_invoke(func, obj).finally([&obj] { + return obj.close(); + }); + }); +} + +SEASTAR_CONCEPT( +template <typename Object> +concept stoppable = requires (Object o) { + { o.stop() } SEASTAR_DEFERRED_ACTION_NOEXCEPT -> std::same_as<future<>>; +}; +) + +/// Template helper to auto-stop \c obj when destroyed. +/// +/// \tparam Object a class exposing a \c stop() method that returns a \c future<> +/// that is called when the controller is destroyed. +/// +/// Must be used in a seastar thread as the destructor +/// needs to wait on the \c obj stop() future. +template <typename Object> +SEASTAR_CONCEPT( requires stoppable<Object> ) +class [[nodiscard("unassigned deferred_stop")]] deferred_stop { + std::reference_wrapper<Object> _obj; + bool _stopped = false; + + void do_stop() noexcept { + if (!_stopped) { + _stopped = true; + _obj.get().stop().get(); + } + } +public: + /// Construct an object that will auto-stop \c obj when destroyed. + /// \tparam obj the object to auto-stop. + deferred_stop(Object& obj) noexcept : _obj(obj) {} + /// Moves the \c deferred_stop into a new one, and + /// the old one is canceled. + deferred_stop(deferred_stop&& x) noexcept : _obj(x._obj), _stopped(std::exchange(x._stopped, true)) {} + deferred_stop(const deferred_stop&) = delete; + /// Move-assign another \ref deferred_stop. + /// The current \ref deferred_stop is stopped before being assigned. + /// And the other one's state is transferred to the current one. + deferred_stop& operator=(deferred_stop&& x) noexcept { + do_stop(); + _obj = x._obj; + _stopped = std::exchange(x._stopped, true); + return *this; + } + /// Destruct the deferred_stop object and auto-stop \c obj. + ~deferred_stop() { + do_stop(); + } + /// Stop \c obj once now. + void stop_now() noexcept { + assert(!_stopped); + do_stop(); + } + + /// Prevents stop() from being called when this object is destroyed. + /// Cannot call stop_now() any more after this. + void cancel() noexcept { + _stopped = true; + } +}; + +template <typename Stoppable, typename Func> +SEASTAR_CONCEPT( +requires stoppable<Stoppable> && std::invocable<Func, Stoppable&> && + std::is_nothrow_move_constructible_v<Stoppable> && std::is_nothrow_move_constructible_v<Func> +) +inline futurize_t<std::invoke_result_t<Func, Stoppable&>> +with_stoppable(Stoppable&& obj, Func func) noexcept { + return do_with(std::move(obj), [func = std::move(func)] (Stoppable& obj) mutable { + return futurize_invoke(func, obj).finally([&obj] { + return obj.stop(); + }); + }); +} + +} // namespace seastar diff --git a/src/seastar/include/seastar/util/concepts.hh b/src/seastar/include/seastar/util/concepts.hh new file mode 100644 index 000000000..7d26b4d55 --- /dev/null +++ b/src/seastar/include/seastar/util/concepts.hh @@ -0,0 +1,39 @@ +/* + * 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) 2020 ScyllaDB + */ +#pragma once + +#if __has_include(<concepts>) +#include <concepts> +#endif + +#if defined(__cpp_concepts) && __cpp_concepts >= 201907 && \ + defined(__cpp_lib_concepts) && __cpp_lib_concepts >= 201907 + +#define SEASTAR_CONCEPT(x...) x +#define SEASTAR_NO_CONCEPT(x...) + +#else + +#define SEASTAR_CONCEPT(x...) +#define SEASTAR_NO_CONCEPT(x...) x + +#endif + diff --git a/src/seastar/include/seastar/util/conversions.hh b/src/seastar/include/seastar/util/conversions.hh new file mode 100644 index 000000000..627422076 --- /dev/null +++ b/src/seastar/include/seastar/util/conversions.hh @@ -0,0 +1,54 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <cstdlib> +#include <string_view> +#include <vector> + +namespace seastar { + +// Convert a string to a memory size, allowing binary SI +// suffixes (intentionally, even though SI suffixes are +// decimal, to follow existing usage). A string matched +// by following BNF is accetped: +// +// memory_size ::= <digit>+ <suffix>? "i"? "B"? +// suffix ::= ("k" | "K" | "M" | "G" | "T") +// +// for instance: +// +// "5" -> 5 +// "4k" -> (4 << 10) +// "8Mi" -> (8 << 20) +// "7GB" -> (7 << 30) +// "1TiB" -> (1 << 40) +// anything else: exception +size_t parse_memory_size(std::string_view s); + +static inline std::vector<char> string2vector(std::string_view str) { + auto v = std::vector<char>(str.begin(), str.end()); + v.push_back('\0'); + return v; +} + +} diff --git a/src/seastar/include/seastar/util/critical_alloc_section.hh b/src/seastar/include/seastar/util/critical_alloc_section.hh new file mode 100644 index 000000000..5cc6fee89 --- /dev/null +++ b/src/seastar/include/seastar/util/critical_alloc_section.hh @@ -0,0 +1,89 @@ +/* + * 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 2020 ScyllaDB + */ + +#pragma once + +namespace seastar { +namespace memory { + +#ifdef SEASTAR_ENABLE_ALLOC_FAILURE_INJECTION + +/// \cond internal +namespace internal { + +// This variable is used in hot paths so we want to avoid the compiler +// generating TLS init guards for it. In C++20 we have constinit to tell the +// compiler that it can be initialized compile time (although gcc still doesn't +// completely drops the init guards - https://gcc.gnu.org/bugzilla/show_bug.cgi?id=97848). +// In < c++20 we use `__thread` which results in no TLS init guards generated. +#ifdef __cpp_constinit +extern thread_local constinit volatile int critical_alloc_section; +#else +extern __thread volatile int critical_alloc_section; +#endif + +} // namespace internal +/// \endcond + +/// \brief Marks scopes that contain critical allocations. +/// +/// Critical allocations are those, whose failure the application cannot +/// tolerate. In a perfect world, there should be no such allocation, but we +/// don't live in a perfect world. +/// This information is used by other parts of the memory subsystem: +/// * \ref alloc_failure_injector will not inject errors into these scopes. +/// * A memory diagnostics report will be dumped if an allocation fails in these +/// scopes when the memory diagnostics subsystem is configured to dump reports +/// for \ref alloc_failure_kind \ref alloc_failure_kind::critical or above. +/// See \ref set_dump_memory_diagnostics_on_alloc_failure_kind(). +class scoped_critical_alloc_section { +public: + scoped_critical_alloc_section() { + // we assume the critical_alloc_section is thread local + // and there's seastar threads are non-preemptive. + // Otherwise, this would require an atomic variable + internal::critical_alloc_section = internal::critical_alloc_section + 1; + } + ~scoped_critical_alloc_section() { + internal::critical_alloc_section = internal::critical_alloc_section - 1; + } +}; + +/// \brief Is the current context inside a critical alloc section? +/// +/// Will return true if there is at least one \ref scoped_critical_alloc_section +/// alive in the current scope or the scope of any of the caller functions. +inline bool is_critical_alloc_section() { + return bool(internal::critical_alloc_section); +} + +#else // SEASTAR_ENABLE_ALLOC_FAILURE_INJECTION + +struct [[maybe_unused]] scoped_critical_alloc_section {}; + +inline bool is_critical_alloc_section() { + return false; +} + +#endif // SEASTAR_ENABLE_ALLOC_FAILURE_INJECTION + +} // namespace seastar +} // namespace memory diff --git a/src/seastar/include/seastar/util/defer.hh b/src/seastar/include/seastar/util/defer.hh new file mode 100644 index 000000000..175c3b880 --- /dev/null +++ b/src/seastar/include/seastar/util/defer.hh @@ -0,0 +1,75 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <type_traits> +#include <utility> + +#include <seastar/util/concepts.hh> + +#ifdef SEASTAR_DEFERRED_ACTION_REQUIRE_NOEXCEPT +#define SEASTAR_DEFERRED_ACTION_NOEXCEPT noexcept +#else +#define SEASTAR_DEFERRED_ACTION_NOEXCEPT +#endif + +SEASTAR_CONCEPT( +template <typename Func> +concept deferrable_action = requires (Func func) { + { func() } SEASTAR_DEFERRED_ACTION_NOEXCEPT -> std::same_as<void>; +}; +) + +namespace seastar { + +template <typename Func> +SEASTAR_CONCEPT( requires deferrable_action<Func> ) +class [[nodiscard("unassigned deferred_action")]] deferred_action { + Func _func; + bool _cancelled = false; +public: + static_assert(std::is_nothrow_move_constructible<Func>::value, "Func(Func&&) must be noexcept"); + deferred_action(Func&& func) noexcept : _func(std::move(func)) {} + deferred_action(deferred_action&& o) noexcept : _func(std::move(o._func)), _cancelled(o._cancelled) { + o._cancelled = true; + } + deferred_action& operator=(deferred_action&& o) noexcept { + if (this != &o) { + this->~deferred_action(); + new (this) deferred_action(std::move(o)); + } + return *this; + } + deferred_action(const deferred_action&) = delete; + ~deferred_action() { if (!_cancelled) { _func(); }; } + void cancel() { _cancelled = true; } +}; + +template <typename Func> +SEASTAR_CONCEPT( requires deferrable_action<Func> ) +inline +deferred_action<Func> +defer(Func&& func) { + return deferred_action<Func>(std::forward<Func>(func)); +} + +} diff --git a/src/seastar/include/seastar/util/eclipse.hh b/src/seastar/include/seastar/util/eclipse.hh new file mode 100644 index 000000000..25ed6158c --- /dev/null +++ b/src/seastar/include/seastar/util/eclipse.hh @@ -0,0 +1,36 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + * + */ + +#pragma once + +// Workarounds for deficiencies in Eclipse's C++ parser +// +// Tell Eclipse that IN_ECLIPSE is defined so it will ignore all the unknown syntax. + +#ifndef IN_ECLIPSE + +#else + +// Eclipse doesn't grok alignof +#define alignof sizeof + +#endif diff --git a/src/seastar/include/seastar/util/exceptions.hh b/src/seastar/include/seastar/util/exceptions.hh new file mode 100644 index 000000000..9b58e987e --- /dev/null +++ b/src/seastar/include/seastar/util/exceptions.hh @@ -0,0 +1,45 @@ +/* + * 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 2020 ScyllaDB + */ + +#pragma once + +#include <seastar/util/std-compat.hh> + +namespace seastar { + +/// \brief make a filesystem_error for system calls with a single file operand. +/// +/// \param what - describes the action that failed +/// \param path - path of the file that hit the error +/// \param error - the system error number (see errno(3)) +/// +std::filesystem::filesystem_error make_filesystem_error(const std::string& what, std::filesystem::path path, int error); + +/// \brief make a filesystem_error for system calls with two file operands. +/// +/// \param what - describes the action that failed +/// \param path1, path2 - paths of the files that hit the error +/// \param error - the system error number (see errno(3)) +/// +std::filesystem::filesystem_error make_filesystem_error(const std::string& what, std::filesystem::path path1, std::filesystem::path path2, int error); + +} // namespace seastar diff --git a/src/seastar/include/seastar/util/file.hh b/src/seastar/include/seastar/util/file.hh new file mode 100644 index 000000000..5de4d6a61 --- /dev/null +++ b/src/seastar/include/seastar/util/file.hh @@ -0,0 +1,97 @@ +/* + * 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 2020 ScyllaDB + */ + +#pragma once + +#include <seastar/core/future.hh> +#include <seastar/core/fstream.hh> +#include <seastar/core/sstring.hh> +#include <seastar/core/reactor.hh> +#include <seastar/util/std-compat.hh> +#include <seastar/util/short_streams.hh> + +namespace seastar { + +/// Recursively removes a directory and all of its contents. +/// +/// \param path path of the directory to recursively remove +/// +/// \note +/// Unlike `rm -rf` path has to be a directory and may not refer to a regular file. +/// +/// The function flushes the parent directory of the removed path and so guaranteeing that +/// the remove is stable on disk. +/// +/// The function bails out on first error. In that case, some files and/or sub-directories +/// (and their contents) may be left behind at the level in which the error was detected. +/// +future<> recursive_remove_directory(std::filesystem::path path) noexcept; + +/// @} + +/// \defgroup fileio-util File and Stream Utilities +/// \ingroup fileio-module +/// +/// \brief +/// These utilities are provided to help perform operations on files and I/O streams. + +namespace util { + +/// \addtogroup fileio-util +/// @{ + +template <typename Func> +SEASTAR_CONCEPT(requires requires(Func func, input_stream<char>& in) { + { func(in) }; +}) +auto with_file_input_stream(const std::filesystem::path& path, Func func, file_open_options file_opts = {}, file_input_stream_options input_stream_opts = {}) { + static_assert(std::is_nothrow_move_constructible_v<Func>); + return open_file_dma(path.native(), open_flags::ro, std::move(file_opts)).then( + [func = std::move(func), input_stream_opts = std::move(input_stream_opts)] (file f) mutable { + return do_with(make_file_input_stream(std::move(f), std::move(input_stream_opts)), + [func = std::move(func)] (input_stream<char>& in) mutable { + return futurize_invoke(std::move(func), in).finally([&in] { + return in.close(); + }); + }); + }); +} + +/// Returns all bytes from the file until eof, accessible in chunks. +/// +/// \note use only on short files to avoid running out of memory. +/// +/// \param path path of the file to be read. +future<std::vector<temporary_buffer<char>>> read_entire_file(std::filesystem::path path); + +/// Returns all bytes from the file until eof as a single buffer. +/// +/// \note use only on short files to avoid running out of memory. +/// +/// \param path path of the file to be read. +future<sstring> read_entire_file_contiguous(std::filesystem::path path); + +/// @} + +} // namespace util + +} // namespace seastar diff --git a/src/seastar/include/seastar/util/function_input_iterator.hh b/src/seastar/include/seastar/util/function_input_iterator.hh new file mode 100644 index 000000000..1155db505 --- /dev/null +++ b/src/seastar/include/seastar/util/function_input_iterator.hh @@ -0,0 +1,72 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +namespace seastar { + +template <typename Function, typename State> +struct function_input_iterator { + Function _func; + State _state; +public: + function_input_iterator(Function func, State state) + : _func(func), _state(state) { + } + function_input_iterator(const function_input_iterator&) = default; + function_input_iterator(function_input_iterator&&) = default; + function_input_iterator& operator=(const function_input_iterator&) = default; + function_input_iterator& operator=(function_input_iterator&&) = default; + auto operator*() const { + return _func(); + } + function_input_iterator& operator++() { + ++_state; + return *this; + } + function_input_iterator operator++(int) { + function_input_iterator ret{*this}; + ++_state; + return ret; + } + bool operator==(const function_input_iterator& x) const { + return _state == x._state; + } + bool operator!=(const function_input_iterator& x) const { + return !operator==(x); + } +}; + +template <typename Function, typename State> +inline +function_input_iterator<Function, State> +make_function_input_iterator(Function func, State state) { + return function_input_iterator<Function, State>(func, state); +} + +template <typename Function, typename State> +inline +function_input_iterator<Function, State> +make_function_input_iterator(Function&& func) { + return function_input_iterator<Function, State>(func, State{}); +} + +} diff --git a/src/seastar/include/seastar/util/gcc6-concepts.hh b/src/seastar/include/seastar/util/gcc6-concepts.hh new file mode 100644 index 000000000..1f4ac8565 --- /dev/null +++ b/src/seastar/include/seastar/util/gcc6-concepts.hh @@ -0,0 +1,25 @@ +/* + * 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) 2016 ScyllaDB + */ +#pragma once + +#define GCC6_CONCEPT(x...) +#define GCC6_NO_CONCEPT(x...) x + diff --git a/src/seastar/include/seastar/util/indirect.hh b/src/seastar/include/seastar/util/indirect.hh new file mode 100644 index 000000000..e544a5f1a --- /dev/null +++ b/src/seastar/include/seastar/util/indirect.hh @@ -0,0 +1,74 @@ +/* + * 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) 2016 ScyllaDB + */ + +#pragma once + +#include <memory> + +namespace seastar { + +// This header defines functors for comparing and hashing pointers by pointed-to values instead of pointer addresses. +// +// Examples: +// +// std::multiset<shared_ptr<sstring>, indirect_less<shared_ptr<sstring>>> _multiset; +// +// std::unordered_map<shared_ptr<sstring>, bool, +// indirect_hash<shared_ptr<sstring>>, indirect_equal_to<shared_ptr<sstring>>> _unordered_map; +// + +template<typename Pointer, typename Equal = std::equal_to<typename std::pointer_traits<Pointer>::element_type>> +struct indirect_equal_to { + Equal _eq; + indirect_equal_to(Equal eq = Equal()) : _eq(std::move(eq)) {} + bool operator()(const Pointer& i1, const Pointer& i2) const { + if (bool(i1) ^ bool(i2)) { + return false; + } + return !i1 || _eq(*i1, *i2); + } +}; + +template<typename Pointer, typename Less = std::less<typename std::pointer_traits<Pointer>::element_type>> +struct indirect_less { + Less _cmp; + indirect_less(Less cmp = Less()) : _cmp(std::move(cmp)) {} + bool operator()(const Pointer& i1, const Pointer& i2) const { + if (i1 && i2) { + return _cmp(*i1, *i2); + } + return !i1 && i2; + } +}; + +template<typename Pointer, typename Hash = std::hash<typename std::pointer_traits<Pointer>::element_type>> +struct indirect_hash { + Hash _h; + indirect_hash(Hash h = Hash()) : _h(std::move(h)) {} + size_t operator()(const Pointer& p) const { + if (p) { + return _h(*p); + } + return 0; + } +}; + +} diff --git a/src/seastar/include/seastar/util/internal/iovec_utils.hh b/src/seastar/include/seastar/util/internal/iovec_utils.hh new file mode 100644 index 000000000..b58942e97 --- /dev/null +++ b/src/seastar/include/seastar/util/internal/iovec_utils.hh @@ -0,0 +1,55 @@ +/* + * 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) 2022 ScyllaDB. + */ + +#include <sys/types.h> + +#pragma once + +namespace seastar { +namespace internal { + +inline size_t iovec_len(const iovec* begin, size_t len) +{ + size_t ret = 0; + auto end = begin + len; + while (begin != end) { + ret += begin++->iov_len; + } + return ret; +} + +inline size_t iovec_len(const std::vector<iovec>& iov) +{ + size_t ret = 0; + for (auto&& e : iov) { + ret += e.iov_len; + } + return ret; +} + +// Given a properly aligned vector of iovecs, ensures that it respects the +// IOV_MAX limit, by trimming if necessary. The modified vector still satisfied +// the alignment requirements. +// Returns the final total length of all iovecs. +size_t sanitize_iovecs(std::vector<iovec>& iov, size_t disk_alignment) noexcept; + +} // internal namespace +} // seastar namespace diff --git a/src/seastar/include/seastar/util/internal/magic.hh b/src/seastar/include/seastar/util/internal/magic.hh new file mode 100644 index 000000000..298343270 --- /dev/null +++ b/src/seastar/include/seastar/util/internal/magic.hh @@ -0,0 +1,81 @@ +/* + * 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) 2022 ScyllaDB. + */ + +#pragma once + +#include <linux/magic.h> + +namespace seastar { + +#ifndef XFS_SUPER_MAGIC +#define XFS_SUPER_MAGIC 0x58465342 +#endif + +#ifndef EXT2_SUPER_MAGIC +#define EXT2_SUPER_MAGIC 0xEF53 +#endif + +#ifndef EXT3_SUPER_MAGIC +#define EXT3_SUPER_MAGIC 0xEF53 +#endif + +#ifndef EXT4_SUPER_MAGIC +#define EXT4_SUPER_MAGIC 0xEF53 +#endif + +#ifndef NFS_SUPER_MAGIC +#define NFS_SUPER_MAGIC 0x6969 +#endif + +#ifndef TMPFS_MAGIC +#define TMPFS_MAGIC 0x01021994 +#endif + +#ifndef FUSE_SUPER_MAGIC +#define FUSE_SUPER_MAGIC 0x65735546 +#endif + +#ifndef BTRFS_SUPER_MAGIC +#define BTRFS_SUPER_MAGIC 0x9123683E +#endif + +#ifndef HFS_SUPER_MAGIC +#define HFS_SUPER_MAGIC 0x4244 +#endif + +namespace internal { + +class fs_magic { +public: + static constexpr unsigned long xfs = XFS_SUPER_MAGIC; + static constexpr unsigned long ext2 = EXT2_SUPER_MAGIC; + static constexpr unsigned long ext3 = EXT3_SUPER_MAGIC; + static constexpr unsigned long ext4 = EXT4_SUPER_MAGIC; + static constexpr unsigned long nfs = NFS_SUPER_MAGIC; + static constexpr unsigned long tmpfs = TMPFS_MAGIC; + static constexpr unsigned long fuse = FUSE_SUPER_MAGIC; + static constexpr unsigned long btrfs = BTRFS_SUPER_MAGIC; + static constexpr unsigned long hfs = HFS_SUPER_MAGIC; +}; + +} // internal namespace + +} // seastar namespace diff --git a/src/seastar/include/seastar/util/is_smart_ptr.hh b/src/seastar/include/seastar/util/is_smart_ptr.hh new file mode 100644 index 000000000..68fb9b961 --- /dev/null +++ b/src/seastar/include/seastar/util/is_smart_ptr.hh @@ -0,0 +1,34 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <memory> // for std::unique_ptr + +namespace seastar { + +template<typename T> +struct is_smart_ptr : std::false_type {}; + +template<typename T> +struct is_smart_ptr<std::unique_ptr<T>> : std::true_type {}; + +} diff --git a/src/seastar/include/seastar/util/later.hh b/src/seastar/include/seastar/util/later.hh new file mode 100644 index 000000000..d1cc3ab90 --- /dev/null +++ b/src/seastar/include/seastar/util/later.hh @@ -0,0 +1,84 @@ +/* + * 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) 2020 ScyllaDB. + */ + + +#pragma once + +#include <seastar/core/future.hh> + +namespace seastar { + +/// \addtogroup future-util +/// @{ + +/// \brief Returns a ready future. +inline +future<> now() { + return make_ready_future<>(); +} + +/// \brief Returns a future which is not ready but is scheduled to resolve soon. +/// +/// Schedules a future to run "soon". yield() can be used to break long-but-finite +/// loops into pieces. Note that if nothing else is runnable, +/// It will not check for I/O, and so an infinite loop with yield() will just +/// burn CPU. +future<> yield() noexcept; + +/// Yield the cpu if the task quota is exhausted. +/// +/// Check if the current continuation is preempted and yield if so. Otherwise +/// return a ready future. +/// +/// \note Threads and coroutines (see seastar::thread::maybe_yield() and +/// seastar::coroutine::maybe_yield() have their own custom variants, +/// and the various continuation-based loops (do_for_each() and similar) +/// do this automatically. +inline +future<> maybe_yield() noexcept { + if (need_preempt()) { + return yield(); + } else { + return make_ready_future<>(); + } +} + +/// Force the reactor to check for pending I/O +/// +/// Schedules a check for new I/O completions (disk operations completions +/// or network packet arrival) immediately and return a future that is ready +/// when the I/O has been polled for. +/// +/// \note It is very rare to need to call this function. It is better to let the +/// reactor schedule I/O polls itself. +/// \note This has no effect on I/O polling on other shards. +future<> check_for_io_immediately() noexcept; + +/// \brief Returns a future which is not ready but is scheduled to resolve soon. +/// +/// \deprecated Use yield() instead, or check_for_io_immediately() if your really need it. +[[deprecated("Use yield() or check_for_io_immediately()")]] +future<> later() noexcept; + +/// @} + +} // namespace seastar diff --git a/src/seastar/include/seastar/util/lazy.hh b/src/seastar/include/seastar/util/lazy.hh new file mode 100644 index 000000000..4684efb32 --- /dev/null +++ b/src/seastar/include/seastar/util/lazy.hh @@ -0,0 +1,153 @@ +/* + * 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) 2016 Cloudius Systems, Ltd. + */ +#pragma once + +#include <ostream> + +/// \addtogroup logging +/// @{ + +namespace seastar { + +/// \brief This class is a wrapper for a lazy evaluation of a value. +/// +/// The value is evaluated by a functor that gets no parameters, which is +/// provided to a lazy_value constructor. +/// +/// The instance may be created only using seastar::value_of helper function. +/// +/// The evaluation is triggered by operator(). +template<typename Func> +class lazy_eval { +private: + Func _func; + +private: + lazy_eval(Func&& f) : _func(std::forward<Func>(f)) {} + +public: + /// \brief Evaluate a value. + /// + /// \return the evaluated value + auto operator()() { + return _func(); + } + + /// \brief Evaluate a value (const version). + /// + /// \return the evaluated value + auto operator()() const { + return _func(); + } + + template <typename F> + friend lazy_eval<F> value_of(F&& func); +}; + + +/// Create a seastar::lazy_eval object that will use a given functor for +/// evaluating a value when the evaluation is triggered. +/// +/// The actual evaluation is triggered by applying a () operator on a +/// returned object. +/// +/// \param func a functor to evaluate the value +/// +/// \return a lazy_eval object that may be used for evaluating a value +template <typename Func> +inline lazy_eval<Func> value_of(Func&& func) { + return lazy_eval<Func>(std::forward<Func>(func)); +} + +/// \brief This struct is a wrapper for lazy dereferencing a pointer. +/// +/// In particular this is to be used in situations where the value of a +/// pointer has to be converted to string in a lazy manner. Since +/// pointers can be null adding a check at the point of calling the +/// log function for example, will introduce an unnecessary branch in +/// potentially useless code. Using lazy_deref this check can be +/// deferred to the point where the code is actually evaluated. +template <typename T> +struct lazy_deref_wrapper { + const T& p; + + constexpr lazy_deref_wrapper(const T& p) : p(p) { + } +}; + +/// Create a seastar::lazy_deref_wrapper object. +/// +/// The actual dereferencing will happen when the object is inserted +/// into a stream. The pointer is not copied, only a reference is saved +/// to it. Smart pointers are supported as well. +/// +/// \param p a raw pointer or a smart pointer +/// +/// \return a lazy_deref_wrapper object +template <typename T> +lazy_deref_wrapper<T> +lazy_deref(const T& p) { + return lazy_deref_wrapper<T>(p); +} + +} + +namespace std { +/// Output operator for a seastar::lazy_eval<Func> +/// This would allow printing a seastar::lazy_eval<Func> as if it's a regular +/// value. +/// +/// For example: +/// +/// `logger.debug("heavy eval result:{}", seastar::value_of([&] { return <heavy evaluation>; }));` +/// +/// (If a logging level is lower than "debug" the evaluation will not take place.) +/// +/// \tparam Func a functor type +/// \param os ostream to print to +/// \param lf a reference to a lazy_eval<Func> to be printed +/// +/// \return os +template <typename Func> +ostream& operator<<(ostream& os, const seastar::lazy_eval<Func>& lf) { + return os << lf(); +} + +template <typename Func> +ostream& operator<<(ostream& os, seastar::lazy_eval<Func>& lf) { + return os << lf(); +} + +template <typename Func> +ostream& operator<<(ostream& os, seastar::lazy_eval<Func>&& lf) { + return os << lf(); +} + +template <typename T> +ostream& operator<<(ostream& os, seastar::lazy_deref_wrapper<T> ld) { + if (ld.p) { + return os << *ld.p; + } + + return os << "null"; +} +} +/// @} diff --git a/src/seastar/include/seastar/util/log-cli.hh b/src/seastar/include/seastar/util/log-cli.hh new file mode 100644 index 000000000..53b970a9b --- /dev/null +++ b/src/seastar/include/seastar/util/log-cli.hh @@ -0,0 +1,134 @@ +/* + * 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) 2017 ScyllaDB + */ + +#pragma once + +#include <seastar/util/log.hh> +#include <seastar/util/program-options.hh> + +#include <seastar/core/sstring.hh> + +#include <boost/program_options.hpp> + +#include <algorithm> +#include <unordered_map> + +/// \addtogroup logging +/// @{ +namespace seastar { + +/// +/// \brief Configure application logging at run-time with program options. +/// +namespace log_cli { + +/// +/// \brief Options for controlling logging at run-time. +/// +boost::program_options::options_description get_options_description(); + +using log_level_map = std::unordered_map<sstring, log_level>; + +/// Logging configuration. +struct options : public program_options::option_group { + /// \brief Default log level for log messages. + /// + /// Valid values are trace, debug, info, warn, error. + /// Default: \p info + /// \see \ref log_level. + program_options::value<log_level> default_log_level; + /// \brief Map of logger name to log level. + /// + /// The format is `NAME0=LEVEL0[:NAME1=LEVEL1:...]`. + /// Valid logger names can be queried with \p --help-loggers. + /// This option can be specified multiple times. + program_options::value<log_level_map> logger_log_level; + /// Select timestamp style for stdout logs. + /// + /// Default: \p real. + /// \see logger_timestamp_style. + program_options::value<logger_timestamp_style> logger_stdout_timestamps; + /// \brief Send log output to output stream. + /// + /// As selected by \ref logger_ostream_type. + /// Default: \p true. + program_options::value<bool> log_to_stdout; + /// Send log output to. + /// + /// Default: \p stderr. + /// \see \ref seastar::logger_ostream_type. + program_options::value<seastar::logger_ostream_type> logger_ostream_type; + /// Send log output to syslog. + /// + /// Default: \p false. + program_options::value<bool> log_to_syslog; + + /// Print colored tag prefix in log messages sent to output stream. + /// + /// Default: \p true. + program_options::value<bool> log_with_color; + /// \cond internal + options(program_options::option_group* parent_group); + /// \endcond +}; + +/// +/// \brief Print a human-friendly list of the available loggers. +/// +void print_available_loggers(std::ostream& os); + +/// +/// \brief Parse a log-level ({error, warn, info, debug, trace}) string, throwing \c std::runtime_error for an invalid +/// level. +/// +log_level parse_log_level(const sstring&); + +/// \cond internal +void parse_map_associations(const std::string& v, std::function<void(std::string, std::string)> consume_key_value); +/// \endcond + +// +// \brief Parse associations from loggers to log-levels and write the resulting pairs to the output iterator. +// +// \throws \c std::runtime_error for an invalid log-level. +// +template <class OutputIter> +void parse_logger_levels(const program_options::string_map& levels, OutputIter out) { + std::for_each(levels.begin(), levels.end(), [&out](auto&& pair) { + *out++ = std::make_pair(pair.first, parse_log_level(pair.second)); + }); +} + +/// +/// \brief Extract CLI options into a logging configuration. +// +logging_settings extract_settings(const boost::program_options::variables_map&); + +/// +/// \brief Extract \ref options into a logging configuration. +/// +logging_settings extract_settings(const options&); + +} + +} + +/// @} diff --git a/src/seastar/include/seastar/util/log-impl.hh b/src/seastar/include/seastar/util/log-impl.hh new file mode 100644 index 000000000..3baa80728 --- /dev/null +++ b/src/seastar/include/seastar/util/log-impl.hh @@ -0,0 +1,125 @@ +/* + * 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) 2020 Cloudius Systems, Ltd. + */ + +#pragma once + +#include <iterator> + +/// \addtogroup logging +/// @{ + +namespace seastar { + +/// \cond internal +namespace internal { + +/// A buffer to format log messages into. +/// +/// It was designed to allow formatting the entire message into it, without any +/// intermediary buffers. To minimize the amount of reallocations it supports +/// using an external buffer. When this is full it moves to using buffers +/// allocated by itself. +/// To accommodate the most widely used way of formatting messages -- fmt --, +/// it provides an output iterator interface for writing into it. +class log_buf { + char* _begin; + char* _end; + char* _current; + bool _own_buf; + bool _alloc_failure = false; +private: + void free_buffer() noexcept; + void realloc_buffer_and_append(char c) noexcept; + +public: + // inserter_iterator is designed like std::back_insert_iterator: + // operator*, operator++ and operator++(int) are no-ops, + // and all the work happens in operator=, which pushes a character + // to the buffer. + // The iterator stores no state of its own. + // + // inserter_iterator is supposed to be used as an output_iterator. + // That is, assignment is expected to alternate with incrementing. + class inserter_iterator { + public: + using iterator_category = std::output_iterator_tag; + using difference_type = std::ptrdiff_t; + using value_type = void; + using pointer = void; + using reference = void; + + private: + log_buf* _buf; + + public: + explicit inserter_iterator(log_buf& buf) noexcept : _buf(&buf) { } + + inserter_iterator& operator=(char c) noexcept { + if (__builtin_expect(_buf->_current == _buf->_end, false)) { + _buf->realloc_buffer_and_append(c); + return *this; + } + *_buf->_current++ = c; + return *this; + } + inserter_iterator& operator*() noexcept { + return *this; + } + inserter_iterator& operator++() noexcept { + return *this; + } + inserter_iterator operator++(int) noexcept { + return *this; + } + }; + + /// Default ctor. + /// + /// Allocates an internal buffer of 512 bytes. + log_buf(); + /// External buffer ctor. + /// + /// Use the external buffer until its full, then switch to internally + /// allocated buffer. log_buf doesn't take ownership of the buffer. + log_buf(char* external_buf, size_t size) noexcept; + ~log_buf(); + /// Clear the buffer, setting its position back to the start, but does not + /// free any buffers (after this called, size is zero, capacity is unchanged). + /// Any existing iterators are invalidated. + void clear() { _current = _begin; } + /// Create an output iterator which allows writing into the buffer. + inserter_iterator back_insert_begin() noexcept { return inserter_iterator(*this); } + /// The amount of data written so far. + size_t size() const noexcept { return _current - _begin; } + /// The size of the buffer. + size_t capacity() const noexcept { return _end - _begin; } + /// Read only pointer to the buffer. + /// Note that the buffer is not guaranteed to be null terminated. The writer + /// has to ensure that, should it wish to. + const char* data() const noexcept { return _begin; } + /// A view of the buffer content. + std::string_view view() const noexcept { return std::string_view(_begin, size()); } +}; + +} // namespace internal +/// \endcond + +} // namespace seastar diff --git a/src/seastar/include/seastar/util/log.hh b/src/seastar/include/seastar/util/log.hh new file mode 100644 index 000000000..28b66c175 --- /dev/null +++ b/src/seastar/include/seastar/util/log.hh @@ -0,0 +1,545 @@ +/* + * 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) 2015 Cloudius Systems, Ltd. + */ +#pragma once + +#include <seastar/core/sstring.hh> +#include <seastar/util/concepts.hh> +#include <seastar/util/log-impl.hh> +#include <seastar/core/lowres_clock.hh> +#include <seastar/util/std-compat.hh> + +#include <unordered_map> +#include <exception> +#include <iosfwd> +#include <atomic> +#include <mutex> +#include <boost/lexical_cast.hpp> +#include <fmt/format.h> + + +/// \addtogroup logging +/// @{ + +namespace seastar { + +/// \brief log level used with \see {logger} +/// used with the logger.do_log method. +/// Levels are in increasing order. That is if you want to see debug(3) logs you +/// will also see error(0), warn(1), info(2). +/// +enum class log_level { + error, + warn, + info, + debug, + trace, +}; + +std::ostream& operator<<(std::ostream& out, log_level level); +std::istream& operator>>(std::istream& in, log_level& level); +} + +// Boost doesn't auto-deduce the existence of the streaming operators for some reason + +namespace boost { +template<> +seastar::log_level lexical_cast(const std::string& source); + +} + +namespace seastar { + +class logger; +class logger_registry; + +/// \brief Logger class for ostream or syslog. +/// +/// Java style api for logging. +/// \code {.cpp} +/// static seastar::logger logger("lsa-api"); +/// logger.info("Triggering compaction"); +/// \endcode +/// The output format is: (depending on level) +/// DEBUG %Y-%m-%d %T,%03d [shard 0] - "your msg" \n +/// +/// It is possible to rate-limit log messages, see \ref logger::rate_limit. +class logger { + sstring _name; + std::atomic<log_level> _level = { log_level::info }; + static std::ostream* _out; + static std::atomic<bool> _ostream; + static std::atomic<bool> _syslog; + static unsigned _shard_field_width; + static inline thread_local bool silent = false; + +public: + class log_writer { + public: + virtual ~log_writer() = default; + virtual internal::log_buf::inserter_iterator operator()(internal::log_buf::inserter_iterator) = 0; + }; + template <typename Func> + SEASTAR_CONCEPT(requires requires (Func fn, internal::log_buf::inserter_iterator it) { + it = fn(it); + }) + class lambda_log_writer : public log_writer { + Func _func; + public: + lambda_log_writer(Func&& func) : _func(std::forward<Func>(func)) { } + virtual ~lambda_log_writer() override = default; + virtual internal::log_buf::inserter_iterator operator()(internal::log_buf::inserter_iterator it) override { return _func(it); } + }; + + /// \cond internal + /// \brief used to hold the log format string and the caller's source_location. + struct format_info { + /// implicitly construct format_info from a const char* format string. + /// \param fmt - {fmt} style format string + format_info(const char* format, compat::source_location loc = compat::source_location::current()) noexcept + : format(format) + , loc(loc) + {} + /// implicitly construct format_info from a std::string_view format string. + /// \param fmt - {fmt} style format string_view + format_info(std::string_view format, compat::source_location loc = compat::source_location::current()) noexcept + : format(format) + , loc(loc) + {} + /// implicitly construct format_info with no format string. + format_info(compat::source_location loc = compat::source_location::current()) noexcept + : format() + , loc(loc) + {} + std::string_view format; + compat::source_location loc; + }; + +private: + + // We can't use an std::function<> as it potentially allocates. + void do_log(log_level level, log_writer& writer); + void failed_to_log(std::exception_ptr ex, format_info fmt) noexcept; + + class silencer { + public: + silencer() noexcept { + silent = true; + } + + ~silencer() { + silent = false; + } + }; + +public: + /// Apply a rate limit to log message(s) + /// + /// Pass this to \ref logger::log() to apply a rate limit to the message. + /// The rate limit is applied to all \ref logger::log() calls this rate + /// limit is passed to. Example: + /// + /// void handle_request() { + /// static thread_local logger::rate_limit my_rl(std::chrono::seconds(10)); + /// // ... + /// my_log.log(log_level::info, my_rl, "a message we don't want to log on every request, only at most once each 10 seconds"); + /// // ... + /// } + /// + /// The rate limit ensures that at most one message per interval will be + /// logged. If there were messages dropped due to rate-limiting the + /// following snippet will be prepended to the first non-dropped log + /// messages: + /// + /// (rate limiting dropped $N similar messages) + /// + /// Where $N is the number of messages dropped. + class rate_limit { + friend class logger; + + using clock = lowres_clock; + + private: + clock::duration _interval; + clock::time_point _next; + uint64_t _dropped_messages = 0; + + private: + bool check(); + bool has_dropped_messages() const { return bool(_dropped_messages); } + uint64_t get_and_reset_dropped_messages() { + return std::exchange(_dropped_messages, 0); + } + + public: + explicit rate_limit(std::chrono::milliseconds interval); + }; + +public: + explicit logger(sstring name); + logger(logger&& x); + ~logger(); + + bool is_shard_zero() noexcept; + + /// Test if desired log level is enabled + /// + /// \param level - enum level value (info|error...) + /// \return true if the log level has been enabled. + bool is_enabled(log_level level) const noexcept { + return __builtin_expect(level <= _level.load(std::memory_order_relaxed), false) && !silent; + } + + /// logs to desired level if enabled, otherwise we ignore the log line + /// + /// \param fmt - {fmt} style format string (implictly converted to struct logger::format_info) + /// or a logger::format_info passed down the call chain. + /// \param args - args to print string + /// + template <typename... Args> + void log(log_level level, format_info fmt, Args&&... args) noexcept { + if (is_enabled(level)) { + try { + lambda_log_writer writer([&] (internal::log_buf::inserter_iterator it) { +#if FMT_VERSION >= 80000 + return fmt::format_to(it, fmt::runtime(fmt.format), std::forward<Args>(args)...); +#else + return fmt::format_to(it, fmt.format, std::forward<Args>(args)...); +#endif + }); + do_log(level, writer); + } catch (...) { + failed_to_log(std::current_exception(), std::move(fmt)); + } + } + } + + /// logs with a rate limit to desired level if enabled, otherwise we ignore the log line + /// + /// If there were messages dropped due to rate-limiting the following snippet + /// will be prepended to the first non-dropped log messages: + /// + /// (rate limiting dropped $N similar messages) + /// + /// Where $N is the number of messages dropped. + /// + /// \param rl - the \ref rate_limit to apply to this log + /// \param fmt - {fmt} style format string (implictly converted to struct logger::format_info) + /// or a logger::format_info passed down the call chain. + /// \param args - args to print string + /// + template <typename... Args> + void log(log_level level, rate_limit& rl, format_info fmt, Args&&... args) noexcept { + if (is_enabled(level) && rl.check()) { + try { + lambda_log_writer writer([&] (internal::log_buf::inserter_iterator it) { + if (rl.has_dropped_messages()) { + it = fmt::format_to(it, "(rate limiting dropped {} similar messages) ", rl.get_and_reset_dropped_messages()); + } +#if FMT_VERSION >= 80000 + return fmt::format_to(it, fmt::runtime(fmt.format), std::forward<Args>(args)...); +#else + return fmt::format_to(it, fmt.format, std::forward<Args>(args)...); +#endif + }); + do_log(level, writer); + } catch (...) { + failed_to_log(std::current_exception(), std::move(fmt)); + } + } + } + + /// \cond internal + /// logs to desired level if enabled, otherwise we ignore the log line + /// + /// \param writer a function which writes directly to the underlying log buffer + /// \param fmt - optional logger::format_info passed down the call chain. + /// + /// This is a low level method for use cases where it is very important to + /// avoid any allocations. The \arg writer will be passed a + /// internal::log_buf::inserter_iterator that allows it to write into the log + /// buffer directly, avoiding the use of any intermediary buffers. + void log(log_level level, log_writer& writer, format_info fmt = {}) noexcept { + if (is_enabled(level)) { + try { + do_log(level, writer); + } catch (...) { + failed_to_log(std::current_exception(), std::move(fmt)); + } + } + } + /// logs to desired level if enabled, otherwise we ignore the log line + /// + /// \param writer a function which writes directly to the underlying log buffer + /// \param fmt - optional logger::format_info passed down the call chain. + /// + /// This is a low level method for use cases where it is very important to + /// avoid any allocations. The \arg writer will be passed a + /// internal::log_buf::inserter_iterator that allows it to write into the log + /// buffer directly, avoiding the use of any intermediary buffers. + /// This is rate-limited version, see \ref rate_limit. + void log(log_level level, rate_limit& rl, log_writer& writer, format_info fmt = {}) noexcept { + if (is_enabled(level) && rl.check()) { + try { + lambda_log_writer writer_wrapper([&] (internal::log_buf::inserter_iterator it) { + if (rl.has_dropped_messages()) { + it = fmt::format_to(it, "(rate limiting dropped {} similar messages) ", rl.get_and_reset_dropped_messages()); + } + return writer(it); + }); + do_log(level, writer_wrapper); + } catch (...) { + failed_to_log(std::current_exception(), std::move(fmt)); + } + } + } + /// \endcond + + /// Log with error tag: + /// ERROR %Y-%m-%d %T,%03d [shard 0] - "your msg" \n + /// + /// \param fmt - {fmt} style format string (implictly converted to struct logger::format_info) + /// or a logger::format_info passed down the call chain. + /// \param args - args to print string + /// + template <typename... Args> + void error(format_info fmt, Args&&... args) noexcept { + log(log_level::error, std::move(fmt), std::forward<Args>(args)...); + } + /// Log with warning tag: + /// WARN %Y-%m-%d %T,%03d [shard 0] - "your msg" \n + /// + /// \param fmt - {fmt} style format string (implictly converted to struct logger::format_info) + /// or a logger::format_info passed down the call chain. + /// \param args - args to print string + /// + template <typename... Args> + void warn(format_info fmt, Args&&... args) noexcept { + log(log_level::warn, std::move(fmt), std::forward<Args>(args)...); + } + /// Log with info tag: + /// INFO %Y-%m-%d %T,%03d [shard 0] - "your msg" \n + /// + /// \param fmt - {fmt} style format string (implictly converted to struct logger::format_info) + /// or a logger::format_info passed down the call chain. + /// \param args - args to print string + /// + template <typename... Args> + void info(format_info fmt, Args&&... args) noexcept { + log(log_level::info, std::move(fmt), std::forward<Args>(args)...); + } + /// Log with info tag on shard zero only: + /// INFO %Y-%m-%d %T,%03d [shard 0] - "your msg" \n + /// + /// \param fmt - {fmt} style format string (implictly converted to struct logger::format_info) + /// or a logger::format_info passed down the call chain. + /// \param args - args to print string + /// + template <typename... Args> + void info0(format_info fmt, Args&&... args) noexcept { + if (is_shard_zero()) { + log(log_level::info, std::move(fmt), std::forward<Args>(args)...); + } + } + /// Log with debug tag: + /// DEBUG %Y-%m-%d %T,%03d [shard 0] - "your msg" \n + /// + /// \param fmt - {fmt} style format string (implictly converted to struct logger::format_info) + /// or a logger::format_info passed down the call chain. + /// \param args - args to print string + /// + template <typename... Args> + void debug(format_info fmt, Args&&... args) noexcept { + log(log_level::debug, std::move(fmt), std::forward<Args>(args)...); + } + /// Log with trace tag: + /// TRACE %Y-%m-%d %T,%03d [shard 0] - "your msg" \n + /// + /// \param fmt - {fmt} style format string (implictly converted to struct logger::format_info) + /// or a logger::format_info passed down the call chain. + /// \param args - args to print string + /// + template <typename... Args> + void trace(format_info fmt, Args&&... args) noexcept { + log(log_level::trace, std::move(fmt), std::forward<Args>(args)...); + } + + /// \return name of the logger. Usually one logger per module + /// + const sstring& name() const noexcept { + return _name; + } + + /// \return current log level for this logger + /// + log_level level() const noexcept { + return _level.load(std::memory_order_relaxed); + } + + /// \param level - set the log level + /// + void set_level(log_level level) noexcept { + _level.store(level, std::memory_order_relaxed); + } + + /// Set output stream, default is std::cerr + static void set_ostream(std::ostream& out) noexcept; + + /// Also output to ostream. default is true + static void set_ostream_enabled(bool enabled) noexcept; + + /// Also output to stdout. default is true + [[deprecated("Use set_ostream_enabled instead")]] + static void set_stdout_enabled(bool enabled) noexcept; + + /// Also output to syslog. default is false + /// + /// NOTE: syslog() can block, which will stall the reactor thread. + /// this should be rare (will have to fill the pipe buffer + /// before syslogd can clear it) but can happen. + static void set_syslog_enabled(bool enabled) noexcept; + + /// Set the width of shard id field in log messages + /// + /// \c this_shard_id() is printed as a part of the prefix in logging + /// messages, like "[shard 42]", where \c 42 is the decimal number of the + /// current shard id printed with a minimal width. + /// + /// \param width the minimal width of the shard id field + static void set_shard_field_width(unsigned width) noexcept; + + /// enable/disable the colored tag in ostream + /// + /// \note this is a noop if fmtlib's version is less than 6.0 + static void set_with_color(bool enabled) noexcept; +}; + +/// \brief used to keep a static registry of loggers +/// since the typical use case is to do: +/// \code {.cpp} +/// static seastar::logger("my_module"); +/// \endcode +/// this class is used to wrap around the static map +/// that holds pointers to all logs +/// +class logger_registry { + mutable std::mutex _mutex; + std::unordered_map<sstring, logger*> _loggers; +public: + /// loops through all registered loggers and sets the log level + /// Note: this method locks + /// + /// \param level - desired level: error,info,... + void set_all_loggers_level(log_level level); + + /// Given a name for a logger returns the log_level enum + /// Note: this method locks + /// + /// \return log_level for the given logger name + log_level get_logger_level(sstring name) const; + + /// Sets the log level for a given logger + /// Note: this method locks + /// + /// \param name - name of logger + /// \param level - desired level of logging + void set_logger_level(sstring name, log_level level); + + /// Returns a list of registered loggers + /// Note: this method locks + /// + /// \return all registered loggers + std::vector<sstring> get_all_logger_names(); + + /// Registers a logger with the static map + /// Note: this method locks + /// + void register_logger(logger* l); + /// Unregisters a logger with the static map + /// Note: this method locks + /// + void unregister_logger(logger* l); + /// Swaps the logger given the from->name() in the static map + /// Note: this method locks + /// + void moved(logger* from, logger* to); +}; + +logger_registry& global_logger_registry(); + +/// \brief Timestamp style. +enum class logger_timestamp_style { + none, + boot, + real, +}; + +/// \brief Output stream to use for logging. +enum class logger_ostream_type { + none, + stdout, + stderr, +}; + +struct logging_settings final { + std::unordered_map<sstring, log_level> logger_levels; + log_level default_level; + bool stdout_enabled; + bool syslog_enabled; + bool with_color; + logger_timestamp_style stdout_timestamp_style = logger_timestamp_style::real; + logger_ostream_type logger_ostream = logger_ostream_type::stderr; +}; + +/// Shortcut for configuring the logging system all at once. +/// +void apply_logging_settings(const logging_settings&); + +/// \cond internal + +extern thread_local uint64_t logging_failures; + +sstring pretty_type_name(const std::type_info&); + +sstring level_name(log_level level); + +template <typename T> +class logger_for : public logger { +public: + logger_for() : logger(pretty_type_name(typeid(T))) {} +}; + +/// \endcond +} // end seastar namespace + +// Pretty-printer for exceptions to be logged, e.g., std::current_exception(). +namespace std { +std::ostream& operator<<(std::ostream&, const std::exception_ptr&); +std::ostream& operator<<(std::ostream&, const std::exception&); +std::ostream& operator<<(std::ostream&, const std::system_error&); +} + +#if FMT_VERSION >= 90000 +template <> struct fmt::formatter<std::exception_ptr> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<std::exception> : fmt::ostream_formatter {}; +template <> struct fmt::formatter<std::system_error> : fmt::ostream_formatter {}; +#endif + +/// @} diff --git a/src/seastar/include/seastar/util/memory_diagnostics.hh b/src/seastar/include/seastar/util/memory_diagnostics.hh new file mode 100644 index 000000000..aa0fd097d --- /dev/null +++ b/src/seastar/include/seastar/util/memory_diagnostics.hh @@ -0,0 +1,99 @@ +/* + * 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 2020 ScyllaDB + */ + +#pragma once + +#include <seastar/core/sstring.hh> +#include <seastar/util/noncopyable_function.hh> + +namespace seastar { + +enum class log_level; + +namespace memory { + +/// \brief The kind of allocation failures to dump diagnostics report for. +/// +/// Note that if the seastar_memory logger is set to level debug, there will +/// be a report dumped for any allocation failure, regardless of this +/// configuration. +enum class alloc_failure_kind { + /// Dump diagnostic error report for none of the allocation failures. + none, + /// Dump diagnostic error report for critical allocation failures, see + /// \ref scoped_critical_alloc_section. + critical, + /// Dump diagnostic error report for all the allocation failures. + all, +}; + +/// \brief Configure when memory diagnostics are dumped. +/// +/// See \ref alloc_failure_kind on available options. +/// Applies configuration on all shards. +void set_dump_memory_diagnostics_on_alloc_failure_kind(alloc_failure_kind); + +/// \brief Configure when memory diagnostics are dumped. +/// +/// String version. See \ref alloc_failure_kind on available options. +/// Applies configuration on all shards. +void set_dump_memory_diagnostics_on_alloc_failure_kind(std::string_view); + +/// \brief A functor which writes its argument into the diagnostics report. +using memory_diagnostics_writer = noncopyable_function<void(std::string_view)>; + +/// \brief Set a producer of additional diagnostic information. +/// +/// This allows the application running on top of seastar to add its own part to +/// the diagnostics dump. The application can supply higher level diagnostics +/// information, that might help explain how the memory was consumed. +/// +/// The application specific part will be added just below the main stats +/// (free/used/total memory). +/// +/// \param producer - the functor to produce the additional diagnostics, specific +/// to the application, to be added to the generated report. The producer is +/// passed a writer functor, which it can use to add its parts to the report. +/// +/// \note As the report is generated at a time when allocations are failing, the +/// producer should try as hard as possible to not allocate while producing +/// the output. +void set_additional_diagnostics_producer(noncopyable_function<void(memory_diagnostics_writer)> producer); + +/// Generate and return a diagnostics report as a string. +/// +/// Note that contrary to the automated report generation (triggered by +/// allocation failure), this method does allocate memory and can fail in +/// low-memory conditions. +sstring generate_memory_diagnostics_report(); + +namespace internal { +/// Log the memory diagnostics to the internal logger in the same way as +/// during an allocation failure, at the given log level. These reports +/// are not rate limited, unlike the internally generated reports which +/// are limited to 1 per 10 seconds. +/// +/// This method attempts to avoid any allocations. +void log_memory_diagnostics_report(log_level lvl); +} + +} // namespace memory +} // namespace seastar diff --git a/src/seastar/include/seastar/util/noncopyable_function.hh b/src/seastar/include/seastar/util/noncopyable_function.hh new file mode 100644 index 000000000..2c9e69593 --- /dev/null +++ b/src/seastar/include/seastar/util/noncopyable_function.hh @@ -0,0 +1,234 @@ +/* + * 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) 2017 ScyllaDB Ltd. + */ + +#pragma once + +#include <seastar/util/used_size.hh> +#include <seastar/util/concepts.hh> +#include <utility> +#include <type_traits> +#include <functional> + +namespace seastar { + +template <typename Signature> +class noncopyable_function; + +namespace internal { + +class noncopyable_function_base { +private: + noncopyable_function_base() = default; + static constexpr size_t nr_direct = 32; + union [[gnu::may_alias]] storage { + char direct[nr_direct]; + void* indirect; + }; + using move_type = void (*)(noncopyable_function_base* from, noncopyable_function_base* to); + using destroy_type = void (*)(noncopyable_function_base* func); + + static void empty_move(noncopyable_function_base*, noncopyable_function_base*) {} + static void empty_destroy(noncopyable_function_base*) {} + + static void indirect_move(noncopyable_function_base* from, noncopyable_function_base* to) { + using void_ptr = void*; + new (&to->_storage.indirect) void_ptr(from->_storage.indirect); + } + + template <size_t N> + static void trivial_direct_move(noncopyable_function_base* from, noncopyable_function_base* to) { + // We use bytewise copy here since we lost the type. This means that + // we will copy any holes/padding not initialized by the move + // constructor in direct_vtable_for::initialize(). This is okay, + // since we won't use those holes/padding, but gcc doesn't know + // that, and complains. Silence it. +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wuninitialized" + // Avoid including <algorithm> just for this + for (unsigned i = 0; i != N; ++i) { + to->_storage.direct[i] = from->_storage.direct[i]; + } +#pragma GCC diagnostic pop + } + + static void trivial_direct_destroy(noncopyable_function_base*) { + } + +private: + storage _storage; + + template <typename Signature> + friend class seastar::noncopyable_function; +}; + +template<typename FirstArg = void, typename... RemainingArgs> +struct is_nothrow_if_object { + static constexpr bool value = is_nothrow_if_object<FirstArg>::value && is_nothrow_if_object<RemainingArgs...>::value; +}; + +template<typename Arg> +struct is_nothrow_if_object<Arg> { + static constexpr bool value = !std::is_object<Arg>::value || std::is_nothrow_move_constructible<Arg>::value; +}; + +template<> +struct is_nothrow_if_object<> { + static constexpr bool value = true; +}; + +} + +/// A clone of \c std::function, but only invokes the move constructor +/// of the contained function. +template <typename Ret, typename... Args, bool Noexcept> +class noncopyable_function<Ret (Args...) noexcept(Noexcept)> : private internal::noncopyable_function_base { + using call_type = Ret (*)(const noncopyable_function* func, Args...); + struct vtable { + const call_type call; + const move_type move; + const destroy_type destroy; + }; +private: + const vtable* _vtable; +private: + static Ret empty_call(const noncopyable_function*, [[maybe_unused]] Args... args) { + throw std::bad_function_call(); + } + + static constexpr vtable _s_empty_vtable = {empty_call, empty_move, empty_destroy}; + + template <typename Func> + struct direct_vtable_for { + static Func* access(noncopyable_function* func) { return reinterpret_cast<Func*>(func->_storage.direct); } + static const Func* access(const noncopyable_function* func) { return reinterpret_cast<const Func*>(func->_storage.direct); } + static Func* access(noncopyable_function_base* func) { return access(static_cast<noncopyable_function*>(func)); } + static Ret call(const noncopyable_function* func, Args... args) noexcept(Noexcept) { + return (*access(const_cast<noncopyable_function*>(func)))(std::forward<Args>(args)...); + } + static void move(noncopyable_function_base* from, noncopyable_function_base* to) { + new (access(to)) Func(std::move(*access(from))); + destroy(from); + } + static constexpr move_type select_move_thunk() { + bool can_trivially_move = std::is_trivially_move_constructible<Func>::value + && std::is_trivially_destructible<Func>::value; + return can_trivially_move ? trivial_direct_move<internal::used_size<Func>::value> : move; + } + static void destroy(noncopyable_function_base* func) { + access(func)->~Func(); + } + static constexpr destroy_type select_destroy_thunk() { + return std::is_trivially_destructible<Func>::value ? trivial_direct_destroy : destroy; + } + static void initialize(Func&& from, noncopyable_function* to) { + new (access(to)) Func(std::move(from)); + } + static constexpr vtable make_vtable() { return { call, select_move_thunk(), select_destroy_thunk() }; } + static const vtable s_vtable; + }; + template <typename Func> + struct indirect_vtable_for { + static Func* access(noncopyable_function* func) { return reinterpret_cast<Func*>(func->_storage.indirect); } + static const Func* access(const noncopyable_function* func) { return reinterpret_cast<const Func*>(func->_storage.indirect); } + static Func* access(noncopyable_function_base* func) { return access(static_cast<noncopyable_function*>(func)); } + static Ret call(const noncopyable_function* func, Args... args) noexcept(Noexcept) { + return (*access(const_cast<noncopyable_function*>(func)))(std::forward<Args>(args)...); + } + static void destroy(noncopyable_function_base* func) { + delete access(func); + } + static void initialize(Func&& from, noncopyable_function* to) { + to->_storage.indirect = new Func(std::move(from)); + } + static constexpr vtable make_vtable() { return { call, indirect_move, destroy }; } + static const vtable s_vtable; + }; + template <typename Func, bool Direct = true> + struct select_vtable_for : direct_vtable_for<Func> {}; + template <typename Func> + struct select_vtable_for<Func, false> : indirect_vtable_for<Func> {}; + template <typename Func> + static constexpr bool is_direct() { + return sizeof(Func) <= nr_direct && alignof(Func) <= alignof(storage) + && std::is_nothrow_move_constructible<Func>::value; + } + template <typename Func> + struct vtable_for : select_vtable_for<Func, is_direct<Func>()> {}; +public: + noncopyable_function() noexcept : _vtable(&_s_empty_vtable) {} + template <typename Func> + SEASTAR_CONCEPT( requires std::is_invocable_r_v<Ret, Func, Args...> ) + noncopyable_function(Func func) { + static_assert(!Noexcept || noexcept(std::declval<Func>()(std::declval<Args>()...))); + vtable_for<Func>::initialize(std::move(func), this); + _vtable = &vtable_for<Func>::s_vtable; + } + template <typename Object, typename... AllButFirstArg> + noncopyable_function(Ret (Object::*member)(AllButFirstArg...) noexcept(Noexcept)) : noncopyable_function(std::mem_fn(member)) {} + template <typename Object, typename... AllButFirstArg> + noncopyable_function(Ret (Object::*member)(AllButFirstArg...) const noexcept(Noexcept)) : noncopyable_function(std::mem_fn(member)) {} + + ~noncopyable_function() { + _vtable->destroy(this); + } + + noncopyable_function(const noncopyable_function&) = delete; + noncopyable_function& operator=(const noncopyable_function&) = delete; + + noncopyable_function(noncopyable_function&& x) noexcept : _vtable(std::exchange(x._vtable, &_s_empty_vtable)) { + _vtable->move(&x, this); + } + + noncopyable_function& operator=(noncopyable_function&& x) noexcept { + if (this != &x) { + this->~noncopyable_function(); + new (this) noncopyable_function(std::move(x)); + } + return *this; + } + + Ret operator()(Args... args) const noexcept(Noexcept) { + static_assert(!Noexcept || internal::is_nothrow_if_object<Args...>::value); + return _vtable->call(this, std::forward<Args>(args)...); + } + + explicit operator bool() const { + return _vtable != &_s_empty_vtable; + } +}; + + +template <typename Ret, typename... Args, bool Noexcept> +constexpr typename noncopyable_function<Ret (Args...) noexcept(Noexcept)>::vtable noncopyable_function<Ret (Args...) noexcept(Noexcept)>::_s_empty_vtable; + +template <typename Ret, typename... Args, bool Noexcept> +template <typename Func> +const typename noncopyable_function<Ret (Args...) noexcept(Noexcept)>::vtable noncopyable_function<Ret (Args...) noexcept(Noexcept)>::direct_vtable_for<Func>::s_vtable + = noncopyable_function<Ret (Args...) noexcept(Noexcept)>::direct_vtable_for<Func>::make_vtable(); + + +template <typename Ret, typename... Args, bool Noexcept> +template <typename Func> +const typename noncopyable_function<Ret (Args...) noexcept(Noexcept)>::vtable noncopyable_function<Ret (Args...) noexcept(Noexcept)>::indirect_vtable_for<Func>::s_vtable + = noncopyable_function<Ret (Args...) noexcept(Noexcept)>::indirect_vtable_for<Func>::make_vtable(); + +} + diff --git a/src/seastar/include/seastar/util/optimized_optional.hh b/src/seastar/include/seastar/util/optimized_optional.hh new file mode 100644 index 000000000..60febceae --- /dev/null +++ b/src/seastar/include/seastar/util/optimized_optional.hh @@ -0,0 +1,101 @@ +/* + * 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) 2017 ScyllaDB + */ + +#pragma once + +#include <seastar/util/concepts.hh> +#include <seastar/util/std-compat.hh> + +#include <type_traits> +#include <iostream> + +namespace seastar { + +SEASTAR_CONCEPT( + +template<typename T> +concept OptimizableOptional = + std::is_default_constructible<T>::value + && std::is_nothrow_move_assignable<T>::value + && requires(const T& obj) { + { bool(obj) } noexcept; + }; + +) + +/// \c optimized_optional<> is intended mainly for use with classes that store +/// their data externally and expect pointer to this data to be always non-null. +/// In such case there is no real need for another flag signifying whether +/// the optional is engaged. +template<typename T> +class optimized_optional { + T _object; +public: + optimized_optional() = default; + optimized_optional(std::nullopt_t) noexcept { } + optimized_optional(const T& obj) : _object(obj) { } + optimized_optional(T&& obj) noexcept : _object(std::move(obj)) { } + optimized_optional(std::optional<T>&& obj) noexcept { + if (obj) { + _object = std::move(*obj); + } + } + optimized_optional(const optimized_optional&) = default; + optimized_optional(optimized_optional&&) = default; + + optimized_optional& operator=(std::nullopt_t) noexcept { + _object = T(); + return *this; + } + template<typename U> + std::enable_if_t<std::is_same<std::decay_t<U>, T>::value, optimized_optional&> + operator=(U&& obj) noexcept { + _object = std::forward<U>(obj); + return *this; + } + optimized_optional& operator=(const optimized_optional&) = default; + optimized_optional& operator=(optimized_optional&&) = default; + + explicit operator bool() const noexcept { + return bool(_object); + } + + T* operator->() noexcept { return &_object; } + const T* operator->() const noexcept { return &_object; } + + T& operator*() noexcept { return _object; } + const T& operator*() const noexcept { return _object; } + + bool operator==(const optimized_optional& other) const { + return _object == other._object; + } + bool operator!=(const optimized_optional& other) const { + return _object != other._object; + } + friend std::ostream& operator<<(std::ostream& out, const optimized_optional& opt) { + if (!opt) { + return out << "null"; + } + return out << *opt; + } +}; + +} diff --git a/src/seastar/include/seastar/util/print_safe.hh b/src/seastar/include/seastar/util/print_safe.hh new file mode 100644 index 000000000..d7da26cf1 --- /dev/null +++ b/src/seastar/include/seastar/util/print_safe.hh @@ -0,0 +1,131 @@ +/* + * 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 2016 ScyllaDB + */ + +#pragma once + +#include <seastar/util/concepts.hh> +#include <cassert> +#include <cstring> +#if __cplusplus > 201703L +#include <concepts> +#endif +#include <stdio.h> +#include <unistd.h> + +namespace seastar { + +// +// Collection of async-signal safe printing functions. +// + +// Outputs string to stderr. +// Async-signal safe. +inline +void print_safe(const char *str, size_t len) noexcept { + while (len) { + auto result = write(STDERR_FILENO, str, len); + if (result > 0) { + len -= result; + str += result; + } else if (result == 0) { + break; + } else { + if (errno == EINTR) { + // retry + } else { + break; // what can we do? + } + } + } +} + +// Outputs string to stderr. +// Async-signal safe. +inline +void print_safe(const char *str) noexcept { + print_safe(str, strlen(str)); +} + +// Fills a buffer with a hexadecimal representation of an integer +// and returns a pointer to the first character. +// For example, convert_hex_safe(buf, 4, uint16_t(12)) fills the buffer with " c". +template<typename Integral, char Padding = ' '> +SEASTAR_CONCEPT( requires std::integral<Integral> ) +char* convert_hex_safe(char *buf, size_t bufsz, Integral n) noexcept { + const char *digits = "0123456789abcdef"; + memset(buf, Padding, bufsz); + auto* p = buf + bufsz; + do { + assert(p > buf); + *--p = digits[n & 0xf]; + n >>= 4; + } while (n); + return p; +} + +// Fills a buffer with a zero-padded hexadecimal representation of an integer. +// For example, convert_zero_padded_hex_safe(buf, 4, uint16_t(12)) fills the buffer with "000c". +template<typename Integral> +SEASTAR_CONCEPT( requires std::integral<Integral> ) +void convert_zero_padded_hex_safe(char *buf, size_t bufsz, Integral n) noexcept { + convert_hex_safe<'0'>(buf, bufsz, n); +} + +// Prints zero-padded hexadecimal representation of an integer to stderr. +// For example, print_zero_padded_hex_safe(uint16_t(12)) prints "000c". +// Async-signal safe. +template<typename Integral> +SEASTAR_CONCEPT ( requires std::signed_integral<Integral> ) +void print_zero_padded_hex_safe(Integral n) noexcept { + char buf[sizeof(n) * 2]; + convert_zero_padded_hex_safe(buf, sizeof(buf), n); + print_safe(buf, sizeof(buf)); +} + +// Fills a buffer with a decimal representation of an integer. +// The argument bufsz is the maximum size of the buffer. +// For example, print_decimal_safe(buf, 16, 12) prints "12". +template<typename Integral> +SEASTAR_CONCEPT( requires std::unsigned_integral<Integral> ) +size_t convert_decimal_safe(char *buf, size_t bufsz, Integral n) noexcept { + char tmp[sizeof(n) * 3]; + unsigned i = bufsz; + do { + assert(i > 0); + tmp[--i] = '0' + n % 10; + n /= 10; + } while (n); + memcpy(buf, tmp + i, sizeof(tmp) - i); + return sizeof(tmp) - i; +} + +// Prints decimal representation of an integer to stderr. +// For example, print_decimal_safe(12) prints "12". +// Async-signal safe. +template<typename Integral> +void print_decimal_safe(Integral n) noexcept { + char buf[sizeof(n) * 3]; + unsigned i = sizeof(buf); + auto len = convert_decimal_safe(buf, i, n); + print_safe(buf, len); +} + +} diff --git a/src/seastar/include/seastar/util/process.hh b/src/seastar/include/seastar/util/process.hh new file mode 100644 index 000000000..d38a6111e --- /dev/null +++ b/src/seastar/include/seastar/util/process.hh @@ -0,0 +1,106 @@ +/* + * 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) 2022 Kefu Chai ( tchaikov@gmail.com ) + */ + +#pragma once + +#include <sys/types.h> +#include <algorithm> +#include <filesystem> +#include <initializer_list> +#include <iterator> +#include <string_view> +#include <utility> +#include <variant> +#include <vector> +#include <fmt/format.h> +#include <seastar/core/iostream.hh> +#include <seastar/core/posix.hh> +#include <seastar/core/sstring.hh> + +namespace seastar::experimental { + +/// The optional parameters for spawning a subprocess +/// +/// \note see \c execve(2) for more details on \c argv and \c env. +struct spawn_parameters { + /// The arguments passed to the program + std::vector<sstring> argv; + /// The environment variables for the program + std::vector<sstring> env; +}; + +/// Interact with a spawned subprocess +/// +/// \note the spawned subprocess should always be \c wait()'ed. Otherwise, +/// the Seastar application spawning the subprocess will leave us with +/// one ore more zombie subprocesses after it exists. +class process { + struct create_tag {}; + /// Spawn a subprocess using \c posix_spawn(3) + /// + /// \param pathname the full path to the executable + /// \param params parameters for spawning the subprocess + /// + /// \returns a \c process instance representing the spawned subprocess + static future<process> spawn(const std::filesystem::path& pathname, + spawn_parameters params); + /// Spawn a subprocess using \c posix_spawn(3) + /// + /// \param pathname the full path to the executable + /// + /// \returns a \c process instance representing the spawned subprocess + static future<process> spawn(const std::filesystem::path& pathname); +public: + process(create_tag, pid_t pid, file_desc&& stdin, file_desc&& stdout, file_desc&& stderr); + /// Return an writable stream which provides input from the child process + output_stream<char> stdin(); + /// Return an writable stream which provides stdout output from the child process + input_stream<char> stdout(); + /// Return an writable stream which provides stderr output from the child process + input_stream<char> stderr(); + struct wait_exited { + int exit_code; + }; + struct wait_signaled { + int terminating_signal; + }; + using wait_status = std::variant<wait_exited, wait_signaled>; + /// Wait until the child process exits or terminates + /// + /// \returns the exit status + future<wait_status> wait(); + /// Stop the process using SIGTERM + void terminate(); + /// Force the process to exit using SIGKILL + void kill(); + +private: + const pid_t _pid; + file_desc _stdin; + file_desc _stdout; + file_desc _stderr; + + friend future<process> spawn_process(const std::filesystem::path&, + spawn_parameters); + friend future<process> spawn_process(const std::filesystem::path&); +}; +} diff --git a/src/seastar/include/seastar/util/program-options.hh b/src/seastar/include/seastar/util/program-options.hh new file mode 100644 index 000000000..73848f346 --- /dev/null +++ b/src/seastar/include/seastar/util/program-options.hh @@ -0,0 +1,619 @@ +/* + * 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) 2017 ScyllaDB + */ + +#pragma once + +#include <seastar/core/sstring.hh> +#include <seastar/core/print.hh> + +#include <boost/any.hpp> +#include <boost/intrusive/list.hpp> + +#include <string> +#include <unordered_map> +#include <vector> +#include <set> + +/// \defgroup program-options Program Options +/// +/// \brief Infrastructure for configuring a seastar application +/// +/// The program-options infrastructure allows configuring seastar both by C++ +/// code and by command-line and/or config files. This is achieved by +/// providing a set of self-describing and self-validating value types as well +/// as option groups to allow grouping them into arbitrary tree structures. +/// Seastar modules expose statically declared option structs, which derive from +/// \ref option_group and contain various concrete \ref basic_value members +/// comprising the required configuration. These structs are self-describing, and +/// self-validating, the name of the option group as well as the list of its +/// \ref basic_value member can be queried run-time. + +namespace seastar { + +namespace program_options { + +/// +/// \brief Wrapper for command-line options with arbitrary string associations. +/// +/// This type, to be used with Boost.Program_options, will result in an option that stores an arbitrary number of +/// string associations. +/// +/// Values are specified in the form "key0=value0:[key1=value1:...]". Options of this type can be specified multiple +/// times, and the values will be merged (with the last-provided value for a key taking precedence). +/// +/// \note We need a distinct type (rather than a simple type alias) for overload resolution in the implementation, but +/// advertizing our inheritance of \c std::unordered_map would introduce the possibility of memory leaks since STL +/// containers do not declare virtual destructors. +/// +class string_map final : private std::unordered_map<sstring, sstring> { +private: + using base = std::unordered_map<sstring, sstring>; +public: + using base::value_type; + using base::key_type; + using base::mapped_type; + + using base::base; + using base::at; + using base::find; + using base::count; + using base::emplace; + using base::clear; + using base::operator[]; + using base::begin; + using base::end; + + friend bool operator==(const string_map&, const string_map&); + friend bool operator!=(const string_map&, const string_map&); +}; + +inline bool operator==(const string_map& lhs, const string_map& rhs) { + return static_cast<const string_map::base&>(lhs) == static_cast<const string_map::base&>(rhs); +} + +inline bool operator!=(const string_map& lhs, const string_map& rhs) { + return !(lhs == rhs); +} + +/// +/// \brief Query the value of a key in a \c string_map, or a default value if the key doesn't exist. +/// +sstring get_or_default(const string_map&, const sstring& key, const sstring& def = sstring()); + +std::istream& operator>>(std::istream& is, string_map&); +std::ostream& operator<<(std::ostream& os, const string_map&); + +/// \cond internal + +// +// Required implementation hook for Boost.Program_options. +// +void validate(boost::any& out, const std::vector<std::string>& in, string_map*, int); + +using list_base_hook = boost::intrusive::list_base_hook<boost::intrusive::link_mode<boost::intrusive::auto_unlink>>; + +} // namespace program_options + +enum class log_level; +enum class logger_timestamp_style; +enum class logger_ostream_type; + +namespace memory { + enum class alloc_failure_kind; +} + +namespace program_options { + +/// \endcond + +/// \addtogroup program-options +/// @{ + +class option_group; + +/// Visitor interface for \ref option_group::describe(). +/// +/// See \ref option_group::describe() for more details on the visiting algorithm. +class options_descriptor { +public: + /// Visit the start of the group. + /// + /// Called when entering a group. Groups can be nested, in which case there + /// will be another call to this method, before the current groups is closed. + /// \returns whether visitor is interested in the group: \p true - visit, + /// \p false - skip. + virtual bool visit_group_start(const std::string& name, bool used) = 0; + /// Visit the end of the group. + /// + /// Called after all values and nested groups were visited in the current + /// group. + virtual void visit_group_end() = 0; + + /// Visit value metadata, common across all value types. + /// + /// Called at the start of visiting a value. After this, a call to the + /// appropriate \ref visit_value() overload (or \ref visit_selection_value()) + /// follows. + /// \returns whether visitor is interested in the value: \p true - visit, + /// \p false - skip. + virtual bool visit_value_metadata(const std::string& name, const std::string& description, bool used) = 0; + + /// Visit a switch (\ref value<std::monostate>). + virtual void visit_value() = 0; + /// Visit a value (\ref value), \p default_val is null when value has no default. + virtual void visit_value(const bool* default_val) = 0; + /// Visit a value (\ref value), \p default_val is null when value has no default. + virtual void visit_value(const int* default_val) = 0; + /// Visit a value (\ref value), \p default_val is null when value has no default. + virtual void visit_value(const unsigned* default_val) = 0; + /// Visit a value (\ref value), \p default_val is null when value has no default. + virtual void visit_value(const float* default_val) = 0; + /// Visit a value (\ref value), \p default_val is null when value has no default. + virtual void visit_value(const double* default_val) = 0; + /// Visit a value (\ref value), \p default_val is null when value has no default. + virtual void visit_value(const std::string* default_val) = 0; + /// Visit a value (\ref value), \p default_val is null when value has no default. + virtual void visit_value(const std::set<unsigned>* default_val) = 0; + /// Visit a value (\ref value), \p default_val is null when value has no default. + virtual void visit_value(const log_level* default_val) = 0; + /// Visit a value (\ref value), \p default_val is null when value has no default. + virtual void visit_value(const logger_timestamp_style* default_val) = 0; + /// Visit a value (\ref value), \p default_val is null when value has no default. + virtual void visit_value(const logger_ostream_type* default_val) = 0; + /// Visit a value (\ref value), \p default_val is null when value has no default. + virtual void visit_value(const memory::alloc_failure_kind* default_val) = 0; + /// Visit a value (\ref value), \p default_val is null when value has no default. + virtual void visit_value(const std::unordered_map<sstring, log_level>* default_val) = 0; + + /// Visit a selection value (\ref selection_value), \p default_candidate is null when there is no default candidate. + virtual void visit_selection_value(const std::vector<std::string>& candidate_names, const std::size_t* default_candidate) = 0; +}; + +/// Visitor interface \ref option_group::mutate(). +/// +/// See \ref option_group::mutate() for more details on the visiting algorithm. +class options_mutator { +public: + /// Visit the start of the group. + /// + /// Called when entering a group. Groups can be nested, in which case there + /// will be another call to this method, before the current groups is closed. + /// \returns whether visitor is interested in the group: \p true - visit, + /// \p false - skip. + virtual bool visit_group_start(const std::string& name, bool used) = 0; + /// Visit the end of the group. + /// + /// Called after all values and nested groups were visited in the current + /// group. + virtual void visit_group_end() = 0; + + /// Visit value metadata, common across all value types. + /// + /// Called at the start of visiting a value. After this, a call to the + /// appropriate \ref visit_value() overload (or \ref visit_selection_value()) + /// follows. + /// \returns whether visitor is interested in the value: \p true - visit, + /// \p false - skip. + virtual bool visit_value_metadata(const std::string& name, bool used) = 0; + + /// Visit a switch (\ref value<std::monostate>), switch is set to returned value. + virtual bool visit_value() = 0; + /// Visit and optionally mutate a value (\ref value), should return true if value was mutated. + virtual bool visit_value(bool& val) = 0; + /// Visit a value (\ref value), \p default_val is null when value has no default. + virtual bool visit_value(int& val) = 0; + /// Visit and optionally mutate a value (\ref value), should return true if value was mutated. + virtual bool visit_value(unsigned& val) = 0; + /// Visit and optionally mutate a value (\ref value), should return true if value was mutated. + virtual bool visit_value(float& val) = 0; + /// Visit and optionally mutate a value (\ref value), should return true if value was mutated. + virtual bool visit_value(double& val) = 0; + /// Visit and optionally mutate a value (\ref value), should return true if value was mutated. + virtual bool visit_value(std::string& val) = 0; + /// Visit and optionally mutate a value (\ref value), should return true if value was mutated. + virtual bool visit_value(std::set<unsigned>& val) = 0; + /// Visit and optionally mutate a value (\ref value), should return true if value was mutated. + virtual bool visit_value(log_level& val) = 0; + /// Visit and optionally mutate a value (\ref value), should return true if value was mutated. + virtual bool visit_value(logger_timestamp_style& val) = 0; + /// Visit and optionally mutate a value (\ref value), should return true if value was mutated. + virtual bool visit_value(logger_ostream_type& val) = 0; + /// Visit and optionally mutate a value (\ref value), should return true if value was mutated. + virtual bool visit_value(memory::alloc_failure_kind& val) = 0; + /// Visit and optionally mutate a value (\ref value), should return true if value was mutated. + virtual bool visit_value(std::unordered_map<sstring, log_level>& val) = 0; + + /// Visit and optionally mutate a selection value (\ref selection_value), should return true if value was mutated. + virtual bool visit_selection_value(const std::vector<std::string>& candidate_names, std::size_t& selected_candidate) = 0; +}; + +/// A tag type used to construct unused \ref option_group and \ref basic_value objects. +struct unused {}; + +class basic_value; + +/// A group of options. +/// +/// \ref option_group is the basis for organizing options. It can hold a number +/// of \ref basic_value objects. These are typically also its members: +/// +/// struct my_option_group : public option_group { +/// value<> opt1; +/// value<bool> opt2; +/// ... +/// +/// my_option_group() +/// : option_group(nullptr, "My option group") +/// , opt1(this, "opt1", ... +/// , opt2(this, "opt2", ... +/// , ... +/// { } +/// }; +/// +/// Option groups can also be nested and using this property one can build a +/// tree of option groups and values. This tree then can be visited using the +/// two visitor methods exposed by \ref option_group: +/// * \ref describe() +/// * \ref mutate() +/// +/// Using these two visitors one can easily implement glue code to expose an +/// entire options tree to the command line. Use \ref describe() to build the +/// command-line level description of the objects (using e.g. +/// boost::program_options) and after parsing the provided command-line options +/// use \ref mutate() to propagate the extracted values back into the options +/// tree. How this is done is entirely up to the visitors, the above methods +/// only offer an API to visit each group and value in the tree, they don't make +/// any assumption about how the visitor works and what its purpose is. +class option_group : public list_base_hook { + friend class basic_value; + +public: + using value_list_type = boost::intrusive::list< + basic_value, + boost::intrusive::base_hook<list_base_hook>, + boost::intrusive::constant_time_size<false>>; + + using option_group_list_type = boost::intrusive::list< + option_group, + boost::intrusive::base_hook<list_base_hook>, + boost::intrusive::constant_time_size<false>>; + +private: + option_group* _parent; + bool _used = true; + std::string _name; + value_list_type _values; + option_group_list_type _subgroups; + +public: + /// Construct an option group. + /// + /// \param parent - the parent option-group, this option group will become a + /// sub option group of the parent group + /// \param name - the name of the option group + explicit option_group(option_group* parent, std::string name); + /// Construct an unused option group. + /// + /// \param parent - the parent option-group, this option group will become a + /// sub option group of the parent group + /// \param name - the name of the option group + explicit option_group(option_group* parent, std::string name, unused); + option_group(option_group&&); + option_group(const option_group&) = delete; + virtual ~option_group() = default; + + option_group& operator=(option_group&&) = delete; + option_group& operator=(const option_group&) = delete; + + /// Does the option group has any values contained in it? + operator bool () const { return !_values.empty(); } + bool used() const { return _used; } + const std::string& name() const { return _name; } + const value_list_type& values() const { return _values; } + value_list_type& values() { return _values; } + + /// Describe the content of this option group to the visitor. + /// + /// The content is visited in a depth-first manner: + /// * First the option groups itself is visited with + /// \ref options_descriptor::visit_group_start(). If this returns \p false + /// the entire content of the group, including all its subgroups and values + /// are skipped and \ref options_descriptor::visit_group_end() is called + /// immediately. Otherwise visiting the content of the group proceeds. + /// * All the values contained therein are visited. For each value the + /// following happens: + /// - First \ref options_descriptor::visit_value_metadata() is called + /// with generic metadata that all values have. If this return + /// \p false the value is skipped, otherwise visiting the value + /// proceeds. + /// - Then the appropriate overload of + /// \ref options_descriptor::visit_value() is called, with a pointer + /// to the default value of the respective value. The pointer is null + /// if there is no default value. + /// - For \ref selection_value, + /// \ref options_descriptor::visit_selection_value() will be called + /// instead of \ref options_descriptor::visit_value(). After the value + /// is visited, the \ref option_group instance belonging to each + /// candidate (if set) will be visited. + /// * All the nested \ref option_group instances in the current group are + /// visited. + /// * Finally \ref options_descriptor::visit_group_end() is called. + void describe(options_descriptor& descriptor) const; + /// Mutate the content of this option group by the visitor. + /// + /// The visiting algorithm is identical to that of \ref describe(), with the + /// following differences: + /// * \ref options_mutator::visit_value() is allowed to mutate the value + /// through the passed-in reference. It should return \p true if it did so + /// and \p false otherwise. + /// * When visiting a selection value, only the nested group belonging to + /// the selected value is visited afterwards. + void mutate(options_mutator& mutator); +}; + +/// A basic configuration option value. +/// +/// This serves as the common base-class of all the concrete value types. +class basic_value : public list_base_hook { + friend class option_group; + +public: + option_group* _group; + bool _used = true; + std::string _name; + std::string _description; + +private: + virtual void do_describe(options_descriptor& descriptor) const = 0; + virtual void do_mutate(options_mutator& mutator) = 0; + +public: + basic_value(option_group& group, bool used, std::string name, std::string description); + basic_value(basic_value&&); + basic_value(const basic_value&) = delete; + virtual ~basic_value() = default; + + basic_value& operator=(basic_value&&) = delete; + basic_value& operator=(const basic_value&) = delete; + + bool used() const { return _used; } + const std::string& name() const { return _name; } + const std::string& description() const { return _description; } + + void describe(options_descriptor& descriptor) const; + void mutate(options_mutator& mutator); +}; + +/// A configuration option value. +/// +/// \tparam T the type of the contained value. +template <typename T = std::monostate> +class value : public basic_value { + std::optional<T> _value; + bool _defaulted = true; + +private: + virtual void do_describe(options_descriptor& descriptor) const override { + auto* val = _value ? &*_value : nullptr; + descriptor.visit_value(val); + } + virtual void do_mutate(options_mutator& mutator) override { + T val; + if (mutator.visit_value(val)) { + _value = std::move(val); + _defaulted = false; + } + } + void do_set_value(T value, bool defaulted) { + _value = std::move(value); + _defaulted = defaulted; + } + +public: + /// Construct a value. + /// + /// \param group - the group containing this value + /// \param name - the name of this value + /// \param default_value - the default value, can be unset + /// \param description - the description of the value + value(option_group& group, std::string name, std::optional<T> default_value, std::string description) + : basic_value(group, true, std::move(name), std::move(description)) + , _value(std::move(default_value)) + { } + /// Construct an unused value. + value(option_group& group, std::string name, unused) + : basic_value(group, false, std::move(name), {}) + { } + value(value&&) = default; + /// Is there a contained value? + operator bool () const { return bool(_value); } + /// Does this value still contain a default-value? + bool defaulted() const { return _defaulted; } + /// Return the contained value, assumes there is one, see \ref operator bool(). + const T& get_value() const { return _value.value(); } + T& get_value() { return _value.value(); } + void set_default_value(T value) { do_set_value(std::move(value), true); } + void set_value(T value) { do_set_value(std::move(value), false); } +}; + +/// A switch-style configuration option value. +/// +/// Contains no value, can be set or not. +template <> +class value<std::monostate> : public basic_value { + std::optional<bool> _set; + +private: + virtual void do_describe(options_descriptor& descriptor) const override { + descriptor.visit_value(); + } + virtual void do_mutate(options_mutator& mutator) override { + bool is_set = mutator.visit_value(); + if (_set.has_value()) { + // override the value only if it is not preset + if (is_set) { + _set = true; + } + } else { + _set = is_set; + } + } + +public: + /// Construct a value. + /// + /// \param group - the group containing this value + /// \param name - the name of this value + /// \param description - the description of the value + value(option_group& group, std::string name, std::string description) + : basic_value(group, true, std::move(name), std::move(description)) + { } + /// Construct an unused value. + value(option_group& group, std::string name, unused) + : basic_value(group, false, std::move(name), {}) + { } + /// Is the option set? + operator bool () const { return _set ? _set.value() : false; } + void set_value() { _set = true; } + void unset_value() { _set = false; } +}; + +/// A selection value, allows selection from multiple candidates. +/// +/// The candidates objects are of an opaque type which may not accessible to +/// whoever is choosing between the available candidates. This allows the user +/// selecting between seastar internal types without exposing them. +/// Each candidate has a name, which is what the users choose based on. Each +/// candidate can also have an associated \ref option_group containing related +/// candidate-specific configuration options, allowing further configuring the +/// selected candidate. The code exposing the candidates should document the +/// concrete types these can be down-casted to. +template <typename T = std::monostate> +class selection_value : public basic_value { +public: + using deleter = std::function<void(T*)>; + using value_handle = std::unique_ptr<T, deleter>; + struct candidate { + std::string name; + value_handle value; + std::unique_ptr<option_group> opts; + }; + using candidates = std::vector<candidate>; + +private: + static constexpr size_t no_selected_candidate = -1; + +private: + candidates _candidates; + size_t _selected_candidate = no_selected_candidate; + bool _defaulted = true; + +private: + std::vector<std::string> get_candidate_names() const { + std::vector<std::string> candidate_names; + candidate_names.reserve(_candidates.size()); + for (const auto& c : _candidates) { + candidate_names.push_back(c.name); + } + return candidate_names; + } + virtual void do_describe(options_descriptor& descriptor) const override { + descriptor.visit_selection_value(get_candidate_names(), _selected_candidate == no_selected_candidate ? nullptr : &_selected_candidate); + for (auto& c : _candidates) { + if (c.opts) { + c.opts->describe(descriptor); + } + } + } + virtual void do_mutate(options_mutator& mutator) override { + if (mutator.visit_selection_value(get_candidate_names(), _selected_candidate)) { + _defaulted = false; + } + if (_selected_candidate != no_selected_candidate) { + auto& c = _candidates.at(_selected_candidate); + if (c.opts) { + c.opts->mutate(mutator); + } + } + } + size_t find_candidate(const std::string& candidate_name) { + auto it = find_if(_candidates.begin(), _candidates.end(), [&] (const auto& candidate) { + return candidate.name == candidate_name; + }); + if (it == _candidates.end()) { + throw std::invalid_argument(fmt::format("find_candidate(): failed to find candidate {}", candidate_name)); + } + return it - _candidates.begin(); + } + + option_group* do_select_candidate(std::string candidate_name, bool defaulted) { + _selected_candidate = find_candidate(candidate_name); + _defaulted = defaulted; + return _candidates.at(_selected_candidate).opts.get(); + } + +public: + /// Construct a value. + /// + /// \param group - the group containing this value + /// \param name - the name of this value + /// \param candidates - the available candidates + /// \param default_candidates - the name of the default candidate + /// \param description - the description of the value + selection_value(option_group& group, std::string name, candidates candidates, std::string default_candidate, std::string description) + : basic_value(group, true, std::move(name), std::move(description)) + , _candidates(std::move(candidates)) + , _selected_candidate(find_candidate(default_candidate)) + { } + selection_value(option_group& group, std::string name, candidates candidates, std::string description) + : basic_value(group, true, std::move(name), std::move(description)) + , _candidates(std::move(candidates)) + { } + /// Construct an unused value. + selection_value(option_group& group, std::string name, unused) + : basic_value(group, false, std::move(name), {}) + { } + /// Was there a candidate selected (default also counts)? + operator bool () const { return _selected_candidate != no_selected_candidate; } + /// Is the currently selected candidate the default one? + bool defaulted() const { return _defaulted; } + /// Get the name of the currently selected candidate (assumes there is one selected, see \operator bool()). + const std::string& get_selected_candidate_name() const { return _candidates.at(_selected_candidate).name; } + /// Get the options of the currently selected candidate (assumes there is one selected, see \operator bool()). + const option_group* get_selected_candidate_opts() const { return _candidates.at(_selected_candidate).opts.get(); } + /// Get the options of the currently selected candidate (assumes there is one selected, see \operator bool()). + option_group* get_selected_candidate_opts() { return _candidates.at(_selected_candidate).opts.get(); } + T& get_selected_candidate() const { return *_candidates.at(_selected_candidate).value; } + /// Select a candidate. + /// + /// \param candidate_name - the name of the to-be-selected candidate. + option_group* select_candidate(std::string candidate_name) { return do_select_candidate(candidate_name, false); } + /// Select a candidate and make it the default. + /// + /// \param candidate_name - the name of the to-be-selected candidate. + option_group* select_default_candidate(std::string candidate_name) { return do_select_candidate(candidate_name, true); } +}; + +/// @} + +} + +} diff --git a/src/seastar/include/seastar/util/read_first_line.hh b/src/seastar/include/seastar/util/read_first_line.hh new file mode 100644 index 000000000..bf77d9e39 --- /dev/null +++ b/src/seastar/include/seastar/util/read_first_line.hh @@ -0,0 +1,14 @@ +#include <seastar/util/std-compat.hh> +#include <seastar/core/sstring.hh> +#include <boost/lexical_cast.hpp> + +namespace seastar { + +sstring read_first_line(std::filesystem::path sys_file); + +template <typename Type> +Type read_first_line_as(std::filesystem::path sys_file) { + return boost::lexical_cast<Type>(read_first_line(sys_file)); +} + +} diff --git a/src/seastar/include/seastar/util/reference_wrapper.hh b/src/seastar/include/seastar/util/reference_wrapper.hh new file mode 100644 index 000000000..f9c540373 --- /dev/null +++ b/src/seastar/include/seastar/util/reference_wrapper.hh @@ -0,0 +1,74 @@ +/* + * 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) 2017 ScyllaDB Ltd. + */ + +#pragma once + +namespace seastar { + +/// \addtogroup utilities +/// @{ + +/// Wrapper for lvalue references +/// +/// reference_wrapper wraps a lvalue reference into a copyable and assignable +/// object. It is very similar to std::reference_wrapper except that it doesn't +/// allow implicit construction from a reference and the only way to construct +/// it is to use either ref() or cref(). The reason for that discrepancy (and +/// also the reason why seastar::reference_wrapper was introduced) is that it +/// server different purpose than std::reference_wrapper. The latter protects +/// references from decaying and allows copying and assigning them. +/// seastar::reference_wrapper is used mainly to force user to explicitly +/// state that object is passed by reference thus reducing the chances that +/// the referred object being prematurely destroyed in case the execution +/// is deferred to a continuation. +template<typename T> +class reference_wrapper { + T* _pointer; + + explicit reference_wrapper(T& object) noexcept : _pointer(&object) { } + + template<typename U> + friend reference_wrapper<U> ref(U&) noexcept; + template<typename U> + friend reference_wrapper<const U> cref(const U&) noexcept; +public: + using type = T; + + operator T&() const noexcept { return *_pointer; } + T& get() const noexcept { return *_pointer; } + +}; + +/// Wraps reference in a reference_wrapper +template<typename T> +inline reference_wrapper<T> ref(T& object) noexcept { + return reference_wrapper<T>(object); +} + +/// Wraps constant reference in a reference_wrapper +template<typename T> +inline reference_wrapper<const T> cref(const T& object) noexcept { + return reference_wrapper<const T>(object); +} + +/// @} + +} diff --git a/src/seastar/include/seastar/util/shared_token_bucket.hh b/src/seastar/include/seastar/util/shared_token_bucket.hh new file mode 100644 index 000000000..0ad7653bc --- /dev/null +++ b/src/seastar/include/seastar/util/shared_token_bucket.hh @@ -0,0 +1,216 @@ +/* + * 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) 2022 ScyllaDB + */ + +#pragma once + +#include <seastar/util/concepts.hh> +#include <atomic> +#include <chrono> +#include <cmath> + +namespace seastar { +namespace internal { + +static inline uint64_t wrapping_difference(const uint64_t& a, const uint64_t& b) noexcept { + return std::max<int64_t>(a - b, 0); +} + +static inline uint64_t fetch_add(std::atomic<uint64_t>& a, uint64_t b) noexcept { + return a.fetch_add(b); +} + +SEASTAR_CONCEPT( +template <typename T> +concept supports_wrapping_arithmetics = requires (T a, std::atomic<T> atomic_a, T b) { + { fetch_add(atomic_a, b) } noexcept -> std::same_as<T>; + { wrapping_difference(a, b) } noexcept -> std::same_as<T>; + { a + b } noexcept -> std::same_as<T>; +}; +) + +enum class capped_release { yes, no }; + +template <typename T, capped_release Capped> +struct rovers; + +template <typename T> +struct rovers<T, capped_release::yes> { + using atomic_rover = std::atomic<T>; + + atomic_rover tail; + atomic_rover head; + atomic_rover ceil; + + rovers(T limit) noexcept : tail(0), head(0), ceil(limit) {} + + T max_extra(T) const noexcept { + return wrapping_difference(ceil.load(std::memory_order_relaxed), head.load(std::memory_order_relaxed)); + } + + void release(T tokens) { + fetch_add(ceil, tokens); + } +}; + +template <typename T> +struct rovers<T, capped_release::no> { + using atomic_rover = std::atomic<T>; + + atomic_rover tail; + atomic_rover head; + + rovers(T) noexcept : tail(0), head(0) {} + + T max_extra(T limit) const noexcept { + return wrapping_difference(tail.load(std::memory_order_relaxed) + limit, head.load(std::memory_order_relaxed)); + } + + void release(T) { + std::abort(); // FIXME shouldn't even be compiled + } +}; + +template <typename T, typename Period, capped_release Capped, typename Clock = std::chrono::steady_clock> +SEASTAR_CONCEPT( requires std::is_nothrow_copy_constructible_v<T> && supports_wrapping_arithmetics<T> ) +class shared_token_bucket { + using rate_resolution = std::chrono::duration<double, Period>; + + T _replenish_rate; + const T _replenish_limit; + const T _replenish_threshold; + std::atomic<typename Clock::time_point> _replenished; + + /* + * The token bucket is implemented as a pair of wrapping monotonic + * counters (called rovers) one chasing the other. Getting a token + * from the bucket is increasing the tail, replenishing a token back + * is increasing the head. If increased tail overruns the head then + * the bucket is empty and we have to wait. The shard that grabs tail + * earlier will be "woken up" earlier, so they form a queue. + * + * The top rover is needed to implement two buckets actually. The + * tokens are not just replenished by timer. They are replenished by + * timer from the second bucket. And the second bucket only get a + * token in it after the request that grabbed it from the first bucket + * completes and returns it back. + */ + + using rovers_t = rovers<T, Capped>; + static_assert(rovers_t::atomic_rover::is_always_lock_free); + rovers_t _rovers; + + T tail() const noexcept { return _rovers.tail.load(std::memory_order_relaxed); } + T head() const noexcept { return _rovers.head.load(std::memory_order_relaxed); } + + /* + * Need to make sure that the multiplication in accumulated_in() doesn't + * overflow. Not to introduce an extra branch there, define that the + * replenish period is not larger than this delta and limit the rate with + * the value that can overflow it. + * + * The additional /=2 in max_rate math is to make extra sure that the + * overflow doesn't break wrapping_difference sign tricks. + */ + static constexpr rate_resolution max_delta = std::chrono::duration_cast<rate_resolution>(std::chrono::hours(1)); +public: + static constexpr T max_rate = std::numeric_limits<T>::max() / 2 / max_delta.count(); + +private: + static constexpr T accumulated(T rate, rate_resolution delta) noexcept { + return std::round(rate * delta.count()); + } +#ifndef __clang__ + // std::round() is constexpr only since C++23 (but g++ doesn't care) + static_assert(accumulated(max_rate, max_delta) <= std::numeric_limits<T>::max()); +#endif + +public: + shared_token_bucket(T rate, T limit, T threshold, bool add_replenish_iffset = true) noexcept + : _replenish_rate(std::min(rate, max_rate)) + , _replenish_limit(limit) + , _replenish_threshold(std::clamp(threshold, (T)1, limit)) + // pretend it was replenished yesterday to spot overflows early + , _replenished(Clock::now() - std::chrono::hours(add_replenish_iffset ? 24 : 0)) + , _rovers(_replenish_limit) + {} + + T grab(T tokens) noexcept { + return fetch_add(_rovers.tail, tokens) + tokens; + } + + void release(T tokens) noexcept { + _rovers.release(tokens); + } + + void replenish(typename Clock::time_point now) noexcept { + auto ts = _replenished.load(std::memory_order_relaxed); + + if (now <= ts) { + return; + } + + auto delta = now - ts; + auto extra = accumulated_in(delta); + + if (extra >= _replenish_threshold) { + if (!_replenished.compare_exchange_weak(ts, ts + delta)) { + return; // next time or another shard + } + + fetch_add(_rovers.head, std::min(extra, _rovers.max_extra(_replenish_limit))); + } + } + + T deficiency(T from) const noexcept { + return wrapping_difference(from, head()); + } + + template <typename Rep, typename Per> + static auto rate_cast(const std::chrono::duration<Rep, Per> delta) noexcept { + return std::chrono::duration_cast<rate_resolution>(delta); + } + + // the number of tokens accumulated for the given time frame + template <typename Rep, typename Per> + T accumulated_in(const std::chrono::duration<Rep, Per> delta) const noexcept { + auto delta_at_rate = std::min(rate_cast(delta), max_delta); + return accumulated(_replenish_rate, delta_at_rate); + } + + // Estimated time to process the given amount of tokens + // (peer of accumulated_in helper) + rate_resolution duration_for(T tokens) const noexcept { + return rate_resolution(tokens / _replenish_rate); + } + + T rate() const noexcept { return _replenish_rate; } + T limit() const noexcept { return _replenish_limit; } + T threshold() const noexcept { return _replenish_threshold; } + typename Clock::time_point replenished_ts() const noexcept { return _replenished; } + + void update_rate(T rate) noexcept { + _replenish_rate = std::min(rate, max_rate); + } +}; + +} // internal namespace +} // seastar namespace diff --git a/src/seastar/include/seastar/util/short_streams.hh b/src/seastar/include/seastar/util/short_streams.hh new file mode 100644 index 000000000..510cc2a45 --- /dev/null +++ b/src/seastar/include/seastar/util/short_streams.hh @@ -0,0 +1,51 @@ +/* + * 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) 2021 ScyllaDB + */ + +#include <seastar/core/future.hh> +#include <seastar/core/iostream.hh> +#include <seastar/core/temporary_buffer.hh> + +namespace seastar { + +namespace util { + +/// Returns all bytes from the stream until eof, accessible in chunks. +/// +/// \note use only on short streams to avoid running out of memory. +/// +/// \param inp \ref input_stream to be read. +future<std::vector<temporary_buffer<char>>> read_entire_stream(input_stream<char>& inp); + +/// Returns all bytes from the stream until eof as a single buffer. +/// +/// \note use only on short streams to avoid running out of memory. +/// +/// \param inp \ref input_stream to be read. +future<sstring> read_entire_stream_contiguous(input_stream<char>& inp); + +/// Ignores all bytes in the stream, until eos. +/// +/// \param inp \ref input_stream to be read. +future<> skip_entire_stream(input_stream<char>& inp); + +} // namespace util + +} // namespace seastar
\ No newline at end of file diff --git a/src/seastar/include/seastar/util/source_location-compat.hh b/src/seastar/include/seastar/util/source_location-compat.hh new file mode 100644 index 000000000..47a14fa5e --- /dev/null +++ b/src/seastar/include/seastar/util/source_location-compat.hh @@ -0,0 +1,57 @@ +/* + * 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) 2021 ScyllaDB + */ + +#pragma once + +namespace seastar::internal { + +class source_location { + const char* _file; + const char* _func; + int _line; + int _col; + + constexpr source_location(const char* file, const char* func, int line, int col) noexcept + : _file(file) + , _func(func) + , _line(line) + , _col(col) + { } +public: + constexpr source_location() noexcept + : _file("unknown") + , _func(_file) + , _line(0) + , _col(0) + { } + + static source_location current(const char* file = __builtin_FILE(), const char* func = __builtin_FUNCTION(), int line = __builtin_LINE(), int col = 0) noexcept { + return source_location(file, func, line, col); + } + + constexpr const char* file_name() const noexcept { return _file; } + constexpr const char* function_name() const noexcept { return _func; } + constexpr int line() const noexcept { return _line; } + constexpr int column() const noexcept { return _col; } +}; + +} // namespace seastar::internal diff --git a/src/seastar/include/seastar/util/spinlock.hh b/src/seastar/include/seastar/util/spinlock.hh new file mode 100644 index 000000000..12774e650 --- /dev/null +++ b/src/seastar/include/seastar/util/spinlock.hh @@ -0,0 +1,104 @@ +/* + * 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 2016 ScyllaDB + */ + +#pragma once + +#include <atomic> +#include <cassert> + +#if defined(__x86_64__) || defined(__i386__) +#include <xmmintrin.h> +#endif + +namespace seastar { + +namespace internal { +#if defined(__x86_64__) || defined(__i386__) + +/// \brief Puts the current CPU thread into a "relaxed" state. +/// +/// This function is supposed to significantly improve the performance in situations like spinlocks when process spins +/// in a tight loop waiting for a lock. The actual implementation is different on different platforms. For more details +/// look for "Pause Intrinsic" for x86 version, and for "yield" assembly instruction documentation for Power platform. +[[gnu::always_inline]] +inline void cpu_relax() { + _mm_pause(); +} + +#elif defined(__PPC__) + +[[gnu::always_inline]] +inline void cpu_relax() { + __asm__ volatile("yield"); +} + +#elif defined(__s390x__) || defined(__zarch__) + +// FIXME: there must be a better way +[[gnu::always_inline]] +inline void cpu_relax() {} + +#elif defined(__aarch64__) + +[[gnu::always_inline]] +inline void cpu_relax() { + __asm__ volatile("yield"); +} + +#else + +[[gnu::always_inline]] +inline void cpu_relax() {} +#warn "Using an empty cpu_relax() for this architecture" + +#endif + + +} + +namespace util { + +// Spin lock implementation. +// BasicLockable. +// Async-signal safe. +// unlock() "synchronizes with" lock(). +class spinlock { + std::atomic<bool> _busy = { false }; +public: + spinlock() = default; + spinlock(const spinlock&) = delete; + ~spinlock() { assert(!_busy.load(std::memory_order_relaxed)); } + bool try_lock() noexcept { + return !_busy.exchange(true, std::memory_order_acquire); + } + void lock() noexcept { + while (_busy.exchange(true, std::memory_order_acquire)) { + internal::cpu_relax(); + } + } + void unlock() noexcept { + _busy.store(false, std::memory_order_release); + } +}; + +} + +} diff --git a/src/seastar/include/seastar/util/std-compat.hh b/src/seastar/include/seastar/util/std-compat.hh new file mode 100644 index 000000000..e3a9cb73f --- /dev/null +++ b/src/seastar/include/seastar/util/std-compat.hh @@ -0,0 +1,78 @@ +/* + * 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 + +#include <optional> +#include <string_view> +#include <variant> + +#include <filesystem> + +#if __has_include(<memory_resource>) +#include <memory_resource> +#else +#include <experimental/memory_resource> +namespace std::pmr { + using namespace std::experimental::pmr; +} +#endif + +#if defined(__cpp_impl_coroutine) || defined(__cpp_coroutines) +#if __has_include(<coroutine>) +#define SEASTAR_COROUTINES_ENABLED +#else +#error Please use a C++ compiler with C++20 coroutines support +#endif +#endif + +// Defining SEASTAR_ASAN_ENABLED in here is a bit of a hack, but +// convenient since it is build system independent and in practice +// everything includes this header. + +#ifndef __has_feature +#define __has_feature(x) 0 +#endif + +// clang uses __has_feature, gcc defines __SANITIZE_ADDRESS__ +#if __has_feature(address_sanitizer) || defined(__SANITIZE_ADDRESS__) +#define SEASTAR_ASAN_ENABLED +#endif + +#if __has_include(<source_location>) +#include <source_location> +#endif + +#if defined(__cpp_lib_source_location) && !defined(SEASTAR_BROKEN_SOURCE_LOCATION) +namespace seastar::compat { +using source_location = std::source_location; +} +#elif __has_include(<experimental/source_location>) && !defined(SEASTAR_BROKEN_SOURCE_LOCATION) +#include <experimental/source_location> +namespace seastar::compat { +using source_location = std::experimental::source_location; +} +#else +#include <seastar/util/source_location-compat.hh> +namespace seastar::compat { +using source_location = seastar::internal::source_location; +} +#endif diff --git a/src/seastar/include/seastar/util/tmp_file.hh b/src/seastar/include/seastar/util/tmp_file.hh new file mode 100644 index 000000000..bfb1b23f7 --- /dev/null +++ b/src/seastar/include/seastar/util/tmp_file.hh @@ -0,0 +1,199 @@ +/* + * 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 2020 ScyllaDB + */ + +#pragma once + +#include <seastar/core/future.hh> +#include <seastar/core/file.hh> +#include <seastar/core/thread.hh> +#include <seastar/util/std-compat.hh> + +namespace seastar { + +const std::filesystem::path& default_tmpdir(); +void set_default_tmpdir(std::filesystem::path); + +class tmp_file { + std::filesystem::path _path; + file _file; + bool _is_open = false; + + static_assert(std::is_nothrow_constructible<std::filesystem::path>::value, + "filesystem::path's constructor must not throw"); + static_assert(std::is_nothrow_move_constructible<std::filesystem::path>::value, + "filesystem::path's move constructor must not throw"); +public: + tmp_file() noexcept = default; + tmp_file(const tmp_file&) = delete; + tmp_file(tmp_file&& x) noexcept; + + tmp_file& operator=(tmp_file&&) noexcept = default; + + ~tmp_file(); + + future<> open(std::filesystem::path path_template = default_tmpdir(), + open_flags oflags = open_flags::rw, + file_open_options options = {}) noexcept; + future<> close() noexcept; + future<> remove() noexcept; + + template <typename Func> + static future<> do_with(std::filesystem::path path_template, Func&& func, + open_flags oflags = open_flags::rw, + file_open_options options = {}) noexcept { + static_assert(std::is_nothrow_move_constructible<Func>::value, + "Func's move constructor must not throw"); + return seastar::do_with(tmp_file(), [func = std::move(func), path_template = std::move(path_template), oflags, options = std::move(options)] (tmp_file& t) mutable { + return t.open(std::move(path_template), oflags, std::move(options)).then([&t, func = std::move(func)] () mutable { + return func(t); + }).finally([&t] { + return t.close().finally([&t] { + return t.remove(); + }); + }); + }); + } + + template <typename Func> + static future<> do_with(Func&& func) noexcept { + return do_with(default_tmpdir(), std::move(func)); + } + + bool has_path() const { + return !_path.empty(); + } + + bool is_open() const { + return _is_open; + } + + const std::filesystem::path& get_path() const { + return _path; + } + + file& get_file() { + return _file; + } +}; + +/// Returns a future for an opened tmp_file exclusively created by the function. +/// +/// \param path_template - path where the file is to be created, +/// optionally including a template for the file name. +/// \param oflags - optional \ref open_flags (open_flags::create | open_flags::exclusive are added to those by default) +/// \param options - additional \ref file_open_options, e.g. for setting the created file permission. +/// +/// \note +/// path_template may optionally include a filename template in the last component of the path. +/// The template is indicated by two or more consecutive XX's. +/// Those will be replaced in the result path by a unique string. +/// +/// If no filename template is found, then path_template is assumed to refer to the directory where +/// the temporary file is to be created at (a.k.a. the parent directory) and `default_tmp_name_template` +/// is appended to the path as the filename template. +/// +/// The parent directory must exist and be writable to the current process. +/// +future<tmp_file> make_tmp_file(std::filesystem::path path_template = default_tmpdir(), + open_flags oflags = open_flags::rw, file_open_options options = {}) noexcept; + +class tmp_dir { + std::filesystem::path _path; + +public: + tmp_dir() = default; + tmp_dir(const tmp_dir&) = delete; + tmp_dir(tmp_dir&& x) = default; + + tmp_dir& operator=(tmp_dir&&) noexcept = default; + + ~tmp_dir(); + + future<> create(std::filesystem::path path_template = default_tmpdir(), + file_permissions create_permissions = file_permissions::default_dir_permissions) noexcept; + future<> remove() noexcept; + + template <typename Func> + SEASTAR_CONCEPT( requires std::is_nothrow_move_constructible_v<Func> ) + static future<> do_with(std::filesystem::path path_template, Func&& func, + file_permissions create_permissions = file_permissions::default_dir_permissions) noexcept { + static_assert(std::is_nothrow_move_constructible_v<Func>, + "Func's move constructor must not throw"); + return seastar::do_with(tmp_dir(), [func = std::move(func), path_template = std::move(path_template), create_permissions] (tmp_dir& t) mutable { + return t.create(std::move(path_template), create_permissions).then([&t, func = std::move(func)] () mutable { + return func(t); + }).finally([&t] { + return t.remove(); + }); + }); + } + + template <typename Func> + static future<> do_with(Func&& func) noexcept { + return do_with(default_tmpdir(), std::move(func)); + } + + template <typename Func> + + SEASTAR_CONCEPT( requires std::is_nothrow_move_constructible_v<Func> ) + static future<> do_with_thread(Func&& func) noexcept { + static_assert(std::is_nothrow_move_constructible_v<Func>, + "Func's move constructor must not throw"); + return async([func = std::move(func)] () mutable { + auto t = tmp_dir(); + t.create().get(); + futurize_invoke(func, t).finally([&t] { + return t.remove(); + }).get(); + }); + } + + bool has_path() const { + return !_path.empty(); + } + + const std::filesystem::path& get_path() const { + return _path; + } +}; + +/// Returns a future for a tmp_dir exclusively created by the function. +/// +/// \param path_template - path where the file is to be created, +/// optionally including a template for the file name. +/// \param create_permissions - optional permissions for the newly created directory. +/// +/// \note +/// path_template may optionally include a name template in the last component of the path. +/// The template is indicated by two or more consecutive XX's. +/// Those will be replaced in the result path by a unique string. +/// +/// If no name template is found, then path_template is assumed to refer to the directory where +/// the temporary dir is to be created at (a.k.a. the parent directory) and `default_tmp_name_template` +/// is appended to the path as the name template for the to-be-created directory. +/// +/// The parent directory must exist and be writable to the current process. +/// +future<tmp_dir> make_tmp_dir(std::filesystem::path path_template = default_tmpdir(), + file_permissions create_permissions = file_permissions::default_dir_permissions) noexcept; + +} // namespace seastar diff --git a/src/seastar/include/seastar/util/transform_iterator.hh b/src/seastar/include/seastar/util/transform_iterator.hh new file mode 100644 index 000000000..95ea1ce51 --- /dev/null +++ b/src/seastar/include/seastar/util/transform_iterator.hh @@ -0,0 +1,57 @@ +/* + * 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) 2014 Cloudius Systems, Ltd. + */ + +#pragma once + +namespace seastar { + +template <typename Iterator, typename Func> +class transform_iterator { + Iterator _i; + Func _f; +public: + transform_iterator(Iterator i, Func f) : _i(i), _f(f) {} + auto operator*() { return _f(*_i); } + transform_iterator& operator++() { + ++_i; + return *this; + } + transform_iterator operator++(int) { + transform_iterator ret(*this); + _i++; + return ret; + } + bool operator==(const transform_iterator& x) const { + return _i == x._i; + } + bool operator!=(const transform_iterator& x) const { + return !operator==(x); + } +}; + +template <typename Iterator, typename Func> +inline +transform_iterator<Iterator, Func> +make_transform_iterator(Iterator i, Func f) { + return transform_iterator<Iterator, Func>(i, f); +} + +} diff --git a/src/seastar/include/seastar/util/tuple_utils.hh b/src/seastar/include/seastar/util/tuple_utils.hh new file mode 100644 index 000000000..477f339f9 --- /dev/null +++ b/src/seastar/include/seastar/util/tuple_utils.hh @@ -0,0 +1,174 @@ +/* + * 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) 2017 ScyllaDB. + */ + +#pragma once + +#include <tuple> +#include <utility> +#include <stddef.h> + +namespace seastar { + +/// \cond internal +namespace internal { + +template<typename Tuple> +Tuple untuple(Tuple t) { + return t; +} + +template<typename T> +T untuple(std::tuple<T> t) { + return std::get<0>(std::move(t)); +} + +template<typename Tuple, typename Function, size_t... I> +void tuple_for_each_helper(Tuple&& t, Function&& f, std::index_sequence<I...>&&) { + auto ignore_me = { (f(std::get<I>(std::forward<Tuple>(t))), 1)... }; + (void)ignore_me; +} + +template<typename Tuple, typename MapFunction, size_t... I> +auto tuple_map_helper(Tuple&& t, MapFunction&& f, std::index_sequence<I...>&&) { + return std::make_tuple(f(std::get<I>(std::forward<Tuple>(t)))...); +} + +template<size_t I, typename IndexSequence> +struct prepend; + +template<size_t I, size_t... Is> +struct prepend<I, std::index_sequence<Is...>> { + using type = std::index_sequence<I, Is...>; +}; + +template<template<typename> class Filter, typename Tuple, typename IndexSequence> +struct tuple_filter; + +template<template<typename> class Filter, typename T, typename... Ts, size_t I, size_t... Is> +struct tuple_filter<Filter, std::tuple<T, Ts...>, std::index_sequence<I, Is...>> { + using tail = typename tuple_filter<Filter, std::tuple<Ts...>, std::index_sequence<Is...>>::type; + using type = std::conditional_t<Filter<T>::value, typename prepend<I, tail>::type, tail>; +}; + +template<template<typename> class Filter> +struct tuple_filter<Filter, std::tuple<>, std::index_sequence<>> { + using type = std::index_sequence<>; +}; + +template<typename Tuple, size_t... I> +auto tuple_filter_helper(Tuple&& t, std::index_sequence<I...>&&) { + return std::make_tuple(std::get<I>(std::forward<Tuple>(t))...); +} + +} +/// \endcond + +/// \addtogroup utilities +/// @{ + +/// Applies type transformation to all types in tuple +/// +/// Member type `type` is set to a tuple type which is a result of applying +/// transformation `MapClass<T>::type` to each element `T` of the input tuple +/// type. +/// +/// \tparam MapClass class template defining type transformation +/// \tparam Tuple input tuple type +template<template<typename> class MapClass, typename Tuple> +struct tuple_map_types; + +/// @} + +template<template<typename> class MapClass, typename... Elements> +struct tuple_map_types<MapClass, std::tuple<Elements...>> { + using type = std::tuple<typename MapClass<Elements>::type...>; +}; + +/// \addtogroup utilities +/// @{ + +/// Filters elements in tuple by their type +/// +/// Returns a tuple containing only those elements which type `T` caused +/// expression FilterClass<T>::value to be true. +/// +/// \tparam FilterClass class template having an element value set to true for elements that +/// should be present in the result +/// \param t tuple to filter +/// \return a tuple contaning elements which type passed the test +template<template<typename> class FilterClass, typename... Elements> +auto tuple_filter_by_type(const std::tuple<Elements...>& t) { + using sequence = typename internal::tuple_filter<FilterClass, std::tuple<Elements...>, + std::index_sequence_for<Elements...>>::type; + return internal::tuple_filter_helper(t, sequence()); +} +template<template<typename> class FilterClass, typename... Elements> +auto tuple_filter_by_type(std::tuple<Elements...>&& t) { + using sequence = typename internal::tuple_filter<FilterClass, std::tuple<Elements...>, + std::index_sequence_for<Elements...>>::type; + return internal::tuple_filter_helper(std::move(t), sequence()); +} + +/// Applies function to all elements in tuple +/// +/// Applies given function to all elements in the tuple and returns a tuple +/// of results. +/// +/// \param t original tuple +/// \param f function to apply +/// \return tuple of results returned by f for each element in t +template<typename Function, typename... Elements> +auto tuple_map(const std::tuple<Elements...>& t, Function&& f) { + return internal::tuple_map_helper(t, std::forward<Function>(f), + std::index_sequence_for<Elements...>()); +} +template<typename Function, typename... Elements> +auto tuple_map(std::tuple<Elements...>&& t, Function&& f) { + return internal::tuple_map_helper(std::move(t), std::forward<Function>(f), + std::index_sequence_for<Elements...>()); +} + +/// Iterate over all elements in tuple +/// +/// Iterates over given tuple and calls the specified function for each of +/// it elements. +/// +/// \param t a tuple to iterate over +/// \param f function to call for each tuple element +template<typename Function, typename... Elements> +void tuple_for_each(const std::tuple<Elements...>& t, Function&& f) { + return internal::tuple_for_each_helper(t, std::forward<Function>(f), + std::index_sequence_for<Elements...>()); +} +template<typename Function, typename... Elements> +void tuple_for_each(std::tuple<Elements...>& t, Function&& f) { + return internal::tuple_for_each_helper(t, std::forward<Function>(f), + std::index_sequence_for<Elements...>()); +} +template<typename Function, typename... Elements> +void tuple_for_each(std::tuple<Elements...>&& t, Function&& f) { + return internal::tuple_for_each_helper(std::move(t), std::forward<Function>(f), + std::index_sequence_for<Elements...>()); +} + +/// @} + +} diff --git a/src/seastar/include/seastar/util/used_size.hh b/src/seastar/include/seastar/util/used_size.hh new file mode 100644 index 000000000..7b4212d7e --- /dev/null +++ b/src/seastar/include/seastar/util/used_size.hh @@ -0,0 +1,36 @@ +/* + * 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) 2020 ScyllaDB Ltd. + */ + +#pragma once + +#include <stddef.h> +#include <type_traits> + +namespace seastar { +namespace internal { +// Empty types have a size of 1, but that byte is not actually +// used. This helper is used to avoid accessing that byte. +template<typename T> +struct used_size { + static constexpr size_t value = std::is_empty<T>::value ? 0 : sizeof(T); +}; +} +} diff --git a/src/seastar/include/seastar/util/variant_utils.hh b/src/seastar/include/seastar/util/variant_utils.hh new file mode 100644 index 000000000..ddb2192d2 --- /dev/null +++ b/src/seastar/include/seastar/util/variant_utils.hh @@ -0,0 +1,102 @@ +/* + * 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) 2017 ScyllaDB. + */ + +#pragma once + +#include <seastar/util/std-compat.hh> + +namespace seastar { + +/// \cond internal +namespace internal { + +template<typename... Args> +struct variant_visitor : Args... { + variant_visitor(Args&&... a) : Args(std::move(a))... {} + using Args::operator()...; +}; + +template<typename... Args> variant_visitor(Args&&...) -> variant_visitor<Args...>; + +} +/// \endcond + +/// \addtogroup utilities +/// @{ + +/// Creates a visitor from function objects. +/// +/// Returns a visitor object comprised of the provided function objects. Can be +/// used with std::variant or any other custom variant implementation. +/// +/// \param args function objects each accepting one or some types stored in the variant as input +template <typename... Args> +auto make_visitor(Args&&... args) +{ + return internal::variant_visitor<Args...>(std::forward<Args>(args)...); +} + +/// Applies a static visitor comprised of supplied lambdas to a variant. +/// Note that the lambdas should cover all the types that the variant can possibly hold. +/// +/// Returns the common type of return types of all lambdas. +/// +/// \tparam Variant the type of a variant +/// \tparam Args types of lambda objects +/// \param variant the variant object +/// \param args lambda objects each accepting one or some types stored in the variant as input +/// \return +template <typename Variant, typename... Args> +inline auto visit(Variant&& variant, Args&&... args) +{ + static_assert(sizeof...(Args) > 0, "At least one lambda must be provided for visitation"); + return std::visit( + make_visitor(std::forward<Args>(args)...), + variant); +} + +namespace internal { +template<typename... Args> +struct castable_variant { + std::variant<Args...> var; + + template<typename... SuperArgs> + operator std::variant<SuperArgs...>() && { + return std::visit([] (auto&& x) { + return std::variant<SuperArgs...>(std::move(x)); + }, var); + } +}; +} + +template<typename... Args> +internal::castable_variant<Args...> variant_cast(std::variant<Args...>&& var) { + return {std::move(var)}; +} + +template<typename... Args> +internal::castable_variant<Args...> variant_cast(const std::variant<Args...>& var) { + return {var}; +} + +/// @} + +} diff --git a/src/seastar/include/seastar/websocket/server.hh b/src/seastar/include/seastar/websocket/server.hh new file mode 100644 index 000000000..e627d0848 --- /dev/null +++ b/src/seastar/include/seastar/websocket/server.hh @@ -0,0 +1,337 @@ +/* + * 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 2021 ScyllaDB + */ + +#pragma once + +#include <map> +#include <functional> + +#include <seastar/http/request_parser.hh> +#include <seastar/core/seastar.hh> +#include <seastar/core/sstring.hh> +#include <seastar/net/api.hh> +#include <seastar/core/gate.hh> +#include <seastar/core/queue.hh> +#include <seastar/core/when_all.hh> + +namespace seastar::experimental::websocket { + +using handler_t = std::function<future<>(input_stream<char>&, output_stream<char>&)>; + +class server; + +/// \defgroup websocket WebSocket +/// \addtogroup websocket +/// @{ + +/*! + * \brief an error in handling a WebSocket connection + */ +class exception : public std::exception { + std::string _msg; +public: + exception(std::string_view msg) : _msg(msg) {} + virtual const char* what() const noexcept { + return _msg.c_str(); + } +}; + +/*! + * \brief Possible type of a websocket frame. + */ +enum opcodes { + CONTINUATION = 0x0, + TEXT = 0x1, + BINARY = 0x2, + CLOSE = 0x8, + PING = 0x9, + PONG = 0xA, + INVALID = 0xFF, +}; + +struct frame_header { + static constexpr uint8_t FIN = 7; + static constexpr uint8_t RSV1 = 6; + static constexpr uint8_t RSV2 = 5; + static constexpr uint8_t RSV3 = 4; + static constexpr uint8_t MASKED = 7; + + uint8_t fin : 1; + uint8_t rsv1 : 1; + uint8_t rsv2 : 1; + uint8_t rsv3 : 1; + uint8_t opcode : 4; + uint8_t masked : 1; + uint8_t length : 7; + frame_header(const char* input) { + this->fin = (input[0] >> FIN) & 1; + this->rsv1 = (input[0] >> RSV1) & 1; + this->rsv2 = (input[0] >> RSV2) & 1; + this->rsv3 = (input[0] >> RSV3) & 1; + this->opcode = input[0] & 0b1111; + this->masked = (input[1] >> MASKED) & 1; + this->length = (input[1] & 0b1111111); + } + // Returns length of the rest of the header. + uint64_t get_rest_of_header_length() { + size_t next_read_length = sizeof(uint32_t); // Masking key + if (length == 126) { + next_read_length += sizeof(uint16_t); + } else if (length == 127) { + next_read_length += sizeof(uint64_t); + } + return next_read_length; + } + uint8_t get_fin() {return fin;} + uint8_t get_rsv1() {return rsv1;} + uint8_t get_rsv2() {return rsv2;} + uint8_t get_rsv3() {return rsv3;} + uint8_t get_opcode() {return opcode;} + uint8_t get_masked() {return masked;} + uint8_t get_length() {return length;} + + bool is_opcode_known() { + //https://datatracker.ietf.org/doc/html/rfc6455#section-5.1 + return opcode < 0xA && !(opcode < 0x8 && opcode > 0x2); + } +}; + +class websocket_parser { + enum class parsing_state : uint8_t { + flags_and_payload_data, + payload_length_and_mask, + payload + }; + enum class connection_state : uint8_t { + valid, + closed, + error + }; + using consumption_result_t = consumption_result<char>; + using buff_t = temporary_buffer<char>; + // What parser is currently doing. + parsing_state _state; + // State of connection - can be valid, closed or should be closed + // due to error. + connection_state _cstate; + sstring _buffer; + std::unique_ptr<frame_header> _header; + uint64_t _payload_length; + uint32_t _masking_key; + buff_t _result; + + static future<consumption_result_t> dont_stop() { + return make_ready_future<consumption_result_t>(continue_consuming{}); + } + static future<consumption_result_t> stop(buff_t data) { + return make_ready_future<consumption_result_t>(stop_consuming(std::move(data))); + } + + // Removes mask from payload given in p. + void remove_mask(buff_t& p, size_t n) { + char *payload = p.get_write(); + for (uint64_t i = 0, j = 0; i < n; ++i, j = (j + 1) % 4) { + payload[i] ^= static_cast<char>(((_masking_key << (j * 8)) >> 24)); + } + } +public: + websocket_parser() : _state(parsing_state::flags_and_payload_data), + _cstate(connection_state::valid), + _payload_length(0), + _masking_key(0) {} + future<consumption_result_t> operator()(temporary_buffer<char> data); + bool is_valid() { return _cstate == connection_state::valid; } + bool eof() { return _cstate == connection_state::closed; } + opcodes opcode() const; + buff_t result(); +}; + +/*! + * \brief a WebSocket connection + */ +class connection : public boost::intrusive::list_base_hook<> { + using buff_t = temporary_buffer<char>; + + /*! + * \brief Implementation of connection's data source. + */ + class connection_source_impl final : public data_source_impl { + queue<buff_t>* data; + + public: + connection_source_impl(queue<buff_t>* data) : data(data) {} + + virtual future<buff_t> get() override { + return data->pop_eventually().then_wrapped([](future<buff_t> f){ + try { + return make_ready_future<buff_t>(std::move(f.get())); + } catch(...) { + return current_exception_as_future<buff_t>(); + } + }); + } + + virtual future<> close() override { + data->push(buff_t(0)); + return make_ready_future<>(); + } + }; + + /*! + * \brief Implementation of connection's data sink. + */ + class connection_sink_impl final : public data_sink_impl { + queue<buff_t>* data; + public: + connection_sink_impl(queue<buff_t>* data) : data(data) {} + + virtual future<> put(net::packet d) override { + net::fragment f = d.frag(0); + return data->push_eventually(temporary_buffer<char>{std::move(f.base), f.size}); + } + + size_t buffer_size() const noexcept override { + return data->max_size(); + } + + virtual future<> close() override { + data->push(buff_t(0)); + return make_ready_future<>(); + } + }; + + future<> close(bool send_close); + + /*! + * \brief This function processess received PING frame. + * https://datatracker.ietf.org/doc/html/rfc6455#section-5.5.2 + */ + future<> handle_ping(); + /*! + * \brief This function processess received PONG frame. + * https://datatracker.ietf.org/doc/html/rfc6455#section-5.5.3 + */ + future<> handle_pong(); + + static const size_t PIPE_SIZE = 512; + server& _server; + connected_socket _fd; + input_stream<char> _read_buf; + output_stream<char> _write_buf; + http_request_parser _http_parser; + bool _done = false; + + websocket_parser _websocket_parser; + queue <temporary_buffer<char>> _input_buffer; + input_stream<char> _input; + queue <temporary_buffer<char>> _output_buffer; + output_stream<char> _output; + + sstring _subprotocol; + handler_t _handler; +public: + /*! + * \param server owning \ref server + * \param fd established socket used for communication + */ + connection(server& server, connected_socket&& fd) + : _server(server) + , _fd(std::move(fd)) + , _read_buf(_fd.input()) + , _write_buf(_fd.output()) + , _input_buffer{PIPE_SIZE} + , _output_buffer{PIPE_SIZE} + { + _input = input_stream<char>{data_source{ + std::make_unique<connection_source_impl>(&_input_buffer)}}; + _output = output_stream<char>{data_sink{ + std::make_unique<connection_sink_impl>(&_output_buffer)}}; + on_new_connection(); + } + ~connection(); + + /*! + * \brief serve WebSocket protocol on a connection + */ + future<> process(); + /*! + * \brief close the socket + */ + void shutdown(); + + future<> close(); + +protected: + future<> read_loop(); + future<> read_one(); + future<> read_http_upgrade_request(); + future<> response_loop(); + void on_new_connection(); + /*! + * \brief Packs buff in websocket frame and sends it to the client. + */ + future<> send_data(opcodes opcode, temporary_buffer<char>&& buff); + +}; + +/*! + * \brief a WebSocket server + * + * A server capable of establishing and serving connections + * over WebSocket protocol. + */ +class server { + std::vector<server_socket> _listeners; + boost::intrusive::list<connection> _connections; + std::map<std::string, handler_t> _handlers; + future<> _accept_fut = make_ready_future<>(); + bool _stopped = false; +public: + /*! + * \brief listen for a WebSocket connection on given address + * \param addr address to listen on + */ + void listen(socket_address addr); + /*! + * \brief listen for a WebSocket connection on given address with custom options + * \param addr address to listen on + * \param lo custom listen options (\ref listen_options) + */ + void listen(socket_address addr, listen_options lo); + + /*! + * Stops the server and shuts down all active connections + */ + future<> stop(); + + bool is_handler_registered(std::string const& name); + + void register_handler(std::string&& name, handler_t handler); + + friend class connection; +protected: + void do_accepts(int which); + future<> do_accept_one(int which); +}; + +/// }@ + +} |