From 7a91062375c9b13456713708f10365a118a0d999 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Mon, 17 Feb 2025 16:36:21 +0000 Subject: [PATCH] treewide: replace assert with SEASTAR_ASSERT This prevents the need to undefined NDEBUG in seastar source and prevents headers from conditionally excluding these checks. --- CMakeLists.txt | 6 --- apps/io_tester/io_tester.cc | 3 +- apps/memcached/memcache.cc | 11 ++--- apps/rpc_tester/rpc_tester.cc | 3 +- demos/block_discard_demo.cc | 3 +- demos/file_demo.cc | 15 ++++--- demos/sharded_parameter_demo.cc | 4 +- demos/udp_zero_copy_demo.cc | 5 ++- include/seastar/core/abort_source.hh | 3 +- include/seastar/core/abortable_fifo.hh | 5 ++- include/seastar/core/chunked_fifo.hh | 3 +- include/seastar/core/expiring_fifo.hh | 3 +- include/seastar/core/fair_queue.hh | 3 +- include/seastar/core/future.hh | 35 +++++++-------- include/seastar/core/gate.hh | 11 ++--- include/seastar/core/internal/io_intent.hh | 3 +- include/seastar/core/internal/io_request.hh | 4 +- include/seastar/core/iostream-impl.hh | 9 ++-- include/seastar/core/iostream.hh | 9 ++-- include/seastar/core/loop.hh | 3 +- include/seastar/core/on_internal_error.hh | 4 +- include/seastar/core/posix.hh | 5 ++- include/seastar/core/queue.hh | 3 +- include/seastar/core/ragel.hh | 3 +- include/seastar/core/rwlock.hh | 5 ++- include/seastar/core/scheduling_specific.hh | 5 ++- include/seastar/core/semaphore.hh | 14 +++--- include/seastar/core/sharded.hh | 11 ++--- include/seastar/core/shared_mutex.hh | 6 +-- include/seastar/core/slab.hh | 23 +++++----- include/seastar/core/sstring.hh | 10 ++--- include/seastar/core/thread.hh | 3 +- include/seastar/core/thread_cputime_clock.hh | 4 +- include/seastar/core/timer-set.hh | 3 +- include/seastar/core/timer.hh | 3 +- include/seastar/coroutine/generator.hh | 36 +++++++-------- include/seastar/net/ethernet.hh | 4 +- include/seastar/net/net.hh | 3 +- include/seastar/net/packet.hh | 12 ++--- include/seastar/net/tcp.hh | 7 +-- include/seastar/rpc/rpc_impl.hh | 3 +- include/seastar/rpc/rpc_types.hh | 3 +- include/seastar/util/assert.hh | 36 +++++++++++++++ include/seastar/util/backtrace.hh | 3 +- include/seastar/util/closeable.hh | 5 ++- include/seastar/util/print_safe.hh | 6 +-- include/seastar/util/spinlock.hh | 4 +- src/core/condition-variable.cc | 4 +- src/core/exception_hacks.cc | 4 +- src/core/fair_queue.cc | 25 ++++++----- src/core/file.cc | 3 +- src/core/fsqual.cc | 7 +-- src/core/fstream.cc | 8 ++-- src/core/future.cc | 14 +++--- src/core/io_queue.cc | 6 +-- src/core/memory.cc | 22 +++++----- src/core/metrics.cc | 1 + src/core/posix.cc | 8 ++-- src/core/prometheus.cc | 3 +- src/core/reactor.cc | 46 ++++++++++---------- src/core/reactor_backend.cc | 22 +++++----- src/core/resource.cc | 17 ++++---- src/core/scollectd.cc | 4 +- src/core/systemwide_memory_barrier.cc | 8 ++-- src/core/thread.cc | 3 +- src/core/thread_pool.cc | 6 +-- src/core/vla.hh | 4 +- src/http/request.cc | 4 +- src/net/dns.cc | 7 +-- src/net/dpdk.cc | 43 +++++++++--------- src/net/native-stack-impl.hh | 5 ++- src/net/native-stack.cc | 7 +-- src/net/net.cc | 7 +-- src/net/posix-stack.cc | 12 ++--- src/net/tcp.cc | 4 +- src/net/tls.cc | 12 ++--- src/net/virtio.cc | 18 ++++---- src/rpc/lz4_fragmented_compressor.cc | 3 +- src/rpc/rpc.cc | 11 ++--- src/testing/test_runner.cc | 3 +- src/util/file.cc | 1 + src/util/log.cc | 5 +++ src/util/process.cc | 6 +-- src/util/tmp_file.cc | 15 ++++--- src/websocket/common.cc | 3 +- src/websocket/parser.cc | 3 +- tests/perf/linux_perf_event.cc | 5 +-- tests/perf/shared_token_bucket.cc | 7 +-- tests/unit/allocator_test.cc | 6 +-- tests/unit/circular_buffer_test.cc | 5 ++- tests/unit/directory_test.cc | 11 ++--- tests/unit/distributed_test.cc | 7 +-- tests/unit/fair_queue_test.cc | 3 +- tests/unit/file_io_test.cc | 7 +-- tests/unit/fstream_test.cc | 5 ++- tests/unit/futures_test.cc | 5 ++- tests/unit/httpd_test.cc | 3 +- tests/unit/io_queue_test.cc | 7 +-- tests/unit/loopback_socket.hh | 9 ++-- tests/unit/mock_file.hh | 3 +- tests/unit/rpc_test.cc | 3 +- tests/unit/scheduling_group_test.cc | 2 +- tests/unit/sharded_test.cc | 11 ++--- tests/unit/shared_ptr_test.cc | 4 +- tests/unit/slab_test.cc | 12 ++--- tests/unit/socket_test.cc | 3 +- tests/unit/thread_test.cc | 5 ++- tests/unit/tls_test.cc | 8 ++-- tests/unit/websocket_test.cc | 6 +-- 109 files changed, 495 insertions(+), 391 deletions(-) create mode 100644 include/seastar/util/assert.hh diff --git a/CMakeLists.txt b/CMakeLists.txt index 1dc9a36c1d..00ffdba84c 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -802,7 +802,6 @@ target_include_directories (seastar set (Seastar_PRIVATE_CXX_FLAGS -fno-semantic-interposition - -UNDEBUG -Wall -Werror -Wimplicit-fallthrough @@ -904,11 +903,6 @@ endif () set( MEMORYCHECK_COMMAND_OPTIONS "--error-exitcode=1 --leak-check=no --trace-children=yes" ) include (CTest) -# -# We want asserts enabled on all modes, but cmake defaults to passing -# -DNDEBUG in some modes. We add -UNDEBUG to our private options to -# reenable it. To force asserts off pass -DNDEBUG in -# Seastar_CXX_FLAGS. # # To disable -Werror, pass -Wno-error to Seastar_CXX_FLAGS. # diff --git a/apps/io_tester/io_tester.cc b/apps/io_tester/io_tester.cc index 628ef7c5c1..910e0e25e2 100644 --- a/apps/io_tester/io_tester.cc +++ b/apps/io_tester/io_tester.cc @@ -33,6 +33,7 @@ #include #include #include +#include #include #include #include @@ -640,7 +641,7 @@ class io_class_data : public class_data { void emit_one_metrics(YAML::Emitter& out, sstring m_name) { const auto& values = seastar::metrics::impl::get_value_map(); const auto& mf = values.find(m_name); - assert(mf != values.end()); + SEASTAR_ASSERT(mf != values.end()); for (auto&& mi : mf->second) { auto&& cname = mi.first.labels().find("class"); if (cname != mi.first.labels().end() && cname->second == name()) { diff --git a/apps/memcached/memcache.cc b/apps/memcached/memcache.cc index 90307081c5..5d302fb775 100644 --- a/apps/memcached/memcache.cc +++ b/apps/memcached/memcache.cc @@ -43,6 +43,7 @@ #include #include #include +#include #include #include #include "ascii.hh" @@ -162,8 +163,8 @@ class item : public slab_item_base { , _key_size(key.key().size()) , _ascii_prefix_size(ascii_prefix.size()) { - assert(_key_size <= std::numeric_limits::max()); - assert(_ascii_prefix_size <= std::numeric_limits::max()); + SEASTAR_ASSERT(_key_size <= std::numeric_limits::max()); + SEASTAR_ASSERT(_ascii_prefix_size <= std::numeric_limits::max()); // storing key memcpy(_data, key.key().c_str(), _key_size); // storing ascii_prefix @@ -255,7 +256,7 @@ class item : public slab_item_base { } friend inline void intrusive_ptr_add_ref(item* it) { - assert(it->_ref_count >= 0); + SEASTAR_ASSERT(it->_ref_count >= 0); ++it->_ref_count; if (it->_ref_count == 2) { slab->lock_item(it); @@ -269,7 +270,7 @@ class item : public slab_item_base { } else if (it->_ref_count == 0) { slab->free(it); } - assert(it->_ref_count >= 0); + SEASTAR_ASSERT(it->_ref_count >= 0); } friend struct item_key_cmp; @@ -469,7 +470,7 @@ class cache { intrusive_ptr_add_ref(new_item); auto insert_result = _cache.insert(*new_item); - assert(insert_result.second); + SEASTAR_ASSERT(insert_result.second); if (insertion.expiry.ever_expires() && _alive.insert(*new_item)) { _timer.rearm(new_item->get_timeout()); } diff --git a/apps/rpc_tester/rpc_tester.cc b/apps/rpc_tester/rpc_tester.cc index 99b6bd8daa..3965cd3892 100644 --- a/apps/rpc_tester/rpc_tester.cc +++ b/apps/rpc_tester/rpc_tester.cc @@ -43,6 +43,7 @@ #include #include #include +#include using namespace seastar; using namespace boost::accumulators; @@ -388,7 +389,7 @@ class job_rpc : public job { future<> call_write(unsigned dummy, const payload_t& pl) { return _rpc.make_client(rpc_verb::WRITE)(*_client, pl).then([exp = pl.size()] (auto res) { - assert(res == exp); + SEASTAR_ASSERT(res == exp); return make_ready_future<>(); }); } diff --git a/demos/block_discard_demo.cc b/demos/block_discard_demo.cc index b14c5ed017..0b7089453c 100644 --- a/demos/block_discard_demo.cc +++ b/demos/block_discard_demo.cc @@ -25,6 +25,7 @@ #include #include #include +#include #include using namespace seastar; @@ -53,7 +54,7 @@ int main(int ac, char** av) { // Discard asynchronously, siganl when done. (void)ft->f.stat().then([ft] (struct stat st) mutable { - assert(S_ISBLK(st.st_mode)); + SEASTAR_ASSERT(S_ISBLK(st.st_mode)); auto offset = 0; auto length = max * 4096; return ft->f.discard(offset, length).then([ft] () mutable { diff --git a/demos/file_demo.cc b/demos/file_demo.cc index 1ce9a42475..76f5dd5727 100644 --- a/demos/file_demo.cc +++ b/demos/file_demo.cc @@ -36,6 +36,7 @@ #include #include #include +#include #include #include @@ -45,9 +46,9 @@ constexpr size_t aligned_size = 4096; future<> verify_data_file(file& f, temporary_buffer& rbuf, const temporary_buffer& wbuf) { return f.dma_read(0, rbuf.get_write(), aligned_size).then([&rbuf, &wbuf] (size_t count) { - assert(count == aligned_size); + SEASTAR_ASSERT(count == aligned_size); fmt::print(" verifying {} bytes\n", count); - assert(!memcmp(rbuf.get(), wbuf.get(), aligned_size)); + SEASTAR_ASSERT(!memcmp(rbuf.get(), wbuf.get(), aligned_size)); }); } @@ -55,7 +56,7 @@ future open_data_file(sstring meta_filename, temporary_buffer& rbuf) fmt::print(" retrieving data filename from {}\n", meta_filename); return with_file(open_file_dma(meta_filename, open_flags::ro), [&rbuf] (file& f) { return f.dma_read(0, rbuf.get_write(), aligned_size).then([&rbuf] (size_t count) { - assert(count == aligned_size); + SEASTAR_ASSERT(count == aligned_size); auto data_filename = sstring(rbuf.get()); fmt::print(" opening {}\n", data_filename); return open_file_dma(data_filename, open_flags::ro); @@ -77,7 +78,7 @@ future<> demo_with_file() { auto count = with_file(open_file_dma(filename, open_flags::rw | open_flags::create), [&wbuf] (file& f) { return f.dma_write(0, wbuf.get(), aligned_size); }).get(); - assert(count == aligned_size); + SEASTAR_ASSERT(count == aligned_size); }; // print the data_filename into the write buffer @@ -213,14 +214,14 @@ future<> demo_with_io_intent() { f.dma_read(0, rbuf.get(), aligned_size).get(); // First part of the buffer must coincide with the overwritten data - assert(!memcmp(rbuf.get(), wbuf_n.get(), half_aligned_size)); + SEASTAR_ASSERT(!memcmp(rbuf.get(), wbuf_n.get(), half_aligned_size)); if (cancelled) { // Second part -- with the old data ... - assert(!memcmp(rbuf.get() + half_aligned_size, wbuf.get() + half_aligned_size, half_aligned_size)); + SEASTAR_ASSERT(!memcmp(rbuf.get() + half_aligned_size, wbuf.get() + half_aligned_size, half_aligned_size)); } else { // ... or with new if the cancellation didn't happen - assert(!memcmp(rbuf.get() + half_aligned_size, wbuf.get() + half_aligned_size, half_aligned_size)); + SEASTAR_ASSERT(!memcmp(rbuf.get() + half_aligned_size, wbuf.get() + half_aligned_size, half_aligned_size)); } }); } diff --git a/demos/sharded_parameter_demo.cc b/demos/sharded_parameter_demo.cc index 09b53a9aba..be4a85fdd5 100644 --- a/demos/sharded_parameter_demo.cc +++ b/demos/sharded_parameter_demo.cc @@ -25,8 +25,8 @@ #include #include #include +#include #include -#include // This is some service that we wish to run on all shards. class service_one { @@ -71,7 +71,7 @@ int main(int ac, char** av) { auto stop_s2 = seastar::deferred_stop(s2); s2.invoke_on_all([] (service_two& s2) { - assert(s2.get_resource_allocation() == 3); + SEASTAR_ASSERT(s2.get_resource_allocation() == 3); }).get(); }); }); diff --git a/demos/udp_zero_copy_demo.cc b/demos/udp_zero_copy_demo.cc index e976ac3ec9..1c64aba86d 100644 --- a/demos/udp_zero_copy_demo.cc +++ b/demos/udp_zero_copy_demo.cc @@ -27,6 +27,7 @@ #include #include #include +#include #include #include #include @@ -101,7 +102,7 @@ class server { _chunk_distribution = std::uniform_int_distribution(0, _mem_size - _chunk_size * 3); - assert(3 * _chunk_size <= _packet_size); + SEASTAR_ASSERT(3 * _chunk_size <= _packet_size); // Run sender in background. (void)keep_doing([this] { @@ -117,7 +118,7 @@ class server { chunk += _chunk_size; (void)_out->write(chunk, _chunk_size); (void)_out->flush(); - assert(_packets.size() == 1); + SEASTAR_ASSERT(_packets.size() == 1); return send(dgram.get_src(), std::move(_packets[0])); } else { auto chunk = next_chunk(); diff --git a/include/seastar/core/abort_source.hh b/include/seastar/core/abort_source.hh index 8781d47cc5..418db0fbd2 100644 --- a/include/seastar/core/abort_source.hh +++ b/include/seastar/core/abort_source.hh @@ -21,6 +21,7 @@ #pragma once +#include #include #include #include @@ -132,7 +133,7 @@ private: return; } _ex = ex.value_or(get_default_exception()); - assert(_ex); + SEASTAR_ASSERT(_ex); auto subs = std::move(_subscriptions); while (!subs.empty()) { subscription& s = subs.front(); diff --git a/include/seastar/core/abortable_fifo.hh b/include/seastar/core/abortable_fifo.hh index fed55f2d6b..b7156eb8fd 100644 --- a/include/seastar/core/abortable_fifo.hh +++ b/include/seastar/core/abortable_fifo.hh @@ -24,6 +24,7 @@ #include #include #include +#include #include #ifndef SEASTAR_MODULE @@ -111,7 +112,7 @@ public: 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); + SEASTAR_ASSERT(o._size == 0); } abortable_fifo& operator=(abortable_fifo&& o) noexcept { @@ -236,7 +237,7 @@ public: if (!_list.empty()) { e = &_list.back(); } - assert(!e->sub); + SEASTAR_ASSERT(!e->sub); auto aborter = [this, e] (const std::optional& ex_opt) noexcept { if constexpr (aborter_ex) { _on_abort(*e->payload, ex_opt); diff --git a/include/seastar/core/chunked_fifo.hh b/include/seastar/core/chunked_fifo.hh index 7033b852bf..930d1e1968 100644 --- a/include/seastar/core/chunked_fifo.hh +++ b/include/seastar/core/chunked_fifo.hh @@ -25,6 +25,7 @@ #include #include #include +#include #include #endif @@ -342,7 +343,7 @@ void chunked_fifo::clear() noexcept { template void chunked_fifo::pop_front_n(size_t n) noexcept { while (n) { - assert(_front_chunk && "pop_front_n n too large"); + SEASTAR_ASSERT(_front_chunk && "pop_front_n n too large"); auto target = _front_chunk; unsigned delete_count = std::min(target->size(), n); diff --git a/include/seastar/core/expiring_fifo.hh b/include/seastar/core/expiring_fifo.hh index 9eb44a1612..dbd414cff8 100644 --- a/include/seastar/core/expiring_fifo.hh +++ b/include/seastar/core/expiring_fifo.hh @@ -29,6 +29,7 @@ #include #include #include +#include #include #endif @@ -108,7 +109,7 @@ public: 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); + SEASTAR_ASSERT(o._size == 0); } expiring_fifo& operator=(expiring_fifo&& o) noexcept { diff --git a/include/seastar/core/fair_queue.hh b/include/seastar/core/fair_queue.hh index bbfde6c37d..217264312e 100644 --- a/include/seastar/core/fair_queue.hh +++ b/include/seastar/core/fair_queue.hh @@ -26,6 +26,7 @@ #include #include #include +#include #include #include @@ -318,7 +319,7 @@ private: } void assert_enough_capacity() const noexcept { - assert(c.size() < c.capacity()); + SEASTAR_ASSERT(c.size() < c.capacity()); } }; diff --git a/include/seastar/core/future.hh b/include/seastar/core/future.hh index c1ea3bcc21..fdce014302 100644 --- a/include/seastar/core/future.hh +++ b/include/seastar/core/future.hh @@ -37,6 +37,7 @@ #include #include #include +#include #include #include #include @@ -416,7 +417,7 @@ struct future_state_base { 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); + SEASTAR_ASSERT(st >= state::exception_min); } any(std::exception_ptr&& e) noexcept { set_exception(std::move(e)); @@ -506,21 +507,21 @@ public: void ignore() noexcept; void set_exception(std::exception_ptr&& ex) noexcept { - assert(_u.st == state::future); + SEASTAR_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); + SEASTAR_ASSERT(_u.st == state::future); *this = std::move(state); } std::exception_ptr get_exception() && noexcept { - assert(_u.st >= state::exception_min); + SEASTAR_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); + SEASTAR_ASSERT(_u.st >= state::exception_min); return _u.ex; } template @@ -607,7 +608,7 @@ struct future_state : public future_state_base, private internal::uninitialized } template void set(A&&... a) noexcept { - assert(_u.st == state::future); + SEASTAR_ASSERT(_u.st == state::future); new (this) future_state(ready_future_marker(), std::forward(a)...); } future_state(exception_future_marker, std::exception_ptr&& ex) noexcept : future_state_base(std::move(ex)) { } @@ -616,21 +617,21 @@ struct future_state : public future_state_base, private internal::uninitialized 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); + SEASTAR_ASSERT(_u.st == state::result); return static_cast(this->uninitialized_get()); } T&& take_value() && noexcept { - assert(_u.st == state::result); + SEASTAR_ASSERT(_u.st == state::result); _u.st = state::result_unavailable; return static_cast(this->uninitialized_get()); } template const std::enable_if_t, U>& get_value() const& noexcept(copy_noexcept) { - assert(_u.st == state::result); + SEASTAR_ASSERT(_u.st == state::result); return this->uninitialized_get(); } T&& take() && { - assert(available()); + SEASTAR_ASSERT(available()); if (_u.st >= state::exception_min) { std::move(*this).rethrow_exception(); } @@ -638,14 +639,14 @@ struct future_state : public future_state_base, private internal::uninitialized return static_cast(this->uninitialized_get()); } T&& get() && { - assert(available()); + SEASTAR_ASSERT(available()); if (_u.st >= state::exception_min) { std::move(*this).rethrow_exception(); } return static_cast(this->uninitialized_get()); } const T& get() const& { - assert(available()); + SEASTAR_ASSERT(available()); if (_u.st >= state::exception_min) { rethrow_exception(); } @@ -868,7 +869,7 @@ public: // 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); + SEASTAR_ASSERT(ptr->_u.st == future_state_base::state::future); new (ptr) future_state(std::move(state)); make_ready(); } @@ -1243,7 +1244,7 @@ private: : _state(std::move(state)) { } internal::promise_base_with_type get_promise() noexcept { - assert(!_promise); + SEASTAR_ASSERT(!_promise); return internal::promise_base_with_type(this); } internal::promise_base_with_type* detach_promise() noexcept { @@ -1750,7 +1751,7 @@ public: private: void set_task(task& t) noexcept { - assert(_promise); + SEASTAR_ASSERT(_promise); _promise->set_task(&t); } @@ -1759,7 +1760,7 @@ private: callback->set_state(get_available_state_ref()); ::seastar::schedule(callback); } else { - assert(_promise); + SEASTAR_ASSERT(_promise); schedule(callback); } @@ -1903,7 +1904,7 @@ template inline future promise::get_future() noexcept { - assert(!this->_future && this->_state && !this->_task); + SEASTAR_ASSERT(!this->_future && this->_state && !this->_task); return future(this); } diff --git a/include/seastar/core/gate.hh b/include/seastar/core/gate.hh index c9bcd13fa6..d0d4ca312e 100644 --- a/include/seastar/core/gate.hh +++ b/include/seastar/core/gate.hh @@ -25,6 +25,7 @@ #include #include +#include #include #include #include @@ -80,7 +81,7 @@ public: } gate& operator=(gate&& x) noexcept { if (this != &x) { - assert(!_count && "gate reassigned with outstanding requests"); + SEASTAR_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); @@ -88,7 +89,7 @@ public: return *this; } ~gate() { - assert(!_count && "gate destroyed with outstanding requests"); + SEASTAR_ASSERT(!_count && "gate destroyed with outstanding requests"); assert_not_held_when_destroyed(); } /// Tries to register an in-progress request. @@ -141,7 +142,7 @@ public: /// 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"); + SEASTAR_ASSERT(!_stopped && "seastar::gate::close() cannot be called more than once"); _stopped = std::make_optional(promise<>()); if (!_count) { _stopped->set_value(); @@ -300,10 +301,10 @@ private: #ifdef SEASTAR_GATE_HOLDER_DEBUG SEASTAR_MODULE_EXPORT inline void gate::assert_not_held_when_moved() const noexcept { - assert(_holders.empty() && "gate moved with outstanding holders"); + SEASTAR_ASSERT(_holders.empty() && "gate moved with outstanding holders"); } inline void gate::assert_not_held_when_destroyed() const noexcept { - assert(_holders.empty() && "gate destroyed with outstanding holders"); + SEASTAR_ASSERT(_holders.empty() && "gate destroyed with outstanding holders"); } #endif // SEASTAR_GATE_HOLDER_DEBUG diff --git a/include/seastar/core/internal/io_intent.hh b/include/seastar/core/internal/io_intent.hh index da0e7101ef..694e8a0881 100644 --- a/include/seastar/core/internal/io_intent.hh +++ b/include/seastar/core/internal/io_intent.hh @@ -21,6 +21,7 @@ #pragma once +#include #include #ifndef SEASTAR_MODULE #include @@ -59,7 +60,7 @@ public: public: link() noexcept : _ref(nullptr) {} - ~link() { assert(_ref == nullptr); } + ~link() { SEASTAR_ASSERT(_ref == nullptr); } void enqueue(cancellable_queue& cq) noexcept { cq.push_back(*this); diff --git a/include/seastar/core/internal/io_request.hh b/include/seastar/core/internal/io_request.hh index 72c2770b60..759ddb97f4 100644 --- a/include/seastar/core/internal/io_request.hh +++ b/include/seastar/core/internal/io_request.hh @@ -23,7 +23,7 @@ #include #include -#include +#include #include #include #include @@ -426,7 +426,7 @@ public: io_direction_and_length(int idx, size_t val) noexcept : _directed_length((val << 1) | idx) { - assert(idx == read_idx || idx == write_idx); + SEASTAR_ASSERT(idx == read_idx || idx == write_idx); } }; diff --git a/include/seastar/core/iostream-impl.hh b/include/seastar/core/iostream-impl.hh index abe583956e..8611aded6e 100644 --- a/include/seastar/core/iostream-impl.hh +++ b/include/seastar/core/iostream-impl.hh @@ -27,6 +27,7 @@ #include #include #include +#include #include namespace seastar { @@ -115,7 +116,7 @@ future<> output_stream::write(net::packet p) noexcept { static_assert(std::is_same_v, "packet works on char"); try { if (p.len() != 0) { - assert(!_end && "Mixing buffered writes and zero-copy writes not supported yet"); + SEASTAR_ASSERT(!_end && "Mixing buffered writes and zero-copy writes not supported yet"); if (_zc_bufs) { _zc_bufs.append(std::move(p)); @@ -143,7 +144,7 @@ future<> output_stream::write(temporary_buffer p) noexcept { if (p.empty()) { return make_ready_future<>(); } - assert(!_end && "Mixing buffered writes and zero-copy writes not supported yet"); + SEASTAR_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(); @@ -324,7 +325,7 @@ input_stream::detach() && { template future<> output_stream::split_and_put(temporary_buffer buf) noexcept { - assert(_end == 0); + SEASTAR_ASSERT(_end == 0); return repeat([this, buf = std::move(buf)] () mutable { if (buf.size() < _size) { @@ -358,7 +359,7 @@ template future<> output_stream::slow_write(const char_type* buf, size_t n) noexcept { try { - assert(!_zc_bufs && "Mixing buffered writes and zero-copy writes not supported yet"); + SEASTAR_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) { diff --git a/include/seastar/core/iostream.hh b/include/seastar/core/iostream.hh index 279bc4fe12..0d920ce864 100644 --- a/include/seastar/core/iostream.hh +++ b/include/seastar/core/iostream.hh @@ -39,6 +39,7 @@ #include #include #include +#include #include #include #ifndef SEASTAR_MODULE @@ -129,7 +130,7 @@ public: // 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"); + SEASTAR_ASSERT(false && "Data sink must have the buffer_size() method overload"); return 0; } @@ -142,7 +143,7 @@ public: } virtual void on_batch_flush_error() noexcept { - assert(false && "Data sink must implement on_batch_flush_error() method"); + SEASTAR_ASSERT(false && "Data sink must implement on_batch_flush_error() method"); } protected: @@ -430,9 +431,9 @@ public: output_stream& operator=(output_stream&&) noexcept = default; ~output_stream() { if (_batch_flushes) { - assert(!_in_batch && "Was this stream properly closed?"); + SEASTAR_ASSERT(!_in_batch && "Was this stream properly closed?"); } else { - assert(!_end && !_zc_bufs && "Was this stream properly closed?"); + SEASTAR_ASSERT(!_end && !_zc_bufs && "Was this stream properly closed?"); } } future<> write(const char_type* buf, size_t n) noexcept; diff --git a/include/seastar/core/loop.hh b/include/seastar/core/loop.hh index 015614cf46..89716ed4c9 100644 --- a/include/seastar/core/loop.hh +++ b/include/seastar/core/loop.hh @@ -33,6 +33,7 @@ #endif #include #include +#include #include #include #include @@ -692,7 +693,7 @@ max_concurrent_for_each(Iterator begin, Sentinel end, size_t max_concurrent, Fun { } }; - assert(max_concurrent > 0); + SEASTAR_ASSERT(max_concurrent > 0); try { return do_with(state(std::move(begin), std::move(end), max_concurrent, std::forward(func)), [] (state& s) { diff --git a/include/seastar/core/on_internal_error.hh b/include/seastar/core/on_internal_error.hh index a89ac55a90..5e6d89992a 100644 --- a/include/seastar/core/on_internal_error.hh +++ b/include/seastar/core/on_internal_error.hh @@ -21,7 +21,7 @@ #pragma once - +#include #include #include #ifndef SEASTAR_MODULE @@ -69,7 +69,7 @@ void on_internal_error_noexcept(logger& logger, std::string_view reason) noexcep /// /// 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(). +/// This overload can be used to replace SEASTAR_ASSERT(). [[noreturn]] void on_fatal_internal_error(logger& logger, std::string_view reason) noexcept; SEASTAR_MODULE_EXPORT_END diff --git a/include/seastar/core/posix.hh b/include/seastar/core/posix.hh index a7d1baaf97..cdb32f8421 100644 --- a/include/seastar/core/posix.hh +++ b/include/seastar/core/posix.hh @@ -49,6 +49,7 @@ #include "abort_on_ebadf.hh" #include #include +#include #include #include @@ -100,7 +101,7 @@ public: return *this; } void close() { - assert(_fd != -1); + SEASTAR_ASSERT(_fd != -1); auto r = ::close(_fd); throw_system_error_on(r == -1, "close"); _fd = -1; @@ -517,7 +518,7 @@ void pin_this_thread(unsigned cpu_id) { CPU_ZERO(&cs); CPU_SET(cpu_id, &cs); auto r = pthread_setaffinity_np(pthread_self(), sizeof(cs), &cs); - assert(r == 0); + SEASTAR_ASSERT(r == 0); (void)r; } diff --git a/include/seastar/core/queue.hh b/include/seastar/core/queue.hh index 88f2367421..bb9b0fc667 100644 --- a/include/seastar/core/queue.hh +++ b/include/seastar/core/queue.hh @@ -23,6 +23,7 @@ #include #include +#include #include #ifndef SEASTAR_MODULE #include @@ -213,7 +214,7 @@ T queue::pop() noexcept { // 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()); + SEASTAR_ASSERT(!_q.empty()); T data = std::move(_q.front()); _q.pop(); return data; diff --git a/include/seastar/core/ragel.hh b/include/seastar/core/ragel.hh index 9c7ccf9cca..f73eca67aa 100644 --- a/include/seastar/core/ragel.hh +++ b/include/seastar/core/ragel.hh @@ -24,6 +24,7 @@ #ifndef SEASTAR_MODULE #include #include +#include #include #include #include @@ -116,7 +117,7 @@ protected: 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); + SEASTAR_ASSERT(_fsm_stack_size > old); std::unique_ptr 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); diff --git a/include/seastar/core/rwlock.hh b/include/seastar/core/rwlock.hh index 0fad7a95f1..fbd50d4875 100644 --- a/include/seastar/core/rwlock.hh +++ b/include/seastar/core/rwlock.hh @@ -25,6 +25,7 @@ #include #endif #include +#include #include namespace seastar { @@ -112,7 +113,7 @@ public: /// is called, one of the fibers waiting on \ref write_lock will be allowed /// to proceed. void read_unlock() { - assert(_sem.current() < max_ops); + SEASTAR_ASSERT(_sem.current() < max_ops); _sem.signal(); } @@ -132,7 +133,7 @@ public: /// 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); + SEASTAR_ASSERT(_sem.current() == 0); _sem.signal(max_ops); } diff --git a/include/seastar/core/scheduling_specific.hh b/include/seastar/core/scheduling_specific.hh index bdb51110d7..c920f14117 100644 --- a/include/seastar/core/scheduling_specific.hh +++ b/include/seastar/core/scheduling_specific.hh @@ -22,6 +22,7 @@ #ifndef SEASTAR_MODULE #include #include +#include #include #include #include @@ -127,7 +128,7 @@ template 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); + SEASTAR_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() && @@ -172,7 +173,7 @@ T& scheduling_group_get_specific(scheduling_group_key key) noexcept { // 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); + SEASTAR_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(data.per_scheduling_group_data[sg_id].specific_vals[key.id()].get()); } diff --git a/include/seastar/core/semaphore.hh b/include/seastar/core/semaphore.hh index 67d7ef569e..fe74eebe46 100644 --- a/include/seastar/core/semaphore.hh +++ b/include/seastar/core/semaphore.hh @@ -28,9 +28,9 @@ #include #include #include +#include #include #ifndef SEASTAR_MODULE -#include #include #include #include @@ -208,11 +208,11 @@ private: used_flag() = default; used_flag(used_flag&& o) noexcept { - assert(!_used && "semaphore cannot be moved after it has been used"); + SEASTAR_ASSERT(!_used && "semaphore cannot be moved after it has been used"); } used_flag& operator=(used_flag&& o) noexcept { if (this != &o) { - assert(!_used && !o._used && "semaphore cannot be moved after it has been used"); + SEASTAR_ASSERT(!_used && !o._used && "semaphore cannot be moved after it has been used"); } return *this; } @@ -271,7 +271,7 @@ public: , _used(std::move(other._used)) { // semaphore cannot be moved with non-empty waiting list - assert(other._wait_list.empty()); + SEASTAR_ASSERT(other._wait_list.empty()); } /// Move-assigns a semaphore object from a moved-from semaphore object, @@ -283,8 +283,8 @@ public: /// \param other the moved-from semaphore object. basic_semaphore& operator=(basic_semaphore&& other) noexcept(std::is_nothrow_move_assignable_v) { // semaphore cannot be moved with non-empty waiting list - assert(_wait_list.empty()); - assert(other._wait_list.empty()); + SEASTAR_ASSERT(_wait_list.empty()); + SEASTAR_ASSERT(other._wait_list.empty()); if (this != &other) { exception_factory::operator=(other); _count = other._count; @@ -593,7 +593,7 @@ public: /// /// \return the updated semaphore_units object void adopt(semaphore_units&& other) noexcept { - assert(other._sem == _sem); + SEASTAR_ASSERT(other._sem == _sem); _n += other.release(); } diff --git a/include/seastar/core/sharded.hh b/include/seastar/core/sharded.hh index 16342d8e4e..5f4e886cb7 100644 --- a/include/seastar/core/sharded.hh +++ b/include/seastar/core/sharded.hh @@ -28,6 +28,7 @@ #include #include #include +#include #include #include @@ -578,7 +579,7 @@ SEASTAR_MODULE_EXPORT_END template sharded::~sharded() { - assert(_instances.empty()); + SEASTAR_ASSERT(_instances.empty()); } namespace internal { @@ -651,7 +652,7 @@ template future<> sharded::start_single(Args&&... args) noexcept { try { - assert(_instances.empty()); + SEASTAR_ASSERT(_instances.empty()); _instances.resize(1); return smp::submit_to(0, [this, args = std::make_tuple(std::forward(args)...)] () mutable { _instances[0].service = std::apply([this] (Args... args) { @@ -893,19 +894,19 @@ sharded::invoke_on(R range, Func func, Args... args) noexcept { template const Service& sharded::local() const noexcept { - assert(local_is_initialized()); + SEASTAR_ASSERT(local_is_initialized()); return *_instances[this_shard_id()].service; } template Service& sharded::local() noexcept { - assert(local_is_initialized()); + SEASTAR_ASSERT(local_is_initialized()); return *_instances[this_shard_id()].service; } template shared_ptr sharded::local_shared() noexcept { - assert(local_is_initialized()); + SEASTAR_ASSERT(local_is_initialized()); return _instances[this_shard_id()].service; } diff --git a/include/seastar/core/shared_mutex.hh b/include/seastar/core/shared_mutex.hh index f2590e6b52..02d40890c4 100644 --- a/include/seastar/core/shared_mutex.hh +++ b/include/seastar/core/shared_mutex.hh @@ -25,8 +25,8 @@ #include #include #include +#include #include -#include #include #include #include @@ -98,7 +98,7 @@ public: } /// Unlocks a \c shared_mutex after a previous call to \ref lock_shared(). void unlock_shared() noexcept { - assert(_readers > 0); + SEASTAR_ASSERT(_readers > 0); --_readers; wake(); } @@ -129,7 +129,7 @@ public: } /// Unlocks a \c shared_mutex after a previous call to \ref lock(). void unlock() noexcept { - assert(_writer); + SEASTAR_ASSERT(_writer); _writer = false; wake(); } diff --git a/include/seastar/core/slab.hh b/include/seastar/core/slab.hh index 820d06268c..7d4dfdc560 100644 --- a/include/seastar/core/slab.hh +++ b/include/seastar/core/slab.hh @@ -31,6 +31,7 @@ #include #include #include +#include namespace seastar { @@ -107,7 +108,7 @@ public: } void* allocate_object() { - assert(!_free_objects.empty()); + SEASTAR_ASSERT(!_free_objects.empty()); auto object = reinterpret_cast(_free_objects.back()); _free_objects.pop_back(); return object; @@ -158,7 +159,7 @@ private: Item& victim = reinterpret_cast(_lru.back()); uint32_t index = victim.get_slab_page_index(); - assert(victim.is_unlocked()); + SEASTAR_ASSERT(victim.is_unlocked()); _lru.erase(_lru.iterator_to(reinterpret_cast(victim))); // WARNING: You need to make sure that erase_func will not release victim back to slab. erase_func(victim); @@ -191,7 +192,7 @@ public: template Item *create(Args&&... args) { - assert(!_free_slab_pages.empty()); + SEASTAR_ASSERT(!_free_slab_pages.empty()); auto& desc = _free_slab_pages.back(); auto object = desc.allocate_object(); if (desc.empty()) { @@ -214,7 +215,7 @@ public: } // allocate descriptor to slab page. slab_page_desc *desc = nullptr; - assert(_size % alignment == 0); + SEASTAR_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); @@ -266,7 +267,7 @@ public: } void remove_desc_from_free_list(slab_page_desc& desc) { - assert(desc.slab_class_id() == _slab_class_id); + SEASTAR_ASSERT(desc.slab_class_id() == _slab_class_id); _free_slab_pages.erase(_free_slab_pages.iterator_to(desc)); } }; @@ -301,7 +302,7 @@ private: } // get descriptor of the least-recently-used slab page and related info. auto& desc = _slab_page_desc_lru.back(); - assert(desc.refcnt() == 0); + SEASTAR_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(); @@ -332,7 +333,7 @@ private: } } Item* item = reinterpret_cast(object); - assert(item->is_unlocked()); + SEASTAR_ASSERT(item->is_unlocked()); slab_class->remove_item_from_lru(item); _erase_func(*item); _stats.frees++; @@ -367,7 +368,7 @@ private: _slab_class_sizes.push_back(size); _slab_classes.emplace_back(size, slab_class_id); size *= growth_factor; - assert(slab_class_id < std::numeric_limits::max()); + SEASTAR_ASSERT(slab_class_id < std::numeric_limits::max()); slab_class_id++; } _slab_class_sizes.push_back(_max_object_size); @@ -392,7 +393,7 @@ private: } slab_class* get_slab_class(const uint8_t slab_class_id) { - assert(slab_class_id >= 0 && slab_class_id < _slab_classes.size()); + SEASTAR_ASSERT(slab_class_id >= 0 && slab_class_id < _slab_classes.size()); return &_slab_classes[slab_class_id]; } @@ -410,8 +411,8 @@ private: 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); + SEASTAR_ASSERT(desc != nullptr); + SEASTAR_ASSERT(desc->magic() == SLAB_MAGIC_NUMBER); return *desc; } diff --git a/include/seastar/core/sstring.hh b/include/seastar/core/sstring.hh index 20547d472a..42a09dfe6f 100644 --- a/include/seastar/core/sstring.hh +++ b/include/seastar/core/sstring.hh @@ -24,7 +24,6 @@ #ifndef SEASTAR_MODULE #include #include -#include #if __has_include() #include #endif @@ -42,6 +41,7 @@ #include #endif #endif +#include #include #include #include @@ -258,7 +258,7 @@ public: } size_t find(const char_type* c_str, size_t pos, size_t len2) const noexcept { - assert(c_str != nullptr || len2 == 0); + SEASTAR_ASSERT(c_str != nullptr || len2 == 0); if (pos > size()) { return npos; } @@ -361,7 +361,7 @@ public: *this = basic_sstring(initialized_later(), n); } size_t r = std::move(op)(data(), n); - assert(r <= n); + SEASTAR_ASSERT(r <= n); resize(r); } @@ -470,7 +470,7 @@ public: */ reference front() noexcept { - assert(!empty()); + SEASTAR_ASSERT(!empty()); return *str(); } @@ -481,7 +481,7 @@ public: */ const_reference front() const noexcept { - assert(!empty()); + SEASTAR_ASSERT(!empty()); return *str(); } diff --git a/include/seastar/core/thread.hh b/include/seastar/core/thread.hh index 14811fb6e4..cb56a99941 100644 --- a/include/seastar/core/thread.hh +++ b/include/seastar/core/thread.hh @@ -30,6 +30,7 @@ #include #include #include +#include #include #include #include @@ -168,7 +169,7 @@ public: /// \brief Destroys a \c thread object. /// /// The thread must not represent a running thread of execution (see join()). - ~thread() { assert(!_context || _context->_joined); } + ~thread() { SEASTAR_ASSERT(!_context || _context->_joined); } /// \brief Waits for thread execution to terminate. /// /// Waits for thread execution to terminate, and marks the thread object as not diff --git a/include/seastar/core/thread_cputime_clock.hh b/include/seastar/core/thread_cputime_clock.hh index 5b8467194c..2af59228d8 100644 --- a/include/seastar/core/thread_cputime_clock.hh +++ b/include/seastar/core/thread_cputime_clock.hh @@ -24,8 +24,8 @@ #ifndef SEASTAR_MODULE #include #include -#include #endif +#include namespace seastar { @@ -41,7 +41,7 @@ public: struct timespec tp; [[gnu::unused]] auto ret = clock_gettime(CLOCK_THREAD_CPUTIME_ID, &tp); - assert(ret == 0); + SEASTAR_ASSERT(ret == 0); return time_point(tp.tv_nsec * 1ns + tp.tv_sec * 1s); } }; diff --git a/include/seastar/core/timer-set.hh b/include/seastar/core/timer-set.hh index 686a5f2919..046dcc4e52 100644 --- a/include/seastar/core/timer-set.hh +++ b/include/seastar/core/timer-set.hh @@ -15,6 +15,7 @@ #include #include +#include #ifndef SEASTAR_MODULE #include #include @@ -78,7 +79,7 @@ private: } auto index = bitsets::count_leading_zeros(timestamp ^ _last); - assert(index < n_buckets - 1); + SEASTAR_ASSERT(index < n_buckets - 1); return index; } diff --git a/include/seastar/core/timer.hh b/include/seastar/core/timer.hh index 1389b321da..86a054432c 100644 --- a/include/seastar/core/timer.hh +++ b/include/seastar/core/timer.hh @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #ifndef SEASTAR_MODULE @@ -97,7 +98,7 @@ private: bool _expired = false; void readd_periodic() noexcept; void arm_state(time_point until, std::optional period) noexcept { - assert(!_armed); + SEASTAR_ASSERT(!_armed); _period = period; _armed = true; _expired = false; diff --git a/include/seastar/coroutine/generator.hh b/include/seastar/coroutine/generator.hh index 2459858461..ef84ca7d56 100644 --- a/include/seastar/coroutine/generator.hh +++ b/include/seastar/coroutine/generator.hh @@ -21,11 +21,11 @@ #pragma once -#include #include #include #include #include +#include namespace seastar::coroutine::experimental { @@ -84,9 +84,9 @@ public: template U> suspend_always yield_value(U&& value) noexcept { - assert(_generator); + SEASTAR_ASSERT(_generator); _generator->put_next_value(std::forward(value)); - assert(_wait_for_next_value); + SEASTAR_ASSERT(_wait_for_next_value); _wait_for_next_value->set_value(); _wait_for_next_value = {}; return {}; @@ -94,19 +94,19 @@ public: generator_type get_return_object() noexcept; void set_generator(generator_type* g) noexcept { - assert(!_generator); + SEASTAR_ASSERT(!_generator); _generator = g; } suspend_always initial_suspend() const noexcept { return {}; } suspend_never final_suspend() const noexcept { - assert(_generator); + SEASTAR_ASSERT(_generator); _generator->on_finished(); return {}; } seastar::future<> wait_for_next_value() noexcept { - assert(!_wait_for_next_value); + SEASTAR_ASSERT(!_wait_for_next_value); return _wait_for_next_value.emplace().get_future(); } @@ -183,20 +183,20 @@ public: if (ready) { return {make_ready_future()}; } else { - assert(!_wait_for_free_space); + SEASTAR_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); + SEASTAR_ASSERT(!_generator); _generator = g; } suspend_always initial_suspend() const noexcept { return {}; } suspend_never final_suspend() const noexcept { - assert(_generator); + SEASTAR_ASSERT(_generator); _generator->on_finished(); return {}; } @@ -210,12 +210,12 @@ public: } seastar::future<> wait_for_next_value() noexcept { - assert(!_wait_for_next_value); + SEASTAR_ASSERT(!_wait_for_next_value); return _wait_for_next_value.emplace().get_future(); } void on_reclaim_free_space() noexcept { - assert(_wait_for_free_space); + SEASTAR_ASSERT(_wait_for_free_space); _wait_for_free_space->set_value(); _wait_for_free_space = {}; } @@ -271,8 +271,8 @@ public: } next_value_type await_resume() { - assert(_next_value_future.available()); - assert(_generator); + SEASTAR_ASSERT(_next_value_future.available()); + SEASTAR_ASSERT(_generator); return _generator->take_next_value(); } }; @@ -329,7 +329,7 @@ public: : _coro{coro} , _promise{promise} , _buffer_capacity{buffer_capacity} { - assert(_promise); + SEASTAR_ASSERT(_promise); _promise->set_generator(this); } generator(const generator&) = delete; @@ -398,7 +398,7 @@ public: void unhandled_exception() noexcept { // called by promise's unhandled_exception() - assert(!_exception); + SEASTAR_ASSERT(!_exception); _exception = std::current_exception(); } }; @@ -420,7 +420,7 @@ public: promise_type* promise) noexcept : _coro{coro} , _promise{promise} { - assert(_promise); + SEASTAR_ASSERT(_promise); _promise->set_generator(this); } generator(const generator&) = delete; @@ -474,7 +474,7 @@ public: } void unhandled_exception() noexcept { - assert(!_exception); + SEASTAR_ASSERT(!_exception); _exception = std::current_exception(); } }; @@ -483,7 +483,7 @@ namespace internal { template void generator_unbuffered_promise::return_void() noexcept { - assert(_wait_for_next_value); + SEASTAR_ASSERT(_wait_for_next_value); _wait_for_next_value->set_value(); _wait_for_next_value = {}; } diff --git a/include/seastar/net/ethernet.hh b/include/seastar/net/ethernet.hh index e82f91b33e..06d27bddeb 100644 --- a/include/seastar/net/ethernet.hh +++ b/include/seastar/net/ethernet.hh @@ -26,9 +26,9 @@ #endif #include -#include #include #include +#include namespace seastar { @@ -43,7 +43,7 @@ struct ethernet_address { } ethernet_address(std::initializer_list eaddr) noexcept { - assert(eaddr.size() == mac.size()); + SEASTAR_ASSERT(eaddr.size() == mac.size()); std::copy(eaddr.begin(), eaddr.end(), mac.begin()); } diff --git a/include/seastar/net/net.hh b/include/seastar/net/net.hh index f35d8ca97d..65cdafe9a0 100644 --- a/include/seastar/net/net.hh +++ b/include/seastar/net/net.hh @@ -30,6 +30,7 @@ #include #include #include +#include #include namespace seastar { @@ -56,7 +57,7 @@ public: return end_idx; } void push_back(uint8_t b) { - assert(end_idx < sizeof(data)); + SEASTAR_ASSERT(end_idx < sizeof(data)); data[end_idx++] = b; } void push_back(uint16_t b) { diff --git a/include/seastar/net/packet.hh b/include/seastar/net/packet.hh index c7cc943655..46e3fb1f73 100644 --- a/include/seastar/net/packet.hh +++ b/include/seastar/net/packet.hh @@ -24,11 +24,11 @@ #include #include #include +#include #include #include #ifndef SEASTAR_MODULE #include -#include #include #include #include @@ -148,7 +148,7 @@ class packet final { return copy(old.get(), std::max(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)); + SEASTAR_ASSERT(nr_frags == uint16_t(nr_frags)); return ::operator new(size + nr_frags * sizeof(fragment)); } // Matching the operator new above @@ -342,7 +342,7 @@ packet::impl::impl(size_t nr_frags) noexcept inline packet::impl::impl(fragment frag, size_t nr_frags) : _len(frag.size), _allocated_frags(nr_frags) { - assert(_allocated_frags > _nr_frags); + SEASTAR_ASSERT(_allocated_frags > _nr_frags); if (frag.size <= internal_data_size) { _headroom -= frag.size; _frags[0] = { _data + _headroom, frag.size }; @@ -533,7 +533,7 @@ Header* packet::get_header(size_t offset) { inline void packet::trim_front(size_t how_much) noexcept { - assert(how_much <= _impl->_len); + SEASTAR_ASSERT(how_much <= _impl->_len); _impl->_len -= how_much; size_t i = 0; while (how_much && how_much >= _impl->_frags[i].size) { @@ -555,7 +555,7 @@ void packet::trim_front(size_t how_much) noexcept { inline void packet::trim_back(size_t how_much) noexcept { - assert(how_much <= _impl->_len); + SEASTAR_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) { @@ -622,7 +622,7 @@ packet packet::share(size_t offset, size_t len) { offset = 0; } n._impl->_offload_info = _impl->_offload_info; - assert(!n._impl->_deleter); + SEASTAR_ASSERT(!n._impl->_deleter); n._impl->_deleter = _impl->_deleter.share(); return n; } diff --git a/include/seastar/net/tcp.hh b/include/seastar/net/tcp.hh index e2f5a567c8..74e98c7e55 100644 --- a/include/seastar/net/tcp.hh +++ b/include/seastar/net/tcp.hh @@ -42,6 +42,7 @@ #include #include #include +#include #include namespace seastar { @@ -1730,7 +1731,7 @@ void tcp::tcb::output_one(bool data_retransmit) { // 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))); + SEASTAR_ASSERT((_snd.window > 0) || ((_snd.window == 0) && (len <= 1))); queue_packet(std::move(p)); } @@ -2094,7 +2095,7 @@ tcp_seq tcp::tcb::get_isn() { gnutls_hash(md5_hash_handle, hash, 3 * sizeof(hash[0])); gnutls_hash(md5_hash_handle, _isn_secret.key, sizeof(_isn_secret.key)); // reuse "hash" for the output of digest - assert(sizeof(hash) == gnutls_hash_get_len(GNUTLS_DIG_MD5)); + SEASTAR_ASSERT(sizeof(hash) == gnutls_hash_get_len(GNUTLS_DIG_MD5)); gnutls_hash_deinit(md5_hash_handle, hash); auto seq = hash[0]; auto m = duration_cast(clock_type::now().time_since_epoch()); @@ -2113,7 +2114,7 @@ std::optional tcp::tcb::get_packet() return std::optional(); } - assert(!_packetq.empty()); + SEASTAR_ASSERT(!_packetq.empty()); auto p = std::move(_packetq.front()); _packetq.pop_front(); diff --git a/include/seastar/rpc/rpc_impl.hh b/include/seastar/rpc/rpc_impl.hh index 54d8b1d99c..e2fbae4c94 100644 --- a/include/seastar/rpc/rpc_impl.hh +++ b/include/seastar/rpc/rpc_impl.hh @@ -25,6 +25,7 @@ #include #include #include +#include #include #include #include @@ -910,7 +911,7 @@ template sink_impl::~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()); + SEASTAR_ASSERT(this->_con->get()->sink_closed()); } template diff --git a/include/seastar/rpc/rpc_types.hh b/include/seastar/rpc/rpc_types.hh index 58ff247b11..d69a38b338 100644 --- a/include/seastar/rpc/rpc_types.hh +++ b/include/seastar/rpc/rpc_types.hh @@ -32,6 +32,7 @@ #include #include #include +#include #include #include #include @@ -106,7 +107,7 @@ struct client_info { template T& retrieve_auxiliary(const sstring& key) { auto it = user_data.find(key); - assert(it != user_data.end()); + SEASTAR_ASSERT(it != user_data.end()); return std::any_cast(it->second); } template diff --git a/include/seastar/util/assert.hh b/include/seastar/util/assert.hh new file mode 100644 index 0000000000..2c8eba9018 --- /dev/null +++ b/include/seastar/util/assert.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) 2025 Redpanda Data. + */ + +#pragma once + +namespace seastar::internal { +[[noreturn]] void assert_fail(const char *msg, const char *file, int line, + const char *func); +} + +/// Like assert(), but independent of NDEBUG. Active in all build modes. +#define SEASTAR_ASSERT(x) \ + do { \ + if (!(x)) [[unlikely]] { \ + seastar::internal::assert_fail(#x, __FILE__, __LINE__, \ + __PRETTY_FUNCTION__); \ + } \ + } while (0) diff --git a/include/seastar/util/backtrace.hh b/include/seastar/util/backtrace.hh index ec4b959734..0676004a44 100644 --- a/include/seastar/util/backtrace.hh +++ b/include/seastar/util/backtrace.hh @@ -25,6 +25,7 @@ #include #include #include +#include #include #ifndef SEASTAR_MODULE @@ -231,7 +232,7 @@ public: * @return original exception message followed by a backtrace */ virtual const char* what() const noexcept override { - assert(_backtrace); + SEASTAR_ASSERT(_backtrace); return _backtrace->c_str(); } }; diff --git a/include/seastar/util/closeable.hh b/include/seastar/util/closeable.hh index 6d575f018a..b3d2340672 100644 --- a/include/seastar/util/closeable.hh +++ b/include/seastar/util/closeable.hh @@ -24,6 +24,7 @@ #include #include #include +#include #include /// \file @@ -79,7 +80,7 @@ public: } /// Close \c obj once now. void close_now() noexcept { - assert(!_closed); + SEASTAR_ASSERT(!_closed); do_close(); } @@ -148,7 +149,7 @@ public: } /// Stop \c obj once now. void stop_now() noexcept { - assert(!_stopped); + SEASTAR_ASSERT(!_stopped); do_stop(); } diff --git a/include/seastar/util/print_safe.hh b/include/seastar/util/print_safe.hh index 01be04dada..1387675f4e 100644 --- a/include/seastar/util/print_safe.hh +++ b/include/seastar/util/print_safe.hh @@ -22,7 +22,6 @@ #pragma once #ifndef SEASTAR_MODULE -#include #include #include #include @@ -30,6 +29,7 @@ #include #endif +#include #include namespace seastar { @@ -77,7 +77,7 @@ char* convert_hex_safe(char *buf, size_t bufsz, Integral n) noexcept { memset(buf, Padding, bufsz); auto* p = buf + bufsz; do { - assert(p > buf); + SEASTAR_ASSERT(p > buf); *--p = digits[n & 0xf]; n >>= 4; } while (n); @@ -112,7 +112,7 @@ 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); + SEASTAR_ASSERT(i > 0); tmp[--i] = '0' + n % 10; n /= 10; } while (n); diff --git a/include/seastar/util/spinlock.hh b/include/seastar/util/spinlock.hh index 71d996d44f..1efe4235fa 100644 --- a/include/seastar/util/spinlock.hh +++ b/include/seastar/util/spinlock.hh @@ -22,10 +22,10 @@ #pragma once #include +#include #ifndef SEASTAR_MODULE #include -#include #if defined(__x86_64__) || defined(__i386__) #include @@ -90,7 +90,7 @@ class spinlock { public: spinlock() = default; spinlock(const spinlock&) = delete; - ~spinlock() { assert(!_busy.load(std::memory_order_relaxed)); } + ~spinlock() { SEASTAR_ASSERT(!_busy.load(std::memory_order_relaxed)); } bool try_lock() noexcept { return !_busy.exchange(true, std::memory_order_acquire); } diff --git a/src/core/condition-variable.cc b/src/core/condition-variable.cc index 6467ce8c8e..37f2b026e9 100644 --- a/src/core/condition-variable.cc +++ b/src/core/condition-variable.cc @@ -20,13 +20,13 @@ */ #ifdef SEASTAR_MODULE module; -#include #include #include module seastar; #else #include #endif +#include namespace seastar { @@ -43,7 +43,7 @@ condition_variable::~condition_variable() { } void condition_variable::add_waiter(waiter& w) noexcept { - assert(!_signalled); // should not have snuck between + SEASTAR_ASSERT(!_signalled); // should not have snuck between if (_ex) { w.set_exception(_ex); return; diff --git a/src/core/exception_hacks.cc b/src/core/exception_hacks.cc index 9ac7ff0d10..423b400044 100644 --- a/src/core/exception_hacks.cc +++ b/src/core/exception_hacks.cc @@ -58,7 +58,6 @@ module; #include #include -#include #include #include @@ -69,6 +68,7 @@ module seastar; #include #include #endif +#include namespace seastar { using dl_iterate_fn = int (*) (int (*callback) (struct dl_phdr_info *info, size_t size, void *data), void *data); @@ -77,7 +77,7 @@ using dl_iterate_fn = int (*) (int (*callback) (struct dl_phdr_info *info, size_ static dl_iterate_fn dl_iterate_phdr_org() { static dl_iterate_fn org = [] { auto org = (dl_iterate_fn)dlsym (RTLD_NEXT, "dl_iterate_phdr"); - assert(org); + SEASTAR_ASSERT(org); return org; }(); return org; diff --git a/src/core/fair_queue.cc b/src/core/fair_queue.cc index 81069d156a..5ac5f9c949 100644 --- a/src/core/fair_queue.cc +++ b/src/core/fair_queue.cc @@ -39,6 +39,7 @@ module seastar; #include #include #endif +#include namespace seastar { @@ -109,7 +110,7 @@ fair_group::fair_group(config cfg, unsigned nr_queues) } auto fair_group::grab_capacity(capacity_t cap) noexcept -> capacity_t { - assert(cap <= _token_bucket.limit()); + SEASTAR_ASSERT(cap <= _token_bucket.limit()); return _token_bucket.grab(cap); } @@ -169,12 +170,12 @@ fair_queue::fair_queue(fair_group& group, config cfg) fair_queue::~fair_queue() { for (const auto& fq : _priority_classes) { - assert(!fq); + SEASTAR_ASSERT(!fq); } } void fair_queue::push_priority_class(priority_class_data& pc) noexcept { - assert(pc._plugged && !pc._queued); + SEASTAR_ASSERT(pc._plugged && !pc._queued); _handles.assert_enough_capacity(); _handles.push(&pc); pc._queued = true; @@ -201,13 +202,13 @@ void fair_queue::push_priority_class_from_idle(priority_class_data& pc) noexcept // ATTN: This can only be called on pc that is from _handles.top() void fair_queue::pop_priority_class(priority_class_data& pc) noexcept { - assert(pc._queued); + SEASTAR_ASSERT(pc._queued); pc._queued = false; _handles.pop(); } void fair_queue::plug_priority_class(priority_class_data& pc) noexcept { - assert(!pc._plugged); + SEASTAR_ASSERT(!pc._plugged); pc._plugged = true; if (!pc._queue.empty()) { push_priority_class_from_idle(pc); @@ -219,7 +220,7 @@ void fair_queue::plug_class(class_id cid) noexcept { } void fair_queue::unplug_priority_class(priority_class_data& pc) noexcept { - assert(pc._plugged); + SEASTAR_ASSERT(pc._plugged); pc._plugged = false; } @@ -249,7 +250,7 @@ void fair_queue::register_priority_class(class_id id, uint32_t shares) { if (id >= _priority_classes.size()) { _priority_classes.resize(id + 1); } else { - assert(!_priority_classes[id]); + SEASTAR_ASSERT(!_priority_classes[id]); } _handles.reserve(_nr_classes + 1); @@ -259,15 +260,15 @@ void fair_queue::register_priority_class(class_id id, uint32_t shares) { void fair_queue::unregister_priority_class(class_id id) { auto& pclass = _priority_classes[id]; - assert(pclass); + SEASTAR_ASSERT(pclass); pclass.reset(); _nr_classes--; } void fair_queue::update_shares_for_class(class_id id, uint32_t shares) { - assert(id < _priority_classes.size()); + SEASTAR_ASSERT(id < _priority_classes.size()); auto& pc = _priority_classes[id]; - assert(pc); + SEASTAR_ASSERT(pc); pc->update_shares(shares); } @@ -390,7 +391,7 @@ void fair_queue::dispatch_requests(std::function cb) { // for the highest-priority request as of now. But since group head didn't touch // it yet, there's no good way to cancel it, so we have no choice but to wait // until the touch time. - assert(available.ready_tokens == 0); + SEASTAR_ASSERT(available.ready_tokens == 0); break; } @@ -430,7 +431,7 @@ void fair_queue::dispatch_requests(std::function cb) { } } - assert(_handles.empty() || available.ready_tokens == 0); + SEASTAR_ASSERT(_handles.empty() || available.ready_tokens == 0); // Note: if IO cancellation happens, it's possible that we are still holding some tokens in `ready` here. // diff --git a/src/core/file.cc b/src/core/file.cc index b77bef8efc..c89d22395e 100644 --- a/src/core/file.cc +++ b/src/core/file.cc @@ -31,6 +31,7 @@ module; #include #include #include +#include #define __user /* empty */ // for xfs includes, below @@ -695,7 +696,7 @@ append_challenged_posix_file_impl::~append_challenged_posix_file_impl() { // // It is safe to destory it if nothing is queued. // Note that posix_file_impl::~posix_file_impl auto-closes the file descriptor. - assert(_q.empty() && (_logical_size == _committed_size || _closing_state == state::closed)); + SEASTAR_ASSERT(_q.empty() && (_logical_size == _committed_size || _closing_state == state::closed)); } bool diff --git a/src/core/fsqual.cc b/src/core/fsqual.cc index 72f78fdae5..3a129a26da 100644 --- a/src/core/fsqual.cc +++ b/src/core/fsqual.cc @@ -20,6 +20,7 @@ */ #include +#include #include #include #include @@ -80,7 +81,7 @@ bool filesystem_has_good_aio_support(sstring directory, bool verbose) { with_ctxsw_counting(ctxsw, [&] { auto r = io_submit(ioctx, 1, cmds); throw_system_error_on(r == -1, "io_submit"); - assert(r == 1); + SEASTAR_ASSERT(r == 1); }); struct io_event ioev; int n = -1; @@ -88,9 +89,9 @@ bool filesystem_has_good_aio_support(sstring directory, bool verbose) { n = io_getevents(ioctx, 1, 1, &ioev, nullptr); throw_system_error_on((n == -1) && (errno != EINTR) , "io_getevents"); } while (n == -1); - assert(n == 1); + SEASTAR_ASSERT(n == 1); throw_kernel_error(long(ioev.res)); - assert(long(ioev.res) == bufsize); + SEASTAR_ASSERT(long(ioev.res) == bufsize); } auto rate = float(ctxsw) / nr; bool ok = rate < 0.1; diff --git a/src/core/fstream.cc b/src/core/fstream.cc index 7aec494110..6a0f8ed040 100644 --- a/src/core/fstream.cc +++ b/src/core/fstream.cc @@ -27,10 +27,10 @@ module; #include #include #include -#include #include #include #include +#include #ifdef SEASTAR_MODULE module seastar; @@ -220,7 +220,7 @@ class file_data_source_impl : public data_source_impl { virtual ~file_data_source_impl() override { // If the data source hasn't been closed, we risk having reads in progress // that will try to access freed memory. - assert(_reads_in_progress == 0); + SEASTAR_ASSERT(_reads_in_progress == 0); } virtual future> get() override { if (!_read_buffers.empty() && !_read_buffers.front()._ready.available()) { @@ -242,7 +242,7 @@ class file_data_source_impl : public data_source_impl { uint64_t dropped = 0; while (n) { if (_read_buffers.empty()) { - assert(n <= _remain); + SEASTAR_ASSERT(n <= _remain); _pos += n; _remain -= n; break; @@ -429,7 +429,7 @@ class file_data_sink_impl : public data_sink_impl { // put() must usually be of chunks multiple of file::dma_alignment. // Only the last part can have an unaligned length. If put() was // called again with an unaligned pos, we have a bug in the caller. - assert(!(pos & (_file.disk_write_dma_alignment() - 1))); + SEASTAR_ASSERT(!(pos & (_file.disk_write_dma_alignment() - 1))); bool truncate = false; auto p = static_cast(buf.get()); size_t buf_size = buf.size(); diff --git a/src/core/future.cc b/src/core/future.cc index 10cfc13541..3e4fb27f68 100644 --- a/src/core/future.cc +++ b/src/core/future.cc @@ -21,7 +21,6 @@ #ifdef SEASTAR_MODULE module; -#include #include #include #include @@ -34,6 +33,7 @@ module seastar; #include #include #endif +#include namespace seastar { @@ -91,12 +91,12 @@ promise_base::promise_base(promise_base&& x) noexcept { void promise_base::clear() noexcept { if (__builtin_expect(bool(_task), false)) { - assert(_state && !_state->available()); + SEASTAR_ASSERT(_state && !_state->available()); set_to_broken_promise(*_state); ::seastar::schedule(std::exchange(_task, nullptr)); } if (_future) { - assert(_state); + SEASTAR_ASSERT(_state); if (!_state->available()) { set_to_broken_promise(*_state); } @@ -167,7 +167,7 @@ void future_state_base::ignore() noexcept { case state::invalid: case state::future: case state::result_unavailable: - assert(0 && "invalid state for ignore"); + SEASTAR_ASSERT(0 && "invalid state for ignore"); case state::result: _u.st = state::result_unavailable; break; @@ -244,7 +244,7 @@ void reactor::test::with_allow_abandoned_failed_futures(unsigned count, noncopya seastar_logger.set_level(log_level::error); func(); auto after = engine()._abandoned_failed_futures; - assert(after - before == count); + SEASTAR_ASSERT(after - before == count); engine()._abandoned_failed_futures = before; seastar_logger.set_level(old_level); } @@ -268,7 +268,7 @@ class thread_wake_task final : public task { void internal::future_base::do_wait() noexcept { auto thread = thread_impl::get(); - assert(thread); + SEASTAR_ASSERT(thread); thread_wake_task wake_task{thread}; wake_task.make_backtrace(); _promise->set_task(&wake_task); @@ -276,7 +276,7 @@ void internal::future_base::do_wait() noexcept { } void internal::future_base::set_coroutine(task& coroutine) noexcept { - assert(_promise); + SEASTAR_ASSERT(_promise); _promise->set_task(&coroutine); } diff --git a/src/core/io_queue.cc b/src/core/io_queue.cc index 353abc817d..25219baafc 100644 --- a/src/core/io_queue.cc +++ b/src/core/io_queue.cc @@ -23,7 +23,6 @@ module; #endif -#include #include #include #include @@ -34,6 +33,7 @@ module; #include #include #include +#include #ifdef SEASTAR_MODULE module seastar; @@ -478,7 +478,7 @@ std::vector io_request::split_iovec(size_t max_length) { } if (vecs.size() > 0) { - assert(remaining < max_length); + SEASTAR_ASSERT(remaining < max_length); auto req = sub_req_iovec(pos, vecs); parts.push_back({ std::move(req), max_length - remaining, std::move(vecs) }); } @@ -668,7 +668,7 @@ io_queue::~io_queue() { // // And that will happen only when there are no more fibers to run. If we ever change // that, then this has to change. - assert(_queued_requests == 0); + SEASTAR_ASSERT(_queued_requests == 0); for (auto&& pc_data : _priority_classes) { if (pc_data) { for (auto&& s : _streams) { diff --git a/src/core/memory.cc b/src/core/memory.cc index 85d9b13ccc..aed37ba138 100644 --- a/src/core/memory.cc +++ b/src/core/memory.cc @@ -73,7 +73,6 @@ module; #endif -#include #include #include #include @@ -81,6 +80,8 @@ module; #include #include +#include + #include #include @@ -93,7 +94,6 @@ module; #include #include #include -#include #include #include #include @@ -129,7 +129,7 @@ module seastar; #endif #ifdef SEASTAR_DEBUG -#define dassert(expr) assert(expr) +#define dassert(expr) SEASTAR_ASSERT(expr) #else #define dassert(expr) do {} while(false) #endif @@ -146,7 +146,7 @@ void* internal::allocate_aligned_buffer_impl(size_t size, size_t align) { } else if (r == EINVAL) { throw std::runtime_error(format("Invalid alignment of {:d}; allocating {:d} bytes", align, size)); } else { - assert(r == 0); + SEASTAR_ASSERT(r == 0); return ret; } } @@ -346,8 +346,8 @@ static char* mem_base() { ::munmap(known + mem_base_alloc, cr + 2 * mem_base_alloc - (known + mem_base_alloc)); // extremely unlikely for mmap to return a mapping at 0, but our detection of free(null) // depends on it not doing that so check it - assert(known != nullptr); - assert(reinterpret_cast(known) != 0); + SEASTAR_ASSERT(known != nullptr); + SEASTAR_ASSERT(reinterpret_cast(known) != 0); }); return known; } @@ -713,7 +713,7 @@ cpu_pages::link(page_list& list, page* span) { } void cpu_pages::free_span_no_merge(uint32_t span_start, uint32_t nr_pages) { - assert(nr_pages); + SEASTAR_ASSERT(nr_pages); nr_free_pages += nr_pages; auto span = &pages[span_start]; auto span_end = &pages[span_start + nr_pages - 1]; @@ -1170,7 +1170,7 @@ cpu_pages::do_foreign_free(void* ptr) { } void cpu_pages::shrink(void* ptr, size_t new_size) { - assert(object_cpu_id(ptr) == cpu_id); + SEASTAR_ASSERT(object_cpu_id(ptr) == cpu_id); page* span = to_page(ptr); if (span->pool) { return; @@ -1214,7 +1214,7 @@ bool cpu_pages::initialize() { cpu_id = cpu_id_gen.fetch_add(1, std::memory_order_relaxed); local_expected_cpu_id = (static_cast(cpu_id) << cpu_id_shift) | reinterpret_cast(mem_base()); - assert(cpu_id < max_cpus); + SEASTAR_ASSERT(cpu_id < max_cpus); all_cpus[cpu_id] = this; auto base = mem_base() + (size_t(cpu_id) << cpu_id_shift); auto size = 32 << 20; // Small size for bootstrap @@ -1361,7 +1361,7 @@ void cpu_pages::schedule_reclaim() { } memory::memory_layout cpu_pages::memory_layout() { - assert(is_initialized()); + SEASTAR_ASSERT(is_initialized()); return { reinterpret_cast(memory), reinterpret_cast(memory) + nr_pages * page_size @@ -2244,7 +2244,7 @@ void* calloc(size_t nmemb, size_t size) { return nullptr; } auto s1 = __int128(nmemb) * __int128(size); - assert(s1 == size_t(s1)); + SEASTAR_ASSERT(s1 == size_t(s1)); size_t s = s1; auto p = malloc(s); if (p) { diff --git a/src/core/metrics.cc b/src/core/metrics.cc index 92fe0dc4d7..e52de13756 100644 --- a/src/core/metrics.cc +++ b/src/core/metrics.cc @@ -26,6 +26,7 @@ module; #include #include #include +#include #include #include #include diff --git a/src/core/posix.cc b/src/core/posix.cc index cf2e95f8d9..a6c5820faa 100644 --- a/src/core/posix.cc +++ b/src/core/posix.cc @@ -24,7 +24,6 @@ module; #endif #include -#include #include #include #include @@ -41,6 +40,7 @@ module seastar; #include #include #endif +#include namespace seastar { @@ -150,11 +150,11 @@ posix_thread::posix_thread(posix_thread&& x) } posix_thread::~posix_thread() { - assert(!_valid); + SEASTAR_ASSERT(!_valid); } void posix_thread::join() { - assert(_valid); + SEASTAR_ASSERT(_valid); pthread_join(_pthread, NULL); _valid = false; } @@ -162,7 +162,7 @@ void posix_thread::join() { std::set get_current_cpuset() { cpu_set_t cs; auto r = pthread_getaffinity_np(pthread_self(), sizeof(cs), &cs); - assert(r == 0); + SEASTAR_ASSERT(r == 0); std::set ret; unsigned nr = CPU_COUNT(&cs); for (int cpu = 0; cpu < CPU_SETSIZE && ret.size() < nr; cpu++) { diff --git a/src/core/prometheus.cc b/src/core/prometheus.cc index 642e041b6f..53bf093f59 100644 --- a/src/core/prometheus.cc +++ b/src/core/prometheus.cc @@ -37,6 +37,7 @@ #include #include #include +#include #include #include #include @@ -883,7 +884,7 @@ class metrics_handler : public httpd::handler_base { // This assert is obviously true. It is in here just to // silence a bogus gcc warning: // https://gcc.gnu.org/bugzilla/show_bug.cgi?id=89337 - assert(name.length() >= 3); + SEASTAR_ASSERT(name.length() >= 3); name.resize(name.length() - 3); return true; } diff --git a/src/core/reactor.cc b/src/core/reactor.cc index 95e5d1eb24..0fa5e23df4 100644 --- a/src/core/reactor.cc +++ b/src/core/reactor.cc @@ -24,7 +24,6 @@ module; #endif #include -#include #include #include #include @@ -182,6 +181,7 @@ module seastar; #include #endif #endif // SEASTAR_MODULE +#include namespace seastar { @@ -236,7 +236,7 @@ seastar::logger seastar_logger("seastar"); seastar::logger sched_logger("scheduler"); shard_id reactor::cpu_id() const { - assert(_id == this_shard_id()); + SEASTAR_ASSERT(_id == this_shard_id()); return _id; } @@ -413,7 +413,7 @@ reactor::do_recv_some(pollable_fd_state& fd, internal::buffer_allocator* ba) { future<> reactor::send_all(pollable_fd_state& fd, const void* buffer, size_t len) { - assert(len); + SEASTAR_ASSERT(len); return send_all_part(fd, buffer, len, 0); } @@ -773,7 +773,7 @@ class backtrace_buffer { } void reserve(size_t len) noexcept { - assert(len < _max_size); + SEASTAR_ASSERT(len < _max_size); if (_pos + len >= _max_size) { flush(); } @@ -885,8 +885,8 @@ auto install_signal_handler_stack() { #endif static sstring shorten_name(const sstring& name, size_t length) { - assert(!name.empty()); - assert(length > 0); + SEASTAR_ASSERT(!name.empty()); + SEASTAR_ASSERT(length > 0); namespace ba = boost::algorithm; using split_iter_t = ba::split_iterator; @@ -908,7 +908,7 @@ static sstring shorten_name(const sstring& name, size_t length) { output != last && split_it != split_last; ++split_it) { auto& part = *split_it; - assert(part.size() > 0); + SEASTAR_ASSERT(part.size() > 0); // convert "hello_world" to "hw" *output++ = part[0]; } @@ -965,7 +965,7 @@ reactor::task_queue::register_stats() { void reactor::task_queue::rename(sstring new_name, sstring new_shortname) { - assert(!new_name.empty()); + SEASTAR_ASSERT(!new_name.empty()); if (_name != new_name) { _name = new_name; if (new_shortname.empty()) { @@ -1041,7 +1041,7 @@ reactor::reactor(std::shared_ptr smp, alien::instance& alien, unsigned id, sigemptyset(&mask); sigaddset(&mask, internal::cpu_stall_detector::signal_number()); auto r = ::pthread_sigmask(SIG_UNBLOCK, &mask, NULL); - assert(r == 0); + SEASTAR_ASSERT(r == 0); memory::set_reclaim_hook([this] (std::function reclaim_fn) { add_high_priority_task(make_task(default_scheduling_group(), [fn = std::move(reclaim_fn)] { fn(); @@ -1054,7 +1054,7 @@ reactor::~reactor() { sigemptyset(&mask); sigaddset(&mask, internal::cpu_stall_detector::signal_number()); auto r = ::pthread_sigmask(SIG_BLOCK, &mask, NULL); - assert(r == 0); + SEASTAR_ASSERT(r == 0); _backend->stop_tick(); auto eraser = [](auto& list) { @@ -2427,7 +2427,7 @@ void reactor::del_timer(timer* tmr) noexcept { } void reactor::at_exit(noncopyable_function ()> func) { - assert(!_stopping); + SEASTAR_ASSERT(!_stopping); _exit_funcs.push_back(std::move(func)); } @@ -2440,7 +2440,7 @@ future<> reactor::run_exit_tasks() { } void reactor::stop() { - assert(_id == 0); + SEASTAR_ASSERT(_id == 0); _smp->cleanup_cpu(); if (!std::exchange(_stopping, true)) { // Run exit tasks locally and then stop all other engines @@ -2939,7 +2939,7 @@ reactor::wakeup() { uint64_t one = 1; auto res = ::write(_notify_eventfd.get(), &one, sizeof(one)); - assert(res == sizeof(one) && "write(2) failed on _reactor._notify_eventfd"); + SEASTAR_ASSERT(res == sizeof(one) && "write(2) failed on _reactor._notify_eventfd"); } void reactor::start_aio_eventfd_loop() { @@ -2965,7 +2965,7 @@ void reactor::stop_aio_eventfd_loop() { } uint64_t one = 1; auto res = ::write(_aio_eventfd->get_fd(), &one, 8); - assert(res == 8 && "write(2) failed on _reactor._aio_eventfd"); + SEASTAR_ASSERT(res == 8 && "write(2) failed on _reactor._aio_eventfd"); } inline @@ -3232,7 +3232,7 @@ int reactor::do_run() { sa_block_notifier.sa_handler = &reactor::block_notifier; sa_block_notifier.sa_flags = SA_RESTART; auto r = sigaction(internal::cpu_stall_detector::signal_number(), &sa_block_notifier, nullptr); - assert(r == 0); + SEASTAR_ASSERT(r == 0); bool idle = false; @@ -3682,14 +3682,14 @@ readable_eventfd writeable_eventfd::read_side() { } file_desc writeable_eventfd::try_create_eventfd(size_t initial) { - assert(size_t(int(initial)) == initial); + SEASTAR_ASSERT(size_t(int(initial)) == initial); return file_desc::eventfd(initial, EFD_CLOEXEC); } void writeable_eventfd::signal(size_t count) { uint64_t c = count; auto r = _fd.write(&c, sizeof(c)); - assert(r == sizeof(c)); + SEASTAR_ASSERT(r == sizeof(c)); } writeable_eventfd readable_eventfd::write_side() { @@ -3697,7 +3697,7 @@ writeable_eventfd readable_eventfd::write_side() { } file_desc readable_eventfd::try_create_eventfd(size_t initial) { - assert(size_t(int(initial)) == initial); + SEASTAR_ASSERT(size_t(int(initial)) == initial); return file_desc::eventfd(initial, EFD_CLOEXEC | EFD_NONBLOCK); } @@ -3705,7 +3705,7 @@ future readable_eventfd::wait() { return engine().readable(*_fd._s).then([this] { uint64_t count; int r = ::read(_fd.get_fd(), &count, sizeof(count)); - assert(r == sizeof(count)); + SEASTAR_ASSERT(r == sizeof(count)); return make_ready_future(count); }); } @@ -3915,13 +3915,13 @@ void smp::arrive_at_event_loop_end() { } void smp::allocate_reactor(unsigned id, reactor_backend_selector rbs, reactor_config cfg) { - assert(!reactor_holder); + SEASTAR_ASSERT(!reactor_holder); // we cannot just write "local_engin = new reactor" since reactor's constructor // uses local_engine void *buf; int r = posix_memalign(&buf, cache_line_size, sizeof(reactor)); - assert(r == 0); + SEASTAR_ASSERT(r == 0); *internal::this_shard_id_ptr() = id; local_engine = new (buf) reactor(this->shared_from_this(), _alien, id, std::move(rbs), cfg); reactor_holder.reset(local_engine); @@ -4491,7 +4491,7 @@ void smp::configure(const smp_options& smp_opts, const reactor_options& reactor_ auto assign_io_queues = [&ioq_topology] (shard_id shard) { for (auto& [dev, io_info] : ioq_topology) { auto queue = std::move(io_info.queues[shard]); - assert(queue); + SEASTAR_ASSERT(queue); engine()._io_queues.emplace(dev, std::move(queue)); auto num_io_groups = io_info.groups.size(); @@ -5032,7 +5032,7 @@ create_scheduling_group(sstring name, sstring shortname, float shares) noexcept return make_exception_future(std::runtime_error(fmt::format("Scheduling group limit exceeded while creating {}", name))); } auto id = static_cast(aid); - assert(id < max_scheduling_groups()); + SEASTAR_ASSERT(id < max_scheduling_groups()); auto sg = scheduling_group(id); return smp::invoke_on_all([sg, name, shortname, shares] { return engine().init_scheduling_group(sg, name, shortname, shares); diff --git a/src/core/reactor_backend.cc b/src/core/reactor_backend.cc index d2369e19e8..5162fcb8dc 100644 --- a/src/core/reactor_backend.cc +++ b/src/core/reactor_backend.cc @@ -23,7 +23,6 @@ module; #endif #include -#include #include #include #include @@ -36,6 +35,7 @@ module; #include #include #include +#include #ifdef SEASTAR_HAVE_URING #include @@ -302,7 +302,7 @@ bool aio_storage_context::reap_completions(bool allow_retry) if (n == -1 && errno == EINTR) { n = 0; } - assert(n >= 0); + SEASTAR_ASSERT(n >= 0); for (size_t i = 0; i < size_t(n); ++i) { auto iocb = get_iocb(_ev_buffer[i]); if (_ev_buffer[i].res == -EAGAIN && allow_retry) { @@ -339,7 +339,7 @@ aio_general_context::~aio_general_context() { } void aio_general_context::queue(linux_abi::iocb* iocb) { - assert(last < end); + SEASTAR_ASSERT(last < end); *last++ = iocb; } @@ -360,7 +360,7 @@ size_t aio_general_context::flush() { // allow retrying for 1 second retry_until = clock::now() + 1s; } else { - assert(clock::now() < retry_until); + SEASTAR_ASSERT(clock::now() < retry_until); } } auto nr = last - iocbs.get(); @@ -461,7 +461,7 @@ void preempt_io_context::reset_preemption_monitor() { bool preempt_io_context::service_preempting_io() { linux_abi::io_event a[2]; auto r = io_getevents(_context.io_context, 0, 2, a, 0); - assert(r != -1); + SEASTAR_ASSERT(r != -1); bool did_work = r > 0; for (unsigned i = 0; i != unsigned(r); ++i) { auto desc = get_user_data(a[i]); @@ -500,7 +500,7 @@ bool reactor_backend_aio::await_events(int timeout, const sigset_t* active_sigma if (r == -1 && errno == EINTR) { return true; } - assert(r != -1); + SEASTAR_ASSERT(r != -1); for (unsigned i = 0; i != unsigned(r); ++i) { did_work = true; auto& event = batch[i]; @@ -534,7 +534,7 @@ reactor_backend_aio::reactor_backend_aio(reactor& r) sigset_t mask = make_sigset_mask(hrtimer_signal()); auto e = ::pthread_sigmask(SIG_BLOCK, &mask, NULL); - assert(e == 0); + SEASTAR_ASSERT(e == 0); } bool reactor_backend_aio::reap_kernel_completions() { @@ -772,7 +772,7 @@ reactor_backend_epoll::task_quota_timer_thread_fn() { pfds[1].fd = _steady_clock_timer_timer_thread.get(); pfds[1].events = POLL_IN; int r = poll(pfds, 2, -1); - assert(r != -1); + SEASTAR_ASSERT(r != -1); uint64_t events; if (pfds[0].revents & POLL_IN) { @@ -852,7 +852,7 @@ reactor_backend_epoll::wait_and_process(int timeout, const sigset_t* active_sigm if (nr == -1 && errno == EINTR) { return false; // gdb can cause this } - assert(nr != -1); + SEASTAR_ASSERT(nr != -1); for (int i = 0; i < nr; ++i) { auto& evt = eevt[i]; auto pfd = reinterpret_cast(evt.data.ptr); @@ -1004,7 +1004,7 @@ future<> reactor_backend_epoll::get_epoll_future(pollable_fd_state& pfd, int eve eevt.events = pfd.events_epoll; eevt.data.ptr = &pfd; int r = ::epoll_ctl(_epollfd.get(), ctl, pfd.fd.get(), &eevt); - assert(r == 0); + SEASTAR_ASSERT(r == 0); _need_epoll_events = true; } @@ -1244,7 +1244,7 @@ class reactor_backend_uring final : public reactor_backend { // Note: for hrtimer_completion we can have spurious wakeups, // since we wait for this using both _preempt_io_context and the // ring. So don't assert that we read anything. - assert(!ret || *ret == 8); + SEASTAR_ASSERT(!ret || *ret == 8); _armed = false; } void maybe_rearm(reactor_backend_uring& be) { diff --git a/src/core/resource.cc b/src/core/resource.cc index d6364d6fa0..12593cb810 100644 --- a/src/core/resource.cc +++ b/src/core/resource.cc @@ -36,6 +36,7 @@ module; #include #include #include +#include #if SEASTAR_HAVE_HWLOC #include #endif @@ -332,7 +333,7 @@ static size_t alloc_from_node(cpu& this_cpu, hwloc_obj_t node, std::unordered_ma if (taken) { used_mem[node] += taken; auto node_id = hwloc_bitmap_first(node->nodeset); - assert(node_id != -1); + SEASTAR_ASSERT(node_id != -1); this_cpu.mem.push_back({taken, unsigned(node_id)}); } return taken; @@ -432,7 +433,7 @@ allocate_io_queues(hwloc_topology_t topology, std::vector cpus, std::unorde if (num_io_groups == 0) { num_io_groups = numa_nodes.size(); - assert(num_io_groups != 0); + SEASTAR_ASSERT(num_io_groups != 0); seastar_logger.debug("Auto-configure {} IO groups", num_io_groups); } else if (num_io_groups > cpus.size()) { // User may be playing with --smp option, but num_io_groups was independently @@ -449,7 +450,7 @@ allocate_io_queues(hwloc_topology_t topology, std::vector cpus, std::unorde } idx++; } - assert(0); + SEASTAR_ASSERT(0); }; auto cpu_sets = distribute_objects(topology, num_io_groups); @@ -588,7 +589,7 @@ resources allocate(configuration& c) { throw std::runtime_error("number of processing units must be positive"); } auto machine_depth = hwloc_get_type_depth(topology, HWLOC_OBJ_MACHINE); - assert(hwloc_get_nbobjs_by_depth(topology, machine_depth) == 1); + SEASTAR_ASSERT(hwloc_get_nbobjs_by_depth(topology, machine_depth) == 1); auto machine = hwloc_get_obj_by_depth(topology, machine_depth, 0); auto available_memory = get_memory_from_hwloc_obj(machine); if (!available_memory) { @@ -614,7 +615,7 @@ resources allocate(configuration& c) { for (auto&& cs : cpu_sets()) { auto cpu_id = hwloc_bitmap_first(cs); - assert(cpu_id != -1); + SEASTAR_ASSERT(cpu_id != -1); auto pu = hwloc_get_pu_obj_by_os_index(topology, cpu_id); auto node = get_numa_node_for_pu(topology, pu); if (node == nullptr) { @@ -625,7 +626,7 @@ resources allocate(configuration& c) { // the system as a single-node configuration. While this code supports // multi-node setups, the fallback behavior is safe and will function // correctly in this case. - assert(num_nodes == 1); + SEASTAR_ASSERT(num_nodes == 1); auto local_memory = get_machine_memory_from_sysconf(); set_memory_to_hwloc_obj(node, local_memory); seastar_logger.warn("hwloc failed to detect NUMA node memory size, using memory size fetched from sysfs"); @@ -686,7 +687,7 @@ resources allocate(configuration& c) { // Divide local memory to cpus for (auto&& cs : cpu_sets()) { auto cpu_id = hwloc_bitmap_first(cs); - assert(cpu_id != -1); + SEASTAR_ASSERT(cpu_id != -1); auto node = cpu_to_node.at(cpu_id); cpu this_cpu; this_cpu.cpu_id = cpu_id; @@ -709,7 +710,7 @@ resources allocate(configuration& c) { if (obj == node) break; } - assert(!remain); + SEASTAR_ASSERT(!remain); ret.cpus.push_back(std::move(this_cpu)); } diff --git a/src/core/scollectd.cc b/src/core/scollectd.cc index 26110ce741..8eb39f2792 100644 --- a/src/core/scollectd.cc +++ b/src/core/scollectd.cc @@ -24,7 +24,6 @@ module; #endif #include -#include #include #include #include @@ -48,6 +47,7 @@ module seastar; #include "core/scollectd-impl.hh" #endif +#include namespace seastar { @@ -233,7 +233,7 @@ struct cpwriter { write(v.ui()); // unsigned int 64, big endian break; default: - assert(0); + SEASTAR_ASSERT(0); } } cpwriter & write(const sstring & s) { diff --git a/src/core/systemwide_memory_barrier.cc b/src/core/systemwide_memory_barrier.cc index 65fa8f067f..71493c064c 100644 --- a/src/core/systemwide_memory_barrier.cc +++ b/src/core/systemwide_memory_barrier.cc @@ -25,7 +25,6 @@ module; #include #include -#include #include #if SEASTAR_HAS_MEMBARRIER @@ -42,6 +41,7 @@ module seastar; #include #include #endif +#include namespace seastar { @@ -90,7 +90,7 @@ systemwide_memory_barrier() { PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0) ; - assert(mem != MAP_FAILED); + SEASTAR_ASSERT(mem != MAP_FAILED); // If the user specified --lock-memory, then madvise() below will fail // with EINVAL, so we unlock here: @@ -98,7 +98,7 @@ systemwide_memory_barrier() { // munlock may fail on old kernels if we don't have permission. That's not // a problem, since if we don't have permission to unlock, we didn't have // permissions to lock. - assert(r == 0 || errno == EPERM); + SEASTAR_ASSERT(r == 0 || errno == EPERM); return reinterpret_cast(mem); }(); @@ -108,7 +108,7 @@ systemwide_memory_barrier() { // a side effect of executing a memory barrier on those threads // FIXME: does this work on ARM? int r2 = madvise(mem, getpagesize(), MADV_DONTNEED); - assert(r2 == 0); + SEASTAR_ASSERT(r2 == 0); } struct alignas(cache_line_size) aligned_flag { diff --git a/src/core/thread.cc b/src/core/thread.cc index 228cea2c66..2b8c00cd1d 100644 --- a/src/core/thread.cc +++ b/src/core/thread.cc @@ -44,6 +44,7 @@ module seastar; #include #include #include +#include #endif /// \cond internal @@ -196,7 +197,7 @@ thread_context::thread_context(thread_attributes attr, noncopyable_function -#include #include #include #include @@ -37,6 +36,7 @@ module seastar; #include #include "core/thread_pool.hh" #endif +#include namespace seastar { @@ -53,7 +53,7 @@ void thread_pool::work(sstring name) { while (true) { uint64_t count; auto r = ::read(inter_thread_wq._start_eventfd.get_read_fd(), &count, sizeof(count)); - assert(r == sizeof(count)); + SEASTAR_ASSERT(r == sizeof(count)); if (_stopped.load(std::memory_order_relaxed)) { break; } @@ -71,7 +71,7 @@ void thread_pool::work(sstring name) { if (_main_thread_idle.load(std::memory_order_relaxed)) { uint64_t one = 1; auto res = ::write(_reactor._notify_eventfd.get(), &one, 8); - assert(res == 8 && "write(2) failed on _reactor._notify_eventfd"); + SEASTAR_ASSERT(res == 8 && "write(2) failed on _reactor._notify_eventfd"); } } } diff --git a/src/core/vla.hh b/src/core/vla.hh index fb9bcba0dd..5a128f65ff 100644 --- a/src/core/vla.hh +++ b/src/core/vla.hh @@ -22,8 +22,8 @@ #pragma once #include +#include #include -#include namespace seastar { @@ -47,7 +47,7 @@ make_struct_with_vla(E S::*last, size_t nr) { auto fake = reinterpret_cast(0); size_t offset = reinterpret_cast(&(fake->*last)); size_t element_size = sizeof((fake->*last)[0]); - assert(offset == sizeof(S)); + SEASTAR_ASSERT(offset == sizeof(S)); auto p = std::unique_ptr( reinterpret_cast(::malloc(offset + element_size * nr))); auto s = std::unique_ptr(new (p.get()) S()); diff --git a/src/http/request.cc b/src/http/request.cc index cc84e8e2c7..aace53f29d 100644 --- a/src/http/request.cc +++ b/src/http/request.cc @@ -23,7 +23,6 @@ module; #endif -#include #include #include @@ -34,6 +33,7 @@ module seastar; #include #include #endif +#include namespace seastar { namespace http { @@ -52,7 +52,7 @@ sstring request::format_url() const { } sstring request::request_line() const { - assert(!_version.empty()); + SEASTAR_ASSERT(!_version.empty()); return _method + " " + format_url() + " HTTP/" + _version + "\r\n"; } diff --git a/src/net/dns.cc b/src/net/dns.cc index e40c6e9eed..124b9b3c9a 100644 --- a/src/net/dns.cc +++ b/src/net/dns.cc @@ -26,6 +26,7 @@ #include #include +#include #include #include @@ -703,11 +704,11 @@ dns_resolver::impl::make_hostent(const ::hostent& host) { while (*p != nullptr) { switch (host.h_addrtype) { case AF_INET: - assert(size_t(host.h_length) >= sizeof(in_addr)); + SEASTAR_ASSERT(size_t(host.h_length) >= sizeof(in_addr)); e.addr_list.emplace_back(*reinterpret_cast(*p)); break; case AF_INET6: - assert(size_t(host.h_length) >= sizeof(in6_addr)); + SEASTAR_ASSERT(size_t(host.h_length) >= sizeof(in6_addr)); e.addr_list.emplace_back(*reinterpret_cast(*p)); break; default: @@ -825,7 +826,7 @@ dns_resolver::impl::do_connect(ares_socket_t fd, const sockaddr * addr, socklen_ dns_log.trace("Connect {}({})->{}", fd, int(e.typ), sa); - assert(e.avail == 0); + SEASTAR_ASSERT(e.avail == 0); e.avail = POLLOUT|POLLIN; // until we know otherwise diff --git a/src/net/dpdk.cc b/src/net/dpdk.cc index c18db5cc86..64584c5270 100644 --- a/src/net/dpdk.cc +++ b/src/net/dpdk.cc @@ -58,6 +58,7 @@ module seastar; #include #include #include +#include #include #include #include @@ -293,7 +294,7 @@ class dpdk_xstats { void update_xstats() { auto len = rte_eth_xstats_get(_port_id, _xstats, _len); - assert(len == _len); + SEASTAR_ASSERT(len == _len); } uint64_t get_value(const xstat_id id) { @@ -330,7 +331,7 @@ class dpdk_xstats { void update_xstat_names() { auto len = rte_eth_xstats_get_names(_port_id, _xstat_names, _len); - assert(len == _len); + SEASTAR_ASSERT(len == _len); } void update_offsets() { @@ -509,7 +510,7 @@ class dpdk_device : public device { virtual future<> link_ready() override { return _link_ready_promise.get_future(); } virtual std::unique_ptr init_local_queue(const program_options::option_group& opts, uint16_t qid) override; virtual unsigned hash2qid(uint32_t hash) override { - assert(_redir_table.size()); + SEASTAR_ASSERT(_redir_table.size()); return _redir_table[hash & (_redir_table.size() - 1)]; } uint16_t port_idx() { return _port_idx; } @@ -645,7 +646,7 @@ class dpdk_qp : public net::qp { head->l3_len = oi.ip_hdr_len; if (oi.tso_seg_size) { - assert(oi.needs_ip_csum); + SEASTAR_ASSERT(oi.needs_ip_csum); head->ol_flags |= RTE_MBUF_F_TX_TCP_SEG; head->l4_len = oi.tcp_hdr_len; head->tso_segsz = oi.tso_seg_size; @@ -777,7 +778,7 @@ class dpdk_qp : public net::qp { cur_seg_offset = 0; // FIXME: assert in a fast-path - remove!!! - assert(cur_seg); + SEASTAR_ASSERT(cur_seg); } } } @@ -862,8 +863,8 @@ class dpdk_qp : public net::qp { rte_mbuf* m; - // TODO: assert() in a fast path! Remove me ASAP! - assert(frag.size); + // TODO: SEASTAR_ASSERT() in a fast path! Remove me ASAP! + SEASTAR_ASSERT(frag.size); // Create a HEAD of mbufs' cluster and set the first bytes into it len = do_one_buf(qp, head, base, left_to_set); @@ -1281,8 +1282,8 @@ class dpdk_qp : public net::qp { uint32_t _send(circular_buffer& pb, Func packet_to_tx_buf_p) { if (_tx_burst.size() == 0) { for (auto&& p : pb) { - // TODO: assert() in a fast path! Remove me ASAP! - assert(p.len()); + // TODO: SEASTAR_ASSERT() in a fast path! Remove me ASAP! + SEASTAR_ASSERT(p.len()); tx_buf* buf = packet_to_tx_buf_p(std::move(p)); if (!buf) { @@ -1437,7 +1438,7 @@ class dpdk_qp : public net::qp { int dpdk_device::init_port_start() { - assert(_port_idx < rte_eth_dev_count_avail()); + SEASTAR_ASSERT(_port_idx < rte_eth_dev_count_avail()); rte_eth_dev_info_get(_port_idx, &_dev_info); @@ -1544,7 +1545,7 @@ int dpdk_device::init_port_start() if (_num_queues > 1) { if (_dev_info.reta_size) { // RETA size should be a power of 2 - assert((_dev_info.reta_size & (_dev_info.reta_size - 1)) == 0); + SEASTAR_ASSERT((_dev_info.reta_size & (_dev_info.reta_size - 1)) == 0); // Set the RSS table to the correct size _redir_table.resize(_dev_info.reta_size); @@ -1577,7 +1578,7 @@ int dpdk_device::init_port_start() // all together. If this assumption breaks we need to rework the below logic // by splitting the csum offload feature bit into separate bits for IPv4, // TCP and UDP. - assert(((_dev_info.rx_offload_capa & RTE_ETH_RX_OFFLOAD_IPV4_CKSUM) && + SEASTAR_ASSERT(((_dev_info.rx_offload_capa & RTE_ETH_RX_OFFLOAD_IPV4_CKSUM) && (_dev_info.rx_offload_capa & RTE_ETH_RX_OFFLOAD_UDP_CKSUM) && (_dev_info.rx_offload_capa & RTE_ETH_RX_OFFLOAD_TCP_CKSUM)) || (!(_dev_info.rx_offload_capa & RTE_ETH_RX_OFFLOAD_IPV4_CKSUM) && @@ -1616,7 +1617,7 @@ int dpdk_device::init_port_start() // or not set all together. If this assumption breaks we need to rework the // below logic by splitting the csum offload feature bit into separate bits // for TCP and UDP. - assert(((_dev_info.tx_offload_capa & RTE_ETH_TX_OFFLOAD_UDP_CKSUM) && + SEASTAR_ASSERT(((_dev_info.tx_offload_capa & RTE_ETH_TX_OFFLOAD_UDP_CKSUM) && (_dev_info.tx_offload_capa & RTE_ETH_TX_OFFLOAD_TCP_CKSUM)) || (!(_dev_info.tx_offload_capa & RTE_ETH_TX_OFFLOAD_UDP_CKSUM) && !(_dev_info.tx_offload_capa & RTE_ETH_TX_OFFLOAD_TCP_CKSUM))); @@ -1831,7 +1832,7 @@ bool dpdk_qp::init_rx_mbuf_pool() // for (int i = 0; i < mbufs_per_queue_rx; i++) { rte_mbuf* m = rte_pktmbuf_alloc(_pktmbuf_pool_rx); - assert(m); + SEASTAR_ASSERT(m); _rx_free_bufs.push_back(m); } @@ -2120,14 +2121,14 @@ bool dpdk_qp::rx_gc() (void **)_rx_free_bufs.data(), _rx_free_bufs.size()); - // TODO: assert() in a fast path! Remove me ASAP! - assert(_num_rx_free_segs >= _rx_free_bufs.size()); + // TODO: SEASTAR_ASSERT() in a fast path! Remove me ASAP! + SEASTAR_ASSERT(_num_rx_free_segs >= _rx_free_bufs.size()); _num_rx_free_segs -= _rx_free_bufs.size(); _rx_free_bufs.clear(); - // TODO: assert() in a fast path! Remove me ASAP! - assert((_rx_free_pkts.empty() && !_num_rx_free_segs) || + // TODO: SEASTAR_ASSERT() in a fast path! Remove me ASAP! + SEASTAR_ASSERT((_rx_free_pkts.empty() && !_num_rx_free_segs) || (!_rx_free_pkts.empty() && _num_rx_free_segs)); } } @@ -2240,7 +2241,7 @@ void dpdk_device::set_rss_table() std::unique_ptr dpdk_device::init_local_queue(const program_options::option_group& opts, uint16_t qid) { auto net_opts = dynamic_cast(&opts); - assert(net_opts); + SEASTAR_ASSERT(net_opts); std::unique_ptr qp; if (net_opts->_hugepages) { @@ -2271,8 +2272,8 @@ std::unique_ptr create_dpdk_net_device( { static bool called = false; - assert(!called); - assert(dpdk::eal::initialized); + SEASTAR_ASSERT(!called); + SEASTAR_ASSERT(dpdk::eal::initialized); called = true; diff --git a/src/net/native-stack-impl.hh b/src/net/native-stack-impl.hh index cd50017949..5eea97eccf 100644 --- a/src/net/native-stack-impl.hh +++ b/src/net/native-stack-impl.hh @@ -23,6 +23,7 @@ #include #include +#include #include namespace seastar { @@ -120,10 +121,10 @@ public: virtual future connect(socket_address sa, socket_address local, transport proto = transport::TCP) override { //TODO: implement SCTP - assert(proto == transport::TCP); + SEASTAR_ASSERT(proto == transport::TCP); // FIXME: local is ignored since native stack does not support multiple IPs yet - assert(sa.as_posix_sockaddr().sa_family == AF_INET); + SEASTAR_ASSERT(sa.as_posix_sockaddr().sa_family == AF_INET); _conn = make_lw_shared(_proto.connect(sa)); return _conn->connected().then([conn = _conn]() mutable { diff --git a/src/net/native-stack.cc b/src/net/native-stack.cc index 2e3c9d1e73..68711779d1 100644 --- a/src/net/native-stack.cc +++ b/src/net/native-stack.cc @@ -23,7 +23,6 @@ module; #endif -#include #include #include #include @@ -32,6 +31,8 @@ module; #include #include +#include + #include #include #include @@ -178,7 +179,7 @@ class native_network_stack : public network_stack { virtual future<> initialize() override; static future> create(const program_options::option_group& opts) { auto ns_opts = dynamic_cast(&opts); - assert(ns_opts); + SEASTAR_ASSERT(ns_opts); if (this_shard_id() == 0) { create_native_net_device(*ns_opts); } @@ -231,7 +232,7 @@ native_network_stack::native_network_stack(const native_stack_options& opts, std server_socket native_network_stack::listen(socket_address sa, listen_options opts) { - assert(sa.family() == AF_INET || sa.is_unspecified()); + SEASTAR_ASSERT(sa.family() == AF_INET || sa.is_unspecified()); return tcpv4_listen(_inet.get_tcp(), ntohs(sa.as_posix_sockaddr_in().sin_port), opts); } diff --git a/src/net/net.cc b/src/net/net.cc index e4038ab760..f478cabced 100644 --- a/src/net/net.cc +++ b/src/net/net.cc @@ -40,6 +40,7 @@ module seastar; #include #include #endif +#include namespace seastar { @@ -201,7 +202,7 @@ qp::~qp() { } void qp::configure_proxies(const std::map& cpu_weights) { - assert(!cpu_weights.empty()); + SEASTAR_ASSERT(!cpu_weights.empty()); if ((cpu_weights.size() == 1 && cpu_weights.begin()->first == this_shard_id())) { // special case queue sending to self only, to avoid requiring a hash value return; @@ -244,7 +245,7 @@ device::receive(std::function (packet)> next_packet) { } void device::set_local_queue(std::unique_ptr dev) { - assert(!_queues[this_shard_id()]); + SEASTAR_ASSERT(!_queues[this_shard_id()]); _queues[this_shard_id()] = dev.get(); engine().at_destroy([dev = std::move(dev)] {}); } @@ -295,7 +296,7 @@ interface::register_l3(eth_protocol_num proto_num, std::function (packet p, ethernet_address from)> next, std::function forward) { auto i = _proto_map.emplace(std::piecewise_construct, std::make_tuple(uint16_t(proto_num)), std::forward_as_tuple(std::move(forward))); - assert(i.second); + SEASTAR_ASSERT(i.second); l3_rx_stream& l3_rx = i.first->second; return l3_rx.packet_stream.listen(std::move(next)).done(); } diff --git a/src/net/posix-stack.cc b/src/net/posix-stack.cc index 6c4adb6229..1ca4145963 100644 --- a/src/net/posix-stack.cc +++ b/src/net/posix-stack.cc @@ -23,7 +23,6 @@ module; #endif -#include #include #include #include @@ -39,6 +38,7 @@ module; #include #include #include +#include #ifdef SEASTAR_MODULE module seastar; @@ -184,7 +184,7 @@ class posix_sctp_connected_socket_operations : public posix_connected_socket_ope class posix_unix_stream_connected_socket_operations : public posix_connected_socket_operations { public: virtual void set_nodelay(file_desc& fd, bool nodelay) const override { - assert(nodelay); // make sure nobody actually tries to use this non-existing functionality + SEASTAR_ASSERT(nodelay); // make sure nobody actually tries to use this non-existing functionality } virtual bool get_nodelay(file_desc& fd) const override { return true; @@ -570,7 +570,7 @@ future posix_ap_server_socket_impl::accept() { } else { try { auto i = sockets.emplace(std::piecewise_construct, std::make_tuple(t_sa), std::make_tuple()); - assert(i.second); + SEASTAR_ASSERT(i.second); return i.first->second.get_future(); } catch (...) { return make_exception_future(std::current_exception()); @@ -867,7 +867,7 @@ class posix_datagram_channel : public datagram_channel_impl { } virtual bool is_closed() const override { return _closed; } socket_address local_address() const override { - assert(_address.u.sas.ss_family != AF_INET6 || (_address.addr_length > 20)); + SEASTAR_ASSERT(_address.u.sas.ss_family != AF_INET6 || (_address.addr_length > 20)); return _address; } }; @@ -877,14 +877,14 @@ future<> posix_datagram_channel::send(const socket_address& dst, const char *mes auto a = dst; resolve_outgoing_address(a); return _fd.sendto(a, message, len) - .then([len] (size_t size) { assert(size == len); }); + .then([len] (size_t size) { SEASTAR_ASSERT(size == len); }); } future<> posix_datagram_channel::send(const socket_address& dst, packet p) { auto len = p.len(); _send.prepare(dst, std::move(p)); return _fd.sendmsg(&_send._hdr) - .then([len] (size_t size) { assert(size == len); }); + .then([len] (size_t size) { SEASTAR_ASSERT(size == len); }); } udp_channel diff --git a/src/net/tcp.cc b/src/net/tcp.cc index f61f9d5de0..0fb21406e4 100644 --- a/src/net/tcp.cc +++ b/src/net/tcp.cc @@ -23,7 +23,6 @@ module; #include #include -#include #include #include #include @@ -36,6 +35,7 @@ module seastar; #include #include "net/native-stack-impl.hh" #endif +#include namespace seastar { @@ -124,7 +124,7 @@ uint8_t tcp_option::fill(void* h, const tcp_hdr* th, uint8_t options_size) { eol.write(off); size += option_len::eol; } - assert(size == options_size); + SEASTAR_ASSERT(size == options_size); return size; } diff --git a/src/net/tls.cc b/src/net/tls.cc index 1c96d1a2d1..3aebe81ab1 100644 --- a/src/net/tls.cc +++ b/src/net/tls.cc @@ -32,6 +32,8 @@ module; #include #include +#include + #include #include #include @@ -193,7 +195,7 @@ static auto get_gtls_string = [](auto func, auto... args) noexcept { if (ret != GNUTLS_E_SHORT_MEMORY_BUFFER) { return std::make_pair(ret, sstring{}); } - assert(size != 0); + SEASTAR_ASSERT(size != 0); sstring res(sstring::initialized_later{}, size - 1); ret = func(args..., res.data(), &size); return std::make_pair(ret, res); @@ -1158,7 +1160,7 @@ class session : public enable_lw_shared_from_this { } ~session() { - assert(_output_pending.available()); + SEASTAR_ASSERT(_output_pending.available()); } typedef temporary_buffer buf_type; @@ -1355,7 +1357,7 @@ class session : public enable_lw_shared_from_this { // then extract subject and issuer from the (leaf) peer certificate and invoke the callback auto dn = extract_dn_information(); - assert(dn.has_value()); // otherwise we couldn't have gotten here + SEASTAR_ASSERT(dn.has_value()); // otherwise we couldn't have gotten here // a switch here might look overelaborate, however, // the compiler will warn us if someone alters the definition of type @@ -1444,7 +1446,7 @@ class session : public enable_lw_shared_from_this { typedef net::fragment* frag_iter; future<> do_put(frag_iter i, frag_iter e) { - assert(_output_pending.available()); + SEASTAR_ASSERT(_output_pending.available()); return do_for_each(i, e, [this](net::fragment& f) { auto ptr = f.base; auto size = f.size; @@ -1577,7 +1579,7 @@ class session : public enable_lw_shared_from_this { case GNUTLS_E_AGAIN: // We only send "bye" alert, letting a "normal" (either pending, or subsequent) // read deal with reading the expected EOF alert. - assert(gnutls_record_get_direction(*this) == 1); + SEASTAR_ASSERT(gnutls_record_get_direction(*this) == 1); return wait_for_output().then([this] { return do_shutdown(); }); diff --git a/src/net/virtio.cc b/src/net/virtio.cc index f53c8329a2..1fa248db1b 100644 --- a/src/net/virtio.cc +++ b/src/net/virtio.cc @@ -25,7 +25,6 @@ module; #include #include -#include #include #include #include @@ -36,6 +35,7 @@ module; #include #include #include +#include #ifdef SEASTAR_MODULE module seastar; @@ -358,7 +358,7 @@ template inline unsigned vring::allocate_desc() { - assert(_free_head != -1); + SEASTAR_ASSERT(_free_head != -1); auto desc = _free_head; if (desc == _free_last) { _free_last = _free_head = -1; @@ -631,7 +631,7 @@ qp::txq::post(circular_buffer& pb) { packet q = packet(fragment{reinterpret_cast(&vhdr), _dev._header_len}, std::move(p)); auto fut = _ring.available_descriptors().wait(q.nr_frags()); - assert(fut.available()); // how it cannot? + SEASTAR_ASSERT(fut.available()); // how it cannot? _packets.emplace_back(packet_as_buffer_chain{ std::move(q) }); } _ring.post(_packets.begin(), _packets.end()); @@ -695,7 +695,7 @@ qp::rxq::complete_buffer(single_buffer&& bc, size_t len) { // First buffer if (_remaining_buffers == 0) { auto hdr = reinterpret_cast(frag_buf); - assert(hdr->num_buffers >= 1); + SEASTAR_ASSERT(hdr->num_buffers >= 1); _remaining_buffers = hdr->num_buffers; frag_buf += _dev._header_len; frag_len -= _dev._header_len; @@ -734,7 +734,7 @@ qp::rxq::complete_buffer(single_buffer&& bc, size_t len) { static std::unique_ptr virtio_buffer(size_t size) { void* ret; auto r = posix_memalign(&ret, 4096, size); - assert(r == 0); + SEASTAR_ASSERT(r == 0); bzero(ret, size); return std::unique_ptr(reinterpret_cast(ret)); } @@ -825,7 +825,7 @@ qp_vhost::qp_vhost(device *dev, const native_stack_options& opts) // this fd to VHOST_NET_SET_BACKEND, the Linux kernel keeps the reference // to it and it's fine to close the file descriptor. file_desc tap_fd(file_desc::open("/dev/net/tun", O_RDWR | O_NONBLOCK)); - assert(tap_device.size() + 1 <= IFNAMSIZ); + SEASTAR_ASSERT(tap_device.size() + 1 <= IFNAMSIZ); ifreq ifr = {}; ifr.ifr_flags = IFF_TAP | IFF_NO_PI | IFF_ONE_QUEUE | IFF_VNET_HDR; strcpy(ifr.ifr_ifrn.ifrn_name, tap_device.c_str()); @@ -886,12 +886,12 @@ qp_vhost::qp_vhost(device *dev, const native_stack_options& opts) std::unique_ptr device::init_local_queue(const program_options::option_group& opts, uint16_t qid) { static bool called = false; - assert(!qid); - assert(!called); + SEASTAR_ASSERT(!qid); + SEASTAR_ASSERT(!called); called = true; auto net_opts = dynamic_cast(&opts); - assert(net_opts); + SEASTAR_ASSERT(net_opts); return std::make_unique(this, *net_opts); } diff --git a/src/rpc/lz4_fragmented_compressor.cc b/src/rpc/lz4_fragmented_compressor.cc index 905fe56ba0..4fc50e562b 100644 --- a/src/rpc/lz4_fragmented_compressor.cc +++ b/src/rpc/lz4_fragmented_compressor.cc @@ -21,6 +21,7 @@ #include #include +#include #include // LZ4_DECODER_RING_BUFFER_SIZE macro is introduced since v1.8.2 @@ -140,7 +141,7 @@ snd_buf lz4_fragmented_compressor::compress(size_t head_space, snd_buf data) { auto src_ptr = src->get() + src_current_offset; if (src->size() - src_current_offset < size) { auto left = size; - assert(lin_buf_size > size); + SEASTAR_ASSERT(lin_buf_size > size); if (lin_buf_size - lin_off < size) { lin_off = 0; } diff --git a/src/rpc/rpc.cc b/src/rpc/rpc.cc index 84336a0856..6135f0d43f 100644 --- a/src/rpc/rpc.cc +++ b/src/rpc/rpc.cc @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -239,7 +240,7 @@ namespace rpc { } void connection::withdraw(outgoing_entry::container_t::iterator it, std::exception_ptr ex) { - assert(it != _outgoing_queue.end()); + SEASTAR_ASSERT(it != _outgoing_queue.end()); auto pit = std::prev(it); // Previous entry's (pit's) done future will schedule current entry (it) @@ -573,7 +574,7 @@ namespace rpc { } }); if (eof && !bufs.empty()) { - assert(_stream_queue.empty()); + SEASTAR_ASSERT(_stream_queue.empty()); _stream_queue.push(rcv_buf(-1U)); // push eof marker back for next read to notice it } }); @@ -816,7 +817,7 @@ namespace rpc { void client::abort_all_streams() { while (!_streams.empty()) { auto&& s = _streams.begin(); - assert(s->second->get_owner_shard() == this_shard_id()); // abort can be called only locally + SEASTAR_ASSERT(s->second->get_owner_shard() == this_shard_id()); // abort can be called only locally s->second->get()->abort(); _streams.erase(s); } @@ -1342,14 +1343,14 @@ future<> server::connection::send_unknown_verb_reply(std::optionalprocess(); }); }).then_wrapped([this] (future<>&& f){ try { f.get(); - assert(false); + SEASTAR_ASSERT(false); } catch (...) { _ss_stopped.set_value(); } diff --git a/src/testing/test_runner.cc b/src/testing/test_runner.cc index 5005fdc938..22eb7c1a77 100644 --- a/src/testing/test_runner.cc +++ b/src/testing/test_runner.cc @@ -26,6 +26,7 @@ #include #include #include +#include namespace seastar { @@ -133,7 +134,7 @@ test_runner::run_sync(std::function()> task) { exchanger e; _task.give([task = std::move(task), &e] { - assert(engine_is_ready()); + SEASTAR_ASSERT(engine_is_ready()); try { return task().then_wrapped([&e](auto&& f) { try { diff --git a/src/util/file.cc b/src/util/file.cc index d280860ad7..839b3a5080 100644 --- a/src/util/file.cc +++ b/src/util/file.cc @@ -26,6 +26,7 @@ module; #include #include +#include #include #include #include diff --git a/src/util/log.cc b/src/util/log.cc index acbe25ddd3..896eb48b0c 100644 --- a/src/util/log.cc +++ b/src/util/log.cc @@ -123,6 +123,11 @@ namespace seastar { namespace internal { +[[noreturn]] void assert_fail(const char* msg, const char* file, int line, const char* func) { + printf("%s:%u: %s: Assertion `%s` failed.\n", file, line, func, msg); + __builtin_trap(); +} + void log_buf::free_buffer() noexcept { if (_own_buf) { delete[] _begin; diff --git a/src/util/process.cc b/src/util/process.cc index 561e792b01..06c83ccd8b 100644 --- a/src/util/process.cc +++ b/src/util/process.cc @@ -22,7 +22,6 @@ #ifdef SEASTAR_MODULE module; -#include #include #include #include @@ -38,6 +37,7 @@ module seastar; #include #include #endif +#include namespace seastar::experimental { @@ -119,7 +119,7 @@ future process::wait() { if (WIFEXITED(wstatus)) { return wait_exited{WEXITSTATUS(wstatus)}; } else { - assert(WIFSIGNALED(wstatus)); + SEASTAR_ASSERT(WIFSIGNALED(wstatus)); return wait_signaled{WTERMSIG(wstatus)}; } }); @@ -135,7 +135,7 @@ void process::kill() { future process::spawn(const std::filesystem::path& pathname, spawn_parameters params) { - assert(!params.argv.empty()); + SEASTAR_ASSERT(!params.argv.empty()); return engine().spawn(pathname.native(), std::move(params.argv), std::move(params.env)).then_unpack( [] (pid_t pid, file_desc stdin_pipe, file_desc stdout_pipe, file_desc stderr_pipe) { return make_ready_future(create_tag{}, pid, std::move(stdin_pipe), std::move(stdout_pipe), std::move(stderr_pipe)); diff --git a/src/util/tmp_file.cc b/src/util/tmp_file.cc index d475cf9e09..5901e43a26 100644 --- a/src/util/tmp_file.cc +++ b/src/util/tmp_file.cc @@ -24,6 +24,7 @@ #include #include +#include #include #include #include @@ -47,7 +48,7 @@ generate_tmp_name(const fs::path& path_template) { parent = path_template; filename = default_tmp_name_template; pos = filename.find("XX"); - assert(pos != std::string::npos); + SEASTAR_ASSERT(pos != std::string::npos); } auto end = filename.size(); static constexpr char charset[] = "0123456789abcdef"; @@ -82,13 +83,13 @@ tmp_file::tmp_file(tmp_file&& x) noexcept } tmp_file::~tmp_file() { - assert(!has_path()); - assert(!is_open()); + SEASTAR_ASSERT(!has_path()); + SEASTAR_ASSERT(!is_open()); } future<> tmp_file::open(fs::path path_template, open_flags oflags, file_open_options options) noexcept { - assert(!has_path()); - assert(!is_open()); + SEASTAR_ASSERT(!has_path()); + SEASTAR_ASSERT(!is_open()); oflags |= open_flags::create | open_flags::exclusive; fs::path path; try { @@ -132,11 +133,11 @@ make_tmp_file(fs::path path_template, open_flags oflags, file_open_options optio } tmp_dir::~tmp_dir() { - assert(!has_path()); + SEASTAR_ASSERT(!has_path()); } future<> tmp_dir::create(fs::path path_template, file_permissions create_permissions) noexcept { - assert(!has_path()); + SEASTAR_ASSERT(!has_path()); fs::path path; try { path = generate_tmp_name(std::move(path_template)); diff --git a/src/websocket/common.cc b/src/websocket/common.cc index 040529b603..2320181f38 100644 --- a/src/websocket/common.cc +++ b/src/websocket/common.cc @@ -22,6 +22,7 @@ #include #include #include +#include #include #include #include @@ -132,7 +133,7 @@ future<> connection::read_one() { std::string sha1_base64(std::string_view source) { unsigned char hash[20]; - assert(sizeof(hash) == gnutls_hash_get_len(GNUTLS_DIG_SHA1)); + SEASTAR_ASSERT(sizeof(hash) == gnutls_hash_get_len(GNUTLS_DIG_SHA1)); if (int ret = gnutls_hash_fast(GNUTLS_DIG_SHA1, source.data(), source.size(), hash); ret != GNUTLS_E_SUCCESS) { throw websocket::exception(fmt::format("gnutls_hash_fast: {}", gnutls_strerror(ret))); diff --git a/src/websocket/parser.cc b/src/websocket/parser.cc index 8de5fa753c..e4f0e0a436 100644 --- a/src/websocket/parser.cc +++ b/src/websocket/parser.cc @@ -18,6 +18,7 @@ #include #include +#include namespace seastar::experimental::websocket { @@ -46,7 +47,7 @@ future websocket_parser::operator()( // the rest of code is structured. The else branch will never increase // _buffer.length() to >=2 and other paths to this condition will always // have buffer cleared. - assert(_buffer.length() < 2); + SEASTAR_ASSERT(_buffer.length() < 2); size_t hlen = _buffer.length(); _buffer.append(data.get(), 2 - hlen); diff --git a/tests/perf/linux_perf_event.cc b/tests/perf/linux_perf_event.cc index f36a94e9f9..b4c76c0014 100644 --- a/tests/perf/linux_perf_event.cc +++ b/tests/perf/linux_perf_event.cc @@ -24,9 +24,8 @@ * This file was copied from Scylla (https://github.com/scylladb/scylla) */ -#include - #include +#include #include #include @@ -64,7 +63,7 @@ linux_perf_event::read() { } uint64_t ret; auto res = ::read(_fd, &ret, sizeof(ret)); - assert(res == sizeof(ret) && "read(2) failed on perf_event fd"); + SEASTAR_ASSERT(res == sizeof(ret) && "read(2) failed on perf_event fd"); return ret; } diff --git a/tests/perf/shared_token_bucket.cc b/tests/perf/shared_token_bucket.cc index 38a07d770e..38cbf7b5d8 100644 --- a/tests/perf/shared_token_bucket.cc +++ b/tests/perf/shared_token_bucket.cc @@ -25,6 +25,7 @@ #include #include #include +#include #include #include @@ -155,7 +156,7 @@ struct worker : public seastar::peering_sharded_service> { } future work(std::function(std::chrono::duration d)> do_sleep) { - assert(tokens == 0); + SEASTAR_ASSERT(tokens == 0); auto start = clock_type::now(); // Run for 1 second. The perf suite would restart this method several times return do_until([end = start + std::chrono::seconds(1)] { return clock_type::now() >= end; }, @@ -267,7 +268,7 @@ struct hog { {} void work() { - assert(!stopped.has_value()); + SEASTAR_ASSERT(!stopped.has_value()); keep_going = true; stopped = do_until([this] { return !keep_going; }, [this] { @@ -283,7 +284,7 @@ struct hog { } future<> terminate() { - assert(stopped.has_value()); + SEASTAR_ASSERT(stopped.has_value()); keep_going = false; auto f = std::move(*stopped); stopped.reset(); diff --git a/tests/unit/allocator_test.cc b/tests/unit/allocator_test.cc index fd0ada76c7..9522d2a7b3 100644 --- a/tests/unit/allocator_test.cc +++ b/tests/unit/allocator_test.cc @@ -23,11 +23,11 @@ #include #include #include +#include #include #include #include #include -#include #include #include #include @@ -47,7 +47,7 @@ struct allocation { allocation(allocation&& x) noexcept = default; void verify() { if (data) { - assert(std::find_if(data.get(), data.get() + n, [this] (char c) { + SEASTAR_ASSERT(std::find_if(data.get(), data.get() + n, [this] (char c) { return c != poison; }) == data.get() + n); } @@ -98,7 +98,7 @@ struct test17_concrete : test17 { static_assert(sizeof(value_type) == N, "language does not guarantee size >= align"); virtual handle alloc() const override { auto ptr = new value_type(); - assert((reinterpret_cast(ptr) & (N - 1)) == 0); + SEASTAR_ASSERT((reinterpret_cast(ptr) & (N - 1)) == 0); return handle{this, ptr}; } virtual void free(void* ptr) const override { diff --git a/tests/unit/circular_buffer_test.cc b/tests/unit/circular_buffer_test.cc index 61a20495cb..ea2d27edde 100644 --- a/tests/unit/circular_buffer_test.cc +++ b/tests/unit/circular_buffer_test.cc @@ -33,6 +33,7 @@ #endif #include +#include using namespace seastar; @@ -79,8 +80,8 @@ BOOST_AUTO_TEST_CASE(test_erasing_at_beginning_or_end_does_not_invalidate_iterat int* ptr_to_3 = &buf[2]; auto iterator_to_3 = buf.begin() + 2; - assert(*ptr_to_3 == 3); - assert(*iterator_to_3 == 3); + SEASTAR_ASSERT(*ptr_to_3 == 3); + SEASTAR_ASSERT(*iterator_to_3 == 3); buf.erase(buf.begin(), buf.begin() + 2); diff --git a/tests/unit/directory_test.cc b/tests/unit/directory_test.cc index ff37014974..006b398300 100644 --- a/tests/unit/directory_test.cc +++ b/tests/unit/directory_test.cc @@ -28,6 +28,7 @@ #include #include #include +#include using namespace seastar; @@ -51,7 +52,7 @@ const char* de_type_desc(directory_entry_type t) case directory_entry_type::socket: return "socket"; } - assert(0 && "should not get here"); + SEASTAR_ASSERT(0 && "should not get here"); return nullptr; } @@ -69,9 +70,9 @@ future<> lister_test() { future<> report(directory_entry de) { return file_stat(de.name, follow_symlink::no).then([de = std::move(de)] (stat_data sd) { if (de.type) { - assert(*de.type == sd.type); + SEASTAR_ASSERT(*de.type == sd.type); } else { - assert(sd.type == directory_entry_type::unknown); + SEASTAR_ASSERT(sd.type == directory_entry_type::unknown); } fmt::print("{} (type={})\n", de.name, de_type_desc(sd.type)); return make_ready_future<>(); @@ -91,9 +92,9 @@ future<> lister_generator_test(file f) { while (auto de = co_await lister()) { auto sd = co_await file_stat(de->name, follow_symlink::no); if (de->type) { - assert(*de->type == sd.type); + SEASTAR_ASSERT(*de->type == sd.type); } else { - assert(sd.type == directory_entry_type::unknown); + SEASTAR_ASSERT(sd.type == directory_entry_type::unknown); } fmt::print("{} (type={})\n", de->name, de_type_desc(sd.type)); } diff --git a/tests/unit/distributed_test.cc b/tests/unit/distributed_test.cc index ead09f6815..2ede720a2f 100644 --- a/tests/unit/distributed_test.cc +++ b/tests/unit/distributed_test.cc @@ -28,6 +28,7 @@ #include #include #include +#include #include #include #include @@ -50,7 +51,7 @@ struct async_service : public seastar::async_sharded_service { }); } virtual void check() { - assert(!deleted); + SEASTAR_ASSERT(!deleted); } future<> stop() { return make_ready_future<>(); } }; @@ -351,8 +352,8 @@ SEASTAR_TEST_CASE(test_smp_service_groups) { bunch1.get(); bunch2.get(); if (smp::count > 1) { - assert(rm1.max_concurrent_observed == 1); - assert(rm2.max_concurrent_observed == 1000); + SEASTAR_ASSERT(rm1.max_concurrent_observed == 1); + SEASTAR_ASSERT(rm2.max_concurrent_observed == 1000); } destroy_smp_service_group(ssg1).get(); destroy_smp_service_group(ssg2).get(); diff --git a/tests/unit/fair_queue_test.cc b/tests/unit/fair_queue_test.cc index 3574f46dd7..d8e50093a0 100644 --- a/tests/unit/fair_queue_test.cc +++ b/tests/unit/fair_queue_test.cc @@ -29,6 +29,7 @@ #include #include #include +#include #include #include #include @@ -169,7 +170,7 @@ class test_env { // // The ratios argument is the ratios towards the first class void verify(sstring name, std::vector ratios, unsigned expected_error = 1) { - assert(ratios.size() == _results.size()); + SEASTAR_ASSERT(ratios.size() == _results.size()); auto str = name + ":"; for (auto i = 0ul; i < _results.size(); ++i) { str += format(" r[{:d}] = {:d}", i, _results[i]); diff --git a/tests/unit/file_io_test.cc b/tests/unit/file_io_test.cc index 3bce589bf3..174315c021 100644 --- a/tests/unit/file_io_test.cc +++ b/tests/unit/file_io_test.cc @@ -36,6 +36,7 @@ #include #include #include +#include #include #include #include @@ -233,7 +234,7 @@ SEASTAR_TEST_CASE(test_iov_max) { while (left) { auto written = f.dma_write(position, iovecs).get(); iovecs.erase(iovecs.begin(), iovecs.begin() + written / buffer_size); - assert(written % buffer_size == 0); + SEASTAR_ASSERT(written % buffer_size == 0); position += written; left -= written; } @@ -252,7 +253,7 @@ SEASTAR_TEST_CASE(test_iov_max) { while (left) { auto read = f.dma_read(position, iovecs).get(); iovecs.erase(iovecs.begin(), iovecs.begin() + read / buffer_size); - assert(read % buffer_size == 0); + SEASTAR_ASSERT(read % buffer_size == 0); position += read; left -= read; } @@ -740,7 +741,7 @@ SEASTAR_TEST_CASE(test_nowait_flag_correctness) { auto is_tmpfs = [&] (sstring filename) { struct ::statfs buf; int fd = ::open(filename.c_str(), static_cast(open_flags::ro)); - assert(fd != -1); + SEASTAR_ASSERT(fd != -1); auto r = ::fstatfs(fd, &buf); if (r == -1) { return false; diff --git a/tests/unit/fstream_test.cc b/tests/unit/fstream_test.cc index d8a53a149a..5203ceb044 100644 --- a/tests/unit/fstream_test.cc +++ b/tests/unit/fstream_test.cc @@ -34,6 +34,7 @@ #include #include #include +#include #include #include #include @@ -161,7 +162,7 @@ SEASTAR_TEST_CASE(test_consume_skip_bytes) { if (_count == 8000) { return make_ready_future(skip_bytes{2000 - buf.size()}); } else { - assert(buf.empty()); + SEASTAR_ASSERT(buf.empty()); return make_ready_future(continue_consuming{}); } return make_ready_future(continue_consuming{}); @@ -453,7 +454,7 @@ SEASTAR_TEST_CASE(test_fstream_slow_start) { mock_file->set_allowed_read_requests(requests_at_full_speed); auto buf = fstr.read().get(); BOOST_CHECK_EQUAL(buf.size(), 0u); - assert(buf.size() == 0); + SEASTAR_ASSERT(buf.size() == 0); }; auto read_while_file_at_full_speed = [&] (auto fstr) { diff --git a/tests/unit/futures_test.cc b/tests/unit/futures_test.cc index 1f79d425a7..c919256b58 100644 --- a/tests/unit/futures_test.cc +++ b/tests/unit/futures_test.cc @@ -45,6 +45,7 @@ #include #include #include +#include #include #include #include @@ -565,8 +566,8 @@ template future<> when_all_but_one_succeed(Container& futures, size_t leave_out) { auto sz = futures.size(); - assert(sz >= 1); - assert(leave_out < sz); + SEASTAR_ASSERT(sz >= 1); + SEASTAR_ASSERT(leave_out < sz); std::vector> all_but_one_tmp; all_but_one_tmp.reserve(sz - 1); for (size_t i = 0 ; i < sz; i++){ diff --git a/tests/unit/httpd_test.cc b/tests/unit/httpd_test.cc index cc8a7e8654..b918a2a8a7 100644 --- a/tests/unit/httpd_test.cc +++ b/tests/unit/httpd_test.cc @@ -26,6 +26,7 @@ #include #include #include +#include #include #include #include @@ -1037,7 +1038,7 @@ SEASTAR_TEST_CASE(test_client_abort_new_conn) { class delayed_factory : public http::experimental::connection_factory { public: virtual future make(abort_source* as) override { - assert(as != nullptr); + SEASTAR_ASSERT(as != nullptr); return sleep_abortable(std::chrono::seconds(1), *as).then([] { return make_exception_future(std::runtime_error("Shouldn't happen")); }); diff --git a/tests/unit/io_queue_test.cc b/tests/unit/io_queue_test.cc index 9edcbc7c0f..a9ea54df46 100644 --- a/tests/unit/io_queue_test.cc +++ b/tests/unit/io_queue_test.cc @@ -33,6 +33,7 @@ #include #include #include +#include #include using namespace seastar; @@ -412,7 +413,7 @@ SEASTAR_TEST_CASE(test_request_iovec_split) { }; auto check_buffer = [&large_buffer] (size_t len, char value) { - assert(len < sizeof(large_buffer)); + SEASTAR_ASSERT(len < sizeof(large_buffer)); bool fill_match = true; bool train_match = true; for (unsigned i = 0; i < sizeof(large_buffer); i++) { @@ -463,12 +464,12 @@ SEASTAR_TEST_CASE(test_request_iovec_split) { ::iovec iov; iov.iov_base = reinterpret_cast(large_buffer + total); iov.iov_len = dice(reng); - assert(iov.iov_len != 0); + SEASTAR_ASSERT(iov.iov_len != 0); total += iov.iov_len; vecs.push_back(std::move(iov)); } - assert(total > 0); + SEASTAR_ASSERT(total > 0); clear_buffer(); bump_buffer(vecs); check_buffer(total, 1); diff --git a/tests/unit/loopback_socket.hh b/tests/unit/loopback_socket.hh index 3b9df90188..3bafd7de20 100644 --- a/tests/unit/loopback_socket.hh +++ b/tests/unit/loopback_socket.hh @@ -32,6 +32,7 @@ #include #include #include +#include namespace seastar { @@ -106,7 +107,7 @@ public: } future<> wait_input_shutdown() { - assert(!_shutdown.has_value()); + SEASTAR_ASSERT(!_shutdown.has_value()); _shutdown.emplace(); return _shutdown->get_future(); } @@ -268,14 +269,14 @@ public: } server_socket get_server_socket() { - assert(this_shard_id() < _shards_count); + SEASTAR_ASSERT(this_shard_id() < _shards_count); if (!_pending[this_shard_id()]) { _pending[this_shard_id()] = make_lw_shared>(_pending_capacity); } return server_socket(std::make_unique(_pending[this_shard_id()])); } future<> make_new_server_connection(foreign_ptr> b1, lw_shared_ptr b2) { - assert(this_shard_id() < _shards_count); + SEASTAR_ASSERT(this_shard_id() < _shards_count); if (!_pending[this_shard_id()]) { _pending[this_shard_id()] = make_lw_shared>(_pending_capacity); } @@ -288,7 +289,7 @@ public: return _shard++ % _shards_count; } void destroy_shard(unsigned shard) { - assert(shard < _shards_count); + SEASTAR_ASSERT(shard < _shards_count); _pending[shard] = nullptr; } future<> destroy_all_shards() { diff --git a/tests/unit/mock_file.hh b/tests/unit/mock_file.hh index 52281b7663..2c5f972123 100644 --- a/tests/unit/mock_file.hh +++ b/tests/unit/mock_file.hh @@ -25,6 +25,7 @@ #include #include +#include namespace seastar { @@ -42,7 +43,7 @@ private: _verify_length(length); } BOOST_CHECK(_allowed_read_requests); - assert(_allowed_read_requests); + SEASTAR_ASSERT(_allowed_read_requests); _allowed_read_requests--; return length; } diff --git a/tests/unit/rpc_test.cc b/tests/unit/rpc_test.cc index 78d7943d47..f7ed2559ab 100644 --- a/tests/unit/rpc_test.cc +++ b/tests/unit/rpc_test.cc @@ -36,6 +36,7 @@ #include #include #include +#include #include #include #include @@ -213,7 +214,7 @@ class rpc_test_env { future<> unregister_handler(MsgType t) { auto it = std::find(_handlers.begin(), _handlers.end(), t); - assert(it != _handlers.end()); + SEASTAR_ASSERT(it != _handlers.end()); _handlers.erase(it); return proto().unregister_handler(t); } diff --git a/tests/unit/scheduling_group_test.cc b/tests/unit/scheduling_group_test.cc index c31d039e37..2b045b7203 100644 --- a/tests/unit/scheduling_group_test.cc +++ b/tests/unit/scheduling_group_test.cc @@ -382,7 +382,7 @@ SEASTAR_THREAD_TEST_CASE(sg_key_constructor_exception_when_creating_new_key) { } ~thrower() { // Shouldn't get here because the constructor shouldn't succeed - BOOST_ASSERT(false); + SEASTAR_ASSERT(false); } }; scheduling_group_key_config thrower_conf = make_scheduling_group_key_config(); diff --git a/tests/unit/sharded_test.cc b/tests/unit/sharded_test.cc index 5daadb82ba..d17c4a8538 100644 --- a/tests/unit/sharded_test.cc +++ b/tests/unit/sharded_test.cc @@ -24,6 +24,7 @@ #include #include #include +#include #include @@ -42,7 +43,7 @@ class invoke_on_during_stop final : public peering_sharded_service results) { for (auto& x : results) { - assert(x == 1); + SEASTAR_ASSERT(x == 1); } }).get(); s.stop().get(); @@ -139,7 +140,7 @@ SEASTAR_THREAD_TEST_CASE(invoke_map_returns_future_value) { return make_ready_future(m.x); }).then([] (std::vector results) { for (auto& x : results) { - assert(x == 1); + SEASTAR_ASSERT(x == 1); } }).get(); s.stop().get(); @@ -154,7 +155,7 @@ SEASTAR_THREAD_TEST_CASE(invoke_map_returns_future_value_from_thread) { }); }).then([] (std::vector results) { for (auto& x : results) { - assert(x == 1); + SEASTAR_ASSERT(x == 1); } }).get(); s.stop().get(); @@ -240,7 +241,7 @@ class coordinator_synced_shard_map : public peering_sharded_service p = ::make_shared(); { auto p2 = dynamic_pointer_cast(p); - BOOST_ASSERT(!p2); + SEASTAR_ASSERT(!p2); } - BOOST_ASSERT(!A::destroyed); + SEASTAR_ASSERT(!A::destroyed); } class C : public enable_shared_from_this { diff --git a/tests/unit/slab_test.cc b/tests/unit/slab_test.cc index 7588dd79d8..1b4723f06e 100644 --- a/tests/unit/slab_test.cc +++ b/tests/unit/slab_test.cc @@ -22,8 +22,8 @@ */ #include -#include #include +#include using namespace seastar; @@ -61,12 +61,12 @@ static void test_allocation_1(const double growth_factor, const unsigned slab_li std::vector items; - assert(slab_limit_size % size == 0); + SEASTAR_ASSERT(slab_limit_size % size == 0); for (auto i = 0u; i < (slab_limit_size / size); i++) { auto item = slab.create(size); items.push_back(item); } - assert(slab.create(size) == nullptr); + SEASTAR_ASSERT(slab.create(size) == nullptr); free_vector(slab, items); std::cout << __FUNCTION__ << " done!\n"; @@ -91,7 +91,7 @@ static void test_allocation_2(const double growth_factor, const unsigned slab_li auto class_size = slab.class_size(size); auto per_slab_page = max_object_size / class_size; auto available_slab_pages = slab_limit_size / max_object_size; - assert(allocations == (per_slab_page * available_slab_pages)); + SEASTAR_ASSERT(allocations == (per_slab_page * available_slab_pages)); free_vector(slab, items); std::cout << __FUNCTION__ << " done!\n"; @@ -108,10 +108,10 @@ static void test_allocation_with_lru(const double growth_factor, const unsigned auto max = slab_limit_size / max_object_size; for (auto i = 0u; i < max * 1000; i++) { auto item = slab.create(size); - assert(item != nullptr); + SEASTAR_ASSERT(item != nullptr); _cache.push_front(*item); } - assert(evictions == max * 999); + SEASTAR_ASSERT(evictions == max * 999); _cache.clear(); diff --git a/tests/unit/socket_test.cc b/tests/unit/socket_test.cc index fa1955bdc3..9ba420d2fa 100644 --- a/tests/unit/socket_test.cc +++ b/tests/unit/socket_test.cc @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #include @@ -101,7 +102,7 @@ SEASTAR_TEST_CASE(socket_skip_test) { // expected return; } - assert(!"Skipping data from socket is likely stuck"); + SEASTAR_ASSERT(!"Skipping data from socket is likely stuck"); }); accept_result accepted = ss.accept().get(); diff --git a/tests/unit/thread_test.cc b/tests/unit/thread_test.cc index c86fef74d8..89408b883a 100644 --- a/tests/unit/thread_test.cc +++ b/tests/unit/thread_test.cc @@ -29,6 +29,7 @@ #include #include #include +#include #include #include @@ -200,7 +201,7 @@ static void* pagealign(void* ptr, size_t page_size) { static thread_local struct sigaction default_old_sigsegv_handler; static void bypass_stack_guard(int sig, siginfo_t* si, void* ctx) { - assert(sig == SIGSEGV); + SEASTAR_ASSERT(sig == SIGSEGV); int flags = get_mprotect_flags(ctx); stack_guard_bypassed = (flags & PROT_WRITE); if (!stack_guard_bypassed) { @@ -208,7 +209,7 @@ static void bypass_stack_guard(int sig, siginfo_t* si, void* ctx) { } size_t page_size = getpagesize(); auto mp_result = mprotect(pagealign(si->si_addr, page_size), page_size, PROT_READ | PROT_WRITE); - assert(mp_result == 0); + SEASTAR_ASSERT(mp_result == 0); } // This test will fail with a regular stack size, because we only probe diff --git a/tests/unit/tls_test.cc b/tests/unit/tls_test.cc index 39b5fe4ef7..0a1646e64e 100644 --- a/tests/unit/tls_test.cc +++ b/tests/unit/tls_test.cc @@ -572,7 +572,7 @@ static future<> run_echo_test(sstring message, auto server = ::make_shared>(); auto addr = ::make_ipv4_address( {0x7f000001, port}); - assert(do_read || loops == 1); + SEASTAR_ASSERT(do_read || loops == 1); future<> f = make_ready_future(); @@ -1075,7 +1075,7 @@ SEASTAR_THREAD_TEST_CASE(test_reload_tolerance) { auto end = std::chrono::system_clock::now(); - BOOST_ASSERT(nfails == 0 || (end - start) > 4s); + SEASTAR_ASSERT(nfails == 0 || (end - start) > 4s); } SEASTAR_THREAD_TEST_CASE(test_reload_by_move) { @@ -1160,7 +1160,7 @@ SEASTAR_THREAD_TEST_CASE(test_reload_by_move) { p = promise(); try { with_timeout(std::chrono::steady_clock::now() + 3s, p.get_future()).get(); - BOOST_ASSERT(i == 0); + SEASTAR_ASSERT(i == 0); } catch (timed_out_error&) { // ok break; @@ -1778,4 +1778,4 @@ SEASTAR_THREAD_TEST_CASE(test_reload_certificates_with_only_shard0_notify) { BOOST_CHECK_EQUAL(sstring(buf.begin(), buf.end()), "apa"); } -} \ No newline at end of file +} diff --git a/tests/unit/websocket_test.cc b/tests/unit/websocket_test.cc index 46a0bb540d..873930a9f7 100644 --- a/tests/unit/websocket_test.cc +++ b/tests/unit/websocket_test.cc @@ -80,7 +80,7 @@ future<> test_websocket_handshake_common(std::string subprotocol) { parser.init(); input.consume(parser).get(); std::unique_ptr resp = parser.get_parsed_response(); - BOOST_ASSERT(resp); + SEASTAR_ASSERT(resp); sstring websocket_accept = resp->_headers["Sec-WebSocket-Accept"]; // Trim possible whitespace prefix auto it = std::find_if(websocket_accept.begin(), websocket_accept.end(), ::isalnum); @@ -234,11 +234,11 @@ SEASTAR_TEST_CASE(test_websocket_parser_split) { break; } - BOOST_ASSERT(parser.is_valid()); + SEASTAR_ASSERT(parser.is_valid()); results.push_back(seastar::to_sstring(parser.result())); } - BOOST_ASSERT(!parser.is_valid()); + SEASTAR_ASSERT(!parser.is_valid()); BOOST_REQUIRE_EQUAL(0, parser.result().size()); std::vector expected = {