aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorilnurkh <ilnurkh@yandex-team.com>2023-10-09 23:39:40 +0300
committerilnurkh <ilnurkh@yandex-team.com>2023-10-09 23:57:14 +0300
commite601ca03f859335d57ecff2e5aa6af234b6052ed (patch)
treede519a847e58a1b3993fcbfe05ff44cc946a3e24
parentbbf2b6878af3854815a2c0ecb07a687071787639 (diff)
downloadydb-e601ca03f859335d57ecff2e5aa6af234b6052ed.tar.gz
Y_VERIFY->Y_ABORT_UNLESS at ^l
https://clubs.at.yandex-team.ru/arcadia/29404
-rw-r--r--library/cpp/actors/core/actor.cpp8
-rw-r--r--library/cpp/actors/core/actor.h8
-rw-r--r--library/cpp/actors/core/actor_bootstrapped.h2
-rw-r--r--library/cpp/actors/core/actor_coroutine.cpp14
-rw-r--r--library/cpp/actors/core/actor_virtual.h2
-rw-r--r--library/cpp/actors/core/actorid.h2
-rw-r--r--library/cpp/actors/core/actorsystem.cpp8
-rw-r--r--library/cpp/actors/core/actorsystem.h2
-rw-r--r--library/cpp/actors/core/balancer.cpp14
-rw-r--r--library/cpp/actors/core/benchmark_ut.cpp20
-rw-r--r--library/cpp/actors/core/callstack.cpp4
-rw-r--r--library/cpp/actors/core/config.h6
-rw-r--r--library/cpp/actors/core/cpu_manager.cpp4
-rw-r--r--library/cpp/actors/core/event.h4
-rw-r--r--library/cpp/actors/core/event_load.h2
-rw-r--r--library/cpp/actors/core/event_pb.cpp36
-rw-r--r--library/cpp/actors/core/event_pb.h4
-rw-r--r--library/cpp/actors/core/events_undelivered.cpp4
-rw-r--r--library/cpp/actors/core/executor_pool_base.cpp12
-rw-r--r--library/cpp/actors/core/executor_pool_basic.cpp2
-rw-r--r--library/cpp/actors/core/executor_pool_basic_ut.cpp2
-rw-r--r--library/cpp/actors/core/executor_pool_united.cpp10
-rw-r--r--library/cpp/actors/core/executor_pool_united_ut.cpp2
-rw-r--r--library/cpp/actors/core/interconnect.cpp12
-rw-r--r--library/cpp/actors/core/invoke.h4
-rw-r--r--library/cpp/actors/core/io_dispatcher.cpp6
-rw-r--r--library/cpp/actors/core/log.cpp4
-rw-r--r--library/cpp/actors/core/log_settings.cpp6
-rw-r--r--library/cpp/actors/core/mailbox.cpp8
-rw-r--r--library/cpp/actors/core/mailbox.h6
-rw-r--r--library/cpp/actors/core/mailbox_queue_revolving.h2
-rw-r--r--library/cpp/actors/core/mon.h4
-rw-r--r--library/cpp/actors/core/mon_ut.cpp2
-rw-r--r--library/cpp/actors/core/performance_ut.cpp6
-rw-r--r--library/cpp/actors/core/scheduler_actor.cpp8
-rw-r--r--library/cpp/actors/core/scheduler_basic.cpp6
-rw-r--r--library/cpp/actors/cppcoro/corobenchmark/main.cpp2
-rw-r--r--library/cpp/actors/cppcoro/task_actor.cpp26
-rw-r--r--library/cpp/actors/cppcoro/task_actor_ut.cpp2
-rw-r--r--library/cpp/actors/cppcoro/task_group.h18
-rw-r--r--library/cpp/actors/cppcoro/task_ut.cpp4
-rw-r--r--library/cpp/actors/dnscachelib/dnscache.cpp2
-rw-r--r--library/cpp/actors/dnsresolver/dnsresolver.cpp22
-rw-r--r--library/cpp/actors/dnsresolver/dnsresolver.h4
-rw-r--r--library/cpp/actors/dnsresolver/dnsresolver_caching.cpp14
-rw-r--r--library/cpp/actors/dnsresolver/dnsresolver_caching_ut.cpp12
-rw-r--r--library/cpp/actors/dnsresolver/dnsresolver_ut.cpp2
-rw-r--r--library/cpp/actors/examples/01_ping_pong/main.cpp4
-rw-r--r--library/cpp/actors/examples/02_discovery/lookup.cpp2
-rw-r--r--library/cpp/actors/examples/02_discovery/main.cpp2
-rw-r--r--library/cpp/actors/helpers/activeactors.h4
-rw-r--r--library/cpp/actors/helpers/mon_histogram_helper.h2
-rw-r--r--library/cpp/actors/helpers/pool_stats_collector.h4
-rw-r--r--library/cpp/actors/http/http.h2
-rw-r--r--library/cpp/actors/http/http_proxy_acceptor.cpp2
-rw-r--r--library/cpp/actors/http/http_proxy_incoming.cpp2
-rw-r--r--library/cpp/actors/interconnect/channel_scheduler.h2
-rw-r--r--library/cpp/actors/interconnect/event_holder_pool.h4
-rw-r--r--library/cpp/actors/interconnect/handshake_broker.h6
-rw-r--r--library/cpp/actors/interconnect/interconnect_address.cpp2
-rw-r--r--library/cpp/actors/interconnect/interconnect_channel.cpp26
-rw-r--r--library/cpp/actors/interconnect/interconnect_counters.cpp4
-rw-r--r--library/cpp/actors/interconnect/interconnect_handshake.cpp36
-rw-r--r--library/cpp/actors/interconnect/interconnect_nameserver_dynamic.cpp2
-rw-r--r--library/cpp/actors/interconnect/interconnect_nameserver_table.cpp2
-rw-r--r--library/cpp/actors/interconnect/interconnect_proxy_wrapper.cpp2
-rw-r--r--library/cpp/actors/interconnect/interconnect_stream.cpp36
-rw-r--r--library/cpp/actors/interconnect/interconnect_tcp_input_session.cpp40
-rw-r--r--library/cpp/actors/interconnect/interconnect_tcp_proxy.cpp52
-rw-r--r--library/cpp/actors/interconnect/interconnect_tcp_proxy.h18
-rw-r--r--library/cpp/actors/interconnect/interconnect_tcp_server.cpp4
-rw-r--r--library/cpp/actors/interconnect/interconnect_tcp_session.cpp22
-rw-r--r--library/cpp/actors/interconnect/mock/ic_mock.cpp14
-rw-r--r--library/cpp/actors/interconnect/outgoing_stream.h4
-rw-r--r--library/cpp/actors/interconnect/packet.h2
-rw-r--r--library/cpp/actors/interconnect/poller_actor.cpp6
-rw-r--r--library/cpp/actors/interconnect/poller_actor_darwin.h8
-rw-r--r--library/cpp/actors/interconnect/poller_actor_linux.h2
-rw-r--r--library/cpp/actors/interconnect/poller_actor_win.h2
-rw-r--r--library/cpp/actors/interconnect/profiler.h8
-rw-r--r--library/cpp/actors/interconnect/ut/channel_scheduler_ut.cpp2
-rw-r--r--library/cpp/actors/interconnect/ut/dynamic_proxy_ut.cpp6
-rw-r--r--library/cpp/actors/interconnect/ut/interconnect_ut.cpp14
-rw-r--r--library/cpp/actors/interconnect/ut/large.cpp4
-rw-r--r--library/cpp/actors/interconnect/ut/lib/interrupter.h4
-rw-r--r--library/cpp/actors/interconnect/ut/poller_actor_ut.cpp6
-rw-r--r--library/cpp/actors/interconnect/ut/sticking_ut.cpp2
-rw-r--r--library/cpp/actors/interconnect/ut_fat/main.cpp12
-rw-r--r--library/cpp/actors/memory_log/memlog.cpp18
-rw-r--r--library/cpp/actors/memory_log/mmap.cpp8
-rw-r--r--library/cpp/actors/prof/tag.cpp8
-rw-r--r--library/cpp/actors/testlib/test_runtime.cpp94
-rw-r--r--library/cpp/actors/testlib/test_runtime.h8
-rw-r--r--library/cpp/actors/util/cpu_load_log.h2
-rw-r--r--library/cpp/actors/util/local_process_key.h6
-rw-r--r--library/cpp/actors/util/rc_buf.h18
-rw-r--r--library/cpp/actors/util/recentwnd.h2
-rw-r--r--library/cpp/actors/util/rope.h8
-rw-r--r--library/cpp/actors/util/shared_data_ut.cpp4
-rw-r--r--library/cpp/actors/util/thread_load_log.h6
-rw-r--r--library/cpp/actors/util/timerfd.h6
-rw-r--r--library/cpp/actors/util/unordered_cache.h2
-rw-r--r--library/cpp/actors/wilson/wilson_profile_span.cpp4
-rw-r--r--library/cpp/actors/wilson/wilson_trace.h4
-rw-r--r--library/cpp/actors/wilson/wilson_uploader.cpp2
-rw-r--r--library/cpp/balloc/lib/balloc.h2
-rw-r--r--library/cpp/binsaver/buffered_io.cpp4
-rw-r--r--library/cpp/bit_io/bitoutput.h2
-rw-r--r--library/cpp/blockcodecs/core/stream.cpp2
-rw-r--r--library/cpp/blockcodecs/fuzz/main.cpp8
-rw-r--r--library/cpp/codecs/codecs.cpp2
-rw-r--r--library/cpp/codecs/codecs_registry.cpp2
-rw-r--r--library/cpp/containers/comptrie/write_trie_backwards.cpp2
-rw-r--r--library/cpp/containers/concurrent_hash/concurrent_hash.h6
-rw-r--r--library/cpp/containers/paged_vector/paged_vector.h2
-rw-r--r--library/cpp/containers/stack_vector/stack_vec.h2
-rw-r--r--library/cpp/coroutine/engine/coroutine_ut.cpp2
-rw-r--r--library/cpp/coroutine/engine/impl.cpp2
-rw-r--r--library/cpp/coroutine/engine/stack/benchmark/alloc_bm.cpp4
-rw-r--r--library/cpp/coroutine/engine/stack/stack_allocator.inl8
-rw-r--r--library/cpp/coroutine/engine/stack/stack_guards.h2
-rw-r--r--library/cpp/coroutine/engine/stack/stack_pool.inl2
-rw-r--r--library/cpp/coroutine/engine/stack/stack_storage.h6
-rw-r--r--library/cpp/coroutine/engine/stack/stack_utils.cpp8
-rw-r--r--library/cpp/coroutine/engine/trampoline.h4
-rw-r--r--library/cpp/getopt/small/completer.cpp2
-rw-r--r--library/cpp/getopt/small/completer.h2
-rw-r--r--library/cpp/getopt/small/completion_generator.cpp4
-rw-r--r--library/cpp/grpc/client/grpc_client_low.cpp10
-rw-r--r--library/cpp/grpc/client/grpc_client_low.h40
-rw-r--r--library/cpp/grpc/server/grpc_request.cpp4
-rw-r--r--library/cpp/grpc/server/grpc_request.h22
-rw-r--r--library/cpp/grpc/server/grpc_server.cpp4
-rw-r--r--library/cpp/grpc/server/grpc_server.h10
-rw-r--r--library/cpp/histogram/adaptive/adaptive_histogram.cpp32
-rw-r--r--library/cpp/histogram/adaptive/block_histogram.cpp12
-rw-r--r--library/cpp/histogram/adaptive/fixed_bin_histogram.cpp2
-rw-r--r--library/cpp/ipv6_address/ipv6_address.cpp14
-rw-r--r--library/cpp/ipv6_address/ut/ipv6_address_ut.cpp2
-rw-r--r--library/cpp/json/writer/json_value.cpp4
-rw-r--r--library/cpp/lcs/lcs_via_lis.h2
-rw-r--r--library/cpp/lfalloc/lf_allocX64.h2
-rw-r--r--library/cpp/logger/global/global.h2
-rw-r--r--library/cpp/logger/global/rty_formater.cpp4
-rw-r--r--library/cpp/logger/uninitialized_creator.cpp2
-rw-r--r--library/cpp/lwtrace/example3/my_action.h2
-rw-r--r--library/cpp/lwtrace/kill_action.cpp2
-rw-r--r--library/cpp/lwtrace/lwprobe.h12
-rw-r--r--library/cpp/lwtrace/mon/analytics/data.h8
-rw-r--r--library/cpp/lwtrace/shuttle.h2
-rw-r--r--library/cpp/lwtrace/start.cpp2
-rw-r--r--library/cpp/lwtrace/trace.cpp2
-rw-r--r--library/cpp/malloc/nalf/alloc_helpers.h4
-rw-r--r--library/cpp/messagebus/actor/actor.h4
-rw-r--r--library/cpp/messagebus/actor/executor.cpp6
-rw-r--r--library/cpp/messagebus/actor/queue_for_actor.h2
-rw-r--r--library/cpp/messagebus/async_result.h2
-rw-r--r--library/cpp/messagebus/config/netaddr.cpp2
-rw-r--r--library/cpp/messagebus/config/session_config.cpp4
-rw-r--r--library/cpp/messagebus/duration_histogram.cpp2
-rw-r--r--library/cpp/messagebus/event_loop.cpp10
-rw-r--r--library/cpp/messagebus/futex_like.cpp4
-rw-r--r--library/cpp/messagebus/handler_impl.h4
-rw-r--r--library/cpp/messagebus/locator.cpp4
-rw-r--r--library/cpp/messagebus/message.cpp10
-rw-r--r--library/cpp/messagebus/message.h2
-rw-r--r--library/cpp/messagebus/message_status_counter.cpp4
-rw-r--r--library/cpp/messagebus/messqueue.cpp4
-rw-r--r--library/cpp/messagebus/misc/test_sync.h8
-rw-r--r--library/cpp/messagebus/network.cpp8
-rw-r--r--library/cpp/messagebus/nondestroying_holder.h4
-rw-r--r--library/cpp/messagebus/oldmodule/module.cpp26
-rw-r--r--library/cpp/messagebus/protobuf/ybusbuf.cpp2
-rw-r--r--library/cpp/messagebus/rain_check/core/coro_stack.cpp4
-rw-r--r--library/cpp/messagebus/rain_check/core/simple_ut.cpp2
-rw-r--r--library/cpp/messagebus/rain_check/core/task.cpp2
-rw-r--r--library/cpp/messagebus/rain_check/http/client.cpp10
-rw-r--r--library/cpp/messagebus/rain_check/messagebus/messagebus_client_ut.cpp6
-rw-r--r--library/cpp/messagebus/rain_check/test/helper/misc.cpp4
-rw-r--r--library/cpp/messagebus/remote_client_connection.cpp8
-rw-r--r--library/cpp/messagebus/remote_client_session_semaphore.cpp4
-rw-r--r--library/cpp/messagebus/remote_connection.cpp30
-rw-r--r--library/cpp/messagebus/remote_server_connection.cpp2
-rw-r--r--library/cpp/messagebus/remote_server_session.cpp2
-rw-r--r--library/cpp/messagebus/remote_server_session_semaphore.cpp6
-rw-r--r--library/cpp/messagebus/scheduler/scheduler.cpp6
-rw-r--r--library/cpp/messagebus/session_impl.cpp36
-rw-r--r--library/cpp/messagebus/session_job_count.cpp2
-rw-r--r--library/cpp/messagebus/shutdown_state.cpp6
-rw-r--r--library/cpp/messagebus/storage.cpp10
-rw-r--r--library/cpp/messagebus/synchandler.cpp6
-rw-r--r--library/cpp/messagebus/test/example/client/client.cpp2
-rw-r--r--library/cpp/messagebus/test/helper/alloc_counter.h2
-rw-r--r--library/cpp/messagebus/test/helper/example.cpp12
-rw-r--r--library/cpp/messagebus/test/helper/wait_for.h2
-rw-r--r--library/cpp/messagebus/test/perftest/perftest.cpp10
-rw-r--r--library/cpp/messagebus/test/perftest/simple_proto.cpp2
-rw-r--r--library/cpp/messagebus/test/ut/messagebus_ut.cpp30
-rw-r--r--library/cpp/messagebus/test/ut/module_client_ut.cpp14
-rw-r--r--library/cpp/messagebus/test/ut/module_server_ut.cpp4
-rw-r--r--library/cpp/messagebus/test/ut/one_way_ut.cpp2
-rw-r--r--library/cpp/messagebus/use_after_free_checker.cpp4
-rw-r--r--library/cpp/messagebus/use_count_checker.cpp2
-rw-r--r--library/cpp/messagebus/vector_swaps.h4
-rw-r--r--library/cpp/messagebus/www/html_output.h2
-rw-r--r--library/cpp/messagebus/www/www.cpp18
-rw-r--r--library/cpp/monlib/dynamic_counters/contention_ut.cpp2
-rw-r--r--library/cpp/monlib/dynamic_counters/counters.cpp12
-rw-r--r--library/cpp/monlib/metrics/ewma.cpp2
-rw-r--r--library/cpp/monlib/service/monservice.cpp6
-rw-r--r--library/cpp/monlib/service/pages/index_mon_page.cpp6
-rw-r--r--library/cpp/monlib/service/pages/mon_page.cpp4
-rw-r--r--library/cpp/neh/http2.cpp4
-rw-r--r--library/cpp/neh/https.cpp6
-rw-r--r--library/cpp/neh/netliba.cpp4
-rw-r--r--library/cpp/neh/netliba_udp_http.cpp2
-rw-r--r--library/cpp/netliba/socket/socket.cpp54
-rw-r--r--library/cpp/netliba/v6/ib_buffers.h6
-rw-r--r--library/cpp/netliba/v6/ib_collective.cpp30
-rw-r--r--library/cpp/netliba/v6/ib_cs.cpp18
-rw-r--r--library/cpp/netliba/v6/ib_low.h4
-rw-r--r--library/cpp/netliba/v6/udp_client_server.cpp6
-rw-r--r--library/cpp/netliba/v6/udp_http.cpp4
-rw-r--r--library/cpp/object_factory/object_factory.h2
-rw-r--r--library/cpp/openssl/big_integer/big_integer.cpp2
-rw-r--r--library/cpp/openssl/method/io.cpp2
-rw-r--r--library/cpp/protobuf/json/proto2json_printer.cpp8
-rw-r--r--library/cpp/resource/registry.cpp6
-rw-r--r--library/cpp/scheme/tests/fuzz_ops/lib/vm_defs.h2
-rw-r--r--library/cpp/sighandler/async_signals_handler.cpp16
-rw-r--r--library/cpp/skiff/skiff_schema.cpp2
-rw-r--r--library/cpp/skiff/skiff_validator.cpp4
-rw-r--r--library/cpp/skiff/zerocopy_output_writer-inl.h2
-rw-r--r--library/cpp/streams/brotli/brotli.cpp6
-rw-r--r--library/cpp/string_utils/base32/fuzz/main.cpp2
-rw-r--r--library/cpp/string_utils/base64/fuzz/lib/main.cpp2
-rw-r--r--library/cpp/tdigest/tdigest.cpp2
-rw-r--r--library/cpp/terminate_handler/segv_handler.cpp2
-rw-r--r--library/cpp/testing/common/env.cpp2
-rw-r--r--library/cpp/testing/common/network.cpp8
-rw-r--r--library/cpp/testing/unittest/registar.cpp10
-rw-r--r--library/cpp/testing/unittest/utmain.cpp4
-rw-r--r--library/cpp/threading/local_executor/local_executor.cpp2
-rw-r--r--library/cpp/threading/mux_event/mux_event.h2
-rw-r--r--library/cpp/threading/queue/mpmc_unordered_ring.cpp2
-rw-r--r--library/cpp/type_info/builder.cpp6
-rw-r--r--library/cpp/type_info/type.cpp8
-rw-r--r--library/cpp/type_info/type.h68
-rw-r--r--library/cpp/type_info/type_list.h2
-rw-r--r--library/cpp/unicode/set/unicode_set_lexer.h2
-rw-r--r--library/cpp/unified_agent_client/async_joiner.h4
-rw-r--r--library/cpp/unified_agent_client/client_impl.cpp44
-rw-r--r--library/cpp/unified_agent_client/clock.cpp8
-rw-r--r--library/cpp/unified_agent_client/enum.h2
-rw-r--r--library/cpp/unified_agent_client/examples/ua_grpc_client/main.cpp2
-rw-r--r--library/cpp/unified_agent_client/grpc_io.cpp6
-rw-r--r--library/cpp/uri/benchmark/main.cpp4
-rw-r--r--library/cpp/yconf/patcher/config_patcher.cpp4
-rw-r--r--library/cpp/yconf/patcher/unstrict_config.cpp6
-rw-r--r--library/cpp/yconf/patcher/unstrict_config.h2
-rw-r--r--library/cpp/yt/threading/at_fork.cpp2
-rw-r--r--library/cpp/ytalloc/impl/bridge.cpp4
-rw-r--r--library/cpp/ytalloc/impl/core-inl.h12
263 files changed, 1014 insertions, 1014 deletions
diff --git a/library/cpp/actors/core/actor.cpp b/library/cpp/actors/core/actor.cpp
index 73edd82067..4f7d4c1493 100644
--- a/library/cpp/actors/core/actor.cpp
+++ b/library/cpp/actors/core/actor.cpp
@@ -18,7 +18,7 @@ namespace NActors {
ui64 count = value >> T::TimestampBits;
count += Increment;
- Y_VERIFY((count & ~T::CountMask) == 0);
+ Y_ABORT_UNLESS((count & ~T::CountMask) == 0);
ui64 timestamp = value;
if (Increment == 1 && count == 1) {
@@ -53,7 +53,7 @@ namespace NActors {
used += (static_cast<ui64>(time) - value) & TimestampMask;
}
- Y_VERIFY(LastUsageTimestamp <= time);
+ Y_ABORT_UNLESS(LastUsageTimestamp <= time);
ui64 passed = time - LastUsageTimestamp;
LastUsageTimestamp = time;
@@ -188,7 +188,7 @@ namespace NActors {
void IActor::Die(const TActorContext& ctx) {
if (ctx.SelfID)
- Y_VERIFY(ctx.SelfID == SelfActorId);
+ Y_ABORT_UNLESS(ctx.SelfID == SelfActorId);
PassAway();
}
@@ -206,7 +206,7 @@ namespace NActors {
}
void TActorVirtualBehaviour::Receive(IActor* actor, std::unique_ptr<IEventHandle> ev) {
- Y_VERIFY(!!ev && ev->GetBase());
+ Y_ABORT_UNLESS(!!ev && ev->GetBase());
ev->GetBase()->Execute(actor, std::move(ev));
}
diff --git a/library/cpp/actors/core/actor.h b/library/cpp/actors/core/actor.h
index 3f9a5751a4..d151425680 100644
--- a/library/cpp/actors/core/actor.h
+++ b/library/cpp/actors/core/actor.h
@@ -475,7 +475,7 @@ namespace NActors {
}
~TRecurseContext() {
- Y_VERIFY(TlsActivationContext == this, "TlsActivationContext mismatch; probably InvokeOtherActor was invoked from a coroutine");
+ Y_ABORT_UNLESS(TlsActivationContext == this, "TlsActivationContext mismatch; probably InvokeOtherActor was invoked from a coroutine");
TlsActivationContext = Prev;
}
} context(actor.SelfId());
@@ -902,7 +902,7 @@ namespace NActors {
template <ESendingType SendingType>
TActorId IActor::Register(IActor* actor, TMailboxType::EType mailboxType, ui32 poolId) const noexcept {
- Y_VERIFY(actor);
+ Y_ABORT_UNLESS(actor);
return TlsActivationContext->ExecutorThread.RegisterActor<SendingType>(actor, mailboxType, poolId, SelfActorId);
}
@@ -910,8 +910,8 @@ namespace NActors {
template <ESendingType SendingType>
TActorId TActorSystem::Register(IActor* actor, TMailboxType::EType mailboxType, ui32 executorPool,
ui64 revolvingCounter, const TActorId& parentId) {
- Y_VERIFY(actor);
- Y_VERIFY(executorPool < ExecutorPoolCount, "executorPool# %" PRIu32 ", ExecutorPoolCount# %" PRIu32,
+ Y_ABORT_UNLESS(actor);
+ Y_ABORT_UNLESS(executorPool < ExecutorPoolCount, "executorPool# %" PRIu32 ", ExecutorPoolCount# %" PRIu32,
(ui32)executorPool, (ui32)ExecutorPoolCount);
if constexpr (SendingType == ESendingType::Common) {
return CpuManager->GetExecutorPool(executorPool)->Register(actor, mailboxType, revolvingCounter, parentId);
diff --git a/library/cpp/actors/core/actor_bootstrapped.h b/library/cpp/actors/core/actor_bootstrapped.h
index 5d3d381d6e..70a6163bc5 100644
--- a/library/cpp/actors/core/actor_bootstrapped.h
+++ b/library/cpp/actors/core/actor_bootstrapped.h
@@ -16,7 +16,7 @@ namespace NActors {
}
STFUNC(StateBootstrap) {
- Y_VERIFY(ev->GetTypeRewrite() == TEvents::TSystem::Bootstrap, "Unexpected bootstrap message");
+ Y_ABORT_UNLESS(ev->GetTypeRewrite() == TEvents::TSystem::Bootstrap, "Unexpected bootstrap message");
using T = decltype(&TDerived::Bootstrap);
TDerived& self = static_cast<TDerived&>(*this);
if constexpr (std::is_invocable_v<T, TDerived, const TActorContext&>) {
diff --git a/library/cpp/actors/core/actor_coroutine.cpp b/library/cpp/actors/core/actor_coroutine.cpp
index 8dde637bf2..9b0c81fa26 100644
--- a/library/cpp/actors/core/actor_coroutine.cpp
+++ b/library/cpp/actors/core/actor_coroutine.cpp
@@ -57,7 +57,7 @@ namespace NActors {
}
// ensure we have no unprocessed event and return back to actor system to receive one
- Y_VERIFY(!Finished);
+ Y_ABORT_UNLESS(!Finished);
// obtain pending event and ensure we've got one
while (THolder<IEventHandle> event = ReturnToActorSystem()) {
@@ -72,7 +72,7 @@ namespace NActors {
bool TActorCoroImpl::ProcessEvent(THolder<IEventHandle> ev) {
if (!SelfActorId) { // process bootstrap message, extract actor ids
- Y_VERIFY(ev->GetTypeRewrite() == TEvents::TSystem::Bootstrap);
+ Y_ABORT_UNLESS(ev->GetTypeRewrite() == TEvents::TSystem::Bootstrap);
SelfActorId = ev->Recipient;
ParentActorId = ev->Sender;
ev.Reset();
@@ -97,7 +97,7 @@ namespace NActors {
void TActorCoroImpl::Resume(THolder<IEventHandle> ev) {
BeforeResume();
- Y_VERIFY(!PendingEvent);
+ Y_ABORT_UNLESS(!PendingEvent);
PendingEvent.Swap(ev);
#if CORO_THROUGH_THREADS
@@ -106,7 +106,7 @@ namespace NActors {
OutEvent.Wait();
#else
// save caller context for a later return
- Y_VERIFY(!ActorSystemContext);
+ Y_ABORT_UNLESS(!ActorSystemContext);
TExceptionSafeContext actorSystemContext;
ActorSystemContext = &actorSystemContext;
@@ -114,7 +114,7 @@ namespace NActors {
ActorSystemContext->SwitchTo(&FiberContext);
#endif
- Y_VERIFY(!PendingEvent);
+ Y_ABORT_UNLESS(!PendingEvent);
}
void TActorCoroImpl::DoRun() {
@@ -150,7 +150,7 @@ namespace NActors {
}
#else
TExceptionSafeContext* returnContext = std::exchange(ActorSystemContext, nullptr);
- Y_VERIFY(returnContext);
+ Y_ABORT_UNLESS(returnContext);
if (StoreTlsState) {
StoreTlsState(this);
}
@@ -165,7 +165,7 @@ namespace NActors {
} else {
// we have returned from the actor system and it kindly asks us to terminate the coroutine as it is being
// stopped
- Y_VERIFY(InvokedFromDtor);
+ Y_ABORT_UNLESS(InvokedFromDtor);
throw TDtorException();
}
}
diff --git a/library/cpp/actors/core/actor_virtual.h b/library/cpp/actors/core/actor_virtual.h
index c9c34c4729..9b02660c65 100644
--- a/library/cpp/actors/core/actor_virtual.h
+++ b/library/cpp/actors/core/actor_virtual.h
@@ -21,7 +21,7 @@ public:
{
Y_VERIFY_DEBUG(dynamic_cast<TEvent*>(Handle->GetBase()));
Event = static_cast<TEvent*>(Handle->GetBase());
- Y_VERIFY(Event);
+ Y_ABORT_UNLESS(Event);
}
};
diff --git a/library/cpp/actors/core/actorid.h b/library/cpp/actors/core/actorid.h
index a9cac589b2..df1cac1c4e 100644
--- a/library/cpp/actors/core/actorid.h
+++ b/library/cpp/actors/core/actorid.h
@@ -50,7 +50,7 @@ namespace NActors {
}
explicit TActorId(ui32 nodeId, const TStringBuf& x) noexcept {
- Y_VERIFY(x.size() <= MaxServiceIDLength, "service id is too long");
+ Y_ABORT_UNLESS(x.size() <= MaxServiceIDLength, "service id is too long");
Raw.N.LocalId = 0;
Raw.N.Hint = 0;
Raw.N.NodeId = nodeId | ServiceMask;
diff --git a/library/cpp/actors/core/actorsystem.cpp b/library/cpp/actors/core/actorsystem.cpp
index 9207e3407e..b73c545134 100644
--- a/library/cpp/actors/core/actorsystem.cpp
+++ b/library/cpp/actors/core/actorsystem.cpp
@@ -106,7 +106,7 @@ namespace NActors {
if (recpNodeId != NodeId && recpNodeId != 0) {
// if recipient is not local one - rewrite with forward instruction
Y_VERIFY_DEBUG(!ev->HasEvent() || ev->GetBase()->IsSerializable());
- Y_VERIFY(ev->Recipient == recipient,
+ Y_ABORT_UNLESS(ev->Recipient == recipient,
"Event rewrite from %s to %s would be lost via interconnect",
ev->Recipient.ToString().c_str(),
recipient.ToString().c_str());
@@ -255,7 +255,7 @@ namespace NActors {
}
void TActorSystem::Start() {
- Y_VERIFY(StartExecuted == false);
+ Y_ABORT_UNLESS(StartExecuted == false);
StartExecuted = true;
ScheduleQueue.Reset(new NSchedulerQueue::TQueueType());
@@ -273,7 +273,7 @@ namespace NActors {
TActorSetupCmd& x = setup.ProxyActors[i];
if (x.Actor) {
Interconnect[i] = Register(x.Actor.release(), x.MailboxType, x.PoolId, i);
- Y_VERIFY(!!Interconnect[i]);
+ Y_ABORT_UNLESS(!!Interconnect[i]);
}
}
ProxyWrapperFactory = std::move(SystemSetup->Interconnect.ProxyWrapperFactory);
@@ -284,7 +284,7 @@ namespace NActors {
for (ui32 i = 0, e = (ui32)SystemSetup->LocalServices.size(); i != e; ++i) {
std::pair<TActorId, TActorSetupCmd>& x = SystemSetup->LocalServices[i];
const TActorId xid = Register(x.second.Actor.release(), x.second.MailboxType, x.second.PoolId, i);
- Y_VERIFY(!!xid);
+ Y_ABORT_UNLESS(!!xid);
if (!!x.first)
RegisterLocalService(x.first, xid);
}
diff --git a/library/cpp/actors/core/actorsystem.h b/library/cpp/actors/core/actorsystem.h
index 64a8e827d2..1b18abc787 100644
--- a/library/cpp/actors/core/actorsystem.h
+++ b/library/cpp/actors/core/actorsystem.h
@@ -117,7 +117,7 @@ namespace NActors {
ui32 GetThreads(ui32 poolId) const {
auto result = GetThreadsOptional(poolId);
- Y_VERIFY(result, "undefined pool id: %" PRIu32, (ui32)poolId);
+ Y_ABORT_UNLESS(result, "undefined pool id: %" PRIu32, (ui32)poolId);
return *result;
}
diff --git a/library/cpp/actors/core/balancer.cpp b/library/cpp/actors/core/balancer.cpp
index d82701bbfb..e09c834543 100644
--- a/library/cpp/actors/core/balancer.cpp
+++ b/library/cpp/actors/core/balancer.cpp
@@ -51,11 +51,11 @@ namespace NActors {
ui64 idle = std::clamp<i64>(1024 - cpuIdle * 512, 0, 1023);
ui64 scale = std::clamp<i64>(1024 - scaleFactor * 32, 0, 1023);
- Y_VERIFY(ui64(load) < (1ull << 2ull));
- Y_VERIFY(ui64(priority) < (1ull << 8ull));
- Y_VERIFY(ui64(scale) < (1ull << 10ull));
- Y_VERIFY(ui64(idle) < (1ull << 10ull));
- Y_VERIFY(ui64(poolId) < (1ull << 6ull));
+ Y_ABORT_UNLESS(ui64(load) < (1ull << 2ull));
+ Y_ABORT_UNLESS(ui64(priority) < (1ull << 8ull));
+ Y_ABORT_UNLESS(ui64(scale) < (1ull << 10ull));
+ Y_ABORT_UNLESS(ui64(idle) < (1ull << 10ull));
+ Y_ABORT_UNLESS(ui64(poolId) < (1ull << 6ull));
static_assert(ui64(MaxPools) <= (1ull << 6ull));
@@ -135,7 +135,7 @@ namespace NActors {
Config.MinCpus = std::clamp<ui32>(Config.MinCpus, 1, Config.Cpus);
Config.MaxCpus = Max<ui32>(Config.MaxCpus, Config.Cpus);
} else {
- Y_VERIFY(Config.Cpus == 0,
+ Y_ABORT_UNLESS(Config.Cpus == 0,
"Unexpected negative Config.Cpus# %" PRIi64,
(i64)Config.Cpus);
Config.MinCpus = 0;
@@ -205,7 +205,7 @@ namespace NActors {
}
void TBalancer::SetPoolStats(TPoolId pool, const TBalancerStats& stats) {
- Y_VERIFY(pool < MaxPools);
+ Y_ABORT_UNLESS(pool < MaxPools);
TPool& p = Pools[pool];
p.Prev = p.Next;
p.Next = stats;
diff --git a/library/cpp/actors/core/benchmark_ut.cpp b/library/cpp/actors/core/benchmark_ut.cpp
index 12ef30ecb2..380e983b92 100644
--- a/library/cpp/actors/core/benchmark_ut.cpp
+++ b/library/cpp/actors/core/benchmark_ut.cpp
@@ -210,8 +210,8 @@ Y_UNIT_TEST_SUITE(ActorSystemBenchmark) {
: Params(params)
, ActiveEventRegistry_(activeEventRegistry)
{
- Y_VERIFY(!Params.Container.empty());
- Y_VERIFY(Params.Right - Params.Left + 1 <= static_cast<i32>(Params.Container.size()),
+ Y_ABORT_UNLESS(!Params.Container.empty());
+ Y_ABORT_UNLESS(Params.Right - Params.Left + 1 <= static_cast<i32>(Params.Container.size()),
"left: %d, right: %d, cont.size: %d", Params.Left, Params.Right, static_cast<i32>(Params.Container.size()));
ActiveEventRegistry_.SetActive(this);
}
@@ -242,7 +242,7 @@ Y_UNIT_TEST_SUITE(ActorSystemBenchmark) {
switch (ev->GetTypeRewrite()) {
hFunc(TEvQuickSort, Handle);
default:
- Y_VERIFY(false);
+ Y_ABORT_UNLESS(false);
}
}
@@ -373,7 +373,7 @@ Y_UNIT_TEST_SUITE(ActorSystemBenchmark) {
BENCH_START(thread);
- Y_VERIFY(threadPool.AddAndOwn(THolder(new TQuickSortTask(params, activeThreadRegistry))));
+ Y_ABORT_UNLESS(threadPool.AddAndOwn(THolder(new TQuickSortTask(params, activeThreadRegistry))));
UNIT_ASSERT_C(activeThreadRegistry.WaitForAllInactive(60s), "timeout");
threaPoolSortDurationTotal += std::chrono::duration_cast<std::chrono::microseconds>(BENCH_END(thread));
@@ -491,7 +491,7 @@ Y_UNIT_TEST_SUITE(ActorSystemBenchmark) {
switch (ev->GetTypeRewrite()) {
hFunc(TEvKvSendRequests, Handle);
default:
- Y_VERIFY(false);
+ Y_ABORT_UNLESS(false);
}
}
@@ -529,7 +529,7 @@ Y_UNIT_TEST_SUITE(ActorSystemBenchmark) {
switch (ev->GetTypeRewrite()) {
hFunc(TEvKvSearch, Handle);
default:
- Y_VERIFY(false);
+ Y_ABORT_UNLESS(false);
}
}
@@ -587,7 +587,7 @@ Y_UNIT_TEST_SUITE(ActorSystemBenchmark) {
}
}
- Y_VERIFY(keys.size() >= requestsNumber);
+ Y_ABORT_UNLESS(keys.size() >= requestsNumber);
std::random_shuffle(keys.begin(), keys.end());
keys.resize(requestsNumber);
@@ -709,7 +709,7 @@ Y_UNIT_TEST_SUITE(ActorSystemBenchmark) {
BENCH_START(kvSearch);
for (auto& key : keysToSearch) {
- Y_VERIFY(threadPool.AddAndOwn(THolder(new TKvSearchTask(key, dict))));
+ Y_ABORT_UNLESS(threadPool.AddAndOwn(THolder(new TKvSearchTask(key, dict))));
}
// CondVar logic gives too much of overhead (2-10 times more than just sleep_for)
@@ -897,7 +897,7 @@ Y_UNIT_TEST_SUITE(ActorSystemBenchmark) {
hFunc(TEvSumSendRequests, HandleRequest);
hFunc(TEvSumVectorResult, HandleResult);
default:
- Y_VERIFY(false);
+ Y_ABORT_UNLESS(false);
}
}
@@ -974,7 +974,7 @@ Y_UNIT_TEST_SUITE(ActorSystemBenchmark) {
switch (ev->GetTypeRewrite()) {
hFunc(TEvSumVector, Handle);
default:
- Y_VERIFY(false);
+ Y_ABORT_UNLESS(false);
}
}
diff --git a/library/cpp/actors/core/callstack.cpp b/library/cpp/actors/core/callstack.cpp
index 9297c1a079..559cc73550 100644
--- a/library/cpp/actors/core/callstack.cpp
+++ b/library/cpp/actors/core/callstack.cpp
@@ -19,7 +19,7 @@ namespace NActors {
void EnableActorCallstack() {
if (ActorBackTraceEnableCounter == 0) {
- Y_VERIFY(PreviousFormatBackTrace == 0);
+ Y_ABORT_UNLESS(PreviousFormatBackTrace == 0);
PreviousFormatBackTrace = SetFormatBackTraceFn(ActorFormatBackTrace);
}
@@ -30,7 +30,7 @@ namespace NActors {
--ActorBackTraceEnableCounter;
if (ActorBackTraceEnableCounter == 0) {
- Y_VERIFY(PreviousFormatBackTrace);
+ Y_ABORT_UNLESS(PreviousFormatBackTrace);
SetFormatBackTraceFn(PreviousFormatBackTrace);
PreviousFormatBackTrace = 0;
}
diff --git a/library/cpp/actors/core/config.h b/library/cpp/actors/core/config.h
index ab35998241..38aaf77797 100644
--- a/library/cpp/actors/core/config.h
+++ b/library/cpp/actors/core/config.h
@@ -146,7 +146,7 @@ namespace NActors {
ui32 GetThreads(ui32 poolId) const {
auto result = GetThreadsOptional(poolId);
- Y_VERIFY(result, "undefined pool id: %" PRIu32, (ui32)poolId);
+ Y_ABORT_UNLESS(result, "undefined pool id: %" PRIu32, (ui32)poolId);
return *result;
}
};
@@ -218,7 +218,7 @@ namespace NActors {
TCpuAllocationConfig(const TCpuMask& available, const TCpuManagerConfig& cfg) {
for (const TUnitedExecutorPoolConfig& pool : cfg.United) {
- Y_VERIFY(pool.PoolId < MaxPools, "wrong PoolId of united executor pool: %s(%d)",
+ Y_ABORT_UNLESS(pool.PoolId < MaxPools, "wrong PoolId of united executor pool: %s(%d)",
pool.PoolName.c_str(), (pool.PoolId));
}
ui32 allocated[MaxPools] = {0};
@@ -240,7 +240,7 @@ namespace NActors {
}
}
for (const TUnitedExecutorPoolConfig& pool : cfg.United) {
- Y_VERIFY(allocated[pool.PoolId] > 0, "unable to allocate cpu for united executor pool: %s(%d)",
+ Y_ABORT_UNLESS(allocated[pool.PoolId] > 0, "unable to allocate cpu for united executor pool: %s(%d)",
pool.PoolName.c_str(), (pool.PoolId));
}
}
diff --git a/library/cpp/actors/core/cpu_manager.cpp b/library/cpp/actors/core/cpu_manager.cpp
index 3ec6cea5a3..df4ea486e4 100644
--- a/library/cpp/actors/core/cpu_manager.cpp
+++ b/library/cpp/actors/core/cpu_manager.cpp
@@ -17,7 +17,7 @@ namespace NActors {
Executors.Reset(setup->Executors.Release());
for (ui32 excIdx = 0; excIdx != ExecutorPoolCount; ++excIdx) {
IExecutorPool* pool = Executors[excIdx].Get();
- Y_VERIFY(dynamic_cast<TUnitedExecutorPool*>(pool) == nullptr,
+ Y_ABORT_UNLESS(dynamic_cast<TUnitedExecutorPool*>(pool) == nullptr,
"united executor pool is prohibited in explicit mode of NActors::TCpuManager");
}
} else {
@@ -103,7 +103,7 @@ namespace NActors {
void TCpuManager::Cleanup() {
for (ui32 round = 0, done = 0; done < ExecutorPoolCount; ++round) {
- Y_VERIFY(round < 10, "actorsystem cleanup could not be completed in 10 rounds");
+ Y_ABORT_UNLESS(round < 10, "actorsystem cleanup could not be completed in 10 rounds");
done = 0;
for (ui32 excIdx = 0; excIdx != ExecutorPoolCount; ++excIdx) {
if (Executors[excIdx]->Cleanup()) {
diff --git a/library/cpp/actors/core/event.h b/library/cpp/actors/core/event.h
index 67ab519945..4f6aefc0ee 100644
--- a/library/cpp/actors/core/event.h
+++ b/library/cpp/actors/core/event.h
@@ -139,8 +139,8 @@ namespace NActors {
}
static ui32 MakeFlags(ui32 channel, TEventFlags flags) {
- Y_VERIFY(channel < (1 << ChannelBits));
- Y_VERIFY(flags < (1 << ChannelShift));
+ Y_ABORT_UNLESS(channel < (1 << ChannelBits));
+ Y_ABORT_UNLESS(flags < (1 << ChannelShift));
return (flags | (channel << ChannelShift));
}
diff --git a/library/cpp/actors/core/event_load.h b/library/cpp/actors/core/event_load.h
index 0062ee40db..c776026cc4 100644
--- a/library/cpp/actors/core/event_load.h
+++ b/library/cpp/actors/core/event_load.h
@@ -93,7 +93,7 @@ namespace NActors {
}
TRope EraseBack(size_t count) {
- Y_VERIFY(count <= Rope.GetSize());
+ Y_ABORT_UNLESS(count <= Rope.GetSize());
TRope::TIterator iter = Rope.End();
iter -= count;
return Rope.Extract(iter, Rope.End());
diff --git a/library/cpp/actors/core/event_pb.cpp b/library/cpp/actors/core/event_pb.cpp
index a2bf14aa02..a72c52efc2 100644
--- a/library/cpp/actors/core/event_pb.cpp
+++ b/library/cpp/actors/core/event_pb.cpp
@@ -15,7 +15,7 @@ namespace NActors {
}
void TRopeStream::BackUp(int count) {
- Y_VERIFY(count <= TotalByteCount);
+ Y_ABORT_UNLESS(count <= TotalByteCount);
Iter -= count;
TotalByteCount -= count;
}
@@ -39,7 +39,7 @@ namespace NActors {
TCoroutineChunkSerializer::~TCoroutineChunkSerializer() {
CancelFlag = true;
Resume();
- Y_VERIFY(Finished);
+ Y_ABORT_UNLESS(Finished);
}
bool TCoroutineChunkSerializer::AllowsAliasing() const {
@@ -47,7 +47,7 @@ namespace NActors {
}
void TCoroutineChunkSerializer::Produce(const void *data, size_t size) {
- Y_VERIFY(size <= SizeRemain);
+ Y_ABORT_UNLESS(size <= SizeRemain);
SizeRemain -= size;
TotalSerializedDataSize += size;
@@ -63,9 +63,9 @@ namespace NActors {
}
bool TCoroutineChunkSerializer::WriteAliasedRaw(const void* data, int size) {
- Y_VERIFY(!CancelFlag);
- Y_VERIFY(!AbortFlag);
- Y_VERIFY(size >= 0);
+ Y_ABORT_UNLESS(!CancelFlag);
+ Y_ABORT_UNLESS(!AbortFlag);
+ Y_ABORT_UNLESS(size >= 0);
while (size) {
if (const size_t bytesToAppend = Min<size_t>(size, SizeRemain)) {
const void *produce = data;
@@ -89,15 +89,15 @@ namespace NActors {
}
bool TCoroutineChunkSerializer::Next(void** data, int* size) {
- Y_VERIFY(!CancelFlag);
- Y_VERIFY(!AbortFlag);
+ Y_ABORT_UNLESS(!CancelFlag);
+ Y_ABORT_UNLESS(!AbortFlag);
if (!SizeRemain) {
InnerContext.SwitchTo(BufFeedContext);
if (CancelFlag || AbortFlag) {
return false;
}
}
- Y_VERIFY(SizeRemain);
+ Y_ABORT_UNLESS(SizeRemain);
*data = BufferPtr;
*size = SizeRemain;
BufferPtr += SizeRemain;
@@ -109,11 +109,11 @@ namespace NActors {
if (!count) {
return;
}
- Y_VERIFY(count > 0);
- Y_VERIFY(!Chunks.empty());
+ Y_ABORT_UNLESS(count > 0);
+ Y_ABORT_UNLESS(!Chunks.empty());
TChunk& buf = Chunks.back();
- Y_VERIFY((size_t)count <= buf.second);
- Y_VERIFY(buf.first + buf.second == BufferPtr, "buf# %p:%zu BufferPtr# %p SizeRemain# %zu NumChunks# %zu",
+ Y_ABORT_UNLESS((size_t)count <= buf.second);
+ Y_ABORT_UNLESS(buf.first + buf.second == BufferPtr, "buf# %p:%zu BufferPtr# %p SizeRemain# %zu NumChunks# %zu",
buf.first, buf.second, BufferPtr, SizeRemain, Chunks.size());
buf.second -= count;
if (!buf.second) {
@@ -151,7 +151,7 @@ namespace NActors {
Y_VERIFY_DEBUG(size);
// transfer control to the coroutine
- Y_VERIFY(Event);
+ Y_ABORT_UNLESS(Event);
Chunks.clear();
Resume();
@@ -159,21 +159,21 @@ namespace NActors {
}
void TCoroutineChunkSerializer::SetSerializingEvent(const IEventBase *event) {
- Y_VERIFY(Event == nullptr);
+ Y_ABORT_UNLESS(Event == nullptr);
Event = event;
TotalSerializedDataSize = 0;
AbortFlag = false;
}
void TCoroutineChunkSerializer::Abort() {
- Y_VERIFY(Event);
+ Y_ABORT_UNLESS(Event);
AbortFlag = true;
Resume();
}
void TCoroutineChunkSerializer::DoRun() {
while (!CancelFlag) {
- Y_VERIFY(Event);
+ Y_ABORT_UNLESS(Event);
SerializationSuccess = !AbortFlag && Event->SerializeToArcadiaStream(this);
Event = nullptr;
if (!CancelFlag) { // cancel flag may have been received during serialization
@@ -208,7 +208,7 @@ namespace NActors {
}
bool TAllocChunkSerializer::WriteAliasedRaw(const void*, int) {
- Y_VERIFY(false);
+ Y_ABORT_UNLESS(false);
return false;
}
diff --git a/library/cpp/actors/core/event_pb.h b/library/cpp/actors/core/event_pb.h
index a4fdb33fb4..b6789df2fc 100644
--- a/library/cpp/actors/core/event_pb.h
+++ b/library/cpp/actors/core/event_pb.h
@@ -303,7 +303,7 @@ namespace NActors {
}
const TRope& GetPayload(ui32 id) const {
- Y_VERIFY(id < Payload.size());
+ Y_ABORT_UNLESS(id < Payload.size());
return Payload[id];
}
@@ -352,7 +352,7 @@ namespace NActors {
total += section.Size;
}
size_t serialized = CalculateSerializedSize();
- Y_VERIFY(total == serialized, "total# %zu serialized# %zu byteSize# %zd Payload.size# %zu", total,
+ Y_ABORT_UNLESS(total == serialized, "total# %zu serialized# %zu byteSize# %zd Payload.size# %zu", total,
serialized, byteSize, Payload.size());
#endif
}
diff --git a/library/cpp/actors/core/events_undelivered.cpp b/library/cpp/actors/core/events_undelivered.cpp
index 70b9ea2d71..7804d5c09d 100644
--- a/library/cpp/actors/core/events_undelivered.cpp
+++ b/library/cpp/actors/core/events_undelivered.cpp
@@ -7,7 +7,7 @@ namespace NActors {
}
bool TEvents::TEvUndelivered::SerializeToArcadiaStream(TChunkSerializer *serializer) const {
- Y_VERIFY(!Unsure); // these are local-only events generated by Interconnect
+ Y_ABORT_UNLESS(!Unsure); // these are local-only events generated by Interconnect
return serializer->WriteString(&Data);
}
@@ -31,7 +31,7 @@ namespace NActors {
IEventBase* TEvents::TEvUndelivered::Load(TEventSerializedData* bufs) {
TString str = bufs->GetString();
- Y_VERIFY(str.size() == (sizeof(ui32) + sizeof(ui32)));
+ Y_ABORT_UNLESS(str.size() == (sizeof(ui32) + sizeof(ui32)));
const char* p = str.data();
const ui64 sourceType = ReadUnaligned<ui32>(p + 0);
const ui64 reason = ReadUnaligned<ui32>(p + 4);
diff --git a/library/cpp/actors/core/executor_pool_base.cpp b/library/cpp/actors/core/executor_pool_base.cpp
index ba7ab0e7be..3658736a7c 100644
--- a/library/cpp/actors/core/executor_pool_base.cpp
+++ b/library/cpp/actors/core/executor_pool_base.cpp
@@ -39,8 +39,8 @@ namespace NActors {
}
auto accountUsage = [&](ui32 activityType, double usage) {
- Y_VERIFY(0 <= usage);
- Y_VERIFY(usage <= 1);
+ Y_ABORT_UNLESS(0 <= usage);
+ Y_ABORT_UNLESS(usage <= 1);
int bin = Min<int>(9, usage * 10);
++stats.UsageByActivity[activityType][bin];
};
@@ -50,7 +50,7 @@ namespace NActors {
with_lock (StuckObserverMutex) {
for (size_t i = 0; i < Actors.size(); ++i) {
IActor *actor = Actors[i];
- Y_VERIFY(actor->StuckIndex == i);
+ Y_ABORT_UNLESS(actor->StuckIndex == i);
const TDuration delta = now - actor->LastReceiveTimestamp;
if (delta > TDuration::Seconds(30)) {
++stats.StuckActorsByActivity[actor->GetActivityType()];
@@ -137,7 +137,7 @@ namespace NActors {
Y_VERIFY_DEBUG(at < Stats.ActorsAliveByActivity.size());
if (at >= Stats.MaxActivityType()) {
at = TActorTypeOperator::GetActorActivityIncorrectIndex();
- Y_VERIFY(at < Stats.ActorsAliveByActivity.size());
+ Y_ABORT_UNLESS(at < Stats.ActorsAliveByActivity.size());
}
AtomicIncrement(Stats.ActorsAliveByActivity[at]);
#endif
@@ -179,7 +179,7 @@ namespace NActors {
#ifdef ACTORSLIB_COLLECT_EXEC_STATS
if (ActorSystem->MonitorStuckActors()) {
with_lock (StuckObserverMutex) {
- Y_VERIFY(actor->StuckIndex == Max<size_t>());
+ Y_ABORT_UNLESS(actor->StuckIndex == Max<size_t>());
actor->StuckIndex = Actors.size();
Actors.push_back(actor);
}
@@ -236,7 +236,7 @@ namespace NActors {
#ifdef ACTORSLIB_COLLECT_EXEC_STATS
if (ActorSystem->MonitorStuckActors()) {
with_lock (StuckObserverMutex) {
- Y_VERIFY(actor->StuckIndex == Max<size_t>());
+ Y_ABORT_UNLESS(actor->StuckIndex == Max<size_t>());
actor->StuckIndex = Actors.size();
Actors.push_back(actor);
}
diff --git a/library/cpp/actors/core/executor_pool_basic.cpp b/library/cpp/actors/core/executor_pool_basic.cpp
index 54b6e26268..762ab32190 100644
--- a/library/cpp/actors/core/executor_pool_basic.cpp
+++ b/library/cpp/actors/core/executor_pool_basic.cpp
@@ -162,7 +162,7 @@ namespace NActors {
#endif
TAtomic state = AtomicLoad(&threadCtx.WaitingFlag);
- Y_VERIFY(state == TThreadCtx::WS_NONE, "WaitingFlag# %d", int(state));
+ Y_ABORT_UNLESS(state == TThreadCtx::WS_NONE, "WaitingFlag# %d", int(state));
if (SpinThreshold > 0 && !needToBlock) {
// spin configured period
diff --git a/library/cpp/actors/core/executor_pool_basic_ut.cpp b/library/cpp/actors/core/executor_pool_basic_ut.cpp
index ecc889211f..516eeda2c6 100644
--- a/library/cpp/actors/core/executor_pool_basic_ut.cpp
+++ b/library/cpp/actors/core/executor_pool_basic_ut.cpp
@@ -73,7 +73,7 @@ private:
Y_UNUSED(ev);
Action();
TAtomicBase count = AtomicDecrement(Counter);
- Y_VERIFY(count != Max<TAtomicBase>());
+ Y_ABORT_UNLESS(count != Max<TAtomicBase>());
if (count) {
Send(Receiver, new TEvMsg());
}
diff --git a/library/cpp/actors/core/executor_pool_united.cpp b/library/cpp/actors/core/executor_pool_united.cpp
index 2e689dec59..a4a9b26269 100644
--- a/library/cpp/actors/core/executor_pool_united.cpp
+++ b/library/cpp/actors/core/executor_pool_united.cpp
@@ -258,7 +258,7 @@ namespace NActors {
// Set specified pool as current, it requires scan
for (Current = 0; Current < Size && pool != Items[Current].GetPoolId(); Current++) {}
- Y_VERIFY(Current < Size);
+ Y_ABORT_UNLESS(Current < Size);
}
// Account currently running pool till now (ts)
@@ -266,7 +266,7 @@ namespace NActors {
// Skip time slice for the first run and when time goes backwards (e.g. rdtsc is reset to zero on cpu reset)
if (Y_LIKELY(Ts > 0 && Ts <= ts)) {
TPoolId pool = CurrentPool();
- Y_VERIFY(pool < MaxPools);
+ Y_ABORT_UNLESS(pool < MaxPools);
Items[Current].Account(MinVRunTs, (ts - Ts) * InvWeights[pool]);
}
Ts = ts; // propagate time
@@ -370,7 +370,7 @@ namespace NActors {
// Select pool to execute
wctx.PoolId = United->AssignedPool(wctx);
- Y_VERIFY(wctx.PoolId != CpuShared);
+ Y_ABORT_UNLESS(wctx.PoolId != CpuShared);
if (United->TryAcquireToken(wctx.PoolId)) {
return true;
}
@@ -1123,7 +1123,7 @@ namespace NActors {
// NOTE: leave gaps for not united pools (default-initialized)
for (const TUnitedExecutorPoolConfig& cfg : unitedPools) {
TPool& pool = Pools[cfg.PoolId];
- Y_VERIFY(cfg.PoolId < MaxPools);
+ Y_ABORT_UNLESS(cfg.PoolId < MaxPools);
pool.PoolId = cfg.PoolId;
pool.Concurrency = cfg.Concurrency ? cfg.Concurrency : Config.CpuCount;
pool.ExecutorPool = nullptr; // should be set later using SetupPool()
@@ -1173,7 +1173,7 @@ namespace NActors {
cpu.LocalManager->AddWorker(workerId);
// Setup worker
- Y_VERIFY(workerId < WorkerCount);
+ Y_ABORT_UNLESS(workerId < WorkerCount);
Workers[workerId].Thread.Reset(new TExecutorThread(
workerId,
cpu.CpuId,
diff --git a/library/cpp/actors/core/executor_pool_united_ut.cpp b/library/cpp/actors/core/executor_pool_united_ut.cpp
index 2871e016fe..df3e2d29d8 100644
--- a/library/cpp/actors/core/executor_pool_united_ut.cpp
+++ b/library/cpp/actors/core/executor_pool_united_ut.cpp
@@ -78,7 +78,7 @@ private:
Y_UNUSED(ev);
Action();
TAtomicBase count = AtomicDecrement(Counter);
- Y_VERIFY(count != Max<TAtomicBase>());
+ Y_ABORT_UNLESS(count != Max<TAtomicBase>());
if (count) {
Send(Receiver, new TEvMsg());
}
diff --git a/library/cpp/actors/core/interconnect.cpp b/library/cpp/actors/core/interconnect.cpp
index 192fb68769..8671fb2db7 100644
--- a/library/cpp/actors/core/interconnect.cpp
+++ b/library/cpp/actors/core/interconnect.cpp
@@ -59,14 +59,14 @@ namespace NActors {
for (int i = 0, count = descriptor->field_count(); i < count; ++i) {
const NProtoBuf::FieldDescriptor *field = descriptor->field(i);
if (reflection->HasField(*locp, field)) {
- Y_VERIFY(field->type() == NProtoBuf::FieldDescriptor::TYPE_STRING, "Location# %s", makeString().data());
+ Y_ABORT_UNLESS(field->type() == NProtoBuf::FieldDescriptor::TYPE_STRING, "Location# %s", makeString().data());
Items.emplace_back(TKeys::E(field->number()), reflection->GetString(*locp, field));
}
}
const NProtoBuf::UnknownFieldSet& unknown = locp->unknown_fields();
for (int i = 0, count = unknown.field_count(); i < count; ++i) {
const NProtoBuf::UnknownField& field = unknown.field(i);
- Y_VERIFY(field.type() == NProtoBuf::UnknownField::TYPE_LENGTH_DELIMITED, "Location# %s", makeString().data());
+ Y_ABORT_UNLESS(field.type() == NProtoBuf::UnknownField::TYPE_LENGTH_DELIMITED, "Location# %s", makeString().data());
Items.emplace_back(TKeys::E(field.number()), field.length_delimited());
}
std::sort(Items.begin(), Items.end());
@@ -86,7 +86,7 @@ namespace NActors {
NActorsInterconnect::TNodeLocation TNodeLocation::ParseLocation(const TString& s) {
NActorsInterconnect::TNodeLocation res;
const bool success = res.ParseFromString(s);
- Y_VERIFY(success);
+ Y_ABORT_UNLESS(success);
return res;
}
@@ -134,7 +134,7 @@ namespace NActors {
Serialize(&pb, false);
TString s;
const bool success = pb.SerializeToString(&s);
- Y_VERIFY(success);
+ Y_ABORT_UNLESS(success);
return s;
}
@@ -153,7 +153,7 @@ namespace NActors {
case TKeys::Module: {
const bool success = TryFromString(value, moduleId);
- Y_VERIFY(success);
+ Y_ABORT_UNLESS(success);
break;
}
@@ -166,7 +166,7 @@ namespace NActors {
case TKeys::Unit: {
const bool success = TryFromString(value, unitId);
- Y_VERIFY(success);
+ Y_ABORT_UNLESS(success);
break;
}
diff --git a/library/cpp/actors/core/invoke.h b/library/cpp/actors/core/invoke.h
index e22d500853..6151c32bfa 100644
--- a/library/cpp/actors/core/invoke.h
+++ b/library/cpp/actors/core/invoke.h
@@ -50,7 +50,7 @@ namespace NActors {
if constexpr (std::is_same_v<TArg, std::exception_ptr>) {
std::rethrow_exception(arg);
} else if constexpr (std::is_void_v<T>) {
- Y_VERIFY(!arg.has_value());
+ Y_ABORT_UNLESS(!arg.has_value());
} else if (auto *value = std::any_cast<T>(&arg)) {
return std::move(*value);
} else {
@@ -122,7 +122,7 @@ namespace NActors {
}
void StateFunc(STFUNC_SIG) {
- Y_VERIFY(ev->GetTypeRewrite() == TEvents::TSystem::Wakeup);
+ Y_ABORT_UNLESS(ev->GetTypeRewrite() == TEvents::TSystem::Wakeup);
auto g = TBase::PassAwayGuard();
Executor();
}
diff --git a/library/cpp/actors/core/io_dispatcher.cpp b/library/cpp/actors/core/io_dispatcher.cpp
index 6bd753f2e0..c7d28c63e0 100644
--- a/library/cpp/actors/core/io_dispatcher.cpp
+++ b/library/cpp/actors/core/io_dispatcher.cpp
@@ -79,7 +79,7 @@ namespace NActors {
void StopOne() {
with_lock (Mutex) {
++NumThreadsToStop;
- Y_VERIFY(NumThreadsToStop);
+ Y_ABORT_UNLESS(NumThreadsToStop);
}
CondVar.Signal();
}
@@ -144,7 +144,7 @@ namespace NActors {
}
void StopThread() {
- Y_VERIFY(Threads.size());
+ Y_ABORT_UNLESS(Threads.size());
TaskQueue.StopOne();
*NumThreads = --NumRunningThreads;
++*ThreadsStopped;
@@ -185,7 +185,7 @@ namespace NActors {
void HandleThreadStopped(TAutoPtr<IEventHandle> ev) {
auto it = Threads.find(ev->Cookie);
- Y_VERIFY(it != Threads.end());
+ Y_ABORT_UNLESS(it != Threads.end());
it->second->Join();
Threads.erase(it);
}
diff --git a/library/cpp/actors/core/log.cpp b/library/cpp/actors/core/log.cpp
index 71b5d46c22..18f43a2fed 100644
--- a/library/cpp/actors/core/log.cpp
+++ b/library/cpp/actors/core/log.cpp
@@ -529,7 +529,7 @@ namespace NActors {
struct tm localTime;
time.LocalTime(&localTime);
int r = strftime(buf, TimeBufSize, "%Y-%m-%d-%H-%M-%S", &localTime);
- Y_VERIFY(r != 0);
+ Y_ABORT_UNLESS(r != 0);
return buf;
}
@@ -671,7 +671,7 @@ namespace NActors {
}
void Pop(const TLogContextGuard& context) {
- Y_VERIFY(Stack.size() && Stack.back().GetId() == context.GetId());
+ Y_ABORT_UNLESS(Stack.size() && Stack.back().GetId() == context.GetId());
Stack.pop_back();
}
diff --git a/library/cpp/actors/core/log_settings.cpp b/library/cpp/actors/core/log_settings.cpp
index 321af18d71..fafaf892eb 100644
--- a/library/cpp/actors/core/log_settings.cpp
+++ b/library/cpp/actors/core/log_settings.cpp
@@ -51,8 +51,8 @@ namespace NActors {
}
void TSettings::Append(EComponent minVal, EComponent maxVal, EComponentToStringFunc func) {
- Y_VERIFY(minVal >= 0, "NLog::TSettings: minVal must be non-negative");
- Y_VERIFY(maxVal > minVal, "NLog::TSettings: maxVal must be greater than minVal");
+ Y_ABORT_UNLESS(minVal >= 0, "NLog::TSettings: minVal must be non-negative");
+ Y_ABORT_UNLESS(maxVal > minVal, "NLog::TSettings: maxVal must be greater than minVal");
// update bounds
if (!MaxVal || minVal < MinVal) {
@@ -83,7 +83,7 @@ namespace NActors {
// assign new names but validate if newly added members were not used before
for (int i = minVal; i <= maxVal; i++) {
- Y_VERIFY(!ComponentNames[i], "component name at %d already set: %s",
+ Y_ABORT_UNLESS(!ComponentNames[i], "component name at %d already set: %s",
i, ComponentNames[i].data());
ComponentNames[i] = func(i);
}
diff --git a/library/cpp/actors/core/mailbox.cpp b/library/cpp/actors/core/mailbox.cpp
index b28fdc0771..de746f0ecb 100644
--- a/library/cpp/actors/core/mailbox.cpp
+++ b/library/cpp/actors/core/mailbox.cpp
@@ -45,7 +45,7 @@ namespace NActors {
const ui32 sx = TMailbox::AlignedSize();
for (ui8* x = begin; x + sx <= end; x += sx) {
TMailbox* mailbox = reinterpret_cast<TMailbox*>(x);
- Y_VERIFY(IsGoodForCleanup(mailbox));
+ Y_ABORT_UNLESS(IsGoodForCleanup(mailbox));
mailbox->ExecutionState = Max<ui32>();
mailbox->~TMailbox();
}
@@ -57,7 +57,7 @@ namespace NActors {
bool done = true;
for (ui8* x = begin; x + sx <= end; x += sx) {
TMailbox* mailbox = reinterpret_cast<TMailbox*>(x);
- Y_VERIFY(IsGoodForCleanup(mailbox));
+ Y_ABORT_UNLESS(IsGoodForCleanup(mailbox));
done &= mailbox->CleanupActors() && mailbox->CleanupEvents();
}
return done;
@@ -138,7 +138,7 @@ namespace NActors {
const ui32 lineIndex = (hint & LineIndexMask) >> LineIndexShift;
const ui32 lineHint = hint & LineHintMask;
- Y_VERIFY((lineIndex < MaxLines) && (lineHint < LineSize / 64));
+ Y_ABORT_UNLESS((lineIndex < MaxLines) && (lineHint < LineSize / 64));
if (lineHint == 0)
return nullptr;
@@ -172,7 +172,7 @@ namespace NActors {
const ui32 lineIndex = (hint & LineIndexMask) >> LineIndexShift;
const ui32 lineHint = hint & LineHintMask;
- Y_VERIFY((lineIndex < MaxLines) && (lineHint < LineSize / 64));
+ Y_ABORT_UNLESS((lineIndex < MaxLines) && (lineHint < LineSize / 64));
if (lineHint == 0)
return false;
diff --git a/library/cpp/actors/core/mailbox.h b/library/cpp/actors/core/mailbox.h
index 0f1c3abc10..0daafa83ac 100644
--- a/library/cpp/actors/core/mailbox.h
+++ b/library/cpp/actors/core/mailbox.h
@@ -222,7 +222,7 @@ namespace NActors {
switch (ActorPack) {
case TMailboxActorPack::Simple: {
- Y_VERIFY(ActorsInfo.Simple.ActorId == localActorId);
+ Y_ABORT_UNLESS(ActorsInfo.Simple.ActorId == localActorId);
actorToDestruct = ActorsInfo.Simple.Actor;
ActorsInfo.Simple.ActorId = 0;
@@ -231,7 +231,7 @@ namespace NActors {
}
case TMailboxActorPack::Map: {
TActorMap::iterator it = ActorsInfo.Map.ActorsMap->find(localActorId);
- Y_VERIFY(it != ActorsInfo.Map.ActorsMap->end());
+ Y_ABORT_UNLESS(it != ActorsInfo.Map.ActorsMap->end());
actorToDestruct = it->second;
ActorsInfo.Map.ActorsMap->erase(it);
@@ -260,7 +260,7 @@ namespace NActors {
break;
}
}
- Y_VERIFY(found);
+ Y_ABORT_UNLESS(found);
if (ActorsInfo.Array.ActorsCount == 1) {
const TActorPair Actor = ActorsInfo.Array.ActorsArray->Actors[0];
diff --git a/library/cpp/actors/core/mailbox_queue_revolving.h b/library/cpp/actors/core/mailbox_queue_revolving.h
index b0e78a18db..22b0332095 100644
--- a/library/cpp/actors/core/mailbox_queue_revolving.h
+++ b/library/cpp/actors/core/mailbox_queue_revolving.h
@@ -173,7 +173,7 @@ namespace NActors {
}
bool TryPush(T x) {
- Y_VERIFY(x != 0);
+ Y_ABORT_UNLESS(x != 0);
for (ui32 i = 0; i != TWriteConcurrency; ++i) {
if (RelaxedLoad(&WriteTo[i]) != nullptr) {
diff --git a/library/cpp/actors/core/mon.h b/library/cpp/actors/core/mon.h
index ca2001e5f6..ba5debbd17 100644
--- a/library/cpp/actors/core/mon.h
+++ b/library/cpp/actors/core/mon.h
@@ -96,7 +96,7 @@ namespace NActors {
static TString MakeSerializedQuery(const NActorsProto::TRemoteHttpInfo& info) {
TString s(1, '\0');
const bool success = info.AppendToString(&s);
- Y_VERIFY(success);
+ Y_ABORT_UNLESS(success);
return s;
}
@@ -158,7 +158,7 @@ namespace NActors {
res->Query = s;
res->ExtendedQuery.emplace();
const bool success = res->ExtendedQuery->ParseFromZeroCopyStream(&stream);
- Y_VERIFY(success);
+ Y_ABORT_UNLESS(success);
return res.release();
} else {
return new TEvRemoteHttpInfo(s);
diff --git a/library/cpp/actors/core/mon_ut.cpp b/library/cpp/actors/core/mon_ut.cpp
index a2991e8a10..fa5dbbe71e 100644
--- a/library/cpp/actors/core/mon_ut.cpp
+++ b/library/cpp/actors/core/mon_ut.cpp
@@ -16,7 +16,7 @@ Y_UNIT_TEST_SUITE(ActorSystemMon) {
TAllocChunkSerializer ser;
const bool success = ev->SerializeToArcadiaStream(&ser);
- Y_VERIFY(success);
+ Y_ABORT_UNLESS(success);
auto buffer = ser.Release(ev->CreateSerializationInfo());
std::unique_ptr<TEvRemoteHttpInfo> restored(dynamic_cast<TEvRemoteHttpInfo*>(TEvRemoteHttpInfo::Load(buffer.Get())));
UNIT_ASSERT(restored->Query == ev->Query);
diff --git a/library/cpp/actors/core/performance_ut.cpp b/library/cpp/actors/core/performance_ut.cpp
index 51d10a5e80..3c1a0ed143 100644
--- a/library/cpp/actors/core/performance_ut.cpp
+++ b/library/cpp/actors/core/performance_ut.cpp
@@ -100,7 +100,7 @@ Y_UNIT_TEST_SUITE(ActorSystemPerformance) {
switch (ev->GetTypeRewrite()) {
HFunc(TEventLocalDolbilkaOld, Handle);
default:
- Y_VERIFY(false);
+ Y_ABORT_UNLESS(false);
}
}
@@ -162,13 +162,13 @@ Y_UNIT_TEST_SUITE(ActorSystemPerformance) {
while (dNew->GetDurationInProgress() < TDuration::Seconds(1000) && !dNew->IsFinished()) {
Sleep(TDuration::Seconds(1));
}
- Y_VERIFY(dNew->IsFinished());
+ Y_ABORT_UNLESS(dNew->IsFinished());
TDolbilkaOld* dOld = new TDolbilkaOld;
runtime->Register(dOld);
while (dOld->GetDurationInProgress() < TDuration::Seconds(1000) && !dOld->IsFinished()) {
Sleep(TDuration::Seconds(1));
}
- Y_VERIFY(dOld->IsFinished());
+ Y_ABORT_UNLESS(dOld->IsFinished());
std::unique_ptr<TDolbilkaSimple> dSimple(new TDolbilkaSimple);
IDolbilkaSimple* dSimpleIface = dSimple.get();
while (dSimpleIface->ProcessEvent()) {
diff --git a/library/cpp/actors/core/scheduler_actor.cpp b/library/cpp/actors/core/scheduler_actor.cpp
index db2bcf2791..513505b5d7 100644
--- a/library/cpp/actors/core/scheduler_actor.cpp
+++ b/library/cpp/actors/core/scheduler_actor.cpp
@@ -21,7 +21,7 @@ namespace NActors {
TTimerDescriptor()
: Descriptor(timerfd_create(CLOCK_MONOTONIC, TFD_NONBLOCK))
{
- Y_VERIFY(Descriptor != -1, "timerfd_create() failed with %s", strerror(errno));
+ Y_ABORT_UNLESS(Descriptor != -1, "timerfd_create() failed with %s", strerror(errno));
}
~TTimerDescriptor() override {
@@ -92,9 +92,9 @@ namespace NActors {
new_time.it_value.tv_nsec = Cfg.ResolutionMicroseconds * 1000;
new_time.it_interval.tv_nsec = Cfg.ResolutionMicroseconds * 1000;
int ret = timerfd_settime(TimerDescriptor->GetDescriptor(), 0, &new_time, NULL);
- Y_VERIFY(ret != -1, "timerfd_settime() failed with %s", strerror(errno));
+ Y_ABORT_UNLESS(ret != -1, "timerfd_settime() failed with %s", strerror(errno));
const bool success = ctx.Send(PollerActor, new TEvPollerRegister(TimerDescriptor, SelfId(), {}));
- Y_VERIFY(success);
+ Y_ABORT_UNLESS(success);
RealTime = RelaxedLoad(CurrentTimestamp);
MonotonicTime = RelaxedLoad(CurrentMonotonic);
@@ -141,7 +141,7 @@ namespace NActors {
continue;
}
}
- Y_VERIFY(bytesRead == sizeof(expired), "Error while reading from timerfd, strerror# %s", strerror(errno));
+ Y_ABORT_UNLESS(bytesRead == sizeof(expired), "Error while reading from timerfd, strerror# %s", strerror(errno));
UpdateTime();
ui32 eventsGottenFromQueues = 0;
diff --git a/library/cpp/actors/core/scheduler_basic.cpp b/library/cpp/actors/core/scheduler_basic.cpp
index 020f640ac0..3b4e0a08c1 100644
--- a/library/cpp/actors/core/scheduler_basic.cpp
+++ b/library/cpp/actors/core/scheduler_basic.cpp
@@ -43,11 +43,11 @@ namespace NActors {
, StopFlag(false)
, ScheduleMap(3600)
{
- Y_VERIFY(!Config.UseSchedulerActor, "Cannot create scheduler thread because Config.UseSchedulerActor# true");
+ Y_ABORT_UNLESS(!Config.UseSchedulerActor, "Cannot create scheduler thread because Config.UseSchedulerActor# true");
}
TBasicSchedulerThread::~TBasicSchedulerThread() {
- Y_VERIFY(!MainCycle);
+ Y_ABORT_UNLESS(!MainCycle);
}
void TBasicSchedulerThread::CycleFunc() {
@@ -220,7 +220,7 @@ namespace NActors {
}
void TBasicSchedulerThread::PrepareSchedules(NSchedulerQueue::TReader** readers, ui32 scheduleReadersCount) {
- Y_VERIFY(scheduleReadersCount > 0);
+ Y_ABORT_UNLESS(scheduleReadersCount > 0);
TotalReaders = scheduleReadersCount;
Readers.Reset(new NSchedulerQueue::TReader*[scheduleReadersCount]);
Copy(readers, readers + scheduleReadersCount, Readers.Get());
diff --git a/library/cpp/actors/cppcoro/corobenchmark/main.cpp b/library/cpp/actors/cppcoro/corobenchmark/main.cpp
index 20b4d63243..49504e7105 100644
--- a/library/cpp/actors/cppcoro/corobenchmark/main.cpp
+++ b/library/cpp/actors/cppcoro/corobenchmark/main.cpp
@@ -62,7 +62,7 @@ Y_CPU_BENCHMARK(TaskCalls, iface) {
AwaitThenCallback(IterateTaskValues(iface.Iterations()), [&]{
finished = true;
});
- Y_VERIFY(finished);
+ Y_ABORT_UNLESS(finished);
}
Y_CPU_BENCHMARK(CoroAwaits, iface) {
diff --git a/library/cpp/actors/cppcoro/task_actor.cpp b/library/cpp/actors/cppcoro/task_actor.cpp
index d55db4eb04..8a9451c8e5 100644
--- a/library/cpp/actors/cppcoro/task_actor.cpp
+++ b/library/cpp/actors/cppcoro/task_actor.cpp
@@ -11,7 +11,7 @@ namespace NActors {
struct TCurrentTaskActorGuard {
TCurrentTaskActorGuard(TTaskActorImpl* current) noexcept {
- Y_VERIFY(TlsCurrentTaskActor == nullptr);
+ Y_ABORT_UNLESS(TlsCurrentTaskActor == nullptr);
TlsCurrentTaskActor = current;
}
@@ -56,7 +56,7 @@ namespace NActors {
: TActor(&TThis::StateBoot)
, Task(std::move(task))
{
- Y_VERIFY(Task);
+ Y_ABORT_UNLESS(Task);
}
~TTaskActorImpl() {
@@ -74,7 +74,7 @@ namespace NActors {
}
STATEFN(StateBoot) {
- Y_VERIFY(ev->GetTypeRewrite() == TEvents::TSystem::Bootstrap, "Expected bootstrap event");
+ Y_ABORT_UNLESS(ev->GetTypeRewrite() == TEvents::TSystem::Bootstrap, "Expected bootstrap event");
TCurrentTaskActorGuard guard(this);
Become(&TThis::StateWork);
AwaitThenCallback(std::move(Task).WhenDone(),
@@ -94,7 +94,7 @@ namespace NActors {
switch (ev->GetTypeRewrite()) {
hFunc(TEvResumeTask, Handle);
default:
- Y_VERIFY(EventAwaiter);
+ Y_ABORT_UNLESS(EventAwaiter);
Event.reset(ev.Release());
std::exchange(EventAwaiter, {}).resume();
}
@@ -109,17 +109,17 @@ namespace NActors {
bool Check() {
if (Result->Finished) {
- Y_VERIFY(!EventAwaiter, "Task terminated while waiting for the next event");
+ Y_ABORT_UNLESS(!EventAwaiter, "Task terminated while waiting for the next event");
PassAway();
return false;
}
- Y_VERIFY(EventAwaiter, "Task suspended without waiting for the next event");
+ Y_ABORT_UNLESS(EventAwaiter, "Task suspended without waiting for the next event");
return true;
}
void WaitForEvent(std::coroutine_handle<> h) noexcept {
- Y_VERIFY(!EventAwaiter, "Task cannot have multiple awaiters for the next event");
+ Y_ABORT_UNLESS(!EventAwaiter, "Task cannot have multiple awaiters for the next event");
EventAwaiter = h;
}
@@ -127,7 +127,7 @@ namespace NActors {
if (Stopped) {
throw TTaskCancelled();
}
- Y_VERIFY(Event, "Task does not have current event");
+ Y_ABORT_UNLESS(Event, "Task does not have current event");
return std::move(Event);
}
@@ -141,12 +141,12 @@ namespace NActors {
};
void TTaskActorNextEvent::await_suspend(std::coroutine_handle<> h) noexcept {
- Y_VERIFY(TlsCurrentTaskActor, "Not in a task actor context");
+ Y_ABORT_UNLESS(TlsCurrentTaskActor, "Not in a task actor context");
TlsCurrentTaskActor->WaitForEvent(h);
}
std::unique_ptr<IEventHandle> TTaskActorNextEvent::await_resume() {
- Y_VERIFY(TlsCurrentTaskActor, "Not in a task actor context");
+ Y_ABORT_UNLESS(TlsCurrentTaskActor, "Not in a task actor context");
return TlsCurrentTaskActor->FinishWaitForEvent();
}
@@ -155,17 +155,17 @@ namespace NActors {
}
TActorIdentity TTaskActor::SelfId() noexcept {
- Y_VERIFY(TlsCurrentTaskActor, "Not in a task actor context");
+ Y_ABORT_UNLESS(TlsCurrentTaskActor, "Not in a task actor context");
return TlsCurrentTaskActor->SelfId();
}
TActorId TTaskActor::ParentId() noexcept {
- Y_VERIFY(TlsCurrentTaskActor, "Not in a task actor context");
+ Y_ABORT_UNLESS(TlsCurrentTaskActor, "Not in a task actor context");
return TlsCurrentTaskActor->ParentId;
}
void TAfterAwaiter::await_suspend(std::coroutine_handle<> h) noexcept {
- Y_VERIFY(TlsCurrentTaskActor, "Not in a task actor context");
+ Y_ABORT_UNLESS(TlsCurrentTaskActor, "Not in a task actor context");
TlsCurrentTaskActor->Schedule(Duration, new TEvResumeTask(h, &Result));
}
diff --git a/library/cpp/actors/cppcoro/task_actor_ut.cpp b/library/cpp/actors/cppcoro/task_actor_ut.cpp
index 8037dd418d..79d68b64c2 100644
--- a/library/cpp/actors/cppcoro/task_actor_ut.cpp
+++ b/library/cpp/actors/cppcoro/task_actor_ut.cpp
@@ -27,7 +27,7 @@ Y_UNIT_TEST_SUITE(TaskActor) {
TTask<void> SimpleResponder() {
for (;;) {
auto ev = co_await TTaskActor::NextEvent;
- Y_VERIFY(ev->GetTypeRewrite() == TEvRequest::EventType);
+ Y_ABORT_UNLESS(ev->GetTypeRewrite() == TEvRequest::EventType);
auto* msg = ev->Get<TEvRequest>();
Y_UNUSED(msg);
TTaskActor::SelfId().Send(ev->Sender, new TEvResponse);
diff --git a/library/cpp/actors/cppcoro/task_group.h b/library/cpp/actors/cppcoro/task_group.h
index b2496f57eb..0f7a91262b 100644
--- a/library/cpp/actors/cppcoro/task_group.h
+++ b/library/cpp/actors/cppcoro/task_group.h
@@ -77,8 +77,8 @@ namespace NActors {
// Continuation may wake up on another thread
return std::noop_coroutine();
}
- Y_VERIFY(currentValue != (void*)MarkerAwaiting, "TaskGroup is suspending with an awaiting marker");
- Y_VERIFY(currentValue != (void*)MarkerDetached, "TaskGroup is suspending with a detached marker");
+ Y_ABORT_UNLESS(currentValue != (void*)MarkerAwaiting, "TaskGroup is suspending with an awaiting marker");
+ Y_ABORT_UNLESS(currentValue != (void*)MarkerDetached, "TaskGroup is suspending with a detached marker");
// Race: ready queue is not actually empty
Continuation = {};
return h;
@@ -89,9 +89,9 @@ namespace NActors {
std::unique_ptr<TTaskGroupResult<T>> result;
if (ReadyQueue == nullptr) {
void* headValue = LastReady.exchange(nullptr, std::memory_order_acq_rel);
- Y_VERIFY(headValue != (void*)MarkerAwaiting, "TaskGroup is resuming with an awaiting marker");
- Y_VERIFY(headValue != (void*)MarkerDetached, "TaskGroup is resuming with a detached marker");
- Y_VERIFY(headValue, "TaskGroup is resuming with an empty queue");
+ Y_ABORT_UNLESS(headValue != (void*)MarkerAwaiting, "TaskGroup is resuming with an awaiting marker");
+ Y_ABORT_UNLESS(headValue != (void*)MarkerDetached, "TaskGroup is resuming with a detached marker");
+ Y_ABORT_UNLESS(headValue, "TaskGroup is resuming with an empty queue");
TTaskGroupResult<T>* head = reinterpret_cast<TTaskGroupResult<T>*>(headValue);
while (head) {
auto* next = std::exchange(head->Next, nullptr);
@@ -100,7 +100,7 @@ namespace NActors {
head = next;
}
}
- Y_VERIFY(ReadyQueue != nullptr);
+ Y_ABORT_UNLESS(ReadyQueue != nullptr);
result.reset(ReadyQueue);
ReadyQueue = std::exchange(result->Next, nullptr);
return result;
@@ -122,8 +122,8 @@ namespace NActors {
void Detach() noexcept {
// After this exchange all new results will be discarded
void* headValue = LastReady.exchange((void*)MarkerDetached, std::memory_order_acq_rel);
- Y_VERIFY(headValue != (void*)MarkerAwaiting, "TaskGroup is detaching with an awaiting marker");
- Y_VERIFY(headValue != (void*)MarkerDetached, "TaskGroup is detaching with a detached marker");
+ Y_ABORT_UNLESS(headValue != (void*)MarkerAwaiting, "TaskGroup is detaching with an awaiting marker");
+ Y_ABORT_UNLESS(headValue != (void*)MarkerDetached, "TaskGroup is detaching with a detached marker");
if (headValue) {
Dispose(reinterpret_cast<TTaskGroupResult<T>*>(headValue));
}
@@ -270,7 +270,7 @@ namespace NActors {
{}
bool await_ready() const noexcept {
- Y_VERIFY(TaskGroup_.TaskCount_ > 0, "Not enough tasks to await");
+ Y_ABORT_UNLESS(TaskGroup_.TaskCount_ > 0, "Not enough tasks to await");
--TaskGroup_.TaskCount_;
return TaskGroup_.Sink_->Ready();
}
diff --git a/library/cpp/actors/cppcoro/task_ut.cpp b/library/cpp/actors/cppcoro/task_ut.cpp
index 24ea9d0700..a1ed5426fc 100644
--- a/library/cpp/actors/cppcoro/task_ut.cpp
+++ b/library/cpp/actors/cppcoro/task_ut.cpp
@@ -109,13 +109,13 @@ Y_UNIT_TEST_SUITE(Task) {
}
void Resume(T result) {
- Y_VERIFY(Next && !Next.done());
+ Y_ABORT_UNLESS(Next && !Next.done());
NextResult = result;
std::exchange(Next, {}).resume();
}
void Cancel() {
- Y_VERIFY(Next && !Next.done());
+ Y_ABORT_UNLESS(Next && !Next.done());
NextResult.reset();
std::exchange(Next, {}).resume();
}
diff --git a/library/cpp/actors/dnscachelib/dnscache.cpp b/library/cpp/actors/dnscachelib/dnscache.cpp
index b72b0aea26..c24888caa5 100644
--- a/library/cpp/actors/dnscachelib/dnscache.cpp
+++ b/library/cpp/actors/dnscachelib/dnscache.cpp
@@ -445,7 +445,7 @@ TString TDnsCache::THost::AddrsV6ToString() const {
TDnsCache::TAresLibInit::TAresLibInit() {
#ifdef _win_
const auto res = ares_library_init(ARES_LIB_INIT_ALL);
- Y_VERIFY(res == 0);
+ Y_ABORT_UNLESS(res == 0);
#endif
}
diff --git a/library/cpp/actors/dnsresolver/dnsresolver.cpp b/library/cpp/actors/dnsresolver/dnsresolver.cpp
index fb66be8616..bf88d7a41c 100644
--- a/library/cpp/actors/dnsresolver/dnsresolver.cpp
+++ b/library/cpp/actors/dnsresolver/dnsresolver.cpp
@@ -19,7 +19,7 @@ namespace NDnsResolver {
protected:
TAresLibraryInitBase() noexcept {
int status = ares_library_init(ARES_LIB_INIT_ALL);
- Y_VERIFY(status == ARES_SUCCESS, "Unexpected failure to initialize c-ares library");
+ Y_ABORT_UNLESS(status == ARES_SUCCESS, "Unexpected failure to initialize c-ares library");
}
~TAresLibraryInitBase() noexcept {
@@ -31,7 +31,7 @@ namespace NDnsResolver {
protected:
TCallbackQueueBase() noexcept {
int err = SocketPair(Sockets, false, true);
- Y_VERIFY(err == 0, "Unexpected failure to create a socket pair");
+ Y_ABORT_UNLESS(err == 0, "Unexpected failure to create a socket pair");
SetNonBlock(Sockets[0]);
SetNonBlock(Sockets[1]);
}
@@ -46,7 +46,7 @@ namespace NDnsResolver {
using TCallbackQueue = NThreading::THTSwapQueue<TCallback>;
void PushCallback(TCallback callback) {
- Y_VERIFY(callback, "Cannot push an empty callback");
+ Y_ABORT_UNLESS(callback, "Cannot push an empty callback");
CallbackQueue.Push(std::move(callback)); // this is a lockfree queue
// Wake up worker thread on the first activation
@@ -56,7 +56,7 @@ namespace NDnsResolver {
#ifdef _win_
ret = send(SignalSock(), &ch, 1, 0);
if (ret == -1) {
- Y_VERIFY(WSAGetLastError() == WSAEWOULDBLOCK, "Unexpected send error");
+ Y_ABORT_UNLESS(WSAGetLastError() == WSAEWOULDBLOCK, "Unexpected send error");
return;
}
#else
@@ -64,11 +64,11 @@ namespace NDnsResolver {
ret = send(SignalSock(), &ch, 1, 0);
} while (ret == -1 && errno == EINTR);
if (ret == -1) {
- Y_VERIFY(errno == EAGAIN || errno == EWOULDBLOCK, "Unexpected send error");
+ Y_ABORT_UNLESS(errno == EAGAIN || errno == EWOULDBLOCK, "Unexpected send error");
return;
}
#endif
- Y_VERIFY(ret == 1, "Unexpected send result");
+ Y_ABORT_UNLESS(ret == 1, "Unexpected send result");
}
}
@@ -96,7 +96,7 @@ namespace NDnsResolver {
if (errno == EAGAIN || errno == EWOULDBLOCK) {
break;
}
- Y_VERIFY(errno == EINTR, "Unexpected recv error");
+ Y_ABORT_UNLESS(errno == EINTR, "Unexpected recv error");
#endif
}
@@ -105,7 +105,7 @@ namespace NDnsResolver {
// It's impossible to get signalled while Activations == 0
// We must set Activations = 0 to receive new signals
size_t count = Activations.exchange(0, std::memory_order_acq_rel);
- Y_VERIFY(count != 0);
+ Y_ABORT_UNLESS(count != 0);
// N.B. due to the way HTSwap works we may not be able to pop
// all callbacks on this activation, however we expect a new
@@ -190,7 +190,7 @@ namespace NDnsResolver {
}
int err = ares_init_options(&AresChannel, &options, optmask);
- Y_VERIFY(err == 0, "Unexpected failure to initialize c-ares channel");
+ Y_ABORT_UNLESS(err == 0, "Unexpected failure to initialize c-ares channel");
if (Options.Servers) {
TStringBuilder csv;
@@ -201,7 +201,7 @@ namespace NDnsResolver {
csv << server;
}
err = ares_set_servers_ports_csv(AresChannel, csv.c_str());
- Y_VERIFY(err == 0, "Unexpected failure to set a list of dns servers: %s", ares_strerror(err));
+ Y_ABORT_UNLESS(err == 0, "Unexpected failure to set a list of dns servers: %s", ares_strerror(err));
}
}
@@ -219,7 +219,7 @@ namespace NDnsResolver {
})
void Handle(TEvents::TEvPoison::TPtr&) {
- Y_VERIFY(!Stopped);
+ Y_ABORT_UNLESS(!Stopped);
PushCallback([this] {
// Cancel all current ares requests (will send notifications)
diff --git a/library/cpp/actors/dnsresolver/dnsresolver.h b/library/cpp/actors/dnsresolver/dnsresolver.h
index 88fc74df7d..1121c31e51 100644
--- a/library/cpp/actors/dnsresolver/dnsresolver.h
+++ b/library/cpp/actors/dnsresolver/dnsresolver.h
@@ -70,13 +70,13 @@ namespace NDnsResolver {
const TIPv6Addr& GetAddrV6() const {
const TIPv6Addr* p = std::get_if<TIPv6Addr>(&Addr);
- Y_VERIFY(p, "Result is not an ipv6 address");
+ Y_ABORT_UNLESS(p, "Result is not an ipv6 address");
return *p;
}
const TIPv4Addr& GetAddrV4() const {
const TIPv4Addr* p = std::get_if<TIPv4Addr>(&Addr);
- Y_VERIFY(p, "Result is not an ipv4 address");
+ Y_ABORT_UNLESS(p, "Result is not an ipv4 address");
return *p;
}
};
diff --git a/library/cpp/actors/dnsresolver/dnsresolver_caching.cpp b/library/cpp/actors/dnsresolver/dnsresolver_caching.cpp
index d16fc28e5f..bc3e5c3d7e 100644
--- a/library/cpp/actors/dnsresolver/dnsresolver_caching.cpp
+++ b/library/cpp/actors/dnsresolver/dnsresolver_caching.cpp
@@ -137,7 +137,7 @@ namespace NDnsResolver {
void Handle(TEvDns::TEvGetHostByNameResult::TPtr& ev) {
auto waitingIt = WaitingRequests.find(ev->Cookie);
- Y_VERIFY(waitingIt != WaitingRequests.end(), "Unexpected reply, reqId=%" PRIu64, ev->Cookie);
+ Y_ABORT_UNLESS(waitingIt != WaitingRequests.end(), "Unexpected reply, reqId=%" PRIu64, ev->Cookie);
auto waitingInfo = waitingIt->second;
WaitingRequests.erase(waitingIt);
@@ -161,7 +161,7 @@ namespace NDnsResolver {
switch (ev->Get()->SourceType) {
case TEvDns::TEvGetHostByName::EventType: {
auto waitingIt = WaitingRequests.find(ev->Cookie);
- Y_VERIFY(waitingIt != WaitingRequests.end(), "Unexpected TEvUndelivered, reqId=%" PRIu64, ev->Cookie);
+ Y_ABORT_UNLESS(waitingIt != WaitingRequests.end(), "Unexpected TEvUndelivered, reqId=%" PRIu64, ev->Cookie);
auto waitingInfo = waitingIt->second;
WaitingRequests.erase(waitingIt);
@@ -253,7 +253,7 @@ namespace NDnsResolver {
EnsureRequest(state, req->Name, family, now);
if (state.IsHardExpired(now)) {
- Y_VERIFY(state.Waiting);
+ Y_ABORT_UNLESS(state.Waiting);
if (MonCounters) {
++*MonCounters->CacheMisses;
}
@@ -396,7 +396,7 @@ namespace NDnsResolver {
auto& req = WaitingRequests[reqId];
req.Position = NameToState.find(name);
req.Family = family;
- Y_VERIFY(req.Position != NameToState.end());
+ Y_ABORT_UNLESS(req.Position != NameToState.end());
Send(Upstream, new TEvDns::TEvGetHostByName(name, family), IEventHandle::FlagTrackDelivery, reqId);
state.Waiting = true;
@@ -484,7 +484,7 @@ namespace NDnsResolver {
}
auto& state = it->second.StateByFamily(family);
- Y_VERIFY(state.Waiting, "Got error for a state we are not waiting");
+ Y_ABORT_UNLESS(state.Waiting, "Got error for a state we are not waiting");
state.Waiting = false;
// When we have a cached positive reply, don't overwrite it with spurious errors
@@ -532,7 +532,7 @@ namespace NDnsResolver {
}
auto& state = it->second.StateByFamily(family);
- Y_VERIFY(state.Waiting, "Got reply for a state we are not waiting");
+ Y_ABORT_UNLESS(state.Waiting, "Got reply for a state we are not waiting");
state.Waiting = false;
state.Status = ARES_SUCCESS;
@@ -567,7 +567,7 @@ namespace NDnsResolver {
}
bool& flag = family.*FamilyToFlag;
- Y_VERIFY(flag);
+ Y_ABORT_UNLESS(flag);
heap.pop();
flag = false;
diff --git a/library/cpp/actors/dnsresolver/dnsresolver_caching_ut.cpp b/library/cpp/actors/dnsresolver/dnsresolver_caching_ut.cpp
index 89a7e9ab36..60a45f6fba 100644
--- a/library/cpp/actors/dnsresolver/dnsresolver_caching_ut.cpp
+++ b/library/cpp/actors/dnsresolver/dnsresolver_caching_ut.cpp
@@ -21,14 +21,14 @@ Y_UNIT_TEST_SUITE(CachingDnsResolver) {
TString operator()(const struct in6_addr& addr) const {
char dst[INET6_ADDRSTRLEN];
auto res = ares_inet_ntop(AF_INET6, &addr, dst, INET6_ADDRSTRLEN);
- Y_VERIFY(res, "Cannot convert ipv6 address");
+ Y_ABORT_UNLESS(res, "Cannot convert ipv6 address");
return dst;
}
TString operator()(const struct in_addr& addr) const {
char dst[INET_ADDRSTRLEN];
auto res = ares_inet_ntop(AF_INET, &addr, dst, INET_ADDRSTRLEN);
- Y_VERIFY(res, "Cannot convert ipv4 address");
+ Y_ABORT_UNLESS(res, "Cannot convert ipv4 address");
return dst;
}
};
@@ -46,7 +46,7 @@ Y_UNIT_TEST_SUITE(CachingDnsResolver) {
TVector<struct in_addr> AddrsV4;
static TMockReply Error(int status, TDuration delay = DefaultDelay) {
- Y_VERIFY(status != 0);
+ Y_ABORT_UNLESS(status != 0);
TMockReply reply;
reply.Status = status;
reply.Delay = delay;
@@ -65,7 +65,7 @@ Y_UNIT_TEST_SUITE(CachingDnsResolver) {
for (const TString& addr : addrs) {
void* dst = &reply.AddrsV6.emplace_back();
int status = ares_inet_pton(AF_INET6, addr.c_str(), dst);
- Y_VERIFY(status == 1, "Invalid ipv6 address: %s", addr.c_str());
+ Y_ABORT_UNLESS(status == 1, "Invalid ipv6 address: %s", addr.c_str());
}
return reply;
}
@@ -76,7 +76,7 @@ Y_UNIT_TEST_SUITE(CachingDnsResolver) {
for (const TString& addr : addrs) {
void* dst = &reply.AddrsV4.emplace_back();
int status = ares_inet_pton(AF_INET, addr.c_str(), dst);
- Y_VERIFY(status == 1, "Invalid ipv4 address: %s", addr.c_str());
+ Y_ABORT_UNLESS(status == 1, "Invalid ipv4 address: %s", addr.c_str());
}
return reply;
}
@@ -90,7 +90,7 @@ Y_UNIT_TEST_SUITE(CachingDnsResolver) {
}
friend TMockReply operator+(const TMockReply& a, const TMockReply& b) {
- Y_VERIFY(a.Status == b.Status);
+ Y_ABORT_UNLESS(a.Status == b.Status);
TMockReply result;
result.Status = a.Status;
result.Delay = Max(a.Delay, b.Delay);
diff --git a/library/cpp/actors/dnsresolver/dnsresolver_ut.cpp b/library/cpp/actors/dnsresolver/dnsresolver_ut.cpp
index 0c343a805c..93c4b832e2 100644
--- a/library/cpp/actors/dnsresolver/dnsresolver_ut.cpp
+++ b/library/cpp/actors/dnsresolver/dnsresolver_ut.cpp
@@ -18,7 +18,7 @@ Y_UNIT_TEST_SUITE(DnsResolver) {
TSilentUdpServer() {
TSockAddrInet addr("127.0.0.1", 0);
int err = Socket.Bind(&addr);
- Y_VERIFY(err == 0, "Cannot bind a udp socket");
+ Y_ABORT_UNLESS(err == 0, "Cannot bind a udp socket");
Port = addr.GetPort();
}
};
diff --git a/library/cpp/actors/examples/01_ping_pong/main.cpp b/library/cpp/actors/examples/01_ping_pong/main.cpp
index c9223f78ef..437f06eadd 100644
--- a/library/cpp/actors/examples/01_ping_pong/main.cpp
+++ b/library/cpp/actors/examples/01_ping_pong/main.cpp
@@ -81,8 +81,8 @@ public:
};
THolder<TActorSystemSetup> BuildActorSystemSetup(ui32 threads, ui32 pools) {
- Y_VERIFY(threads > 0 && threads < 100);
- Y_VERIFY(pools > 0 && pools < 10);
+ Y_ABORT_UNLESS(threads > 0 && threads < 100);
+ Y_ABORT_UNLESS(pools > 0 && pools < 10);
auto setup = MakeHolder<TActorSystemSetup>();
diff --git a/library/cpp/actors/examples/02_discovery/lookup.cpp b/library/cpp/actors/examples/02_discovery/lookup.cpp
index bcae477034..fb136a431c 100644
--- a/library/cpp/actors/examples/02_discovery/lookup.cpp
+++ b/library/cpp/actors/examples/02_discovery/lookup.cpp
@@ -106,7 +106,7 @@ public:
{}
void Bootstrap() {
- Y_VERIFY(Config->Replicas.size() > 0);
+ Y_ABORT_UNLESS(Config->Replicas.size() > 0);
TotalReplicas = Config->Replicas.size();
RequestActors.reserve(TotalReplicas);
diff --git a/library/cpp/actors/examples/02_discovery/main.cpp b/library/cpp/actors/examples/02_discovery/main.cpp
index 379fd6de84..9dec850c77 100644
--- a/library/cpp/actors/examples/02_discovery/main.cpp
+++ b/library/cpp/actors/examples/02_discovery/main.cpp
@@ -31,7 +31,7 @@ void OnTerminate(int) {
}
THolder<TActorSystemSetup> BuildActorSystemSetup(ui32 nodeId, ui32 threads, NMonitoring::TDynamicCounters &counters) {
- Y_VERIFY(threads > 0 && threads < 100);
+ Y_ABORT_UNLESS(threads > 0 && threads < 100);
auto setup = MakeHolder<TActorSystemSetup>();
diff --git a/library/cpp/actors/helpers/activeactors.h b/library/cpp/actors/helpers/activeactors.h
index 0fdb0fab10..ec482e93c8 100644
--- a/library/cpp/actors/helpers/activeactors.h
+++ b/library/cpp/actors/helpers/activeactors.h
@@ -14,7 +14,7 @@ namespace NActors {
public:
void Insert(const TActorId &aid) {
bool inserted = insert(aid).second;
- Y_VERIFY(inserted);
+ Y_ABORT_UNLESS(inserted);
}
void Insert(const TActiveActors &moreActors) {
@@ -25,7 +25,7 @@ namespace NActors {
void Erase(const TActorId &aid) {
auto num = erase(aid);
- Y_VERIFY(num == 1);
+ Y_ABORT_UNLESS(num == 1);
}
size_t KillAndClear(const TActorContext &ctx) {
diff --git a/library/cpp/actors/helpers/mon_histogram_helper.h b/library/cpp/actors/helpers/mon_histogram_helper.h
index a9a57e3823..2c5ef0bbee 100644
--- a/library/cpp/actors/helpers/mon_histogram_helper.h
+++ b/library/cpp/actors/helpers/mon_histogram_helper.h
@@ -37,7 +37,7 @@ namespace NActors {
void Add(ui64 val) {
Y_ASSERT(FirstBucketVal != 0);
Y_ASSERT(BucketCount != 0);
- Y_VERIFY(val <= (1ULL << 63ULL));
+ Y_ABORT_UNLESS(val <= (1ULL << 63ULL));
size_t ind = 0;
if (val > FirstBucketVal) {
ind = GetValueBitCount((2 * val - 1) / FirstBucketVal) - 1;
diff --git a/library/cpp/actors/helpers/pool_stats_collector.h b/library/cpp/actors/helpers/pool_stats_collector.h
index 9e6550dace..1f9a6a1ebb 100644
--- a/library/cpp/actors/helpers/pool_stats_collector.h
+++ b/library/cpp/actors/helpers/pool_stats_collector.h
@@ -62,7 +62,7 @@ private:
void Set(const TExecutorThreadStats& stats) {
for (ui32 i : xrange(stats.MaxActivityType())) {
- Y_VERIFY(i < GetActivityTypeCount());
+ Y_ABORT_UNLESS(i < GetActivityTypeCount());
ui64 ticks = stats.ElapsedTicksByActivity[i];
ui64 events = stats.ReceivedEventsByActivity[i];
ui64 actors = stats.ActorsAliveByActivity[i];
@@ -91,7 +91,7 @@ private:
private:
void InitCountersForActivity(ui32 activityType) {
- Y_VERIFY(activityType < GetActivityTypeCount());
+ Y_ABORT_UNLESS(activityType < GetActivityTypeCount());
auto bucketName = TString(GetActivityTypeName(activityType));
diff --git a/library/cpp/actors/http/http.h b/library/cpp/actors/http/http.h
index 8b35de737e..a2d8004a64 100644
--- a/library/cpp/actors/http/http.h
+++ b/library/cpp/actors/http/http.h
@@ -647,7 +647,7 @@ public:
Stage = ERenderStage::Error;
break;
}
- Y_VERIFY(size == BufferType::Size());
+ Y_ABORT_UNLESS(size == BufferType::Size());
}
TStringBuf GetRawData() const {
diff --git a/library/cpp/actors/http/http_proxy_acceptor.cpp b/library/cpp/actors/http/http_proxy_acceptor.cpp
index 4ff09ec7aa..f9ee1d8032 100644
--- a/library/cpp/actors/http/http_proxy_acceptor.cpp
+++ b/library/cpp/actors/http/http_proxy_acceptor.cpp
@@ -132,7 +132,7 @@ protected:
}
int err = errno;
if (err == EAGAIN || err == EWOULDBLOCK) { // request poller for further connection polling
- Y_VERIFY(PollerToken);
+ Y_ABORT_UNLESS(PollerToken);
PollerToken->Request(true, false);
}
}
diff --git a/library/cpp/actors/http/http_proxy_incoming.cpp b/library/cpp/actors/http/http_proxy_incoming.cpp
index f6d26ec3eb..68bd5f6ecb 100644
--- a/library/cpp/actors/http/http_proxy_incoming.cpp
+++ b/library/cpp/actors/http/http_proxy_incoming.cpp
@@ -222,7 +222,7 @@ protected:
while (CurrentResponse != nullptr) {
size_t size = CurrentResponse->Size();
if (size == 0) {
- Y_VERIFY(Requests.front() == CurrentResponse->GetRequest());
+ Y_ABORT_UNLESS(Requests.front() == CurrentResponse->GetRequest());
bool close = CurrentResponse->IsConnectionClose();
Requests.pop_front();
CleanupResponse(CurrentResponse);
diff --git a/library/cpp/actors/interconnect/channel_scheduler.h b/library/cpp/actors/interconnect/channel_scheduler.h
index 1528810d0d..3040601ef2 100644
--- a/library/cpp/actors/interconnect/channel_scheduler.h
+++ b/library/cpp/actors/interconnect/channel_scheduler.h
@@ -43,7 +43,7 @@ namespace NActors {
}
TEventOutputChannel *PickChannelWithLeastConsumedWeight() {
- Y_VERIFY(!Heap.empty());
+ Y_ABORT_UNLESS(!Heap.empty());
return Heap.front().Channel;
}
diff --git a/library/cpp/actors/interconnect/event_holder_pool.h b/library/cpp/actors/interconnect/event_holder_pool.h
index 59df9bd4c8..0afa1d7a7c 100644
--- a/library/cpp/actors/interconnect/event_holder_pool.h
+++ b/library/cpp/actors/interconnect/event_holder_pool.h
@@ -17,12 +17,12 @@ namespace NActors {
~TEvFreeItems() {
if (Counter) {
TAtomicBase res = Counter->fetch_sub(NumBytes) - NumBytes;
- Y_VERIFY(res >= 0);
+ Y_ABORT_UNLESS(res >= 0);
}
}
bool GetInLineForDestruction(const TIntrusivePtr<TInterconnectProxyCommon>& common) {
- Y_VERIFY(!Counter);
+ Y_ABORT_UNLESS(!Counter);
const auto& counter = common->DestructorQueueSize;
const auto& max = common->MaxDestructorQueueSize;
if (counter && (TAtomicBase)(counter->fetch_add(NumBytes) + NumBytes) > max) {
diff --git a/library/cpp/actors/interconnect/handshake_broker.h b/library/cpp/actors/interconnect/handshake_broker.h
index d2e1b8b6ac..d657e7dd51 100644
--- a/library/cpp/actors/interconnect/handshake_broker.h
+++ b/library/cpp/actors/interconnect/handshake_broker.h
@@ -77,7 +77,7 @@ namespace NActors {
}
const size_t n = WaiterLookup.erase(waiter);
- Y_VERIFY(n == 1);
+ Y_ABORT_UNLESS(n == 1);
Send(waiter, new TEvHandshakeBrokerPermit());
PermittedLeases.insert(waiter);
@@ -105,7 +105,7 @@ namespace NActors {
} else {
const auto [it, inserted] = WaiterLookup.try_emplace(sender,
Waiters.insert(Waiters.end(), sender));
- Y_VERIFY(inserted);
+ Y_ABORT_UNLESS(inserted);
}
}
@@ -114,7 +114,7 @@ namespace NActors {
if (!PermittedLeases.erase(sender)) {
// Lease was not permitted yet, remove sender from Waiters queue
const auto it = WaiterLookup.find(sender);
- Y_VERIFY(it != WaiterLookup.end());
+ Y_ABORT_UNLESS(it != WaiterLookup.end());
Waiters.erase(it->second);
WaiterLookup.erase(it);
}
diff --git a/library/cpp/actors/interconnect/interconnect_address.cpp b/library/cpp/actors/interconnect/interconnect_address.cpp
index d6adb8098d..124cd61325 100644
--- a/library/cpp/actors/interconnect/interconnect_address.cpp
+++ b/library/cpp/actors/interconnect/interconnect_address.cpp
@@ -15,7 +15,7 @@ namespace NInterconnect {
TAddress::TAddress(NAddr::IRemoteAddr& addr) {
socklen_t len = addr.Len();
- Y_VERIFY(len <= sizeof(Addr));
+ Y_ABORT_UNLESS(len <= sizeof(Addr));
memcpy(&Addr.Generic, addr.Addr(), len);
}
diff --git a/library/cpp/actors/interconnect/interconnect_channel.cpp b/library/cpp/actors/interconnect/interconnect_channel.cpp
index 6987c344de..ab6af8f8d4 100644
--- a/library/cpp/actors/interconnect/interconnect_channel.cpp
+++ b/library/cpp/actors/interconnect/interconnect_channel.cpp
@@ -20,7 +20,7 @@ namespace NActors {
auto traceId = event.Span.GetTraceId();
event.Span.EndOk();
- Y_VERIFY(SerializationInfo);
+ Y_ABORT_UNLESS(SerializationInfo);
const ui32 flags = (event.Descr.Flags & ~IEventHandle::FlagForwardOnNondelivery) |
(SerializationInfo->IsExtendedFormat ? IEventHandle::FlagExtendedFormat : 0);
@@ -65,7 +65,7 @@ namespace NActors {
bool TEventOutputChannel::FeedBuf(TTcpPacketOutTask& task, ui64 serial, ui64 *weightConsumed) {
for (;;) {
- Y_VERIFY(!Queue.empty());
+ Y_ABORT_UNLESS(!Queue.empty());
TEventHolder& event = Queue.front();
switch (State) {
@@ -125,7 +125,7 @@ namespace NActors {
for (const auto& section : SerializationInfo->Sections) {
totalSectionSize += section.Size;
}
- Y_VERIFY(totalSectionSize == event.EventSerializedSize);
+ Y_ABORT_UNLESS(totalSectionSize == event.EventSerializedSize);
}
while (SectionIndex != SerializationInfo->Sections.size()) {
@@ -142,7 +142,7 @@ namespace NActors {
if (section.IsInline && Params.UseXdcShuffle) {
type = static_cast<ui8>(EXdcCommand::DECLARE_SECTION_INLINE);
}
- Y_VERIFY(p <= std::end(sectionInfo));
+ Y_ABORT_UNLESS(p <= std::end(sectionInfo));
const size_t declareLen = p - sectionInfo;
if (sizeof(TChannelPart) + XdcData.size() + declareLen <= task.GetInternalFreeAmount() &&
@@ -209,7 +209,7 @@ namespace NActors {
}
complete = Chunker.IsComplete();
if (complete) {
- Y_VERIFY(Chunker.IsSuccessfull());
+ Y_ABORT_UNLESS(Chunker.IsSuccessfull());
}
}
} else if (event.Buffer) {
@@ -223,7 +223,7 @@ namespace NActors {
} else {
Y_FAIL();
}
- Y_VERIFY(!complete || event.EventActuallySerialized == event.EventSerializedSize,
+ Y_ABORT_UNLESS(!complete || event.EventActuallySerialized == event.EventSerializedSize,
"EventActuallySerialized# %" PRIu32 " EventSerializedSize# %" PRIu32 " Type# 0x%08" PRIx32,
event.EventActuallySerialized, event.EventSerializedSize, event.Descr.Type);
@@ -244,7 +244,7 @@ namespace NActors {
IsPartInline = false;
PartLenRemain = Max<size_t>();
} else {
- Y_VERIFY(SectionIndex < sections.size());
+ Y_ABORT_UNLESS(SectionIndex < sections.size());
IsPartInline = sections[SectionIndex].IsInline;
while (SectionIndex < sections.size() && IsPartInline == sections[SectionIndex].IsInline) {
PartLenRemain += sections[SectionIndex].Size;
@@ -276,7 +276,7 @@ namespace NActors {
const bool complete = SerializeEvent<false>(task, event, &bytesSerialized);
Y_VERIFY_DEBUG(bytesSerialized);
- Y_VERIFY(bytesSerialized <= Max<ui16>());
+ Y_ABORT_UNLESS(bytesSerialized <= Max<ui16>());
TChannelPart part{
.ChannelFlags = ChannelId,
@@ -304,7 +304,7 @@ namespace NActors {
size_t bytesSerialized = 0;
const bool complete = SerializeEvent<true>(task, event, &bytesSerialized);
- Y_VERIFY(0 < bytesSerialized && bytesSerialized <= Max<ui16>());
+ Y_ABORT_UNLESS(0 < bytesSerialized && bytesSerialized <= Max<ui16>());
char buffer[partSize];
TChannelPart *part = reinterpret_cast<TChannelPart*>(buffer);
@@ -338,12 +338,12 @@ namespace NActors {
void TEventOutputChannel::NotifyUndelivered() {
LOG_DEBUG_IC_SESSION("ICOCH89", "Notyfying about Undelivered messages! NotYetConfirmed size: %zu, Queue size: %zu", NotYetConfirmed.size(), Queue.size());
if (State == EState::BODY && Queue.front().Event) {
- Y_VERIFY(!Chunker.IsComplete()); // chunk must have an event being serialized
- Y_VERIFY(!Queue.empty()); // this event must be the first event in queue
+ Y_ABORT_UNLESS(!Chunker.IsComplete()); // chunk must have an event being serialized
+ Y_ABORT_UNLESS(!Queue.empty()); // this event must be the first event in queue
TEventHolder& event = Queue.front();
- Y_VERIFY(Chunker.GetCurrentEvent() == event.Event.Get()); // ensure the event is valid
+ Y_ABORT_UNLESS(Chunker.GetCurrentEvent() == event.Event.Get()); // ensure the event is valid
Chunker.Abort(); // stop serializing current event
- Y_VERIFY(Chunker.IsComplete());
+ Y_ABORT_UNLESS(Chunker.IsComplete());
}
for (auto& item : NotYetConfirmed) {
if (item.Descr.Flags & IEventHandle::FlagGenerateUnsureUndelivered) { // notify only when unsure flag is set
diff --git a/library/cpp/actors/interconnect/interconnect_counters.cpp b/library/cpp/actors/interconnect/interconnect_counters.cpp
index 3278160aef..1c55eab650 100644
--- a/library/cpp/actors/interconnect/interconnect_counters.cpp
+++ b/library/cpp/actors/interconnect/interconnect_counters.cpp
@@ -312,7 +312,7 @@ namespace {
}
const TOutputChannel& GetOutputChannel(ui16 index) const {
- Y_VERIFY(Initialized);
+ Y_ABORT_UNLESS(Initialized);
const auto it = OutputChannels.find(index);
return it != OutputChannels.end() ? it->second : OtherOutputChannel;
}
@@ -642,7 +642,7 @@ namespace {
}
const TOutputChannel& GetOutputChannel(ui16 index) const {
- Y_VERIFY(Initialized_);
+ Y_ABORT_UNLESS(Initialized_);
const auto it = OutputChannels_.find(index);
return it != OutputChannels_.end() ? it->second : OtherOutputChannel_;
}
diff --git a/library/cpp/actors/interconnect/interconnect_handshake.cpp b/library/cpp/actors/interconnect/interconnect_handshake.cpp
index cb4788a33c..3c1e90115a 100644
--- a/library/cpp/actors/interconnect/interconnect_handshake.cpp
+++ b/library/cpp/actors/interconnect/interconnect_handshake.cpp
@@ -154,13 +154,13 @@ namespace NActors {
}
void RegisterInPoller() {
- Y_VERIFY(!PollerToken);
+ Y_ABORT_UNLESS(!PollerToken);
const bool success = Actor->Send(MakePollerActorId(), new TEvPollerRegister(Socket, Actor->SelfActorId, Actor->SelfActorId));
- Y_VERIFY(success);
+ Y_ABORT_UNLESS(success);
auto result = Actor->WaitForSpecificEvent<TEvPollerRegisterResult>("RegisterPoller");
PollerToken = std::move(result->Get()->PollerToken);
- Y_VERIFY(PollerToken);
- Y_VERIFY(PollerToken->RefCount() == 1); // ensure exclusive ownership
+ Y_ABORT_UNLESS(PollerToken);
+ Y_ABORT_UNLESS(PollerToken->RefCount() == 1); // ensure exclusive ownership
}
void WaitPoller(bool read, bool write, TString state) {
@@ -170,7 +170,7 @@ namespace NActors {
template <typename TDataPtr, typename TSendRecvFunc>
void Process(TDataPtr buffer, size_t len, TSendRecvFunc&& sendRecv, bool read, bool write, TString state) {
- Y_VERIFY(Socket);
+ Y_ABORT_UNLESS(Socket);
NInterconnect::TStreamSocket* sock = Socket.Get();
ssize_t (NInterconnect::TStreamSocket::*pfn)(TDataPtr, size_t, TString*) const = sendRecv;
size_t processed = 0;
@@ -208,7 +208,7 @@ namespace NActors {
void ResetPollerToken() {
if (PollerToken) {
- Y_VERIFY(PollerToken->RefCount() == 1);
+ Y_ABORT_UNLESS(PollerToken->RefCount() == 1);
PollerToken.Reset(); // ensure we are going to destroy poller token here as we will re-register the socket within other actor
}
}
@@ -255,9 +255,9 @@ namespace NActors {
, HandshakeKind("outgoing handshake")
, Params(std::move(params))
{
- Y_VERIFY(SelfVirtualId);
- Y_VERIFY(SelfVirtualId.NodeId());
- Y_VERIFY(PeerNodeId);
+ Y_ABORT_UNLESS(SelfVirtualId);
+ Y_ABORT_UNLESS(SelfVirtualId.NodeId());
+ Y_ABORT_UNLESS(PeerNodeId);
HandshakeBroker = MakeHandshakeBrokerOutId();
// generate random handshake id
@@ -272,7 +272,7 @@ namespace NActors {
, ExternalDataChannel(this, nullptr)
, HandshakeKind("incoming handshake")
{
- Y_VERIFY(MainChannel);
+ Y_ABORT_UNLESS(MainChannel);
PeerAddr = TString::Uninitialized(1024);
if (GetRemoteAddr(*MainChannel.GetSocketRef(), PeerAddr.Detach(), PeerAddr.size())) {
PeerAddr.resize(strlen(PeerAddr.data()));
@@ -342,7 +342,7 @@ namespace NActors {
if (ProgramInfo) {
if (Params.UseExternalDataChannel) {
if (incoming) {
- Y_VERIFY(SubscribedForConnection);
+ Y_ABORT_UNLESS(SubscribedForConnection);
auto ev = WaitForSpecificEvent<TEvReportConnection>("WaitInboundXdcStream");
SubscribedForConnection = false;
if (ev->Get()->HandshakeId != *HandshakeId) {
@@ -370,10 +370,10 @@ namespace NActors {
if (ProgramInfo) {
LOG_LOG_IC_X(NActorsServices::INTERCONNECT, "ICH04", NLog::PRI_INFO, "handshake succeeded");
- Y_VERIFY(NextPacketFromPeer);
+ Y_ABORT_UNLESS(NextPacketFromPeer);
MainChannel.ResetPollerToken();
ExternalDataChannel.ResetPollerToken();
- Y_VERIFY(!ExternalDataChannel == !Params.UseExternalDataChannel);
+ Y_ABORT_UNLESS(!ExternalDataChannel == !Params.UseExternalDataChannel);
SendToProxy(MakeHolder<TEvHandshakeDone>(std::move(MainChannel.GetSocketRef()), PeerVirtualId, SelfVirtualId,
*NextPacketFromPeer, ProgramInfo->Release(), std::move(Params), std::move(ExternalDataChannel.GetSocketRef())));
}
@@ -837,7 +837,7 @@ namespace NActors {
}
addresses.emplace_back(r.GetAddress(), static_cast<ui16>(r.GetPort()));
} else {
- Y_VERIFY(ev->GetTypeRewrite() == ui32(ENetwork::ResolveError));
+ Y_ABORT_UNLESS(ev->GetTypeRewrite() == ui32(ENetwork::ResolveError));
Fail(TEvHandshakeFail::HANDSHAKE_FAIL_PERMANENT, "DNS resolve error: " + ev->Get<TEvResolveError>()->Explain
+ ", Unresolved host# " + ev->Get<TEvResolveError>()->Host, true);
}
@@ -1072,7 +1072,7 @@ namespace NActors {
if (auto ev = reply->CastAsLocal<TEvHandshakeReplyOK>()) {
// issue successful reply to the peer
auto& record = ev->Record;
- Y_VERIFY(record.HasSuccess());
+ Y_ABORT_UNLESS(record.HasSuccess());
auto& success = *record.MutableSuccess();
SetupClusterUUID(success);
SetupCompatibilityInfo(success);
@@ -1106,7 +1106,7 @@ namespace NActors {
void SendExBlock(TConnection& connection, const T& proto, const char* what) {
TString data;
Y_PROTOBUF_SUPPRESS_NODISCARD proto.SerializeToString(&data);
- Y_VERIFY(data.size() <= TExHeader::MaxSize);
+ Y_ABORT_UNLESS(data.size() <= TExHeader::MaxSize);
ReportProto(proto, Sprintf("SendExBlock %s", what).data());
@@ -1137,7 +1137,7 @@ namespace NActors {
private:
void SendToProxy(THolder<IEventBase> ev) {
- Y_VERIFY(PeerNodeId);
+ Y_ABORT_UNLESS(PeerNodeId);
Send(GetActorSystem()->InterconnectProxy(PeerNodeId), ev.Release());
}
@@ -1204,7 +1204,7 @@ namespace NActors {
}
THolder<TEvInterconnect::TNodeInfo> GetPeerNodeInfo() {
- Y_VERIFY(PeerNodeId);
+ Y_ABORT_UNLESS(PeerNodeId);
Send(Common->NameserviceId, new TEvInterconnect::TEvGetNode(PeerNodeId, TActivationContext::Now() +
(Deadline - TActivationContext::Monotonic())));
auto response = WaitForSpecificEvent<TEvInterconnect::TEvNodeInfo>("GetPeerNodeInfo");
diff --git a/library/cpp/actors/interconnect/interconnect_nameserver_dynamic.cpp b/library/cpp/actors/interconnect/interconnect_nameserver_dynamic.cpp
index 1a563454df..867b4b5d39 100644
--- a/library/cpp/actors/interconnect/interconnect_nameserver_dynamic.cpp
+++ b/library/cpp/actors/interconnect/interconnect_nameserver_dynamic.cpp
@@ -96,7 +96,7 @@ namespace NActors {
, NodeTable(setup->StaticNodeTable)
, PendingPeriod(pendingPeriod)
{
- Y_VERIFY(setup->IsEntriesUnique());
+ Y_ABORT_UNLESS(setup->IsEntriesUnique());
}
STFUNC(StateFunc) {
diff --git a/library/cpp/actors/interconnect/interconnect_nameserver_table.cpp b/library/cpp/actors/interconnect/interconnect_nameserver_table.cpp
index e5507b9838..ac565f6f8c 100644
--- a/library/cpp/actors/interconnect/interconnect_nameserver_table.cpp
+++ b/library/cpp/actors/interconnect/interconnect_nameserver_table.cpp
@@ -21,7 +21,7 @@ namespace NActors {
: TInterconnectNameserverBase<TInterconnectNameserverTable>(&TInterconnectNameserverTable::StateFunc, setup->StaticNodeTable)
, Config(setup)
{
- Y_VERIFY(Config->IsEntriesUnique());
+ Y_ABORT_UNLESS(Config->IsEntriesUnique());
}
STFUNC(StateFunc) {
diff --git a/library/cpp/actors/interconnect/interconnect_proxy_wrapper.cpp b/library/cpp/actors/interconnect/interconnect_proxy_wrapper.cpp
index 0692cea963..2c02c632c2 100644
--- a/library/cpp/actors/interconnect/interconnect_proxy_wrapper.cpp
+++ b/library/cpp/actors/interconnect/interconnect_proxy_wrapper.cpp
@@ -30,7 +30,7 @@ namespace NActors {
if (Mock) {
Proxy = actor;
}
- Y_VERIFY(Proxy);
+ Y_ABORT_UNLESS(Proxy);
}
InvokeOtherActor(*Proxy, &IActor::Receive, ev);
}
diff --git a/library/cpp/actors/interconnect/interconnect_stream.cpp b/library/cpp/actors/interconnect/interconnect_stream.cpp
index 3082e08a78..51a23923af 100644
--- a/library/cpp/actors/interconnect/interconnect_stream.cpp
+++ b/library/cpp/actors/interconnect/interconnect_stream.cpp
@@ -97,7 +97,7 @@ namespace NInterconnect {
const SOCKET res = ::socket(domain, SOCK_STREAM | SOCK_NONBLOCK, 0);
if (res == -1) {
const int err = LastSocketError();
- Y_VERIFY(err != EMFILE && err != ENFILE);
+ Y_ABORT_UNLESS(err != EMFILE && err != ENFILE);
if (error) {
*error = err;
}
@@ -224,7 +224,7 @@ namespace NInterconnect {
const SOCKET res = ::socket(domain, SOCK_DGRAM, 0);
if (res == -1) {
const int err = LastSocketError();
- Y_VERIFY(err != EMFILE && err != ENFILE);
+ Y_ABORT_UNLESS(err != EMFILE && err != ENFILE);
}
return std::make_shared<TDatagramSocket>(res);
}
@@ -283,14 +283,14 @@ namespace NInterconnect {
InitOpenSSL();
#if OPENSSL_VERSION_NUMBER < 0x10100000L
Ctx.reset(SSL_CTX_new(TLSv1_2_method()));
- Y_VERIFY(Ctx, "SSL_CTX_new() failed");
+ Y_ABORT_UNLESS(Ctx, "SSL_CTX_new() failed");
#else
Ctx.reset(SSL_CTX_new(TLS_method()));
- Y_VERIFY(Ctx, "SSL_CTX_new() failed");
+ Y_ABORT_UNLESS(Ctx, "SSL_CTX_new() failed");
ret = SSL_CTX_set_min_proto_version(Ctx.get(), TLS1_2_VERSION);
- Y_VERIFY(ret == 1, "failed to set min proto version");
+ Y_ABORT_UNLESS(ret == 1, "failed to set min proto version");
ret = SSL_CTX_set_max_proto_version(Ctx.get(), TLS1_2_VERSION);
- Y_VERIFY(ret == 1, "failed to set max proto version");
+ Y_ABORT_UNLESS(ret == 1, "failed to set max proto version");
#endif
SSL_CTX_set_verify(Ctx.get(), SSL_VERIFY_PEER | SSL_VERIFY_FAIL_IF_NO_PEER_CERT, &Verify);
SSL_CTX_set_mode(*this, SSL_MODE_ENABLE_PARTIAL_WRITE | SSL_MODE_ACCEPT_MOVING_WRITE_BUFFER);
@@ -298,13 +298,13 @@ namespace NInterconnect {
// apply certificates in SSL context
if (certificate) {
std::unique_ptr<BIO, TDeleter> bio(BIO_new_mem_buf(certificate.data(), certificate.size()));
- Y_VERIFY(bio);
+ Y_ABORT_UNLESS(bio);
// first certificate in the chain is expected to be a leaf
std::unique_ptr<X509, TDeleter> cert(PEM_read_bio_X509(bio.get(), nullptr, nullptr, nullptr));
- Y_VERIFY(cert, "failed to parse certificate");
+ Y_ABORT_UNLESS(cert, "failed to parse certificate");
ret = SSL_CTX_use_certificate(Ctx.get(), cert.get());
- Y_VERIFY(ret == 1);
+ Y_ABORT_UNLESS(ret == 1);
// loading additional certificates in the chain, if any
while(true) {
@@ -313,25 +313,25 @@ namespace NInterconnect {
break;
}
ret = SSL_CTX_add0_chain_cert(Ctx.get(), ca);
- Y_VERIFY(ret == 1);
+ Y_ABORT_UNLESS(ret == 1);
// we must not free memory if certificate was added successfully by SSL_CTX_add0_chain_cert
}
}
if (privateKey) {
std::unique_ptr<BIO, TDeleter> bio(BIO_new_mem_buf(privateKey.data(), privateKey.size()));
- Y_VERIFY(bio);
+ Y_ABORT_UNLESS(bio);
std::unique_ptr<RSA, TDeleter> pkey(PEM_read_bio_RSAPrivateKey(bio.get(), nullptr, nullptr, nullptr));
- Y_VERIFY(pkey);
+ Y_ABORT_UNLESS(pkey);
ret = SSL_CTX_use_RSAPrivateKey(Ctx.get(), pkey.get());
- Y_VERIFY(ret == 1);
+ Y_ABORT_UNLESS(ret == 1);
}
if (caFilePath) {
ret = SSL_CTX_load_verify_locations(Ctx.get(), caFilePath.data(), nullptr);
- Y_VERIFY(ret == 1);
+ Y_ABORT_UNLESS(ret == 1);
}
int success = SSL_CTX_set_cipher_list(Ctx.get(), ciphers ? ciphers.data() : "AES128-GCM-SHA256");
- Y_VERIFY(success, "failed to set cipher list");
+ Y_ABORT_UNLESS(success, "failed to set cipher list");
}
operator SSL_CTX*() const {
@@ -386,7 +386,7 @@ namespace NInterconnect {
TImpl(SSL_CTX *ctx, int fd)
: Ssl(SSL_new(ctx))
{
- Y_VERIFY(Ssl, "SSL_new() failed");
+ Y_ABORT_UNLESS(Ssl, "SSL_new() failed");
SSL_set_fd(Ssl, fd);
SSL_set_ex_data(Ssl, TSecureSocketContext::TImpl::GetExIndex(), &ErrorDescription);
}
@@ -490,7 +490,7 @@ namespace NInterconnect {
if (BlockedSend && BlockedSend->first == msg && BlockedSend->second < len) {
len = BlockedSend->second;
}
- Y_VERIFY(!BlockedSend || *BlockedSend == std::make_pair(msg, len));
+ Y_ABORT_UNLESS(!BlockedSend || *BlockedSend == std::make_pair(msg, len));
const ssize_t res = Operate(msg, len, &SSL_write_ex, err);
if (res == -EAGAIN) {
BlockedSend.emplace(msg, len);
@@ -510,7 +510,7 @@ namespace NInterconnect {
if (BlockedReceive && BlockedReceive->first == msg && BlockedReceive->second < len) {
len = BlockedReceive->second;
}
- Y_VERIFY(!BlockedReceive || *BlockedReceive == std::make_pair(msg, len));
+ Y_ABORT_UNLESS(!BlockedReceive || *BlockedReceive == std::make_pair(msg, len));
const ssize_t res = Operate(msg, len, &SSL_read_ex, err);
if (res == -EAGAIN) {
BlockedReceive.emplace(msg, len);
diff --git a/library/cpp/actors/interconnect/interconnect_tcp_input_session.cpp b/library/cpp/actors/interconnect/interconnect_tcp_input_session.cpp
index 3b42884355..41fd36a824 100644
--- a/library/cpp/actors/interconnect/interconnect_tcp_input_session.cpp
+++ b/library/cpp/actors/interconnect/interconnect_tcp_input_session.cpp
@@ -18,7 +18,7 @@ namespace NActors {
void TReceiveContext::TPerChannelContext::ApplyCatchBuffer() {
if (auto buffer = std::exchange(XdcCatchBuffer, {})) {
- Y_VERIFY(XdcCatchBytesRead >= buffer.size());
+ Y_ABORT_UNLESS(XdcCatchBytesRead >= buffer.size());
const size_t offset = XdcCatchBytesRead % buffer.size();
const char *begin = buffer.data();
@@ -109,10 +109,10 @@ namespace NActors {
, Metrics(std::move(metrics))
, DeadPeerTimeout(deadPeerTimeout)
{
- Y_VERIFY(Context);
- Y_VERIFY(Socket);
- Y_VERIFY(SessionId);
- Y_VERIFY(!Params.UseExternalDataChannel == !XdcSocket);
+ Y_ABORT_UNLESS(Context);
+ Y_ABORT_UNLESS(Socket);
+ Y_ABORT_UNLESS(SessionId);
+ Y_ABORT_UNLESS(!Params.UseExternalDataChannel == !XdcSocket);
Metrics->SetClockSkewMicrosec(0);
@@ -120,7 +120,7 @@ namespace NActors {
// ensure that we do not spawn new session while the previous one is still alive
TAtomicBase sessions = AtomicIncrement(Context->NumInputSessions);
- Y_VERIFY(sessions == 1, "sessions# %" PRIu64, ui64(sessions));
+ Y_ABORT_UNLESS(sessions == 1, "sessions# %" PRIu64, ui64(sessions));
// calculate number of bytes to catch
for (auto& context : Context->ChannelArray) {
@@ -270,7 +270,7 @@ namespace NActors {
if (!UpdateFromInputSession) {
UpdateFromInputSession = MakeHolder<TEvUpdateFromInputSession>(ConfirmedByInput, numDataBytes, ping);
} else {
- Y_VERIFY(ConfirmedByInput >= UpdateFromInputSession->ConfirmedByInput);
+ Y_ABORT_UNLESS(ConfirmedByInput >= UpdateFromInputSession->ConfirmedByInput);
UpdateFromInputSession->ConfirmedByInput = ConfirmedByInput;
UpdateFromInputSession->NumDataBytes += numDataBytes;
UpdateFromInputSession->Ping = Min(UpdateFromInputSession->Ping, ping);
@@ -303,7 +303,7 @@ namespace NActors {
break;
case EUpdateState::INFLIGHT_AND_PENDING:
- Y_VERIFY(UpdateFromInputSession);
+ Y_ABORT_UNLESS(UpdateFromInputSession);
break;
default:
@@ -328,7 +328,7 @@ namespace NActors {
void TInputSessionTCP::ProcessHeader() {
TTcpPacketHeader_v2 header;
const bool success = IncomingData.ExtractFrontPlain(&header, sizeof(header));
- Y_VERIFY(success);
+ Y_ABORT_UNLESS(success);
PayloadSize = header.PayloadLength;
const ui64 serial = header.Serial;
const ui64 confirm = header.Confirm;
@@ -444,7 +444,7 @@ namespace NActors {
if (part.IsXdc()) { // external data channel command packet
XdcCommands.resize(part.Size);
const bool success = Payload.ExtractFrontPlain(XdcCommands.data(), XdcCommands.size());
- Y_VERIFY(success);
+ Y_ABORT_UNLESS(success);
ProcessXdcCommand(channel, context);
} else if (IgnorePayload) { // throw payload out
Payload.EraseFront(part.Size);
@@ -459,7 +459,7 @@ namespace NActors {
}
const bool success = Payload.ExtractFrontPlain(&v2, sizeof(v2));
- Y_VERIFY(success);
+ Y_ABORT_UNLESS(success);
pendingEvent.EventData = TEventData{
v2.Type,
@@ -511,7 +511,7 @@ namespace NActors {
numXdcBytesRead -= n;
if (front.XdcUnreadBytes) { // we haven't finished this packet yet
- Y_VERIFY(!numXdcBytesRead);
+ Y_ABORT_UNLESS(!numXdcBytesRead);
break;
}
@@ -548,7 +548,7 @@ namespace NActors {
pendingEvent.SerializationInfo.Sections.push_back(TEventSectionInfo{headroom, size, tailroom,
alignment, isInline});
- Y_VERIFY(!isInline || Params.UseXdcShuffle);
+ Y_ABORT_UNLESS(!isInline || Params.UseXdcShuffle);
if (!isInline) {
// allocate buffer and push it into the payload
auto buffer = TRcBuf::Uninitialized(size, headroom, tailroom);
@@ -653,7 +653,7 @@ namespace NActors {
rope.ExtractFront(rope.size(), &payload);
}
// and ensure there is no unprocessed external payload
- Y_VERIFY(!pendingEvent.ExternalPayload);
+ Y_ABORT_UNLESS(!pendingEvent.ExternalPayload);
#if IC_FORCE_HARDENED_PACKET_CHECKS
if (descr.Len != payload.GetSize()) {
@@ -703,7 +703,7 @@ namespace NActors {
return;
case EUpdateState::CONFIRMING:
- Y_VERIFY(UpdateFromInputSession);
+ Y_ABORT_UNLESS(UpdateFromInputSession);
if (Context->UpdateState.compare_exchange_weak(state, EUpdateState::INFLIGHT)) {
Send(SessionId, UpdateFromInputSession.Release());
return;
@@ -776,14 +776,14 @@ namespace NActors {
return false;
}
- Y_VERIFY(recvres > 0);
+ Y_ABORT_UNLESS(recvres > 0);
Metrics->AddTotalBytesRead(recvres);
BytesReadFromSocket += recvres;
size_t numBuffersCovered = 0;
while (recvres) {
- Y_VERIFY(!Buffers.empty());
+ Y_ABORT_UNLESS(!Buffers.empty());
auto& buffer = Buffers.front();
const size_t bytes = Min<size_t>(recvres, buffer.size());
recvres -= bytes;
@@ -949,14 +949,14 @@ namespace NActors {
}
}
- Y_VERIFY(recvres > 0);
+ Y_ABORT_UNLESS(recvres > 0);
Metrics->AddTotalBytesRead(recvres);
*numDataBytes += recvres;
BytesReadFromXdcSocket += recvres;
// cut the XdcInputQ deque
for (size_t bytesToCut = recvres; bytesToCut; ) {
- Y_VERIFY(!XdcInputQ.empty());
+ Y_ABORT_UNLESS(!XdcInputQ.empty());
auto& [channel, span] = XdcInputQ.front();
size_t n = Min(bytesToCut, span.size());
bytesToCut -= n;
@@ -964,7 +964,7 @@ namespace NActors {
XdcInputQ.pop_front();
} else {
span = span.SubSpan(n, Max<size_t>());
- Y_VERIFY(!bytesToCut);
+ Y_ABORT_UNLESS(!bytesToCut);
}
Y_VERIFY_DEBUG(n);
diff --git a/library/cpp/actors/interconnect/interconnect_tcp_proxy.cpp b/library/cpp/actors/interconnect/interconnect_tcp_proxy.cpp
index 13f2f2dd83..c1994b2d71 100644
--- a/library/cpp/actors/interconnect/interconnect_tcp_proxy.cpp
+++ b/library/cpp/actors/interconnect/interconnect_tcp_proxy.cpp
@@ -28,10 +28,10 @@ namespace NActors {
, SecureContext(new NInterconnect::TSecureSocketContext(Common->Settings.Certificate, Common->Settings.PrivateKey,
Common->Settings.CaFilePath, Common->Settings.CipherList))
{
- Y_VERIFY(Common);
- Y_VERIFY(Common->NameserviceId);
+ Y_ABORT_UNLESS(Common);
+ Y_ABORT_UNLESS(Common->NameserviceId);
if (DynamicPtr) {
- Y_VERIFY(!*DynamicPtr);
+ Y_ABORT_UNLESS(!*DynamicPtr);
*DynamicPtr = this;
}
}
@@ -63,7 +63,7 @@ namespace NActors {
void TInterconnectProxyTCP::RequestNodeInfo(STATEFN_SIG) {
ICPROXY_PROFILED;
- Y_VERIFY(!IncomingHandshakeActor && !OutgoingHandshakeActor && !PendingIncomingHandshakeEvents && !PendingSessionEvents);
+ Y_ABORT_UNLESS(!IncomingHandshakeActor && !OutgoingHandshakeActor && !PendingIncomingHandshakeEvents && !PendingSessionEvents);
EnqueueSessionEvent(ev);
StartConfiguring();
}
@@ -72,7 +72,7 @@ namespace NActors {
ICPROXY_PROFILED;
if (!Terminated) {
- Y_VERIFY(!IncomingHandshakeActor && !OutgoingHandshakeActor && !PendingIncomingHandshakeEvents && !PendingSessionEvents);
+ Y_ABORT_UNLESS(!IncomingHandshakeActor && !OutgoingHandshakeActor && !PendingIncomingHandshakeEvents && !PendingSessionEvents);
EnqueueIncomingHandshakeEvent(ev);
StartConfiguring();
}
@@ -85,7 +85,7 @@ namespace NActors {
void TInterconnectProxyTCP::StartConfiguring() {
ICPROXY_PROFILED;
- Y_VERIFY(!IncomingHandshakeActor && !OutgoingHandshakeActor);
+ Y_ABORT_UNLESS(!IncomingHandshakeActor && !OutgoingHandshakeActor);
// issue node info request
Send(Common->NameserviceId, new TEvInterconnect::TEvGetNode(PeerNodeId));
@@ -99,7 +99,7 @@ namespace NActors {
void TInterconnectProxyTCP::Configure(TEvInterconnect::TEvNodeInfo::TPtr& ev) {
ICPROXY_PROFILED;
- Y_VERIFY(!IncomingHandshakeActor && !OutgoingHandshakeActor && !Session);
+ Y_ABORT_UNLESS(!IncomingHandshakeActor && !OutgoingHandshakeActor && !Session);
if (!ev->Get()->Node) {
TransitToErrorState("cannot get node info");
@@ -170,11 +170,11 @@ namespace NActors {
DropOutgoingHandshake();
// ensure that we have session
- Y_VERIFY(Session);
+ Y_ABORT_UNLESS(Session);
// ensure that we have both virtual ids
- Y_VERIFY(SessionVirtualId);
- Y_VERIFY(RemoteSessionVirtualId);
+ Y_ABORT_UNLESS(SessionVirtualId);
+ Y_ABORT_UNLESS(RemoteSessionVirtualId);
// create and register handshake actor
OutgoingHandshakeActor = Register(CreateOutgoingHandshakeActor(Common, SessionVirtualId,
@@ -187,10 +187,10 @@ namespace NActors {
THolder<IEventBase> event) {
ICPROXY_PROFILED;
- Y_VERIFY(!IncomingHandshakeActor);
+ Y_ABORT_UNLESS(!IncomingHandshakeActor);
IncomingHandshakeActor = handshakeId;
IncomingHandshakeActorFilledIn = TActivationContext::Now();
- Y_VERIFY(!LastSerialFromIncomingHandshake || *LastSerialFromIncomingHandshake <= peerLocalId);
+ Y_ABORT_UNLESS(!LastSerialFromIncomingHandshake || *LastSerialFromIncomingHandshake <= peerLocalId);
LastSerialFromIncomingHandshake = peerLocalId;
if (OutgoingHandshakeActor && SelfId().NodeId() < PeerNodeId) {
@@ -201,12 +201,12 @@ namespace NActors {
// Check that we are in one of acceptable states that would properly handle handshake statuses.
const auto state = CurrentStateFunc();
- Y_VERIFY(state == &TThis::PendingConnection || state == &TThis::StateWork, "invalid handshake request in state# %s", State);
+ Y_ABORT_UNLESS(state == &TThis::PendingConnection || state == &TThis::StateWork, "invalid handshake request in state# %s", State);
} else {
LOG_DEBUG_IC("ICP07", "issued incoming handshake reply");
// No race, so we can send reply immediately.
- Y_VERIFY(!HeldHandshakeReply);
+ Y_ABORT_UNLESS(!HeldHandshakeReply);
Send(IncomingHandshakeActor, event.Release());
// Start waiting for handshake reply, if not yet started; also, if session is already created, then we don't
@@ -215,7 +215,7 @@ namespace NActors {
LOG_INFO_IC("ICP08", "No active sessions, becoming PendingConnection");
SwitchToState(__LINE__, "PendingConnection", &TThis::PendingConnection);
} else {
- Y_VERIFY(CurrentStateFunc() == &TThis::StateWork);
+ Y_ABORT_UNLESS(CurrentStateFunc() == &TThis::StateWork);
}
}
}
@@ -342,15 +342,15 @@ namespace NActors {
// drop any pending XDC subscriptions
ConnectionSubscriptions.clear();
- Y_VERIFY(!IncomingHandshakeActor && !OutgoingHandshakeActor);
+ Y_ABORT_UNLESS(!IncomingHandshakeActor && !OutgoingHandshakeActor);
SwitchToState(__LINE__, "StateWork", &TThis::StateWork);
if (Session) {
// this is continuation request, check that virtual ids match
- Y_VERIFY(SessionVirtualId == msg->Self && RemoteSessionVirtualId == msg->Peer);
+ Y_ABORT_UNLESS(SessionVirtualId == msg->Self && RemoteSessionVirtualId == msg->Peer);
} else {
// this is initial request, check that we have virtual ids not filled in
- Y_VERIFY(!SessionVirtualId && !RemoteSessionVirtualId);
+ Y_ABORT_UNLESS(!SessionVirtualId && !RemoteSessionVirtualId);
}
auto error = [&](const char* description) {
@@ -374,7 +374,7 @@ namespace NActors {
}
// ensure that we have session local/peer virtual ids
- Y_VERIFY(Session && SessionVirtualId && RemoteSessionVirtualId);
+ Y_ABORT_UNLESS(Session && SessionVirtualId && RemoteSessionVirtualId);
// Set up new connection for the session.
IActor::InvokeOtherActor(*Session, &TInterconnectSessionTCP::SetNewConnection, ev);
@@ -405,7 +405,7 @@ namespace NActors {
DropOutgoingHandshake(false);
if (IEventBase* reply = HeldHandshakeReply.Release()) {
- Y_VERIFY(IncomingHandshakeActor);
+ Y_ABORT_UNLESS(IncomingHandshakeActor);
LOG_DEBUG_IC("ICP26", "sent held handshake reply to %s", IncomingHandshakeActor.ToString().data());
Send(IncomingHandshakeActor, reply);
}
@@ -532,7 +532,7 @@ namespace NActors {
void TInterconnectProxyTCP::UnregisterSession(TInterconnectSessionTCP* session) {
ICPROXY_PROFILED;
- Y_VERIFY(Session && Session == session && SessionID);
+ Y_ABORT_UNLESS(Session && Session == session && SessionID);
LOG_INFO_IC("ICP30", "unregister session Session# %s VirtualId# %s", SessionID.ToString().data(),
SessionVirtualId.ToString().data());
@@ -606,7 +606,7 @@ namespace NActors {
void TInterconnectProxyTCP::ForwardSessionEventToSession(STATEFN_SIG) {
ICPROXY_PROFILED;
- Y_VERIFY(Session && SessionID);
+ Y_ABORT_UNLESS(Session && SessionID);
ValidateEvent(ev, "ForwardSessionEventToSession");
InvokeOtherActor(*Session, &TInterconnectSessionTCP::Receive, ev);
}
@@ -753,8 +753,8 @@ namespace NActors {
UpdateErrorStateLog(TActivationContext::Now(), "permanent conclusive", explanation);
}
- Y_VERIFY(Session == nullptr);
- Y_VERIFY(!SessionID);
+ Y_ABORT_UNLESS(Session == nullptr);
+ Y_ABORT_UNLESS(!SessionID);
// recalculate wakeup timeout -- if this is the first failure, then we sleep for default timeout; otherwise we
// sleep N times longer than the previous try, but not longer than desired number of seconds
@@ -825,7 +825,7 @@ namespace NActors {
void TInterconnectProxyTCP::HandleCleanupEventQueue() {
ICPROXY_PROFILED;
- Y_VERIFY(CleanupEventQueueScheduled);
+ Y_ABORT_UNLESS(CleanupEventQueueScheduled);
CleanupEventQueueScheduled = false;
CleanupEventQueue();
ScheduleCleanupEventQueue();
@@ -935,7 +935,7 @@ namespace NActors {
IActor::InvokeOtherActor(*Session, &TInterconnectSessionTCP::Terminate, TDisconnectReason());
}
if (DynamicPtr) {
- Y_VERIFY(*DynamicPtr == this);
+ Y_ABORT_UNLESS(*DynamicPtr == this);
*DynamicPtr = nullptr;
}
// TODO: unregister actor mon page
diff --git a/library/cpp/actors/interconnect/interconnect_tcp_proxy.h b/library/cpp/actors/interconnect/interconnect_tcp_proxy.h
index 009d5f1c21..b03b406195 100644
--- a/library/cpp/actors/interconnect/interconnect_tcp_proxy.h
+++ b/library/cpp/actors/interconnect/interconnect_tcp_proxy.h
@@ -153,15 +153,15 @@ namespace NActors {
void Ignore(TEvHandshakeDone::TPtr& ev) {
ICPROXY_PROFILED;
- Y_VERIFY(ev->Sender != IncomingHandshakeActor);
- Y_VERIFY(ev->Sender != OutgoingHandshakeActor);
+ Y_ABORT_UNLESS(ev->Sender != IncomingHandshakeActor);
+ Y_ABORT_UNLESS(ev->Sender != OutgoingHandshakeActor);
}
void Ignore(TEvHandshakeFail::TPtr& ev) {
ICPROXY_PROFILED;
- Y_VERIFY(ev->Sender != IncomingHandshakeActor);
- Y_VERIFY(ev->Sender != OutgoingHandshakeActor);
+ Y_ABORT_UNLESS(ev->Sender != IncomingHandshakeActor);
+ Y_ABORT_UNLESS(ev->Sender != OutgoingHandshakeActor);
LogHandshakeFail(ev, true);
}
@@ -176,7 +176,7 @@ namespace NActors {
State = name;
StateSwitchTime = TActivationContext::Now();
Become(std::forward<TArgs>(args)...);
- Y_VERIFY(!Terminated || CurrentStateFunc() == &TThis::HoldByError); // ensure we never escape this state
+ Y_ABORT_UNLESS(!Terminated || CurrentStateFunc() == &TThis::HoldByError); // ensure we never escape this state
if (CurrentStateFunc() != &TThis::PendingActivation) {
PassAwayTimestamp = TMonotonic::Max();
} else if (DynamicPtr) {
@@ -195,14 +195,14 @@ namespace NActors {
void SwitchToInitialState() {
ICPROXY_PROFILED;
- Y_VERIFY(!PendingSessionEvents && !PendingIncomingHandshakeEvents, "%s PendingSessionEvents# %zu"
+ Y_ABORT_UNLESS(!PendingSessionEvents && !PendingIncomingHandshakeEvents, "%s PendingSessionEvents# %zu"
" PendingIncomingHandshakeEvents# %zu State# %s", LogPrefix.data(), PendingSessionEvents.size(),
PendingIncomingHandshakeEvents.size(), State);
SwitchToState(__LINE__, "PendingActivation", &TThis::PendingActivation);
}
void HandlePassAwayIfNeeded() {
- Y_VERIFY(PassAwayScheduled);
+ Y_ABORT_UNLESS(PassAwayScheduled);
const TMonotonic now = TActivationContext::Monotonic();
if (now >= PassAwayTimestamp) {
PassAway();
@@ -367,7 +367,7 @@ namespace NActors {
Y_VERIFY_DEBUG(false, "%s", msg.data());
}
- Y_VERIFY(ev->GetTypeRewrite() != TEvInterconnect::EvForward || ev->Recipient.NodeId() == PeerNodeId,
+ Y_ABORT_UNLESS(ev->GetTypeRewrite() != TEvInterconnect::EvForward || ev->Recipient.NodeId() == PeerNodeId,
"Recipient/Proxy NodeId mismatch Recipient# %s Type# 0x%08" PRIx32 " PeerNodeId# %" PRIu32 " Func# %s",
ev->Recipient.ToString().data(), ev->Type, PeerNodeId, func);
}
@@ -482,7 +482,7 @@ namespace NActors {
}
// ensure we have no current session
- Y_VERIFY(!Session);
+ Y_ABORT_UNLESS(!Session);
// switch to pending connection state -- we wait for handshakes, we want more handshakes!
SwitchToState(__LINE__, "PendingConnection", &TThis::PendingConnection);
diff --git a/library/cpp/actors/interconnect/interconnect_tcp_server.cpp b/library/cpp/actors/interconnect/interconnect_tcp_server.cpp
index ede35b0b8b..09c5987e81 100644
--- a/library/cpp/actors/interconnect/interconnect_tcp_server.cpp
+++ b/library/cpp/actors/interconnect/interconnect_tcp_server.cpp
@@ -85,7 +85,7 @@ namespace NActors {
callback(Port, TlsActivationContext->ExecutorThread.ActorSystem);
}
const bool success = ctx.Send(MakePollerActorId(), new TEvPollerRegister(Listener, SelfId(), {}));
- Y_VERIFY(success);
+ Y_ABORT_UNLESS(success);
Become(&TThis::Listen);
}
@@ -104,7 +104,7 @@ namespace NActors {
ctx.Register(CreateIncomingHandshakeActor(ProxyCommonCtx, std::move(socket)));
continue;
} else if (-r != EAGAIN && -r != EWOULDBLOCK) {
- Y_VERIFY(-r != ENFILE && -r != EMFILE && !ExternalSocket);
+ Y_ABORT_UNLESS(-r != ENFILE && -r != EMFILE && !ExternalSocket);
LOG_ERROR_IC("ICL06", "Listen failed: %s (%s:%u)", strerror(-r), Address.data(), Port);
Listener.Reset();
PollerToken.Reset();
diff --git a/library/cpp/actors/interconnect/interconnect_tcp_session.cpp b/library/cpp/actors/interconnect/interconnect_tcp_session.cpp
index b58ca345a9..3644706e51 100644
--- a/library/cpp/actors/interconnect/interconnect_tcp_session.cpp
+++ b/library/cpp/actors/interconnect/interconnect_tcp_session.cpp
@@ -238,10 +238,10 @@ namespace NActors {
// register our socket in poller actor
LOG_DEBUG_IC_SESSION("ICS11", "registering socket in PollerActor");
const bool success = Send(MakePollerActorId(), new TEvPollerRegister(Socket, ReceiverId, SelfId()));
- Y_VERIFY(success);
+ Y_ABORT_UNLESS(success);
if (XdcSocket) {
const bool success = Send(MakePollerActorId(), new TEvPollerRegister(XdcSocket, ReceiverId, SelfId()));
- Y_VERIFY(success);
+ Y_ABORT_UNLESS(success);
}
LostConnectionWatchdog.Disarm();
@@ -273,7 +273,7 @@ namespace NActors {
ForcedWriteLength = 0;
const ui64 serial = OutputCounter - SendQueue.size() + 1;
- Y_VERIFY(serial > LastConfirmed, "%s serial# %" PRIu64 " LastConfirmed# %" PRIu64, LogPrefix.data(), serial, LastConfirmed);
+ Y_ABORT_UNLESS(serial > LastConfirmed, "%s serial# %" PRIu64 " LastConfirmed# %" PRIu64, LogPrefix.data(), serial, LastConfirmed);
LOG_DEBUG_IC_SESSION("ICS06", "rewind SendQueue size# %zu LastConfirmed# %" PRIu64 " NextSerial# %" PRIu64,
SendQueue.size(), LastConfirmed, serial);
@@ -677,7 +677,7 @@ namespace NActors {
TStackVec<TConstIoVec, iovLimit> wbuffers;
stream.ProduceIoVec(wbuffers, maxElementsInIOV, maxBytes);
- Y_VERIFY(!wbuffers.empty());
+ Y_ABORT_UNLESS(!wbuffers.empty());
TString err;
ssize_t r = 0;
@@ -784,11 +784,11 @@ namespace NActors {
serial = ++OutputCounter;
// fill the data packet
- Y_VERIFY(NumEventsInQueue);
+ Y_ABORT_UNLESS(NumEventsInQueue);
LWPROBE_IF_TOO_LONG(SlowICFillSendingBuffer, Proxy->PeerNodeId, ms) {
FillSendingBuffer(packet, serial);
}
- Y_VERIFY(!packet.IsEmpty());
+ Y_ABORT_UNLESS(!packet.IsEmpty());
InflightDataAmount += packet.GetDataSize();
Proxy->Metrics->AddInflightDataAmount(packet.GetDataSize());
@@ -818,7 +818,7 @@ namespace NActors {
const size_t outgoingStreamSizeAfter = stream.CalculateOutgoingSize();
const size_t xdcStreamSizeAfter = XdcStream.CalculateOutgoingSize();
- Y_VERIFY(outgoingStreamSizeAfter == outgoingStreamSizeBefore + packetSize &&
+ Y_ABORT_UNLESS(outgoingStreamSizeAfter == outgoingStreamSizeBefore + packetSize &&
xdcStreamSizeAfter == xdcStreamSizeBefore + packet.GetExternalSize(),
"outgoingStreamSizeBefore# %zu outgoingStreamSizeAfter# %zu packetSize# %zu"
" xdcStreamSizeBefore# %zu xdcStreamSizeAfter# %zu externalSize# %" PRIu32,
@@ -848,7 +848,7 @@ namespace NActors {
void TInterconnectSessionTCP::DropConfirmed(ui64 confirm) {
LOG_DEBUG_IC_SESSION("ICS23", "confirm count: %" PRIu64, confirm);
- Y_VERIFY(LastConfirmed <= confirm && confirm <= OutputCounter,
+ Y_ABORT_UNLESS(LastConfirmed <= confirm && confirm <= OutputCounter,
"%s confirm# %" PRIu64 " LastConfirmed# %" PRIu64 " OutputCounter# %" PRIu64,
LogPrefix.data(), confirm, LastConfirmed, OutputCounter);
LastConfirmed = confirm;
@@ -907,7 +907,7 @@ namespace NActors {
void TInterconnectSessionTCP::FillSendingBuffer(TTcpPacketOutTask& task, ui64 serial) {
ui32 bytesGenerated = 0;
- Y_VERIFY(NumEventsInQueue);
+ Y_ABORT_UNLESS(NumEventsInQueue);
while (NumEventsInQueue) {
TEventOutputChannel *channel = ChannelScheduler->PickChannelWithLeastConsumedWeight();
Y_VERIFY_DEBUG(!channel->IsEmpty());
@@ -934,7 +934,7 @@ namespace NActors {
if (eventDone) {
++MessagesWrittenToBuffer;
- Y_VERIFY(NumEventsInQueue);
+ Y_ABORT_UNLESS(NumEventsInQueue);
--NumEventsInQueue;
if (!NumEventsInQueue) {
@@ -947,7 +947,7 @@ namespace NActors {
}
}
- Y_VERIFY(bytesGenerated); // ensure we are not stalled in serialization
+ Y_ABORT_UNLESS(bytesGenerated); // ensure we are not stalled in serialization
}
ui32 TInterconnectSessionTCP::CalculateQueueUtilization() {
diff --git a/library/cpp/actors/interconnect/mock/ic_mock.cpp b/library/cpp/actors/interconnect/mock/ic_mock.cpp
index 875f4ff553..9178e52da8 100644
--- a/library/cpp/actors/interconnect/mock/ic_mock.cpp
+++ b/library/cpp/actors/interconnect/mock/ic_mock.cpp
@@ -46,7 +46,7 @@ namespace NActors {
void Attach(ui32 nodeId, TActorSystem *as, const TActorId& actorId) {
TPeerInfo *peer = GetPeer(nodeId);
auto guard = TWriteGuard(peer->Mutex);
- Y_VERIFY(!peer->ActorSystem);
+ Y_ABORT_UNLESS(!peer->ActorSystem);
peer->ActorSystem = as;
peer->ProxyId = actorId;
as->DeferPreStop([peer] {
@@ -119,7 +119,7 @@ namespace NActors {
for (const auto& kv : Subscribers) {
Send(kv.first, new TEvInterconnect::TEvNodeDisconnected(Proxy->PeerNodeId), 0, kv.second);
}
- Y_VERIFY(Proxy->Session == this);
+ Y_ABORT_UNLESS(Proxy->Session == this);
Proxy->Session = nullptr;
PassAway();
}
@@ -242,7 +242,7 @@ namespace NActors {
}
void HandleNodeInfo(TEvInterconnect::TEvNodeInfo::TPtr ev) {
- Y_VERIFY(IsWaitingForNodeInfo);
+ Y_ABORT_UNLESS(IsWaitingForNodeInfo);
if (!ev->Get()->Node) {
PeerNodeStatus = EPeerNodeStatus::MISSING;
} else {
@@ -339,7 +339,7 @@ namespace NActors {
}
void PeerInject(std::deque<std::unique_ptr<IEventHandle>>&& messages) {
- Y_VERIFY(Session);
+ Y_ABORT_UNLESS(Session);
return State.Inject(PeerNodeId, std::move(messages), Common->LocalScopeId, Session->SessionId);
}
@@ -362,9 +362,9 @@ namespace NActors {
public:
IActor *CreateProxyMock(ui32 nodeId, ui32 peerNodeId, TInterconnectProxyCommon::TPtr common) {
- Y_VERIFY(nodeId != peerNodeId);
- Y_VERIFY(nodeId);
- Y_VERIFY(peerNodeId);
+ Y_ABORT_UNLESS(nodeId != peerNodeId);
+ Y_ABORT_UNLESS(nodeId);
+ Y_ABORT_UNLESS(peerNodeId);
const ui64 key = std::min(nodeId, peerNodeId) | ui64(std::max(nodeId, peerNodeId)) << 32;
auto it = States.try_emplace(key, key).first;
return new TProxyMockActor(nodeId, peerNodeId, it->second, std::move(common));
diff --git a/library/cpp/actors/interconnect/outgoing_stream.h b/library/cpp/actors/interconnect/outgoing_stream.h
index 197b9219c2..23982c4a3e 100644
--- a/library/cpp/actors/interconnect/outgoing_stream.h
+++ b/library/cpp/actors/interconnect/outgoing_stream.h
@@ -57,7 +57,7 @@ namespace NInterconnect {
for (auto it = SendQueue.begin() + SendQueuePos; it != SendQueue.end(); ++it) {
res += it->Span.size();
}
- Y_VERIFY(UnsentBytes == res - SendOffset);
+ Y_ABORT_UNLESS(UnsentBytes == res - SendOffset);
#endif
return UnsentBytes;
}
@@ -70,7 +70,7 @@ namespace NInterconnect {
if (maxLen && AppendOffset == BufferSize) { // we have no free buffer, allocate one
Buffers.emplace_back(static_cast<TBuffer*>(malloc(sizeof(TBuffer))));
AppendBuffer = Buffers.back().get();
- Y_VERIFY(AppendBuffer);
+ Y_ABORT_UNLESS(AppendBuffer);
AppendBuffer->RefCount = 1; // through AppendBuffer pointer
AppendBuffer->Index = Buffers.size() - 1;
AppendOffset = 0;
diff --git a/library/cpp/actors/interconnect/packet.h b/library/cpp/actors/interconnect/packet.h
index df0bf0a8c7..105cc41ce3 100644
--- a/library/cpp/actors/interconnect/packet.h
+++ b/library/cpp/actors/interconnect/packet.h
@@ -224,7 +224,7 @@ struct TTcpPacketOutTask : TNonCopyable {
}
void Finish(ui64 serial, ui64 confirm) {
- Y_VERIFY(InternalSize <= Max<ui16>());
+ Y_ABORT_UNLESS(InternalSize <= Max<ui16>());
TTcpPacketHeader_v2 header{
confirm,
diff --git a/library/cpp/actors/interconnect/poller_actor.cpp b/library/cpp/actors/interconnect/poller_actor.cpp
index 9dbd9838aa..e3ddbe8b98 100644
--- a/library/cpp/actors/interconnect/poller_actor.cpp
+++ b/library/cpp/actors/interconnect/poller_actor.cpp
@@ -137,7 +137,7 @@ namespace NActors {
Y_FAIL("WriteEnd.Write() failed with %s", strerror(err));
}
} else {
- Y_VERIFY(nwritten);
+ Y_ABORT_UNLESS(nwritten);
break;
}
}
@@ -160,13 +160,13 @@ namespace NActors {
Y_FAIL("read() failed with %s", strerror(errno));
}
} else {
- Y_VERIFY(n);
+ Y_ABORT_UNLESS(n);
}
}
}
bool ProcessSyncOpQueue() {
- Y_VERIFY(!SyncOperationsQ.IsEmpty());
+ Y_ABORT_UNLESS(!SyncOperationsQ.IsEmpty());
do {
TPollerSyncOperationWrapper *op = SyncOperationsQ.Top();
if (auto *unregister = std::get_if<TPollerUnregisterSocket>(&op->Operation)) {
diff --git a/library/cpp/actors/interconnect/poller_actor_darwin.h b/library/cpp/actors/interconnect/poller_actor_darwin.h
index 31c1144794..f091972502 100644
--- a/library/cpp/actors/interconnect/poller_actor_darwin.h
+++ b/library/cpp/actors/interconnect/poller_actor_darwin.h
@@ -13,7 +13,7 @@ namespace NActors {
do {
rc = kevent(KqDescriptor, ev, size, nullptr, 0, nullptr);
} while (rc == -1 && errno == EINTR);
- Y_VERIFY(rc != -1, "kevent() failed with %s", strerror(errno));
+ Y_ABORT_UNLESS(rc != -1, "kevent() failed with %s", strerror(errno));
}
public:
@@ -22,14 +22,14 @@ namespace NActors {
{
// create kqueue
KqDescriptor = kqueue();
- Y_VERIFY(KqDescriptor != -1, "kqueue() failed with %s", strerror(errno));
+ Y_ABORT_UNLESS(KqDescriptor != -1, "kqueue() failed with %s", strerror(errno));
// set close-on-exit flag
{
int flags = fcntl(KqDescriptor, F_GETFD);
- Y_VERIFY(flags >= 0, "fcntl(F_GETFD) failed with %s", strerror(errno));
+ Y_ABORT_UNLESS(flags >= 0, "fcntl(F_GETFD) failed with %s", strerror(errno));
int rc = fcntl(KqDescriptor, F_SETFD, flags | FD_CLOEXEC);
- Y_VERIFY(rc != -1, "fcntl(F_SETFD, +FD_CLOEXEC) failed with %s", strerror(errno));
+ Y_ABORT_UNLESS(rc != -1, "fcntl(F_SETFD, +FD_CLOEXEC) failed with %s", strerror(errno));
}
// register pipe's read end in poller
diff --git a/library/cpp/actors/interconnect/poller_actor_linux.h b/library/cpp/actors/interconnect/poller_actor_linux.h
index 6bd2cc258f..f19f53f730 100644
--- a/library/cpp/actors/interconnect/poller_actor_linux.h
+++ b/library/cpp/actors/interconnect/poller_actor_linux.h
@@ -13,7 +13,7 @@ namespace NActors {
: TPollerThreadBase(actorSystem)
{
EpollDescriptor = epoll_create1(EPOLL_CLOEXEC);
- Y_VERIFY(EpollDescriptor != -1, "epoll_create1() failed with %s", strerror(errno));
+ Y_ABORT_UNLESS(EpollDescriptor != -1, "epoll_create1() failed with %s", strerror(errno));
epoll_event event;
event.data.ptr = nullptr;
diff --git a/library/cpp/actors/interconnect/poller_actor_win.h b/library/cpp/actors/interconnect/poller_actor_win.h
index e593cbafd1..9666c32be7 100644
--- a/library/cpp/actors/interconnect/poller_actor_win.h
+++ b/library/cpp/actors/interconnect/poller_actor_win.h
@@ -97,7 +97,7 @@ namespace NActors {
void Request(const TIntrusivePtr<TSocketRecord>& record, bool read, bool write) {
with_lock (Mutex) {
const auto it = Descriptors.find(record->Socket->GetDescriptor());
- Y_VERIFY(it != Descriptors.end());
+ Y_ABORT_UNLESS(it != Descriptors.end());
it->second->Flags |= (read ? READ : 0) | (write ? WRITE : 0);
}
ExecuteSyncOperation(TPollerWakeup());
diff --git a/library/cpp/actors/interconnect/profiler.h b/library/cpp/actors/interconnect/profiler.h
index 77a59e3179..11dac077ea 100644
--- a/library/cpp/actors/interconnect/profiler.h
+++ b/library/cpp/actors/interconnect/profiler.h
@@ -67,7 +67,7 @@ namespace NActors {
TString Format() const {
TDeque<TItem>::iterator it = Items.begin();
TString res = FormatLevel(it);
- Y_VERIFY(it == Items.end());
+ Y_ABORT_UNLESS(it == Items.end());
return res;
}
@@ -85,13 +85,13 @@ namespace NActors {
TVector<TRecord> records;
while (it != Items.end() && it->Type != EType::EXIT) {
- Y_VERIFY(it->Type == EType::ENTRY);
+ Y_ABORT_UNLESS(it->Type == EType::ENTRY);
const TString marker = Sprintf("%s:%d", it->Marker, it->Line);
const ui64 begin = it->Timestamp;
++it;
const TString interior = FormatLevel(it);
- Y_VERIFY(it != Items.end());
- Y_VERIFY(it->Type == EType::EXIT);
+ Y_ABORT_UNLESS(it != Items.end());
+ Y_ABORT_UNLESS(it->Type == EType::EXIT);
const ui64 end = it->Timestamp;
records.push_back(TRecord{marker, end - begin, interior});
++it;
diff --git a/library/cpp/actors/interconnect/ut/channel_scheduler_ut.cpp b/library/cpp/actors/interconnect/ut/channel_scheduler_ut.cpp
index 2f8e575cb0..98e81d7781 100644
--- a/library/cpp/actors/interconnect/ut/channel_scheduler_ut.cpp
+++ b/library/cpp/actors/interconnect/ut/channel_scheduler_ut.cpp
@@ -62,7 +62,7 @@ Y_UNIT_TEST_SUITE(ChannelScheduler) {
ui64 weightConsumed = 0;
numEvents -= channel->FeedBuf(task, 0, &weightConsumed);
ui32 after = task.GetDataSize();
- Y_VERIFY(after >= before);
+ Y_ABORT_UNLESS(after >= before);
scheduler.FinishPick(weightConsumed, 0);
const ui32 bytesAdded = after - before;
if (!bytesAdded) {
diff --git a/library/cpp/actors/interconnect/ut/dynamic_proxy_ut.cpp b/library/cpp/actors/interconnect/ut/dynamic_proxy_ut.cpp
index 3c474979dc..4cc6a3cc99 100644
--- a/library/cpp/actors/interconnect/ut/dynamic_proxy_ut.cpp
+++ b/library/cpp/actors/interconnect/ut/dynamic_proxy_ut.cpp
@@ -50,11 +50,11 @@ public:
for (const auto& [queueId, st] : state) {
ui32 expected = 0;
for (const ui32 index : st.Ok) {
- Y_VERIFY(index == expected);
+ Y_ABORT_UNLESS(index == expected);
++expected;
}
for (const ui32 index : st.Error) {
- Y_VERIFY(index == expected);
+ Y_ABORT_UNLESS(index == expected);
++expected;
}
if (st.Error.size()) {
@@ -158,7 +158,7 @@ void RaceTestIter(ui32 numThreads, ui32 count) {
}
for (THPTimer timer; !arriveQueue.Done(); TDuration::MilliSeconds(10)) {
- Y_VERIFY(timer.Passed() < 10);
+ Y_ABORT_UNLESS(timer.Passed() < 10);
}
nodes.clear();
diff --git a/library/cpp/actors/interconnect/ut/interconnect_ut.cpp b/library/cpp/actors/interconnect/ut/interconnect_ut.cpp
index 3596bffd5a..8d1b47c0c6 100644
--- a/library/cpp/actors/interconnect/ut/interconnect_ut.cpp
+++ b/library/cpp/actors/interconnect/ut/interconnect_ut.cpp
@@ -27,7 +27,7 @@ public:
void Subscribe() {
Cerr << (TStringBuilder() << "Subscribe" << Endl);
- Y_VERIFY(!SubscribeInFlight);
+ Y_ABORT_UNLESS(!SubscribeInFlight);
SubscribeInFlight = true;
Send(TActivationContext::InterconnectProxy(Recipient.NodeId()), new TEvents::TEvSubscribe);
}
@@ -57,11 +57,11 @@ public:
// Cerr << (TStringBuilder() << "Receive# " << ev->Cookie << Endl);
if (const auto it = InFlight.find(ev->Cookie); it != InFlight.end()) {
auto& [s2cIt, hash] = it->second;
- Y_VERIFY(hash == ev->GetChainBuffer()->GetString());
+ Y_ABORT_UNLESS(hash == ev->GetChainBuffer()->GetString());
SessionToCookie.erase(s2cIt);
InFlight.erase(it);
} else if (const auto it = Tentative.find(ev->Cookie); it != Tentative.end()) {
- Y_VERIFY(it->second == ev->GetChainBuffer()->GetString());
+ Y_ABORT_UNLESS(it->second == ev->GetChainBuffer()->GetString());
Tentative.erase(it);
} else {
Y_FAIL("Cookie# %" PRIu64, ev->Cookie);
@@ -71,9 +71,9 @@ public:
void Handle(TEvInterconnect::TEvNodeConnected::TPtr ev) {
Cerr << (TStringBuilder() << "TEvNodeConnected" << Endl);
- Y_VERIFY(SubscribeInFlight);
+ Y_ABORT_UNLESS(SubscribeInFlight);
SubscribeInFlight = false;
- Y_VERIFY(!SessionId);
+ Y_ABORT_UNLESS(!SessionId);
SessionId = ev->Sender;
IssueQueries();
}
@@ -82,11 +82,11 @@ public:
Cerr << (TStringBuilder() << "TEvNodeDisconnected" << Endl);
SubscribeInFlight = false;
if (SessionId) {
- Y_VERIFY(SessionId == ev->Sender);
+ Y_ABORT_UNLESS(SessionId == ev->Sender);
auto r = SessionToCookie.equal_range(SessionId);
for (auto it = r.first; it != r.second; ++it) {
const auto inFlightIt = InFlight.find(it->second);
- Y_VERIFY(inFlightIt != InFlight.end());
+ Y_ABORT_UNLESS(inFlightIt != InFlight.end());
Tentative.emplace(inFlightIt->first, inFlightIt->second.second);
InFlight.erase(it->second);
}
diff --git a/library/cpp/actors/interconnect/ut/large.cpp b/library/cpp/actors/interconnect/ut/large.cpp
index 192d733325..725238242a 100644
--- a/library/cpp/actors/interconnect/ut/large.cpp
+++ b/library/cpp/actors/interconnect/ut/large.cpp
@@ -57,10 +57,10 @@ Y_UNIT_TEST_SUITE(LargeMessage) {
const auto& record = ev->Get()->Record;
Cerr << "RECEIVED TEvTest\n";
if (record.GetSequenceNumber() == 1) {
- Y_VERIFY(!SessionId);
+ Y_ABORT_UNLESS(!SessionId);
SessionId = ev->InterconnectSession;
} else if (record.GetSequenceNumber() == 3) {
- Y_VERIFY(SessionId != ev->InterconnectSession);
+ Y_ABORT_UNLESS(SessionId != ev->InterconnectSession);
Done.Signal();
} else {
Y_FAIL("incorrect sequence number");
diff --git a/library/cpp/actors/interconnect/ut/lib/interrupter.h b/library/cpp/actors/interconnect/ut/lib/interrupter.h
index 48851de2c5..b00985573a 100644
--- a/library/cpp/actors/interconnect/ut/lib/interrupter.h
+++ b/library/cpp/actors/interconnect/ut/lib/interrupter.h
@@ -86,8 +86,8 @@ public:
{
SetReuseAddressAndPort(ListenSocket);
TSockAddrInet6 addr(Address.data(), listenPort);
- Y_VERIFY(ListenSocket.Bind(&addr) == 0);
- Y_VERIFY(ListenSocket.Listen(5) == 0);
+ Y_ABORT_UNLESS(ListenSocket.Bind(&addr) == 0);
+ Y_ABORT_UNLESS(ListenSocket.Listen(5) == 0);
DelayTraffic = (Bandwidth == 0.0) ? false : true;
diff --git a/library/cpp/actors/interconnect/ut/poller_actor_ut.cpp b/library/cpp/actors/interconnect/ut/poller_actor_ut.cpp
index 23d846a2fd..67921ef9bc 100644
--- a/library/cpp/actors/interconnect/ut/poller_actor_ut.cpp
+++ b/library/cpp/actors/interconnect/ut/poller_actor_ut.cpp
@@ -36,7 +36,7 @@ std::pair<TTestSocketPtr, TTestSocketPtr> NonBlockSockets() {
std::pair<TTestSocketPtr, TTestSocketPtr> TcpSockets() {
// create server (listening) socket
SOCKET server = socket(AF_INET, SOCK_STREAM, 0);
- Y_VERIFY(server != -1, "socket() failed with %s", strerror(errno));
+ Y_ABORT_UNLESS(server != -1, "socket() failed with %s", strerror(errno));
// bind it to local address with automatically picked port
sockaddr_in addr;
@@ -57,7 +57,7 @@ std::pair<TTestSocketPtr, TTestSocketPtr> TcpSockets() {
// create client socket
SOCKET client = socket(AF_INET, SOCK_STREAM, 0);
- Y_VERIFY(client != -1, "socket() failed with %s", strerror(errno));
+ Y_ABORT_UNLESS(client != -1, "socket() failed with %s", strerror(errno));
// connect to server
if (connect(client, (sockaddr*)&addr, len) == -1) {
@@ -66,7 +66,7 @@ std::pair<TTestSocketPtr, TTestSocketPtr> TcpSockets() {
// accept connection from the other side
SOCKET accepted = accept(server, nullptr, nullptr);
- Y_VERIFY(accepted != -1, "accept() failed with %s", strerror(errno));
+ Y_ABORT_UNLESS(accepted != -1, "accept() failed with %s", strerror(errno));
// close server socket
closesocket(server);
diff --git a/library/cpp/actors/interconnect/ut/sticking_ut.cpp b/library/cpp/actors/interconnect/ut/sticking_ut.cpp
index 510baa3a1f..2fa3d0933e 100644
--- a/library/cpp/actors/interconnect/ut/sticking_ut.cpp
+++ b/library/cpp/actors/interconnect/ut/sticking_ut.cpp
@@ -71,7 +71,7 @@ public:
if (MaxRTT < rtt) {
MaxRTT = rtt;
Cerr << "Updated MaxRTT# " << MaxRTT << Endl;
- Y_VERIFY(MaxRTT <= TDuration::MilliSeconds(500));
+ Y_ABORT_UNLESS(MaxRTT <= TDuration::MilliSeconds(500));
}
--PingInFlight;
Action();
diff --git a/library/cpp/actors/interconnect/ut_fat/main.cpp b/library/cpp/actors/interconnect/ut_fat/main.cpp
index c9c77300bb..abd1cd289a 100644
--- a/library/cpp/actors/interconnect/ut_fat/main.cpp
+++ b/library/cpp/actors/interconnect/ut_fat/main.cpp
@@ -52,21 +52,21 @@ Y_UNIT_TEST_SUITE(InterconnectUnstableConnection) {
SendMessage(ctx);
}
} else {
- Y_VERIFY(record != InFly.end());
+ Y_ABORT_UNLESS(record != InFly.end());
}
}
void Handle(TEvTestResponse::TPtr& ev, const TActorContext& ctx) override {
- Y_VERIFY(InFly);
+ Y_ABORT_UNLESS(InFly);
const NInterconnectTest::TEvTestResponse& record = ev->Get()->Record;
- Y_VERIFY(record.HasConfirmedSequenceNumber());
+ Y_ABORT_UNLESS(record.HasConfirmedSequenceNumber());
if (!(SendFlags & IEventHandle::FlagGenerateUnsureUndelivered)) {
while (record.GetConfirmedSequenceNumber() != InFly.front()) {
InFly.pop_front();
--InFlySize;
}
}
- Y_VERIFY(record.GetConfirmedSequenceNumber() == InFly.front(), "got# %" PRIu64 " expected# %" PRIu64,
+ Y_ABORT_UNLESS(record.GetConfirmedSequenceNumber() == InFly.front(), "got# %" PRIu64 " expected# %" PRIu64,
record.GetConfirmedSequenceNumber(), InFly.front());
InFly.pop_front();
--InFlySize;
@@ -87,8 +87,8 @@ Y_UNIT_TEST_SUITE(InterconnectUnstableConnection) {
void Handle(TEvTest::TPtr& ev, const TActorContext& /*ctx*/) override {
const NInterconnectTest::TEvTest& m = ev->Get()->Record;
- Y_VERIFY(m.HasSequenceNumber());
- Y_VERIFY(m.GetSequenceNumber() >= ReceivedCount, "got #%" PRIu64 " expected at least #%" PRIu64,
+ Y_ABORT_UNLESS(m.HasSequenceNumber());
+ Y_ABORT_UNLESS(m.GetSequenceNumber() >= ReceivedCount, "got #%" PRIu64 " expected at least #%" PRIu64,
m.GetSequenceNumber(), ReceivedCount);
++ReceivedCount;
SenderNode->Send(ev->Sender, new TEvTestResponse(m.GetSequenceNumber()));
diff --git a/library/cpp/actors/memory_log/memlog.cpp b/library/cpp/actors/memory_log/memlog.cpp
index 17a52847d7..263c5c5079 100644
--- a/library/cpp/actors/memory_log/memlog.cpp
+++ b/library/cpp/actors/memory_log/memlog.cpp
@@ -66,7 +66,7 @@ unsigned TMemoryLog::GetSelfCpu() noexcept {
unsigned cpu;
if (Y_LIKELY(FastGetCpu != nullptr)) {
auto result = FastGetCpu(&cpu, nullptr, nullptr);
- Y_VERIFY(result == 0);
+ Y_ABORT_UNLESS(result == 0);
return cpu;
} else {
return 0;
@@ -110,7 +110,7 @@ TMemoryLog::TMemoryLog(size_t totalSize, size_t grainSize)
, FreeGrains(DEFAULT_TOTAL_SIZE / DEFAULT_GRAIN_SIZE * 2)
, Buf(totalSize)
{
- Y_VERIFY(DEFAULT_TOTAL_SIZE % DEFAULT_GRAIN_SIZE == 0);
+ Y_ABORT_UNLESS(DEFAULT_TOTAL_SIZE % DEFAULT_GRAIN_SIZE == 0);
NumberOfGrains = DEFAULT_TOTAL_SIZE / DEFAULT_GRAIN_SIZE;
for (size_t i = 0; i < NumberOfGrains; ++i) {
@@ -118,7 +118,7 @@ TMemoryLog::TMemoryLog(size_t totalSize, size_t grainSize)
}
NumberOfCpus = NSystemInfo::NumberOfCpus();
- Y_VERIFY(NumberOfGrains > NumberOfCpus);
+ Y_ABORT_UNLESS(NumberOfGrains > NumberOfCpus);
ActiveGrains.Reset(new TGrain*[NumberOfCpus]);
for (size_t i = 0; i < NumberOfCpus; ++i) {
ActiveGrains[i] = GetGrain(i);
@@ -267,7 +267,7 @@ bool MemLogWrite(const char* begin, size_t msgSize, bool addLF) noexcept {
// check for format for snprintf
constexpr size_t prologSize = 48;
alignas(TMemoryLog::MemcpyAlignment) char prolog[prologSize + 1];
- Y_VERIFY(AlignDown(&prolog, TMemoryLog::MemcpyAlignment) == &prolog);
+ Y_ABORT_UNLESS(AlignDown(&prolog, TMemoryLog::MemcpyAlignment) == &prolog);
int snprintfResult = snprintf(prolog, prologSize + 1,
"TS %020" PRIu64 " TI %020" PRIu64 " ", GetCycleCountFast(), threadId);
@@ -275,7 +275,7 @@ bool MemLogWrite(const char* begin, size_t msgSize, bool addLF) noexcept {
if (snprintfResult < 0) {
return false;
}
- Y_VERIFY(snprintfResult == prologSize);
+ Y_ABORT_UNLESS(snprintfResult == prologSize);
amount += prologSize;
if (addLF) {
@@ -336,7 +336,7 @@ bool MemLogVPrintF(const char* format, va_list params) noexcept {
// alignment required by NoCacheMemcpy
alignas(TMemoryLog::MemcpyAlignment) char buf[TMemoryLog::MAX_MESSAGE_SIZE];
- Y_VERIFY(AlignDown(&buf, TMemoryLog::MemcpyAlignment) == &buf);
+ Y_ABORT_UNLESS(AlignDown(&buf, TMemoryLog::MemcpyAlignment) == &buf);
int prologSize = snprintf(buf,
TMemoryLog::MAX_MESSAGE_SIZE - 2,
@@ -347,7 +347,7 @@ bool MemLogVPrintF(const char* format, va_list params) noexcept {
if (Y_UNLIKELY(prologSize < 0)) {
return false;
}
- Y_VERIFY((ui32)prologSize <= TMemoryLog::MAX_MESSAGE_SIZE);
+ Y_ABORT_UNLESS((ui32)prologSize <= TMemoryLog::MAX_MESSAGE_SIZE);
int add = vsnprintf(
&buf[prologSize],
@@ -357,11 +357,11 @@ bool MemLogVPrintF(const char* format, va_list params) noexcept {
if (Y_UNLIKELY(add < 0)) {
return false;
}
- Y_VERIFY(add >= 0);
+ Y_ABORT_UNLESS(add >= 0);
auto totalSize = prologSize + add;
buf[totalSize++] = '\n';
- Y_VERIFY((ui32)totalSize <= TMemoryLog::MAX_MESSAGE_SIZE);
+ Y_ABORT_UNLESS((ui32)totalSize <= TMemoryLog::MAX_MESSAGE_SIZE);
return BareMemLogWrite(buf, totalSize) != nullptr;
}
diff --git a/library/cpp/actors/memory_log/mmap.cpp b/library/cpp/actors/memory_log/mmap.cpp
index 201998d343..1fe734235e 100644
--- a/library/cpp/actors/memory_log/mmap.cpp
+++ b/library/cpp/actors/memory_log/mmap.cpp
@@ -9,7 +9,7 @@
#endif
void TMemoryLog::TMMapArea::MMap(size_t amount) {
- Y_VERIFY(amount > 0);
+ Y_ABORT_UNLESS(amount > 0);
#if defined(_unix_)
constexpr int mmapProt = PROT_READ | PROT_WRITE;
@@ -46,14 +46,14 @@ void TMemoryLog::TMMapArea::MUnmap() {
#if defined(_unix_)
int result = ::munmap(BufPtr, Size);
- Y_VERIFY(result == 0);
+ Y_ABORT_UNLESS(result == 0);
#elif defined(_win_)
BOOL result = ::UnmapViewOfFile(BufPtr);
- Y_VERIFY(result != 0);
+ Y_ABORT_UNLESS(result != 0);
result = ::CloseHandle(Mapping);
- Y_VERIFY(result != 0);
+ Y_ABORT_UNLESS(result != 0);
Mapping = 0;
#endif
diff --git a/library/cpp/actors/prof/tag.cpp b/library/cpp/actors/prof/tag.cpp
index e755ed8ca9..99248a135f 100644
--- a/library/cpp/actors/prof/tag.cpp
+++ b/library/cpp/actors/prof/tag.cpp
@@ -30,21 +30,21 @@ namespace NProfiling {
}
ui32 MakeTag(const char* s) {
- Y_VERIFY(s);
+ Y_ABORT_UNLESS(s);
with_lock (Mutex) {
return Tags.string_to_atom(s);
}
}
ui32 MakeTags(const TVector<const char*>& ss) {
- Y_VERIFY(ss);
+ Y_ABORT_UNLESS(ss);
with_lock (Mutex) {
ui32 baseTag = Tags.string_to_atom(ss[0]);
ui32 nextTag = baseTag + 1;
for (auto i = ss.begin() + 1; i != ss.end(); ++i, ++nextTag) {
- Y_VERIFY(*i);
+ Y_ABORT_UNLESS(*i);
ui32 ctag = Tags.string_to_atom(*i);
- Y_VERIFY(ctag == nextTag);
+ Y_ABORT_UNLESS(ctag == nextTag);
}
return baseTag;
}
diff --git a/library/cpp/actors/testlib/test_runtime.cpp b/library/cpp/actors/testlib/test_runtime.cpp
index dad62ca697..256116ce32 100644
--- a/library/cpp/actors/testlib/test_runtime.cpp
+++ b/library/cpp/actors/testlib/test_runtime.cpp
@@ -67,7 +67,7 @@ namespace NActors {
if (MailboxTable) {
for (ui32 round = 0; !MailboxTable->Cleanup(); ++round)
- Y_VERIFY(round < 10, "cyclic event/actor spawn while trying to shutdown actorsystem stub");
+ Y_ABORT_UNLESS(round < 10, "cyclic event/actor spawn while trying to shutdown actorsystem stub");
}
if (ActorSystem)
@@ -108,7 +108,7 @@ namespace NActors {
if (!Runtime->EventFilterFunc(*Runtime, ev)) {
ui32 nodeId = ev->GetRecipientRewrite().NodeId();
- Y_VERIFY(nodeId != 0);
+ Y_ABORT_UNLESS(nodeId != 0);
ui32 mailboxHint = ev->GetRecipientRewrite().Hint();
Runtime->GetMailbox(nodeId, mailboxHint).Send(ev);
Runtime->MailboxesHasEvents.Signal();
@@ -127,7 +127,7 @@ namespace NActors {
void TEventMailBox::Send(TAutoPtr<IEventHandle> ev) {
IEventHandle* ptr = ev.Get();
- Y_VERIFY(ptr);
+ Y_ABORT_UNLESS(ptr);
#ifdef DEBUG_ORDER_EVENTS
ui64 counter = NextToSend++;
TrackSent[ptr] = counter;
@@ -141,7 +141,7 @@ namespace NActors {
#ifdef DEBUG_ORDER_EVENTS
auto it = TrackSent.find(result.Get());
if (it != TrackSent.end()) {
- Y_VERIFY(ExpectedReceive == it->second);
+ Y_ABORT_UNLESS(ExpectedReceive == it->second);
TrackSent.erase(result.Get());
++ExpectedReceive;
}
@@ -378,7 +378,7 @@ namespace NActors {
if (!Runtime->EventFilterFunc(*Runtime, ev)) {
ui32 nodeId = ev->GetRecipientRewrite().NodeId();
- Y_VERIFY(nodeId != 0);
+ Y_ABORT_UNLESS(nodeId != 0);
TNodeDataBase* node = Runtime->Nodes[nodeId].Get();
if (!AllowSendFrom(node, ev)) {
@@ -752,8 +752,8 @@ namespace NActors {
}
void TTestActorRuntimeBase::AddLocalService(const TActorId& actorId, TActorSetupCmd cmd, ui32 nodeIndex) {
- Y_VERIFY(!IsInitialized);
- Y_VERIFY(nodeIndex < NodeCount);
+ Y_ABORT_UNLESS(!IsInitialized);
+ Y_ABORT_UNLESS(nodeIndex < NodeCount);
auto node = Nodes[nodeIndex + FirstNodeId];
if (!node) {
node = GetNodeFactory().CreateNode();
@@ -766,7 +766,7 @@ namespace NActors {
void TTestActorRuntimeBase::InitNodes() {
NextNodeId += NodeCount;
- Y_VERIFY(NodeCount > 0);
+ Y_ABORT_UNLESS(NodeCount > 0);
for (ui32 nodeIndex = 0; nodeIndex < NodeCount; ++nodeIndex) {
auto nodeIt = Nodes.emplace(FirstNodeId + nodeIndex, GetNodeFactory().CreateNode()).first;
@@ -800,7 +800,7 @@ namespace NActors {
}
void TTestActorRuntimeBase::SetLogBackend(const TAutoPtr<TLogBackend> logBackend) {
- Y_VERIFY(!IsInitialized);
+ Y_ABORT_UNLESS(!IsInitialized);
TGuard<TMutex> guard(Mutex);
LogBackend = logBackend;
}
@@ -819,13 +819,13 @@ namespace NActors {
TInstant TTestActorRuntimeBase::GetCurrentTime() const {
TGuard<TMutex> guard(Mutex);
- Y_VERIFY(!UseRealThreads);
+ Y_ABORT_UNLESS(!UseRealThreads);
return TInstant::MicroSeconds(CurrentTimestamp);
}
TMonotonic TTestActorRuntimeBase::GetCurrentMonotonicTime() const {
TGuard<TMutex> guard(Mutex);
- Y_VERIFY(!UseRealThreads);
+ Y_ABORT_UNLESS(!UseRealThreads);
return TMonotonic::MicroSeconds(CurrentTimestamp);
}
@@ -836,7 +836,7 @@ namespace NActors {
Cerr << "UpdateCurrentTime(" << counter << "," << newTime << ")\n";
}
TGuard<TMutex> guard(Mutex);
- Y_VERIFY(!UseRealThreads);
+ Y_ABORT_UNLESS(!UseRealThreads);
if (newTime.MicroSeconds() > CurrentTimestamp) {
CurrentTimestamp = newTime.MicroSeconds();
for (auto& kv : Nodes) {
@@ -851,17 +851,17 @@ namespace NActors {
}
TIntrusivePtr<ITimeProvider> TTestActorRuntimeBase::GetTimeProvider() {
- Y_VERIFY(!UseRealThreads);
+ Y_ABORT_UNLESS(!UseRealThreads);
return TimeProvider;
}
TIntrusivePtr<IMonotonicTimeProvider> TTestActorRuntimeBase::GetMonotonicTimeProvider() {
- Y_VERIFY(!UseRealThreads);
+ Y_ABORT_UNLESS(!UseRealThreads);
return MonotonicTimeProvider;
}
ui32 TTestActorRuntimeBase::GetNodeId(ui32 index) const {
- Y_VERIFY(index < NodeCount);
+ Y_ABORT_UNLESS(index < NodeCount);
return FirstNodeId + index;
}
@@ -896,11 +896,11 @@ namespace NActors {
TActorId TTestActorRuntimeBase::Register(IActor* actor, ui32 nodeIndex, ui32 poolId, TMailboxType::EType mailboxType,
ui64 revolvingCounter, const TActorId& parentId) {
- Y_VERIFY(nodeIndex < NodeCount);
+ Y_ABORT_UNLESS(nodeIndex < NodeCount);
TGuard<TMutex> guard(Mutex);
TNodeDataBase* node = Nodes[FirstNodeId + nodeIndex].Get();
if (UseRealThreads) {
- Y_VERIFY(poolId < node->ExecutorPools.size());
+ Y_ABORT_UNLESS(poolId < node->ExecutorPools.size());
return node->ExecutorPools[poolId]->Register(actor, mailboxType, revolvingCounter, parentId);
}
@@ -964,11 +964,11 @@ namespace NActors {
TActorId TTestActorRuntimeBase::Register(IActor *actor, ui32 nodeIndex, ui32 poolId, TMailboxHeader *mailbox, ui32 hint,
const TActorId& parentId) {
- Y_VERIFY(nodeIndex < NodeCount);
+ Y_ABORT_UNLESS(nodeIndex < NodeCount);
TGuard<TMutex> guard(Mutex);
TNodeDataBase* node = Nodes[FirstNodeId + nodeIndex].Get();
if (UseRealThreads) {
- Y_VERIFY(poolId < node->ExecutorPools.size());
+ Y_ABORT_UNLESS(poolId < node->ExecutorPools.size());
return node->ExecutorPools[poolId]->Register(actor, mailbox, hint, parentId);
}
@@ -988,7 +988,7 @@ namespace NActors {
TActorId TTestActorRuntimeBase::RegisterService(const TActorId& serviceId, const TActorId& actorId, ui32 nodeIndex) {
TGuard<TMutex> guard(Mutex);
- Y_VERIFY(nodeIndex < NodeCount);
+ Y_ABORT_UNLESS(nodeIndex < NodeCount);
TNodeDataBase* node = Nodes[FirstNodeId + nodeIndex].Get();
if (!UseRealThreads) {
IActor* actor = FindActor(actorId, node);
@@ -1001,7 +1001,7 @@ namespace NActors {
TActorId TTestActorRuntimeBase::AllocateEdgeActor(ui32 nodeIndex) {
TGuard<TMutex> guard(Mutex);
- Y_VERIFY(nodeIndex < NodeCount);
+ Y_ABORT_UNLESS(nodeIndex < NodeCount);
TActorId edgeActor = Register(new TEdgeActor(this), nodeIndex);
EdgeActors.insert(edgeActor);
EdgeActorByMailbox[TEventMailboxId(edgeActor.NodeId(), edgeActor.Hint())] = edgeActor;
@@ -1020,7 +1020,7 @@ namespace NActors {
TEventsList TTestActorRuntimeBase::CaptureMailboxEvents(ui32 hint, ui32 nodeId) {
TGuard<TMutex> guard(Mutex);
- Y_VERIFY(nodeId >= FirstNodeId && nodeId < FirstNodeId + NodeCount);
+ Y_ABORT_UNLESS(nodeId >= FirstNodeId && nodeId < FirstNodeId + NodeCount);
TEventsList result;
GetMailbox(nodeId, hint).Capture(result);
return result;
@@ -1029,7 +1029,7 @@ namespace NActors {
void TTestActorRuntimeBase::PushFront(TAutoPtr<IEventHandle>& ev) {
TGuard<TMutex> guard(Mutex);
ui32 nodeId = ev->GetRecipientRewrite().NodeId();
- Y_VERIFY(nodeId != 0);
+ Y_ABORT_UNLESS(nodeId != 0);
GetMailbox(nodeId, ev->GetRecipientRewrite().Hint()).PushFront(ev);
}
@@ -1039,7 +1039,7 @@ namespace NActors {
if (*rit) {
auto& ev = *rit;
ui32 nodeId = ev->GetRecipientRewrite().NodeId();
- Y_VERIFY(nodeId != 0);
+ Y_ABORT_UNLESS(nodeId != 0);
GetMailbox(nodeId, ev->GetRecipientRewrite().Hint()).PushFront(ev);
}
}
@@ -1049,7 +1049,7 @@ namespace NActors {
void TTestActorRuntimeBase::PushMailboxEventsFront(ui32 hint, ui32 nodeId, TEventsList& events) {
TGuard<TMutex> guard(Mutex);
- Y_VERIFY(nodeId >= FirstNodeId && nodeId < FirstNodeId + NodeCount);
+ Y_ABORT_UNLESS(nodeId >= FirstNodeId && nodeId < FirstNodeId + NodeCount);
TEventsList result;
GetMailbox(nodeId, hint).PushFront(events);
events.clear();
@@ -1118,7 +1118,7 @@ namespace NActors {
Runtime.GetMailbox(edgeActor.NodeId(), edgeActor.Hint()).Capture(events);
auto mboxId = TEventMailboxId(edgeActor.NodeId(), edgeActor.Hint());
auto storeIt = Store.find(mboxId);
- Y_VERIFY(storeIt == Store.end());
+ Y_ABORT_UNLESS(storeIt == Store.end());
storeIt = Store.insert(std::make_pair(mboxId, new TEventMailBox)).first;
storeIt->second->PushFront(events);
if (!events.empty())
@@ -1256,7 +1256,7 @@ namespace NActors {
}
}
- Y_VERIFY(mboxIt != currentMailboxes.end());
+ Y_ABORT_UNLESS(mboxIt != currentMailboxes.end());
if (!isIgnored && !CurrentDispatchContext->PrevContext && !restrictedMailboxes &&
mboxIt->second->IsEmpty() &&
mboxIt->second->IsScheduledEmpty() &&
@@ -1267,7 +1267,7 @@ namespace NActors {
if (mboxIt == currentMailboxes.end()) {
mboxIt = currentMailboxes.begin();
}
- Y_VERIFY(endWithMboxIt != currentMailboxes.end());
+ Y_ABORT_UNLESS(endWithMboxIt != currentMailboxes.end());
if (mboxIt == endWithMboxIt) {
break;
}
@@ -1425,7 +1425,7 @@ namespace NActors {
void TTestActorRuntimeBase::Send(TAutoPtr<IEventHandle> ev, ui32 senderNodeIndex, bool viaActorSystem) {
TGuard<TMutex> guard(Mutex);
- Y_VERIFY(senderNodeIndex < NodeCount, "senderNodeIndex# %" PRIu32 " < NodeCount# %" PRIu32,
+ Y_ABORT_UNLESS(senderNodeIndex < NodeCount, "senderNodeIndex# %" PRIu32 " < NodeCount# %" PRIu32,
senderNodeIndex, NodeCount);
SendInternal(ev, senderNodeIndex, viaActorSystem);
}
@@ -1436,7 +1436,7 @@ namespace NActors {
void TTestActorRuntimeBase::Schedule(TAutoPtr<IEventHandle> ev, const TDuration& duration, ui32 nodeIndex) {
TGuard<TMutex> guard(Mutex);
- Y_VERIFY(nodeIndex < NodeCount);
+ Y_ABORT_UNLESS(nodeIndex < NodeCount);
ui32 nodeId = FirstNodeId + nodeIndex;
ui32 mailboxHint = ev->GetRecipientRewrite().Hint();
TInstant deadline = TInstant::MicroSeconds(CurrentTimestamp) + duration;
@@ -1461,7 +1461,7 @@ namespace NActors {
TActorId TTestActorRuntimeBase::GetLocalServiceId(const TActorId& serviceId, ui32 nodeIndex) {
TGuard<TMutex> guard(Mutex);
- Y_VERIFY(nodeIndex < NodeCount);
+ Y_ABORT_UNLESS(nodeIndex < NodeCount);
TNodeDataBase* node = Nodes[FirstNodeId + nodeIndex].Get();
return node->ActorSystem->LookupLocalService(serviceId);
}
@@ -1471,7 +1471,7 @@ namespace NActors {
ui32 dispatchCount = 0;
if (!edgeFilter.empty()) {
for (auto edgeActor : edgeFilter) {
- Y_VERIFY(EdgeActors.contains(edgeActor), "%s is not an edge actor", ToString(edgeActor).data());
+ Y_ABORT_UNLESS(EdgeActors.contains(edgeActor), "%s is not an edge actor", ToString(edgeActor).data());
}
}
const TSet<TActorId>& edgeActors = edgeFilter.empty() ? EdgeActors : edgeFilter;
@@ -1501,15 +1501,15 @@ namespace NActors {
}
}
- Y_VERIFY(dispatchCount < 1000, "Hard limit to prevent endless loop");
+ Y_ABORT_UNLESS(dispatchCount < 1000, "Hard limit to prevent endless loop");
}
}
TActorId TTestActorRuntimeBase::GetInterconnectProxy(ui32 nodeIndexFrom, ui32 nodeIndexTo) {
TGuard<TMutex> guard(Mutex);
- Y_VERIFY(nodeIndexFrom < NodeCount);
- Y_VERIFY(nodeIndexTo < NodeCount);
- Y_VERIFY(nodeIndexFrom != nodeIndexTo);
+ Y_ABORT_UNLESS(nodeIndexFrom < NodeCount);
+ Y_ABORT_UNLESS(nodeIndexTo < NodeCount);
+ Y_ABORT_UNLESS(nodeIndexFrom != nodeIndexTo);
TNodeDataBase* node = Nodes[FirstNodeId + nodeIndexFrom].Get();
return node->ActorSystem->InterconnectProxy(FirstNodeId + nodeIndexTo);
}
@@ -1528,13 +1528,13 @@ namespace NActors {
IActor* TTestActorRuntimeBase::FindActor(const TActorId& actorId, ui32 nodeIndex) const {
TGuard<TMutex> guard(Mutex);
if (nodeIndex == Max<ui32>()) {
- Y_VERIFY(actorId.NodeId());
+ Y_ABORT_UNLESS(actorId.NodeId());
nodeIndex = actorId.NodeId() - FirstNodeId;
}
- Y_VERIFY(nodeIndex < NodeCount);
+ Y_ABORT_UNLESS(nodeIndex < NodeCount);
auto nodeIt = Nodes.find(FirstNodeId + nodeIndex);
- Y_VERIFY(nodeIt != Nodes.end());
+ Y_ABORT_UNLESS(nodeIt != Nodes.end());
TNodeDataBase* node = nodeIt->second.Get();
return FindActor(actorId, node);
}
@@ -1561,7 +1561,7 @@ namespace NActors {
TIntrusivePtr<NMonitoring::TDynamicCounters> TTestActorRuntimeBase::GetDynamicCounters(ui32 nodeIndex) {
TGuard<TMutex> guard(Mutex);
- Y_VERIFY(nodeIndex < NodeCount);
+ Y_ABORT_UNLESS(nodeIndex < NodeCount);
ui32 nodeId = FirstNodeId + nodeIndex;
TNodeDataBase* node = Nodes[nodeId].Get();
return node->DynamicCounters;
@@ -1572,7 +1572,7 @@ namespace NActors {
}
void TTestActorRuntimeBase::SendInternal(TAutoPtr<IEventHandle> ev, ui32 nodeIndex, bool viaActorSystem) {
- Y_VERIFY(nodeIndex < NodeCount);
+ Y_ABORT_UNLESS(nodeIndex < NodeCount);
ui32 nodeId = FirstNodeId + nodeIndex;
TNodeDataBase* node = Nodes[nodeId].Get();
ui32 targetNode = ev->GetRecipientRewrite().NodeId();
@@ -1581,7 +1581,7 @@ namespace NActors {
targetNodeIndex = nodeIndex;
} else {
targetNodeIndex = targetNode - FirstNodeId;
- Y_VERIFY(targetNodeIndex < NodeCount);
+ Y_ABORT_UNLESS(targetNodeIndex < NodeCount);
}
if (viaActorSystem || UseRealThreads || ev->GetRecipientRewrite().IsService() || (targetNodeIndex != nodeIndex)) {
@@ -1589,7 +1589,7 @@ namespace NActors {
return;
}
- Y_VERIFY(!ev->GetRecipientRewrite().IsService() && (targetNodeIndex == nodeIndex));
+ Y_ABORT_UNLESS(!ev->GetRecipientRewrite().IsService() && (targetNodeIndex == nodeIndex));
if (!AllowSendFrom(node, ev)) {
return;
@@ -1744,7 +1744,7 @@ namespace NActors {
}
TActorSystem* TTestActorRuntimeBase::SingleSys() const {
- Y_VERIFY(Nodes.size() == 1, "Works only for single system env");
+ Y_ABORT_UNLESS(Nodes.size() == 1, "Works only for single system env");
return Nodes.begin()->second->ActorSystem.Get();
}
@@ -1758,7 +1758,7 @@ namespace NActors {
TActorSystem* TTestActorRuntimeBase::GetActorSystem(ui32 nodeId) {
auto it = Nodes.find(GetNodeId(nodeId));
- Y_VERIFY(it != Nodes.end());
+ Y_ABORT_UNLESS(it != Nodes.end());
return it->second->ActorSystem.Get();
}
@@ -1833,7 +1833,7 @@ namespace NActors {
, ReplyChecker(createReplyChecker())
{
if (IsSync) {
- Y_VERIFY(!runtime->IsRealThreads());
+ Y_ABORT_UNLESS(!runtime->IsRealThreads());
}
}
@@ -1859,7 +1859,7 @@ namespace NActors {
}
STFUNC(Reply) {
- Y_VERIFY(!HasReply);
+ Y_ABORT_UNLESS(!HasReply);
IEventHandle *requestEv = Context->Queue->Head();
TActorId originalSender = requestEv->Sender;
HasReply = !ReplyChecker->IsWaitingForMoreResponses(ev.Get());
diff --git a/library/cpp/actors/testlib/test_runtime.h b/library/cpp/actors/testlib/test_runtime.h
index 12c99ba8e6..80853e357c 100644
--- a/library/cpp/actors/testlib/test_runtime.h
+++ b/library/cpp/actors/testlib/test_runtime.h
@@ -298,7 +298,7 @@ namespace NActors {
template<typename T>
void AppendToLogSettings(NLog::EComponent minVal, NLog::EComponent maxVal, T func) {
- Y_VERIFY(!IsInitialized);
+ Y_ABORT_UNLESS(!IsInitialized);
for (const auto& pair : Nodes) {
pair.second->LogSettings->Append(minVal, maxVal, func);
@@ -332,7 +332,7 @@ namespace NActors {
}, {}, simTimeout);
if (simTimeout == TDuration::Max())
- Y_VERIFY(handle);
+ Y_ABORT_UNLESS(handle);
if (handle) {
return handle->Get<TEvent>();
@@ -364,7 +364,7 @@ namespace NActors {
}, edgeFilter, simTimeout);
if (simTimeout == TDuration::Max())
- Y_VERIFY(handle);
+ Y_ABORT_UNLESS(handle);
return handle;
}
@@ -419,7 +419,7 @@ namespace NActors {
return true;
}, {}, simTimeout);
if (simTimeout == TDuration::Max())
- Y_VERIFY(handle);
+ Y_ABORT_UNLESS(handle);
if (handle) {
return std::make_tuple(handle->Type == TEvents::EventType
? handle->Get<TEvents>()
diff --git a/library/cpp/actors/util/cpu_load_log.h b/library/cpp/actors/util/cpu_load_log.h
index e4ae612246..225f7148da 100644
--- a/library/cpp/actors/util/cpu_load_log.h
+++ b/library/cpp/actors/util/cpu_load_log.h
@@ -147,7 +147,7 @@ struct TMinusOneCpuEstimator {
ui64 Delay[BitsSize];
ui64 MaxLatencyIncreaseWithOneLessCpu(TCpuLoadLog<DataSize>** logs, i64 logCount, ui64 timeNs, ui64 periodNs) {
- Y_VERIFY(logCount > 0);
+ Y_ABORT_UNLESS(logCount > 0);
ui64 endTimeNs = timeNs;
ui64 lastTimeNs = timeNs;
diff --git a/library/cpp/actors/util/local_process_key.h b/library/cpp/actors/util/local_process_key.h
index 077de8cddc..bff8bef81b 100644
--- a/library/cpp/actors/util/local_process_key.h
+++ b/library/cpp/actors/util/local_process_key.h
@@ -40,7 +40,7 @@ public:
}
TStringBuf GetNameByIndex(size_t index) const {
- Y_VERIFY(index < Names.size());
+ Y_ABORT_UNLESS(index < Names.size());
return Names[index];
}
@@ -64,7 +64,7 @@ public:
const ui32 index = TLocalProcessKeyStateIndexConstructor<T>::BuildCurrentIndex(name, Names.size());
auto x = Map.emplace(name, index);
if (x.second) {
- Y_VERIFY(index < Names.size(), "a lot of actors or tags for memory monitoring");
+ Y_ABORT_UNLESS(index < Names.size(), "a lot of actors or tags for memory monitoring");
Names[index] = name;
}
@@ -131,7 +131,7 @@ public:
static size_t GetIndex(const EnumT key) {
ui32 index = static_cast<ui32>(key);
- Y_VERIFY(index < Enum2Index.size());
+ Y_ABORT_UNLESS(index < Enum2Index.size());
return Enum2Index[index];
}
diff --git a/library/cpp/actors/util/rc_buf.h b/library/cpp/actors/util/rc_buf.h
index 7b6c68d269..638a410f6b 100644
--- a/library/cpp/actors/util/rc_buf.h
+++ b/library/cpp/actors/util/rc_buf.h
@@ -718,7 +718,7 @@ class TRcBuf {
explicit TRcBuf(TInternalBackend s, const char *data, size_t size)
: Backend(std::move(s))
{
- Y_VERIFY(Backend.GetData().data() == nullptr ||
+ Y_ABORT_UNLESS(Backend.GetData().data() == nullptr ||
(Backend.GetCookies() && Backend.GetCookies()->Begin == data && Backend.GetCookies()->End == data + size));
Begin = data;
End = data + size;
@@ -735,9 +735,9 @@ class TRcBuf {
TRcBuf(TOwnedPiece, const char *data, size_t size, const TRcBuf& from)
: TRcBuf(from.Backend, {data, size})
{
- Y_VERIFY(data >= from.GetData());
- Y_VERIFY(data < from.GetData() + from.GetSize());
- Y_VERIFY(data + size <= from.GetData() + from.GetSize());
+ Y_ABORT_UNLESS(data >= from.GetData());
+ Y_ABORT_UNLESS(data < from.GetData() + from.GetSize());
+ Y_ABORT_UNLESS(data + size <= from.GetData() + from.GetSize());
Backend.UpdateCookiesUnsafe(Begin, End);
}
@@ -794,9 +794,9 @@ public:
TRcBuf(TPiece, const char *data, size_t size, const TRcBuf& from)
: TRcBuf(from.Backend, {data, size})
{
- Y_VERIFY(data >= from.GetData());
- Y_VERIFY(data < from.GetData() + from.GetSize());
- Y_VERIFY(data + size <= from.GetData() + from.GetSize());
+ Y_ABORT_UNLESS(data >= from.GetData());
+ Y_ABORT_UNLESS(data < from.GetData() + from.GetSize());
+ Y_ABORT_UNLESS(data + size <= from.GetData() + from.GetSize());
}
TRcBuf(TPiece, const char *begin, const char *end, const TRcBuf& from)
@@ -1068,12 +1068,12 @@ public:
}
void TrimBack(size_t size) {
- Y_VERIFY(size <= GetSize());
+ Y_ABORT_UNLESS(size <= GetSize());
End = End - (GetSize() - size);
}
void TrimFront(size_t size) {
- Y_VERIFY(size <= GetSize());
+ Y_ABORT_UNLESS(size <= GetSize());
Begin = Begin + (GetSize() - size);
}
diff --git a/library/cpp/actors/util/recentwnd.h b/library/cpp/actors/util/recentwnd.h
index ba1ede6f29..0f5ee17fa0 100644
--- a/library/cpp/actors/util/recentwnd.h
+++ b/library/cpp/actors/util/recentwnd.h
@@ -53,7 +53,7 @@ public:
}
void ResetWnd(ui32 wndSize) {
- Y_VERIFY(wndSize != 0);
+ Y_ABORT_UNLESS(wndSize != 0);
MaxWndSize_ = wndSize;
if (Window_.size() > MaxWndSize_) {
Window_.erase(Window_.begin(),
diff --git a/library/cpp/actors/util/rope.h b/library/cpp/actors/util/rope.h
index 201ce06f0d..d1385f6b75 100644
--- a/library/cpp/actors/util/rope.h
+++ b/library/cpp/actors/util/rope.h
@@ -30,7 +30,7 @@ class TRopeAlignedBuffer : public IContiguousChunk {
, Capacity(size)
, Offset((Alignment - reinterpret_cast<uintptr_t>(Data)) & (Alignment - 1))
{
- Y_VERIFY(Offset <= Alignment - MallocAlignment);
+ Y_ABORT_UNLESS(Offset <= Alignment - MallocAlignment);
}
public:
@@ -153,8 +153,8 @@ private:
void CheckValid() const {
#ifndef NDEBUG
- Y_VERIFY(ValidityToken == Rope->GetValidityToken());
- Y_VERIFY(Iter == Rope->Chain.end() || Iter->Backend);
+ Y_ABORT_UNLESS(ValidityToken == Rope->GetValidityToken());
+ Y_ABORT_UNLESS(Iter == Rope->Chain.end() || Iter->Backend);
#endif
}
@@ -505,7 +505,7 @@ public:
}
void ExtractFront(size_t num, TRope *dest) {
- Y_VERIFY(Size >= num);
+ Y_ABORT_UNLESS(Size >= num);
if (num == Size && !*dest) {
*dest = std::move(*this);
return;
diff --git a/library/cpp/actors/util/shared_data_ut.cpp b/library/cpp/actors/util/shared_data_ut.cpp
index af38fc8f0c..2f7dc2ccc8 100644
--- a/library/cpp/actors/util/shared_data_ut.cpp
+++ b/library/cpp/actors/util/shared_data_ut.cpp
@@ -83,12 +83,12 @@ namespace NActors {
header->RefCount = 1;
header->Owner = this;
char* data = raw + sizeof(THeader);
- Y_VERIFY(Allocated_.insert(data).second);
+ Y_ABORT_UNLESS(Allocated_.insert(data).second);
return TSharedData::AttachUnsafe(data, size);
}
void Deallocate(char* data) noexcept {
- Y_VERIFY(Allocated_.erase(data) > 0);
+ Y_ABORT_UNLESS(Allocated_.erase(data) > 0);
char* raw = data - sizeof(THeader);
y_deallocate(raw);
Deallocated_.push_back(data);
diff --git a/library/cpp/actors/util/thread_load_log.h b/library/cpp/actors/util/thread_load_log.h
index b4b34d47bb..132e99a52d 100644
--- a/library/cpp/actors/util/thread_load_log.h
+++ b/library/cpp/actors/util/thread_load_log.h
@@ -279,7 +279,7 @@ private:
public:
template <typename T>
ui64 MaxLatencyIncreaseWithOneLessCpu(T **threadLoads, ui32 threadCount, ui64 timeNs, ui64 periodNs) {
- Y_VERIFY(threadCount > 0);
+ Y_ABORT_UNLESS(threadCount > 0);
struct TTimeSlotData {
typename T::TimeSlotType Load;
@@ -329,7 +329,7 @@ public:
auto timeSlotShiftCount = slotIndex - firstThreadLoadData.Index;
maxTimeSlotShiftCount = std::max(maxTimeSlotShiftCount, timeSlotShiftCount);
auto res = firstThreadLoadDataQueue.pop();
- Y_VERIFY(res);
+ Y_ABORT_UNLESS(res);
}
}
@@ -347,7 +347,7 @@ public:
// The current load of the first thread can be later
// processed by the following time slots of other threads
auto res = firstThreadLoadDataQueue.push({firstThreadTimeSlotValue, slotIndex});
- Y_VERIFY(res);
+ Y_ABORT_UNLESS(res);
}
}
}
diff --git a/library/cpp/actors/util/timerfd.h b/library/cpp/actors/util/timerfd.h
index 3189e2a672..78ae27e2ee 100644
--- a/library/cpp/actors/util/timerfd.h
+++ b/library/cpp/actors/util/timerfd.h
@@ -15,7 +15,7 @@ struct TTimerFd: public TNonCopyable {
TTimerFd() {
Fd = timerfd_create(CLOCK_MONOTONIC, 0);
- Y_VERIFY(Fd != -1, "timerfd_create(CLOCK_MONOTONIC, 0) -> -1; errno:%d: %s", int(errno), strerror(errno));
+ Y_ABORT_UNLESS(Fd != -1, "timerfd_create(CLOCK_MONOTONIC, 0) -> -1; errno:%d: %s", int(errno), strerror(errno));
}
~TTimerFd() {
@@ -34,7 +34,7 @@ struct TTimerFd: public TNonCopyable {
void Wait() {
ui64 expirations;
ssize_t s = read(Fd, &expirations, sizeof(ui64));
- Y_UNUSED(s); // Y_VERIFY(s == sizeof(ui64));
+ Y_UNUSED(s); // Y_ABORT_UNLESS(s == sizeof(ui64));
}
void Wake() {
@@ -48,7 +48,7 @@ private:
spec.it_interval.tv_sec = 0;
spec.it_interval.tv_nsec = 0;
int ret = timerfd_settime(Fd, 0, &spec, nullptr);
- Y_VERIFY(ret != -1, "timerfd_settime(%d, 0, %" PRIu64 "ns, 0) -> %d; errno:%d: %s", Fd, ns, ret, int(errno), strerror(errno));
+ Y_ABORT_UNLESS(ret != -1, "timerfd_settime(%d, 0, %" PRIu64 "ns, 0) -> %d; errno:%d: %s", Fd, ns, ret, int(errno), strerror(errno));
}
};
diff --git a/library/cpp/actors/util/unordered_cache.h b/library/cpp/actors/util/unordered_cache.h
index 76f036c0cf..0ab1a3d220 100644
--- a/library/cpp/actors/util/unordered_cache.h
+++ b/library/cpp/actors/util/unordered_cache.h
@@ -127,7 +127,7 @@ public:
}
~TUnorderedCache() {
- Y_VERIFY(!Pop(0));
+ Y_ABORT_UNLESS(!Pop(0));
for (ui64 i = 0; i < Concurrency; ++i) {
if (ReadSlots[i].ReadFrom) {
diff --git a/library/cpp/actors/wilson/wilson_profile_span.cpp b/library/cpp/actors/wilson/wilson_profile_span.cpp
index 3939be3c94..e908ed2b8a 100644
--- a/library/cpp/actors/wilson/wilson_profile_span.cpp
+++ b/library/cpp/actors/wilson/wilson_profile_span.cpp
@@ -148,9 +148,9 @@ TProfileSpan::TGuard::~TGuard() {
if (!Owner.Enabled) {
return;
}
- Y_VERIFY(CurrentNodeDuration->IsDouble());
+ Y_ABORT_UNLESS(CurrentNodeDuration->IsDouble());
CurrentNodeDuration->SetValue((Now() - Start).MicroSeconds() * 0.000001 + CurrentNodeDuration->GetDoubleRobust());
- Y_VERIFY(Owner.CurrentJsonPath.size());
+ Y_ABORT_UNLESS(Owner.CurrentJsonPath.size());
Owner.CurrentJsonPath.pop_back();
if (Owner.CurrentJsonPath.empty()) {
Owner.LastNoGuards = Now();
diff --git a/library/cpp/actors/wilson/wilson_trace.h b/library/cpp/actors/wilson/wilson_trace.h
index 668d32e306..1675250566 100644
--- a/library/cpp/actors/wilson/wilson_trace.h
+++ b/library/cpp/actors/wilson/wilson_trace.h
@@ -34,8 +34,8 @@ namespace NWilson {
if (timeToLive == Max<ui32>()) {
timeToLive = 4095;
}
- Y_VERIFY(verbosity <= 15);
- Y_VERIFY(timeToLive <= 4095);
+ Y_ABORT_UNLESS(verbosity <= 15);
+ Y_ABORT_UNLESS(timeToLive <= 4095);
SpanId = spanId;
Verbosity = verbosity;
TimeToLive = timeToLive;
diff --git a/library/cpp/actors/wilson/wilson_uploader.cpp b/library/cpp/actors/wilson/wilson_uploader.cpp
index 2390d5a376..13fca19015 100644
--- a/library/cpp/actors/wilson/wilson_uploader.cpp
+++ b/library/cpp/actors/wilson/wilson_uploader.cpp
@@ -172,7 +172,7 @@ namespace NWilson {
}
void HandleWakeup() {
- Y_VERIFY(WakeupScheduled);
+ Y_ABORT_UNLESS(WakeupScheduled);
WakeupScheduled = false;
CheckIfDone();
TryToSend();
diff --git a/library/cpp/balloc/lib/balloc.h b/library/cpp/balloc/lib/balloc.h
index 019c9cb7de..f9b17cf035 100644
--- a/library/cpp/balloc/lib/balloc.h
+++ b/library/cpp/balloc/lib/balloc.h
@@ -131,7 +131,7 @@ namespace NBalloc {
for (res = Head; res; res = Head) {
TNode* keepNext = res->Next;
if (DoCas(&Head, keepNext, res)) {
- //Y_VERIFY(keepNext == res->Next);
+ //Y_ABORT_UNLESS(keepNext == res->Next);
break;
}
}
diff --git a/library/cpp/binsaver/buffered_io.cpp b/library/cpp/binsaver/buffered_io.cpp
index dd88b04bc5..eb291840b0 100644
--- a/library/cpp/binsaver/buffered_io.cpp
+++ b/library/cpp/binsaver/buffered_io.cpp
@@ -1,7 +1,7 @@
#include "buffered_io.h"
i64 IBinaryStream::LongWrite(const void* userBuffer, i64 size) {
- Y_VERIFY(size >= 0, "IBinaryStream::Write() called with a negative buffer size.");
+ Y_ABORT_UNLESS(size >= 0, "IBinaryStream::Write() called with a negative buffer size.");
i64 leftToWrite = size;
while (leftToWrite != 0) {
@@ -19,7 +19,7 @@ i64 IBinaryStream::LongWrite(const void* userBuffer, i64 size) {
}
i64 IBinaryStream::LongRead(void* userBuffer, i64 size) {
- Y_VERIFY(size >= 0, "IBinaryStream::Read() called with a negative buffer size.");
+ Y_ABORT_UNLESS(size >= 0, "IBinaryStream::Read() called with a negative buffer size.");
i64 leftToRead = size;
while (leftToRead != 0) {
diff --git a/library/cpp/bit_io/bitoutput.h b/library/cpp/bit_io/bitoutput.h
index 2b886c1f02..cc9291233e 100644
--- a/library/cpp/bit_io/bitoutput.h
+++ b/library/cpp/bit_io/bitoutput.h
@@ -148,7 +148,7 @@ namespace NBitIO {
public:
void WriteData(const char* begin, const char* end) {
size_t sz = end - begin;
- Y_VERIFY(sz <= Left, " ");
+ Y_ABORT_UNLESS(sz <= Left, " ");
memcpy(Data, begin, sz);
Data += sz;
Left -= sz;
diff --git a/library/cpp/blockcodecs/core/stream.cpp b/library/cpp/blockcodecs/core/stream.cpp
index 4f7db3c32b..39e8786db6 100644
--- a/library/cpp/blockcodecs/core/stream.cpp
+++ b/library/cpp/blockcodecs/core/stream.cpp
@@ -100,7 +100,7 @@ void TCodedOutput::DoWrite(const void* buf, size_t len) {
in += avail;
len -= avail;
- Y_VERIFY(FlushImpl(), "flush on writing failed");
+ Y_ABORT_UNLESS(FlushImpl(), "flush on writing failed");
}
}
diff --git a/library/cpp/blockcodecs/fuzz/main.cpp b/library/cpp/blockcodecs/fuzz/main.cpp
index 763c6c5a10..a89fee7ab7 100644
--- a/library/cpp/blockcodecs/fuzz/main.cpp
+++ b/library/cpp/blockcodecs/fuzz/main.cpp
@@ -45,8 +45,8 @@ static void DoDecodeEncode(const TPackUnpackCase& case_) {
TransferData(&decoded, &co);
co.Flush();
- Y_VERIFY((case_.GetData().size() > 0) == (cno.Counter() > 0));
- Y_VERIFY((case_.GetData().size() > 0) == (decoded.Str().size() > 0));
+ Y_ABORT_UNLESS((case_.GetData().size() > 0) == (cno.Counter() > 0));
+ Y_ABORT_UNLESS((case_.GetData().size() > 0) == (decoded.Str().size() > 0));
}
static void DoEncodeDecode(const TPackUnpackCase& case_) {
@@ -62,8 +62,8 @@ static void DoEncodeDecode(const TPackUnpackCase& case_) {
TDecodedInput di(&encoded, codec);
TransferData(&di, &decoded);
- Y_VERIFY((case_.GetData().size() > 0) == (encoded.Str().size() > 0));
- Y_VERIFY(case_.GetData() == decoded.Str());
+ Y_ABORT_UNLESS((case_.GetData().size() > 0) == (encoded.Str().size() > 0));
+ Y_ABORT_UNLESS(case_.GetData() == decoded.Str());
}
DEFINE_BINARY_PROTO_FUZZER(const TPackUnpackCase& case_) {
diff --git a/library/cpp/codecs/codecs.cpp b/library/cpp/codecs/codecs.cpp
index b17a3156d2..ee76a38c69 100644
--- a/library/cpp/codecs/codecs.cpp
+++ b/library/cpp/codecs/codecs.cpp
@@ -12,7 +12,7 @@ namespace NCodecs {
Y_ENSURE_EX(p->AlreadyTrained(), TCodecException() << "untrained codec " << p->GetName());
const TString& n = p->GetName();
- Y_VERIFY(n.size() <= Max<ui16>());
+ Y_ABORT_UNLESS(n.size() <= Max<ui16>());
::Save(out, (ui16)n.size());
out->Write(n.data(), n.size());
p->Save(out);
diff --git a/library/cpp/codecs/codecs_registry.cpp b/library/cpp/codecs/codecs_registry.cpp
index 17d07062ab..9bd16d3978 100644
--- a/library/cpp/codecs/codecs_registry.cpp
+++ b/library/cpp/codecs/codecs_registry.cpp
@@ -24,7 +24,7 @@ namespace NCodecs {
void TCodecRegistry::RegisterFactory(TFactoryPtr fac) {
TVector<TString> names = fac->ListNames();
for (const auto& name : names) {
- Y_VERIFY(!Registry.contains(name), "already has %s", name.data());
+ Y_ABORT_UNLESS(!Registry.contains(name), "already has %s", name.data());
Registry[name] = fac;
}
}
diff --git a/library/cpp/containers/comptrie/write_trie_backwards.cpp b/library/cpp/containers/comptrie/write_trie_backwards.cpp
index fd8c28b0ed..40d63e7d65 100644
--- a/library/cpp/containers/comptrie/write_trie_backwards.cpp
+++ b/library/cpp/containers/comptrie/write_trie_backwards.cpp
@@ -101,7 +101,7 @@ namespace NCompactTrie {
memmove(data, pos, end - pos);
break;
default:
- Y_VERIFY(false);
+ Y_ABORT_UNLESS(false);
}
return end - pos;
diff --git a/library/cpp/containers/concurrent_hash/concurrent_hash.h b/library/cpp/containers/concurrent_hash/concurrent_hash.h
index 042bf0c040..2c0f68cd8c 100644
--- a/library/cpp/containers/concurrent_hash/concurrent_hash.h
+++ b/library/cpp/containers/concurrent_hash/concurrent_hash.h
@@ -39,19 +39,19 @@ public:
const V& GetUnsafe(const K& key) const {
typename TActualMap::const_iterator it = Map.find(key);
- Y_VERIFY(it != Map.end(), "not found by key");
+ Y_ABORT_UNLESS(it != Map.end(), "not found by key");
return it->second;
}
V& GetUnsafe(const K& key) {
typename TActualMap::iterator it = Map.find(key);
- Y_VERIFY(it != Map.end(), "not found by key");
+ Y_ABORT_UNLESS(it != Map.end(), "not found by key");
return it->second;
}
V RemoveUnsafe(const K& key) {
typename TActualMap::iterator it = Map.find(key);
- Y_VERIFY(it != Map.end(), "removing non-existent key");
+ Y_ABORT_UNLESS(it != Map.end(), "removing non-existent key");
V r = std::move(it->second);
Map.erase(it);
return r;
diff --git a/library/cpp/containers/paged_vector/paged_vector.h b/library/cpp/containers/paged_vector/paged_vector.h
index 0a6dd8f83c..0fe1660763 100644
--- a/library/cpp/containers/paged_vector/paged_vector.h
+++ b/library/cpp/containers/paged_vector/paged_vector.h
@@ -395,7 +395,7 @@ namespace NPagedVector {
if (pagepart)
CurrentPage().resize(pagepart);
- Y_VERIFY(sz == size(), "%" PRIu64 " %" PRIu64, (ui64)sz, (ui64)size());
+ Y_ABORT_UNLESS(sz == size(), "%" PRIu64 " %" PRIu64, (ui64)sz, (ui64)size());
}
reference at(size_t idx) {
diff --git a/library/cpp/containers/stack_vector/stack_vec.h b/library/cpp/containers/stack_vector/stack_vec.h
index 91ae8a4778..25a8658fd3 100644
--- a/library/cpp/containers/stack_vector/stack_vec.h
+++ b/library/cpp/containers/stack_vector/stack_vec.h
@@ -74,7 +74,7 @@ namespace NPrivate {
void deallocate(T* p, size_type n) {
if (p >= reinterpret_cast<T*>(&StackBasedStorage[0]) &&
p < reinterpret_cast<T*>(&StackBasedStorage[CountOnStack])) {
- Y_VERIFY(IsStorageUsed);
+ Y_ABORT_UNLESS(IsStorageUsed);
IsStorageUsed = false;
} else {
FallbackAllocator().deallocate(p, n);
diff --git a/library/cpp/coroutine/engine/coroutine_ut.cpp b/library/cpp/coroutine/engine/coroutine_ut.cpp
index b4d0b62543..de56d0ed2b 100644
--- a/library/cpp/coroutine/engine/coroutine_ut.cpp
+++ b/library/cpp/coroutine/engine/coroutine_ut.cpp
@@ -91,7 +91,7 @@ void TCoroTest::TestException() {
auto f1 = [&f2run](TCont* c) {
struct TCtx {
~TCtx() {
- Y_VERIFY(!*F2);
+ Y_ABORT_UNLESS(!*F2);
C->Yield();
}
diff --git a/library/cpp/coroutine/engine/impl.cpp b/library/cpp/coroutine/engine/impl.cpp
index ac116af0c1..5f2e4a53f6 100644
--- a/library/cpp/coroutine/engine/impl.cpp
+++ b/library/cpp/coroutine/engine/impl.cpp
@@ -146,7 +146,7 @@ TContExecutor::TContExecutor(
}
TContExecutor::~TContExecutor() {
- Y_VERIFY(Allocated_ == 0, "leaked %u coroutines", (ui32)Allocated_);
+ Y_ABORT_UNLESS(Allocated_ == 0, "leaked %u coroutines", (ui32)Allocated_);
}
void TContExecutor::Execute() noexcept {
diff --git a/library/cpp/coroutine/engine/stack/benchmark/alloc_bm.cpp b/library/cpp/coroutine/engine/stack/benchmark/alloc_bm.cpp
index 0a71b02e62..6573dd0dc9 100644
--- a/library/cpp/coroutine/engine/stack/benchmark/alloc_bm.cpp
+++ b/library/cpp/coroutine/engine/stack/benchmark/alloc_bm.cpp
@@ -17,14 +17,14 @@ namespace NCoro::NStack::NBenchmark {
constexpr size_t ManyStacks = 4096;
void BasicOperations(TStackHolder& stack) {
- Y_VERIFY(!stack.Get().empty());
+ Y_ABORT_UNLESS(!stack.Get().empty());
stack.LowerCanaryOk();
stack.UpperCanaryOk();
}
void WriteStack(TStackHolder& stack) {
auto memory = stack.Get();
- Y_VERIFY(!memory.empty());
+ Y_ABORT_UNLESS(!memory.empty());
stack.LowerCanaryOk();
stack.UpperCanaryOk();
for (size_t i = PageSize / 2; i < memory.size(); i += PageSize * 2) {
diff --git a/library/cpp/coroutine/engine/stack/stack_allocator.inl b/library/cpp/coroutine/engine/stack/stack_allocator.inl
index 2f8856da46..5d2de52f0d 100644
--- a/library/cpp/coroutine/engine/stack/stack_allocator.inl
+++ b/library/cpp/coroutine/engine/stack/stack_allocator.inl
@@ -52,7 +52,7 @@ namespace NCoro::NStack {
, Guard_(GetGuard<TGuard>())
{
#ifdef _linux_
- Y_VERIFY(sysconf(_SC_PAGESIZE) == PageSize);
+ Y_ABORT_UNLESS(sysconf(_SC_PAGESIZE) == PageSize);
#endif
}
@@ -64,7 +64,7 @@ namespace NCoro::NStack {
if (pool == Pools_.end()) {
Y_ASSERT(Pools_.size() < 1000); // too many different sizes for coroutine stacks
auto [newPool, success] = Pools_.emplace(alignedSize, TPool<TGuard>{alignedSize, PoolSettings_, Guard_});
- Y_VERIFY(success, "Failed to add new coroutine pool");
+ Y_ABORT_UNLESS(success, "Failed to add new coroutine pool");
pool = newPool;
}
return pool->second.AllocStack(name);
@@ -73,7 +73,7 @@ namespace NCoro::NStack {
template<typename TGuard>
void TPoolAllocator<TGuard>::DoFreeStack(NDetails::TStack& stack) noexcept {
auto pool = Pools_.find(stack.GetSize());
- Y_VERIFY(pool != Pools_.end(), "Attempt to free stack from another allocator");
+ Y_ABORT_UNLESS(pool != Pools_.end(), "Attempt to free stack from another allocator");
pool->second.FreeStack(stack);
}
@@ -117,7 +117,7 @@ namespace NCoro::NStack {
char* rawPtr = nullptr;
char* alignedPtr = nullptr; // with extra space for previous guard in this type of allocator
- Y_VERIFY(GetAlignedMemory((alignedSize + Guard_.GetPageAlignedSize()) / PageSize, rawPtr, alignedPtr)); // + memory for previous guard
+ Y_ABORT_UNLESS(GetAlignedMemory((alignedSize + Guard_.GetPageAlignedSize()) / PageSize, rawPtr, alignedPtr)); // + memory for previous guard
char* alignedStackMemory = alignedPtr + Guard_.GetPageAlignedSize(); // after previous guard
// Default allocator sets both guards, because it doesn't have memory chunk with previous stack and guard on it
diff --git a/library/cpp/coroutine/engine/stack/stack_guards.h b/library/cpp/coroutine/engine/stack/stack_guards.h
index f22000b152..b7f93f2160 100644
--- a/library/cpp/coroutine/engine/stack/stack_guards.h
+++ b/library/cpp/coroutine/engine/stack/stack_guards.h
@@ -42,7 +42,7 @@ namespace NCoro::NStack {
Y_ASSERT(size >= Canary.size()); // stack should have enough space to place guard
if (checkPrevious) {
- Y_VERIFY(CheckOverflow(stack), "Previous stack was corrupted");
+ Y_ABORT_UNLESS(CheckOverflow(stack), "Previous stack was corrupted");
}
auto guardPos = (char*) stack + size - Canary.size();
memcpy(guardPos, Canary.data(), Canary.size());
diff --git a/library/cpp/coroutine/engine/stack/stack_pool.inl b/library/cpp/coroutine/engine/stack/stack_pool.inl
index 503e75664e..0aa1f3a4d0 100644
--- a/library/cpp/coroutine/engine/stack/stack_pool.inl
+++ b/library/cpp/coroutine/engine/stack/stack_pool.inl
@@ -91,7 +91,7 @@ namespace NCoro::NStack {
TMemory memory;
const auto res = GetAlignedMemory(totalSizeInPages, memory.Raw, memory.Aligned);
- Y_VERIFY(res, "Failed to allocate memory for coro stack pool");
+ Y_ABORT_UNLESS(res, "Failed to allocate memory for coro stack pool");
NextToAlloc_ = memory.Aligned + Guard_.GetPageAlignedSize(); // skip first guard page
Guard_.Protect(memory.Aligned, Guard_.GetPageAlignedSize(), false); // protect first guard page
diff --git a/library/cpp/coroutine/engine/stack/stack_storage.h b/library/cpp/coroutine/engine/stack/stack_storage.h
index 8327ce6fe0..1a55d4a8c1 100644
--- a/library/cpp/coroutine/engine/stack/stack_storage.h
+++ b/library/cpp/coroutine/engine/stack/stack_storage.h
@@ -40,7 +40,7 @@ namespace NCoro::NStack {
template<typename TGuard>
NDetails::TStack TStorage::GetStack(const TGuard& guard, const char* name) {
- Y_VERIFY(!IsEmpty()); // check before call
+ Y_ABORT_UNLESS(!IsEmpty()); // check before call
void* newStack = nullptr;
if (!Full_.empty()) {
@@ -52,8 +52,8 @@ namespace NCoro::NStack {
Released_.pop_back();
}
- Y_VERIFY(guard.CheckOverflow(newStack), "corrupted stack in pool");
- Y_VERIFY(guard.CheckOverride(newStack, StackSize_), "corrupted stack in pool");
+ Y_ABORT_UNLESS(guard.CheckOverflow(newStack), "corrupted stack in pool");
+ Y_ABORT_UNLESS(guard.CheckOverride(newStack, StackSize_), "corrupted stack in pool");
return NDetails::TStack{newStack, newStack, StackSize_, name};
}
diff --git a/library/cpp/coroutine/engine/stack/stack_utils.cpp b/library/cpp/coroutine/engine/stack/stack_utils.cpp
index 6865772e32..d8fae1ae1f 100644
--- a/library/cpp/coroutine/engine/stack/stack_utils.cpp
+++ b/library/cpp/coroutine/engine/stack/stack_utils.cpp
@@ -35,12 +35,12 @@ namespace NCoro::NStack {
#ifdef _linux_
void ReleaseRss(char* alignedPtr, size_t numOfPages) noexcept {
- Y_VERIFY( !((size_t)alignedPtr & PageSizeMask), "Not aligned pointer to release RSS memory");
+ Y_ABORT_UNLESS( !((size_t)alignedPtr & PageSizeMask), "Not aligned pointer to release RSS memory");
if (!numOfPages) {
return;
}
if (auto res = madvise((void*) alignedPtr, numOfPages * PageSize, MADV_DONTNEED); res) {
- Y_VERIFY(errno == EAGAIN || errno == ENOMEM, "Failed to release memory");
+ Y_ABORT_UNLESS(errno == EAGAIN || errno == ENOMEM, "Failed to release memory");
}
}
#else
@@ -50,12 +50,12 @@ namespace NCoro::NStack {
#ifdef _linux_
size_t CountMapped(char* alignedPtr, size_t numOfPages) noexcept {
- Y_VERIFY( !((size_t)alignedPtr & PageSizeMask) );
+ Y_ABORT_UNLESS( !((size_t)alignedPtr & PageSizeMask) );
Y_ASSERT(numOfPages);
size_t result = 0;
unsigned char* mappedPages = (unsigned char*) calloc(numOfPages, numOfPages);
- Y_VERIFY(mappedPages);
+ Y_ABORT_UNLESS(mappedPages);
Y_DEFER {
free(mappedPages);
};
diff --git a/library/cpp/coroutine/engine/trampoline.h b/library/cpp/coroutine/engine/trampoline.h
index 37b61cf015..5e499445f2 100644
--- a/library/cpp/coroutine/engine/trampoline.h
+++ b/library/cpp/coroutine/engine/trampoline.h
@@ -39,8 +39,8 @@ namespace NCoro {
}
void SwitchTo(TExceptionSafeContext* ctx) noexcept {
- Y_VERIFY(Stack_.LowerCanaryOk(), "Stack overflow (%s)", ContName());
- Y_VERIFY(Stack_.UpperCanaryOk(), "Stack override (%s)", ContName());
+ Y_ABORT_UNLESS(Stack_.LowerCanaryOk(), "Stack overflow (%s)", ContName());
+ Y_ABORT_UNLESS(Stack_.UpperCanaryOk(), "Stack override (%s)", ContName());
Ctx_.SwitchTo(ctx);
}
diff --git a/library/cpp/getopt/small/completer.cpp b/library/cpp/getopt/small/completer.cpp
index 3fff684adb..f9a14b8234 100644
--- a/library/cpp/getopt/small/completer.cpp
+++ b/library/cpp/getopt/small/completer.cpp
@@ -256,7 +256,7 @@ namespace NLastGetopt::NComp {
}
void TCustomCompleter::RegisterCustomCompleter(TCustomCompleter* completer) noexcept {
- Y_VERIFY(completer);
+ Y_ABORT_UNLESS(completer);
completer->Next_ = Head;
Head = completer;
}
diff --git a/library/cpp/getopt/small/completer.h b/library/cpp/getopt/small/completer.h
index 4136f13add..20f190c41a 100644
--- a/library/cpp/getopt/small/completer.h
+++ b/library/cpp/getopt/small/completer.h
@@ -174,7 +174,7 @@ namespace NLastGetopt::NComp {
TMultipartCustomCompleter(TStringBuf sep)
: Sep_(sep)
{
- Y_VERIFY(!Sep_.empty());
+ Y_ABORT_UNLESS(!Sep_.empty());
}
public:
diff --git a/library/cpp/getopt/small/completion_generator.cpp b/library/cpp/getopt/small/completion_generator.cpp
index ac41988217..d893afb40a 100644
--- a/library/cpp/getopt/small/completion_generator.cpp
+++ b/library/cpp/getopt/small/completion_generator.cpp
@@ -24,13 +24,13 @@ namespace NLastGetopt {
TCompletionGenerator::TCompletionGenerator(const TModChooser* modChooser)
: Options_(modChooser)
{
- Y_VERIFY(modChooser != nullptr);
+ Y_ABORT_UNLESS(modChooser != nullptr);
}
TCompletionGenerator::TCompletionGenerator(const TOpts* opts)
: Options_(opts)
{
- Y_VERIFY(opts != nullptr);
+ Y_ABORT_UNLESS(opts != nullptr);
}
void TZshCompletionGenerator::Generate(TStringBuf command, IOutputStream& stream) {
diff --git a/library/cpp/grpc/client/grpc_client_low.cpp b/library/cpp/grpc/client/grpc_client_low.cpp
index 5794e1ecb1..d06bbb2a7d 100644
--- a/library/cpp/grpc/client/grpc_client_low.cpp
+++ b/library/cpp/grpc/client/grpc_client_low.cpp
@@ -176,7 +176,7 @@ void TChannelPool::DeleteExpiredStubsHolders() {
void TChannelPool::EraseFromQueueByTime(const TInstant& lastUseTime, const TString& channelId) {
auto [begin, end] = LastUsedQueue_.equal_range(lastUseTime);
auto pos = std::find_if(begin, end, [&](auto a){return a.second == channelId;});
- Y_VERIFY(pos != LastUsedQueue_.end(), "data corruption at TChannelPool");
+ Y_ABORT_UNLESS(pos != LastUsedQueue_.end(), "data corruption at TChannelPool");
LastUsedQueue_.erase(pos);
}
@@ -209,7 +209,7 @@ class TGRpcClientLow::TContextImpl final
public:
~TContextImpl() override {
- Y_VERIFY(CountChildren() == 0,
+ Y_ABORT_UNLESS(CountChildren() == 0,
"Destructor called with non-empty children");
if (Parent) {
@@ -239,7 +239,7 @@ public:
std::unique_lock<std::mutex> guard(Mutex);
auto removed = RemoveChild(child);
- Y_VERIFY(removed, "Unexpected ForgetContext(%p)", child);
+ Y_ABORT_UNLESS(removed, "Unexpected ForgetContext(%p)", child);
}
IQueueClientContextPtr CreateContext() override {
@@ -266,7 +266,7 @@ public:
}
grpc::CompletionQueue* CompletionQueue() override {
- Y_VERIFY(Owner, "Uninitialized context");
+ Y_ABORT_UNLESS(Owner, "Uninitialized context");
return CQ;
}
@@ -326,7 +326,7 @@ public:
}
void SubscribeCancel(TCallback callback) override {
- Y_VERIFY(callback, "SubscribeCancel called with an empty callback");
+ Y_ABORT_UNLESS(callback, "SubscribeCancel called with an empty callback");
{
std::unique_lock<std::mutex> guard(Mutex);
diff --git a/library/cpp/grpc/client/grpc_client_low.h b/library/cpp/grpc/client/grpc_client_low.h
index d9a061035d..e3f786242d 100644
--- a/library/cpp/grpc/client/grpc_client_low.h
+++ b/library/cpp/grpc/client/grpc_client_low.h
@@ -555,7 +555,7 @@ public:
explicit TStreamRequestReadProcessor(TReaderCallback&& callback)
: Callback(std::move(callback))
{
- Y_VERIFY(Callback, "Missing connected callback");
+ Y_ABORT_UNLESS(Callback, "Missing connected callback");
}
void Cancel() override {
@@ -580,7 +580,7 @@ public:
{
std::unique_lock<std::mutex> guard(Mutex);
- Y_VERIFY(!ReadActive, "Multiple Read/Finish calls detected");
+ Y_ABORT_UNLESS(!ReadActive, "Multiple Read/Finish calls detected");
if (!Finished && !HasInitialMetadata) {
ReadActive = true;
ReadCallback = std::move(callback);
@@ -609,7 +609,7 @@ public:
{
std::unique_lock<std::mutex> guard(Mutex);
- Y_VERIFY(!ReadActive, "Multiple Read/Finish calls detected");
+ Y_ABORT_UNLESS(!ReadActive, "Multiple Read/Finish calls detected");
if (!Finished) {
ReadActive = true;
ReadCallback = std::move(callback);
@@ -637,7 +637,7 @@ public:
{
std::unique_lock<std::mutex> guard(Mutex);
- Y_VERIFY(!ReadActive, "Multiple Read/Finish calls detected");
+ Y_ABORT_UNLESS(!ReadActive, "Multiple Read/Finish calls detected");
if (!Finished) {
ReadActive = true;
FinishCallback = std::move(callback);
@@ -658,7 +658,7 @@ public:
}
void AddFinishedCallback(TReadCallback callback) override {
- Y_VERIFY(callback, "Unexpected empty callback");
+ Y_ABORT_UNLESS(callback, "Unexpected empty callback");
TGrpcStatus status;
@@ -709,8 +709,8 @@ private:
{
std::unique_lock<std::mutex> guard(Mutex);
- Y_VERIFY(ReadActive, "Unexpected Read done callback");
- Y_VERIFY(!ReadFinished, "Unexpected ReadFinished flag");
+ Y_ABORT_UNLESS(ReadActive, "Unexpected Read done callback");
+ Y_ABORT_UNLESS(!ReadFinished, "Unexpected ReadFinished flag");
if (!ok || Cancelled) {
ReadFinished = true;
@@ -781,7 +781,7 @@ private:
finishedCallbacks.swap(FinishedCallbacks);
if (Callback) {
- Y_VERIFY(!ReadActive);
+ Y_ABORT_UNLESS(!ReadActive);
startCallback = std::move(Callback);
Callback = nullptr;
} else if (ReadActive) {
@@ -857,7 +857,7 @@ public:
explicit TStreamRequestReadWriteProcessor(TConnectedCallback&& callback)
: ConnectedCallback(std::move(callback))
{
- Y_VERIFY(ConnectedCallback, "Missing connected callback");
+ Y_ABORT_UNLESS(ConnectedCallback, "Missing connected callback");
}
void Cancel() override {
@@ -908,7 +908,7 @@ public:
{
std::unique_lock<std::mutex> guard(Mutex);
- Y_VERIFY(!ReadActive, "Multiple Read/Finish calls detected");
+ Y_ABORT_UNLESS(!ReadActive, "Multiple Read/Finish calls detected");
if (!Finished && !HasInitialMetadata) {
ReadActive = true;
ReadCallback = std::move(callback);
@@ -937,7 +937,7 @@ public:
{
std::unique_lock<std::mutex> guard(Mutex);
- Y_VERIFY(!ReadActive, "Multiple Read/Finish calls detected");
+ Y_ABORT_UNLESS(!ReadActive, "Multiple Read/Finish calls detected");
if (!Finished) {
ReadActive = true;
ReadCallback = std::move(callback);
@@ -965,7 +965,7 @@ public:
{
std::unique_lock<std::mutex> guard(Mutex);
- Y_VERIFY(!ReadActive, "Multiple Read/Finish calls detected");
+ Y_ABORT_UNLESS(!ReadActive, "Multiple Read/Finish calls detected");
if (!Finished) {
ReadActive = true;
FinishCallback = std::move(callback);
@@ -991,7 +991,7 @@ public:
}
void AddFinishedCallback(TReadCallback callback) override {
- Y_VERIFY(callback, "Unexpected empty callback");
+ Y_ABORT_UNLESS(callback, "Unexpected empty callback");
TGrpcStatus status;
@@ -1065,8 +1065,8 @@ private:
{
std::unique_lock<std::mutex> guard(Mutex);
- Y_VERIFY(ReadActive, "Unexpected Read done callback");
- Y_VERIFY(!ReadFinished, "Unexpected ReadFinished flag");
+ Y_ABORT_UNLESS(ReadActive, "Unexpected Read done callback");
+ Y_ABORT_UNLESS(!ReadFinished, "Unexpected ReadFinished flag");
if (!ok || Cancelled || WriteFinished) {
ReadFinished = true;
@@ -1103,8 +1103,8 @@ private:
{
std::unique_lock<std::mutex> guard(Mutex);
- Y_VERIFY(WriteActive, "Unexpected Write done callback");
- Y_VERIFY(!WriteFinished, "Unexpected WriteFinished flag");
+ Y_ABORT_UNLESS(WriteActive, "Unexpected Write done callback");
+ Y_ABORT_UNLESS(!WriteFinished, "Unexpected WriteFinished flag");
if (ok) {
okCallback.swap(WriteCallback);
@@ -1167,7 +1167,7 @@ private:
finishedCallbacks.swap(FinishedCallbacks);
if (ConnectedCallback) {
- Y_VERIFY(!ReadActive);
+ Y_ABORT_UNLESS(!ReadActive);
connectedCallback = std::move(ConnectedCallback);
ConnectedCallback = nullptr;
} else if (ReadActive) {
@@ -1320,7 +1320,7 @@ private:
: Stub_(TGRpcService::NewStub(ci))
, Provider_(provider)
{
- Y_VERIFY(Provider_, "Connection does not have a queue provider");
+ Y_ABORT_UNLESS(Provider_, "Connection does not have a queue provider");
}
TServiceConnection(TStubsHolder& holder,
@@ -1328,7 +1328,7 @@ private:
: Stub_(holder.GetOrCreateStub<TStub>())
, Provider_(provider)
{
- Y_VERIFY(Provider_, "Connection does not have a queue provider");
+ Y_ABORT_UNLESS(Provider_, "Connection does not have a queue provider");
}
std::shared_ptr<TStub> Stub_;
diff --git a/library/cpp/grpc/server/grpc_request.cpp b/library/cpp/grpc/server/grpc_request.cpp
index d18a32776f..7030992173 100644
--- a/library/cpp/grpc/server/grpc_request.cpp
+++ b/library/cpp/grpc/server/grpc_request.cpp
@@ -13,7 +13,7 @@ public:
void Enqueue(std::function<void()>&& fn, bool urgent) override {
with_lock(Mtx_) {
if (!UrgentQueue_.empty() || !NormalQueue_.empty()) {
- Y_VERIFY(!StreamIsReady_);
+ Y_ABORT_UNLESS(!StreamIsReady_);
}
auto& queue = urgent ? UrgentQueue_ : NormalQueue_;
if (StreamIsReady_ && queue.empty()) {
@@ -30,7 +30,7 @@ public:
size_t left = 0;
std::function<void()> fn;
with_lock(Mtx_) {
- Y_VERIFY(!StreamIsReady_);
+ Y_ABORT_UNLESS(!StreamIsReady_);
auto& queue = UrgentQueue_.empty() ? NormalQueue_ : UrgentQueue_;
if (queue.empty()) {
// Both queues are empty
diff --git a/library/cpp/grpc/server/grpc_request.h b/library/cpp/grpc/server/grpc_request.h
index caf47e2677..5c4cf7c2b8 100644
--- a/library/cpp/grpc/server/grpc_request.h
+++ b/library/cpp/grpc/server/grpc_request.h
@@ -76,7 +76,7 @@ public:
{
AuthState_ = Server_->NeedAuth() ? TAuthState(true) : TAuthState(false);
Request_ = google::protobuf::Arena::CreateMessage<TIn>(&Arena_);
- Y_VERIFY(Request_);
+ Y_ABORT_UNLESS(Request_);
GRPC_LOG_DEBUG(Logger_, "[%p] created request Name# %s", this, Name_);
FinishPromise_ = NThreading::NewPromise<EFinishStatus>();
}
@@ -104,7 +104,7 @@ public:
{
AuthState_ = Server_->NeedAuth() ? TAuthState(true) : TAuthState(false);
Request_ = google::protobuf::Arena::CreateMessage<TIn>(&Arena_);
- Y_VERIFY(Request_);
+ Y_ABORT_UNLESS(Request_);
GRPC_LOG_DEBUG(Logger_, "[%p] created streaming request Name# %s", this, Name_);
FinishPromise_ = NThreading::NewPromise<EFinishStatus>();
StreamAdaptor_ = CreateStreamAdaptor();
@@ -157,7 +157,7 @@ public:
}
void DestroyRequest() override {
- Y_VERIFY(!CallInProgress_, "Unexpected DestroyRequest while another grpc call is still in progress");
+ Y_ABORT_UNLESS(!CallInProgress_, "Unexpected DestroyRequest while another grpc call is still in progress");
RequestDestroyed_ = true;
if (RequestRegistered_) {
Server_->DeregisterRequestCtx(this);
@@ -262,16 +262,16 @@ private:
}
void OnBeforeCall() {
- Y_VERIFY(!RequestDestroyed_, "Cannot start grpc calls after request is already destroyed");
- Y_VERIFY(!Finished_, "Cannot start grpc calls after request is finished");
+ Y_ABORT_UNLESS(!RequestDestroyed_, "Cannot start grpc calls after request is already destroyed");
+ Y_ABORT_UNLESS(!Finished_, "Cannot start grpc calls after request is finished");
bool wasInProgress = std::exchange(CallInProgress_, true);
- Y_VERIFY(!wasInProgress, "Another grpc call is already in progress");
+ Y_ABORT_UNLESS(!wasInProgress, "Another grpc call is already in progress");
}
void OnAfterCall() {
- Y_VERIFY(!RequestDestroyed_, "Finished grpc call after request is already destroyed");
+ Y_ABORT_UNLESS(!RequestDestroyed_, "Finished grpc call after request is already destroyed");
bool wasInProgress = std::exchange(CallInProgress_, false);
- Y_VERIFY(wasInProgress, "Finished grpc call that was not in progress");
+ Y_ABORT_UNLESS(wasInProgress, "Finished grpc call that was not in progress");
}
void WriteDataOk(NProtoBuf::Message* resp, ui32 status) {
@@ -290,7 +290,7 @@ private:
StateFunc_ = &TThis::SetFinishDone;
ResponseSize = sz;
ResponseStatus = status;
- Y_VERIFY(this->Context.c_call());
+ Y_ABORT_UNLESS(this->Context.c_call());
OnBeforeCall();
Finished_ = true;
Writer_->Finish(TUniversalResponseRef<TOut>(resp), grpc::Status::OK, GetGRpcTag());
@@ -346,7 +346,7 @@ private:
}
void FinishGrpcStatus(grpc::StatusCode code, const TString& msg, const TString& details, bool urgent) {
- Y_VERIFY(code != grpc::OK);
+ Y_ABORT_UNLESS(code != grpc::OK);
if (code == grpc::StatusCode::UNAUTHENTICATED) {
Counters_->CountNotAuthenticated();
} else if (code == grpc::StatusCode::RESOURCE_EXHAUSTED) {
@@ -395,7 +395,7 @@ private:
ok ? "true" : "false", makeRequestString().data(), this->Context.peer().c_str());
if (this->Context.c_call() == nullptr) {
- Y_VERIFY(!ok);
+ Y_ABORT_UNLESS(!ok);
// One ref by OnFinishTag, grpc will not call this tag if no request received
UnRef();
} else if (!(RequestRegistered_ = Server_->RegisterRequestCtx(this))) {
diff --git a/library/cpp/grpc/server/grpc_server.cpp b/library/cpp/grpc/server/grpc_server.cpp
index 9ba7d29659..a38d4c9da6 100644
--- a/library/cpp/grpc/server/grpc_server.cpp
+++ b/library/cpp/grpc/server/grpc_server.cpp
@@ -44,7 +44,7 @@ TGRpcServer::TGRpcServer(const TServerOptions& opts)
{}
TGRpcServer::~TGRpcServer() {
- Y_VERIFY(Ts.empty());
+ Y_ABORT_UNLESS(Ts.empty());
Services_.clear();
}
@@ -193,7 +193,7 @@ void TGRpcServer::Stop() {
if (Server_) {
i64 sec = Options_.GRpcShutdownDeadline.Seconds();
- Y_VERIFY(Options_.GRpcShutdownDeadline.NanoSecondsOfSecond() <= Max<i32>());
+ Y_ABORT_UNLESS(Options_.GRpcShutdownDeadline.NanoSecondsOfSecond() <= Max<i32>());
i32 nanosecOfSec = Options_.GRpcShutdownDeadline.NanoSecondsOfSecond();
Server_->Shutdown(gpr_timespec{sec, nanosecOfSec, GPR_TIMESPAN});
}
diff --git a/library/cpp/grpc/server/grpc_server.h b/library/cpp/grpc/server/grpc_server.h
index fc1826b922..2d746de99b 100644
--- a/library/cpp/grpc/server/grpc_server.h
+++ b/library/cpp/grpc/server/grpc_server.h
@@ -155,7 +155,7 @@ public:
i64 prev;
do {
prev = AtomicGet(CurInFlightReqs_);
- Y_VERIFY(prev >= 0);
+ Y_ABORT_UNLESS(prev >= 0);
if (Limit_ && prev > Limit_) {
return false;
}
@@ -166,7 +166,7 @@ public:
void Dec() {
i64 newVal = AtomicDecrement(CurInFlightReqs_);
- Y_VERIFY(newVal >= 0);
+ Y_ABORT_UNLESS(newVal >= 0);
}
i64 GetCurrentInFlight() const {
@@ -334,18 +334,18 @@ public:
}
auto r = shard.Requests_.emplace(req);
- Y_VERIFY(r.second, "Ctx already registered");
+ Y_ABORT_UNLESS(r.second, "Ctx already registered");
}
return true;
}
void DeregisterRequestCtx(ICancelableContext* req) {
- Y_VERIFY(req->ShardIndex != size_t(-1), "Ctx does not have an assigned shard index");
+ Y_ABORT_UNLESS(req->ShardIndex != size_t(-1), "Ctx does not have an assigned shard index");
auto& shard = Shards_[req->ShardIndex];
with_lock(shard.Lock_) {
- Y_VERIFY(shard.Requests_.erase(req), "Ctx is not registered");
+ Y_ABORT_UNLESS(shard.Requests_.erase(req), "Ctx is not registered");
}
}
diff --git a/library/cpp/histogram/adaptive/adaptive_histogram.cpp b/library/cpp/histogram/adaptive/adaptive_histogram.cpp
index 3e91ed1131..4bc4f63cda 100644
--- a/library/cpp/histogram/adaptive/adaptive_histogram.cpp
+++ b/library/cpp/histogram/adaptive/adaptive_histogram.cpp
@@ -82,7 +82,7 @@ namespace NKiwiAggr {
histo.GetType() == HT_ADAPTIVE_WARD_HISTOGRAM ||
histo.GetType() == HT_ADAPTIVE_HISTOGRAM)
{
- Y_VERIFY(histo.FreqSize() == histo.PositionSize(), "Corrupted histo");
+ Y_ABORT_UNLESS(histo.FreqSize() == histo.PositionSize(), "Corrupted histo");
for (size_t j = 0; j < histo.FreqSize(); ++j) {
double value = histo.GetPosition(j);
double weight = histo.GetFreq(j);
@@ -182,7 +182,7 @@ namespace NKiwiAggr {
}
void TAdaptiveHistogram::FromProto(const THistogram& histo) {
- Y_VERIFY(histo.HasType(), "Attempt to parse TAdaptiveHistogram from THistogram protobuf with no Type field set");
+ Y_ABORT_UNLESS(histo.HasType(), "Attempt to parse TAdaptiveHistogram from THistogram protobuf with no Type field set");
;
switch (histo.GetType()) { // check that histogram type could be deduced
case HT_ADAPTIVE_DISTANCE_HISTOGRAM:
@@ -352,7 +352,7 @@ namespace NKiwiAggr {
}
double TAdaptiveHistogram::CalcUpperBound(double sum) {
- Y_VERIFY(sum >= 0, "Sum must be >= 0");
+ Y_ABORT_UNLESS(sum >= 0, "Sum must be >= 0");
if (sum == 0.0) {
return MinValue;
}
@@ -393,7 +393,7 @@ namespace NKiwiAggr {
}
double TAdaptiveHistogram::CalcLowerBound(double sum) {
- Y_VERIFY(sum >= 0, "Sum must be >= 0");
+ Y_ABORT_UNLESS(sum >= 0, "Sum must be >= 0");
if (sum == 0.0) {
return MaxValue;
}
@@ -511,13 +511,13 @@ namespace NKiwiAggr {
++rightBin;
TWeightedValue newBin(value, weight + currentBin->second);
if (rightBin != Bins.end()) {
- Y_VERIFY(BinsByQuality.erase(CalcQuality(*currentBin, *rightBin)) == 1, "Erase failed");
+ Y_ABORT_UNLESS(BinsByQuality.erase(CalcQuality(*currentBin, *rightBin)) == 1, "Erase failed");
BinsByQuality.insert(CalcQuality(newBin, *rightBin));
}
if (currentBin != Bins.begin()) {
TPairSet::iterator leftBin = currentBin;
--leftBin;
- Y_VERIFY(BinsByQuality.erase(CalcQuality(*leftBin, *currentBin)) == 1, "Erase failed");
+ Y_ABORT_UNLESS(BinsByQuality.erase(CalcQuality(*leftBin, *currentBin)) == 1, "Erase failed");
BinsByQuality.insert(CalcQuality(*leftBin, newBin));
}
Bins.erase(currentBin);
@@ -532,7 +532,7 @@ namespace NKiwiAggr {
if (rightBin == Bins.end()) {
BinsByQuality.insert(CalcQuality(*leftBin, weightedValue));
} else {
- Y_VERIFY(BinsByQuality.erase(CalcQuality(*leftBin, *rightBin)) == 1, "Erase failed");
+ Y_ABORT_UNLESS(BinsByQuality.erase(CalcQuality(*leftBin, *rightBin)) == 1, "Erase failed");
BinsByQuality.insert(CalcQuality(*leftBin, weightedValue));
BinsByQuality.insert(CalcQuality(weightedValue, *rightBin));
}
@@ -545,20 +545,20 @@ namespace NKiwiAggr {
void TAdaptiveHistogram::Erase(double value) {
TPairSet::iterator currentBin = Bins.lower_bound(TWeightedValue(value, -1.0));
- Y_VERIFY(currentBin != Bins.end() && currentBin->first == value, "Can't find bin that should be erased");
+ Y_ABORT_UNLESS(currentBin != Bins.end() && currentBin->first == value, "Can't find bin that should be erased");
TPairSet::iterator rightBin = currentBin;
++rightBin;
if (currentBin == Bins.begin()) {
- Y_VERIFY(rightBin != Bins.end(), "No right bin for the first bin");
- Y_VERIFY(BinsByQuality.erase(CalcQuality(*currentBin, *rightBin)) != 0, "Erase failed");
+ Y_ABORT_UNLESS(rightBin != Bins.end(), "No right bin for the first bin");
+ Y_ABORT_UNLESS(BinsByQuality.erase(CalcQuality(*currentBin, *rightBin)) != 0, "Erase failed");
} else {
TPairSet::iterator leftBin = currentBin;
--leftBin;
if (rightBin == Bins.end()) {
- Y_VERIFY(BinsByQuality.erase(CalcQuality(*leftBin, *currentBin)) != 0, "Erase failed");
+ Y_ABORT_UNLESS(BinsByQuality.erase(CalcQuality(*leftBin, *currentBin)) != 0, "Erase failed");
} else {
- Y_VERIFY(BinsByQuality.erase(CalcQuality(*leftBin, *currentBin)) != 0, "Erase failed");
- Y_VERIFY(BinsByQuality.erase(CalcQuality(*currentBin, *rightBin)) != 0, "Erase failed");
+ Y_ABORT_UNLESS(BinsByQuality.erase(CalcQuality(*leftBin, *currentBin)) != 0, "Erase failed");
+ Y_ABORT_UNLESS(BinsByQuality.erase(CalcQuality(*currentBin, *rightBin)) != 0, "Erase failed");
BinsByQuality.insert(CalcQuality(*leftBin, *rightBin));
}
}
@@ -567,12 +567,12 @@ namespace NKiwiAggr {
void TAdaptiveHistogram::Shrink() {
TPairSet::iterator worstBin = BinsByQuality.begin();
- Y_VERIFY(worstBin != BinsByQuality.end(), "No right bin for the first bin");
+ Y_ABORT_UNLESS(worstBin != BinsByQuality.end(), "No right bin for the first bin");
TPairSet::iterator leftBin = Bins.lower_bound(TWeightedValue(worstBin->second, -1.0));
- Y_VERIFY(leftBin != Bins.end() && leftBin->first == worstBin->second, "Can't find worst bin");
+ Y_ABORT_UNLESS(leftBin != Bins.end() && leftBin->first == worstBin->second, "Can't find worst bin");
TPairSet::iterator rightBin = leftBin;
++rightBin;
- Y_VERIFY(rightBin != Bins.end(), "Can't find right bin");
+ Y_ABORT_UNLESS(rightBin != Bins.end(), "Can't find right bin");
TWeightedValue newBin((leftBin->first * leftBin->second + rightBin->first * rightBin->second) / (leftBin->second + rightBin->second), leftBin->second + rightBin->second);
if (Bins.size() > 2) {
diff --git a/library/cpp/histogram/adaptive/block_histogram.cpp b/library/cpp/histogram/adaptive/block_histogram.cpp
index 6586d13ff6..55337488ac 100644
--- a/library/cpp/histogram/adaptive/block_histogram.cpp
+++ b/library/cpp/histogram/adaptive/block_histogram.cpp
@@ -149,7 +149,7 @@ namespace NKiwiAggr {
histo.GetType() == HT_ADAPTIVE_WARD_HISTOGRAM ||
histo.GetType() == HT_ADAPTIVE_HISTOGRAM)
{
- Y_VERIFY(histo.FreqSize() == histo.PositionSize(), "Corrupted histo");
+ Y_ABORT_UNLESS(histo.FreqSize() == histo.PositionSize(), "Corrupted histo");
for (size_t j = 0; j < histo.FreqSize(); ++j) {
double value = histo.GetPosition(j);
double weight = histo.GetFreq(j);
@@ -204,7 +204,7 @@ namespace NKiwiAggr {
}
void TBlockHistogram::FromProto(const THistogram& histo) {
- Y_VERIFY(histo.HasType(), "Attempt to parse TBlockHistogram from THistogram protobuf with no Type field set");
+ Y_ABORT_UNLESS(histo.HasType(), "Attempt to parse TBlockHistogram from THistogram protobuf with no Type field set");
;
switch (histo.GetType()) { // check that histogram type is correct
case HT_ADAPTIVE_DISTANCE_HISTOGRAM:
@@ -286,7 +286,7 @@ namespace NKiwiAggr {
}
void TBlockHistogram::SortAndShrink(size_t intervals, bool final) {
- Y_VERIFY(intervals > 0);
+ Y_ABORT_UNLESS(intervals > 0);
if (Bins.size() <= intervals) {
return;
@@ -382,7 +382,7 @@ namespace NKiwiAggr {
ui32 a = (ui32)(bins[b].Prev() - bins);
ui32 c = (ui32)(bins[b].Next() - bins);
ui32 d = (ui32)(bins[b].Next()->Next() - bins);
- Y_VERIFY(Bins[c].second != -1);
+ Y_ABORT_UNLESS(Bins[c].second != -1);
double mass = Bins[b].second + Bins[c].second;
Bins[c].first = (Bins[b].first * Bins[b].second + Bins[c].first * Bins[c].second) / mass;
@@ -411,7 +411,7 @@ namespace NKiwiAggr {
Bins.resize(pos);
PrevSize = pos;
- Y_VERIFY(pos == intervals);
+ Y_ABORT_UNLESS(pos == intervals);
}
double TBlockHistogram::GetSumInRange(double leftBound, double rightBound) {
@@ -528,7 +528,7 @@ namespace NKiwiAggr {
}
void TBlockWardHistogram::FastGreedyShrink(size_t intervals) {
- Y_VERIFY(intervals > 0);
+ Y_ABORT_UNLESS(intervals > 0);
if (Bins.size() <= intervals) {
return;
diff --git a/library/cpp/histogram/adaptive/fixed_bin_histogram.cpp b/library/cpp/histogram/adaptive/fixed_bin_histogram.cpp
index 558aba9e2d..a6bcd3a517 100644
--- a/library/cpp/histogram/adaptive/fixed_bin_histogram.cpp
+++ b/library/cpp/histogram/adaptive/fixed_bin_histogram.cpp
@@ -512,7 +512,7 @@ namespace NKiwiAggr {
}
void TFixedBinHistogram::Shrink(double newReferencePoint, double newMaxValue) {
- Y_VERIFY(newReferencePoint < newMaxValue, "Invalid Shrink()");
+ Y_ABORT_UNLESS(newReferencePoint < newMaxValue, "Invalid Shrink()");
memset(&(ReserveFreqs[0]), 0, ReserveFreqs.size() * sizeof(double));
double newBinRange = CalcBinRange(newReferencePoint, newMaxValue);
diff --git a/library/cpp/ipv6_address/ipv6_address.cpp b/library/cpp/ipv6_address/ipv6_address.cpp
index b23534b74f..75007d56ee 100644
--- a/library/cpp/ipv6_address/ipv6_address.cpp
+++ b/library/cpp/ipv6_address/ipv6_address.cpp
@@ -17,7 +17,7 @@
namespace {
// reinterpret_cast from memory, where most significant bit is first
inline ui128 FromMemMSF(const char* memPtr) {
- Y_VERIFY(memPtr, " ");
+ Y_ABORT_UNLESS(memPtr, " ");
return ui128{
*reinterpret_cast<const ui64*>(memPtr),
*(reinterpret_cast<const ui64*>(memPtr) + 1)
@@ -174,20 +174,20 @@ void TIpv6Address::ToSockaddrAndSocklen(sockaddr_in& sockAddrIPv4,
sockAddrSize = sizeof(sockAddrIPv6);
sockAddrPtr = reinterpret_cast<sockaddr*>(&sockAddrIPv6);
} else
- Y_VERIFY(false);
+ Y_ABORT_UNLESS(false);
}
void TIpv6Address::ToInAddr(in_addr& Addr4) const {
- Y_VERIFY(Type_ == TIpv6Address::Ipv4);
+ Y_ABORT_UNLESS(Type_ == TIpv6Address::Ipv4);
Zero(Addr4);
ui32 Value = GetLow(Ip);
- Y_VERIFY(Value == GetLow(Ip), " ");
- Y_VERIFY(GetHigh(Ip) == 0, " ");
+ Y_ABORT_UNLESS(Value == GetLow(Ip), " ");
+ Y_ABORT_UNLESS(GetHigh(Ip) == 0, " ");
Addr4.s_addr = SwapBytes(Value);
}
void TIpv6Address::ToIn6Addr(in6_addr& Addr6) const {
- Y_VERIFY(Type_ == TIpv6Address::Ipv6);
+ Y_ABORT_UNLESS(Type_ == TIpv6Address::Ipv6);
Zero(Addr6);
ui64 Raw[2] = {GetHigh(Ip), GetLow(Ip)};
@@ -242,7 +242,7 @@ TIpv6Address TIpv6Address::Normalized() const noexcept {
return *this;
TIpv6Address Result = TryToExtractIpv4From6();
- Y_VERIFY(Result.IsNull() == false);
+ Y_ABORT_UNLESS(Result.IsNull() == false);
return Result;
}
diff --git a/library/cpp/ipv6_address/ut/ipv6_address_ut.cpp b/library/cpp/ipv6_address/ut/ipv6_address_ut.cpp
index 73bb7cffdd..f23aabe0ee 100644
--- a/library/cpp/ipv6_address/ut/ipv6_address_ut.cpp
+++ b/library/cpp/ipv6_address/ut/ipv6_address_ut.cpp
@@ -22,7 +22,7 @@ TResult IpRes(TString Ip, TIpPort Port) {
bool Ok;
THostAddressAndPort HostAddressAndPort;
HostAddressAndPort.Ip = TIpv6Address::FromString(Ip, Ok);
- Y_VERIFY(Ok);
+ Y_ABORT_UNLESS(Ok);
HostAddressAndPort.Port = Port;
return TResult(HostAddressAndPort, {}, {});
}
diff --git a/library/cpp/json/writer/json_value.cpp b/library/cpp/json/writer/json_value.cpp
index c61e8d1dc4..6d57ef669a 100644
--- a/library/cpp/json/writer/json_value.cpp
+++ b/library/cpp/json/writer/json_value.cpp
@@ -18,7 +18,7 @@ static bool
AreJsonMapsEqual(const NJson::TJsonValue& lhs, const NJson::TJsonValue& rhs) {
using namespace NJson;
- Y_VERIFY(lhs.GetType() == JSON_MAP, "lhs has not a JSON_MAP type.");
+ Y_ABORT_UNLESS(lhs.GetType() == JSON_MAP, "lhs has not a JSON_MAP type.");
if (rhs.GetType() != JSON_MAP)
return false;
@@ -46,7 +46,7 @@ static bool
AreJsonArraysEqual(const NJson::TJsonValue& lhs, const NJson::TJsonValue& rhs) {
using namespace NJson;
- Y_VERIFY(lhs.GetType() == JSON_ARRAY, "lhs has not a JSON_ARRAY type.");
+ Y_ABORT_UNLESS(lhs.GetType() == JSON_ARRAY, "lhs has not a JSON_ARRAY type.");
if (rhs.GetType() != JSON_ARRAY)
return false;
diff --git a/library/cpp/lcs/lcs_via_lis.h b/library/cpp/lcs/lcs_via_lis.h
index d26733d94e..fe0ac297fc 100644
--- a/library/cpp/lcs/lcs_via_lis.h
+++ b/library/cpp/lcs/lcs_via_lis.h
@@ -144,7 +144,7 @@ namespace NLCS {
for (auto it = cover.begin() + 1; it != cover.end(); ++it) {
auto pit = UpperBound(it->begin(), it->end(), resbuf.back(), std::greater<ui32>());
- Y_VERIFY(pit != it->end(), " ");
+ Y_ABORT_UNLESS(pit != it->end(), " ");
resbuf.push_back(*pit);
}
diff --git a/library/cpp/lfalloc/lf_allocX64.h b/library/cpp/lfalloc/lf_allocX64.h
index b962fdf30b..ae753925c3 100644
--- a/library/cpp/lfalloc/lf_allocX64.h
+++ b/library/cpp/lfalloc/lf_allocX64.h
@@ -683,7 +683,7 @@ class TLFAllocFreeList {
for (res = Head; res; res = Head) {
TNode* keepNext = res->Next;
if (DoCas(&Head, keepNext, res) == res) {
- //Y_VERIFY(keepNext == res->Next);
+ //Y_ABORT_UNLESS(keepNext == res->Next);
break;
}
}
diff --git a/library/cpp/logger/global/global.h b/library/cpp/logger/global/global.h
index cbe71b16ea..dece990083 100644
--- a/library/cpp/logger/global/global.h
+++ b/library/cpp/logger/global/global.h
@@ -81,7 +81,7 @@ public:
do { \
if (Y_UNLIKELY(!(expr))) { \
FATAL_LOG << Sprintf(msg, ##__VA_ARGS__) << Endl; \
- Y_VERIFY(false, msg, ##__VA_ARGS__); \
+ Y_ABORT_UNLESS(false, msg, ##__VA_ARGS__); \
}; \
} while (0);
diff --git a/library/cpp/logger/global/rty_formater.cpp b/library/cpp/logger/global/rty_formater.cpp
index 305f8470c5..66657e7162 100644
--- a/library/cpp/logger/global/rty_formater.cpp
+++ b/library/cpp/logger/global/rty_formater.cpp
@@ -12,7 +12,7 @@ namespace {
constexpr size_t LocalTimeSBufferSize = sizeof("2017-07-24 12:20:34.313 +0300");
size_t PrintLocalTimeS(const TInstant instant, char* const begin, const char* const end) {
- Y_VERIFY(static_cast<size_t>(end - begin) >= LocalTimeSBufferSize);
+ Y_ABORT_UNLESS(static_cast<size_t>(end - begin) >= LocalTimeSBufferSize);
struct tm tm;
instant.LocalTime(&tm);
@@ -22,7 +22,7 @@ namespace {
pos += strftime(pos, end - pos, "%Y-%m-%d %H:%M:%S.", &tm);
pos += sprintf(pos, "%03" PRIu32, instant.MilliSecondsOfSecond());
pos += strftime(pos, end - pos, " %z", &tm);
- Y_VERIFY(LocalTimeSBufferSize - 1 == pos - begin); // together with Y_VERIFY above this also implies pos<=end
+ Y_ABORT_UNLESS(LocalTimeSBufferSize - 1 == pos - begin); // together with Y_VERIFY above this also implies pos<=end
return (pos - begin);
}
}
diff --git a/library/cpp/logger/uninitialized_creator.cpp b/library/cpp/logger/uninitialized_creator.cpp
index 26dd168529..748127356e 100644
--- a/library/cpp/logger/uninitialized_creator.cpp
+++ b/library/cpp/logger/uninitialized_creator.cpp
@@ -43,7 +43,7 @@ bool TLogBackendCreatorUninitialized::Init(const IInitContext& ctx) {
void TLogBackendCreatorUninitialized::ToJson(NJson::TJsonValue& value) const {
- Y_VERIFY(Slave, "Serialization off uninitialized LogBackendCreator");
+ Y_ABORT_UNLESS(Slave, "Serialization off uninitialized LogBackendCreator");
Slave->ToJson(value);
}
diff --git a/library/cpp/lwtrace/example3/my_action.h b/library/cpp/lwtrace/example3/my_action.h
index 9a04293ba2..a8185ab9da 100644
--- a/library/cpp/lwtrace/example3/my_action.h
+++ b/library/cpp/lwtrace/example3/my_action.h
@@ -33,7 +33,7 @@ public:
// Outputs a line to opened file
// Can be called from DoExecute() and must be thread-safe
void Output(const TString& line) {
- Y_VERIFY(File);
+ Y_ABORT_UNLESS(File);
TGuard<TMutex> g(Mutex); // Because DoExecute() call can come from any thread
*File << line << Endl;
}
diff --git a/library/cpp/lwtrace/kill_action.cpp b/library/cpp/lwtrace/kill_action.cpp
index 2b74dc4587..124cc5f15a 100644
--- a/library/cpp/lwtrace/kill_action.cpp
+++ b/library/cpp/lwtrace/kill_action.cpp
@@ -15,7 +15,7 @@ bool TKillActionExecutor::DoExecute(TOrbit&, const TParams&) {
abort();
#else
int r = kill(getpid(), SIGABRT);
- Y_VERIFY(r == 0, "kill failed");
+ Y_ABORT_UNLESS(r == 0, "kill failed");
return true;
#endif
}
diff --git a/library/cpp/lwtrace/lwprobe.h b/library/cpp/lwtrace/lwprobe.h
index 801fc3861b..ca6e4eaf39 100644
--- a/library/cpp/lwtrace/lwprobe.h
+++ b/library/cpp/lwtrace/lwprobe.h
@@ -33,28 +33,28 @@ namespace NLWTrace {
probe.Init();
// initialize TEvent
- Y_VERIFY(IsCppIdentifier(Name), "probe '%s' is not C++ identifier", Name.data());
- Y_VERIFY(IsCppIdentifier(Provider), "provider '%s' is not C++ identifier in probe %s", Provider.data(), Name.data());
+ Y_ABORT_UNLESS(IsCppIdentifier(Name), "probe '%s' is not C++ identifier", Name.data());
+ Y_ABORT_UNLESS(IsCppIdentifier(Provider), "provider '%s' is not C++ identifier in probe %s", Provider.data(), Name.data());
probe.Event.Name = Name.c_str();
Zero(probe.Event.Groups);
probe.Event.Groups[0] = Provider.c_str();
auto i = Groups.begin(), ie = Groups.end();
- Y_VERIFY(Groups.size() < LWTRACE_MAX_GROUPS, "too many groups in probe %s", Name.data());
+ Y_ABORT_UNLESS(Groups.size() < LWTRACE_MAX_GROUPS, "too many groups in probe %s", Name.data());
for (size_t n = 1; n < LWTRACE_MAX_GROUPS && i != ie; n++, ++i) {
- Y_VERIFY(IsCppIdentifier(*i), "group '%s' is not C++ identifier in probe %s", i->data(), Name.data());
+ Y_ABORT_UNLESS(IsCppIdentifier(*i), "group '%s' is not C++ identifier in probe %s", i->data(), Name.data());
probe.Event.Groups[n] = i->c_str();
}
// initialize TSignature
using TUsrSign = TUserSignature<LWTRACE_TEMPLATE_ARGS>;
- Y_VERIFY(TUsrSign::ParamCount == (int)Params.size(), "param count mismatch in probe %s: %d != %d",
+ Y_ABORT_UNLESS(TUsrSign::ParamCount == (int)Params.size(), "param count mismatch in probe %s: %d != %d",
Name.data(), int(Params.size()), TUsrSign::ParamCount);
TSignature& signature = probe.Event.Signature;
signature.ParamTypes = TUsrSign::ParamTypes;
Zero(signature.ParamNames);
auto j = Params.begin(), je = Params.end();
for (size_t n = 0; n < LWTRACE_MAX_PARAMS && j != je; n++, ++j) {
- Y_VERIFY(IsCppIdentifier(*j), "param '%s' is not C++ identifier in probe %s", j->data(), Name.data());
+ Y_ABORT_UNLESS(IsCppIdentifier(*j), "param '%s' is not C++ identifier in probe %s", j->data(), Name.data());
signature.ParamNames[n] = j->c_str();
}
signature.ParamCount = TUsrSign::ParamCount;
diff --git a/library/cpp/lwtrace/mon/analytics/data.h b/library/cpp/lwtrace/mon/analytics/data.h
index 4b643fe20b..281b246d34 100644
--- a/library/cpp/lwtrace/mon/analytics/data.h
+++ b/library/cpp/lwtrace/mon/analytics/data.h
@@ -83,15 +83,15 @@ struct TMatrix : public TVector<double> {
double& Cell(size_t row, size_t col)
{
- Y_VERIFY(row < Rows);
- Y_VERIFY(col < Cols);
+ Y_ABORT_UNLESS(row < Rows);
+ Y_ABORT_UNLESS(col < Cols);
return operator[](row * Cols + col);
}
double Cell(size_t row, size_t col) const
{
- Y_VERIFY(row < Rows);
- Y_VERIFY(col < Cols);
+ Y_ABORT_UNLESS(row < Rows);
+ Y_ABORT_UNLESS(col < Cols);
return operator[](row * Cols + col);
}
diff --git a/library/cpp/lwtrace/shuttle.h b/library/cpp/lwtrace/shuttle.h
index d8e0987c35..c3e31a8223 100644
--- a/library/cpp/lwtrace/shuttle.h
+++ b/library/cpp/lwtrace/shuttle.h
@@ -205,7 +205,7 @@ namespace NLWTrace {
void AddShuttle(const TShuttlePtr& shuttle) {
NotConcurrent([&] (TShuttlePtr& head) {
- Y_VERIFY(!shuttle->GetNext());
+ Y_ABORT_UNLESS(!shuttle->GetNext());
shuttle->SetNext(head);
head = shuttle;
});
diff --git a/library/cpp/lwtrace/start.cpp b/library/cpp/lwtrace/start.cpp
index 121d5472b6..e4bb6e13c8 100644
--- a/library/cpp/lwtrace/start.cpp
+++ b/library/cpp/lwtrace/start.cpp
@@ -26,7 +26,7 @@ namespace {
TString script = TUnbufferedFileInput(path).ReadAll();
TQuery query;
bool ok = google::protobuf::TextFormat::ParseFromString(script, &query);
- Y_VERIFY(ok, "failed to parse protobuf");
+ Y_ABORT_UNLESS(ok, "failed to parse protobuf");
Singleton<TTraceManagerHolder>()->TraceManager.New("env", query);
}
diff --git a/library/cpp/lwtrace/trace.cpp b/library/cpp/lwtrace/trace.cpp
index 3c974c85a0..76e64a3025 100644
--- a/library/cpp/lwtrace/trace.cpp
+++ b/library/cpp/lwtrace/trace.cpp
@@ -50,7 +50,7 @@ namespace NLWTrace {
return; // silently skip probe double registration
}
TIds::key_type key(probe->Event.GetProvider(), probe->Event.Name);
- Y_VERIFY(Ids.count(key) == 0, "duplicate provider:probe pair %s:%s", key.first.data(), key.second.data());
+ Y_ABORT_UNLESS(Ids.count(key) == 0, "duplicate provider:probe pair %s:%s", key.first.data(), key.second.data());
Probes.emplace(probe, box);
Ids.insert(key);
}
diff --git a/library/cpp/malloc/nalf/alloc_helpers.h b/library/cpp/malloc/nalf/alloc_helpers.h
index a48f5a41ee..0e3649d76a 100644
--- a/library/cpp/malloc/nalf/alloc_helpers.h
+++ b/library/cpp/malloc/nalf/alloc_helpers.h
@@ -29,12 +29,12 @@ struct TSystemAllocHelper {
// (allocator themself)
void* operator new(size_t sz) {
- Y_VERIFY(sz == sizeof(TFinal));
+ Y_ABORT_UNLESS(sz == sizeof(TFinal));
return NNumaAwareLockFreeAllocator::SystemAllocation(sz);
}
void* operator new[](size_t sz) {
- Y_VERIFY(sz == sizeof(TFinal));
+ Y_ABORT_UNLESS(sz == sizeof(TFinal));
return NNumaAwareLockFreeAllocator::SystemAllocation(sz);
}
diff --git a/library/cpp/messagebus/actor/actor.h b/library/cpp/messagebus/actor/actor.h
index 9b8f20298a..a7e706cf21 100644
--- a/library/cpp/messagebus/actor/actor.h
+++ b/library/cpp/messagebus/actor/actor.h
@@ -82,14 +82,14 @@ namespace NActor {
void ScheduleHereAtMostOnce() {
if (Tasks.AddTask()) {
bool fetched = Tasks.FetchTask();
- Y_VERIFY(fetched, "happens");
+ Y_ABORT_UNLESS(fetched, "happens");
DoAct();
// if someone added more tasks, schedule them
if (Tasks.FetchTask()) {
bool added = Tasks.AddTask();
- Y_VERIFY(!added, "happens");
+ Y_ABORT_UNLESS(!added, "happens");
EnqueueWork();
}
}
diff --git a/library/cpp/messagebus/actor/executor.cpp b/library/cpp/messagebus/actor/executor.cpp
index 0c095b1677..70233799e3 100644
--- a/library/cpp/messagebus/actor/executor.cpp
+++ b/library/cpp/messagebus/actor/executor.cpp
@@ -178,7 +178,7 @@ void TExecutor::Init() {
AtomicSet(ExitWorkers, 0);
- Y_VERIFY(Config.WorkerCount > 0);
+ Y_ABORT_UNLESS(Config.WorkerCount > 0);
for (size_t i = 0; i < Config.WorkerCount; i++) {
WorkerThreads.push_back(new TExecutorWorker(this));
@@ -215,7 +215,7 @@ void TExecutor::EnqueueWork(TArrayRef<IWorkItem* const> wis) {
return;
if (Y_UNLIKELY(AtomicGet(ExitWorkers) != 0)) {
- Y_VERIFY(WorkItems.Empty(), "executor %s: cannot add tasks after queue shutdown", Config.Name);
+ Y_ABORT_UNLESS(WorkItems.Empty(), "executor %s: cannot add tasks after queue shutdown", Config.Name);
}
TWhatThreadDoesPushPop pp("executor: EnqueueWork");
@@ -319,7 +319,7 @@ void TExecutor::ProcessWorkQueueHere() {
}
void TExecutor::RunWorker() {
- Y_VERIFY(!ThreadCurrentExecutor, "state check");
+ Y_ABORT_UNLESS(!ThreadCurrentExecutor, "state check");
ThreadCurrentExecutor = this;
SetCurrentThreadName("wrkr");
diff --git a/library/cpp/messagebus/actor/queue_for_actor.h b/library/cpp/messagebus/actor/queue_for_actor.h
index 40fa536b82..99d0d4c89e 100644
--- a/library/cpp/messagebus/actor/queue_for_actor.h
+++ b/library/cpp/messagebus/actor/queue_for_actor.h
@@ -21,7 +21,7 @@ namespace NActor {
public:
~TQueueForActor() {
- Y_VERIFY(Queue.IsEmpty());
+ Y_ABORT_UNLESS(Queue.IsEmpty());
}
bool IsEmpty() {
diff --git a/library/cpp/messagebus/async_result.h b/library/cpp/messagebus/async_result.h
index d24dde284a..be8c9c1f95 100644
--- a/library/cpp/messagebus/async_result.h
+++ b/library/cpp/messagebus/async_result.h
@@ -24,7 +24,7 @@ private:
public:
void SetResult(const T& result) {
TGuard<TMutex> guard(Mutex);
- Y_VERIFY(!Result, "cannot set result twice");
+ Y_ABORT_UNLESS(!Result, "cannot set result twice");
Result = result;
CondVar.BroadCast();
diff --git a/library/cpp/messagebus/config/netaddr.cpp b/library/cpp/messagebus/config/netaddr.cpp
index 962ac538e2..4660711a0c 100644
--- a/library/cpp/messagebus/config/netaddr.cpp
+++ b/library/cpp/messagebus/config/netaddr.cpp
@@ -66,7 +66,7 @@ namespace NBus {
TNetAddr::TNetAddr(TAutoPtr<IRemoteAddr> addr)
: Ptr(addr)
{
- Y_VERIFY(!!Ptr);
+ Y_ABORT_UNLESS(!!Ptr);
}
namespace {
diff --git a/library/cpp/messagebus/config/session_config.cpp b/library/cpp/messagebus/config/session_config.cpp
index fbbbb106c9..3188bf62e2 100644
--- a/library/cpp/messagebus/config/session_config.cpp
+++ b/library/cpp/messagebus/config/session_config.cpp
@@ -31,12 +31,12 @@ static int ParseToSForMessageBus(const char* option) {
TStringBuf str(option);
if (str.StartsWith("0x")) {
str = str.Tail(2);
- Y_VERIFY(str.length() == 2, "ToS must be a number between 0x00 and 0xFF");
+ Y_ABORT_UNLESS(str.length() == 2, "ToS must be a number between 0x00 and 0xFF");
tos = String2Byte(str.data());
} else {
tos = FromString<int>(option);
}
- Y_VERIFY(tos >= 0 && tos <= 255, "ToS must be between 0x00 and 0xFF");
+ Y_ABORT_UNLESS(tos >= 0 && tos <= 255, "ToS must be between 0x00 and 0xFF");
return tos;
}
diff --git a/library/cpp/messagebus/duration_histogram.cpp b/library/cpp/messagebus/duration_histogram.cpp
index 32a0001d41..ac3783b600 100644
--- a/library/cpp/messagebus/duration_histogram.cpp
+++ b/library/cpp/messagebus/duration_histogram.cpp
@@ -48,7 +48,7 @@ namespace {
}
TString TDurationHistogram::LabelBefore(unsigned i) {
- Y_VERIFY(i < Buckets);
+ Y_ABORT_UNLESS(i < Buckets);
TDuration d = Singleton<TMarks>()->Marks[i];
diff --git a/library/cpp/messagebus/event_loop.cpp b/library/cpp/messagebus/event_loop.cpp
index 4240a5cd96..9a9d90508d 100644
--- a/library/cpp/messagebus/event_loop.cpp
+++ b/library/cpp/messagebus/event_loop.cpp
@@ -280,7 +280,7 @@ TEventLoop::TImpl::TImpl(const char* name)
void TEventLoop::TImpl::Run() {
bool res = AtomicCas(&RunningState, EVENT_LOOP_RUNNING, EVENT_LOOP_CREATED);
- Y_VERIFY(res, "Invalid mbus event loop state");
+ Y_ABORT_UNLESS(res, "Invalid mbus event loop state");
if (!!Name) {
SetCurrentThreadName(Name);
@@ -308,7 +308,7 @@ void TEventLoop::TImpl::Run() {
SOCKET socket = -1;
while (SocketsToRemove.Dequeue(&socket)) {
TGuard<TMutex> guard(Mutex);
- Y_VERIFY(Data.erase(socket) == 1, "must be removed once");
+ Y_ABORT_UNLESS(Data.erase(socket) == 1, "must be removed once");
}
}
@@ -324,7 +324,7 @@ void TEventLoop::TImpl::Run() {
res = AtomicCas(&RunningState, EVENT_LOOP_STOPPED, EVENT_LOOP_RUNNING);
- Y_VERIFY(res);
+ Y_ABORT_UNLESS(res);
StoppedEvent.Signal();
}
@@ -340,13 +340,13 @@ void TEventLoop::TImpl::Stop() {
}
TChannelPtr TEventLoop::TImpl::Register(TSocket socket, TEventHandlerPtr eventHandler, void* cookie) {
- Y_VERIFY(socket != INVALID_SOCKET, "must be a valid socket");
+ Y_ABORT_UNLESS(socket != INVALID_SOCKET, "must be a valid socket");
TChannelPtr channel = new TChannel(new TChannel::TImpl(this, socket, eventHandler, cookie));
TGuard<TMutex> guard(Mutex);
- Y_VERIFY(Data.insert(std::make_pair(socket, channel)).second, "must not be already inserted");
+ Y_ABORT_UNLESS(Data.insert(std::make_pair(socket, channel)).second, "must not be already inserted");
return channel;
}
diff --git a/library/cpp/messagebus/futex_like.cpp b/library/cpp/messagebus/futex_like.cpp
index 7f965126db..71998065a6 100644
--- a/library/cpp/messagebus/futex_like.cpp
+++ b/library/cpp/messagebus/futex_like.cpp
@@ -31,7 +31,7 @@ void TFutexLike::Wake(size_t count) {
count = Max<int>();
}
int r = futex(&Value, FUTEX_WAKE, count, nullptr, nullptr, 0);
- Y_VERIFY(r >= 0, "futex_wake failed: %s", strerror(errno));
+ Y_ABORT_UNLESS(r >= 0, "futex_wake failed: %s", strerror(errno));
#else
TGuard<TMutex> guard(Mutex);
if (count == 1) {
@@ -45,7 +45,7 @@ void TFutexLike::Wake(size_t count) {
void TFutexLike::Wait(int expected) {
#ifdef _linux_
int r = futex(&Value, FUTEX_WAIT, expected, nullptr, nullptr, 0);
- Y_VERIFY(r >= 0 || errno == EWOULDBLOCK, "futex_wait failed: %s", strerror(errno));
+ Y_ABORT_UNLESS(r >= 0 || errno == EWOULDBLOCK, "futex_wait failed: %s", strerror(errno));
#else
TGuard<TMutex> guard(Mutex);
if (expected == Get()) {
diff --git a/library/cpp/messagebus/handler_impl.h b/library/cpp/messagebus/handler_impl.h
index 6593f04cc3..ae4d823400 100644
--- a/library/cpp/messagebus/handler_impl.h
+++ b/library/cpp/messagebus/handler_impl.h
@@ -15,8 +15,8 @@ namespace NBus {
}
inline void TOnMessageContext::AckMessage(TBusIdentity& ident) {
- Y_VERIFY(Ident.LocalFlags == NPrivate::MESSAGE_IN_WORK);
- Y_VERIFY(ident.LocalFlags == 0);
+ Y_ABORT_UNLESS(Ident.LocalFlags == NPrivate::MESSAGE_IN_WORK);
+ Y_ABORT_UNLESS(ident.LocalFlags == 0);
Ident.Swap(ident);
}
diff --git a/library/cpp/messagebus/locator.cpp b/library/cpp/messagebus/locator.cpp
index e38a35c426..e49716f166 100644
--- a/library/cpp/messagebus/locator.cpp
+++ b/library/cpp/messagebus/locator.cpp
@@ -250,7 +250,7 @@ namespace NBus {
int TBusLocator::LocateAll(TBusService service, TBusKey key, TVector<TNetAddr>& addrs) {
TGuard<TMutex> G(Lock);
- Y_VERIFY(addrs.empty(), "Non emtpy addresses");
+ Y_ABORT_UNLESS(addrs.empty(), "Non emtpy addresses");
TServiceId serviceId = GetServiceId(service);
TItems::const_iterator it;
@@ -361,7 +361,7 @@ namespace NBus {
int TBusLocator::LocateKeys(TBusService service, TBusKeyVec& keys, bool onlyLocal) {
TGuard<TMutex> G(Lock);
- Y_VERIFY(keys.empty(), "Non empty keys");
+ Y_ABORT_UNLESS(keys.empty(), "Non empty keys");
TServiceId serviceId = GetServiceId(service);
TItems::const_iterator it;
diff --git a/library/cpp/messagebus/message.cpp b/library/cpp/messagebus/message.cpp
index 20784860f3..ac8caa5c56 100644
--- a/library/cpp/messagebus/message.cpp
+++ b/library/cpp/messagebus/message.cpp
@@ -23,10 +23,10 @@ namespace NBus {
TBusIdentity::~TBusIdentity() {
// TODO: print local flags
#ifndef NDEBUG
- Y_VERIFY(LocalFlags == 0, "local flags must be zero at this point; message type is %s",
+ Y_ABORT_UNLESS(LocalFlags == 0, "local flags must be zero at this point; message type is %s",
MessageType.value_or("unknown").c_str());
#else
- Y_VERIFY(LocalFlags == 0, "local flags must be zero at this point");
+ Y_ABORT_UNLESS(LocalFlags == 0, "local flags must be zero at this point");
#endif
}
@@ -47,8 +47,8 @@ namespace NBus {
}
void TBusIdentity::Unpack(const char* src) {
- Y_VERIFY(LocalFlags == 0);
- Y_VERIFY(!Connection);
+ Y_ABORT_UNLESS(LocalFlags == 0);
+ Y_ABORT_UNLESS(!Connection);
memcpy(this, src, sizeof(TBusIdentity));
}
@@ -88,7 +88,7 @@ namespace NBus {
TBusMessage::~TBusMessage() {
#ifndef NDEBUG
- Y_VERIFY(GetHeader()->Id != YBUS_KEYINVALID, "must not be invalid key, message type: %d, ", int(Type));
+ Y_ABORT_UNLESS(GetHeader()->Id != YBUS_KEYINVALID, "must not be invalid key, message type: %d, ", int(Type));
GetHeader()->Id = YBUS_KEYINVALID;
Data = (void*)17;
CheckClean();
diff --git a/library/cpp/messagebus/message.h b/library/cpp/messagebus/message.h
index 005ca10c65..2027f24983 100644
--- a/library/cpp/messagebus/message.h
+++ b/library/cpp/messagebus/message.h
@@ -93,7 +93,7 @@ namespace NBus {
void SetMessageType(const std::type_info& messageTypeInfo) {
#ifndef NDEBUG
- Y_VERIFY(!MessageType, "state check");
+ Y_ABORT_UNLESS(!MessageType, "state check");
MessageType = TypeName(messageTypeInfo);
#else
Y_UNUSED(messageTypeInfo);
diff --git a/library/cpp/messagebus/message_status_counter.cpp b/library/cpp/messagebus/message_status_counter.cpp
index 891c8f5bb2..3d3f3c344e 100644
--- a/library/cpp/messagebus/message_status_counter.cpp
+++ b/library/cpp/messagebus/message_status_counter.cpp
@@ -28,7 +28,7 @@ TString TMessageStatusCounter::PrintToString() const {
bool hasZeros = false;
for (size_t i = 0; i < MESSAGE_STATUS_COUNT; ++i) {
if (i == MESSAGE_OK) {
- Y_VERIFY(Counts[i] == 0);
+ Y_ABORT_UNLESS(Counts[i] == 0);
continue;
}
if (Counts[i] != 0) {
@@ -59,7 +59,7 @@ void TMessageStatusCounter::FillErrorsProtobuf(TConnectionStatusMonRecord* statu
status->clear_errorcountbystatus();
for (size_t i = 0; i < MESSAGE_STATUS_COUNT; ++i) {
if (i == MESSAGE_OK) {
- Y_VERIFY(Counts[i] == 0);
+ Y_ABORT_UNLESS(Counts[i] == 0);
continue;
}
if (Counts[i] != 0) {
diff --git a/library/cpp/messagebus/messqueue.cpp b/library/cpp/messagebus/messqueue.cpp
index 3474d62705..85baa3c383 100644
--- a/library/cpp/messagebus/messqueue.cpp
+++ b/library/cpp/messagebus/messqueue.cpp
@@ -169,7 +169,7 @@ void TBusMessageQueue::Add(TIntrusivePtr<TBusSessionImpl> session) {
void TBusMessageQueue::Remove(TBusSession* session) {
TGuard<TMutex> scope(Lock);
TList<TIntrusivePtr<TBusSessionImpl>>::iterator it = std::find(Sessions.begin(), Sessions.end(), session);
- Y_VERIFY(it != Sessions.end(), "do not destroy session twice");
+ Y_ABORT_UNLESS(it != Sessions.end(), "do not destroy session twice");
Sessions.erase(it);
}
@@ -185,7 +185,7 @@ void TBusMessageQueue::DestroyAllSessions() {
}
for (auto& session : sessions) {
- Y_VERIFY(session->IsDown(), "Session must be shut down prior to queue shutdown");
+ Y_ABORT_UNLESS(session->IsDown(), "Session must be shut down prior to queue shutdown");
}
}
diff --git a/library/cpp/messagebus/misc/test_sync.h b/library/cpp/messagebus/misc/test_sync.h
index be3f4f20b8..07dde1dfcd 100644
--- a/library/cpp/messagebus/misc/test_sync.h
+++ b/library/cpp/messagebus/misc/test_sync.h
@@ -32,7 +32,7 @@ public:
void WaitFor(unsigned n) {
TGuard<TMutex> guard(Mutex);
- Y_VERIFY(Current <= n, "too late, waiting for %d, already %d", n, Current);
+ Y_ABORT_UNLESS(Current <= n, "too late, waiting for %d, already %d", n, Current);
while (n > Current) {
CondVar.WaitI(Mutex);
@@ -42,7 +42,7 @@ public:
void WaitForAndIncrement(unsigned n) {
TGuard<TMutex> guard(Mutex);
- Y_VERIFY(Current <= n, "too late, waiting for %d, already %d", n, Current);
+ Y_ABORT_UNLESS(Current <= n, "too late, waiting for %d, already %d", n, Current);
while (n > Current) {
CondVar.WaitI(Mutex);
@@ -55,7 +55,7 @@ public:
void CheckAndIncrement(unsigned n) {
TGuard<TMutex> guard(Mutex);
- Y_VERIFY(Current == n, "must be %d, currently %d", n, Current);
+ Y_ABORT_UNLESS(Current == n, "must be %d, currently %d", n, Current);
DoInc();
CondVar.BroadCast();
@@ -64,7 +64,7 @@ public:
void Check(unsigned n) {
TGuard<TMutex> guard(Mutex);
- Y_VERIFY(Current == n, "must be %d, currently %d", n, Current);
+ Y_ABORT_UNLESS(Current == n, "must be %d, currently %d", n, Current);
}
private:
diff --git a/library/cpp/messagebus/network.cpp b/library/cpp/messagebus/network.cpp
index 304bedae5a..a4afafa94e 100644
--- a/library/cpp/messagebus/network.cpp
+++ b/library/cpp/messagebus/network.cpp
@@ -11,7 +11,7 @@ using namespace NBus::NPrivate;
namespace {
TBindResult BindOnPortProto(int port, int af, bool reusePort) {
- Y_VERIFY(af == AF_INET || af == AF_INET6, "wrong af");
+ Y_ABORT_UNLESS(af == AF_INET || af == AF_INET6, "wrong af");
SOCKET fd = ::socket(af, SOCK_STREAM, 0);
if (fd == INVALID_SOCKET) {
@@ -82,7 +82,7 @@ namespace {
}
std::pair<unsigned, TVector<TBindResult>> AggregateBindResults(TBindResult&& r1, TBindResult&& r2) {
- Y_VERIFY(r1.Addr.GetPort() == r2.Addr.GetPort(), "internal");
+ Y_ABORT_UNLESS(r1.Addr.GetPort() == r2.Addr.GetPort(), "internal");
std::pair<unsigned, TVector<TBindResult>> r;
r.second.reserve(2);
@@ -138,7 +138,7 @@ ssize_t NBus::NPrivate::SocketSend(SOCKET s, TArrayRef<const char> data) {
#endif
ssize_t r = ::send(s, data.data(), data.size(), flags);
if (r < 0) {
- Y_VERIFY(LastSystemError() != EBADF, "bad fd");
+ Y_ABORT_UNLESS(LastSystemError() != EBADF, "bad fd");
}
return r;
}
@@ -150,7 +150,7 @@ ssize_t NBus::NPrivate::SocketRecv(SOCKET s, TArrayRef<char> buffer) {
#endif
ssize_t r = ::recv(s, buffer.data(), buffer.size(), flags);
if (r < 0) {
- Y_VERIFY(LastSystemError() != EBADF, "bad fd");
+ Y_ABORT_UNLESS(LastSystemError() != EBADF, "bad fd");
}
return r;
}
diff --git a/library/cpp/messagebus/nondestroying_holder.h b/library/cpp/messagebus/nondestroying_holder.h
index f4725d696f..bd56e5335c 100644
--- a/library/cpp/messagebus/nondestroying_holder.h
+++ b/library/cpp/messagebus/nondestroying_holder.h
@@ -16,7 +16,7 @@ public:
}
~TNonDestroyingHolder() {
- Y_VERIFY(!*this, "stored object must be explicitly released");
+ Y_ABORT_UNLESS(!*this, "stored object must be explicitly released");
}
};
@@ -34,6 +34,6 @@ public:
}
inline ~TNonDestroyingAutoPtr() {
- Y_VERIFY(!*this, "stored object must be explicitly released");
+ Y_ABORT_UNLESS(!*this, "stored object must be explicitly released");
}
};
diff --git a/library/cpp/messagebus/oldmodule/module.cpp b/library/cpp/messagebus/oldmodule/module.cpp
index 24bd778799..4ec37a4d53 100644
--- a/library/cpp/messagebus/oldmodule/module.cpp
+++ b/library/cpp/messagebus/oldmodule/module.cpp
@@ -143,14 +143,14 @@ namespace NBus {
~TBusModuleImpl() override {
// Shutdown cannot be called from destructor,
// because module has virtual methods.
- Y_VERIFY(State != RUNNING, "if running, must explicitly call Shutdown() before destructor");
+ Y_ABORT_UNLESS(State != RUNNING, "if running, must explicitly call Shutdown() before destructor");
Scheduler.Stop();
while (!Jobs.empty()) {
DestroyJob(Jobs.front());
}
- Y_VERIFY(JobCount == 0, "state check");
+ Y_ABORT_UNLESS(JobCount == 0, "state check");
}
void OnMessageReceived(TAutoPtr<TBusMessage> msg, TOnMessageContext&);
@@ -336,7 +336,7 @@ namespace NBus {
}
TNetAddr TBusJob::GetPeerAddrNetAddr() const {
- Y_VERIFY(!!OnMessageContext);
+ Y_ABORT_UNLESS(!!OnMessageContext);
return OnMessageContext.GetPeerAddrNetAddr();
}
@@ -452,7 +452,7 @@ namespace NBus {
}
}
- Y_VERIFY(!(Pending.size() == 0 && Handler == nullptr && Status == MESSAGE_OK && !ReplySent),
+ Y_ABORT_UNLESS(!(Pending.size() == 0 && Handler == nullptr && Status == MESSAGE_OK && !ReplySent),
"Handler returned NULL without Cancel() or SendReply() for message=%016" PRIx64 " type=%d",
Message->GetHeader()->Id, Message->GetHeader()->Type);
@@ -542,7 +542,7 @@ namespace NBus {
void TBusJob::SendReply(TBusMessageAutoPtr reply) {
CheckThreadCurrentJob();
- Y_VERIFY(!ReplySent, "cannot call SendReply twice");
+ Y_ABORT_UNLESS(!ReplySent, "cannot call SendReply twice");
ReplySent = true;
if (!OnMessageContext)
return;
@@ -581,8 +581,8 @@ namespace NBus {
void TBusJob::Sleep(int milliSeconds) {
CheckThreadCurrentJob();
- Y_VERIFY(Pending.empty(), "sleep is not allowed when there are pending job");
- Y_VERIFY(SleepUntil == 0, "must not override sleep");
+ Y_ABORT_UNLESS(Pending.empty(), "sleep is not allowed when there are pending job");
+ Y_ABORT_UNLESS(SleepUntil == 0, "must not override sleep");
SleepUntil = Now() + milliSeconds;
}
@@ -665,8 +665,8 @@ namespace NBus {
}
bool TBusModule::StartInput() {
- Y_VERIFY(Impl->State == TBusModuleImpl::CREATED, "state check");
- Y_VERIFY(!!Impl->Queue, "state check");
+ Y_ABORT_UNLESS(Impl->State == TBusModuleImpl::CREATED, "state check");
+ Y_ABORT_UNLESS(!!Impl->Queue, "state check");
Impl->State = TBusModuleImpl::RUNNING;
Y_ASSERT(!Impl->ExternalSession);
@@ -774,7 +774,7 @@ void TBusModuleImpl::DestroyJob(TJobRunner* job) {
{
TWhatThreadDoesAcquireGuard<TMutex> G(Lock, "modules: acquiring lock for DestroyJob");
int jobCount = AtomicDecrement(JobCount);
- Y_VERIFY(jobCount >= 0, "decremented too much");
+ Y_ABORT_UNLESS(jobCount >= 0, "decremented too much");
Jobs.erase(job->JobStorageIterator);
if (AtomicGet(State) == STOPPED) {
@@ -789,7 +789,7 @@ void TBusModuleImpl::DestroyJob(TJobRunner* job) {
void TBusModuleImpl::OnMessageReceived(TAutoPtr<TBusMessage> msg0, TOnMessageContext& context) {
TBusMessage* msg = !!msg0 ? msg0.Get() : context.GetMessage();
- Y_VERIFY(!!msg);
+ Y_ABORT_UNLESS(!!msg);
THolder<TJobRunner> jobRunner(new TJobRunner(Module->CreateJobInstance(msg)));
jobRunner->Job->MessageHolder.Reset(msg0.Release());
@@ -834,8 +834,8 @@ void TBusModuleImpl::Shutdown() {
}
EMessageStatus TBusModule::StartJob(TAutoPtr<TBusMessage> message) {
- Y_VERIFY(Impl->State == TBusModuleImpl::RUNNING);
- Y_VERIFY(!!Impl->Queue);
+ Y_ABORT_UNLESS(Impl->State == TBusModuleImpl::RUNNING);
+ Y_ABORT_UNLESS(!!Impl->Queue);
if ((unsigned)AtomicGet(Impl->JobCount) >= Impl->ModuleConfig.StarterMaxInFlight) {
return MESSAGE_BUSY;
diff --git a/library/cpp/messagebus/protobuf/ybusbuf.cpp b/library/cpp/messagebus/protobuf/ybusbuf.cpp
index 63415b3737..7212e59590 100644
--- a/library/cpp/messagebus/protobuf/ybusbuf.cpp
+++ b/library/cpp/messagebus/protobuf/ybusbuf.cpp
@@ -58,7 +58,7 @@ void TBusBufferProtocol::Serialize(const TBusMessage* mess, TBuffer& data) {
data.Reserve(data.Size() + size);
char* after = (char*)bmess->GetRecord()->SerializeWithCachedSizesToArray((ui8*)data.Pos());
- Y_VERIFY(after - data.Pos() == size);
+ Y_ABORT_UNLESS(after - data.Pos() == size);
data.Advance(size);
}
diff --git a/library/cpp/messagebus/rain_check/core/coro_stack.cpp b/library/cpp/messagebus/rain_check/core/coro_stack.cpp
index 83b984ca6e..96ab66865c 100644
--- a/library/cpp/messagebus/rain_check/core/coro_stack.cpp
+++ b/library/cpp/messagebus/rain_check/core/coro_stack.cpp
@@ -12,8 +12,8 @@ using namespace NRainCheck::NPrivate;
TCoroStack::TCoroStack(size_t size)
: SizeValue(size)
{
- Y_VERIFY(size % sizeof(ui32) == 0);
- Y_VERIFY(size >= 0x1000);
+ Y_ABORT_UNLESS(size % sizeof(ui32) == 0);
+ Y_ABORT_UNLESS(size >= 0x1000);
DataHolder.Reset(malloc(size));
diff --git a/library/cpp/messagebus/rain_check/core/simple_ut.cpp b/library/cpp/messagebus/rain_check/core/simple_ut.cpp
index d4545e05aa..d0afaf19fa 100644
--- a/library/cpp/messagebus/rain_check/core/simple_ut.cpp
+++ b/library/cpp/messagebus/rain_check/core/simple_ut.cpp
@@ -31,7 +31,7 @@ Y_UNIT_TEST_SUITE(RainCheckSimple) {
}
void SleepCompletionCallback(TSubtaskCompletion* completion) {
- Y_VERIFY(completion == &SleepCompletion);
+ Y_ABORT_UNLESS(completion == &SleepCompletion);
TestSync->CheckAndIncrement(1);
Env->SleepService.Sleep(&SleepCompletion, TDuration::MilliSeconds(1));
diff --git a/library/cpp/messagebus/rain_check/core/task.cpp b/library/cpp/messagebus/rain_check/core/task.cpp
index a098437d53..d8aeefac25 100644
--- a/library/cpp/messagebus/rain_check/core/task.cpp
+++ b/library/cpp/messagebus/rain_check/core/task.cpp
@@ -199,7 +199,7 @@ void TTaskRunnerBase::AssertInThisThread() const {
}
TTaskRunnerBase* TTaskRunnerBase::CurrentTask() {
- Y_VERIFY(!!ThreadCurrentTask);
+ Y_ABORT_UNLESS(!!ThreadCurrentTask);
return ThreadCurrentTask;
}
diff --git a/library/cpp/messagebus/rain_check/http/client.cpp b/library/cpp/messagebus/rain_check/http/client.cpp
index 5ef5ceeece..7ac5034e91 100644
--- a/library/cpp/messagebus/rain_check/http/client.cpp
+++ b/library/cpp/messagebus/rain_check/http/client.cpp
@@ -19,7 +19,7 @@ namespace NRainCheck {
THttpCallback(NRainCheck::THttpFuture* future)
: Future(future)
{
- Y_VERIFY(!!future, "future is NULL");
+ Y_ABORT_UNLESS(!!future, "future is NULL");
}
void OnRecv(NNeh::THandle& handle) override {
@@ -57,15 +57,15 @@ namespace NRainCheck {
: GetProtocol(NNeh::ProtocolFactory()->Protocol("http"))
, FullProtocol(NNeh::ProtocolFactory()->Protocol("full"))
{
- Y_VERIFY(!!GetProtocol, "GET protocol is NULL.");
- Y_VERIFY(!!FullProtocol, "POST protocol is NULL.");
+ Y_ABORT_UNLESS(!!GetProtocol, "GET protocol is NULL.");
+ Y_ABORT_UNLESS(!!FullProtocol, "POST protocol is NULL.");
}
THttpClientService::~THttpClientService() {
}
void THttpClientService::SendPost(TString addr, const TString& data, const THttpHeaders& headers, THttpFuture* future) {
- Y_VERIFY(!!future, "future is NULL.");
+ Y_ABORT_UNLESS(!!future, "future is NULL.");
TTaskRunnerBase* current = TTaskRunnerBase::CurrentTask();
future->SetRunning(current);
@@ -88,7 +88,7 @@ namespace NRainCheck {
}
void THttpClientService::Send(const TString& request, THttpFuture* future) {
- Y_VERIFY(!!future, "future is NULL.");
+ Y_ABORT_UNLESS(!!future, "future is NULL.");
TTaskRunnerBase* current = TTaskRunnerBase::CurrentTask();
future->SetRunning(current);
diff --git a/library/cpp/messagebus/rain_check/messagebus/messagebus_client_ut.cpp b/library/cpp/messagebus/rain_check/messagebus/messagebus_client_ut.cpp
index 1b3618558b..35bb7abd8e 100644
--- a/library/cpp/messagebus/rain_check/messagebus/messagebus_client_ut.cpp
+++ b/library/cpp/messagebus/rain_check/messagebus/messagebus_client_ut.cpp
@@ -58,7 +58,7 @@ Y_UNIT_TEST_SUITE(RainCheckMessageBusClient) {
TContinueFunc GotReplies() {
for (unsigned i = 0; i < Requests.size(); ++i) {
- Y_VERIFY(Requests[i]->GetStatus() == MESSAGE_OK);
+ Y_ABORT_UNLESS(Requests[i]->GetStatus() == MESSAGE_OK);
VerifyDynamicCast<TExampleResponse*>(Requests[i]->GetResponse());
}
Env->TestSync.CheckAndIncrement(0);
@@ -124,8 +124,8 @@ Y_UNIT_TEST_SUITE(RainCheckMessageBusClient) {
TContinueFunc GotReplies() {
for (unsigned i = 0; i < Requests.size(); ++i) {
- Y_VERIFY(Requests[i]->GetStatus() == MESSAGE_OK);
- Y_VERIFY(!Requests[i]->GetResponse());
+ Y_ABORT_UNLESS(Requests[i]->GetStatus() == MESSAGE_OK);
+ Y_ABORT_UNLESS(!Requests[i]->GetResponse());
}
Env->TestSync.WaitForAndIncrement(2);
return nullptr;
diff --git a/library/cpp/messagebus/rain_check/test/helper/misc.cpp b/library/cpp/messagebus/rain_check/test/helper/misc.cpp
index c0fcb27252..d940d68a4b 100644
--- a/library/cpp/messagebus/rain_check/test/helper/misc.cpp
+++ b/library/cpp/messagebus/rain_check/test/helper/misc.cpp
@@ -5,7 +5,7 @@
using namespace NRainCheck;
void TSpawnNopTasksCoroTask::Run() {
- Y_VERIFY(Count <= Completion.size());
+ Y_ABORT_UNLESS(Count <= Completion.size());
for (unsigned i = 0; i < Count; ++i) {
SpawnSubtask<TNopCoroTask>(Env, &Completion[i], "");
}
@@ -14,7 +14,7 @@ void TSpawnNopTasksCoroTask::Run() {
}
TContinueFunc TSpawnNopTasksSimpleTask::Start() {
- Y_VERIFY(Count <= Completion.size());
+ Y_ABORT_UNLESS(Count <= Completion.size());
for (unsigned i = 0; i < Count; ++i) {
SpawnSubtask<TNopSimpleTask>(Env, &Completion[i], "");
}
diff --git a/library/cpp/messagebus/remote_client_connection.cpp b/library/cpp/messagebus/remote_client_connection.cpp
index 8c7a6db3a8..30a7fc25fb 100644
--- a/library/cpp/messagebus/remote_client_connection.cpp
+++ b/library/cpp/messagebus/remote_client_connection.cpp
@@ -20,7 +20,7 @@ TRemoteClientConnection::TRemoteClientConnection(TRemoteClientSessionPtr session
: TRemoteConnection(session.Get(), id, addr)
, ClientHandler(GetSession()->ClientHandler)
{
- Y_VERIFY(addr.GetPort() > 0, "must connect to non-zero port");
+ Y_ABORT_UNLESS(addr.GetPort() > 0, "must connect to non-zero port");
ScheduleWrite();
}
@@ -35,7 +35,7 @@ TBusMessage* TRemoteClientConnection::PopAck(TBusKey id) {
SOCKET TRemoteClientConnection::CreateSocket(const TNetAddr& addr) {
SOCKET handle = socket(addr.Addr()->sa_family, SOCK_STREAM, 0);
- Y_VERIFY(handle != INVALID_SOCKET, "failed to create socket: %s", LastSystemErrorText());
+ Y_ABORT_UNLESS(handle != INVALID_SOCKET, "failed to create socket: %s", LastSystemErrorText());
TSocketHolder s(handle);
@@ -61,7 +61,7 @@ void TRemoteClientConnection::TryConnect() {
if (AtomicGet(WriterData.Down)) {
return;
}
- Y_VERIFY(!WriterData.Status.Connected);
+ Y_ABORT_UNLESS(!WriterData.Status.Connected);
TInstant now = TInstant::Now();
@@ -306,7 +306,7 @@ EMessageStatus TRemoteClientConnection::SendMessageImpl(TBusMessage* msg, bool w
}
if (wait) {
- Y_VERIFY(!Session->Queue->GetExecutor()->IsInExecutorThread());
+ Y_ABORT_UNLESS(!Session->Queue->GetExecutor()->IsInExecutorThread());
GetSession()->ClientRemoteInFlight.Wait();
} else {
if (!GetSession()->ClientRemoteInFlight.TryWait()) {
diff --git a/library/cpp/messagebus/remote_client_session_semaphore.cpp b/library/cpp/messagebus/remote_client_session_semaphore.cpp
index f877ed4257..ad37fae87c 100644
--- a/library/cpp/messagebus/remote_client_session_semaphore.cpp
+++ b/library/cpp/messagebus/remote_client_session_semaphore.cpp
@@ -12,12 +12,12 @@ TRemoteClientSessionSemaphore::TRemoteClientSessionSemaphore(TAtomicBase limit,
, Current(0)
, StopSignal(0)
{
- Y_VERIFY(limit > 0, "limit must be > 0");
+ Y_ABORT_UNLESS(limit > 0, "limit must be > 0");
Y_UNUSED(Name);
}
TRemoteClientSessionSemaphore::~TRemoteClientSessionSemaphore() {
- Y_VERIFY(AtomicGet(Current) == 0);
+ Y_ABORT_UNLESS(AtomicGet(Current) == 0);
}
bool TRemoteClientSessionSemaphore::TryAcquire() {
diff --git a/library/cpp/messagebus/remote_connection.cpp b/library/cpp/messagebus/remote_connection.cpp
index 740a7f6f8e..24c2153730 100644
--- a/library/cpp/messagebus/remote_connection.cpp
+++ b/library/cpp/messagebus/remote_connection.cpp
@@ -54,7 +54,7 @@ namespace NBus {
}
TRemoteConnection::~TRemoteConnection() {
- Y_VERIFY(ReplyQueue.IsEmpty());
+ Y_ABORT_UNLESS(ReplyQueue.IsEmpty());
}
TRemoteConnection::TWriterData::TWriterData()
@@ -67,8 +67,8 @@ namespace NBus {
}
TRemoteConnection::TWriterData::~TWriterData() {
- Y_VERIFY(AtomicGet(Down));
- Y_VERIFY(SendQueue.Empty());
+ Y_ABORT_UNLESS(AtomicGet(Down));
+ Y_ABORT_UNLESS(SendQueue.Empty());
}
bool TRemoteConnection::TReaderData::HasBytesInBuf(size_t bytes) noexcept {
@@ -78,15 +78,15 @@ namespace NBus {
}
void TRemoteConnection::TWriterData::SetChannel(NEventLoop::TChannelPtr channel) {
- Y_VERIFY(!Channel, "must not have channel");
- Y_VERIFY(Buffer.GetBuffer().Empty() && Buffer.LeftSize() == 0, "buffer must be empty");
- Y_VERIFY(State == WRITER_FILLING, "state must be initial");
+ Y_ABORT_UNLESS(!Channel, "must not have channel");
+ Y_ABORT_UNLESS(Buffer.GetBuffer().Empty() && Buffer.LeftSize() == 0, "buffer must be empty");
+ Y_ABORT_UNLESS(State == WRITER_FILLING, "state must be initial");
Channel = channel;
}
void TRemoteConnection::TReaderData::SetChannel(NEventLoop::TChannelPtr channel) {
- Y_VERIFY(!Channel, "must not have channel");
- Y_VERIFY(Buffer.Empty(), "buffer must be empty");
+ Y_ABORT_UNLESS(!Channel, "must not have channel");
+ Y_ABORT_UNLESS(Buffer.Empty(), "buffer must be empty");
Channel = channel;
}
@@ -119,7 +119,7 @@ namespace NBus {
}
TRemoteConnection::TReaderData::~TReaderData() {
- Y_VERIFY(AtomicGet(Down));
+ Y_ABORT_UNLESS(AtomicGet(Down));
}
void TRemoteConnection::Send(TNonDestroyingAutoPtr<TBusMessage> msg) {
@@ -197,13 +197,13 @@ namespace NBus {
}
void TRemoteConnection::ProcessItem(TWriterTag, TReconnectTag, ui32 socketVersion) {
- Y_VERIFY(socketVersion <= WriterData.SocketVersion, "something weird");
+ Y_ABORT_UNLESS(socketVersion <= WriterData.SocketVersion, "something weird");
if (WriterData.SocketVersion != socketVersion) {
return;
}
- Y_VERIFY(WriterData.Status.Connected, "must be connected at this point");
- Y_VERIFY(!!WriterData.Channel, "must have channel at this point");
+ Y_ABORT_UNLESS(WriterData.Status.Connected, "must be connected at this point");
+ Y_ABORT_UNLESS(!!WriterData.Channel, "must have channel at this point");
WriterData.Status.Connected = false;
WriterData.DropChannel();
@@ -670,7 +670,7 @@ namespace NBus {
void TRemoteConnection::CallSerialize(TBusMessage* msg, TBuffer& buffer) const {
size_t posForAssertion = buffer.Size();
Proto->Serialize(msg, buffer);
- Y_VERIFY(buffer.Size() >= posForAssertion,
+ Y_ABORT_UNLESS(buffer.Size() >= posForAssertion,
"incorrect Serialize implementation, pos before serialize: %d, pos after serialize: %d",
int(posForAssertion), int(buffer.Size()));
}
@@ -808,7 +808,7 @@ namespace NBus {
TBusMessage* r = DeserializeMessage(readDataRef, &header, &ReaderData.Status.Incremental.MessageCounter, &deserializeFailureStatus);
if (!r) {
- Y_VERIFY(deserializeFailureStatus != MESSAGE_OK, "state check");
+ Y_ABORT_UNLESS(deserializeFailureStatus != MESSAGE_OK, "state check");
LWPROBE(Error, ToString(deserializeFailureStatus), ToString(PeerAddr), "");
ReaderData.Status.Incremental.StatusCounter[deserializeFailureStatus] += 1;
ScheduleShutdownOnServerOrReconnectOnClient(deserializeFailureStatus, false);
@@ -960,7 +960,7 @@ namespace NBus {
}
void TRemoteConnection::FireClientConnectionEvent(TClientConnectionEvent::EType type) {
- Y_VERIFY(Session->IsSource_, "state check");
+ Y_ABORT_UNLESS(Session->IsSource_, "state check");
TClientConnectionEvent event(type, ConnectionId, PeerAddr);
TRemoteClientSession* session = CheckedCast<TRemoteClientSession*>(Session.Get());
session->ClientHandler->OnClientConnectionEvent(event);
diff --git a/library/cpp/messagebus/remote_server_connection.cpp b/library/cpp/messagebus/remote_server_connection.cpp
index 74be34ded9..cc2336d20d 100644
--- a/library/cpp/messagebus/remote_server_connection.cpp
+++ b/library/cpp/messagebus/remote_server_connection.cpp
@@ -19,7 +19,7 @@ void TRemoteServerConnection::Init(SOCKET socket, TInstant now) {
WriterData.Status.ConnectTime = now;
WriterData.Status.Connected = true;
- Y_VERIFY(socket != INVALID_SOCKET, "must be a valid socket");
+ Y_ABORT_UNLESS(socket != INVALID_SOCKET, "must be a valid socket");
TSocket readSocket(socket);
TSocket writeSocket = readSocket;
diff --git a/library/cpp/messagebus/remote_server_session.cpp b/library/cpp/messagebus/remote_server_session.cpp
index 6abbf88a60..98ca6436f6 100644
--- a/library/cpp/messagebus/remote_server_session.cpp
+++ b/library/cpp/messagebus/remote_server_session.cpp
@@ -201,6 +201,6 @@ void TRemoteServerSession::PauseInput(bool pause) {
}
unsigned TRemoteServerSession::GetActualListenPort() {
- Y_VERIFY(Config.ListenPort > 0, "state check");
+ Y_ABORT_UNLESS(Config.ListenPort > 0, "state check");
return Config.ListenPort;
}
diff --git a/library/cpp/messagebus/remote_server_session_semaphore.cpp b/library/cpp/messagebus/remote_server_session_semaphore.cpp
index 6094a3586e..ffea5fb85c 100644
--- a/library/cpp/messagebus/remote_server_session_semaphore.cpp
+++ b/library/cpp/messagebus/remote_server_session_semaphore.cpp
@@ -16,14 +16,14 @@ TRemoteServerSessionSemaphore::TRemoteServerSessionSemaphore(
, PausedByUser(0)
, StopSignal(0)
{
- Y_VERIFY(limitCount > 0, "limit must be > 0");
+ Y_ABORT_UNLESS(limitCount > 0, "limit must be > 0");
Y_UNUSED(Name);
}
TRemoteServerSessionSemaphore::~TRemoteServerSessionSemaphore() {
- Y_VERIFY(AtomicGet(CurrentCount) == 0);
+ Y_ABORT_UNLESS(AtomicGet(CurrentCount) == 0);
// TODO: fix spider and enable
- //Y_VERIFY(AtomicGet(CurrentSize) == 0);
+ //Y_ABORT_UNLESS(AtomicGet(CurrentSize) == 0);
}
bool TRemoteServerSessionSemaphore::TryWait() {
diff --git a/library/cpp/messagebus/scheduler/scheduler.cpp b/library/cpp/messagebus/scheduler/scheduler.cpp
index 5a5fe52894..149d12c41b 100644
--- a/library/cpp/messagebus/scheduler/scheduler.cpp
+++ b/library/cpp/messagebus/scheduler/scheduler.cpp
@@ -23,7 +23,7 @@ TScheduler::TScheduler()
}
TScheduler::~TScheduler() {
- Y_VERIFY(StopThread, "state check");
+ Y_ABORT_UNLESS(StopThread, "state check");
}
size_t TScheduler::Size() const {
@@ -34,7 +34,7 @@ size_t TScheduler::Size() const {
void TScheduler::Stop() {
{
TGuard<TLock> guard(Lock);
- Y_VERIFY(!StopThread, "Scheduler already stopped");
+ Y_ABORT_UNLESS(!StopThread, "Scheduler already stopped");
StopThread = true;
CondVar.Signal();
}
@@ -98,7 +98,7 @@ void TScheduler::SchedulerThread() {
}
// signal comes if either scheduler is to be stopped of there's work to do
- Y_VERIFY(!!NextItem, "state check");
+ Y_ABORT_UNLESS(!!NextItem, "state check");
if (TInstant::Now() < NextItem->GetScheduleTime()) {
// NextItem is updated since WaitD
diff --git a/library/cpp/messagebus/session_impl.cpp b/library/cpp/messagebus/session_impl.cpp
index ddf9f360c4..a3b480584c 100644
--- a/library/cpp/messagebus/session_impl.cpp
+++ b/library/cpp/messagebus/session_impl.cpp
@@ -76,10 +76,10 @@ namespace {
copy.ConnectTimeout = copy.SendTimeout;
}
- Y_VERIFY(copy.SendTimeout > 0, "SendTimeout must be > 0");
- Y_VERIFY(copy.TotalTimeout > 0, "TotalTimeout must be > 0");
- Y_VERIFY(copy.ConnectTimeout > 0, "ConnectTimeout must be > 0");
- Y_VERIFY(copy.TotalTimeout >= copy.SendTimeout, "TotalTimeout must be >= SendTimeout");
+ Y_ABORT_UNLESS(copy.SendTimeout > 0, "SendTimeout must be > 0");
+ Y_ABORT_UNLESS(copy.TotalTimeout > 0, "TotalTimeout must be > 0");
+ Y_ABORT_UNLESS(copy.ConnectTimeout > 0, "ConnectTimeout must be > 0");
+ Y_ABORT_UNLESS(copy.TotalTimeout >= copy.SendTimeout, "TotalTimeout must be >= SendTimeout");
if (!copy.Name) {
copy.Name = name;
@@ -117,10 +117,10 @@ TBusSessionImpl::TBusSessionImpl(bool isSource, TBusMessageQueue* queue, TBusPro
}
TBusSessionImpl::~TBusSessionImpl() {
- Y_VERIFY(Down);
- Y_VERIFY(ShutdownCompleteEvent.WaitT(TDuration::Zero()));
- Y_VERIFY(!WriteEventLoop.IsRunning());
- Y_VERIFY(!ReadEventLoop.IsRunning());
+ Y_ABORT_UNLESS(Down);
+ Y_ABORT_UNLESS(ShutdownCompleteEvent.WaitT(TDuration::Zero()));
+ Y_ABORT_UNLESS(!WriteEventLoop.IsRunning());
+ Y_ABORT_UNLESS(!ReadEventLoop.IsRunning());
}
TBusSessionStatus::TBusSessionStatus()
@@ -136,7 +136,7 @@ void TBusSessionImpl::Shutdown() {
return;
}
- Y_VERIFY(Queue->IsRunning(), "Session must be shut down prior to queue shutdown");
+ Y_ABORT_UNLESS(Queue->IsRunning(), "Session must be shut down prior to queue shutdown");
TUseAfterFreeCheckerGuard handlerAliveCheckedGuard(ErrorHandler->UseAfterFreeChecker);
@@ -205,7 +205,7 @@ size_t TBusSessionImpl::GetInFlightImpl(const TNetAddr& addr) const {
}
void TBusSessionImpl::GetInFlightBulk(TArrayRef<const TNetAddr> addrs, TArrayRef<size_t> results) const {
- Y_VERIFY(addrs.size() == results.size(), "input.size != output.size");
+ Y_ABORT_UNLESS(addrs.size() == results.size(), "input.size != output.size");
for (size_t i = 0; i < addrs.size(); ++i) {
results[i] = GetInFlightImpl(addrs[i]);
}
@@ -221,7 +221,7 @@ size_t TBusSessionImpl::GetConnectSyscallsNumForTestImpl(const TNetAddr& addr) c
}
void TBusSessionImpl::GetConnectSyscallsNumBulkForTest(TArrayRef<const TNetAddr> addrs, TArrayRef<size_t> results) const {
- Y_VERIFY(addrs.size() == results.size(), "input.size != output.size");
+ Y_ABORT_UNLESS(addrs.size() == results.size(), "input.size != output.size");
for (size_t i = 0; i < addrs.size(); ++i) {
results[i] = GetConnectSyscallsNumForTestImpl(addrs[i]);
}
@@ -232,7 +232,7 @@ void TBusSessionImpl::FillStatus() {
TSessionDumpStatus TBusSessionImpl::GetStatusRecordInternal() {
// Probably useless, because it returns cached info now
- Y_VERIFY(!Queue->GetExecutor()->IsInExecutorThread(),
+ Y_ABORT_UNLESS(!Queue->GetExecutor()->IsInExecutorThread(),
"GetStatus must not be called from executor thread");
TGuard<TMutex> guard(StatusData.StatusDumpCachedMutex);
@@ -248,7 +248,7 @@ TString TBusSessionImpl::GetStatus(ui16 flags) {
}
TConnectionStatusMonRecord TBusSessionImpl::GetStatusProtobuf() {
- Y_VERIFY(!Queue->GetExecutor()->IsInExecutorThread(),
+ Y_ABORT_UNLESS(!Queue->GetExecutor()->IsInExecutorThread(),
"GetStatus must not be called from executor thread");
TGuard<TMutex> guard(StatusData.StatusDumpCachedMutex);
@@ -471,8 +471,8 @@ void TBusSessionImpl::Act(TConnectionTag) {
EShutdownState shutdownState = ConnectionsData.ShutdownState.State.Get();
if (shutdownState == SS_SHUTDOWN_COMPLETE) {
- Y_VERIFY(GetRemoveConnectionQueue()->IsEmpty());
- Y_VERIFY(GetOnAcceptQueue()->IsEmpty());
+ Y_ABORT_UNLESS(GetRemoveConnectionQueue()->IsEmpty());
+ Y_ABORT_UNLESS(GetOnAcceptQueue()->IsEmpty());
}
GetRemoveConnectionQueue()->DequeueAllLikelyEmpty();
@@ -495,7 +495,7 @@ void TBusSessionImpl::Listen(const TVector<TBindResult>& bindTo, TBusMessageQueu
if (actualPort == -1) {
actualPort = br.Addr.GetPort();
} else {
- Y_VERIFY(actualPort == br.Addr.GetPort(), "state check");
+ Y_ABORT_UNLESS(actualPort == br.Addr.GetPort(), "state check");
}
if (Config.SocketToS >= 0) {
SetSocketToS(*br.Socket, &(br.Addr), Config.SocketToS);
@@ -531,7 +531,7 @@ void TBusSessionImpl::InsertConnectionLockAcquired(TRemoteConnection* connection
// after reconnect, if previous connections wasn't shutdown yet
bool inserted2 = ConnectionsById.insert(std::make_pair(connection->ConnectionId, connection)).second;
- Y_VERIFY(inserted2, "state check: must be inserted (2)");
+ Y_ABORT_UNLESS(inserted2, "state check: must be inserted (2)");
SendSnapshotToStatusActor();
}
@@ -614,7 +614,7 @@ TRemoteConnectionPtr TBusSessionImpl::GetConnection(const TBusSocketAddr& addr,
return TRemoteConnectionPtr();
}
- Y_VERIFY(IsSource_, "must be source");
+ Y_ABORT_UNLESS(IsSource_, "must be source");
TRemoteConnectionPtr c(new TRemoteClientConnection(VerifyDynamicCast<TRemoteClientSession*>(this), ++LastConnectionId, addr.ToNetAddr()));
InsertConnectionLockAcquired(c.Get());
diff --git a/library/cpp/messagebus/session_job_count.cpp b/library/cpp/messagebus/session_job_count.cpp
index 33322b1910..bdbf35c2b9 100644
--- a/library/cpp/messagebus/session_job_count.cpp
+++ b/library/cpp/messagebus/session_job_count.cpp
@@ -11,7 +11,7 @@ TBusSessionJobCount::TBusSessionJobCount()
}
TBusSessionJobCount::~TBusSessionJobCount() {
- Y_VERIFY(JobCount == 0, "must be 0 job count to destroy job");
+ Y_ABORT_UNLESS(JobCount == 0, "must be 0 job count to destroy job");
}
void TBusSessionJobCount::WaitForZero() {
diff --git a/library/cpp/messagebus/shutdown_state.cpp b/library/cpp/messagebus/shutdown_state.cpp
index a4e2bfa8b2..afe07890c6 100644
--- a/library/cpp/messagebus/shutdown_state.cpp
+++ b/library/cpp/messagebus/shutdown_state.cpp
@@ -3,11 +3,11 @@
#include <util/system/yassert.h>
void TAtomicShutdownState::ShutdownCommand() {
- Y_VERIFY(State.CompareAndSet(SS_RUNNING, SS_SHUTDOWN_COMMAND));
+ Y_ABORT_UNLESS(State.CompareAndSet(SS_RUNNING, SS_SHUTDOWN_COMMAND));
}
void TAtomicShutdownState::CompleteShutdown() {
- Y_VERIFY(State.CompareAndSet(SS_SHUTDOWN_COMMAND, SS_SHUTDOWN_COMPLETE));
+ Y_ABORT_UNLESS(State.CompareAndSet(SS_SHUTDOWN_COMMAND, SS_SHUTDOWN_COMPLETE));
ShutdownComplete.Signal();
}
@@ -16,5 +16,5 @@ bool TAtomicShutdownState::IsRunning() {
}
TAtomicShutdownState::~TAtomicShutdownState() {
- Y_VERIFY(SS_SHUTDOWN_COMPLETE == State.Get());
+ Y_ABORT_UNLESS(SS_SHUTDOWN_COMPLETE == State.Get());
}
diff --git a/library/cpp/messagebus/storage.cpp b/library/cpp/messagebus/storage.cpp
index efefc87340..402ec63c93 100644
--- a/library/cpp/messagebus/storage.cpp
+++ b/library/cpp/messagebus/storage.cpp
@@ -8,7 +8,7 @@ namespace NBus {
}
TTimedMessages::~TTimedMessages() {
- Y_VERIFY(Items.empty());
+ Y_ABORT_UNLESS(Items.empty());
}
void TTimedMessages::PushBack(TNonDestroyingAutoPtr<TBusMessage> m) {
@@ -64,8 +64,8 @@ namespace NBus {
}
TSyncAckMessages::~TSyncAckMessages() {
- Y_VERIFY(KeyToMessage.empty());
- Y_VERIFY(TimedItems.empty());
+ Y_ABORT_UNLESS(KeyToMessage.empty());
+ Y_ABORT_UNLESS(TimedItems.empty());
}
void TSyncAckMessages::Push(TBusMessagePtrAndHeader& m) {
@@ -77,7 +77,7 @@ namespace NBus {
TValue value = {m.MessagePtr.Release()};
std::pair<TKeyToMessage::iterator, bool> p = KeyToMessage.insert(TKeyToMessage::value_type(m.Header.Id, value));
- Y_VERIFY(p.second, "non-unique id; %s", value.Message->Describe().data());
+ Y_ABORT_UNLESS(p.second, "non-unique id; %s", value.Message->Describe().data());
TTimedItem item = {m.Header.Id, m.Header.SendTime};
TimedItems.push_back(item);
@@ -147,7 +147,7 @@ namespace NBus {
void TSyncAckMessages::RemoveAll(const TMessagesPtrs& messages) {
for (auto message : messages) {
TKeyToMessage::iterator it = KeyToMessage.find(message->GetHeader()->Id);
- Y_VERIFY(it != KeyToMessage.end(), "delete non-existent message");
+ Y_ABORT_UNLESS(it != KeyToMessage.end(), "delete non-existent message");
KeyToMessage.erase(it);
}
}
diff --git a/library/cpp/messagebus/synchandler.cpp b/library/cpp/messagebus/synchandler.cpp
index 8e891d66b3..daf7153489 100644
--- a/library/cpp/messagebus/synchandler.cpp
+++ b/library/cpp/messagebus/synchandler.cpp
@@ -76,7 +76,7 @@ public:
private:
void SignalResult(TBusSyncMessageData* data, TBusMessage* pReply, EMessageStatus status) const {
- Y_VERIFY(data, "Message data is set to NULL.");
+ Y_ABORT_UNLESS(data, "Message data is set to NULL.");
TGuard<TMutex> G(data->ReplyLock);
data->Reply = pReply;
data->ReplyStatus = status;
@@ -115,7 +115,7 @@ namespace NBus {
}
TBusMessage* SendSyncMessage(TBusMessage* pMessage, EMessageStatus& status, const TNetAddr* addr = nullptr) {
- Y_VERIFY(!Queue->GetExecutor()->IsInExecutorThread(),
+ Y_ABORT_UNLESS(!Queue->GetExecutor()->IsInExecutorThread(),
"SendSyncMessage must not be called from executor thread");
TBusMessage* reply = nullptr;
@@ -134,7 +134,7 @@ namespace NBus {
if (status == MESSAGE_OK) {
data->ReplyEvent.Wait(data->ReplyLock);
TBusSyncMessageData* rdata = static_cast<TBusSyncMessageData*>(pMessage->Data);
- Y_VERIFY(rdata == data.Get(), "Message data pointer should not be modified.");
+ Y_ABORT_UNLESS(rdata == data.Get(), "Message data pointer should not be modified.");
reply = rdata->Reply;
status = rdata->ReplyStatus;
}
diff --git a/library/cpp/messagebus/test/example/client/client.cpp b/library/cpp/messagebus/test/example/client/client.cpp
index 89b5f2c9be..8d0f0584e0 100644
--- a/library/cpp/messagebus/test/example/client/client.cpp
+++ b/library/cpp/messagebus/test/example/client/client.cpp
@@ -23,7 +23,7 @@ namespace NCalculator {
}
void OnReply(TAutoPtr<TBusMessage> request, TAutoPtr<TBusMessage> response0) override {
- Y_VERIFY(response0->GetHeader()->Type == TResponse::MessageType, "wrong response");
+ Y_ABORT_UNLESS(response0->GetHeader()->Type == TResponse::MessageType, "wrong response");
TResponse* response = VerifyDynamicCast<TResponse*>(response0.Get());
if (request->GetHeader()->Type == TRequestSum::MessageType) {
TRequestSum* requestSum = VerifyDynamicCast<TRequestSum*>(request.Get());
diff --git a/library/cpp/messagebus/test/helper/alloc_counter.h b/library/cpp/messagebus/test/helper/alloc_counter.h
index 7c4010714f..93972b52dd 100644
--- a/library/cpp/messagebus/test/helper/alloc_counter.h
+++ b/library/cpp/messagebus/test/helper/alloc_counter.h
@@ -16,6 +16,6 @@ public:
}
~TAllocCounter() {
- Y_VERIFY(AtomicDecrement(*CountPtr) >= 0, "released too many");
+ Y_ABORT_UNLESS(AtomicDecrement(*CountPtr) >= 0, "released too many");
}
};
diff --git a/library/cpp/messagebus/test/helper/example.cpp b/library/cpp/messagebus/test/helper/example.cpp
index 7c6d704042..1ef2809111 100644
--- a/library/cpp/messagebus/test/helper/example.cpp
+++ b/library/cpp/messagebus/test/helper/example.cpp
@@ -68,11 +68,11 @@ TExampleProtocol::~TExampleProtocol() {
// so it could be reported in test
return;
}
- Y_VERIFY(0 == AtomicGet(RequestCount), "protocol %s: must be 0 requests allocated, actually %d", GetService(), int(RequestCount));
- Y_VERIFY(0 == AtomicGet(ResponseCount), "protocol %s: must be 0 responses allocated, actually %d", GetService(), int(ResponseCount));
- Y_VERIFY(0 == AtomicGet(RequestCountDeserialized), "protocol %s: must be 0 requests deserialized allocated, actually %d", GetService(), int(RequestCountDeserialized));
- Y_VERIFY(0 == AtomicGet(ResponseCountDeserialized), "protocol %s: must be 0 responses deserialized allocated, actually %d", GetService(), int(ResponseCountDeserialized));
- Y_VERIFY(0 == AtomicGet(StartCount), "protocol %s: must be 0 start objects allocated, actually %d", GetService(), int(StartCount));
+ Y_ABORT_UNLESS(0 == AtomicGet(RequestCount), "protocol %s: must be 0 requests allocated, actually %d", GetService(), int(RequestCount));
+ Y_ABORT_UNLESS(0 == AtomicGet(ResponseCount), "protocol %s: must be 0 responses allocated, actually %d", GetService(), int(ResponseCount));
+ Y_ABORT_UNLESS(0 == AtomicGet(RequestCountDeserialized), "protocol %s: must be 0 requests deserialized allocated, actually %d", GetService(), int(RequestCountDeserialized));
+ Y_ABORT_UNLESS(0 == AtomicGet(ResponseCountDeserialized), "protocol %s: must be 0 responses deserialized allocated, actually %d", GetService(), int(ResponseCountDeserialized));
+ Y_ABORT_UNLESS(0 == AtomicGet(StartCount), "protocol %s: must be 0 start objects allocated, actually %d", GetService(), int(StartCount));
}
void TExampleProtocol::Serialize(const TBusMessage* message, TBuffer& buffer) {
@@ -277,5 +277,5 @@ void TExampleServer::OnMessage(TOnMessageContext& mess) {
status = mess.SendReplyMove(reply);
}
- Y_VERIFY(status == MESSAGE_OK, "failed to send reply: %s", ToString(status).data());
+ Y_ABORT_UNLESS(status == MESSAGE_OK, "failed to send reply: %s", ToString(status).data());
}
diff --git a/library/cpp/messagebus/test/helper/wait_for.h b/library/cpp/messagebus/test/helper/wait_for.h
index f09958d4c0..b8363900b8 100644
--- a/library/cpp/messagebus/test/helper/wait_for.h
+++ b/library/cpp/messagebus/test/helper/wait_for.h
@@ -10,5 +10,5 @@
Sleep(TDuration::MilliSeconds(1)); \
} \
/* TODO: use UNIT_ASSERT if in unittest thread */ \
- Y_VERIFY(condition, "condition failed after 10 seconds wait"); \
+ Y_ABORT_UNLESS(condition, "condition failed after 10 seconds wait"); \
} while (0)
diff --git a/library/cpp/messagebus/test/perftest/perftest.cpp b/library/cpp/messagebus/test/perftest/perftest.cpp
index 8489319278..3084c9e01c 100644
--- a/library/cpp/messagebus/test/perftest/perftest.cpp
+++ b/library/cpp/messagebus/test/perftest/perftest.cpp
@@ -150,7 +150,7 @@ TAutoPtr<TBusMessage> NewRequest() {
void CheckRequest(TPerftestRequest* request) {
const TString& data = request->Record.GetData();
for (size_t i = 0; i != data.size(); ++i) {
- Y_VERIFY(data.at(i) == '?', "must be question mark");
+ Y_ABORT_UNLESS(data.at(i) == '?', "must be question mark");
}
}
@@ -164,7 +164,7 @@ TAutoPtr<TPerftestResponse> NewResponse(TPerftestRequest* request) {
void CheckResponse(TPerftestResponse* response) {
const TString& data = response->Record.GetData();
for (size_t i = 0; i != data.size(); ++i) {
- Y_VERIFY(data.at(i) == '.', "must be dot");
+ Y_ABORT_UNLESS(data.at(i) == '.', "must be dot");
}
}
@@ -416,8 +416,8 @@ public:
: TPerftestServerCommon("server")
, TBusModule("fast")
{
- Y_VERIFY(CreatePrivateSessions(Bus.Get()), "failed to initialize dupdetect module");
- Y_VERIFY(StartInput(), "failed to start input");
+ Y_ABORT_UNLESS(CreatePrivateSessions(Bus.Get()), "failed to initialize dupdetect module");
+ Y_ABORT_UNLESS(StartInput(), "failed to start input");
}
~TPerftestUsingModule() override {
@@ -479,7 +479,7 @@ TVector<TNetAddr> ParseNodes(const TString nodes) {
for (int i = 0; i < int(numh); i++) {
const TNetworkAddress& networkAddress = ParseNetworkAddress(hosts[i].data());
- Y_VERIFY(networkAddress.Begin() != networkAddress.End(), "no addresses");
+ Y_ABORT_UNLESS(networkAddress.Begin() != networkAddress.End(), "no addresses");
r.push_back(TNetAddr(networkAddress, &*networkAddress.Begin()));
}
diff --git a/library/cpp/messagebus/test/perftest/simple_proto.cpp b/library/cpp/messagebus/test/perftest/simple_proto.cpp
index 19d6c15b9d..3e28413325 100644
--- a/library/cpp/messagebus/test/perftest/simple_proto.cpp
+++ b/library/cpp/messagebus/test/perftest/simple_proto.cpp
@@ -7,7 +7,7 @@
using namespace NBus;
void TSimpleProtocol::Serialize(const TBusMessage* mess, TBuffer& data) {
- Y_VERIFY(typeid(TSimpleMessage) == typeid(*mess));
+ Y_ABORT_UNLESS(typeid(TSimpleMessage) == typeid(*mess));
const TSimpleMessage* typed = static_cast<const TSimpleMessage*>(mess);
data.Append((const char*)&typed->Payload, 4);
}
diff --git a/library/cpp/messagebus/test/ut/messagebus_ut.cpp b/library/cpp/messagebus/test/ut/messagebus_ut.cpp
index 8d2f4aa444..8f6aa6a295 100644
--- a/library/cpp/messagebus/test/ut/messagebus_ut.cpp
+++ b/library/cpp/messagebus/test/ut/messagebus_ut.cpp
@@ -31,7 +31,7 @@ namespace {
}
void OnReply(TAutoPtr<TBusMessage> mess, TAutoPtr<TBusMessage> reply) override {
- Y_VERIFY(AtomicGet(SentCompleted), "must be completed");
+ Y_ABORT_UNLESS(AtomicGet(SentCompleted), "must be completed");
TExampleClient::OnReply(mess, reply);
@@ -138,7 +138,7 @@ Y_UNIT_TEST_SUITE(TMessageBusTests) {
void OnError(TAutoPtr<TBusMessage> message, EMessageStatus status) override {
Y_UNUSED(message);
- Y_VERIFY(status == MESSAGE_CONNECT_FAILED, "must be MESSAGE_CONNECT_FAILED, got %s", ToString(status).data());
+ Y_ABORT_UNLESS(status == MESSAGE_CONNECT_FAILED, "must be MESSAGE_CONNECT_FAILED, got %s", ToString(status).data());
TestSync.CheckAndIncrement((failures++) * 2);
}
@@ -159,7 +159,7 @@ Y_UNIT_TEST_SUITE(TMessageBusTests) {
status = client.Session->SendMessageAutoPtr(message, &noServerAddr);
}
- Y_VERIFY(status == MESSAGE_OK, "must be MESSAGE_OK, got %s", ToString(status).data());
+ Y_ABORT_UNLESS(status == MESSAGE_OK, "must be MESSAGE_OK, got %s", ToString(status).data());
if (count == 0) {
// lame way to wait until it is connected
@@ -266,7 +266,7 @@ Y_UNIT_TEST_SUITE(TMessageBusTests) {
TSystemEvent ErrorHappened;
void OnError(TAutoPtr<TBusMessage>, EMessageStatus status) override {
- Y_VERIFY(status == MESSAGE_CONNECT_FAILED || status == MESSAGE_TIMEOUT, "got status: %s", ToString(status).data());
+ Y_ABORT_UNLESS(status == MESSAGE_CONNECT_FAILED || status == MESSAGE_TIMEOUT, "got status: %s", ToString(status).data());
ErrorHappened.Signal();
}
};
@@ -327,7 +327,7 @@ Y_UNIT_TEST_SUITE(TMessageBusTests) {
}
void OnMessage(TOnMessageContext& mess) override {
- Y_VERIFY(mess.IsConnectionAlive(), "connection should be alive here");
+ Y_ABORT_UNLESS(mess.IsConnectionAlive(), "connection should be alive here");
TAutoPtr<TOnMessageContext> delayedMsg(new TOnMessageContext);
delayedMsg->Swap(mess);
auto g(Guard(Lock_));
@@ -378,7 +378,7 @@ Y_UNIT_TEST_SUITE(TMessageBusTests) {
void OnError(TAutoPtr<TBusMessage> mess, EMessageStatus status) override {
Y_UNUSED(mess);
- Y_VERIFY(status == MESSAGE_SHUTDOWN, "only shutdown allowed, got %s", ToString(status).data());
+ Y_ABORT_UNLESS(status == MESSAGE_SHUTDOWN, "only shutdown allowed, got %s", ToString(status).data());
}
};
@@ -432,7 +432,7 @@ Y_UNIT_TEST_SUITE(TMessageBusTests) {
void OnError(TAutoPtr<TBusMessage> mess, EMessageStatus status) override {
Y_UNUSED(mess);
- Y_VERIFY(status == MESSAGE_SHUTDOWN, "only shutdown allowed");
+ Y_ABORT_UNLESS(status == MESSAGE_SHUTDOWN, "only shutdown allowed");
}
};
@@ -505,7 +505,7 @@ Y_UNIT_TEST_SUITE(TMessageBusTests) {
void OnError(TAutoPtr<TBusMessage>, EMessageStatus status) override {
TestSync.WaitForAndIncrement(1);
- Y_VERIFY(status == MESSAGE_MESSAGE_TOO_LARGE, "status");
+ Y_ABORT_UNLESS(status == MESSAGE_MESSAGE_TOO_LARGE, "status");
}
};
@@ -697,7 +697,7 @@ Y_UNIT_TEST_SUITE(TMessageBusTests) {
void OnError(TAutoPtr<TBusMessage> mess, EMessageStatus status) override {
TestSync.WaitForAndIncrement(0);
- Y_VERIFY(status == MESSAGE_CONNECT_FAILED || status == MESSAGE_TIMEOUT, "must be connection failed, got %s", ToString(status).data());
+ Y_ABORT_UNLESS(status == MESSAGE_CONNECT_FAILED || status == MESSAGE_TIMEOUT, "must be connection failed, got %s", ToString(status).data());
mess.Destroy();
TestSync.CheckAndIncrement(1);
}
@@ -726,7 +726,7 @@ Y_UNIT_TEST_SUITE(TMessageBusTests) {
void OnError(TAutoPtr<TBusMessage> message, EMessageStatus status) override {
TestSync.CheckAndIncrement(0);
- Y_VERIFY(status == MESSAGE_CONNECT_FAILED, "must be MESSAGE_CONNECT_FAILED, got %s", ToString(status).data());
+ Y_ABORT_UNLESS(status == MESSAGE_CONNECT_FAILED, "must be MESSAGE_CONNECT_FAILED, got %s", ToString(status).data());
// check reset is possible here
message->Reset();
@@ -755,7 +755,7 @@ Y_UNIT_TEST_SUITE(TMessageBusTests) {
client.Session->Shutdown();
ok = client.Session->SendMessageOneWay(message);
- Y_VERIFY(ok == MESSAGE_SHUTDOWN, "must be shutdown when sending during shutdown, got %s", ToString(ok).data());
+ Y_ABORT_UNLESS(ok == MESSAGE_SHUTDOWN, "must be shutdown when sending during shutdown, got %s", ToString(ok).data());
// check reset is possible here
message->Reset();
@@ -1074,7 +1074,7 @@ Y_UNIT_TEST_SUITE(TMessageBusTests) {
EMessageStatus status = client.Session->SendMessageOneWay(new TExampleRequest(&client.Proto.RequestCount),
&noServerAddr);
- Y_VERIFY(status == MESSAGE_OK, "must be MESSAGE_OK, got %s", ToString(status).data());
+ Y_ABORT_UNLESS(status == MESSAGE_OK, "must be MESSAGE_OK, got %s", ToString(status).data());
client.TestSync.WaitForAndIncrement(count * 2 + 1);
// First connection attempt is for connect call; second one is to get connect result.
@@ -1085,7 +1085,7 @@ Y_UNIT_TEST_SUITE(TMessageBusTests) {
EMessageStatus status = client.Session->SendMessageOneWay(new TExampleRequest(&client.Proto.RequestCount),
&noServerAddr);
- Y_VERIFY(status == MESSAGE_OK, "must be MESSAGE_OK, got %s", ToString(status).data());
+ Y_ABORT_UNLESS(status == MESSAGE_OK, "must be MESSAGE_OK, got %s", ToString(status).data());
client.TestSync.WaitForAndIncrement(count * 2 + 1);
// First connection attempt is for connect call; second one is to get connect result.
@@ -1107,7 +1107,7 @@ Y_UNIT_TEST_SUITE(TMessageBusTests) {
EMessageStatus status = client.Session->SendMessageOneWay(new TExampleRequest(&client.Proto.RequestCount),
&noServerAddr);
- Y_VERIFY(status == MESSAGE_OK, "must be MESSAGE_OK, got %s", ToString(status).data());
+ Y_ABORT_UNLESS(status == MESSAGE_OK, "must be MESSAGE_OK, got %s", ToString(status).data());
client.TestSync.WaitForAndIncrement(count * 2 + 1);
// First connection attempt is for connect call; second one is to get connect result.
@@ -1134,7 +1134,7 @@ Y_UNIT_TEST_SUITE(TMessageBusTests) {
EMessageStatus status = client.Session->SendMessageOneWay(new TExampleRequest(&client.Proto.RequestCount),
&noServerAddr);
- Y_VERIFY(status == MESSAGE_OK, "must be MESSAGE_OK, got %s", ToString(status).data());
+ Y_ABORT_UNLESS(status == MESSAGE_OK, "must be MESSAGE_OK, got %s", ToString(status).data());
client.TestSync.WaitForAndIncrement(count * 2 + 1);
// First connection attempt is for connect call; second one is to get connect result.
diff --git a/library/cpp/messagebus/test/ut/module_client_ut.cpp b/library/cpp/messagebus/test/ut/module_client_ut.cpp
index ebfe185cc6..a172b06bd2 100644
--- a/library/cpp/messagebus/test/ut/module_client_ut.cpp
+++ b/library/cpp/messagebus/test/ut/module_client_ut.cpp
@@ -120,13 +120,13 @@ Y_UNIT_TEST_SUITE(BusJobTest) {
void ReplyHandler(TBusJob*, EMessageStatus status, TBusMessage* mess, TBusMessage* reply) {
Y_UNUSED(mess);
Y_UNUSED(reply);
- Y_VERIFY(status == MESSAGE_OK, "failed to get reply: %s", ToCString(status));
+ Y_ABORT_UNLESS(status == MESSAGE_OK, "failed to get reply: %s", ToCString(status));
}
TJobHandler HandleReplies(TBusJob* job, TBusMessage* mess) {
Y_UNUSED(mess);
RepliesLatch.CountDown();
- Y_VERIFY(RepliesLatch.Await(TDuration::Seconds(10)), "failed to get answers");
+ Y_ABORT_UNLESS(RepliesLatch.Await(TDuration::Seconds(10)), "failed to get answers");
job->Cancel(MESSAGE_UNKNOWN);
return nullptr;
}
@@ -178,9 +178,9 @@ Y_UNIT_TEST_SUITE(BusJobTest) {
}
void ReplyHandler(TBusJob*, EMessageStatus status, TBusMessage* req, TBusMessage* resp) {
- Y_VERIFY(status == MESSAGE_CONNECT_FAILED || status == MESSAGE_TIMEOUT, "got wrong status: %s", ToString(status).data());
- Y_VERIFY(req == SentMessage, "checking request");
- Y_VERIFY(resp == nullptr, "checking response");
+ Y_ABORT_UNLESS(status == MESSAGE_CONNECT_FAILED || status == MESSAGE_TIMEOUT, "got wrong status: %s", ToString(status).data());
+ Y_ABORT_UNLESS(req == SentMessage, "checking request");
+ Y_ABORT_UNLESS(resp == nullptr, "checking response");
GotReplyLatch.CountDown();
}
@@ -279,7 +279,7 @@ Y_UNIT_TEST_SUITE(BusJobTest) {
void ReplyHandler(TBusJob* job, EMessageStatus status, TBusMessage* mess, TBusMessage* reply) {
Y_UNUSED(mess);
Y_UNUSED(reply);
- Y_VERIFY(status == MESSAGE_OK, "failed to get reply");
+ Y_ABORT_UNLESS(status == MESSAGE_OK, "failed to get reply");
if (AtomicIncrement(ReplyCount) == 1) {
TestSync->WaitForAndIncrement(1);
job->SendReply(new TExampleResponse(&Proto.ResponseCount));
@@ -338,7 +338,7 @@ Y_UNIT_TEST_SUITE(BusJobTest) {
}
void HandleReply(TBusJob*, EMessageStatus status, TBusMessage*, TBusMessage*) {
- Y_VERIFY(status == MESSAGE_SHUTDOWN, "got %s", ToCString(status));
+ Y_ABORT_UNLESS(status == MESSAGE_SHUTDOWN, "got %s", ToCString(status));
TestSync.CheckAndIncrement(1);
}
diff --git a/library/cpp/messagebus/test/ut/module_server_ut.cpp b/library/cpp/messagebus/test/ut/module_server_ut.cpp
index 88fe1dd9b6..b0e6ac38da 100644
--- a/library/cpp/messagebus/test/ut/module_server_ut.cpp
+++ b/library/cpp/messagebus/test/ut/module_server_ut.cpp
@@ -49,7 +49,7 @@ Y_UNIT_TEST_SUITE(ModuleServerTests) {
TJobHandler Start(TBusJob* job, TBusMessage* mess) override {
WaitTwoRequestsLatch.CountDown();
- Y_VERIFY(WaitTwoRequestsLatch.Await(TDuration::Seconds(5)), "oops");
+ Y_ABORT_UNLESS(WaitTwoRequestsLatch.Await(TDuration::Seconds(5)), "oops");
VerifyDynamicCast<TExampleRequest*>(mess);
@@ -83,7 +83,7 @@ Y_UNIT_TEST_SUITE(ModuleServerTests) {
MessageReceivedEvent.Signal();
- Y_VERIFY(ClientDiedEvent.WaitT(TDuration::Seconds(5)), "oops");
+ Y_ABORT_UNLESS(ClientDiedEvent.WaitT(TDuration::Seconds(5)), "oops");
job->SendReply(new TExampleResponse(&Proto.ResponseCount));
return nullptr;
diff --git a/library/cpp/messagebus/test/ut/one_way_ut.cpp b/library/cpp/messagebus/test/ut/one_way_ut.cpp
index 9c21227e2b..0b90985289 100644
--- a/library/cpp/messagebus/test/ut/one_way_ut.cpp
+++ b/library/cpp/messagebus/test/ut/one_way_ut.cpp
@@ -168,7 +168,7 @@ Y_UNIT_TEST_SUITE(TMessageBusTests_OneWay) {
void OnError(TAutoPtr<TBusMessage> mess, EMessageStatus status) override {
Y_UNUSED(mess);
- Y_VERIFY(status == MESSAGE_MESSAGE_TOO_LARGE, "wrong status: %s", ToCString(status));
+ Y_ABORT_UNLESS(status == MESSAGE_MESSAGE_TOO_LARGE, "wrong status: %s", ToCString(status));
GotTooLarge.Signal();
}
diff --git a/library/cpp/messagebus/use_after_free_checker.cpp b/library/cpp/messagebus/use_after_free_checker.cpp
index 4904e7c614..0b35fcbdfe 100644
--- a/library/cpp/messagebus/use_after_free_checker.cpp
+++ b/library/cpp/messagebus/use_after_free_checker.cpp
@@ -13,10 +13,10 @@ TUseAfterFreeChecker::TUseAfterFreeChecker()
}
TUseAfterFreeChecker::~TUseAfterFreeChecker() {
- Y_VERIFY(Magic == VALID, "Corrupted");
+ Y_ABORT_UNLESS(Magic == VALID, "Corrupted");
Magic = INVALID;
}
void TUseAfterFreeChecker::CheckNotFreed() const {
- Y_VERIFY(Magic == VALID, "Freed or corrupted");
+ Y_ABORT_UNLESS(Magic == VALID, "Freed or corrupted");
}
diff --git a/library/cpp/messagebus/use_count_checker.cpp b/library/cpp/messagebus/use_count_checker.cpp
index 32eee10431..ae18d9f994 100644
--- a/library/cpp/messagebus/use_count_checker.cpp
+++ b/library/cpp/messagebus/use_count_checker.cpp
@@ -8,7 +8,7 @@ TUseCountChecker::TUseCountChecker() {
TUseCountChecker::~TUseCountChecker() {
auto count = Counter.Val();
- Y_VERIFY(count == 0, "must not release when count is not zero: %ld", (long)count);
+ Y_ABORT_UNLESS(count == 0, "must not release when count is not zero: %ld", (long)count);
}
void TUseCountChecker::Inc() {
diff --git a/library/cpp/messagebus/vector_swaps.h b/library/cpp/messagebus/vector_swaps.h
index b920bcf03e..f0d23c6078 100644
--- a/library/cpp/messagebus/vector_swaps.h
+++ b/library/cpp/messagebus/vector_swaps.h
@@ -122,7 +122,7 @@ public:
size_t newCapacity = FastClp2(n);
TVectorSwaps<T> tmp;
tmp.Start = (T*)malloc(sizeof(T) * newCapacity);
- Y_VERIFY(!!tmp.Start);
+ Y_ABORT_UNLESS(!!tmp.Start);
tmp.EndOfStorage = tmp.Start + newCapacity;
@@ -146,7 +146,7 @@ public:
template <class TIterator>
void insert(iterator pos, TIterator b, TIterator e) {
- Y_VERIFY(pos == end(), "TODO: only insert at the end is implemented");
+ Y_ABORT_UNLESS(pos == end(), "TODO: only insert at the end is implemented");
size_t count = e - b;
diff --git a/library/cpp/messagebus/www/html_output.h b/library/cpp/messagebus/www/html_output.h
index 27e77adefa..cf9cba831d 100644
--- a/library/cpp/messagebus/www/html_output.h
+++ b/library/cpp/messagebus/www/html_output.h
@@ -10,7 +10,7 @@
extern Y_POD_THREAD(IOutputStream*) HtmlOutputStreamPtr;
static IOutputStream& HtmlOutputStream() {
- Y_VERIFY(!!HtmlOutputStreamPtr);
+ Y_ABORT_UNLESS(!!HtmlOutputStreamPtr);
return *HtmlOutputStreamPtr;
}
diff --git a/library/cpp/messagebus/www/www.cpp b/library/cpp/messagebus/www/www.cpp
index 6c8b4bc732..a52ad404ff 100644
--- a/library/cpp/messagebus/www/www.cpp
+++ b/library/cpp/messagebus/www/www.cpp
@@ -38,7 +38,7 @@ namespace {
TVector<std::pair<TString, TValuePtr>> Entries;
TValuePtr FindByName(TStringBuf name) {
- Y_VERIFY(!!name);
+ Y_ABORT_UNLESS(!!name);
for (unsigned i = 0; i < Entries.size(); ++i) {
if (Entries[i].first == name) {
@@ -49,7 +49,7 @@ namespace {
}
TString FindNameByPtr(TValuePtr value) {
- Y_VERIFY(!!value);
+ Y_ABORT_UNLESS(!!value);
for (unsigned i = 0; i < Entries.size(); ++i) {
if (Entries[i].second.Get() == value.Get()) {
@@ -61,7 +61,7 @@ namespace {
}
void Add(TValuePtr p) {
- Y_VERIFY(!!p);
+ Y_ABORT_UNLESS(!!p);
// Do not add twice
for (unsigned i = 0; i < Entries.size(); ++i) {
@@ -175,27 +175,27 @@ struct TBusWww::TImpl {
TMutex Mutex;
void RegisterClientSession(TBusClientSessionPtr s) {
- Y_VERIFY(!!s);
+ Y_ABORT_UNLESS(!!s);
TGuard<TMutex> g(Mutex);
ClientSessions.Add(s.Get());
Queues.Add(s->GetQueue());
}
void RegisterServerSession(TBusServerSessionPtr s) {
- Y_VERIFY(!!s);
+ Y_ABORT_UNLESS(!!s);
TGuard<TMutex> g(Mutex);
ServerSessions.Add(s.Get());
Queues.Add(s->GetQueue());
}
void RegisterQueue(TBusMessageQueuePtr q) {
- Y_VERIFY(!!q);
+ Y_ABORT_UNLESS(!!q);
TGuard<TMutex> g(Mutex);
Queues.Add(q);
}
void RegisterModule(TBusModule* module) {
- Y_VERIFY(!!module);
+ Y_ABORT_UNLESS(!!module);
TGuard<TMutex> g(Mutex);
{
@@ -227,7 +227,7 @@ struct TBusWww::TImpl {
serverSession = ServerSessions.FindByName(sessionName);
session = serverSession.Get();
}
- Y_VERIFY(!!session);
+ Y_ABORT_UNLESS(!!session);
return Queues.FindNameByPtr(session->GetQueue());
}
@@ -831,7 +831,7 @@ struct TBusWwwHttpServer::TImpl: public THttpServer::ICallBack {
THttpServer HttpServer;
static THttpServer::TOptions MakeHttpServerOptions(unsigned port) {
- Y_VERIFY(port > 0);
+ Y_ABORT_UNLESS(port > 0);
THttpServer::TOptions r;
r.Port = port;
return r;
diff --git a/library/cpp/monlib/dynamic_counters/contention_ut.cpp b/library/cpp/monlib/dynamic_counters/contention_ut.cpp
index 8798044ee3..166b345250 100644
--- a/library/cpp/monlib/dynamic_counters/contention_ut.cpp
+++ b/library/cpp/monlib/dynamic_counters/contention_ut.cpp
@@ -47,7 +47,7 @@ Y_UNIT_TEST_SUITE(TDynamicCountersContentionTest) {
// acts like a coroutine
Ev.Wait();
auto ctr = Counters->GetSubgroup("label", "value")->GetCounter("name");
- Y_VERIFY(*ctr == 42);
+ Y_ABORT_UNLESS(*ctr == 42);
Response.Signal();
}
};
diff --git a/library/cpp/monlib/dynamic_counters/counters.cpp b/library/cpp/monlib/dynamic_counters/counters.cpp
index 8cc6f6e835..09455be2d7 100644
--- a/library/cpp/monlib/dynamic_counters/counters.cpp
+++ b/library/cpp/monlib/dynamic_counters/counters.cpp
@@ -121,7 +121,7 @@ void TDynamicCounters::RemoveSubgroupChain(const std::vector<std::pair<TString,
const auto& [name, value] = chain[i];
auto& base = basePointers.back();
basePointers.push_back(base->GetSubgroup(name, value));
- Y_VERIFY(basePointers.back());
+ Y_ABORT_UNLESS(basePointers.back());
}
for (size_t i = chain.size(); i-- && basePointers[i]->RemoveSubgroup(chain[i].first, chain[i].second); ) {}
}
@@ -158,16 +158,16 @@ bool TDynamicCounters::RemoveSubgroup(const TString& name, const TString& value)
void TDynamicCounters::ReplaceSubgroup(const TString& name, const TString& value, TIntrusivePtr<TDynamicCounters> subgroup) {
auto g = LockForUpdate("ReplaceSubgroup", name, value);
const auto it = Counters.find({name, value});
- Y_VERIFY(it != Counters.end() && AsDynamicCounters(it->second));
+ Y_ABORT_UNLESS(it != Counters.end() && AsDynamicCounters(it->second));
it->second = std::move(subgroup);
}
void TDynamicCounters::MergeWithSubgroup(const TString& name, const TString& value) {
auto g = LockForUpdate("MergeWithSubgroup", name, value);
auto it = Counters.find({name, value});
- Y_VERIFY(it != Counters.end());
+ Y_ABORT_UNLESS(it != Counters.end());
TIntrusivePtr<TDynamicCounters> subgroup = AsDynamicCounters(it->second);
- Y_VERIFY(subgroup);
+ Y_ABORT_UNLESS(subgroup);
Counters.erase(it);
Counters.merge(subgroup->Resign());
AtomicAdd(ExpiringCount, AtomicSwap(&subgroup->ExpiringCount, 0));
@@ -187,10 +187,10 @@ void TDynamicCounters::ResetCounters(bool derivOnly) {
}
void TDynamicCounters::RegisterCountable(const TString& name, const TString& value, TCountablePtr countable) {
- Y_VERIFY(countable);
+ Y_ABORT_UNLESS(countable);
auto g = LockForUpdate("RegisterCountable", name, value);
const bool inserted = Counters.emplace(TChildId(name, value), std::move(countable)).second;
- Y_VERIFY(inserted);
+ Y_ABORT_UNLESS(inserted);
}
void TDynamicCounters::RegisterSubgroup(const TString& name, const TString& value, TIntrusivePtr<TDynamicCounters> subgroup) {
diff --git a/library/cpp/monlib/metrics/ewma.cpp b/library/cpp/monlib/metrics/ewma.cpp
index 8a296c3225..e8ed49131b 100644
--- a/library/cpp/monlib/metrics/ewma.cpp
+++ b/library/cpp/monlib/metrics/ewma.cpp
@@ -19,7 +19,7 @@ namespace {
, Alpha_{alpha}
, Interval_{interval.Seconds()}
{
- Y_VERIFY(metric != nullptr, "Passing nullptr metric is not allowed");
+ Y_ABORT_UNLESS(metric != nullptr, "Passing nullptr metric is not allowed");
}
~TExpMovingAverage() override = default;
diff --git a/library/cpp/monlib/service/monservice.cpp b/library/cpp/monlib/service/monservice.cpp
index d1b9cda1d2..45986a62d2 100644
--- a/library/cpp/monlib/service/monservice.cpp
+++ b/library/cpp/monlib/service/monservice.cpp
@@ -23,7 +23,7 @@ TMonService2::TMonService2(const THttpServerOptions& options, const TString& tit
, IndexMonPage(new TIndexMonPage("", Title))
, AuthProvider_{std::move(auth)}
{
- Y_VERIFY(!!title);
+ Y_ABORT_UNLESS(!!title);
time_t t = time(nullptr);
ctime_r(&t, StartTime);
}
@@ -34,7 +34,7 @@ TMonService2::TMonService2(const THttpServerOptions& options, TSimpleSharedPtr<I
, IndexMonPage(new TIndexMonPage("", Title))
, AuthProvider_{std::move(auth)}
{
- Y_VERIFY(!!title);
+ Y_ABORT_UNLESS(!!title);
time_t t = time(nullptr);
ctime_r(&t, StartTime);
}
@@ -79,7 +79,7 @@ void TMonService2::OutputIndexBody(IOutputStream& out) {
void TMonService2::ServeRequest(IOutputStream& out, const NMonitoring::IHttpRequest& request) {
TString path = request.GetPath();
- Y_VERIFY(path.StartsWith('/'));
+ Y_ABORT_UNLESS(path.StartsWith('/'));
if (AuthProvider_) {
const auto authResult = AuthProvider_->Check(request);
diff --git a/library/cpp/monlib/service/pages/index_mon_page.cpp b/library/cpp/monlib/service/pages/index_mon_page.cpp
index c9b2f82cc0..87cc149146 100644
--- a/library/cpp/monlib/service/pages/index_mon_page.cpp
+++ b/library/cpp/monlib/service/pages/index_mon_page.cpp
@@ -20,7 +20,7 @@ void TIndexMonPage::Output(IMonHttpRequest& request) {
return;
}
- Y_VERIFY(pathInfo.StartsWith('/'));
+ Y_ABORT_UNLESS(pathInfo.StartsWith('/'));
TMonPagePtr found;
// analogous to CGI PATH_INFO
@@ -31,11 +31,11 @@ void TIndexMonPage::Output(IMonHttpRequest& request) {
if (TPagesByPath::iterator i = PagesByPath.find(pathTmp); i != PagesByPath.end()) {
found = *i->second;
pathInfo = request.GetPathInfo().substr(pathTmp.size());
- Y_VERIFY(pathInfo.empty() || pathInfo.StartsWith('/'));
+ Y_ABORT_UNLESS(pathInfo.empty() || pathInfo.StartsWith('/'));
break;
}
size_t slash = pathTmp.find_last_of('/');
- Y_VERIFY(slash != TString::npos);
+ Y_ABORT_UNLESS(slash != TString::npos);
pathTmp = pathTmp.substr(0, slash);
if (!pathTmp) {
break;
diff --git a/library/cpp/monlib/service/pages/mon_page.cpp b/library/cpp/monlib/service/pages/mon_page.cpp
index 72033b1699..929c54b084 100644
--- a/library/cpp/monlib/service/pages/mon_page.cpp
+++ b/library/cpp/monlib/service/pages/mon_page.cpp
@@ -6,8 +6,8 @@ IMonPage::IMonPage(const TString& path, const TString& title)
: Path(path)
, Title(title)
{
- Y_VERIFY(!Path.StartsWith('/'));
- Y_VERIFY(!Path.EndsWith('/'));
+ Y_ABORT_UNLESS(!Path.StartsWith('/'));
+ Y_ABORT_UNLESS(!Path.EndsWith('/'));
}
void IMonPage::OutputNavBar(IOutputStream& out) {
diff --git a/library/cpp/neh/http2.cpp b/library/cpp/neh/http2.cpp
index feaa45c5c9..a685195edf 100644
--- a/library/cpp/neh/http2.cpp
+++ b/library/cpp/neh/http2.cpp
@@ -1760,7 +1760,7 @@ namespace {
TAtomicBase oldReqId;
do {
oldReqId = AtomicGet(PrimaryResponse_);
- Y_VERIFY(oldReqId, "race inside http pipelining");
+ Y_ABORT_UNLESS(oldReqId, "race inside http pipelining");
} while (!AtomicCas(&PrimaryResponse_, requestId, oldReqId));
ProcessResponsesData();
@@ -1768,7 +1768,7 @@ namespace {
TAtomicBase oldReqId = AtomicGet(PrimaryResponse_);
if (oldReqId) {
while (!AtomicCas(&PrimaryResponse_, 0, oldReqId)) {
- Y_VERIFY(oldReqId == AtomicGet(PrimaryResponse_), "race inside http pipelining [2]");
+ Y_ABORT_UNLESS(oldReqId == AtomicGet(PrimaryResponse_), "race inside http pipelining [2]");
}
}
}
diff --git a/library/cpp/neh/https.cpp b/library/cpp/neh/https.cpp
index a5b0920d97..f1a2ec9e7e 100644
--- a/library/cpp/neh/https.cpp
+++ b/library/cpp/neh/https.cpp
@@ -1871,7 +1871,7 @@ namespace NNeh {
}
void SetHttpOutputConnectionsLimits(size_t softLimit, size_t hardLimit) {
- Y_VERIFY(
+ Y_ABORT_UNLESS(
hardLimit > softLimit,
"invalid output fd limits; hardLimit=%" PRISZT ", softLimit=%" PRISZT,
hardLimit, softLimit);
@@ -1880,7 +1880,7 @@ namespace NNeh {
}
void SetHttpInputConnectionsLimits(size_t softLimit, size_t hardLimit) {
- Y_VERIFY(
+ Y_ABORT_UNLESS(
hardLimit > softLimit,
"invalid output fd limits; hardLimit=%" PRISZT ", softLimit=%" PRISZT,
hardLimit, softLimit);
@@ -1889,7 +1889,7 @@ namespace NNeh {
}
void SetHttpInputConnectionsTimeouts(unsigned minSec, unsigned maxSec) {
- Y_VERIFY(
+ Y_ABORT_UNLESS(
maxSec > minSec,
"invalid input fd limits timeouts; maxSec=%u, minSec=%u",
maxSec, minSec);
diff --git a/library/cpp/neh/netliba.cpp b/library/cpp/neh/netliba.cpp
index f69906f3ba..1f46ed0c1e 100644
--- a/library/cpp/neh/netliba.cpp
+++ b/library/cpp/neh/netliba.cpp
@@ -188,7 +188,7 @@ namespace {
UpdateInFly();
TInFly::iterator it = InFly_.find(resp->ReqId);
- Y_VERIFY(it != InFly_.end(), "incorrect incoming message");
+ Y_ABORT_UNLESS(it != InFly_.end(), "incorrect incoming message");
TRequestRef& req = it->second;
@@ -220,7 +220,7 @@ namespace {
UpdateInFly();
TInFly::iterator it = InFly_.find(guid);
- Y_VERIFY(it != InFly_.end(), "incorrect complete notification");
+ Y_ABORT_UNLESS(it != InFly_.end(), "incorrect complete notification");
it->second->OnRequestAck();
}
diff --git a/library/cpp/neh/netliba_udp_http.cpp b/library/cpp/neh/netliba_udp_http.cpp
index a4df426f02..1e89347a13 100644
--- a/library/cpp/neh/netliba_udp_http.cpp
+++ b/library/cpp/neh/netliba_udp_http.cpp
@@ -282,7 +282,7 @@ namespace NNehNetliba {
}
void SendRequest(const TUdpAddress& addr, const TString& url, const TString& data, const TGUID& reqId) override {
- Y_VERIFY(
+ Y_ABORT_UNLESS(
data.size() < MAX_PACKET_SIZE,
"data size is too large; data.size()=%" PRISZT ", MAX_PACKET_SIZE=%" PRISZT,
data.size(), MAX_PACKET_SIZE);
diff --git a/library/cpp/netliba/socket/socket.cpp b/library/cpp/netliba/socket/socket.cpp
index c10236229b..ca8a64d25c 100644
--- a/library/cpp/netliba/socket/socket.cpp
+++ b/library/cpp/netliba/socket/socket.cpp
@@ -188,24 +188,24 @@ namespace NNetlibaSocket {
}
{
int flag = 0;
- Y_VERIFY(SetSockOpt(IPPROTO_IPV6, IPV6_V6ONLY, (const char*)&flag, sizeof(flag)) == 0, "IPV6_V6ONLY failed");
+ Y_ABORT_UNLESS(SetSockOpt(IPPROTO_IPV6, IPV6_V6ONLY, (const char*)&flag, sizeof(flag)) == 0, "IPV6_V6ONLY failed");
}
{
int flag = 1;
- Y_VERIFY(SetSockOpt(SOL_SOCKET, SO_REUSEADDR, (const char*)&flag, sizeof(flag)) == 0, "SO_REUSEADDR failed");
+ Y_ABORT_UNLESS(SetSockOpt(SOL_SOCKET, SO_REUSEADDR, (const char*)&flag, sizeof(flag)) == 0, "SO_REUSEADDR failed");
}
#if defined(_win_)
unsigned long dummy = 1;
ioctlsocket(S, FIONBIO, &dummy);
#else
- Y_VERIFY(fcntl(S, F_SETFL, O_NONBLOCK) == 0, "fnctl failed: %s (errno = %d)", LastSystemErrorText(), LastSystemError());
- Y_VERIFY(fcntl(S, F_SETFD, FD_CLOEXEC) == 0, "fnctl failed: %s (errno = %d)", LastSystemErrorText(), LastSystemError());
+ Y_ABORT_UNLESS(fcntl(S, F_SETFL, O_NONBLOCK) == 0, "fnctl failed: %s (errno = %d)", LastSystemErrorText(), LastSystemError());
+ Y_ABORT_UNLESS(fcntl(S, F_SETFD, FD_CLOEXEC) == 0, "fnctl failed: %s (errno = %d)", LastSystemErrorText(), LastSystemError());
{
int flag = 1;
#ifndef IPV6_RECVPKTINFO /* Darwin platforms require this */
- Y_VERIFY(SetSockOpt(IPPROTO_IPV6, IPV6_PKTINFO, (const char*)&flag, sizeof(flag)) == 0, "IPV6_PKTINFO failed");
+ Y_ABORT_UNLESS(SetSockOpt(IPPROTO_IPV6, IPV6_PKTINFO, (const char*)&flag, sizeof(flag)) == 0, "IPV6_PKTINFO failed");
#else
- Y_VERIFY(SetSockOpt(IPPROTO_IPV6, IPV6_RECVPKTINFO, (const char*)&flag, sizeof(flag)) == 0, "IPV6_RECVPKTINFO failed");
+ Y_ABORT_UNLESS(SetSockOpt(IPPROTO_IPV6, IPV6_RECVPKTINFO, (const char*)&flag, sizeof(flag)) == 0, "IPV6_RECVPKTINFO failed");
#endif
}
#endif
@@ -280,17 +280,17 @@ namespace NNetlibaSocket {
#if defined(_win_)
DWORD flag = 0;
socklen_t sz = sizeof(flag);
- Y_VERIFY(GetSockOpt(IPPROTO_IP, IP_DONTFRAGMENT, (char*)&flag, &sz) == 0, "");
+ Y_ABORT_UNLESS(GetSockOpt(IPPROTO_IP, IP_DONTFRAGMENT, (char*)&flag, &sz) == 0, "");
return flag;
#elif defined(_linux_)
int flag = 0;
socklen_t sz = sizeof(flag);
- Y_VERIFY(GetSockOpt(IPPROTO_IPV6, IPV6_MTU_DISCOVER, (char*)&flag, &sz) == 0, "");
+ Y_ABORT_UNLESS(GetSockOpt(IPPROTO_IPV6, IPV6_MTU_DISCOVER, (char*)&flag, &sz) == 0, "");
return flag == IPV6_PMTUDISC_DO;
#elif !defined(_darwin_)
int flag = 0;
socklen_t sz = sizeof(flag);
- Y_VERIFY(GetSockOpt(IPPROTO_IPV6, IPV6_DONTFRAG, (char*)&flag, &sz) == 0, "");
+ Y_ABORT_UNLESS(GetSockOpt(IPPROTO_IPV6, IPV6_DONTFRAG, (char*)&flag, &sz) == 0, "");
return flag;
#endif
return false;
@@ -346,10 +346,10 @@ namespace NNetlibaSocket {
int TAbstractSocket::SendMMsg(TMMsgHdr* msgvec, unsigned int vlen, unsigned int flags) {
Y_ASSERT(IsValid());
- Y_VERIFY(SendMMsgFunc, "sendmmsg is not supported!");
+ Y_ABORT_UNLESS(SendMMsgFunc, "sendmmsg is not supported!");
TReadGuard rg(Mutex);
static bool checked = 0;
- Y_VERIFY(checked || (checked = !IsFragmentationForbiden()), "Send methods of this class expect default EnableFragmentation behavior");
+ Y_ABORT_UNLESS(checked || (checked = !IsFragmentationForbiden()), "Send methods of this class expect default EnableFragmentation behavior");
return SendMMsgFunc(S, msgvec, vlen, flags);
}
@@ -357,23 +357,23 @@ namespace NNetlibaSocket {
Y_ASSERT(IsValid());
#ifdef _win32_
static bool checked = 0;
- Y_VERIFY(hdr->msg_iov->iov_len == 1, "Scatter/gather is currenly not supported on Windows");
+ Y_ABORT_UNLESS(hdr->msg_iov->iov_len == 1, "Scatter/gather is currenly not supported on Windows");
if (hdr->Tos || frag == FF_DONT_FRAG) {
TWriteGuard wg(Mutex);
if (frag == FF_DONT_FRAG) {
ForbidFragmentation();
} else {
- Y_VERIFY(checked || (checked = !IsFragmentationForbiden()), "Send methods of this class expect default EnableFragmentation behavior");
+ Y_ABORT_UNLESS(checked || (checked = !IsFragmentationForbiden()), "Send methods of this class expect default EnableFragmentation behavior");
}
int originalTos;
if (hdr->Tos) {
socklen_t sz = sizeof(originalTos);
- Y_VERIFY(GetSockOpt(IPPROTO_IP, IP_TOS, (char*)&originalTos, &sz) == 0, "");
- Y_VERIFY(SetSockOpt(IPPROTO_IP, IP_TOS, (char*)&hdr->Tos, sizeof(hdr->Tos)) == 0, "");
+ Y_ABORT_UNLESS(GetSockOpt(IPPROTO_IP, IP_TOS, (char*)&originalTos, &sz) == 0, "");
+ Y_ABORT_UNLESS(SetSockOpt(IPPROTO_IP, IP_TOS, (char*)&hdr->Tos, sizeof(hdr->Tos)) == 0, "");
}
const ssize_t rv = sendto(S, hdr->msg_iov->iov_base, hdr->msg_iov->iov_len, flags, (sockaddr*)hdr->msg_name, hdr->msg_namelen);
if (hdr->Tos) {
- Y_VERIFY(SetSockOpt(IPPROTO_IP, IP_TOS, (char*)&originalTos, sizeof(originalTos)) == 0, "");
+ Y_ABORT_UNLESS(SetSockOpt(IPPROTO_IP, IP_TOS, (char*)&originalTos, sizeof(originalTos)) == 0, "");
}
if (frag == FF_DONT_FRAG) {
EnableFragmentation();
@@ -381,7 +381,7 @@ namespace NNetlibaSocket {
return rv;
}
TReadGuard rg(Mutex);
- Y_VERIFY(checked || (checked = !IsFragmentationForbiden()), "Send methods of this class expect default EnableFragmentation behavior");
+ Y_ABORT_UNLESS(checked || (checked = !IsFragmentationForbiden()), "Send methods of this class expect default EnableFragmentation behavior");
return sendto(S, hdr->msg_iov->iov_base, hdr->msg_iov->iov_len, flags, (sockaddr*)hdr->msg_name, hdr->msg_namelen);
#else
if (frag == FF_DONT_FRAG) {
@@ -395,7 +395,7 @@ namespace NNetlibaSocket {
TReadGuard rg(Mutex);
#ifndef _darwin_
static bool checked = 0;
- Y_VERIFY(checked || (checked = !IsFragmentationForbiden()), "Send methods of this class expect default EnableFragmentation behavior");
+ Y_ABORT_UNLESS(checked || (checked = !IsFragmentationForbiden()), "Send methods of this class expect default EnableFragmentation behavior");
#endif
return sendmsg(S, hdr, flags);
#endif
@@ -501,13 +501,13 @@ namespace NNetlibaSocket {
void TAbstractSocket::CloseImpl() {
if (IsValid()) {
Poller.Unwait(S);
- Y_VERIFY(closesocket(S) == 0, "closesocket failed: %s (errno = %d)", LastSystemErrorText(), LastSystemError());
+ Y_ABORT_UNLESS(closesocket(S) == 0, "closesocket failed: %s (errno = %d)", LastSystemErrorText(), LastSystemError());
}
S = INVALID_SOCKET;
}
void TAbstractSocket::WaitImpl(float timeoutSec) const {
- Y_VERIFY(IsValid(), "something went wrong");
+ Y_ABORT_UNLESS(IsValid(), "something went wrong");
Poller.WaitT(TDuration::Seconds(timeoutSec));
}
@@ -526,7 +526,7 @@ namespace NNetlibaSocket {
Y_ASSERT(IsValid());
#ifdef _win32_
- Y_VERIFY(hdr->msg_iov->iov_len == 1, "Scatter/gather is currenly not supported on Windows");
+ Y_ABORT_UNLESS(hdr->msg_iov->iov_len == 1, "Scatter/gather is currenly not supported on Windows");
return recvfrom(S, hdr->msg_iov->iov_base, hdr->msg_iov->iov_len, flags, (sockaddr*)hdr->msg_name, &hdr->msg_namelen);
#else
return recvmsg(S, hdr, flags);
@@ -559,7 +559,7 @@ namespace NNetlibaSocket {
// thread-safe
int TAbstractSocket::RecvMMsgImpl(TMMsgHdr* msgvec, unsigned int vlen, unsigned int flags, timespec* timeout) {
Y_ASSERT(IsValid());
- Y_VERIFY(RecvMMsgFunc, "recvmmsg is not supported!");
+ Y_ABORT_UNLESS(RecvMMsgFunc, "recvmmsg is not supported!");
return RecvMMsgFunc(S, msgvec, vlen, flags, timeout);
}
@@ -648,7 +648,7 @@ namespace NNetlibaSocket {
ssize_t RecvMsg(TMsgHdr* hdr, int flags) override {
Y_UNUSED(hdr);
Y_UNUSED(flags);
- Y_VERIFY(false, "Use TBasicSocket for RecvMsg call! TRecvMMsgSocket implementation must use memcpy which is suboptimal and thus forbidden!");
+ Y_ABORT_UNLESS(false, "Use TBasicSocket for RecvMsg call! TRecvMMsgSocket implementation must use memcpy which is suboptimal and thus forbidden!");
}
TUdpRecvPacket* Recv(sockaddr_in6* addr, sockaddr_in6* dstAddr, int netlibaVersion) override;
};
@@ -842,7 +842,7 @@ public:
AtomicSwap(&NumThreadsToDie, (int)RecvThreads.size());
CancelWaitImpl();
- Y_VERIFY(AllThreadsAreDead.WaitT(TDuration::Seconds(30)), "TMTRecvSocket destruction failed");
+ Y_ABORT_UNLESS(AllThreadsAreDead.WaitT(TDuration::Seconds(30)), "TMTRecvSocket destruction failed");
CloseImpl();
}
@@ -869,7 +869,7 @@ public:
}
bool IsRecvMsgSupported() const { return false; }
- ssize_t RecvMsg(TMsgHdr* hdr, int flags) { Y_VERIFY(false, "Use TBasicSocket for RecvMsg call! TMTRecvSocket implementation must use memcpy which is suboptimal and thus forbidden!"); }
+ ssize_t RecvMsg(TMsgHdr* hdr, int flags) { Y_ABORT_UNLESS(false, "Use TBasicSocket for RecvMsg call! TMTRecvSocket implementation must use memcpy which is suboptimal and thus forbidden!"); }
};
*/
@@ -939,7 +939,7 @@ public:
ssize_t RecvMsg(TMsgHdr* hdr, int flags) override {
Y_UNUSED(hdr);
Y_UNUSED(flags);
- Y_VERIFY(false, "Use TBasicSocket for RecvMsg call! TDualStackSocket implementation must use memcpy which is suboptimal and thus forbidden!");
+ Y_ABORT_UNLESS(false, "Use TBasicSocket for RecvMsg call! TDualStackSocket implementation must use memcpy which is suboptimal and thus forbidden!");
}
TUdpRecvPacket* Recv(sockaddr_in6* addr, sockaddr_in6* dstAddr, int netlibaVersion) override;
@@ -989,7 +989,7 @@ public:
AtomicSwap(&ShouldDie, 1);
CancelWaitImpl();
- Y_VERIFY(DieEvent.WaitT(TDuration::Seconds(30)), "TDualStackSocket::Close failed");
+ Y_ABORT_UNLESS(DieEvent.WaitT(TDuration::Seconds(30)), "TDualStackSocket::Close failed");
TBase::Close();
}
diff --git a/library/cpp/netliba/v6/ib_buffers.h b/library/cpp/netliba/v6/ib_buffers.h
index 96d83ff654..8847250137 100644
--- a/library/cpp/netliba/v6/ib_buffers.h
+++ b/library/cpp/netliba/v6/ib_buffers.h
@@ -42,7 +42,7 @@ namespace NNetliba {
void AddBlock() {
if (FirstFreeBlock == Blocks.size()) {
- Y_VERIFY(0, "run out of buffers");
+ Y_ABORT_UNLESS(0, "run out of buffers");
}
Blocks[FirstFreeBlock].Alloc(IBCtx);
size_t start = (FirstFreeBlock == 0) ? 1 : FirstFreeBlock * BLOCK_SIZE;
@@ -95,7 +95,7 @@ namespace NNetliba {
}
int PostSend(TPtrArg<TRCQueuePair> qp, const void* data, size_t len) {
if (len > SMALL_PKT_SIZE) {
- Y_VERIFY(0, "buffer overrun");
+ Y_ABORT_UNLESS(0, "buffer overrun");
}
if (len <= MAX_INLINE_DATA_SIZE) {
qp->PostSend(nullptr, 0, data, len);
@@ -112,7 +112,7 @@ namespace NNetliba {
void PostSend(TPtrArg<TUDQueuePair> qp, TPtrArg<TAddressHandle> ah, int remoteQPN, int remoteQKey,
const void* data, size_t len) {
if (len > SMALL_PKT_SIZE - 40) {
- Y_VERIFY(0, "buffer overrun");
+ Y_ABORT_UNLESS(0, "buffer overrun");
}
ui64 id = AllocBuf();
TSingleBlock& blk = Blocks[id >> BLOCK_SIZE_LN];
diff --git a/library/cpp/netliba/v6/ib_collective.cpp b/library/cpp/netliba/v6/ib_collective.cpp
index 87ea166366..af5dd6d284 100644
--- a/library/cpp/netliba/v6/ib_collective.cpp
+++ b/library/cpp/netliba/v6/ib_collective.cpp
@@ -73,7 +73,7 @@ namespace NNetliba {
Ops.resize(colSize);
}
void Transfer(int srcRank, int dstRank, int sl, int rangeBeg, int rangeFin, int id) {
- Y_VERIFY(id < 64, "recv mask overflow");
+ Y_ABORT_UNLESS(id < 64, "recv mask overflow");
Ops[srcRank].OutList.push_back(TMergeRecord::TTransfer(dstRank, sl, rangeBeg, rangeFin, id));
Ops[dstRank].InList.push_back(TMergeRecord::TInTransfer(srcRank, sl));
Ops[dstRank].RecvMask |= ui64(1) << id;
@@ -188,7 +188,7 @@ namespace NNetliba {
for (int k = 1; k < groupSize; ++k) {
int h1 = myGroup[k - 1];
int h2 = myGroup[k];
- Y_VERIFY(hostCoverage[h1].Fin == hostCoverage[h2].Beg, "Invalid host order in CreateGroupMerge()");
+ Y_ABORT_UNLESS(hostCoverage[h1].Fin == hostCoverage[h2].Beg, "Invalid host order in CreateGroupMerge()");
}
switch (mode) {
@@ -303,12 +303,12 @@ namespace NNetliba {
if (cur == prev + 1) {
isIncrement = false;
} else {
- Y_VERIFY(cur == 0, "ib_hosts, wrapped to non-zero");
- Y_VERIFY(prev == gcount[groupType] - 1, "ib_hosts, structure is irregular");
+ Y_ABORT_UNLESS(cur == 0, "ib_hosts, wrapped to non-zero");
+ Y_ABORT_UNLESS(prev == gcount[groupType] - 1, "ib_hosts, structure is irregular");
isIncrement = true;
}
} else {
- Y_VERIFY(prev == cur, "ib_hosts, structure is irregular");
+ Y_ABORT_UNLESS(prev == cur, "ib_hosts, structure is irregular");
}
}
}
@@ -333,7 +333,7 @@ namespace NNetliba {
if (newIter == 0) {
newIter = nn;
} else {
- Y_VERIFY(newIter == nn, "groups should be of same size");
+ Y_ABORT_UNLESS(newIter == nn, "groups should be of same size");
}
}
baseIter = newIter;
@@ -429,13 +429,13 @@ namespace NNetliba {
while ((recvMask & iter.RecvMask) != iter.RecvMask) {
int rv = CQ->Poll(&wc, 1);
if (rv > 0) {
- Y_VERIFY(wc.status == IBV_WC_SUCCESS, "AllGather::Sync fail, status %d", (int)wc.status);
+ Y_ABORT_UNLESS(wc.status == IBV_WC_SUCCESS, "AllGather::Sync fail, status %d", (int)wc.status);
if (wc.opcode == IBV_WC_RECV_RDMA_WITH_IMM) {
//printf("Got %d\n", wc.imm_data);
++recvDebt;
ui64 newBit = ui64(1) << wc.imm_data;
if (recvMask & newBit) {
- Y_VERIFY((FutureRecvMask & newBit) == 0, "data from 2 Sync() ahead is impossible");
+ Y_ABORT_UNLESS((FutureRecvMask & newBit) == 0, "data from 2 Sync() ahead is impossible");
FutureRecvMask |= newBit;
} else {
recvMask |= newBit;
@@ -604,7 +604,7 @@ namespace NNetliba {
}
bool Resize(const TVector<size_t>& szPerRank) override {
- Y_VERIFY(szPerRank.ysize() == ColSize, "Invalid size array");
+ Y_ABORT_UNLESS(szPerRank.ysize() == ColSize, "Invalid size array");
TVector<size_t> offsets;
offsets.push_back(0);
@@ -732,7 +732,7 @@ namespace NNetliba {
ibv_wc wc;
int rv = CQ->Poll(&wc, 1);
if (rv > 0) {
- Y_VERIFY(wc.status == IBV_WC_SUCCESS, "WaitForMsg() fail, status %d", (int)wc.status);
+ Y_ABORT_UNLESS(wc.status == IBV_WC_SUCCESS, "WaitForMsg() fail, status %d", (int)wc.status);
if (wc.opcode & IBV_WC_RECV) {
BP->RequestPostRecv();
if (tbl->NeedQPN(wc.qp_num)) {
@@ -753,7 +753,7 @@ namespace NNetliba {
}
bool ProcessSendCompletion(const ibv_wc& wc) {
- Y_VERIFY(wc.status == IBV_WC_SUCCESS, "WaitForMsg() fail, status %d", (int)wc.status);
+ Y_ABORT_UNLESS(wc.status == IBV_WC_SUCCESS, "WaitForMsg() fail, status %d", (int)wc.status);
if (wc.opcode & IBV_WC_RECV) {
BP->RequestPostRecv();
Pending.push_back(TPendingMessage(wc.qp_num, wc.wr_id));
@@ -806,7 +806,7 @@ namespace NNetliba {
for (;;) {
int rv = CQ->Poll(&wc, 1);
if (rv > 0) {
- Y_VERIFY(wc.status == IBV_WC_SUCCESS, "WaitForMsg() fail, status %d", (int)wc.status);
+ Y_ABORT_UNLESS(wc.status == IBV_WC_SUCCESS, "WaitForMsg() fail, status %d", (int)wc.status);
if (wc.opcode & IBV_WC_RECV) {
BP->RequestPostRecv();
if ((int)wc.qp_num == qpn) {
@@ -957,7 +957,7 @@ namespace NNetliba {
IAllGather* CreateAllGather(const TVector<size_t>& szPerRank) override {
const TMergePlan& plan = MergePlan;
- Y_VERIFY(szPerRank.ysize() == ColSize, "Invalid size array");
+ Y_ABORT_UNLESS(szPerRank.ysize() == ColSize, "Invalid size array");
size_t totalSize = 0;
for (int i = 0; i < szPerRank.ysize(); ++i) {
@@ -1004,7 +1004,7 @@ namespace NNetliba {
iter.RecvMask = rr.RecvMask;
}
bool rv = res->Resize(szPerRank);
- Y_VERIFY(rv, "oops");
+ Y_ABORT_UNLESS(rv, "oops");
return res;
}
@@ -1087,7 +1087,7 @@ namespace NNetliba {
res->ReadyOffsetMult = currentDataOffset;
bool rv = res->Resize(dataSize);
- Y_VERIFY(rv, "oops");
+ Y_ABORT_UNLESS(rv, "oops");
return res;
}
diff --git a/library/cpp/netliba/v6/ib_cs.cpp b/library/cpp/netliba/v6/ib_cs.cpp
index 1b904e3358..93fc4ef67e 100644
--- a/library/cpp/netliba/v6/ib_cs.cpp
+++ b/library/cpp/netliba/v6/ib_cs.cpp
@@ -200,7 +200,7 @@ namespace NNetliba {
return z;
}
}
- Y_VERIFY(0, "no send by guid");
+ Y_ABORT_UNLESS(0, "no send by guid");
return SendQueue.begin();
}
TDeque<TQueuedSend>::iterator GetSend(TIBMsgHandle msgHandle) {
@@ -209,7 +209,7 @@ namespace NNetliba {
return z;
}
}
- Y_VERIFY(0, "no send by handle");
+ Y_ABORT_UNLESS(0, "no send by handle");
return SendQueue.begin();
}
TDeque<TQueuedRecv>::iterator GetRecv(const TGUID& packetGuid) {
@@ -218,7 +218,7 @@ namespace NNetliba {
return z;
}
}
- Y_VERIFY(0, "no recv by guid");
+ Y_ABORT_UNLESS(0, "no recv by guid");
return RecvQueue.begin();
}
void PostRDMA(TQueuedSend& qs) {
@@ -354,7 +354,7 @@ namespace NNetliba {
//printf("Remove peer %p from hash (QPN %d)\n", peer.Get(), peer->QP->GetQPN());
TPeerChannelHash::iterator z = Channels.find(peer->QP->GetQPN());
if (z == Channels.end()) {
- Y_VERIFY(0, "peer failed for unregistered peer");
+ Y_ABORT_UNLESS(0, "peer failed for unregistered peer");
}
Channels.erase(z);
}
@@ -447,7 +447,7 @@ namespace NNetliba {
TDeque<TQueuedRecv>::iterator z = peer->GetRecv(cmd.PacketGuid);
TQueuedRecv& qr = *z;
#ifdef _DEBUG
- Y_VERIFY(MurmurHash<ui64>(qr.Data->GetData(), qr.Data->GetSize()) == cmd.DataHash || cmd.DataHash == 0, "RDMA data hash mismatch");
+ Y_ABORT_UNLESS(MurmurHash<ui64>(qr.Data->GetData(), qr.Data->GetSize()) == cmd.DataHash || cmd.DataHash == 0, "RDMA data hash mismatch");
#endif
TRequest* req = new TRequest;
req->Address = peer->PeerAddress;
@@ -525,12 +525,12 @@ namespace NNetliba {
BP.FreeBuf(wc->wr_id);
}
} else {
- Y_VERIFY(0, "got completion without outstanding messages");
+ Y_ABORT_UNLESS(0, "got completion without outstanding messages");
}
} else {
//printf("Got completion for non existing qpn %d, bufId %d (status %d)\n", wc->qp_num, (int)wc->wr_id, (int)wc->status);
if (wc->status == IBV_WC_SUCCESS) {
- Y_VERIFY(0, "only errors should go unmatched");
+ Y_ABORT_UNLESS(0, "only errors should go unmatched");
}
// no need to free buf since it has to be freed in PeerFailed()
}
@@ -617,7 +617,7 @@ namespace NNetliba {
// received msg
if ((int)wc.qp_num == WelcomeQPN) {
if (wc.status != IBV_WC_SUCCESS) {
- Y_VERIFY(0, "ud recv op completed with error %d\n", (int)wc.status);
+ Y_ABORT_UNLESS(0, "ud recv op completed with error %d\n", (int)wc.status);
}
Y_ASSERT(wc.opcode == IBV_WC_RECV | IBV_WC_SEND);
ParseWelcomePacket(&wc);
@@ -645,7 +645,7 @@ namespace NNetliba {
}
TDeque<TQueuedSend>::iterator z = peer->GetSend(msgHandle);
if (z == peer->SendQueue.end()) {
- Y_VERIFY(0, "peer %p, copy completed, msg %d not found?\n", peer.Get(), (int)msgHandle);
+ Y_ABORT_UNLESS(0, "peer %p, copy completed, msg %d not found?\n", peer.Get(), (int)msgHandle);
continue;
}
TQueuedSend& qs = *z;
diff --git a/library/cpp/netliba/v6/ib_low.h b/library/cpp/netliba/v6/ib_low.h
index e3afd9df2e..42e11ed941 100644
--- a/library/cpp/netliba/v6/ib_low.h
+++ b/library/cpp/netliba/v6/ib_low.h
@@ -13,7 +13,7 @@ namespace NNetliba {
int rv = (x); \
if (rv != 0) { \
fprintf(stderr, "check_z failed, errno = %d\n", errno); \
- Y_VERIFY(0, "check_z"); \
+ Y_ABORT_UNLESS(0, "check_z"); \
} \
}
@@ -167,7 +167,7 @@ namespace NNetliba {
//};
int rv = ibv_poll_cq(CQ, bufSize, res);
if (rv < 0) {
- Y_VERIFY(0, "ibv_poll_cq failed");
+ Y_ABORT_UNLESS(0, "ibv_poll_cq failed");
}
if (rv > 0) {
//printf("Completed wr\n");
diff --git a/library/cpp/netliba/v6/udp_client_server.cpp b/library/cpp/netliba/v6/udp_client_server.cpp
index 3eaf6e5e96..e7cde45242 100644
--- a/library/cpp/netliba/v6/udp_client_server.cpp
+++ b/library/cpp/netliba/v6/udp_client_server.cpp
@@ -438,14 +438,14 @@ namespace NNetliba {
if (IB->GetSendResult(&sr)) {
TIBtoTransferKeyHash::iterator z = IBKeyToTransferKey.find(sr.Handle);
if (z == IBKeyToTransferKey.end()) {
- Y_VERIFY(0, "unknown handle returned from IB");
+ Y_ABORT_UNLESS(0, "unknown handle returned from IB");
}
TTransferKey transferKey = z->second;
IBKeyToTransferKey.erase(z);
TUdpOutXferHash::iterator i = SendQueue.find(transferKey);
if (i == SendQueue.end()) {
- Y_VERIFY(0, "IBKeyToTransferKey refers nonexisting xfer");
+ Y_ABORT_UNLESS(0, "IBKeyToTransferKey refers nonexisting xfer");
}
if (sr.Success) {
TUdpOutTransfer& xfer = i->second;
@@ -1034,7 +1034,7 @@ namespace NNetliba {
case ACK_RESEND_NOSHMEM: {
// abort execution here
// failed to open shmem on recv side, need to transmit data without using shmem
- Y_VERIFY(0, "not implemented yet");
+ Y_ABORT_UNLESS(0, "not implemented yet");
break;
}
case PING: {
diff --git a/library/cpp/netliba/v6/udp_http.cpp b/library/cpp/netliba/v6/udp_http.cpp
index 9fa0b07818..f318554a76 100644
--- a/library/cpp/netliba/v6/udp_http.cpp
+++ b/library/cpp/netliba/v6/udp_http.cpp
@@ -829,7 +829,7 @@ namespace NNetliba {
void SendRequestImpl(const TUdpAddress& addr, const TString& url, TVector<char>* data, const TGUID& reqId,
TWaitResponse* wr, TRequesterUserQueues* userQueues) {
if (data && data->size() > MAX_PACKET_SIZE) {
- Y_VERIFY(0, "data size is too large");
+ Y_ABORT_UNLESS(0, "data size is too large");
}
//printf("SendRequest(%s)\n", url.c_str());
if (wr)
@@ -873,7 +873,7 @@ namespace NNetliba {
void SendResponseImpl(const TGUID& reqId, EPacketPriority prior, TVector<char>* data) // non-virtual, for direct call from TRequestOps
{
if (data && data->size() > MAX_PACKET_SIZE) {
- Y_VERIFY(0, "data size is too large");
+ Y_ABORT_UNLESS(0, "data size is too large");
}
SendRespList.Enqueue(new TSendResponse(reqId, prior, data));
Host->CancelWait();
diff --git a/library/cpp/object_factory/object_factory.h b/library/cpp/object_factory/object_factory.h
index fb96de2d46..c2680b2301 100644
--- a/library/cpp/object_factory/object_factory.h
+++ b/library/cpp/object_factory/object_factory.h
@@ -128,7 +128,7 @@ namespace NObjectFactory {
template <class... Args>
static THolder<TProduct> VerifiedConstruct(Args&&... args) {
auto result = MakeHolder(std::forward<Args>(args)...);
- Y_VERIFY(result, "Construct by factory failed");
+ Y_ABORT_UNLESS(result, "Construct by factory failed");
return result;
}
diff --git a/library/cpp/openssl/big_integer/big_integer.cpp b/library/cpp/openssl/big_integer/big_integer.cpp
index 9b6802369a..49a60fd975 100644
--- a/library/cpp/openssl/big_integer/big_integer.cpp
+++ b/library/cpp/openssl/big_integer/big_integer.cpp
@@ -40,7 +40,7 @@ size_t TBigInteger::NumBytes() const noexcept {
size_t TBigInteger::ToRegion(void* to) const noexcept {
const auto ret = BN_bn2bin(Impl_, (unsigned char*)to);
- Y_VERIFY(ret >= 0, "it happens");
+ Y_ABORT_UNLESS(ret >= 0, "it happens");
return ret;
}
diff --git a/library/cpp/openssl/method/io.cpp b/library/cpp/openssl/method/io.cpp
index d184b6456c..533cb04180 100644
--- a/library/cpp/openssl/method/io.cpp
+++ b/library/cpp/openssl/method/io.cpp
@@ -10,7 +10,7 @@ namespace {
TAbstractIO* IO(BIO* bio) noexcept {
void* ptr = BIO_get_data(bio);
- Y_VERIFY(ptr);
+ Y_ABORT_UNLESS(ptr);
return static_cast<TAbstractIO*>(ptr);
}
diff --git a/library/cpp/protobuf/json/proto2json_printer.cpp b/library/cpp/protobuf/json/proto2json_printer.cpp
index 51a947be08..2b2c9b1713 100644
--- a/library/cpp/protobuf/json/proto2json_printer.cpp
+++ b/library/cpp/protobuf/json/proto2json_printer.cpp
@@ -211,7 +211,7 @@ namespace NProtobufJson {
const FieldDescriptor& field,
IJsonOutput& json,
TStringBuf key) {
- Y_VERIFY(!field.is_repeated(), "field is repeated.");
+ Y_ABORT_UNLESS(!field.is_repeated(), "field is repeated.");
if (!key) {
key = MakeKey(field);
@@ -303,7 +303,7 @@ namespace NProtobufJson {
const FieldDescriptor& field,
IJsonOutput& json,
TStringBuf key) {
- Y_VERIFY(field.is_repeated(), "field isn't repeated.");
+ Y_ABORT_UNLESS(field.is_repeated(), "field isn't repeated.");
const bool isMap = field.is_map() && GetConfig().MapAsObject;
if (!key) {
@@ -405,10 +405,10 @@ namespace NProtobufJson {
void TProto2JsonPrinter::PrintKeyValue(const NProtoBuf::Message& proto,
IJsonOutput& json) {
const FieldDescriptor* keyField = proto.GetDescriptor()->FindFieldByName("key");
- Y_VERIFY(keyField, "Map entry key field not found.");
+ Y_ABORT_UNLESS(keyField, "Map entry key field not found.");
TString key = MakeKey(proto, *keyField);
const FieldDescriptor* valueField = proto.GetDescriptor()->FindFieldByName("value");
- Y_VERIFY(valueField, "Map entry value field not found.");
+ Y_ABORT_UNLESS(valueField, "Map entry value field not found.");
PrintField(proto, *valueField, json, key);
}
diff --git a/library/cpp/resource/registry.cpp b/library/cpp/resource/registry.cpp
index 6f5ff85c63..c5d6543c65 100644
--- a/library/cpp/resource/registry.cpp
+++ b/library/cpp/resource/registry.cpp
@@ -28,14 +28,14 @@ namespace {
size_t vsize = GetCodec()->DecompressedLength(value);
size_t dsize = GetCodec()->DecompressedLength(data);
if (vsize + dsize < 1000) {
- Y_VERIFY(false, "Redefinition of key %s:\n"
+ Y_ABORT_UNLESS(false, "Redefinition of key %s:\n"
" old value: %s,\n"
" new value: %s.",
TString{key}.Quote().c_str(),
Decompress(value).Quote().c_str(),
Decompress(data).Quote().c_str());
} else {
- Y_VERIFY(false, "Redefinition of key %s,"
+ Y_ABORT_UNLESS(false, "Redefinition of key %s,"
" old size: %zu,"
" new size: %zu.",
TString{key}.Quote().c_str(), vsize, dsize);
@@ -46,7 +46,7 @@ namespace {
(*this)[key] = &D_.back();
}
- Y_VERIFY(size() == Count(), "size mismatch");
+ Y_ABORT_UNLESS(size() == Count(), "size mismatch");
}
bool Has(const TStringBuf key) const override {
diff --git a/library/cpp/scheme/tests/fuzz_ops/lib/vm_defs.h b/library/cpp/scheme/tests/fuzz_ops/lib/vm_defs.h
index 9a0ddf7351..6bc74c96c6 100644
--- a/library/cpp/scheme/tests/fuzz_ops/lib/vm_defs.h
+++ b/library/cpp/scheme/tests/fuzz_ops/lib/vm_defs.h
@@ -233,7 +233,7 @@ namespace NSc::NUt {
public:
template <class T>
bool SetArg(TMaybe<T> arg) {
- Y_VERIFY(CurrArg < Y_ARRAY_SIZE(Arg));
+ Y_ABORT_UNLESS(CurrArg < Y_ARRAY_SIZE(Arg));
if (arg) {
Arg[CurrArg++] = *arg;
return true;
diff --git a/library/cpp/sighandler/async_signals_handler.cpp b/library/cpp/sighandler/async_signals_handler.cpp
index 52e2c204f8..0c20dfccaa 100644
--- a/library/cpp/sighandler/async_signals_handler.cpp
+++ b/library/cpp/sighandler/async_signals_handler.cpp
@@ -33,7 +33,7 @@ namespace {
while (totalBytesWritten != bufsize) {
const ssize_t result = write(fd, (const char*)buf + totalBytesWritten, bufsize - totalBytesWritten);
- Y_VERIFY(result >= 0 || (result == -1 && errno == EINTR), "write failed: %s (errno = %d)", strerror(errno), errno);
+ Y_ABORT_UNLESS(result >= 0 || (result == -1 && errno == EINTR), "write failed: %s (errno = %d)", strerror(errno), errno);
totalBytesWritten += static_cast<size_t>(result);
}
}
@@ -72,7 +72,7 @@ namespace {
ui8 signum;
const ssize_t bytesRead = read(SignalPipeReadFd, &signum, 1);
- Y_VERIFY(bytesRead >= 0 || (bytesRead == -1 && errno == EINTR), "read failed: %s (errno = %d)", strerror(errno), errno);
+ Y_ABORT_UNLESS(bytesRead >= 0 || (bytesRead == -1 && errno == EINTR), "read failed: %s (errno = %d)", strerror(errno), errno);
if (AtomicAdd(ShouldDie, 0) != 0) {
DieEvent.Signal();
@@ -90,7 +90,7 @@ namespace {
TReadGuard dnd(HandlersLock);
const TEventHandlerPtr* handler = Handlers.FindPtr(signum);
- Y_VERIFY(handler && handler->Get(), "Async signal handler is not set, it's a bug!");
+ Y_ABORT_UNLESS(handler && handler->Get(), "Async signal handler is not set, it's a bug!");
handler->Get()->Handle(signum);
}
}
@@ -127,16 +127,16 @@ namespace {
if (result != 0 && errno == ENOSYS) { // linux older than 2.6.27 returns "not implemented"
#endif
- Y_VERIFY(pipe(filedes) == 0, "pipe failed: %s (errno = %d)", strerror(errno), errno);
+ Y_ABORT_UNLESS(pipe(filedes) == 0, "pipe failed: %s (errno = %d)", strerror(errno), errno);
SignalPipeReadFd = filedes[0];
SIGNAL_PIPE_WRITE_FD = filedes[1];
- Y_VERIFY(fcntl(SignalPipeReadFd, F_SETFD, FD_CLOEXEC) == 0, "fcntl failed: %s (errno = %d)", strerror(errno), errno);
- Y_VERIFY(fcntl(SIGNAL_PIPE_WRITE_FD, F_SETFD, FD_CLOEXEC) == 0, "fcntl failed: %s (errno = %d)", strerror(errno), errno);
+ Y_ABORT_UNLESS(fcntl(SignalPipeReadFd, F_SETFD, FD_CLOEXEC) == 0, "fcntl failed: %s (errno = %d)", strerror(errno), errno);
+ Y_ABORT_UNLESS(fcntl(SIGNAL_PIPE_WRITE_FD, F_SETFD, FD_CLOEXEC) == 0, "fcntl failed: %s (errno = %d)", strerror(errno), errno);
#ifdef _linux_
} else {
- Y_VERIFY(result == 0, "pipe2 failed: %s (errno = %d)", strerror(errno), errno);
+ Y_ABORT_UNLESS(result == 0, "pipe2 failed: %s (errno = %d)", strerror(errno), errno);
SignalPipeReadFd = filedes[0];
SIGNAL_PIPE_WRITE_FD = filedes[1];
}
@@ -177,7 +177,7 @@ namespace {
a.sa_sigaction = PipeWriterSignalHandler;
a.sa_flags = SA_SIGINFO | SA_RESTART;
- Y_VERIFY(!sigaction(signum, &a, nullptr), "sigaction failed: %s (errno = %d)", strerror(errno), errno);
+ Y_ABORT_UNLESS(!sigaction(signum, &a, nullptr), "sigaction failed: %s (errno = %d)", strerror(errno), errno);
}
}
};
diff --git a/library/cpp/skiff/skiff_schema.cpp b/library/cpp/skiff/skiff_schema.cpp
index c762896ad0..1355316756 100644
--- a/library/cpp/skiff/skiff_schema.cpp
+++ b/library/cpp/skiff/skiff_schema.cpp
@@ -127,7 +127,7 @@ const TSkiffSchemaList& TSkiffSchema::GetChildren() const
TSimpleTypeSchema::TSimpleTypeSchema(EWireType type)
: TSkiffSchema(type)
{
- Y_VERIFY(IsSimpleType(type));
+ Y_ABORT_UNLESS(IsSimpleType(type));
}
////////////////////////////////////////////////////////////////////////////////
diff --git a/library/cpp/skiff/skiff_validator.cpp b/library/cpp/skiff/skiff_validator.cpp
index 1b1b98d5a6..0406e9fdc2 100644
--- a/library/cpp/skiff/skiff_validator.cpp
+++ b/library/cpp/skiff/skiff_validator.cpp
@@ -81,7 +81,7 @@ public:
void PopValidator()
{
- Y_VERIFY(!ValidatorStack_.empty());
+ Y_ABORT_UNLESS(!ValidatorStack_.empty());
ValidatorStack_.pop();
if (!ValidatorStack_.empty()) {
ValidatorStack_.top()->OnChildDone(this);
@@ -97,7 +97,7 @@ public:
IValidatorNode* Top() const
{
- Y_VERIFY(!ValidatorStack_.empty());
+ Y_ABORT_UNLESS(!ValidatorStack_.empty());
return ValidatorStack_.top();
}
diff --git a/library/cpp/skiff/zerocopy_output_writer-inl.h b/library/cpp/skiff/zerocopy_output_writer-inl.h
index 6bd067c9fa..6da5e38f69 100644
--- a/library/cpp/skiff/zerocopy_output_writer-inl.h
+++ b/library/cpp/skiff/zerocopy_output_writer-inl.h
@@ -23,7 +23,7 @@ ui64 TZeroCopyOutputStreamWriter::RemainingBytes() const
void TZeroCopyOutputStreamWriter::Advance(size_t bytes)
{
- Y_VERIFY(bytes <= RemainingBytes_);
+ Y_ABORT_UNLESS(bytes <= RemainingBytes_);
Current_ += bytes;
RemainingBytes_ -= bytes;
}
diff --git a/library/cpp/streams/brotli/brotli.cpp b/library/cpp/streams/brotli/brotli.cpp
index 38052cb688..4e6ce64531 100644
--- a/library/cpp/streams/brotli/brotli.cpp
+++ b/library/cpp/streams/brotli/brotli.cpp
@@ -55,7 +55,7 @@ public:
void Finish() {
Flush();
DoWrite(nullptr, 0, BROTLI_OPERATION_FINISH);
- Y_VERIFY(BrotliEncoderIsFinished(EncoderState_));
+ Y_ABORT_UNLESS(BrotliEncoderIsFinished(EncoderState_));
}
private:
@@ -171,7 +171,7 @@ public:
ythrow yexception() << "Brotli decoder failed to decompress buffer: "
<< BrotliDecoderErrorString(code);
} else if (result == BROTLI_DECODER_RESULT_NEEDS_MORE_OUTPUT) {
- Y_VERIFY(availableOut != size,
+ Y_ABORT_UNLESS(availableOut != size,
"Buffer passed to read in Brotli decoder is too small");
break;
}
@@ -213,7 +213,7 @@ private:
}
void ResetState() {
- Y_VERIFY(BrotliDecoderIsFinished(DecoderState_));
+ Y_ABORT_UNLESS(BrotliDecoderIsFinished(DecoderState_));
FreeDecoder();
InitDecoder();
}
diff --git a/library/cpp/string_utils/base32/fuzz/main.cpp b/library/cpp/string_utils/base32/fuzz/main.cpp
index f2255c5aa0..0a4a30ceab 100644
--- a/library/cpp/string_utils/base32/fuzz/main.cpp
+++ b/library/cpp/string_utils/base32/fuzz/main.cpp
@@ -8,7 +8,7 @@ extern "C" int LLVMFuzzerTestOneInput(const ui8* data, size_t size)
const std::string_view example{reinterpret_cast<const char*>(data), size};
const auto converted = Base32StrictDecode(Base32Encode(example));
- Y_VERIFY(example == converted);
+ Y_ABORT_UNLESS(example == converted);
return 0;
}
diff --git a/library/cpp/string_utils/base64/fuzz/lib/main.cpp b/library/cpp/string_utils/base64/fuzz/lib/main.cpp
index 28547ae7a5..3367f44cf4 100644
--- a/library/cpp/string_utils/base64/fuzz/lib/main.cpp
+++ b/library/cpp/string_utils/base64/fuzz/lib/main.cpp
@@ -7,7 +7,7 @@ extern "C" int LLVMFuzzerTestOneInput(const ui8* data, size_t size) {
const TStringBuf example{reinterpret_cast<const char*>(data), size};
const auto converted = Base64Decode(Base64Encode(example));
- Y_VERIFY(example == converted);
+ Y_ABORT_UNLESS(example == converted);
return 0;
}
diff --git a/library/cpp/tdigest/tdigest.cpp b/library/cpp/tdigest/tdigest.cpp
index 3bd6d2d7e5..2381db1b14 100644
--- a/library/cpp/tdigest/tdigest.cpp
+++ b/library/cpp/tdigest/tdigest.cpp
@@ -23,7 +23,7 @@ TDigest::TDigest(TStringBuf serializedDigest)
: N(0)
{
NTDigest::TDigest digest;
- Y_VERIFY(digest.ParseFromArray(serializedDigest.data(), serializedDigest.size()));
+ Y_ABORT_UNLESS(digest.ParseFromArray(serializedDigest.data(), serializedDigest.size()));
Delta = digest.GetDelta();
K = digest.GetK();
for (int i = 0; i < digest.centroids_size(); ++i) {
diff --git a/library/cpp/terminate_handler/segv_handler.cpp b/library/cpp/terminate_handler/segv_handler.cpp
index f24ece4125..732e802159 100644
--- a/library/cpp/terminate_handler/segv_handler.cpp
+++ b/library/cpp/terminate_handler/segv_handler.cpp
@@ -29,6 +29,6 @@ static void SegvHandler(int sig) {
void InstallSegvHandler() {
#ifndef _win_
sig_t r = signal(SIGSEGV, &SegvHandler);
- Y_VERIFY(r != SIG_ERR, "signal failed: %s", strerror(errno));
+ Y_ABORT_UNLESS(r != SIG_ERR, "signal failed: %s", strerror(errno));
#endif // !_win_
}
diff --git a/library/cpp/testing/common/env.cpp b/library/cpp/testing/common/env.cpp
index 6585de8584..13f3f186af 100644
--- a/library/cpp/testing/common/env.cpp
+++ b/library/cpp/testing/common/env.cpp
@@ -110,7 +110,7 @@ const TString& GetTestParam(TStringBuf name, const TString& def) {
const TString& GetGlobalResource(TStringBuf name) {
auto& resources = NPrivate::GetTestEnv().GlobalResources;
auto it = resources.find(name.data());
- Y_VERIFY(it != resources.end());
+ Y_ABORT_UNLESS(it != resources.end());
return it->second;
}
diff --git a/library/cpp/testing/common/network.cpp b/library/cpp/testing/common/network.cpp
index 7a7ff2544c..a1f9f22e5a 100644
--- a/library/cpp/testing/common/network.cpp
+++ b/library/cpp/testing/common/network.cpp
@@ -105,7 +105,7 @@ namespace {
if (!SyncDir_.IsDefined()) {
SyncDir_ = TFsPath(GetSystemTempDir()) / "testing_port_locks";
}
- Y_VERIFY(SyncDir_.IsDefined());
+ Y_ABORT_UNLESS(SyncDir_.IsDefined());
NFs::MakeDirectoryRecursive(SyncDir_);
Ranges_ = GetPortRanges();
@@ -113,7 +113,7 @@ namespace {
for (auto [left, right] : Ranges_) {
TotalCount_ += right - left;
}
- Y_VERIFY(0 != TotalCount_);
+ Y_ABORT_UNLESS(0 != TotalCount_);
DisableRandomPorts_ = !GetEnv("NO_RANDOM_PORTS").empty();
}
@@ -141,7 +141,7 @@ namespace {
}
TVector<NTesting::TPortHolder> GetFreePortsRange(size_t count) const {
- Y_VERIFY(count > 0);
+ Y_ABORT_UNLESS(count > 0);
TVector<NTesting::TPortHolder> ports(Reserve(count));
for (size_t i = 0; i < Retries; ++i) {
for (auto[left, right] : Ranges_) {
@@ -194,7 +194,7 @@ namespace {
TSockAddrInet6 addr("::", port);
if (sock.Bind(&addr) != 0) {
lock->Release();
- Y_VERIFY(EADDRINUSE == LastSystemError(), "unexpected error: %d, port: %d", LastSystemError(), port);
+ Y_ABORT_UNLESS(EADDRINUSE == LastSystemError(), "unexpected error: %d, port: %d", LastSystemError(), port);
return nullptr;
}
return MakeHolder<TPortGuard>(port, std::move(lock));
diff --git a/library/cpp/testing/unittest/registar.cpp b/library/cpp/testing/unittest/registar.cpp
index 4f032fa00b..675d1f6003 100644
--- a/library/cpp/testing/unittest/registar.cpp
+++ b/library/cpp/testing/unittest/registar.cpp
@@ -34,8 +34,8 @@ currentTest;
::NUnitTest::TRaiseErrorHandler RaiseErrorHandler;
void ::NUnitTest::NPrivate::RaiseError(const char* what, const TString& msg, bool fatalFailure) {
- Y_VERIFY(UnittestThread, "%s in non-unittest thread with message:\n%s", what, msg.data());
- Y_VERIFY(GetCurrentTest());
+ Y_ABORT_UNLESS(UnittestThread, "%s in non-unittest thread with message:\n%s", what, msg.data());
+ Y_ABORT_UNLESS(GetCurrentTest());
if (RaiseErrorHandler) {
RaiseErrorHandler(what, msg, fatalFailure);
@@ -53,17 +53,17 @@ void ::NUnitTest::NPrivate::RaiseError(const char* what, const TString& msg, boo
}
void ::NUnitTest::SetRaiseErrorHandler(::NUnitTest::TRaiseErrorHandler handler) {
- Y_VERIFY(UnittestThread);
+ Y_ABORT_UNLESS(UnittestThread);
RaiseErrorHandler = std::move(handler);
}
void ::NUnitTest::NPrivate::SetUnittestThread(bool unittestThread) {
- Y_VERIFY(UnittestThread != unittestThread, "state check");
+ Y_ABORT_UNLESS(UnittestThread != unittestThread, "state check");
UnittestThread = unittestThread;
}
void ::NUnitTest::NPrivate::SetCurrentTest(TTestBase* test) {
- Y_VERIFY(!test || !currentTest, "state check");
+ Y_ABORT_UNLESS(!test || !currentTest, "state check");
currentTest = test;
}
diff --git a/library/cpp/testing/unittest/utmain.cpp b/library/cpp/testing/unittest/utmain.cpp
index cae713e3ac..1daabad4b8 100644
--- a/library/cpp/testing/unittest/utmain.cpp
+++ b/library/cpp/testing/unittest/utmain.cpp
@@ -566,7 +566,7 @@ private:
ythrow yexception() << "Forked test finished with unknown status";
}
case TShellCommand::SHELL_RUNNING: {
- Y_VERIFY(false, "This can't happen, we used sync mode, it's a bug!");
+ Y_ABORT_UNLESS(false, "This can't happen, we used sync mode, it's a bug!");
}
case TShellCommand::SHELL_INTERNAL_ERROR: {
ythrow yexception() << "Forked test failed with internal error: " << cmd.GetInternalError();
@@ -706,7 +706,7 @@ int NUnitTest::RunMain(int argc, char** argv) {
memset(&sa, 0, sizeof(sa));
sa.sa_handler = GracefulShutdownHandler;
sa.sa_flags = SA_SIGINFO | SA_RESTART;
- Y_VERIFY(!sigaction(SIGUSR2, &sa, nullptr));
+ Y_ABORT_UNLESS(!sigaction(SIGUSR2, &sa, nullptr));
}
#endif
NTesting::THook::CallBeforeInit();
diff --git a/library/cpp/threading/local_executor/local_executor.cpp b/library/cpp/threading/local_executor/local_executor.cpp
index 1440a3a67a..4b4f69c0cb 100644
--- a/library/cpp/threading/local_executor/local_executor.cpp
+++ b/library/cpp/threading/local_executor/local_executor.cpp
@@ -313,7 +313,7 @@ void NPar::ILocalExecutor::ExecRange(TLocallyExecutableFunction exec, int firstI
}
void NPar::ILocalExecutor::ExecRangeWithThrow(TLocallyExecutableFunction exec, int firstId, int lastId, int flags) {
- Y_VERIFY((flags & WAIT_COMPLETE) != 0, "ExecRangeWithThrow() requires WAIT_COMPLETE to wait if exceptions arise.");
+ Y_ABORT_UNLESS((flags & WAIT_COMPLETE) != 0, "ExecRangeWithThrow() requires WAIT_COMPLETE to wait if exceptions arise.");
if (TryExecRangeSequentially(exec, firstId, lastId, flags)) {
return;
}
diff --git a/library/cpp/threading/mux_event/mux_event.h b/library/cpp/threading/mux_event/mux_event.h
index 6ff32ee49d..b25e1acbbb 100644
--- a/library/cpp/threading/mux_event/mux_event.h
+++ b/library/cpp/threading/mux_event/mux_event.h
@@ -25,7 +25,7 @@ public:
Y_UNUSED(rmode);
}
~TMuxEvent() {
- Y_VERIFY(WaitList.empty(), "");
+ Y_ABORT_UNLESS(WaitList.empty(), "");
}
// TODO: potentially unsafe, but currently I can't add "virtual" to TSystemEvent methods
diff --git a/library/cpp/threading/queue/mpmc_unordered_ring.cpp b/library/cpp/threading/queue/mpmc_unordered_ring.cpp
index 160547f594..a3a0d42565 100644
--- a/library/cpp/threading/queue/mpmc_unordered_ring.cpp
+++ b/library/cpp/threading/queue/mpmc_unordered_ring.cpp
@@ -2,7 +2,7 @@
namespace NThreading {
TMPMCUnorderedRing::TMPMCUnorderedRing(size_t size) {
- Y_VERIFY(size > 0);
+ Y_ABORT_UNLESS(size > 0);
RingSize = size;
RingBuffer.Reset(new void*[size]);
memset(&RingBuffer[0], 0, sizeof(void*) * size);
diff --git a/library/cpp/type_info/builder.cpp b/library/cpp/type_info/builder.cpp
index 008e0af754..9e10cb71ed 100644
--- a/library/cpp/type_info/builder.cpp
+++ b/library/cpp/type_info/builder.cpp
@@ -139,7 +139,7 @@ namespace NTi {
}
TStructBuilderRaw& TStructBuilderRaw::AddMember() & noexcept {
- Y_VERIFY(CanAddMember());
+ Y_ABORT_UNLESS(CanAddMember());
Members_.emplace_back(*PendingMemberName_, *PendingMemberType_);
DiscardMember();
return *this;
@@ -311,7 +311,7 @@ namespace NTi {
}
TTupleBuilderRaw& TTupleBuilderRaw::AddElement() & noexcept {
- Y_VERIFY(CanAddElement());
+ Y_ABORT_UNLESS(CanAddElement());
Elements_.emplace_back(*PendingElementType_);
DiscardElement();
return *this;
@@ -452,7 +452,7 @@ namespace NTi {
}
const TTaggedType* TTaggedBuilderRaw::BuildRaw() {
- Y_VERIFY(CanBuild());
+ Y_ABORT_UNLESS(CanBuild());
return Factory_->New<TTaggedType>(Nothing(), *Item_, *Tag_);
}
}
diff --git a/library/cpp/type_info/type.cpp b/library/cpp/type_info/type.cpp
index cee58a0a79..b0f3ccd46c 100644
--- a/library/cpp/type_info/type.cpp
+++ b/library/cpp/type_info/type.cpp
@@ -228,14 +228,14 @@ namespace NTi {
factory.SaveCache(result);
}
- Y_VERIFY(result->GetTypeName() == type->GetTypeName());
+ Y_ABORT_UNLESS(result->GetTypeName() == type->GetTypeName());
Y_VERIFY_DEBUG(result->GetHash() == type->GetHash());
return static_cast<const T*>(result);
}
bool operator==(const TType& lhs, const TType& rhs) {
- Y_VERIFY(&lhs);
- Y_VERIFY(&rhs);
+ Y_ABORT_UNLESS(&lhs);
+ Y_ABORT_UNLESS(&rhs);
return NEq::TStrictlyEqual().IgnoreHash(&lhs, &rhs);
}
@@ -1019,7 +1019,7 @@ namespace NTi {
}
void TStructType::MakeSortedMembers(TStructType::TMembers members, TArrayRef<size_t> sortedItems) {
- Y_VERIFY(members.size() == sortedItems.size());
+ Y_ABORT_UNLESS(members.size() == sortedItems.size());
for (size_t i = 0; i < members.size(); ++i) {
sortedItems[i] = i;
diff --git a/library/cpp/type_info/type.h b/library/cpp/type_info/type.h
index 1577f1cee3..ea148c364d 100644
--- a/library/cpp/type_info/type.h
+++ b/library/cpp/type_info/type.h
@@ -1698,7 +1698,7 @@ namespace NTi {
}
const TPrimitiveType* TType::AsPrimitiveRaw() const noexcept {
- Y_VERIFY(IsPrimitive());
+ Y_ABORT_UNLESS(IsPrimitive());
return static_cast<const TPrimitiveType*>(this);
}
@@ -1711,7 +1711,7 @@ namespace NTi {
}
const TVoidType* TType::AsVoidRaw() const noexcept {
- Y_VERIFY(IsVoid());
+ Y_ABORT_UNLESS(IsVoid());
return static_cast<const TVoidType*>(this);
}
@@ -1724,7 +1724,7 @@ namespace NTi {
}
const TNullType* TType::AsNullRaw() const noexcept {
- Y_VERIFY(IsNull());
+ Y_ABORT_UNLESS(IsNull());
return static_cast<const TNullType*>(this);
}
@@ -1737,7 +1737,7 @@ namespace NTi {
}
const TBoolType* TType::AsBoolRaw() const noexcept {
- Y_VERIFY(IsBool());
+ Y_ABORT_UNLESS(IsBool());
return static_cast<const TBoolType*>(this);
}
@@ -1750,7 +1750,7 @@ namespace NTi {
}
const TInt8Type* TType::AsInt8Raw() const noexcept {
- Y_VERIFY(IsInt8());
+ Y_ABORT_UNLESS(IsInt8());
return static_cast<const TInt8Type*>(this);
}
@@ -1763,7 +1763,7 @@ namespace NTi {
}
const TInt16Type* TType::AsInt16Raw() const noexcept {
- Y_VERIFY(IsInt16());
+ Y_ABORT_UNLESS(IsInt16());
return static_cast<const TInt16Type*>(this);
}
@@ -1776,7 +1776,7 @@ namespace NTi {
}
const TInt32Type* TType::AsInt32Raw() const noexcept {
- Y_VERIFY(IsInt32());
+ Y_ABORT_UNLESS(IsInt32());
return static_cast<const TInt32Type*>(this);
}
@@ -1789,7 +1789,7 @@ namespace NTi {
}
const TInt64Type* TType::AsInt64Raw() const noexcept {
- Y_VERIFY(IsInt64());
+ Y_ABORT_UNLESS(IsInt64());
return static_cast<const TInt64Type*>(this);
}
@@ -1802,7 +1802,7 @@ namespace NTi {
}
const TUint8Type* TType::AsUint8Raw() const noexcept {
- Y_VERIFY(IsUint8());
+ Y_ABORT_UNLESS(IsUint8());
return static_cast<const TUint8Type*>(this);
}
@@ -1815,7 +1815,7 @@ namespace NTi {
}
const TUint16Type* TType::AsUint16Raw() const noexcept {
- Y_VERIFY(IsUint16());
+ Y_ABORT_UNLESS(IsUint16());
return static_cast<const TUint16Type*>(this);
}
@@ -1828,7 +1828,7 @@ namespace NTi {
}
const TUint32Type* TType::AsUint32Raw() const noexcept {
- Y_VERIFY(IsUint32());
+ Y_ABORT_UNLESS(IsUint32());
return static_cast<const TUint32Type*>(this);
}
@@ -1841,7 +1841,7 @@ namespace NTi {
}
const TUint64Type* TType::AsUint64Raw() const noexcept {
- Y_VERIFY(IsUint64());
+ Y_ABORT_UNLESS(IsUint64());
return static_cast<const TUint64Type*>(this);
}
@@ -1854,7 +1854,7 @@ namespace NTi {
}
const TFloatType* TType::AsFloatRaw() const noexcept {
- Y_VERIFY(IsFloat());
+ Y_ABORT_UNLESS(IsFloat());
return static_cast<const TFloatType*>(this);
}
@@ -1867,7 +1867,7 @@ namespace NTi {
}
const TDoubleType* TType::AsDoubleRaw() const noexcept {
- Y_VERIFY(IsDouble());
+ Y_ABORT_UNLESS(IsDouble());
return static_cast<const TDoubleType*>(this);
}
@@ -1880,7 +1880,7 @@ namespace NTi {
}
const TStringType* TType::AsStringRaw() const noexcept {
- Y_VERIFY(IsString());
+ Y_ABORT_UNLESS(IsString());
return static_cast<const TStringType*>(this);
}
@@ -1893,7 +1893,7 @@ namespace NTi {
}
const TUtf8Type* TType::AsUtf8Raw() const noexcept {
- Y_VERIFY(IsUtf8());
+ Y_ABORT_UNLESS(IsUtf8());
return static_cast<const TUtf8Type*>(this);
}
@@ -1906,7 +1906,7 @@ namespace NTi {
}
const TDateType* TType::AsDateRaw() const noexcept {
- Y_VERIFY(IsDate());
+ Y_ABORT_UNLESS(IsDate());
return static_cast<const TDateType*>(this);
}
@@ -1919,7 +1919,7 @@ namespace NTi {
}
const TDatetimeType* TType::AsDatetimeRaw() const noexcept {
- Y_VERIFY(IsDatetime());
+ Y_ABORT_UNLESS(IsDatetime());
return static_cast<const TDatetimeType*>(this);
}
@@ -1932,7 +1932,7 @@ namespace NTi {
}
const TTimestampType* TType::AsTimestampRaw() const noexcept {
- Y_VERIFY(IsTimestamp());
+ Y_ABORT_UNLESS(IsTimestamp());
return static_cast<const TTimestampType*>(this);
}
@@ -1945,7 +1945,7 @@ namespace NTi {
}
const TTzDateType* TType::AsTzDateRaw() const noexcept {
- Y_VERIFY(IsTzDate());
+ Y_ABORT_UNLESS(IsTzDate());
return static_cast<const TTzDateType*>(this);
}
@@ -1958,7 +1958,7 @@ namespace NTi {
}
const TTzDatetimeType* TType::AsTzDatetimeRaw() const noexcept {
- Y_VERIFY(IsTzDatetime());
+ Y_ABORT_UNLESS(IsTzDatetime());
return static_cast<const TTzDatetimeType*>(this);
}
@@ -1971,7 +1971,7 @@ namespace NTi {
}
const TTzTimestampType* TType::AsTzTimestampRaw() const noexcept {
- Y_VERIFY(IsTzTimestamp());
+ Y_ABORT_UNLESS(IsTzTimestamp());
return static_cast<const TTzTimestampType*>(this);
}
@@ -1984,7 +1984,7 @@ namespace NTi {
}
const TIntervalType* TType::AsIntervalRaw() const noexcept {
- Y_VERIFY(IsInterval());
+ Y_ABORT_UNLESS(IsInterval());
return static_cast<const TIntervalType*>(this);
}
@@ -1997,7 +1997,7 @@ namespace NTi {
}
const TDecimalType* TType::AsDecimalRaw() const noexcept {
- Y_VERIFY(IsDecimal());
+ Y_ABORT_UNLESS(IsDecimal());
return static_cast<const TDecimalType*>(this);
}
@@ -2010,7 +2010,7 @@ namespace NTi {
}
const TJsonType* TType::AsJsonRaw() const noexcept {
- Y_VERIFY(IsJson());
+ Y_ABORT_UNLESS(IsJson());
return static_cast<const TJsonType*>(this);
}
@@ -2023,7 +2023,7 @@ namespace NTi {
}
const TYsonType* TType::AsYsonRaw() const noexcept {
- Y_VERIFY(IsYson());
+ Y_ABORT_UNLESS(IsYson());
return static_cast<const TYsonType*>(this);
}
@@ -2036,7 +2036,7 @@ namespace NTi {
}
const TUuidType* TType::AsUuidRaw() const noexcept {
- Y_VERIFY(IsUuid());
+ Y_ABORT_UNLESS(IsUuid());
return static_cast<const TUuidType*>(this);
}
@@ -2049,7 +2049,7 @@ namespace NTi {
}
const TOptionalType* TType::AsOptionalRaw() const noexcept {
- Y_VERIFY(IsOptional());
+ Y_ABORT_UNLESS(IsOptional());
return static_cast<const TOptionalType*>(this);
}
@@ -2062,7 +2062,7 @@ namespace NTi {
}
const TListType* TType::AsListRaw() const noexcept {
- Y_VERIFY(IsList());
+ Y_ABORT_UNLESS(IsList());
return static_cast<const TListType*>(this);
}
@@ -2075,7 +2075,7 @@ namespace NTi {
}
const TDictType* TType::AsDictRaw() const noexcept {
- Y_VERIFY(IsDict());
+ Y_ABORT_UNLESS(IsDict());
return static_cast<const TDictType*>(this);
}
@@ -2088,7 +2088,7 @@ namespace NTi {
}
const TStructType* TType::AsStructRaw() const noexcept {
- Y_VERIFY(IsStruct());
+ Y_ABORT_UNLESS(IsStruct());
return static_cast<const TStructType*>(this);
}
@@ -2101,7 +2101,7 @@ namespace NTi {
}
const TTupleType* TType::AsTupleRaw() const noexcept {
- Y_VERIFY(IsTuple());
+ Y_ABORT_UNLESS(IsTuple());
return static_cast<const TTupleType*>(this);
}
@@ -2114,7 +2114,7 @@ namespace NTi {
}
const TVariantType* TType::AsVariantRaw() const noexcept {
- Y_VERIFY(IsVariant());
+ Y_ABORT_UNLESS(IsVariant());
return static_cast<const TVariantType*>(this);
}
@@ -2127,7 +2127,7 @@ namespace NTi {
}
const TTaggedType* TType::AsTaggedRaw() const noexcept {
- Y_VERIFY(IsTagged());
+ Y_ABORT_UNLESS(IsTagged());
return static_cast<const TTaggedType*>(this);
}
diff --git a/library/cpp/type_info/type_list.h b/library/cpp/type_info/type_list.h
index c87c51918d..8d0c1db88b 100644
--- a/library/cpp/type_info/type_list.h
+++ b/library/cpp/type_info/type_list.h
@@ -176,7 +176,7 @@ namespace NTi {
/// Enumerator values should not relied upon, therefore users should not cast `NTi::ETypeName`
/// to `NTi::EPrimitiveTypeName` using `static_cast`.
inline constexpr EPrimitiveTypeName ToPrimitiveTypeName(ETypeName typeName) {
- Y_VERIFY(IsPrimitive(typeName));
+ Y_ABORT_UNLESS(IsPrimitive(typeName));
// Note: there's a test in ut/type_list.cpp that checks this is a safe conversion
return static_cast<EPrimitiveTypeName>(typeName);
}
diff --git a/library/cpp/unicode/set/unicode_set_lexer.h b/library/cpp/unicode/set/unicode_set_lexer.h
index c584f317fc..95b563671f 100644
--- a/library/cpp/unicode/set/unicode_set_lexer.h
+++ b/library/cpp/unicode/set/unicode_set_lexer.h
@@ -40,7 +40,7 @@ namespace NUnicode {
}
inline void PushBack() {
- Y_VERIFY(!UseLast, "Double TUnicodeSetLexer::PushBack()");
+ Y_ABORT_UNLESS(!UseLast, "Double TUnicodeSetLexer::PushBack()");
UseLast = true;
}
};
diff --git a/library/cpp/unified_agent_client/async_joiner.h b/library/cpp/unified_agent_client/async_joiner.h
index ce392ef7bc..23494cc6f7 100644
--- a/library/cpp/unified_agent_client/async_joiner.h
+++ b/library/cpp/unified_agent_client/async_joiner.h
@@ -13,13 +13,13 @@ namespace NUnifiedAgent {
inline i64 Ref(i64 count = 1) noexcept {
const auto result = Refs.fetch_add(count);
- Y_VERIFY(result >= 1, "already joined");
+ Y_ABORT_UNLESS(result >= 1, "already joined");
return result;
}
inline i64 UnRef() noexcept {
const auto prev = Refs.fetch_sub(1);
- Y_VERIFY(prev >= 1);
+ Y_ABORT_UNLESS(prev >= 1);
if (prev == 1) {
auto p = Promise;
p.SetValue();
diff --git a/library/cpp/unified_agent_client/client_impl.cpp b/library/cpp/unified_agent_client/client_impl.cpp
index 31845fab61..17e2cd6b9d 100644
--- a/library/cpp/unified_agent_client/client_impl.cpp
+++ b/library/cpp/unified_agent_client/client_impl.cpp
@@ -67,7 +67,7 @@ namespace NUnifiedAgent::NPrivate {
TClient::~TClient() {
with_lock(Lock) {
- Y_VERIFY(ActiveSessions.empty(), "active sessions found");
+ Y_ABORT_UNLESS(ActiveSessions.empty(), "active sessions found");
EnsureStoppedNoLock();
@@ -106,7 +106,7 @@ namespace NUnifiedAgent::NPrivate {
void TClient::UnregisterSession(TClientSession* session) {
with_lock(Lock) {
const auto it = Find(ActiveSessions, session);
- Y_VERIFY(it != ActiveSessions.end());
+ Y_ABORT_UNLESS(it != ActiveSessions.end());
ActiveSessions.erase(it);
}
}
@@ -213,8 +213,8 @@ namespace NUnifiedAgent::NPrivate {
return;
}
- Y_VERIFY(grpc_is_initialized());
- Y_VERIFY(grpc_core::Fork::Enabled());
+ Y_ABORT_UNLESS(grpc_is_initialized());
+ Y_ABORT_UNLESS(grpc_core::Fork::Enabled());
with_lock(Lock) {
Clients.push_back(&client);
@@ -228,7 +228,7 @@ namespace NUnifiedAgent::NPrivate {
with_lock(Lock) {
const auto it = Find(Clients, &client);
- Y_VERIFY(it != Clients.end());
+ Y_ABORT_UNLESS(it != Clients.end());
Clients.erase(it);
}
}
@@ -418,7 +418,7 @@ namespace NUnifiedAgent::NPrivate {
void TClientSession::DoStart() {
// Lock must be held
- Y_VERIFY(!Started);
+ Y_ABORT_UNLESS(!Started);
YLOG_DEBUG("starting");
Client->EnsureStarted();
@@ -447,7 +447,7 @@ namespace NUnifiedAgent::NPrivate {
}, &AsyncJoiner));
EventNotification = MakeHolder<TGrpcNotification>(Client->GetCompletionQueue(),
MakeIOCallback([this](EIOStatus status) {
- Y_VERIFY(status == EIOStatus::Ok);
+ Y_ABORT_UNLESS(status == EIOStatus::Ok);
Poll();
}, &AsyncJoiner));
@@ -471,8 +471,8 @@ namespace NUnifiedAgent::NPrivate {
YLOG_INFO("MakeGrpcCall, session already closed");
return;
}
- Y_VERIFY(!ForcedCloseStarted);
- Y_VERIFY(!ActiveGrpcCall);
+ Y_ABORT_UNLESS(!ForcedCloseStarted);
+ Y_ABORT_UNLESS(!ActiveGrpcCall);
ActiveGrpcCall = MakeIntrusive<TGrpcCall>(*this);
ActiveGrpcCall->Start();
++Counters->GrpcCalls;
@@ -655,7 +655,7 @@ namespace NUnifiedAgent::NPrivate {
});
if (auto it = begin(batch); it != closeIt) {
- Y_VERIFY(!CloseStarted);
+ Y_ABORT_UNLESS(!CloseStarted);
do {
auto& e = std::get<TMessageReceivedEvent>(*it++);
WriteQueue.push_back({std::move(e.Message), e.Size, false});
@@ -715,7 +715,7 @@ namespace NUnifiedAgent::NPrivate {
TClientSession::TRequestBuilder::TAddResult TClientSession::TRequestBuilder::TryAddMessage(
const TPendingMessage& message, size_t seqNo) {
- Y_VERIFY(!CountersInvalid);
+ Y_ABORT_UNLESS(!CountersInvalid);
{
// add item to pwRequest to increase calculated size
PwTarget->DataBatch.SeqNo.Add(seqNo);
@@ -768,7 +768,7 @@ namespace NUnifiedAgent::NPrivate {
if (it == MetaItems.end()) {
batch.AddMeta()->SetKey(m.first);
auto insertResult = MetaItems.insert({m.first, {batch.MetaSize() - 1}});
- Y_VERIFY(insertResult.second);
+ Y_ABORT_UNLESS(insertResult.second);
metaItemBuilder = &insertResult.first->second;
} else {
metaItemBuilder = &it->second;
@@ -792,7 +792,7 @@ namespace NUnifiedAgent::NPrivate {
}
void TClientSession::PrepareWriteBatchRequest(NUnifiedAgentProto::Request& target) {
- Y_VERIFY(AckSeqNo.Defined());
+ Y_ABORT_UNLESS(AckSeqNo.Defined());
TRequestBuilder requestBuilder(target, Client->GetParameters().GrpcMaxMessageSize, AgentMaxReceiveMessage);
const auto startIndex = NextIndex - TrimmedCount;
for (size_t i = startIndex; i < WriteQueue.size(); ++i) {
@@ -830,7 +830,7 @@ namespace NUnifiedAgent::NPrivate {
if (messagesCount == 0) {
return;
}
- Y_VERIFY(requestBuilder.GetSerializedRequestSize() == target.ByteSizeLong(),
+ Y_ABORT_UNLESS(requestBuilder.GetSerializedRequestSize() == target.ByteSizeLong(),
"failed to calculate size for message [%s]", target.ShortDebugString().c_str());
GrpcInflightMessages += messagesCount;
GrpcInflightBytes += requestBuilder.GetRequestPayloadSize();
@@ -893,8 +893,8 @@ namespace NUnifiedAgent::NPrivate {
}
void TClientSession::OnGrpcCallFinished() {
- Y_VERIFY(!Closed);
- Y_VERIFY(ActiveGrpcCall);
+ Y_ABORT_UNLESS(!Closed);
+ Y_ABORT_UNLESS(ActiveGrpcCall);
ActiveGrpcCall = nullptr;
if (CloseStarted && (ForcedCloseStarted || WriteQueue.empty())) {
DoClose();
@@ -928,9 +928,9 @@ namespace NUnifiedAgent::NPrivate {
}
void TClientSession::DoClose() {
- Y_VERIFY(CloseStarted);
- Y_VERIFY(!Closed);
- Y_VERIFY(!ClosePromise.HasValue());
+ Y_ABORT_UNLESS(CloseStarted);
+ Y_ABORT_UNLESS(!Closed);
+ Y_ABORT_UNLESS(!ClosePromise.HasValue());
MakeGrpcCallTimer->Cancel();
ForceCloseTimer->Cancel();
PollTimer->Cancel();
@@ -1091,7 +1091,7 @@ namespace NUnifiedAgent::NPrivate {
}
void TGrpcCall::EndAccept(EIOStatus status) {
- Y_VERIFY(AcceptPending);
+ Y_ABORT_UNLESS(AcceptPending);
AcceptPending = false;
if (CheckHasError(status, "EndAccept")) {
return;
@@ -1109,7 +1109,7 @@ namespace NUnifiedAgent::NPrivate {
return;
}
if (!ErrorOccured && status == EIOStatus::Error && WritesBlocked) {
- Y_VERIFY(!WritePending);
+ Y_ABORT_UNLESS(!WritePending);
YLOG_DEBUG("EndRead ReadsDone");
ReadsDone = true;
if (WritesDone) {
@@ -1189,7 +1189,7 @@ namespace NUnifiedAgent::NPrivate {
void TGrpcCall::EndWritesDone(EIOStatus status) {
YLOG_DEBUG(Sprintf("EndWritesDone [%s]", ToString(status).c_str()));
- Y_VERIFY(!WritePending && !WritesDone && WritesDonePending);
+ Y_ABORT_UNLESS(!WritePending && !WritesDone && WritesDonePending);
WritesDonePending = false;
WritesDone = true;
if (CheckHasError(status, "EndWriteDone")) {
diff --git a/library/cpp/unified_agent_client/clock.cpp b/library/cpp/unified_agent_client/clock.cpp
index 192c998a02..9c0e59c20e 100644
--- a/library/cpp/unified_agent_client/clock.cpp
+++ b/library/cpp/unified_agent_client/clock.cpp
@@ -2,13 +2,13 @@
namespace NUnifiedAgent {
void TClock::Configure() {
- Y_VERIFY(!Configured_);
+ Y_ABORT_UNLESS(!Configured_);
Configured_ = true;
}
void TClock::SetBase(TInstant value) {
- Y_VERIFY(Configured_);
+ Y_ABORT_UNLESS(Configured_);
Base_.store(value.GetValue());
}
@@ -18,13 +18,13 @@ namespace NUnifiedAgent {
}
void TClock::ResetBaseWithShift() {
- Y_VERIFY(Configured_);
+ Y_ABORT_UNLESS(Configured_);
Shift_.store(static_cast<i64>(Base_.exchange(0)) - static_cast<i64>(::Now().GetValue()));
}
void TClock::SetShift(TDuration value) {
- Y_VERIFY(Configured_);
+ Y_ABORT_UNLESS(Configured_);
Shift_.fetch_add(value.GetValue());
}
diff --git a/library/cpp/unified_agent_client/enum.h b/library/cpp/unified_agent_client/enum.h
index ad08efb723..1eef790819 100644
--- a/library/cpp/unified_agent_client/enum.h
+++ b/library/cpp/unified_agent_client/enum.h
@@ -13,7 +13,7 @@ namespace NUnifiedAgent {
TEnumNames* result = new TEnumNames(names.size());
size_t index = 0;
for (const auto& p: names) {
- Y_VERIFY(static_cast<size_t>(p.first) == index);
+ Y_ABORT_UNLESS(static_cast<size_t>(p.first) == index);
(*result)[index++] = &p.second;
}
return result;
diff --git a/library/cpp/unified_agent_client/examples/ua_grpc_client/main.cpp b/library/cpp/unified_agent_client/examples/ua_grpc_client/main.cpp
index a9eb423d13..8f247cf9ad 100644
--- a/library/cpp/unified_agent_client/examples/ua_grpc_client/main.cpp
+++ b/library/cpp/unified_agent_client/examples/ua_grpc_client/main.cpp
@@ -59,7 +59,7 @@ bool TryParseMeta(const TString& s, THashMap<TString, TString>& meta) {
bool TryParseLine(const TString& line, TVector<TString>& lineItems) {
lineItems = StringSplitter(line).Split('|').ToList<TString>();
- Y_VERIFY(lineItems.size() >= 1);
+ Y_ABORT_UNLESS(lineItems.size() >= 1);
if (lineItems.size() > 2) {
Cout << "invalid line format, expected 'k1=v1,k2=v2|payload' or just 'payload'" << Endl;
return false;
diff --git a/library/cpp/unified_agent_client/grpc_io.cpp b/library/cpp/unified_agent_client/grpc_io.cpp
index b59cfb674a..569e04dde8 100644
--- a/library/cpp/unified_agent_client/grpc_io.cpp
+++ b/library/cpp/unified_agent_client/grpc_io.cpp
@@ -36,10 +36,10 @@ namespace NUnifiedAgent {
grpc_core::ApplicationCallbackExecCtx callbackExecCtx;
grpc_core::ExecCtx execCtx;
IOCallback->Ref();
- Y_VERIFY(grpc_cq_begin_op(CompletionQueue.cq(), this));
+ Y_ABORT_UNLESS(grpc_cq_begin_op(CompletionQueue.cq(), this));
grpc_cq_end_op(CompletionQueue.cq(), this, y_absl::OkStatus(),
[](void* self, grpc_cq_completion*) {
- Y_VERIFY(static_cast<TGrpcNotification*>(self)->InQueue.exchange(false));
+ Y_ABORT_UNLESS(static_cast<TGrpcNotification*>(self)->InQueue.exchange(false));
},
this, Completion.Get());
}
@@ -81,7 +81,7 @@ namespace NUnifiedAgent {
}
void TGrpcTimer::OnIOCompleted(EIOStatus status) {
- Y_VERIFY(AlarmIsSet);
+ Y_ABORT_UNLESS(AlarmIsSet);
if (NextTriggerTime) {
Alarm.Set(&CompletionQueue, InstantToTimespec(*NextTriggerTime), this);
NextTriggerTime.Clear();
diff --git a/library/cpp/uri/benchmark/main.cpp b/library/cpp/uri/benchmark/main.cpp
index d39704877e..31c6a3be32 100644
--- a/library/cpp/uri/benchmark/main.cpp
+++ b/library/cpp/uri/benchmark/main.cpp
@@ -28,7 +28,7 @@ Y_CPU_BENCHMARK(Parsing, iface) {
NUri::TUri uri;
auto parseResult = uri.Parse(url, uri.FeaturesAll);
Y_DO_NOT_OPTIMIZE_AWAY(parseResult);
- Y_VERIFY(parseResult == NUri::TState::ParsedOK, "cannot parse %s: %d", url.c_str(), static_cast<ui32>(parseResult));
+ Y_ABORT_UNLESS(parseResult == NUri::TState::ParsedOK, "cannot parse %s: %d", url.c_str(), static_cast<ui32>(parseResult));
}
}
}
@@ -38,7 +38,7 @@ Y_CPU_BENCHMARK(ParsingAndCopying, iface) {
for (auto&& url : URLS) {
NUri::TUri uri;
auto parseResult = uri.Parse(url, uri.FeaturesAll);
- Y_VERIFY(parseResult == NUri::TState::ParsedOK, "cannot parse %s: %d", url.c_str(), static_cast<ui32>(parseResult));
+ Y_ABORT_UNLESS(parseResult == NUri::TState::ParsedOK, "cannot parse %s: %d", url.c_str(), static_cast<ui32>(parseResult));
auto copy = uri;
Y_DO_NOT_OPTIMIZE_AWAY(copy);
}
diff --git a/library/cpp/yconf/patcher/config_patcher.cpp b/library/cpp/yconf/patcher/config_patcher.cpp
index 93b8649705..c395817b48 100644
--- a/library/cpp/yconf/patcher/config_patcher.cpp
+++ b/library/cpp/yconf/patcher/config_patcher.cpp
@@ -99,7 +99,7 @@ namespace {
const TYandexConfig::Section* source,
const TYandexConfig::Section* target,
const TString& parentPrefix = TString()) {
- Y_VERIFY(target);
+ Y_ABORT_UNLESS(target);
const TString& prefix = parentPrefix ? (parentPrefix + ".") : parentPrefix;
for (const auto& [name, value]: target->GetDirectives()) {
@@ -137,7 +137,7 @@ namespace {
const bool needsIndex = targetSections.size() > 1;
if (targetSections.empty()) {
- Y_VERIFY(source);
+ Y_ABORT_UNLESS(source);
container[prefix + sectionName] = "__remove_all__";
} else {
for (const size_t i: xrange(targetSections.size())) {
diff --git a/library/cpp/yconf/patcher/unstrict_config.cpp b/library/cpp/yconf/patcher/unstrict_config.cpp
index 5cb4c351a3..a4fb613210 100644
--- a/library/cpp/yconf/patcher/unstrict_config.cpp
+++ b/library/cpp/yconf/patcher/unstrict_config.cpp
@@ -159,7 +159,7 @@ TYandexConfig::Section* TUnstrictConfig::GetSection(TYandexConfig::Section* sect
}
TVector<const TYandexConfig::Section*> TUnstrictConfig::GetSections(const TYandexConfig::Section* section, TPathIterator begin, TPathIterator end) const {
- Y_VERIFY(section);
+ Y_ABORT_UNLESS(section);
if (begin == end)
return {section};
@@ -183,7 +183,7 @@ TVector<const TYandexConfig::Section*> TUnstrictConfig::GetSections(const TYande
}
TVector<TYandexConfig::Section*> TUnstrictConfig::GetSections(TYandexConfig::Section* section, TPathIterator begin, TPathIterator end, bool add) {
- Y_VERIFY(section);
+ Y_ABORT_UNLESS(section);
if (begin == end)
return {section};
@@ -283,7 +283,7 @@ bool TUnstrictConfig::PatchEntry(const TString& path, const TString& value, cons
}
[[nodiscard]] bool TUnstrictConfig::ParseJson(const NJson::TJsonValue& json) {
- Y_VERIFY(ParseMemory(""));
+ Y_ABORT_UNLESS(ParseMemory(""));
return ParseJson(json, TString());
}
diff --git a/library/cpp/yconf/patcher/unstrict_config.h b/library/cpp/yconf/patcher/unstrict_config.h
index 4990fbe86d..80ffeab890 100644
--- a/library/cpp/yconf/patcher/unstrict_config.h
+++ b/library/cpp/yconf/patcher/unstrict_config.h
@@ -50,7 +50,7 @@ private:
, BeginIndex(beginIndex)
, EndIndex(endIndex)
{
- Y_VERIFY(EndIndex >= BeginIndex);
+ Y_ABORT_UNLESS(EndIndex >= BeginIndex);
}
inline TPathUnit(const TCiString& name, size_t index)
: TPathUnit(name, index, index)
diff --git a/library/cpp/yt/threading/at_fork.cpp b/library/cpp/yt/threading/at_fork.cpp
index 6cf1a7fbfe..2046918fcb 100644
--- a/library/cpp/yt/threading/at_fork.cpp
+++ b/library/cpp/yt/threading/at_fork.cpp
@@ -29,7 +29,7 @@ public:
TAtForkHandler child)
{
int index = AtForkHandlerCount_++;
- Y_VERIFY(index < MaxAtForkHandlerSets);
+ Y_ABORT_UNLESS(index < MaxAtForkHandlerSets);
auto& set = AtForkHandlerSets_[index];
set.Prepare = std::move(prepare);
set.Parent = std::move(parent);
diff --git a/library/cpp/ytalloc/impl/bridge.cpp b/library/cpp/ytalloc/impl/bridge.cpp
index 5444a8d48b..95a4761998 100644
--- a/library/cpp/ytalloc/impl/bridge.cpp
+++ b/library/cpp/ytalloc/impl/bridge.cpp
@@ -155,9 +155,9 @@ extern "C" void* valloc(size_t size)
extern "C" void* aligned_alloc(size_t alignment, size_t size)
{
// Alignment must be a power of two.
- Y_VERIFY((alignment & (alignment - 1)) == 0);
+ Y_ABORT_UNLESS((alignment & (alignment - 1)) == 0);
// Alignment must not exceed the page size.
- Y_VERIFY(alignment <= PageSize);
+ Y_ABORT_UNLESS(alignment <= PageSize);
if (alignment <= 16) {
// Proper alignment here is automatic.
return Allocate(size);
diff --git a/library/cpp/ytalloc/impl/core-inl.h b/library/cpp/ytalloc/impl/core-inl.h
index c02a983a2c..64e95188f4 100644
--- a/library/cpp/ytalloc/impl/core-inl.h
+++ b/library/cpp/ytalloc/impl/core-inl.h
@@ -1237,7 +1237,7 @@ private:
if (result != 0) {
auto error = errno;
// Failure is possible for locked pages.
- Y_VERIFY(error == EINVAL);
+ Y_ABORT_UNLESS(error == EINVAL);
}
}
@@ -1895,7 +1895,7 @@ public:
static void SetCurrentMemoryTag(TMemoryTag tag)
{
- Y_VERIFY(tag <= MaxMemoryTag);
+ Y_ABORT_UNLESS(tag <= MaxMemoryTag);
(&ThreadControlWord_)->Parts.MemoryTag = tag;
}
@@ -1929,13 +1929,13 @@ private:
void RefThreadState(TThreadState* state)
{
auto result = ++state->RefCounter;
- Y_VERIFY(result > 1);
+ Y_ABORT_UNLESS(result > 1);
}
void UnrefThreadState(TThreadState* state)
{
auto result = --state->RefCounter;
- Y_VERIFY(result >= 0);
+ Y_ABORT_UNLESS(result >= 0);
if (result == 0) {
DestroyThreadState(state);
}
@@ -2518,7 +2518,7 @@ void* TSystemAllocator::Allocate(size_t size)
void* mmappedPtr;
while (true) {
auto currentPtr = CurrentPtr_.fetch_add(rawSize);
- Y_VERIFY(currentPtr + rawSize <= SystemZoneEnd);
+ Y_ABORT_UNLESS(currentPtr + rawSize <= SystemZoneEnd);
mmappedPtr = MappedMemoryManager->Map(
currentPtr,
rawSize,
@@ -4404,7 +4404,7 @@ Y_FORCE_INLINE TThreadState* TThreadManager::FindThreadState()
InitializeGlobals();
// InitializeGlobals must not allocate.
- Y_VERIFY(!ThreadState_);
+ Y_ABORT_UNLESS(!ThreadState_);
ThreadState_ = ThreadManager->AllocateThreadState();
(&ThreadControlWord_)->Parts.ThreadStateValid = true;