aboutsummaryrefslogtreecommitdiffstats
path: root/library/cpp/actors/core
diff options
context:
space:
mode:
authorDevtools Arcadia <arcadia-devtools@yandex-team.ru>2022-02-07 18:08:42 +0300
committerDevtools Arcadia <arcadia-devtools@mous.vla.yp-c.yandex.net>2022-02-07 18:08:42 +0300
commit1110808a9d39d4b808aef724c861a2e1a38d2a69 (patch)
treee26c9fed0de5d9873cce7e00bc214573dc2195b7 /library/cpp/actors/core
downloadydb-1110808a9d39d4b808aef724c861a2e1a38d2a69.tar.gz
intermediate changes
ref:cde9a383711a11544ce7e107a78147fb96cc4029
Diffstat (limited to 'library/cpp/actors/core')
-rw-r--r--library/cpp/actors/core/README.md99
-rw-r--r--library/cpp/actors/core/actor.cpp172
-rw-r--r--library/cpp/actors/core/actor.h530
-rw-r--r--library/cpp/actors/core/actor_bootstrapped.h37
-rw-r--r--library/cpp/actors/core/actor_coroutine.cpp165
-rw-r--r--library/cpp/actors/core/actor_coroutine.h174
-rw-r--r--library/cpp/actors/core/actor_coroutine_ut.cpp141
-rw-r--r--library/cpp/actors/core/actor_ut.cpp578
-rw-r--r--library/cpp/actors/core/actorid.cpp34
-rw-r--r--library/cpp/actors/core/actorid.h196
-rw-r--r--library/cpp/actors/core/actorsystem.cpp277
-rw-r--r--library/cpp/actors/core/actorsystem.h367
-rw-r--r--library/cpp/actors/core/actorsystem_ut.cpp45
-rw-r--r--library/cpp/actors/core/ask.cpp74
-rw-r--r--library/cpp/actors/core/ask.h18
-rw-r--r--library/cpp/actors/core/ask_ut.cpp131
-rw-r--r--library/cpp/actors/core/balancer.cpp293
-rw-r--r--library/cpp/actors/core/balancer.h27
-rw-r--r--library/cpp/actors/core/balancer_ut.cpp225
-rw-r--r--library/cpp/actors/core/buffer.cpp93
-rw-r--r--library/cpp/actors/core/buffer.h62
-rw-r--r--library/cpp/actors/core/callstack.cpp93
-rw-r--r--library/cpp/actors/core/callstack.h58
-rw-r--r--library/cpp/actors/core/config.h239
-rw-r--r--library/cpp/actors/core/cpu_manager.cpp108
-rw-r--r--library/cpp/actors/core/cpu_manager.h57
-rw-r--r--library/cpp/actors/core/cpu_state.h215
-rw-r--r--library/cpp/actors/core/defs.h69
-rw-r--r--library/cpp/actors/core/event.cpp38
-rw-r--r--library/cpp/actors/core/event.h344
-rw-r--r--library/cpp/actors/core/event_load.h112
-rw-r--r--library/cpp/actors/core/event_local.h74
-rw-r--r--library/cpp/actors/core/event_pb.cpp223
-rw-r--r--library/cpp/actors/core/event_pb.h500
-rw-r--r--library/cpp/actors/core/event_pb_payload_ut.cpp154
-rw-r--r--library/cpp/actors/core/event_pb_ut.cpp71
-rw-r--r--library/cpp/actors/core/events.h222
-rw-r--r--library/cpp/actors/core/events_undelivered.cpp60
-rw-r--r--library/cpp/actors/core/executelater.h87
-rw-r--r--library/cpp/actors/core/executor_pool_base.cpp168
-rw-r--r--library/cpp/actors/core/executor_pool_base.h49
-rw-r--r--library/cpp/actors/core/executor_pool_basic.cpp431
-rw-r--r--library/cpp/actors/core/executor_pool_basic.h111
-rw-r--r--library/cpp/actors/core/executor_pool_basic_ut.cpp435
-rw-r--r--library/cpp/actors/core/executor_pool_io.cpp151
-rw-r--r--library/cpp/actors/core/executor_pool_io.h49
-rw-r--r--library/cpp/actors/core/executor_pool_united.cpp1428
-rw-r--r--library/cpp/actors/core/executor_pool_united.h135
-rw-r--r--library/cpp/actors/core/executor_pool_united_ut.cpp338
-rw-r--r--library/cpp/actors/core/executor_thread.cpp563
-rw-r--r--library/cpp/actors/core/executor_thread.h112
-rw-r--r--library/cpp/actors/core/hfunc.h84
-rw-r--r--library/cpp/actors/core/interconnect.cpp170
-rw-r--r--library/cpp/actors/core/interconnect.h248
-rw-r--r--library/cpp/actors/core/invoke.h110
-rw-r--r--library/cpp/actors/core/io_dispatcher.cpp234
-rw-r--r--library/cpp/actors/core/io_dispatcher.h38
-rw-r--r--library/cpp/actors/core/lease.h56
-rw-r--r--library/cpp/actors/core/log.cpp753
-rw-r--r--library/cpp/actors/core/log.h369
-rw-r--r--library/cpp/actors/core/log_iface.h109
-rw-r--r--library/cpp/actors/core/log_settings.cpp230
-rw-r--r--library/cpp/actors/core/log_settings.h176
-rw-r--r--library/cpp/actors/core/log_ut.cpp185
-rw-r--r--library/cpp/actors/core/mailbox.cpp551
-rw-r--r--library/cpp/actors/core/mailbox.h553
-rw-r--r--library/cpp/actors/core/mailbox_queue_revolving.h214
-rw-r--r--library/cpp/actors/core/mailbox_queue_simple.h34
-rw-r--r--library/cpp/actors/core/memory_track.cpp38
-rw-r--r--library/cpp/actors/core/memory_track.h293
-rw-r--r--library/cpp/actors/core/memory_tracker.cpp103
-rw-r--r--library/cpp/actors/core/memory_tracker.h53
-rw-r--r--library/cpp/actors/core/memory_tracker_ut.cpp262
-rw-r--r--library/cpp/actors/core/mon.h234
-rw-r--r--library/cpp/actors/core/mon_stats.h147
-rw-r--r--library/cpp/actors/core/monotonic.cpp23
-rw-r--r--library/cpp/actors/core/monotonic.h111
-rw-r--r--library/cpp/actors/core/probes.cpp28
-rw-r--r--library/cpp/actors/core/probes.h176
-rw-r--r--library/cpp/actors/core/process_stats.cpp303
-rw-r--r--library/cpp/actors/core/process_stats.h66
-rw-r--r--library/cpp/actors/core/scheduler_actor.cpp279
-rw-r--r--library/cpp/actors/core/scheduler_actor.h29
-rw-r--r--library/cpp/actors/core/scheduler_actor_ut.cpp100
-rw-r--r--library/cpp/actors/core/scheduler_basic.cpp274
-rw-r--r--library/cpp/actors/core/scheduler_basic.h81
-rw-r--r--library/cpp/actors/core/scheduler_cookie.cpp84
-rw-r--r--library/cpp/actors/core/scheduler_cookie.h78
-rw-r--r--library/cpp/actors/core/scheduler_queue.h120
-rw-r--r--library/cpp/actors/core/servicemap.h168
-rw-r--r--library/cpp/actors/core/ut/ya.make46
-rw-r--r--library/cpp/actors/core/worker_context.cpp7
-rw-r--r--library/cpp/actors/core/worker_context.h175
-rw-r--r--library/cpp/actors/core/ya.make123
94 files changed, 17947 insertions, 0 deletions
diff --git a/library/cpp/actors/core/README.md b/library/cpp/actors/core/README.md
new file mode 100644
index 0000000000..439a8dd459
--- /dev/null
+++ b/library/cpp/actors/core/README.md
@@ -0,0 +1,99 @@
+## Memory tracker
+
+https://a.yandex-team.ru/arc/trunk/arcadia/library/cpp/actors/core/memory_track.h
+
+Использование:
+
+* отслеживание аллокаций экземпляров конкретного класса через new/delete и new[]/delete[]
+* отслеживание аллокаций в контейнерах
+* ручное отслеживание моментов аллокации/деаллокации
+
+----
+
+### Отслеживание аллокаций класса через new/delete
+
+Использование с автоматически генерируемой меткой:
+
+```cpp
+#include <library/cpp/actors/core/memory_track.h>
+
+struct TTypeLabeled
+ : public NActors::NMemory::TTrack<TTypeLabeled>
+{
+ char payload[16];
+};
+```
+
+Использование с пользовательским именем метки:
+
+```cpp
+#include <library/cpp/actors/core/memory_track.h>
+
+static const char NamedLabel[] = "NamedLabel";
+
+struct TNameLabeled
+ : public NActors::NMemory::TTrack<TNameLabeled, NamedLabel>
+{
+ char payload[32];
+};
+```
+
+----
+
+### Отслеживание аллокаций в контейнерах
+
+```cpp
+#include <library/cpp/actors/core/memory_track.h>
+
+static const char InContainerLabel[] = "InContainerLabel";
+
+struct TInContainer {
+ char payload[16];
+};
+
+std::vector<TInContainer, NActors::NMemory::TAlloc<TInContainer>> vecT;
+
+std::vector<TInContainer, NActors::NMemory::TAlloc<TInContainer, InContainerLabel>> vecN;
+
+using TKey = int;
+
+std::map<TKey, TInContainer, std::less<TKey>,
+ NActors::NMemory::TAlloc<std::pair<const TKey, TInContainer>>> mapT;
+
+std::map<TKey, TInContainer, std::less<TKey>,
+ NActors::NMemory::TAlloc<std::pair<const TKey, TInContainer>, InContainerLabel>> mapN;
+
+std::unordered_map<TKey, TInContainer, std::hash<TKey>, std::equal_to<TKey>,
+ NActors::NMemory::TAlloc<std::pair<const TKey, TInContainer>>> umapT;
+
+std::unordered_map<TKey, TInContainer, std::hash<TKey>, std::equal_to<TKey>,
+ NActors::NMemory::TAlloc<std::pair<const TKey, TInContainer>, InContainerLabel>> umapN;
+```
+
+----
+
+### Ручное отслеживание аллокаций/деаллокаций
+
+```cpp
+#include <library/cpp/actors/core/memory_track.h>
+
+static const char ManualLabel[] = "ManualLabel";
+
+...
+NActors::NMemory::TLabel<ManualLabel>::Add(size);
+
+...
+NActors::NMemory::TLabel<ManualLabel>::Sub(size);
+```
+
+----
+
+### Собираемые метрики
+
+Сервис **utils**, пользовательская метка **label**, сенсоры:
+
+- MT/Count: количество аллокаций в моменте
+- MT/Memory: аллоцированная память в моменте
+- MT/PeakCount: пиковое значение количества аллокаций (сэмплится с фиксированной частотой)
+- MT/PeakMemory: пиковое значение аллоцированной памяти
+
diff --git a/library/cpp/actors/core/actor.cpp b/library/cpp/actors/core/actor.cpp
new file mode 100644
index 0000000000..6f9ba6a42b
--- /dev/null
+++ b/library/cpp/actors/core/actor.cpp
@@ -0,0 +1,172 @@
+#include "actor.h"
+#include "executor_thread.h"
+#include "mailbox.h"
+#include <library/cpp/actors/util/datetime.h>
+
+namespace NActors {
+ Y_POD_THREAD(TActivationContext*)
+ TlsActivationContext((TActivationContext*)nullptr);
+
+ bool TActorContext::Send(const TActorId& recipient, IEventBase* ev, ui32 flags, ui64 cookie, NWilson::TTraceId traceId) const {
+ return Send(new IEventHandle(recipient, SelfID, ev, flags, cookie, nullptr, std::move(traceId)));
+ }
+
+ bool TActorContext::Send(TAutoPtr<IEventHandle> ev) const {
+ return ExecutorThread.Send(ev);
+ }
+
+ void IActor::Registered(TActorSystem* sys, const TActorId& owner) {
+ // fallback to legacy method, do not use it anymore
+ if (auto eh = AfterRegister(SelfId(), owner))
+ sys->Send(eh);
+ }
+
+ void IActor::Describe(IOutputStream &out) const noexcept {
+ SelfActorId.Out(out);
+ }
+
+ bool IActor::Send(const TActorId& recipient, IEventBase* ev, ui32 flags, ui64 cookie, NWilson::TTraceId traceId) const noexcept {
+ return SelfActorId.Send(recipient, ev, flags, cookie, std::move(traceId));
+ }
+
+ bool TActivationContext::Send(TAutoPtr<IEventHandle> ev) {
+ return TlsActivationContext->ExecutorThread.Send(ev);
+ }
+
+ void TActivationContext::Schedule(TInstant deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie) {
+ TlsActivationContext->ExecutorThread.Schedule(deadline, ev, cookie);
+ }
+
+ void TActivationContext::Schedule(TMonotonic deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie) {
+ TlsActivationContext->ExecutorThread.Schedule(deadline, ev, cookie);
+ }
+
+ void TActivationContext::Schedule(TDuration delta, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie) {
+ TlsActivationContext->ExecutorThread.Schedule(delta, ev, cookie);
+ }
+
+ bool TActorIdentity::Send(const TActorId& recipient, IEventBase* ev, ui32 flags, ui64 cookie, NWilson::TTraceId traceId) const {
+ return TActivationContext::Send(new IEventHandle(recipient, *this, ev, flags, cookie, nullptr, std::move(traceId)));
+ }
+
+ void TActorIdentity::Schedule(TInstant deadline, IEventBase* ev, ISchedulerCookie* cookie) const {
+ return TActivationContext::Schedule(deadline, new IEventHandle(*this, {}, ev), cookie);
+ }
+
+ void TActorIdentity::Schedule(TMonotonic deadline, IEventBase* ev, ISchedulerCookie* cookie) const {
+ return TActivationContext::Schedule(deadline, new IEventHandle(*this, {}, ev), cookie);
+ }
+
+ void TActorIdentity::Schedule(TDuration delta, IEventBase* ev, ISchedulerCookie* cookie) const {
+ return TActivationContext::Schedule(delta, new IEventHandle(*this, {}, ev), cookie);
+ }
+
+ TActorId TActivationContext::RegisterWithSameMailbox(IActor* actor, TActorId parentId) {
+ Y_VERIFY_DEBUG(parentId);
+ auto& ctx = *TlsActivationContext;
+ return ctx.ExecutorThread.RegisterActor(actor, &ctx.Mailbox, parentId.Hint(), parentId);
+ }
+
+ TActorId TActorContext::RegisterWithSameMailbox(IActor* actor) const {
+ return ExecutorThread.RegisterActor(actor, &Mailbox, SelfID.Hint(), SelfID);
+ }
+
+ TActorId IActor::RegisterWithSameMailbox(IActor* actor) const noexcept {
+ return TlsActivationContext->ExecutorThread.RegisterActor(actor, &TlsActivationContext->Mailbox, SelfActorId.Hint(), SelfActorId);
+ }
+
+ TActorId TActivationContext::Register(IActor* actor, TActorId parentId, TMailboxType::EType mailboxType, ui32 poolId) {
+ return TlsActivationContext->ExecutorThread.RegisterActor(actor, mailboxType, poolId, parentId);
+ }
+
+ TActorId TActivationContext::InterconnectProxy(ui32 destinationNodeId) {
+ return TlsActivationContext->ExecutorThread.ActorSystem->InterconnectProxy(destinationNodeId);
+ }
+
+ TActorSystem* TActivationContext::ActorSystem() {
+ return TlsActivationContext->ExecutorThread.ActorSystem;
+ }
+
+ i64 TActivationContext::GetCurrentEventTicks() {
+ return GetCycleCountFast() - TlsActivationContext->EventStart;
+ }
+
+ double TActivationContext::GetCurrentEventTicksAsSeconds() {
+ return NHPTimer::GetSeconds(GetCurrentEventTicks());
+ }
+
+ TActorId TActorContext::Register(IActor* actor, TMailboxType::EType mailboxType, ui32 poolId) const {
+ return ExecutorThread.RegisterActor(actor, mailboxType, poolId, SelfID);
+ }
+
+ TActorId IActor::Register(IActor* actor, TMailboxType::EType mailboxType, ui32 poolId) const noexcept {
+ return TlsActivationContext->ExecutorThread.RegisterActor(actor, mailboxType, poolId, SelfActorId);
+ }
+
+ void TActorContext::Schedule(TInstant deadline, IEventBase* ev, ISchedulerCookie* cookie) const {
+ ExecutorThread.Schedule(deadline, new IEventHandle(SelfID, TActorId(), ev), cookie);
+ }
+
+ void TActorContext::Schedule(TMonotonic deadline, IEventBase* ev, ISchedulerCookie* cookie) const {
+ ExecutorThread.Schedule(deadline, new IEventHandle(SelfID, TActorId(), ev), cookie);
+ }
+
+ void TActorContext::Schedule(TDuration delta, IEventBase* ev, ISchedulerCookie* cookie) const {
+ ExecutorThread.Schedule(delta, new IEventHandle(SelfID, TActorId(), ev), cookie);
+ }
+
+ void IActor::Schedule(TInstant deadline, IEventBase* ev, ISchedulerCookie* cookie) const noexcept {
+ TlsActivationContext->ExecutorThread.Schedule(deadline, new IEventHandle(SelfActorId, TActorId(), ev), cookie);
+ }
+
+ void IActor::Schedule(TMonotonic deadline, IEventBase* ev, ISchedulerCookie* cookie) const noexcept {
+ TlsActivationContext->ExecutorThread.Schedule(deadline, new IEventHandle(SelfActorId, TActorId(), ev), cookie);
+ }
+
+ void IActor::Schedule(TDuration delta, IEventBase* ev, ISchedulerCookie* cookie) const noexcept {
+ TlsActivationContext->ExecutorThread.Schedule(delta, new IEventHandle(SelfActorId, TActorId(), ev), cookie);
+ }
+
+ TInstant TActivationContext::Now() {
+ return TlsActivationContext->ExecutorThread.ActorSystem->Timestamp();
+ }
+
+ TMonotonic TActivationContext::Monotonic() {
+ return TlsActivationContext->ExecutorThread.ActorSystem->Monotonic();
+ }
+
+ TInstant TActorContext::Now() const {
+ return ExecutorThread.ActorSystem->Timestamp();
+ }
+
+ TMonotonic TActorContext::Monotonic() const {
+ return ExecutorThread.ActorSystem->Monotonic();
+ }
+
+ NLog::TSettings* TActivationContext::LoggerSettings() const {
+ return ExecutorThread.ActorSystem->LoggerSettings();
+ }
+
+ std::pair<ui32, ui32> TActorContext::CountMailboxEvents(ui32 maxTraverse) const {
+ return Mailbox.CountMailboxEvents(SelfID.LocalId(), maxTraverse);
+ }
+
+ std::pair<ui32, ui32> IActor::CountMailboxEvents(ui32 maxTraverse) const {
+ return TlsActivationContext->Mailbox.CountMailboxEvents(SelfActorId.LocalId(), maxTraverse);
+ }
+
+ void IActor::Die(const TActorContext& ctx) {
+ if (ctx.SelfID)
+ Y_VERIFY(ctx.SelfID == SelfActorId);
+ PassAway();
+ }
+
+ void IActor::PassAway() {
+ auto& cx = *TlsActivationContext;
+ cx.ExecutorThread.UnregisterActor(&cx.Mailbox, SelfActorId.LocalId());
+ }
+
+ double IActor::GetElapsedTicksAsSeconds() const {
+ return NHPTimer::GetSeconds(ElapsedTicks);
+ }
+}
diff --git a/library/cpp/actors/core/actor.h b/library/cpp/actors/core/actor.h
new file mode 100644
index 0000000000..ed29bd14b9
--- /dev/null
+++ b/library/cpp/actors/core/actor.h
@@ -0,0 +1,530 @@
+#pragma once
+
+#include "event.h"
+#include "monotonic.h"
+#include <util/system/tls.h>
+#include <library/cpp/actors/util/local_process_key.h>
+
+namespace NActors {
+ class TActorSystem;
+ class TMailboxTable;
+ struct TMailboxHeader;
+
+ class TExecutorThread;
+ class IActor;
+ class ISchedulerCookie;
+
+ namespace NLog {
+ struct TSettings;
+ }
+
+ struct TActorContext;
+
+ struct TActivationContext {
+ public:
+ TMailboxHeader& Mailbox;
+ TExecutorThread& ExecutorThread;
+ const NHPTimer::STime EventStart;
+
+ protected:
+ explicit TActivationContext(TMailboxHeader& mailbox, TExecutorThread& executorThread, NHPTimer::STime eventStart)
+ : Mailbox(mailbox)
+ , ExecutorThread(executorThread)
+ , EventStart(eventStart)
+ {
+ }
+
+ public:
+ static bool Send(TAutoPtr<IEventHandle> ev);
+
+ /**
+ * Schedule one-shot event that will be send at given time point in the future.
+ *
+ * @param deadline the wallclock time point in future when event must be send
+ * @param ev the event to send
+ * @param cookie cookie that will be piggybacked with event
+ */
+ static void Schedule(TInstant deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie = nullptr);
+
+ /**
+ * Schedule one-shot event that will be send at given time point in the future.
+ *
+ * @param deadline the monotonic time point in future when event must be send
+ * @param ev the event to send
+ * @param cookie cookie that will be piggybacked with event
+ */
+ static void Schedule(TMonotonic deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie = nullptr);
+
+ /**
+ * Schedule one-shot event that will be send after given delay.
+ *
+ * @param delta the time from now to delay event sending
+ * @param ev the event to send
+ * @param cookie cookie that will be piggybacked with event
+ */
+ static void Schedule(TDuration delta, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie = nullptr);
+
+ static TInstant Now();
+ static TMonotonic Monotonic();
+ NLog::TSettings* LoggerSettings() const;
+
+ // register new actor in ActorSystem on new fresh mailbox.
+ static TActorId Register(IActor* actor, TActorId parentId = TActorId(), TMailboxType::EType mailboxType = TMailboxType::HTSwap, ui32 poolId = Max<ui32>());
+
+ // Register new actor in ActorSystem on same _mailbox_ as current actor.
+ // There is one thread per mailbox to execute actor, which mean
+ // no _cpu core scalability_ for such actors.
+ // This method of registration can be usefull if multiple actors share
+ // some memory.
+ static TActorId RegisterWithSameMailbox(IActor* actor, TActorId parentId);
+
+ static const TActorContext& AsActorContext();
+ static TActorContext ActorContextFor(TActorId id);
+
+ static TActorId InterconnectProxy(ui32 nodeid);
+ static TActorSystem* ActorSystem();
+
+ static i64 GetCurrentEventTicks();
+ static double GetCurrentEventTicksAsSeconds();
+ };
+
+ struct TActorContext: public TActivationContext {
+ const TActorId SelfID;
+
+ explicit TActorContext(TMailboxHeader& mailbox, TExecutorThread& executorThread, NHPTimer::STime eventStart, const TActorId& selfID)
+ : TActivationContext(mailbox, executorThread, eventStart)
+ , SelfID(selfID)
+ {
+ }
+
+ bool Send(const TActorId& recipient, IEventBase* ev, ui32 flags = 0, ui64 cookie = 0, NWilson::TTraceId traceId = {}) const;
+ template <typename TEvent>
+ bool Send(const TActorId& recipient, THolder<TEvent> ev, ui32 flags = 0, ui64 cookie = 0, NWilson::TTraceId traceId = {}) const {
+ return Send(recipient, static_cast<IEventBase*>(ev.Release()), flags, cookie, std::move(traceId));
+ }
+ bool Send(TAutoPtr<IEventHandle> ev) const;
+
+ TInstant Now() const;
+ TMonotonic Monotonic() const;
+
+ /**
+ * Schedule one-shot event that will be send at given time point in the future.
+ *
+ * @param deadline the wallclock time point in future when event must be send
+ * @param ev the event to send
+ * @param cookie cookie that will be piggybacked with event
+ */
+ void Schedule(TInstant deadline, IEventBase* ev, ISchedulerCookie* cookie = nullptr) const;
+
+ /**
+ * Schedule one-shot event that will be send at given time point in the future.
+ *
+ * @param deadline the monotonic time point in future when event must be send
+ * @param ev the event to send
+ * @param cookie cookie that will be piggybacked with event
+ */
+ void Schedule(TMonotonic deadline, IEventBase* ev, ISchedulerCookie* cookie = nullptr) const;
+
+ /**
+ * Schedule one-shot event that will be send after given delay.
+ *
+ * @param delta the time from now to delay event sending
+ * @param ev the event to send
+ * @param cookie cookie that will be piggybacked with event
+ */
+ void Schedule(TDuration delta, IEventBase* ev, ISchedulerCookie* cookie = nullptr) const;
+
+ TActorContext MakeFor(const TActorId& otherId) const {
+ return TActorContext(Mailbox, ExecutorThread, EventStart, otherId);
+ }
+
+ // register new actor in ActorSystem on new fresh mailbox.
+ TActorId Register(IActor* actor, TMailboxType::EType mailboxType = TMailboxType::HTSwap, ui32 poolId = Max<ui32>()) const;
+
+ // Register new actor in ActorSystem on same _mailbox_ as current actor.
+ // There is one thread per mailbox to execute actor, which mean
+ // no _cpu core scalability_ for such actors.
+ // This method of registration can be usefull if multiple actors share
+ // some memory.
+ TActorId RegisterWithSameMailbox(IActor* actor) const;
+
+ std::pair<ui32, ui32> CountMailboxEvents(ui32 maxTraverse = Max<ui32>()) const;
+ };
+
+ extern Y_POD_THREAD(TActivationContext*) TlsActivationContext;
+
+ struct TActorIdentity: public TActorId {
+ explicit TActorIdentity(TActorId actorId)
+ : TActorId(actorId)
+ {
+ }
+
+ void operator=(TActorId actorId) {
+ *this = TActorIdentity(actorId);
+ }
+
+ bool Send(const TActorId& recipient, IEventBase* ev, ui32 flags = 0, ui64 cookie = 0, NWilson::TTraceId traceId = {}) const;
+ void Schedule(TInstant deadline, IEventBase* ev, ISchedulerCookie* cookie = nullptr) const;
+ void Schedule(TMonotonic deadline, IEventBase* ev, ISchedulerCookie* cookie = nullptr) const;
+ void Schedule(TDuration delta, IEventBase* ev, ISchedulerCookie* cookie = nullptr) const;
+ };
+
+ class IActor;
+
+ class IActorOps : TNonCopyable {
+ public:
+ virtual void Describe(IOutputStream&) const noexcept = 0;
+ virtual bool Send(const TActorId& recipient, IEventBase*, ui32 flags = 0, ui64 cookie = 0, NWilson::TTraceId traceId = {}) const noexcept = 0;
+
+ /**
+ * Schedule one-shot event that will be send at given time point in the future.
+ *
+ * @param deadline the wallclock time point in future when event must be send
+ * @param ev the event to send
+ * @param cookie cookie that will be piggybacked with event
+ */
+ virtual void Schedule(TInstant deadline, IEventBase* ev, ISchedulerCookie* cookie = nullptr) const noexcept = 0;
+
+ /**
+ * Schedule one-shot event that will be send at given time point in the future.
+ *
+ * @param deadline the monotonic time point in future when event must be send
+ * @param ev the event to send
+ * @param cookie cookie that will be piggybacked with event
+ */
+ virtual void Schedule(TMonotonic deadline, IEventBase* ev, ISchedulerCookie* cookie = nullptr) const noexcept = 0;
+
+ /**
+ * Schedule one-shot event that will be send after given delay.
+ *
+ * @param delta the time from now to delay event sending
+ * @param ev the event to send
+ * @param cookie cookie that will be piggybacked with event
+ */
+ virtual void Schedule(TDuration delta, IEventBase* ev, ISchedulerCookie* cookie = nullptr) const noexcept = 0;
+
+ virtual TActorId Register(IActor*, TMailboxType::EType mailboxType = TMailboxType::HTSwap, ui32 poolId = Max<ui32>()) const noexcept = 0;
+ virtual TActorId RegisterWithSameMailbox(IActor*) const noexcept = 0;
+ };
+
+ class TDecorator;
+
+ class IActor : protected IActorOps {
+ public:
+ typedef void (IActor::*TReceiveFunc)(TAutoPtr<IEventHandle>& ev, const TActorContext& ctx);
+
+ private:
+ TReceiveFunc StateFunc;
+ TActorIdentity SelfActorId;
+ i64 ElapsedTicks;
+ ui64 HandledEvents;
+
+ friend void DoActorInit(TActorSystem*, IActor*, const TActorId&, const TActorId&);
+ friend class TDecorator;
+
+ public:
+ /// @sa services.proto NKikimrServices::TActivity::EType
+ enum EActorActivity {
+ OTHER = 0,
+ ACTOR_SYSTEM = 1,
+ ACTORLIB_COMMON = 2,
+ ACTORLIB_STATS = 3,
+ LOG_ACTOR = 4,
+ INTERCONNECT_PROXY_TCP = 12,
+ INTERCONNECT_SESSION_TCP = 13,
+ INTERCONNECT_COMMON = 171,
+ SELF_PING_ACTOR = 207,
+ TEST_ACTOR_RUNTIME = 283,
+ INTERCONNECT_HANDSHAKE = 284,
+ INTERCONNECT_POLLER = 285,
+ INTERCONNECT_SESSION_KILLER = 286,
+ ACTOR_SYSTEM_SCHEDULER_ACTOR = 312,
+ ACTOR_FUTURE_CALLBACK = 337,
+ INTERCONNECT_MONACTOR = 362,
+ INTERCONNECT_LOAD_ACTOR = 376,
+ INTERCONNECT_LOAD_RESPONDER = 377,
+ NAMESERVICE = 450,
+ DNS_RESOLVER = 481,
+ INTERCONNECT_PROXY_WRAPPER = 546,
+ };
+
+ using EActivityType = EActorActivity;
+ ui32 ActivityType;
+
+ protected:
+ IActor(TReceiveFunc stateFunc, ui32 activityType = OTHER)
+ : StateFunc(stateFunc)
+ , SelfActorId(TActorId())
+ , ElapsedTicks(0)
+ , HandledEvents(0)
+ , ActivityType(activityType)
+ {
+ }
+
+ public:
+ virtual ~IActor() {
+ } // must not be called for registered actors, see Die method instead
+
+ protected:
+ virtual void Die(const TActorContext& ctx); // would unregister actor so call exactly once and only from inside of message processing
+ virtual void PassAway();
+
+ public:
+ template <typename T>
+ void Become(T stateFunc) {
+ StateFunc = static_cast<TReceiveFunc>(stateFunc);
+ }
+
+ template <typename T, typename... TArgs>
+ void Become(T stateFunc, const TActorContext& ctx, TArgs&&... args) {
+ StateFunc = static_cast<TReceiveFunc>(stateFunc);
+ ctx.Schedule(std::forward<TArgs>(args)...);
+ }
+
+ template <typename T, typename... TArgs>
+ void Become(T stateFunc, TArgs&&... args) {
+ StateFunc = static_cast<TReceiveFunc>(stateFunc);
+ Schedule(std::forward<TArgs>(args)...);
+ }
+
+ protected:
+ void SetActivityType(ui32 activityType) {
+ ActivityType = activityType;
+ }
+
+ public:
+ TReceiveFunc CurrentStateFunc() const {
+ return StateFunc;
+ }
+
+ // NOTE: exceptions must not escape state function but if an exception hasn't be caught
+ // by the actor then we want to crash an see the stack
+ void Receive(TAutoPtr<IEventHandle>& ev, const TActorContext& ctx) {
+ (this->*StateFunc)(ev, ctx);
+ HandledEvents++;
+ }
+
+ // must be called to wrap any call trasitions from one actor to another
+ template<typename TActor, typename TMethod, typename... TArgs>
+ static decltype((std::declval<TActor>().*std::declval<TMethod>())(std::declval<TArgs>()...))
+ InvokeOtherActor(TActor& actor, TMethod&& method, TArgs&&... args) {
+ struct TRecurseContext : TActorContext {
+ TActivationContext *Prev;
+ TRecurseContext(const TActorId& actorId)
+ : TActorContext(TActivationContext::ActorContextFor(actorId))
+ , Prev(TlsActivationContext)
+ {
+ TlsActivationContext = this;
+ }
+ ~TRecurseContext() {
+ TlsActivationContext = Prev;
+ }
+ } context(actor.SelfId());
+ return (actor.*method)(std::forward<TArgs>(args)...);
+ }
+
+ virtual void Registered(TActorSystem* sys, const TActorId& owner);
+
+ virtual TAutoPtr<IEventHandle> AfterRegister(const TActorId& self, const TActorId& parentId) {
+ Y_UNUSED(self);
+ Y_UNUSED(parentId);
+ return TAutoPtr<IEventHandle>();
+ }
+
+ i64 GetElapsedTicks() const {
+ return ElapsedTicks;
+ }
+ double GetElapsedTicksAsSeconds() const;
+ void AddElapsedTicks(i64 ticks) {
+ ElapsedTicks += ticks;
+ }
+ auto GetActivityType() const {
+ return ActivityType;
+ }
+ ui64 GetHandledEvents() const {
+ return HandledEvents;
+ }
+ TActorIdentity SelfId() const {
+ return SelfActorId;
+ }
+
+ protected:
+ void Describe(IOutputStream&) const noexcept override;
+ bool Send(const TActorId& recipient, IEventBase* ev, ui32 flags = 0, ui64 cookie = 0, NWilson::TTraceId traceId = {}) const noexcept final;
+ template <typename TEvent>
+ bool Send(const TActorId& recipient, THolder<TEvent> ev, ui32 flags = 0, ui64 cookie = 0, NWilson::TTraceId traceId = {}) const{
+ return Send(recipient, static_cast<IEventBase*>(ev.Release()), flags, cookie, std::move(traceId));
+ }
+
+ template <class TEvent, class ... TEventArgs>
+ bool Send(TActorId recipient, TEventArgs&& ... args) const {
+ return Send(recipient, MakeHolder<TEvent>(std::forward<TEventArgs>(args)...));
+ }
+
+ void Schedule(TInstant deadline, IEventBase* ev, ISchedulerCookie* cookie = nullptr) const noexcept final;
+ void Schedule(TMonotonic deadline, IEventBase* ev, ISchedulerCookie* cookie = nullptr) const noexcept final;
+ void Schedule(TDuration delta, IEventBase* ev, ISchedulerCookie* cookie = nullptr) const noexcept final;
+
+ // register new actor in ActorSystem on new fresh mailbox.
+ TActorId Register(IActor* actor, TMailboxType::EType mailboxType = TMailboxType::HTSwap, ui32 poolId = Max<ui32>()) const noexcept final;
+
+ // Register new actor in ActorSystem on same _mailbox_ as current actor.
+ // There is one thread per mailbox to execute actor, which mean
+ // no _cpu core scalability_ for such actors.
+ // This method of registration can be usefull if multiple actors share
+ // some memory.
+ TActorId RegisterWithSameMailbox(IActor* actor) const noexcept final;
+
+ std::pair<ui32, ui32> CountMailboxEvents(ui32 maxTraverse = Max<ui32>()) const;
+
+ private:
+ void ChangeSelfId(TActorId actorId) {
+ SelfActorId = actorId;
+ }
+ };
+
+ struct TActorActivityTag {};
+
+ inline size_t GetActivityTypeCount() {
+ return TLocalProcessKeyState<TActorActivityTag>::GetInstance().GetCount();
+ }
+
+ inline TStringBuf GetActivityTypeName(size_t index) {
+ return TLocalProcessKeyState<TActorActivityTag>::GetInstance().GetNameByIndex(index);
+ }
+
+ template <typename TDerived>
+ class TActor: public IActor {
+ private:
+ template <typename T, typename = const char*>
+ struct HasActorName: std::false_type { };
+ template <typename T>
+ struct HasActorName<T, decltype((void)T::ActorName, (const char*)nullptr)>: std::true_type { };
+
+ static ui32 GetActivityTypeIndex() {
+ if constexpr(HasActorName<TDerived>::value) {
+ return TLocalProcessKey<TActorActivityTag, TDerived::ActorName>::GetIndex();
+ } else {
+ using TActorActivity = decltype(((TDerived*)nullptr)->ActorActivityType());
+ // if constexpr(std::is_enum<TActorActivity>::value) {
+ return TEnumProcessKey<TActorActivityTag, TActorActivity>::GetIndex(
+ TDerived::ActorActivityType());
+ //} else {
+ // for int, ui32, ...
+ // return TEnumProcessKey<TActorActivityTag, IActor::EActorActivity>::GetIndex(
+ // static_cast<IActor::EActorActivity>(TDerived::ActorActivityType()));
+ //}
+ }
+ }
+
+ protected:
+ //* Comment this function to find unmarked activities
+ static constexpr IActor::EActivityType ActorActivityType() {
+ return EActorActivity::OTHER;
+ } //*/
+
+ // static constexpr char ActorName[] = "UNNAMED";
+
+ TActor(void (TDerived::*func)(TAutoPtr<IEventHandle>& ev, const TActorContext& ctx), ui32 activityType = GetActivityTypeIndex())
+ : IActor(static_cast<TReceiveFunc>(func), activityType)
+ { }
+
+ public:
+ typedef TDerived TThis;
+ };
+
+
+#define STFUNC_SIG TAutoPtr< ::NActors::IEventHandle>&ev, const ::NActors::TActorContext &ctx
+#define STATEFN_SIG TAutoPtr<::NActors::IEventHandle>& ev
+#define STFUNC(funcName) void funcName(TAutoPtr< ::NActors::IEventHandle>& ev, const ::NActors::TActorContext& ctx)
+#define STATEFN(funcName) void funcName(TAutoPtr< ::NActors::IEventHandle>& ev, const ::NActors::TActorContext& )
+
+#define STRICT_STFUNC(NAME, HANDLERS) \
+ void NAME(STFUNC_SIG) { \
+ Y_UNUSED(ctx); \
+ switch (const ui32 etype = ev->GetTypeRewrite()) { \
+ HANDLERS \
+ default: \
+ Y_VERIFY_DEBUG(false, "%s: unexpected message type 0x%08" PRIx32, __func__, etype); \
+ } \
+ }
+
+ inline const TActorContext& TActivationContext::AsActorContext() {
+ TActivationContext* tls = TlsActivationContext;
+ return *static_cast<TActorContext*>(tls);
+ }
+
+ inline TActorContext TActivationContext::ActorContextFor(TActorId id) {
+ auto& tls = *TlsActivationContext;
+ return TActorContext(tls.Mailbox, tls.ExecutorThread, tls.EventStart, id);
+ }
+
+ class TDecorator : public IActor {
+ protected:
+ THolder<IActor> Actor;
+
+ public:
+ TDecorator(THolder<IActor>&& actor)
+ : IActor(static_cast<TReceiveFunc>(&TDecorator::State), actor->GetActivityType())
+ , Actor(std::move(actor))
+ {
+ }
+
+ void Registered(TActorSystem* sys, const TActorId& owner) override {
+ Actor->ChangeSelfId(SelfId());
+ Actor->Registered(sys, owner);
+ }
+
+ virtual bool DoBeforeReceiving(TAutoPtr<IEventHandle>& /*ev*/, const TActorContext& /*ctx*/) {
+ return true;
+ }
+
+ virtual void DoAfterReceiving(const TActorContext& /*ctx*/)
+ {
+ }
+
+ STFUNC(State) {
+ if (DoBeforeReceiving(ev, ctx)) {
+ Actor->Receive(ev, ctx);
+ DoAfterReceiving(ctx);
+ }
+ }
+ };
+
+ // TTestDecorator doesn't work with the real actor system
+ struct TTestDecorator : public TDecorator {
+ TTestDecorator(THolder<IActor>&& actor)
+ : TDecorator(std::move(actor))
+ {
+ }
+
+ virtual ~TTestDecorator() = default;
+
+ // This method must be called in the test actor system
+ bool BeforeSending(TAutoPtr<IEventHandle>& ev)
+ {
+ bool send = true;
+ TTestDecorator *decorator = dynamic_cast<TTestDecorator*>(Actor.Get());
+ if (decorator) {
+ send = decorator->BeforeSending(ev);
+ }
+ return send && ev && DoBeforeSending(ev);
+ }
+
+ virtual bool DoBeforeSending(TAutoPtr<IEventHandle>& /*ev*/) {
+ return true;
+ }
+ };
+}
+
+template <>
+inline void Out<NActors::TActorIdentity>(IOutputStream& o, const NActors::TActorIdentity& x) {
+ return x.Out(o);
+}
+
+template <>
+struct THash<NActors::TActorIdentity> {
+ inline ui64 operator()(const NActors::TActorIdentity& x) const {
+ return x.Hash();
+ }
+};
diff --git a/library/cpp/actors/core/actor_bootstrapped.h b/library/cpp/actors/core/actor_bootstrapped.h
new file mode 100644
index 0000000000..a37887c939
--- /dev/null
+++ b/library/cpp/actors/core/actor_bootstrapped.h
@@ -0,0 +1,37 @@
+#pragma once
+
+#include "actor.h"
+#include "events.h"
+
+namespace NActors {
+ template<typename T> struct dependent_false : std::false_type {};
+
+ template<typename TDerived>
+ class TActorBootstrapped : public TActor<TDerived> {
+ protected:
+ TAutoPtr<IEventHandle> AfterRegister(const TActorId& self, const TActorId& parentId) override {
+ return new IEventHandle(TEvents::TSystem::Bootstrap, 0, self, parentId, {}, 0);
+ }
+
+ STFUNC(StateBootstrap) {
+ Y_VERIFY(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&>) {
+ self.Bootstrap(ctx);
+ } else if constexpr (std::is_invocable_v<T, TDerived, const TActorId&, const TActorContext&>) {
+ self.Bootstrap(ev->Sender, ctx);
+ } else if constexpr (std::is_invocable_v<T, TDerived>) {
+ self.Bootstrap();
+ } else if constexpr (std::is_invocable_v<T, TDerived, const TActorId&>) {
+ self.Bootstrap(ev->Sender);
+ } else {
+ static_assert(dependent_false<TDerived>::value, "No correct Bootstrap() signature");
+ }
+ }
+
+ TActorBootstrapped()
+ : TActor<TDerived>(&TDerived::StateBootstrap)
+ {}
+ };
+}
diff --git a/library/cpp/actors/core/actor_coroutine.cpp b/library/cpp/actors/core/actor_coroutine.cpp
new file mode 100644
index 0000000000..0ab4d2b24d
--- /dev/null
+++ b/library/cpp/actors/core/actor_coroutine.cpp
@@ -0,0 +1,165 @@
+#include "actor_coroutine.h"
+#include "executor_thread.h"
+
+#include <util/system/sanitizers.h>
+#include <util/system/type_name.h>
+
+namespace NActors {
+ static constexpr size_t StackOverflowGap = 4096;
+ static char GoodStack[StackOverflowGap];
+
+ static struct TInitGoodStack {
+ TInitGoodStack() {
+ // fill stack with some pseudo-random pattern
+ for (size_t k = 0; k < StackOverflowGap; ++k) {
+ GoodStack[k] = k + k * 91;
+ }
+ }
+ } initGoodStack;
+
+ TActorCoroImpl::TActorCoroImpl(size_t stackSize, bool allowUnhandledPoisonPill, bool allowUnhandledDtor)
+ : Stack(stackSize)
+ , AllowUnhandledPoisonPill(allowUnhandledPoisonPill)
+ , AllowUnhandledDtor(allowUnhandledDtor)
+ , FiberClosure{this, TArrayRef(Stack.Begin(), Stack.End())}
+ , FiberContext(FiberClosure)
+ {
+#ifndef NDEBUG
+ char* p;
+#if STACK_GROW_DOWN
+ p = Stack.Begin();
+#else
+ p = Stack.End() - StackOverflowGap;
+#endif
+ memcpy(p, GoodStack, StackOverflowGap);
+#endif
+ }
+
+ TActorCoroImpl::~TActorCoroImpl() {
+ if (!Finished && !NSan::TSanIsOn()) { // only resume when we have bootstrapped and Run() was entered and not yet finished; in other case simply terminate
+ Y_VERIFY(!PendingEvent);
+ Resume();
+ }
+ }
+
+ bool TActorCoroImpl::Send(TAutoPtr<IEventHandle> ev) {
+ return GetActorContext().ExecutorThread.Send(ev);
+ }
+
+ THolder<IEventHandle> TActorCoroImpl::WaitForEvent(TInstant deadline) {
+ const ui64 cookie = ++WaitCookie;
+ if (deadline != TInstant::Max()) {
+ ActorContext->ExecutorThread.Schedule(deadline - Now(), new IEventHandle(SelfActorId, {}, new TEvCoroTimeout,
+ 0, cookie));
+ }
+
+ // ensure we have no unprocessed event and return back to actor system to receive one
+ Y_VERIFY(!PendingEvent);
+ ReturnToActorSystem();
+
+ // obtain pending event and ensure we've got one
+ while (THolder<IEventHandle> event = std::exchange(PendingEvent, {})) {
+ if (event->GetTypeRewrite() != TEvents::TSystem::CoroTimeout) {
+ // special handling for poison pill -- we throw exception
+ if (event->GetTypeRewrite() == TEvents::TEvPoisonPill::EventType) {
+ throw TPoisonPillException();
+ }
+
+ // otherwise just return received event
+ return event;
+ } else if (event->Cookie == cookie) {
+ return nullptr; // it is not a race -- we've got timeout exactly for our current wait
+ } else {
+ ReturnToActorSystem(); // drop this event and wait for the next one
+ }
+ }
+ Y_FAIL("no pending event");
+ }
+
+ const TActorContext& TActorCoroImpl::GetActorContext() const {
+ Y_VERIFY(ActorContext);
+ return *ActorContext;
+ }
+
+ bool TActorCoroImpl::ProcessEvent(THolder<IEventHandle> ev) {
+ Y_VERIFY(!PendingEvent);
+ if (!SelfActorId) { // process bootstrap message, extract actor ids
+ Y_VERIFY(ev->GetTypeRewrite() == TEvents::TSystem::Bootstrap);
+ SelfActorId = ev->Recipient;
+ ParentActorId = ev->Sender;
+ } else { // process further messages
+ PendingEvent = std::move(ev);
+ }
+
+ // prepare actor context for in-coroutine use
+ TActivationContext *ac = TlsActivationContext;
+ TlsActivationContext = nullptr;
+ TActorContext ctx(ac->Mailbox, ac->ExecutorThread, ac->EventStart, SelfActorId);
+ ActorContext = &ctx;
+
+ Resume();
+
+ // drop actor context
+ TlsActivationContext = ac;
+ ActorContext = nullptr;
+
+ return Finished;
+ }
+
+ void TActorCoroImpl::Resume() {
+ // save caller context for a later return
+ Y_VERIFY(!ActorSystemContext);
+ TExceptionSafeContext actorSystemContext;
+ ActorSystemContext = &actorSystemContext;
+
+ // go to actor coroutine
+ BeforeResume();
+ ActorSystemContext->SwitchTo(&FiberContext);
+
+ // check for stack overflow
+#ifndef NDEBUG
+ const char* p;
+#if STACK_GROW_DOWN
+ p = Stack.Begin();
+#else
+ p = Stack.End() - StackOverflowGap;
+#endif
+ Y_VERIFY_DEBUG(memcmp(p, GoodStack, StackOverflowGap) == 0);
+#endif
+ }
+
+ void TActorCoroImpl::DoRun() {
+ try {
+ if (ActorContext) { // ActorContext may be nullptr here if the destructor was invoked before bootstrapping
+ Y_VERIFY(!PendingEvent);
+ Run();
+ }
+ } catch (const TPoisonPillException& /*ex*/) {
+ if (!AllowUnhandledPoisonPill) {
+ Y_FAIL("unhandled TPoisonPillException");
+ }
+ } catch (const TDtorException& /*ex*/) {
+ if (!AllowUnhandledDtor) {
+ Y_FAIL("unhandled TDtorException");
+ }
+ } catch (const std::exception& ex) {
+ Y_FAIL("unhandled exception of type %s", TypeName(ex).data());
+ } catch (...) {
+ Y_FAIL("unhandled exception of type not derived from std::exception");
+ }
+ Finished = true;
+ ReturnToActorSystem();
+ }
+
+ void TActorCoroImpl::ReturnToActorSystem() {
+ TExceptionSafeContext* returnContext = std::exchange(ActorSystemContext, nullptr);
+ Y_VERIFY(returnContext);
+ FiberContext.SwitchTo(returnContext);
+ if (!PendingEvent) {
+ // we have returned from the actor system and it kindly asks us to terminate the coroutine as it is being
+ // stopped
+ throw TDtorException();
+ }
+ }
+
+}
diff --git a/library/cpp/actors/core/actor_coroutine.h b/library/cpp/actors/core/actor_coroutine.h
new file mode 100644
index 0000000000..6bcb768eaf
--- /dev/null
+++ b/library/cpp/actors/core/actor_coroutine.h
@@ -0,0 +1,174 @@
+#pragma once
+
+#include <util/system/context.h>
+#include <util/system/filemap.h>
+
+#include "actor_bootstrapped.h"
+#include "executor_thread.h"
+#include "event_local.h"
+
+namespace NActors {
+
+ class TActorCoro;
+
+ class TActorCoroImpl : public ITrampoLine {
+ TMappedAllocation Stack;
+ bool AllowUnhandledPoisonPill;
+ bool AllowUnhandledDtor;
+ TContClosure FiberClosure;
+ TExceptionSafeContext FiberContext;
+ TExceptionSafeContext* ActorSystemContext = nullptr;
+ THolder<IEventHandle> PendingEvent;
+ bool Finished = false;
+ ui64 WaitCookie = 0;
+ TActorContext *ActorContext = nullptr;
+
+ protected:
+ TActorIdentity SelfActorId = TActorIdentity(TActorId());
+ TActorId ParentActorId;
+
+ private:
+ template <typename TFirstEvent, typename... TOtherEvents>
+ struct TIsOneOf: public TIsOneOf<TOtherEvents...> {
+ bool operator()(IEventHandle& ev) const {
+ return ev.GetTypeRewrite() == TFirstEvent::EventType || TIsOneOf<TOtherEvents...>()(ev);
+ }
+ };
+
+ template <typename TSingleEvent>
+ struct TIsOneOf<TSingleEvent> {
+ bool operator()(IEventHandle& ev) const {
+ return ev.GetTypeRewrite() == TSingleEvent::EventType;
+ }
+ };
+
+ struct TEvCoroTimeout : TEventLocal<TEvCoroTimeout, TEvents::TSystem::CoroTimeout> {};
+
+ protected:
+ struct TPoisonPillException : yexception {};
+ struct TDtorException : yexception {};
+
+ public:
+ TActorCoroImpl(size_t stackSize, bool allowUnhandledPoisonPill = false, bool allowUnhandledDtor = false);
+ // specify stackSize explicitly for each actor; don't forget about overflow control gap
+
+ virtual ~TActorCoroImpl();
+
+ virtual void Run() = 0;
+
+ virtual void BeforeResume() {}
+
+ // Handle all events that are not expected in wait loops.
+ virtual void ProcessUnexpectedEvent(TAutoPtr<IEventHandle> ev) = 0;
+
+ // Release execution ownership and wait for some event to arrive. When PoisonPill event is received, then
+ // TPoisonPillException is thrown.
+ THolder<IEventHandle> WaitForEvent(TInstant deadline = TInstant::Max());
+
+ // Wait for specific event set by filter functor. Function returns first event that matches filter. On any other
+ // kind of event ProcessUnexpectedEvent() is called.
+ //
+ // Example: WaitForSpecificEvent([](IEventHandle& ev) { return ev.Cookie == 42; });
+ template <typename TFunc>
+ THolder<IEventHandle> WaitForSpecificEvent(TFunc&& filter, TInstant deadline = TInstant::Max()) {
+ for (;;) {
+ if (THolder<IEventHandle> event = WaitForEvent(deadline); !event) {
+ return nullptr;
+ } else if (filter(*event)) {
+ return event;
+ } else {
+ ProcessUnexpectedEvent(event);
+ }
+ }
+ }
+
+ // Wait for specific event or set of events. Function returns first event that matches enlisted type. On any other
+ // kind of event ProcessUnexpectedEvent() is called.
+ //
+ // Example: WaitForSpecificEvent<TEvReadResult, TEvFinished>();
+ template <typename TFirstEvent, typename TSecondEvent, typename... TOtherEvents>
+ THolder<IEventHandle> WaitForSpecificEvent(TInstant deadline = TInstant::Max()) {
+ TIsOneOf<TFirstEvent, TSecondEvent, TOtherEvents...> filter;
+ return WaitForSpecificEvent(filter, deadline);
+ }
+
+ // Wait for single specific event.
+ template <typename TEventType>
+ THolder<typename TEventType::THandle> WaitForSpecificEvent(TInstant deadline = TInstant::Max()) {
+ auto filter = [](IEventHandle& ev) {
+ return ev.GetTypeRewrite() == TEventType::EventType;
+ };
+ THolder<IEventHandle> event = WaitForSpecificEvent(filter, deadline);
+ return THolder<typename TEventType::THandle>(static_cast<typename TEventType::THandle*>(event ? event.Release() : nullptr));
+ }
+
+ protected: // Actor System compatibility section
+ const TActorContext& GetActorContext() const;
+ TActorSystem *GetActorSystem() const { return GetActorContext().ExecutorThread.ActorSystem; }
+ TInstant Now() const { return GetActorContext().Now(); }
+
+ bool Send(const TActorId& recipient, IEventBase* ev, ui32 flags = 0, ui64 cookie = 0, NWilson::TTraceId traceId = {}) {
+ return GetActorContext().Send(recipient, ev, flags, cookie, std::move(traceId));
+ }
+
+ template <typename TEvent>
+ bool Send(const TActorId& recipient, THolder<TEvent> ev, ui32 flags = 0, ui64 cookie = 0, NWilson::TTraceId traceId = {}) {
+ return GetActorContext().Send(recipient, ev.Release(), flags, cookie, std::move(traceId));
+ }
+
+ bool Send(TAutoPtr<IEventHandle> ev);
+
+ void Schedule(TDuration delta, IEventBase* ev, ISchedulerCookie* cookie = nullptr) {
+ return GetActorContext().Schedule(delta, ev, cookie);
+ }
+
+ void Schedule(TInstant deadline, IEventBase* ev, ISchedulerCookie* cookie = nullptr) {
+ return GetActorContext().Schedule(deadline, ev, cookie);
+ }
+
+ void Schedule(TMonotonic deadline, IEventBase* ev, ISchedulerCookie* cookie = nullptr) {
+ return GetActorContext().Schedule(deadline, ev, cookie);
+ }
+
+ TActorId Register(IActor* actor, TMailboxType::EType mailboxType = TMailboxType::HTSwap, ui32 poolId = Max<ui32>()) {
+ return GetActorContext().Register(actor, mailboxType, poolId);
+ }
+
+ TActorId RegisterWithSameMailbox(IActor* actor) {
+ return GetActorContext().RegisterWithSameMailbox(actor);
+ }
+
+ private:
+ friend class TActorCoro;
+ bool ProcessEvent(THolder<IEventHandle> ev);
+
+ private:
+ /* Resume() function goes to actor coroutine context and continues (or starts) to execute it until actor finishes
+ * his job or it is blocked on WaitForEvent. Then the function returns. */
+ void Resume();
+ void ReturnToActorSystem();
+ void DoRun() override final;
+ };
+
+ class TActorCoro : public IActor {
+ THolder<TActorCoroImpl> Impl;
+
+ public:
+ TActorCoro(THolder<TActorCoroImpl> impl, ui32 activityType = IActor::ACTORLIB_COMMON)
+ : IActor(static_cast<TReceiveFunc>(&TActorCoro::StateFunc), activityType)
+ , Impl(std::move(impl))
+ {}
+
+ TAutoPtr<IEventHandle> AfterRegister(const TActorId& self, const TActorId& parent) override {
+ return new IEventHandle(TEvents::TSystem::Bootstrap, 0, self, parent, {}, 0);
+ }
+
+ private:
+ STATEFN(StateFunc) {
+ if (Impl->ProcessEvent(ev)) {
+ PassAway();
+ }
+ }
+ };
+
+}
diff --git a/library/cpp/actors/core/actor_coroutine_ut.cpp b/library/cpp/actors/core/actor_coroutine_ut.cpp
new file mode 100644
index 0000000000..951512b877
--- /dev/null
+++ b/library/cpp/actors/core/actor_coroutine_ut.cpp
@@ -0,0 +1,141 @@
+#include "actor_coroutine.h"
+#include "actorsystem.h"
+#include "executor_pool_basic.h"
+#include "scheduler_basic.h"
+#include "events.h"
+#include "event_local.h"
+#include "hfunc.h"
+#include <library/cpp/testing/unittest/registar.h>
+
+#include <util/system/sanitizers.h>
+
+using namespace NActors;
+
+Y_UNIT_TEST_SUITE(ActorCoro) {
+ enum {
+ Begin = EventSpaceBegin(TEvents::ES_USERSPACE),
+ Request,
+ Response,
+ Enough
+ };
+
+ struct TEvRequest: public TEventLocal<TEvRequest, Request> {
+ };
+
+ struct TEvResponse: public TEventLocal<TEvResponse, Response> {
+ };
+
+ struct TEvEnough: public TEventLocal<TEvEnough, Enough> {
+ };
+
+ class TBasicResponderActor: public TActorBootstrapped<TBasicResponderActor> {
+ TDeque<TActorId> RespondTo;
+
+ public:
+ TBasicResponderActor() {
+ }
+
+ void Bootstrap(const TActorContext& /*ctx*/) {
+ Become(&TBasicResponderActor::StateFunc);
+ }
+
+ STFUNC(StateFunc) {
+ switch (ev->GetTypeRewrite()) {
+ HFunc(TEvRequest, Handle);
+ HFunc(TEvents::TEvWakeup, Handle);
+ HFunc(TEvents::TEvPoisonPill, Handle);
+ }
+ }
+
+ void Handle(TEvRequest::TPtr& ev, const TActorContext& ctx) {
+ RespondTo.push_back(ev->Sender);
+ ctx.Schedule(TDuration::Seconds(1), new TEvents::TEvWakeup);
+ }
+
+ void Handle(TEvents::TEvWakeup::TPtr& /*ev*/, const TActorContext& ctx) {
+ ctx.Send(RespondTo.front(), new TEvResponse());
+ RespondTo.pop_front();
+ }
+
+ void Handle(TEvents::TEvPoisonPill::TPtr& /*ev*/, const TActorContext& ctx) {
+ Die(ctx);
+ }
+ };
+
+ class TCoroActor: public TActorCoroImpl {
+ TManualEvent& DoneEvent;
+ TAtomic& ItemsProcessed;
+ bool Finish;
+
+ public:
+ TCoroActor(TManualEvent& doneEvent, TAtomic& itemsProcessed)
+ : TActorCoroImpl(1 << 20)
+ , DoneEvent(doneEvent)
+ , ItemsProcessed(itemsProcessed)
+ , Finish(false)
+ {
+ }
+
+ void Run() override {
+ TActorId child = GetActorContext().Register(new TBasicResponderActor);
+ ui32 itemsProcessed = 0;
+ try {
+ while (!Finish) {
+ GetActorContext().Send(child, new TEvRequest());
+ THolder<IEventHandle> resp = WaitForSpecificEvent<TEvResponse>();
+ UNIT_ASSERT_EQUAL(resp->GetTypeRewrite(), TEvResponse::EventType);
+ ++itemsProcessed;
+ }
+ } catch (const TPoisonPillException& /*ex*/) {
+ }
+ GetActorContext().Send(child, new TEvents::TEvPoisonPill);
+
+ AtomicSet(ItemsProcessed, itemsProcessed);
+ DoneEvent.Signal();
+ }
+
+ void ProcessUnexpectedEvent(TAutoPtr<IEventHandle> event) override {
+ if (event->GetTypeRewrite() == Enough) {
+ Finish = true;
+ }
+ }
+ };
+
+ void Check(THolder<IEventBase> && message) {
+ THolder<TActorSystemSetup> setup = MakeHolder<TActorSystemSetup>();
+ setup->NodeId = 0;
+ setup->ExecutorsCount = 1;
+ setup->Executors.Reset(new TAutoPtr<IExecutorPool>[setup->ExecutorsCount]);
+ for (ui32 i = 0; i < setup->ExecutorsCount; ++i) {
+ setup->Executors[i] = new TBasicExecutorPool(i, 5, 10, "basic");
+ }
+ setup->Scheduler = new TBasicSchedulerThread;
+
+ TActorSystem actorSystem(setup);
+
+ actorSystem.Start();
+
+ TManualEvent doneEvent;
+ TAtomic itemsProcessed = 0;
+ TActorId actor = actorSystem.Register(new TActorCoro(MakeHolder<TCoroActor>(doneEvent, itemsProcessed)));
+ NanoSleep(3UL * 1000 * 1000 * 1000);
+ actorSystem.Send(actor, message.Release());
+ doneEvent.WaitI();
+
+ UNIT_ASSERT(AtomicGet(itemsProcessed) >= 2);
+
+ actorSystem.Stop();
+ }
+
+ Y_UNIT_TEST(Basic) {
+ if (NSan::TSanIsOn()) {
+ // TODO https://st.yandex-team.ru/DEVTOOLS-3154
+ return;
+ }
+ Check(MakeHolder<TEvEnough>());
+ }
+
+ Y_UNIT_TEST(PoisonPill) {
+ Check(MakeHolder<TEvents::TEvPoisonPill>());
+ }
+}
diff --git a/library/cpp/actors/core/actor_ut.cpp b/library/cpp/actors/core/actor_ut.cpp
new file mode 100644
index 0000000000..e1b765ec72
--- /dev/null
+++ b/library/cpp/actors/core/actor_ut.cpp
@@ -0,0 +1,578 @@
+#include "actor.cpp"
+#include "events.h"
+#include "actorsystem.h"
+#include "executor_pool_basic.h"
+#include "scheduler_basic.h"
+#include "actor_bootstrapped.h"
+
+#include <library/cpp/actors/util/threadparkpad.h>
+#include <library/cpp/testing/unittest/registar.h>
+
+#include <util/generic/algorithm.h>
+#include <util/system/atomic.h>
+#include <util/system/rwlock.h>
+#include <util/system/hp_timer.h>
+
+using namespace NActors;
+
+struct TTestEndDecorator : TDecorator {
+ TThreadParkPad* Pad;
+ TAtomic* ActorsAlive;
+
+ TTestEndDecorator(THolder<IActor>&& actor, TThreadParkPad* pad, TAtomic* actorsAlive)
+ : TDecorator(std::move(actor))
+ , Pad(pad)
+ , ActorsAlive(actorsAlive)
+ {
+ AtomicIncrement(*ActorsAlive);
+ }
+
+ ~TTestEndDecorator() {
+ if (AtomicDecrement(*ActorsAlive) == 0) {
+ Pad->Unpark();
+ }
+ }
+};
+
+Y_UNIT_TEST_SUITE(ActorBenchmark) {
+ static constexpr bool DefaultNoRealtime = true;
+ static constexpr ui32 DefaultSpinThreshold = 1000000;
+ static constexpr ui32 TotalEventsAmount = 1000;
+
+ class TDummyActor : public TActor<TDummyActor> {
+ public:
+ TDummyActor() : TActor<TDummyActor>(&TDummyActor::StateFunc) {}
+ STFUNC(StateFunc) {
+ (void)ev;
+ (void)ctx;
+ }
+ };
+
+ enum ERole {
+ Leader,
+ Follower
+ };
+
+ class TSendReceiveActor : public TActorBootstrapped<TSendReceiveActor> {
+ public:
+ static constexpr auto ActorActivityType() {
+ return ACTORLIB_COMMON;
+ }
+
+ TSendReceiveActor(double* elapsedTime, TActorId receiver, bool allocation, ERole role, ui32 neighbours = 0)
+ : EventsCounter(TotalEventsAmount)
+ , ElapsedTime(elapsedTime)
+ , Receiver(receiver)
+ , AllocatesMemory(allocation)
+ , Role(role)
+ , MailboxNeighboursCount(neighbours)
+ {}
+
+ void Bootstrap(const TActorContext &ctx) {
+ if (!Receiver) {
+ this->Receiver = SelfId();
+ } else {
+ EventsCounter /= 2; // We want to measure CPU requirement for one-way send
+ }
+ Timer.Reset();
+ Become(&TThis::StateFunc);
+ for (ui32 i = 0; i < MailboxNeighboursCount; ++i) {
+ ctx.RegisterWithSameMailbox(new TDummyActor());
+ }
+ if (Role == Leader) {
+ Send(Receiver, new TEvents::TEvPing());
+ }
+ }
+
+ STATEFN(StateFunc) {
+ if (EventsCounter == 0 && ElapsedTime != nullptr) {
+ *ElapsedTime = Timer.Passed() / TotalEventsAmount;
+ PassAway();
+ }
+
+ if (AllocatesMemory) {
+ Send(ev->Sender, new TEvents::TEvPing());
+ } else {
+ std::swap(*const_cast<TActorId*>(&ev->Sender), *const_cast<TActorId*>(&ev->Recipient));
+ ev->DropRewrite();
+ TActivationContext::Send(ev.Release());
+ }
+ EventsCounter--;
+ }
+
+ private:
+ THPTimer Timer;
+ ui64 EventsCounter;
+ double* ElapsedTime;
+ TActorId Receiver;
+ bool AllocatesMemory;
+ ERole Role;
+ ui32 MailboxNeighboursCount;
+ };
+
+ void AddBasicPool(THolder<TActorSystemSetup>& setup, ui32 threads, bool activateEveryEvent) {
+ TBasicExecutorPoolConfig basic;
+ basic.PoolId = setup->GetExecutorsCount();
+ basic.PoolName = TStringBuilder() << "b" << basic.PoolId;
+ basic.Threads = threads;
+ basic.SpinThreshold = DefaultSpinThreshold;
+ basic.TimePerMailbox = TDuration::Hours(1);
+ if (activateEveryEvent) {
+ basic.EventsPerMailbox = 1;
+ } else {
+ basic.EventsPerMailbox = Max<ui32>();
+ }
+ setup->CpuManager.Basic.emplace_back(std::move(basic));
+ }
+
+ void AddUnitedPool(THolder<TActorSystemSetup>& setup, ui32 concurrency, bool activateEveryEvent) {
+ TUnitedExecutorPoolConfig united;
+ united.PoolId = setup->GetExecutorsCount();
+ united.PoolName = TStringBuilder() << "u" << united.PoolId;
+ united.Concurrency = concurrency;
+ united.TimePerMailbox = TDuration::Hours(1);
+ if (activateEveryEvent) {
+ united.EventsPerMailbox = 1;
+ } else {
+ united.EventsPerMailbox = Max<ui32>();
+ }
+ setup->CpuManager.United.emplace_back(std::move(united));
+ }
+
+ THolder<TActorSystemSetup> GetActorSystemSetup(ui32 unitedCpuCount, bool preemption) {
+ auto setup = MakeHolder<NActors::TActorSystemSetup>();
+ setup->NodeId = 1;
+ setup->CpuManager.UnitedWorkers.CpuCount = unitedCpuCount;
+ setup->CpuManager.UnitedWorkers.SpinThresholdUs = DefaultSpinThreshold;
+ setup->CpuManager.UnitedWorkers.NoRealtime = DefaultNoRealtime;
+ if (preemption) {
+ setup->CpuManager.UnitedWorkers.PoolLimitUs = 500;
+ setup->CpuManager.UnitedWorkers.EventLimitUs = 100;
+ setup->CpuManager.UnitedWorkers.LimitPrecisionUs = 100;
+ } else {
+ setup->CpuManager.UnitedWorkers.PoolLimitUs = 100'000'000'000;
+ setup->CpuManager.UnitedWorkers.EventLimitUs = 10'000'000'000;
+ setup->CpuManager.UnitedWorkers.LimitPrecisionUs = 10'000'000'000;
+ }
+ setup->Scheduler = new TBasicSchedulerThread(NActors::TSchedulerConfig(512, 0));
+ return setup;
+ }
+
+ enum class EPoolType {
+ Basic,
+ United
+ };
+
+ THolder<TActorSystemSetup> InitActorSystemSetup(EPoolType poolType, ui32 poolsCount, ui32 threads, bool activateEveryEvent, bool preemption) {
+ if (poolType == EPoolType::Basic) {
+ THolder<TActorSystemSetup> setup = GetActorSystemSetup(0, false);
+ for (ui32 i = 0; i < poolsCount; ++i) {
+ AddBasicPool(setup, threads, activateEveryEvent);
+ }
+ return setup;
+ } else if (poolType == EPoolType::United) {
+ THolder<TActorSystemSetup> setup = GetActorSystemSetup(poolsCount * threads, preemption);
+ for (ui32 i = 0; i < poolsCount; ++i) {
+ AddUnitedPool(setup, threads, activateEveryEvent);
+ }
+ return setup;
+ }
+ Y_FAIL();
+ }
+
+ double BenchSendReceive(bool allocation, NActors::TMailboxType::EType mType, EPoolType poolType) {
+ THolder<TActorSystemSetup> setup = InitActorSystemSetup(poolType, 1, 1, false, false);
+ TActorSystem actorSystem(setup);
+ actorSystem.Start();
+
+ TThreadParkPad pad;
+ TAtomic actorsAlive = 0;
+ double elapsedTime = 0;
+ THolder<IActor> endActor{
+ new TTestEndDecorator(THolder(
+ new TSendReceiveActor(&elapsedTime, {}, allocation, Leader)), &pad, &actorsAlive)};
+
+ actorSystem.Register(endActor.Release(), mType);
+
+ pad.Park();
+ actorSystem.Stop();
+
+ return 1e9 * elapsedTime;
+ }
+
+ double BenchSendActivateReceive(ui32 poolsCount, ui32 threads, bool allocation, EPoolType poolType) {
+ THolder<TActorSystemSetup> setup = InitActorSystemSetup(poolType, poolsCount, threads, true, false);
+ TActorSystem actorSystem(setup);
+ actorSystem.Start();
+
+ TThreadParkPad pad;
+ TAtomic actorsAlive = 0;
+ double elapsedTime = 0;
+ ui32 followerPoolId = 0;
+
+ ui32 leaderPoolId = poolsCount == 1 ? 0 : 1;
+ TActorId followerId = actorSystem.Register(
+ new TSendReceiveActor(nullptr, {}, allocation, Follower), TMailboxType::HTSwap, followerPoolId);
+ THolder<IActor> leader{
+ new TTestEndDecorator(THolder(
+ new TSendReceiveActor(&elapsedTime, followerId, allocation, Leader)), &pad, &actorsAlive)};
+ actorSystem.Register(leader.Release(), TMailboxType::HTSwap, leaderPoolId);
+
+ pad.Park();
+ actorSystem.Stop();
+
+ return 1e9 * elapsedTime;
+ }
+
+ double BenchSendActivateReceiveWithMailboxNeighbours(ui32 MailboxNeighbourActors, EPoolType poolType) {
+ THolder<TActorSystemSetup> setup = InitActorSystemSetup(poolType, 1, 1, false, false);
+ TActorSystem actorSystem(setup);
+ actorSystem.Start();
+
+ TThreadParkPad pad;
+ TAtomic actorsAlive = 0;
+ double elapsedTime = 0;
+
+ TActorId followerId = actorSystem.Register(
+ new TSendReceiveActor(nullptr, {}, false, Follower, MailboxNeighbourActors), TMailboxType::HTSwap);
+ THolder<IActor> leader{
+ new TTestEndDecorator(THolder(
+ new TSendReceiveActor(&elapsedTime, followerId, false, Leader, MailboxNeighbourActors)), &pad, &actorsAlive)};
+ actorSystem.Register(leader.Release(), TMailboxType::HTSwap);
+
+ pad.Park();
+ actorSystem.Stop();
+
+ return 1e9 * elapsedTime;
+ }
+
+ double BenchContentedThreads(ui32 threads, ui32 actorsPairsCount, EPoolType poolType) {
+ THolder<TActorSystemSetup> setup = InitActorSystemSetup(poolType, 1, threads, true, false);
+ TActorSystem actorSystem(setup);
+ actorSystem.Start();
+
+ TThreadParkPad pad;
+ TAtomic actorsAlive = 0;
+ THPTimer Timer;
+
+ TVector<double> dummy(actorsPairsCount);
+ Timer.Reset();
+ for (ui32 i = 0; i < actorsPairsCount; ++i) {
+ ui32 followerPoolId = 0;
+ ui32 leaderPoolId = 0;
+ TActorId followerId = actorSystem.Register(
+ new TSendReceiveActor(nullptr, {}, true, Follower), TMailboxType::HTSwap, followerPoolId);
+ THolder<IActor> leader{
+ new TTestEndDecorator(THolder(
+ new TSendReceiveActor(&dummy[i], followerId, true, Leader)), &pad, &actorsAlive)};
+ actorSystem.Register(leader.Release(), TMailboxType::HTSwap, leaderPoolId);
+ }
+
+ pad.Park();
+ auto elapsedTime = Timer.Passed() / TotalEventsAmount;
+ actorSystem.Stop();
+
+ return 1e9 * elapsedTime;
+ }
+
+ auto Mean(const TVector<double>& data) {
+ return Accumulate(data.begin(), data.end(), 0.0) / data.size();
+ }
+
+ auto Deviation(const TVector<double>& data) {
+ auto mean = Mean(data);
+ double deviation = 0.0;
+ for (const auto& x : data) {
+ deviation += (x - mean) * (x - mean);
+ }
+ return std::sqrt(deviation / data.size());
+ }
+
+ struct TStats {
+ double Mean;
+ double Deviation;
+ TString ToString() {
+ return TStringBuilder() << Mean << " ± " << Deviation << " ns " << std::ceil(Deviation / Mean * 1000) / 10.0 << "%";
+ }
+ };
+
+ template <typename Func>
+ TStats CountStats(Func func, ui32 itersCount = 5) {
+ TVector<double> elapsedTimes;
+ for (ui32 i = 0; i < itersCount; ++i) {
+ auto elapsedTime = func();
+ elapsedTimes.push_back(elapsedTime);
+ }
+ return {Mean(elapsedTimes), Deviation(elapsedTimes)};
+ }
+
+ TVector<NActors::TMailboxType::EType> MailboxTypes = {
+ TMailboxType::Simple,
+ TMailboxType::Revolving,
+ TMailboxType::HTSwap,
+ TMailboxType::ReadAsFilled,
+ TMailboxType::TinyReadAsFilled
+ };
+
+ Y_UNIT_TEST(SendReceive1Pool1ThreadAlloc) {
+ for (const auto& mType : MailboxTypes) {
+ auto stats = CountStats([mType] {
+ return BenchSendReceive(true, mType, EPoolType::Basic);
+ });
+ Cerr << stats.ToString() << " " << mType << Endl;
+ }
+ }
+
+ Y_UNIT_TEST(SendReceive1Pool1ThreadAllocUnited) {
+ for (const auto& mType : MailboxTypes) {
+ auto stats = CountStats([mType] {
+ return BenchSendReceive(true, mType, EPoolType::United);
+ });
+ Cerr << stats.ToString() << " " << mType << Endl;
+ }
+ }
+
+ Y_UNIT_TEST(SendReceive1Pool1ThreadNoAlloc) {
+ for (const auto& mType : MailboxTypes) {
+ auto stats = CountStats([mType] {
+ return BenchSendReceive(false, mType, EPoolType::Basic);
+ });
+ Cerr << stats.ToString() << " " << mType << Endl;
+ }
+ }
+
+ Y_UNIT_TEST(SendReceive1Pool1ThreadNoAllocUnited) {
+ for (const auto& mType : MailboxTypes) {
+ auto stats = CountStats([mType] {
+ return BenchSendReceive(false, mType, EPoolType::United);
+ });
+ Cerr << stats.ToString() << " " << mType << Endl;
+ }
+ }
+
+ Y_UNIT_TEST(SendActivateReceive1Pool1ThreadAlloc) {
+ auto stats = CountStats([] {
+ return BenchSendActivateReceive(1, 1, true, EPoolType::Basic);
+ });
+ Cerr << stats.ToString() << Endl;
+ }
+
+ Y_UNIT_TEST(SendActivateReceive1Pool1ThreadAllocUnited) {
+ auto stats = CountStats([] {
+ return BenchSendActivateReceive(1, 1, true, EPoolType::United);
+ });
+ Cerr << stats.ToString() << Endl;
+ }
+
+ Y_UNIT_TEST(SendActivateReceive1Pool1ThreadNoAlloc) {
+ auto stats = CountStats([] {
+ return BenchSendActivateReceive(1, 1, false, EPoolType::Basic);
+ });
+ Cerr << stats.ToString() << Endl;
+ }
+
+ Y_UNIT_TEST(SendActivateReceive1Pool1ThreadNoAllocUnited) {
+ auto stats = CountStats([] {
+ return BenchSendActivateReceive(1, 1, false, EPoolType::United);
+ });
+ Cerr << stats.ToString() << Endl;
+ }
+
+ Y_UNIT_TEST(SendActivateReceive1Pool2ThreadsAlloc) {
+ auto stats = CountStats([] {
+ return BenchSendActivateReceive(1, 2, true, EPoolType::Basic);
+ });
+ Cerr << stats.ToString() << Endl;
+ }
+
+ Y_UNIT_TEST(SendActivateReceive1Pool2ThreadsAllocUnited) {
+ auto stats = CountStats([] {
+ return BenchSendActivateReceive(1, 2, true, EPoolType::United);
+ });
+ Cerr << stats.ToString() << Endl;
+ }
+
+ Y_UNIT_TEST(SendActivateReceive1Pool2ThreadsNoAlloc) {
+ auto stats = CountStats([] {
+ return BenchSendActivateReceive(1, 2, false, EPoolType::Basic);
+ });
+ Cerr << stats.ToString() << Endl;
+ }
+
+ Y_UNIT_TEST(SendActivateReceive1Pool2ThreadsNoAllocUnited) {
+ auto stats = CountStats([] {
+ return BenchSendActivateReceive(1, 2, false, EPoolType::United);
+ });
+ Cerr << stats.ToString() << Endl;
+ }
+
+ Y_UNIT_TEST(SendActivateReceive2Pool1ThreadAlloc) {
+ auto stats = CountStats([] {
+ return BenchSendActivateReceive(2, 1, true, EPoolType::Basic);
+ });
+ Cerr << stats.ToString() << Endl;
+ }
+
+ Y_UNIT_TEST(SendActivateReceive2Pool1ThreadAllocUnited) {
+ auto stats = CountStats([] {
+ return BenchSendActivateReceive(2, 1, true, EPoolType::United);
+ });
+ Cerr << stats.ToString() << Endl;
+ }
+
+ Y_UNIT_TEST(SendActivateReceive2Pool1ThreadNoAlloc) {
+ auto stats = CountStats([] {
+ return BenchSendActivateReceive(2, 1, false, EPoolType::Basic);
+ });
+ Cerr << stats.ToString() << Endl;
+ }
+
+ Y_UNIT_TEST(SendActivateReceive2Pool1ThreadNoAllocUnited) {
+ auto stats = CountStats([] {
+ return BenchSendActivateReceive(2, 1, false, EPoolType::United);
+ });
+ Cerr << stats.ToString() << Endl;
+ }
+
+ void RunBenchContentedThreads(ui32 threads, EPoolType poolType) {
+ for (ui32 actorPairs = 1; actorPairs <= 2 * threads; actorPairs++) {
+ auto stats = CountStats([threads, actorPairs, poolType] {
+ return BenchContentedThreads(threads, actorPairs, poolType);
+ });
+ Cerr << stats.ToString() << " actorPairs: " << actorPairs << Endl;
+ }
+ }
+
+ Y_UNIT_TEST(SendActivateReceive1Pool1Threads) { RunBenchContentedThreads(1, EPoolType::Basic); }
+ Y_UNIT_TEST(SendActivateReceive1Pool1ThreadsUnited) { RunBenchContentedThreads(1, EPoolType::United); }
+ Y_UNIT_TEST(SendActivateReceive1Pool2Threads) { RunBenchContentedThreads(2, EPoolType::Basic); }
+ Y_UNIT_TEST(SendActivateReceive1Pool2ThreadsUnited) { RunBenchContentedThreads(2, EPoolType::United); }
+ Y_UNIT_TEST(SendActivateReceive1Pool3Threads) { RunBenchContentedThreads(3, EPoolType::Basic); }
+ Y_UNIT_TEST(SendActivateReceive1Pool3ThreadsUnited) { RunBenchContentedThreads(3, EPoolType::United); }
+ Y_UNIT_TEST(SendActivateReceive1Pool4Threads) { RunBenchContentedThreads(4, EPoolType::Basic); }
+ Y_UNIT_TEST(SendActivateReceive1Pool4ThreadsUnited) { RunBenchContentedThreads(4, EPoolType::United); }
+ Y_UNIT_TEST(SendActivateReceive1Pool5Threads) { RunBenchContentedThreads(5, EPoolType::Basic); }
+ Y_UNIT_TEST(SendActivateReceive1Pool5ThreadsUnited) { RunBenchContentedThreads(5, EPoolType::United); }
+ Y_UNIT_TEST(SendActivateReceive1Pool6Threads) { RunBenchContentedThreads(6, EPoolType::Basic); }
+ Y_UNIT_TEST(SendActivateReceive1Pool6ThreadsUnited) { RunBenchContentedThreads(6, EPoolType::United); }
+ Y_UNIT_TEST(SendActivateReceive1Pool7Threads) { RunBenchContentedThreads(7, EPoolType::Basic); }
+ Y_UNIT_TEST(SendActivateReceive1Pool7ThreadsUnited) { RunBenchContentedThreads(7, EPoolType::United); }
+ Y_UNIT_TEST(SendActivateReceive1Pool8Threads) { RunBenchContentedThreads(8, EPoolType::Basic); }
+ Y_UNIT_TEST(SendActivateReceive1Pool8ThreadsUnited) { RunBenchContentedThreads(8, EPoolType::United); }
+
+ Y_UNIT_TEST(SendActivateReceiveWithMailboxNeighbours) {
+ TVector<ui32> NeighbourActors = {0, 1, 2, 3, 4, 5, 6, 7, 8, 16, 32, 64, 128, 256};
+ for (const auto& neighbour : NeighbourActors) {
+ auto stats = CountStats([neighbour] {
+ return BenchSendActivateReceiveWithMailboxNeighbours(neighbour, EPoolType::Basic);
+ });
+ Cerr << stats.ToString() << " neighbourActors: " << neighbour << Endl;
+ }
+ }
+
+ Y_UNIT_TEST(SendActivateReceiveWithMailboxNeighboursUnited) {
+ TVector<ui32> NeighbourActors = {0, 1, 2, 3, 4, 5, 6, 7, 8, 16, 32, 64, 128, 256};
+ for (const auto& neighbour : NeighbourActors) {
+ auto stats = CountStats([neighbour] {
+ return BenchSendActivateReceiveWithMailboxNeighbours(neighbour, EPoolType::United);
+ });
+ Cerr << stats.ToString() << " neighbourActors: " << neighbour << Endl;
+ }
+ }
+}
+
+Y_UNIT_TEST_SUITE(TestDecorator) {
+ struct TPingDecorator : TDecorator {
+ TAutoPtr<IEventHandle> SavedEvent = nullptr;
+ ui64* Counter;
+
+ TPingDecorator(THolder<IActor>&& actor, ui64* counter)
+ : TDecorator(std::move(actor))
+ , Counter(counter)
+ {
+ }
+
+ bool DoBeforeReceiving(TAutoPtr<IEventHandle>& ev, const TActorContext& ctx) override {
+ *Counter += 1;
+ if (ev->Type != TEvents::THelloWorld::Pong) {
+ TAutoPtr<IEventHandle> pingEv = new IEventHandle(SelfId(), SelfId(), new TEvents::TEvPing());
+ SavedEvent = ev;
+ Actor->Receive(pingEv, ctx);
+ } else {
+ Actor->Receive(SavedEvent, ctx);
+ }
+ return false;
+ }
+ };
+
+ struct TPongDecorator : TDecorator {
+ ui64* Counter;
+
+ TPongDecorator(THolder<IActor>&& actor, ui64* counter)
+ : TDecorator(std::move(actor))
+ , Counter(counter)
+ {
+ }
+
+ bool DoBeforeReceiving(TAutoPtr<IEventHandle>& ev, const TActorContext&) override {
+ *Counter += 1;
+ if (ev->Type == TEvents::THelloWorld::Ping) {
+ TAutoPtr<IEventHandle> pongEv = new IEventHandle(SelfId(), SelfId(), new TEvents::TEvPong());
+ Send(SelfId(), new TEvents::TEvPong());
+ return false;
+ }
+ return true;
+ }
+ };
+
+ struct TTestActor : TActorBootstrapped<TTestActor> {
+ static constexpr char ActorName[] = "TestActor";
+
+ void Bootstrap()
+ {
+ const auto& activityTypeIndex = GetActivityType();
+ Y_ENSURE(activityTypeIndex < GetActivityTypeCount());
+ Y_ENSURE(GetActivityTypeName(activityTypeIndex) == "TestActor");
+ PassAway();
+ }
+ };
+
+ Y_UNIT_TEST(Basic) {
+ THolder<TActorSystemSetup> setup = MakeHolder<TActorSystemSetup>();
+ setup->NodeId = 0;
+ setup->ExecutorsCount = 1;
+ setup->Executors.Reset(new TAutoPtr<IExecutorPool>[setup->ExecutorsCount]);
+ for (ui32 i = 0; i < setup->ExecutorsCount; ++i) {
+ setup->Executors[i] = new TBasicExecutorPool(i, 1, 10, "basic");
+ }
+ setup->Scheduler = new TBasicSchedulerThread;
+
+ TActorSystem actorSystem(setup);
+ actorSystem.Start();
+
+ THolder<IActor> innerActor = MakeHolder<TTestActor>();
+ ui64 pongCounter = 0;
+ THolder<IActor> pongActor = MakeHolder<TPongDecorator>(std::move(innerActor), &pongCounter);
+ ui64 pingCounter = 0;
+ THolder<IActor> pingActor = MakeHolder<TPingDecorator>(std::move(pongActor), &pingCounter);
+
+ TThreadParkPad pad;
+ TAtomic actorsAlive = 0;
+
+ THolder<IActor> endActor = MakeHolder<TTestEndDecorator>(std::move(pingActor), &pad, &actorsAlive);
+ actorSystem.Register(endActor.Release(), TMailboxType::HTSwap);
+
+ pad.Park();
+ actorSystem.Stop();
+ UNIT_ASSERT(pongCounter == 2 && pingCounter == 2);
+ }
+
+ Y_UNIT_TEST(LocalProcessKey) {
+ static constexpr char ActorName[] = "TestActor";
+
+ UNIT_ASSERT((TEnumProcessKey<TActorActivityTag, IActor::EActorActivity>::GetName(IActor::INTERCONNECT_PROXY_TCP) == "INTERCONNECT_PROXY_TCP"));
+
+ UNIT_ASSERT((TLocalProcessKey<TActorActivityTag, ActorName>::GetName() == ActorName));
+ UNIT_ASSERT((TEnumProcessKey<TActorActivityTag, IActor::EActorActivity>::GetIndex(IActor::INTERCONNECT_PROXY_TCP) == IActor::INTERCONNECT_PROXY_TCP));
+ }
+}
diff --git a/library/cpp/actors/core/actorid.cpp b/library/cpp/actors/core/actorid.cpp
new file mode 100644
index 0000000000..ccda035eac
--- /dev/null
+++ b/library/cpp/actors/core/actorid.cpp
@@ -0,0 +1,34 @@
+#include "actorid.h"
+#include <util/string/builder.h>
+#include <util/string/cast.h>
+
+namespace NActors {
+ void TActorId::Out(IOutputStream& o) const {
+ o << "[" << NodeId() << ":" << LocalId() << ":" << Hint() << "]";
+ }
+
+ TString TActorId::ToString() const {
+ TString x;
+ TStringOutput o(x);
+ Out(o);
+ return x;
+ }
+
+ bool TActorId::Parse(const char* buf, ui32 sz) {
+ if (sz < 4 || buf[0] != '[' || buf[sz - 1] != ']')
+ return false;
+
+ size_t semicolons[2];
+ TStringBuf str(buf, sz);
+ semicolons[0] = str.find(':', 1);
+ if (semicolons[0] == TStringBuf::npos)
+ return false;
+ semicolons[1] = str.find(':', semicolons[0] + 1);
+ if (semicolons[1] == TStringBuf::npos)
+ return false;
+
+ bool success = TryFromString(buf + 1, semicolons[0] - 1, Raw.N.NodeId) && TryFromString(buf + semicolons[0] + 1, semicolons[1] - semicolons[0] - 1, Raw.N.LocalId) && TryFromString(buf + semicolons[1] + 1, sz - semicolons[1] - 2, Raw.N.Hint);
+
+ return success;
+ }
+}
diff --git a/library/cpp/actors/core/actorid.h b/library/cpp/actors/core/actorid.h
new file mode 100644
index 0000000000..d972b1a0ff
--- /dev/null
+++ b/library/cpp/actors/core/actorid.h
@@ -0,0 +1,196 @@
+#pragma once
+
+#include "defs.h"
+#include <util/stream/output.h> // for IOutputStream
+#include <util/generic/hash.h>
+
+namespace NActors {
+ // used as global uniq address of actor
+ // also could be used to transport service id (12 byte strings placed in hint-localid)
+ // highest 1 bit of node - mark of service id
+ // next 11 bits of node-id - pool id
+ // next 20 bits - node id itself
+
+ struct TActorId {
+ static constexpr ui32 MaxServiceIDLength = 12;
+ static constexpr ui32 MaxPoolID = 0x000007FF;
+ static constexpr ui32 MaxNodeId = 0x000FFFFF;
+ static constexpr ui32 PoolIndexShift = 20;
+ static constexpr ui32 PoolIndexMask = MaxPoolID << PoolIndexShift;
+ static constexpr ui32 ServiceMask = 0x80000000;
+ static constexpr ui32 NodeIdMask = MaxNodeId;
+
+ private:
+ union {
+ struct {
+ ui64 LocalId;
+ ui32 Hint;
+ ui32 NodeId;
+ } N;
+
+ struct {
+ ui64 X1;
+ ui64 X2;
+ } X;
+
+ ui8 Buf[16];
+ } Raw;
+
+ public:
+ TActorId() noexcept {
+ Raw.X.X1 = 0;
+ Raw.X.X2 = 0;
+ }
+
+ explicit TActorId(ui32 nodeId, ui32 poolId, ui64 localId, ui32 hint) noexcept {
+ Y_VERIFY_DEBUG(poolId <= MaxPoolID);
+ Raw.N.LocalId = localId;
+ Raw.N.Hint = hint;
+ Raw.N.NodeId = nodeId | (poolId << PoolIndexShift);
+ }
+
+ explicit TActorId(ui32 nodeId, const TStringBuf& x) noexcept {
+ Y_VERIFY(x.size() <= MaxServiceIDLength, "service id is too long");
+ Raw.N.LocalId = 0;
+ Raw.N.Hint = 0;
+ Raw.N.NodeId = nodeId | ServiceMask;
+ memcpy(Raw.Buf, x.data(), x.size());
+ }
+
+ explicit TActorId(ui64 x1, ui64 x2) noexcept {
+ Raw.X.X1 = x1;
+ Raw.X.X2 = x2;
+ }
+
+ explicit operator bool() const noexcept {
+ return Raw.X.X1 != 0 || Raw.X.X2 != 0;
+ }
+
+ ui64 LocalId() const noexcept {
+ return Raw.N.LocalId;
+ }
+
+ ui32 Hint() const noexcept {
+ return Raw.N.Hint;
+ }
+
+ ui32 NodeId() const noexcept {
+ return Raw.N.NodeId & NodeIdMask;
+ }
+
+ bool IsService() const noexcept {
+ return (Raw.N.NodeId & ServiceMask);
+ }
+
+ TStringBuf ServiceId() const noexcept {
+ Y_VERIFY_DEBUG(IsService());
+ return TStringBuf((const char*)Raw.Buf, MaxServiceIDLength);
+ }
+
+ static ui32 PoolIndex(ui32 nodeid) noexcept {
+ return ((nodeid & PoolIndexMask) >> PoolIndexShift);
+ }
+
+ ui32 PoolID() const noexcept {
+ return PoolIndex(Raw.N.NodeId);
+ }
+
+ ui64 RawX1() const noexcept {
+ return Raw.X.X1;
+ }
+
+ ui64 RawX2() const noexcept {
+ return Raw.X.X2;
+ }
+
+ bool operator<(const TActorId& x) const noexcept {
+ const ui64 s1 = Raw.X.X1;
+ const ui64 s2 = Raw.X.X2;
+ const ui64 x1 = x.Raw.X.X1;
+ const ui64 x2 = x.Raw.X.X2;
+
+ return (s1 != x1) ? (s1 < x1) : (s2 < x2);
+ }
+
+ bool operator!=(const TActorId& x) const noexcept {
+ return Raw.X.X1 != x.Raw.X.X1 || Raw.X.X2 != x.Raw.X.X2;
+ }
+
+ bool operator==(const TActorId& x) const noexcept {
+ return !(x != *this);
+ }
+
+ ui64 Hash() const noexcept {
+ const ui32* x = (const ui32*)Raw.Buf;
+
+ const ui64 x1 = x[0] * 0x001DFF3D8DC48F5Dull;
+ const ui64 x2 = x[1] * 0x179CA10C9242235Dull;
+ const ui64 x3 = x[2] * 0x0F530CAD458B0FB1ull;
+ const ui64 x4 = x[3] * 0xB5026F5AA96619E9ull;
+
+ const ui64 z1 = x1 + x2;
+ const ui64 z2 = x3 + x4;
+
+ const ui64 sum = 0x5851F42D4C957F2D + z1 + z2;
+
+ return (sum >> 32) | (sum << 32);
+ }
+
+ ui32 Hash32() const noexcept {
+ const ui32* x = (const ui32*)Raw.Buf;
+
+ const ui64 x1 = x[0] * 0x001DFF3D8DC48F5Dull;
+ const ui64 x2 = x[1] * 0x179CA10C9242235Dull;
+ const ui64 x3 = x[2] * 0x0F530CAD458B0FB1ull;
+ const ui64 x4 = x[3] * 0xB5026F5AA96619E9ull;
+
+ const ui64 z1 = x1 + x2;
+ const ui64 z2 = x3 + x4;
+
+ const ui64 sum = 0x5851F42D4C957F2D + z1 + z2;
+
+ return sum >> 32;
+ }
+
+ struct THash {
+ ui64 operator()(const TActorId& actorId) const noexcept {
+ return actorId.Hash();
+ }
+ };
+
+ struct THash32 {
+ ui64 operator()(const TActorId& actorId) const noexcept {
+ return actorId.Hash();
+ }
+ };
+
+ struct TOrderedCmp {
+ bool operator()(const TActorId &left, const TActorId &right) const noexcept {
+ Y_VERIFY_DEBUG(!left.IsService() && !right.IsService(), "ordered compare works for plain actorids only");
+ const ui32 n1 = left.NodeId();
+ const ui32 n2 = right.NodeId();
+
+ return (n1 != n2) ? (n1 < n2) : left.LocalId() < right.LocalId();
+ }
+ };
+
+ TString ToString() const;
+ void Out(IOutputStream& o) const;
+ bool Parse(const char* buf, ui32 sz);
+ };
+
+ static_assert(sizeof(TActorId) == 16, "expect sizeof(TActorId) == 16");
+ static_assert(MaxPools < TActorId::MaxPoolID); // current implementation of united pool has limit MaxPools on pool id
+}
+
+template <>
+inline void Out<NActors::TActorId>(IOutputStream& o, const NActors::TActorId& x) {
+ return x.Out(o);
+}
+
+template <>
+struct THash<NActors::TActorId> {
+ inline ui64 operator()(const NActors::TActorId& x) const {
+ return x.Hash();
+ }
+};
diff --git a/library/cpp/actors/core/actorsystem.cpp b/library/cpp/actors/core/actorsystem.cpp
new file mode 100644
index 0000000000..c58698a206
--- /dev/null
+++ b/library/cpp/actors/core/actorsystem.cpp
@@ -0,0 +1,277 @@
+#include "defs.h"
+#include "actorsystem.h"
+#include "callstack.h"
+#include "cpu_manager.h"
+#include "mailbox.h"
+#include "events.h"
+#include "interconnect.h"
+#include "servicemap.h"
+#include "scheduler_queue.h"
+#include "scheduler_actor.h"
+#include "log.h"
+#include "probes.h"
+#include "ask.h"
+#include <library/cpp/actors/util/affinity.h>
+#include <library/cpp/actors/util/datetime.h>
+#include <util/generic/hash.h>
+#include <util/system/rwlock.h>
+#include <util/random/random.h>
+
+namespace NActors {
+ LWTRACE_USING(ACTORLIB_PROVIDER);
+
+ struct TActorSystem::TServiceMap : TNonCopyable {
+ NActors::TServiceMap<TActorId, TActorId, TActorId::THash> LocalMap;
+ TTicketLock Lock;
+
+ TActorId RegisterLocalService(const TActorId& serviceId, const TActorId& actorId) {
+ TTicketLock::TGuard guard(&Lock);
+ const TActorId old = LocalMap.Update(serviceId, actorId);
+ return old;
+ }
+
+ TActorId LookupLocal(const TActorId& x) {
+ return LocalMap.Find(x);
+ }
+ };
+
+ TActorSystem::TActorSystem(THolder<TActorSystemSetup>& setup, void* appData,
+ TIntrusivePtr<NLog::TSettings> loggerSettings)
+ : NodeId(setup->NodeId)
+ , CpuManager(new TCpuManager(setup))
+ , ExecutorPoolCount(CpuManager->GetExecutorsCount())
+ , Scheduler(setup->Scheduler)
+ , InterconnectCount((ui32)setup->Interconnect.ProxyActors.size())
+ , CurrentTimestamp(0)
+ , CurrentMonotonic(0)
+ , CurrentIDCounter(RandomNumber<ui64>())
+ , SystemSetup(setup.Release())
+ , DefSelfID(NodeId, "actorsystem")
+ , AppData0(appData)
+ , LoggerSettings0(loggerSettings)
+ , StartExecuted(false)
+ , StopExecuted(false)
+ , CleanupExecuted(false)
+ {
+ ServiceMap.Reset(new TServiceMap());
+ }
+
+ TActorSystem::~TActorSystem() {
+ Cleanup();
+ }
+
+ bool TActorSystem::Send(TAutoPtr<IEventHandle> ev) const {
+ if (Y_UNLIKELY(!ev))
+ return false;
+
+#ifdef USE_ACTOR_CALLSTACK
+ ev->Callstack.TraceIfEmpty();
+#endif
+
+ TActorId recipient = ev->GetRecipientRewrite();
+ const ui32 recpNodeId = recipient.NodeId();
+
+ 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,
+ "Event rewrite from %s to %s would be lost via interconnect",
+ ev->Recipient.ToString().c_str(),
+ recipient.ToString().c_str());
+ recipient = InterconnectProxy(recpNodeId);
+ ev->Rewrite(TEvInterconnect::EvForward, recipient);
+ }
+ if (recipient.IsService()) {
+ TActorId target = ServiceMap->LookupLocal(recipient);
+ if (!target && IsInterconnectProxyId(recipient) && ProxyWrapperFactory) {
+ const TActorId actorId = ProxyWrapperFactory(const_cast<TActorSystem*>(this),
+ GetInterconnectProxyNode(recipient));
+ with_lock(ProxyCreationLock) {
+ target = ServiceMap->LookupLocal(recipient);
+ if (!target) {
+ target = actorId;
+ ServiceMap->RegisterLocalService(recipient, target);
+ }
+ }
+ if (target != actorId) {
+ // a race has occured, terminate newly created actor
+ Send(new IEventHandle(TEvents::TSystem::Poison, 0, actorId, {}, nullptr, 0));
+ }
+ }
+ recipient = target;
+ ev->Rewrite(ev->GetTypeRewrite(), recipient);
+ }
+
+ Y_VERIFY_DEBUG(recipient == ev->GetRecipientRewrite());
+ const ui32 recpPool = recipient.PoolID();
+ if (recipient && recpPool < ExecutorPoolCount) {
+ if (CpuManager->GetExecutorPool(recpPool)->Send(ev)) {
+ return true;
+ }
+ }
+
+ Send(ev->ForwardOnNondelivery(TEvents::TEvUndelivered::ReasonActorUnknown));
+ return false;
+ }
+
+ bool TActorSystem::Send(const TActorId& recipient, IEventBase* ev, ui32 flags) const {
+ return this->Send(new IEventHandle(recipient, DefSelfID, ev, flags));
+ }
+
+ void TActorSystem::Schedule(TInstant deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie) const {
+ Schedule(deadline - Timestamp(), ev, cookie);
+ }
+
+ void TActorSystem::Schedule(TMonotonic deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie) const {
+ const auto current = Monotonic();
+ if (deadline < current)
+ deadline = current;
+
+ TTicketLock::TGuard guard(&ScheduleLock);
+ ScheduleQueue->Writer.Push(deadline.MicroSeconds(), ev.Release(), cookie);
+ }
+
+ void TActorSystem::Schedule(TDuration delta, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie) const {
+ const auto deadline = Monotonic() + delta;
+
+ TTicketLock::TGuard guard(&ScheduleLock);
+ ScheduleQueue->Writer.Push(deadline.MicroSeconds(), ev.Release(), cookie);
+ }
+
+ TActorId TActorSystem::Register(IActor* actor, TMailboxType::EType mailboxType, ui32 executorPool, ui64 revolvingCounter,
+ const TActorId& parentId) {
+ Y_VERIFY(executorPool < ExecutorPoolCount, "executorPool# %" PRIu32 ", ExecutorPoolCount# %" PRIu32,
+ (ui32)executorPool, (ui32)ExecutorPoolCount);
+ return CpuManager->GetExecutorPool(executorPool)->Register(actor, mailboxType, revolvingCounter, parentId);
+ }
+
+ NThreading::TFuture<THolder<IEventBase>> TActorSystem::AskGeneric(TMaybe<ui32> expectedEventType,
+ TActorId recipient, THolder<IEventBase> event,
+ TDuration timeout) {
+ auto promise = NThreading::NewPromise<THolder<IEventBase>>();
+ Register(MakeAskActor(expectedEventType, recipient, std::move(event), timeout, promise).Release());
+ return promise.GetFuture();
+ }
+
+ ui64 TActorSystem::AllocateIDSpace(ui64 count) {
+ Y_VERIFY_DEBUG(count < Max<ui32>() / 65536);
+
+ static_assert(sizeof(TAtomic) == sizeof(ui64), "expect sizeof(TAtomic) == sizeof(ui64)");
+
+ // get high 32 bits as seconds from epoch
+ // it could wrap every century, but we don't expect any actor-reference to live this long so such wrap will do no harm
+ const ui64 timeFromEpoch = TInstant::MicroSeconds(RelaxedLoad(&CurrentTimestamp)).Seconds();
+
+ // get low 32 bits as counter value
+ ui32 lowPartEnd = (ui32)(AtomicAdd(CurrentIDCounter, count));
+ while (lowPartEnd < count) // if our request crosses 32bit boundary - retry
+ lowPartEnd = (ui32)(AtomicAdd(CurrentIDCounter, count));
+
+ const ui64 lowPart = lowPartEnd - count;
+ const ui64 ret = (timeFromEpoch << 32) | lowPart;
+
+ return ret;
+ }
+
+ TActorId TActorSystem::InterconnectProxy(ui32 destinationNode) const {
+ if (destinationNode < InterconnectCount)
+ return Interconnect[destinationNode];
+ else if (destinationNode != NodeId)
+ return MakeInterconnectProxyId(destinationNode);
+ else
+ return TActorId();
+ }
+
+ ui32 TActorSystem::BroadcastToProxies(const std::function<IEventHandle*(const TActorId&)>& eventFabric) {
+ // TODO: get rid of this method
+ for (ui32 i = 0; i < InterconnectCount; ++i) {
+ Send(eventFabric(Interconnect[i]));
+ }
+ return InterconnectCount;
+ }
+
+ TActorId TActorSystem::LookupLocalService(const TActorId& x) const {
+ return ServiceMap->LookupLocal(x);
+ }
+
+ TActorId TActorSystem::RegisterLocalService(const TActorId& serviceId, const TActorId& actorId) {
+ // TODO: notify old actor about demotion
+ return ServiceMap->RegisterLocalService(serviceId, actorId);
+ }
+
+ void TActorSystem::GetPoolStats(ui32 poolId, TExecutorPoolStats& poolStats, TVector<TExecutorThreadStats>& statsCopy) const {
+ CpuManager->GetPoolStats(poolId, poolStats, statsCopy);
+ }
+
+ void TActorSystem::Start() {
+ Y_VERIFY(StartExecuted == false);
+ StartExecuted = true;
+
+ ScheduleQueue.Reset(new NSchedulerQueue::TQueueType());
+ TVector<NSchedulerQueue::TReader*> scheduleReaders;
+ scheduleReaders.push_back(&ScheduleQueue->Reader);
+ CpuManager->PrepareStart(scheduleReaders, this);
+ Scheduler->Prepare(this, &CurrentTimestamp, &CurrentMonotonic);
+ Scheduler->PrepareSchedules(&scheduleReaders.front(), (ui32)scheduleReaders.size());
+
+ // setup interconnect proxies
+ {
+ const TInterconnectSetup& setup = SystemSetup->Interconnect;
+ Interconnect.Reset(new TActorId[InterconnectCount + 1]);
+ for (ui32 i = 0, e = InterconnectCount; i != e; ++i) {
+ const TActorSetupCmd& x = setup.ProxyActors[i];
+ if (x.Actor) {
+ Interconnect[i] = Register(x.Actor, x.MailboxType, x.PoolId, i);
+ Y_VERIFY(!!Interconnect[i]);
+ }
+ }
+ ProxyWrapperFactory = std::move(SystemSetup->Interconnect.ProxyWrapperFactory);
+ }
+
+ // setup local services
+ {
+ for (ui32 i = 0, e = (ui32)SystemSetup->LocalServices.size(); i != e; ++i) {
+ const std::pair<TActorId, TActorSetupCmd>& x = SystemSetup->LocalServices[i];
+ const TActorId xid = Register(x.second.Actor, x.second.MailboxType, x.second.PoolId, i);
+ Y_VERIFY(!!xid);
+ if (!!x.first)
+ RegisterLocalService(x.first, xid);
+ }
+ }
+
+ // ok, setup complete, we could destroy setup config
+ SystemSetup.Destroy();
+
+ Scheduler->PrepareStart();
+ CpuManager->Start();
+ Send(MakeSchedulerActorId(), new TEvSchedulerInitialize(scheduleReaders, &CurrentTimestamp, &CurrentMonotonic));
+ Scheduler->Start();
+ }
+
+ void TActorSystem::Stop() {
+ if (StopExecuted || !StartExecuted)
+ return;
+
+ StopExecuted = true;
+
+ for (auto&& fn : std::exchange(DeferredPreStop, {})) {
+ fn();
+ }
+
+ Scheduler->PrepareStop();
+ CpuManager->PrepareStop();
+ Scheduler->Stop();
+ CpuManager->Shutdown();
+ }
+
+ void TActorSystem::Cleanup() {
+ Stop();
+ if (CleanupExecuted || !StartExecuted)
+ return;
+ CleanupExecuted = true;
+ CpuManager->Cleanup();
+ Scheduler.Destroy();
+ }
+
+ ui32 TActorSystem::MemProfActivityBase;
+}
diff --git a/library/cpp/actors/core/actorsystem.h b/library/cpp/actors/core/actorsystem.h
new file mode 100644
index 0000000000..40499d7586
--- /dev/null
+++ b/library/cpp/actors/core/actorsystem.h
@@ -0,0 +1,367 @@
+#pragma once
+
+#include "defs.h"
+
+#include "actor.h"
+#include "balancer.h"
+#include "config.h"
+#include "event.h"
+#include "log_settings.h"
+#include "scheduler_cookie.h"
+#include "mon_stats.h"
+
+#include <library/cpp/threading/future/future.h>
+#include <library/cpp/actors/util/ticket_lock.h>
+
+#include <util/generic/vector.h>
+#include <util/datetime/base.h>
+#include <util/system/mutex.h>
+
+namespace NActors {
+ class TActorSystem;
+ class TCpuManager;
+ class IExecutorPool;
+ struct TWorkerContext;
+
+ inline TActorId MakeInterconnectProxyId(ui32 destNodeId) {
+ char data[12];
+ memcpy(data, "ICProxy@", 8);
+ memcpy(data + 8, &destNodeId, sizeof(ui32));
+ return TActorId(0, TStringBuf(data, 12));
+ }
+
+ inline bool IsInterconnectProxyId(const TActorId& actorId) {
+ return actorId.IsService() && !memcmp(actorId.ServiceId().data(), "ICProxy@", 8);
+ }
+
+ inline ui32 GetInterconnectProxyNode(const TActorId& actorId) {
+ ui32 nodeId;
+ memcpy(&nodeId, actorId.ServiceId().data() + 8, sizeof(ui32));
+ return nodeId;
+ }
+
+ namespace NSchedulerQueue {
+ class TReader;
+ struct TQueueType;
+ }
+
+ class IExecutorPool : TNonCopyable {
+ public:
+ const ui32 PoolId;
+
+ TAtomic ActorRegistrations;
+ TAtomic DestroyedActors;
+
+ IExecutorPool(ui32 poolId)
+ : PoolId(poolId)
+ , ActorRegistrations(0)
+ , DestroyedActors(0)
+ {
+ }
+
+ virtual ~IExecutorPool() {
+ }
+
+ // for workers
+ virtual ui32 GetReadyActivation(TWorkerContext& wctx, ui64 revolvingCounter) = 0;
+ virtual void ReclaimMailbox(TMailboxType::EType mailboxType, ui32 hint, TWorkerId workerId, ui64 revolvingCounter) = 0;
+
+ /**
+ * Schedule one-shot event that will be send at given time point in the future.
+ *
+ * @param deadline the wallclock time point in future when event must be send
+ * @param ev the event to send
+ * @param cookie cookie that will be piggybacked with event
+ * @param workerId index of thread which will perform event dispatching
+ */
+ virtual void Schedule(TInstant deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) = 0;
+
+ /**
+ * Schedule one-shot event that will be send at given time point in the future.
+ *
+ * @param deadline the monotonic time point in future when event must be send
+ * @param ev the event to send
+ * @param cookie cookie that will be piggybacked with event
+ * @param workerId index of thread which will perform event dispatching
+ */
+ virtual void Schedule(TMonotonic deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) = 0;
+
+ /**
+ * Schedule one-shot event that will be send after given delay.
+ *
+ * @param delta the time from now to delay event sending
+ * @param ev the event to send
+ * @param cookie cookie that will be piggybacked with event
+ * @param workerId index of thread which will perform event dispatching
+ */
+ virtual void Schedule(TDuration delta, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) = 0;
+
+ // for actorsystem
+ virtual bool Send(TAutoPtr<IEventHandle>& ev) = 0;
+ virtual void ScheduleActivation(ui32 activation) = 0;
+ virtual void ScheduleActivationEx(ui32 activation, ui64 revolvingCounter) = 0;
+ virtual TActorId Register(IActor* actor, TMailboxType::EType mailboxType, ui64 revolvingCounter, const TActorId& parentId) = 0;
+ virtual TActorId Register(IActor* actor, TMailboxHeader* mailbox, ui32 hint, const TActorId& parentId) = 0;
+
+ // lifecycle stuff
+ virtual void Prepare(TActorSystem* actorSystem, NSchedulerQueue::TReader** scheduleReaders, ui32* scheduleSz) = 0;
+ virtual void Start() = 0;
+ virtual void PrepareStop() = 0;
+ virtual void Shutdown() = 0;
+ virtual bool Cleanup() = 0;
+
+ virtual void GetCurrentStats(TExecutorPoolStats& poolStats, TVector<TExecutorThreadStats>& statsCopy) const {
+ // TODO: make pure virtual and override everywhere
+ Y_UNUSED(poolStats);
+ Y_UNUSED(statsCopy);
+ }
+
+ virtual TString GetName() const {
+ return TString();
+ }
+
+ virtual ui32 GetThreads() const {
+ return 1;
+ }
+
+ // generic
+ virtual TAffinity* Affinity() const = 0;
+
+ virtual void SetRealTimeMode() const {}
+ };
+
+ // could be proxy to in-pool schedulers (for NUMA-aware executors)
+ class ISchedulerThread : TNonCopyable {
+ public:
+ virtual ~ISchedulerThread() {
+ }
+
+ virtual void Prepare(TActorSystem* actorSystem, volatile ui64* currentTimestamp, volatile ui64* currentMonotonic) = 0;
+ virtual void PrepareSchedules(NSchedulerQueue::TReader** readers, ui32 scheduleReadersCount) = 0;
+ virtual void PrepareStart() { /* empty */ }
+ virtual void Start() = 0;
+ virtual void PrepareStop() = 0;
+ virtual void Stop() = 0;
+ };
+
+ struct TActorSetupCmd {
+ TMailboxType::EType MailboxType;
+ ui32 PoolId;
+ IActor* Actor;
+
+ TActorSetupCmd()
+ : MailboxType(TMailboxType::HTSwap)
+ , PoolId(0)
+ , Actor(nullptr)
+ {
+ }
+
+ TActorSetupCmd(IActor* actor, TMailboxType::EType mailboxType, ui32 poolId)
+ : MailboxType(mailboxType)
+ , PoolId(poolId)
+ , Actor(actor)
+ {
+ }
+
+ void Set(IActor* actor, TMailboxType::EType mailboxType, ui32 poolId) {
+ MailboxType = mailboxType;
+ PoolId = poolId;
+ Actor = actor;
+ }
+ };
+
+ using TProxyWrapperFactory = std::function<TActorId(TActorSystem*, ui32)>;
+
+ struct TInterconnectSetup {
+ TVector<TActorSetupCmd> ProxyActors;
+ TProxyWrapperFactory ProxyWrapperFactory;
+ };
+
+ struct TActorSystemSetup {
+ ui32 NodeId = 0;
+
+ // Either Executors or CpuManager must be initialized
+ ui32 ExecutorsCount = 0;
+ TArrayHolder<TAutoPtr<IExecutorPool>> Executors;
+
+ TAutoPtr<IBalancer> Balancer; // main implementation will be implicitly created if not set
+
+ TCpuManagerConfig CpuManager;
+
+ TAutoPtr<ISchedulerThread> Scheduler;
+ ui32 MaxActivityType = 5; // for default entries
+
+ TInterconnectSetup Interconnect;
+
+ using TLocalServices = TVector<std::pair<TActorId, TActorSetupCmd>>;
+ TLocalServices LocalServices;
+
+ ui32 GetExecutorsCount() const {
+ return Executors ? ExecutorsCount : CpuManager.GetExecutorsCount();
+ }
+
+ TString GetPoolName(ui32 poolId) const {
+ return Executors ? Executors[poolId]->GetName() : CpuManager.GetPoolName(poolId);
+ }
+
+ ui32 GetThreads(ui32 poolId) const {
+ return Executors ? Executors[poolId]->GetThreads() : CpuManager.GetThreads(poolId);
+ }
+ };
+
+ class TActorSystem : TNonCopyable {
+ struct TServiceMap;
+
+ public:
+ const ui32 NodeId;
+
+ private:
+ THolder<TCpuManager> CpuManager;
+ const ui32 ExecutorPoolCount;
+
+ TAutoPtr<ISchedulerThread> Scheduler;
+ THolder<TServiceMap> ServiceMap;
+
+ const ui32 InterconnectCount;
+ TArrayHolder<TActorId> Interconnect;
+
+ volatile ui64 CurrentTimestamp;
+ volatile ui64 CurrentMonotonic;
+ volatile ui64 CurrentIDCounter;
+
+ THolder<NSchedulerQueue::TQueueType> ScheduleQueue;
+ mutable TTicketLock ScheduleLock;
+
+ friend class TExecutorThread;
+
+ THolder<TActorSystemSetup> SystemSetup;
+ TActorId DefSelfID;
+ void* AppData0;
+ TIntrusivePtr<NLog::TSettings> LoggerSettings0;
+ TProxyWrapperFactory ProxyWrapperFactory;
+ TMutex ProxyCreationLock;
+
+ bool StartExecuted;
+ bool StopExecuted;
+ bool CleanupExecuted;
+
+ std::deque<std::function<void()>> DeferredPreStop;
+ public:
+ TActorSystem(THolder<TActorSystemSetup>& setup, void* appData = nullptr,
+ TIntrusivePtr<NLog::TSettings> loggerSettings = TIntrusivePtr<NLog::TSettings>(nullptr));
+ ~TActorSystem();
+
+ void Start();
+ void Stop();
+ void Cleanup();
+
+ TActorId Register(IActor* actor, TMailboxType::EType mailboxType = TMailboxType::HTSwap, ui32 executorPool = 0,
+ ui64 revolvingCounter = 0, const TActorId& parentId = TActorId());
+
+ bool Send(TAutoPtr<IEventHandle> ev) const;
+ bool Send(const TActorId& recipient, IEventBase* ev, ui32 flags = 0) const;
+
+ /**
+ * Schedule one-shot event that will be send at given time point in the future.
+ *
+ * @param deadline the wallclock time point in future when event must be send
+ * @param ev the event to send
+ * @param cookie cookie that will be piggybacked with event
+ */
+ void Schedule(TInstant deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie = nullptr) const;
+
+ /**
+ * Schedule one-shot event that will be send at given time point in the future.
+ *
+ * @param deadline the monotonic time point in future when event must be send
+ * @param ev the event to send
+ * @param cookie cookie that will be piggybacked with event
+ */
+ void Schedule(TMonotonic deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie = nullptr) const;
+
+ /**
+ * Schedule one-shot event that will be send after given delay.
+ *
+ * @param delta the time from now to delay event sending
+ * @param ev the event to send
+ * @param cookie cookie that will be piggybacked with event
+ */
+ void Schedule(TDuration delta, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie = nullptr) const;
+
+ /**
+ * A way to interact with actors from non-actor context.
+ *
+ * This method will send the `event` to the `recipient` and then will wait for a response. When response arrives,
+ * it will be passed to the future. If response is not of type `T`, the future will resolve into an exception.
+ *
+ * @tparam T expected response type. Must be derived from `TEventBase`,
+ * or use `IEventBase` to catch any response.
+ * @param actorSystem actor system that will be used to register an actor that'll wait for response.
+ * @param recipient who will get a request.
+ * @param event a request message.
+ * @return future that will be resolved when a message from `recipient` arrives.
+ */
+ template <typename T>
+ [[nodiscard]]
+ NThreading::TFuture<THolder<T>> Ask(TActorId recipient, THolder<IEventBase> event, TDuration timeout = TDuration::Max()) {
+ if constexpr (std::is_same_v<T, IEventBase>) {
+ return AskGeneric(Nothing(), recipient, std::move(event), timeout);
+ } else {
+ return AskGeneric(T::EventType, recipient, std::move(event), timeout)
+ .Apply([](const NThreading::TFuture<THolder<IEventBase>>& ev) {
+ return THolder<T>(static_cast<T*>(const_cast<THolder<IEventBase>&>(ev.GetValueSync()).Release())); // =(
+ });
+ }
+ }
+
+ [[nodiscard]]
+ NThreading::TFuture<THolder<IEventBase>> AskGeneric(
+ TMaybe<ui32> expectedEventType,
+ TActorId recipient,
+ THolder<IEventBase> event,
+ TDuration timeout);
+
+ ui64 AllocateIDSpace(ui64 count);
+
+ TActorId InterconnectProxy(ui32 destinationNode) const;
+ ui32 BroadcastToProxies(const std::function<IEventHandle*(const TActorId&)>&);
+
+ void UpdateLinkStatus(ui8 status, ui32 destinationNode);
+ ui8 LinkStatus(ui32 destinationNode);
+
+ TActorId LookupLocalService(const TActorId& x) const;
+ TActorId RegisterLocalService(const TActorId& serviceId, const TActorId& actorId);
+
+ ui32 GetMaxActivityType() const {
+ return SystemSetup ? SystemSetup->MaxActivityType : 1;
+ }
+
+ TInstant Timestamp() const {
+ return TInstant::MicroSeconds(RelaxedLoad(&CurrentTimestamp));
+ }
+
+ TMonotonic Monotonic() const {
+ return TMonotonic::MicroSeconds(RelaxedLoad(&CurrentMonotonic));
+ }
+
+ template <typename T>
+ T* AppData() const {
+ return (T*)AppData0;
+ }
+
+ NLog::TSettings* LoggerSettings() const {
+ return LoggerSettings0.Get();
+ }
+
+ void GetPoolStats(ui32 poolId, TExecutorPoolStats& poolStats, TVector<TExecutorThreadStats>& statsCopy) const;
+
+ void DeferPreStop(std::function<void()> fn) {
+ DeferredPreStop.push_back(std::move(fn));
+ }
+
+ /* This is the base for memory profiling tags.
+ System sets memory profiling tag for debug version of lfalloc.
+ The tag is set as "base_tag + actor_activity_type". */
+ static ui32 MemProfActivityBase;
+ };
+}
diff --git a/library/cpp/actors/core/actorsystem_ut.cpp b/library/cpp/actors/core/actorsystem_ut.cpp
new file mode 100644
index 0000000000..231d6f0ca1
--- /dev/null
+++ b/library/cpp/actors/core/actorsystem_ut.cpp
@@ -0,0 +1,45 @@
+#include "actorsystem.h"
+
+#include <library/cpp/actors/testlib/test_runtime.h>
+#include <library/cpp/testing/unittest/registar.h>
+
+using namespace NActors;
+
+Y_UNIT_TEST_SUITE(TActorSystemTest) {
+
+ class TTestActor: public TActor<TTestActor> {
+ public:
+ TTestActor()
+ : TActor{&TThis::Main}
+ {
+ }
+
+ STATEFN(Main) {
+ Y_UNUSED(ev);
+ }
+ };
+
+ THolder<TTestActorRuntimeBase> CreateRuntime() {
+ auto runtime = MakeHolder<TTestActorRuntimeBase>();
+ runtime->SetScheduledEventFilter([](auto&&, auto&&, auto&&, auto&&) { return false; });
+ runtime->Initialize();
+ return runtime;
+ }
+
+ Y_UNIT_TEST(LocalService) {
+ THolder<TTestActorRuntimeBase> runtime = CreateRuntime();
+ auto actorA = runtime->Register(new TTestActor);
+ auto actorB = runtime->Register(new TTestActor);
+
+ TActorId myServiceId{0, TStringBuf{"my-service"}};
+
+ auto prevActorId = runtime->RegisterService(myServiceId, actorA);
+ UNIT_ASSERT(!prevActorId);
+ UNIT_ASSERT_EQUAL(runtime->GetLocalServiceId(myServiceId), actorA);
+
+ prevActorId = runtime->RegisterService(myServiceId, actorB);
+ UNIT_ASSERT(prevActorId);
+ UNIT_ASSERT_EQUAL(prevActorId, actorA);
+ UNIT_ASSERT_EQUAL(runtime->GetLocalServiceId(myServiceId), actorB);
+ }
+}
diff --git a/library/cpp/actors/core/ask.cpp b/library/cpp/actors/core/ask.cpp
new file mode 100644
index 0000000000..0054c9a906
--- /dev/null
+++ b/library/cpp/actors/core/ask.cpp
@@ -0,0 +1,74 @@
+#include "ask.h"
+
+#include "actor_bootstrapped.h"
+#include "actorid.h"
+#include "event.h"
+#include "hfunc.h"
+
+namespace NActors {
+ namespace {
+ class TAskActor: public TActorBootstrapped<TAskActor> {
+ enum {
+ Timeout = EventSpaceBegin(TEvents::ES_PRIVATE),
+ };
+
+ // We can't use the standard timeout event because recipient may send us one.
+ struct TTimeout: public TEventLocal<TTimeout, Timeout> {
+ };
+
+ public:
+ TAskActor(
+ TMaybe<ui32> expectedEventType,
+ TActorId recipient,
+ THolder<IEventBase> event,
+ TDuration timeout,
+ const NThreading::TPromise<THolder<IEventBase>>& promise)
+ : ExpectedEventType_(expectedEventType)
+ , Recipient_(recipient)
+ , Event_(std::move(event))
+ , Timeout_(timeout)
+ , Promise_(promise)
+ {
+ }
+
+ public:
+ void Bootstrap() {
+ Send(Recipient_, std::move(Event_));
+ Become(&TAskActor::Waiting);
+
+ if (Timeout_ != TDuration::Max()) {
+ Schedule(Timeout_, new TTimeout);
+ }
+ }
+
+ STATEFN(Waiting) {
+ if (ev->GetTypeRewrite() == TTimeout::EventType) {
+ Promise_.SetException(std::make_exception_ptr(yexception() << "ask timeout"));
+ } else if (!ExpectedEventType_ || ev->GetTypeRewrite() == ExpectedEventType_) {
+ Promise_.SetValue(ev->ReleaseBase());
+ } else {
+ Promise_.SetException(std::make_exception_ptr(yexception() << "received unexpected response " << ev->GetBase()->ToString()));
+ }
+
+ PassAway();
+ }
+
+ public:
+ TMaybe<ui32> ExpectedEventType_;
+ TActorId Recipient_;
+ THolder<IEventBase> Event_;
+ TDuration Timeout_;
+ NThreading::TPromise<THolder<IEventBase>> Promise_;
+ };
+ }
+
+ THolder<IActor> MakeAskActor(
+ TMaybe<ui32> expectedEventType,
+ TActorId recipient,
+ THolder<IEventBase> event,
+ TDuration timeout,
+ const NThreading::TPromise<THolder<IEventBase>>& promise)
+ {
+ return MakeHolder<TAskActor>(expectedEventType, std::move(recipient), std::move(event), timeout, promise);
+ }
+}
diff --git a/library/cpp/actors/core/ask.h b/library/cpp/actors/core/ask.h
new file mode 100644
index 0000000000..036f1833a4
--- /dev/null
+++ b/library/cpp/actors/core/ask.h
@@ -0,0 +1,18 @@
+#pragma once
+
+#include "actor.h"
+#include "event.h"
+
+#include <library/cpp/threading/future/future.h>
+
+namespace NActors {
+ /**
+ * See `TActorSystem::Ask`.
+ */
+ THolder<IActor> MakeAskActor(
+ TMaybe<ui32> expectedEventType,
+ TActorId recipient,
+ THolder<IEventBase> event,
+ TDuration timeout,
+ const NThreading::TPromise<THolder<IEventBase>>& promise);
+}
diff --git a/library/cpp/actors/core/ask_ut.cpp b/library/cpp/actors/core/ask_ut.cpp
new file mode 100644
index 0000000000..e72ebdba9b
--- /dev/null
+++ b/library/cpp/actors/core/ask_ut.cpp
@@ -0,0 +1,131 @@
+#include <library/cpp/testing/unittest/registar.h>
+
+#include "actorsystem.h"
+
+#include <library/cpp/actors/testlib/test_runtime.h>
+
+using namespace NActors;
+
+class TPingPong: public TActor<TPingPong> {
+public:
+ TPingPong()
+ : TActor(&TPingPong::Main)
+ {
+ }
+
+ STATEFN(Main) {
+ switch (ev->GetTypeRewrite()) {
+ hFunc(TEvents::TEvPing, OnPing);
+ hFunc(TEvents::TEvBlob, OnBlob);
+ }
+ }
+
+ void OnPing(const TEvents::TEvPing::TPtr& ev) {
+ Send(ev->Sender, new TEvents::TEvPong);
+ }
+
+ void OnBlob(const TEvents::TEvBlob::TPtr& ev) {
+ Send(ev->Sender, ev->Release().Release());
+ }
+};
+
+class TPing: public TActor<TPing> {
+public:
+ TPing()
+ : TActor(&TPing::Main)
+ {
+ }
+
+ STATEFN(Main) {
+ Y_UNUSED(ev);
+ }
+};
+
+THolder<TTestActorRuntimeBase> CreateRuntime() {
+ auto runtime = MakeHolder<TTestActorRuntimeBase>();
+ runtime->SetScheduledEventFilter([](auto&&, auto&&, auto&&, auto&&) { return false; });
+ runtime->Initialize();
+ return runtime;
+}
+
+Y_UNIT_TEST_SUITE(AskActor) {
+ Y_UNIT_TEST(Ok) {
+ auto runtime = CreateRuntime();
+ auto pingpong = runtime->Register(new TPingPong);
+
+ {
+ auto fut = runtime->GetAnyNodeActorSystem()->Ask<TEvents::TEvPong>(
+ pingpong,
+ THolder(new TEvents::TEvPing));
+ runtime->DispatchEvents();
+ fut.ExtractValueSync();
+ }
+
+ {
+ auto fut = runtime->GetAnyNodeActorSystem()->Ask<TEvents::TEvBlob>(
+ pingpong,
+ THolder(new TEvents::TEvBlob("hello!")));
+ runtime->DispatchEvents();
+ auto ev = fut.ExtractValueSync();
+ UNIT_ASSERT_VALUES_EQUAL(ev->Blob, "hello!");
+ }
+
+ {
+ auto fut = runtime->GetAnyNodeActorSystem()->Ask<IEventBase>(
+ pingpong,
+ THolder(new TEvents::TEvPing));
+ runtime->DispatchEvents();
+ auto ev = fut.ExtractValueSync();
+ UNIT_ASSERT_VALUES_EQUAL(ev->Type(), TEvents::TEvPong::EventType);
+ }
+ }
+
+ Y_UNIT_TEST(Err) {
+ auto runtime = CreateRuntime();
+ auto pingpong = runtime->Register(new TPingPong);
+
+ {
+ auto fut = runtime->GetAnyNodeActorSystem()->Ask<TEvents::TEvBlob>(
+ pingpong,
+ THolder(new TEvents::TEvPing));
+ runtime->DispatchEvents();
+ UNIT_ASSERT_EXCEPTION_CONTAINS(
+ fut.ExtractValueSync(),
+ yexception,
+ "received unexpected response HelloWorld: Pong");
+ }
+ }
+
+ Y_UNIT_TEST(Timeout) {
+ auto runtime = CreateRuntime();
+ auto ping = runtime->Register(new TPing);
+
+ {
+ auto fut = runtime->GetAnyNodeActorSystem()->Ask<TEvents::TEvPong>(
+ ping,
+ THolder(new TEvents::TEvPing),
+ TDuration::Seconds(1));
+ auto start = runtime->GetCurrentTime();
+ runtime->DispatchEvents({}, TDuration::Seconds(5));
+ UNIT_ASSERT_EXCEPTION_CONTAINS(
+ fut.ExtractValueSync(),
+ yexception,
+ "ask timeout");
+ UNIT_ASSERT_VALUES_EQUAL(runtime->GetCurrentTime() - start, TDuration::Seconds(1));
+ }
+
+ {
+ auto fut = runtime->GetAnyNodeActorSystem()->Ask<IEventBase>(
+ ping,
+ THolder(new TEvents::TEvPing),
+ TDuration::Seconds(1));
+ auto start = runtime->GetCurrentTime();
+ runtime->DispatchEvents({}, TDuration::Seconds(5));
+ UNIT_ASSERT_EXCEPTION_CONTAINS(
+ fut.ExtractValueSync(),
+ yexception,
+ "ask timeout");
+ UNIT_ASSERT_VALUES_EQUAL(runtime->GetCurrentTime() - start, TDuration::Seconds(1));
+ }
+ }
+}
diff --git a/library/cpp/actors/core/balancer.cpp b/library/cpp/actors/core/balancer.cpp
new file mode 100644
index 0000000000..cc5417b0b5
--- /dev/null
+++ b/library/cpp/actors/core/balancer.cpp
@@ -0,0 +1,293 @@
+#include "balancer.h"
+
+#include "probes.h"
+
+#include <library/cpp/actors/util/intrinsics.h>
+#include <library/cpp/actors/util/datetime.h>
+
+#include <util/system/spinlock.h>
+
+#include <algorithm>
+
+namespace NActors {
+ LWTRACE_USING(ACTORLIB_PROVIDER);
+
+ // Describes balancing-related state of pool, the most notable is `Importance` to add new cpu
+ struct TLevel {
+ // Balancer will try to give more cpu to overloaded pools
+ enum ELoadClass {
+ Underloaded = 0,
+ Moderate = 1,
+ Overloaded = 2,
+ };
+
+ double ScaleFactor;
+ ELoadClass LoadClass;
+ ui64 Importance; // pool with lower importance is allowed to pass cpu to pool with higher, but the opposite is forbidden
+
+ TLevel() {}
+
+ TLevel(const TBalancingConfig& cfg, TPoolId poolId, ui64 currentCpus, double cpuIdle) {
+ ScaleFactor = double(currentCpus) / cfg.Cpus;
+ if (cpuIdle > 1.3) { // TODO: add a better underload criterion, based on estimated latency w/o 1 cpu
+ LoadClass = Underloaded;
+ } else if (cpuIdle < 0.2) { // TODO: add a better overload criterion, based on latency
+ LoadClass = Overloaded;
+ } else {
+ LoadClass = Moderate;
+ }
+ Importance = MakeImportance(LoadClass, cfg.Priority, ScaleFactor, cpuIdle, poolId);
+ }
+
+ private:
+ // Importance is simple ui64 value (from highest to lowest):
+ // 2 Bits: LoadClass
+ // 8 Bits: Priority
+ // 10 Bits: -ScaleFactor (for max-min fairness with weights equal to TBalancingConfig::Cpus)
+ // 10 Bits: -CpuIdle
+ // 6 Bits: PoolId
+ static ui64 MakeImportance(ELoadClass load, ui8 priority, double scaleFactor, double cpuIdle, TPoolId poolId) {
+ 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));
+
+ static_assert(ui64(MaxPools) <= (1ull << 6ull));
+
+ ui64 importance =
+ (ui64(load) << ui64(6 + 10 + 10 + 8)) |
+ (ui64(priority) << ui64(6 + 10 + 10)) |
+ (ui64(scale) << ui64(6 + 10)) |
+ (ui64(idle) << ui64(6)) |
+ ui64(poolId);
+ return importance;
+ }
+ };
+
+ // Main balancer implemenation
+ class TBalancer: public IBalancer {
+ private:
+ struct TCpu;
+ struct TPool;
+
+ bool Disabled = true;
+ TSpinLock Lock;
+ ui64 NextBalanceTs;
+ TVector<TCpu> Cpus; // Indexed by CpuId, can have gaps
+ TVector<TPool> Pools; // Indexed by PoolId, can have gaps
+ TBalancerConfig Config;
+
+ public:
+ // Setup
+ TBalancer(const TBalancerConfig& config, const TVector<TUnitedExecutorPoolConfig>& unitedPools, ui64 ts);
+ bool AddCpu(const TCpuAllocation& cpuAlloc, TCpuState* cpu) override;
+ ~TBalancer();
+
+ // Balancing
+ bool TryLock(ui64 ts) override;
+ void SetPoolStats(TPoolId pool, const TBalancerStats& stats) override;
+ void Balance() override;
+ void Unlock() override;
+
+ private:
+ void MoveCpu(TPool& from, TPool& to);
+ };
+
+ struct TBalancer::TPool {
+ TBalancingConfig Config;
+ TPoolId PoolId;
+ TString PoolName;
+
+ // Input data for balancing
+ TBalancerStats Prev;
+ TBalancerStats Next;
+
+ // Derived stats
+ double CpuLoad;
+ double CpuIdle;
+
+ // Classification
+ // NOTE: We want to avoid passing cpu back and forth, so we must consider not only current level,
+ // NOTE: but expected levels after movements also
+ TLevel CurLevel; // Level with current amount of cpu
+ TLevel AddLevel; // Level after one cpu acception
+ TLevel SubLevel; // Level after one cpu donation
+
+ // Balancing state
+ ui64 CurrentCpus = 0; // Total number of cpus assigned for this pool (zero means pools is not balanced)
+ ui64 PrevCpus = 0; // Cpus in last period
+
+ explicit TPool(const TBalancingConfig& cfg = {})
+ : Config(cfg)
+ {}
+
+ void Configure(const TBalancingConfig& cfg, const TString& poolName) {
+ Config = cfg;
+ // Enforce constraints
+ Config.MinCpus = std::clamp<ui32>(Config.MinCpus, 1, Config.Cpus);
+ Config.MaxCpus = Max<ui32>(Config.MaxCpus, Config.Cpus);
+ PoolName = poolName;
+ }
+ };
+
+ struct TBalancer::TCpu {
+ TCpuState* State = nullptr; // Cpu state, nullptr means cpu is not used (gap)
+ TCpuAllocation Alloc;
+ TPoolId Current;
+ TPoolId Assigned;
+ };
+
+ TBalancer::TBalancer(const TBalancerConfig& config, const TVector<TUnitedExecutorPoolConfig>& unitedPools, ui64 ts)
+ : NextBalanceTs(ts)
+ , Config(config)
+ {
+ for (TPoolId pool = 0; pool < MaxPools; pool++) {
+ Pools.emplace_back();
+ Pools.back().PoolId = pool;
+ }
+ for (const TUnitedExecutorPoolConfig& united : unitedPools) {
+ Pools[united.PoolId].Configure(united.Balancing, united.PoolName);
+ }
+ }
+
+ TBalancer::~TBalancer() {
+ }
+
+ bool TBalancer::AddCpu(const TCpuAllocation& cpuAlloc, TCpuState* state) {
+ // Setup
+ TCpuId cpuId = cpuAlloc.CpuId;
+ if (Cpus.size() <= cpuId) {
+ Cpus.resize(cpuId + 1);
+ }
+ TCpu& cpu = Cpus[cpuId];
+ cpu.State = state;
+ cpu.Alloc = cpuAlloc;
+
+ // Fill every pool with cpus up to TBalancingConfig::Cpus
+ TPoolId pool = 0;
+ for (TPool& p : Pools) {
+ if (p.CurrentCpus < p.Config.Cpus) {
+ p.CurrentCpus++;
+ break;
+ }
+ pool++;
+ }
+ if (pool != MaxPools) { // cpu under balancer control
+ state->SwitchPool(pool);
+ state->AssignPool(pool);
+ Disabled = false;
+ return true;
+ }
+ return false; // non-balanced cpu
+ }
+
+ bool TBalancer::TryLock(ui64 ts) {
+ if (!Disabled && NextBalanceTs < ts && Lock.TryAcquire()) {
+ NextBalanceTs = ts + Us2Ts(Config.PeriodUs);
+ return true;
+ }
+ return false;
+ }
+
+ void TBalancer::SetPoolStats(TPoolId pool, const TBalancerStats& stats) {
+ Y_VERIFY(pool < MaxPools);
+ TPool& p = Pools[pool];
+ p.Prev = p.Next;
+ p.Next = stats;
+ }
+
+ void TBalancer::Balance() {
+ // Update every cpu state
+ for (TCpu& cpu : Cpus) {
+ if (cpu.State) {
+ cpu.State->Load(cpu.Assigned, cpu.Current);
+ if (cpu.Current < MaxPools && cpu.Current != cpu.Assigned) {
+ return; // previous movement has not been applied yet, wait
+ }
+ }
+ }
+
+ // Process stats, classify and compute pool importance
+ TStackVec<TPool*, MaxPools> order;
+ for (TPool& pool : Pools) {
+ if (pool.Config.Cpus == 0) {
+ continue; // skip gaps (non-existent or non-united pools)
+ }
+ if (pool.Prev.Ts == 0 || pool.Prev.Ts >= pool.Next.Ts) {
+ return; // invalid stats
+ }
+
+ // Compute derived stats
+ pool.CpuLoad = (pool.Next.CpuUs - pool.Prev.CpuUs) / Ts2Us(pool.Next.Ts - pool.Prev.Ts);
+ if (pool.Prev.IdleUs == ui64(-1) || pool.Next.IdleUs == ui64(-1)) {
+ pool.CpuIdle = pool.CurrentCpus - pool.CpuLoad; // for tests
+ } else {
+ pool.CpuIdle = (pool.Next.IdleUs - pool.Prev.IdleUs) / Ts2Us(pool.Next.Ts - pool.Prev.Ts);
+ }
+
+ // Compute levels
+ pool.CurLevel = TLevel(pool.Config, pool.PoolId, pool.CurrentCpus, pool.CpuIdle);
+ pool.AddLevel = TLevel(pool.Config, pool.PoolId, pool.CurrentCpus + 1, pool.CpuIdle); // we expect taken cpu to became utilized
+ pool.SubLevel = TLevel(pool.Config, pool.PoolId, pool.CurrentCpus - 1, pool.CpuIdle - 1);
+
+ // Prepare for balancing
+ pool.PrevCpus = pool.CurrentCpus;
+ order.push_back(&pool);
+ }
+
+ // Sort pools by importance
+ std::sort(order.begin(), order.end(), [] (TPool* l, TPool* r) {return l->CurLevel.Importance < r->CurLevel.Importance; });
+ for (TPool* pool : order) {
+ LWPROBE(PoolStats, pool->PoolId, pool->PoolName, pool->CurrentCpus, pool->CurLevel.LoadClass, pool->Config.Priority, pool->CurLevel.ScaleFactor, pool->CpuIdle, pool->CpuLoad, pool->CurLevel.Importance, pool->AddLevel.Importance, pool->SubLevel.Importance);
+ }
+
+ // Move cpus from lower importance to higher importance pools
+ for (auto toIter = order.rbegin(); toIter != order.rend(); ++toIter) {
+ TPool& to = **toIter;
+ if (to.CurLevel.LoadClass == TLevel::Overloaded && // if pool is overloaded
+ to.CurrentCpus < to.Config.MaxCpus) // and constraints would not be violated
+ {
+ for (auto fromIter = order.begin(); (*fromIter)->CurLevel.Importance < to.CurLevel.Importance; ++fromIter) {
+ TPool& from = **fromIter;
+ if (from.CurrentCpus == from.PrevCpus && // if not balanced yet
+ from.CurrentCpus > from.Config.MinCpus && // and constraints would not be violated
+ from.SubLevel.Importance < to.AddLevel.Importance) // and which of two pools is more important would not change after cpu movement
+ {
+ MoveCpu(from, to);
+ from.CurrentCpus--;
+ to.CurrentCpus++;
+ break;
+ }
+ }
+ }
+ }
+ }
+
+ void TBalancer::MoveCpu(TBalancer::TPool& from, TBalancer::TPool& to) {
+ for (auto ci = Cpus.rbegin(), ce = Cpus.rend(); ci != ce; ci++) {
+ TCpu& cpu = *ci;
+ if (!cpu.State) {
+ continue;
+ }
+ if (cpu.Assigned == from.PoolId) {
+ cpu.State->AssignPool(to.PoolId);
+ cpu.Assigned = to.PoolId;
+ LWPROBE(MoveCpu, from.PoolId, to.PoolId, from.PoolName, to.PoolName, cpu.Alloc.CpuId);
+ return;
+ }
+ }
+ Y_FAIL();
+ }
+
+ void TBalancer::Unlock() {
+ Lock.Release();
+ }
+
+ IBalancer* MakeBalancer(const TBalancerConfig& config, const TVector<TUnitedExecutorPoolConfig>& unitedPools, ui64 ts) {
+ return new TBalancer(config, unitedPools, ts);
+ }
+}
diff --git a/library/cpp/actors/core/balancer.h b/library/cpp/actors/core/balancer.h
new file mode 100644
index 0000000000..9763ec79e1
--- /dev/null
+++ b/library/cpp/actors/core/balancer.h
@@ -0,0 +1,27 @@
+#pragma once
+
+#include "defs.h"
+#include "config.h"
+#include "cpu_state.h"
+
+namespace NActors {
+ // Per-pool statistics used by balancer
+ struct TBalancerStats {
+ ui64 Ts = 0; // Measurement timestamp
+ ui64 CpuUs = 0; // Total cpu microseconds consumed by pool on all cpus since start
+ ui64 IdleUs = ui64(-1); // Total cpu microseconds in spinning or waiting on futex
+ };
+
+ // Pool cpu balancer
+ struct IBalancer {
+ virtual ~IBalancer() {}
+ virtual bool AddCpu(const TCpuAllocation& cpuAlloc, TCpuState* cpu) = 0;
+ virtual bool TryLock(ui64 ts) = 0;
+ virtual void SetPoolStats(TPoolId pool, const TBalancerStats& stats) = 0;
+ virtual void Balance() = 0;
+ virtual void Unlock() = 0;
+ // TODO: add method for reconfiguration on fly
+ };
+
+ IBalancer* MakeBalancer(const TBalancerConfig& config, const TVector<TUnitedExecutorPoolConfig>& unitedPools, ui64 ts);
+}
diff --git a/library/cpp/actors/core/balancer_ut.cpp b/library/cpp/actors/core/balancer_ut.cpp
new file mode 100644
index 0000000000..7e5e95f4b9
--- /dev/null
+++ b/library/cpp/actors/core/balancer_ut.cpp
@@ -0,0 +1,225 @@
+#include "balancer.h"
+
+#include <library/cpp/actors/util/datetime.h>
+#include <library/cpp/lwtrace/all.h>
+#include <library/cpp/testing/unittest/registar.h>
+
+#include <util/stream/str.h>
+
+using namespace NActors;
+
+////////////////////////////////////////////////////////////////////////////////
+
+Y_UNIT_TEST_SUITE(PoolCpuBalancer) {
+ struct TTest {
+ TCpuManagerConfig Config;
+ TCpuMask Available;
+ THolder<IBalancer> Balancer;
+ TVector<TCpuState> CpuStates;
+ TVector<ui64> CpuUs;
+ ui64 Now = 0;
+
+ void SetCpuCount(size_t count) {
+ Config.UnitedWorkers.CpuCount = count;
+ for (TCpuId cpuId = 0; cpuId < count; cpuId++) {
+ Available.Set(cpuId);
+ }
+ }
+
+ void AddPool(ui32 minCpus, ui32 cpus, ui32 maxCpus, ui8 priority = 0) {
+ TUnitedExecutorPoolConfig u;
+ u.PoolId = TPoolId(Config.United.size());
+ u.Balancing.Cpus = cpus;
+ u.Balancing.MinCpus = minCpus;
+ u.Balancing.MaxCpus = maxCpus;
+ u.Balancing.Priority = priority;
+ Config.United.push_back(u);
+ }
+
+ void Start() {
+ TCpuAllocationConfig allocation(Available, Config);
+ Balancer.Reset(MakeBalancer(Config.UnitedWorkers.Balancer, Config.United, 0));
+ CpuStates.resize(allocation.Items.size()); // do not resize it later to avoid dangling pointers
+ CpuUs.resize(CpuStates.size());
+ for (const TCpuAllocation& cpuAlloc : allocation.Items) {
+ bool added = Balancer->AddCpu(cpuAlloc, &CpuStates[cpuAlloc.CpuId]);
+ UNIT_ASSERT(added);
+ }
+ }
+
+ void Balance(ui64 deltaTs, const TVector<ui64>& cpuUs) {
+ Now += deltaTs;
+ ui64 ts = Now;
+ if (Balancer->TryLock(ts)) {
+ for (TPoolId pool = 0; pool < cpuUs.size(); pool++) {
+ CpuUs[pool] += cpuUs[pool];
+ TBalancerStats stats;
+ stats.Ts = ts;
+ stats.CpuUs = CpuUs[pool];
+ Balancer->SetPoolStats(pool, stats);
+ }
+ Balancer->Balance();
+ Balancer->Unlock();
+ }
+ }
+
+ void ApplyMovements() {
+ for (TCpuState& state : CpuStates) {
+ TPoolId current;
+ TPoolId assigned;
+ state.Load(assigned, current);
+ state.SwitchPool(assigned);
+ }
+ }
+
+ static TString ToStr(const TVector<ui64>& values) {
+ TStringStream ss;
+ ss << "{";
+ for (auto v : values) {
+ ss << " " << v;
+ }
+ ss << " }";
+ return ss.Str();
+ }
+
+ void AssertPoolsCurrentCpus(const TVector<ui64>& cpuRequired) {
+ TVector<ui64> cpuCurrent;
+ cpuCurrent.resize(cpuRequired.size());
+ for (TCpuState& state : CpuStates) {
+ TPoolId current;
+ TPoolId assigned;
+ state.Load(assigned, current);
+ cpuCurrent[current]++;
+ }
+ for (TPoolId pool = 0; pool < cpuRequired.size(); pool++) {
+ UNIT_ASSERT_C(cpuCurrent[pool] == cpuRequired[pool],
+ "cpu distribution mismatch, required " << ToStr(cpuRequired) << " but got " << ToStr(cpuCurrent));
+ }
+ }
+ };
+
+ Y_UNIT_TEST(StartLwtrace) {
+ NLWTrace::StartLwtraceFromEnv();
+ }
+
+ Y_UNIT_TEST(AllOverloaded) {
+ TTest t;
+ int cpus = 10;
+ t.SetCpuCount(cpus);
+ t.AddPool(1, 1, 10); // pool=0
+ t.AddPool(1, 2, 10); // pool=1
+ t.AddPool(1, 3, 10); // pool=2
+ t.AddPool(1, 4, 10); // pool=2
+ t.Start();
+ ui64 dts = 1.01 * Us2Ts(t.Config.UnitedWorkers.Balancer.PeriodUs);
+ ui64 totalCpuUs = cpus * Ts2Us(dts); // pretend every pool has consumed as whole actorsystem, overload
+ for (int i = 0; i < cpus; i++) {
+ t.Balance(dts, {totalCpuUs, totalCpuUs, totalCpuUs, totalCpuUs});
+ t.ApplyMovements();
+ }
+ t.AssertPoolsCurrentCpus({1, 2, 3, 4});
+ }
+
+ Y_UNIT_TEST(OneOverloaded) {
+ TTest t;
+ int cpus = 10;
+ t.SetCpuCount(cpus);
+ t.AddPool(1, 1, 10); // pool=0
+ t.AddPool(1, 2, 10); // pool=1
+ t.AddPool(1, 3, 10); // pool=2
+ t.AddPool(1, 4, 10); // pool=2
+ t.Start();
+ ui64 dts = 1.01 * Us2Ts(t.Config.UnitedWorkers.Balancer.PeriodUs);
+ ui64 totalCpuUs = cpus * Ts2Us(dts);
+ for (int i = 0; i < cpus; i++) {
+ t.Balance(dts, {totalCpuUs, 0, 0, 0});
+ t.ApplyMovements();
+ }
+ t.AssertPoolsCurrentCpus({7, 1, 1, 1});
+ for (int i = 0; i < cpus; i++) {
+ t.Balance(dts, {0, totalCpuUs, 0, 0});
+ t.ApplyMovements();
+ }
+ t.AssertPoolsCurrentCpus({1, 7, 1, 1});
+ for (int i = 0; i < cpus; i++) {
+ t.Balance(dts, {0, 0, totalCpuUs, 0});
+ t.ApplyMovements();
+ }
+ t.AssertPoolsCurrentCpus({1, 1, 7, 1});
+ for (int i = 0; i < cpus; i++) {
+ t.Balance(dts, {0, 0, 0, totalCpuUs});
+ t.ApplyMovements();
+ }
+ t.AssertPoolsCurrentCpus({1, 1, 1, 7});
+ }
+
+ Y_UNIT_TEST(TwoOverloadedFairness) {
+ TTest t;
+ int cpus = 10;
+ t.SetCpuCount(cpus);
+ t.AddPool(1, 1, 10); // pool=0
+ t.AddPool(1, 2, 10); // pool=1
+ t.AddPool(1, 3, 10); // pool=2
+ t.AddPool(1, 4, 10); // pool=2
+ t.Start();
+ ui64 dts = 1.01 * Us2Ts(t.Config.UnitedWorkers.Balancer.PeriodUs);
+ ui64 totalCpuUs = cpus * Ts2Us(dts);
+ for (int i = 0; i < cpus; i++) {
+ t.Balance(dts, {totalCpuUs, totalCpuUs, 0, 0});
+ t.ApplyMovements();
+ }
+ t.AssertPoolsCurrentCpus({3, 5, 1, 1});
+ for (int i = 0; i < cpus; i++) {
+ t.Balance(dts, {totalCpuUs, 0, totalCpuUs, 0});
+ t.ApplyMovements();
+ }
+ t.AssertPoolsCurrentCpus({2, 1, 6, 1});
+ for (int i = 0; i < cpus; i++) {
+ t.Balance(dts, {totalCpuUs, 0, 0, totalCpuUs});
+ t.ApplyMovements();
+ }
+ t.AssertPoolsCurrentCpus({2, 1, 1, 6});
+ for (int i = 0; i < cpus; i++) {
+ t.Balance(dts, {0, totalCpuUs, totalCpuUs, 0});
+ t.ApplyMovements();
+ }
+ t.AssertPoolsCurrentCpus({1, 3, 5, 1});
+ for (int i = 0; i < cpus; i++) {
+ t.Balance(dts, {0, totalCpuUs, 0, totalCpuUs});
+ t.ApplyMovements();
+ }
+ t.AssertPoolsCurrentCpus({1, 3, 1, 5});
+ for (int i = 0; i < cpus; i++) {
+ t.Balance(dts, {0, 0, totalCpuUs, totalCpuUs});
+ t.ApplyMovements();
+ }
+ t.AssertPoolsCurrentCpus({1, 1, 3, 5});
+ }
+
+ Y_UNIT_TEST(TwoOverloadedPriority) {
+ TTest t;
+ int cpus = 20;
+ t.SetCpuCount(cpus);
+ t.AddPool(1, 5, 20, 0); // pool=0
+ t.AddPool(1, 5, 20, 1); // pool=1
+ t.AddPool(1, 5, 20, 2); // pool=2
+ t.AddPool(1, 5, 20, 3); // pool=3
+ t.Start();
+ ui64 dts = 1.01 * Us2Ts(t.Config.UnitedWorkers.Balancer.PeriodUs);
+ ui64 mErlang = Ts2Us(dts) / 1000;
+ for (int i = 0; i < cpus; i++) {
+ t.Balance(dts, {20000 * mErlang, 2500 * mErlang, 4500 * mErlang, 9500 * mErlang});
+ t.ApplyMovements();
+ }
+ t.AssertPoolsCurrentCpus({2, 3, 5, 10});
+ t.Balance(dts, {20000 * mErlang, 2500 * mErlang, 4500 * mErlang, 8500 * mErlang});
+ t.ApplyMovements();
+ t.AssertPoolsCurrentCpus({3, 3, 5, 9});
+ // NOTE: this operation require one move, but we do not make global analysis, so multiple steps (1->2 & 0->1) are required (can be optimized later)
+ for (int i = 0; i < 3; i++) {
+ t.Balance(dts, {20000 * mErlang, 2500 * mErlang, 5500 * mErlang, 8500 * mErlang});
+ t.ApplyMovements();
+ }
+ t.AssertPoolsCurrentCpus({2, 3, 6, 9});
+ }
+}
diff --git a/library/cpp/actors/core/buffer.cpp b/library/cpp/actors/core/buffer.cpp
new file mode 100644
index 0000000000..48128d76ef
--- /dev/null
+++ b/library/cpp/actors/core/buffer.cpp
@@ -0,0 +1,93 @@
+#include "buffer.h"
+
+#include <util/system/yassert.h>
+
+#include <algorithm>
+
+TBufferBase::TBufferBase(size_t size) noexcept
+ : Size(size)
+{
+}
+
+size_t
+TBufferBase::GetSize() const noexcept {
+ return Size;
+}
+
+void TBufferBase::SetSize(size_t size) noexcept {
+ Size = size;
+}
+
+/////////////////////////////////////////////////////////////////////
+
+template <typename PointerType>
+TBufferBaseT<PointerType>::TBufferBaseT(PointerType data, size_t size) noexcept
+ : TBufferBase(size)
+ , Data(data)
+{
+}
+
+template <typename PointerType>
+PointerType
+TBufferBaseT<PointerType>::GetPointer() const noexcept {
+ return Data;
+}
+
+template <typename PointerType>
+void TBufferBaseT<PointerType>::Assign(PointerType data, size_t size) noexcept {
+ Data = data;
+ Size = size;
+}
+
+template <>
+void TBufferBaseT<void*>::Cut(size_t offset) noexcept {
+ Y_VERIFY_DEBUG(offset <= Size);
+ Data = static_cast<char*>(Data) + offset;
+ TBufferBase::Size -= offset;
+}
+
+template <>
+void TBufferBaseT<const void*>::Cut(size_t offset) noexcept {
+ Y_VERIFY_DEBUG(offset <= Size);
+ Data = static_cast<const char*>(Data) + offset;
+ TBufferBase::Size -= offset;
+}
+
+template class TBufferBaseT<void*>;
+template class TBufferBaseT<const void*>;
+
+/////////////////////////////////////////////////////////////////////
+
+TConstBuffer::TConstBuffer(const void* data, size_t size) noexcept
+ : TBufferBaseT<const void*>(data, size)
+{
+}
+
+TConstBuffer::TConstBuffer(const TMutableBuffer& buffer) noexcept
+ : TBufferBaseT<const void*>(buffer.GetPointer(), buffer.GetSize())
+{
+}
+
+TConstBuffer
+TConstBuffer::Offset(ptrdiff_t offset, size_t size) const noexcept {
+ return TConstBuffer(static_cast<const char*>(Data) + offset, std::min(Size - offset, size));
+}
+
+////////////////////////////////////////////////////////////////////////////////
+
+TMutableBuffer::TMutableBuffer(void* data, size_t size) noexcept
+ : TBufferBaseT<void*>(data, size)
+{
+}
+
+TMutableBuffer
+TMutableBuffer::Offset(ptrdiff_t offset, size_t size) const noexcept {
+ return TMutableBuffer(static_cast<char*>(Data) + offset, std::min(Size - offset, size));
+}
+
+size_t
+TMutableBuffer::CopyFrom(const TConstBuffer& buffer) const noexcept {
+ const auto size = std::min(Size, buffer.Size);
+ std::memcpy(Data, buffer.Data, size);
+ return size;
+}
diff --git a/library/cpp/actors/core/buffer.h b/library/cpp/actors/core/buffer.h
new file mode 100644
index 0000000000..95425046d6
--- /dev/null
+++ b/library/cpp/actors/core/buffer.h
@@ -0,0 +1,62 @@
+#pragma once
+
+#include <limits>
+
+class TConstBuffer;
+class TMutableBuffer;
+
+class TBufferBase {
+public:
+ size_t GetSize() const noexcept;
+
+ void SetSize(size_t newSize) noexcept;
+
+protected:
+ TBufferBase(size_t size = 0) noexcept;
+
+ size_t Size;
+};
+
+template <typename PointerType>
+class TBufferBaseT: public TBufferBase {
+public:
+ PointerType GetPointer() const noexcept;
+
+ void Cut(size_t offset) noexcept;
+
+ void Assign(PointerType data = nullptr, size_t size = 0U) noexcept;
+
+protected:
+ TBufferBaseT(PointerType data, size_t size) noexcept;
+
+ PointerType Data;
+};
+
+/// Represents constant memory buffer, but do not owns it.
+class TConstBuffer: public TBufferBaseT<const void*> {
+ friend class TMutableBuffer;
+
+public:
+ TConstBuffer(const TMutableBuffer& buffer) noexcept;
+
+ TConstBuffer(const void* data = nullptr, size_t size = 0U) noexcept;
+
+ TConstBuffer Offset(ptrdiff_t offset, size_t size = std::numeric_limits<size_t>::max()) const noexcept;
+};
+
+/// Represents mutable memory buffer, but do not owns it.
+class TMutableBuffer: public TBufferBaseT<void*> {
+ friend class TConstBuffer;
+
+public:
+ TMutableBuffer(void* data = nullptr, size_t size = 0U) noexcept;
+
+ TMutableBuffer(const TMutableBuffer& value) noexcept
+ : TBufferBaseT<void*>(value)
+ {
+ }
+
+ TMutableBuffer Offset(ptrdiff_t offset, size_t size = std::numeric_limits<size_t>::max()) const noexcept;
+
+ size_t CopyFrom(const TConstBuffer& buffer) const noexcept;
+};
diff --git a/library/cpp/actors/core/callstack.cpp b/library/cpp/actors/core/callstack.cpp
new file mode 100644
index 0000000000..9297c1a079
--- /dev/null
+++ b/library/cpp/actors/core/callstack.cpp
@@ -0,0 +1,93 @@
+#include "callstack.h"
+#include <util/thread/singleton.h>
+
+#ifdef USE_ACTOR_CALLSTACK
+
+namespace NActors {
+ namespace {
+ void (*PreviousFormatBackTrace)(IOutputStream*) = 0;
+ ui32 ActorBackTraceEnableCounter = 0;
+ }
+
+ void ActorFormatBackTrace(IOutputStream* out) {
+ TStringStream str;
+ PreviousFormatBackTrace(&str);
+ str << Endl;
+ TCallstack::DumpCallstack(str);
+ *out << str.Str();
+ }
+
+ void EnableActorCallstack() {
+ if (ActorBackTraceEnableCounter == 0) {
+ Y_VERIFY(PreviousFormatBackTrace == 0);
+ PreviousFormatBackTrace = SetFormatBackTraceFn(ActorFormatBackTrace);
+ }
+
+ ++ActorBackTraceEnableCounter;
+ }
+
+ void DisableActorCallstack() {
+ --ActorBackTraceEnableCounter;
+
+ if (ActorBackTraceEnableCounter == 0) {
+ Y_VERIFY(PreviousFormatBackTrace);
+ SetFormatBackTraceFn(PreviousFormatBackTrace);
+ PreviousFormatBackTrace = 0;
+ }
+ }
+
+ TCallstack::TCallstack()
+ : BeginIdx(0)
+ , Size(0)
+ , LinesToSkip(0)
+ {
+ }
+
+ void TCallstack::SetLinesToSkip() {
+ TTrace record;
+ LinesToSkip = BackTrace(record.Data, TTrace::CAPACITY);
+ }
+
+ void TCallstack::Trace() {
+ size_t currentIdx = (BeginIdx + Size) % RECORDS;
+ if (Size == RECORDS) {
+ ++BeginIdx;
+ } else {
+ ++Size;
+ }
+ TTrace& record = Record[currentIdx];
+ record.Size = BackTrace(record.Data, TTrace::CAPACITY);
+ record.LinesToSkip = LinesToSkip;
+ }
+
+ void TCallstack::TraceIfEmpty() {
+ if (Size == 0) {
+ LinesToSkip = 0;
+ Trace();
+ }
+ }
+
+ TCallstack& TCallstack::GetTlsCallstack() {
+ return *FastTlsSingleton<TCallstack>();
+ }
+
+ void TCallstack::DumpCallstack(TStringStream& str) {
+ TCallstack& callstack = GetTlsCallstack();
+ for (int i = callstack.Size - 1; i >= 0; --i) {
+ TTrace& record = callstack.Record[(callstack.BeginIdx + i) % RECORDS];
+ str << Endl << "Trace entry " << i << Endl << Endl;
+ size_t size = record.Size;
+ if (size > record.LinesToSkip && size < TTrace::CAPACITY) {
+ size -= record.LinesToSkip;
+ }
+ if (size > RECORDS_TO_SKIP) {
+ FormatBackTrace(&str, &record.Data[RECORDS_TO_SKIP], size - RECORDS_TO_SKIP);
+ } else {
+ FormatBackTrace(&str, record.Data, size);
+ }
+ str << Endl;
+ }
+ }
+}
+
+#endif
diff --git a/library/cpp/actors/core/callstack.h b/library/cpp/actors/core/callstack.h
new file mode 100644
index 0000000000..176717d2ae
--- /dev/null
+++ b/library/cpp/actors/core/callstack.h
@@ -0,0 +1,58 @@
+#pragma once
+
+#ifndef NDEBUG
+//#define ENABLE_ACTOR_CALLSTACK
+#endif
+
+#ifdef ENABLE_ACTOR_CALLSTACK
+#include "defs.h"
+#include <util/system/backtrace.h>
+#include <util/stream/str.h>
+#include <util/generic/deque.h>
+#define USE_ACTOR_CALLSTACK
+
+namespace NActors {
+ struct TCallstack {
+ struct TTrace {
+ static const size_t CAPACITY = 50;
+ void* Data[CAPACITY];
+ size_t Size;
+ size_t LinesToSkip;
+
+ TTrace()
+ : Size(0)
+ , LinesToSkip(0)
+ {
+ }
+ };
+
+ static const size_t RECORDS = 8;
+ static const size_t RECORDS_TO_SKIP = 2;
+ TTrace Record[RECORDS];
+ size_t BeginIdx;
+ size_t Size;
+ size_t LinesToSkip;
+
+ TCallstack();
+ void SetLinesToSkip();
+ void Trace();
+ void TraceIfEmpty();
+ static TCallstack& GetTlsCallstack();
+ static void DumpCallstack(TStringStream& str);
+ };
+
+ void EnableActorCallstack();
+ void DisableActorCallstack();
+
+}
+
+#else
+
+namespace NActors {
+ inline void EnableActorCallstack(){};
+
+ inline void DisableActorCallstack(){};
+
+}
+
+#endif
diff --git a/library/cpp/actors/core/config.h b/library/cpp/actors/core/config.h
new file mode 100644
index 0000000000..2486bf4c43
--- /dev/null
+++ b/library/cpp/actors/core/config.h
@@ -0,0 +1,239 @@
+#pragma once
+
+#include "defs.h"
+#include <library/cpp/actors/util/cpumask.h>
+#include <library/cpp/monlib/dynamic_counters/counters.h>
+#include <util/datetime/base.h>
+#include <util/generic/ptr.h>
+#include <util/generic/string.h>
+#include <util/generic/vector.h>
+
+namespace NActors {
+
+ struct TBalancingConfig {
+ // Default cpu count (used during overload). Zero value disables this pool balancing
+ // 1) Sum of `Cpus` on all pools cannot be changed without restart
+ // (changing cpu mode between Shared and Assigned is not implemented yet)
+ // 2) This sum must be equal to TUnitedWorkersConfig::CpuCount,
+ // otherwise `CpuCount - SUM(Cpus)` cpus will be in Shared mode (i.e. actorsystem 2.0)
+ ui32 Cpus = 0;
+
+ ui32 MinCpus = 0; // Lower balancing bound, should be at least 1, and not greater than `Cpus`
+ ui32 MaxCpus = 0; // Higher balancing bound, should be not lower than `Cpus`
+ ui8 Priority = 0; // Priority of pool to obtain cpu due to balancing (higher is better)
+ ui64 ToleratedLatencyUs = 0; // p100-latency threshold indicating that more cpus are required by pool
+ };
+
+ struct TBalancerConfig {
+ ui64 PeriodUs = 15000000; // Time between balancer steps
+ };
+
+ struct TBasicExecutorPoolConfig {
+ static constexpr TDuration DEFAULT_TIME_PER_MAILBOX = TDuration::MilliSeconds(10);
+ static constexpr ui32 DEFAULT_EVENTS_PER_MAILBOX = 100;
+
+ ui32 PoolId = 0;
+ TString PoolName;
+ ui32 Threads = 1;
+ ui64 SpinThreshold = 100;
+ TCpuMask Affinity; // Executor thread affinity
+ TDuration TimePerMailbox = DEFAULT_TIME_PER_MAILBOX;
+ ui32 EventsPerMailbox = DEFAULT_EVENTS_PER_MAILBOX;
+ int RealtimePriority = 0;
+ ui32 MaxActivityType = 1;
+ };
+
+ struct TIOExecutorPoolConfig {
+ ui32 PoolId = 0;
+ TString PoolName;
+ ui32 Threads = 1;
+ TCpuMask Affinity; // Executor thread affinity
+ ui32 MaxActivityType = 1;
+ };
+
+ struct TUnitedExecutorPoolConfig {
+ static constexpr TDuration DEFAULT_TIME_PER_MAILBOX = TDuration::MilliSeconds(10);
+ static constexpr ui32 DEFAULT_EVENTS_PER_MAILBOX = 100;
+
+ ui32 PoolId = 0;
+ TString PoolName;
+
+ // Resource sharing
+ ui32 Concurrency = 0; // Limits simultaneously running mailboxes count if set to non-zero value (do not set if Balancing.Cpus != 0)
+ TPoolWeight Weight = 0; // Weight in fair cpu-local pool scheduler
+ TCpuMask Allowed; // Allowed CPUs for workers to run this pool on (ignored if balancer works, i.e. actorsystem 1.5)
+
+ // Single mailbox execution limits
+ TDuration TimePerMailbox = DEFAULT_TIME_PER_MAILBOX;
+ ui32 EventsPerMailbox = DEFAULT_EVENTS_PER_MAILBOX;
+
+ // Introspection
+ ui32 MaxActivityType = 1;
+
+ // Long-term balancing
+ TBalancingConfig Balancing;
+ };
+
+ struct TUnitedWorkersConfig {
+ ui32 CpuCount = 0; // Total CPUs running united workers (i.e. TBasicExecutorPoolConfig::Threads analog); set to zero to disable united workers
+ ui64 SpinThresholdUs = 100; // Limit for active spinning in case all pools became idle
+ ui64 PoolLimitUs = 500; // Soft limit on pool execution
+ ui64 EventLimitUs = 100; // Hard limit on last event execution exceeding pool limit
+ ui64 LimitPrecisionUs = 100; // Maximum delay of timer on limit excess (delay needed to avoid settimer syscall on every pool switch)
+ ui64 FastWorkerPriority = 10; // Real-time priority of workers not exceeding hard limits
+ ui64 IdleWorkerPriority = 20; // Real-time priority of standby workers waiting for hard preemption on timers (should be greater than FastWorkerPriority)
+ TCpuMask Allowed; // Allowed CPUs for workers to run on (every worker has affinity for exactly one cpu)
+ bool NoRealtime = false; // For environments w/o permissions for RT-threads
+ bool NoAffinity = false; // For environments w/o permissions for cpu affinity
+ TBalancerConfig Balancer;
+ };
+
+ struct TCpuManagerConfig {
+ TUnitedWorkersConfig UnitedWorkers;
+ TVector<TBasicExecutorPoolConfig> Basic;
+ TVector<TIOExecutorPoolConfig> IO;
+ TVector<TUnitedExecutorPoolConfig> United;
+
+ ui32 GetExecutorsCount() const {
+ return Basic.size() + IO.size() + United.size();
+ }
+
+ TString GetPoolName(ui32 poolId) const {
+ for (const auto& p : Basic) {
+ if (p.PoolId == poolId) {
+ return p.PoolName;
+ }
+ }
+ for (const auto& p : IO) {
+ if (p.PoolId == poolId) {
+ return p.PoolName;
+ }
+ }
+ for (const auto& p : United) {
+ if (p.PoolId == poolId) {
+ return p.PoolName;
+ }
+ }
+ Y_FAIL("undefined pool id: %" PRIu32, (ui32)poolId);
+ }
+
+ ui32 GetThreads(ui32 poolId) const {
+ for (const auto& p : Basic) {
+ if (p.PoolId == poolId) {
+ return p.Threads;
+ }
+ }
+ for (const auto& p : IO) {
+ if (p.PoolId == poolId) {
+ return p.Threads;
+ }
+ }
+ for (const auto& p : United) {
+ if (p.PoolId == poolId) {
+ return p.Concurrency ? p.Concurrency : UnitedWorkers.CpuCount;
+ }
+ }
+ Y_FAIL("undefined pool id: %" PRIu32, (ui32)poolId);
+ }
+ };
+
+ struct TSchedulerConfig {
+ TSchedulerConfig(
+ ui64 resolution = 1024,
+ ui64 spinThreshold = 100,
+ ui64 progress = 10000,
+ bool useSchedulerActor = false)
+ : ResolutionMicroseconds(resolution)
+ , SpinThreshold(spinThreshold)
+ , ProgressThreshold(progress)
+ , UseSchedulerActor(useSchedulerActor)
+ {}
+
+ ui64 ResolutionMicroseconds = 1024;
+ ui64 SpinThreshold = 100;
+ ui64 ProgressThreshold = 10000;
+ bool UseSchedulerActor = false; // False is default because tests use scheduler thread
+ ui64 RelaxedSendPaceEventsPerSecond = 200000;
+ ui64 RelaxedSendPaceEventsPerCycle = RelaxedSendPaceEventsPerSecond * ResolutionMicroseconds / 1000000;
+ // For resolution >= 250000 microseconds threshold is SendPace
+ // For resolution <= 250 microseconds threshold is 20 * SendPace
+ ui64 RelaxedSendThresholdEventsPerSecond = RelaxedSendPaceEventsPerSecond *
+ (20 - ((20 - 1) * ClampVal(ResolutionMicroseconds, ui64(250), ui64(250000)) - 250) / (250000 - 250));
+ ui64 RelaxedSendThresholdEventsPerCycle = RelaxedSendThresholdEventsPerSecond * ResolutionMicroseconds / 1000000;
+
+ // Optional subsection for scheduler counters (usually subsystem=utils)
+ NMonitoring::TDynamicCounterPtr MonCounters = nullptr;
+ };
+
+ struct TCpuAllocation {
+ struct TPoolAllocation {
+ TPoolId PoolId;
+ TPoolWeight Weight;
+
+ TPoolAllocation(TPoolId poolId = 0, TPoolWeight weight = 0)
+ : PoolId(poolId)
+ , Weight(weight)
+ {}
+ };
+
+ TCpuId CpuId;
+ TVector<TPoolAllocation> AllowedPools;
+
+ TPoolsMask GetPoolsMask() const {
+ TPoolsMask mask = 0;
+ for (const auto& pa : AllowedPools) {
+ if (pa.PoolId < MaxPools) {
+ mask &= (1ull << pa.PoolId);
+ }
+ }
+ return mask;
+ }
+
+ bool HasPool(TPoolId pool) const {
+ for (const auto& pa : AllowedPools) {
+ if (pa.PoolId == pool) {
+ return true;
+ }
+ }
+ return false;
+ }
+ };
+
+ struct TCpuAllocationConfig {
+ TVector<TCpuAllocation> Items;
+
+ 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)",
+ pool.PoolName.c_str(), (pool.PoolId));
+ }
+ ui32 allocated[MaxPools] = {0};
+ for (TCpuId cpu = 0; cpu < available.Size() && Items.size() < cfg.UnitedWorkers.CpuCount; cpu++) {
+ if (available.IsSet(cpu)) {
+ TCpuAllocation item;
+ item.CpuId = cpu;
+ for (const TUnitedExecutorPoolConfig& pool : cfg.United) {
+ if (cfg.UnitedWorkers.Allowed.IsEmpty() || cfg.UnitedWorkers.Allowed.IsSet(cpu)) {
+ if (pool.Allowed.IsEmpty() || pool.Allowed.IsSet(cpu)) {
+ item.AllowedPools.emplace_back(pool.PoolId, pool.Weight);
+ allocated[pool.PoolId]++;
+ }
+ }
+ }
+ if (!item.AllowedPools.empty()) {
+ Items.push_back(item);
+ }
+ }
+ }
+ for (const TUnitedExecutorPoolConfig& pool : cfg.United) {
+ Y_VERIFY(allocated[pool.PoolId] > 0, "unable to allocate cpu for united executor pool: %s(%d)",
+ pool.PoolName.c_str(), (pool.PoolId));
+ }
+ }
+
+ operator bool() const {
+ return !Items.empty();
+ }
+ };
+
+}
diff --git a/library/cpp/actors/core/cpu_manager.cpp b/library/cpp/actors/core/cpu_manager.cpp
new file mode 100644
index 0000000000..39089b5d83
--- /dev/null
+++ b/library/cpp/actors/core/cpu_manager.cpp
@@ -0,0 +1,108 @@
+#include "cpu_manager.h"
+#include "probes.h"
+
+namespace NActors {
+ LWTRACE_USING(ACTORLIB_PROVIDER);
+
+ void TCpuManager::Setup() {
+ TAffinity available;
+ available.Current();
+ TCpuAllocationConfig allocation(available, Config);
+
+ if (allocation) {
+ if (!Balancer) {
+ Balancer.Reset(MakeBalancer(Config.UnitedWorkers.Balancer, Config.United, GetCycleCountFast()));
+ }
+ UnitedWorkers.Reset(new TUnitedWorkers(Config.UnitedWorkers, Config.United, allocation, Balancer.Get()));
+ }
+
+ Executors.Reset(new TAutoPtr<IExecutorPool>[ExecutorPoolCount]);
+
+ for (ui32 excIdx = 0; excIdx != ExecutorPoolCount; ++excIdx) {
+ Executors[excIdx].Reset(CreateExecutorPool(excIdx));
+ }
+ }
+
+ void TCpuManager::PrepareStart(TVector<NSchedulerQueue::TReader*>& scheduleReaders, TActorSystem* actorSystem) {
+ if (UnitedWorkers) {
+ UnitedWorkers->Prepare(actorSystem, scheduleReaders);
+ }
+ for (ui32 excIdx = 0; excIdx != ExecutorPoolCount; ++excIdx) {
+ NSchedulerQueue::TReader* readers;
+ ui32 readersCount = 0;
+ Executors[excIdx]->Prepare(actorSystem, &readers, &readersCount);
+ for (ui32 i = 0; i != readersCount; ++i, ++readers) {
+ scheduleReaders.push_back(readers);
+ }
+ }
+ }
+
+ void TCpuManager::Start() {
+ if (UnitedWorkers) {
+ UnitedWorkers->Start();
+ }
+ for (ui32 excIdx = 0; excIdx != ExecutorPoolCount; ++excIdx) {
+ Executors[excIdx]->Start();
+ }
+ }
+
+ void TCpuManager::PrepareStop() {
+ for (ui32 excIdx = 0; excIdx != ExecutorPoolCount; ++excIdx) {
+ Executors[excIdx]->PrepareStop();
+ }
+ if (UnitedWorkers) {
+ UnitedWorkers->PrepareStop();
+ }
+ }
+
+ void TCpuManager::Shutdown() {
+ for (ui32 excIdx = 0; excIdx != ExecutorPoolCount; ++excIdx) {
+ Executors[excIdx]->Shutdown();
+ }
+ if (UnitedWorkers) {
+ UnitedWorkers->Shutdown();
+ }
+ for (ui32 round = 0, done = 0; done < ExecutorPoolCount && round < 3; ++round) {
+ done = 0;
+ for (ui32 excIdx = 0; excIdx != ExecutorPoolCount; ++excIdx) {
+ if (Executors[excIdx]->Cleanup()) {
+ ++done;
+ }
+ }
+ }
+ }
+
+ 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");
+ done = 0;
+ for (ui32 excIdx = 0; excIdx != ExecutorPoolCount; ++excIdx) {
+ if (Executors[excIdx]->Cleanup()) {
+ ++done;
+ }
+ }
+ }
+ Executors.Destroy();
+ UnitedWorkers.Destroy();
+ }
+
+ IExecutorPool* TCpuManager::CreateExecutorPool(ui32 poolId) {
+ for (TBasicExecutorPoolConfig& cfg : Config.Basic) {
+ if (cfg.PoolId == poolId) {
+ return new TBasicExecutorPool(cfg);
+ }
+ }
+ for (TIOExecutorPoolConfig& cfg : Config.IO) {
+ if (cfg.PoolId == poolId) {
+ return new TIOExecutorPool(cfg);
+ }
+ }
+ for (TUnitedExecutorPoolConfig& cfg : Config.United) {
+ if (cfg.PoolId == poolId) {
+ IExecutorPool* result = new TUnitedExecutorPool(cfg, UnitedWorkers.Get());
+ return result;
+ }
+ }
+ Y_FAIL("missing PoolId: %d", int(poolId));
+ }
+}
diff --git a/library/cpp/actors/core/cpu_manager.h b/library/cpp/actors/core/cpu_manager.h
new file mode 100644
index 0000000000..454035477b
--- /dev/null
+++ b/library/cpp/actors/core/cpu_manager.h
@@ -0,0 +1,57 @@
+#pragma once
+
+#include "actorsystem.h"
+#include "executor_pool_basic.h"
+#include "executor_pool_io.h"
+#include "executor_pool_united.h"
+
+namespace NActors {
+ class TCpuManager : public TNonCopyable {
+ const ui32 ExecutorPoolCount;
+ TArrayHolder<TAutoPtr<IExecutorPool>> Executors;
+ THolder<TUnitedWorkers> UnitedWorkers;
+ THolder<IBalancer> Balancer;
+ TCpuManagerConfig Config;
+ public:
+ explicit TCpuManager(THolder<TActorSystemSetup>& setup)
+ : ExecutorPoolCount(setup->GetExecutorsCount())
+ , Balancer(setup->Balancer)
+ , Config(setup->CpuManager)
+ {
+ if (setup->Executors) { // Explicit mode w/o united pools
+ Executors.Reset(setup->Executors.Release());
+ for (ui32 excIdx = 0; excIdx != ExecutorPoolCount; ++excIdx) {
+ IExecutorPool* pool = Executors[excIdx].Get();
+ Y_VERIFY(dynamic_cast<TUnitedExecutorPool*>(pool) == nullptr,
+ "united executor pool is prohibited in explicit mode of NActors::TCpuManager");
+ }
+ } else {
+ Setup();
+ }
+ }
+
+ void Setup();
+ void PrepareStart(TVector<NSchedulerQueue::TReader*>& scheduleReaders, TActorSystem* actorSystem);
+ void Start();
+ void PrepareStop();
+ void Shutdown();
+ void Cleanup();
+
+ ui32 GetExecutorsCount() const {
+ return ExecutorPoolCount;
+ }
+
+ IExecutorPool* GetExecutorPool(ui32 poolId) {
+ return Executors[poolId].Get();
+ }
+
+ void GetPoolStats(ui32 poolId, TExecutorPoolStats& poolStats, TVector<TExecutorThreadStats>& statsCopy) const {
+ if (poolId < ExecutorPoolCount) {
+ Executors[poolId]->GetCurrentStats(poolStats, statsCopy);
+ }
+ }
+
+ private:
+ IExecutorPool* CreateExecutorPool(ui32 poolId);
+ };
+}
diff --git a/library/cpp/actors/core/cpu_state.h b/library/cpp/actors/core/cpu_state.h
new file mode 100644
index 0000000000..b8030149a7
--- /dev/null
+++ b/library/cpp/actors/core/cpu_state.h
@@ -0,0 +1,215 @@
+#pragma once
+
+#include "defs.h"
+
+#include <library/cpp/actors/util/futex.h>
+
+namespace NActors {
+
+ class alignas(64) TCpuState {
+ // Atomic cachelign-aligned 64-bit state, see description below
+ TAtomic State = 0;
+ char Padding[64 - sizeof(TAtomic)];
+
+ // Bits 0-31: Currently executing pool
+ // - value less than MaxPools means cpu is executing corresponding pool (fast-worker is executing or waiting for slow-workers)
+ // - one of Cpu* values in case of idle cpu
+ // - used as futex by blocked fast-worker
+ static constexpr ui64 CurrentBits = 32;
+ static constexpr ui64 CurrentMask = ui64((1ull << CurrentBits) - 1);
+
+ // Bits 32-63: Assigned pool
+ // - value is set by balancer
+ // - NOT used as futex
+ // - Not balanced
+ static constexpr ui64 AssignedOffs = 32;
+ static constexpr ui64 AssignedMask = ~CurrentMask;
+
+ public:
+ TCpuState() {
+ Y_UNUSED(Padding);
+ }
+
+ void Load(TPoolId& assigned, TPoolId& current) const {
+ TAtomicBase state = AtomicLoad(&State);
+ assigned = (state & AssignedMask) >> AssignedOffs;
+ current = state & CurrentMask;
+ }
+
+ TPoolId CurrentPool() const {
+ return TPoolId(AtomicLoad(&State) & CurrentMask);
+ }
+
+ void SwitchPool(TPoolId pool) {
+ while (true) {
+ TAtomicBase state = AtomicLoad(&State);
+ if (AtomicCas(&State, (state & ~CurrentMask) | pool, state)) {
+ return;
+ }
+ }
+ }
+
+ TPoolId AssignedPool() const {
+ return TPoolId((AtomicLoad(&State) & AssignedMask) >> AssignedOffs);
+ }
+
+ // Assigns new pool to cpu and wakes it up if cpu is idle
+ void AssignPool(TPoolId pool) {
+ while (true) {
+ TAtomicBase state = AtomicLoad(&State);
+ TPoolId current(state & CurrentMask);
+ if (Y_UNLIKELY(current == CpuStopped)) {
+ return; // it would be better to shutdown instead of balancing
+ }
+ // Idle cpu must be woken up after balancing to handle pending tokens (if any) in assigned/schedulable pool(s)
+ if (current == CpuSpinning) {
+ if (AtomicCas(&State, (ui64(pool) << AssignedOffs) | pool, state)) {
+ return; // successfully woken up
+ }
+ } else if (current == CpuBlocked) {
+ if (AtomicCas(&State, (ui64(pool) << AssignedOffs) | pool, state)) {
+ FutexWake();
+ return; // successfully woken up
+ }
+ } else {
+ if (AtomicCas(&State, (ui64(pool) << AssignedOffs) | (state & ~AssignedMask), state)) {
+ return; // wakeup is not required
+ }
+ }
+ }
+ }
+
+ void Stop() {
+ while (true) {
+ TAtomicBase state = AtomicLoad(&State);
+ if (AtomicCas(&State, (state & ~CurrentMask) | CpuStopped, state)) {
+ FutexWake();
+ return; // successfully stopped
+ }
+ }
+ }
+
+ // Start waiting, returns false in case of actorsystem shutdown
+ bool StartSpinning() {
+ while (true) {
+ TAtomicBase state = AtomicLoad(&State);
+ TPoolId current(state & CurrentMask);
+ if (Y_UNLIKELY(current == CpuStopped)) {
+ return false;
+ }
+ Y_VERIFY_DEBUG(current < MaxPools, "unexpected already waiting state of cpu (%d)", (int)current);
+ if (AtomicCas(&State, (state & ~CurrentMask) | CpuSpinning, state)) { // successfully marked as spinning
+ return true;
+ }
+ }
+ }
+
+ bool StartBlocking() {
+ while (true) {
+ TAtomicBase state = AtomicLoad(&State);
+ TPoolId current(state & CurrentMask);
+ if (current == CpuSpinning) {
+ if (AtomicCas(&State, (state & ~CurrentMask) | CpuBlocked, state)) {
+ return false; // successful switch
+ }
+ } else {
+ return true; // wakeup
+ }
+ }
+ }
+
+ bool Block(ui64 timeoutNs, TPoolId& result) {
+#ifdef _linux_
+ timespec timeout;
+ timeout.tv_sec = timeoutNs / 1'000'000'000;
+ timeout.tv_nsec = timeoutNs % 1'000'000'000;
+ SysFutex(Futex(), FUTEX_WAIT_PRIVATE, CpuBlocked, &timeout, nullptr, 0);
+#else
+ NanoSleep(timeoutNs); // non-linux wake is not supported, cpu will go idle on wake after blocked state
+#endif
+ TAtomicBase state = AtomicLoad(&State);
+ TPoolId current(state & CurrentMask);
+ if (current == CpuBlocked) {
+ return false; // timeout
+ } else {
+ result = current;
+ return true; // wakeup
+ }
+ }
+
+ enum EWakeResult {
+ Woken, // successfully woken up
+ NotIdle, // cpu is already not idle
+ Forbidden, // cpu is assigned to another pool
+ Stopped, // cpu is shutdown
+ };
+
+ EWakeResult WakeWithoutToken(TPoolId pool) {
+ while (true) {
+ TAtomicBase state = RelaxedLoad(&State);
+ TPoolId current(state & CurrentMask);
+ TPoolId assigned((state & AssignedMask) >> AssignedOffs);
+ if (assigned == CpuShared || assigned == pool) {
+ if (current == CpuSpinning) {
+ if (AtomicCas(&State, (state & ~CurrentMask) | pool, state)) {
+ return Woken;
+ }
+ } else if (current == CpuBlocked) {
+ if (AtomicCas(&State, (state & ~CurrentMask) | pool, state)) {
+ FutexWake();
+ return Woken;
+ }
+ } else if (current == CpuStopped) {
+ return Stopped;
+ } else {
+ return NotIdle;
+ }
+ } else {
+ return Forbidden;
+ }
+ }
+ }
+
+ EWakeResult WakeWithTokenAcquired(TPoolId token) {
+ while (true) {
+ TAtomicBase state = RelaxedLoad(&State);
+ TPoolId current(state & CurrentMask);
+ // NOTE: We ignore assigned value because we already have token, so
+ // NOTE: not assigned pool may be run here. This will be fixed
+ // NOTE: after we finish with current activation
+ if (current == CpuSpinning) {
+ if (AtomicCas(&State, (state & ~CurrentMask) | token, state)) {
+ return Woken;
+ }
+ } else if (current == CpuBlocked) {
+ if (AtomicCas(&State, (state & ~CurrentMask) | token, state)) {
+ FutexWake();
+ return Woken;
+ }
+ } else if (current == CpuStopped) {
+ return Stopped;
+ } else {
+ return NotIdle;
+ }
+ }
+ }
+
+ bool IsPoolReassigned(TPoolId current) const {
+ TAtomicBase state = AtomicLoad(&State);
+ TPoolId assigned((state & AssignedMask) >> AssignedOffs);
+ return assigned != current;
+ }
+
+ private:
+ void* Futex() {
+ return (void*)&State; // little endian assumed
+ }
+
+ void FutexWake() {
+#ifdef _linux_
+ SysFutex(Futex(), FUTEX_WAKE_PRIVATE, 1, nullptr, nullptr, 0);
+#endif
+ }
+ };
+
+}
diff --git a/library/cpp/actors/core/defs.h b/library/cpp/actors/core/defs.h
new file mode 100644
index 0000000000..980b7d767b
--- /dev/null
+++ b/library/cpp/actors/core/defs.h
@@ -0,0 +1,69 @@
+#pragma once
+
+// unique tag to fix pragma once gcc glueing: ./library/actorlib/core/defs.h
+
+#include <library/cpp/actors/util/defs.h>
+#include <util/generic/hash.h>
+#include <util/string/printf.h>
+
+// Enables collection of
+// event send/receive counts
+// activation time histograms
+// event processing time histograms
+#define ACTORSLIB_COLLECT_EXEC_STATS
+
+namespace NActors {
+ using TPoolId = ui8;
+ using TPoolsMask = ui64;
+ static constexpr TPoolId PoolBits = 6;
+ static constexpr TPoolId MaxPools = (1 << PoolBits) - 1; // maximum amount of pools (poolid=63 is reserved)
+ static constexpr TPoolsMask WaitPoolsFlag = (1ull << MaxPools); // wait-for-slow-workers flag bitmask
+
+ // Special TPoolId values used by TCpuState
+ static constexpr TPoolId CpuSpinning = MaxPools; // fast-worker is actively spinning, no slow-workers
+ static constexpr TPoolId CpuBlocked = MaxPools + 1; // fast-worker is blocked, no slow-workers
+ static constexpr TPoolId CpuStopped = TPoolId(-1); // special value indicating worker should stop
+ static constexpr TPoolId CpuShared = MaxPools; // special value for `assigned` meaning balancer disabled, pool scheduler is used instead
+
+ using TPoolWeight = ui16;
+ static constexpr TPoolWeight MinPoolWeight = 1;
+ static constexpr TPoolWeight DefPoolWeight = 32;
+ static constexpr TPoolWeight MaxPoolWeight = 1024;
+
+ using TWorkerId = ui16;
+ static constexpr TWorkerId WorkerBits = 11;
+ static constexpr TWorkerId MaxWorkers = 1 << WorkerBits;
+
+ using TThreadId = ui64;
+ static constexpr TThreadId UnknownThreadId = ui64(-1);
+
+ struct TMailboxType {
+ enum EType {
+ Inherited = -1, // inherit mailbox from parent
+ Simple = 0, // simplest queue under producer lock. fastest in no-contention case
+ Revolving = 1, // somewhat outdated, tries to be wait-free. replaced by ReadAsFilled
+ HTSwap = 2, // other simple lf queue, suggested for low-contention case
+ ReadAsFilled = 3, // wait-free queue, suggested for high-contention or latency critical
+ TinyReadAsFilled = 4, // same as 3 but with lower overhead
+ //Inplace;
+ //Direct;
+ //Virtual
+ };
+ };
+
+ struct TScopeId : std::pair<ui64, ui64> {
+ using TBase = std::pair<ui64, ui64>;
+ using TBase::TBase;
+ static const TScopeId LocallyGenerated;
+ };
+
+ static inline TString ScopeIdToString(const TScopeId& scopeId) {
+ return Sprintf("<%" PRIu64 ":%" PRIu64 ">", scopeId.first, scopeId.second);
+ }
+
+}
+
+template<>
+struct hash<NActors::TScopeId> : hash<std::pair<ui64, ui64>> {};
+
+class TAffinity;
diff --git a/library/cpp/actors/core/event.cpp b/library/cpp/actors/core/event.cpp
new file mode 100644
index 0000000000..33f8ce2aaf
--- /dev/null
+++ b/library/cpp/actors/core/event.cpp
@@ -0,0 +1,38 @@
+#include "event.h"
+#include "event_pb.h"
+
+namespace NActors {
+
+ const TScopeId TScopeId::LocallyGenerated{
+ Max<ui64>(), Max<ui64>()
+ };
+
+ TIntrusivePtr<TEventSerializedData> IEventHandle::ReleaseChainBuffer() {
+ if (Buffer) {
+ TIntrusivePtr<TEventSerializedData> result;
+ DoSwap(result, Buffer);
+ Event.Reset();
+ return result;
+ }
+ if (Event) {
+ TAllocChunkSerializer serializer;
+ Event->SerializeToArcadiaStream(&serializer);
+ auto chainBuf = serializer.Release(Event->IsExtendedFormat());
+ Event.Reset();
+ return chainBuf;
+ }
+ return new TEventSerializedData;
+ }
+
+ TIntrusivePtr<TEventSerializedData> IEventHandle::GetChainBuffer() {
+ if (Buffer)
+ return Buffer;
+ if (Event) {
+ TAllocChunkSerializer serializer;
+ Event->SerializeToArcadiaStream(&serializer);
+ Buffer = serializer.Release(Event->IsExtendedFormat());
+ return Buffer;
+ }
+ return new TEventSerializedData;
+ }
+}
diff --git a/library/cpp/actors/core/event.h b/library/cpp/actors/core/event.h
new file mode 100644
index 0000000000..6ff02aaf94
--- /dev/null
+++ b/library/cpp/actors/core/event.h
@@ -0,0 +1,344 @@
+#pragma once
+
+#include "defs.h"
+#include "actorid.h"
+#include "callstack.h"
+#include "event_load.h"
+
+#include <library/cpp/actors/wilson/wilson_trace.h>
+
+#include <util/system/hp_timer.h>
+#include <util/generic/maybe.h>
+
+namespace NActors {
+ class TChunkSerializer;
+
+ class ISerializerToStream {
+ public:
+ virtual bool SerializeToArcadiaStream(TChunkSerializer*) const = 0;
+ };
+
+ class IEventBase
+ : TNonCopyable,
+ public ISerializerToStream {
+ public:
+ // actual typing is performed by IEventHandle
+
+ virtual ~IEventBase() {
+ }
+
+ virtual TString ToStringHeader() const = 0;
+ virtual TString ToString() const {
+ return ToStringHeader();
+ }
+ virtual ui32 CalculateSerializedSize() const {
+ return 0;
+ }
+ virtual ui32 Type() const = 0;
+ virtual bool SerializeToArcadiaStream(TChunkSerializer*) const = 0;
+ virtual bool IsSerializable() const = 0;
+ virtual bool IsExtendedFormat() const {
+ return false;
+ }
+ virtual ui32 CalculateSerializedSizeCached() const {
+ return CalculateSerializedSize();
+ }
+ };
+
+ // fat handle
+ class IEventHandle : TNonCopyable {
+ struct TOnNondelivery {
+ TActorId Recipient;
+
+ TOnNondelivery(const TActorId& recipient)
+ : Recipient(recipient)
+ {
+ }
+ };
+
+ public:
+ template <typename TEv>
+ inline TEv* CastAsLocal() const noexcept {
+ auto fits = GetTypeRewrite() == TEv::EventType;
+
+ return fits ? static_cast<TEv*>(Event.Get()) : nullptr;
+ }
+
+ template <typename TEventType>
+ TEventType* Get() {
+ if (Type != TEventType::EventType)
+ Y_FAIL("Event type %" PRIu32 " doesn't match the expected type %" PRIu32, Type, TEventType::EventType);
+
+ if (!Event) {
+ Event.Reset(TEventType::Load(Buffer.Get()));
+ }
+
+ if (Event) {
+ return static_cast<TEventType*>(Event.Get());
+ }
+
+ Y_FAIL("Failed to Load() event type %" PRIu32 " class %s", Type, TypeName<TEventType>().data());
+ }
+
+ template <typename T>
+ TAutoPtr<T> Release() {
+ TAutoPtr<T> x = Get<T>();
+ Y_UNUSED(Event.Release());
+ Buffer.Reset();
+ return x;
+ }
+
+ enum EFlags {
+ FlagTrackDelivery = 1 << 0,
+ FlagForwardOnNondelivery = 1 << 1,
+ FlagSubscribeOnSession = 1 << 2,
+ FlagUseSubChannel = 1 << 3,
+ FlagGenerateUnsureUndelivered = 1 << 4,
+ FlagExtendedFormat = 1 << 5,
+ };
+
+ const ui32 Type;
+ const ui32 Flags;
+ const TActorId Recipient;
+ const TActorId Sender;
+ const ui64 Cookie;
+ const TScopeId OriginScopeId = TScopeId::LocallyGenerated; // filled in when the message is received from Interconnect
+
+ // if set, used by ActorSystem/Interconnect to report tracepoints
+ NWilson::TTraceId TraceId;
+
+ // filled if feeded by interconnect session
+ const TActorId InterconnectSession;
+
+#ifdef ACTORSLIB_COLLECT_EXEC_STATS
+ ::NHPTimer::STime SendTime;
+#endif
+
+ static const size_t ChannelBits = 12;
+ static const size_t ChannelShift = (sizeof(ui32) << 3) - ChannelBits;
+
+#ifdef USE_ACTOR_CALLSTACK
+ TCallstack Callstack;
+#endif
+ ui16 GetChannel() const noexcept {
+ return Flags >> ChannelShift;
+ }
+
+ ui64 GetSubChannel() const noexcept {
+ return Flags & FlagUseSubChannel ? Sender.LocalId() : 0ULL;
+ }
+
+ static ui32 MakeFlags(ui32 channel, ui32 flags) {
+ Y_VERIFY(channel < (1 << ChannelBits));
+ Y_VERIFY(flags < (1 << ChannelShift));
+ return (flags | (channel << ChannelShift));
+ }
+
+ private:
+ THolder<IEventBase> Event;
+ TIntrusivePtr<TEventSerializedData> Buffer;
+
+ TActorId RewriteRecipient;
+ ui32 RewriteType;
+
+ THolder<TOnNondelivery> OnNondeliveryHolder; // only for local events
+
+ public:
+ void Rewrite(ui32 typeRewrite, TActorId recipientRewrite) {
+ RewriteRecipient = recipientRewrite;
+ RewriteType = typeRewrite;
+ }
+
+ void DropRewrite() {
+ RewriteRecipient = Recipient;
+ RewriteType = Type;
+ }
+
+ const TActorId& GetRecipientRewrite() const {
+ return RewriteRecipient;
+ }
+
+ ui32 GetTypeRewrite() const {
+ return RewriteType;
+ }
+
+ TActorId GetForwardOnNondeliveryRecipient() const {
+ return OnNondeliveryHolder.Get() ? OnNondeliveryHolder->Recipient : TActorId();
+ }
+
+ IEventHandle(const TActorId& recipient, const TActorId& sender, IEventBase* ev, ui32 flags = 0, ui64 cookie = 0,
+ const TActorId* forwardOnNondelivery = nullptr, NWilson::TTraceId traceId = {})
+ : Type(ev->Type())
+ , Flags(flags)
+ , Recipient(recipient)
+ , Sender(sender)
+ , Cookie(cookie)
+ , TraceId(std::move(traceId))
+#ifdef ACTORSLIB_COLLECT_EXEC_STATS
+ , SendTime(0)
+#endif
+ , Event(ev)
+ , RewriteRecipient(Recipient)
+ , RewriteType(Type)
+ {
+ if (forwardOnNondelivery)
+ OnNondeliveryHolder.Reset(new TOnNondelivery(*forwardOnNondelivery));
+ }
+
+ IEventHandle(ui32 type,
+ ui32 flags,
+ const TActorId& recipient,
+ const TActorId& sender,
+ TIntrusivePtr<TEventSerializedData> buffer,
+ ui64 cookie,
+ const TActorId* forwardOnNondelivery = nullptr,
+ NWilson::TTraceId traceId = {})
+ : Type(type)
+ , Flags(flags)
+ , Recipient(recipient)
+ , Sender(sender)
+ , Cookie(cookie)
+ , TraceId(std::move(traceId))
+#ifdef ACTORSLIB_COLLECT_EXEC_STATS
+ , SendTime(0)
+#endif
+ , Buffer(std::move(buffer))
+ , RewriteRecipient(Recipient)
+ , RewriteType(Type)
+ {
+ if (forwardOnNondelivery)
+ OnNondeliveryHolder.Reset(new TOnNondelivery(*forwardOnNondelivery));
+ }
+
+ // Special ctor for events from interconnect.
+ IEventHandle(const TActorId& session,
+ ui32 type,
+ ui32 flags,
+ const TActorId& recipient,
+ const TActorId& sender,
+ TIntrusivePtr<TEventSerializedData> buffer,
+ ui64 cookie,
+ TScopeId originScopeId,
+ NWilson::TTraceId traceId) noexcept
+ : Type(type)
+ , Flags(flags)
+ , Recipient(recipient)
+ , Sender(sender)
+ , Cookie(cookie)
+ , OriginScopeId(originScopeId)
+ , TraceId(std::move(traceId))
+ , InterconnectSession(session)
+#ifdef ACTORSLIB_COLLECT_EXEC_STATS
+ , SendTime(0)
+#endif
+ , Buffer(std::move(buffer))
+ , RewriteRecipient(Recipient)
+ , RewriteType(Type)
+ {
+ }
+
+ TIntrusivePtr<TEventSerializedData> GetChainBuffer();
+ TIntrusivePtr<TEventSerializedData> ReleaseChainBuffer();
+
+ ui32 GetSize() const {
+ if (Buffer) {
+ return Buffer->GetSize();
+ } else if (Event) {
+ return Event->CalculateSerializedSize();
+ } else {
+ return 0;
+ }
+ }
+
+ bool HasBuffer() const {
+ return bool(Buffer);
+ }
+
+ bool HasEvent() const {
+ return bool(Event);
+ }
+
+ IEventBase* GetBase() {
+ if (!Event) {
+ if (!Buffer)
+ return nullptr;
+ else
+ ythrow TWithBackTrace<yexception>() << "don't know how to load the event from buffer";
+ }
+
+ return Event.Get();
+ }
+
+ TAutoPtr<IEventBase> ReleaseBase() {
+ TAutoPtr<IEventBase> x = GetBase();
+ Y_UNUSED(Event.Release());
+ Buffer.Reset();
+ return x;
+ }
+
+ TAutoPtr<IEventHandle> Forward(const TActorId& dest) {
+ if (Event)
+ return new IEventHandle(dest, Sender, Event.Release(), Flags, Cookie, nullptr, std::move(TraceId));
+ else
+ return new IEventHandle(Type, Flags, dest, Sender, Buffer, Cookie, nullptr, std::move(TraceId));
+ }
+
+ TAutoPtr<IEventHandle> ForwardOnNondelivery(ui32 reason, bool unsure = false);
+ };
+
+ template <typename TEventType>
+ class TEventHandle: public IEventHandle {
+ TEventHandle(); // we never made instance of TEventHandle
+ public:
+ TEventType* Get() {
+ return IEventHandle::Get<TEventType>();
+ }
+
+ TAutoPtr<TEventType> Release() {
+ return IEventHandle::Release<TEventType>();
+ }
+ };
+
+ static_assert(sizeof(TEventHandle<IEventBase>) == sizeof(IEventHandle), "expect sizeof(TEventHandle<IEventBase>) == sizeof(IEventHandle)");
+
+ template <typename TEventType, ui32 EventType0>
+ class TEventBase: public IEventBase {
+ public:
+ static constexpr ui32 EventType = EventType0;
+ ui32 Type() const override {
+ return EventType0;
+ }
+ // still abstract
+
+ typedef TEventHandle<TEventType> THandle;
+ typedef TAutoPtr<THandle> TPtr;
+ };
+
+#define DEFINE_SIMPLE_LOCAL_EVENT(eventType, header) \
+ TString ToStringHeader() const override { \
+ return TString(header); \
+ } \
+ bool SerializeToArcadiaStream(NActors::TChunkSerializer*) const override { \
+ Y_FAIL("Local event " #eventType " is not serializable"); \
+ } \
+ static IEventBase* Load(NActors::TEventSerializedData*) { \
+ Y_FAIL("Local event " #eventType " has no load method"); \
+ } \
+ bool IsSerializable() const override { \
+ return false; \
+ }
+
+#define DEFINE_SIMPLE_NONLOCAL_EVENT(eventType, header) \
+ TString ToStringHeader() const override { \
+ return TString(header); \
+ } \
+ bool SerializeToArcadiaStream(NActors::TChunkSerializer*) const override { \
+ return true; \
+ } \
+ static IEventBase* Load(NActors::TEventSerializedData*) { \
+ return new eventType(); \
+ } \
+ bool IsSerializable() const override { \
+ return true; \
+ }
+}
diff --git a/library/cpp/actors/core/event_load.h b/library/cpp/actors/core/event_load.h
new file mode 100644
index 0000000000..0dab1dd374
--- /dev/null
+++ b/library/cpp/actors/core/event_load.h
@@ -0,0 +1,112 @@
+#pragma once
+
+#include <util/stream/walk.h>
+#include <util/system/types.h>
+#include <util/generic/string.h>
+#include <library/cpp/actors/util/rope.h>
+#include <library/cpp/actors/wilson/wilson_trace.h>
+
+namespace NActors {
+ class IEventHandle;
+
+ struct TConstIoVec {
+ const void* Data;
+ size_t Size;
+ };
+
+ struct TIoVec {
+ void* Data;
+ size_t Size;
+ };
+
+ class TEventSerializedData
+ : public TThrRefBase
+ {
+ TRope Rope;
+ bool ExtendedFormat = false;
+
+ public:
+ TEventSerializedData() = default;
+
+ TEventSerializedData(TRope&& rope, bool extendedFormat)
+ : Rope(std::move(rope))
+ , ExtendedFormat(extendedFormat)
+ {}
+
+ TEventSerializedData(const TEventSerializedData& original, TString extraBuffer)
+ : Rope(original.Rope)
+ , ExtendedFormat(original.ExtendedFormat)
+ {
+ Append(std::move(extraBuffer));
+ }
+
+ TEventSerializedData(TString buffer, bool extendedFormat)
+ : ExtendedFormat(extendedFormat)
+ {
+ Append(std::move(buffer));
+ }
+
+ void SetExtendedFormat() {
+ ExtendedFormat = true;
+ }
+
+ bool IsExtendedFormat() const {
+ return ExtendedFormat;
+ }
+
+ TRope::TConstIterator GetBeginIter() const {
+ return Rope.Begin();
+ }
+
+ size_t GetSize() const {
+ return Rope.GetSize();
+ }
+
+ TString GetString() const {
+ TString result;
+ result.reserve(GetSize());
+ for (auto it = Rope.Begin(); it.Valid(); it.AdvanceToNextContiguousBlock()) {
+ result.append(it.ContiguousData(), it.ContiguousSize());
+ }
+ return result;
+ }
+
+ TRope EraseBack(size_t count) {
+ Y_VERIFY(count <= Rope.GetSize());
+ TRope::TIterator iter = Rope.End();
+ iter -= count;
+ return Rope.Extract(iter, Rope.End());
+ }
+
+ void Append(TRope&& from) {
+ Rope.Insert(Rope.End(), std::move(from));
+ }
+
+ void Append(TString buffer) {
+ if (buffer) {
+ Rope.Insert(Rope.End(), TRope(std::move(buffer)));
+ }
+ }
+ };
+}
+
+class TChainBufWalk : public IWalkInput {
+ TIntrusivePtr<NActors::TEventSerializedData> Buffer;
+ TRope::TConstIterator Iter;
+
+public:
+ TChainBufWalk(TIntrusivePtr<NActors::TEventSerializedData> buffer)
+ : Buffer(std::move(buffer))
+ , Iter(Buffer->GetBeginIter())
+ {}
+
+private:
+ size_t DoUnboundedNext(const void **ptr) override {
+ const size_t size = Iter.ContiguousSize();
+ *ptr = Iter.ContiguousData();
+ if (Iter.Valid()) {
+ Iter.AdvanceToNextContiguousBlock();
+ }
+ return size;
+ }
+};
diff --git a/library/cpp/actors/core/event_local.h b/library/cpp/actors/core/event_local.h
new file mode 100644
index 0000000000..2845aa94dd
--- /dev/null
+++ b/library/cpp/actors/core/event_local.h
@@ -0,0 +1,74 @@
+#pragma once
+
+#include "event.h"
+#include "scheduler_cookie.h"
+#include "event_load.h"
+#include <util/system/type_name.h>
+
+namespace NActors {
+ template <typename TEv, ui32 TEventType>
+ class TEventLocal: public TEventBase<TEv, TEventType> {
+ public:
+ TString ToStringHeader() const override {
+ return TypeName<TEv>();
+ }
+
+ bool SerializeToArcadiaStream(TChunkSerializer* /*serializer*/) const override {
+ Y_FAIL("Serialization of local event %s type %" PRIu32, TypeName<TEv>().data(), TEventType);
+ }
+
+ bool IsSerializable() const override {
+ return false;
+ }
+
+ static IEventBase* Load(TEventSerializedData*) {
+ Y_FAIL("Loading of local event %s type %" PRIu32, TypeName<TEv>().data(), TEventType);
+ }
+ };
+
+ template <typename TEv, ui32 TEventType>
+ class TEventScheduler: public TEventLocal<TEv, TEventType> {
+ public:
+ TSchedulerCookieHolder Cookie;
+
+ TEventScheduler(ISchedulerCookie* cookie)
+ : Cookie(cookie)
+ {
+ }
+ };
+
+ template <ui32 TEventType>
+ class TEventSchedulerEv: public TEventScheduler<TEventSchedulerEv<TEventType>, TEventType> {
+ public:
+ TEventSchedulerEv(ISchedulerCookie* cookie)
+ : TEventScheduler<TEventSchedulerEv<TEventType>, TEventType>(cookie)
+ {
+ }
+ };
+
+ template <typename TEv, ui32 TEventType>
+ class TEventSimple: public TEventBase<TEv, TEventType> {
+ public:
+ TString ToStringHeader() const override {
+ static TString header(TypeName<TEv>());
+ return header;
+ }
+
+ bool SerializeToArcadiaStream(TChunkSerializer* /*serializer*/) const override {
+ static_assert(sizeof(TEv) == sizeof(TEventSimple<TEv, TEventType>), "Descendant should be an empty class");
+ return true;
+ }
+
+ bool IsSerializable() const override {
+ return true;
+ }
+
+ static IEventBase* Load(NActors::TEventSerializedData*) {
+ return new TEv();
+ }
+
+ static IEventBase* Load(const TString&) {
+ return new TEv();
+ }
+ };
+}
diff --git a/library/cpp/actors/core/event_pb.cpp b/library/cpp/actors/core/event_pb.cpp
new file mode 100644
index 0000000000..018ff9ac34
--- /dev/null
+++ b/library/cpp/actors/core/event_pb.cpp
@@ -0,0 +1,223 @@
+#include "event_pb.h"
+
+namespace NActors {
+ bool TRopeStream::Next(const void** data, int* size) {
+ *data = Iter.ContiguousData();
+ *size = Iter.ContiguousSize();
+ if (size_t(*size + TotalByteCount) > Size) {
+ *size = Size - TotalByteCount;
+ Iter += *size;
+ } else if (Iter.Valid()) {
+ Iter.AdvanceToNextContiguousBlock();
+ }
+ TotalByteCount += *size;
+ return *size != 0;
+ }
+
+ void TRopeStream::BackUp(int count) {
+ Y_VERIFY(count <= TotalByteCount);
+ Iter -= count;
+ TotalByteCount -= count;
+ }
+
+ bool TRopeStream::Skip(int count) {
+ if (static_cast<size_t>(TotalByteCount + count) > Size) {
+ count = Size - TotalByteCount;
+ }
+ Iter += count;
+ TotalByteCount += count;
+ return static_cast<size_t>(TotalByteCount) != Size;
+ }
+
+ TCoroutineChunkSerializer::TCoroutineChunkSerializer()
+ : TotalSerializedDataSize(0)
+ , Stack(64 * 1024)
+ , SelfClosure{this, TArrayRef(Stack.Begin(), Stack.End())}
+ , InnerContext(SelfClosure)
+ {}
+
+ TCoroutineChunkSerializer::~TCoroutineChunkSerializer() {
+ CancelFlag = true;
+ Resume();
+ Y_VERIFY(Finished);
+ }
+
+ bool TCoroutineChunkSerializer::AllowsAliasing() const {
+ return true;
+ }
+
+ bool TCoroutineChunkSerializer::Produce(const void *data, size_t size) {
+ Y_VERIFY(size <= SizeRemain);
+ SizeRemain -= size;
+ TotalSerializedDataSize += size;
+
+ if (NumChunks) {
+ auto& last = Chunks[NumChunks - 1];
+ if (last.first + last.second == data) {
+ last.second += size; // just extend the last buffer
+ return true;
+ }
+ }
+
+ if (NumChunks == MaxChunks) {
+ InnerContext.SwitchTo(BufFeedContext);
+ if (CancelFlag || AbortFlag) {
+ return false;
+ }
+ }
+
+ Y_VERIFY(NumChunks < MaxChunks);
+ Chunks[NumChunks++] = {static_cast<const char*>(data), size};
+ return true;
+ }
+
+ bool TCoroutineChunkSerializer::WriteAliasedRaw(const void* data, int size) {
+ Y_VERIFY(size >= 0);
+ while (size) {
+ if (CancelFlag || AbortFlag) {
+ return false;
+ } else if (const size_t bytesToAppend = Min<size_t>(size, SizeRemain)) {
+ if (!Produce(data, bytesToAppend)) {
+ return false;
+ }
+ data = static_cast<const char*>(data) + bytesToAppend;
+ size -= bytesToAppend;
+ } else {
+ InnerContext.SwitchTo(BufFeedContext);
+ }
+ }
+ return true;
+ }
+
+ bool TCoroutineChunkSerializer::Next(void** data, int* size) {
+ if (CancelFlag || AbortFlag) {
+ return false;
+ }
+ if (!SizeRemain) {
+ InnerContext.SwitchTo(BufFeedContext);
+ if (CancelFlag || AbortFlag) {
+ return false;
+ }
+ }
+ Y_VERIFY(SizeRemain);
+ *data = BufferPtr;
+ *size = SizeRemain;
+ BufferPtr += SizeRemain;
+ return Produce(*data, *size);
+ }
+
+ void TCoroutineChunkSerializer::BackUp(int count) {
+ if (!count) {
+ return;
+ }
+ Y_VERIFY(count > 0);
+ Y_VERIFY(NumChunks);
+ TChunk& buf = Chunks[NumChunks - 1];
+ Y_VERIFY((size_t)count <= buf.second);
+ Y_VERIFY(buf.first + buf.second == BufferPtr);
+ buf.second -= count;
+ if (!buf.second) {
+ --NumChunks;
+ }
+ BufferPtr -= count;
+ SizeRemain += count;
+ TotalSerializedDataSize -= count;
+ }
+
+ void TCoroutineChunkSerializer::Resume() {
+ TContMachineContext feedContext;
+ BufFeedContext = &feedContext;
+ feedContext.SwitchTo(&InnerContext);
+ BufFeedContext = nullptr;
+ }
+
+ bool TCoroutineChunkSerializer::WriteRope(const TRope *rope) {
+ for (auto iter = rope->Begin(); iter.Valid(); iter.AdvanceToNextContiguousBlock()) {
+ if (!WriteAliasedRaw(iter.ContiguousData(), iter.ContiguousSize())) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ bool TCoroutineChunkSerializer::WriteString(const TString *s) {
+ return WriteAliasedRaw(s->data(), s->length());
+ }
+
+ std::pair<TCoroutineChunkSerializer::TChunk*, TCoroutineChunkSerializer::TChunk*> TCoroutineChunkSerializer::FeedBuf(void* data, size_t size) {
+ // fill in base params
+ BufferPtr = static_cast<char*>(data);
+ SizeRemain = size;
+
+ // transfer control to the coroutine
+ Y_VERIFY(Event);
+ NumChunks = 0;
+ Resume();
+
+ return {Chunks, Chunks + NumChunks};
+ }
+
+ void TCoroutineChunkSerializer::SetSerializingEvent(const IEventBase *event) {
+ Y_VERIFY(Event == nullptr);
+ Event = event;
+ TotalSerializedDataSize = 0;
+ AbortFlag = false;
+ }
+
+ void TCoroutineChunkSerializer::Abort() {
+ Y_VERIFY(Event);
+ AbortFlag = true;
+ Resume();
+ }
+
+ void TCoroutineChunkSerializer::DoRun() {
+ while (!CancelFlag) {
+ Y_VERIFY(Event);
+ SerializationSuccess = Event->SerializeToArcadiaStream(this);
+ Event = nullptr;
+ if (!CancelFlag) { // cancel flag may have been received during serialization
+ InnerContext.SwitchTo(BufFeedContext);
+ }
+ }
+ Finished = true;
+ InnerContext.SwitchTo(BufFeedContext);
+ }
+
+ bool TAllocChunkSerializer::Next(void** pdata, int* psize) {
+ if (Backup) {
+ // we have some data in backup rope -- move the first chunk from the backup rope to the buffer and return
+ // pointer to the buffer; it is safe to remove 'const' here as we uniquely own this buffer
+ TRope::TIterator iter = Backup.Begin();
+ *pdata = const_cast<char*>(iter.ContiguousData());
+ *psize = iter.ContiguousSize();
+ iter.AdvanceToNextContiguousBlock();
+ Buffers->Append(Backup.Extract(Backup.Begin(), iter));
+ } else {
+ // no backup buffer, so we have to create new one
+ auto item = TRopeAlignedBuffer::Allocate(4096);
+ *pdata = item->GetBuffer();
+ *psize = item->GetCapacity();
+ Buffers->Append(TRope(std::move(item)));
+ }
+ return true;
+ }
+
+ void TAllocChunkSerializer::BackUp(int count) {
+ Backup.Insert(Backup.Begin(), Buffers->EraseBack(count));
+ }
+
+ bool TAllocChunkSerializer::WriteAliasedRaw(const void*, int) {
+ Y_VERIFY(false);
+ return false;
+ }
+
+ bool TAllocChunkSerializer::WriteRope(const TRope *rope) {
+ Buffers->Append(TRope(*rope));
+ return true;
+ }
+
+ bool TAllocChunkSerializer::WriteString(const TString *s) {
+ Buffers->Append(*s);
+ return true;
+ }
+}
diff --git a/library/cpp/actors/core/event_pb.h b/library/cpp/actors/core/event_pb.h
new file mode 100644
index 0000000000..d7546b901a
--- /dev/null
+++ b/library/cpp/actors/core/event_pb.h
@@ -0,0 +1,500 @@
+#pragma once
+
+#include "event.h"
+#include "event_load.h"
+
+#include <google/protobuf/io/zero_copy_stream.h>
+#include <google/protobuf/arena.h>
+#include <library/cpp/actors/protos/actors.pb.h>
+#include <util/generic/deque.h>
+#include <util/system/context.h>
+#include <util/system/filemap.h>
+#include <array>
+
+namespace NActors {
+
+ class TRopeStream : public NProtoBuf::io::ZeroCopyInputStream {
+ TRope::TConstIterator Iter;
+ const size_t Size;
+
+ public:
+ TRopeStream(TRope::TConstIterator iter, size_t size)
+ : Iter(iter)
+ , Size(size)
+ {}
+
+ bool Next(const void** data, int* size) override;
+ void BackUp(int count) override;
+ bool Skip(int count) override;
+ int64_t ByteCount() const override {
+ return TotalByteCount;
+ }
+
+ private:
+ int64_t TotalByteCount = 0;
+ };
+
+ class TChunkSerializer : public NProtoBuf::io::ZeroCopyOutputStream {
+ public:
+ TChunkSerializer() = default;
+ virtual ~TChunkSerializer() = default;
+
+ virtual bool WriteRope(const TRope *rope) = 0;
+ virtual bool WriteString(const TString *s) = 0;
+ };
+
+ class TAllocChunkSerializer final : public TChunkSerializer {
+ public:
+ bool Next(void** data, int* size) override;
+ void BackUp(int count) override;
+ int64_t ByteCount() const override {
+ return Buffers->GetSize();
+ }
+ bool WriteAliasedRaw(const void* data, int size) override;
+
+ // WARNING: these methods require owner to retain ownership and immutability of passed objects
+ bool WriteRope(const TRope *rope) override;
+ bool WriteString(const TString *s) override;
+
+ inline TIntrusivePtr<TEventSerializedData> Release(bool extendedFormat) {
+ if (extendedFormat) {
+ Buffers->SetExtendedFormat();
+ }
+ return std::move(Buffers);
+ }
+
+ protected:
+ TIntrusivePtr<TEventSerializedData> Buffers = new TEventSerializedData;
+ TRope Backup;
+ };
+
+ class TCoroutineChunkSerializer final : public TChunkSerializer, protected ITrampoLine {
+ public:
+ using TChunk = std::pair<const char*, size_t>;
+
+ TCoroutineChunkSerializer();
+ ~TCoroutineChunkSerializer();
+
+ void SetSerializingEvent(const IEventBase *event);
+ void Abort();
+ std::pair<TChunk*, TChunk*> FeedBuf(void* data, size_t size);
+ bool IsComplete() const {
+ return !Event;
+ }
+ bool IsSuccessfull() const {
+ return SerializationSuccess;
+ }
+ const IEventBase *GetCurrentEvent() const {
+ return Event;
+ }
+
+ bool Next(void** data, int* size) override;
+ void BackUp(int count) override;
+ int64_t ByteCount() const override {
+ return TotalSerializedDataSize;
+ }
+ bool WriteAliasedRaw(const void* data, int size) override;
+ bool AllowsAliasing() const override;
+
+ bool WriteRope(const TRope *rope) override;
+ bool WriteString(const TString *s) override;
+
+ protected:
+ void DoRun() override;
+ void Resume();
+ bool Produce(const void *data, size_t size);
+
+ i64 TotalSerializedDataSize;
+ TMappedAllocation Stack;
+ TContClosure SelfClosure;
+ TContMachineContext InnerContext;
+ TContMachineContext *BufFeedContext = nullptr;
+ char *BufferPtr;
+ size_t SizeRemain;
+ static constexpr size_t MaxChunks = 16;
+ TChunk Chunks[MaxChunks];
+ size_t NumChunks = 0;
+ const IEventBase *Event = nullptr;
+ bool CancelFlag = false;
+ bool AbortFlag;
+ bool SerializationSuccess;
+ bool Finished = false;
+ };
+
+#ifdef ACTORLIB_HUGE_PB_SIZE
+ static const size_t EventMaxByteSize = 140 << 20; // (140MB)
+#else
+ static const size_t EventMaxByteSize = 67108000;
+#endif
+
+ template <typename TEv, typename TRecord /*protobuf record*/, ui32 TEventType, typename TRecHolder>
+ class TEventPBBase: public TEventBase<TEv, TEventType> , public TRecHolder {
+ // a vector of data buffers referenced by record; if filled, then extended serialization mechanism applies
+ TVector<TRope> Payload;
+
+ public:
+ using TRecHolder::Record;
+
+ public:
+ using ProtoRecordType = TRecord;
+
+ TEventPBBase() = default;
+
+ explicit TEventPBBase(const TRecord& rec)
+ {
+ Record = rec;
+ }
+
+ explicit TEventPBBase(TRecord&& rec)
+ {
+ Record = std::move(rec);
+ }
+
+ TString ToStringHeader() const override {
+ return Record.GetTypeName();
+ }
+
+ TString ToString() const override {
+ return Record.ShortDebugString();
+ }
+
+ bool IsSerializable() const override {
+ return true;
+ }
+
+ bool IsExtendedFormat() const override {
+ return static_cast<bool>(Payload);
+ }
+
+ bool SerializeToArcadiaStream(TChunkSerializer* chunker) const override {
+ // serialize payload first
+ if (Payload) {
+ void *data;
+ int size = 0;
+ auto append = [&](const char *p, size_t len) {
+ while (len) {
+ if (size) {
+ const size_t numBytesToCopy = std::min<size_t>(size, len);
+ memcpy(data, p, numBytesToCopy);
+ data = static_cast<char*>(data) + numBytesToCopy;
+ size -= numBytesToCopy;
+ p += numBytesToCopy;
+ len -= numBytesToCopy;
+ } else if (!chunker->Next(&data, &size)) {
+ return false;
+ }
+ }
+ return true;
+ };
+ auto appendNumber = [&](size_t number) {
+ char buf[MaxNumberBytes];
+ return append(buf, SerializeNumber(number, buf));
+ };
+ char marker = PayloadMarker;
+ append(&marker, 1);
+ if (!appendNumber(Payload.size())) {
+ return false;
+ }
+ for (const TRope& rope : Payload) {
+ if (!appendNumber(rope.GetSize())) {
+ return false;
+ }
+ if (rope) {
+ if (size) {
+ chunker->BackUp(std::exchange(size, 0));
+ }
+ if (!chunker->WriteRope(&rope)) {
+ return false;
+ }
+ }
+ }
+ if (size) {
+ chunker->BackUp(size);
+ }
+ }
+
+ return Record.SerializeToZeroCopyStream(chunker);
+ }
+
+ ui32 CalculateSerializedSize() const override {
+ ssize_t result = Record.ByteSize();
+ if (result >= 0 && Payload) {
+ ++result; // marker
+ char buf[MaxNumberBytes];
+ result += SerializeNumber(Payload.size(), buf);
+ for (const TRope& rope : Payload) {
+ result += SerializeNumber(rope.GetSize(), buf);
+ result += rope.GetSize();
+ }
+ }
+ return result;
+ }
+
+ static IEventBase* Load(TIntrusivePtr<TEventSerializedData> input) {
+ THolder<TEventPBBase> ev(new TEv());
+ if (!input->GetSize()) {
+ Y_PROTOBUF_SUPPRESS_NODISCARD ev->Record.ParseFromString(TString());
+ } else {
+ TRope::TConstIterator iter = input->GetBeginIter();
+ ui64 size = input->GetSize();
+
+ if (input->IsExtendedFormat()) {
+ // check marker
+ if (!iter.Valid() || *iter.ContiguousData() != PayloadMarker) {
+ Y_FAIL("invalid event");
+ }
+ // skip marker
+ iter += 1;
+ --size;
+ // parse number of payload ropes
+ size_t numRopes = DeserializeNumber(iter, size);
+ if (numRopes == Max<size_t>()) {
+ Y_FAIL("invalid event");
+ }
+ while (numRopes--) {
+ // parse length of the rope
+ const size_t len = DeserializeNumber(iter, size);
+ if (len == Max<size_t>() || size < len) {
+ Y_FAIL("invalid event len# %zu size# %" PRIu64, len, size);
+ }
+ // extract the rope
+ TRope::TConstIterator begin = iter;
+ iter += len;
+ size -= len;
+ ev->Payload.emplace_back(begin, iter);
+ }
+ }
+
+ // parse the protobuf
+ TRopeStream stream(iter, size);
+ if (!ev->Record.ParseFromZeroCopyStream(&stream)) {
+ Y_FAIL("Failed to parse protobuf event type %" PRIu32 " class %s", TEventType, TypeName(ev->Record).data());
+ }
+ }
+ ev->CachedByteSize = input->GetSize();
+ return ev.Release();
+ }
+
+ size_t GetCachedByteSize() const {
+ if (CachedByteSize == 0) {
+ CachedByteSize = CalculateSerializedSize();
+ }
+ return CachedByteSize;
+ }
+
+ ui32 CalculateSerializedSizeCached() const override {
+ return GetCachedByteSize();
+ }
+
+ void InvalidateCachedByteSize() {
+ CachedByteSize = 0;
+ }
+
+ public:
+ void ReservePayload(size_t size) {
+ Payload.reserve(size);
+ }
+
+ ui32 AddPayload(TRope&& rope) {
+ const ui32 id = Payload.size();
+ Payload.push_back(std::move(rope));
+ InvalidateCachedByteSize();
+ return id;
+ }
+
+ const TRope& GetPayload(ui32 id) const {
+ Y_VERIFY(id < Payload.size());
+ return Payload[id];
+ }
+
+ ui32 GetPayloadCount() const {
+ return Payload.size();
+ }
+
+ void StripPayload() {
+ Payload.clear();
+ }
+
+ protected:
+ mutable size_t CachedByteSize = 0;
+
+ static constexpr char PayloadMarker = 0x07;
+ static constexpr size_t MaxNumberBytes = (sizeof(size_t) * CHAR_BIT + 6) / 7;
+
+ static size_t SerializeNumber(size_t num, char *buffer) {
+ char *begin = buffer;
+ do {
+ *buffer++ = (num & 0x7F) | (num >= 128 ? 0x80 : 0x00);
+ num >>= 7;
+ } while (num);
+ return buffer - begin;
+ }
+
+ static size_t DeserializeNumber(const char **ptr, const char *end) {
+ const char *p = *ptr;
+ size_t res = 0;
+ size_t offset = 0;
+ for (;;) {
+ if (p == end) {
+ return Max<size_t>();
+ }
+ const char byte = *p++;
+ res |= (static_cast<size_t>(byte) & 0x7F) << offset;
+ offset += 7;
+ if (!(byte & 0x80)) {
+ break;
+ }
+ }
+ *ptr = p;
+ return res;
+ }
+
+ static size_t DeserializeNumber(TRope::TConstIterator& iter, ui64& size) {
+ size_t res = 0;
+ size_t offset = 0;
+ for (;;) {
+ if (!iter.Valid()) {
+ return Max<size_t>();
+ }
+ const char byte = *iter.ContiguousData();
+ iter += 1;
+ --size;
+ res |= (static_cast<size_t>(byte) & 0x7F) << offset;
+ offset += 7;
+ if (!(byte & 0x80)) {
+ break;
+ }
+ }
+ return res;
+ }
+ };
+
+ // Protobuf record not using arena
+ template <typename TRecord>
+ struct TRecordHolder {
+ TRecord Record;
+ };
+
+ // Protobuf arena and a record allocated on it
+ template <typename TRecord, size_t InitialBlockSize, size_t MaxBlockSize>
+ struct TArenaRecordHolder {
+ google::protobuf::Arena PbArena;
+ TRecord& Record;
+
+ static const google::protobuf::ArenaOptions GetArenaOptions() {
+ google::protobuf::ArenaOptions opts;
+ opts.initial_block_size = InitialBlockSize;
+ opts.max_block_size = MaxBlockSize;
+ return opts;
+ }
+
+ TArenaRecordHolder()
+ : PbArena(GetArenaOptions())
+ , Record(*google::protobuf::Arena::CreateMessage<TRecord>(&PbArena))
+ {}
+ };
+
+ template <typename TEv, typename TRecord, ui32 TEventType>
+ class TEventPB : public TEventPBBase<TEv, TRecord, TEventType, TRecordHolder<TRecord> > {
+ typedef TEventPBBase<TEv, TRecord, TEventType, TRecordHolder<TRecord> > TPbBase;
+ // NOTE: No extra fields allowed: TEventPB must be a "template typedef"
+ public:
+ using TPbBase::TPbBase;
+ };
+
+ template <typename TEv, typename TRecord, ui32 TEventType, size_t InitialBlockSize = 512, size_t MaxBlockSize = 16*1024>
+ using TEventPBWithArena = TEventPBBase<TEv, TRecord, TEventType, TArenaRecordHolder<TRecord, InitialBlockSize, MaxBlockSize> >;
+
+ template <typename TEv, typename TRecord, ui32 TEventType>
+ class TEventShortDebugPB: public TEventPB<TEv, TRecord, TEventType> {
+ public:
+ using TBase = TEventPB<TEv, TRecord, TEventType>;
+ TEventShortDebugPB() = default;
+ explicit TEventShortDebugPB(const TRecord& rec)
+ : TBase(rec)
+ {
+ }
+ explicit TEventShortDebugPB(TRecord&& rec)
+ : TBase(std::move(rec))
+ {
+ }
+ TString ToString() const override {
+ return TypeName<TEv>() + " { " + TBase::Record.ShortDebugString() + " }";
+ }
+ };
+
+ template <typename TEv, typename TRecord, ui32 TEventType>
+ class TEventPreSerializedPB: public TEventPB<TEv, TRecord, TEventType> {
+ protected:
+ using TBase = TEventPB<TEv, TRecord, TEventType>;
+ using TSelf = TEventPreSerializedPB<TEv, TRecord, TEventType>;
+ using TBase::Record;
+
+ public:
+ TString PreSerializedData; // already serialized PB data (using message::SerializeToString)
+
+ TEventPreSerializedPB() = default;
+
+ explicit TEventPreSerializedPB(const TRecord& rec)
+ : TBase(rec)
+ {
+ }
+
+ explicit TEventPreSerializedPB(TRecord&& rec)
+ : TBase(std::move(rec))
+ {
+ }
+
+ // when remote event received locally this method will merge preserialized data
+ const TRecord& GetRecord() {
+ TRecord& base(TBase::Record);
+ if (!PreSerializedData.empty()) {
+ TRecord copy;
+ Y_PROTOBUF_SUPPRESS_NODISCARD copy.ParseFromString(PreSerializedData);
+ copy.MergeFrom(base);
+ base.Swap(&copy);
+ PreSerializedData.clear();
+ }
+ return TBase::Record;
+ }
+
+ const TRecord& GetRecord() const {
+ return const_cast<TSelf*>(this)->GetRecord();
+ }
+
+ TRecord* MutableRecord() {
+ GetRecord(); // Make sure PreSerializedData is parsed
+ return &(TBase::Record);
+ }
+
+ TString ToString() const override {
+ return GetRecord().ShortDebugString();
+ }
+
+ bool SerializeToArcadiaStream(TChunkSerializer* chunker) const override {
+ return chunker->WriteString(&PreSerializedData) && TBase::SerializeToArcadiaStream(chunker);
+ }
+
+ ui32 CalculateSerializedSize() const override {
+ return PreSerializedData.size() + TBase::CalculateSerializedSize();
+ }
+
+ size_t GetCachedByteSize() const {
+ return PreSerializedData.size() + TBase::GetCachedByteSize();
+ }
+
+ ui32 CalculateSerializedSizeCached() const override {
+ return GetCachedByteSize();
+ }
+ };
+
+ inline TActorId ActorIdFromProto(const NActorsProto::TActorId& actorId) {
+ return TActorId(actorId.GetRawX1(), actorId.GetRawX2());
+ }
+
+ inline void ActorIdToProto(const TActorId& src, NActorsProto::TActorId* dest) {
+ Y_VERIFY_DEBUG(dest);
+ dest->SetRawX1(src.RawX1());
+ dest->SetRawX2(src.RawX2());
+ }
+}
diff --git a/library/cpp/actors/core/event_pb_payload_ut.cpp b/library/cpp/actors/core/event_pb_payload_ut.cpp
new file mode 100644
index 0000000000..eab007bc15
--- /dev/null
+++ b/library/cpp/actors/core/event_pb_payload_ut.cpp
@@ -0,0 +1,154 @@
+#include "event_pb.h"
+#include "events.h"
+
+#include <library/cpp/testing/unittest/registar.h>
+#include <library/cpp/actors/protos/unittests.pb.h>
+
+using namespace NActors;
+
+enum {
+ EvMessageWithPayload = EventSpaceBegin(TEvents::ES_PRIVATE),
+ EvArenaMessage,
+ EvArenaMessageBig,
+ EvMessageWithPayloadPreSerialized
+};
+
+struct TEvMessageWithPayload : TEventPB<TEvMessageWithPayload, TMessageWithPayload, EvMessageWithPayload> {
+ TEvMessageWithPayload() = default;
+ explicit TEvMessageWithPayload(const TMessageWithPayload& p)
+ : TEventPB<TEvMessageWithPayload, TMessageWithPayload, EvMessageWithPayload>(p)
+ {}
+};
+
+struct TEvMessageWithPayloadPreSerialized : TEventPreSerializedPB<TEvMessageWithPayloadPreSerialized, TMessageWithPayload, EvMessageWithPayloadPreSerialized> {
+};
+
+
+TRope MakeStringRope(const TString& message) {
+ return message ? TRope(message) : TRope();
+}
+
+TString MakeString(size_t len) {
+ TString res;
+ for (size_t i = 0; i < len; ++i) {
+ res += RandomNumber<char>();
+ }
+ return res;
+}
+
+Y_UNIT_TEST_SUITE(TEventProtoWithPayload) {
+
+ template <class TEventFrom, class TEventTo>
+ void TestSerializeDeserialize(size_t size1, size_t size2) {
+ static_assert(TEventFrom::EventType == TEventTo::EventType, "Must be same event type");
+
+ TEventFrom msg;
+ msg.Record.SetMeta("hello, world!");
+ msg.Record.AddPayloadId(msg.AddPayload(MakeStringRope(MakeString(size1))));
+ msg.Record.AddPayloadId(msg.AddPayload(MakeStringRope(MakeString(size2))));
+ msg.Record.AddSomeData(MakeString((size1 + size2) % 50 + 11));
+
+ auto serializer = MakeHolder<TAllocChunkSerializer>();
+ msg.SerializeToArcadiaStream(serializer.Get());
+ auto buffers = serializer->Release(msg.IsExtendedFormat());
+ UNIT_ASSERT_VALUES_EQUAL(buffers->GetSize(), msg.CalculateSerializedSize());
+ TString ser = buffers->GetString();
+
+ TString chunkerRes;
+ TCoroutineChunkSerializer chunker;
+ chunker.SetSerializingEvent(&msg);
+ while (!chunker.IsComplete()) {
+ char buffer[4096];
+ auto range = chunker.FeedBuf(buffer, sizeof(buffer));
+ for (auto p = range.first; p != range.second; ++p) {
+ chunkerRes += TString(p->first, p->second);
+ }
+ }
+ UNIT_ASSERT_VALUES_EQUAL(chunkerRes, ser);
+
+ THolder<IEventBase> ev2 = THolder(TEventTo::Load(buffers));
+ TEventTo& msg2 = static_cast<TEventTo&>(*ev2);
+ UNIT_ASSERT_VALUES_EQUAL(msg2.Record.GetMeta(), msg.Record.GetMeta());
+ UNIT_ASSERT_EQUAL(msg2.GetPayload(msg2.Record.GetPayloadId(0)), msg.GetPayload(msg.Record.GetPayloadId(0)));
+ UNIT_ASSERT_EQUAL(msg2.GetPayload(msg2.Record.GetPayloadId(1)), msg.GetPayload(msg.Record.GetPayloadId(1)));
+ }
+
+ template <class TEvent>
+ void TestAllSizes(size_t step1 = 100, size_t step2 = 111) {
+ for (size_t size1 = 0; size1 < 10000; size1 += step1) {
+ for (size_t size2 = 0; size2 < 10000; size2 += step2) {
+ TestSerializeDeserialize<TEvent, TEvent>(size1, size2);
+ }
+ }
+ }
+
+#if (!defined(_tsan_enabled_))
+ Y_UNIT_TEST(SerializeDeserialize) {
+ TestAllSizes<TEvMessageWithPayload>();
+ }
+#endif
+
+
+ struct TEvArenaMessage : TEventPBWithArena<TEvArenaMessage, TMessageWithPayload, EvArenaMessage> {
+ };
+
+ Y_UNIT_TEST(SerializeDeserializeArena) {
+ TestAllSizes<TEvArenaMessage>(500, 111);
+ }
+
+
+ struct TEvArenaMessageBig : TEventPBWithArena<TEvArenaMessageBig, TMessageWithPayload, EvArenaMessageBig, 4000, 32000> {
+ };
+
+ Y_UNIT_TEST(SerializeDeserializeArenaBig) {
+ TestAllSizes<TEvArenaMessageBig>(111, 500);
+ }
+
+
+ // Compatible with TEvArenaMessage but doesn't use arenas
+ struct TEvArenaMessageWithoutArena : TEventPB<TEvArenaMessageWithoutArena, TMessageWithPayload, EvArenaMessage> {
+ };
+
+ Y_UNIT_TEST(Compatibility) {
+ TestSerializeDeserialize<TEvArenaMessage, TEvArenaMessageWithoutArena>(200, 14010);
+ TestSerializeDeserialize<TEvArenaMessageWithoutArena, TEvArenaMessage>(2000, 4010);
+ }
+
+ Y_UNIT_TEST(PreSerializedCompatibility) {
+ // ensure TEventPreSerializedPB and TEventPB are interchangable with no compatibility issues
+ TMessageWithPayload msg;
+ msg.SetMeta("hello, world!");
+ msg.AddPayloadId(123);
+ msg.AddPayloadId(999);
+ msg.AddSomeData("abc");
+ msg.AddSomeData("xyzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzzz");
+
+ TEvMessageWithPayloadPreSerialized e1;
+ Y_PROTOBUF_SUPPRESS_NODISCARD msg.SerializeToString(&e1.PreSerializedData);
+
+ auto serializer1 = MakeHolder<TAllocChunkSerializer>();
+ e1.SerializeToArcadiaStream(serializer1.Get());
+ auto buffers1 = serializer1->Release(e1.IsExtendedFormat());
+ UNIT_ASSERT_VALUES_EQUAL(buffers1->GetSize(), e1.CalculateSerializedSize());
+ TString ser1 = buffers1->GetString();
+
+ TEvMessageWithPayload e2(msg);
+ auto serializer2 = MakeHolder<TAllocChunkSerializer>();
+ e2.SerializeToArcadiaStream(serializer2.Get());
+ auto buffers2 = serializer2->Release(e2.IsExtendedFormat());
+ UNIT_ASSERT_VALUES_EQUAL(buffers2->GetSize(), e2.CalculateSerializedSize());
+ TString ser2 = buffers2->GetString();
+ UNIT_ASSERT_VALUES_EQUAL(ser1, ser2);
+
+ // deserialize
+ auto data = MakeIntrusive<TEventSerializedData>(ser1, false);
+ THolder<TEvMessageWithPayloadPreSerialized> parsedEvent(static_cast<TEvMessageWithPayloadPreSerialized*>(TEvMessageWithPayloadPreSerialized::Load(data)));
+ UNIT_ASSERT_VALUES_EQUAL(parsedEvent->PreSerializedData, ""); // this field is empty after deserialization
+ auto& record = parsedEvent->GetRecord();
+ UNIT_ASSERT_VALUES_EQUAL(record.GetMeta(), msg.GetMeta());
+ UNIT_ASSERT_VALUES_EQUAL(record.PayloadIdSize(), msg.PayloadIdSize());
+ UNIT_ASSERT_VALUES_EQUAL(record.PayloadIdSize(), 2);
+ UNIT_ASSERT_VALUES_EQUAL(record.GetPayloadId(0), msg.GetPayloadId(0));
+ UNIT_ASSERT_VALUES_EQUAL(record.GetPayloadId(1), msg.GetPayloadId(1));
+ }
+}
diff --git a/library/cpp/actors/core/event_pb_ut.cpp b/library/cpp/actors/core/event_pb_ut.cpp
new file mode 100644
index 0000000000..a16c3092b3
--- /dev/null
+++ b/library/cpp/actors/core/event_pb_ut.cpp
@@ -0,0 +1,71 @@
+#include "event_pb.h"
+
+#include <library/cpp/testing/unittest/registar.h>
+#include <library/cpp/actors/protos/unittests.pb.h>
+
+Y_UNIT_TEST_SUITE(TEventSerialization) {
+ struct TMockEvent: public NActors::IEventBase {
+ TBigMessage* msg;
+ bool
+ SerializeToArcadiaStream(NActors::TChunkSerializer* chunker) const override {
+ return msg->SerializeToZeroCopyStream(chunker);
+ }
+ bool IsSerializable() const override {
+ return true;
+ }
+ TString ToStringHeader() const override {
+ return TString();
+ }
+ virtual TString Serialize() const {
+ return TString();
+ }
+ ui32 Type() const override {
+ return 0;
+ };
+ };
+
+ Y_UNIT_TEST(Coroutine) {
+ TString strA(507, 'a');
+ TString strB(814, 'b');
+ TString strC(198, 'c');
+
+ TBigMessage bm;
+
+ TSimple* simple0 = bm.AddSimples();
+ simple0->SetStr1(strA);
+ simple0->SetStr2(strB);
+ simple0->SetNumber1(213431324);
+
+ TSimple* simple1 = bm.AddSimples();
+ simple1->SetStr1(strC);
+ simple1->SetStr2(strA);
+ simple1->SetNumber1(21039313);
+
+ bm.AddManyStr(strA);
+ bm.AddManyStr(strC);
+ bm.AddManyStr(strB);
+
+ bm.SetOneMoreStr(strB);
+ bm.SetYANumber(394143);
+
+ TString bmSerialized;
+ Y_PROTOBUF_SUPPRESS_NODISCARD bm.SerializeToString(&bmSerialized);
+ UNIT_ASSERT_UNEQUAL(bmSerialized.size(), 0);
+
+ NActors::TCoroutineChunkSerializer chunker;
+ for (int i = 0; i < 4; ++i) {
+ TMockEvent event;
+ event.msg = &bm;
+ chunker.SetSerializingEvent(&event);
+ char buf1[87];
+ TString bmChunkedSerialized;
+ while (!chunker.IsComplete()) {
+ auto range = chunker.FeedBuf(&buf1[0], sizeof(buf1));
+ for (auto p = range.first; p != range.second; ++p) {
+ bmChunkedSerialized.append(p->first, p->second);
+ }
+ }
+ UNIT_ASSERT_EQUAL(bmSerialized, bmChunkedSerialized);
+ }
+ }
+}
diff --git a/library/cpp/actors/core/events.h b/library/cpp/actors/core/events.h
new file mode 100644
index 0000000000..702cf50fad
--- /dev/null
+++ b/library/cpp/actors/core/events.h
@@ -0,0 +1,222 @@
+#pragma once
+
+#include "event.h"
+#include "event_pb.h"
+
+#include <library/cpp/actors/protos/actors.pb.h>
+#include <util/system/unaligned_mem.h>
+
+namespace NActors {
+ struct TEvents {
+ enum EEventSpace {
+ ES_HELLOWORLD = 0,
+ ES_SYSTEM = 1,
+ ES_INTERCONNECT = 2,
+ ES_INTERCONNECT_MSGBUS = 3,
+ ES_DNS = 4,
+ ES_SOCKET_POLLER = 5,
+ ES_LOGGER = 6,
+ ES_MON = 7,
+ ES_INTERCONNECT_TCP = 8,
+ ES_PROFILER = 9,
+ ES_YF = 10,
+ ES_HTTP = 11,
+
+ ES_USERSPACE = 4096,
+
+ ES_PRIVATE = (1 << 15) - 16,
+ ES_MAX = (1 << 15),
+ };
+
+#define EventSpaceBegin(eventSpace) (eventSpace << 16u)
+#define EventSpaceEnd(eventSpace) ((eventSpace << 16u) + (1u << 16u))
+
+ struct THelloWorld {
+ enum {
+ Start = EventSpaceBegin(ES_HELLOWORLD),
+ Ping,
+ Pong,
+ Blob,
+ End
+ };
+
+ static_assert(End < EventSpaceEnd(ES_HELLOWORLD), "expect End < EventSpaceEnd(ES_HELLOWORLD)");
+ };
+
+ struct TEvPing: public TEventBase<TEvPing, THelloWorld::Ping> {
+ DEFINE_SIMPLE_NONLOCAL_EVENT(TEvPing, "HelloWorld: Ping");
+ };
+
+ struct TEvPong: public TEventBase<TEvPong, THelloWorld::Pong> {
+ DEFINE_SIMPLE_NONLOCAL_EVENT(TEvPong, "HelloWorld: Pong");
+ };
+
+ struct TEvBlob: public TEventBase<TEvBlob, THelloWorld::Blob> {
+ const TString Blob;
+
+ TEvBlob(const TString& blob) noexcept
+ : Blob(blob)
+ {
+ }
+
+ TString ToStringHeader() const noexcept override {
+ return "THelloWorld::Blob";
+ }
+
+ bool SerializeToArcadiaStream(TChunkSerializer *serializer) const override {
+ return serializer->WriteString(&Blob);
+ }
+
+ static IEventBase* Load(TEventSerializedData* bufs) noexcept {
+ return new TEvBlob(bufs->GetString());
+ }
+
+ bool IsSerializable() const override {
+ return true;
+ }
+ };
+
+ struct TSystem {
+ enum {
+ Start = EventSpaceBegin(ES_SYSTEM),
+ Bootstrap, // generic bootstrap event
+ Wakeup, // generic timeout
+ Subscribe, // generic subscribe to something
+ Unsubscribe, // generic unsubscribe from something
+ Delivered, // event delivered
+ Undelivered, // event undelivered
+ Poison, // request actor to shutdown
+ Completed, // generic async job result event
+ PoisonTaken, // generic Poison taken (reply to PoisonPill event, i.e. died completely)
+ FlushLog,
+ CallbackCompletion,
+ CallbackException,
+ Gone, // Generic notification of actor death
+ TrackActor,
+ UntrackActor,
+ InvokeResult,
+ CoroTimeout,
+ InvokeQuery,
+ End,
+
+ // Compatibility section
+ PoisonPill = Poison,
+ ActorDied = Gone,
+ };
+
+ static_assert(End < EventSpaceEnd(ES_SYSTEM), "expect End < EventSpaceEnd(ES_SYSTEM)");
+ };
+
+ struct TEvBootstrap: public TEventBase<TEvBootstrap, TSystem::Bootstrap> {
+ DEFINE_SIMPLE_LOCAL_EVENT(TEvBootstrap, "System: TEvBootstrap")
+ };
+
+ struct TEvPoison : public TEventBase<TEvPoison, TSystem::Poison> {
+ DEFINE_SIMPLE_NONLOCAL_EVENT(TEvPoison, "System: TEvPoison")
+ };
+
+ struct TEvWakeup: public TEventBase<TEvWakeup, TSystem::Wakeup> {
+ DEFINE_SIMPLE_LOCAL_EVENT(TEvWakeup, "System: TEvWakeup")
+
+ TEvWakeup(ui64 tag = 0) : Tag(tag) { }
+
+ const ui64 Tag = 0;
+ };
+
+ struct TEvSubscribe: public TEventBase<TEvSubscribe, TSystem::Subscribe> {
+ DEFINE_SIMPLE_LOCAL_EVENT(TEvSubscribe, "System: TEvSubscribe")
+ };
+
+ struct TEvUnsubscribe: public TEventBase<TEvUnsubscribe, TSystem::Unsubscribe> {
+ DEFINE_SIMPLE_LOCAL_EVENT(TEvUnsubscribe, "System: TEvUnsubscribe")
+ };
+
+ struct TEvUndelivered: public TEventBase<TEvUndelivered, TSystem::Undelivered> {
+ enum EReason {
+ ReasonUnknown,
+ ReasonActorUnknown,
+ Disconnected
+ };
+ const ui32 SourceType;
+ const EReason Reason;
+ const bool Unsure;
+ const TString Data;
+
+ TEvUndelivered(ui32 sourceType, ui32 reason, bool unsure = false)
+ : SourceType(sourceType)
+ , Reason(static_cast<EReason>(reason))
+ , Unsure(unsure)
+ , Data(MakeData(sourceType, reason))
+ {}
+
+ TString ToStringHeader() const override;
+ bool SerializeToArcadiaStream(TChunkSerializer *serializer) const override;
+ static IEventBase* Load(TEventSerializedData* bufs);
+ bool IsSerializable() const override;
+
+ ui32 CalculateSerializedSize() const override { return 2 * sizeof(ui32); }
+
+ static void Out(IOutputStream& o, EReason x);
+
+ private:
+ static TString MakeData(ui32 sourceType, ui32 reason) {
+ TString s = TString::Uninitialized(sizeof(ui32) + sizeof(ui32));
+ char *p = s.Detach();
+ WriteUnaligned<ui32>(p + 0, sourceType);
+ WriteUnaligned<ui32>(p + 4, reason);
+ return s;
+ }
+ };
+
+ struct TEvCompleted: public TEventBase<TEvCompleted, TSystem::Completed> {
+ const ui32 Id;
+ const ui32 Status;
+ TEvCompleted(ui32 id = 0, ui32 status = 0)
+ : Id(id)
+ , Status(status)
+ {
+ }
+
+ DEFINE_SIMPLE_LOCAL_EVENT(TEvCompleted, "System: TEvCompleted")
+ };
+
+ struct TEvPoisonTaken: public TEventBase<TEvPoisonTaken, TSystem::PoisonTaken> {
+ DEFINE_SIMPLE_LOCAL_EVENT(TEvPoisonTaken, "System: TEvPoisonTaken")
+ };
+
+ struct TEvFlushLog: public TEventBase<TEvFlushLog, TSystem::FlushLog> {
+ DEFINE_SIMPLE_LOCAL_EVENT(TEvFlushLog, "System: TEvFlushLog")
+ };
+
+ struct TEvCallbackException: public TEventPB<TEvCallbackException,
+ NActorsProto::TCallbackException,
+ TSystem::CallbackException> {
+ TEvCallbackException(const TActorId& id, const TString& msg) {
+ ActorIdToProto(id, Record.MutableActorId());
+ Record.SetExceptionMessage(msg);
+ }
+ };
+
+ struct TEvCallbackCompletion: public TEventPB<TEvCallbackCompletion,
+ NActorsProto::TActorId,
+ TSystem::CallbackCompletion> {
+ TEvCallbackCompletion(const TActorId& id) {
+ ActorIdToProto(id, &Record);
+ }
+ };
+
+ struct TEvGone: public TEventBase<TEvGone, TSystem::Gone> {
+ DEFINE_SIMPLE_LOCAL_EVENT(TEvGone, "System: TEvGone")
+ };
+
+ struct TEvInvokeResult;
+
+ using TEvPoisonPill = TEvPoison; // Legacy name, deprecated
+ using TEvActorDied = TEvGone;
+ };
+}
+
+template <>
+inline void Out<NActors::TEvents::TEvUndelivered::EReason>(IOutputStream& o, NActors::TEvents::TEvUndelivered::EReason x) {
+ NActors::TEvents::TEvUndelivered::Out(o, x);
+}
diff --git a/library/cpp/actors/core/events_undelivered.cpp b/library/cpp/actors/core/events_undelivered.cpp
new file mode 100644
index 0000000000..23deaffd10
--- /dev/null
+++ b/library/cpp/actors/core/events_undelivered.cpp
@@ -0,0 +1,60 @@
+#include "events.h"
+#include "actorsystem.h"
+
+namespace NActors {
+ TString TEvents::TEvUndelivered::ToStringHeader() const {
+ return "TSystem::Undelivered";
+ }
+
+ bool TEvents::TEvUndelivered::SerializeToArcadiaStream(TChunkSerializer *serializer) const {
+ Y_VERIFY(!Unsure); // these are local-only events generated by Interconnect
+ return serializer->WriteString(&Data);
+ }
+
+ void TEvents::TEvUndelivered::Out(IOutputStream& o, EReason x) {
+ switch (x) {
+ case ReasonActorUnknown:
+ o << "ActorUnknown";
+ break;
+ case Disconnected:
+ o << "Disconnected";
+ break;
+ default:
+ o << "Undefined";
+ break;
+ }
+ }
+
+ bool TEvents::TEvUndelivered::IsSerializable() const {
+ return true;
+ }
+
+ IEventBase* TEvents::TEvUndelivered::Load(TEventSerializedData* bufs) {
+ TString str = bufs->GetString();
+ Y_VERIFY(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);
+ return new TEvUndelivered(sourceType, reason);
+ }
+
+ TAutoPtr<IEventHandle> IEventHandle::ForwardOnNondelivery(ui32 reason, bool unsure) {
+ if (Flags & FlagForwardOnNondelivery) {
+ const ui32 updatedFlags = Flags & ~(FlagForwardOnNondelivery | FlagSubscribeOnSession);
+ const TActorId recp = OnNondeliveryHolder ? OnNondeliveryHolder->Recipient : TActorId();
+
+ if (Event)
+ return new IEventHandle(recp, Sender, Event.Release(), updatedFlags, Cookie, &Recipient, TraceId.Clone());
+ else
+ return new IEventHandle(Type, updatedFlags, recp, Sender, Buffer, Cookie, &Recipient, TraceId.Clone());
+ }
+
+ if (Flags & FlagTrackDelivery) {
+ const ui32 updatedFlags = Flags & ~(FlagTrackDelivery | FlagSubscribeOnSession | FlagGenerateUnsureUndelivered);
+ return new IEventHandle(Sender, Recipient, new TEvents::TEvUndelivered(Type, reason, unsure), updatedFlags,
+ Cookie, nullptr, TraceId.Clone());
+ }
+
+ return nullptr;
+ }
+}
diff --git a/library/cpp/actors/core/executelater.h b/library/cpp/actors/core/executelater.h
new file mode 100644
index 0000000000..e7a13c1005
--- /dev/null
+++ b/library/cpp/actors/core/executelater.h
@@ -0,0 +1,87 @@
+#pragma once
+
+#include "actor_bootstrapped.h"
+
+#include <utility>
+
+namespace NActors {
+ template <typename TCallback>
+ class TExecuteLater: public TActorBootstrapped<TExecuteLater<TCallback>> {
+ public:
+ static constexpr IActor::EActivityType ActorActivityType() {
+ return IActor::ACTORLIB_COMMON;
+ }
+
+ TExecuteLater(
+ TCallback&& callback,
+ IActor::EActivityType activityType,
+ ui32 channel = 0,
+ ui64 cookie = 0,
+ const TActorId& reportCompletionTo = TActorId(),
+ const TActorId& reportExceptionTo = TActorId()) noexcept
+ : Callback(std::move(callback))
+ , Channel(channel)
+ , Cookie(cookie)
+ , ReportCompletionTo(reportCompletionTo)
+ , ReportExceptionTo(reportExceptionTo)
+ {
+ this->SetActivityType(activityType);
+ }
+
+ void Bootstrap(const TActorContext& ctx) noexcept {
+ try {
+ {
+ /* RAII, Callback should be destroyed right before sending
+ TEvCallbackCompletion */
+
+ auto local = std::move(Callback);
+ using T = decltype(local);
+
+ if constexpr (std::is_invocable_v<T, const TActorContext&>) {
+ local(ctx);
+ } else {
+ local();
+ }
+ }
+
+ if (ReportCompletionTo) {
+ ctx.Send(ReportCompletionTo,
+ new TEvents::TEvCallbackCompletion(ctx.SelfID),
+ Channel, Cookie);
+ }
+ } catch (...) {
+ if (ReportExceptionTo) {
+ const TString msg = CurrentExceptionMessage();
+ ctx.Send(ReportExceptionTo,
+ new TEvents::TEvCallbackException(ctx.SelfID, msg),
+ Channel, Cookie);
+ }
+ }
+
+ this->Die(ctx);
+ }
+
+ private:
+ TCallback Callback;
+ const ui32 Channel;
+ const ui64 Cookie;
+ const TActorId ReportCompletionTo;
+ const TActorId ReportExceptionTo;
+ };
+
+ template <typename T>
+ IActor* CreateExecuteLaterActor(
+ T&& func,
+ IActor::EActivityType activityType,
+ ui32 channel = 0,
+ ui64 cookie = 0,
+ const TActorId& reportCompletionTo = TActorId(),
+ const TActorId& reportExceptionTo = TActorId()) noexcept {
+ return new TExecuteLater<T>(std::forward<T>(func),
+ activityType,
+ channel,
+ cookie,
+ reportCompletionTo,
+ reportExceptionTo);
+ }
+}
diff --git a/library/cpp/actors/core/executor_pool_base.cpp b/library/cpp/actors/core/executor_pool_base.cpp
new file mode 100644
index 0000000000..c3b9999168
--- /dev/null
+++ b/library/cpp/actors/core/executor_pool_base.cpp
@@ -0,0 +1,168 @@
+#include "executor_pool_base.h"
+#include "executor_thread.h"
+#include "mailbox.h"
+#include "probes.h"
+#include <library/cpp/actors/util/datetime.h>
+
+namespace NActors {
+ LWTRACE_USING(ACTORLIB_PROVIDER);
+
+ void DoActorInit(TActorSystem* sys, IActor* actor, const TActorId& self, const TActorId& owner) {
+ actor->SelfActorId = self;
+ actor->Registered(sys, owner);
+ }
+
+ TExecutorPoolBaseMailboxed::TExecutorPoolBaseMailboxed(ui32 poolId, ui32 maxActivityType)
+ : IExecutorPool(poolId)
+ , ActorSystem(nullptr)
+ , MailboxTable(new TMailboxTable)
+#ifdef ACTORSLIB_COLLECT_EXEC_STATS
+ , Stats(maxActivityType)
+#endif
+ {}
+
+ TExecutorPoolBaseMailboxed::~TExecutorPoolBaseMailboxed() {
+ MailboxTable.Destroy();
+ }
+
+ TExecutorPoolBase::TExecutorPoolBase(ui32 poolId, ui32 threads, TAffinity* affinity, ui32 maxActivityType)
+ : TExecutorPoolBaseMailboxed(poolId, maxActivityType)
+ , PoolThreads(threads)
+ , ThreadsAffinity(affinity)
+ {}
+
+ TExecutorPoolBase::~TExecutorPoolBase() {
+ while (Activations.Pop(0))
+ ;
+ }
+
+ void TExecutorPoolBaseMailboxed::ReclaimMailbox(TMailboxType::EType mailboxType, ui32 hint, TWorkerId workerId, ui64 revolvingWriteCounter) {
+ Y_UNUSED(workerId);
+ MailboxTable->ReclaimMailbox(mailboxType, hint, revolvingWriteCounter);
+ }
+
+ ui64 TExecutorPoolBaseMailboxed::AllocateID() {
+ return ActorSystem->AllocateIDSpace(1);
+ }
+
+ bool TExecutorPoolBaseMailboxed::Send(TAutoPtr<IEventHandle>& ev) {
+ Y_VERIFY_DEBUG(ev->GetRecipientRewrite().PoolID() == PoolId);
+#ifdef ACTORSLIB_COLLECT_EXEC_STATS
+ RelaxedStore(&ev->SendTime, (::NHPTimer::STime)GetCycleCountFast());
+#endif
+ return MailboxTable->SendTo(ev, this);
+ }
+
+ void TExecutorPoolBase::ScheduleActivation(ui32 activation) {
+ ScheduleActivationEx(activation, AtomicIncrement(ActivationsRevolvingCounter));
+ }
+
+ TActorId TExecutorPoolBaseMailboxed::Register(IActor* actor, TMailboxType::EType mailboxType, ui64 revolvingWriteCounter, const TActorId& parentId) {
+ NHPTimer::STime hpstart = GetCycleCountFast();
+#ifdef ACTORSLIB_COLLECT_EXEC_STATS
+ ui32 at = actor->GetActivityType();
+ if (at >= Stats.MaxActivityType())
+ at = 0;
+ AtomicIncrement(Stats.ActorsAliveByActivity[at]);
+#endif
+ AtomicIncrement(ActorRegistrations);
+
+ // first step - find good enough mailbox
+ ui32 hint = 0;
+ TMailboxHeader* mailbox = nullptr;
+
+ if (revolvingWriteCounter == 0)
+ revolvingWriteCounter = AtomicIncrement(RegisterRevolvingCounter);
+
+ {
+ ui32 hintBackoff = 0;
+
+ while (hint == 0) {
+ hint = MailboxTable->AllocateMailbox(mailboxType, ++revolvingWriteCounter);
+ mailbox = MailboxTable->Get(hint);
+
+ if (!mailbox->LockFromFree()) {
+ MailboxTable->ReclaimMailbox(mailboxType, hintBackoff, ++revolvingWriteCounter);
+ hintBackoff = hint;
+ hint = 0;
+ }
+ }
+
+ MailboxTable->ReclaimMailbox(mailboxType, hintBackoff, ++revolvingWriteCounter);
+ }
+
+ const ui64 localActorId = AllocateID();
+
+ // ok, got mailbox
+ mailbox->AttachActor(localActorId, actor);
+
+ // do init
+ const TActorId actorId(ActorSystem->NodeId, PoolId, localActorId, hint);
+ DoActorInit(ActorSystem, actor, actorId, parentId);
+
+ // Once we unlock the mailbox the actor starts running and we cannot use the pointer any more
+ actor = nullptr;
+
+ switch (mailboxType) {
+ case TMailboxType::Simple:
+ UnlockFromExecution((TMailboxTable::TSimpleMailbox*)mailbox, this, false, hint, MaxWorkers, ++revolvingWriteCounter);
+ break;
+ case TMailboxType::Revolving:
+ UnlockFromExecution((TMailboxTable::TRevolvingMailbox*)mailbox, this, false, hint, MaxWorkers, ++revolvingWriteCounter);
+ break;
+ case TMailboxType::HTSwap:
+ UnlockFromExecution((TMailboxTable::THTSwapMailbox*)mailbox, this, false, hint, MaxWorkers, ++revolvingWriteCounter);
+ break;
+ case TMailboxType::ReadAsFilled:
+ UnlockFromExecution((TMailboxTable::TReadAsFilledMailbox*)mailbox, this, false, hint, MaxWorkers, ++revolvingWriteCounter);
+ break;
+ case TMailboxType::TinyReadAsFilled:
+ UnlockFromExecution((TMailboxTable::TTinyReadAsFilledMailbox*)mailbox, this, false, hint, MaxWorkers, ++revolvingWriteCounter);
+ break;
+ default:
+ Y_FAIL();
+ }
+
+ NHPTimer::STime elapsed = GetCycleCountFast() - hpstart;
+ if (elapsed > 1000000) {
+ LWPROBE(SlowRegisterNew, PoolId, NHPTimer::GetSeconds(elapsed) * 1000.0);
+ }
+
+ return actorId;
+ }
+
+ TActorId TExecutorPoolBaseMailboxed::Register(IActor* actor, TMailboxHeader* mailbox, ui32 hint, const TActorId& parentId) {
+ NHPTimer::STime hpstart = GetCycleCountFast();
+#ifdef ACTORSLIB_COLLECT_EXEC_STATS
+ ui32 at = actor->GetActivityType();
+ if (at >= Stats.MaxActivityType())
+ at = 0;
+ AtomicIncrement(Stats.ActorsAliveByActivity[at]);
+#endif
+ AtomicIncrement(ActorRegistrations);
+
+ const ui64 localActorId = AllocateID();
+ mailbox->AttachActor(localActorId, actor);
+
+ const TActorId actorId(ActorSystem->NodeId, PoolId, localActorId, hint);
+ DoActorInit(ActorSystem, actor, actorId, parentId);
+ NHPTimer::STime elapsed = GetCycleCountFast() - hpstart;
+ if (elapsed > 1000000) {
+ LWPROBE(SlowRegisterAdd, PoolId, NHPTimer::GetSeconds(elapsed) * 1000.0);
+ }
+
+ return actorId;
+ }
+
+ TAffinity* TExecutorPoolBase::Affinity() const {
+ return ThreadsAffinity.Get();
+ }
+
+ bool TExecutorPoolBaseMailboxed::Cleanup() {
+ return MailboxTable->Cleanup();
+ }
+
+ ui32 TExecutorPoolBase::GetThreads() const {
+ return PoolThreads;
+ }
+}
diff --git a/library/cpp/actors/core/executor_pool_base.h b/library/cpp/actors/core/executor_pool_base.h
new file mode 100644
index 0000000000..c84ce1af77
--- /dev/null
+++ b/library/cpp/actors/core/executor_pool_base.h
@@ -0,0 +1,49 @@
+#pragma once
+
+#include "actorsystem.h"
+#include "executor_thread.h"
+#include "scheduler_queue.h"
+#include <library/cpp/actors/util/affinity.h>
+#include <library/cpp/actors/util/unordered_cache.h>
+#include <library/cpp/actors/util/threadparkpad.h>
+
+namespace NActors {
+ class TExecutorPoolBaseMailboxed: public IExecutorPool {
+ protected:
+ TActorSystem* ActorSystem;
+ THolder<TMailboxTable> MailboxTable;
+#ifdef ACTORSLIB_COLLECT_EXEC_STATS
+ // Need to have per pool object to collect stats like actor registrations (because
+ // registrations might be done in threads from other pools)
+ TExecutorThreadStats Stats;
+#endif
+ TAtomic RegisterRevolvingCounter = 0;
+ ui64 AllocateID();
+ public:
+ TExecutorPoolBaseMailboxed(ui32 poolId, ui32 maxActivityType);
+ ~TExecutorPoolBaseMailboxed();
+ void ReclaimMailbox(TMailboxType::EType mailboxType, ui32 hint, TWorkerId workerId, ui64 revolvingWriteCounter) override;
+ bool Send(TAutoPtr<IEventHandle>& ev) override;
+ TActorId Register(IActor* actor, TMailboxType::EType mailboxType, ui64 revolvingWriteCounter, const TActorId& parentId) override;
+ TActorId Register(IActor* actor, TMailboxHeader* mailbox, ui32 hint, const TActorId& parentId) override;
+ bool Cleanup() override;
+ };
+
+ class TExecutorPoolBase: public TExecutorPoolBaseMailboxed {
+ protected:
+ const ui32 PoolThreads;
+ TIntrusivePtr<TAffinity> ThreadsAffinity;
+ TAtomic Semaphore = 0;
+ TUnorderedCache<ui32, 512, 4> Activations;
+ TAtomic ActivationsRevolvingCounter = 0;
+ volatile bool StopFlag = false;
+ public:
+ TExecutorPoolBase(ui32 poolId, ui32 threads, TAffinity* affinity, ui32 maxActivityType);
+ ~TExecutorPoolBase();
+ void ScheduleActivation(ui32 activation) override;
+ TAffinity* Affinity() const override;
+ ui32 GetThreads() const override;
+ };
+
+ void DoActorInit(TActorSystem*, IActor*, const TActorId&, const TActorId&);
+}
diff --git a/library/cpp/actors/core/executor_pool_basic.cpp b/library/cpp/actors/core/executor_pool_basic.cpp
new file mode 100644
index 0000000000..4dce16939a
--- /dev/null
+++ b/library/cpp/actors/core/executor_pool_basic.cpp
@@ -0,0 +1,431 @@
+#include "executor_pool_basic.h"
+#include "probes.h"
+#include "mailbox.h"
+#include <library/cpp/actors/util/affinity.h>
+#include <library/cpp/actors/util/datetime.h>
+
+#ifdef _linux_
+#include <pthread.h>
+#endif
+
+namespace NActors {
+ LWTRACE_USING(ACTORLIB_PROVIDER);
+
+ constexpr TDuration TBasicExecutorPool::DEFAULT_TIME_PER_MAILBOX;
+
+ TBasicExecutorPool::TBasicExecutorPool(
+ ui32 poolId,
+ ui32 threads,
+ ui64 spinThreshold,
+ const TString& poolName,
+ TAffinity* affinity,
+ TDuration timePerMailbox,
+ ui32 eventsPerMailbox,
+ int realtimePriority,
+ ui32 maxActivityType)
+ : TExecutorPoolBase(poolId, threads, affinity, maxActivityType)
+ , SpinThreshold(spinThreshold)
+ , SpinThresholdCycles(spinThreshold * NHPTimer::GetCyclesPerSecond() * 0.000001) // convert microseconds to cycles
+ , Threads(new TThreadCtx[threads])
+ , PoolName(poolName)
+ , TimePerMailbox(timePerMailbox)
+ , EventsPerMailbox(eventsPerMailbox)
+ , RealtimePriority(realtimePriority)
+ , ThreadUtilization(0)
+ , MaxUtilizationCounter(0)
+ , MaxUtilizationAccumulator(0)
+ , ThreadCount(threads)
+ {
+ }
+
+ TBasicExecutorPool::TBasicExecutorPool(const TBasicExecutorPoolConfig& cfg)
+ : TBasicExecutorPool(
+ cfg.PoolId,
+ cfg.Threads,
+ cfg.SpinThreshold,
+ cfg.PoolName,
+ new TAffinity(cfg.Affinity),
+ cfg.TimePerMailbox,
+ cfg.EventsPerMailbox,
+ cfg.RealtimePriority,
+ cfg.MaxActivityType
+ )
+ {}
+
+ TBasicExecutorPool::~TBasicExecutorPool() {
+ Threads.Destroy();
+ }
+
+ ui32 TBasicExecutorPool::GetReadyActivation(TWorkerContext& wctx, ui64 revolvingCounter) {
+ ui32 workerId = wctx.WorkerId;
+ Y_VERIFY_DEBUG(workerId < PoolThreads);
+
+ NHPTimer::STime elapsed = 0;
+ NHPTimer::STime parked = 0;
+ NHPTimer::STime blocked = 0;
+ NHPTimer::STime hpstart = GetCycleCountFast();
+ NHPTimer::STime hpnow;
+
+ TThreadCtx& threadCtx = Threads[workerId];
+ AtomicSet(threadCtx.WaitingFlag, TThreadCtx::WS_NONE);
+
+ if (Y_UNLIKELY(AtomicGet(threadCtx.BlockedFlag) != TThreadCtx::BS_NONE)) {
+ do {
+ if (AtomicCas(&threadCtx.BlockedFlag, TThreadCtx::BS_BLOCKED, TThreadCtx::BS_BLOCKING)) {
+ hpnow = GetCycleCountFast();
+ elapsed += hpnow - hpstart;
+ if (threadCtx.BlockedPad.Park()) // interrupted
+ return 0;
+ hpstart = GetCycleCountFast();
+ blocked += hpstart - hpnow;
+ }
+ } while (AtomicGet(threadCtx.BlockedFlag) != TThreadCtx::BS_NONE && !AtomicLoad(&StopFlag));
+ }
+
+ const TAtomic x = AtomicDecrement(Semaphore);
+
+ if (x < 0) {
+#if defined ACTORSLIB_COLLECT_EXEC_STATS
+ if (AtomicGetAndIncrement(ThreadUtilization) == 0) {
+ // Initially counter contains -t0, the pool start timestamp
+ // When the first thread goes to sleep we add t1, so the counter
+ // becomes t1-t0 >= 0, or the duration of max utilization so far.
+ // If the counter was negative and becomes positive, that means
+ // counter just turned into a duration and we should store that
+ // duration. Otherwise another thread raced with us and
+ // subtracted some other timestamp t2.
+ const i64 t = GetCycleCountFast();
+ const i64 x = AtomicGetAndAdd(MaxUtilizationCounter, t);
+ if (x < 0 && x + t > 0)
+ AtomicStore(&MaxUtilizationAccumulator, x + t);
+ }
+#endif
+
+ Y_VERIFY(AtomicLoad(&threadCtx.WaitingFlag) == TThreadCtx::WS_NONE);
+
+ if (SpinThreshold > 0) {
+ // spin configured period
+ AtomicSet(threadCtx.WaitingFlag, TThreadCtx::WS_ACTIVE);
+ ui64 start = GetCycleCountFast();
+ bool doSpin = true;
+ while (true) {
+ for (ui32 j = 0; doSpin && j < 12; ++j) {
+ if (GetCycleCountFast() >= (start + SpinThresholdCycles)) {
+ doSpin = false;
+ break;
+ }
+ for (ui32 i = 0; i < 12; ++i) {
+ if (AtomicLoad(&threadCtx.WaitingFlag) == TThreadCtx::WS_ACTIVE) {
+ SpinLockPause();
+ } else {
+ doSpin = false;
+ break;
+ }
+ }
+ }
+ if (!doSpin) {
+ break;
+ }
+ if (RelaxedLoad(&StopFlag)) {
+ break;
+ }
+ }
+ // then - sleep
+ if (AtomicLoad(&threadCtx.WaitingFlag) == TThreadCtx::WS_ACTIVE) {
+ if (AtomicCas(&threadCtx.WaitingFlag, TThreadCtx::WS_BLOCKED, TThreadCtx::WS_ACTIVE)) {
+ do {
+ hpnow = GetCycleCountFast();
+ elapsed += hpnow - hpstart;
+ if (threadCtx.Pad.Park()) // interrupted
+ return 0;
+ hpstart = GetCycleCountFast();
+ parked += hpstart - hpnow;
+ } while (AtomicLoad(&threadCtx.WaitingFlag) == TThreadCtx::WS_BLOCKED);
+ }
+ }
+ } else {
+ AtomicSet(threadCtx.WaitingFlag, TThreadCtx::WS_BLOCKED);
+ do {
+ hpnow = GetCycleCountFast();
+ elapsed += hpnow - hpstart;
+ if (threadCtx.Pad.Park()) // interrupted
+ return 0;
+ hpstart = GetCycleCountFast();
+ parked += hpstart - hpnow;
+ } while (AtomicLoad(&threadCtx.WaitingFlag) == TThreadCtx::WS_BLOCKED);
+ }
+
+ Y_VERIFY_DEBUG(AtomicLoad(&StopFlag) || AtomicLoad(&threadCtx.WaitingFlag) == TThreadCtx::WS_RUNNING);
+
+#if defined ACTORSLIB_COLLECT_EXEC_STATS
+ if (AtomicDecrement(ThreadUtilization) == 0) {
+ // When we started sleeping counter contained t1-t0, or the
+ // last duration of max utilization. Now we subtract t2 >= t1,
+ // which turns counter negative again, and the next sleep cycle
+ // at timestamp t3 would be adding some new duration t3-t2.
+ // If the counter was positive and becomes negative that means
+ // there are no current races with other threads and we should
+ // store the last positive duration we observed. Multiple
+ // threads may be adding and subtracting values in potentially
+ // arbitrary order, which would cause counter to oscillate
+ // around zero. When it crosses zero is a good indication of a
+ // correct value.
+ const i64 t = GetCycleCountFast();
+ const i64 x = AtomicGetAndAdd(MaxUtilizationCounter, -t);
+ if (x > 0 && x - t < 0)
+ AtomicStore(&MaxUtilizationAccumulator, x);
+ }
+#endif
+ } else {
+ AtomicSet(threadCtx.WaitingFlag, TThreadCtx::WS_RUNNING);
+ }
+
+ // ok, has work suggested, must dequeue
+ while (!RelaxedLoad(&StopFlag)) {
+ if (const ui32 activation = Activations.Pop(++revolvingCounter)) {
+ hpnow = GetCycleCountFast();
+ elapsed += hpnow - hpstart;
+ wctx.AddElapsedCycles(IActor::ACTOR_SYSTEM, elapsed);
+ if (parked > 0) {
+ wctx.AddParkedCycles(parked);
+ }
+ if (blocked > 0) {
+ wctx.AddBlockedCycles(blocked);
+ }
+ return activation;
+ }
+ SpinLockPause();
+ }
+
+ // stopping, die!
+ return 0;
+ }
+
+ inline void TBasicExecutorPool::WakeUpLoop() {
+ for (ui32 i = 0;;) {
+ TThreadCtx& threadCtx = Threads[i % PoolThreads];
+ switch (AtomicLoad(&threadCtx.WaitingFlag)) {
+ case TThreadCtx::WS_NONE:
+ case TThreadCtx::WS_RUNNING:
+ ++i;
+ break;
+ case TThreadCtx::WS_ACTIVE: // in active spin-lock, just set flag
+ if (AtomicCas(&threadCtx.WaitingFlag, TThreadCtx::WS_RUNNING, TThreadCtx::WS_ACTIVE)) {
+ return;
+ }
+ break;
+ case TThreadCtx::WS_BLOCKED:
+ if (AtomicCas(&threadCtx.WaitingFlag, TThreadCtx::WS_RUNNING, TThreadCtx::WS_BLOCKED)) {
+ threadCtx.Pad.Unpark();
+ return;
+ }
+ break;
+ default:
+ Y_FAIL();
+ }
+ }
+ }
+
+ void TBasicExecutorPool::ScheduleActivationEx(ui32 activation, ui64 revolvingCounter) {
+ Activations.Push(activation, revolvingCounter);
+ const TAtomic x = AtomicIncrement(Semaphore);
+ if (x <= 0) { // we must find someone to wake-up
+ WakeUpLoop();
+ }
+ }
+
+ void TBasicExecutorPool::GetCurrentStats(TExecutorPoolStats& poolStats, TVector<TExecutorThreadStats>& statsCopy) const {
+ poolStats.MaxUtilizationTime = RelaxedLoad(&MaxUtilizationAccumulator) / (i64)(NHPTimer::GetCyclesPerSecond() / 1000);
+
+ statsCopy.resize(PoolThreads + 1);
+ // Save counters from the pool object
+ statsCopy[0] = TExecutorThreadStats();
+ statsCopy[0].Aggregate(Stats);
+ // Per-thread stats
+ for (size_t i = 0; i < PoolThreads; ++i) {
+ Threads[i].Thread->GetCurrentStats(statsCopy[i + 1]);
+ }
+ }
+
+ void TBasicExecutorPool::Prepare(TActorSystem* actorSystem, NSchedulerQueue::TReader** scheduleReaders, ui32* scheduleSz) {
+ TAffinityGuard affinityGuard(Affinity());
+
+ ActorSystem = actorSystem;
+
+ ScheduleReaders.Reset(new NSchedulerQueue::TReader[PoolThreads]);
+ ScheduleWriters.Reset(new NSchedulerQueue::TWriter[PoolThreads]);
+
+ for (ui32 i = 0; i != PoolThreads; ++i) {
+ Threads[i].Thread.Reset(
+ new TExecutorThread(
+ i,
+ 0, // CpuId is not used in BASIC pool
+ actorSystem,
+ this,
+ MailboxTable.Get(),
+ PoolName,
+ TimePerMailbox,
+ EventsPerMailbox));
+ ScheduleWriters[i].Init(ScheduleReaders[i]);
+ }
+
+ *scheduleReaders = ScheduleReaders.Get();
+ *scheduleSz = PoolThreads;
+ }
+
+ void TBasicExecutorPool::Start() {
+ TAffinityGuard affinityGuard(Affinity());
+
+ ThreadUtilization = 0;
+ AtomicAdd(MaxUtilizationCounter, -(i64)GetCycleCountFast());
+
+ for (ui32 i = 0; i != PoolThreads; ++i) {
+ Threads[i].Thread->Start();
+ }
+ }
+
+ void TBasicExecutorPool::PrepareStop() {
+ AtomicStore(&StopFlag, true);
+ for (ui32 i = 0; i != PoolThreads; ++i) {
+ Threads[i].Pad.Interrupt();
+ Threads[i].BlockedPad.Interrupt();
+ }
+ }
+
+ void TBasicExecutorPool::Shutdown() {
+ for (ui32 i = 0; i != PoolThreads; ++i)
+ Threads[i].Thread->Join();
+ }
+
+ void TBasicExecutorPool::Schedule(TInstant deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) {
+ Y_VERIFY_DEBUG(workerId < PoolThreads);
+
+ Schedule(deadline - ActorSystem->Timestamp(), ev, cookie, workerId);
+ }
+
+ void TBasicExecutorPool::Schedule(TMonotonic deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) {
+ Y_VERIFY_DEBUG(workerId < PoolThreads);
+
+ const auto current = ActorSystem->Monotonic();
+ if (deadline < current)
+ deadline = current;
+
+ ScheduleWriters[workerId].Push(deadline.MicroSeconds(), ev.Release(), cookie);
+ }
+
+ void TBasicExecutorPool::Schedule(TDuration delta, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) {
+ Y_VERIFY_DEBUG(workerId < PoolThreads);
+
+ const auto deadline = ActorSystem->Monotonic() + delta;
+ ScheduleWriters[workerId].Push(deadline.MicroSeconds(), ev.Release(), cookie);
+ }
+
+ void TBasicExecutorPool::SetRealTimeMode() const {
+// TODO: musl-libc version of `sched_param` struct is for some reason different from pthread
+// version in Ubuntu 12.04
+#if defined(_linux_) && !defined(_musl_)
+ if (RealtimePriority != 0) {
+ pthread_t threadSelf = pthread_self();
+ sched_param param = {RealtimePriority};
+ if (pthread_setschedparam(threadSelf, SCHED_FIFO, &param)) {
+ Y_FAIL("Cannot set realtime priority");
+ }
+ }
+#else
+ Y_UNUSED(RealtimePriority);
+#endif
+ }
+
+ ui32 TBasicExecutorPool::GetThreadCount() const {
+ return AtomicGet(ThreadCount);
+ }
+
+ void TBasicExecutorPool::SetThreadCount(ui32 threads) {
+ threads = Max(1u, Min(PoolThreads, threads));
+ with_lock (ChangeThreadsLock) {
+ size_t prevCount = GetThreadCount();
+ AtomicSet(ThreadCount, threads);
+ if (prevCount < threads) {
+ for (size_t i = prevCount; i < threads; ++i) {
+ bool repeat = true;
+ while (repeat) {
+ switch (AtomicGet(Threads[i].BlockedFlag)) {
+ case TThreadCtx::BS_BLOCKING:
+ if (AtomicCas(&Threads[i].BlockedFlag, TThreadCtx::BS_NONE, TThreadCtx::BS_BLOCKING)) {
+ // thread not entry to blocked loop
+ repeat = false;
+ }
+ break;
+ case TThreadCtx::BS_BLOCKED:
+ // thread entry to blocked loop and we wake it
+ AtomicSet(Threads[i].BlockedFlag, TThreadCtx::BS_NONE);
+ Threads[i].BlockedPad.Unpark();
+ repeat = false;
+ break;
+ default:
+ // thread mustn't has TThreadCtx::BS_NONE because last time it was started to block
+ Y_FAIL("BlockedFlag is not TThreadCtx::BS_BLOCKING and TThreadCtx::BS_BLOCKED when thread was waked up");
+ }
+ }
+ }
+ } else if (prevCount > threads) {
+ // at first, start to block
+ for (size_t i = threads; i < prevCount; ++i) {
+ Y_VERIFY(AtomicGet(Threads[i].BlockedFlag) == TThreadCtx::BS_NONE);
+ AtomicSet(Threads[i].BlockedFlag, TThreadCtx::BS_BLOCKING);
+ }
+ // after check need to wake up threads
+ for (size_t idx = threads; idx < prevCount; ++idx) {
+ TThreadCtx& threadCtx = Threads[idx];
+ auto waitingFlag = AtomicGet(threadCtx.WaitingFlag);
+ auto blockedFlag = AtomicGet(threadCtx.BlockedFlag);
+ // while thread has this states (WS_NONE and BS_BLOCKING) we can't guess which way thread will go.
+ // Either go to sleep and it will have to wake up,
+ // or go to execute task and after completion will be blocked.
+ while (waitingFlag == TThreadCtx::WS_NONE && blockedFlag == TThreadCtx::BS_BLOCKING) {
+ waitingFlag = AtomicGet(threadCtx.WaitingFlag);
+ blockedFlag = AtomicGet(threadCtx.BlockedFlag);
+ }
+ // next states:
+ // 1) WS_ACTIVE BS_BLOCKING - waiting and start spinig | need wake up to block
+ // 2) WS_BLOCKED BS_BLOCKING - waiting and start sleep | need wake up to block
+ // 3) WS_RUNNING BS_BLOCKING - start execute | not need wake up, will block after executing
+ // 4) WS_NONE BS_BLOCKED - blocked | not need wake up, already blocked
+
+ if (waitingFlag == TThreadCtx::WS_ACTIVE || waitingFlag == TThreadCtx::WS_BLOCKED) {
+ // need wake up
+ Y_VERIFY(blockedFlag == TThreadCtx::BS_BLOCKING);
+
+ // creaty empty mailBoxHint, where LineIndex == 1 and LineHint == 0, and activations will be ignored
+ constexpr auto emptyMailBoxHint = TMailboxTable::LineIndexMask & -TMailboxTable::LineIndexMask;
+ ui64 revolvingCounter = AtomicGet(ActivationsRevolvingCounter);
+
+ Activations.Push(emptyMailBoxHint, revolvingCounter);
+
+ auto x = AtomicIncrement(Semaphore);
+ if (x <= 0) {
+ // try wake up. if success then go to next thread
+ switch (waitingFlag){
+ case TThreadCtx::WS_ACTIVE: // in active spin-lock, just set flag
+ if (AtomicCas(&threadCtx.WaitingFlag, TThreadCtx::WS_RUNNING, TThreadCtx::WS_ACTIVE)) {
+ continue;
+ }
+ break;
+ case TThreadCtx::WS_BLOCKED:
+ if (AtomicCas(&threadCtx.WaitingFlag, TThreadCtx::WS_RUNNING, TThreadCtx::WS_BLOCKED)) {
+ threadCtx.Pad.Unpark();
+ continue;
+ }
+ break;
+ default:
+ ; // other thread woke this sleeping thread
+ }
+ // if thread has already been awakened then we must awaken the other
+ WakeUpLoop();
+ }
+ }
+ }
+ }
+ }
+ }
+}
diff --git a/library/cpp/actors/core/executor_pool_basic.h b/library/cpp/actors/core/executor_pool_basic.h
new file mode 100644
index 0000000000..023190f7fe
--- /dev/null
+++ b/library/cpp/actors/core/executor_pool_basic.h
@@ -0,0 +1,111 @@
+#pragma once
+
+#include "actorsystem.h"
+#include "executor_thread.h"
+#include "scheduler_queue.h"
+#include "executor_pool_base.h"
+#include <library/cpp/actors/util/unordered_cache.h>
+#include <library/cpp/actors/util/threadparkpad.h>
+#include <library/cpp/monlib/dynamic_counters/counters.h>
+
+#include <util/system/mutex.h>
+
+namespace NActors {
+ class TBasicExecutorPool: public TExecutorPoolBase {
+ struct TThreadCtx {
+ TAutoPtr<TExecutorThread> Thread;
+ TThreadParkPad Pad;
+ TThreadParkPad BlockedPad;
+ TAtomic WaitingFlag;
+ TAtomic BlockedFlag;
+
+ // different threads must spin/block on different cache-lines.
+ // we add some padding bytes to enforce this rule
+ static const size_t SizeWithoutPadding = sizeof(TAutoPtr<TExecutorThread>) + 2 * sizeof(TThreadParkPad) + 2 * sizeof(TAtomic);
+ ui8 Padding[64 - SizeWithoutPadding];
+ static_assert(64 >= SizeWithoutPadding);
+
+ enum EWaitState {
+ WS_NONE,
+ WS_ACTIVE,
+ WS_BLOCKED,
+ WS_RUNNING
+ };
+
+ enum EBlockedState {
+ BS_NONE,
+ BS_BLOCKING,
+ BS_BLOCKED
+ };
+
+ TThreadCtx()
+ : WaitingFlag(WS_NONE)
+ , BlockedFlag(BS_NONE)
+ {
+ }
+ };
+
+ const ui64 SpinThreshold;
+ const ui64 SpinThresholdCycles;
+
+ TArrayHolder<TThreadCtx> Threads;
+
+ TArrayHolder<NSchedulerQueue::TReader> ScheduleReaders;
+ TArrayHolder<NSchedulerQueue::TWriter> ScheduleWriters;
+
+ const TString PoolName;
+ const TDuration TimePerMailbox;
+ const ui32 EventsPerMailbox;
+
+ const int RealtimePriority;
+
+ TAtomic ThreadUtilization;
+ TAtomic MaxUtilizationCounter;
+ TAtomic MaxUtilizationAccumulator;
+
+ TAtomic ThreadCount;
+ TMutex ChangeThreadsLock;
+
+ public:
+ static constexpr TDuration DEFAULT_TIME_PER_MAILBOX = TBasicExecutorPoolConfig::DEFAULT_TIME_PER_MAILBOX;
+ static constexpr ui32 DEFAULT_EVENTS_PER_MAILBOX = TBasicExecutorPoolConfig::DEFAULT_EVENTS_PER_MAILBOX;
+
+ TBasicExecutorPool(ui32 poolId,
+ ui32 threads,
+ ui64 spinThreshold,
+ const TString& poolName = "",
+ TAffinity* affinity = nullptr,
+ TDuration timePerMailbox = DEFAULT_TIME_PER_MAILBOX,
+ ui32 eventsPerMailbox = DEFAULT_EVENTS_PER_MAILBOX,
+ int realtimePriority = 0,
+ ui32 maxActivityType = 1);
+ explicit TBasicExecutorPool(const TBasicExecutorPoolConfig& cfg);
+ ~TBasicExecutorPool();
+
+ ui32 GetReadyActivation(TWorkerContext& wctx, ui64 revolvingReadCounter) override;
+
+ void Schedule(TInstant deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) override;
+ void Schedule(TMonotonic deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) override;
+ void Schedule(TDuration delta, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) override;
+
+ void ScheduleActivationEx(ui32 activation, ui64 revolvingWriteCounter) override;
+
+ void Prepare(TActorSystem* actorSystem, NSchedulerQueue::TReader** scheduleReaders, ui32* scheduleSz) override;
+ void Start() override;
+ void PrepareStop() override;
+ void Shutdown() override;
+
+ void GetCurrentStats(TExecutorPoolStats& poolStats, TVector<TExecutorThreadStats>& statsCopy) const override;
+ TString GetName() const override {
+ return PoolName;
+ }
+
+ void SetRealTimeMode() const override;
+
+ ui32 GetThreadCount() const;
+ void SetThreadCount(ui32 threads);
+
+ private:
+ void WakeUpLoop();
+ };
+}
diff --git a/library/cpp/actors/core/executor_pool_basic_ut.cpp b/library/cpp/actors/core/executor_pool_basic_ut.cpp
new file mode 100644
index 0000000000..76dff693af
--- /dev/null
+++ b/library/cpp/actors/core/executor_pool_basic_ut.cpp
@@ -0,0 +1,435 @@
+#include "actorsystem.h"
+#include "executor_pool_basic.h"
+#include "hfunc.h"
+#include "scheduler_basic.h"
+
+#include <library/cpp/actors/util/should_continue.h>
+
+#include <library/cpp/testing/unittest/registar.h>
+#include <library/cpp/actors/protos/unittests.pb.h>
+
+using namespace NActors;
+
+////////////////////////////////////////////////////////////////////////////////
+
+struct TEvMsg : public NActors::TEventBase<TEvMsg, 10347> {
+ DEFINE_SIMPLE_LOCAL_EVENT(TEvMsg, "ExecutorPoolTest: Msg");
+};
+
+////////////////////////////////////////////////////////////////////////////////
+
+class TTestSenderActor : public IActor {
+private:
+ using EActivityType = IActor::EActivityType ;
+ using EActorActivity = IActor::EActorActivity;
+
+private:
+ TAtomic Counter;
+ TActorId Receiver;
+
+ std::function<void(void)> Action;
+
+public:
+ TTestSenderActor(std::function<void(void)> action = [](){},
+ EActivityType activityType = EActorActivity::OTHER)
+ : IActor(static_cast<TReceiveFunc>(&TTestSenderActor::Execute), activityType)
+ , Action(action)
+ {}
+
+ void Start(TActorId receiver, size_t count)
+ {
+ AtomicSet(Counter, count);
+ Receiver = receiver;
+ }
+
+ void Stop() {
+ while (true) {
+ if (GetCounter() == 0) {
+ break;
+ }
+
+ Sleep(TDuration::MilliSeconds(1));
+ }
+ }
+
+ size_t GetCounter() const {
+ return AtomicGet(Counter);
+ }
+
+private:
+ STFUNC(Execute)
+ {
+ Y_UNUSED(ctx);
+ switch (ev->GetTypeRewrite()) {
+ hFunc(TEvMsg, Handle);
+ }
+ }
+
+ void Handle(TEvMsg::TPtr &ev)
+ {
+ Y_UNUSED(ev);
+ Action();
+ TAtomicBase count = AtomicDecrement(Counter);
+ Y_VERIFY(count != Max<TAtomicBase>());
+ if (count) {
+ Send(Receiver, new TEvMsg());
+ }
+ }
+};
+
+THolder<TActorSystemSetup> GetActorSystemSetup(TBasicExecutorPool* pool)
+{
+ auto setup = MakeHolder<NActors::TActorSystemSetup>();
+ setup->NodeId = 1;
+ setup->ExecutorsCount = 1;
+ setup->Executors.Reset(new TAutoPtr<NActors::IExecutorPool>[1]);
+ setup->Executors[0] = pool;
+ setup->Scheduler = new TBasicSchedulerThread(NActors::TSchedulerConfig(512, 0));
+ return setup;
+}
+
+Y_UNIT_TEST_SUITE(BasicExecutorPool) {
+
+ Y_UNIT_TEST(DecreaseIncreaseThreadsCount) {
+ const size_t msgCount = 1e4;
+ const size_t size = 4;
+ const size_t halfSize = size / 2;
+ TBasicExecutorPool* executorPool = new TBasicExecutorPool(0, size, 50);
+
+ auto setup = GetActorSystemSetup(executorPool);
+ TActorSystem actorSystem(setup);
+ actorSystem.Start();
+
+ executorPool->SetThreadCount(halfSize);
+ TTestSenderActor* actors[size];
+ TActorId actorIds[size];
+ for (size_t i = 0; i < size; ++i) {
+ actors[i] = new TTestSenderActor();
+ actorIds[i] = actorSystem.Register(actors[i]);
+ }
+
+ const int testCount = 2;
+
+ TExecutorPoolStats poolStats[testCount];
+ TVector<TExecutorThreadStats> statsCopy[testCount];
+
+ for (size_t testIdx = 0; testIdx < testCount; ++testIdx) {
+ for (size_t i = 0; i < size; ++i) {
+ actors[i]->Start(actors[i]->SelfId(), msgCount);
+ }
+ for (size_t i = 0; i < size; ++i) {
+ actorSystem.Send(actorIds[i], new TEvMsg());
+ }
+
+ Sleep(TDuration::MilliSeconds(100));
+
+ for (size_t i = 0; i < size; ++i) {
+ actors[i]->Stop();
+ }
+
+ executorPool->GetCurrentStats(poolStats[testIdx], statsCopy[testIdx]);
+ }
+
+ for (size_t i = 1; i <= halfSize; ++i) {
+ UNIT_ASSERT_UNEQUAL(statsCopy[0][i].ReceivedEvents, statsCopy[1][i].ReceivedEvents);
+ }
+
+ for (size_t i = halfSize + 1; i <= size; ++i) {
+ UNIT_ASSERT_EQUAL(statsCopy[0][i].ReceivedEvents, statsCopy[1][i].ReceivedEvents);
+ }
+
+ executorPool->SetThreadCount(size);
+
+ for (size_t testIdx = 0; testIdx < testCount; ++testIdx) {
+ for (size_t i = 0; i < size; ++i) {
+ actors[i]->Start(actors[i]->SelfId(), msgCount);
+ }
+ for (size_t i = 0; i < size; ++i) {
+ actorSystem.Send(actorIds[i], new TEvMsg());
+ }
+
+ Sleep(TDuration::MilliSeconds(100));
+
+ for (size_t i = 0; i < size; ++i) {
+ actors[i]->Stop();
+ }
+
+ executorPool->GetCurrentStats(poolStats[testIdx], statsCopy[testIdx]);
+ }
+
+ for (size_t i = 1; i <= size; ++i) {
+ UNIT_ASSERT_UNEQUAL(statsCopy[0][i].ReceivedEvents, statsCopy[1][i].ReceivedEvents);
+ }
+ }
+
+ Y_UNIT_TEST(ChangeCount) {
+ const size_t msgCount = 1e3;
+ const size_t size = 4;
+ const size_t halfSize = size / 2;
+ TBasicExecutorPool* executorPool = new TBasicExecutorPool(0, size, 50);
+
+ auto begin = TInstant::Now();
+
+ auto setup = GetActorSystemSetup(executorPool);
+ TActorSystem actorSystem(setup);
+ actorSystem.Start();
+ executorPool->SetThreadCount(halfSize);
+
+ TTestSenderActor* actors[size];
+ TActorId actorIds[size];
+ for (size_t i = 0; i < size; ++i) {
+ actors[i] = new TTestSenderActor();
+ actorIds[i] = actorSystem.Register(actors[i]);
+ }
+
+ for (size_t i = 0; i < size; ++i) {
+ actors[i]->Start(actorIds[i], msgCount);
+ }
+ for (size_t i = 0; i < size; ++i) {
+ actorSystem.Send(actorIds[i], new TEvMsg());
+ }
+
+ const i32 N = 6;
+ const i32 threadsCouns[N] = { 1, 3, 2, 3, 1, 4 };
+
+ ui64 counter = 0;
+
+ TTestSenderActor* changerActor = new TTestSenderActor([&]{
+ executorPool->SetThreadCount(threadsCouns[counter]);
+ counter++;
+ if (counter == N) {
+ counter = 0;
+ }
+ });
+ TActorId changerActorId = actorSystem.Register(changerActor);
+ changerActor->Start(changerActorId, msgCount);
+ actorSystem.Send(changerActorId, new TEvMsg());
+
+ while (true) {
+ size_t maxCounter = 0;
+ for (size_t i = 0; i < size; ++i) {
+ maxCounter = Max(maxCounter, actors[i]->GetCounter());
+ }
+
+ if (maxCounter == 0) {
+ break;
+ }
+
+ auto now = TInstant::Now();
+ UNIT_ASSERT_C(now - begin < TDuration::Seconds(5), "Max counter is " << maxCounter);
+
+ Sleep(TDuration::MilliSeconds(1));
+ }
+
+ changerActor->Stop();
+ }
+
+ Y_UNIT_TEST(CheckCompleteOne) {
+ const size_t size = 4;
+ const size_t msgCount = 1e4;
+ TBasicExecutorPool* executorPool = new TBasicExecutorPool(0, size, 50);
+
+ auto setup = GetActorSystemSetup(executorPool);
+ TActorSystem actorSystem(setup);
+ actorSystem.Start();
+
+ auto begin = TInstant::Now();
+
+ auto actor = new TTestSenderActor();
+ auto actorId = actorSystem.Register(actor);
+ actor->Start(actor->SelfId(), msgCount);
+ actorSystem.Send(actorId, new TEvMsg());
+
+ while (actor->GetCounter()) {
+ auto now = TInstant::Now();
+ UNIT_ASSERT_C(now - begin < TDuration::Seconds(5), "Counter is " << actor->GetCounter());
+
+ Sleep(TDuration::MilliSeconds(1));
+ }
+ }
+
+ Y_UNIT_TEST(CheckCompleteAll) {
+ const size_t size = 4;
+ const size_t msgCount = 1e4;
+ TBasicExecutorPool* executorPool = new TBasicExecutorPool(0, size, 50);
+
+ auto setup = GetActorSystemSetup(executorPool);
+ TActorSystem actorSystem(setup);
+ actorSystem.Start();
+
+ auto begin = TInstant::Now();
+
+ TTestSenderActor* actors[size];
+ TActorId actorIds[size];
+
+ for (size_t i = 0; i < size; ++i) {
+ actors[i] = new TTestSenderActor();
+ actorIds[i] = actorSystem.Register(actors[i]);
+ }
+ for (size_t i = 0; i < size; ++i) {
+ actors[i]->Start(actors[i]->SelfId(), msgCount);
+ }
+ for (size_t i = 0; i < size; ++i) {
+ actorSystem.Send(actorIds[i], new TEvMsg());
+ }
+
+
+ while (true) {
+ size_t maxCounter = 0;
+ for (size_t i = 0; i < size; ++i) {
+ maxCounter = Max(maxCounter, actors[i]->GetCounter());
+ }
+
+ if (maxCounter == 0) {
+ break;
+ }
+
+ auto now = TInstant::Now();
+ UNIT_ASSERT_C(now - begin < TDuration::Seconds(5), "Max counter is " << maxCounter);
+
+ Sleep(TDuration::MilliSeconds(1));
+ }
+ }
+
+ Y_UNIT_TEST(CheckCompleteOver) {
+ const size_t size = 4;
+ const size_t actorsCount = size * 2;
+ const size_t msgCount = 1e4;
+ TBasicExecutorPool* executorPool = new TBasicExecutorPool(0, size, 50);
+
+ auto setup = GetActorSystemSetup(executorPool);
+ TActorSystem actorSystem(setup);
+ actorSystem.Start();
+
+ auto begin = TInstant::Now();
+
+ TTestSenderActor* actors[actorsCount];
+ TActorId actorIds[actorsCount];
+
+ for (size_t i = 0; i < actorsCount; ++i) {
+ actors[i] = new TTestSenderActor();
+ actorIds[i] = actorSystem.Register(actors[i]);
+ }
+ for (size_t i = 0; i < actorsCount; ++i) {
+ actors[i]->Start(actors[i]->SelfId(), msgCount);
+ }
+ for (size_t i = 0; i < actorsCount; ++i) {
+ actorSystem.Send(actorIds[i], new TEvMsg());
+ }
+
+
+ while (true) {
+ size_t maxCounter = 0;
+ for (size_t i = 0; i < actorsCount; ++i) {
+ maxCounter = Max(maxCounter, actors[i]->GetCounter());
+ }
+
+ if (maxCounter == 0) {
+ break;
+ }
+
+ auto now = TInstant::Now();
+ UNIT_ASSERT_C(now - begin < TDuration::Seconds(5), "Max counter is " << maxCounter);
+
+ Sleep(TDuration::MilliSeconds(1));
+ }
+ }
+
+ Y_UNIT_TEST(CheckCompleteRoundRobinOver) {
+ const size_t size = 4;
+ const size_t actorsCount = size * 2;
+ const size_t msgCount = 1e2;
+ TBasicExecutorPool* executorPool = new TBasicExecutorPool(0, size, 50);
+
+ auto setup = GetActorSystemSetup(executorPool);
+ TActorSystem actorSystem(setup);
+ actorSystem.Start();
+
+ auto begin = TInstant::Now();
+
+ TTestSenderActor* actors[actorsCount];
+ TActorId actorIds[actorsCount];
+
+ for (size_t i = 0; i < actorsCount; ++i) {
+ actors[i] = new TTestSenderActor();
+ actorIds[i] = actorSystem.Register(actors[i]);
+ }
+ for (size_t i = 0; i < actorsCount; ++i) {
+ actors[i]->Start(actorIds[(i + 1) % actorsCount], msgCount);
+ }
+ for (size_t i = 0; i < actorsCount; ++i) {
+ actorSystem.Send(actorIds[i], new TEvMsg());
+ }
+
+ while (true) {
+ size_t maxCounter = 0;
+ for (size_t i = 0; i < actorsCount; ++i) {
+ maxCounter = Max(maxCounter, actors[i]->GetCounter());
+ }
+
+ if (maxCounter == 0) {
+ break;
+ }
+
+ auto now = TInstant::Now();
+ UNIT_ASSERT_C(now - begin < TDuration::Seconds(5), "Max counter is " << maxCounter);
+
+ Sleep(TDuration::MilliSeconds(1));
+ }
+ }
+
+ Y_UNIT_TEST(CheckStats) {
+ const size_t size = 4;
+ const size_t msgCount = 1e4;
+ TBasicExecutorPool* executorPool = new TBasicExecutorPool(0, size, 50);
+
+ auto setup = GetActorSystemSetup(executorPool);
+ TActorSystem actorSystem(setup);
+ actorSystem.Start();
+
+ auto begin = TInstant::Now();
+
+ auto actor = new TTestSenderActor();
+ auto actorId = actorSystem.Register(actor);
+ actor->Start(actor->SelfId(), msgCount);
+ actorSystem.Send(actorId, new TEvMsg());
+
+ while (actor->GetCounter()) {
+ auto now = TInstant::Now();
+ UNIT_ASSERT_C(now - begin < TDuration::Seconds(5), "Counter is " << actor->GetCounter());
+
+ Sleep(TDuration::MilliSeconds(1));
+ }
+
+ TVector<TExecutorThreadStats> stats;
+ TExecutorPoolStats poolStats;
+ actorSystem.GetPoolStats(0, poolStats, stats);
+ // Sum all per-thread counters into the 0th element
+ for (ui32 idx = 1; idx < stats.size(); ++idx) {
+ stats[0].Aggregate(stats[idx]);
+ }
+
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].SentEvents, msgCount - 1);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].ReceivedEvents, msgCount);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].PreemptedEvents, 0);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].NonDeliveredEvents, 0);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].EmptyMailboxActivation, 0);
+ //UNIT_ASSERT_VALUES_EQUAL(stats[0].CpuNs, 0); // depends on total duration of test, so undefined
+ UNIT_ASSERT(stats[0].ElapsedTicks > 0);
+ UNIT_ASSERT(stats[0].ParkedTicks > 0);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].BlockedTicks, 0);
+ UNIT_ASSERT(stats[0].ActivationTimeHistogram.TotalSamples >= msgCount / TBasicExecutorPoolConfig::DEFAULT_EVENTS_PER_MAILBOX);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].EventDeliveryTimeHistogram.TotalSamples, msgCount);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].EventProcessingCountHistogram.TotalSamples, msgCount);
+ UNIT_ASSERT(stats[0].EventProcessingTimeHistogram.TotalSamples > 0);
+ UNIT_ASSERT(stats[0].ElapsedTicksByActivity[0] > 0);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].ReceivedEventsByActivity[0], msgCount);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].ActorsAliveByActivity[0], 1);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].ScheduledEventsByActivity[0], 0);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].PoolActorRegistrations, 1);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].PoolDestroyedActors, 0);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].PoolAllocatedMailboxes, 4095); // one line
+ UNIT_ASSERT(stats[0].MailboxPushedOutByTime + stats[0].MailboxPushedOutByEventCount >= msgCount / TBasicExecutorPoolConfig::DEFAULT_EVENTS_PER_MAILBOX);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].MailboxPushedOutBySoftPreemption, 0);
+ }
+}
diff --git a/library/cpp/actors/core/executor_pool_io.cpp b/library/cpp/actors/core/executor_pool_io.cpp
new file mode 100644
index 0000000000..fb557ae6b0
--- /dev/null
+++ b/library/cpp/actors/core/executor_pool_io.cpp
@@ -0,0 +1,151 @@
+#include "executor_pool_io.h"
+#include "mailbox.h"
+#include <library/cpp/actors/util/affinity.h>
+#include <library/cpp/actors/util/datetime.h>
+
+namespace NActors {
+ TIOExecutorPool::TIOExecutorPool(ui32 poolId, ui32 threads, const TString& poolName, TAffinity* affinity, ui32 maxActivityType)
+ : TExecutorPoolBase(poolId, threads, affinity, maxActivityType)
+ , Threads(new TThreadCtx[threads])
+ , PoolName(poolName)
+ {}
+
+ TIOExecutorPool::TIOExecutorPool(const TIOExecutorPoolConfig& cfg)
+ : TIOExecutorPool(
+ cfg.PoolId,
+ cfg.Threads,
+ cfg.PoolName,
+ new TAffinity(cfg.Affinity),
+ cfg.MaxActivityType
+ )
+ {}
+
+ TIOExecutorPool::~TIOExecutorPool() {
+ Threads.Destroy();
+ while (ThreadQueue.Pop(0))
+ ;
+ }
+
+ ui32 TIOExecutorPool::GetReadyActivation(TWorkerContext& wctx, ui64 revolvingCounter) {
+ ui32 workerId = wctx.WorkerId;
+ Y_VERIFY_DEBUG(workerId < PoolThreads);
+
+ NHPTimer::STime elapsed = 0;
+ NHPTimer::STime parked = 0;
+ NHPTimer::STime hpstart = GetCycleCountFast();
+ NHPTimer::STime hpnow;
+
+ const TAtomic x = AtomicDecrement(Semaphore);
+ if (x < 0) {
+ TThreadCtx& threadCtx = Threads[workerId];
+ ThreadQueue.Push(workerId + 1, revolvingCounter);
+ hpnow = GetCycleCountFast();
+ elapsed += hpnow - hpstart;
+ if (threadCtx.Pad.Park())
+ return 0;
+ hpstart = GetCycleCountFast();
+ parked += hpstart - hpnow;
+ }
+
+ while (!RelaxedLoad(&StopFlag)) {
+ if (const ui32 activation = Activations.Pop(++revolvingCounter)) {
+ hpnow = GetCycleCountFast();
+ elapsed += hpnow - hpstart;
+ wctx.AddElapsedCycles(IActor::ACTOR_SYSTEM, elapsed);
+ if (parked > 0) {
+ wctx.AddParkedCycles(parked);
+ }
+ return activation;
+ }
+ SpinLockPause();
+ }
+
+ return 0;
+ }
+
+ void TIOExecutorPool::Schedule(TInstant deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) {
+ Schedule(deadline - ActorSystem->Timestamp(), ev, cookie, workerId);
+ }
+
+ void TIOExecutorPool::Schedule(TMonotonic deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) {
+ Y_UNUSED(workerId);
+
+ const auto current = ActorSystem->Monotonic();
+ if (deadline < current)
+ deadline = current;
+
+ TTicketLock::TGuard guard(&ScheduleLock);
+ ScheduleQueue->Writer.Push(deadline.MicroSeconds(), ev.Release(), cookie);
+ }
+
+ void TIOExecutorPool::Schedule(TDuration delta, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) {
+ Y_UNUSED(workerId);
+ const auto deadline = ActorSystem->Monotonic() + delta;
+
+ TTicketLock::TGuard guard(&ScheduleLock);
+ ScheduleQueue->Writer.Push(deadline.MicroSeconds(), ev.Release(), cookie);
+ }
+
+ void TIOExecutorPool::ScheduleActivationEx(ui32 activation, ui64 revolvingWriteCounter) {
+ Activations.Push(activation, revolvingWriteCounter);
+ const TAtomic x = AtomicIncrement(Semaphore);
+ if (x <= 0) {
+ for (;; ++revolvingWriteCounter) {
+ if (const ui32 x = ThreadQueue.Pop(revolvingWriteCounter)) {
+ const ui32 threadIdx = x - 1;
+ Threads[threadIdx].Pad.Unpark();
+ return;
+ }
+ SpinLockPause();
+ }
+ }
+ }
+
+ void TIOExecutorPool::Prepare(TActorSystem* actorSystem, NSchedulerQueue::TReader** scheduleReaders, ui32* scheduleSz) {
+ TAffinityGuard affinityGuard(Affinity());
+
+ ActorSystem = actorSystem;
+
+ ScheduleQueue.Reset(new NSchedulerQueue::TQueueType());
+
+ for (ui32 i = 0; i != PoolThreads; ++i) {
+ Threads[i].Thread.Reset(new TExecutorThread(i, 0, actorSystem, this, MailboxTable.Get(), PoolName));
+ }
+
+ *scheduleReaders = &ScheduleQueue->Reader;
+ *scheduleSz = 1;
+ }
+
+ void TIOExecutorPool::Start() {
+ TAffinityGuard affinityGuard(Affinity());
+
+ for (ui32 i = 0; i != PoolThreads; ++i)
+ Threads[i].Thread->Start();
+ }
+
+ void TIOExecutorPool::PrepareStop() {
+ AtomicStore(&StopFlag, true);
+ for (ui32 i = 0; i != PoolThreads; ++i)
+ Threads[i].Pad.Interrupt();
+ }
+
+ void TIOExecutorPool::Shutdown() {
+ for (ui32 i = 0; i != PoolThreads; ++i)
+ Threads[i].Thread->Join();
+ }
+
+ void TIOExecutorPool::GetCurrentStats(TExecutorPoolStats& /*poolStats*/, TVector<TExecutorThreadStats>& statsCopy) const {
+ statsCopy.resize(PoolThreads + 1);
+ // Save counters from the pool object
+ statsCopy[0] = TExecutorThreadStats();
+ statsCopy[0].Aggregate(Stats);
+ // Per-thread stats
+ for (size_t i = 0; i < PoolThreads; ++i) {
+ Threads[i].Thread->GetCurrentStats(statsCopy[i + 1]);
+ }
+ }
+
+ TString TIOExecutorPool::GetName() const {
+ return PoolName;
+ }
+}
diff --git a/library/cpp/actors/core/executor_pool_io.h b/library/cpp/actors/core/executor_pool_io.h
new file mode 100644
index 0000000000..e576d642a1
--- /dev/null
+++ b/library/cpp/actors/core/executor_pool_io.h
@@ -0,0 +1,49 @@
+#pragma once
+
+#include "actorsystem.h"
+#include "executor_thread.h"
+#include "scheduler_queue.h"
+#include "executor_pool_base.h"
+#include <library/cpp/actors/util/ticket_lock.h>
+#include <library/cpp/actors/util/unordered_cache.h>
+#include <library/cpp/actors/util/threadparkpad.h>
+#include <util/system/condvar.h>
+
+namespace NActors {
+ class TIOExecutorPool: public TExecutorPoolBase {
+ struct TThreadCtx {
+ TAutoPtr<TExecutorThread> Thread;
+ TThreadParkPad Pad;
+ };
+
+ TArrayHolder<TThreadCtx> Threads;
+ TUnorderedCache<ui32, 512, 4> ThreadQueue;
+
+ THolder<NSchedulerQueue::TQueueType> ScheduleQueue;
+ TTicketLock ScheduleLock;
+
+ const TString PoolName;
+
+ public:
+ TIOExecutorPool(ui32 poolId, ui32 threads, const TString& poolName = "", TAffinity* affinity = nullptr,
+ ui32 maxActivityType = 1);
+ explicit TIOExecutorPool(const TIOExecutorPoolConfig& cfg);
+ ~TIOExecutorPool();
+
+ ui32 GetReadyActivation(TWorkerContext& wctx, ui64 revolvingCounter) override;
+
+ void Schedule(TInstant deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) override;
+ void Schedule(TMonotonic deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) override;
+ void Schedule(TDuration delta, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) override;
+
+ void ScheduleActivationEx(ui32 activation, ui64 revolvingWriteCounter) override;
+
+ void Prepare(TActorSystem* actorSystem, NSchedulerQueue::TReader** scheduleReaders, ui32* scheduleSz) override;
+ void Start() override;
+ void PrepareStop() override;
+ void Shutdown() override;
+
+ void GetCurrentStats(TExecutorPoolStats& poolStats, TVector<TExecutorThreadStats>& statsCopy) const override;
+ TString GetName() const override;
+ };
+}
diff --git a/library/cpp/actors/core/executor_pool_united.cpp b/library/cpp/actors/core/executor_pool_united.cpp
new file mode 100644
index 0000000000..dac6245635
--- /dev/null
+++ b/library/cpp/actors/core/executor_pool_united.cpp
@@ -0,0 +1,1428 @@
+#include "executor_pool_united.h"
+
+#include "balancer.h"
+#include "cpu_state.h"
+#include "executor_thread.h"
+#include "probes.h"
+#include "mailbox.h"
+#include "scheduler_queue.h"
+#include <library/cpp/actors/util/affinity.h>
+#include <library/cpp/actors/util/datetime.h>
+#include <library/cpp/actors/util/futex.h>
+#include <library/cpp/actors/util/intrinsics.h>
+#include <library/cpp/actors/util/timerfd.h>
+
+#include <util/system/datetime.h>
+#include <util/system/hp_timer.h>
+
+#include <algorithm>
+
+namespace NActors {
+ LWTRACE_USING(ACTORLIB_PROVIDER);
+
+ struct TUnitedWorkers::TWorker: public TNonCopyable {
+ TAutoPtr<TExecutorThread> Thread;
+ volatile TThreadId ThreadId = UnknownThreadId;
+ NSchedulerQueue::TQueueType SchedulerQueue;
+ };
+
+ struct TUnitedWorkers::TPool: public TNonCopyable {
+ TAtomic Waiters = 0; // Number of idle cpus, waiting for activations in this pool
+ char Padding[64 - sizeof(TAtomic)];
+
+ TUnorderedCache<ui32, 512, 4> Activations; // MPMC-queue for mailbox activations
+ TAtomic Active = 0; // Number of mailboxes ready for execution or currently executing
+ TAtomic Tokens = 0; // Pending tokens (token is required for worker to start execution, guarantees concurrency limit and activation availability)
+ volatile bool StopFlag = false;
+
+ // Configuration
+ TPoolId PoolId;
+ TAtomicBase Concurrency; // Max concurrent workers running this pool
+ IExecutorPool* ExecutorPool;
+ TMailboxTable* MailboxTable;
+ ui64 TimePerMailboxTs;
+ ui32 EventsPerMailbox;
+
+ // Cpus this pool is allowed to run on
+ // Cpus are specified in wake order
+ TStackVec<TCpu*, 15> WakeOrderCpus;
+
+ ~TPool() {
+ while (Activations.Pop(0)) {}
+ }
+
+ void Stop() {
+ AtomicStore(&StopFlag, true);
+ }
+
+ bool IsUnited() const {
+ return WakeOrderCpus.size();
+ }
+
+ // Add activation of newly scheduled mailbox. Returns generated token (unless concurrency is exceeded)
+ bool PushActivation(ui32 activation, ui64 revolvingCounter) {
+ Activations.Push(activation, revolvingCounter);
+ TAtomicBase active = AtomicIncrement(Active);
+ if (active <= Concurrency) { // token generated
+ AtomicIncrement(Tokens);
+ return true;
+ }
+ return false;
+ }
+
+ template <bool Relaxed>
+ static bool TryAcquireTokenImpl(TAtomic* tokens) {
+ while (true) {
+ TAtomicBase value;
+ if constexpr (Relaxed) {
+ value = RelaxedLoad(tokens);
+ } else {
+ value = AtomicLoad(tokens);
+ }
+ if (value > 0) {
+ if (AtomicCas(tokens, value - 1, value)) {
+ return true; // token acquired
+ }
+ } else {
+ return false; // no more tokens
+ }
+ }
+ }
+
+ // Try acquire pending token. Must be done before execution
+ bool TryAcquireToken() {
+ return TryAcquireTokenImpl<false>(&Tokens);
+ }
+
+ // Try acquire pending token. Must be done before execution
+ bool TryAcquireTokenRelaxed() {
+ return TryAcquireTokenImpl<true>(&Tokens);
+ }
+
+ // Get activation. Requires acquired token.
+ void BeginExecution(ui32& activation, ui64 revolvingCounter) {
+ while (!RelaxedLoad(&StopFlag)) {
+ if (activation = Activations.Pop(++revolvingCounter)) {
+ return;
+ }
+ SpinLockPause();
+ }
+ activation = 0; // should stop
+ }
+
+ // End currently active execution and start new one if token is available.
+ // Reuses token if it's not destroyed.
+ // Returned `true` means successful switch, `activation` is filled.
+ // Returned `false` means execution has ended, no need to call StopExecution()
+ bool NextExecution(ui32& activation, ui64 revolvingCounter) {
+ if (AtomicDecrement(Active) >= Concurrency) { // reuse just released token
+ BeginExecution(activation, revolvingCounter);
+ return true;
+ } else if (TryAcquireToken()) { // another token acquired
+ BeginExecution(activation, revolvingCounter);
+ return true;
+ }
+ return false; // no more tokens available
+ }
+
+ // Stop active execution. Returns released token (unless it is destroyed)
+ bool StopExecution() {
+ TAtomicBase active = AtomicDecrement(Active);
+ if (active >= Concurrency) { // token released
+ AtomicIncrement(Tokens);
+ return true;
+ }
+ return false; // token destroyed
+ }
+
+ // Switch worker context into this pool
+ void Switch(TWorkerContext& wctx, ui64 softDeadlineTs, TExecutorThreadStats& stats) {
+ wctx.Switch(ExecutorPool, MailboxTable, TimePerMailboxTs, EventsPerMailbox, softDeadlineTs, &stats);
+ }
+ };
+
+ class TPoolScheduler {
+ class TSchedulable {
+ // Lower PoolBits store PoolId
+ // All other higher bits store virtual runtime in cycles
+ using TValue = ui64;
+ TValue Value;
+
+ static constexpr ui64 PoolIdMask = ui64((1ull << PoolBits) - 1);
+ static constexpr ui64 VRunTsMask = ~PoolIdMask;
+
+ public:
+ explicit TSchedulable(TPoolId poolId = MaxPools, ui64 vrunts = 0)
+ : Value((poolId & PoolIdMask) | (vrunts & VRunTsMask))
+ {}
+
+ TPoolId GetPoolId() const {
+ return Value & PoolIdMask;
+ }
+
+ ui64 GetVRunTs() const {
+ // Do not truncate pool id
+ // NOTE: it decrease accuracy, but improves performance
+ return Value;
+ }
+
+ ui64 GetPreciseVRunTs() const {
+ return Value & VRunTsMask;
+ }
+
+ void SetVRunTs(ui64 vrunts) {
+ Value = (Value & PoolIdMask) | (vrunts & VRunTsMask);
+ }
+
+ void Account(ui64 base, ui64 ts) {
+ // Add at least minimum amount to change Value
+ SetVRunTs(base + Max(ts, PoolIdMask + 1));
+ }
+ };
+
+ // For min-heap of Items
+ struct TCmp {
+ bool operator()(TSchedulable lhs, TSchedulable rhs) const {
+ return lhs.GetVRunTs() > rhs.GetVRunTs();
+ }
+ };
+
+ TPoolId Size = 0; // total number of pools on this cpu
+ TPoolId Current = 0; // index of current pool in `Items`
+
+ // At the beginning `Current` items are orginized as binary min-heap -- ready to be scheduled
+ // The rest `Size - Current` items are unordered (required to keep track of last vrunts)
+ TSchedulable Items[MaxPools]; // virtual runtime in cycles for each pool
+ ui64 MinVRunTs = 0; // virtual runtime used by waking pools (system's vrunts)
+ ui64 Ts = 0; // real timestamp of current execution start (for accounting)
+
+ // Maps PoolId into it's inverse weight
+ ui64 InvWeights[MaxPools];
+ static constexpr ui64 VRunTsOverflow = ui64(1ull << 62ull) / MaxPoolWeight;
+
+ public:
+ void AddPool(TPoolId pool, TPoolWeight weight) {
+ Items[Size] = TSchedulable(pool, MinVRunTs);
+ Size++;
+ InvWeights[pool] = MaxPoolWeight / std::clamp(weight ? weight : DefPoolWeight, MinPoolWeight, MaxPoolWeight);
+ }
+
+ // Iterate over pools in scheduling order
+ // should be used in construction:
+ // for (TPoolId pool = Begin(); pool != End(); pool = Next())
+ TPoolId Begin() {
+ // Wrap vruntime around to avoid overflow, if required
+ if (Y_UNLIKELY(MinVRunTs >= VRunTsOverflow)) {
+ for (TPoolId i = 0; i < Size; i++) {
+ ui64 ts = Items[i].GetPreciseVRunTs();
+ Items[i].SetVRunTs(ts >= VRunTsOverflow ? ts - VRunTsOverflow : 0);
+ }
+ MinVRunTs -= VRunTsOverflow;
+ }
+ Current = Size;
+ std::make_heap(Items, Items + Current, TCmp());
+ return Next();
+ }
+
+ constexpr TPoolId End() const {
+ return MaxPools;
+ }
+
+ TPoolId Next() {
+ if (Current > 0) {
+ std::pop_heap(Items, Items + Current, TCmp());
+ Current--;
+ return CurrentPool();
+ } else {
+ return End();
+ }
+ }
+
+ // Scheduling was successful, we are going to run CurrentPool()
+ void Scheduled() {
+ MinVRunTs = Max(MinVRunTs, Items[Current].GetPreciseVRunTs());
+ // NOTE: Ts is propagated on Account() to avoid gaps
+ }
+
+ // Schedule specific pool that woke up cpu after idle
+ void ScheduledAfterIdle(TPoolId pool, ui64 ts) {
+ if (Y_UNLIKELY(ts < Ts)) { // anomaly: time goes backwards (e.g. rdtsc is reset to zero on cpu reset)
+ Ts = ts; // just skip anomalous time slice
+ return;
+ }
+ MinVRunTs += (ts - Ts) * (MaxPoolWeight / DefPoolWeight); // propagate system's vrunts to blur difference between pools
+ Ts = ts; // propagate time w/o accounting to any pool
+
+ // Set specified pool as current, it requires scan
+ for (Current = 0; Current < Size && pool != Items[Current].GetPoolId(); Current++) {}
+ Y_VERIFY(Current < Size);
+ }
+
+ // Account currently running pool till now (ts)
+ void Account(ui64 ts) {
+ // 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);
+ Items[Current].Account(MinVRunTs, (ts - Ts) * InvWeights[pool]);
+ }
+ Ts = ts; // propagate time
+ }
+
+ TPoolId CurrentPool() const {
+ return Items[Current].GetPoolId();
+ }
+ };
+
+ // Cyclic array of timers for idle workers to wait for hard preemption on
+ struct TIdleQueue: public TNonCopyable {
+ TArrayHolder<TTimerFd> Timers;
+ size_t Size;
+ TAtomic EnqueueCounter = 0;
+ TAtomic DequeueCounter = 0;
+
+ explicit TIdleQueue(size_t size)
+ : Timers(new TTimerFd[size])
+ , Size(size)
+ {}
+
+ void Stop() {
+ for (size_t i = 0; i < Size; i++) {
+ Timers[i].Wake();
+ }
+ }
+
+ // Returns timer which new idle-worker should wait for
+ TTimerFd* Enqueue() {
+ return &Timers[AtomicGetAndIncrement(EnqueueCounter) % Size];
+ }
+
+ // Returns timer that hard preemption should trigger to wake idle-worker
+ TTimerFd* Dequeue() {
+ return &Timers[AtomicGetAndIncrement(DequeueCounter) % Size];
+ }
+ };
+
+ // Base class for cpu-local managers that help workers on single cpu to cooperate
+ struct TCpuLocalManager: public TThrRefBase {
+ TUnitedWorkers* United;
+
+ explicit TCpuLocalManager(TUnitedWorkers* united)
+ : United(united)
+ {}
+
+ virtual TWorkerId WorkerCount() const = 0;
+ virtual void AddWorker(TWorkerId workerId) = 0;
+ virtual void Stop() = 0;
+ };
+
+ // Represents cpu with single associated worker that is able to execute any pool.
+ // It always executes pool assigned by balancer and switch pool only if assigned pool has changed
+ struct TAssignedCpu: public TCpuLocalManager {
+ bool Started = false;
+
+ TAssignedCpu(TUnitedWorkers* united)
+ : TCpuLocalManager(united)
+ {}
+
+ TWorkerId WorkerCount() const override {
+ return 1;
+ }
+
+ void AddWorker(TWorkerId workerId) override {
+ Y_UNUSED(workerId);
+ }
+
+ ui32 GetReadyActivation(TWorkerContext& wctx, ui64 revolvingCounter) {
+ ui32 activation;
+ if (Y_UNLIKELY(!Started)) {
+ Started = true;
+ } else if (Y_UNLIKELY(United->IsPoolReassigned(wctx))) {
+ United->StopExecution(wctx.PoolId); // stop current execution and switch pool if reassigned
+ } else if (United->NextExecution(wctx.PoolId, activation, revolvingCounter)) {
+ return activation; // another activation from currently executing pool (or 0 if stopped)
+ }
+
+ // Switch to another pool, it blocks until token is acquired
+ if (Y_UNLIKELY(!SwitchPool(wctx))) {
+ return 0; // stopped
+ }
+ United->SwitchPool(wctx, 0);
+ United->BeginExecution(wctx.PoolId, activation, revolvingCounter);
+ return activation;
+ }
+
+ void Stop() override {
+ }
+
+ private:
+ // Sets next pool to run, and acquires token, blocks if there are no tokens
+ bool SwitchPool(TWorkerContext& wctx) {
+ if (Y_UNLIKELY(United->IsStopped())) {
+ return false;
+ }
+
+ // Run balancer (if it's time to)
+ United->Balance();
+
+ // Select pool to execute
+ wctx.PoolId = United->AssignedPool(wctx);
+ Y_VERIFY(wctx.PoolId != CpuShared);
+ if (United->TryAcquireToken(wctx.PoolId)) {
+ return true;
+ }
+
+ // No more work -- wait for activations (spinning, then blocked)
+ wctx.PoolId = United->Idle(wctx.PoolId, wctx);
+
+ // Wakeup or stop occured
+ if (Y_UNLIKELY(wctx.PoolId == CpuStopped)) {
+ return false;
+ }
+ return true; // United->Idle() has already acquired token
+ }
+ };
+
+ // Lock-free data structure that help workers on single cpu to discover their state and do hard preemptions
+ struct TSharedCpu: public TCpuLocalManager {
+ // Current lease
+ volatile TLease::TValue CurrentLease;
+ char Padding1[64 - sizeof(TLease)];
+
+ // Slow pools
+ // the highest bit: 1=wait-for-slow-workers mode 0=else
+ // any lower bit (poolId is bit position): 1=pool-is-slow 0=pool-is-fast
+ volatile TPoolsMask SlowPoolsMask = 0;
+ char Padding2[64 - sizeof(TPoolsMask)];
+
+ // Must be accessed under never expiring lease to avoid races
+ TPoolScheduler PoolSched;
+ TWorkerId FastWorker = MaxWorkers;
+ TTimerFd* PreemptionTimer = nullptr;
+ ui64 HardPreemptionTs = 0;
+ bool Started = false;
+
+ TIdleQueue IdleQueue;
+
+ struct TConfig {
+ const TCpuId CpuId;
+ const TWorkerId Workers;
+ ui64 SoftLimitTs;
+ ui64 HardLimitTs;
+ ui64 EventLimitTs;
+ ui64 LimitPrecisionTs;
+ const int IdleWorkerPriority;
+ const int FastWorkerPriority;
+ const bool NoRealtime;
+ const bool NoAffinity;
+ const TCpuAllocation CpuAlloc;
+
+ TConfig(const TCpuAllocation& allocation, const TUnitedWorkersConfig& united)
+ : CpuId(allocation.CpuId)
+ , Workers(allocation.AllowedPools.size() + 1)
+ , SoftLimitTs(Us2Ts(united.PoolLimitUs))
+ , HardLimitTs(Us2Ts(united.PoolLimitUs + united.EventLimitUs))
+ , EventLimitTs(Us2Ts(united.EventLimitUs))
+ , LimitPrecisionTs(Us2Ts(united.LimitPrecisionUs))
+ , IdleWorkerPriority(std::clamp<ui64>(united.IdleWorkerPriority ? united.IdleWorkerPriority : 20, 1, 99))
+ , FastWorkerPriority(std::clamp<ui64>(united.FastWorkerPriority ? united.FastWorkerPriority : 10, 1, IdleWorkerPriority - 1))
+ , NoRealtime(united.NoRealtime)
+ , NoAffinity(united.NoAffinity)
+ , CpuAlloc(allocation)
+ {}
+ };
+
+ TConfig Config;
+ TVector<TWorkerId> Workers;
+
+ TSharedCpu(const TConfig& cfg, TUnitedWorkers* united)
+ : TCpuLocalManager(united)
+ , IdleQueue(cfg.Workers)
+ , Config(cfg)
+ {
+ for (const auto& pa : Config.CpuAlloc.AllowedPools) {
+ PoolSched.AddPool(pa.PoolId, pa.Weight);
+ }
+ }
+
+ TWorkerId WorkerCount() const override {
+ return Config.Workers;
+ }
+
+ void AddWorker(TWorkerId workerId) override {
+ if (Workers.empty()) {
+ // Grant lease to the first worker
+ AtomicStore(&CurrentLease, TLease(workerId, NeverExpire).Value);
+ }
+ Workers.push_back(workerId);
+ }
+
+ ui32 GetReadyActivation(TWorkerContext& wctx, ui64 revolvingCounter) {
+ ui32 activation;
+ if (!wctx.Lease.IsNeverExpiring()) {
+ if (wctx.SoftDeadlineTs < GetCycleCountFast()) { // stop if lease has expired or is near to be expired
+ United->StopExecution(wctx.PoolId);
+ } else if (United->NextExecution(wctx.PoolId, activation, revolvingCounter)) {
+ return activation; // another activation from currently executing pool (or 0 if stopped)
+ }
+ }
+
+ // Switch to another pool, it blocks until token is acquired
+ if (Y_UNLIKELY(!SwitchPool(wctx))) {
+ return 0; // stopped
+ }
+ United->BeginExecution(wctx.PoolId, activation, revolvingCounter);
+ return activation;
+ }
+
+ void Stop() override {
+ IdleQueue.Stop();
+ }
+
+ private:
+ enum EPriority {
+ IdlePriority, // highest (real-time, Config.IdleWorkerPriority)
+ FastPriority, // normal (real-time, Config.FastWorkerPriority)
+ SlowPriority, // lowest (not real-time)
+ };
+
+ enum EWorkerAction {
+ // Fast-worker
+ ExecuteFast,
+ WaitForSlow,
+
+ // Slow-worker
+ BecameIdle,
+ WakeFast,
+
+ // Idle-worker
+ BecameFast,
+ Standby,
+
+ // Common
+ Stopped,
+ };
+
+ // Thread-safe; should be called from worker
+ // Blocks for idle-workers; sets lease and next pool to run
+ bool SwitchPool(TWorkerContext& wctx) {
+ TTimerFd* idleTimer = nullptr;
+ while (true) {
+ if (DisablePreemptionAndTryExtend(wctx.Lease)) { // if fast-worker
+ if (Y_UNLIKELY(!Started)) {
+ SetPriority(0, FastPriority);
+ Started = true;
+ }
+ while (true) {
+ switch (FastWorkerAction(wctx)) {
+ case ExecuteFast:
+ United->SwitchPool(wctx, wctx.Lease.GetPreciseExpireTs() - Config.EventLimitTs);
+ EnablePreemptionAndGrant(wctx.Lease);
+ return true;
+ case WaitForSlow:
+ FastWorkerSleep(GetCycleCountFast() + Config.SoftLimitTs);
+ break;
+ case Stopped: return false;
+ default: Y_FAIL();
+ }
+ }
+ } else if (wctx.Lease.IsNeverExpiring()) { // if idle-worker
+ switch (IdleWorkerAction(idleTimer, wctx.Lease.GetWorkerId())) {
+ case BecameFast:
+ SetPriority(0, FastPriority);
+ break; // try acquire new lease
+ case Standby:
+ if (!idleTimer) {
+ idleTimer = IdleQueue.Enqueue();
+ }
+ SetPriority(0, IdlePriority);
+ idleTimer->Wait();
+ break;
+ case Stopped: return false;
+ default: Y_FAIL();
+ }
+ } else { // lease has expired and hard preemption occured, so we are executing in a slow-worker
+ wctx.IncrementPreemptedEvents();
+ switch (SlowWorkerAction(wctx.PoolId)) {
+ case WakeFast:
+ WakeFastWorker();
+ [[fallthrough]]; // no break; pass through
+ case BecameIdle:
+ wctx.Lease = wctx.Lease.NeverExpire();
+ wctx.PoolId = MaxPools;
+ idleTimer = nullptr;
+ break;
+ case Stopped: return false;
+ default: Y_FAIL();
+ }
+ }
+ }
+ }
+
+ enum ETryRunPool {
+ RunFastPool,
+ RunSlowPool,
+ NoTokens,
+ };
+
+ ETryRunPool TryRun(TPoolId pool) {
+ while (true) {
+ // updates WaitPoolsFlag in SlowPoolsMask according to scheduled pool slowness
+ TPoolsMask slow = AtomicLoad(&SlowPoolsMask);
+ if ((1ull << pool) & slow) { // we are about to execute slow pool (fast-worker will just wait, token is NOT required)
+ if (slow & WaitPoolsFlag) {
+ return RunSlowPool; // wait flag is already set
+ } else {
+ if (AtomicCas(&SlowPoolsMask, slow | WaitPoolsFlag, slow)) { // try set wait flag
+ return RunSlowPool; // wait flag has been successfully set
+ }
+ }
+ } else { // we are about to execute fast pool, token required
+ if (slow & WaitPoolsFlag) { // reset wait flag if required
+ if (AtomicCas(&SlowPoolsMask, slow & ~WaitPoolsFlag, slow)) { // try reset wait flag
+ return United->TryAcquireToken(pool) ? RunFastPool : NoTokens; // wait flag has been successfully reset
+ }
+ } else {
+ return United->TryAcquireToken(pool) ? RunFastPool : NoTokens; // wait flag is already reset
+ }
+ }
+ }
+ }
+
+ EWorkerAction FastWorkerAction(TWorkerContext& wctx) {
+ if (Y_UNLIKELY(United->IsStopped())) {
+ return Stopped;
+ }
+
+ // Account current pool
+ ui64 ts = GetCycleCountFast();
+ PoolSched.Account(ts);
+
+ // Select next pool to execute
+ for (wctx.PoolId = PoolSched.Begin(); wctx.PoolId != PoolSched.End(); wctx.PoolId = PoolSched.Next()) {
+ switch (TryRun(wctx.PoolId)) {
+ case RunFastPool:
+ PoolSched.Scheduled();
+ wctx.Lease = PostponePreemption(wctx.Lease.GetWorkerId(), ts);
+ return ExecuteFast;
+ case RunSlowPool:
+ PoolSched.Scheduled();
+ ResetPreemption(wctx.Lease.GetWorkerId(), ts); // there is no point in preemption during wait
+ return WaitForSlow;
+ case NoTokens: // concurrency limit reached, or no more work in pool
+ break; // just try next pool (if any)
+ }
+ }
+
+ // No more work, no slow-workers -- wait for activations (active, then blocked)
+ wctx.PoolId = United->Idle(CpuShared, wctx);
+
+ // Wakeup or stop occured
+ if (Y_UNLIKELY(wctx.PoolId == CpuStopped)) {
+ return Stopped;
+ }
+ ts = GetCycleCountFast();
+ PoolSched.ScheduledAfterIdle(wctx.PoolId, ts);
+ wctx.Lease = PostponePreemption(wctx.Lease.GetWorkerId(), ts);
+ return ExecuteFast; // United->Idle() has already acquired token
+ }
+
+ EWorkerAction IdleWorkerAction(TTimerFd* idleTimer, TWorkerId workerId) {
+ if (Y_UNLIKELY(United->IsStopped())) {
+ return Stopped;
+ }
+ if (!idleTimer) { // either worker start or became idle -- hard preemption is not required
+ return Standby;
+ }
+
+ TLease lease = TLease(AtomicLoad(&CurrentLease));
+ ui64 ts = GetCycleCountFast();
+ if (lease.GetExpireTs() < ts) { // current lease has expired
+ if (TryBeginHardPreemption(lease)) {
+ SetPoolIsSlowFlag(PoolSched.CurrentPool());
+ TWorkerId preempted = lease.GetWorkerId();
+ SetPriority(United->GetWorkerThreadId(preempted), SlowPriority);
+ LWPROBE(HardPreemption, Config.CpuId, PoolSched.CurrentPool(), preempted, workerId);
+ EndHardPreemption(workerId);
+ return BecameFast;
+ } else {
+ // Lease has been changed just now, no way we need preemption right now, so no retry needed
+ return Standby;
+ }
+ } else {
+ // Lease has not expired yet (maybe never expiring lease)
+ return Standby;
+ }
+ }
+
+ EWorkerAction SlowWorkerAction(TPoolId pool) {
+ if (Y_UNLIKELY(United->IsStopped())) {
+ return Stopped;
+ }
+ while (true) {
+ TPoolsMask slow = AtomicLoad(&SlowPoolsMask);
+ if (slow & (1ull << pool)) {
+ if (slow == (1ull << pool) & WaitPoolsFlag) { // the last slow pool is going to became fast
+ if (AtomicCas(&SlowPoolsMask, 0, slow)) { // reset both pool-is-slow flag and WaitPoolsFlag
+ return WakeFast;
+ }
+ } else { // there are (a) several slow-worker or (b) one slow-worker w/o waiting fast-worker
+ if (AtomicCas(&SlowPoolsMask, slow & ~(1ull << pool), slow)) { // reset pool-is-slow flag
+ return BecameIdle;
+ }
+ }
+ } else {
+ // SlowWorkerAction has been called between TryBeginHardPreemption and SetPoolIsSlowFlag
+ // flag for this pool is not set yet, but we can be sure pool is slow:
+ // - because SlowWorkerAction has been called;
+ // - this mean lease has expired and hard preemption occured.
+ // So just wait other worker to call SetPoolIsSlowFlag
+ LWPROBE(SlowWorkerActionRace, Config.CpuId, pool, slow);
+ }
+ }
+ }
+
+ void SetPoolIsSlowFlag(TPoolId pool) {
+ while (true) {
+ TPoolsMask slow = AtomicLoad(&SlowPoolsMask);
+ if ((slow & (1ull << pool)) == 0) { // if pool is fast
+ if (AtomicCas(&SlowPoolsMask, slow | (1ull << pool), slow)) { // set pool-is-slow flag
+ return;
+ }
+ } else {
+ Y_FAIL("two slow-workers executing the same pool on the same core");
+ return; // pool is already slow
+ }
+ }
+ }
+
+ bool TryBeginHardPreemption(TLease lease) {
+ return AtomicCas(&CurrentLease, HardPreemptionLease, lease);
+ }
+
+ void EndHardPreemption(TWorkerId to) {
+ ATOMIC_COMPILER_BARRIER();
+ if (!AtomicCas(&CurrentLease, TLease(to, NeverExpire), HardPreemptionLease)) {
+ Y_FAIL("hard preemption failed");
+ }
+ }
+
+ bool DisablePreemptionAndTryExtend(TLease lease) {
+ return AtomicCas(&CurrentLease, lease.NeverExpire(), lease);
+ }
+
+ void EnablePreemptionAndGrant(TLease lease) {
+ ATOMIC_COMPILER_BARRIER();
+ if (!AtomicCas(&CurrentLease, lease, lease.NeverExpire())) {
+ Y_FAIL("lease grant failed");
+ }
+ }
+
+ void FastWorkerSleep(ui64 deadlineTs) {
+ while (true) {
+ TPoolsMask slow = AtomicLoad(&SlowPoolsMask);
+ if ((slow & WaitPoolsFlag) == 0) {
+ return; // woken by WakeFast action
+ }
+ ui64 ts = GetCycleCountFast();
+ if (deadlineTs <= ts) {
+ if (AtomicCas(&SlowPoolsMask, slow & ~WaitPoolsFlag, slow)) { // try reset wait flag
+ return; // wait flag has been successfully reset after timeout
+ }
+ } else { // should wait
+ ui64 timeoutNs = Ts2Ns(deadlineTs - ts);
+#ifdef _linux_
+ timespec timeout;
+ timeout.tv_sec = timeoutNs / 1'000'000'000;
+ timeout.tv_nsec = timeoutNs % 1'000'000'000;
+ SysFutex(FastWorkerFutex(), FUTEX_WAIT_PRIVATE, FastWorkerFutexValue(slow), &timeout, nullptr, 0);
+#else
+ NanoSleep(timeoutNs); // non-linux wake is not supported, cpu will go idle on slow -> fast switch
+#endif
+ }
+ }
+ }
+
+ void WakeFastWorker() {
+#ifdef _linux_
+ SysFutex(FastWorkerFutex(), FUTEX_WAKE_PRIVATE, 1, nullptr, nullptr, 0);
+#endif
+ }
+
+#ifdef _linux_
+ ui32* FastWorkerFutex() {
+ // Actually we wait on one highest bit, but futex value size is 4 bytes on all platforms
+ static_assert(sizeof(TPoolsMask) >= 4, "cannot be used as futex value on linux");
+ return (ui32*)&SlowPoolsMask + 1; // higher 32 bits (little endian assumed)
+ }
+
+ ui32 FastWorkerFutexValue(TPoolsMask slow) {
+ return ui32(slow >> 32); // higher 32 bits
+ }
+#endif
+
+ void SetPriority(TThreadId tid, EPriority priority) {
+ if (Config.NoRealtime) {
+ return;
+ }
+#ifdef _linux_
+ int policy;
+ struct sched_param param;
+ switch (priority) {
+ case IdlePriority:
+ policy = SCHED_FIFO;
+ param.sched_priority = Config.IdleWorkerPriority;
+ break;
+ case FastPriority:
+ policy = SCHED_FIFO;
+ param.sched_priority = Config.FastWorkerPriority;
+ break;
+ case SlowPriority:
+ policy = SCHED_OTHER;
+ param.sched_priority = 0;
+ break;
+ }
+ int ret = sched_setscheduler(tid, policy, &param);
+ switch (ret) {
+ case 0: return;
+ case EINVAL:
+ Y_FAIL("sched_setscheduler(%" PRIu64 ", %d, %d) -> EINVAL", tid, policy, param.sched_priority);
+ case EPERM:
+ // Requirements:
+ // * CAP_SYS_NICE capability to run real-time processes and set cpu affinity.
+ // Either run under root or set application capabilities:
+ // sudo setcap cap_sys_nice=eip BINARY
+ // * Non-zero rt-runtime (in case cgroups are used).
+ // Either (a) disable global limit on RT processes bandwidth:
+ // sudo sysctl -w kernel.sched_rt_runtime_us=-1
+ // Or (b) set non-zero rt-runtime for your cgroup:
+ // echo -1 > /sys/fs/cgroup/cpu/[cgroup]/cpu.rt_runtime_us
+ // (also set the same value for every parent cgroup)
+ // https://www.kernel.org/doc/Documentation/scheduler/sched-rt-group.txt
+ Y_FAIL("sched_setscheduler(%" PRIu64 ", %d, %d) -> EPERM", tid, policy, param.sched_priority);
+ case ESRCH:
+ Y_FAIL("sched_setscheduler(%" PRIu64 ", %d, %d) -> ESRCH", tid, policy, param.sched_priority);
+ default:
+ Y_FAIL("sched_setscheduler(%" PRIu64 ", %d, %d) -> %d", tid, policy, param.sched_priority, ret);
+ }
+#else
+ Y_UNUSED(tid);
+ Y_UNUSED(priority);
+#endif
+ }
+
+ void ResetPreemption(TWorkerId fastWorkerId, ui64 ts) {
+ if (Y_UNLIKELY(!PreemptionTimer)) {
+ return;
+ }
+ if (FastWorker == fastWorkerId && HardPreemptionTs > 0) {
+ PreemptionTimer->Reset();
+ LWPROBE(ResetPreemptionTimer, Config.CpuId, FastWorker, PreemptionTimer->Fd, Ts2Ms(ts), Ts2Ms(HardPreemptionTs));
+ HardPreemptionTs = 0;
+ }
+ }
+
+ TLease PostponePreemption(TWorkerId fastWorkerId, ui64 ts) {
+ // Select new timer after hard preemption
+ if (FastWorker != fastWorkerId) {
+ FastWorker = fastWorkerId;
+ PreemptionTimer = IdleQueue.Dequeue();
+ HardPreemptionTs = 0;
+ }
+
+ ui64 hardPreemptionTs = ts + Config.HardLimitTs;
+ if (hardPreemptionTs > HardPreemptionTs) {
+ // Reset timer (at most once in TickIntervalTs, sacrifice precision)
+ HardPreemptionTs = hardPreemptionTs + Config.LimitPrecisionTs;
+ PreemptionTimer->Set(HardPreemptionTs);
+ LWPROBE(SetPreemptionTimer, Config.CpuId, FastWorker, PreemptionTimer->Fd, Ts2Ms(ts), Ts2Ms(HardPreemptionTs));
+ }
+
+ return TLease(fastWorkerId, hardPreemptionTs);
+ }
+ };
+
+ // Proxy for start and switching TUnitedExecutorPool-s on single cpu via GetReadyActivation()
+ // (does not implement any other method in IExecutorPool)
+ class TCpuExecutorPool: public IExecutorPool {
+ const TString Name;
+
+ public:
+ explicit TCpuExecutorPool(const TString& name)
+ : IExecutorPool(MaxPools)
+ , Name(name)
+ {}
+
+ TString GetName() const override {
+ return Name;
+ }
+
+ void SetRealTimeMode() const override {
+ // derived classes controls rt-priority - do nothing
+ }
+
+ // Should never be called
+ void ReclaimMailbox(TMailboxType::EType, ui32, TWorkerId, ui64) override { Y_FAIL(); }
+ void Schedule(TInstant, TAutoPtr<IEventHandle>, ISchedulerCookie*, TWorkerId) override { Y_FAIL(); }
+ void Schedule(TMonotonic, TAutoPtr<IEventHandle>, ISchedulerCookie*, TWorkerId) override { Y_FAIL(); }
+ void Schedule(TDuration, TAutoPtr<IEventHandle>, ISchedulerCookie*, TWorkerId) override { Y_FAIL(); }
+ bool Send(TAutoPtr<IEventHandle>&) override { Y_FAIL(); }
+ void ScheduleActivation(ui32) override { Y_FAIL(); }
+ void ScheduleActivationEx(ui32, ui64) override { Y_FAIL(); }
+ TActorId Register(IActor*, TMailboxType::EType, ui64, const TActorId&) override { Y_FAIL(); }
+ TActorId Register(IActor*, TMailboxHeader*, ui32, const TActorId&) override { Y_FAIL(); }
+ void Prepare(TActorSystem*, NSchedulerQueue::TReader**, ui32*) override { Y_FAIL(); }
+ void Start() override { Y_FAIL(); }
+ void PrepareStop() override { Y_FAIL(); }
+ void Shutdown() override { Y_FAIL(); }
+ bool Cleanup() override { Y_FAIL(); }
+ };
+
+ // Proxy executor pool working with cpu-local scheduler (aka actorsystem 2.0)
+ class TSharedCpuExecutorPool: public TCpuExecutorPool {
+ TSharedCpu* Local;
+ TIntrusivePtr<TAffinity> SingleCpuAffinity; // no migration support yet
+ public:
+ explicit TSharedCpuExecutorPool(TSharedCpu* local, const TUnitedWorkersConfig& config)
+ : TCpuExecutorPool("u-" + ToString(local->Config.CpuId))
+ , Local(local)
+ , SingleCpuAffinity(config.NoAffinity ? nullptr : new TAffinity(TCpuMask(local->Config.CpuId)))
+ {}
+
+ TAffinity* Affinity() const override {
+ return SingleCpuAffinity.Get();
+ }
+
+ ui32 GetReadyActivation(TWorkerContext& wctx, ui64 revolvingCounter) override {
+ return Local->GetReadyActivation(wctx, revolvingCounter);
+ }
+ };
+
+ // Proxy executor pool working with balancer and assigned pools (aka actorsystem 1.5)
+ class TAssignedCpuExecutorPool: public TCpuExecutorPool {
+ TAssignedCpu* Local;
+ TIntrusivePtr<TAffinity> CpuAffinity;
+ public:
+ explicit TAssignedCpuExecutorPool(TAssignedCpu* local, const TUnitedWorkersConfig& config)
+ : TCpuExecutorPool("United")
+ , Local(local)
+ , CpuAffinity(config.NoAffinity ? nullptr : new TAffinity(config.Allowed))
+ {}
+
+ TAffinity* Affinity() const override {
+ return CpuAffinity.Get();
+ }
+
+ ui32 GetReadyActivation(TWorkerContext& wctx, ui64 revolvingCounter) override {
+ return Local->GetReadyActivation(wctx, revolvingCounter);
+ }
+ };
+
+ // Representation of a single cpu and it's state visible to other cpus and pools
+ struct TUnitedWorkers::TCpu: public TNonCopyable {
+ struct TScopedWaiters {
+ TCpu& Cpu;
+ TPool* AssignedPool; // nullptr if CpuShared
+
+ // Subscribe on wakeups from allowed pools
+ TScopedWaiters(TCpu& cpu, TPool* assignedPool) : Cpu(cpu), AssignedPool(assignedPool) {
+ if (!AssignedPool) {
+ for (TPool* pool : Cpu.AllowedPools) {
+ AtomicIncrement(pool->Waiters);
+ }
+ } else {
+ AtomicIncrement(AssignedPool->Waiters);
+ }
+ }
+
+ // Unsubscribe from pools we've subscribed on
+ ~TScopedWaiters() {
+ if (!AssignedPool) {
+ for (TPool* pool : Cpu.AllowedPools) {
+ AtomicDecrement(pool->Waiters);
+ }
+ } else {
+ AtomicDecrement(AssignedPool->Waiters);
+ }
+ }
+ };
+
+ // Current cpu state important for other cpus and balancer
+ TCpuState State;
+
+ // Thread-safe per pool stats
+ // NOTE: It's guaranteed that cpu never executes two instance of the same pool
+ TVector<TExecutorThreadStats> PoolStats;
+
+ // Configuration
+ TCpuId CpuId;
+ THolder<TCpuLocalManager> LocalManager;
+ THolder<TCpuExecutorPool> ExecutorPool;
+
+ // Pools allowed to run on this cpu
+ TStackVec<TPool*, 15> AllowedPools;
+
+ void Stop() {
+ if (LocalManager) {
+ State.Stop();
+ LocalManager->Stop();
+ }
+ }
+
+ bool StartSpinning(TUnitedWorkers* united, TPool* assignedPool, TPoolId& result) {
+ // Mark cpu as idle
+ if (Y_UNLIKELY(!State.StartSpinning())) {
+ result = CpuStopped;
+ return true;
+ }
+
+ // Avoid using multiple atomic seq_cst loads in cycle, use barrier once and relaxed ops
+ AtomicBarrier();
+
+ // Check there is no pending tokens (can be released before Waiters increment)
+ if (!assignedPool) {
+ for (TPool* pool : AllowedPools) {
+ if (pool->TryAcquireTokenRelaxed()) {
+ result = WakeWithTokenAcquired(united, pool->PoolId);
+ return true; // token acquired or stop
+ }
+ }
+ } else {
+ if (assignedPool->TryAcquireTokenRelaxed()) {
+ result = WakeWithTokenAcquired(united, assignedPool->PoolId);
+ return true; // token acquired or stop
+ }
+ }
+
+ // At this point we can be sure wakeup won't be lost
+ // So we can actively spin or block w/o checking for pending tokens
+ return false;
+ }
+
+ bool ActiveWait(ui64 spinThresholdTs, TPoolId& result) {
+ ui64 deadline = GetCycleCountFast() + spinThresholdTs;
+ while (GetCycleCountFast() < deadline) {
+ for (ui32 i = 0; i < 12; ++i) {
+ TPoolId current = State.CurrentPool();
+ if (current == CpuSpinning) {
+ SpinLockPause();
+ } else {
+ result = current;
+ return true; // wakeup
+ }
+ }
+ }
+ return false; // spin threshold exceeded, no wakeups
+ }
+
+ bool StartBlocking(TPoolId& result) {
+ // Switch into blocked state
+ if (State.StartBlocking()) {
+ result = State.CurrentPool();
+ return true;
+ } else {
+ return false;
+ }
+ }
+
+ bool BlockedWait(TPoolId& result, ui64 timeoutNs) {
+ return State.Block(timeoutNs, result);
+ }
+
+ void SwitchPool(TPoolId pool) {
+ return State.SwitchPool(pool);
+ }
+
+ private:
+ TPoolId WakeWithTokenAcquired(TUnitedWorkers* united, TPoolId token) {
+ switch (State.WakeWithTokenAcquired(token)) {
+ case TCpuState::Woken: // we've got token and successfully woken up this cpu
+ // NOTE: sending thread may also wakeup another worker, which wont be able to acquire token and will go idle (it's ok)
+ return token;
+ case TCpuState::NotIdle: { // wakeup event has also occured
+ TPoolId wakeup = State.CurrentPool();
+ if (wakeup != token) { // token and wakeup for different pools
+ united->TryWake(wakeup); // rewake another cpu to avoid losing wakeup
+ }
+ return token;
+ }
+ case TCpuState::Forbidden:
+ Y_FAIL();
+ case TCpuState::Stopped:
+ return CpuStopped;
+ }
+ }
+ };
+
+ TUnitedWorkers::TUnitedWorkers(
+ const TUnitedWorkersConfig& config,
+ const TVector<TUnitedExecutorPoolConfig>& unitedPools,
+ const TCpuAllocationConfig& allocation,
+ IBalancer* balancer)
+ : Balancer(balancer)
+ , Config(config)
+ , Allocation(allocation)
+ {
+ // Find max pool id and initialize pools
+ PoolCount = 0;
+ for (const TCpuAllocation& cpuAlloc : allocation.Items) {
+ for (const auto& pa : cpuAlloc.AllowedPools) {
+ PoolCount = Max<size_t>(PoolCount, pa.PoolId + 1);
+ }
+ }
+ Pools.Reset(new TPool[PoolCount]);
+
+ // Find max cpu id and initialize cpus
+ CpuCount = 0;
+ for (const TCpuAllocation& cpuAlloc : allocation.Items) {
+ CpuCount = Max<size_t>(CpuCount, cpuAlloc.CpuId + 1);
+ }
+ Cpus.Reset(new TCpu[CpuCount]);
+
+ // Setup allocated cpus
+ // NOTE: leave gaps for not allocated cpus (default-initialized)
+ WorkerCount = 0;
+ for (const TCpuAllocation& cpuAlloc : allocation.Items) {
+ TCpu& cpu = Cpus[cpuAlloc.CpuId];
+ cpu.CpuId = cpuAlloc.CpuId;
+ cpu.PoolStats.resize(PoolCount); // NOTE: also may have gaps
+ for (const auto& pa : cpuAlloc.AllowedPools) {
+ cpu.AllowedPools.emplace_back(&Pools[pa.PoolId]);
+ }
+
+ // Setup balancing and cpu-local manager
+ if (!Balancer->AddCpu(cpuAlloc, &cpu.State)) {
+ cpu.State.SwitchPool(0); // set initial state to non-idle to avoid losing wakeups on start
+ cpu.State.AssignPool(CpuShared);
+ TSharedCpu* local = new TSharedCpu(TSharedCpu::TConfig(cpuAlloc, Config), this);
+ cpu.LocalManager.Reset(local);
+ cpu.ExecutorPool.Reset(new TSharedCpuExecutorPool(local, Config));
+ } else {
+ TAssignedCpu* local = new TAssignedCpu(this);
+ cpu.LocalManager.Reset(local);
+ cpu.ExecutorPool.Reset(new TAssignedCpuExecutorPool(local, Config));
+ }
+ WorkerCount += cpu.LocalManager->WorkerCount();
+ }
+
+ // Initialize workers
+ Workers.Reset(new TWorker[WorkerCount]);
+
+ // Setup pools
+ // NOTE: leave gaps for not united pools (default-initialized)
+ for (const TUnitedExecutorPoolConfig& cfg : unitedPools) {
+ TPool& pool = Pools[cfg.PoolId];
+ Y_VERIFY(cfg.PoolId < MaxPools);
+ pool.PoolId = cfg.PoolId;
+ pool.Concurrency = cfg.Concurrency ? cfg.Concurrency : Config.CpuCount;
+ pool.ExecutorPool = nullptr; // should be set later using SetupPool()
+ pool.MailboxTable = nullptr; // should be set later using SetupPool()
+ pool.TimePerMailboxTs = DurationToCycles(cfg.TimePerMailbox);
+ pool.EventsPerMailbox = cfg.EventsPerMailbox;
+
+ // Reinitialize per cpu pool stats with right MaxActivityType
+ for (const TCpuAllocation& cpuAlloc : allocation.Items) {
+ TCpu& cpu = Cpus[cpuAlloc.CpuId];
+ cpu.PoolStats[cfg.PoolId] = TExecutorThreadStats(cfg.MaxActivityType);
+ }
+
+ // Setup WakeOrderCpus: left to right exclusive cpus, then left to right shared cpus.
+ // Waking exclusive cpus first reduce load on shared cpu and improve latency isolation, which is
+ // the point of using exclusive cpu. But note that number of actively spinning idle cpus may increase,
+ // so cpu consumption on light load is higher.
+ for (const TCpuAllocation& cpuAlloc : allocation.Items) {
+ TCpu& cpu = Cpus[cpuAlloc.CpuId];
+ if (cpu.AllowedPools.size() == 1 && cpu.AllowedPools[0] == &pool) {
+ pool.WakeOrderCpus.emplace_back(&cpu);
+ }
+ }
+ for (const TCpuAllocation& cpuAlloc : allocation.Items) {
+ TCpu& cpu = Cpus[cpuAlloc.CpuId];
+ if (cpu.AllowedPools.size() > 1 && cpuAlloc.HasPool(pool.PoolId)) {
+ pool.WakeOrderCpus.emplace_back(&cpu);
+ }
+ }
+ }
+ }
+
+ TUnitedWorkers::~TUnitedWorkers() {
+ }
+
+ void TUnitedWorkers::Prepare(TActorSystem* actorSystem, TVector<NSchedulerQueue::TReader*>& scheduleReaders) {
+ // Setup allocated cpus
+ // NOTE: leave gaps for not allocated cpus (default-initialized)
+ TWorkerId workers = 0;
+ for (TCpuId cpuId = 0; cpuId < CpuCount; cpuId++) {
+ TCpu& cpu = Cpus[cpuId];
+
+ // Setup cpu-local workers
+ if (cpu.LocalManager) {
+ for (size_t i = 0; i < cpu.LocalManager->WorkerCount(); i++) {
+ TWorkerId workerId = workers++;
+ cpu.LocalManager->AddWorker(workerId);
+
+ // Setup worker
+ Y_VERIFY(workerId < WorkerCount);
+ Workers[workerId].Thread.Reset(new TExecutorThread(
+ workerId,
+ cpu.CpuId,
+ actorSystem,
+ cpu.ExecutorPool.Get(), // use cpu-local manager as proxy executor for all workers on cpu
+ nullptr, // MailboxTable is pool-specific, will be set on pool switch
+ cpu.ExecutorPool->GetName()));
+ // NOTE: TWorker::ThreadId will be initialized after in Start()
+
+ scheduleReaders.push_back(&Workers[workerId].SchedulerQueue.Reader);
+ }
+ }
+ }
+ }
+
+ void TUnitedWorkers::Start() {
+ for (TWorkerId workerId = 0; workerId < WorkerCount; workerId++) {
+ Workers[workerId].Thread->Start();
+ }
+ for (TWorkerId workerId = 0; workerId < WorkerCount; workerId++) {
+ AtomicStore(&Workers[workerId].ThreadId, Workers[workerId].Thread->GetThreadId());
+ }
+ }
+
+ inline TThreadId TUnitedWorkers::GetWorkerThreadId(TWorkerId workerId) const {
+ volatile TThreadId* threadId = &Workers[workerId].ThreadId;
+#ifdef _linux_
+ while (AtomicLoad(threadId) == UnknownThreadId) {
+ NanoSleep(1000);
+ }
+#endif
+ return AtomicLoad(threadId);
+ }
+
+ inline NSchedulerQueue::TWriter* TUnitedWorkers::GetScheduleWriter(TWorkerId workerId) const {
+ return &Workers[workerId].SchedulerQueue.Writer;
+ }
+
+ void TUnitedWorkers::SetupPool(TPoolId pool, IExecutorPool* executorPool, TMailboxTable* mailboxTable) {
+ Pools[pool].ExecutorPool = executorPool;
+ Pools[pool].MailboxTable = mailboxTable;
+ }
+
+ void TUnitedWorkers::PrepareStop() {
+ AtomicStore(&StopFlag, true);
+ for (TPoolId pool = 0; pool < PoolCount; pool++) {
+ Pools[pool].Stop();
+ }
+ for (TCpuId cpuId = 0; cpuId < CpuCount; cpuId++) {
+ Cpus[cpuId].Stop();
+ }
+ }
+
+ void TUnitedWorkers::Shutdown() {
+ for (TWorkerId workerId = 0; workerId < WorkerCount; workerId++) {
+ Workers[workerId].Thread->Join();
+ }
+ }
+
+ inline void TUnitedWorkers::PushActivation(TPoolId pool, ui32 activation, ui64 revolvingCounter) {
+ if (Pools[pool].PushActivation(activation, revolvingCounter)) { // token generated
+ TryWake(pool);
+ }
+ }
+
+ inline bool TUnitedWorkers::TryAcquireToken(TPoolId pool) {
+ return Pools[pool].TryAcquireToken();
+ }
+
+ inline void TUnitedWorkers::TryWake(TPoolId pool) {
+ // Avoid using multiple atomic seq_cst loads in cycle, use barrier once
+ AtomicBarrier();
+
+ // Scan every allowed cpu in pool's wakeup order and try to wake the first idle cpu
+ if (RelaxedLoad(&Pools[pool].Waiters) > 0) {
+ for (TCpu* cpu : Pools[pool].WakeOrderCpus) {
+ if (cpu->State.WakeWithoutToken(pool) == TCpuState::Woken) {
+ return; // successful wake up
+ }
+ }
+ }
+
+ // Cpu has not been woken up
+ }
+
+ inline void TUnitedWorkers::BeginExecution(TPoolId pool, ui32& activation, ui64 revolvingCounter) {
+ Pools[pool].BeginExecution(activation, revolvingCounter);
+ }
+
+ inline bool TUnitedWorkers::NextExecution(TPoolId pool, ui32& activation, ui64 revolvingCounter) {
+ return Pools[pool].NextExecution(activation, revolvingCounter);
+ }
+
+ inline void TUnitedWorkers::StopExecution(TPoolId pool) {
+ if (Pools[pool].StopExecution()) { // pending token
+ TryWake(pool);
+ }
+ }
+
+ inline void TUnitedWorkers::Balance() {
+ ui64 ts = GetCycleCountFast();
+ if (Balancer->TryLock(ts)) {
+ for (TPoolId pool = 0; pool < PoolCount; pool++) {
+ if (Pools[pool].IsUnited()) {
+ ui64 ElapsedTs = 0;
+ ui64 ParkedTs = 0;
+ for (TCpu* cpu : Pools[pool].WakeOrderCpus) {
+ const TExecutorThreadStats& cpuStats = cpu->PoolStats[pool];
+ ElapsedTs += cpuStats.ElapsedTicks;
+ ParkedTs += cpuStats.ParkedTicks;
+ }
+ TBalancerStats stats;
+ stats.Ts = ts;
+ stats.CpuUs = Ts2Us(ElapsedTs);
+ stats.IdleUs = Ts2Us(ParkedTs);
+ Balancer->SetPoolStats(pool, stats);
+ }
+ }
+ Balancer->Balance();
+ Balancer->Unlock();
+ }
+ }
+
+ inline TPoolId TUnitedWorkers::AssignedPool(TWorkerContext& wctx) {
+ return Cpus[wctx.CpuId].State.AssignedPool();
+ }
+
+ inline bool TUnitedWorkers::IsPoolReassigned(TWorkerContext& wctx) {
+ return Cpus[wctx.CpuId].State.IsPoolReassigned(wctx.PoolId);
+ }
+
+ inline void TUnitedWorkers::SwitchPool(TWorkerContext& wctx, ui64 softDeadlineTs) {
+ Pools[wctx.PoolId].Switch(wctx, softDeadlineTs, Cpus[wctx.CpuId].PoolStats[wctx.PoolId]);
+ Cpus[wctx.CpuId].SwitchPool(wctx.PoolId);
+ }
+
+ TPoolId TUnitedWorkers::Idle(TPoolId assigned, TWorkerContext& wctx) {
+ wctx.SwitchToIdle();
+
+ TPoolId result;
+ TTimeTracker timeTracker;
+ TCpu& cpu = Cpus[wctx.CpuId];
+ TPool* assignedPool = assigned == CpuShared ? nullptr : &Pools[assigned];
+ TCpu::TScopedWaiters scopedWaiters(cpu, assignedPool);
+ while (true) {
+ if (cpu.StartSpinning(this, assignedPool, result)) {
+ break; // token already acquired (or stop)
+ }
+ result = WaitSequence(cpu, wctx, timeTracker);
+ if (Y_UNLIKELY(result == CpuStopped) || TryAcquireToken(result)) {
+ break; // token acquired (or stop)
+ }
+ }
+
+ wctx.AddElapsedCycles(IActor::ACTOR_SYSTEM, timeTracker.Elapsed());
+ return result;
+ }
+
+ TPoolId TUnitedWorkers::WaitSequence(TCpu& cpu, TWorkerContext& wctx, TTimeTracker& timeTracker) {
+ TPoolId result;
+ if (cpu.ActiveWait(Us2Ts(Config.SpinThresholdUs), result)) {
+ wctx.AddElapsedCycles(IActor::ACTOR_SYSTEM, timeTracker.Elapsed());
+ return result;
+ }
+ if (cpu.StartBlocking(result)) {
+ wctx.AddElapsedCycles(IActor::ACTOR_SYSTEM, timeTracker.Elapsed());
+ return result;
+ }
+ wctx.AddElapsedCycles(IActor::ACTOR_SYSTEM, timeTracker.Elapsed());
+ bool wakeup;
+ do {
+ wakeup = cpu.BlockedWait(result, Config.Balancer.PeriodUs * 1000);
+ wctx.AddParkedCycles(timeTracker.Elapsed());
+ } while (!wakeup);
+ return result;
+ }
+
+ void TUnitedWorkers::GetCurrentStats(TPoolId pool, TVector<TExecutorThreadStats>& statsCopy) const {
+ size_t idx = 1;
+ statsCopy.resize(idx + Pools[pool].WakeOrderCpus.size());
+ for (TCpu* cpu : Pools[pool].WakeOrderCpus) {
+ TExecutorThreadStats& s = statsCopy[idx++];
+ s = TExecutorThreadStats();
+ s.Aggregate(cpu->PoolStats[pool]);
+ }
+ }
+
+ TUnitedExecutorPool::TUnitedExecutorPool(const TUnitedExecutorPoolConfig& cfg, TUnitedWorkers* united)
+ : TExecutorPoolBaseMailboxed(cfg.PoolId, cfg.MaxActivityType)
+ , United(united)
+ , PoolName(cfg.PoolName)
+ {
+ United->SetupPool(TPoolId(cfg.PoolId), this, MailboxTable.Get());
+ }
+
+ void TUnitedExecutorPool::Prepare(TActorSystem* actorSystem, NSchedulerQueue::TReader** scheduleReaders, ui32* scheduleSz) {
+ ActorSystem = actorSystem;
+
+ // Schedule readers are initialized through TUnitedWorkers::Prepare
+ *scheduleReaders = nullptr;
+ *scheduleSz = 0;
+ }
+
+ void TUnitedExecutorPool::Start() {
+ // workers are actually started in TUnitedWorkers::Start()
+ }
+
+ void TUnitedExecutorPool::PrepareStop() {
+ }
+
+ void TUnitedExecutorPool::Shutdown() {
+ // workers are actually joined in TUnitedWorkers::Shutdown()
+ }
+
+ TAffinity* TUnitedExecutorPool::Affinity() const {
+ Y_FAIL(); // should never be called, TCpuExecutorPool is used instead
+ }
+
+ ui32 TUnitedExecutorPool::GetThreads() const {
+ return 0;
+ }
+
+ ui32 TUnitedExecutorPool::GetReadyActivation(TWorkerContext&, ui64) {
+ Y_FAIL(); // should never be called, TCpu*ExecutorPool is used instead
+ }
+
+ inline void TUnitedExecutorPool::ScheduleActivation(ui32 activation) {
+ TUnitedExecutorPool::ScheduleActivationEx(activation, AtomicIncrement(ActivationsRevolvingCounter));
+ }
+
+ inline void TUnitedExecutorPool::ScheduleActivationEx(ui32 activation, ui64 revolvingCounter) {
+ United->PushActivation(PoolId, activation, revolvingCounter);
+ }
+
+ void TUnitedExecutorPool::Schedule(TInstant deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) {
+ TUnitedExecutorPool::Schedule(deadline - ActorSystem->Timestamp(), ev, cookie, workerId);
+ }
+
+ void TUnitedExecutorPool::Schedule(TMonotonic deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) {
+ Y_VERIFY_DEBUG(workerId < United->GetWorkerCount());
+ const auto current = ActorSystem->Monotonic();
+ if (deadline < current) {
+ deadline = current;
+ }
+ United->GetScheduleWriter(workerId)->Push(deadline.MicroSeconds(), ev.Release(), cookie);
+ }
+
+ void TUnitedExecutorPool::Schedule(TDuration delta, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) {
+ Y_VERIFY_DEBUG(workerId < United->GetWorkerCount());
+ const auto deadline = ActorSystem->Monotonic() + delta;
+ United->GetScheduleWriter(workerId)->Push(deadline.MicroSeconds(), ev.Release(), cookie);
+ }
+
+ void TUnitedExecutorPool::GetCurrentStats(TExecutorPoolStats& poolStats, TVector<TExecutorThreadStats>& statsCopy) const {
+ Y_UNUSED(poolStats);
+ if (statsCopy.empty()) {
+ statsCopy.resize(1);
+ }
+ statsCopy[0] = TExecutorThreadStats();
+ statsCopy[0].Aggregate(Stats);
+ United->GetCurrentStats(PoolId, statsCopy);
+ }
+}
diff --git a/library/cpp/actors/core/executor_pool_united.h b/library/cpp/actors/core/executor_pool_united.h
new file mode 100644
index 0000000000..a090ba2466
--- /dev/null
+++ b/library/cpp/actors/core/executor_pool_united.h
@@ -0,0 +1,135 @@
+#pragma once
+
+#include "actorsystem.h"
+#include "balancer.h"
+#include "scheduler_queue.h"
+#include "executor_pool_base.h"
+
+#include <library/cpp/actors/util/unordered_cache.h>
+
+#include <library/cpp/monlib/dynamic_counters/counters.h>
+#include <library/cpp/actors/util/unordered_cache.h>
+#include <library/cpp/containers/stack_vector/stack_vec.h>
+
+#include <util/generic/noncopyable.h>
+
+namespace NActors {
+ class TMailboxTable;
+
+ class TUnitedWorkers: public TNonCopyable {
+ struct TWorker;
+ struct TPool;
+ struct TCpu;
+
+ size_t WorkerCount;
+ TArrayHolder<TWorker> Workers; // indexed by WorkerId
+ size_t PoolCount;
+ TArrayHolder<TPool> Pools; // indexed by PoolId, so may include not used (not united) pools
+ size_t CpuCount;
+ TArrayHolder<TCpu> Cpus; // indexed by CpuId, so may include not allocated CPUs
+
+ IBalancer* Balancer; // external pool cpu balancer
+
+ TUnitedWorkersConfig Config;
+ TCpuAllocationConfig Allocation;
+
+ volatile bool StopFlag = false;
+
+ public:
+ TUnitedWorkers(
+ const TUnitedWorkersConfig& config,
+ const TVector<TUnitedExecutorPoolConfig>& unitedPools,
+ const TCpuAllocationConfig& allocation,
+ IBalancer* balancer);
+ ~TUnitedWorkers();
+ void Prepare(TActorSystem* actorSystem, TVector<NSchedulerQueue::TReader*>& scheduleReaders);
+ void Start();
+ void PrepareStop();
+ void Shutdown();
+
+ bool IsStopped() const {
+ return RelaxedLoad(&StopFlag);
+ }
+
+ TWorkerId GetWorkerCount() const {
+ return WorkerCount;
+ }
+
+ // Returns thread id of a worker
+ TThreadId GetWorkerThreadId(TWorkerId workerId) const;
+
+ // Returns per worker schedule writers
+ NSchedulerQueue::TWriter* GetScheduleWriter(TWorkerId workerId) const;
+
+ // Sets executor for specified pool
+ void SetupPool(TPoolId pool, IExecutorPool* executorPool, TMailboxTable* mailboxTable);
+
+ // Add activation of newly scheduled mailbox and wake cpu to execute it if required
+ void PushActivation(TPoolId pool, ui32 activation, ui64 revolvingCounter);
+
+ // Try acquire pending token. Must be done before execution
+ bool TryAcquireToken(TPoolId pool);
+
+ // Try to wake idle cpu waiting for tokens on specified pool
+ void TryWake(TPoolId pool);
+
+ // Get activation from pool; requires pool's token
+ void BeginExecution(TPoolId pool, ui32& activation, ui64 revolvingCounter);
+
+ // Stop currently active execution and start new one if token is available
+ // NOTE: Reuses token if it's not destroyed
+ bool NextExecution(TPoolId pool, ui32& activation, ui64 revolvingCounter);
+
+ // Stop active execution
+ void StopExecution(TPoolId pool);
+
+ // Runs balancer to assign pools to cpus
+ void Balance();
+
+ // Returns pool to be executed by worker or `CpuShared`
+ TPoolId AssignedPool(TWorkerContext& wctx);
+
+ // Checks if balancer has assigned another pool for worker's cpu
+ bool IsPoolReassigned(TWorkerContext& wctx);
+
+ // Switch worker context into specified pool
+ void SwitchPool(TWorkerContext& wctx, ui64 softDeadlineTs);
+
+ // Wait for tokens from any pool allowed on specified cpu
+ TPoolId Idle(TPoolId assigned, TWorkerContext& wctx);
+
+ // Fill stats for specified pool
+ void GetCurrentStats(TPoolId pool, TVector<TExecutorThreadStats>& statsCopy) const;
+
+ private:
+ TPoolId WaitSequence(TCpu& cpu, TWorkerContext& wctx, TTimeTracker& timeTracker);
+ };
+
+ class TUnitedExecutorPool: public TExecutorPoolBaseMailboxed {
+ TUnitedWorkers* United;
+ const TString PoolName;
+ TAtomic ActivationsRevolvingCounter = 0;
+ public:
+ TUnitedExecutorPool(const TUnitedExecutorPoolConfig& cfg, TUnitedWorkers* united);
+
+ void Prepare(TActorSystem* actorSystem, NSchedulerQueue::TReader** scheduleReaders, ui32* scheduleSz) override;
+ void Start() override;
+ void PrepareStop() override;
+ void Shutdown() override;
+
+ TAffinity* Affinity() const override;
+ ui32 GetThreads() const override;
+ ui32 GetReadyActivation(TWorkerContext& wctx, ui64 revolvingReadCounter) override;
+ void ScheduleActivation(ui32 activation) override;
+ void ScheduleActivationEx(ui32 activation, ui64 revolvingWriteCounter) override;
+ void Schedule(TInstant deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) override;
+ void Schedule(TMonotonic deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) override;
+ void Schedule(TDuration delta, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) override;
+
+ void GetCurrentStats(TExecutorPoolStats& poolStats, TVector<TExecutorThreadStats>& statsCopy) const override;
+
+ TString GetName() const override {
+ return PoolName;
+ }
+ };
+}
diff --git a/library/cpp/actors/core/executor_pool_united_ut.cpp b/library/cpp/actors/core/executor_pool_united_ut.cpp
new file mode 100644
index 0000000000..d4df17f1b8
--- /dev/null
+++ b/library/cpp/actors/core/executor_pool_united_ut.cpp
@@ -0,0 +1,338 @@
+#include "actorsystem.h"
+#include "executor_pool_basic.h"
+#include "hfunc.h"
+#include "scheduler_basic.h"
+
+#include <library/cpp/actors/util/should_continue.h>
+
+#include <library/cpp/testing/unittest/registar.h>
+#include <library/cpp/actors/protos/unittests.pb.h>
+
+using namespace NActors;
+
+////////////////////////////////////////////////////////////////////////////////
+
+struct TEvMsg : public NActors::TEventBase<TEvMsg, 10347> {
+ DEFINE_SIMPLE_LOCAL_EVENT(TEvMsg, "ExecutorPoolTest: Msg");
+};
+
+////////////////////////////////////////////////////////////////////////////////
+
+inline ui64 DoTimedWork(ui64 workUs) {
+ ui64 startUs = ThreadCPUTime();
+ ui64 endUs = startUs + workUs;
+ ui64 nowUs = startUs;
+ do {
+ ui64 endTs = GetCycleCountFast() + Us2Ts(endUs - nowUs);
+ while (GetCycleCountFast() <= endTs) {}
+ nowUs = ThreadCPUTime();
+ } while (nowUs <= endUs);
+ return nowUs - startUs;
+}
+
+class TTestSenderActor : public IActor {
+private:
+ using EActivityType = IActor::EActivityType ;
+ using EActorActivity = IActor::EActorActivity;
+
+private:
+ TAtomic Counter;
+ TActorId Receiver;
+
+ std::function<void(void)> Action;
+
+public:
+ TTestSenderActor(std::function<void(void)> action = [](){},
+ EActivityType activityType = EActorActivity::OTHER)
+ : IActor(static_cast<TReceiveFunc>(&TTestSenderActor::Execute), activityType)
+ , Action(action)
+ {}
+
+ void Start(TActorId receiver, size_t count) {
+ AtomicSet(Counter, count);
+ Receiver = receiver;
+ }
+
+ void Stop() {
+ while (true) {
+ if (GetCounter() == 0) {
+ break;
+ }
+
+ Sleep(TDuration::MilliSeconds(1));
+ }
+ }
+
+ size_t GetCounter() const {
+ return AtomicGet(Counter);
+ }
+
+private:
+ STFUNC(Execute) {
+ Y_UNUSED(ctx);
+ switch (ev->GetTypeRewrite()) {
+ hFunc(TEvMsg, Handle);
+ }
+ }
+
+ void Handle(TEvMsg::TPtr &ev) {
+ Y_UNUSED(ev);
+ Action();
+ TAtomicBase count = AtomicDecrement(Counter);
+ Y_VERIFY(count != Max<TAtomicBase>());
+ if (count) {
+ Send(Receiver, new TEvMsg());
+ }
+ }
+};
+
+// Single cpu balancer that switches pool on every activation; not thread-safe
+struct TRoundRobinBalancer: public IBalancer {
+ TCpuState* State;
+ TMap<TPoolId, TPoolId> NextPool;
+
+ bool AddCpu(const TCpuAllocation& cpuAlloc, TCpuState* cpu) override {
+ State = cpu;
+ TPoolId prev = cpuAlloc.AllowedPools.rbegin()->PoolId;
+ for (auto& p : cpuAlloc.AllowedPools) {
+ NextPool[prev] = p.PoolId;
+ prev = p.PoolId;
+ }
+ return true;
+ }
+
+ bool TryLock(ui64) override { return true; }
+ void SetPoolStats(TPoolId, const TBalancerStats&) override {}
+ void Unlock() override {}
+
+ void Balance() override {
+ TPoolId assigned;
+ TPoolId current;
+ State->Load(assigned, current);
+ State->AssignPool(NextPool[assigned]);
+ }
+};
+
+void AddUnitedPool(THolder<TActorSystemSetup>& setup, ui32 concurrency = 0) {
+ TUnitedExecutorPoolConfig united;
+ united.PoolId = setup->GetExecutorsCount();
+ united.Concurrency = concurrency;
+ setup->CpuManager.United.emplace_back(std::move(united));
+}
+
+THolder<TActorSystemSetup> GetActorSystemSetup(ui32 cpuCount) {
+ auto setup = MakeHolder<NActors::TActorSystemSetup>();
+ setup->NodeId = 1;
+ setup->CpuManager.UnitedWorkers.CpuCount = cpuCount;
+ setup->CpuManager.UnitedWorkers.NoRealtime = true; // unavailable in test environment
+ setup->Scheduler = new TBasicSchedulerThread(NActors::TSchedulerConfig(512, 0));
+ return setup;
+}
+
+Y_UNIT_TEST_SUITE(UnitedExecutorPool) {
+
+#ifdef _linux_
+
+ Y_UNIT_TEST(OnePoolManyCpus) {
+ const size_t msgCount = 1e4;
+ auto setup = GetActorSystemSetup(4);
+ AddUnitedPool(setup);
+ TActorSystem actorSystem(setup);
+ actorSystem.Start();
+
+ auto begin = TInstant::Now();
+
+ auto actor = new TTestSenderActor();
+ auto actorId = actorSystem.Register(actor);
+ actor->Start(actor->SelfId(), msgCount);
+ actorSystem.Send(actorId, new TEvMsg());
+
+ while (actor->GetCounter()) {
+ auto now = TInstant::Now();
+ UNIT_ASSERT_C(now - begin < TDuration::Seconds(5), "Counter is " << actor->GetCounter());
+
+ Sleep(TDuration::MilliSeconds(1));
+ }
+
+ TVector<TExecutorThreadStats> stats;
+ TExecutorPoolStats poolStats;
+ actorSystem.GetPoolStats(0, poolStats, stats);
+ // Sum all per-thread counters into the 0th element
+ for (ui32 idx = 1; idx < stats.size(); ++idx) {
+ stats[0].Aggregate(stats[idx]);
+ }
+
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].SentEvents, msgCount - 1);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].ReceivedEvents, msgCount);
+ //UNIT_ASSERT_VALUES_EQUAL(stats[0].PreemptedEvents, 0); // depends on execution time and system load, so may be non-zero
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].NonDeliveredEvents, 0);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].EmptyMailboxActivation, 0);
+ //UNIT_ASSERT_VALUES_EQUAL(stats[0].CpuNs, 0); // depends on total duration of test, so undefined
+ UNIT_ASSERT(stats[0].ElapsedTicks > 0);
+ UNIT_ASSERT(stats[0].ParkedTicks == 0); // per-pool parked time does not make sense for united pools
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].BlockedTicks, 0);
+ UNIT_ASSERT(stats[0].ActivationTimeHistogram.TotalSamples >= msgCount / TBasicExecutorPoolConfig::DEFAULT_EVENTS_PER_MAILBOX);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].EventDeliveryTimeHistogram.TotalSamples, msgCount);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].EventProcessingCountHistogram.TotalSamples, msgCount);
+ UNIT_ASSERT(stats[0].EventProcessingTimeHistogram.TotalSamples > 0);
+ UNIT_ASSERT(stats[0].ElapsedTicksByActivity[0] > 0);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].ReceivedEventsByActivity[0], msgCount);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].ActorsAliveByActivity[0], 1);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].ScheduledEventsByActivity[0], 0);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].PoolActorRegistrations, 1);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].PoolDestroyedActors, 0);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].PoolAllocatedMailboxes, 4095); // one line
+ UNIT_ASSERT(stats[0].MailboxPushedOutByTime + stats[0].MailboxPushedOutByEventCount + stats[0].MailboxPushedOutBySoftPreemption >= msgCount / TBasicExecutorPoolConfig::DEFAULT_EVENTS_PER_MAILBOX);
+ }
+
+ Y_UNIT_TEST(ManyPoolsOneSharedCpu) {
+ const size_t msgCount = 1e4;
+ const size_t pools = 4;
+ auto setup = GetActorSystemSetup(1);
+ for (size_t pool = 0; pool < pools; pool++) {
+ AddUnitedPool(setup);
+ }
+ TActorSystem actorSystem(setup);
+ actorSystem.Start();
+
+ auto begin = TInstant::Now();
+
+ TVector<TTestSenderActor*> actors;
+ for (size_t pool = 0; pool < pools; pool++) {
+ auto actor = new TTestSenderActor();
+ auto actorId = actorSystem.Register(actor, TMailboxType::HTSwap, pool);
+ actor->Start(actor->SelfId(), msgCount);
+ actorSystem.Send(actorId, new TEvMsg());
+ actors.push_back(actor);
+ }
+
+ while (true) {
+ size_t left = 0;
+ for (auto actor : actors) {
+ left += actor->GetCounter();
+ }
+ if (left == 0) {
+ break;
+ }
+ auto now = TInstant::Now();
+ UNIT_ASSERT_C(now - begin < TDuration::Seconds(5), "left " << left);
+ Sleep(TDuration::MilliSeconds(1));
+ }
+
+ for (size_t pool = 0; pool < pools; pool++) {
+ TVector<TExecutorThreadStats> stats;
+ TExecutorPoolStats poolStats;
+ actorSystem.GetPoolStats(pool, poolStats, stats);
+ // Sum all per-thread counters into the 0th element
+ for (ui32 idx = 1; idx < stats.size(); ++idx) {
+ stats[0].Aggregate(stats[idx]);
+ }
+
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].ReceivedEvents, msgCount);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].PoolActorRegistrations, 1);
+ }
+ }
+
+ Y_UNIT_TEST(ManyPoolsOneAssignedCpu) {
+ const size_t msgCount = 1e4;
+ const size_t pools = 4;
+ auto setup = GetActorSystemSetup(1);
+ setup->Balancer.Reset(new TRoundRobinBalancer());
+ for (size_t pool = 0; pool < pools; pool++) {
+ AddUnitedPool(setup);
+ }
+ TActorSystem actorSystem(setup);
+ actorSystem.Start();
+
+ auto begin = TInstant::Now();
+
+ TVector<TTestSenderActor*> actors;
+ for (size_t pool = 0; pool < pools; pool++) {
+ auto actor = new TTestSenderActor();
+ auto actorId = actorSystem.Register(actor, TMailboxType::HTSwap, pool);
+ actor->Start(actor->SelfId(), msgCount);
+ actorSystem.Send(actorId, new TEvMsg());
+ actors.push_back(actor);
+ }
+
+ while (true) {
+ size_t left = 0;
+ for (auto actor : actors) {
+ left += actor->GetCounter();
+ }
+ if (left == 0) {
+ break;
+ }
+ auto now = TInstant::Now();
+ UNIT_ASSERT_C(now - begin < TDuration::Seconds(5), "left " << left);
+ Sleep(TDuration::MilliSeconds(1));
+ }
+
+ for (size_t pool = 0; pool < pools; pool++) {
+ TVector<TExecutorThreadStats> stats;
+ TExecutorPoolStats poolStats;
+ actorSystem.GetPoolStats(pool, poolStats, stats);
+ // Sum all per-thread counters into the 0th element
+ for (ui32 idx = 1; idx < stats.size(); ++idx) {
+ stats[0].Aggregate(stats[idx]);
+ }
+
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].ReceivedEvents, msgCount);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].PoolActorRegistrations, 1);
+ }
+ }
+
+ Y_UNIT_TEST(ManyPoolsOneCpuSlowEvents) {
+ const size_t msgCount = 3;
+ const size_t pools = 4;
+ auto setup = GetActorSystemSetup(1);
+ for (size_t pool = 0; pool < pools; pool++) {
+ AddUnitedPool(setup);
+ }
+ TActorSystem actorSystem(setup);
+ actorSystem.Start();
+
+ auto begin = TInstant::Now();
+
+ TVector<TTestSenderActor*> actors;
+ for (size_t pool = 0; pool < pools; pool++) {
+ auto actor = new TTestSenderActor([]() {
+ DoTimedWork(100'000);
+ });
+ auto actorId = actorSystem.Register(actor, TMailboxType::HTSwap, pool);
+ actor->Start(actor->SelfId(), msgCount);
+ actorSystem.Send(actorId, new TEvMsg());
+ actors.push_back(actor);
+ }
+
+ while (true) {
+ size_t left = 0;
+ for (auto actor : actors) {
+ left += actor->GetCounter();
+ }
+ if (left == 0) {
+ break;
+ }
+ auto now = TInstant::Now();
+ UNIT_ASSERT_C(now - begin < TDuration::Seconds(15), "left " << left);
+ Sleep(TDuration::MilliSeconds(1));
+ }
+
+ for (size_t pool = 0; pool < pools; pool++) {
+ TVector<TExecutorThreadStats> stats;
+ TExecutorPoolStats poolStats;
+ actorSystem.GetPoolStats(pool, poolStats, stats);
+ // Sum all per-thread counters into the 0th element
+ for (ui32 idx = 1; idx < stats.size(); ++idx) {
+ stats[0].Aggregate(stats[idx]);
+ }
+
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].ReceivedEvents, msgCount);
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].PreemptedEvents, msgCount); // every 100ms event should be preempted
+ UNIT_ASSERT_VALUES_EQUAL(stats[0].PoolActorRegistrations, 1);
+ }
+ }
+
+#endif
+
+}
diff --git a/library/cpp/actors/core/executor_thread.cpp b/library/cpp/actors/core/executor_thread.cpp
new file mode 100644
index 0000000000..446b651efd
--- /dev/null
+++ b/library/cpp/actors/core/executor_thread.cpp
@@ -0,0 +1,563 @@
+#include "executor_thread.h"
+#include "actorsystem.h"
+#include "callstack.h"
+#include "mailbox.h"
+#include "event.h"
+#include "events.h"
+
+#include <library/cpp/actors/prof/tag.h>
+#include <library/cpp/actors/util/affinity.h>
+#include <library/cpp/actors/util/datetime.h>
+#include <library/cpp/actors/util/thread.h>
+
+#ifdef BALLOC
+#include <library/cpp/balloc/optional/operators.h>
+#endif
+
+#ifdef _linux_
+#include <sys/syscall.h>
+#include <unistd.h>
+#endif
+
+#include <util/system/type_name.h>
+#include <util/system/datetime.h>
+
+LWTRACE_USING(ACTORLIB_PROVIDER)
+
+namespace NActors {
+ constexpr TDuration TExecutorThread::DEFAULT_TIME_PER_MAILBOX;
+
+ TExecutorThread::TExecutorThread(
+ TWorkerId workerId,
+ TWorkerId cpuId,
+ TActorSystem* actorSystem,
+ IExecutorPool* executorPool,
+ TMailboxTable* mailboxTable,
+ const TString& threadName,
+ TDuration timePerMailbox,
+ ui32 eventsPerMailbox)
+ : ActorSystem(actorSystem)
+ , ExecutorPool(executorPool)
+ , Ctx(workerId, cpuId, actorSystem ? actorSystem->GetMaxActivityType() : 1)
+ , ThreadName(threadName)
+ {
+ Ctx.Switch(
+ ExecutorPool,
+ mailboxTable,
+ NHPTimer::GetClockRate() * timePerMailbox.SecondsFloat(),
+ eventsPerMailbox,
+ ui64(-1), // infinite soft deadline
+ &Ctx.WorkerStats);
+ }
+
+ TActorId TExecutorThread::RegisterActor(IActor* actor, TMailboxType::EType mailboxType, ui32 poolId, const TActorId& parentId) {
+ if (poolId == Max<ui32>())
+ return Ctx.Executor->Register(actor, mailboxType, ++RevolvingWriteCounter, parentId ? parentId : CurrentRecipient);
+ else
+ return ActorSystem->Register(actor, mailboxType, poolId, ++RevolvingWriteCounter, parentId ? parentId : CurrentRecipient);
+ }
+
+ TActorId TExecutorThread::RegisterActor(IActor* actor, TMailboxHeader* mailbox, ui32 hint, const TActorId& parentId) {
+ return Ctx.Executor->Register(actor, mailbox, hint, parentId ? parentId : CurrentRecipient);
+ }
+
+ void TExecutorThread::UnregisterActor(TMailboxHeader* mailbox, ui64 localActorId) {
+ IActor* actor = mailbox->DetachActor(localActorId);
+ Ctx.DecrementActorsAliveByActivity(actor->GetActivityType());
+ DyingActors.push_back(THolder(actor));
+ }
+
+ void TExecutorThread::DropUnregistered() {
+ DyingActors.clear(); // here is actual destruction of actors
+ }
+
+ void TExecutorThread::Schedule(TInstant deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie) {
+ ++CurrentActorScheduledEventsCounter;
+ Ctx.Executor->Schedule(deadline, ev, cookie, Ctx.WorkerId);
+ }
+
+ void TExecutorThread::Schedule(TMonotonic deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie) {
+ ++CurrentActorScheduledEventsCounter;
+ Ctx.Executor->Schedule(deadline, ev, cookie, Ctx.WorkerId);
+ }
+
+ void TExecutorThread::Schedule(TDuration delta, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie) {
+ ++CurrentActorScheduledEventsCounter;
+ Ctx.Executor->Schedule(delta, ev, cookie, Ctx.WorkerId);
+ }
+
+ template <class T>
+ inline TString SafeTypeName(T* t) {
+ if (t == nullptr) {
+ return "nullptr";
+ }
+ try {
+ return TypeName(*t);
+ } catch (...) {
+ return "unknown-type";
+ }
+ }
+
+ inline TString ActorTypeName(const IActor* actor, ui32 activityType) {
+ return actor ? SafeTypeName(actor) : ("activityType_" + ToString(activityType) + " (destroyed)");
+ }
+
+ inline void LwTraceSlowDelivery(IEventHandle* ev, const IActor* actor, ui32 poolId, const TActorId& currentRecipient,
+ double delivMs, double sinceActivationMs, ui32 eventsExecutedBefore) {
+ const auto baseEv = (ev && ev->HasEvent()) ? ev->GetBase() : nullptr;
+ LWPROBE(EventSlowDelivery,
+ poolId,
+ delivMs,
+ sinceActivationMs,
+ eventsExecutedBefore,
+ baseEv ? SafeTypeName(baseEv) : (ev ? ToString(ev->Type) : TString("nullptr")),
+ currentRecipient.ToString(),
+ SafeTypeName(actor));
+ }
+
+ inline void LwTraceSlowEvent(IEventHandle* ev, ui32 evTypeForTracing, const IActor* actor, ui32 poolId, ui32 activityType,
+ const TActorId& currentRecipient, double eventMs) {
+ // Event could have been destroyed by actor->Receive();
+ const auto baseEv = (ev && ev->HasEvent()) ? ev->GetBase() : nullptr;
+ LWPROBE(SlowEvent,
+ poolId,
+ eventMs,
+ baseEv ? SafeTypeName(baseEv) : ToString(evTypeForTracing),
+ currentRecipient.ToString(),
+ ActorTypeName(actor, activityType));
+ }
+
+ template <typename TMailbox>
+ void TExecutorThread::Execute(TMailbox* mailbox, ui32 hint) {
+ Y_VERIFY_DEBUG(DyingActors.empty());
+
+ bool reclaimAsFree = false;
+
+ NHPTimer::STime hpstart = GetCycleCountFast();
+ NHPTimer::STime hpprev = hpstart;
+
+ IActor* actor = nullptr;
+ ui32 prevActivityType = std::numeric_limits<ui32>::max();
+ TActorId recipient;
+ for (ui32 executed = 0; executed < Ctx.EventsPerMailbox; ++executed) {
+ TAutoPtr<IEventHandle> ev(mailbox->Pop());
+ if (!!ev) {
+ NHPTimer::STime hpnow;
+ recipient = ev->GetRecipientRewrite();
+ if (actor = mailbox->FindActor(recipient.LocalId())) {
+ TActorContext ctx(*mailbox, *this, hpprev, recipient);
+ TlsActivationContext = &ctx;
+
+#ifdef USE_ACTOR_CALLSTACK
+ TCallstack::GetTlsCallstack() = ev->Callstack;
+ TCallstack::GetTlsCallstack().SetLinesToSkip();
+#endif
+ CurrentRecipient = recipient;
+ CurrentActorScheduledEventsCounter = 0;
+
+ if (executed == 0) {
+ double usec = Ctx.AddActivationStats(AtomicLoad(&mailbox->ScheduleMoment), hpprev);
+ if (usec > 500) {
+ GLOBAL_LWPROBE(ACTORLIB_PROVIDER, SlowActivation, Ctx.PoolId, usec / 1000.0);
+ }
+ }
+
+ i64 usecDeliv = Ctx.AddEventDeliveryStats(ev->SendTime, hpprev);
+ if (usecDeliv > 5000) {
+ double sinceActivationMs = NHPTimer::GetSeconds(hpprev - hpstart) * 1000.0;
+ LwTraceSlowDelivery(ev.Get(), actor, Ctx.PoolId, CurrentRecipient, NHPTimer::GetSeconds(hpprev - ev->SendTime) * 1000.0, sinceActivationMs, executed);
+ }
+
+ ui32 evTypeForTracing = ev->Type;
+
+ ui32 activityType = actor->GetActivityType();
+ if (activityType != prevActivityType) {
+ prevActivityType = activityType;
+ NProfiling::TMemoryTagScope::Reset(ActorSystem->MemProfActivityBase + activityType);
+ }
+
+ actor->Receive(ev, ctx);
+
+ size_t dyingActorsCnt = DyingActors.size();
+ Ctx.UpdateActorsStats(dyingActorsCnt);
+ if (dyingActorsCnt) {
+ DropUnregistered();
+ actor = nullptr;
+ }
+
+ if (mailbox->IsEmpty()) // was not-free and become free, we must reclaim mailbox
+ reclaimAsFree = true;
+
+ hpnow = GetCycleCountFast();
+ NHPTimer::STime elapsed = Ctx.AddEventProcessingStats(hpprev, hpnow, activityType, CurrentActorScheduledEventsCounter);
+ if (elapsed > 1000000) {
+ LwTraceSlowEvent(ev.Get(), evTypeForTracing, actor, Ctx.PoolId, activityType, CurrentRecipient, NHPTimer::GetSeconds(elapsed) * 1000.0);
+ }
+
+ // The actor might have been destroyed
+ if (actor)
+ actor->AddElapsedTicks(elapsed);
+
+ CurrentRecipient = TActorId();
+ } else {
+ TAutoPtr<IEventHandle> nonDelivered = ev->ForwardOnNondelivery(TEvents::TEvUndelivered::ReasonActorUnknown);
+ if (nonDelivered.Get()) {
+ ActorSystem->Send(nonDelivered);
+ } else {
+ Ctx.IncrementNonDeliveredEvents();
+ }
+ hpnow = GetCycleCountFast();
+ }
+
+ hpprev = hpnow;
+
+ // Soft preemption in united pool
+ if (Ctx.SoftDeadlineTs < (ui64)hpnow) {
+ AtomicStore(&mailbox->ScheduleMoment, hpnow);
+ Ctx.IncrementMailboxPushedOutBySoftPreemption();
+ LWTRACK(MailboxPushedOutBySoftPreemption,
+ Ctx.Orbit,
+ Ctx.PoolId,
+ Ctx.Executor->GetName(),
+ executed + 1,
+ CyclesToDuration(hpnow - hpstart),
+ Ctx.WorkerId,
+ recipient.ToString(),
+ SafeTypeName(actor));
+ break;
+ }
+
+ // time limit inside one mailbox passed, let others do some work
+ if (hpnow - hpstart > (i64)Ctx.TimePerMailboxTs) {
+ AtomicStore(&mailbox->ScheduleMoment, hpnow);
+ Ctx.IncrementMailboxPushedOutByTime();
+ LWTRACK(MailboxPushedOutByTime,
+ Ctx.Orbit,
+ Ctx.PoolId,
+ Ctx.Executor->GetName(),
+ executed + 1,
+ CyclesToDuration(hpnow - hpstart),
+ Ctx.WorkerId,
+ recipient.ToString(),
+ SafeTypeName(actor));
+ break;
+ }
+
+ if (executed + 1 == Ctx.EventsPerMailbox) {
+ AtomicStore(&mailbox->ScheduleMoment, hpnow);
+ Ctx.IncrementMailboxPushedOutByEventCount();
+ LWTRACK(MailboxPushedOutByEventCount,
+ Ctx.Orbit,
+ Ctx.PoolId,
+ Ctx.Executor->GetName(),
+ executed + 1,
+ CyclesToDuration(hpnow - hpstart),
+ Ctx.WorkerId,
+ recipient.ToString(),
+ SafeTypeName(actor));
+ break;
+ }
+ } else {
+ if (executed == 0)
+ Ctx.IncrementEmptyMailboxActivation();
+ LWTRACK(MailboxEmpty,
+ Ctx.Orbit,
+ Ctx.PoolId,
+ Ctx.Executor->GetName(),
+ executed,
+ CyclesToDuration(GetCycleCountFast() - hpstart),
+ Ctx.WorkerId,
+ recipient.ToString(),
+ SafeTypeName(actor));
+ break; // empty queue, leave
+ }
+ }
+
+ NProfiling::TMemoryTagScope::Reset(0);
+ TlsActivationContext = nullptr;
+ UnlockFromExecution(mailbox, Ctx.Executor, reclaimAsFree, hint, Ctx.WorkerId, RevolvingWriteCounter);
+ }
+
+ TThreadId TExecutorThread::GetThreadId() const {
+#ifdef _linux_
+ while (AtomicLoad(&ThreadId) == UnknownThreadId) {
+ NanoSleep(1000);
+ }
+#endif
+ return ThreadId;
+ }
+
+ void* TExecutorThread::ThreadProc() {
+#ifdef _linux_
+ pid_t tid = syscall(SYS_gettid);
+ AtomicSet(ThreadId, (ui64)tid);
+#endif
+
+#ifdef BALLOC
+ ThreadDisableBalloc();
+#endif
+
+ if (ThreadName) {
+ ::SetCurrentThreadName(ThreadName);
+ }
+
+ ExecutorPool->SetRealTimeMode();
+ TAffinityGuard affinity(ExecutorPool->Affinity());
+
+ NHPTimer::STime hpnow = GetCycleCountFast();
+ NHPTimer::STime hpprev = hpnow;
+ ui64 execCount = 0;
+ ui64 readyActivationCount = 0;
+ i64 execCycles = 0;
+ i64 nonExecCycles = 0;
+
+ for (;;) {
+ if (ui32 activation = ExecutorPool->GetReadyActivation(Ctx, ++RevolvingReadCounter)) {
+ LWTRACK(ActivationBegin, Ctx.Orbit, Ctx.CpuId, Ctx.PoolId, Ctx.WorkerId, NHPTimer::GetSeconds(Ctx.Lease.GetPreciseExpireTs()) * 1e3);
+ readyActivationCount++;
+ if (TMailboxHeader* header = Ctx.MailboxTable->Get(activation)) {
+ if (header->LockForExecution()) {
+ hpnow = GetCycleCountFast();
+ nonExecCycles += hpnow - hpprev;
+ hpprev = hpnow;
+ switch (header->Type) {
+ case TMailboxType::Simple:
+ Execute(static_cast<TMailboxTable::TSimpleMailbox*>(header), activation);
+ break;
+ case TMailboxType::Revolving:
+ Execute(static_cast<TMailboxTable::TRevolvingMailbox*>(header), activation);
+ break;
+ case TMailboxType::HTSwap:
+ Execute(static_cast<TMailboxTable::THTSwapMailbox*>(header), activation);
+ break;
+ case TMailboxType::ReadAsFilled:
+ Execute(static_cast<TMailboxTable::TReadAsFilledMailbox*>(header), activation);
+ break;
+ case TMailboxType::TinyReadAsFilled:
+ Execute(static_cast<TMailboxTable::TTinyReadAsFilledMailbox*>(header), activation);
+ break;
+ }
+ hpnow = GetCycleCountFast();
+ execCycles += hpnow - hpprev;
+ hpprev = hpnow;
+ execCount++;
+ if (execCycles + nonExecCycles > 39000000) { // every 15 ms at 2.6GHz, so 1000 items is 15 sec (solomon interval)
+ LWPROBE(ExecutorThreadStats, ExecutorPool->PoolId, ExecutorPool->GetName(), Ctx.WorkerId,
+ execCount, readyActivationCount,
+ NHPTimer::GetSeconds(execCycles) * 1000.0, NHPTimer::GetSeconds(nonExecCycles) * 1000.0);
+ execCount = 0;
+ readyActivationCount = 0;
+ execCycles = 0;
+ nonExecCycles = 0;
+ Ctx.UpdateThreadTime();
+ }
+ }
+ }
+ LWTRACK(ActivationEnd, Ctx.Orbit, Ctx.CpuId, Ctx.PoolId, Ctx.WorkerId);
+ Ctx.Orbit.Reset();
+ } else { // no activation means PrepareStop was called so thread must terminate
+ break;
+ }
+ }
+ return nullptr;
+ }
+
+ // there must be barrier and check-read with following cas
+ // or just cas w/o read.
+ // or queue unlocks must be performed with exchange and not generic write
+ // TODO: check performance of those options under contention
+
+ // placed here in hope for better compiler optimization
+
+ bool TMailboxHeader::MarkForSchedule() {
+ AtomicBarrier();
+ for (;;) {
+ const ui32 state = AtomicLoad(&ExecutionState);
+ switch (state) {
+ case TExecutionState::Inactive:
+ if (AtomicUi32Cas(&ExecutionState, TExecutionState::Scheduled, TExecutionState::Inactive))
+ return true;
+ break;
+ case TExecutionState::Scheduled:
+ return false;
+ case TExecutionState::Leaving:
+ if (AtomicUi32Cas(&ExecutionState, TExecutionState::LeavingMarked, TExecutionState::Leaving))
+ return true;
+ break;
+ case TExecutionState::Executing:
+ case TExecutionState::LeavingMarked:
+ return false;
+ case TExecutionState::Free:
+ if (AtomicUi32Cas(&ExecutionState, TExecutionState::FreeScheduled, TExecutionState::Free))
+ return true;
+ break;
+ case TExecutionState::FreeScheduled:
+ return false;
+ case TExecutionState::FreeLeaving:
+ if (AtomicUi32Cas(&ExecutionState, TExecutionState::FreeLeavingMarked, TExecutionState::FreeLeaving))
+ return true;
+ break;
+ case TExecutionState::FreeExecuting:
+ case TExecutionState::FreeLeavingMarked:
+ return false;
+ default:
+ Y_FAIL();
+ }
+ }
+ }
+
+ bool TMailboxHeader::LockForExecution() {
+ AtomicBarrier(); // strictly speaking here should be AtomicBarrier, but as we got mailboxes from queue - this barrier is already set implicitly and could be removed
+ for (;;) {
+ const ui32 state = AtomicLoad(&ExecutionState);
+ switch (state) {
+ case TExecutionState::Inactive:
+ return false;
+ case TExecutionState::Scheduled:
+ if (AtomicUi32Cas(&ExecutionState, TExecutionState::Executing, TExecutionState::Scheduled))
+ return true;
+ break;
+ case TExecutionState::Leaving:
+ case TExecutionState::Executing:
+ case TExecutionState::LeavingMarked:
+ return false;
+ case TExecutionState::Free:
+ if (AtomicUi32Cas(&ExecutionState, TExecutionState::FreeExecuting, TExecutionState::Free))
+ return true;
+ break;
+ case TExecutionState::FreeScheduled:
+ if (AtomicUi32Cas(&ExecutionState, TExecutionState::FreeExecuting, TExecutionState::FreeScheduled))
+ return true;
+ break;
+ case TExecutionState::FreeLeaving:
+ case TExecutionState::FreeExecuting:
+ case TExecutionState::FreeLeavingMarked:
+ return false;
+ default:
+ Y_FAIL();
+ }
+ }
+ }
+
+ bool TMailboxHeader::LockFromFree() {
+ AtomicBarrier();
+ for (;;) {
+ const ui32 state = AtomicLoad(&ExecutionState);
+ switch (state) {
+ case TExecutionState::Inactive:
+ case TExecutionState::Scheduled:
+ case TExecutionState::Leaving:
+ case TExecutionState::Executing:
+ case TExecutionState::LeavingMarked:
+ Y_FAIL();
+ case TExecutionState::Free:
+ if (AtomicUi32Cas(&ExecutionState, TExecutionState::Executing, TExecutionState::Free))
+ return true;
+ break;
+ case TExecutionState::FreeScheduled:
+ if (AtomicUi32Cas(&ExecutionState, TExecutionState::Executing, TExecutionState::FreeScheduled))
+ return true;
+ break;
+ case TExecutionState::FreeLeaving:
+ case TExecutionState::FreeExecuting:
+ case TExecutionState::FreeLeavingMarked:
+ return false;
+ default:
+ Y_FAIL();
+ }
+ }
+ }
+
+ void TMailboxHeader::UnlockFromExecution1() {
+ const ui32 state = AtomicLoad(&ExecutionState);
+ if (state == TExecutionState::Executing)
+ AtomicStore(&ExecutionState, (ui32)TExecutionState::Leaving);
+ else if (state == TExecutionState::FreeExecuting)
+ AtomicStore(&ExecutionState, (ui32)TExecutionState::FreeLeaving);
+ else
+ Y_FAIL();
+ AtomicBarrier();
+ }
+
+ bool TMailboxHeader::UnlockFromExecution2(bool wouldReschedule) {
+ AtomicBarrier();
+ for (;;) {
+ const ui32 state = AtomicLoad(&ExecutionState);
+ switch (state) {
+ case TExecutionState::Inactive:
+ case TExecutionState::Scheduled:
+ Y_FAIL();
+ case TExecutionState::Leaving:
+ if (!wouldReschedule) {
+ if (AtomicUi32Cas(&ExecutionState, TExecutionState::Inactive, TExecutionState::Leaving))
+ return false;
+ } else {
+ if (AtomicUi32Cas(&ExecutionState, TExecutionState::Scheduled, TExecutionState::Leaving))
+ return true;
+ }
+ break;
+ case TExecutionState::Executing:
+ Y_FAIL();
+ case TExecutionState::LeavingMarked:
+ if (AtomicUi32Cas(&ExecutionState, TExecutionState::Scheduled, TExecutionState::LeavingMarked))
+ return true;
+ break;
+ case TExecutionState::Free:
+ case TExecutionState::FreeScheduled:
+ Y_FAIL();
+ case TExecutionState::FreeLeaving:
+ if (!wouldReschedule) {
+ if (AtomicUi32Cas(&ExecutionState, TExecutionState::Free, TExecutionState::FreeLeaving))
+ return false;
+ } else {
+ if (AtomicUi32Cas(&ExecutionState, TExecutionState::FreeScheduled, TExecutionState::FreeLeaving))
+ return true;
+ }
+ break;
+ case TExecutionState::FreeExecuting:
+ Y_FAIL();
+ case TExecutionState::FreeLeavingMarked:
+ if (AtomicUi32Cas(&ExecutionState, TExecutionState::FreeScheduled, TExecutionState::FreeLeavingMarked))
+ return true;
+ break;
+ default:
+ Y_FAIL();
+ }
+ }
+ }
+
+ bool TMailboxHeader::UnlockAsFree(bool wouldReschedule) {
+ AtomicBarrier();
+ for (;;) {
+ const ui32 state = AtomicLoad(&ExecutionState);
+ switch (state) {
+ case TExecutionState::Inactive:
+ case TExecutionState::Scheduled:
+ Y_FAIL();
+ case TExecutionState::Leaving:
+ if (!wouldReschedule) {
+ if (AtomicUi32Cas(&ExecutionState, TExecutionState::Free, TExecutionState::Leaving))
+ return false;
+ } else {
+ if (AtomicUi32Cas(&ExecutionState, TExecutionState::FreeScheduled, TExecutionState::Leaving))
+ return true;
+ }
+ break;
+ case TExecutionState::Executing:
+ Y_FAIL();
+ case TExecutionState::LeavingMarked:
+ if (AtomicUi32Cas(&ExecutionState, TExecutionState::FreeScheduled, TExecutionState::LeavingMarked))
+ return true;
+ break;
+ case TExecutionState::Free:
+ case TExecutionState::FreeScheduled:
+ case TExecutionState::FreeLeaving:
+ case TExecutionState::FreeExecuting:
+ case TExecutionState::FreeLeavingMarked:
+ Y_FAIL();
+ default:
+ Y_FAIL();
+ }
+ }
+ }
+}
diff --git a/library/cpp/actors/core/executor_thread.h b/library/cpp/actors/core/executor_thread.h
new file mode 100644
index 0000000000..9d3c573f0d
--- /dev/null
+++ b/library/cpp/actors/core/executor_thread.h
@@ -0,0 +1,112 @@
+#pragma once
+
+#include "defs.h"
+#include "event.h"
+#include "actor.h"
+#include "actorsystem.h"
+#include "callstack.h"
+#include "probes.h"
+#include "worker_context.h"
+
+#include <library/cpp/actors/util/datetime.h>
+
+#include <util/system/thread.h>
+
+namespace NActors {
+
+ class TExecutorThread: public ISimpleThread {
+ public:
+ static constexpr TDuration DEFAULT_TIME_PER_MAILBOX =
+ TDuration::MilliSeconds(10);
+ static constexpr ui32 DEFAULT_EVENTS_PER_MAILBOX = 100;
+
+ TExecutorThread(TWorkerId workerId,
+ TWorkerId cpuId,
+ TActorSystem* actorSystem,
+ IExecutorPool* executorPool,
+ TMailboxTable* mailboxTable,
+ const TString& threadName,
+ TDuration timePerMailbox = DEFAULT_TIME_PER_MAILBOX,
+ ui32 eventsPerMailbox = DEFAULT_EVENTS_PER_MAILBOX);
+
+ TExecutorThread(TWorkerId workerId,
+ TActorSystem* actorSystem,
+ IExecutorPool* executorPool,
+ TMailboxTable* mailboxTable,
+ const TString& threadName,
+ TDuration timePerMailbox = DEFAULT_TIME_PER_MAILBOX,
+ ui32 eventsPerMailbox = DEFAULT_EVENTS_PER_MAILBOX)
+ : TExecutorThread(workerId, 0, actorSystem, executorPool, mailboxTable, threadName, timePerMailbox, eventsPerMailbox)
+ {}
+
+ TActorId RegisterActor(IActor* actor, TMailboxType::EType mailboxType = TMailboxType::HTSwap, ui32 poolId = Max<ui32>(),
+ const TActorId& parentId = TActorId());
+ TActorId RegisterActor(IActor* actor, TMailboxHeader* mailbox, ui32 hint, const TActorId& parentId = TActorId());
+ void UnregisterActor(TMailboxHeader* mailbox, ui64 localActorId);
+ void DropUnregistered();
+ const std::vector<THolder<IActor>>& GetUnregistered() const { return DyingActors; }
+
+ void Schedule(TInstant deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie = nullptr);
+ void Schedule(TMonotonic deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie = nullptr);
+ void Schedule(TDuration delta, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie = nullptr);
+
+ bool Send(TAutoPtr<IEventHandle> ev) {
+#ifdef USE_ACTOR_CALLSTACK
+ ev->Callstack = TCallstack::GetTlsCallstack();
+ ev->Callstack.Trace();
+#endif
+ Ctx.IncrementSentEvents();
+ return ActorSystem->Send(ev);
+ }
+
+ void GetCurrentStats(TExecutorThreadStats& statsCopy) const {
+ Ctx.GetCurrentStats(statsCopy);
+ }
+
+ TThreadId GetThreadId() const; // blocks, must be called after Start()
+ TWorkerId GetWorkerId() const { return Ctx.WorkerId; }
+
+ private:
+ void* ThreadProc();
+
+ template <typename TMailbox>
+ void Execute(TMailbox* mailbox, ui32 hint);
+
+ public:
+ TActorSystem* const ActorSystem;
+
+ private:
+ // Pool-specific
+ IExecutorPool* const ExecutorPool;
+
+ // Event-specific (currently executing)
+ TVector<THolder<IActor>> DyingActors;
+ TActorId CurrentRecipient;
+ ui64 CurrentActorScheduledEventsCounter = 0;
+
+ // Thread-specific
+ TWorkerContext Ctx;
+ ui64 RevolvingReadCounter = 0;
+ ui64 RevolvingWriteCounter = 0;
+ const TString ThreadName;
+ volatile TThreadId ThreadId = UnknownThreadId;
+ };
+
+ template <typename TMailbox>
+ void UnlockFromExecution(TMailbox* mailbox, IExecutorPool* executorPool, bool asFree, ui32 hint, TWorkerId workerId, ui64& revolvingWriteCounter) {
+ mailbox->UnlockFromExecution1();
+ const bool needReschedule1 = (nullptr != mailbox->Head());
+ if (!asFree) {
+ if (mailbox->UnlockFromExecution2(needReschedule1)) {
+ RelaxedStore<NHPTimer::STime>(&mailbox->ScheduleMoment, GetCycleCountFast());
+ executorPool->ScheduleActivationEx(hint, ++revolvingWriteCounter);
+ }
+ } else {
+ if (mailbox->UnlockAsFree(needReschedule1)) {
+ RelaxedStore<NHPTimer::STime>(&mailbox->ScheduleMoment, GetCycleCountFast());
+ executorPool->ScheduleActivationEx(hint, ++revolvingWriteCounter);
+ }
+ executorPool->ReclaimMailbox(TMailbox::MailboxType, hint, workerId, ++revolvingWriteCounter);
+ }
+ }
+}
diff --git a/library/cpp/actors/core/hfunc.h b/library/cpp/actors/core/hfunc.h
new file mode 100644
index 0000000000..26f3c65013
--- /dev/null
+++ b/library/cpp/actors/core/hfunc.h
@@ -0,0 +1,84 @@
+#pragma once
+
+#include "actor.h"
+#include "executor_thread.h"
+
+#include <util/system/defaults.h>
+
+#define HFunc(TEvType, HandleFunc) \
+ case TEvType::EventType: { \
+ typename TEvType::TPtr* x = reinterpret_cast<typename TEvType::TPtr*>(&ev); \
+ HandleFunc(*x, ctx); \
+ break; \
+ }
+
+#define hFunc(TEvType, HandleFunc) \
+ case TEvType::EventType: { \
+ typename TEvType::TPtr* x = reinterpret_cast<typename TEvType::TPtr*>(&ev); \
+ HandleFunc(*x); \
+ break; \
+ }
+
+#define HFuncTraced(TEvType, HandleFunc) \
+ case TEvType::EventType: { \
+ TRACE_EVENT_TYPE(Y_STRINGIZE(TEvType)); \
+ TEvType::TPtr* x = reinterpret_cast<TEvType::TPtr*>(&ev); \
+ HandleFunc(*x, ctx); \
+ break; \
+ }
+
+#define hFuncTraced(TEvType, HandleFunc) \
+ case TEvType::EventType: { \
+ TRACE_EVENT_TYPE(Y_STRINGIZE(TEvType)); \
+ typename TEvType::TPtr* x = reinterpret_cast<typename TEvType::TPtr*>(&ev); \
+ HandleFunc(*x); \
+ break; \
+ }
+
+#define HTemplFunc(TEvType, HandleFunc) \
+ case TEvType::EventType: { \
+ typename TEvType::TPtr* x = reinterpret_cast<typename TEvType::TPtr*>(&ev); \
+ HandleFunc(*x, ctx); \
+ break; \
+ }
+
+#define hTemplFunc(TEvType, HandleFunc) \
+ case TEvType::EventType: { \
+ typename TEvType::TPtr* x = reinterpret_cast<typename TEvType::TPtr*>(&ev); \
+ HandleFunc(*x); \
+ break; \
+ }
+
+#define SFunc(TEvType, HandleFunc) \
+ case TEvType::EventType: \
+ HandleFunc(ctx); \
+ break;
+
+#define sFunc(TEvType, HandleFunc) \
+ case TEvType::EventType: \
+ HandleFunc(); \
+ break;
+
+#define CFunc(TEventType, HandleFunc) \
+ case TEventType: \
+ HandleFunc(ctx); \
+ break;
+
+#define cFunc(TEventType, HandleFunc) \
+ case TEventType: \
+ HandleFunc(); \
+ break;
+
+#define FFunc(TEventType, HandleFunc) \
+ case TEventType: \
+ HandleFunc(ev, ctx); \
+ break;
+
+#define fFunc(TEventType, HandleFunc) \
+ case TEventType: \
+ HandleFunc(ev); \
+ break;
+
+#define IgnoreFunc(TEvType) \
+ case TEvType::EventType: \
+ break;
diff --git a/library/cpp/actors/core/interconnect.cpp b/library/cpp/actors/core/interconnect.cpp
new file mode 100644
index 0000000000..9fb33413b2
--- /dev/null
+++ b/library/cpp/actors/core/interconnect.cpp
@@ -0,0 +1,170 @@
+#include "interconnect.h"
+#include <util/digest/murmur.h>
+#include <google/protobuf/text_format.h>
+
+namespace NActors {
+
+ TNodeLocation::TNodeLocation(const NActorsInterconnect::TNodeLocation& location) {
+ const NProtoBuf::Descriptor *descriptor = NActorsInterconnect::TNodeLocation::descriptor();
+ const NActorsInterconnect::TNodeLocation *locp = &location;
+ NActorsInterconnect::TNodeLocation temp; // for legacy location case
+
+ // WalleConfig compatibility section
+ if (locp->HasBody()) {
+ if (locp == &location) {
+ temp.CopyFrom(*locp);
+ locp = &temp;
+ }
+ temp.SetUnit(::ToString(temp.GetBody()));
+ temp.ClearBody();
+ }
+
+ // legacy value processing
+ if (locp->HasDataCenterNum() || locp->HasRoomNum() || locp->HasRackNum() || locp->HasBodyNum()) {
+ if (locp == &location) {
+ temp.CopyFrom(*locp);
+ locp = &temp;
+ }
+ LegacyValue = TLegacyValue{temp.GetDataCenterNum(), temp.GetRoomNum(), temp.GetRackNum(), temp.GetBodyNum()};
+ temp.ClearDataCenterNum();
+ temp.ClearRoomNum();
+ temp.ClearRackNum();
+ temp.ClearBodyNum();
+
+ // legacy format must not interfere with new one
+ const NProtoBuf::Reflection *reflection = temp.GetReflection();
+ for (int i = 0, count = descriptor->field_count(); i < count; ++i) {
+ Y_VERIFY(!reflection->HasField(temp, descriptor->field(i)));
+ }
+
+ const auto& v = LegacyValue->DataCenter;
+ const char *p = reinterpret_cast<const char*>(&v);
+ temp.SetDataCenter(TString(p, strnlen(p, sizeof(ui32))));
+ temp.SetModule(::ToString(LegacyValue->Room));
+ temp.SetRack(::ToString(LegacyValue->Rack));
+ temp.SetUnit(::ToString(LegacyValue->Body));
+ }
+
+ auto makeString = [&] {
+ NProtoBuf::TextFormat::Printer p;
+ p.SetSingleLineMode(true);
+ TString s;
+ p.PrintToString(*locp, &s);
+ return s;
+ };
+
+ // modern format parsing
+ const NProtoBuf::Reflection *reflection = locp->GetReflection();
+ 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());
+ 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());
+ Items.emplace_back(TKeys::E(field.number()), field.length_delimited());
+ }
+ std::sort(Items.begin(), Items.end());
+ }
+
+ TNodeLocation::TNodeLocation(TFromSerialized, const TString& s)
+ : TNodeLocation(ParseLocation(s))
+ {}
+
+ NActorsInterconnect::TNodeLocation TNodeLocation::ParseLocation(const TString& s) {
+ NActorsInterconnect::TNodeLocation res;
+ const bool success = res.ParseFromString(s);
+ Y_VERIFY(success);
+ return res;
+ }
+
+ TString TNodeLocation::ToStringUpTo(TKeys::E upToKey) const {
+ const NProtoBuf::Descriptor *descriptor = NActorsInterconnect::TNodeLocation::descriptor();
+
+ TStringBuilder res;
+ for (const auto& [key, value] : Items) {
+ if (upToKey < key) {
+ break;
+ }
+ TString name;
+ if (const NProtoBuf::FieldDescriptor *field = descriptor->FindFieldByNumber(key)) {
+ name = field->options().GetExtension(NActorsInterconnect::PrintName);
+ } else {
+ name = ::ToString(int(key));
+ }
+ if (key != upToKey) {
+ res << name << "=" << value << "/";
+ } else {
+ res << value;
+ }
+ }
+ return res;
+ }
+
+ void TNodeLocation::Serialize(NActorsInterconnect::TNodeLocation *pb) const {
+ const NProtoBuf::Descriptor *descriptor = NActorsInterconnect::TNodeLocation::descriptor();
+ const NProtoBuf::Reflection *reflection = pb->GetReflection();
+ NProtoBuf::UnknownFieldSet *unknown = pb->mutable_unknown_fields();
+ for (const auto& [key, value] : Items) {
+ if (const NProtoBuf::FieldDescriptor *field = descriptor->FindFieldByNumber(key)) {
+ reflection->SetString(pb, field, value);
+ } else {
+ unknown->AddLengthDelimited(key)->assign(value);
+ }
+ }
+ }
+
+ TString TNodeLocation::GetSerializedLocation() const {
+ NActorsInterconnect::TNodeLocation pb;
+ Serialize(&pb);
+ TString s;
+ const bool success = pb.SerializeToString(&s);
+ Y_VERIFY(success);
+ return s;
+ }
+
+ TNodeLocation::TLegacyValue TNodeLocation::GetLegacyValue() const {
+ if (LegacyValue) {
+ return *LegacyValue;
+ }
+
+ ui32 dataCenterId = 0, moduleId = 0, rackId = 0, unitId = 0;
+
+ for (const auto& [key, value] : Items) {
+ switch (key) {
+ case TKeys::DataCenter:
+ memcpy(&dataCenterId, value.data(), Min<size_t>(sizeof(dataCenterId), value.length()));
+ break;
+
+ case TKeys::Module: {
+ const bool success = TryFromString(value, moduleId);
+ Y_VERIFY(success);
+ break;
+ }
+
+ case TKeys::Rack:
+ // hacky way to obtain numeric id by a rack name
+ if (!TryFromString(value, rackId)) {
+ rackId = MurmurHash<ui32>(value.data(), value.length());
+ }
+ break;
+
+ case TKeys::Unit: {
+ const bool success = TryFromString(value, unitId);
+ Y_VERIFY(success);
+ break;
+ }
+
+ default:
+ Y_FAIL("unexpected legacy key# %d", key);
+ }
+ }
+
+ return {dataCenterId, moduleId, rackId, unitId};
+ }
+
+} // NActors
diff --git a/library/cpp/actors/core/interconnect.h b/library/cpp/actors/core/interconnect.h
new file mode 100644
index 0000000000..8d1cbd1e77
--- /dev/null
+++ b/library/cpp/actors/core/interconnect.h
@@ -0,0 +1,248 @@
+#pragma once
+
+#include "events.h"
+#include "event_local.h"
+#include <library/cpp/actors/protos/interconnect.pb.h>
+#include <util/string/cast.h>
+#include <util/string/builder.h>
+
+namespace NActors {
+ class TNodeLocation {
+ public:
+ struct TKeys {
+ enum E : int {
+ DataCenter = 10,
+ Module = 20,
+ Rack = 30,
+ Unit = 40,
+ };
+ };
+
+ struct TLegacyValue {
+ ui32 DataCenter;
+ ui32 Room;
+ ui32 Rack;
+ ui32 Body;
+
+ auto ConvertToTuple() const { return std::make_tuple(DataCenter, Room, Rack, Body); }
+
+ int Compare(const TLegacyValue& other) const {
+ const auto x = ConvertToTuple();
+ const auto y = other.ConvertToTuple();
+ if (x < y) {
+ return -1;
+ } else if (y < x) {
+ return 1;
+ } else {
+ return 0;
+ }
+ }
+
+ friend bool operator ==(const TLegacyValue& x, const TLegacyValue& y) { return x.Compare(y) == 0; }
+ };
+
+ private:
+ std::optional<TLegacyValue> LegacyValue;
+ std::vector<std::pair<TKeys::E, TString>> Items;
+
+ public:
+ // generic ctors
+ TNodeLocation() = default;
+ TNodeLocation(const TNodeLocation&) = default;
+ TNodeLocation(TNodeLocation&&) = default;
+
+ // protobuf-parser ctor
+ explicit TNodeLocation(const NActorsInterconnect::TNodeLocation& location);
+
+ // serialized protobuf ctor
+ static constexpr struct TFromSerialized {} FromSerialized {};
+ TNodeLocation(TFromSerialized, const TString& s);
+
+ // parser helper function
+ static NActorsInterconnect::TNodeLocation ParseLocation(const TString& s);
+
+ // assignment operators
+ TNodeLocation& operator =(const TNodeLocation&) = default;
+ TNodeLocation& operator =(TNodeLocation&&) = default;
+
+ void Serialize(NActorsInterconnect::TNodeLocation *pb) const;
+ TString GetSerializedLocation() const;
+
+ TString GetDataCenterId() const { return ToStringUpTo(TKeys::DataCenter); }
+ TString GetModuleId() const { return ToStringUpTo(TKeys::Module); }
+ TString GetRackId() const { return ToStringUpTo(TKeys::Rack); }
+ TString ToString() const { return ToStringUpTo(TKeys::E(Max<int>())); }
+ TString ToStringUpTo(TKeys::E upToKey) const;
+
+ TLegacyValue GetLegacyValue() const;
+
+ const std::vector<std::pair<TKeys::E, TString>>& GetItems() const { return Items; }
+
+ bool HasKey(TKeys::E key) const {
+ auto comp = [](const auto& p, TKeys::E value) { return p.first < value; };
+ const auto it = std::lower_bound(Items.begin(), Items.end(), key, comp);
+ return it != Items.end() && it->first == key;
+ }
+
+ int Compare(const TNodeLocation& other) const {
+ if (LegacyValue || other.LegacyValue) {
+ return GetLegacyValue().Compare(other.GetLegacyValue());
+ } else if (Items < other.Items) {
+ return -1;
+ } else if (other.Items < Items) {
+ return 1;
+ } else {
+ return 0;
+ }
+ }
+
+ void InheritLegacyValue(const TNodeLocation& other) {
+ LegacyValue = other.GetLegacyValue();
+ }
+
+ friend bool operator ==(const TNodeLocation& x, const TNodeLocation& y) { return x.Compare(y) == 0; }
+ friend bool operator !=(const TNodeLocation& x, const TNodeLocation& y) { return x.Compare(y) != 0; }
+ friend bool operator < (const TNodeLocation& x, const TNodeLocation& y) { return x.Compare(y) < 0; }
+ friend bool operator <=(const TNodeLocation& x, const TNodeLocation& y) { return x.Compare(y) <= 0; }
+ friend bool operator > (const TNodeLocation& x, const TNodeLocation& y) { return x.Compare(y) > 0; }
+ friend bool operator >=(const TNodeLocation& x, const TNodeLocation& y) { return x.Compare(y) >= 0; }
+ };
+
+ struct TEvInterconnect {
+ enum EEv {
+ EvForward = EventSpaceBegin(TEvents::ES_INTERCONNECT),
+ EvResolveNode, // resolve info about node (internal)
+ EvNodeAddress, // node info (internal)
+ EvConnectNode, // request proxy to establish connection (like: we would send something there soon)
+ EvAcceptIncoming,
+ EvNodeConnected, // node connected notify
+ EvNodeDisconnected, // node disconnected notify
+ EvRegisterNode,
+ EvRegisterNodeResult,
+ EvListNodes,
+ EvNodesInfo,
+ EvDisconnect,
+ EvGetNode,
+ EvNodeInfo,
+ EvClosePeerSocket,
+ EvCloseInputSession,
+ EvPoisonSession,
+ EvTerminate,
+ EvEnd
+ };
+
+ enum ESubscribes {
+ SubConnected,
+ SubDisconnected,
+ };
+
+ static_assert(EvEnd < EventSpaceEnd(TEvents::ES_INTERCONNECT), "expect EvEnd < EventSpaceEnd(TEvents::ES_INTERCONNECT)");
+
+ struct TEvResolveNode;
+ struct TEvNodeAddress;
+
+ struct TEvConnectNode: public TEventBase<TEvConnectNode, EvConnectNode> {
+ DEFINE_SIMPLE_LOCAL_EVENT(TEvConnectNode, "TEvInterconnect::TEvConnectNode")
+ };
+
+ struct TEvAcceptIncoming;
+
+ struct TEvNodeConnected: public TEventLocal<TEvNodeConnected, EvNodeConnected> {
+ DEFINE_SIMPLE_LOCAL_EVENT(TEvNodeConnected, "TEvInterconnect::TEvNodeConnected")
+ TEvNodeConnected(ui32 node) noexcept
+ : NodeId(node)
+ {
+ }
+ const ui32 NodeId;
+ };
+
+ struct TEvNodeDisconnected: public TEventLocal<TEvNodeDisconnected, EvNodeDisconnected> {
+ DEFINE_SIMPLE_LOCAL_EVENT(TEvNodeDisconnected, "TEvInterconnect::TEvNodeDisconnected")
+ TEvNodeDisconnected(ui32 node) noexcept
+ : NodeId(node)
+ {
+ }
+ const ui32 NodeId;
+ };
+
+ struct TEvRegisterNode;
+ struct TEvRegisterNodeResult;
+
+ struct TEvListNodes: public TEventLocal<TEvListNodes, EvListNodes> {
+ };
+
+ struct TNodeInfo {
+ ui32 NodeId;
+ TString Address;
+ TString Host;
+ TString ResolveHost;
+ ui16 Port;
+ TNodeLocation Location;
+
+ TNodeInfo() = default;
+ TNodeInfo(const TNodeInfo&) = default;
+ TNodeInfo& operator =(const TNodeInfo&) = default;
+ TNodeInfo(ui32 nodeId,
+ const TString& address,
+ const TString& host,
+ const TString& resolveHost,
+ ui16 port,
+ const TNodeLocation& location)
+ : NodeId(nodeId)
+ , Address(address)
+ , Host(host)
+ , ResolveHost(resolveHost)
+ , Port(port)
+ , Location(location)
+ {
+ }
+
+ operator ui32() const {
+ return NodeId;
+ }
+ };
+
+ struct TEvNodesInfo: public TEventLocal<TEvNodesInfo, EvNodesInfo> {
+ TVector<TNodeInfo> Nodes;
+
+ const TNodeInfo* GetNodeInfo(ui32 nodeId) const {
+ for (const auto& x : Nodes) {
+ if (x.NodeId == nodeId)
+ return &x;
+ }
+ return nullptr;
+ }
+ };
+
+ struct TEvDisconnect;
+
+ struct TEvGetNode: public TEventLocal<TEvGetNode, EvGetNode> {
+ ui32 NodeId;
+ TInstant Deadline;
+
+ TEvGetNode(ui32 nodeId, TInstant deadline = TInstant::Max())
+ : NodeId(nodeId)
+ , Deadline(deadline)
+ {
+ }
+ };
+
+ struct TEvNodeInfo: public TEventLocal<TEvNodeInfo, EvNodeInfo> {
+ TEvNodeInfo(ui32 nodeId)
+ : NodeId(nodeId)
+ {
+ }
+
+ ui32 NodeId;
+ THolder<TNodeInfo> Node;
+ };
+
+ struct TEvClosePeerSocket : TEventLocal<TEvClosePeerSocket, EvClosePeerSocket> {};
+
+ struct TEvCloseInputSession : TEventLocal<TEvCloseInputSession, EvCloseInputSession> {};
+
+ struct TEvPoisonSession : TEventLocal<TEvPoisonSession, EvPoisonSession> {};
+
+ struct TEvTerminate : TEventLocal<TEvTerminate, EvTerminate> {};
+ };
+}
diff --git a/library/cpp/actors/core/invoke.h b/library/cpp/actors/core/invoke.h
new file mode 100644
index 0000000000..931a9767dd
--- /dev/null
+++ b/library/cpp/actors/core/invoke.h
@@ -0,0 +1,110 @@
+#pragma once
+
+#include "actor_bootstrapped.h"
+#include "events.h"
+#include "event_local.h"
+
+#include <any>
+#include <type_traits>
+#include <utility>
+#include <variant>
+
+#include <util/system/type_name.h>
+
+namespace NActors {
+
+ struct TEvents::TEvInvokeResult
+ : TEventLocal<TEvInvokeResult, TSystem::InvokeResult>
+ {
+ using TProcessCallback = std::function<void(TEvInvokeResult&, const TActorContext&)>;
+ TProcessCallback ProcessCallback;
+ std::variant<std::any /* the value */, std::exception_ptr> Result;
+
+ // This constructor creates TEvInvokeResult with the result of calling callback(args...) or exception_ptr,
+ // if exception occurs during evaluation.
+ template<typename TCallback, typename... TArgs>
+ TEvInvokeResult(TProcessCallback&& process, TCallback&& callback, TArgs&&... args)
+ : ProcessCallback(std::move(process))
+ {
+ try {
+ if constexpr (std::is_void_v<std::invoke_result_t<TCallback, TArgs...>>) {
+ // just invoke callback without saving any value
+ std::invoke(std::forward<TCallback>(callback), std::forward<TArgs>(args)...);
+ } else {
+ Result.emplace<std::any>(std::invoke(std::forward<TCallback>(callback), std::forward<TArgs>(args)...));
+ }
+ } catch (...) {
+ Result.emplace<std::exception_ptr>(std::current_exception());
+ }
+ }
+
+ void Process(const TActorContext& ctx) {
+ ProcessCallback(*this, ctx);
+ }
+
+ template<typename TCallback>
+ std::invoke_result_t<TCallback, const TActorContext&> GetResult() {
+ using T = std::invoke_result_t<TCallback, const TActorContext&>;
+ return std::visit([](auto& arg) -> T {
+ using TArg = std::decay_t<decltype(arg)>;
+ 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());
+ } else if (auto *value = std::any_cast<T>(&arg)) {
+ return std::move(*value);
+ } else {
+ Y_FAIL("unspported return type for TEvInvokeResult: actual# %s != expected# %s",
+ TypeName(arg.type()).data(), TypeName<T>().data());
+ }
+ }, Result);
+ }
+ };
+
+ // Invoke Actor is used to make different procedure calls in specific threads pools.
+ //
+ // Actor is created by CreateInvokeActor(callback, complete) where `callback` is the function that will be invoked
+ // upon actor registration, which will issue then TEvInvokeResult to the parent actor with the result of called
+ // function. If the called function throws exception, then the exception will arrive in the result. Receiver of
+ // this message can either handle it by its own means calling ev.GetResult() (which will rethrow exception if it
+ // has occured in called function or return its return value; notice that when there is no return value, then
+ // GetResult() should also be called to prevent losing exception), or invoke ev.Process(), which will invoke
+ // callback provided as `complete` parameter to the CreateInvokeActor function. Complete handler is invoked with
+ // the result-getter lambda as the first argument and the actor system context as the second one. Result-getter
+ // should be called to obtain resulting value or exception like the GetResult() method of the TEvInvokeResult event.
+ //
+ // Notice that `callback` execution usually occurs in separate actor on separate mailbox and should not use parent
+ // actor's class. But `complete` handler is invoked in parent context and can use its contents. Do not forget to
+ // handle TEvInvokeResult event by calling Process/GetResult method, whichever is necessary.
+
+ template<typename TCallback, typename TCompletion, ui32 Activity>
+ class TInvokeActor : public TActorBootstrapped<TInvokeActor<TCallback, TCompletion, Activity>> {
+ TCallback Callback;
+ TCompletion Complete;
+
+ public:
+ static constexpr auto ActorActivityType() {
+ return static_cast<IActor::EActorActivity>(Activity);
+ }
+
+ TInvokeActor(TCallback&& callback, TCompletion&& complete)
+ : Callback(std::move(callback))
+ , Complete(std::move(complete))
+ {}
+
+ void Bootstrap(const TActorId& parentId, const TActorContext& ctx) {
+ auto process = [complete = std::move(Complete)](TEvents::TEvInvokeResult& res, const TActorContext& ctx) {
+ complete([&] { return res.GetResult<TCallback>(); }, ctx);
+ };
+ ctx.Send(parentId, new TEvents::TEvInvokeResult(std::move(process), std::move(Callback), ctx));
+ TActorBootstrapped<TInvokeActor>::Die(ctx);
+ }
+ };
+
+ template<ui32 Activity, typename TCallback, typename TCompletion>
+ std::unique_ptr<IActor> CreateInvokeActor(TCallback&& callback, TCompletion&& complete) {
+ return std::make_unique<TInvokeActor<std::decay_t<TCallback>, std::decay_t<TCompletion>, Activity>>(
+ std::forward<TCallback>(callback), std::forward<TCompletion>(complete));
+ }
+
+} // NActors
diff --git a/library/cpp/actors/core/io_dispatcher.cpp b/library/cpp/actors/core/io_dispatcher.cpp
new file mode 100644
index 0000000000..90699ff16c
--- /dev/null
+++ b/library/cpp/actors/core/io_dispatcher.cpp
@@ -0,0 +1,234 @@
+#include "io_dispatcher.h"
+#include "actor_bootstrapped.h"
+#include "hfunc.h"
+#include <util/system/mutex.h>
+#include <util/system/condvar.h>
+#include <util/system/thread.h>
+#include <map>
+#include <list>
+
+namespace NActors {
+
+ class TIoDispatcherActor : public TActorBootstrapped<TIoDispatcherActor> {
+ enum {
+ EvNotifyThreadStopped = EventSpaceBegin(TEvents::ES_PRIVATE),
+ };
+
+ ////////////////////////////////////////////////////////////////////////////////////////////////////////////////
+ // IO task queue
+ ////////////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+ class TTask {
+ TInstant Timestamp;
+ std::function<void()> Callback;
+
+ public:
+ TTask(TInstant timestamp, TEvInvokeQuery *ev)
+ : Timestamp(timestamp)
+ , Callback(std::move(ev->Callback))
+ {}
+
+ void Execute() {
+ Callback();
+ }
+
+ TInstant GetTimestamp() const {
+ return Timestamp;
+ }
+ };
+
+ class TTaskQueue {
+ std::list<TTask> Tasks;
+ TMutex Mutex;
+ TCondVar CondVar;
+ size_t NumThreadsToStop = 0;
+
+ public:
+ void Enqueue(TInstant timestamp, TEvInvokeQuery *ev) {
+ std::list<TTask> list;
+ list.emplace_back(timestamp, ev);
+ with_lock (Mutex) {
+ Tasks.splice(Tasks.end(), std::move(list));
+ }
+ CondVar.Signal();
+ }
+
+ bool Dequeue(std::list<TTask>& list, bool *sendNotify) {
+ with_lock (Mutex) {
+ CondVar.Wait(Mutex, [&] { return NumThreadsToStop || !Tasks.empty(); });
+ if (NumThreadsToStop) {
+ *sendNotify = NumThreadsToStop != Max<size_t>();
+ if (*sendNotify) {
+ --NumThreadsToStop;
+ }
+ return false;
+ } else {
+ list.splice(list.end(), Tasks, Tasks.begin());
+ return true;
+ }
+ }
+ }
+
+ void Stop() {
+ with_lock (Mutex) {
+ NumThreadsToStop = Max<size_t>();
+ }
+ CondVar.BroadCast();
+ }
+
+ void StopOne() {
+ with_lock (Mutex) {
+ ++NumThreadsToStop;
+ Y_VERIFY(NumThreadsToStop);
+ }
+ CondVar.Signal();
+ }
+
+ std::optional<TInstant> GetEarliestTaskTimestamp() {
+ with_lock (Mutex) {
+ return Tasks.empty() ? std::nullopt : std::make_optional(Tasks.front().GetTimestamp());
+ }
+ }
+ };
+
+ TTaskQueue TaskQueue;
+
+ ////////////////////////////////////////////////////////////////////////////////////////////////////////////////
+ // IO dispatcher threads
+ ////////////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+ class TThread : public ISimpleThread {
+ TIoDispatcherActor& Actor;
+ TActorSystem* const ActorSystem;
+
+ public:
+ TThread(TIoDispatcherActor& actor, TActorSystem *actorSystem)
+ : Actor(actor)
+ , ActorSystem(actorSystem)
+ {
+ Start();
+ }
+
+ void *ThreadProc() override {
+ SetCurrentThreadName("kikimr IO");
+ for (;;) {
+ std::list<TTask> tasks;
+ bool sendNotify;
+ if (!Actor.TaskQueue.Dequeue(tasks, &sendNotify)) {
+ if (sendNotify) {
+ ActorSystem->Send(new IEventHandle(EvNotifyThreadStopped, 0, Actor.SelfId(), TActorId(),
+ nullptr, TThread::CurrentThreadId()));
+ }
+ break;
+ }
+ for (TTask& task : tasks) {
+ task.Execute();
+ ++*Actor.TasksCompleted;
+ }
+ }
+ return nullptr;
+ }
+ };
+
+ static constexpr size_t MinThreadCount = 4;
+ static constexpr size_t MaxThreadCount = 64;
+ std::map<TThread::TId, std::unique_ptr<TThread>> Threads;
+ size_t NumRunningThreads = 0;
+
+ void StartThread() {
+ auto thread = std::make_unique<TThread>(*this, TlsActivationContext->ExecutorThread.ActorSystem);
+ const TThread::TId id = thread->Id();
+ Threads.emplace(id, std::move(thread));
+ *NumThreads = ++NumRunningThreads;
+ ++*ThreadsStarted;
+ }
+
+ void StopThread() {
+ Y_VERIFY(Threads.size());
+ TaskQueue.StopOne();
+ *NumThreads = --NumRunningThreads;
+ ++*ThreadsStopped;
+ }
+
+ ////////////////////////////////////////////////////////////////////////////////////////////////////////////////
+ // Counters
+ ////////////////////////////////////////////////////////////////////////////////////////////////////////////////
+
+ NMonitoring::TDynamicCounters::TCounterPtr NumThreads;
+ NMonitoring::TDynamicCounters::TCounterPtr TasksAdded;
+ NMonitoring::TDynamicCounters::TCounterPtr TasksCompleted;
+ NMonitoring::TDynamicCounters::TCounterPtr ThreadsStarted;
+ NMonitoring::TDynamicCounters::TCounterPtr ThreadsStopped;
+
+ public:
+ TIoDispatcherActor(const NMonitoring::TDynamicCounterPtr& counters)
+ : NumThreads(counters->GetCounter("NumThreads"))
+ , TasksAdded(counters->GetCounter("TasksAdded", true))
+ , TasksCompleted(counters->GetCounter("TasksCompleted", true))
+ , ThreadsStarted(counters->GetCounter("ThreadsStarted", true))
+ , ThreadsStopped(counters->GetCounter("ThreadsStopped", true))
+ {}
+
+ ~TIoDispatcherActor() override {
+ TaskQueue.Stop();
+ }
+
+ void Bootstrap() {
+ while (NumRunningThreads < MinThreadCount) {
+ StartThread();
+ }
+ HandleWakeup();
+ Become(&TThis::StateFunc);
+ }
+
+ void HandleThreadStopped(TAutoPtr<IEventHandle> ev) {
+ auto it = Threads.find(ev->Cookie);
+ Y_VERIFY(it != Threads.end());
+ it->second->Join();
+ Threads.erase(it);
+ }
+
+ void Handle(TEvInvokeQuery::TPtr ev) {
+ ++*TasksAdded;
+ TaskQueue.Enqueue(TActivationContext::Now(), ev->Get());
+ }
+
+ ////////////////////////////////////////////////////////////////////////////////////////////////////////////////
+ // Thread usage counter logic
+ ////////////////////////////////////////////////////////////////////////////////////////////////////////////////
+ std::optional<TInstant> IdleTimestamp;
+ static constexpr TDuration ThreadStartTime = TDuration::MilliSeconds(500);
+ static constexpr TDuration ThreadStopTime = TDuration::MilliSeconds(500);
+
+ void HandleWakeup() {
+ const TInstant now = TActivationContext::Now();
+ std::optional<TInstant> earliest = TaskQueue.GetEarliestTaskTimestamp();
+ if (earliest) {
+ if (now >= *earliest + ThreadStartTime && NumRunningThreads < MaxThreadCount) {
+ StartThread();
+ }
+ IdleTimestamp.reset();
+ } else if (!IdleTimestamp) {
+ IdleTimestamp = now;
+ } else if (now >= *IdleTimestamp + ThreadStopTime) {
+ IdleTimestamp.reset();
+ if (NumRunningThreads > MinThreadCount) {
+ StopThread();
+ }
+ }
+ Schedule(TDuration::MilliSeconds(100), new TEvents::TEvWakeup);
+ }
+
+ STRICT_STFUNC(StateFunc, {
+ fFunc(EvNotifyThreadStopped, HandleThreadStopped);
+ hFunc(TEvInvokeQuery, Handle);
+ cFunc(TEvents::TSystem::Wakeup, HandleWakeup);
+ cFunc(TEvents::TSystem::Poison, PassAway);
+ })
+ };
+
+ IActor *CreateIoDispatcherActor(const NMonitoring::TDynamicCounterPtr& counters) {
+ return new TIoDispatcherActor(counters);
+ }
+
+} // NActors
diff --git a/library/cpp/actors/core/io_dispatcher.h b/library/cpp/actors/core/io_dispatcher.h
new file mode 100644
index 0000000000..b0e4e60d1a
--- /dev/null
+++ b/library/cpp/actors/core/io_dispatcher.h
@@ -0,0 +1,38 @@
+#pragma once
+
+#include "actor.h"
+#include "event_local.h"
+#include "events.h"
+#include "actorsystem.h"
+#include "executor_thread.h"
+#include "executelater.h"
+
+namespace NActors {
+
+ struct TEvInvokeQuery : TEventLocal<TEvInvokeQuery, TEvents::TSystem::InvokeQuery> {
+ std::function<void()> Callback;
+
+ TEvInvokeQuery(std::function<void()>&& callback)
+ : Callback(std::move(callback))
+ {}
+ };
+
+ inline TActorId MakeIoDispatcherActorId() {
+ return TActorId(0, TStringBuf("IoDispatcher", 12));
+ }
+
+ extern IActor *CreateIoDispatcherActor(const NMonitoring::TDynamicCounterPtr& counters);
+
+ /* InvokeIoCallback enqueues callback() to be executed in IO thread pool and then return result in TEvInvokeResult
+ * message to parentId actor.
+ */
+ template<typename TCallback>
+ static void InvokeIoCallback(TCallback&& callback, ui32 poolId, IActor::EActivityType activityType) {
+ if (!TActivationContext::Send(new IEventHandle(MakeIoDispatcherActorId(), TActorId(),
+ new TEvInvokeQuery(callback)))) {
+ TActivationContext::Register(CreateExecuteLaterActor(std::move(callback), activityType), TActorId(),
+ TMailboxType::HTSwap, poolId);
+ }
+ }
+
+} // NActors
diff --git a/library/cpp/actors/core/lease.h b/library/cpp/actors/core/lease.h
new file mode 100644
index 0000000000..650ae7b122
--- /dev/null
+++ b/library/cpp/actors/core/lease.h
@@ -0,0 +1,56 @@
+#pragma once
+
+#include "defs.h"
+
+namespace NActors {
+ // Value representing specific worker's permission for exclusive use of CPU till specific deadline
+ struct TLease {
+ // Lower WorkerBits store current fast worker id
+ // All other higher bits store expiration (hard preemption) timestamp
+ using TValue = ui64;
+ TValue Value;
+
+ static constexpr ui64 WorkerIdMask = ui64((1ull << WorkerBits) - 1);
+ static constexpr ui64 ExpireTsMask = ~WorkerIdMask;
+
+ explicit constexpr TLease(ui64 value)
+ : Value(value)
+ {}
+
+ constexpr TLease(TWorkerId workerId, ui64 expireTs)
+ : Value((workerId & WorkerIdMask) | (expireTs & ExpireTsMask))
+ {}
+
+ TWorkerId GetWorkerId() const {
+ return Value & WorkerIdMask;
+ }
+
+ TLease NeverExpire() const {
+ return TLease(Value | ExpireTsMask);
+ }
+
+ bool IsNeverExpiring() const {
+ return (Value & ExpireTsMask) == ExpireTsMask;
+ }
+
+ ui64 GetExpireTs() const {
+ // Do not truncate worker id
+ // NOTE: it decrease accuracy, but improves performance
+ return Value;
+ }
+
+ ui64 GetPreciseExpireTs() const {
+ return Value & ExpireTsMask;
+ }
+
+ operator TValue() const {
+ return Value;
+ }
+ };
+
+ // Special expire timestamp values
+ static constexpr ui64 NeverExpire = ui64(-1);
+
+ // Special hard-preemption-in-progress lease
+ static constexpr TLease HardPreemptionLease = TLease(TLease::WorkerIdMask, NeverExpire);
+}
diff --git a/library/cpp/actors/core/log.cpp b/library/cpp/actors/core/log.cpp
new file mode 100644
index 0000000000..5f63b5af58
--- /dev/null
+++ b/library/cpp/actors/core/log.cpp
@@ -0,0 +1,753 @@
+#include "log.h"
+#include "log_settings.h"
+
+#include <library/cpp/monlib/service/pages/templates.h>
+
+static_assert(int(NActors::NLog::PRI_EMERG) == int(::TLOG_EMERG), "expect int(NActors::NLog::PRI_EMERG) == int(::TLOG_EMERG)");
+static_assert(int(NActors::NLog::PRI_ALERT) == int(::TLOG_ALERT), "expect int(NActors::NLog::PRI_ALERT) == int(::TLOG_ALERT)");
+static_assert(int(NActors::NLog::PRI_CRIT) == int(::TLOG_CRIT), "expect int(NActors::NLog::PRI_CRIT) == int(::TLOG_CRIT)");
+static_assert(int(NActors::NLog::PRI_ERROR) == int(::TLOG_ERR), "expect int(NActors::NLog::PRI_ERROR) == int(::TLOG_ERR)");
+static_assert(int(NActors::NLog::PRI_WARN) == int(::TLOG_WARNING), "expect int(NActors::NLog::PRI_WARN) == int(::TLOG_WARNING)");
+static_assert(int(NActors::NLog::PRI_NOTICE) == int(::TLOG_NOTICE), "expect int(NActors::NLog::PRI_NOTICE) == int(::TLOG_NOTICE)");
+static_assert(int(NActors::NLog::PRI_INFO) == int(::TLOG_INFO), "expect int(NActors::NLog::PRI_INFO) == int(::TLOG_INFO)");
+static_assert(int(NActors::NLog::PRI_DEBUG) == int(::TLOG_DEBUG), "expect int(NActors::NLog::PRI_DEBUG) == int(::TLOG_DEBUG)");
+static_assert(int(NActors::NLog::PRI_TRACE) == int(::TLOG_RESOURCES), "expect int(NActors::NLog::PRI_TRACE) == int(::TLOG_RESOURCES)");
+
+namespace {
+ struct TRecordWithNewline {
+ ELogPriority Priority;
+ TTempBuf Buf;
+
+ TRecordWithNewline(const TLogRecord& rec)
+ : Priority(rec.Priority)
+ , Buf(rec.Len + 1)
+ {
+ Buf.Append(rec.Data, rec.Len);
+ *Buf.Proceed(1) = '\n';
+ }
+
+ operator TLogRecord() const {
+ return TLogRecord(Priority, Buf.Data(), Buf.Filled());
+ }
+ };
+}
+
+namespace NActors {
+
+ class TLoggerCounters : public ILoggerMetrics {
+ public:
+ TLoggerCounters(TIntrusivePtr<NMonitoring::TDynamicCounters> counters)
+ : DynamicCounters(counters)
+ {
+ ActorMsgs_ = DynamicCounters->GetCounter("ActorMsgs", true);
+ DirectMsgs_ = DynamicCounters->GetCounter("DirectMsgs", true);
+ LevelRequests_ = DynamicCounters->GetCounter("LevelRequests", true);
+ IgnoredMsgs_ = DynamicCounters->GetCounter("IgnoredMsgs", true);
+ DroppedMsgs_ = DynamicCounters->GetCounter("DroppedMsgs", true);
+
+ AlertMsgs_ = DynamicCounters->GetCounter("AlertMsgs", true);
+ EmergMsgs_ = DynamicCounters->GetCounter("EmergMsgs", true);
+ }
+
+ ~TLoggerCounters() = default;
+
+ void IncActorMsgs() override {
+ ++*ActorMsgs_;
+ }
+ void IncDirectMsgs() override {
+ ++*DirectMsgs_;
+ }
+ void IncLevelRequests() override {
+ ++*LevelRequests_;
+ }
+ void IncIgnoredMsgs() override {
+ ++*IgnoredMsgs_;
+ }
+ void IncAlertMsgs() override {
+ ++*AlertMsgs_;
+ }
+ void IncEmergMsgs() override {
+ ++*EmergMsgs_;
+ }
+ void IncDroppedMsgs() override {
+ DroppedMsgs_->Inc();
+ };
+
+ void GetOutputHtml(IOutputStream& str) override {
+ HTML(str) {
+ DIV_CLASS("row") {
+ DIV_CLASS("col-md-12") {
+ H4() {
+ str << "Counters" << Endl;
+ }
+ DynamicCounters->OutputHtml(str);
+ }
+ }
+ }
+ }
+
+ private:
+ NMonitoring::TDynamicCounters::TCounterPtr ActorMsgs_;
+ NMonitoring::TDynamicCounters::TCounterPtr DirectMsgs_;
+ NMonitoring::TDynamicCounters::TCounterPtr LevelRequests_;
+ NMonitoring::TDynamicCounters::TCounterPtr IgnoredMsgs_;
+ NMonitoring::TDynamicCounters::TCounterPtr AlertMsgs_;
+ NMonitoring::TDynamicCounters::TCounterPtr EmergMsgs_;
+ // Dropped while the logger backend was unavailable
+ NMonitoring::TDynamicCounters::TCounterPtr DroppedMsgs_;
+
+ TIntrusivePtr<NMonitoring::TDynamicCounters> DynamicCounters;
+ };
+
+ class TLoggerMetrics : public ILoggerMetrics {
+ public:
+ TLoggerMetrics(std::shared_ptr<NMonitoring::TMetricRegistry> metrics)
+ : Metrics(metrics)
+ {
+ ActorMsgs_ = Metrics->Rate(NMonitoring::TLabels{{"sensor", "logger.actor_msgs"}});
+ DirectMsgs_ = Metrics->Rate(NMonitoring::TLabels{{"sensor", "logger.direct_msgs"}});
+ LevelRequests_ = Metrics->Rate(NMonitoring::TLabels{{"sensor", "logger.level_requests"}});
+ IgnoredMsgs_ = Metrics->Rate(NMonitoring::TLabels{{"sensor", "logger.ignored_msgs"}});
+ DroppedMsgs_ = Metrics->Rate(NMonitoring::TLabels{{"sensor", "logger.dropped_msgs"}});
+
+ AlertMsgs_ = Metrics->Rate(NMonitoring::TLabels{{"sensor", "logger.alert_msgs"}});
+ EmergMsgs_ = Metrics->Rate(NMonitoring::TLabels{{"sensor", "logger.emerg_msgs"}});
+ }
+
+ ~TLoggerMetrics() = default;
+
+ void IncActorMsgs() override {
+ ActorMsgs_->Inc();
+ }
+ void IncDirectMsgs() override {
+ DirectMsgs_->Inc();
+ }
+ void IncLevelRequests() override {
+ LevelRequests_->Inc();
+ }
+ void IncIgnoredMsgs() override {
+ IgnoredMsgs_->Inc();
+ }
+ void IncAlertMsgs() override {
+ AlertMsgs_->Inc();
+ }
+ void IncEmergMsgs() override {
+ EmergMsgs_->Inc();
+ }
+ void IncDroppedMsgs() override {
+ DroppedMsgs_->Inc();
+ };
+
+ void GetOutputHtml(IOutputStream& str) override {
+ HTML(str) {
+ DIV_CLASS("row") {
+ DIV_CLASS("col-md-12") {
+ H4() {
+ str << "Metrics" << Endl;
+ }
+ // TODO: Now, TMetricRegistry does not have the GetOutputHtml function
+ }
+ }
+ }
+ }
+
+ private:
+ NMonitoring::TRate* ActorMsgs_;
+ NMonitoring::TRate* DirectMsgs_;
+ NMonitoring::TRate* LevelRequests_;
+ NMonitoring::TRate* IgnoredMsgs_;
+ NMonitoring::TRate* AlertMsgs_;
+ NMonitoring::TRate* EmergMsgs_;
+ // Dropped while the logger backend was unavailable
+ NMonitoring::TRate* DroppedMsgs_;
+
+ std::shared_ptr<NMonitoring::TMetricRegistry> Metrics;
+ };
+
+ TAtomic TLoggerActor::IsOverflow = 0;
+
+ TLoggerActor::TLoggerActor(TIntrusivePtr<NLog::TSettings> settings,
+ TAutoPtr<TLogBackend> logBackend,
+ TIntrusivePtr<NMonitoring::TDynamicCounters> counters)
+ : TActor(&TLoggerActor::StateFunc)
+ , Settings(settings)
+ , LogBackend(logBackend.Release())
+ , Metrics(std::make_unique<TLoggerCounters>(counters))
+ {
+ }
+
+ TLoggerActor::TLoggerActor(TIntrusivePtr<NLog::TSettings> settings,
+ std::shared_ptr<TLogBackend> logBackend,
+ TIntrusivePtr<NMonitoring::TDynamicCounters> counters)
+ : TActor(&TLoggerActor::StateFunc)
+ , Settings(settings)
+ , LogBackend(logBackend)
+ , Metrics(std::make_unique<TLoggerCounters>(counters))
+ {
+ }
+
+ TLoggerActor::TLoggerActor(TIntrusivePtr<NLog::TSettings> settings,
+ TAutoPtr<TLogBackend> logBackend,
+ std::shared_ptr<NMonitoring::TMetricRegistry> metrics)
+ : TActor(&TLoggerActor::StateFunc)
+ , Settings(settings)
+ , LogBackend(logBackend.Release())
+ , Metrics(std::make_unique<TLoggerMetrics>(metrics))
+ {
+ }
+
+ TLoggerActor::TLoggerActor(TIntrusivePtr<NLog::TSettings> settings,
+ std::shared_ptr<TLogBackend> logBackend,
+ std::shared_ptr<NMonitoring::TMetricRegistry> metrics)
+ : TActor(&TLoggerActor::StateFunc)
+ , Settings(settings)
+ , LogBackend(logBackend)
+ , Metrics(std::make_unique<TLoggerMetrics>(metrics))
+ {
+ }
+
+ TLoggerActor::~TLoggerActor() {
+ }
+
+ void TLoggerActor::Log(TInstant time, NLog::EPriority priority, NLog::EComponent component, const char* c, ...) {
+ Metrics->IncDirectMsgs();
+ if (Settings && Settings->Satisfies(priority, component, 0ull)) {
+ va_list params;
+ va_start(params, c);
+ TString formatted;
+ vsprintf(formatted, c, params);
+
+ auto ok = OutputRecord(time, NLog::EPrio(priority), component, formatted);
+ Y_UNUSED(ok);
+ va_end(params);
+ }
+ }
+
+ void TLoggerActor::Throttle(const NLog::TSettings& settings) {
+ if (AtomicGet(IsOverflow))
+ Sleep(settings.ThrottleDelay);
+ }
+
+ void TLoggerActor::LogIgnoredCount(TInstant now) {
+ TString message = Sprintf("Ignored IgnoredCount# %" PRIu64 " log records due to logger overflow!", IgnoredCount);
+ if (!OutputRecord(now, NActors::NLog::EPrio::Error, Settings->LoggerComponent, message)) {
+ BecomeDefunct();
+ }
+ }
+
+ void TLoggerActor::HandleIgnoredEvent(TLogIgnored::TPtr& ev, const NActors::TActorContext& ctx) {
+ Y_UNUSED(ev);
+ LogIgnoredCount(ctx.Now());
+ IgnoredCount = 0;
+ PassedCount = 0;
+ }
+
+ void TLoggerActor::HandleIgnoredEventDrop() {
+ // logger backend is unavailable, just ignore
+ }
+
+ void TLoggerActor::WriteMessageStat(const NLog::TEvLog& ev) {
+ Metrics->IncActorMsgs();
+
+ const auto prio = ev.Level.ToPrio();
+
+ switch (prio) {
+ case ::NActors::NLog::EPrio::Alert:
+ Metrics->IncAlertMsgs();
+ break;
+ case ::NActors::NLog::EPrio::Emerg:
+ Metrics->IncEmergMsgs();
+ break;
+ default:
+ break;
+ }
+
+ }
+
+ void TLoggerActor::HandleLogEvent(NLog::TEvLog::TPtr& ev, const NActors::TActorContext& ctx) {
+ i64 delayMillisec = (ctx.Now() - ev->Get()->Stamp).MilliSeconds();
+ WriteMessageStat(*ev->Get());
+ if (Settings->AllowDrop) {
+ // Disable throttling if it was enabled previously
+ if (AtomicGet(IsOverflow))
+ AtomicSet(IsOverflow, 0);
+
+ // Check if some records have to be dropped
+ if ((PassedCount > 10 && delayMillisec > (i64)Settings->TimeThresholdMs) || IgnoredCount > 0) {
+ Metrics->IncIgnoredMsgs();
+ if (IgnoredCount == 0) {
+ ctx.Send(ctx.SelfID, new TLogIgnored());
+ }
+ ++IgnoredCount;
+ PassedCount = 0;
+ return;
+ }
+ PassedCount++;
+ } else {
+ // Enable of disable throttling depending on the load
+ if (delayMillisec > (i64)Settings->TimeThresholdMs && !AtomicGet(IsOverflow))
+ AtomicSet(IsOverflow, 1);
+ else if (delayMillisec <= (i64)Settings->TimeThresholdMs && AtomicGet(IsOverflow))
+ AtomicSet(IsOverflow, 0);
+ }
+
+ const auto prio = ev->Get()->Level.ToPrio();
+ if (!OutputRecord(ev->Get()->Stamp, prio, ev->Get()->Component, ev->Get()->Line)) {
+ BecomeDefunct();
+ }
+ }
+
+ void TLoggerActor::BecomeDefunct() {
+ Become(&TThis::StateDefunct);
+ Schedule(WakeupInterval, new TEvents::TEvWakeup);
+ }
+
+ void TLoggerActor::HandleLogComponentLevelRequest(TLogComponentLevelRequest::TPtr& ev, const NActors::TActorContext& ctx) {
+ Metrics->IncLevelRequests();
+ TString explanation;
+ int code = Settings->SetLevel(ev->Get()->Priority, ev->Get()->Component, explanation);
+ ctx.Send(ev->Sender, new TLogComponentLevelResponse(code, explanation));
+ }
+
+ void TLoggerActor::RenderComponentPriorities(IOutputStream& str) {
+ using namespace NLog;
+ HTML(str) {
+ H4() {
+ str << "Priority Settings for the Components";
+ }
+ TABLE_SORTABLE_CLASS("table") {
+ TABLEHEAD() {
+ TABLER() {
+ TABLEH() {
+ str << "Component";
+ }
+ TABLEH() {
+ str << "Level";
+ }
+ TABLEH() {
+ str << "Sampling Level";
+ }
+ TABLEH() {
+ str << "Sampling Rate";
+ }
+ }
+ }
+ TABLEBODY() {
+ for (EComponent i = Settings->MinVal; i < Settings->MaxVal; i++) {
+ auto name = Settings->ComponentName(i);
+ if (!*name)
+ continue;
+ NLog::TComponentSettings componentSettings = Settings->GetComponentSettings(i);
+
+ TABLER() {
+ TABLED() {
+ str << "<a href='logger?c=" << i << "'>" << name << "</a>";
+ }
+ TABLED() {
+ str << PriorityToString(EPrio(componentSettings.Raw.X.Level));
+ }
+ TABLED() {
+ str << PriorityToString(EPrio(componentSettings.Raw.X.SamplingLevel));
+ }
+ TABLED() {
+ str << componentSettings.Raw.X.SamplingRate;
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+
+ /*
+ * Logger INFO:
+ * 1. Current priority settings from components
+ * 2. Number of log messages (via actors events, directly)
+ * 3. Number of messages per components, per priority
+ * 4. Log level changes (last N changes)
+ */
+ void TLoggerActor::HandleMonInfo(NMon::TEvHttpInfo::TPtr& ev, const TActorContext& ctx) {
+ const auto& params = ev->Get()->Request.GetParams();
+ NLog::EComponent component = NLog::InvalidComponent;
+ NLog::EPriority priority = NLog::PRI_DEBUG;
+ NLog::EPriority samplingPriority = NLog::PRI_DEBUG;
+ ui32 samplingRate = 0;
+ bool hasComponent = false;
+ bool hasPriority = false;
+ bool hasSamplingPriority = false;
+ bool hasSamplingRate = false;
+ bool hasAllowDrop = false;
+ int allowDrop = 0;
+ if (params.Has("c")) {
+ if (TryFromString(params.Get("c"), component) && (component == NLog::InvalidComponent || Settings->IsValidComponent(component))) {
+ hasComponent = true;
+ if (params.Has("p")) {
+ int rawPriority;
+ if (TryFromString(params.Get("p"), rawPriority) && NLog::TSettings::IsValidPriority((NLog::EPriority)rawPriority)) {
+ priority = (NLog::EPriority)rawPriority;
+ hasPriority = true;
+ }
+ }
+ if (params.Has("sp")) {
+ int rawPriority;
+ if (TryFromString(params.Get("sp"), rawPriority) && NLog::TSettings::IsValidPriority((NLog::EPriority)rawPriority)) {
+ samplingPriority = (NLog::EPriority)rawPriority;
+ hasSamplingPriority = true;
+ }
+ }
+ if (params.Has("sr")) {
+ if (TryFromString(params.Get("sr"), samplingRate)) {
+ hasSamplingRate = true;
+ }
+ }
+ }
+ }
+ if (params.Has("allowdrop")) {
+ if (TryFromString(params.Get("allowdrop"), allowDrop)) {
+ hasAllowDrop = true;
+ }
+ }
+
+ TStringStream str;
+ if (hasComponent && !hasPriority && !hasSamplingPriority && !hasSamplingRate) {
+ NLog::TComponentSettings componentSettings = Settings->GetComponentSettings(component);
+ ui32 samplingRate = componentSettings.Raw.X.SamplingRate;
+ HTML(str) {
+ DIV_CLASS("row") {
+ DIV_CLASS("col-md-12") {
+ H4() {
+ str << "Current log settings for " << Settings->ComponentName(component) << Endl;
+ }
+ UL() {
+ LI() {
+ str << "Priority: "
+ << NLog::PriorityToString(NLog::EPrio(componentSettings.Raw.X.Level));
+ }
+ LI() {
+ str << "Sampling priority: "
+ << NLog::PriorityToString(NLog::EPrio(componentSettings.Raw.X.SamplingLevel));
+ }
+ LI() {
+ str << "Sampling rate: "
+ << samplingRate;
+ }
+ }
+ }
+ }
+
+ DIV_CLASS("row") {
+ DIV_CLASS("col-md-12") {
+ H4() {
+ str << "Change priority" << Endl;
+ }
+ UL() {
+ for (int p = NLog::PRI_EMERG; p <= NLog::PRI_TRACE; ++p) {
+ LI() {
+ str << "<a href='logger?c=" << component << "&p=" << p << "'>"
+ << NLog::PriorityToString(NLog::EPrio(p)) << "</a>";
+ }
+ }
+ }
+ H4() {
+ str << "Change sampling priority" << Endl;
+ }
+ UL() {
+ for (int p = NLog::PRI_EMERG; p <= NLog::PRI_TRACE; ++p) {
+ LI() {
+ str << "<a href='logger?c=" << component << "&sp=" << p << "'>"
+ << NLog::PriorityToString(NLog::EPrio(p)) << "</a>";
+ }
+ }
+ }
+ H4() {
+ str << "Change sampling rate" << Endl;
+ }
+ str << "<form method=\"GET\">" << Endl;
+ str << "Rate: <input type=\"number\" name=\"sr\" value=\"" << samplingRate << "\"/>" << Endl;
+ str << "<input type=\"hidden\" name=\"c\" value=\"" << component << "\">" << Endl;
+ str << "<input class=\"btn btn-primary\" type=\"submit\" value=\"Change\"/>" << Endl;
+ str << "</form>" << Endl;
+ H4() {
+ str << "<a href='logger'>Cancel</a>" << Endl;
+ }
+ }
+ }
+ }
+
+ } else {
+ TString explanation;
+ if (hasComponent && hasPriority) {
+ Settings->SetLevel(priority, component, explanation);
+ }
+ if (hasComponent && hasSamplingPriority) {
+ Settings->SetSamplingLevel(samplingPriority, component, explanation);
+ }
+ if (hasComponent && hasSamplingRate) {
+ Settings->SetSamplingRate(samplingRate, component, explanation);
+ }
+ if (hasAllowDrop) {
+ Settings->SetAllowDrop(allowDrop);
+ }
+
+ HTML(str) {
+ if (!explanation.empty()) {
+ DIV_CLASS("row") {
+ DIV_CLASS("col-md-12 alert alert-info") {
+ str << explanation;
+ }
+ }
+ }
+
+ DIV_CLASS("row") {
+ DIV_CLASS("col-md-6") {
+ RenderComponentPriorities(str);
+ }
+ DIV_CLASS("col-md-6") {
+ H4() {
+ str << "Change priority for all components";
+ }
+ TABLE_CLASS("table table-condensed") {
+ TABLEHEAD() {
+ TABLER() {
+ TABLEH() {
+ str << "Priority";
+ }
+ }
+ }
+ TABLEBODY() {
+ for (int p = NLog::PRI_EMERG; p <= NLog::PRI_TRACE; ++p) {
+ TABLER() {
+ TABLED() {
+ str << "<a href = 'logger?c=-1&p=" << p << "'>"
+ << NLog::PriorityToString(NLog::EPrio(p)) << "</a>";
+ }
+ }
+ }
+ }
+ }
+ H4() {
+ str << "Change sampling priority for all components";
+ }
+ TABLE_CLASS("table table-condensed") {
+ TABLEHEAD() {
+ TABLER() {
+ TABLEH() {
+ str << "Priority";
+ }
+ }
+ }
+ TABLEBODY() {
+ for (int p = NLog::PRI_EMERG; p <= NLog::PRI_TRACE; ++p) {
+ TABLER() {
+ TABLED() {
+ str << "<a href = 'logger?c=-1&sp=" << p << "'>"
+ << NLog::PriorityToString(NLog::EPrio(p)) << "</a>";
+ }
+ }
+ }
+ }
+ }
+ H4() {
+ str << "Change sampling rate for all components";
+ }
+ str << "<form method=\"GET\">" << Endl;
+ str << "Rate: <input type=\"number\" name=\"sr\" value=\"0\"/>" << Endl;
+ str << "<input type=\"hidden\" name=\"c\" value=\"-1\">" << Endl;
+ str << "<input class=\"btn btn-primary\" type=\"submit\" value=\"Change\"/>" << Endl;
+ str << "</form>" << Endl;
+ H4() {
+ str << "Drop log entries in case of overflow: "
+ << (Settings->AllowDrop ? "Enabled" : "Disabled");
+ }
+ str << "<form method=\"GET\">" << Endl;
+ str << "<input type=\"hidden\" name=\"allowdrop\" value=\"" << (Settings->AllowDrop ? "0" : "1") << "\"/>" << Endl;
+ str << "<input class=\"btn btn-primary\" type=\"submit\" value=\"" << (Settings->AllowDrop ? "Disable" : "Enable") << "\"/>" << Endl;
+ str << "</form>" << Endl;
+ }
+ }
+ Metrics->GetOutputHtml(str);
+ }
+ }
+
+ ctx.Send(ev->Sender, new NMon::TEvHttpInfoRes(str.Str()));
+ }
+
+ constexpr size_t TimeBufSize = 512;
+
+ bool TLoggerActor::OutputRecord(TInstant time, NLog::EPrio priority, NLog::EComponent component,
+ const TString& formatted) noexcept try {
+ const auto logPrio = ::ELogPriority(ui16(priority));
+
+ char buf[TimeBufSize];
+ switch (Settings->Format) {
+ case NActors::NLog::TSettings::PLAIN_FULL_FORMAT: {
+ TStringBuilder logRecord;
+ if (Settings->UseLocalTimestamps) {
+ logRecord << FormatLocalTimestamp(time, buf);
+ } else {
+ logRecord << time;
+ }
+ logRecord
+ << Settings->MessagePrefix
+ << " :" << Settings->ComponentName(component)
+ << " " << PriorityToString(priority)
+ << ": " << formatted;
+ LogBackend->WriteData(
+ TLogRecord(logPrio, logRecord.data(), logRecord.size()));
+ } break;
+
+ case NActors::NLog::TSettings::PLAIN_SHORT_FORMAT: {
+ TStringBuilder logRecord;
+ logRecord
+ << Settings->ComponentName(component)
+ << ": " << formatted;
+ LogBackend->WriteData(
+ TLogRecord(logPrio, logRecord.data(), logRecord.size()));
+ } break;
+
+ case NActors::NLog::TSettings::JSON_FORMAT: {
+ NJsonWriter::TBuf json;
+ json.BeginObject()
+ .WriteKey("@timestamp")
+ .WriteString(Settings->UseLocalTimestamps ? FormatLocalTimestamp(time, buf) : time.ToString().data())
+ .WriteKey("microseconds")
+ .WriteULongLong(time.MicroSeconds())
+ .WriteKey("host")
+ .WriteString(Settings->ShortHostName)
+ .WriteKey("cluster")
+ .WriteString(Settings->ClusterName)
+ .WriteKey("priority")
+ .WriteString(PriorityToString(priority))
+ .WriteKey("npriority")
+ .WriteInt((int)priority)
+ .WriteKey("component")
+ .WriteString(Settings->ComponentName(component))
+ .WriteKey("tag")
+ .WriteString("KIKIMR")
+ .WriteKey("revision")
+ .WriteInt(GetProgramSvnRevision())
+ .WriteKey("message")
+ .WriteString(formatted)
+ .EndObject();
+ auto logRecord = json.Str();
+ LogBackend->WriteData(
+ TLogRecord(logPrio, logRecord.data(), logRecord.size()));
+ } break;
+ }
+
+ return true;
+ } catch (...) {
+ return false;
+ }
+
+ void TLoggerActor::HandleLogEventDrop(const NLog::TEvLog::TPtr& ev) {
+ WriteMessageStat(*ev->Get());
+ Metrics->IncDroppedMsgs();
+ }
+
+ void TLoggerActor::HandleWakeup() {
+ Become(&TThis::StateFunc);
+ }
+
+ const char* TLoggerActor::FormatLocalTimestamp(TInstant time, char* buf) {
+ struct tm localTime;
+ time.LocalTime(&localTime);
+ int r = strftime(buf, TimeBufSize, "%Y-%m-%d-%H-%M-%S", &localTime);
+ Y_VERIFY(r != 0);
+ return buf;
+ }
+
+ TAutoPtr<TLogBackend> CreateSysLogBackend(const TString& ident,
+ bool logPError, bool logCons) {
+ int flags = 0;
+ if (logPError)
+ flags |= TSysLogBackend::LogPerror;
+ if (logCons)
+ flags |= TSysLogBackend::LogCons;
+
+ return new TSysLogBackend(ident.data(), TSysLogBackend::TSYSLOG_LOCAL1, flags);
+ }
+
+ class TStderrBackend: public TLogBackend {
+ public:
+ TStderrBackend() {
+ }
+ void WriteData(const TLogRecord& rec) override {
+#ifdef _MSC_VER
+ if (IsDebuggerPresent()) {
+ TString x;
+ x.reserve(rec.Len + 2);
+ x.append(rec.Data, rec.Len);
+ x.append('\n');
+ OutputDebugString(x.c_str());
+ }
+#endif
+ bool isOk = false;
+ do {
+ try {
+ TRecordWithNewline r(rec);
+ Cerr.Write(r.Buf.Data(), r.Buf.Filled());
+ isOk = true;
+ } catch (TSystemError err) {
+ // Interrupted system call
+ Y_UNUSED(err);
+ }
+ } while (!isOk);
+ }
+
+ void ReopenLog() override {
+ }
+
+ private:
+ const TString Indent;
+ };
+
+ class TLineFileLogBackend: public TFileLogBackend {
+ public:
+ TLineFileLogBackend(const TString& path)
+ : TFileLogBackend(path)
+ {
+ }
+
+ // Append newline after every record
+ void WriteData(const TLogRecord& rec) override {
+ TFileLogBackend::WriteData(TRecordWithNewline(rec));
+ }
+ };
+
+ class TCompositeLogBackend: public TLogBackend {
+ public:
+ TCompositeLogBackend(TVector<TAutoPtr<TLogBackend>>&& underlyingBackends)
+ : UnderlyingBackends(std::move(underlyingBackends))
+ {
+ }
+
+ void WriteData(const TLogRecord& rec) override {
+ for (auto& b: UnderlyingBackends) {
+ b->WriteData(rec);
+ }
+ }
+
+ void ReopenLog() override {
+ }
+
+ private:
+ TVector<TAutoPtr<TLogBackend>> UnderlyingBackends;
+ };
+
+ TAutoPtr<TLogBackend> CreateStderrBackend() {
+ return new TStderrBackend();
+ }
+
+ TAutoPtr<TLogBackend> CreateFileBackend(const TString& fileName) {
+ return new TLineFileLogBackend(fileName);
+ }
+
+ TAutoPtr<TLogBackend> CreateNullBackend() {
+ return new TNullLogBackend();
+ }
+
+ TAutoPtr<TLogBackend> CreateCompositeLogBackend(TVector<TAutoPtr<TLogBackend>>&& underlyingBackends) {
+ return new TCompositeLogBackend(std::move(underlyingBackends));
+ }
+}
diff --git a/library/cpp/actors/core/log.h b/library/cpp/actors/core/log.h
new file mode 100644
index 0000000000..c11a7cf3c1
--- /dev/null
+++ b/library/cpp/actors/core/log.h
@@ -0,0 +1,369 @@
+#pragma once
+
+#include "defs.h"
+
+#include "log_iface.h"
+#include "log_settings.h"
+#include "actorsystem.h"
+#include "events.h"
+#include "event_local.h"
+#include "hfunc.h"
+#include "mon.h"
+
+#include <util/generic/vector.h>
+#include <util/string/printf.h>
+#include <util/string/builder.h>
+#include <library/cpp/logger/all.h>
+#include <library/cpp/monlib/dynamic_counters/counters.h>
+#include <library/cpp/monlib/metrics/metric_registry.h>
+#include <library/cpp/json/writer/json.h>
+#include <library/cpp/svnversion/svnversion.h>
+
+#include <library/cpp/actors/memory_log/memlog.h>
+
+// TODO: limit number of messages per second
+// TODO: make TLogComponentLevelRequest/Response network messages
+
+#define IS_LOG_PRIORITY_ENABLED(actorCtxOrSystem, priority, component) \
+ (static_cast<::NActors::NLog::TSettings*>((actorCtxOrSystem).LoggerSettings()) && \
+ static_cast<::NActors::NLog::TSettings*>((actorCtxOrSystem).LoggerSettings())->Satisfies( \
+ static_cast<::NActors::NLog::EPriority>(priority), \
+ static_cast<::NActors::NLog::EComponent>(component), \
+ 0ull) \
+ )
+
+#define LOG_LOG_SAMPLED_BY(actorCtxOrSystem, priority, component, sampleBy, ...) \
+ do { \
+ ::NActors::NLog::TSettings* mSettings = static_cast<::NActors::NLog::TSettings*>((actorCtxOrSystem).LoggerSettings()); \
+ ::NActors::NLog::EPriority mPriority = static_cast<::NActors::NLog::EPriority>(priority); \
+ ::NActors::NLog::EComponent mComponent = static_cast<::NActors::NLog::EComponent>(component); \
+ if (mSettings && mSettings->Satisfies(mPriority, mComponent, sampleBy)) { \
+ ::NActors::MemLogAdapter( \
+ actorCtxOrSystem, priority, component, __VA_ARGS__); \
+ } \
+ } while (0) /**/
+
+#define LOG_LOG_S_SAMPLED_BY(actorCtxOrSystem, priority, component, sampleBy, stream) \
+ LOG_LOG_SAMPLED_BY(actorCtxOrSystem, priority, component, sampleBy, "%s", [&]() { \
+ TStringBuilder logStringBuilder; \
+ logStringBuilder << stream; \
+ return static_cast<TString>(logStringBuilder); \
+ }().data())
+
+#define LOG_LOG(actorCtxOrSystem, priority, component, ...) LOG_LOG_SAMPLED_BY(actorCtxOrSystem, priority, component, 0ull, __VA_ARGS__)
+#define LOG_LOG_S(actorCtxOrSystem, priority, component, stream) LOG_LOG_S_SAMPLED_BY(actorCtxOrSystem, priority, component, 0ull, stream)
+
+// use these macros for logging via actor system or actor context
+#define LOG_EMERG(actorCtxOrSystem, component, ...) LOG_LOG(actorCtxOrSystem, NActors::NLog::PRI_EMERG, component, __VA_ARGS__)
+#define LOG_ALERT(actorCtxOrSystem, component, ...) LOG_LOG(actorCtxOrSystem, NActors::NLog::PRI_ALERT, component, __VA_ARGS__)
+#define LOG_CRIT(actorCtxOrSystem, component, ...) LOG_LOG(actorCtxOrSystem, NActors::NLog::PRI_CRIT, component, __VA_ARGS__)
+#define LOG_ERROR(actorCtxOrSystem, component, ...) LOG_LOG(actorCtxOrSystem, NActors::NLog::PRI_ERROR, component, __VA_ARGS__)
+#define LOG_WARN(actorCtxOrSystem, component, ...) LOG_LOG(actorCtxOrSystem, NActors::NLog::PRI_WARN, component, __VA_ARGS__)
+#define LOG_NOTICE(actorCtxOrSystem, component, ...) LOG_LOG(actorCtxOrSystem, NActors::NLog::PRI_NOTICE, component, __VA_ARGS__)
+#define LOG_INFO(actorCtxOrSystem, component, ...) LOG_LOG(actorCtxOrSystem, NActors::NLog::PRI_INFO, component, __VA_ARGS__)
+#define LOG_DEBUG(actorCtxOrSystem, component, ...) LOG_LOG(actorCtxOrSystem, NActors::NLog::PRI_DEBUG, component, __VA_ARGS__)
+#define LOG_TRACE(actorCtxOrSystem, component, ...) LOG_LOG(actorCtxOrSystem, NActors::NLog::PRI_TRACE, component, __VA_ARGS__)
+
+#define LOG_EMERG_S(actorCtxOrSystem, component, stream) LOG_LOG_S(actorCtxOrSystem, NActors::NLog::PRI_EMERG, component, stream)
+#define LOG_ALERT_S(actorCtxOrSystem, component, stream) LOG_LOG_S(actorCtxOrSystem, NActors::NLog::PRI_ALERT, component, stream)
+#define LOG_CRIT_S(actorCtxOrSystem, component, stream) LOG_LOG_S(actorCtxOrSystem, NActors::NLog::PRI_CRIT, component, stream)
+#define LOG_ERROR_S(actorCtxOrSystem, component, stream) LOG_LOG_S(actorCtxOrSystem, NActors::NLog::PRI_ERROR, component, stream)
+#define LOG_WARN_S(actorCtxOrSystem, component, stream) LOG_LOG_S(actorCtxOrSystem, NActors::NLog::PRI_WARN, component, stream)
+#define LOG_NOTICE_S(actorCtxOrSystem, component, stream) LOG_LOG_S(actorCtxOrSystem, NActors::NLog::PRI_NOTICE, component, stream)
+#define LOG_INFO_S(actorCtxOrSystem, component, stream) LOG_LOG_S(actorCtxOrSystem, NActors::NLog::PRI_INFO, component, stream)
+#define LOG_DEBUG_S(actorCtxOrSystem, component, stream) LOG_LOG_S(actorCtxOrSystem, NActors::NLog::PRI_DEBUG, component, stream)
+#define LOG_TRACE_S(actorCtxOrSystem, component, stream) LOG_LOG_S(actorCtxOrSystem, NActors::NLog::PRI_TRACE, component, stream)
+
+#define LOG_EMERG_SAMPLED_BY(actorCtxOrSystem, component, sampleBy, ...) LOG_LOG_SAMPLED_BY(actorCtxOrSystem, NActors::NLog::PRI_EMERG, component, sampleBy, __VA_ARGS__)
+#define LOG_ALERT_SAMPLED_BY(actorCtxOrSystem, component, sampleBy, ...) LOG_LOG_SAMPLED_BY(actorCtxOrSystem, NActors::NLog::PRI_ALERT, component, sampleBy, __VA_ARGS__)
+#define LOG_CRIT_SAMPLED_BY(actorCtxOrSystem, component, sampleBy, ...) LOG_LOG_SAMPLED_BY(actorCtxOrSystem, NActors::NLog::PRI_CRIT, component, sampleBy, __VA_ARGS__)
+#define LOG_ERROR_SAMPLED_BY(actorCtxOrSystem, component, sampleBy, ...) LOG_LOG_SAMPLED_BY(actorCtxOrSystem, NActors::NLog::PRI_ERROR, component, sampleBy, __VA_ARGS__)
+#define LOG_WARN_SAMPLED_BY(actorCtxOrSystem, component, sampleBy, ...) LOG_LOG_SAMPLED_BY(actorCtxOrSystem, NActors::NLog::PRI_WARN, component, sampleBy, __VA_ARGS__)
+#define LOG_NOTICE_SAMPLED_BY(actorCtxOrSystem, component, sampleBy, ...) LOG_LOG_SAMPLED_BY(actorCtxOrSystem, NActors::NLog::PRI_NOTICE, component, sampleBy, __VA_ARGS__)
+#define LOG_INFO_SAMPLED_BY(actorCtxOrSystem, component, sampleBy, ...) LOG_LOG_SAMPLED_BY(actorCtxOrSystem, NActors::NLog::PRI_INFO, component, sampleBy, __VA_ARGS__)
+#define LOG_DEBUG_SAMPLED_BY(actorCtxOrSystem, component, sampleBy, ...) LOG_LOG_SAMPLED_BY(actorCtxOrSystem, NActors::NLog::PRI_DEBUG, component, sampleBy, __VA_ARGS__)
+#define LOG_TRACE_SAMPLED_BY(actorCtxOrSystem, component, sampleBy, ...) LOG_LOG_SAMPLED_BY(actorCtxOrSystem, NActors::NLog::PRI_TRACE, component, sampleBy, __VA_ARGS__)
+
+#define LOG_EMERG_S_SAMPLED_BY(actorCtxOrSystem, component, sampleBy, stream) LOG_LOG_S_SAMPLED_BY(actorCtxOrSystem, NActors::NLog::PRI_EMERG, component, sampleBy, stream)
+#define LOG_ALERT_S_SAMPLED_BY(actorCtxOrSystem, component, sampleBy, stream) LOG_LOG_S_SAMPLED_BY(actorCtxOrSystem, NActors::NLog::PRI_ALERT, component, sampleBy, stream)
+#define LOG_CRIT_S_SAMPLED_BY(actorCtxOrSystem, component, sampleBy, stream) LOG_LOG_S_SAMPLED_BY(actorCtxOrSystem, NActors::NLog::PRI_CRIT, component, sampleBy, stream)
+#define LOG_ERROR_S_SAMPLED_BY(actorCtxOrSystem, component, sampleBy, stream) LOG_LOG_S_SAMPLED_BY(actorCtxOrSystem, NActors::NLog::PRI_ERROR, component, sampleBy, stream)
+#define LOG_WARN_S_SAMPLED_BY(actorCtxOrSystem, component, sampleBy, stream) LOG_LOG_S_SAMPLED_BY(actorCtxOrSystem, NActors::NLog::PRI_WARN, component, sampleBy, stream)
+#define LOG_NOTICE_S_SAMPLED_BY(actorCtxOrSystem, component, sampleBy, stream) LOG_LOG_S_SAMPLED_BY(actorCtxOrSystem, NActors::NLog::PRI_NOTICE, component, sampleBy, stream)
+#define LOG_INFO_S_SAMPLED_BY(actorCtxOrSystem, component, sampleBy, stream) LOG_LOG_S_SAMPLED_BY(actorCtxOrSystem, NActors::NLog::PRI_INFO, component, sampleBy, stream)
+#define LOG_DEBUG_S_SAMPLED_BY(actorCtxOrSystem, component, sampleBy, stream) LOG_LOG_S_SAMPLED_BY(actorCtxOrSystem, NActors::NLog::PRI_DEBUG, component, sampleBy, stream)
+#define LOG_TRACE_S_SAMPLED_BY(actorCtxOrSystem, component, sampleBy, stream) LOG_LOG_S_SAMPLED_BY(actorCtxOrSystem, NActors::NLog::PRI_TRACE, component, sampleBy, stream)
+
+// Log Throttling
+#define LOG_LOG_THROTTLE(throttler, actorCtxOrSystem, priority, component, ...) \
+ do { \
+ if ((throttler).Kick()) { \
+ LOG_LOG(actorCtxOrSystem, priority, component, __VA_ARGS__); \
+ } \
+ } while (0) /**/
+
+#define TRACE_EVENT(component) \
+ const auto& currentTracer = component; \
+ if (ev->HasEvent()) { \
+ LOG_TRACE(*TlsActivationContext, currentTracer, "%s, received event# %" PRIu32 ", Sender %s, Recipient %s: %s", \
+ __FUNCTION__, ev->Type, ev->Sender.ToString().data(), SelfId().ToString().data(), ev->GetBase()->ToString().substr(0, 1000).data()); \
+ } else { \
+ LOG_TRACE(*TlsActivationContext, currentTracer, "%s, received event# %" PRIu32 ", Sender %s, Recipient %s", \
+ __FUNCTION__, ev->Type, ev->Sender.ToString().data(), ev->Recipient.ToString().data()); \
+ }
+#define TRACE_EVENT_TYPE(eventType) LOG_TRACE(*TlsActivationContext, currentTracer, "%s, processing event %s", __FUNCTION__, eventType)
+
+class TLog;
+class TLogBackend;
+
+namespace NActors {
+ class TLoggerActor;
+
+ ////////////////////////////////////////////////////////////////////////////////
+ // SET LOG LEVEL FOR A COMPONENT
+ ////////////////////////////////////////////////////////////////////////////////
+ class TLogComponentLevelRequest: public TEventLocal<TLogComponentLevelRequest, int(NLog::EEv::LevelReq)> {
+ public:
+ // set given priority for the component
+ TLogComponentLevelRequest(NLog::EPriority priority, NLog::EComponent component)
+ : Priority(priority)
+ , Component(component)
+ {
+ }
+
+ // set given priority for all components
+ TLogComponentLevelRequest(NLog::EPriority priority)
+ : Priority(priority)
+ , Component(NLog::InvalidComponent)
+ {
+ }
+
+ protected:
+ NLog::EPriority Priority;
+ NLog::EComponent Component;
+
+ friend class TLoggerActor;
+ };
+
+ class TLogComponentLevelResponse: public TEventLocal<TLogComponentLevelResponse, int(NLog::EEv::LevelResp)> {
+ public:
+ TLogComponentLevelResponse(int code, const TString& explanation)
+ : Code(code)
+ , Explanation(explanation)
+ {
+ }
+
+ int GetCode() const {
+ return Code;
+ }
+
+ const TString& GetExplanation() const {
+ return Explanation;
+ }
+
+ protected:
+ int Code;
+ TString Explanation;
+ };
+
+ class TLogIgnored: public TEventLocal<TLogIgnored, int(NLog::EEv::Ignored)> {
+ public:
+ TLogIgnored() {
+ }
+ };
+
+ ////////////////////////////////////////////////////////////////////////////////
+ // LOGGER ACTOR
+ ////////////////////////////////////////////////////////////////////////////////
+ class ILoggerMetrics {
+ public:
+ virtual ~ILoggerMetrics() = default;
+
+ virtual void IncActorMsgs() = 0;
+ virtual void IncDirectMsgs() = 0;
+ virtual void IncLevelRequests() = 0;
+ virtual void IncIgnoredMsgs() = 0;
+ virtual void IncAlertMsgs() = 0;
+ virtual void IncEmergMsgs() = 0;
+ virtual void IncDroppedMsgs() = 0;
+
+ virtual void GetOutputHtml(IOutputStream&) = 0;
+ };
+
+ class TLoggerActor: public TActor<TLoggerActor> {
+ public:
+ static constexpr IActor::EActivityType ActorActivityType() {
+ return IActor::LOG_ACTOR;
+ }
+
+ TLoggerActor(TIntrusivePtr<NLog::TSettings> settings,
+ TAutoPtr<TLogBackend> logBackend,
+ TIntrusivePtr<NMonitoring::TDynamicCounters> counters);
+ TLoggerActor(TIntrusivePtr<NLog::TSettings> settings,
+ std::shared_ptr<TLogBackend> logBackend,
+ TIntrusivePtr<NMonitoring::TDynamicCounters> counters);
+ TLoggerActor(TIntrusivePtr<NLog::TSettings> settings,
+ TAutoPtr<TLogBackend> logBackend,
+ std::shared_ptr<NMonitoring::TMetricRegistry> metrics);
+ TLoggerActor(TIntrusivePtr<NLog::TSettings> settings,
+ std::shared_ptr<TLogBackend> logBackend,
+ std::shared_ptr<NMonitoring::TMetricRegistry> metrics);
+ ~TLoggerActor();
+
+ void StateFunc(TAutoPtr<IEventHandle>& ev, const TActorContext& ctx) {
+ switch (ev->GetTypeRewrite()) {
+ HFunc(TLogIgnored, HandleIgnoredEvent);
+ HFunc(NLog::TEvLog, HandleLogEvent);
+ HFunc(TLogComponentLevelRequest, HandleLogComponentLevelRequest);
+ HFunc(NMon::TEvHttpInfo, HandleMonInfo);
+ }
+ }
+
+ STFUNC(StateDefunct) {
+ switch (ev->GetTypeRewrite()) {
+ cFunc(TLogIgnored::EventType, HandleIgnoredEventDrop);
+ hFunc(NLog::TEvLog, HandleLogEventDrop);
+ HFunc(TLogComponentLevelRequest, HandleLogComponentLevelRequest);
+ HFunc(NMon::TEvHttpInfo, HandleMonInfo);
+ cFunc(TEvents::TEvWakeup::EventType, HandleWakeup);
+ }
+ }
+
+ // Directly call logger instead of sending a message
+ void Log(TInstant time, NLog::EPriority priority, NLog::EComponent component, const char* c, ...);
+
+ static void Throttle(const NLog::TSettings& settings);
+
+ private:
+ TIntrusivePtr<NLog::TSettings> Settings;
+ std::shared_ptr<TLogBackend> LogBackend;
+ ui64 IgnoredCount = 0;
+ ui64 PassedCount = 0;
+ static TAtomic IsOverflow;
+ TDuration WakeupInterval{TDuration::Seconds(5)};
+ std::unique_ptr<ILoggerMetrics> Metrics;
+
+ void BecomeDefunct();
+ void HandleIgnoredEvent(TLogIgnored::TPtr& ev, const NActors::TActorContext& ctx);
+ void HandleIgnoredEventDrop();
+ void HandleLogEvent(NLog::TEvLog::TPtr& ev, const TActorContext& ctx);
+ void HandleLogEventDrop(const NLog::TEvLog::TPtr& ev);
+ void HandleLogComponentLevelRequest(TLogComponentLevelRequest::TPtr& ev, const TActorContext& ctx);
+ void HandleMonInfo(NMon::TEvHttpInfo::TPtr& ev, const TActorContext& ctx);
+ void HandleWakeup();
+ [[nodiscard]] bool OutputRecord(TInstant time, NLog::EPrio priority, NLog::EComponent component, const TString& formatted) noexcept;
+ void RenderComponentPriorities(IOutputStream& str);
+ void LogIgnoredCount(TInstant now);
+ void WriteMessageStat(const NLog::TEvLog& ev);
+ static const char* FormatLocalTimestamp(TInstant time, char* buf);
+ };
+
+ ////////////////////////////////////////////////////////////////////////////////
+ // LOG THROTTLING
+ // TTrivialLogThrottler -- log a message every 'period' duration
+ // Use case:
+ // TTrivialLogThrottler throttler(TDuration::Minutes(1));
+ // ....
+ // LOG_LOG_THROTTLE(throttler, ctx, NActors::NLog::PRI_ERROR, SOME, "Error");
+ ////////////////////////////////////////////////////////////////////////////////
+ class TTrivialLogThrottler {
+ public:
+ TTrivialLogThrottler(TDuration period)
+ : Period(period)
+ {
+ }
+
+ // return value:
+ // true -- write to log
+ // false -- don't write to log, throttle
+ bool Kick() {
+ auto now = TInstant::Now();
+ if (now >= (LastWrite + Period)) {
+ LastWrite = now;
+ return true;
+ } else {
+ return false;
+ }
+ }
+
+ private:
+ TInstant LastWrite;
+ TDuration Period;
+ };
+
+ ////////////////////////////////////////////////////////////////////////////////
+ // SYSLOG BACKEND
+ ////////////////////////////////////////////////////////////////////////////////
+ TAutoPtr<TLogBackend> CreateSysLogBackend(const TString& ident,
+ bool logPError, bool logCons);
+ TAutoPtr<TLogBackend> CreateStderrBackend();
+ TAutoPtr<TLogBackend> CreateFileBackend(const TString& fileName);
+ TAutoPtr<TLogBackend> CreateNullBackend();
+ TAutoPtr<TLogBackend> CreateCompositeLogBackend(TVector<TAutoPtr<TLogBackend>>&& underlyingBackends);
+
+ /////////////////////////////////////////////////////////////////////
+ // Logging adaptors for memory log and logging into filesystem
+ /////////////////////////////////////////////////////////////////////
+
+ namespace NDetail {
+ inline void Y_PRINTF_FORMAT(2, 3) PrintfV(TString& dst, const char* format, ...) {
+ va_list params;
+ va_start(params, format);
+ vsprintf(dst, format, params);
+ va_end(params);
+ }
+
+ inline void PrintfV(TString& dst, const char* format, va_list params) {
+ vsprintf(dst, format, params);
+ }
+ } // namespace NDetail
+
+ template <typename TCtx>
+ inline void DeliverLogMessage(TCtx& ctx, NLog::EPriority mPriority, NLog::EComponent mComponent, TString &&str)
+ {
+ const NLog::TSettings *mSettings = ctx.LoggerSettings();
+ TLoggerActor::Throttle(*mSettings);
+ ctx.Send(new IEventHandle(mSettings->LoggerActorId, TActorId(), new NLog::TEvLog(mPriority, mComponent, std::move(str))));
+ }
+
+ template <typename TCtx, typename... TArgs>
+ inline void MemLogAdapter(
+ TCtx& actorCtxOrSystem,
+ NLog::EPriority mPriority,
+ NLog::EComponent mComponent,
+ const char* format, TArgs&&... params) {
+ TString Formatted;
+
+
+ if constexpr (sizeof... (params) > 0) {
+ NDetail::PrintfV(Formatted, format, std::forward<TArgs>(params)...);
+ } else {
+ NDetail::PrintfV(Formatted, "%s", format);
+ }
+
+ MemLogWrite(Formatted.data(), Formatted.size(), true);
+ DeliverLogMessage(actorCtxOrSystem, mPriority, mComponent, std::move(Formatted));
+ }
+
+ template <typename TCtx>
+ Y_WRAPPER inline void MemLogAdapter(
+ TCtx& actorCtxOrSystem,
+ NLog::EPriority mPriority,
+ NLog::EComponent mComponent,
+ const TString& str) {
+
+ MemLogWrite(str.data(), str.size(), true);
+ DeliverLogMessage(actorCtxOrSystem, mPriority, mComponent, TString(str));
+ }
+
+ template <typename TCtx>
+ Y_WRAPPER inline void MemLogAdapter(
+ TCtx& actorCtxOrSystem,
+ NLog::EPriority mPriority,
+ NLog::EComponent mComponent,
+ TString&& str) {
+
+ MemLogWrite(str.data(), str.size(), true);
+ DeliverLogMessage(actorCtxOrSystem, mPriority, mComponent, std::move(str));
+ }
+}
diff --git a/library/cpp/actors/core/log_iface.h b/library/cpp/actors/core/log_iface.h
new file mode 100644
index 0000000000..b331db9ca8
--- /dev/null
+++ b/library/cpp/actors/core/log_iface.h
@@ -0,0 +1,109 @@
+#pragma once
+
+#include "events.h"
+#include "event_local.h"
+
+namespace NActors {
+ namespace NLog {
+ using EComponent = int;
+
+ enum EPriority : ui16 { // migrate it to EPrio whenever possible
+ PRI_EMERG,
+ PRI_ALERT,
+ PRI_CRIT,
+ PRI_ERROR,
+ PRI_WARN,
+ PRI_NOTICE,
+ PRI_INFO,
+ PRI_DEBUG,
+ PRI_TRACE
+ };
+
+ enum class EPrio : ui16 {
+ Emerg = 0,
+ Alert = 1,
+ Crit = 2,
+ Error = 3,
+ Warn = 4,
+ Notice = 5,
+ Info = 6,
+ Debug = 7,
+ Trace = 8,
+ };
+
+ struct TLevel {
+ TLevel(ui32 raw)
+ : Raw(raw)
+ {
+ }
+
+ TLevel(EPrio prio)
+ : Raw((ui16(prio) + 1) << 8)
+ {
+ }
+
+ EPrio ToPrio() const noexcept {
+ const auto major = Raw >> 8;
+
+ return major > 0 ? EPrio(major - 1) : EPrio::Emerg;
+ }
+
+ bool IsUrgentAbortion() const noexcept {
+ return (Raw >> 8) == 0;
+ }
+
+ /* Generalized monotonic level value composed with major and minor
+ levels. Minor is used for verbosity within major, basic EPrio
+ mapped to (EPrio + 1, 0) and Major = 0 is reserved as special
+ space with meaning like EPrio::Emerg but with extened actions.
+ Thus logger should map Major = 0 to EPrio::Emerg if it have no
+ idea how to handle special emergency actions.
+ */
+
+ ui32 Raw = 0; // ((ui16(EPrio) + 1) << 8) | ui8(minor)
+ };
+
+ enum class EEv {
+ Log = EventSpaceBegin(TEvents::ES_LOGGER),
+ LevelReq,
+ LevelResp,
+ Ignored,
+ End
+ };
+
+ static_assert(int(EEv::End) < EventSpaceEnd(TEvents::ES_LOGGER), "");
+
+ class TEvLog: public TEventLocal<TEvLog, int(EEv::Log)> {
+ public:
+ TEvLog(TInstant stamp, TLevel level, EComponent comp, const TString &line)
+ : Stamp(stamp)
+ , Level(level)
+ , Component(comp)
+ , Line(line)
+ {
+ }
+
+ TEvLog(TInstant stamp, TLevel level, EComponent comp, TString &&line)
+ : Stamp(stamp)
+ , Level(level)
+ , Component(comp)
+ , Line(std::move(line))
+ {
+ }
+
+ TEvLog(EPriority prio, EComponent comp, TString line, TInstant time = TInstant::Now())
+ : Stamp(time)
+ , Level(EPrio(prio))
+ , Component(comp)
+ , Line(std::move(line))
+ {
+ }
+
+ const TInstant Stamp = TInstant::Max();
+ const TLevel Level;
+ const EComponent Component = 0;
+ TString Line;
+ };
+
+ }
+}
diff --git a/library/cpp/actors/core/log_settings.cpp b/library/cpp/actors/core/log_settings.cpp
new file mode 100644
index 0000000000..f52f2fc5d2
--- /dev/null
+++ b/library/cpp/actors/core/log_settings.cpp
@@ -0,0 +1,230 @@
+#include "log_settings.h"
+
+#include <util/stream/str.h>
+
+namespace NActors {
+ namespace NLog {
+ TSettings::TSettings(const TActorId& loggerActorId, const EComponent loggerComponent,
+ EComponent minVal, EComponent maxVal, EComponentToStringFunc func,
+ EPriority defPriority, EPriority defSamplingPriority,
+ ui32 defSamplingRate, ui64 timeThresholdMs)
+ : LoggerActorId(loggerActorId)
+ , LoggerComponent(loggerComponent)
+ , TimeThresholdMs(timeThresholdMs)
+ , AllowDrop(true)
+ , ThrottleDelay(TDuration::MilliSeconds(100))
+ , MinVal(0)
+ , MaxVal(0)
+ , Mask(0)
+ , DefPriority(defPriority)
+ , DefSamplingPriority(defSamplingPriority)
+ , DefSamplingRate(defSamplingRate)
+ , UseLocalTimestamps(false)
+ , Format(PLAIN_FULL_FORMAT)
+ , ShortHostName("")
+ , ClusterName("")
+ {
+ Append(minVal, maxVal, func);
+ }
+
+ TSettings::TSettings(const TActorId& loggerActorId, const EComponent loggerComponent,
+ EPriority defPriority, EPriority defSamplingPriority,
+ ui32 defSamplingRate, ui64 timeThresholdMs)
+ : LoggerActorId(loggerActorId)
+ , LoggerComponent(loggerComponent)
+ , TimeThresholdMs(timeThresholdMs)
+ , AllowDrop(true)
+ , ThrottleDelay(TDuration::MilliSeconds(100))
+ , MinVal(0)
+ , MaxVal(0)
+ , Mask(0)
+ , DefPriority(defPriority)
+ , DefSamplingPriority(defSamplingPriority)
+ , DefSamplingRate(defSamplingRate)
+ , UseLocalTimestamps(false)
+ , Format(PLAIN_FULL_FORMAT)
+ , ShortHostName("")
+ , ClusterName("")
+ {
+ }
+
+ 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");
+
+ // update bounds
+ if (!MaxVal || minVal < MinVal) {
+ MinVal = minVal;
+ }
+
+ if (!MaxVal || maxVal > MaxVal) {
+ MaxVal = maxVal;
+
+ // expand ComponentNames to the new bounds
+ auto oldMask = Mask;
+ Mask = PowerOf2Mask(MaxVal);
+
+ TArrayHolder<TAtomic> oldComponentInfo(new TAtomic[Mask + 1]);
+ ComponentInfo.Swap(oldComponentInfo);
+ int startVal = oldMask ? oldMask + 1 : 0;
+ for (int i = 0; i < startVal; i++) {
+ AtomicSet(ComponentInfo[i], AtomicGet(oldComponentInfo[i]));
+ }
+
+ TComponentSettings defSetting(DefPriority, DefSamplingPriority, DefSamplingRate);
+ for (int i = startVal; i < Mask + 1; i++) {
+ AtomicSet(ComponentInfo[i], defSetting.Raw.Data);
+ }
+
+ ComponentNames.resize(Mask + 1);
+ }
+
+ // 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",
+ i, ComponentNames[i].data());
+ ComponentNames[i] = func(i);
+ }
+ }
+
+ int TSettings::SetLevelImpl(
+ const TString& name, bool isSampling,
+ EPriority priority, EComponent component, TString& explanation) {
+ TString titleName(name);
+ titleName.to_title();
+
+ // check priority
+ if (!IsValidPriority(priority)) {
+ TStringStream str;
+ str << "Invalid " << name;
+ explanation = str.Str();
+ return 1;
+ }
+
+ if (component == InvalidComponent) {
+ for (int i = 0; i < Mask + 1; i++) {
+ TComponentSettings settings = AtomicGet(ComponentInfo[i]);
+ if (isSampling) {
+ settings.Raw.X.SamplingLevel = priority;
+ } else {
+ settings.Raw.X.Level = priority;
+ }
+ AtomicSet(ComponentInfo[i], settings.Raw.Data);
+ }
+
+ TStringStream str;
+
+ str << titleName
+ << " for all components has been changed to "
+ << PriorityToString(EPrio(priority));
+ explanation = str.Str();
+ return 0;
+ } else {
+ if (!IsValidComponent(component)) {
+ explanation = "Invalid component";
+ return 1;
+ }
+ TComponentSettings settings = AtomicGet(ComponentInfo[component]);
+ EPriority oldPriority;
+ if (isSampling) {
+ oldPriority = (EPriority)settings.Raw.X.SamplingLevel;
+ settings.Raw.X.SamplingLevel = priority;
+ } else {
+ oldPriority = (EPriority)settings.Raw.X.Level;
+ settings.Raw.X.Level = priority;
+ }
+ AtomicSet(ComponentInfo[component], settings.Raw.Data);
+ TStringStream str;
+ str << titleName << " for the component " << ComponentNames[component]
+ << " has been changed from " << PriorityToString(EPrio(oldPriority))
+ << " to " << PriorityToString(EPrio(priority));
+ explanation = str.Str();
+ return 0;
+ }
+ }
+
+ int TSettings::SetLevel(EPriority priority, EComponent component, TString& explanation) {
+ return SetLevelImpl("priority", false,
+ priority, component, explanation);
+ }
+
+ int TSettings::SetSamplingLevel(EPriority priority, EComponent component, TString& explanation) {
+ return SetLevelImpl("sampling priority", true,
+ priority, component, explanation);
+ }
+
+ int TSettings::SetSamplingRate(ui32 sampling, EComponent component, TString& explanation) {
+ if (component == InvalidComponent) {
+ for (int i = 0; i < Mask + 1; i++) {
+ TComponentSettings settings = AtomicGet(ComponentInfo[i]);
+ settings.Raw.X.SamplingRate = sampling;
+ AtomicSet(ComponentInfo[i], settings.Raw.Data);
+ }
+ TStringStream str;
+ str << "Sampling rate for all components has been changed to " << sampling;
+ explanation = str.Str();
+ } else {
+ if (!IsValidComponent(component)) {
+ explanation = "Invalid component";
+ return 1;
+ }
+ TComponentSettings settings = AtomicGet(ComponentInfo[component]);
+ ui32 oldSampling = settings.Raw.X.SamplingRate;
+ settings.Raw.X.SamplingRate = sampling;
+ AtomicSet(ComponentInfo[component], settings.Raw.Data);
+ TStringStream str;
+ str << "Sampling rate for the component " << ComponentNames[component]
+ << " has been changed from " << oldSampling
+ << " to " << sampling;
+ explanation = str.Str();
+ }
+ return 0;
+ }
+
+ int TSettings::PowerOf2Mask(int val) {
+ int mask = 1;
+ while ((val & mask) != val) {
+ mask <<= 1;
+ mask |= 1;
+ }
+ return mask;
+ }
+
+ bool TSettings::IsValidPriority(EPriority priority) {
+ return priority == PRI_EMERG || priority == PRI_ALERT ||
+ priority == PRI_CRIT || priority == PRI_ERROR ||
+ priority == PRI_WARN || priority == PRI_NOTICE ||
+ priority == PRI_INFO || priority == PRI_DEBUG || priority == PRI_TRACE;
+ }
+
+ bool TSettings::IsValidComponent(EComponent component) {
+ return (MinVal <= component) && (component <= MaxVal) && !ComponentNames[component].empty();
+ }
+
+ void TSettings::SetAllowDrop(bool val) {
+ AllowDrop = val;
+ }
+
+ void TSettings::SetThrottleDelay(TDuration value) {
+ ThrottleDelay = value;
+ }
+
+ void TSettings::SetUseLocalTimestamps(bool value) {
+ UseLocalTimestamps = value;
+ }
+
+ EComponent TSettings::FindComponent(const TStringBuf& componentName) const {
+ if (componentName.empty())
+ return InvalidComponent;
+
+ for (EComponent component = MinVal; component <= MaxVal; ++component) {
+ if (ComponentNames[component] == componentName)
+ return component;
+ }
+
+ return InvalidComponent;
+ }
+
+ }
+
+}
diff --git a/library/cpp/actors/core/log_settings.h b/library/cpp/actors/core/log_settings.h
new file mode 100644
index 0000000000..7fe4504edd
--- /dev/null
+++ b/library/cpp/actors/core/log_settings.h
@@ -0,0 +1,176 @@
+#pragma once
+
+#include "actor.h"
+#include "log_iface.h"
+#include <util/generic/vector.h>
+#include <util/digest/murmur.h>
+#include <util/random/easy.h>
+
+namespace NActors {
+ namespace NLog {
+ inline const char* PriorityToString(EPrio priority) {
+ switch (priority) {
+ case EPrio::Emerg:
+ return "EMERG";
+ case EPrio::Alert:
+ return "ALERT";
+ case EPrio::Crit:
+ return "CRIT";
+ case EPrio::Error:
+ return "ERROR";
+ case EPrio::Warn:
+ return "WARN";
+ case EPrio::Notice:
+ return "NOTICE";
+ case EPrio::Info:
+ return "INFO";
+ case EPrio::Debug:
+ return "DEBUG";
+ case EPrio::Trace:
+ return "TRACE";
+ default:
+ return "UNKNOWN";
+ }
+ }
+
+ // You can structure your program to have multiple logical components.
+ // In this case you can set different log priorities for different
+ // components. And you can change component's priority while system
+ // is running. Suspect a component has a bug? Turn DEBUG priority level on
+ // for this component.
+ static const int InvalidComponent = -1;
+
+ // Functions converts EComponent id to string
+ using EComponentToStringFunc = std::function<const TString&(EComponent)>;
+ ;
+
+ // Log settings
+ struct TComponentSettings {
+ union {
+ struct {
+ ui32 SamplingRate;
+ ui8 SamplingLevel;
+ ui8 Level;
+ } X;
+
+ ui64 Data;
+ } Raw;
+
+ TComponentSettings(TAtomicBase data) {
+ Raw.Data = (ui64)data;
+ }
+
+ TComponentSettings(ui8 level, ui8 samplingLevel, ui32 samplingRate) {
+ Raw.X.Level = level;
+ Raw.X.SamplingLevel = samplingLevel;
+ Raw.X.SamplingRate = samplingRate;
+ }
+ };
+
+ struct TSettings: public TThrRefBase {
+ public:
+ TActorId LoggerActorId;
+ EComponent LoggerComponent;
+ ui64 TimeThresholdMs;
+ bool AllowDrop;
+ TDuration ThrottleDelay;
+ TArrayHolder<TAtomic> ComponentInfo;
+ TVector<TString> ComponentNames;
+ EComponent MinVal;
+ EComponent MaxVal;
+ EComponent Mask;
+ EPriority DefPriority;
+ EPriority DefSamplingPriority;
+ ui32 DefSamplingRate;
+ bool UseLocalTimestamps;
+
+ enum ELogFormat {
+ PLAIN_FULL_FORMAT,
+ PLAIN_SHORT_FORMAT,
+ JSON_FORMAT
+ };
+ ELogFormat Format;
+ TString ShortHostName;
+ TString ClusterName;
+ TString MessagePrefix;
+
+ // The best way to provide minVal, maxVal and func is to have
+ // protobuf enumeration of components. In this case protoc
+ // automatically generates YOURTYPE_MIN, YOURTYPE_MAX and
+ // YOURTYPE_Name for you.
+ TSettings(const TActorId& loggerActorId, const EComponent loggerComponent,
+ EComponent minVal, EComponent maxVal, EComponentToStringFunc func,
+ EPriority defPriority, EPriority defSamplingPriority = PRI_DEBUG,
+ ui32 defSamplingRate = 0, ui64 timeThresholdMs = 1000);
+
+ TSettings(const TActorId& loggerActorId, const EComponent loggerComponent,
+ EPriority defPriority, EPriority defSamplingPriority = PRI_DEBUG,
+ ui32 defSamplingRate = 0, ui64 timeThresholdMs = 1000);
+
+ void Append(EComponent minVal, EComponent maxVal, EComponentToStringFunc func);
+
+ template <typename T>
+ void Append(T minVal, T maxVal, const TString& (*func)(T)) {
+ Append(
+ static_cast<EComponent>(minVal),
+ static_cast<EComponent>(maxVal),
+ [func](EComponent c) -> const TString& {
+ return func(static_cast<T>(c));
+ }
+ );
+ }
+
+ inline bool Satisfies(EPriority priority, EComponent component, ui64 sampleBy = 0) const {
+ // by using Mask we don't get outside of array boundaries
+ TComponentSettings settings = GetComponentSettings(component);
+ if (priority > settings.Raw.X.Level) {
+ if (priority > settings.Raw.X.SamplingLevel) {
+ return false; // priority > both levels ==> do not log
+ }
+ // priority <= sampling level ==> apply sampling
+ ui32 samplingRate = settings.Raw.X.SamplingRate;
+ if (samplingRate) {
+ ui32 samplingValue = sampleBy ? MurmurHash<ui32>((const char*)&sampleBy, sizeof(sampleBy))
+ : samplingRate != 1 ? RandomNumber<ui32>() : 0;
+ return (samplingValue % samplingRate == 0);
+ } else {
+ // sampling rate not set ==> do not log
+ return false;
+ }
+ } else {
+ // priority <= log level ==> log
+ return true;
+ }
+ }
+
+ inline TComponentSettings GetComponentSettings(EComponent component) const {
+ Y_VERIFY_DEBUG((component & Mask) == component);
+ // by using Mask we don't get outside of array boundaries
+ return TComponentSettings(AtomicGet(ComponentInfo[component & Mask]));
+ }
+
+ const char* ComponentName(EComponent component) const {
+ Y_VERIFY_DEBUG((component & Mask) == component);
+ return ComponentNames[component & Mask].data();
+ }
+
+ int SetLevel(EPriority priority, EComponent component, TString& explanation);
+ int SetSamplingLevel(EPriority priority, EComponent component, TString& explanation);
+ int SetSamplingRate(ui32 sampling, EComponent component, TString& explanation);
+ EComponent FindComponent(const TStringBuf& componentName) const;
+ static int PowerOf2Mask(int val);
+ static bool IsValidPriority(EPriority priority);
+ bool IsValidComponent(EComponent component);
+ void SetAllowDrop(bool val);
+ void SetThrottleDelay(TDuration value);
+ void SetUseLocalTimestamps(bool value);
+
+ private:
+ int SetLevelImpl(
+ const TString& name, bool isSampling,
+ EPriority priority, EComponent component, TString& explanation);
+ };
+
+ }
+
+}
diff --git a/library/cpp/actors/core/log_ut.cpp b/library/cpp/actors/core/log_ut.cpp
new file mode 100644
index 0000000000..09b5f88ea2
--- /dev/null
+++ b/library/cpp/actors/core/log_ut.cpp
@@ -0,0 +1,185 @@
+#include "log.h"
+
+#include <library/cpp/testing/unittest/registar.h>
+
+#include <library/cpp/actors/testlib/test_runtime.h>
+
+using namespace NMonitoring;
+using namespace NActors;
+using namespace NActors::NLog;
+
+namespace {
+ const TString& ServiceToString(int) {
+ static const TString FAKE{"FAKE"};
+ return FAKE;
+ }
+
+ TIntrusivePtr<TSettings> DefaultSettings() {
+ auto loggerId = TActorId{0, "Logger"};
+ auto s = MakeIntrusive<TSettings>(loggerId, 0, EPriority::PRI_TRACE);
+ s->SetAllowDrop(false);
+ s->Append(0, 1, ServiceToString);
+ return s;
+ }
+
+ TIntrusivePtr<TSettings> DroppingSettings(ui64 timeThresholdMs) {
+ auto loggerId = TActorId{0, "Logger"};
+ auto s = MakeIntrusive<TSettings>(
+ loggerId,
+ 0,
+ EPriority::PRI_TRACE,
+ EPriority::PRI_DEBUG,
+ (ui32)0,
+ timeThresholdMs);
+ s->Append(0, 1, ServiceToString);
+ return s;
+ }
+
+ class TMockBackend: public TLogBackend {
+ public:
+ using TWriteImpl = std::function<void(const TLogRecord&)>;
+ using TReopenImpl = std::function<void()>;
+
+ static void REOPEN_NOP() { }
+
+ TMockBackend(TWriteImpl writeImpl, TReopenImpl reopenImpl = REOPEN_NOP)
+ : WriteImpl_{writeImpl}
+ , ReopenImpl_{reopenImpl}
+ {
+ }
+
+ void WriteData(const TLogRecord& r) override {
+ WriteImpl_(r);
+ }
+
+ void ReopenLog() override { }
+
+ void SetWriteImpl(TWriteImpl writeImpl) {
+ WriteImpl_ = writeImpl;
+ }
+
+ private:
+ TWriteImpl WriteImpl_;
+ TReopenImpl ReopenImpl_;
+ };
+
+ void ThrowAlways(const TLogRecord&) {
+ ythrow yexception();
+ };
+
+ struct TFixture {
+ TFixture(
+ TIntrusivePtr<TSettings> settings,
+ TMockBackend::TWriteImpl writeImpl = ThrowAlways)
+ {
+ Runtime.Initialize();
+ LogBackend.reset(new TMockBackend{writeImpl});
+ LoggerActor = Runtime.Register(new TLoggerActor{settings, LogBackend, Counters});
+ Runtime.SetScheduledEventFilter([] (auto&&, auto&&, auto&&, auto) {
+ return false;
+ });
+ }
+
+ TFixture(TMockBackend::TWriteImpl writeImpl = ThrowAlways)
+ : TFixture(DefaultSettings(), writeImpl)
+ {}
+
+ void WriteLog() {
+ Runtime.Send(new IEventHandle{LoggerActor, {}, new TEvLog(TInstant::Zero(), TLevel{EPrio::Emerg}, 0, "foo")});
+ }
+
+ void WriteLog(TInstant ts) {
+ Runtime.Send(new IEventHandle{LoggerActor, {}, new TEvLog(ts, TLevel{EPrio::Emerg}, 0, "foo")});
+ }
+
+ void Wakeup() {
+ Runtime.Send(new IEventHandle{LoggerActor, {}, new TEvents::TEvWakeup});
+ }
+
+ TIntrusivePtr<TDynamicCounters> Counters{MakeIntrusive<TDynamicCounters>()};
+ std::shared_ptr<TMockBackend> LogBackend;
+ TActorId LoggerActor;
+ TTestActorRuntimeBase Runtime;
+ };
+}
+
+
+Y_UNIT_TEST_SUITE(TLoggerActorTest) {
+ Y_UNIT_TEST(NoCrashOnWriteFailure) {
+ TFixture test;
+ test.WriteLog();
+ // everything is okay as long as we get here
+ }
+
+ Y_UNIT_TEST(SubsequentWritesAreIgnored) {
+ size_t count{0};
+ auto countWrites = [&count] (auto&& r) {
+ count++;
+ ThrowAlways(r);
+ };
+
+ TFixture test{countWrites};
+ test.WriteLog();
+ UNIT_ASSERT_VALUES_EQUAL(count, 1);
+
+ // at this point we should have started dropping messages
+ for (auto i = 0; i < 5; ++i) {
+ test.WriteLog();
+ }
+
+ UNIT_ASSERT_VALUES_EQUAL(count, 1);
+ }
+
+ Y_UNIT_TEST(LoggerCanRecover) {
+ TFixture test;
+ test.WriteLog();
+
+ TVector<TString> messages;
+ auto acceptWrites = [&] (const TLogRecord& r) {
+ messages.emplace_back(r.Data, r.Len);
+ };
+
+ auto scheduled = test.Runtime.CaptureScheduledEvents();
+ UNIT_ASSERT_VALUES_EQUAL(scheduled.size(), 1);
+
+ test.LogBackend->SetWriteImpl(acceptWrites);
+ test.Wakeup();
+
+ const auto COUNT = 10;
+ for (auto i = 0; i < COUNT; ++i) {
+ test.WriteLog();
+ }
+
+ UNIT_ASSERT_VALUES_EQUAL(messages.size(), COUNT);
+ }
+
+ Y_UNIT_TEST(ShouldObeyTimeThresholdMsWhenOverloaded) {
+ TFixture test{DroppingSettings(5000)};
+
+ TVector<TString> messages;
+ auto acceptWrites = [&] (const TLogRecord& r) {
+ messages.emplace_back(r.Data, r.Len);
+ };
+
+ test.LogBackend->SetWriteImpl(acceptWrites);
+ test.Wakeup();
+
+ const auto COUNT = 11;
+ for (auto i = 0; i < COUNT; ++i) {
+ test.WriteLog();
+ }
+
+ UNIT_ASSERT_VALUES_EQUAL(messages.size(), COUNT);
+
+ test.Runtime.AdvanceCurrentTime(TDuration::Seconds(20));
+ auto now = test.Runtime.GetCurrentTime();
+
+ test.WriteLog(now - TDuration::Seconds(5));
+
+ UNIT_ASSERT_VALUES_EQUAL(messages.size(), COUNT + 1);
+
+ test.WriteLog(now - TDuration::Seconds(6));
+
+ UNIT_ASSERT_VALUES_EQUAL(messages.size(), COUNT + 1);
+ }
+}
diff --git a/library/cpp/actors/core/mailbox.cpp b/library/cpp/actors/core/mailbox.cpp
new file mode 100644
index 0000000000..d84b4f9e46
--- /dev/null
+++ b/library/cpp/actors/core/mailbox.cpp
@@ -0,0 +1,551 @@
+#include "mailbox.h"
+#include "actorsystem.h"
+
+#include <library/cpp/actors/util/datetime.h>
+
+#include <util/system/sanitizers.h>
+
+namespace NActors {
+ TMailboxTable::TMailboxTable()
+ : LastAllocatedLine(0)
+ , AllocatedMailboxCount(0)
+ , CachedSimpleMailboxes(0)
+ , CachedRevolvingMailboxes(0)
+ , CachedHTSwapMailboxes(0)
+ , CachedReadAsFilledMailboxes(0)
+ , CachedTinyReadAsFilledMailboxes(0)
+ {
+ memset((void*)Lines, 0, sizeof(Lines));
+ }
+
+ bool IsGoodForCleanup(const TMailboxHeader* header) {
+ switch (AtomicLoad(&header->ExecutionState)) {
+ case TMailboxHeader::TExecutionState::Inactive:
+ case TMailboxHeader::TExecutionState::Scheduled:
+ return true;
+ case TMailboxHeader::TExecutionState::Leaving:
+ case TMailboxHeader::TExecutionState::Executing:
+ case TMailboxHeader::TExecutionState::LeavingMarked:
+ return false;
+ case TMailboxHeader::TExecutionState::Free:
+ case TMailboxHeader::TExecutionState::FreeScheduled:
+ return true;
+ case TMailboxHeader::TExecutionState::FreeLeaving:
+ case TMailboxHeader::TExecutionState::FreeExecuting:
+ case TMailboxHeader::TExecutionState::FreeLeavingMarked:
+ return false;
+ default:
+ Y_FAIL();
+ }
+ }
+
+ template <typename TMailbox>
+ void DestructMailboxLine(ui8* begin, ui8* end) {
+ const ui32 sx = TMailbox::AlignedSize();
+ for (ui8* x = begin; x + sx <= end; x += sx) {
+ TMailbox* mailbox = reinterpret_cast<TMailbox*>(x);
+ Y_VERIFY(IsGoodForCleanup(mailbox));
+ mailbox->ExecutionState = Max<ui32>();
+ mailbox->~TMailbox();
+ }
+ }
+
+ template <typename TMailbox>
+ bool CleanupMailboxLine(ui8* begin, ui8* end) {
+ const ui32 sx = TMailbox::AlignedSize();
+ bool done = true;
+ for (ui8* x = begin; x + sx <= end; x += sx) {
+ TMailbox* mailbox = reinterpret_cast<TMailbox*>(x);
+ Y_VERIFY(IsGoodForCleanup(mailbox));
+ done &= mailbox->CleanupActors() && mailbox->CleanupEvents();
+ }
+ return done;
+ }
+
+ TMailboxTable::~TMailboxTable() {
+ // on cleanup we must traverse everything and free stuff
+ for (ui32 i = 0; i < LastAllocatedLine; ++i) {
+ if (TMailboxLineHeader* lineHeader = Lines[i]) {
+ switch (lineHeader->MailboxType) {
+ case TMailboxType::Simple:
+ DestructMailboxLine<TSimpleMailbox>((ui8*)lineHeader + 64, (ui8*)lineHeader + LineSize);
+ break;
+ case TMailboxType::Revolving:
+ DestructMailboxLine<TRevolvingMailbox>((ui8*)lineHeader + 64, (ui8*)lineHeader + LineSize);
+ break;
+ case TMailboxType::HTSwap:
+ DestructMailboxLine<THTSwapMailbox>((ui8*)lineHeader + 64, (ui8*)lineHeader + LineSize);
+ break;
+ case TMailboxType::ReadAsFilled:
+ DestructMailboxLine<TReadAsFilledMailbox>((ui8*)lineHeader + 64, (ui8*)lineHeader + LineSize);
+ break;
+ case TMailboxType::TinyReadAsFilled:
+ DestructMailboxLine<TTinyReadAsFilledMailbox>((ui8*)lineHeader + 64, (ui8*)lineHeader + LineSize);
+ break;
+ default:
+ Y_FAIL();
+ }
+
+ lineHeader->~TMailboxLineHeader();
+ free(lineHeader);
+ Lines[i] = nullptr;
+ }
+ }
+
+ while (MailboxCacheSimple.Pop(0))
+ ;
+ while (MailboxCacheRevolving.Pop(0))
+ ;
+ while (MailboxCacheHTSwap.Pop(0))
+ ;
+ while (MailboxCacheReadAsFilled.Pop(0))
+ ;
+ while (MailboxCacheTinyReadAsFilled.Pop(0))
+ ;
+ }
+
+ bool TMailboxTable::Cleanup() {
+ bool done = true;
+ for (ui32 i = 0; i < LastAllocatedLine; ++i) {
+ if (TMailboxLineHeader* lineHeader = Lines[i]) {
+ switch (lineHeader->MailboxType) {
+ case TMailboxType::Simple:
+ done &= CleanupMailboxLine<TSimpleMailbox>((ui8*)lineHeader + 64, (ui8*)lineHeader + LineSize);
+ break;
+ case TMailboxType::Revolving:
+ done &= CleanupMailboxLine<TRevolvingMailbox>((ui8*)lineHeader + 64, (ui8*)lineHeader + LineSize);
+ break;
+ case TMailboxType::HTSwap:
+ done &= CleanupMailboxLine<THTSwapMailbox>((ui8*)lineHeader + 64, (ui8*)lineHeader + LineSize);
+ break;
+ case TMailboxType::ReadAsFilled:
+ done &= CleanupMailboxLine<TReadAsFilledMailbox>((ui8*)lineHeader + 64, (ui8*)lineHeader + LineSize);
+ break;
+ case TMailboxType::TinyReadAsFilled:
+ done &= CleanupMailboxLine<TTinyReadAsFilledMailbox>((ui8*)lineHeader + 64, (ui8*)lineHeader + LineSize);
+ break;
+ default:
+ Y_FAIL();
+ }
+ }
+ }
+ return done;
+ }
+
+ TMailboxHeader* TMailboxTable::Get(ui32 hint) {
+ // get line
+ const ui32 lineIndex = (hint & LineIndexMask) >> LineIndexShift;
+ const ui32 lineHint = hint & LineHintMask;
+
+ Y_VERIFY((lineIndex < MaxLines) && (lineHint < LineSize / 64));
+ if (lineHint == 0)
+ return nullptr;
+
+ if (TMailboxLineHeader* const x = AtomicLoad(Lines + lineIndex)) {
+ switch (x->MailboxType) {
+ case TMailboxType::Simple:
+ return TSimpleMailbox::Get(lineHint, x);
+ case TMailboxType::Revolving:
+ return TRevolvingMailbox::Get(lineHint, x);
+ case TMailboxType::HTSwap:
+ return THTSwapMailbox::Get(lineHint, x);
+ case TMailboxType::ReadAsFilled:
+ return TReadAsFilledMailbox::Get(lineHint, x);
+ case TMailboxType::TinyReadAsFilled:
+ return TTinyReadAsFilledMailbox::Get(lineHint, x);
+ default:
+ Y_VERIFY_DEBUG(false);
+ break;
+ }
+ }
+
+ return nullptr;
+ }
+
+ bool TMailboxTable::SendTo(TAutoPtr<IEventHandle>& ev, IExecutorPool* executorPool) {
+ const TActorId& recipient = ev->GetRecipientRewrite();
+ const ui32 hint = recipient.Hint();
+
+ // copy-paste from Get to avoid duplicated type-switches
+ const ui32 lineIndex = (hint & LineIndexMask) >> LineIndexShift;
+ const ui32 lineHint = hint & LineHintMask;
+
+ Y_VERIFY((lineIndex < MaxLines) && (lineHint < LineSize / 64));
+ if (lineHint == 0)
+ return false;
+
+ if (TMailboxLineHeader* const x = AtomicLoad(Lines + lineIndex)) {
+ switch (x->MailboxType) {
+ case TMailboxType::Simple: {
+ TSimpleMailbox* const mailbox = TSimpleMailbox::Get(lineHint, x);
+#if (!defined(_tsan_enabled_))
+ Y_VERIFY_DEBUG(mailbox->Type == (ui32)x->MailboxType);
+#endif
+ mailbox->Queue.Push(ev.Release());
+ if (mailbox->MarkForSchedule()) {
+ RelaxedStore<NHPTimer::STime>(&mailbox->ScheduleMoment, GetCycleCountFast());
+ executorPool->ScheduleActivation(hint);
+ }
+ }
+ return true;
+ case TMailboxType::Revolving: {
+ // The actorid could be stale and coming from a different machine. If local process has restarted than
+ // the stale actorid coming from a remote machine might be referencing an actor with simple mailbox
+ // which is smaller than revolving mailbox. In this cases 'lineHint' index might be greater than actual
+ // array size. Normally its ok to store stale event to other actor's valid mailbox beacuse Receive will
+ // compare receiver actor id and discard stale event. But in this case we should discard the event right away
+ // instead of trying to enque it to a mailbox at invalid address.
+ // NOTE: lineHint is 1-based
+ static_assert(TSimpleMailbox::AlignedSize() <= TRevolvingMailbox::AlignedSize(),
+ "We expect that one line can store more simple mailboxes than revolving mailboxes");
+ if (lineHint > TRevolvingMailbox::MaxMailboxesInLine())
+ return false;
+
+ TRevolvingMailbox* const mailbox = TRevolvingMailbox::Get(lineHint, x);
+#if (!defined(_tsan_enabled_))
+ Y_VERIFY_DEBUG(mailbox->Type == (ui32)x->MailboxType);
+#endif
+ mailbox->QueueWriter.Push(ev.Release());
+ if (mailbox->MarkForSchedule()) {
+ RelaxedStore<NHPTimer::STime>(&mailbox->ScheduleMoment, GetCycleCountFast());
+ executorPool->ScheduleActivation(hint);
+ }
+ }
+ return true;
+ case TMailboxType::HTSwap: {
+ THTSwapMailbox* const mailbox = THTSwapMailbox::Get(lineHint, x);
+#if (!defined(_tsan_enabled_))
+ Y_VERIFY_DEBUG(mailbox->Type == (ui32)x->MailboxType);
+#endif
+ mailbox->Queue.Push(ev.Release());
+ if (mailbox->MarkForSchedule()) {
+ RelaxedStore<NHPTimer::STime>(&mailbox->ScheduleMoment, GetCycleCountFast());
+ executorPool->ScheduleActivation(hint);
+ }
+ }
+ return true;
+ case TMailboxType::ReadAsFilled: {
+ if (lineHint > TReadAsFilledMailbox::MaxMailboxesInLine())
+ return false;
+
+ TReadAsFilledMailbox* const mailbox = TReadAsFilledMailbox::Get(lineHint, x);
+#if (!defined(_tsan_enabled_))
+ Y_VERIFY_DEBUG(mailbox->Type == (ui32)x->MailboxType);
+#endif
+ mailbox->Queue.Push(ev.Release());
+ if (mailbox->MarkForSchedule()) {
+ RelaxedStore<NHPTimer::STime>(&mailbox->ScheduleMoment, GetCycleCountFast());
+ executorPool->ScheduleActivation(hint);
+ }
+ }
+ return true;
+ case TMailboxType::TinyReadAsFilled: {
+ if (lineHint > TTinyReadAsFilledMailbox::MaxMailboxesInLine())
+ return false;
+
+ TTinyReadAsFilledMailbox* const mailbox = TTinyReadAsFilledMailbox::Get(lineHint, x);
+#if (!defined(_tsan_enabled_))
+ Y_VERIFY_DEBUG(mailbox->Type == (ui32)x->MailboxType);
+#endif
+ mailbox->Queue.Push(ev.Release());
+ if (mailbox->MarkForSchedule()) {
+ RelaxedStore<NHPTimer::STime>(&mailbox->ScheduleMoment, GetCycleCountFast());
+ executorPool->ScheduleActivation(hint);
+ }
+ }
+ return true;
+ default:
+ Y_FAIL("unknown mailbox type");
+ }
+ }
+
+ return false;
+ }
+
+ ui32 TMailboxTable::AllocateMailbox(TMailboxType::EType type, ui64 revolvingCounter) {
+ ui32 x = TryAllocateMailbox(type, revolvingCounter);
+ if (x == 0)
+ x = AllocateNewLine(type);
+ return x;
+ }
+
+ ui32 TMailboxTable::TryAllocateMailbox(TMailboxType::EType type, ui64 revolvingCounter) {
+ switch (type) {
+ case TMailboxType::Simple:
+ do {
+ if (ui32 ret = MailboxCacheSimple.Pop(revolvingCounter)) {
+ AtomicDecrement(CachedSimpleMailboxes);
+ return ret;
+ }
+ } while (AtomicGet(CachedSimpleMailboxes) > (MailboxCacheSimple.Concurrency * 512));
+ return 0;
+ case TMailboxType::Revolving:
+ do {
+ if (ui32 ret = MailboxCacheRevolving.Pop(revolvingCounter)) {
+ AtomicDecrement(CachedRevolvingMailboxes);
+ return ret;
+ }
+ } while (AtomicGet(CachedRevolvingMailboxes) > (MailboxCacheRevolving.Concurrency * 512));
+ return 0;
+ case TMailboxType::HTSwap:
+ do {
+ if (ui32 ret = MailboxCacheHTSwap.Pop(revolvingCounter)) {
+ AtomicDecrement(CachedHTSwapMailboxes);
+ return ret;
+ }
+ } while (AtomicGet(CachedHTSwapMailboxes) > (MailboxCacheHTSwap.Concurrency * 512));
+ return 0;
+ case TMailboxType::ReadAsFilled:
+ do {
+ if (ui32 ret = MailboxCacheReadAsFilled.Pop(revolvingCounter)) {
+ AtomicDecrement(CachedReadAsFilledMailboxes);
+ return ret;
+ }
+ } while (AtomicGet(CachedReadAsFilledMailboxes) > (MailboxCacheReadAsFilled.Concurrency * 512));
+ return 0;
+ case TMailboxType::TinyReadAsFilled:
+ do {
+ if (ui32 ret = MailboxCacheTinyReadAsFilled.Pop(revolvingCounter)) {
+ AtomicDecrement(CachedTinyReadAsFilledMailboxes);
+ return ret;
+ }
+ } while (AtomicGet(CachedTinyReadAsFilledMailboxes) > (MailboxCacheTinyReadAsFilled.Concurrency * 512));
+ return 0;
+ default:
+ Y_FAIL("Unknown mailbox type");
+ }
+ }
+
+ void TMailboxTable::ReclaimMailbox(TMailboxType::EType type, ui32 hint, ui64 revolvingCounter) {
+ if (hint != 0) {
+ switch (type) {
+ case TMailboxType::Simple:
+ MailboxCacheSimple.Push(hint, revolvingCounter);
+ AtomicIncrement(CachedSimpleMailboxes);
+ break;
+ case TMailboxType::Revolving:
+ MailboxCacheRevolving.Push(hint, revolvingCounter);
+ AtomicIncrement(CachedRevolvingMailboxes);
+ break;
+ case TMailboxType::HTSwap:
+ MailboxCacheHTSwap.Push(hint, revolvingCounter);
+ AtomicIncrement(CachedHTSwapMailboxes);
+ break;
+ case TMailboxType::ReadAsFilled:
+ MailboxCacheReadAsFilled.Push(hint, revolvingCounter);
+ AtomicIncrement(CachedReadAsFilledMailboxes);
+ break;
+ case TMailboxType::TinyReadAsFilled:
+ MailboxCacheTinyReadAsFilled.Push(hint, revolvingCounter);
+ AtomicIncrement(CachedTinyReadAsFilledMailboxes);
+ break;
+ default:
+ Y_FAIL();
+ }
+ }
+ }
+
+ TMailboxHeader::TMailboxHeader(TMailboxType::EType type)
+ : ExecutionState(TExecutionState::Free)
+ , Reserved(0)
+ , Type(type)
+ , ActorPack(TMailboxActorPack::Simple)
+ , Knobs(0)
+ {
+ ActorsInfo.Simple.ActorId = 0;
+ ActorsInfo.Simple.Actor = nullptr;
+ }
+
+ TMailboxHeader::~TMailboxHeader() {
+ CleanupActors();
+ }
+
+ bool TMailboxHeader::CleanupActors() {
+ bool done = true;
+ switch (ActorPack) {
+ case TMailboxActorPack::Simple: {
+ if (ActorsInfo.Simple.ActorId != 0) {
+ delete ActorsInfo.Simple.Actor;
+ done = false;
+ }
+ break;
+ }
+ case TMailboxActorPack::Map: {
+ for (auto& [actorId, actor] : *ActorsInfo.Map.ActorsMap) {
+ delete actor;
+ }
+ delete ActorsInfo.Map.ActorsMap;
+ done = false;
+ break;
+ }
+ case TMailboxActorPack::Array: {
+ for (ui64 i = 0; i < ActorsInfo.Array.ActorsCount; ++i) {
+ delete ActorsInfo.Array.ActorsArray->Actors[i].Actor;
+ }
+ delete ActorsInfo.Array.ActorsArray;
+ done = false;
+ break;
+ }
+ }
+ ActorPack = TMailboxActorPack::Simple;
+ ActorsInfo.Simple.ActorId = 0;
+ ActorsInfo.Simple.Actor = nullptr;
+ return done;
+ }
+
+ std::pair<ui32, ui32> TMailboxHeader::CountMailboxEvents(ui64 localActorId, ui32 maxTraverse) {
+ switch (Type) {
+ case TMailboxType::Simple:
+ return static_cast<TMailboxTable::TSimpleMailbox*>(this)->CountSimpleMailboxEvents(localActorId, maxTraverse);
+ case TMailboxType::Revolving:
+ return static_cast<TMailboxTable::TRevolvingMailbox*>(this)->CountRevolvingMailboxEvents(localActorId, maxTraverse);
+ default:
+ return {0, 0};
+ }
+ }
+
+ TMailboxTable::TSimpleMailbox::TSimpleMailbox()
+ : TMailboxHeader(TMailboxType::Simple)
+ , ScheduleMoment(0)
+ {
+ }
+
+ TMailboxTable::TSimpleMailbox::~TSimpleMailbox() {
+ CleanupEvents();
+ }
+
+ bool TMailboxTable::TSimpleMailbox::CleanupEvents() {
+ const bool done = (Queue.Head() == nullptr);
+ while (IEventHandle* ev = Queue.Pop())
+ delete ev;
+ return done;
+ }
+
+ std::pair<ui32, ui32> TMailboxTable::TSimpleMailbox::CountSimpleMailboxEvents(ui64 localActorId, ui32 maxTraverse) {
+ ui32 local = 0;
+ ui32 total = 0;
+
+ auto it = Queue.ReadIterator();
+ while (IEventHandle* x = it.Next()) {
+ ++total;
+ if (x->GetRecipientRewrite().LocalId() == localActorId)
+ ++local;
+ if (total >= maxTraverse)
+ break;
+ }
+
+ return std::make_pair(local, total);
+ }
+
+ TMailboxTable::TRevolvingMailbox::TRevolvingMailbox()
+ : TMailboxHeader(TMailboxType::Revolving)
+ , QueueWriter(QueueReader)
+ , Reserved1(0)
+ , Reserved2(0)
+ , ScheduleMoment(0)
+ {
+ }
+
+ TMailboxTable::TRevolvingMailbox::~TRevolvingMailbox() {
+ CleanupEvents();
+ }
+
+ bool TMailboxTable::TRevolvingMailbox::CleanupEvents() {
+ const bool done = (QueueReader.Head() == nullptr);
+ while (IEventHandle* ev = QueueReader.Pop())
+ delete ev;
+ return done;
+ }
+
+ std::pair<ui32, ui32> TMailboxTable::TRevolvingMailbox::CountRevolvingMailboxEvents(ui64 localActorId, ui32 maxTraverse) {
+ ui32 local = 0;
+ ui32 total = 0;
+
+ auto it = QueueReader.Iterator();
+
+ while (IEventHandle* x = it.Next()) {
+ ++total;
+ if (x->GetRecipientRewrite().LocalId() == localActorId)
+ ++local;
+ if (total >= maxTraverse)
+ break;
+ }
+
+ return std::make_pair(local, total);
+ }
+
+ template <typename T>
+ static ui32 InitNewLine(ui8* x, ui8* end) {
+ const ui32 sx = T::AlignedSize();
+
+ for (ui32 index = 1; x + sx <= end; x += sx, ++index)
+ ::new (x) T();
+
+ return sx;
+ }
+
+ ui32 TMailboxTable::AllocateNewLine(TMailboxType::EType type) {
+ ui8* ptr = (ui8*)malloc(LineSize);
+ ui8* end = ptr + LineSize;
+
+ const ui32 lineIndex = (ui32)AtomicIncrement(LastAllocatedLine) - 1;
+ const ui32 lineIndexMask = (lineIndex << LineIndexShift) & LineIndexMask;
+
+ // first 64 bytes is TMailboxLineHeader
+ TMailboxLineHeader* header = ::new (ptr) TMailboxLineHeader(type, lineIndex);
+
+ ui8* x = ptr + 64;
+ ui32 sx = 0;
+ TMailboxCache* cache = nullptr;
+ TAtomic* counter = nullptr;
+
+ switch (type) {
+ case TMailboxType::Simple:
+ sx = InitNewLine<TSimpleMailbox>(x, end);
+ cache = &MailboxCacheSimple;
+ counter = &CachedSimpleMailboxes;
+ break;
+ case TMailboxType::Revolving:
+ sx = InitNewLine<TRevolvingMailbox>(x, end);
+ cache = &MailboxCacheRevolving;
+ counter = &CachedRevolvingMailboxes;
+ break;
+ case TMailboxType::HTSwap:
+ sx = InitNewLine<THTSwapMailbox>(x, end);
+ cache = &MailboxCacheHTSwap;
+ counter = &CachedHTSwapMailboxes;
+ break;
+ case TMailboxType::ReadAsFilled:
+ sx = InitNewLine<TReadAsFilledMailbox>(x, end);
+ cache = &MailboxCacheReadAsFilled;
+ counter = &CachedReadAsFilledMailboxes;
+ break;
+ case TMailboxType::TinyReadAsFilled:
+ sx = InitNewLine<TTinyReadAsFilledMailbox>(x, end);
+ cache = &MailboxCacheTinyReadAsFilled;
+ counter = &CachedTinyReadAsFilledMailboxes;
+ break;
+ default:
+ Y_FAIL();
+ }
+
+ AtomicStore(Lines + lineIndex, header);
+
+ ui32 ret = lineIndexMask | 1;
+
+ ui32 index = 2;
+ for (ui32 endIndex = LineSize / sx; index != endIndex;) {
+ const ui32 bufSize = 8;
+ ui32 buf[bufSize];
+ ui32 bufIndex;
+ for (bufIndex = 0; index != endIndex && bufIndex != bufSize; ++bufIndex, ++index)
+ buf[bufIndex] = lineIndexMask | index;
+ cache->PushBulk(buf, bufIndex, index);
+ AtomicAdd(*counter, bufIndex);
+ }
+
+ AtomicAdd(AllocatedMailboxCount, index - 1);
+
+ return ret;
+ }
+}
diff --git a/library/cpp/actors/core/mailbox.h b/library/cpp/actors/core/mailbox.h
new file mode 100644
index 0000000000..0bd9c4d314
--- /dev/null
+++ b/library/cpp/actors/core/mailbox.h
@@ -0,0 +1,553 @@
+#pragma once
+
+#include "defs.h"
+#include "event.h"
+#include "actor.h"
+#include "mailbox_queue_simple.h"
+#include "mailbox_queue_revolving.h"
+#include <library/cpp/actors/util/unordered_cache.h>
+#include <library/cpp/threading/queue/mpsc_htswap.h>
+#include <library/cpp/threading/queue/mpsc_read_as_filled.h>
+#include <util/generic/hash.h>
+#include <util/system/hp_timer.h>
+#include <util/generic/ptr.h>
+// TODO: clean all broken arcadia atomic stuff and replace with intrinsics
+
+namespace NActors {
+ class IActor;
+ class IExecutorPool;
+
+ const ui64 ARRAY_CAPACITY = 8;
+
+ // structure of hint:
+ // 1 bit: is service or direct hint
+ // 2 bits: pool index
+ // 17 bits: line
+ // 12 bits: index of mailbox inside of line
+
+ struct TMailboxHeader {
+ struct TMailboxActorPack {
+ enum EType {
+ Simple = 0,
+ Array = 1,
+ Map = 2
+ };
+ };
+
+ using TActorMap = THashMap<ui64, IActor*>;
+
+ struct TExecutionState {
+ enum EState {
+ // normal states
+ Inactive = 0,
+ Scheduled = 1,
+ Leaving = 2,
+ Executing = 3,
+ LeavingMarked = 4,
+ // states for free mailboxes (they can still be scheduled so we need duplicates)
+ Free = 5,
+ FreeScheduled = 6,
+ FreeLeaving = 7,
+ FreeExecuting = 8,
+ FreeLeavingMarked = 9,
+ };
+ };
+
+ volatile ui32 ExecutionState;
+ ui32 Reserved : 4; // never changes, always zero
+ ui32 Type : 4; // never changes
+ ui32 ActorPack : 2;
+ ui32 Knobs : 22;
+
+ struct TActorPair {
+ IActor *Actor;
+ ui64 ActorId;
+ };
+
+ struct alignas(64) TActorArray {
+ TActorPair Actors[ARRAY_CAPACITY];
+ };
+
+ union TActorsInfo {
+ TActorPair Simple;
+ struct {
+ TActorArray* ActorsArray;
+ ui64 ActorsCount;
+ } Array;
+ struct {
+ TActorMap* ActorsMap;
+ } Map;
+ } ActorsInfo;
+
+ TMailboxHeader(TMailboxType::EType type);
+ ~TMailboxHeader();
+
+ bool CleanupActors();
+
+ // this interface is used exclusively by executor thread, so implementation is there
+
+ bool MarkForSchedule(); // we put something in queue, check should we schedule?
+
+ bool LockForExecution(); // we got activation, try to lock mailbox
+ bool LockFromFree(); // try to claim mailbox from recycled (could fail if other thread process garbage)
+
+ void UnlockFromExecution1(); // prepare for releasing lock
+ bool UnlockFromExecution2(bool wouldReschedule); // proceed with releasing lock
+ bool UnlockAsFree(bool wouldReschedule); // preceed with releasing lock, but mark as free one
+
+ bool IsEmpty() const noexcept {
+ return (ActorPack == TMailboxActorPack::Simple && ActorsInfo.Simple.ActorId == 0);
+ }
+
+ template<typename T>
+ void ForEach(T&& callback) noexcept {
+ switch (ActorPack) {
+ case TMailboxActorPack::Simple:
+ if (ActorsInfo.Simple.ActorId) {
+ callback(ActorsInfo.Simple.ActorId, ActorsInfo.Simple.Actor);
+ }
+ break;
+
+ case TMailboxActorPack::Map:
+ for (const auto& [actorId, actor] : *ActorsInfo.Map.ActorsMap) {
+ callback(actorId, actor);
+ }
+ break;
+
+ case TMailboxActorPack::Array:
+ for (ui64 i = 0; i < ActorsInfo.Array.ActorsCount; ++i) {
+ auto& row = ActorsInfo.Array.ActorsArray->Actors[i];
+ callback(row.ActorId, row.Actor);
+ }
+ break;
+ }
+ }
+
+ IActor* FindActor(ui64 localActorId) noexcept {
+ switch (ActorPack) {
+ case TMailboxActorPack::Simple: {
+ if (ActorsInfo.Simple.ActorId == localActorId)
+ return ActorsInfo.Simple.Actor;
+ break;
+ }
+ case TMailboxActorPack::Map: {
+ TActorMap::iterator it = ActorsInfo.Map.ActorsMap->find(localActorId);
+ if (it != ActorsInfo.Map.ActorsMap->end())
+ return it->second;
+ break;
+ }
+ case TMailboxActorPack::Array: {
+ for (ui64 i = 0; i < ActorsInfo.Array.ActorsCount; ++i) {
+ if (ActorsInfo.Array.ActorsArray->Actors[i].ActorId == localActorId) {
+ return ActorsInfo.Array.ActorsArray->Actors[i].Actor;
+ }
+ }
+ break;
+ }
+ default:
+ Y_FAIL();
+ }
+ return nullptr;
+ }
+
+ void AttachActor(ui64 localActorId, IActor* actor) noexcept {
+ switch (ActorPack) {
+ case TMailboxActorPack::Simple: {
+ if (ActorsInfo.Simple.ActorId == 0) {
+ ActorsInfo.Simple.ActorId = localActorId;
+ ActorsInfo.Simple.Actor = actor;
+ return;
+ } else {
+ auto ar = new TActorArray;
+ ar->Actors[0] = ActorsInfo.Simple;
+ ar->Actors[1] = TActorPair{actor, localActorId};
+ ActorsInfo.Array.ActorsCount = 2;
+ ActorPack = TMailboxActorPack::Array;
+ ActorsInfo.Array.ActorsArray = ar;
+ }
+ break;
+ }
+ case TMailboxActorPack::Map: {
+ ActorsInfo.Map.ActorsMap->insert(TActorMap::value_type(localActorId, actor));
+ break;
+ }
+ case TMailboxActorPack::Array: {
+ if (ActorsInfo.Array.ActorsCount == ARRAY_CAPACITY) {
+ TActorMap* mp = new TActorMap();
+ for (ui64 i = 0; i < ARRAY_CAPACITY; ++i) {
+ mp->emplace(ActorsInfo.Array.ActorsArray->Actors[i].ActorId, ActorsInfo.Array.ActorsArray->Actors[i].Actor);
+ }
+ mp->emplace(localActorId, actor);
+ ActorPack = TMailboxActorPack::Map;
+ ActorsInfo.Array.ActorsCount = 0;
+ delete ActorsInfo.Array.ActorsArray;
+ ActorsInfo.Map.ActorsMap = mp;
+ } else {
+ ActorsInfo.Array.ActorsArray->Actors[ActorsInfo.Array.ActorsCount++] = TActorPair{actor, localActorId};
+ }
+ break;
+ }
+ default:
+ Y_FAIL();
+ }
+ }
+
+ IActor* DetachActor(ui64 localActorId) noexcept {
+ Y_VERIFY_DEBUG(FindActor(localActorId) != nullptr);
+
+ IActor* actorToDestruct = nullptr;
+
+ switch (ActorPack) {
+ case TMailboxActorPack::Simple: {
+ Y_VERIFY(ActorsInfo.Simple.ActorId == localActorId);
+ actorToDestruct = ActorsInfo.Simple.Actor;
+
+ ActorsInfo.Simple.ActorId = 0;
+ ActorsInfo.Simple.Actor = nullptr;
+ break;
+ }
+ case TMailboxActorPack::Map: {
+ TActorMap::iterator it = ActorsInfo.Map.ActorsMap->find(localActorId);
+ Y_VERIFY(it != ActorsInfo.Map.ActorsMap->end());
+
+ actorToDestruct = it->second;
+ ActorsInfo.Map.ActorsMap->erase(it);
+
+ if (ActorsInfo.Map.ActorsMap->size() == ARRAY_CAPACITY) {
+ auto ar = new TActorArray;
+ ui64 i = 0;
+ for (auto& [actorId, actor] : *ActorsInfo.Map.ActorsMap) {
+ ar->Actors[i++] = TActorPair{actor, actorId};
+ }
+ delete ActorsInfo.Map.ActorsMap;
+ ActorPack = TMailboxActorPack::Array;
+ ActorsInfo.Array.ActorsArray = ar;
+ ActorsInfo.Array.ActorsCount = ARRAY_CAPACITY;
+ }
+ break;
+ }
+ case TMailboxActorPack::Array: {
+ bool found = false;
+ for (ui64 i = 0; i < ActorsInfo.Array.ActorsCount; ++i) {
+ if (ActorsInfo.Array.ActorsArray->Actors[i].ActorId == localActorId) {
+ found = true;
+ actorToDestruct = ActorsInfo.Array.ActorsArray->Actors[i].Actor;
+ ActorsInfo.Array.ActorsArray->Actors[i] = ActorsInfo.Array.ActorsArray->Actors[ActorsInfo.Array.ActorsCount - 1];
+ ActorsInfo.Array.ActorsCount -= 1;
+ break;
+ }
+ }
+ Y_VERIFY(found);
+
+ if (ActorsInfo.Array.ActorsCount == 1) {
+ const TActorPair Actor = ActorsInfo.Array.ActorsArray->Actors[0];
+ delete ActorsInfo.Array.ActorsArray;
+ ActorPack = TMailboxActorPack::Simple;
+ ActorsInfo.Simple = Actor;
+ }
+ break;
+ }
+ }
+
+ return actorToDestruct;
+ }
+
+ std::pair<ui32, ui32> CountMailboxEvents(ui64 localActorId, ui32 maxTraverse);
+ };
+
+ class TMailboxTable : TNonCopyable {
+ private:
+ struct TMailboxLineHeader {
+ const TMailboxType::EType MailboxType;
+ const ui32 Index;
+ // some more stuff in first cache line, then goes mailboxes
+ ui8 Padding[52];
+
+ TMailboxLineHeader(TMailboxType::EType type, ui32 index)
+ : MailboxType(type)
+ , Index(index)
+ {
+ }
+ };
+ static_assert(sizeof(TMailboxLineHeader) <= 64, "expect sizeof(TMailboxLineHeader) <= 64");
+
+ constexpr static ui64 MaxLines = 131000; // somewhat less then 2^17.
+ constexpr static ui64 LineSize = 262144; // 64 * 2^12.
+
+ TAtomic LastAllocatedLine;
+ TAtomic AllocatedMailboxCount;
+
+ typedef TUnorderedCache<ui32, 512, 4> TMailboxCache;
+ TMailboxCache MailboxCacheSimple;
+ TAtomic CachedSimpleMailboxes;
+ TMailboxCache MailboxCacheRevolving;
+ TAtomic CachedRevolvingMailboxes;
+ TMailboxCache MailboxCacheHTSwap;
+ TAtomic CachedHTSwapMailboxes;
+ TMailboxCache MailboxCacheReadAsFilled;
+ TAtomic CachedReadAsFilledMailboxes;
+ TMailboxCache MailboxCacheTinyReadAsFilled;
+ TAtomic CachedTinyReadAsFilledMailboxes;
+
+ // and here goes large chunk of lines
+ // presented as array of static size to avoid sync on access
+ TMailboxLineHeader* volatile Lines[MaxLines];
+
+ ui32 AllocateNewLine(TMailboxType::EType type);
+ ui32 TryAllocateMailbox(TMailboxType::EType type, ui64 revolvingCounter);
+
+ public:
+ TMailboxTable();
+ ~TMailboxTable();
+
+ bool Cleanup(); // returns true if nothing found to destruct (so nothing new is possible to be created)
+
+ static const ui32 LineIndexShift = 12;
+ static const ui32 LineIndexMask = 0x1FFFFu << LineIndexShift;
+ static const ui32 LineHintMask = 0xFFFu;
+ static const ui32 PoolIndexShift = TActorId::PoolIndexShift;
+ static const ui32 PoolIndexMask = TActorId::PoolIndexMask;
+
+ static ui32 LineIndex(ui32 hint) {
+ return ((hint & LineIndexMask) >> LineIndexShift);
+ }
+ static ui32 PoolIndex(ui32 hint) {
+ return TActorId::PoolIndex(hint);
+ }
+
+ TMailboxHeader* Get(ui32 hint);
+ ui32 AllocateMailbox(TMailboxType::EType type, ui64 revolvingCounter);
+ void ReclaimMailbox(TMailboxType::EType type, ui32 hint, ui64 revolvingCounter);
+ ui64 GetAllocatedMailboxCount() const {
+ return RelaxedLoad(&AllocatedMailboxCount);
+ }
+
+ bool SendTo(TAutoPtr<IEventHandle>& ev, IExecutorPool* executorPool);
+
+ struct TSimpleMailbox: public TMailboxHeader {
+ // 4 bytes - state
+ // 4 bytes - knobs
+ // 8 bytes - actorid
+ // 8 bytes - actor*
+ TSimpleMailboxQueue<IEventHandle*, 64> Queue; // 24 + 8 bytes (body, lock)
+ NHPTimer::STime ScheduleMoment;
+
+ TSimpleMailbox();
+ ~TSimpleMailbox();
+
+ IEventHandle* Pop() {
+ return Queue.Pop();
+ }
+ IEventHandle* Head() {
+ return Queue.Head();
+ }
+
+ static TSimpleMailbox* Get(ui32 hint, void* line) {
+ return (TSimpleMailbox*)((ui8*)line + hint * 64); //
+ }
+ static const TMailboxType::EType MailboxType = TMailboxType::Simple;
+ constexpr static ui32 AlignedSize() {
+ return ((sizeof(TSimpleMailbox) + 63) / 64) * 64;
+ }
+
+ std::pair<ui32, ui32> CountSimpleMailboxEvents(ui64 localActorId, ui32 maxTraverse);
+ bool CleanupEvents();
+ };
+
+ static_assert(sizeof(TSimpleMailbox) == 64, "expect sizeof(TSimpleMailbox) == 64");
+
+ struct TRevolvingMailbox: public TMailboxHeader {
+ // 4 bytes - state
+ // 4 bytes - knobs
+ // 8 bytes - actorid
+ // 8 bytes - actor*
+ TRevolvingMailboxQueue<IEventHandle*, 3, 128>::TReader QueueReader; // 8 * 3 + 4 * 3 + (padding): 40 bytes
+ // here goes next cache-line, so less writers<-> reader interference
+ TRevolvingMailboxQueue<IEventHandle*, 3, 128>::TWriter QueueWriter; // 8 * 3 + 4 * 3 + 8 : 48 bytes
+ ui32 Reserved1;
+ ui32 Reserved2;
+ NHPTimer::STime ScheduleMoment;
+
+ TRevolvingMailbox();
+ ~TRevolvingMailbox();
+
+ IEventHandle* Pop() {
+ return QueueReader.Pop();
+ }
+ IEventHandle* Head() {
+ return QueueReader.Head();
+ }
+
+ static TRevolvingMailbox* Get(ui32 hint, void* line) {
+ return (TRevolvingMailbox*)((ui8*)line + 64 + (hint - 1) * 128);
+ }
+
+ constexpr static ui64 MaxMailboxesInLine() {
+ return (LineSize - 64) / AlignedSize();
+ }
+ static const TMailboxType::EType MailboxType = TMailboxType::Revolving;
+ constexpr static ui32 AlignedSize() {
+ return ((sizeof(TRevolvingMailbox) + 63) / 64) * 64;
+ }
+
+ std::pair<ui32, ui32> CountRevolvingMailboxEvents(ui64 localActorId, ui32 maxTraverse);
+ bool CleanupEvents();
+ };
+
+ static_assert(sizeof(TRevolvingMailbox) == 128, "expect sizeof(TRevolvingMailbox) == 128");
+
+ struct THTSwapMailbox: public TMailboxHeader {
+ using TQueueType = NThreading::THTSwapQueue<IEventHandle*>;
+
+ TQueueType Queue;
+ NHPTimer::STime ScheduleMoment;
+ char Padding_[16];
+
+ THTSwapMailbox()
+ : TMailboxHeader(TMailboxType::HTSwap)
+ , ScheduleMoment(0)
+ {
+ }
+
+ ~THTSwapMailbox() {
+ CleanupEvents();
+ }
+
+ IEventHandle* Pop() {
+ return Queue.Pop();
+ }
+
+ IEventHandle* Head() {
+ return Queue.Peek();
+ }
+
+ static THTSwapMailbox* Get(ui32 hint, void* line) {
+ return (THTSwapMailbox*)((ui8*)line + 64 + (hint - 1) * 64);
+ }
+
+ constexpr static ui64 MaxMailboxesInLine() {
+ return (LineSize - 64) / AlignedSize();
+ }
+
+ static const TMailboxType::EType MailboxType = TMailboxType::HTSwap;
+
+ constexpr static ui32 AlignedSize() {
+ return ((sizeof(THTSwapMailbox) + 63) / 64) * 64;
+ }
+
+ bool CleanupEvents() {
+ const bool done = (Queue.Peek() == nullptr);
+ while (IEventHandle* ev = Queue.Pop())
+ delete ev;
+ return done;
+ }
+ };
+
+ static_assert(sizeof(THTSwapMailbox) == 64,
+ "expect sizeof(THTSwapMailbox) == 64");
+
+ struct TReadAsFilledMailbox: public TMailboxHeader {
+ using TQueueType = NThreading::TReadAsFilledQueue<IEventHandle>;
+
+ TQueueType Queue;
+ NHPTimer::STime ScheduleMoment;
+ char Padding_[8];
+
+ TReadAsFilledMailbox()
+ : TMailboxHeader(TMailboxType::ReadAsFilled)
+ , ScheduleMoment(0)
+ {
+ }
+
+ ~TReadAsFilledMailbox() {
+ CleanupEvents();
+ }
+
+ IEventHandle* Pop() {
+ return Queue.Pop();
+ }
+
+ IEventHandle* Head() {
+ return Queue.Peek();
+ }
+
+ static TReadAsFilledMailbox* Get(ui32 hint, void* line) {
+ return (TReadAsFilledMailbox*)((ui8*)line + 64 + (hint - 1) * 192);
+ }
+
+ constexpr static ui64 MaxMailboxesInLine() {
+ return (LineSize - 64) / AlignedSize();
+ }
+
+ static const TMailboxType::EType MailboxType =
+ TMailboxType::ReadAsFilled;
+
+ constexpr static ui32 AlignedSize() {
+ return ((sizeof(TReadAsFilledMailbox) + 63) / 64) * 64;
+ }
+
+ bool CleanupEvents() {
+ const bool done = (Queue.Peek() == nullptr);
+ while (IEventHandle* ev = Queue.Pop())
+ delete ev;
+ return done;
+ }
+ };
+
+ static_assert(sizeof(TReadAsFilledMailbox) == 192,
+ "expect sizeof(TReadAsFilledMailbox) == 192");
+
+ struct TTinyReadAsFilledMailbox: public TMailboxHeader {
+ using TQueueType = NThreading::TReadAsFilledQueue<
+ IEventHandle,
+ NThreading::TRaFQueueBunchSize<4>>;
+
+ TQueueType Queue;
+ NHPTimer::STime ScheduleMoment;
+ char Padding_[8];
+
+ TTinyReadAsFilledMailbox()
+ : TMailboxHeader(TMailboxType::TinyReadAsFilled)
+ , ScheduleMoment(0)
+ {
+ }
+
+ ~TTinyReadAsFilledMailbox() {
+ CleanupEvents();
+ }
+
+ IEventHandle* Pop() {
+ return Queue.Pop();
+ }
+
+ IEventHandle* Head() {
+ return Queue.Peek();
+ }
+
+ static TTinyReadAsFilledMailbox* Get(ui32 hint, void* line) {
+ return (TTinyReadAsFilledMailbox*)((ui8*)line + 64 + (hint - 1) * 192);
+ }
+
+ constexpr static ui64 MaxMailboxesInLine() {
+ return (LineSize - 64) / AlignedSize();
+ }
+
+ static const TMailboxType::EType MailboxType =
+ TMailboxType::TinyReadAsFilled;
+
+ constexpr static ui32 AlignedSize() {
+ return ((sizeof(TTinyReadAsFilledMailbox) + 63) / 64) * 64;
+ }
+
+ bool CleanupEvents() {
+ const bool done = (Queue.Peek() == nullptr);
+ while (IEventHandle* ev = Queue.Pop())
+ delete ev;
+ return done;
+ }
+ };
+
+ static_assert(sizeof(TTinyReadAsFilledMailbox) == 192,
+ "expect sizeof(TTinyReadAsFilledMailbox) == 192");
+ };
+}
diff --git a/library/cpp/actors/core/mailbox_queue_revolving.h b/library/cpp/actors/core/mailbox_queue_revolving.h
new file mode 100644
index 0000000000..b0e78a18db
--- /dev/null
+++ b/library/cpp/actors/core/mailbox_queue_revolving.h
@@ -0,0 +1,214 @@
+#pragma once
+
+#include "defs.h"
+#include <library/cpp/actors/util/queue_chunk.h>
+
+namespace NActors {
+ // add some concurrency to basic queue to avoid hangs under contention (we pay with memory, so use only when really expect contention)
+ // ordering: every completed push guarantied to seen before any not-yet-initiated push. parallel pushes could reorder (and that is natural for concurrent queues).
+ // try to place reader/writer on different cache-lines to avoid congestion b/w reader and writers.
+ // if strict ordering does not matter - look at TManyOneQueue.
+
+ template <typename T, ui32 TWriteConcurrency = 3, ui32 TSize = 128>
+ class TRevolvingMailboxQueue {
+ static_assert(std::is_integral<T>::value || std::is_pointer<T>::value, "expect std::is_integral<T>::value || std::is_pointer<T>::value");
+
+ struct TValTagPair {
+ volatile T Value;
+ volatile ui64 Tag;
+ };
+
+ typedef TQueueChunk<TValTagPair, TSize> TChunk;
+
+ static_assert(sizeof(TAtomic) == sizeof(TChunk*), "expect sizeof(TAtomic) == sizeof(TChunk*)");
+ static_assert(sizeof(TAtomic) == sizeof(ui64), "expect sizeof(TAtomic) == sizeof(ui64)");
+
+ public:
+ class TWriter;
+
+ class TReader {
+ TChunk* ReadFrom[TWriteConcurrency];
+ ui32 ReadPosition[TWriteConcurrency];
+
+ friend class TRevolvingMailboxQueue<T, TWriteConcurrency, TSize>::TWriter; // for access to ReadFrom in constructor
+
+ bool ChunkHead(ui32 idx, ui64* tag, T* value) {
+ TChunk* head = ReadFrom[idx];
+ const ui32 pos = ReadPosition[idx];
+ if (pos != TChunk::EntriesCount) {
+ if (const T xval = AtomicLoad(&head->Entries[pos].Value)) {
+ const ui64 xtag = head->Entries[pos].Tag;
+ if (xtag < *tag) {
+ *value = xval;
+ *tag = xtag;
+ return true;
+ }
+ }
+ } else if (TChunk* next = AtomicLoad(&head->Next)) {
+ ReadFrom[idx] = next;
+ delete head;
+ ReadPosition[idx] = 0;
+ return ChunkHead(idx, tag, value);
+ }
+
+ return false;
+ }
+
+ T Head(bool pop) {
+ ui64 tag = Max<ui64>();
+ T ret = T{};
+ ui32 idx = 0;
+
+ for (ui32 i = 0; i < TWriteConcurrency; ++i)
+ if (ChunkHead(i, &tag, &ret))
+ idx = i;
+
+ // w/o second pass we could reorder updates with 'already scanned' range
+ if (ret) {
+ for (ui32 i = 0; i < TWriteConcurrency; ++i)
+ if (ChunkHead(i, &tag, &ret))
+ idx = i;
+ }
+
+ if (pop && ret)
+ ++ReadPosition[idx];
+
+ return ret;
+ }
+
+ public:
+ TReader() {
+ for (ui32 i = 0; i != TWriteConcurrency; ++i) {
+ ReadFrom[i] = new TChunk();
+ ReadPosition[i] = 0;
+ }
+ }
+
+ ~TReader() {
+ Y_VERIFY_DEBUG(Head() == 0);
+ for (ui32 i = 0; i < TWriteConcurrency; ++i)
+ delete ReadFrom[i];
+ }
+
+ T Pop() {
+ return Head(true);
+ }
+
+ T Head() {
+ return Head(false);
+ }
+
+ class TReadIterator {
+ TChunk* ReadFrom[TWriteConcurrency];
+ ui32 ReadPosition[TWriteConcurrency];
+
+ bool ChunkHead(ui32 idx, ui64* tag, T* value) {
+ TChunk* head = ReadFrom[idx];
+ const ui32 pos = ReadPosition[idx];
+ if (pos != TChunk::EntriesCount) {
+ if (const T xval = AtomicLoad(&head->Entries[pos].Value)) {
+ const ui64 xtag = head->Entries[pos].Tag;
+ if (xtag < *tag) {
+ *value = xval;
+ *tag = xtag;
+ return true;
+ }
+ }
+ } else if (TChunk* next = AtomicLoad(&head->Next)) {
+ ReadFrom[idx] = next;
+ ReadPosition[idx] = 0;
+ return ChunkHead(idx, tag, value);
+ }
+
+ return false;
+ }
+
+ public:
+ TReadIterator(TChunk* const* readFrom, const ui32* readPosition) {
+ memcpy(ReadFrom, readFrom, TWriteConcurrency * sizeof(TChunk*));
+ memcpy(ReadPosition, readPosition, TWriteConcurrency * sizeof(ui32));
+ }
+
+ T Next() {
+ ui64 tag = Max<ui64>();
+ T ret = T{};
+ ui32 idx = 0;
+
+ for (ui32 i = 0; i < TWriteConcurrency; ++i)
+ if (ChunkHead(i, &tag, &ret))
+ idx = i;
+
+ // w/o second pass we could reorder updates with 'already scanned' range
+ if (ret) {
+ for (ui32 i = 0; i < TWriteConcurrency; ++i)
+ if (ChunkHead(i, &tag, &ret))
+ idx = i;
+ }
+
+ if (ret)
+ ++ReadPosition[idx];
+
+ return ret;
+ }
+ };
+
+ TReadIterator Iterator() const {
+ return TReadIterator(ReadFrom, ReadPosition);
+ }
+ };
+
+ class TWriter {
+ TChunk* volatile WriteTo[TWriteConcurrency];
+ volatile ui64 Tag;
+ ui32 WritePosition[TWriteConcurrency];
+
+ public:
+ TWriter(const TReader& reader)
+ : Tag(0)
+ {
+ for (ui32 i = 0; i != TWriteConcurrency; ++i) {
+ WriteTo[i] = reader.ReadFrom[i];
+ WritePosition[i] = 0;
+ }
+ }
+
+ bool TryPush(T x) {
+ Y_VERIFY(x != 0);
+
+ for (ui32 i = 0; i != TWriteConcurrency; ++i) {
+ if (RelaxedLoad(&WriteTo[i]) != nullptr) {
+ if (TChunk* writeTo = AtomicSwap(&WriteTo[i], nullptr)) {
+ const ui64 nextTag = AtomicIncrement(Tag);
+ Y_VERIFY_DEBUG(nextTag < Max<ui64>());
+ const ui32 writePosition = WritePosition[i];
+ if (writePosition != TChunk::EntriesCount) {
+ writeTo->Entries[writePosition].Tag = nextTag;
+ AtomicStore(&writeTo->Entries[writePosition].Value, x);
+ ++WritePosition[i];
+ } else {
+ TChunk* next = new TChunk();
+ next->Entries[0].Tag = nextTag;
+ next->Entries[0].Value = x;
+ AtomicStore(&writeTo->Next, next);
+ writeTo = next;
+ WritePosition[i] = 1;
+ }
+ AtomicStore(WriteTo + i, writeTo);
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+ ui32 Push(T x) {
+ ui32 spins = 0;
+ while (!TryPush(x)) {
+ ++spins;
+ SpinLockPause();
+ }
+ return spins;
+ }
+ };
+ };
+}
diff --git a/library/cpp/actors/core/mailbox_queue_simple.h b/library/cpp/actors/core/mailbox_queue_simple.h
new file mode 100644
index 0000000000..2e44c21adb
--- /dev/null
+++ b/library/cpp/actors/core/mailbox_queue_simple.h
@@ -0,0 +1,34 @@
+#pragma once
+
+#include "defs.h"
+#include <library/cpp/actors/util/ticket_lock.h>
+#include <library/cpp/actors/util/queue_oneone_inplace.h>
+
+namespace NActors {
+ // dead-simple one-one queue, based on serializability guaranties of x64 and ticket lock to ensure writer unicity.
+ template <typename T, ui32 TSize>
+ class TSimpleMailboxQueue {
+ TOneOneQueueInplace<T, TSize> Queue;
+ TTicketLock Lock;
+
+ public:
+ ui32 Push(T x) noexcept {
+ const ui32 spins = Lock.Acquire();
+ Queue.Push(x);
+ Lock.Release();
+ return spins;
+ }
+
+ T Head() {
+ return Queue.Head();
+ }
+
+ T Pop() {
+ return Queue.Pop();
+ }
+
+ typename TOneOneQueueInplace<T, TSize>::TReadIterator ReadIterator() {
+ return Queue.Iterator();
+ }
+ };
+}
diff --git a/library/cpp/actors/core/memory_track.cpp b/library/cpp/actors/core/memory_track.cpp
new file mode 100644
index 0000000000..5f422116be
--- /dev/null
+++ b/library/cpp/actors/core/memory_track.cpp
@@ -0,0 +1,38 @@
+#include "memory_track.h"
+#include "memory_tracker.h"
+
+namespace NActors {
+namespace NMemory {
+
+namespace NPrivate {
+
+TThreadLocalInfo::TThreadLocalInfo()
+ : Metrics(TMemoryTracker::Instance()->GetCount())
+{
+ TMemoryTracker::Instance()->OnCreateThread(this);
+}
+
+TThreadLocalInfo::~TThreadLocalInfo() {
+ TMemoryTracker::Instance()->OnDestroyThread(this);
+}
+
+TMetric* TThreadLocalInfo::GetMetric(size_t index) {
+ if (Y_UNLIKELY(index >= Metrics.size())) {
+ return &Null;
+ }
+ return &Metrics[index];
+}
+
+const std::vector<TMetric>& TThreadLocalInfo::GetMetrics() const {
+ return Metrics;
+}
+
+size_t TBaseLabel::RegisterStaticMemoryLabel(const char* name, bool hasSensor) {
+ return TMemoryTracker::Instance()->RegisterStaticMemoryLabel(name, hasSensor);
+}
+
+}
+
+}
+}
+
diff --git a/library/cpp/actors/core/memory_track.h b/library/cpp/actors/core/memory_track.h
new file mode 100644
index 0000000000..6035333eeb
--- /dev/null
+++ b/library/cpp/actors/core/memory_track.h
@@ -0,0 +1,293 @@
+#pragma once
+
+#include <vector>
+
+#include <util/system/type_name.h>
+#include <util/thread/singleton.h>
+
+#define ENABLE_MEMORY_TRACKING
+
+namespace NActors {
+namespace NMemory {
+
+namespace NPrivate {
+
+class TMetric {
+ std::atomic<ssize_t> Memory;
+ std::atomic<ssize_t> Count;
+
+ void Copy(const TMetric& other) {
+ Memory.store(other.GetMemory(), std::memory_order_relaxed);
+ Count.store(other.GetCount(), std::memory_order_relaxed);
+ }
+
+public:
+ TMetric()
+ : Memory(0)
+ , Count(0)
+ {}
+
+ inline TMetric(const TMetric& other) {
+ Copy(other);
+ }
+
+ inline TMetric(TMetric&& other) {
+ Copy(other);
+ }
+
+ inline TMetric& operator=(const TMetric& other) {
+ Copy(other);
+ return *this;
+ }
+
+ inline TMetric& operator=(TMetric&& other) {
+ Copy(other);
+ return *this;
+ }
+
+ inline ssize_t GetMemory() const {
+ return Memory.load(std::memory_order_relaxed);
+ }
+ inline void SetMemory(ssize_t value) {
+ Memory.store(value, std::memory_order_relaxed);
+ }
+
+ inline ssize_t GetCount() const {
+ return Count.load(std::memory_order_relaxed);
+ }
+ inline void SetCount(ssize_t value) {
+ Count.store(value, std::memory_order_relaxed);
+ }
+
+ inline void operator+=(const TMetric& other) {
+ SetMemory(GetMemory() + other.GetMemory());
+ SetCount(GetCount() + other.GetCount());
+ }
+
+ inline void CalculatePeak(const TMetric& other) {
+ SetMemory(Max(GetMemory(), other.GetMemory()));
+ SetCount(Max(GetCount(), other.GetCount()));
+ }
+
+ inline void Add(size_t size) {
+ SetMemory(GetMemory() + size);
+ SetCount(GetCount() + 1);
+ }
+
+ inline void Sub(size_t size) {
+ SetMemory(GetMemory() - size);
+ SetCount(GetCount() - 1);
+ }
+};
+
+
+class TThreadLocalInfo {
+public:
+ TThreadLocalInfo();
+ ~TThreadLocalInfo();
+
+ TMetric* GetMetric(size_t index);
+ const std::vector<TMetric>& GetMetrics() const;
+
+private:
+ std::vector<TMetric> Metrics;
+
+ inline static TMetric Null = {};
+};
+
+
+class TBaseLabel {
+protected:
+ static size_t RegisterStaticMemoryLabel(const char* name, bool hasSensor);
+
+ inline static TMetric* GetLocalMetric(size_t index) {
+ return FastTlsSingleton<TThreadLocalInfo>()->GetMetric(index);
+ }
+};
+
+
+template <const char* Name>
+class TNameLabel
+ : TBaseLabel
+{
+public:
+ static void Add(size_t size) {
+#if defined(ENABLE_MEMORY_TRACKING)
+ Y_UNUSED(MetricInit);
+
+ if (Y_UNLIKELY(!Metric)) {
+ Metric = GetLocalMetric(Index);
+ }
+
+ Metric->Add(size);
+#else
+ Y_UNUSED(size);
+#endif
+ }
+
+ static void Sub(size_t size) {
+#if defined(ENABLE_MEMORY_TRACKING)
+ Y_UNUSED(MetricInit);
+
+ if (Y_UNLIKELY(!Metric)) {
+ Metric = GetLocalMetric(Index);
+ }
+
+ Metric->Sub(size);
+#else
+ Y_UNUSED(size);
+#endif
+ }
+
+private:
+#if defined(ENABLE_MEMORY_TRACKING)
+ inline static size_t Index = Max<size_t>();
+ inline static struct TMetricInit {
+ TMetricInit() {
+ Index = RegisterStaticMemoryLabel(Name, true);
+ }
+ } MetricInit;
+
+ inline static thread_local TMetric* Metric = nullptr;
+#endif
+};
+
+
+template <typename TType>
+class TTypeLabel
+ : TBaseLabel
+{
+public:
+ static void Add(size_t size) {
+#if defined(ENABLE_MEMORY_TRACKING)
+ Y_UNUSED(MetricInit);
+
+ if (Y_UNLIKELY(!Metric)) {
+ Metric = GetLocalMetric(Index);
+ }
+
+ Metric->Add(size);
+#else
+ Y_UNUSED(size);
+#endif
+ }
+
+ static void Sub(size_t size) {
+#if defined(ENABLE_MEMORY_TRACKING)
+ Y_UNUSED(MetricInit);
+
+ if (Y_UNLIKELY(!Metric)) {
+ Metric = GetLocalMetric(Index);
+ }
+
+ Metric->Sub(size);
+#else
+ Y_UNUSED(size);
+#endif
+ }
+
+private:
+#if defined(ENABLE_MEMORY_TRACKING)
+ inline static size_t Index = Max<size_t>();
+ inline static struct TMetricInit {
+ TMetricInit() {
+ Index = RegisterStaticMemoryLabel(TypeName<TType>().c_str(), false);
+ }
+ } MetricInit;
+
+ inline static thread_local TMetric* Metric = nullptr;
+#endif
+};
+
+
+template <typename T>
+struct TTrackHelper {
+#if defined(ENABLE_MEMORY_TRACKING)
+ void* operator new(size_t size) {
+ T::Add(size);
+ return malloc(size);
+ }
+
+ void* operator new[](size_t size) {
+ T::Add(size);
+ return malloc(size);
+ }
+
+ void operator delete(void* ptr, size_t size) {
+ T::Sub(size);
+ free(ptr);
+ }
+
+ void operator delete[](void* ptr, size_t size) {
+ T::Sub(size);
+ free(ptr);
+ }
+#endif
+};
+
+template <typename TType, typename T>
+struct TAllocHelper {
+ typedef size_t size_type;
+ typedef TType value_type;
+ typedef TType* pointer;
+ typedef const TType* const_pointer;
+
+ struct propagate_on_container_copy_assignment : public std::false_type {};
+ struct propagate_on_container_move_assignment : public std::false_type {};
+ struct propagate_on_container_swap : public std::false_type {};
+
+ pointer allocate(size_type n, const void* hint = nullptr) {
+ Y_UNUSED(hint);
+ auto size = n * sizeof(TType);
+ T::Add(size);
+ return (pointer)malloc(size);
+ }
+
+ void deallocate(pointer ptr, size_t n) {
+ auto size = n * sizeof(TType);
+ T::Sub(size);
+ free((void*)ptr);
+ }
+};
+
+} // NPrivate
+
+
+template <const char* Name>
+using TLabel = NPrivate::TNameLabel<Name>;
+
+template <typename TType, const char* Name = nullptr>
+struct TTrack
+ : public NPrivate::TTrackHelper<NPrivate::TNameLabel<Name>>
+{
+};
+
+template <typename TType>
+struct TTrack<TType, nullptr>
+ : public NPrivate::TTrackHelper<NPrivate::TTypeLabel<TType>>
+{
+};
+
+template <typename TType, const char* Name = nullptr>
+struct TAlloc
+ : public NPrivate::TAllocHelper<TType, NPrivate::TNameLabel<Name>>
+{
+ template<typename U>
+ struct rebind {
+ typedef TAlloc<U, Name> other;
+ };
+};
+
+template <typename TType>
+struct TAlloc<TType, nullptr>
+ : public NPrivate::TAllocHelper<TType, NPrivate::TTypeLabel<TType>>
+{
+ template<typename U>
+ struct rebind {
+ typedef TAlloc<U> other;
+ };
+};
+
+}
+}
+
diff --git a/library/cpp/actors/core/memory_tracker.cpp b/library/cpp/actors/core/memory_tracker.cpp
new file mode 100644
index 0000000000..8a12452c71
--- /dev/null
+++ b/library/cpp/actors/core/memory_tracker.cpp
@@ -0,0 +1,103 @@
+#include "memory_tracker.h"
+
+#include <util/generic/xrange.h>
+
+namespace NActors {
+namespace NMemory {
+
+namespace NPrivate {
+
+TMemoryTracker* TMemoryTracker::Instance() {
+ return SingletonWithPriority<TMemoryTracker, 0>();
+}
+
+void TMemoryTracker::Initialize() {
+ GlobalMetrics.resize(Indices.size());
+}
+
+const std::map<TString, size_t>& TMemoryTracker::GetMetricIndices() const {
+ return Indices;
+}
+
+const std::unordered_set<size_t>& TMemoryTracker::GetSensors() const {
+ return Sensors;
+}
+
+TString TMemoryTracker::GetName(size_t index) const {
+ return Names[index];
+}
+
+size_t TMemoryTracker::GetCount() const {
+ return Indices.size();
+}
+
+void TMemoryTracker::GatherMetrics(std::vector<TMetric>& metrics) const {
+ metrics.resize(0);
+ auto count = GetCount();
+
+ if (!count || GlobalMetrics.size() != count) {
+ return;
+ }
+
+ TReadGuard guard(LockThreadInfo);
+
+ metrics.resize(count);
+ for (size_t i : xrange(count)) {
+ metrics[i] += GlobalMetrics[i];
+ }
+
+ for (auto info : ThreadInfo) {
+ auto& localMetrics = info->GetMetrics();
+ if (localMetrics.size() == count) {
+ for (size_t i : xrange(count)) {
+ metrics[i] += localMetrics[i];
+ }
+ }
+ }
+}
+
+size_t TMemoryTracker::RegisterStaticMemoryLabel(const char* name, bool hasSensor) {
+ size_t index = 0;
+ auto found = Indices.find(name);
+ if (found == Indices.end()) {
+ TString str(name);
+ auto next = Names.size();
+ Indices.emplace(str, next);
+ Names.push_back(str);
+ index = next;
+ } else {
+ index = found->second;
+ }
+
+ if (hasSensor) {
+ Sensors.emplace(index);
+ }
+ return index;
+}
+
+void TMemoryTracker::OnCreateThread(TThreadLocalInfo* info) {
+ TWriteGuard guard(LockThreadInfo);
+ ThreadInfo.insert(info);
+}
+
+void TMemoryTracker::OnDestroyThread(TThreadLocalInfo* info) {
+ TWriteGuard guard(LockThreadInfo);
+
+ auto count = GetCount();
+ if (count && GlobalMetrics.size() == count) {
+ const auto& localMetrics = info->GetMetrics();
+ if (localMetrics.size() == count) {
+ for (size_t i : xrange(count)) {
+ GlobalMetrics[i] += localMetrics[i];
+ }
+ }
+ }
+
+ ThreadInfo.erase(info);
+}
+
+}
+
+}
+}
+
diff --git a/library/cpp/actors/core/memory_tracker.h b/library/cpp/actors/core/memory_tracker.h
new file mode 100644
index 0000000000..e74508191b
--- /dev/null
+++ b/library/cpp/actors/core/memory_tracker.h
@@ -0,0 +1,53 @@
+#pragma once
+
+#include "memory_track.h"
+
+#include <map>
+#include <unordered_map>
+#include <unordered_set>
+
+#include <util/system/rwlock.h>
+
+namespace NActors {
+namespace NMemory {
+
+namespace NPrivate {
+
+class TMemoryTracker {
+public:
+ static TMemoryTracker* Instance();
+
+ void Initialize();
+
+ const std::map<TString, size_t>& GetMetricIndices() const;
+ const std::unordered_set<size_t>& GetSensors() const;
+ TString GetName(size_t index) const;
+ size_t GetCount() const;
+
+ void GatherMetrics(std::vector<TMetric>& metrics) const;
+
+private:
+ size_t RegisterStaticMemoryLabel(const char* name, bool hasSensor);
+
+ void OnCreateThread(TThreadLocalInfo* info);
+ void OnDestroyThread(TThreadLocalInfo* info);
+
+private:
+ std::map<TString, size_t> Indices;
+ std::vector<TString> Names;
+
+ std::vector<TMetric> GlobalMetrics;
+
+ std::unordered_set<size_t> Sensors;
+
+ std::unordered_set<TThreadLocalInfo*> ThreadInfo;
+ TRWMutex LockThreadInfo;
+
+ friend class TThreadLocalInfo;
+ friend class TBaseLabel;
+};
+
+}
+
+}
+}
diff --git a/library/cpp/actors/core/memory_tracker_ut.cpp b/library/cpp/actors/core/memory_tracker_ut.cpp
new file mode 100644
index 0000000000..d168214da6
--- /dev/null
+++ b/library/cpp/actors/core/memory_tracker_ut.cpp
@@ -0,0 +1,262 @@
+#include "memory_tracker.h"
+
+#include <library/cpp/testing/unittest/registar.h>
+
+#include <util/system/hp_timer.h>
+#include <util/system/thread.h>
+
+namespace NActors {
+namespace NMemory {
+
+Y_UNIT_TEST_SUITE(TMemoryTrackerTest) {
+
+#if defined(ENABLE_MEMORY_TRACKING)
+
+using namespace NPrivate;
+
+size_t FindLabelIndex(const char* label) {
+ auto indices = TMemoryTracker::Instance()->GetMetricIndices();
+ auto it = indices.find(label);
+ UNIT_ASSERT(it != indices.end());
+ return it->second;
+}
+
+
+struct TTypeLabeled
+ : public NActors::NMemory::TTrack<TTypeLabeled>
+{
+ char payload[16];
+};
+
+static constexpr char NamedLabel[] = "NamedLabel";
+
+struct TNameLabeled
+ : public NActors::NMemory::TTrack<TNameLabeled, NamedLabel>
+{
+ char payload[32];
+};
+
+Y_UNIT_TEST(Gathering)
+{
+ TMemoryTracker::Instance()->Initialize();
+
+ auto* typed = new TTypeLabeled;
+ auto* typedArray = new TTypeLabeled[3];
+
+ auto* named = new TNameLabeled;
+ auto* namedArray = new TNameLabeled[5];
+ NActors::NMemory::TLabel<NamedLabel>::Add(100);
+
+ std::vector<TMetric> metrics;
+ TMemoryTracker::Instance()->GatherMetrics(metrics);
+
+ auto typeIndex = FindLabelIndex(TypeName<TTypeLabeled>().c_str());
+ UNIT_ASSERT(typeIndex < metrics.size());
+ UNIT_ASSERT(metrics[typeIndex].GetMemory() == sizeof(TTypeLabeled) * 4 + sizeof(size_t));
+ UNIT_ASSERT(metrics[typeIndex].GetCount() == 2);
+
+ auto nameIndex = FindLabelIndex(NamedLabel);
+ UNIT_ASSERT(nameIndex < metrics.size());
+ UNIT_ASSERT(metrics[nameIndex].GetMemory() == sizeof(TNameLabeled) * 6 + sizeof(size_t) + 100);
+ UNIT_ASSERT(metrics[nameIndex].GetCount() == 3);
+
+ NActors::NMemory::TLabel<NamedLabel>::Sub(100);
+ delete [] namedArray;
+ delete named;
+
+ delete [] typedArray;
+ delete typed;
+
+ TMemoryTracker::Instance()->GatherMetrics(metrics);
+
+ UNIT_ASSERT(metrics[typeIndex].GetMemory() == 0);
+ UNIT_ASSERT(metrics[typeIndex].GetCount() == 0);
+
+ UNIT_ASSERT(metrics[nameIndex].GetMemory() == 0);
+ UNIT_ASSERT(metrics[nameIndex].GetCount() == 0);
+}
+
+
+static constexpr char InContainerLabel[] = "InContainerLabel";
+
+struct TInContainer {
+ char payload[16];
+};
+
+Y_UNIT_TEST(Containers) {
+ TMemoryTracker::Instance()->Initialize();
+
+ std::vector<TInContainer, NActors::NMemory::TAlloc<TInContainer>> vecT;
+ vecT.resize(5);
+
+ std::vector<TInContainer, NActors::NMemory::TAlloc<TInContainer, InContainerLabel>> vecN;
+ vecN.resize(7);
+
+ using TKey = int;
+
+ std::map<TKey, TInContainer, std::less<TKey>,
+ NActors::NMemory::TAlloc<std::pair<const TKey, TInContainer>>> mapT;
+ mapT.emplace(0, TInContainer());
+ mapT.emplace(1, TInContainer());
+
+ std::map<TKey, TInContainer, std::less<TKey>,
+ NActors::NMemory::TAlloc<std::pair<const TKey, TInContainer>, InContainerLabel>> mapN;
+ mapN.emplace(0, TInContainer());
+
+ std::unordered_map<TKey, TInContainer, std::hash<TKey>, std::equal_to<TKey>,
+ NActors::NMemory::TAlloc<std::pair<const TKey, TInContainer>>> umapT;
+ umapT.emplace(0, TInContainer());
+
+ std::unordered_map<TKey, TInContainer, std::hash<TKey>, std::equal_to<TKey>,
+ NActors::NMemory::TAlloc<std::pair<const TKey, TInContainer>, InContainerLabel>> umapN;
+ umapN.emplace(0, TInContainer());
+ umapN.emplace(1, TInContainer());
+
+ std::vector<TMetric> metrics;
+ TMemoryTracker::Instance()->GatherMetrics(metrics);
+
+ auto indices = TMemoryTracker::Instance()->GetMetricIndices();
+ for (auto& [name, index] : indices) {
+ Cerr << "---- " << name
+ << ": memory = " << metrics[index].GetMemory()
+ << ", count = " << metrics[index].GetCount() << Endl;
+ }
+
+ auto vecTIndex = FindLabelIndex(TypeName<TInContainer>().c_str());
+ UNIT_ASSERT(metrics[vecTIndex].GetMemory() >= ssize_t(sizeof(TInContainer) * 5));
+ UNIT_ASSERT(metrics[vecTIndex].GetCount() == 1);
+
+ auto labelIndex = FindLabelIndex(InContainerLabel);
+ UNIT_ASSERT(metrics[labelIndex].GetCount() == 5);
+ UNIT_ASSERT(metrics[labelIndex].GetMemory() >= ssize_t(
+ sizeof(TInContainer) * 7 +
+ sizeof(decltype(mapN)::value_type) +
+ sizeof(decltype(umapN)::value_type) * 2));
+}
+
+
+static constexpr char InThreadLabel[] = "InThreadLabel";
+
+struct TInThread
+ : public NActors::NMemory::TTrack<TInThread, InThreadLabel>
+{
+ char payload[16];
+};
+
+void* ThreadProc(void*) {
+ return new TInThread;
+}
+
+Y_UNIT_TEST(Threads) {
+ TMemoryTracker::Instance()->Initialize();
+
+ auto index = FindLabelIndex(InThreadLabel);
+
+ auto* object1 = new TInThread;
+
+ std::vector<TMetric> metrics;
+ TMemoryTracker::Instance()->GatherMetrics(metrics);
+ UNIT_ASSERT(metrics[index].GetMemory() == sizeof(TInThread));
+ UNIT_ASSERT(metrics[index].GetCount() == 1);
+
+ TThread thread(&ThreadProc, nullptr);
+ thread.Start();
+ auto* object2 = static_cast<TInThread*>(thread.Join());
+
+ TMemoryTracker::Instance()->GatherMetrics(metrics);
+ UNIT_ASSERT(metrics[index].GetMemory() == sizeof(TInThread) * 2);
+ UNIT_ASSERT(metrics[index].GetCount() == 2);
+
+ delete object2;
+
+ TMemoryTracker::Instance()->GatherMetrics(metrics);
+ UNIT_ASSERT(metrics[index].GetMemory() == sizeof(TInThread));
+ UNIT_ASSERT(metrics[index].GetCount() == 1);
+
+ delete object1;
+}
+
+
+struct TNotTracked {
+ char payload[16];
+};
+
+struct TTracked
+ : public NActors::NMemory::TTrack<TTracked>
+{
+ char payload[16];
+};
+
+template <typename T>
+double MeasureAllocations() {
+ constexpr size_t objectsCount = 4 << 20;
+
+ std::vector<T*> objects;
+ objects.resize(objectsCount);
+
+ THPTimer timer;
+
+ for (size_t i = 0; i < objectsCount; ++i) {
+ objects[i] = new T;
+ }
+
+ for (size_t i = 0; i < objectsCount; ++i) {
+ delete objects[i];
+ }
+
+ auto seconds = timer.Passed();
+ Cerr << "---- objects: " << objectsCount << ", time: " << seconds << Endl;
+ return seconds;
+}
+
+Y_UNIT_TEST(Performance) {
+ TMemoryTracker::Instance()->Initialize();
+
+ constexpr size_t Runs = 16;
+
+ Cerr << "---- warmup" << Endl;
+ MeasureAllocations<TNotTracked>();
+ MeasureAllocations<TTracked>();
+
+ std::vector<double> noTrack;
+ std::vector<double> track;
+
+ for (size_t run = 0; run < Runs; ++run) {
+ Cerr << "---- no track" << Endl;
+ auto time = MeasureAllocations<TNotTracked>();
+ noTrack.push_back(time);
+
+ Cerr << "---- track" << Endl;
+ time = MeasureAllocations<TTracked>();
+ track.push_back(time);
+ }
+
+ double meanNoTrack = 0, stddevNoTrack = 0;
+ double meanTrack = 0, stddevTrack = 0;
+ for (size_t i = 0; i < Runs; ++i) {
+ meanNoTrack += noTrack[i];
+ meanTrack += track[i];
+ }
+ meanNoTrack /= Runs;
+ meanTrack /= Runs;
+
+ auto sqr = [](double val) { return val * val; };
+
+ for (size_t i = 0; i < Runs; ++i) {
+ stddevNoTrack += sqr(noTrack[i] - meanNoTrack);
+ stddevTrack += sqr(track[i] - meanTrack);
+ }
+ stddevNoTrack = sqrt(stddevNoTrack / (Runs - 1));
+ stddevTrack = sqrt(stddevTrack / (Runs - 1));
+
+ Cerr << "---- no track - mean: " << meanNoTrack << ", stddev: " << stddevNoTrack << Endl;
+ Cerr << "---- track - mean: " << meanTrack << ", stddev: " << stddevTrack << Endl;
+ Cerr << "---- tracking is slower by " << int((meanTrack / meanNoTrack - 1.0) * 100) << "%" << Endl;
+}
+
+#endif
+
+}
+
+}
+}
diff --git a/library/cpp/actors/core/mon.h b/library/cpp/actors/core/mon.h
new file mode 100644
index 0000000000..c450f2338e
--- /dev/null
+++ b/library/cpp/actors/core/mon.h
@@ -0,0 +1,234 @@
+#pragma once
+
+#include "events.h"
+#include "event_local.h"
+#include <library/cpp/monlib/service/monservice.h>
+#include <library/cpp/monlib/service/pages/mon_page.h>
+
+namespace NActors {
+ namespace NMon {
+ enum {
+ HttpInfo = EventSpaceBegin(NActors::TEvents::ES_MON),
+ HttpInfoRes,
+ RemoteHttpInfo,
+ RemoteHttpInfoRes,
+ RemoteJsonInfoRes,
+ RemoteBinaryInfoRes,
+ End
+ };
+
+ static_assert(End < EventSpaceEnd(NActors::TEvents::ES_MON), "expect End < EventSpaceEnd(NActors::TEvents::ES_MON)");
+
+ // request info from an actor in HTML format
+ struct TEvHttpInfo: public NActors::TEventLocal<TEvHttpInfo, HttpInfo> {
+ TEvHttpInfo(const NMonitoring::IMonHttpRequest& request, int subReqId = 0)
+ : Request(request)
+ , SubRequestId(subReqId)
+ {
+ }
+
+ TEvHttpInfo(const NMonitoring::IMonHttpRequest& request, const TString& userToken)
+ : Request(request)
+ , UserToken(userToken)
+ , SubRequestId(0)
+ {
+ }
+
+ const NMonitoring::IMonHttpRequest& Request;
+ TString UserToken; // built and serialized
+ // SubRequestId != 0 means that we assemble reply from multiple parts and SubRequestId contains this part id
+ int SubRequestId;
+ };
+
+ // base class for HTTP info response
+ struct IEvHttpInfoRes: public NActors::TEventLocal<IEvHttpInfoRes, HttpInfoRes> {
+ enum EContentType {
+ Html,
+ Custom,
+ };
+
+ IEvHttpInfoRes() {
+ }
+
+ virtual ~IEvHttpInfoRes() {
+ }
+
+ virtual void Output(IOutputStream& out) const = 0;
+ virtual EContentType GetContentType() const = 0;
+ };
+
+ // Ready to output HTML in TString
+ struct TEvHttpInfoRes: public IEvHttpInfoRes {
+ TEvHttpInfoRes(const TString& answer, int subReqId = 0, EContentType contentType = Html)
+ : Answer(answer)
+ , SubRequestId(subReqId)
+ , ContentType(contentType)
+ {
+ }
+
+ void Output(IOutputStream& out) const override {
+ out << Answer;
+ }
+
+ EContentType GetContentType() const override {
+ return ContentType;
+ }
+
+ const TString Answer;
+ const int SubRequestId;
+ const EContentType ContentType;
+ };
+
+ struct TEvRemoteHttpInfo: public NActors::TEventBase<TEvRemoteHttpInfo, RemoteHttpInfo> {
+ TEvRemoteHttpInfo() {
+ }
+
+ TEvRemoteHttpInfo(const TString& query)
+ : Query(query)
+ {
+ }
+
+ TEvRemoteHttpInfo(const TString& query, HTTP_METHOD method)
+ : Query(query)
+ , Method(method)
+ {
+ }
+
+ TString Query;
+ HTTP_METHOD Method;
+
+ TString PathInfo() const {
+ const size_t pos = Query.find('?');
+ return (pos == TString::npos) ? TString() : Query.substr(0, pos);
+ }
+
+ TCgiParameters Cgi() const {
+ const size_t pos = Query.find('?');
+ return TCgiParameters((pos == TString::npos) ? TString() : Query.substr(pos + 1));
+ }
+
+ TString ToStringHeader() const override {
+ return "TEvRemoteHttpInfo";
+ }
+
+ bool SerializeToArcadiaStream(TChunkSerializer *serializer) const override {
+ return serializer->WriteString(&Query);
+ }
+
+ ui32 CalculateSerializedSize() const override {
+ return Query.size();
+ }
+
+ bool IsSerializable() const override {
+ return true;
+ }
+
+ static IEventBase* Load(TEventSerializedData* bufs) {
+ return new TEvRemoteHttpInfo(bufs->GetString());
+ }
+
+ HTTP_METHOD GetMethod() const
+ {
+ return Method;
+ }
+ };
+
+ struct TEvRemoteHttpInfoRes: public NActors::TEventBase<TEvRemoteHttpInfoRes, RemoteHttpInfoRes> {
+ TEvRemoteHttpInfoRes() {
+ }
+
+ TEvRemoteHttpInfoRes(const TString& html)
+ : Html(html)
+ {
+ }
+
+ TString Html;
+
+ TString ToStringHeader() const override {
+ return "TEvRemoteHttpInfoRes";
+ }
+
+ bool SerializeToArcadiaStream(TChunkSerializer *serializer) const override {
+ return serializer->WriteString(&Html);
+ }
+
+ ui32 CalculateSerializedSize() const override {
+ return Html.size();
+ }
+
+ bool IsSerializable() const override {
+ return true;
+ }
+
+ static IEventBase* Load(TEventSerializedData* bufs) {
+ return new TEvRemoteHttpInfoRes(bufs->GetString());
+ }
+ };
+
+ struct TEvRemoteJsonInfoRes: public NActors::TEventBase<TEvRemoteJsonInfoRes, RemoteJsonInfoRes> {
+ TEvRemoteJsonInfoRes() {
+ }
+
+ TEvRemoteJsonInfoRes(const TString& json)
+ : Json(json)
+ {
+ }
+
+ TString Json;
+
+ TString ToStringHeader() const override {
+ return "TEvRemoteJsonInfoRes";
+ }
+
+ bool SerializeToArcadiaStream(TChunkSerializer *serializer) const override {
+ return serializer->WriteString(&Json);
+ }
+
+ ui32 CalculateSerializedSize() const override {
+ return Json.size();
+ }
+
+ bool IsSerializable() const override {
+ return true;
+ }
+
+ static IEventBase* Load(TEventSerializedData* bufs) {
+ return new TEvRemoteJsonInfoRes(bufs->GetString());
+ }
+ };
+
+ struct TEvRemoteBinaryInfoRes: public NActors::TEventBase<TEvRemoteBinaryInfoRes, RemoteBinaryInfoRes> {
+ TEvRemoteBinaryInfoRes() {
+ }
+
+ TEvRemoteBinaryInfoRes(const TString& blob)
+ : Blob(blob)
+ {
+ }
+
+ TString Blob;
+
+ TString ToStringHeader() const override {
+ return "TEvRemoteBinaryInfoRes";
+ }
+
+ bool SerializeToArcadiaStream(TChunkSerializer *serializer) const override {
+ return serializer->WriteString(&Blob);
+ }
+
+ ui32 CalculateSerializedSize() const override {
+ return Blob.size();
+ }
+
+ bool IsSerializable() const override {
+ return true;
+ }
+
+ static IEventBase* Load(TEventSerializedData* bufs) {
+ return new TEvRemoteBinaryInfoRes(bufs->GetString());
+ }
+ };
+
+ }
+
+}
diff --git a/library/cpp/actors/core/mon_stats.h b/library/cpp/actors/core/mon_stats.h
new file mode 100644
index 0000000000..d55552af0c
--- /dev/null
+++ b/library/cpp/actors/core/mon_stats.h
@@ -0,0 +1,147 @@
+#pragma once
+
+#include "defs.h"
+#include "actor.h"
+#include <library/cpp/monlib/metrics/histogram_snapshot.h>
+#include <util/system/hp_timer.h>
+
+namespace NActors {
+ struct TLogHistogram : public NMonitoring::IHistogramSnapshot {
+ TLogHistogram() {
+ memset(Buckets, 0, sizeof(Buckets));
+ }
+
+ inline void Add(ui64 val, ui64 inc = 1) {
+ size_t ind = 0;
+#if defined(__clang__) && __clang_major__ == 3 && __clang_minor__ == 7
+ asm volatile("" ::
+ : "memory");
+#endif
+ if (val > 1) {
+ ind = GetValueBitCount(val - 1);
+ }
+#if defined(__clang__) && __clang_major__ == 3 && __clang_minor__ == 7
+ asm volatile("" ::
+ : "memory");
+#endif
+ RelaxedStore(&TotalSamples, RelaxedLoad(&TotalSamples) + inc);
+ RelaxedStore(&Buckets[ind], RelaxedLoad(&Buckets[ind]) + inc);
+ }
+
+ void Aggregate(const TLogHistogram& other) {
+ const ui64 inc = RelaxedLoad(&other.TotalSamples);
+ RelaxedStore(&TotalSamples, RelaxedLoad(&TotalSamples) + inc);
+ for (size_t i = 0; i < Y_ARRAY_SIZE(Buckets); ++i) {
+ Buckets[i] += RelaxedLoad(&other.Buckets[i]);
+ }
+ }
+
+ // IHistogramSnapshot
+ ui32 Count() const override {
+ return Y_ARRAY_SIZE(Buckets);
+ }
+
+ NMonitoring::TBucketBound UpperBound(ui32 index) const override {
+ Y_ASSERT(index < Y_ARRAY_SIZE(Buckets));
+ if (index == 0) {
+ return 1;
+ }
+ return NMonitoring::TBucketBound(1ull << (index - 1)) * 2.0;
+ }
+
+ NMonitoring::TBucketValue Value(ui32 index) const override {
+ Y_ASSERT(index < Y_ARRAY_SIZE(Buckets));
+ return Buckets[index];
+ }
+
+ ui64 TotalSamples = 0;
+ ui64 Buckets[65];
+ };
+
+ struct TExecutorPoolStats {
+ ui64 MaxUtilizationTime = 0;
+ };
+
+ struct TExecutorThreadStats {
+ ui64 SentEvents = 0;
+ ui64 ReceivedEvents = 0;
+ ui64 PreemptedEvents = 0; // Number of events experienced hard preemption
+ ui64 NonDeliveredEvents = 0;
+ ui64 EmptyMailboxActivation = 0;
+ ui64 CpuNs = 0; // nanoseconds thread was executing on CPU (accounts for preemtion)
+ NHPTimer::STime ElapsedTicks = 0;
+ NHPTimer::STime ParkedTicks = 0;
+ NHPTimer::STime BlockedTicks = 0;
+ TLogHistogram ActivationTimeHistogram;
+ TLogHistogram EventDeliveryTimeHistogram;
+ TLogHistogram EventProcessingCountHistogram;
+ TLogHistogram EventProcessingTimeHistogram;
+ TVector<NHPTimer::STime> ElapsedTicksByActivity;
+ TVector<ui64> ReceivedEventsByActivity;
+ TVector<i64> ActorsAliveByActivity; // the sum should be positive, but per-thread might be negative
+ TVector<ui64> ScheduledEventsByActivity;
+ ui64 PoolActorRegistrations = 0;
+ ui64 PoolDestroyedActors = 0;
+ ui64 PoolAllocatedMailboxes = 0;
+ ui64 MailboxPushedOutBySoftPreemption = 0;
+ ui64 MailboxPushedOutByTime = 0;
+ ui64 MailboxPushedOutByEventCount = 0;
+
+ TExecutorThreadStats(size_t activityVecSize = 1) // must be not empty as 0 used as default
+ : ElapsedTicksByActivity(activityVecSize)
+ , ReceivedEventsByActivity(activityVecSize)
+ , ActorsAliveByActivity(activityVecSize)
+ , ScheduledEventsByActivity(activityVecSize)
+ {}
+
+ template <typename T>
+ static void AggregateOne(TVector<T>& self, const TVector<T>& other) {
+ const size_t selfSize = self.size();
+ const size_t otherSize = other.size();
+ if (selfSize < otherSize)
+ self.resize(otherSize);
+ for (size_t at = 0; at < otherSize; ++at)
+ self[at] += RelaxedLoad(&other[at]);
+ }
+
+ void Aggregate(const TExecutorThreadStats& other) {
+ SentEvents += RelaxedLoad(&other.SentEvents);
+ ReceivedEvents += RelaxedLoad(&other.ReceivedEvents);
+ PreemptedEvents += RelaxedLoad(&other.PreemptedEvents);
+ NonDeliveredEvents += RelaxedLoad(&other.NonDeliveredEvents);
+ EmptyMailboxActivation += RelaxedLoad(&other.EmptyMailboxActivation);
+ CpuNs += RelaxedLoad(&other.CpuNs);
+ ElapsedTicks += RelaxedLoad(&other.ElapsedTicks);
+ ParkedTicks += RelaxedLoad(&other.ParkedTicks);
+ BlockedTicks += RelaxedLoad(&other.BlockedTicks);
+ MailboxPushedOutBySoftPreemption += RelaxedLoad(&other.MailboxPushedOutBySoftPreemption);
+ MailboxPushedOutByTime += RelaxedLoad(&other.MailboxPushedOutByTime);
+ MailboxPushedOutByEventCount += RelaxedLoad(&other.MailboxPushedOutByEventCount);
+
+ ActivationTimeHistogram.Aggregate(other.ActivationTimeHistogram);
+ EventDeliveryTimeHistogram.Aggregate(other.EventDeliveryTimeHistogram);
+ EventProcessingCountHistogram.Aggregate(other.EventProcessingCountHistogram);
+ EventProcessingTimeHistogram.Aggregate(other.EventProcessingTimeHistogram);
+
+ AggregateOne(ElapsedTicksByActivity, other.ElapsedTicksByActivity);
+ AggregateOne(ReceivedEventsByActivity, other.ReceivedEventsByActivity);
+ AggregateOne(ActorsAliveByActivity, other.ActorsAliveByActivity);
+ AggregateOne(ScheduledEventsByActivity, other.ScheduledEventsByActivity);
+
+ RelaxedStore(
+ &PoolActorRegistrations,
+ std::max(RelaxedLoad(&PoolActorRegistrations), RelaxedLoad(&other.PoolActorRegistrations)));
+ RelaxedStore(
+ &PoolDestroyedActors,
+ std::max(RelaxedLoad(&PoolDestroyedActors), RelaxedLoad(&other.PoolDestroyedActors)));
+ RelaxedStore(
+ &PoolAllocatedMailboxes,
+ std::max(RelaxedLoad(&PoolAllocatedMailboxes), RelaxedLoad(&other.PoolAllocatedMailboxes)));
+ }
+
+ size_t MaxActivityType() const {
+ return ActorsAliveByActivity.size();
+ }
+ };
+
+}
diff --git a/library/cpp/actors/core/monotonic.cpp b/library/cpp/actors/core/monotonic.cpp
new file mode 100644
index 0000000000..3465149dbe
--- /dev/null
+++ b/library/cpp/actors/core/monotonic.cpp
@@ -0,0 +1,23 @@
+#include "monotonic.h"
+
+#include <chrono>
+
+namespace NActors {
+
+ namespace {
+ // Unfortunately time_since_epoch() is sometimes negative on wine
+ // Remember initial time point at program start and use offsets from that
+ std::chrono::steady_clock::time_point MonotonicOffset = std::chrono::steady_clock::now();
+ }
+
+ ui64 GetMonotonicMicroSeconds() {
+ auto microseconds = std::chrono::duration_cast<std::chrono::microseconds>(std::chrono::steady_clock::now() - MonotonicOffset).count();
+ // Steady clock is supposed to never jump backwards, but it's better to be safe in case of buggy implementations
+ if (Y_UNLIKELY(microseconds < 0)) {
+ microseconds = 0;
+ }
+ // Add one so we never return zero
+ return microseconds + 1;
+ }
+
+} // namespace NActors
diff --git a/library/cpp/actors/core/monotonic.h b/library/cpp/actors/core/monotonic.h
new file mode 100644
index 0000000000..6fceb91dbe
--- /dev/null
+++ b/library/cpp/actors/core/monotonic.h
@@ -0,0 +1,111 @@
+#pragma once
+
+#include <util/datetime/base.h>
+
+namespace NActors {
+
+ /**
+ * Returns current monotonic time in microseconds
+ */
+ ui64 GetMonotonicMicroSeconds();
+
+ /**
+ * Similar to TInstant, but measuring monotonic time
+ */
+ class TMonotonic : public TTimeBase<TMonotonic> {
+ using TBase = TTimeBase<TMonotonic>;
+
+ private:
+ constexpr explicit TMonotonic(TValue value) noexcept
+ : TBase(value)
+ { }
+
+ public:
+ constexpr TMonotonic() noexcept {
+ }
+
+ static constexpr TMonotonic FromValue(TValue value) noexcept {
+ return TMonotonic(value);
+ }
+
+ static inline TMonotonic Now() {
+ return TMonotonic::MicroSeconds(GetMonotonicMicroSeconds());
+ }
+
+ using TBase::Days;
+ using TBase::Hours;
+ using TBase::MicroSeconds;
+ using TBase::MilliSeconds;
+ using TBase::Minutes;
+ using TBase::Seconds;
+
+ static constexpr TMonotonic Max() noexcept {
+ return TMonotonic(::Max<ui64>());
+ }
+
+ static constexpr TMonotonic Zero() noexcept {
+ return TMonotonic();
+ }
+
+ static constexpr TMonotonic MicroSeconds(ui64 us) noexcept {
+ return TMonotonic(TInstant::MicroSeconds(us).GetValue());
+ }
+
+ static constexpr TMonotonic MilliSeconds(ui64 ms) noexcept {
+ return TMonotonic(TInstant::MilliSeconds(ms).GetValue());
+ }
+
+ static constexpr TMonotonic Seconds(ui64 s) noexcept {
+ return TMonotonic(TInstant::Seconds(s).GetValue());
+ }
+
+ static constexpr TMonotonic Minutes(ui64 m) noexcept {
+ return TMonotonic(TInstant::Minutes(m).GetValue());
+ }
+
+ static constexpr TMonotonic Hours(ui64 h) noexcept {
+ return TMonotonic(TInstant::Hours(h).GetValue());
+ }
+
+ static constexpr TMonotonic Days(ui64 d) noexcept {
+ return TMonotonic(TInstant::Days(d).GetValue());
+ }
+
+ template<class T>
+ inline TMonotonic& operator+=(const T& t) noexcept {
+ return (*this = (*this + t));
+ }
+
+ template<class T>
+ inline TMonotonic& operator-=(const T& t) noexcept {
+ return (*this = (*this - t));
+ }
+ };
+} // namespace NActors
+
+Y_DECLARE_PODTYPE(NActors::TMonotonic);
+
+template<>
+struct THash<NActors::TMonotonic> {
+ size_t operator()(const NActors::TMonotonic& key) const {
+ return THash<NActors::TMonotonic::TValue>()(key.GetValue());
+ }
+};
+
+namespace NActors {
+
+ constexpr TDuration operator-(const TMonotonic& l, const TMonotonic& r) {
+ return TInstant::FromValue(l.GetValue()) - TInstant::FromValue(r.GetValue());
+ }
+
+ constexpr TMonotonic operator+(const TMonotonic& l, const TDuration& r) {
+ TInstant result = TInstant::FromValue(l.GetValue()) + r;
+ return TMonotonic::FromValue(result.GetValue());
+ }
+
+ constexpr TMonotonic operator-(const TMonotonic& l, const TDuration& r) {
+ TInstant result = TInstant::FromValue(l.GetValue()) - r;
+ return TMonotonic::FromValue(result.GetValue());
+ }
+
+} // namespace NActors
diff --git a/library/cpp/actors/core/probes.cpp b/library/cpp/actors/core/probes.cpp
new file mode 100644
index 0000000000..7ace83e102
--- /dev/null
+++ b/library/cpp/actors/core/probes.cpp
@@ -0,0 +1,28 @@
+#include "probes.h"
+
+#include "actorsystem.h"
+
+#include <util/string/builder.h>
+
+LWTRACE_DEFINE_PROVIDER(ACTORLIB_PROVIDER);
+
+namespace NActors {
+ TVector<NLWTrace::TDashboard> LWTraceDashboards(TActorSystemSetup* setup) {
+ TVector<NLWTrace::TDashboard> result;
+
+ NLWTrace::TDashboard slowDash;
+ ui32 pools = setup->GetExecutorsCount();
+ size_t top = 30;
+ slowDash.SetName("ActorSystem slow events");
+ slowDash.SetDescription(TStringBuilder() << "TOP" << top << " slow event executions >1M cycles for every pool (refresh page to update)");
+ for (ui32 pool = 0; pool < pools; pool++) {
+ auto* row = slowDash.AddRows();
+ auto* cell = row->AddCells();
+ cell->SetTitle(TStringBuilder() << pool << ":" << setup->GetPoolName(pool));
+ cell->SetUrl(TStringBuilder() << "?mode=log&id=.ACTORLIB_PROVIDER.SlowEvent.ppoolId=" << pool << "&s=eventMs&reverse=y&head=30");
+ }
+ result.push_back(slowDash);
+
+ return result;
+ }
+}
diff --git a/library/cpp/actors/core/probes.h b/library/cpp/actors/core/probes.h
new file mode 100644
index 0000000000..4912d6dd26
--- /dev/null
+++ b/library/cpp/actors/core/probes.h
@@ -0,0 +1,176 @@
+#pragma once
+
+#include <library/cpp/lwtrace/all.h>
+#include <util/generic/vector.h>
+
+#define LWACTORID(x) (x).RawX1(), (x).RawX2(), (x).NodeId(), (x).PoolID()
+#define LWTYPE_ACTORID ui64, ui64, ui32, ui32
+#define LWNAME_ACTORID(n) n "Raw1", n "Raw2", n "NodeId", n "PoolId"
+
+#define ACTORLIB_PROVIDER(PROBE, EVENT, GROUPS, TYPES, NAMES) \
+ PROBE(SlowEvent, GROUPS("ActorLibSlow"), \
+ TYPES(ui32, double, TString, TString, TString), \
+ NAMES("poolId", "eventMs", "eventType", "actorId", "actorType")) \
+ PROBE(EventSlowDelivery, GROUPS("ActorLibSlow"), \
+ TYPES(ui32, double, double, ui64, TString, TString, TString), \
+ NAMES("poolId", "deliveryMs", "sinceActivationMs", "eventProcessedBefore", "eventType", "actorId", "actorType")) \
+ PROBE(SlowActivation, GROUPS("ActorLibSlow"), \
+ TYPES(ui32, double), \
+ NAMES("poolId", "activationMs")) \
+ PROBE(SlowRegisterNew, GROUPS("ActorLibSlow"), \
+ TYPES(ui32, double), \
+ NAMES("poolId", "registerNewMs")) \
+ PROBE(SlowRegisterAdd, GROUPS("ActorLibSlow"), \
+ TYPES(ui32, double), \
+ NAMES("poolId", "registerAddMs")) \
+ PROBE(MailboxPushedOutBySoftPreemption, GROUPS("ActorLibMailbox", "ActorLibMailboxPushedOut"), \
+ TYPES(ui32, TString, ui32, TDuration, ui64, TString, TString), \
+ NAMES("poolId", "pool", "eventsProcessed", "procTimeMs", "workerId", "actorId", "actorType")) \
+ PROBE(MailboxPushedOutByTime, GROUPS("ActorLibMailbox", "ActorLibMailboxPushedOut"), \
+ TYPES(ui32, TString, ui32, TDuration, ui64, TString, TString), \
+ NAMES("poolId", "pool", "eventsProcessed", "procTimeMs", "workerId", "actorId", "actorType")) \
+ PROBE(MailboxPushedOutByEventCount, GROUPS("ActorLibMailbox", "ActorLibMailboxPushedOut"), \
+ TYPES(ui32, TString, ui32, TDuration, ui64, TString, TString), \
+ NAMES("poolId", "pool", "eventsProcessed", "procTimeMs", "workerId", "actorId", "actorType")) \
+ PROBE(MailboxEmpty, GROUPS("ActorLibMailbox"), \
+ TYPES(ui32, TString, ui32, TDuration, ui64, TString, TString), \
+ NAMES("poolId", "pool", "eventsProcessed", "procTimeMs", "workerId", "actorId", "actorType")) \
+ PROBE(ActivationBegin, GROUPS(), \
+ TYPES(ui32, ui32, ui32, double), \
+ NAMES("cpu", "poolId", "workerId", "expireMs")) \
+ PROBE(ActivationEnd, GROUPS(), \
+ TYPES(ui32, ui32, ui32), \
+ NAMES("cpu", "poolId", "workerId")) \
+ PROBE(ExecutorThreadStats, GROUPS("ActorLibStats"), \
+ TYPES(ui32, TString, ui64, ui64, ui64, double, double), \
+ NAMES("poolId", "pool", "workerId", "execCount", "readyActivationCount", "execMs", "nonExecMs")) \
+ PROBE(SlowICReadLoopAdjustSize, GROUPS("ActorLibSlowIC"), \
+ TYPES(double), \
+ NAMES("icReadLoopAdjustSizeMs")) \
+ PROBE(SlowICReadFromSocket, GROUPS("ActorLibSlowIC"), \
+ TYPES(double), \
+ NAMES("icReadFromSocketMs")) \
+ PROBE(SlowICReadLoopSend, GROUPS("ActorLibSlowIC"), \
+ TYPES(double), \
+ NAMES("icReadLoopSendMs")) \
+ PROBE(SlowICAllocPacketBuffer, GROUPS("ActorLibSlowIC"), \
+ TYPES(ui32, double), \
+ NAMES("peerId", "icAllocPacketBufferMs")) \
+ PROBE(SlowICFillSendingBuffer, GROUPS("ActorLibSlowIC"), \
+ TYPES(ui32, double), \
+ NAMES("peerId", "icFillSendingBufferMs")) \
+ PROBE(SlowICPushSentPackets, GROUPS("ActorLibSlowIC"), \
+ TYPES(ui32, double), \
+ NAMES("peerId", "icPushSentPacketsMs")) \
+ PROBE(SlowICPushSendQueue, GROUPS("ActorLibSlowIC"), \
+ TYPES(ui32, double), \
+ NAMES("peerId", "icPushSendQueueMs")) \
+ PROBE(SlowICWriteData, GROUPS("ActorLibSlowIC"), \
+ TYPES(ui32, double), \
+ NAMES("peerId", "icWriteDataMs")) \
+ PROBE(SlowICDropConfirmed, GROUPS("ActorLibSlowIC"), \
+ TYPES(ui32, double), \
+ NAMES("peerId", "icDropConfirmedMs")) \
+ PROBE(ActorsystemScheduler, GROUPS("Durations"), \
+ TYPES(ui64, ui64, ui32, ui32, ui64, ui64), \
+ NAMES("timeUs", "timerfd_expirations", "eventsGottenFromQueues", "eventsSent", \
+ "eventsInSendQueue", "eventSchedulingErrorUs")) \
+ PROBE(ForwardEvent, GROUPS("Orbit", "InterconnectSessionTCP"), \
+ TYPES(ui32, ui32, ui32, LWTYPE_ACTORID, LWTYPE_ACTORID, ui64, ui32), \
+ NAMES("peerId", "type", "flags", LWNAME_ACTORID("r"), LWNAME_ACTORID("s"), \
+ "cookie", "eventSerializedSize")) \
+ PROBE(EnqueueEvent, GROUPS("InterconnectSessionTCP"), \
+ TYPES(ui32, ui64, TDuration, ui16, ui64, ui64), \
+ NAMES("peerId", "numEventsInReadyChannels", "enqueueBlockedTotalMs", "channelId", "queueSizeInEvents", "queueSizeInBytes")) \
+ PROBE(SerializeToPacketBegin, GROUPS("InterconnectSessionTCP"), \
+ TYPES(ui32, ui16, ui64), \
+ NAMES("peerId", "channelId", "outputQueueSize")) \
+ PROBE(SerializeToPacketEnd, GROUPS("InterconnectSessionTCP"), \
+ TYPES(ui32, ui16, ui64, ui64), \
+ NAMES("peerId", "channelId", "outputQueueSize", "offsetInPacket")) \
+ PROBE(FillSendingBuffer, GROUPS("InterconnectSessionTCP"), \
+ TYPES(ui32, ui32, ui64, TDuration), \
+ NAMES("peerId", "taskBytesGenerated", "numEventsInReadyChannelsBehind", "fillBlockedTotalMs")) \
+ PROBE(PacketGenerated, GROUPS("InterconnectSessionTCP"), \
+ TYPES(ui32, ui64, ui64, ui64, ui64), \
+ NAMES("peerId", "bytesUnwritten", "inflightBytes", "packetsGenerated", "packetSize")) \
+ PROBE(PacketWrittenToSocket, GROUPS("InterconnectSessionTCP"), \
+ TYPES(ui32, ui64, bool, ui64, ui64, TDuration, int), \
+ NAMES("peerId", "packetsWrittenToSocket", "triedWriting", "packetDataSize", "bytesUnwritten", "writeBlockedTotalMs", "fd")) \
+ PROBE(GenerateTraffic, GROUPS("InterconnectSessionTCP"), \
+ TYPES(ui32, double, ui64, ui32, ui64), \
+ NAMES("peerId", "generateTrafficMs", "dataBytesSent", "generatedPackets", "generatedBytes")) \
+ PROBE(WriteToSocket, GROUPS("InterconnectSessionTCP"), \
+ TYPES(ui32, ui64, ui64, ui64, ui64, TDuration, int), \
+ NAMES("peerId", "bytesWritten", "packetsWritten", "packetsWrittenToSocket", "bytesUnwritten", "writeBlockedTotalMs", "fd")) \
+ PROBE(UpdateFromInputSession, GROUPS("InterconnectSessionTCP"), \
+ TYPES(ui32, double), \
+ NAMES("peerId", "pingMs")) \
+ PROBE(UnblockByDropConfirmed, GROUPS("InterconnectSessionTCP"), \
+ TYPES(ui32, double), \
+ NAMES("peerId", "updateDeliveryMs")) \
+ PROBE(DropConfirmed, GROUPS("InterconnectSessionTCP"), \
+ TYPES(ui32, ui64, ui64), \
+ NAMES("peerId", "droppedBytes", "inflightBytes")) \
+ PROBE(StartRam, GROUPS("InterconnectSessionTCP"), \
+ TYPES(ui32), \
+ NAMES("peerId")) \
+ PROBE(FinishRam, GROUPS("InterconnectSessionTCP"), \
+ TYPES(ui32, double), \
+ NAMES("peerId", "ramMs")) \
+ PROBE(SkipGenerateTraffic, GROUPS("InterconnectSessionTCP"), \
+ TYPES(ui32, double), \
+ NAMES("peerId", "elapsedSinceRamMs")) \
+ PROBE(StartBatching, GROUPS("InterconnectSessionTCP"), \
+ TYPES(ui32, double), \
+ NAMES("peerId", "batchPeriodMs")) \
+ PROBE(FinishBatching, GROUPS("InterconnectSessionTCP"), \
+ TYPES(ui32, double), \
+ NAMES("peerId", "finishBatchDeliveryMs")) \
+ PROBE(BlockedWrite, GROUPS("InterconnectSessionTCP"), \
+ TYPES(ui32, double, ui64), \
+ NAMES("peerId", "sendQueueSize", "writtenBytes")) \
+ PROBE(ReadyWrite, GROUPS("InterconnectSessionTCP"), \
+ TYPES(ui32, double, double), \
+ NAMES("peerId", "readyWriteDeliveryMs", "blockMs")) \
+ PROBE(EpollStartWaitIn, GROUPS("EpollThread"), \
+ TYPES(), \
+ NAMES()) \
+ PROBE(EpollFinishWaitIn, GROUPS("EpollThread"), \
+ TYPES(i32), \
+ NAMES("eventsCount")) \
+ PROBE(EpollWaitOut, GROUPS("EpollThread"), \
+ TYPES(i32), \
+ NAMES("eventsCount")) \
+ PROBE(EpollSendReadyRead, GROUPS("EpollThread"), \
+ TYPES(bool, bool, int), \
+ NAMES("hangup", "event", "fd")) \
+ PROBE(EpollSendReadyWrite, GROUPS("EpollThread"), \
+ TYPES(bool, bool, int), \
+ NAMES("hangup", "event", "fd")) \
+ PROBE(HardPreemption, GROUPS("UnitedWorker"), \
+ TYPES(ui32, ui32, ui32, ui32), \
+ NAMES("cpu", "prevPoolId", "prevWorkerId", "nextWorkerId")) \
+ PROBE(SetPreemptionTimer, GROUPS("UnitedWorker", "PreemptionTimer"), \
+ TYPES(ui32, ui32, int, double, double), \
+ NAMES("cpu", "workerId", "fd", "nowMs", "preemptMs")) \
+ PROBE(ResetPreemptionTimer, GROUPS("UnitedWorker", "PreemptionTimer"), \
+ TYPES(ui32, ui32, int, double, double), \
+ NAMES("cpu", "workerId", "fd", "nowMs", "preemptMs")) \
+ PROBE(SlowWorkerActionRace, GROUPS("UnitedWorker"), \
+ TYPES(ui32, ui32, ui64), \
+ NAMES("cpu", "poolId", "slowPoolsMask")) \
+ PROBE(PoolStats, GROUPS("PoolCpuBalancer"), \
+ TYPES(ui32, TString, ui64, ui8, ui8, double, double, double, ui64, ui64, ui64), \
+ NAMES("poolId", "pool", "currentCpus", "loadClass", "priority", "scaleFactor", "cpuIdle", "cpuLoad", "importance", "addImportance", "subImportance")) \
+ PROBE(MoveCpu, GROUPS("PoolCpuBalancer"), \
+ TYPES(ui32, ui64, TString, TString, ui32), \
+ NAMES("fromPoolId", "toPoolId", "fromPool", "toPool", "cpu")) \
+ /**/
+
+LWTRACE_DECLARE_PROVIDER(ACTORLIB_PROVIDER)
+
+namespace NActors {
+ struct TActorSystemSetup;
+ TVector<NLWTrace::TDashboard> LWTraceDashboards(TActorSystemSetup* setup);
+}
diff --git a/library/cpp/actors/core/process_stats.cpp b/library/cpp/actors/core/process_stats.cpp
new file mode 100644
index 0000000000..0e1dbd0031
--- /dev/null
+++ b/library/cpp/actors/core/process_stats.cpp
@@ -0,0 +1,303 @@
+#include "actorsystem.h"
+#include "actor_bootstrapped.h"
+#include "hfunc.h"
+#include "process_stats.h"
+
+#include <library/cpp/monlib/dynamic_counters/counters.h>
+#include <library/cpp/monlib/metrics/metric_registry.h>
+
+#include <util/datetime/uptime.h>
+#include <util/system/defaults.h>
+#include <util/stream/file.h>
+#include <util/string/vector.h>
+#include <util/string/split.h>
+
+#ifndef _win_
+#include <sys/user.h>
+#include <sys/sysctl.h>
+#endif
+
+namespace NActors {
+#ifdef _linux_
+
+ namespace {
+ template <typename TVal>
+ static bool ExtractVal(const TString& str, const TString& name, TVal& res) {
+ if (!str.StartsWith(name))
+ return false;
+ size_t pos = name.size();
+ while (pos < str.size() && (str[pos] == ' ' || str[pos] == '\t')) {
+ pos++;
+ }
+ res = atol(str.data() + pos);
+ return true;
+ }
+
+ float TicksPerMillisec() {
+#ifdef _SC_CLK_TCK
+ return sysconf(_SC_CLK_TCK) / 1000.0;
+#else
+ return 1.f;
+#endif
+ }
+ }
+
+ bool TProcStat::Fill(pid_t pid) {
+ try {
+ TString strPid(ToString(pid));
+ TFileInput proc("/proc/" + strPid + "/status");
+ TString str;
+ while (proc.ReadLine(str)) {
+ if (ExtractVal(str, "VmRSS:", Rss))
+ continue;
+ if (ExtractVal(str, "voluntary_ctxt_switches:", VolCtxSwtch))
+ continue;
+ if (ExtractVal(str, "nonvoluntary_ctxt_switches:", NonvolCtxSwtch))
+ continue;
+ }
+ // Convert from kB to bytes
+ Rss *= 1024;
+
+ float tickPerMillisec = TicksPerMillisec();
+
+ TFileInput procStat("/proc/" + strPid + "/stat");
+ procStat.ReadLine(str);
+ if (!str.empty()) {
+ sscanf(str.data(),
+ "%d %*s %c %d %d %d %d %d %u %lu %lu "
+ "%lu %lu %lu %lu %ld %ld %ld %ld %ld "
+ "%ld %llu %lu %ld %lu",
+ &Pid, &State, &Ppid, &Pgrp, &Session, &TtyNr, &TPgid, &Flags, &MinFlt, &CMinFlt,
+ &MajFlt, &CMajFlt, &Utime, &Stime, &CUtime, &CStime, &Priority, &Nice, &NumThreads,
+ &ItRealValue, &StartTime, &Vsize, &RssPages, &RssLim);
+ Utime /= tickPerMillisec;
+ Stime /= tickPerMillisec;
+ CUtime /= tickPerMillisec;
+ CStime /= tickPerMillisec;
+ SystemUptime = ::Uptime();
+ Uptime = SystemUptime - TDuration::MilliSeconds(StartTime / TicksPerMillisec());
+ }
+
+ TFileInput statm("/proc/" + strPid + "/statm");
+ statm.ReadLine(str);
+ TVector<TString> fields;
+ StringSplitter(str).Split(' ').SkipEmpty().Collect(&fields);
+ if (fields.size() >= 7) {
+ ui64 resident = FromString<ui64>(fields[1]);
+ ui64 shared = FromString<ui64>(fields[2]);
+ if (PageSize == 0) {
+ PageSize = ObtainPageSize();
+ }
+ FileRss = shared * PageSize;
+ AnonRss = (resident - shared) * PageSize;
+ }
+
+ TFileInput cgroup("/proc/" + strPid + "/cgroup");
+ TString line;
+ TString memoryCGroup;
+ while (cgroup.ReadLine(line) > 0) {
+ StringSplitter(line).Split(':').Collect(&fields);
+ if (fields.size() > 2 && fields[1] == "memory") {
+ memoryCGroup = fields[2];
+ break;
+ }
+ }
+ if (!memoryCGroup.empty()) {
+ TFileInput limit("/sys/fs/cgroup/memory" + memoryCGroup + "/memory.limit_in_bytes");
+ if (limit.ReadLine(line) > 0) {
+ CGroupMemLim = FromString<ui64>(line);
+ if (CGroupMemLim > (1ULL << 40)) {
+ CGroupMemLim = 0;
+ }
+ }
+ }
+
+ } catch (...) {
+ return false;
+ }
+ return true;
+ }
+
+ long TProcStat::ObtainPageSize() {
+ long sz = sysconf(_SC_PAGESIZE);
+ return sz;
+ }
+
+#else
+
+ bool TProcStat::Fill(pid_t pid) {
+ Y_UNUSED(pid);
+ return false;
+ }
+
+ long TProcStat::ObtainPageSize() {
+ return 0;
+ }
+
+#endif
+
+namespace {
+ // Periodically collects process stats and exposes them as mon counters
+ template <typename TDerived>
+ class TProcStatCollectingActor: public TActorBootstrapped<TProcStatCollectingActor<TDerived>> {
+ public:
+ static constexpr IActor::EActivityType ActorActivityType() {
+ return IActor::ACTORLIB_STATS;
+ }
+
+ TProcStatCollectingActor(TDuration interval)
+ : Interval(interval)
+ {
+ }
+
+ void Bootstrap(const TActorContext& ctx) {
+ ctx.Schedule(Interval, new TEvents::TEvWakeup());
+ Self()->Become(&TDerived::StateWork);
+ }
+
+ STFUNC(StateWork) {
+ switch (ev->GetTypeRewrite()) {
+ CFunc(TEvents::TSystem::Wakeup, Wakeup);
+ }
+ }
+
+ private:
+ void Wakeup(const TActorContext& ctx) {
+ Self()->UpdateCounters(ProcStat);
+ ctx.Schedule(Interval, new TEvents::TEvWakeup());
+ }
+
+ TDerived* Self() {
+ ProcStat.Fill(getpid());
+ return static_cast<TDerived*>(this);
+ }
+
+ private:
+ const TDuration Interval;
+ TProcStat ProcStat;
+ };
+
+ // Periodically collects process stats and exposes them as mon counters
+ class TDynamicCounterCollector: public TProcStatCollectingActor<TDynamicCounterCollector> {
+ using TBase = TProcStatCollectingActor<TDynamicCounterCollector>;
+ public:
+ TDynamicCounterCollector(
+ ui32 intervalSeconds,
+ NMonitoring::TDynamicCounterPtr counters)
+ : TBase{TDuration::Seconds(intervalSeconds)}
+ {
+ ProcStatGroup = counters->GetSubgroup("counters", "utils");
+
+ VmSize = ProcStatGroup->GetCounter("Process/VmSize", false);
+ AnonRssSize = ProcStatGroup->GetCounter("Process/AnonRssSize", false);
+ FileRssSize = ProcStatGroup->GetCounter("Process/FileRssSize", false);
+ CGroupMemLimit = ProcStatGroup->GetCounter("Process/CGroupMemLimit", false);
+ UserTime = ProcStatGroup->GetCounter("Process/UserTime", true);
+ SysTime = ProcStatGroup->GetCounter("Process/SystemTime", true);
+ MinorPageFaults = ProcStatGroup->GetCounter("Process/MinorPageFaults", true);
+ MajorPageFaults = ProcStatGroup->GetCounter("Process/MajorPageFaults", true);
+ UptimeSeconds = ProcStatGroup->GetCounter("Process/UptimeSeconds", false);
+ NumThreads = ProcStatGroup->GetCounter("Process/NumThreads", false);
+ SystemUptimeSeconds = ProcStatGroup->GetCounter("System/UptimeSeconds", false);
+ }
+
+ void UpdateCounters(const TProcStat& procStat) {
+ *VmSize = procStat.Vsize;
+ *AnonRssSize = procStat.AnonRss;
+ *FileRssSize = procStat.FileRss;
+ if (procStat.CGroupMemLim) {
+ *CGroupMemLimit = procStat.CGroupMemLim;
+ }
+ *UserTime = procStat.Utime;
+ *SysTime = procStat.Stime;
+ *MinorPageFaults = procStat.MinFlt;
+ *MajorPageFaults = procStat.MajFlt;
+ *UptimeSeconds = procStat.Uptime.Seconds();
+ *NumThreads = procStat.NumThreads;
+ *SystemUptimeSeconds = procStat.Uptime.Seconds();
+ }
+
+ private:
+ NMonitoring::TDynamicCounterPtr ProcStatGroup;
+ NMonitoring::TDynamicCounters::TCounterPtr VmSize;
+ NMonitoring::TDynamicCounters::TCounterPtr AnonRssSize;
+ NMonitoring::TDynamicCounters::TCounterPtr FileRssSize;
+ NMonitoring::TDynamicCounters::TCounterPtr CGroupMemLimit;
+ NMonitoring::TDynamicCounters::TCounterPtr UserTime;
+ NMonitoring::TDynamicCounters::TCounterPtr SysTime;
+ NMonitoring::TDynamicCounters::TCounterPtr MinorPageFaults;
+ NMonitoring::TDynamicCounters::TCounterPtr MajorPageFaults;
+ NMonitoring::TDynamicCounters::TCounterPtr UptimeSeconds;
+ NMonitoring::TDynamicCounters::TCounterPtr NumThreads;
+ NMonitoring::TDynamicCounters::TCounterPtr SystemUptimeSeconds;
+ };
+
+
+ class TRegistryCollector: public TProcStatCollectingActor<TRegistryCollector> {
+ using TBase = TProcStatCollectingActor<TRegistryCollector>;
+ public:
+ TRegistryCollector(TDuration interval, NMonitoring::TMetricRegistry& registry)
+ : TBase{interval}
+ {
+ VmSize = registry.IntGauge({{"sensor", "process.VmSize"}});
+ AnonRssSize = registry.IntGauge({{"sensor", "process.AnonRssSize"}});
+ FileRssSize = registry.IntGauge({{"sensor", "process.FileRssSize"}});
+ CGroupMemLimit = registry.IntGauge({{"sensor", "process.CGroupMemLimit"}});
+ UptimeSeconds = registry.IntGauge({{"sensor", "process.UptimeSeconds"}});
+ NumThreads = registry.IntGauge({{"sensor", "process.NumThreads"}});
+ SystemUptimeSeconds = registry.IntGauge({{"sensor", "system.UptimeSeconds"}});
+
+ UserTime = registry.Rate({{"sensor", "process.UserTime"}});
+ SysTime = registry.Rate({{"sensor", "process.SystemTime"}});
+ MinorPageFaults = registry.Rate({{"sensor", "process.MinorPageFaults"}});
+ MajorPageFaults = registry.Rate({{"sensor", "process.MajorPageFaults"}});
+ }
+
+ void UpdateCounters(const TProcStat& procStat) {
+ VmSize->Set(procStat.Vsize);
+ AnonRssSize->Set(procStat.AnonRss);
+ FileRssSize->Set(procStat.FileRss);
+ CGroupMemLimit->Set(procStat.CGroupMemLim);
+ UptimeSeconds->Set(procStat.Uptime.Seconds());
+ NumThreads->Set(procStat.NumThreads);
+ SystemUptimeSeconds->Set(procStat.SystemUptime.Seconds());
+
+ // it is ok here to reset and add metric value, because mutation
+ // is performed in siglethreaded context
+
+ UserTime->Reset();
+ UserTime->Add(procStat.Utime);
+
+ SysTime->Reset();
+ SysTime->Add(procStat.Stime);
+
+ MinorPageFaults->Reset();
+ MinorPageFaults->Add(procStat.MinFlt);
+
+ MajorPageFaults->Reset();
+ MajorPageFaults->Add(procStat.MajFlt);
+ }
+
+ private:
+ NMonitoring::TIntGauge* VmSize;
+ NMonitoring::TIntGauge* AnonRssSize;
+ NMonitoring::TIntGauge* FileRssSize;
+ NMonitoring::TIntGauge* CGroupMemLimit;
+ NMonitoring::TRate* UserTime;
+ NMonitoring::TRate* SysTime;
+ NMonitoring::TRate* MinorPageFaults;
+ NMonitoring::TRate* MajorPageFaults;
+ NMonitoring::TIntGauge* UptimeSeconds;
+ NMonitoring::TIntGauge* NumThreads;
+ NMonitoring::TIntGauge* SystemUptimeSeconds;
+ };
+} // namespace
+
+ IActor* CreateProcStatCollector(ui32 intervalSec, NMonitoring::TDynamicCounterPtr counters) {
+ return new TDynamicCounterCollector(intervalSec, counters);
+ }
+
+ IActor* CreateProcStatCollector(TDuration interval, NMonitoring::TMetricRegistry& registry) {
+ return new TRegistryCollector(interval, registry);
+ }
+}
diff --git a/library/cpp/actors/core/process_stats.h b/library/cpp/actors/core/process_stats.h
new file mode 100644
index 0000000000..66346d0b5a
--- /dev/null
+++ b/library/cpp/actors/core/process_stats.h
@@ -0,0 +1,66 @@
+#pragma once
+
+#include "defs.h"
+#include "actor.h"
+
+#include <library/cpp/monlib/dynamic_counters/counters.h>
+
+namespace NMonitoring {
+ class TMetricRegistry;
+}
+
+namespace NActors {
+ struct TProcStat {
+ ui64 Rss;
+ ui64 VolCtxSwtch;
+ ui64 NonvolCtxSwtch;
+
+ int Pid;
+ char State;
+ int Ppid;
+ int Pgrp;
+ int Session;
+ int TtyNr;
+ int TPgid;
+ unsigned Flags;
+ unsigned long MinFlt;
+ unsigned long CMinFlt;
+ unsigned long MajFlt;
+ unsigned long CMajFlt;
+ unsigned long Utime;
+ unsigned long Stime;
+ long CUtime;
+ long CStime;
+ long Priority;
+ long Nice;
+ long NumThreads;
+ long ItRealValue;
+ // StartTime is measured from system boot
+ unsigned long long StartTime;
+ unsigned long Vsize;
+ long RssPages;
+ unsigned long RssLim;
+ ui64 FileRss;
+ ui64 AnonRss;
+ ui64 CGroupMemLim = 0;
+
+ TDuration Uptime;
+ TDuration SystemUptime;
+ // ...
+
+ TProcStat() {
+ Zero(*this);
+ Y_UNUSED(PageSize);
+ }
+
+ bool Fill(pid_t pid);
+
+ private:
+ long PageSize = 0;
+
+ long ObtainPageSize();
+ };
+
+ IActor* CreateProcStatCollector(ui32 intervalSec, NMonitoring::TDynamicCounterPtr counters);
+ IActor* CreateProcStatCollector(TDuration interval, NMonitoring::TMetricRegistry& registry);
+}
diff --git a/library/cpp/actors/core/scheduler_actor.cpp b/library/cpp/actors/core/scheduler_actor.cpp
new file mode 100644
index 0000000000..febc5e40dd
--- /dev/null
+++ b/library/cpp/actors/core/scheduler_actor.cpp
@@ -0,0 +1,279 @@
+#include "actor_bootstrapped.h"
+#include "hfunc.h"
+#include "probes.h"
+#include "scheduler_actor.h"
+#include "scheduler_queue.h"
+
+#include <library/cpp/actors/interconnect/poller_actor.h>
+#include <util/system/hp_timer.h>
+
+#ifdef __linux__
+#include <sys/timerfd.h>
+#include <errno.h>
+
+LWTRACE_USING(ACTORLIB_PROVIDER);
+
+namespace NActors {
+ class TTimerDescriptor: public TSharedDescriptor {
+ const int Descriptor;
+
+ public:
+ TTimerDescriptor()
+ : Descriptor(timerfd_create(CLOCK_MONOTONIC, TFD_NONBLOCK))
+ {
+ Y_VERIFY(Descriptor != -1, "timerfd_create() failed with %s", strerror(errno));
+ }
+
+ ~TTimerDescriptor() override {
+ close(Descriptor);
+ }
+
+ int GetDescriptor() override {
+ return Descriptor;
+ }
+ };
+
+ class TSchedulerActor: public TActor<TSchedulerActor> {
+ const TSchedulerConfig Cfg;
+ TIntrusivePtr<TSharedDescriptor> TimerDescriptor;
+
+ TVector<NSchedulerQueue::TReader*> Readers;
+
+ TActorId PollerActor;
+ TPollerToken::TPtr PollerToken;
+
+ ui64 RealTime;
+ ui64 MonotonicTime;
+
+ ui64 ActiveTick;
+ typedef TMap<ui64, TAutoPtr<NSchedulerQueue::TQueueType>> TMomentMap; // intrasecond queues
+ typedef THashMap<ui64, TAutoPtr<TMomentMap>> TScheduleMap; // over-second schedule
+
+ TScheduleMap ScheduleMap;
+
+ THolder<NThreading::TLegacyFuture<void, false>> MainCycle;
+
+ static const ui64 IntrasecondThreshold = 1048576; // ~second
+ TAutoPtr<TMomentMap> ActiveSec;
+ volatile ui64* CurrentTimestamp = nullptr;
+ volatile ui64* CurrentMonotonic = nullptr;
+ TDeque<TAutoPtr<IEventHandle>> EventsToBeSent;
+
+ public:
+ static constexpr IActor::EActivityType ActorActivityType() {
+ return IActor::ACTOR_SYSTEM_SCHEDULER_ACTOR;
+ }
+
+ TSchedulerActor(const TSchedulerConfig& cfg)
+ : TActor(&TSchedulerActor::StateFunc)
+ , Cfg(cfg)
+ , TimerDescriptor(new TTimerDescriptor())
+ , PollerActor(MakePollerActorId())
+ {
+ Y_ASSERT(Cfg.ResolutionMicroseconds != 0);
+ Y_ASSERT(Cfg.ProgressThreshold != 0);
+ Become(&TSchedulerActor::StateFunc);
+ }
+
+ void Handle(TEvSchedulerInitialize::TPtr& ev, const TActorContext& ctx) {
+ const TEvSchedulerInitialize& evInitialize = *ev->Get();
+ Y_ASSERT(evInitialize.ScheduleReaders.size() != 0);
+ Readers.resize(evInitialize.ScheduleReaders.size());
+ Copy(evInitialize.ScheduleReaders.begin(), evInitialize.ScheduleReaders.end(), Readers.begin());
+
+ Y_ASSERT(evInitialize.CurrentTimestamp != nullptr);
+ CurrentTimestamp = evInitialize.CurrentTimestamp;
+
+ Y_ASSERT(evInitialize.CurrentMonotonic != nullptr);
+ CurrentMonotonic = evInitialize.CurrentMonotonic;
+
+ struct itimerspec new_time;
+ memset(&new_time, 0, sizeof(new_time));
+ 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));
+ const bool success = ctx.Send(PollerActor, new TEvPollerRegister(TimerDescriptor, SelfId(), {}));
+ Y_VERIFY(success);
+
+ RealTime = RelaxedLoad(CurrentTimestamp);
+ MonotonicTime = RelaxedLoad(CurrentMonotonic);
+
+ ActiveTick = AlignUp<ui64>(MonotonicTime, IntrasecondThreshold);
+ }
+
+ void Handle(TEvPollerRegisterResult::TPtr ev, const TActorContext& ctx) {
+ PollerToken = ev->Get()->PollerToken;
+ HandleSchedule(ctx);
+ }
+
+ void UpdateTime() {
+ RealTime = TInstant::Now().MicroSeconds();
+ MonotonicTime = Max(MonotonicTime, GetMonotonicMicroSeconds());
+ AtomicStore(CurrentTimestamp, RealTime);
+ AtomicStore(CurrentMonotonic, MonotonicTime);
+ }
+
+ void TryUpdateTime(NHPTimer::STime* lastTimeUpdate) {
+ NHPTimer::STime hpnow;
+ GetTimeFast(&hpnow);
+ const ui64 elapsedCycles = hpnow > *lastTimeUpdate ? hpnow - *lastTimeUpdate : 0;
+ if (elapsedCycles > Cfg.ResolutionMicroseconds * (NHPTimer::GetCyclesPerSecond() / IntrasecondThreshold)) {
+ UpdateTime();
+ GetTimeFast(lastTimeUpdate);
+ }
+ }
+
+ void HandleSchedule(const TActorContext& ctx) {
+ for (;;) {
+ NHPTimer::STime schedulingStart;
+ GetTimeFast(&schedulingStart);
+ NHPTimer::STime lastTimeUpdate = schedulingStart;
+
+ ui64 expired;
+ ssize_t bytesRead;
+ bytesRead = read(TimerDescriptor->GetDescriptor(), &expired, sizeof(expired));
+ if (bytesRead == -1) {
+ if (errno == EAGAIN) {
+ PollerToken->Request(true, false);
+ break;
+ } else if (errno == EINTR) {
+ continue;
+ }
+ }
+ Y_VERIFY(bytesRead == sizeof(expired), "Error while reading from timerfd, strerror# %s", strerror(errno));
+ UpdateTime();
+
+ ui32 eventsGottenFromQueues = 0;
+ // collect everything from queues
+ for (ui32 i = 0; i != Readers.size(); ++i) {
+ while (NSchedulerQueue::TEntry* x = Readers[i]->Pop()) {
+ const ui64 instant = AlignUp<ui64>(x->InstantMicroseconds, Cfg.ResolutionMicroseconds);
+ IEventHandle* const ev = x->Ev;
+ ISchedulerCookie* const cookie = x->Cookie;
+
+ // check is cookie still valid? looks like it will hurt performance w/o sagnificant memory save
+
+ if (instant <= ActiveTick) {
+ if (!ActiveSec)
+ ActiveSec.Reset(new TMomentMap());
+ TAutoPtr<NSchedulerQueue::TQueueType>& queue = (*ActiveSec)[instant];
+ if (!queue)
+ queue.Reset(new NSchedulerQueue::TQueueType());
+ queue->Writer.Push(instant, ev, cookie);
+ } else {
+ const ui64 intrasecond = AlignUp<ui64>(instant, IntrasecondThreshold);
+ TAutoPtr<TMomentMap>& msec = ScheduleMap[intrasecond];
+ if (!msec)
+ msec.Reset(new TMomentMap());
+ TAutoPtr<NSchedulerQueue::TQueueType>& queue = (*msec)[instant];
+ if (!queue)
+ queue.Reset(new NSchedulerQueue::TQueueType());
+ queue->Writer.Push(instant, ev, cookie);
+ }
+ ++eventsGottenFromQueues;
+ TryUpdateTime(&lastTimeUpdate);
+ }
+ }
+
+ ui64 eventSchedulingErrorUs = 0;
+ // send everything triggered on schedule
+ for (;;) {
+ while (!!ActiveSec && !ActiveSec->empty()) {
+ TMomentMap::iterator it = ActiveSec->begin();
+ if (it->first <= MonotonicTime) {
+ if (NSchedulerQueue::TQueueType* q = it->second.Get()) {
+ while (NSchedulerQueue::TEntry* x = q->Reader.Pop()) {
+ Y_VERIFY_DEBUG(x->InstantMicroseconds <= ActiveTick);
+ if (eventSchedulingErrorUs == 0 && MonotonicTime > x->InstantMicroseconds) {
+ eventSchedulingErrorUs = MonotonicTime - x->InstantMicroseconds;
+ }
+ IEventHandle* ev = x->Ev;
+ ISchedulerCookie* cookie = x->Cookie;
+ if (cookie) {
+ if (cookie->Detach()) {
+ EventsToBeSent.push_back(ev);
+ } else {
+ delete ev;
+ }
+ } else {
+ EventsToBeSent.push_back(ev);
+ }
+ TryUpdateTime(&lastTimeUpdate);
+ }
+ }
+ ActiveSec->erase(it);
+ } else {
+ break;
+ }
+ }
+
+ if (ActiveTick <= MonotonicTime) {
+ Y_VERIFY_DEBUG(!ActiveSec || ActiveSec->empty());
+ ActiveSec.Destroy();
+ ActiveTick += IntrasecondThreshold;
+ TScheduleMap::iterator it = ScheduleMap.find(ActiveTick);
+ if (it != ScheduleMap.end()) {
+ ActiveSec = it->second;
+ ScheduleMap.erase(it);
+ }
+ continue;
+ }
+
+ // ok, if we are here - then nothing is ready, so send step complete
+ break;
+ }
+
+ // Send all from buffer queue
+ const ui64 eventsToBeSentSize = EventsToBeSent.size();
+ ui32 sentCount = 0;
+ if (eventsToBeSentSize > Cfg.RelaxedSendThresholdEventsPerCycle) {
+ sentCount = Cfg.RelaxedSendPaceEventsPerCycle +
+ (eventsToBeSentSize - Cfg.RelaxedSendThresholdEventsPerCycle) / 2;
+ } else {
+ sentCount = Min(eventsToBeSentSize, Cfg.RelaxedSendPaceEventsPerCycle);
+ }
+ for (ui32 i = 0; i < sentCount; ++i) {
+ ctx.Send(EventsToBeSent.front());
+ EventsToBeSent.pop_front();
+ }
+
+ NHPTimer::STime hpnow;
+ GetTimeFast(&hpnow);
+ const ui64 processingTime = hpnow > schedulingStart ? hpnow - schedulingStart : 0;
+ const ui64 elapsedTimeMicroseconds = processingTime / (NHPTimer::GetCyclesPerSecond() / IntrasecondThreshold);
+ LWPROBE(ActorsystemScheduler, elapsedTimeMicroseconds, expired, eventsGottenFromQueues, sentCount,
+ eventsToBeSentSize, eventSchedulingErrorUs);
+ TryUpdateTime(&lastTimeUpdate);
+ }
+ }
+
+ STRICT_STFUNC(StateFunc,
+ HFunc(TEvSchedulerInitialize, Handle)
+ CFunc(TEvPollerReady::EventType, HandleSchedule)
+ CFunc(TEvents::TSystem::PoisonPill, Die)
+ HFunc(TEvPollerRegisterResult, Handle)
+ )
+ };
+
+ IActor* CreateSchedulerActor(const TSchedulerConfig& cfg) {
+ if (cfg.UseSchedulerActor) {
+ return new TSchedulerActor(cfg);
+ } else {
+ return nullptr;
+ }
+ }
+
+}
+
+#else // linux
+
+namespace NActors {
+ IActor* CreateSchedulerActor(const TSchedulerConfig& cfg) {
+ Y_UNUSED(cfg);
+ return nullptr;
+ }
+
+}
+
+#endif // linux
diff --git a/library/cpp/actors/core/scheduler_actor.h b/library/cpp/actors/core/scheduler_actor.h
new file mode 100644
index 0000000000..c2c561b43d
--- /dev/null
+++ b/library/cpp/actors/core/scheduler_actor.h
@@ -0,0 +1,29 @@
+#pragma once
+
+#include "actor.h"
+#include "event_local.h"
+#include "events.h"
+#include "scheduler_basic.h"
+
+namespace NActors {
+ struct TEvSchedulerInitialize : TEventLocal<TEvSchedulerInitialize, TEvents::TSystem::Bootstrap> {
+ TVector<NSchedulerQueue::TReader*> ScheduleReaders;
+ volatile ui64* CurrentTimestamp;
+ volatile ui64* CurrentMonotonic;
+
+ TEvSchedulerInitialize(const TVector<NSchedulerQueue::TReader*>& scheduleReaders, volatile ui64* currentTimestamp, volatile ui64* currentMonotonic)
+ : ScheduleReaders(scheduleReaders)
+ , CurrentTimestamp(currentTimestamp)
+ , CurrentMonotonic(currentMonotonic)
+ {
+ }
+ };
+
+ IActor* CreateSchedulerActor(const TSchedulerConfig& cfg);
+
+ inline TActorId MakeSchedulerActorId() {
+ char x[12] = {'s', 'c', 'h', 'e', 'd', 'u', 'l', 'e', 'r', 's', 'e', 'r'};
+ return TActorId(0, TStringBuf(x, 12));
+ }
+
+}
diff --git a/library/cpp/actors/core/scheduler_actor_ut.cpp b/library/cpp/actors/core/scheduler_actor_ut.cpp
new file mode 100644
index 0000000000..09b7369d36
--- /dev/null
+++ b/library/cpp/actors/core/scheduler_actor_ut.cpp
@@ -0,0 +1,100 @@
+#include "actor_coroutine.h"
+#include "actorsystem.h"
+#include "executor_pool_basic.h"
+#include "scheduler_actor.h"
+#include "scheduler_basic.h"
+#include "events.h"
+#include "event_local.h"
+#include "hfunc.h"
+#include <library/cpp/actors/interconnect/poller_actor.h>
+#include <library/cpp/testing/unittest/registar.h>
+
+#include <util/system/sanitizers.h>
+
+using namespace NActors;
+
+Y_UNIT_TEST_SUITE(SchedulerActor) {
+ class TTestActor: public TActorBootstrapped<TTestActor> {
+ TManualEvent& DoneEvent;
+ TAtomic& EventsProcessed;
+ TInstant LastWakeup;
+ const TAtomicBase EventsTotalCount;
+ const TDuration ScheduleDelta;
+
+ public:
+ TTestActor(TManualEvent& doneEvent, TAtomic& eventsProcessed, TAtomicBase eventsTotalCount, ui32 scheduleDeltaMs)
+ : DoneEvent(doneEvent)
+ , EventsProcessed(eventsProcessed)
+ , EventsTotalCount(eventsTotalCount)
+ , ScheduleDelta(TDuration::MilliSeconds(scheduleDeltaMs))
+ {
+ }
+
+ void Bootstrap(const TActorContext& ctx) {
+ LastWakeup = ctx.Now();
+ Become(&TThis::StateFunc);
+ ctx.Schedule(ScheduleDelta, new TEvents::TEvWakeup());
+ }
+
+ void Handle(TEvents::TEvWakeup::TPtr& /*ev*/, const TActorContext& ctx) {
+ const TInstant now = ctx.Now();
+ UNIT_ASSERT(now - LastWakeup >= ScheduleDelta);
+ LastWakeup = now;
+
+ if (AtomicIncrement(EventsProcessed) == EventsTotalCount) {
+ DoneEvent.Signal();
+ } else {
+ ctx.Schedule(ScheduleDelta, new TEvents::TEvWakeup());
+ }
+ }
+
+ STRICT_STFUNC(StateFunc, {HFunc(TEvents::TEvWakeup, Handle)})
+ };
+
+ void Test(TAtomicBase eventsTotalCount, ui32 scheduleDeltaMs) {
+ THolder<TActorSystemSetup> setup = MakeHolder<TActorSystemSetup>();
+ setup->NodeId = 0;
+ setup->ExecutorsCount = 1;
+ setup->Executors.Reset(new TAutoPtr<IExecutorPool>[setup->ExecutorsCount]);
+ for (ui32 i = 0; i < setup->ExecutorsCount; ++i) {
+ setup->Executors[i] = new TBasicExecutorPool(i, 5, 10, "basic");
+ }
+ // create poller actor (whether platform supports it)
+ TActorId pollerActorId;
+ if (IActor* poller = CreatePollerActor()) {
+ pollerActorId = MakePollerActorId();
+ setup->LocalServices.emplace_back(pollerActorId, TActorSetupCmd(poller, TMailboxType::ReadAsFilled, 0));
+ }
+ TActorId schedulerActorId;
+ if (IActor* schedulerActor = CreateSchedulerActor(TSchedulerConfig())) {
+ schedulerActorId = MakeSchedulerActorId();
+ setup->LocalServices.emplace_back(schedulerActorId, TActorSetupCmd(schedulerActor, TMailboxType::ReadAsFilled, 0));
+ }
+ setup->Scheduler = CreateSchedulerThread(TSchedulerConfig());
+
+ TActorSystem actorSystem(setup);
+
+ actorSystem.Start();
+
+ TManualEvent doneEvent;
+ TAtomic eventsProcessed = 0;
+ actorSystem.Register(new TTestActor(doneEvent, eventsProcessed, eventsTotalCount, scheduleDeltaMs));
+ doneEvent.WaitI();
+
+ UNIT_ASSERT(AtomicGet(eventsProcessed) == eventsTotalCount);
+
+ actorSystem.Stop();
+ }
+
+ Y_UNIT_TEST(LongEvents) {
+ Test(10, 500);
+ }
+
+ Y_UNIT_TEST(MediumEvents) {
+ Test(100, 50);
+ }
+
+ Y_UNIT_TEST(QuickEvents) {
+ Test(1000, 5);
+ }
+}
diff --git a/library/cpp/actors/core/scheduler_basic.cpp b/library/cpp/actors/core/scheduler_basic.cpp
new file mode 100644
index 0000000000..fba200e16b
--- /dev/null
+++ b/library/cpp/actors/core/scheduler_basic.cpp
@@ -0,0 +1,274 @@
+#include "scheduler_basic.h"
+#include "scheduler_queue.h"
+
+#include <library/cpp/actors/util/datetime.h>
+#include <library/cpp/actors/util/thread.h>
+
+#ifdef BALLOC
+#include <library/cpp/balloc/optional/operators.h>
+#endif
+
+namespace NActors {
+
+ struct TBasicSchedulerThread::TMonCounters {
+ NMonitoring::TDynamicCounters::TCounterPtr TimeDelayMs;
+ NMonitoring::TDynamicCounters::TCounterPtr QueueSize;
+ NMonitoring::TDynamicCounters::TCounterPtr EventsSent;
+ NMonitoring::TDynamicCounters::TCounterPtr EventsDropped;
+ NMonitoring::TDynamicCounters::TCounterPtr EventsAdded;
+ NMonitoring::TDynamicCounters::TCounterPtr Iterations;
+ NMonitoring::TDynamicCounters::TCounterPtr Sleeps;
+ NMonitoring::TDynamicCounters::TCounterPtr ElapsedMicrosec;
+
+ TMonCounters(const NMonitoring::TDynamicCounterPtr& counters)
+ : TimeDelayMs(counters->GetCounter("Scheduler/TimeDelayMs", false))
+ , QueueSize(counters->GetCounter("Scheduler/QueueSize", false))
+ , EventsSent(counters->GetCounter("Scheduler/EventsSent", true))
+ , EventsDropped(counters->GetCounter("Scheduler/EventsDropped", true))
+ , EventsAdded(counters->GetCounter("Scheduler/EventsAdded", true))
+ , Iterations(counters->GetCounter("Scheduler/Iterations", true))
+ , Sleeps(counters->GetCounter("Scheduler/Sleeps", true))
+ , ElapsedMicrosec(counters->GetCounter("Scheduler/ElapsedMicrosec", true))
+ { }
+ };
+
+ TBasicSchedulerThread::TBasicSchedulerThread(const TSchedulerConfig& config)
+ : Config(config)
+ , MonCounters(Config.MonCounters ? new TMonCounters(Config.MonCounters) : nullptr)
+ , ActorSystem(nullptr)
+ , CurrentTimestamp(nullptr)
+ , CurrentMonotonic(nullptr)
+ , TotalReaders(0)
+ , StopFlag(false)
+ , ScheduleMap(3600)
+ {
+ Y_VERIFY(!Config.UseSchedulerActor, "Cannot create scheduler thread because Config.UseSchedulerActor# true");
+ }
+
+ TBasicSchedulerThread::~TBasicSchedulerThread() {
+ Y_VERIFY(!MainCycle);
+ }
+
+ void TBasicSchedulerThread::CycleFunc() {
+#ifdef BALLOC
+ ThreadDisableBalloc();
+#endif
+ ::SetCurrentThreadName("Scheduler");
+
+ ui64 currentMonotonic = RelaxedLoad(CurrentMonotonic);
+ ui64 throttledMonotonic = currentMonotonic;
+
+ ui64 activeTick = AlignUp<ui64>(throttledMonotonic, IntrasecondThreshold);
+ TAutoPtr<TMomentMap> activeSec;
+
+ NHPTimer::STime hpprev = GetCycleCountFast();
+ ui64 nextTimestamp = TInstant::Now().MicroSeconds();
+ ui64 nextMonotonic = Max(currentMonotonic, GetMonotonicMicroSeconds());
+
+ while (!AtomicLoad(&StopFlag)) {
+ {
+ const ui64 delta = nextMonotonic - throttledMonotonic;
+ const ui64 elapsedDelta = nextMonotonic - currentMonotonic;
+ const ui64 threshold = Max(Min(Config.ProgressThreshold, 2 * elapsedDelta), ui64(1));
+
+ throttledMonotonic = (delta > threshold) ? throttledMonotonic + threshold : nextMonotonic;
+
+ if (MonCounters) {
+ *MonCounters->TimeDelayMs = (nextMonotonic - throttledMonotonic) / 1000;
+ }
+ }
+ AtomicStore(CurrentTimestamp, nextTimestamp);
+ AtomicStore(CurrentMonotonic, nextMonotonic);
+ currentMonotonic = nextMonotonic;
+
+ if (MonCounters) {
+ ++*MonCounters->Iterations;
+ }
+
+ bool somethingDone = false;
+
+ // first step - send everything triggered on schedule
+ ui64 eventsSent = 0;
+ ui64 eventsDropped = 0;
+ for (;;) {
+ while (!!activeSec && !activeSec->empty()) {
+ TMomentMap::iterator it = activeSec->begin();
+ if (it->first <= throttledMonotonic) {
+ if (NSchedulerQueue::TQueueType* q = it->second.Get()) {
+ while (NSchedulerQueue::TEntry* x = q->Reader.Pop()) {
+ somethingDone = true;
+ Y_VERIFY_DEBUG(x->InstantMicroseconds <= activeTick);
+ IEventHandle* ev = x->Ev;
+ ISchedulerCookie* cookie = x->Cookie;
+ // TODO: lazy send with backoff queue to not hang over contended mailboxes
+ if (cookie) {
+ if (cookie->Detach()) {
+ ActorSystem->Send(ev);
+ ++eventsSent;
+ } else {
+ delete ev;
+ ++eventsDropped;
+ }
+ } else {
+ ActorSystem->Send(ev);
+ ++eventsSent;
+ }
+ }
+ }
+ activeSec->erase(it);
+ } else
+ break;
+ }
+
+ if (activeTick <= throttledMonotonic) {
+ Y_VERIFY_DEBUG(!activeSec || activeSec->empty());
+ activeSec.Destroy();
+ activeTick += IntrasecondThreshold;
+ TScheduleMap::iterator it = ScheduleMap.find(activeTick);
+ if (it != ScheduleMap.end()) {
+ activeSec = it->second;
+ ScheduleMap.erase(it);
+ }
+ continue;
+ }
+
+ // ok, if we are here - then nothing is ready, so send step complete
+ break;
+ }
+
+ // second step - collect everything from queues
+
+ ui64 eventsAdded = 0;
+ for (ui32 i = 0; i != TotalReaders; ++i) {
+ while (NSchedulerQueue::TEntry* x = Readers[i]->Pop()) {
+ somethingDone = true;
+ const ui64 instant = AlignUp<ui64>(x->InstantMicroseconds, Config.ResolutionMicroseconds);
+ IEventHandle* const ev = x->Ev;
+ ISchedulerCookie* const cookie = x->Cookie;
+
+ // check is cookie still valid? looks like it will hurt performance w/o sagnificant memory save
+
+ if (instant <= activeTick) {
+ if (!activeSec)
+ activeSec.Reset(new TMomentMap());
+ TAutoPtr<NSchedulerQueue::TQueueType>& queue = (*activeSec)[instant];
+ if (!queue)
+ queue.Reset(new NSchedulerQueue::TQueueType());
+ queue->Writer.Push(instant, ev, cookie);
+ } else {
+ const ui64 intrasecond = AlignUp<ui64>(instant, IntrasecondThreshold);
+ TAutoPtr<TMomentMap>& msec = ScheduleMap[intrasecond];
+ if (!msec)
+ msec.Reset(new TMomentMap());
+ TAutoPtr<NSchedulerQueue::TQueueType>& queue = (*msec)[instant];
+ if (!queue)
+ queue.Reset(new NSchedulerQueue::TQueueType());
+ queue->Writer.Push(instant, ev, cookie);
+ }
+
+ ++eventsAdded;
+ }
+ }
+
+ NHPTimer::STime hpnow = GetCycleCountFast();
+
+ if (MonCounters) {
+ *MonCounters->QueueSize -= eventsSent + eventsDropped;
+ *MonCounters->QueueSize += eventsAdded;
+ *MonCounters->EventsSent += eventsSent;
+ *MonCounters->EventsDropped += eventsDropped;
+ *MonCounters->EventsAdded += eventsAdded;
+ *MonCounters->ElapsedMicrosec += NHPTimer::GetSeconds(hpnow - hpprev) * 1000000;
+ }
+
+ hpprev = hpnow;
+ nextTimestamp = TInstant::Now().MicroSeconds();
+ nextMonotonic = Max(currentMonotonic, GetMonotonicMicroSeconds());
+
+ // ok complete, if nothing left - sleep
+ if (!somethingDone) {
+ const ui64 nextInstant = AlignDown<ui64>(throttledMonotonic + Config.ResolutionMicroseconds, Config.ResolutionMicroseconds);
+ if (nextMonotonic >= nextInstant) // already in next time-slice
+ continue;
+
+ const ui64 delta = nextInstant - nextMonotonic;
+ if (delta < Config.SpinThreshold) // not so much time left, just spin
+ continue;
+
+ if (MonCounters) {
+ ++*MonCounters->Sleeps;
+ }
+
+ NanoSleep(delta * 1000); // ok, looks like we should sleep a bit.
+
+ // Don't count sleep in elapsed microseconds
+ hpprev = GetCycleCountFast();
+ nextTimestamp = TInstant::Now().MicroSeconds();
+ nextMonotonic = Max(currentMonotonic, GetMonotonicMicroSeconds());
+ }
+ }
+ // ok, die!
+ }
+
+ void TBasicSchedulerThread::Prepare(TActorSystem* actorSystem, volatile ui64* currentTimestamp, volatile ui64* currentMonotonic) {
+ ActorSystem = actorSystem;
+ CurrentTimestamp = currentTimestamp;
+ CurrentMonotonic = currentMonotonic;
+ *CurrentTimestamp = TInstant::Now().MicroSeconds();
+ *CurrentMonotonic = GetMonotonicMicroSeconds();
+ }
+
+ void TBasicSchedulerThread::PrepareSchedules(NSchedulerQueue::TReader** readers, ui32 scheduleReadersCount) {
+ Y_VERIFY(scheduleReadersCount > 0);
+ TotalReaders = scheduleReadersCount;
+ Readers.Reset(new NSchedulerQueue::TReader*[scheduleReadersCount]);
+ Copy(readers, readers + scheduleReadersCount, Readers.Get());
+ }
+
+ void TBasicSchedulerThread::PrepareStart() {
+ // Called after actor system is initialized, but before executor threads
+ // are started, giving us a chance to update current timestamp with a
+ // more recent value, taking initialization time into account. This is
+ // safe to do, since scheduler thread is not started yet, so no other
+ // threads are updating time concurrently.
+ AtomicStore(CurrentTimestamp, TInstant::Now().MicroSeconds());
+ AtomicStore(CurrentMonotonic, Max(RelaxedLoad(CurrentMonotonic), GetMonotonicMicroSeconds()));
+ }
+
+ void TBasicSchedulerThread::Start() {
+ MainCycle.Reset(new NThreading::TLegacyFuture<void, false>(std::bind(&TBasicSchedulerThread::CycleFunc, this)));
+ }
+
+ void TBasicSchedulerThread::PrepareStop() {
+ AtomicStore(&StopFlag, true);
+ }
+
+ void TBasicSchedulerThread::Stop() {
+ MainCycle->Get();
+ MainCycle.Destroy();
+ }
+
+}
+
+#ifdef __linux__
+
+namespace NActors {
+ ISchedulerThread* CreateSchedulerThread(const TSchedulerConfig& config) {
+ if (config.UseSchedulerActor) {
+ return new TMockSchedulerThread();
+ } else {
+ return new TBasicSchedulerThread(config);
+ }
+ }
+
+}
+
+#else // __linux__
+
+namespace NActors {
+ ISchedulerThread* CreateSchedulerThread(const TSchedulerConfig& config) {
+ return new TBasicSchedulerThread(config);
+ }
+}
+
+#endif // __linux__
diff --git a/library/cpp/actors/core/scheduler_basic.h b/library/cpp/actors/core/scheduler_basic.h
new file mode 100644
index 0000000000..2ccde39235
--- /dev/null
+++ b/library/cpp/actors/core/scheduler_basic.h
@@ -0,0 +1,81 @@
+#pragma once
+
+#include "actorsystem.h"
+#include "monotonic.h"
+#include "scheduler_queue.h"
+#include <library/cpp/actors/util/queue_chunk.h>
+#include <library/cpp/threading/future/legacy_future.h>
+#include <util/generic/hash.h>
+#include <util/generic/map.h>
+
+namespace NActors {
+
+ class TBasicSchedulerThread: public ISchedulerThread {
+ // TODO: replace with NUMA-local threads and per-thread schedules
+ const TSchedulerConfig Config;
+
+ struct TMonCounters;
+ const THolder<TMonCounters> MonCounters;
+
+ TActorSystem* ActorSystem;
+ volatile ui64* CurrentTimestamp;
+ volatile ui64* CurrentMonotonic;
+
+ ui32 TotalReaders;
+ TArrayHolder<NSchedulerQueue::TReader*> Readers;
+
+ volatile bool StopFlag;
+
+ typedef TMap<ui64, TAutoPtr<NSchedulerQueue::TQueueType>> TMomentMap; // intrasecond queues
+ typedef THashMap<ui64, TAutoPtr<TMomentMap>> TScheduleMap; // over-second schedule
+
+ TScheduleMap ScheduleMap;
+
+ THolder<NThreading::TLegacyFuture<void, false>> MainCycle;
+
+ static const ui64 IntrasecondThreshold = 1048576; // ~second
+
+ void CycleFunc();
+
+ public:
+ TBasicSchedulerThread(const TSchedulerConfig& config = TSchedulerConfig());
+ ~TBasicSchedulerThread();
+
+ void Prepare(TActorSystem* actorSystem, volatile ui64* currentTimestamp, volatile ui64* currentMonotonic) override;
+ void PrepareSchedules(NSchedulerQueue::TReader** readers, ui32 scheduleReadersCount) override;
+
+ void PrepareStart() override;
+ void Start() override;
+ void PrepareStop() override;
+ void Stop() override;
+ };
+
+ class TMockSchedulerThread: public ISchedulerThread {
+ public:
+ virtual ~TMockSchedulerThread() override {
+ }
+
+ void Prepare(TActorSystem* actorSystem, volatile ui64* currentTimestamp, volatile ui64* currentMonotonic) override {
+ Y_UNUSED(actorSystem);
+ *currentTimestamp = TInstant::Now().MicroSeconds();
+ *currentMonotonic = GetMonotonicMicroSeconds();
+ }
+
+ void PrepareSchedules(NSchedulerQueue::TReader** readers, ui32 scheduleReadersCount) override {
+ Y_UNUSED(readers);
+ Y_UNUSED(scheduleReadersCount);
+ }
+
+ void Start() override {
+ }
+
+ void PrepareStop() override {
+ }
+
+ void Stop() override {
+ }
+ };
+
+ ISchedulerThread* CreateSchedulerThread(const TSchedulerConfig& cfg);
+
+}
diff --git a/library/cpp/actors/core/scheduler_cookie.cpp b/library/cpp/actors/core/scheduler_cookie.cpp
new file mode 100644
index 0000000000..0fa6f543a7
--- /dev/null
+++ b/library/cpp/actors/core/scheduler_cookie.cpp
@@ -0,0 +1,84 @@
+#include "scheduler_cookie.h"
+
+namespace NActors {
+ class TSchedulerCookie2Way: public ISchedulerCookie {
+ TAtomic Value;
+
+ public:
+ TSchedulerCookie2Way()
+ : Value(2)
+ {
+ }
+
+ bool IsArmed() noexcept override {
+ return (AtomicGet(Value) == 2);
+ }
+
+ bool Detach() noexcept override {
+ const ui64 x = AtomicDecrement(Value);
+ if (x == 1)
+ return true;
+
+ if (x == 0) {
+ delete this;
+ return false;
+ }
+
+ Y_FAIL();
+ }
+
+ bool DetachEvent() noexcept override {
+ Y_FAIL();
+ }
+ };
+
+ ISchedulerCookie* ISchedulerCookie::Make2Way() {
+ return new TSchedulerCookie2Way();
+ }
+
+ class TSchedulerCookie3Way: public ISchedulerCookie {
+ TAtomic Value;
+
+ public:
+ TSchedulerCookie3Way()
+ : Value(3)
+ {
+ }
+
+ bool IsArmed() noexcept override {
+ return (AtomicGet(Value) == 3);
+ }
+
+ bool Detach() noexcept override {
+ const ui64 x = AtomicDecrement(Value);
+ if (x == 2)
+ return true;
+ if (x == 1)
+ return false;
+ if (x == 0) {
+ delete this;
+ return false;
+ }
+
+ Y_FAIL();
+ }
+
+ bool DetachEvent() noexcept override {
+ const ui64 x = AtomicDecrement(Value);
+ if (x == 2)
+ return false;
+ if (x == 1)
+ return true;
+ if (x == 0) {
+ delete this;
+ return false;
+ }
+
+ Y_FAIL();
+ }
+ };
+
+ ISchedulerCookie* ISchedulerCookie::Make3Way() {
+ return new TSchedulerCookie3Way();
+ }
+}
diff --git a/library/cpp/actors/core/scheduler_cookie.h b/library/cpp/actors/core/scheduler_cookie.h
new file mode 100644
index 0000000000..2c20ca67f3
--- /dev/null
+++ b/library/cpp/actors/core/scheduler_cookie.h
@@ -0,0 +1,78 @@
+#pragma once
+
+#include "defs.h"
+#include <util/generic/noncopyable.h>
+
+namespace NActors {
+ class ISchedulerCookie : TNonCopyable {
+ protected:
+ virtual ~ISchedulerCookie() {
+ }
+
+ public:
+ virtual bool Detach() noexcept = 0;
+ virtual bool DetachEvent() noexcept = 0;
+ virtual bool IsArmed() noexcept = 0;
+
+ static ISchedulerCookie* Make2Way();
+ static ISchedulerCookie* Make3Way();
+ };
+
+ class TSchedulerCookieHolder : TNonCopyable {
+ ISchedulerCookie* Cookie;
+
+ public:
+ TSchedulerCookieHolder()
+ : Cookie(nullptr)
+ {
+ }
+
+ TSchedulerCookieHolder(ISchedulerCookie* x)
+ : Cookie(x)
+ {
+ }
+
+ ~TSchedulerCookieHolder() {
+ Detach();
+ }
+
+ bool operator==(const TSchedulerCookieHolder& x) const noexcept {
+ return (Cookie == x.Cookie);
+ }
+
+ ISchedulerCookie* Get() const {
+ return Cookie;
+ }
+
+ ISchedulerCookie* Release() {
+ ISchedulerCookie* result = Cookie;
+ Cookie = nullptr;
+ return result;
+ }
+
+ void Reset(ISchedulerCookie* cookie) {
+ Detach();
+ Cookie = cookie;
+ }
+
+ bool Detach() noexcept {
+ if (Cookie) {
+ const bool res = Cookie->Detach();
+ Cookie = nullptr;
+ return res;
+ } else {
+ return false;
+ }
+ }
+
+ bool DetachEvent() noexcept {
+ if (Cookie) {
+ const bool res = Cookie->DetachEvent();
+ Cookie = nullptr;
+ return res;
+ } else {
+ return false;
+ }
+ }
+ };
+}
diff --git a/library/cpp/actors/core/scheduler_queue.h b/library/cpp/actors/core/scheduler_queue.h
new file mode 100644
index 0000000000..3b8fac28f0
--- /dev/null
+++ b/library/cpp/actors/core/scheduler_queue.h
@@ -0,0 +1,120 @@
+#pragma once
+
+#include <library/cpp/actors/util/queue_chunk.h>
+
+namespace NActors {
+ class IEventHandle;
+ class ISchedulerCookie;
+
+ namespace NSchedulerQueue {
+ struct TEntry {
+ ui64 InstantMicroseconds;
+ IEventHandle* Ev;
+ ISchedulerCookie* Cookie;
+ };
+
+ struct TChunk : TQueueChunkDerived<TEntry, 512, TChunk> {};
+
+ class TReader;
+ class TWriter;
+ class TWriterWithPadding;
+
+ class TReader : ::TNonCopyable {
+ TChunk* ReadFrom;
+ ui32 ReadPosition;
+
+ friend class TWriter;
+
+ public:
+ TReader()
+ : ReadFrom(new TChunk())
+ , ReadPosition(0)
+ {
+ }
+
+ ~TReader() {
+ while (TEntry* x = Pop()) {
+ if (x->Cookie)
+ x->Cookie->Detach();
+ delete x->Ev;
+ }
+ delete ReadFrom;
+ }
+
+ TEntry* Pop() {
+ TChunk* head = ReadFrom;
+ if (ReadPosition != TChunk::EntriesCount) {
+ if (AtomicLoad(&head->Entries[ReadPosition].InstantMicroseconds) != 0)
+ return const_cast<TEntry*>(&head->Entries[ReadPosition++]);
+ else
+ return nullptr;
+ } else if (TChunk* next = AtomicLoad(&head->Next)) {
+ ReadFrom = next;
+ delete head;
+ ReadPosition = 0;
+ return Pop();
+ }
+
+ return nullptr;
+ }
+ };
+
+ class TWriter : ::TNonCopyable {
+ TChunk* WriteTo;
+ ui32 WritePosition;
+
+ public:
+ TWriter()
+ : WriteTo(nullptr)
+ , WritePosition(0)
+ {
+ }
+
+ void Init(const TReader& reader) {
+ WriteTo = reader.ReadFrom;
+ WritePosition = 0;
+ }
+
+ void Push(ui64 instantMicrosends, IEventHandle* ev, ISchedulerCookie* cookie) {
+ if (Y_UNLIKELY(instantMicrosends == 0)) {
+ // Protect against Pop() getting stuck forever
+ instantMicrosends = 1;
+ }
+ if (WritePosition != TChunk::EntriesCount) {
+ volatile TEntry& entry = WriteTo->Entries[WritePosition];
+ entry.Cookie = cookie;
+ entry.Ev = ev;
+ AtomicStore(&entry.InstantMicroseconds, instantMicrosends);
+ ++WritePosition;
+ } else {
+ TChunk* next = new TChunk();
+ volatile TEntry& entry = next->Entries[0];
+ entry.Cookie = cookie;
+ entry.Ev = ev;
+ entry.InstantMicroseconds = instantMicrosends;
+ AtomicStore(&WriteTo->Next, next);
+ WriteTo = next;
+ WritePosition = 1;
+ }
+ }
+ };
+
+ class TWriterWithPadding: public TWriter {
+ private:
+ ui8 CacheLinePadding[64 - sizeof(TWriter)];
+
+ void UnusedCacheLinePadding() {
+ Y_UNUSED(CacheLinePadding);
+ }
+ };
+
+ struct TQueueType {
+ TReader Reader;
+ TWriter Writer;
+
+ TQueueType() {
+ Writer.Init(Reader);
+ }
+ };
+ }
+}
diff --git a/library/cpp/actors/core/servicemap.h b/library/cpp/actors/core/servicemap.h
new file mode 100644
index 0000000000..d72e50cae5
--- /dev/null
+++ b/library/cpp/actors/core/servicemap.h
@@ -0,0 +1,168 @@
+#pragma once
+
+#include "defs.h"
+
+namespace NActors {
+ // wait-free one writer multi reader hash-tree for service mapping purposes
+ // on fast updates on same key - could lead to false-negatives, we don't care as such cases are broken from service-map app logic
+
+ template <typename TKey, typename TValue, typename THash, ui64 BaseSize = 256 * 1024, ui64 ExtCount = 4, ui64 ExtBranching = 4>
+ class TServiceMap : TNonCopyable {
+ struct TEntry : TNonCopyable {
+ ui32 CounterIn;
+ ui32 CounterOut;
+ TKey Key;
+ TValue Value;
+
+ TEntry()
+ : CounterIn(0)
+ , CounterOut(0)
+ , Key()
+ , Value()
+ {
+ }
+ };
+
+ struct TBranch : TNonCopyable {
+ TEntry Entries[ExtCount];
+ TBranch* Branches[ExtBranching];
+
+ TBranch() {
+ Fill(Branches, Branches + ExtBranching, (TBranch*)nullptr);
+ }
+ };
+
+ ui32 Counter;
+ TBranch* Line[BaseSize];
+
+ bool ScanBranch(TBranch* branch, const TKey& key, ui64 hash, TValue& ret) {
+ for (ui32 i = 0; i != ExtCount; ++i) {
+ const TEntry& entry = branch->Entries[i];
+ const ui32 counterIn = AtomicLoad(&entry.CounterIn);
+ if (counterIn != 0 && entry.Key == key) {
+ ret = entry.Value;
+ const ui32 counterOut = AtomicLoad(&entry.CounterOut);
+ if (counterOut == counterIn)
+ return true;
+ }
+ }
+
+ const ui64 hash0 = hash % ExtBranching;
+ if (TBranch* next = AtomicLoad(branch->Branches + hash0))
+ return ScanBranch(next, key, hash / ExtBranching, ret);
+
+ return false;
+ }
+
+ void ScanZeroOld(TBranch* branch, const TKey& key, ui64 hash, TEntry** zeroEntry, TEntry*& oldEntry) {
+ for (ui32 i = 0; i != ExtCount; ++i) {
+ TEntry& entry = branch->Entries[i];
+ if (entry.CounterIn == 0) {
+ if (zeroEntry && !*zeroEntry) {
+ *zeroEntry = &entry;
+ if (oldEntry != nullptr)
+ return;
+ }
+ } else {
+ if (entry.Key == key) {
+ oldEntry = &entry;
+ if (!zeroEntry || *zeroEntry)
+ return;
+ }
+ }
+ }
+
+ const ui64 hash0 = hash % ExtBranching;
+ if (TBranch* next = branch->Branches[hash0]) {
+ ScanZeroOld(next, key, hash / ExtBranching, zeroEntry, oldEntry);
+ } else { // found tail, if zeroEntry requested, but not yet found - insert one
+ if (zeroEntry && !*zeroEntry) {
+ TBranch* next = new TBranch();
+ *zeroEntry = next->Entries;
+ AtomicStore(branch->Branches + hash0, next);
+ }
+ }
+ }
+
+ public:
+ TServiceMap()
+ : Counter(0)
+ {
+ Fill(Line, Line + BaseSize, (TBranch*)nullptr);
+ }
+
+ ~TServiceMap() {
+ for (ui64 i = 0; i < BaseSize; ++i) {
+ delete Line[i];
+ }
+ }
+
+ TValue Find(const TKey& key) {
+ THash hashOp;
+ const ui64 hash = hashOp(key);
+ const ui64 hash0 = hash % BaseSize;
+
+ if (TBranch* branch = AtomicLoad(Line + hash0)) {
+ TValue ret;
+ if (ScanBranch(branch, key, hash / BaseSize, ret))
+ return ret;
+ }
+
+ return TValue();
+ }
+
+ // returns true on update, false on insert
+ TValue Update(const TKey& key, const TValue& value) {
+ THash hashOp;
+ const ui64 hash = hashOp(key);
+ const ui64 hash0 = hash % BaseSize;
+
+ TEntry* zeroEntry = nullptr;
+ TEntry* oldEntry = nullptr;
+
+ if (TBranch* branch = Line[hash0]) {
+ ScanZeroOld(branch, key, hash / BaseSize, &zeroEntry, oldEntry);
+ } else {
+ TBranch* next = new TBranch();
+ zeroEntry = next->Entries;
+ AtomicStore(Line + hash0, next);
+ }
+
+ // now we got both entries, first - push new one
+ const ui32 counter = AtomicUi32Increment(&Counter);
+ AtomicStore(&zeroEntry->CounterOut, counter);
+ zeroEntry->Key = key;
+ zeroEntry->Value = value;
+ AtomicStore(&zeroEntry->CounterIn, counter);
+
+ if (oldEntry != nullptr) {
+ const TValue ret = oldEntry->Value;
+ AtomicStore<ui32>(&oldEntry->CounterOut, 0);
+ AtomicStore<ui32>(&oldEntry->CounterIn, 0);
+ return ret;
+ } else {
+ return TValue();
+ }
+ }
+
+ bool Erase(const TKey& key) {
+ THash hashOp;
+ const ui64 hash = hashOp(key);
+ const ui64 hash0 = hash % BaseSize;
+
+ TEntry* oldEntry = 0;
+
+ if (TBranch* branch = Line[hash0]) {
+ ScanZeroOld(branch, key, hash / BaseSize, 0, oldEntry);
+ }
+
+ if (oldEntry != 0) {
+ AtomicStore<ui32>(&oldEntry->CounterOut, 0);
+ AtomicStore<ui32>(&oldEntry->CounterIn, 0);
+ return true;
+ } else {
+ return false;
+ }
+ }
+ };
+}
diff --git a/library/cpp/actors/core/ut/ya.make b/library/cpp/actors/core/ut/ya.make
new file mode 100644
index 0000000000..3ee28d5850
--- /dev/null
+++ b/library/cpp/actors/core/ut/ya.make
@@ -0,0 +1,46 @@
+UNITTEST_FOR(library/cpp/actors/core)
+
+OWNER(
+ alexvru
+ g:kikimr
+)
+
+FORK_SUBTESTS()
+IF (SANITIZER_TYPE)
+ SIZE(LARGE)
+ TIMEOUT(1200)
+ TAG(ya:fat)
+ SPLIT_FACTOR(20)
+ REQUIREMENTS(
+ ram:32
+ )
+ELSE()
+ SIZE(MEDIUM)
+ TIMEOUT(600)
+ REQUIREMENTS(
+ ram:16
+ )
+ENDIF()
+
+
+PEERDIR(
+ library/cpp/actors/interconnect
+ library/cpp/actors/testlib
+)
+
+SRCS(
+ actor_coroutine_ut.cpp
+ actor_ut.cpp
+ actorsystem_ut.cpp
+ ask_ut.cpp
+ balancer_ut.cpp
+ event_pb_payload_ut.cpp
+ event_pb_ut.cpp
+ executor_pool_basic_ut.cpp
+ executor_pool_united_ut.cpp
+ log_ut.cpp
+ memory_tracker_ut.cpp
+ scheduler_actor_ut.cpp
+)
+
+END()
diff --git a/library/cpp/actors/core/worker_context.cpp b/library/cpp/actors/core/worker_context.cpp
new file mode 100644
index 0000000000..ada6c997d4
--- /dev/null
+++ b/library/cpp/actors/core/worker_context.cpp
@@ -0,0 +1,7 @@
+#include "worker_context.h"
+#include "probes.h"
+
+namespace NActors {
+ LWTRACE_USING(ACTORLIB_PROVIDER);
+
+}
diff --git a/library/cpp/actors/core/worker_context.h b/library/cpp/actors/core/worker_context.h
new file mode 100644
index 0000000000..b4c37a7629
--- /dev/null
+++ b/library/cpp/actors/core/worker_context.h
@@ -0,0 +1,175 @@
+#pragma once
+
+#include "defs.h"
+
+#include "actorsystem.h"
+#include "event.h"
+#include "lease.h"
+#include "mailbox.h"
+#include "mon_stats.h"
+
+#include <library/cpp/actors/util/datetime.h>
+#include <library/cpp/actors/util/intrinsics.h>
+#include <library/cpp/actors/util/thread.h>
+
+#include <library/cpp/lwtrace/shuttle.h>
+
+namespace NActors {
+ struct TWorkerContext {
+ const TWorkerId WorkerId;
+ const TCpuId CpuId;
+ TLease Lease;
+ IExecutorPool* Executor = nullptr;
+ TMailboxTable* MailboxTable = nullptr;
+ ui64 TimePerMailboxTs = 0;
+ ui32 EventsPerMailbox = 0;
+ ui64 SoftDeadlineTs = ui64(-1);
+ TExecutorThreadStats* Stats = &WorkerStats; // pool stats
+ TExecutorThreadStats WorkerStats;
+ TPoolId PoolId = MaxPools;
+ mutable NLWTrace::TOrbit Orbit;
+
+ TWorkerContext(TWorkerId workerId, TCpuId cpuId, size_t activityVecSize)
+ : WorkerId(workerId)
+ , CpuId(cpuId)
+ , Lease(WorkerId, NeverExpire)
+ , WorkerStats(activityVecSize)
+ {}
+
+#ifdef ACTORSLIB_COLLECT_EXEC_STATS
+ void GetCurrentStats(TExecutorThreadStats& statsCopy) const {
+ statsCopy = TExecutorThreadStats();
+ statsCopy.Aggregate(*Stats);
+ }
+
+ void AddElapsedCycles(ui32 activityType, i64 elapsed) {
+ Y_VERIFY_DEBUG(activityType < Stats->MaxActivityType());
+ RelaxedStore(&Stats->ElapsedTicks, RelaxedLoad(&Stats->ElapsedTicks) + elapsed);
+ RelaxedStore(&Stats->ElapsedTicksByActivity[activityType], RelaxedLoad(&Stats->ElapsedTicksByActivity[activityType]) + elapsed);
+ }
+
+ void AddParkedCycles(i64 elapsed) {
+ RelaxedStore(&Stats->ParkedTicks, RelaxedLoad(&Stats->ParkedTicks) + elapsed);
+ }
+
+ void AddBlockedCycles(i64 elapsed) {
+ RelaxedStore(&Stats->BlockedTicks, RelaxedLoad(&Stats->BlockedTicks) + elapsed);
+ }
+
+ void IncrementSentEvents() {
+ RelaxedStore(&Stats->SentEvents, RelaxedLoad(&Stats->SentEvents) + 1);
+ }
+
+ void IncrementPreemptedEvents() {
+ RelaxedStore(&Stats->PreemptedEvents, RelaxedLoad(&Stats->PreemptedEvents) + 1);
+ }
+
+ void DecrementActorsAliveByActivity(ui32 activityType) {
+ if (activityType >= Stats->MaxActivityType()) {
+ activityType = 0;
+ }
+ RelaxedStore(&Stats->ActorsAliveByActivity[activityType], Stats->ActorsAliveByActivity[activityType] - 1);
+ }
+
+ inline void IncrementNonDeliveredEvents() {
+ RelaxedStore(&Stats->NonDeliveredEvents, RelaxedLoad(&Stats->NonDeliveredEvents) + 1);
+ }
+
+ inline void IncrementMailboxPushedOutBySoftPreemption() {
+ RelaxedStore(&Stats->MailboxPushedOutBySoftPreemption, RelaxedLoad(&Stats->MailboxPushedOutBySoftPreemption) + 1);
+ }
+
+ inline void IncrementMailboxPushedOutByTime() {
+ RelaxedStore(&Stats->MailboxPushedOutByTime, RelaxedLoad(&Stats->MailboxPushedOutByTime) + 1);
+ }
+
+ inline void IncrementMailboxPushedOutByEventCount() {
+ RelaxedStore(&Stats->MailboxPushedOutByEventCount, RelaxedLoad(&Stats->MailboxPushedOutByEventCount) + 1);
+ }
+
+ inline void IncrementEmptyMailboxActivation() {
+ RelaxedStore(&Stats->EmptyMailboxActivation, RelaxedLoad(&Stats->EmptyMailboxActivation) + 1);
+ }
+
+ double AddActivationStats(i64 scheduleTs, i64 deliveredTs) {
+ i64 ts = deliveredTs > scheduleTs ? deliveredTs - scheduleTs : 0;
+ double usec = NHPTimer::GetSeconds(ts) * 1000000.0;
+ Stats->ActivationTimeHistogram.Add(usec);
+ return usec;
+ }
+
+ ui64 AddEventDeliveryStats(i64 sentTs, i64 deliveredTs) {
+ ui64 usecDeliv = deliveredTs > sentTs ? NHPTimer::GetSeconds(deliveredTs - sentTs) * 1000000 : 0;
+ Stats->EventDeliveryTimeHistogram.Add(usecDeliv);
+ return usecDeliv;
+ }
+
+ i64 AddEventProcessingStats(i64 deliveredTs, i64 processedTs, ui32 activityType, ui64 scheduled) {
+ i64 elapsed = processedTs - deliveredTs;
+ ui64 usecElapsed = NHPTimer::GetSeconds(elapsed) * 1000000;
+ activityType = (activityType >= Stats->MaxActivityType()) ? 0 : activityType;
+ Stats->EventProcessingCountHistogram.Add(usecElapsed);
+ Stats->EventProcessingTimeHistogram.Add(usecElapsed, elapsed);
+ RelaxedStore(&Stats->ReceivedEvents, RelaxedLoad(&Stats->ReceivedEvents) + 1);
+ RelaxedStore(&Stats->ReceivedEventsByActivity[activityType], RelaxedLoad(&Stats->ReceivedEventsByActivity[activityType]) + 1);
+ RelaxedStore(&Stats->ScheduledEventsByActivity[activityType], RelaxedLoad(&Stats->ScheduledEventsByActivity[activityType]) + scheduled);
+ AddElapsedCycles(activityType, elapsed);
+ return elapsed;
+ }
+
+ void UpdateActorsStats(size_t dyingActorsCnt) {
+ if (dyingActorsCnt) {
+ AtomicAdd(Executor->DestroyedActors, dyingActorsCnt);
+ }
+ RelaxedStore(&Stats->PoolDestroyedActors, (ui64)RelaxedLoad(&Executor->DestroyedActors));
+ RelaxedStore(&Stats->PoolActorRegistrations, (ui64)RelaxedLoad(&Executor->ActorRegistrations));
+ RelaxedStore(&Stats->PoolAllocatedMailboxes, MailboxTable->GetAllocatedMailboxCount());
+ }
+
+ void UpdateThreadTime() {
+ RelaxedStore(&WorkerStats.CpuNs, ThreadCPUTime() * 1000);
+ }
+#else
+ void GetCurrentStats(TExecutorThreadStats&) const {}
+ inline void AddElapsedCycles(ui32, i64) {}
+ inline void AddParkedCycles(i64) {}
+ inline void AddBlockedCycles(i64) {}
+ inline void IncrementSentEvents() {}
+ inline void IncrementPreemptedEvents() {}
+ inline void IncrementMailboxPushedOutBySoftPreemption() {}
+ inline void IncrementMailboxPushedOutByTime() {}
+ inline void IncrementMailboxPushedOutByEventCount() {}
+ inline void IncrementEmptyMailboxActivation() {}
+ void DecrementActorsAliveByActivity(ui32) {}
+ void IncrementNonDeliveredEvents() {}
+ double AddActivationStats(i64, i64) { return 0; }
+ ui64 AddEventDeliveryStats(i64, i64) { return 0; }
+ i64 AddEventProcessingStats(i64, i64, ui32, ui64) { return 0; }
+ void UpdateActorsStats(size_t, IExecutorPool*) {}
+ void UpdateThreadTime() {}
+#endif
+
+ void Switch(IExecutorPool* executor,
+ TMailboxTable* mailboxTable,
+ ui64 timePerMailboxTs,
+ ui32 eventsPerMailbox,
+ ui64 softDeadlineTs,
+ TExecutorThreadStats* stats)
+ {
+ Executor = executor;
+ MailboxTable = mailboxTable;
+ TimePerMailboxTs = timePerMailboxTs;
+ EventsPerMailbox = eventsPerMailbox;
+ SoftDeadlineTs = softDeadlineTs;
+ Stats = stats;
+ PoolId = Executor ? Executor->PoolId : MaxPools;
+ }
+
+ void SwitchToIdle() {
+ Executor = nullptr;
+ MailboxTable = nullptr;
+ //Stats = &WorkerStats; // TODO: in actorsystem 2.0 idle stats cannot be related to specific pool
+ PoolId = MaxPools;
+ }
+ };
+}
diff --git a/library/cpp/actors/core/ya.make b/library/cpp/actors/core/ya.make
new file mode 100644
index 0000000000..880a9d00db
--- /dev/null
+++ b/library/cpp/actors/core/ya.make
@@ -0,0 +1,123 @@
+LIBRARY()
+
+OWNER(
+ ddoarn
+ g:kikimr
+)
+
+NO_WSHADOW()
+
+IF (PROFILE_MEMORY_ALLOCATIONS)
+ CFLAGS(-DPROFILE_MEMORY_ALLOCATIONS)
+ENDIF()
+
+IF (ALLOCATOR == "B" OR ALLOCATOR == "BS" OR ALLOCATOR == "C")
+ CXXFLAGS(-DBALLOC)
+ PEERDIR(
+ library/cpp/balloc/optional
+ )
+ENDIF()
+
+SRCS(
+ actor_bootstrapped.h
+ actor_coroutine.cpp
+ actor_coroutine.h
+ actor.cpp
+ actor.h
+ actorid.cpp
+ actorid.h
+ actorsystem.cpp
+ actorsystem.h
+ ask.cpp
+ ask.h
+ balancer.h
+ balancer.cpp
+ buffer.cpp
+ buffer.h
+ callstack.cpp
+ callstack.h
+ config.h
+ cpu_manager.cpp
+ cpu_manager.h
+ cpu_state.h
+ defs.h
+ event.cpp
+ event.h
+ event_load.h
+ event_local.h
+ event_pb.cpp
+ event_pb.h
+ events.h
+ events_undelivered.cpp
+ executelater.h
+ executor_pool_base.cpp
+ executor_pool_base.h
+ executor_pool_basic.cpp
+ executor_pool_basic.h
+ executor_pool_io.cpp
+ executor_pool_io.h
+ executor_pool_united.cpp
+ executor_pool_united.h
+ executor_thread.cpp
+ executor_thread.h
+ hfunc.h
+ interconnect.cpp
+ interconnect.h
+ invoke.h
+ io_dispatcher.cpp
+ io_dispatcher.h
+ lease.h
+ log.cpp
+ log.h
+ log_settings.cpp
+ log_settings.h
+ mailbox.cpp
+ mailbox.h
+ mailbox_queue_revolving.h
+ mailbox_queue_simple.h
+ memory_track.cpp
+ memory_track.h
+ memory_tracker.cpp
+ memory_tracker.h
+ mon.h
+ mon_stats.h
+ monotonic.cpp
+ monotonic.h
+ worker_context.cpp
+ worker_context.h
+ probes.cpp
+ probes.h
+ process_stats.cpp
+ process_stats.h
+ scheduler_actor.cpp
+ scheduler_actor.h
+ scheduler_basic.cpp
+ scheduler_basic.h
+ scheduler_cookie.cpp
+ scheduler_cookie.h
+ scheduler_queue.h
+ servicemap.h
+)
+
+GENERATE_ENUM_SERIALIZATION(defs.h)
+GENERATE_ENUM_SERIALIZATION(actor.h)
+
+PEERDIR(
+ library/cpp/actors/memory_log
+ library/cpp/actors/prof
+ library/cpp/actors/protos
+ library/cpp/actors/util
+ library/cpp/execprofile
+ library/cpp/json/writer
+ library/cpp/logger
+ library/cpp/lwtrace
+ library/cpp/monlib/dynamic_counters
+ library/cpp/svnversion
+ library/cpp/threading/future
+)
+
+END()
+
+RECURSE_FOR_TESTS(
+ ut
+)