aboutsummaryrefslogtreecommitdiffstats
path: root/library/cpp/actors
diff options
context:
space:
mode:
authorserxa <serxa@yandex-team.ru>2022-02-10 16:49:08 +0300
committerDaniil Cherednik <dcherednik@yandex-team.ru>2022-02-10 16:49:08 +0300
commitd6d7db348c2cc64e71243cab9940ee6778f4317d (patch)
treebac67f42a02f9368eb4d329f5d79b77d0a6adc18 /library/cpp/actors
parent8d57b69dee81198a59c39e64704f7dc9f04b4fbf (diff)
downloadydb-d6d7db348c2cc64e71243cab9940ee6778f4317d.tar.gz
Restoring authorship annotation for <serxa@yandex-team.ru>. Commit 1 of 2.
Diffstat (limited to 'library/cpp/actors')
-rw-r--r--library/cpp/actors/core/actor.cpp4
-rw-r--r--library/cpp/actors/core/actor_ut.cpp472
-rw-r--r--library/cpp/actors/core/actorid.h2
-rw-r--r--library/cpp/actors/core/actorsystem.cpp40
-rw-r--r--library/cpp/actors/core/actorsystem.h84
-rw-r--r--library/cpp/actors/core/balancer.cpp586
-rw-r--r--library/cpp/actors/core/balancer.h54
-rw-r--r--library/cpp/actors/core/balancer_ut.cpp450
-rw-r--r--library/cpp/actors/core/config.h478
-rw-r--r--library/cpp/actors/core/cpu_manager.cpp216
-rw-r--r--library/cpp/actors/core/cpu_manager.h114
-rw-r--r--library/cpp/actors/core/cpu_state.h430
-rw-r--r--library/cpp/actors/core/defs.h46
-rw-r--r--library/cpp/actors/core/event_pb.h8
-rw-r--r--library/cpp/actors/core/executor_pool_base.cpp72
-rw-r--r--library/cpp/actors/core/executor_pool_base.h42
-rw-r--r--library/cpp/actors/core/executor_pool_basic.cpp116
-rw-r--r--library/cpp/actors/core/executor_pool_basic.h18
-rw-r--r--library/cpp/actors/core/executor_pool_basic_ut.cpp110
-rw-r--r--library/cpp/actors/core/executor_pool_io.cpp68
-rw-r--r--library/cpp/actors/core/executor_pool_io.h14
-rw-r--r--library/cpp/actors/core/executor_pool_united.cpp2764
-rw-r--r--library/cpp/actors/core/executor_pool_united.h234
-rw-r--r--library/cpp/actors/core/executor_pool_united_ut.cpp404
-rw-r--r--library/cpp/actors/core/executor_thread.cpp268
-rw-r--r--library/cpp/actors/core/executor_thread.h80
-rw-r--r--library/cpp/actors/core/lease.h112
-rw-r--r--library/cpp/actors/core/log.cpp152
-rw-r--r--library/cpp/actors/core/log.h2
-rw-r--r--library/cpp/actors/core/mailbox.cpp16
-rw-r--r--library/cpp/actors/core/mailbox.h2
-rw-r--r--library/cpp/actors/core/mon_stats.h40
-rw-r--r--library/cpp/actors/core/probes.cpp50
-rw-r--r--library/cpp/actors/core/probes.h242
-rw-r--r--library/cpp/actors/core/scheduler_actor.cpp4
-rw-r--r--library/cpp/actors/core/scheduler_basic.cpp12
-rw-r--r--library/cpp/actors/core/scheduler_basic.h4
-rw-r--r--library/cpp/actors/core/ut/ya.make4
-rw-r--r--library/cpp/actors/core/worker_context.cpp14
-rw-r--r--library/cpp/actors/core/worker_context.h350
-rw-r--r--library/cpp/actors/core/ya.make24
-rw-r--r--library/cpp/actors/dnscachelib/dnscache.cpp74
-rw-r--r--library/cpp/actors/dnscachelib/dnscache.h2
-rw-r--r--library/cpp/actors/dnscachelib/probes.cpp6
-rw-r--r--library/cpp/actors/dnscachelib/probes.h12
-rw-r--r--library/cpp/actors/dnscachelib/ya.make4
-rw-r--r--library/cpp/actors/helpers/flow_controlled_queue.cpp6
-rw-r--r--library/cpp/actors/interconnect/interconnect_channel.cpp12
-rw-r--r--library/cpp/actors/interconnect/interconnect_channel.h12
-rw-r--r--library/cpp/actors/interconnect/interconnect_common.h4
-rw-r--r--library/cpp/actors/interconnect/interconnect_tcp_input_session.cpp6
-rw-r--r--library/cpp/actors/interconnect/interconnect_tcp_session.cpp150
-rw-r--r--library/cpp/actors/interconnect/interconnect_tcp_session.h36
-rw-r--r--library/cpp/actors/interconnect/packet.cpp10
-rw-r--r--library/cpp/actors/interconnect/packet.h10
-rw-r--r--library/cpp/actors/interconnect/poller_actor.cpp8
-rw-r--r--library/cpp/actors/interconnect/profiler.h8
-rw-r--r--library/cpp/actors/memory_log/memlog.cpp8
-rw-r--r--library/cpp/actors/testlib/test_runtime.cpp80
-rw-r--r--library/cpp/actors/testlib/test_runtime.h30
-rw-r--r--library/cpp/actors/util/affinity.cpp82
-rw-r--r--library/cpp/actors/util/affinity.h14
-rw-r--r--library/cpp/actors/util/cpumask.h266
-rw-r--r--library/cpp/actors/util/datetime.h164
-rw-r--r--library/cpp/actors/util/thread.h2
-rw-r--r--library/cpp/actors/util/timerfd.h130
-rw-r--r--library/cpp/actors/util/ya.make6
67 files changed, 4672 insertions, 4672 deletions
diff --git a/library/cpp/actors/core/actor.cpp b/library/cpp/actors/core/actor.cpp
index 6f9ba6a42b..d46d3e9b50 100644
--- a/library/cpp/actors/core/actor.cpp
+++ b/library/cpp/actors/core/actor.cpp
@@ -1,7 +1,7 @@
#include "actor.h"
#include "executor_thread.h"
#include "mailbox.h"
-#include <library/cpp/actors/util/datetime.h>
+#include <library/cpp/actors/util/datetime.h>
namespace NActors {
Y_POD_THREAD(TActivationContext*)
@@ -88,7 +88,7 @@ namespace NActors {
}
i64 TActivationContext::GetCurrentEventTicks() {
- return GetCycleCountFast() - TlsActivationContext->EventStart;
+ return GetCycleCountFast() - TlsActivationContext->EventStart;
}
double TActivationContext::GetCurrentEventTicksAsSeconds() {
diff --git a/library/cpp/actors/core/actor_ut.cpp b/library/cpp/actors/core/actor_ut.cpp
index e1b765ec72..0146b5a6de 100644
--- a/library/cpp/actors/core/actor_ut.cpp
+++ b/library/cpp/actors/core/actor_ut.cpp
@@ -35,7 +35,7 @@ struct TTestEndDecorator : TDecorator {
};
Y_UNIT_TEST_SUITE(ActorBenchmark) {
- static constexpr bool DefaultNoRealtime = true;
+ static constexpr bool DefaultNoRealtime = true;
static constexpr ui32 DefaultSpinThreshold = 1000000;
static constexpr ui32 TotalEventsAmount = 1000;
@@ -43,49 +43,49 @@ Y_UNIT_TEST_SUITE(ActorBenchmark) {
public:
TDummyActor() : TActor<TDummyActor>(&TDummyActor::StateFunc) {}
STFUNC(StateFunc) {
- (void)ev;
+ (void)ev;
(void)ctx;
}
};
- enum ERole {
- Leader,
- Follower
- };
-
+ 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)
+ 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
+ } 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());
- }
+ if (Role == Leader) {
+ Send(Receiver, new TEvents::TEvPing());
+ }
}
STATEFN(StateFunc) {
- if (EventsCounter == 0 && ElapsedTime != nullptr) {
+ if (EventsCounter == 0 && ElapsedTime != nullptr) {
*ElapsedTime = Timer.Passed() / TotalEventsAmount;
PassAway();
}
@@ -97,91 +97,91 @@ Y_UNIT_TEST_SUITE(ActorBenchmark) {
ev->DropRewrite();
TActivationContext::Send(ev.Release());
}
- EventsCounter--;
+ EventsCounter--;
}
private:
THPTimer Timer;
- ui64 EventsCounter;
+ ui64 EventsCounter;
double* ElapsedTime;
TActorId Receiver;
bool AllocatesMemory;
- ERole Role;
+ 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;
+ 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>();
}
- Y_FAIL();
- }
-
- double BenchSendReceive(bool allocation, NActors::TMailboxType::EType mType, EPoolType poolType) {
- THolder<TActorSystemSetup> setup = InitActorSystemSetup(poolType, 1, 1, false, false);
+ 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();
@@ -197,86 +197,86 @@ Y_UNIT_TEST_SUITE(ActorBenchmark) {
pad.Park();
actorSystem.Stop();
- return 1e9 * elapsedTime;
+ 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);
+ 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 followerPoolId = 0;
- ui32 leaderPoolId = poolsCount == 1 ? 0 : 1;
- TActorId followerId = actorSystem.Register(
- new TSendReceiveActor(nullptr, {}, allocation, Follower), TMailboxType::HTSwap, followerPoolId);
+ 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);
-
+ actorSystem.Register(leader.Release(), TMailboxType::HTSwap, leaderPoolId);
+
pad.Park();
actorSystem.Stop();
- return 1e9 * elapsedTime;
+ return 1e9 * elapsedTime;
}
- double BenchSendActivateReceiveWithMailboxNeighbours(ui32 MailboxNeighbourActors, EPoolType poolType) {
- THolder<TActorSystemSetup> setup = InitActorSystemSetup(poolType, 1, 1, false, false);
- TActorSystem actorSystem(setup);
+ 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);
+ 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);
+ actorSystem.Register(leader.Release(), TMailboxType::HTSwap);
pad.Park();
actorSystem.Stop();
- return 1e9 * elapsedTime;
+ return 1e9 * elapsedTime;
}
- double BenchContentedThreads(ui32 threads, ui32 actorsPairsCount, EPoolType poolType) {
- THolder<TActorSystemSetup> setup = InitActorSystemSetup(poolType, 1, threads, true, false);
- TActorSystem actorSystem(setup);
+ 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);
+ 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);
+ 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);
+ actorSystem.Register(leader.Release(), TMailboxType::HTSwap, leaderPoolId);
}
pad.Park();
auto elapsedTime = Timer.Passed() / TotalEventsAmount;
actorSystem.Stop();
- return 1e9 * elapsedTime;
+ return 1e9 * elapsedTime;
}
auto Mean(const TVector<double>& data) {
- return Accumulate(data.begin(), data.end(), 0.0) / data.size();
+ return Accumulate(data.begin(), data.end(), 0.0) / data.size();
}
auto Deviation(const TVector<double>& data) {
@@ -285,19 +285,19 @@ Y_UNIT_TEST_SUITE(ActorBenchmark) {
for (const auto& x : data) {
deviation += (x - mean) * (x - mean);
}
- return std::sqrt(deviation / data.size());
+ 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 << "%";
+ return TStringBuilder() << Mean << " ± " << Deviation << " ns " << std::ceil(Deviation / Mean * 1000) / 10.0 << "%";
}
};
template <typename Func>
- TStats CountStats(Func func, ui32 itersCount = 5) {
+ TStats CountStats(Func func, ui32 itersCount = 5) {
TVector<double> elapsedTimes;
for (ui32 i = 0; i < itersCount; ++i) {
auto elapsedTime = func();
@@ -314,171 +314,171 @@ Y_UNIT_TEST_SUITE(ActorBenchmark) {
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) {
+ Y_UNIT_TEST(SendReceive1Pool1ThreadAlloc) {
for (const auto& mType : MailboxTypes) {
auto stats = CountStats([mType] {
- return BenchSendReceive(true, mType, EPoolType::United);
+ 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);
+ 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(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);
+ 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);
+ 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);
+ 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);
+ return BenchSendActivateReceive(1, 2, false, EPoolType::Basic);
});
Cerr << stats.ToString() << Endl;
}
- Y_UNIT_TEST(SendActivateReceive2Pool1ThreadNoAlloc) {
+ Y_UNIT_TEST(SendActivateReceive1Pool2ThreadsNoAllocUnited) {
auto stats = CountStats([] {
- return BenchSendActivateReceive(2, 1, false, EPoolType::Basic);
+ return BenchSendActivateReceive(1, 2, false, EPoolType::United);
});
Cerr << stats.ToString() << Endl;
}
- Y_UNIT_TEST(SendActivateReceive2Pool1ThreadNoAllocUnited) {
+ Y_UNIT_TEST(SendActivateReceive2Pool1ThreadAlloc) {
auto stats = CountStats([] {
- return BenchSendActivateReceive(2, 1, false, EPoolType::United);
+ return BenchSendActivateReceive(2, 1, true, EPoolType::Basic);
});
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(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);
+ return BenchSendActivateReceiveWithMailboxNeighbours(neighbour, EPoolType::Basic);
});
- Cerr << stats.ToString() << " neighbourActors: " << neighbour << Endl;
+ 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) {
diff --git a/library/cpp/actors/core/actorid.h b/library/cpp/actors/core/actorid.h
index d972b1a0ff..b8c41f5dd8 100644
--- a/library/cpp/actors/core/actorid.h
+++ b/library/cpp/actors/core/actorid.h
@@ -180,7 +180,7 @@ namespace NActors {
};
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
+ static_assert(MaxPools < TActorId::MaxPoolID); // current implementation of united pool has limit MaxPools on pool id
}
template <>
diff --git a/library/cpp/actors/core/actorsystem.cpp b/library/cpp/actors/core/actorsystem.cpp
index c58698a206..b2bed85f92 100644
--- a/library/cpp/actors/core/actorsystem.cpp
+++ b/library/cpp/actors/core/actorsystem.cpp
@@ -1,7 +1,7 @@
#include "defs.h"
#include "actorsystem.h"
#include "callstack.h"
-#include "cpu_manager.h"
+#include "cpu_manager.h"
#include "mailbox.h"
#include "events.h"
#include "interconnect.h"
@@ -9,10 +9,10 @@
#include "scheduler_queue.h"
#include "scheduler_actor.h"
#include "log.h"
-#include "probes.h"
+#include "probes.h"
#include "ask.h"
#include <library/cpp/actors/util/affinity.h>
-#include <library/cpp/actors/util/datetime.h>
+#include <library/cpp/actors/util/datetime.h>
#include <util/generic/hash.h>
#include <util/system/rwlock.h>
#include <util/random/random.h>
@@ -38,8 +38,8 @@ namespace NActors {
TActorSystem::TActorSystem(THolder<TActorSystemSetup>& setup, void* appData,
TIntrusivePtr<NLog::TSettings> loggerSettings)
: NodeId(setup->NodeId)
- , CpuManager(new TCpuManager(setup))
- , ExecutorPoolCount(CpuManager->GetExecutorsCount())
+ , CpuManager(new TCpuManager(setup))
+ , ExecutorPoolCount(CpuManager->GetExecutorsCount())
, Scheduler(setup->Scheduler)
, InterconnectCount((ui32)setup->Interconnect.ProxyActors.size())
, CurrentTimestamp(0)
@@ -105,10 +105,10 @@ namespace NActors {
Y_VERIFY_DEBUG(recipient == ev->GetRecipientRewrite());
const ui32 recpPool = recipient.PoolID();
if (recipient && recpPool < ExecutorPoolCount) {
- if (CpuManager->GetExecutorPool(recpPool)->Send(ev)) {
+ if (CpuManager->GetExecutorPool(recpPool)->Send(ev)) {
return true;
- }
- }
+ }
+ }
Send(ev->ForwardOnNondelivery(TEvents::TEvUndelivered::ReasonActorUnknown));
return false;
@@ -142,7 +142,7 @@ namespace NActors {
const TActorId& parentId) {
Y_VERIFY(executorPool < ExecutorPoolCount, "executorPool# %" PRIu32 ", ExecutorPoolCount# %" PRIu32,
(ui32)executorPool, (ui32)ExecutorPoolCount);
- return CpuManager->GetExecutorPool(executorPool)->Register(actor, mailboxType, revolvingCounter, parentId);
+ return CpuManager->GetExecutorPool(executorPool)->Register(actor, mailboxType, revolvingCounter, parentId);
}
NThreading::TFuture<THolder<IEventBase>> TActorSystem::AskGeneric(TMaybe<ui32> expectedEventType,
@@ -199,20 +199,20 @@ namespace NActors {
return ServiceMap->RegisterLocalService(serviceId, actorId);
}
- void TActorSystem::GetPoolStats(ui32 poolId, TExecutorPoolStats& poolStats, TVector<TExecutorThreadStats>& statsCopy) const {
- CpuManager->GetPoolStats(poolId, poolStats, statsCopy);
- }
-
+ 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());
+ ScheduleQueue.Reset(new NSchedulerQueue::TQueueType());
TVector<NSchedulerQueue::TReader*> scheduleReaders;
scheduleReaders.push_back(&ScheduleQueue->Reader);
- CpuManager->PrepareStart(scheduleReaders, this);
+ CpuManager->PrepareStart(scheduleReaders, this);
Scheduler->Prepare(this, &CurrentTimestamp, &CurrentMonotonic);
- Scheduler->PrepareSchedules(&scheduleReaders.front(), (ui32)scheduleReaders.size());
+ Scheduler->PrepareSchedules(&scheduleReaders.front(), (ui32)scheduleReaders.size());
// setup interconnect proxies
{
@@ -243,7 +243,7 @@ namespace NActors {
SystemSetup.Destroy();
Scheduler->PrepareStart();
- CpuManager->Start();
+ CpuManager->Start();
Send(MakeSchedulerActorId(), new TEvSchedulerInitialize(scheduleReaders, &CurrentTimestamp, &CurrentMonotonic));
Scheduler->Start();
}
@@ -259,9 +259,9 @@ namespace NActors {
}
Scheduler->PrepareStop();
- CpuManager->PrepareStop();
+ CpuManager->PrepareStop();
Scheduler->Stop();
- CpuManager->Shutdown();
+ CpuManager->Shutdown();
}
void TActorSystem::Cleanup() {
@@ -269,7 +269,7 @@ namespace NActors {
if (CleanupExecuted || !StartExecuted)
return;
CleanupExecuted = true;
- CpuManager->Cleanup();
+ CpuManager->Cleanup();
Scheduler.Destroy();
}
diff --git a/library/cpp/actors/core/actorsystem.h b/library/cpp/actors/core/actorsystem.h
index 40499d7586..d6d88ba0c2 100644
--- a/library/cpp/actors/core/actorsystem.h
+++ b/library/cpp/actors/core/actorsystem.h
@@ -1,27 +1,27 @@
#pragma once
#include "defs.h"
-
-#include "actor.h"
-#include "balancer.h"
-#include "config.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 TCpuManager;
class IExecutorPool;
- struct TWorkerContext;
+ struct TWorkerContext;
inline TActorId MakeInterconnectProxyId(ui32 destNodeId) {
char data[12];
@@ -62,9 +62,9 @@ namespace NActors {
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;
+ // 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.
@@ -72,9 +72,9 @@ namespace NActors {
* @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
+ * @param workerId index of thread which will perform event dispatching
*/
- virtual void Schedule(TInstant deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) = 0;
+ 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.
@@ -82,9 +82,9 @@ namespace NActors {
* @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
+ * @param workerId index of thread which will perform event dispatching
*/
- virtual void Schedule(TMonotonic deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) = 0;
+ virtual void Schedule(TMonotonic deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) = 0;
/**
* Schedule one-shot event that will be send after given delay.
@@ -92,9 +92,9 @@ namespace NActors {
* @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
+ * @param workerId index of thread which will perform event dispatching
*/
- virtual void Schedule(TDuration delta, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) = 0;
+ virtual void Schedule(TDuration delta, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) = 0;
// for actorsystem
virtual bool Send(TAutoPtr<IEventHandle>& ev) = 0;
@@ -104,7 +104,7 @@ namespace NActors {
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 Prepare(TActorSystem* actorSystem, NSchedulerQueue::TReader** scheduleReaders, ui32* scheduleSz) = 0;
virtual void Start() = 0;
virtual void PrepareStop() = 0;
virtual void Shutdown() = 0;
@@ -115,7 +115,7 @@ namespace NActors {
Y_UNUSED(poolStats);
Y_UNUSED(statsCopy);
}
-
+
virtual TString GetName() const {
return TString();
}
@@ -127,7 +127,7 @@ namespace NActors {
// generic
virtual TAffinity* Affinity() const = 0;
- virtual void SetRealTimeMode() const {}
+ virtual void SetRealTimeMode() const {}
};
// could be proxy to in-pool schedulers (for NUMA-aware executors)
@@ -137,7 +137,7 @@ namespace NActors {
}
virtual void Prepare(TActorSystem* actorSystem, volatile ui64* currentTimestamp, volatile ui64* currentMonotonic) = 0;
- virtual void PrepareSchedules(NSchedulerQueue::TReader** readers, ui32 scheduleReadersCount) = 0;
+ virtual void PrepareSchedules(NSchedulerQueue::TReader** readers, ui32 scheduleReadersCount) = 0;
virtual void PrepareStart() { /* empty */ }
virtual void Start() = 0;
virtual void PrepareStop() = 0;
@@ -180,14 +180,14 @@ namespace NActors {
struct TActorSystemSetup {
ui32 NodeId = 0;
- // Either Executors or CpuManager must be initialized
+ // 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<IBalancer> Balancer; // main implementation will be implicitly created if not set
+
+ TCpuManagerConfig CpuManager;
+
TAutoPtr<ISchedulerThread> Scheduler;
ui32 MaxActivityType = 5; // for default entries
@@ -195,18 +195,18 @@ namespace NActors {
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);
- }
+
+ 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 {
@@ -214,9 +214,9 @@ namespace NActors {
public:
const ui32 NodeId;
-
+
private:
- THolder<TCpuManager> CpuManager;
+ THolder<TCpuManager> CpuManager;
const ui32 ExecutorPoolCount;
TAutoPtr<ISchedulerThread> Scheduler;
@@ -353,7 +353,7 @@ namespace NActors {
return LoggerSettings0.Get();
}
- void GetPoolStats(ui32 poolId, TExecutorPoolStats& poolStats, TVector<TExecutorThreadStats>& statsCopy) const;
+ void GetPoolStats(ui32 poolId, TExecutorPoolStats& poolStats, TVector<TExecutorThreadStats>& statsCopy) const;
void DeferPreStop(std::function<void()> fn) {
DeferredPreStop.push_back(std::move(fn));
diff --git a/library/cpp/actors/core/balancer.cpp b/library/cpp/actors/core/balancer.cpp
index cc5417b0b5..6ee975bec1 100644
--- a/library/cpp/actors/core/balancer.cpp
+++ b/library/cpp/actors/core/balancer.cpp
@@ -1,293 +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);
- }
-}
+#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
index 9763ec79e1..cf4ea776f0 100644
--- a/library/cpp/actors/core/balancer.h
+++ b/library/cpp/actors/core/balancer.h
@@ -1,27 +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);
-}
+#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
index 7e5e95f4b9..ac4ebdccb3 100644
--- a/library/cpp/actors/core/balancer_ut.cpp
+++ b/library/cpp/actors/core/balancer_ut.cpp
@@ -1,225 +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});
- }
-}
+#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/config.h b/library/cpp/actors/core/config.h
index 2486bf4c43..750bdd3a07 100644
--- a/library/cpp/actors/core/config.h
+++ b/library/cpp/actors/core/config.h
@@ -1,239 +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();
- }
- };
-
-}
+#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
index 39089b5d83..26aca975e9 100644
--- a/library/cpp/actors/core/cpu_manager.cpp
+++ b/library/cpp/actors/core/cpu_manager.cpp
@@ -1,108 +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));
- }
-}
+#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
index 454035477b..fc67bfd34c 100644
--- a/library/cpp/actors/core/cpu_manager.h
+++ b/library/cpp/actors/core/cpu_manager.h
@@ -1,57 +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);
- };
-}
+#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
index b8030149a7..34febf7ccf 100644
--- a/library/cpp/actors/core/cpu_state.h
+++ b/library/cpp/actors/core/cpu_state.h
@@ -1,215 +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
- }
- };
-
-}
+#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
index 980b7d767b..d331e98833 100644
--- a/library/cpp/actors/core/defs.h
+++ b/library/cpp/actors/core/defs.h
@@ -13,30 +13,30 @@
#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);
+ 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
diff --git a/library/cpp/actors/core/event_pb.h b/library/cpp/actors/core/event_pb.h
index d7546b901a..bf9695da95 100644
--- a/library/cpp/actors/core/event_pb.h
+++ b/library/cpp/actors/core/event_pb.h
@@ -291,10 +291,10 @@ namespace NActors {
}
public:
- void ReservePayload(size_t size) {
- Payload.reserve(size);
- }
-
+ void ReservePayload(size_t size) {
+ Payload.reserve(size);
+ }
+
ui32 AddPayload(TRope&& rope) {
const ui32 id = Payload.size();
Payload.push_back(std::move(rope));
diff --git a/library/cpp/actors/core/executor_pool_base.cpp b/library/cpp/actors/core/executor_pool_base.cpp
index c3b9999168..7340458631 100644
--- a/library/cpp/actors/core/executor_pool_base.cpp
+++ b/library/cpp/actors/core/executor_pool_base.cpp
@@ -1,8 +1,8 @@
#include "executor_pool_base.h"
#include "executor_thread.h"
#include "mailbox.h"
-#include "probes.h"
-#include <library/cpp/actors/util/datetime.h>
+#include "probes.h"
+#include <library/cpp/actors/util/datetime.h>
namespace NActors {
LWTRACE_USING(ACTORLIB_PROVIDER);
@@ -11,44 +11,44 @@ namespace NActors {
actor->SelfActorId = self;
actor->Registered(sys, owner);
}
-
- TExecutorPoolBaseMailboxed::TExecutorPoolBaseMailboxed(ui32 poolId, ui32 maxActivityType)
+
+ 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();
+ {}
+
+ TExecutorPoolBaseMailboxed::~TExecutorPoolBaseMailboxed() {
+ MailboxTable.Destroy();
}
- TExecutorPoolBase::TExecutorPoolBase(ui32 poolId, ui32 threads, TAffinity* affinity, ui32 maxActivityType)
- : TExecutorPoolBaseMailboxed(poolId, maxActivityType)
- , PoolThreads(threads)
- , ThreadsAffinity(affinity)
- {}
-
+ 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);
+ void TExecutorPoolBaseMailboxed::ReclaimMailbox(TMailboxType::EType mailboxType, ui32 hint, TWorkerId workerId, ui64 revolvingWriteCounter) {
+ Y_UNUSED(workerId);
MailboxTable->ReclaimMailbox(mailboxType, hint, revolvingWriteCounter);
}
- ui64 TExecutorPoolBaseMailboxed::AllocateID() {
+ ui64 TExecutorPoolBaseMailboxed::AllocateID() {
return ActorSystem->AllocateIDSpace(1);
}
- bool TExecutorPoolBaseMailboxed::Send(TAutoPtr<IEventHandle>& ev) {
+ bool TExecutorPoolBaseMailboxed::Send(TAutoPtr<IEventHandle>& ev) {
Y_VERIFY_DEBUG(ev->GetRecipientRewrite().PoolID() == PoolId);
#ifdef ACTORSLIB_COLLECT_EXEC_STATS
- RelaxedStore(&ev->SendTime, (::NHPTimer::STime)GetCycleCountFast());
+ RelaxedStore(&ev->SendTime, (::NHPTimer::STime)GetCycleCountFast());
#endif
return MailboxTable->SendTo(ev, this);
}
@@ -57,8 +57,8 @@ namespace NActors {
ScheduleActivationEx(activation, AtomicIncrement(ActivationsRevolvingCounter));
}
- TActorId TExecutorPoolBaseMailboxed::Register(IActor* actor, TMailboxType::EType mailboxType, ui64 revolvingWriteCounter, const TActorId& parentId) {
- NHPTimer::STime hpstart = GetCycleCountFast();
+ 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())
@@ -72,7 +72,7 @@ namespace NActors {
TMailboxHeader* mailbox = nullptr;
if (revolvingWriteCounter == 0)
- revolvingWriteCounter = AtomicIncrement(RegisterRevolvingCounter);
+ revolvingWriteCounter = AtomicIncrement(RegisterRevolvingCounter);
{
ui32 hintBackoff = 0;
@@ -105,34 +105,34 @@ namespace NActors {
switch (mailboxType) {
case TMailboxType::Simple:
- UnlockFromExecution((TMailboxTable::TSimpleMailbox*)mailbox, this, false, hint, MaxWorkers, ++revolvingWriteCounter);
+ UnlockFromExecution((TMailboxTable::TSimpleMailbox*)mailbox, this, false, hint, MaxWorkers, ++revolvingWriteCounter);
break;
case TMailboxType::Revolving:
- UnlockFromExecution((TMailboxTable::TRevolvingMailbox*)mailbox, this, false, hint, MaxWorkers, ++revolvingWriteCounter);
+ UnlockFromExecution((TMailboxTable::TRevolvingMailbox*)mailbox, this, false, hint, MaxWorkers, ++revolvingWriteCounter);
break;
case TMailboxType::HTSwap:
- UnlockFromExecution((TMailboxTable::THTSwapMailbox*)mailbox, this, false, hint, MaxWorkers, ++revolvingWriteCounter);
+ UnlockFromExecution((TMailboxTable::THTSwapMailbox*)mailbox, this, false, hint, MaxWorkers, ++revolvingWriteCounter);
break;
case TMailboxType::ReadAsFilled:
- UnlockFromExecution((TMailboxTable::TReadAsFilledMailbox*)mailbox, this, false, hint, MaxWorkers, ++revolvingWriteCounter);
+ UnlockFromExecution((TMailboxTable::TReadAsFilledMailbox*)mailbox, this, false, hint, MaxWorkers, ++revolvingWriteCounter);
break;
case TMailboxType::TinyReadAsFilled:
- UnlockFromExecution((TMailboxTable::TTinyReadAsFilledMailbox*)mailbox, this, false, hint, MaxWorkers, ++revolvingWriteCounter);
+ UnlockFromExecution((TMailboxTable::TTinyReadAsFilledMailbox*)mailbox, this, false, hint, MaxWorkers, ++revolvingWriteCounter);
break;
default:
Y_FAIL();
}
- NHPTimer::STime elapsed = GetCycleCountFast() - hpstart;
+ 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();
+ }
+
+ 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())
@@ -146,19 +146,19 @@ namespace NActors {
const TActorId actorId(ActorSystem->NodeId, PoolId, localActorId, hint);
DoActorInit(ActorSystem, actor, actorId, parentId);
- NHPTimer::STime elapsed = GetCycleCountFast() - hpstart;
+ 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() {
+ bool TExecutorPoolBaseMailboxed::Cleanup() {
return MailboxTable->Cleanup();
}
diff --git a/library/cpp/actors/core/executor_pool_base.h b/library/cpp/actors/core/executor_pool_base.h
index c84ce1af77..22247ae1c7 100644
--- a/library/cpp/actors/core/executor_pool_base.h
+++ b/library/cpp/actors/core/executor_pool_base.h
@@ -8,7 +8,7 @@
#include <library/cpp/actors/util/threadparkpad.h>
namespace NActors {
- class TExecutorPoolBaseMailboxed: public IExecutorPool {
+ class TExecutorPoolBaseMailboxed: public IExecutorPool {
protected:
TActorSystem* ActorSystem;
THolder<TMailboxTable> MailboxTable;
@@ -17,33 +17,33 @@ namespace NActors {
// registrations might be done in threads from other pools)
TExecutorThreadStats Stats;
#endif
- TAtomic RegisterRevolvingCounter = 0;
- ui64 AllocateID();
+ TAtomic RegisterRevolvingCounter = 0;
+ ui64 AllocateID();
public:
- TExecutorPoolBaseMailboxed(ui32 poolId, ui32 maxActivityType);
- ~TExecutorPoolBaseMailboxed();
- void ReclaimMailbox(TMailboxType::EType mailboxType, ui32 hint, TWorkerId workerId, ui64 revolvingWriteCounter) override;
+ 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;
+ 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&);
+
+ 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
index 4dce16939a..16541889c6 100644
--- a/library/cpp/actors/core/executor_pool_basic.cpp
+++ b/library/cpp/actors/core/executor_pool_basic.cpp
@@ -1,8 +1,8 @@
#include "executor_pool_basic.h"
-#include "probes.h"
+#include "probes.h"
#include "mailbox.h"
#include <library/cpp/actors/util/affinity.h>
-#include <library/cpp/actors/util/datetime.h>
+#include <library/cpp/actors/util/datetime.h>
#ifdef _linux_
#include <pthread.h>
@@ -12,7 +12,7 @@ namespace NActors {
LWTRACE_USING(ACTORLIB_PROVIDER);
constexpr TDuration TBasicExecutorPool::DEFAULT_TIME_PER_MAILBOX;
-
+
TBasicExecutorPool::TBasicExecutorPool(
ui32 poolId,
ui32 threads,
@@ -23,7 +23,7 @@ namespace NActors {
ui32 eventsPerMailbox,
int realtimePriority,
ui32 maxActivityType)
- : TExecutorPoolBase(poolId, threads, affinity, maxActivityType)
+ : TExecutorPoolBase(poolId, threads, affinity, maxActivityType)
, SpinThreshold(spinThreshold)
, SpinThresholdCycles(spinThreshold * NHPTimer::GetCyclesPerSecond() * 0.000001) // convert microseconds to cycles
, Threads(new TThreadCtx[threads])
@@ -38,45 +38,45 @@ namespace NActors {
{
}
- 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(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);
+ 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 hpstart = GetCycleCountFast();
NHPTimer::STime hpnow;
- TThreadCtx& threadCtx = Threads[workerId];
+ 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();
+ hpnow = GetCycleCountFast();
elapsed += hpnow - hpstart;
if (threadCtx.BlockedPad.Park()) // interrupted
return 0;
- hpstart = GetCycleCountFast();
+ hpstart = GetCycleCountFast();
blocked += hpstart - hpnow;
}
} while (AtomicGet(threadCtx.BlockedFlag) != TThreadCtx::BS_NONE && !AtomicLoad(&StopFlag));
@@ -94,7 +94,7 @@ namespace NActors {
// 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 t = GetCycleCountFast();
const i64 x = AtomicGetAndAdd(MaxUtilizationCounter, t);
if (x < 0 && x + t > 0)
AtomicStore(&MaxUtilizationAccumulator, x + t);
@@ -106,14 +106,14 @@ namespace NActors {
if (SpinThreshold > 0) {
// spin configured period
AtomicSet(threadCtx.WaitingFlag, TThreadCtx::WS_ACTIVE);
- ui64 start = GetCycleCountFast();
+ ui64 start = GetCycleCountFast();
bool doSpin = true;
while (true) {
for (ui32 j = 0; doSpin && j < 12; ++j) {
- if (GetCycleCountFast() >= (start + SpinThresholdCycles)) {
- doSpin = false;
- break;
- }
+ if (GetCycleCountFast() >= (start + SpinThresholdCycles)) {
+ doSpin = false;
+ break;
+ }
for (ui32 i = 0; i < 12; ++i) {
if (AtomicLoad(&threadCtx.WaitingFlag) == TThreadCtx::WS_ACTIVE) {
SpinLockPause();
@@ -122,7 +122,7 @@ namespace NActors {
break;
}
}
- }
+ }
if (!doSpin) {
break;
}
@@ -134,23 +134,23 @@ namespace NActors {
if (AtomicLoad(&threadCtx.WaitingFlag) == TThreadCtx::WS_ACTIVE) {
if (AtomicCas(&threadCtx.WaitingFlag, TThreadCtx::WS_BLOCKED, TThreadCtx::WS_ACTIVE)) {
do {
- hpnow = GetCycleCountFast();
+ hpnow = GetCycleCountFast();
elapsed += hpnow - hpstart;
if (threadCtx.Pad.Park()) // interrupted
return 0;
- hpstart = GetCycleCountFast();
+ hpstart = GetCycleCountFast();
parked += hpstart - hpnow;
} while (AtomicLoad(&threadCtx.WaitingFlag) == TThreadCtx::WS_BLOCKED);
}
- }
+ }
} else {
AtomicSet(threadCtx.WaitingFlag, TThreadCtx::WS_BLOCKED);
do {
- hpnow = GetCycleCountFast();
+ hpnow = GetCycleCountFast();
elapsed += hpnow - hpstart;
if (threadCtx.Pad.Park()) // interrupted
return 0;
- hpstart = GetCycleCountFast();
+ hpstart = GetCycleCountFast();
parked += hpstart - hpnow;
} while (AtomicLoad(&threadCtx.WaitingFlag) == TThreadCtx::WS_BLOCKED);
}
@@ -170,7 +170,7 @@ namespace NActors {
// 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 t = GetCycleCountFast();
const i64 x = AtomicGetAndAdd(MaxUtilizationCounter, -t);
if (x > 0 && x - t < 0)
AtomicStore(&MaxUtilizationAccumulator, x);
@@ -183,17 +183,17 @@ namespace NActors {
// ok, has work suggested, must dequeue
while (!RelaxedLoad(&StopFlag)) {
if (const ui32 activation = Activations.Pop(++revolvingCounter)) {
- hpnow = GetCycleCountFast();
+ hpnow = GetCycleCountFast();
elapsed += hpnow - hpstart;
- wctx.AddElapsedCycles(IActor::ACTOR_SYSTEM, elapsed);
+ wctx.AddElapsedCycles(IActor::ACTOR_SYSTEM, elapsed);
if (parked > 0) {
- wctx.AddParkedCycles(parked);
+ wctx.AddParkedCycles(parked);
}
if (blocked > 0) {
- wctx.AddBlockedCycles(blocked);
+ wctx.AddBlockedCycles(blocked);
}
return activation;
- }
+ }
SpinLockPause();
}
@@ -201,7 +201,7 @@ namespace NActors {
return 0;
}
- inline void TBasicExecutorPool::WakeUpLoop() {
+ inline void TBasicExecutorPool::WakeUpLoop() {
for (ui32 i = 0;;) {
TThreadCtx& threadCtx = Threads[i % PoolThreads];
switch (AtomicLoad(&threadCtx.WaitingFlag)) {
@@ -230,7 +230,7 @@ namespace NActors {
Activations.Push(activation, revolvingCounter);
const TAtomic x = AtomicIncrement(Semaphore);
if (x <= 0) { // we must find someone to wake-up
- WakeUpLoop();
+ WakeUpLoop();
}
}
@@ -247,7 +247,7 @@ namespace NActors {
}
}
- void TBasicExecutorPool::Prepare(TActorSystem* actorSystem, NSchedulerQueue::TReader** scheduleReaders, ui32* scheduleSz) {
+ void TBasicExecutorPool::Prepare(TActorSystem* actorSystem, NSchedulerQueue::TReader** scheduleReaders, ui32* scheduleSz) {
TAffinityGuard affinityGuard(Affinity());
ActorSystem = actorSystem;
@@ -259,7 +259,7 @@ namespace NActors {
Threads[i].Thread.Reset(
new TExecutorThread(
i,
- 0, // CpuId is not used in BASIC pool
+ 0, // CpuId is not used in BASIC pool
actorSystem,
this,
MailboxTable.Get(),
@@ -277,7 +277,7 @@ namespace NActors {
TAffinityGuard affinityGuard(Affinity());
ThreadUtilization = 0;
- AtomicAdd(MaxUtilizationCounter, -(i64)GetCycleCountFast());
+ AtomicAdd(MaxUtilizationCounter, -(i64)GetCycleCountFast());
for (ui32 i = 0; i != PoolThreads; ++i) {
Threads[i].Thread->Start();
@@ -297,27 +297,27 @@ namespace NActors {
Threads[i].Thread->Join();
}
- void TBasicExecutorPool::Schedule(TInstant deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) {
- Y_VERIFY_DEBUG(workerId < PoolThreads);
+ void TBasicExecutorPool::Schedule(TInstant deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) {
+ Y_VERIFY_DEBUG(workerId < PoolThreads);
- Schedule(deadline - ActorSystem->Timestamp(), ev, cookie, workerId);
+ Schedule(deadline - ActorSystem->Timestamp(), ev, cookie, workerId);
}
- void TBasicExecutorPool::Schedule(TMonotonic deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) {
- Y_VERIFY_DEBUG(workerId < PoolThreads);
+ 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);
+ 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);
+ 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);
+ ScheduleWriters[workerId].Push(deadline.MicroSeconds(), ev.Release(), cookie);
}
void TBasicExecutorPool::SetRealTimeMode() const {
@@ -421,7 +421,7 @@ namespace NActors {
; // other thread woke this sleeping thread
}
// if thread has already been awakened then we must awaken the other
- WakeUpLoop();
+ WakeUpLoop();
}
}
}
diff --git a/library/cpp/actors/core/executor_pool_basic.h b/library/cpp/actors/core/executor_pool_basic.h
index 023190f7fe..83224ed9d3 100644
--- a/library/cpp/actors/core/executor_pool_basic.h
+++ b/library/cpp/actors/core/executor_pool_basic.h
@@ -67,8 +67,8 @@ namespace NActors {
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;
+ 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,
@@ -79,18 +79,18 @@ namespace NActors {
ui32 eventsPerMailbox = DEFAULT_EVENTS_PER_MAILBOX,
int realtimePriority = 0,
ui32 maxActivityType = 1);
- explicit TBasicExecutorPool(const TBasicExecutorPoolConfig& cfg);
+ explicit TBasicExecutorPool(const TBasicExecutorPoolConfig& cfg);
~TBasicExecutorPool();
- ui32 GetReadyActivation(TWorkerContext& wctx, ui64 revolvingReadCounter) override;
+ 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 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 Prepare(TActorSystem* actorSystem, NSchedulerQueue::TReader** scheduleReaders, ui32* scheduleSz) override;
void Start() override;
void PrepareStop() override;
void Shutdown() override;
@@ -106,6 +106,6 @@ namespace NActors {
void SetThreadCount(ui32 threads);
private:
- void WakeUpLoop();
+ void WakeUpLoop();
};
}
diff --git a/library/cpp/actors/core/executor_pool_basic_ut.cpp b/library/cpp/actors/core/executor_pool_basic_ut.cpp
index 76dff693af..be87e5df33 100644
--- a/library/cpp/actors/core/executor_pool_basic_ut.cpp
+++ b/library/cpp/actors/core/executor_pool_basic_ut.cpp
@@ -377,59 +377,59 @@ Y_UNIT_TEST_SUITE(BasicExecutorPool) {
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);
- }
+
+ 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
index fb557ae6b0..29e64220db 100644
--- a/library/cpp/actors/core/executor_pool_io.cpp
+++ b/library/cpp/actors/core/executor_pool_io.cpp
@@ -1,59 +1,59 @@
#include "executor_pool_io.h"
#include "mailbox.h"
#include <library/cpp/actors/util/affinity.h>
-#include <library/cpp/actors/util/datetime.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)
+ 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(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);
+ 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 hpstart = GetCycleCountFast();
NHPTimer::STime hpnow;
const TAtomic x = AtomicDecrement(Semaphore);
if (x < 0) {
- TThreadCtx& threadCtx = Threads[workerId];
- ThreadQueue.Push(workerId + 1, revolvingCounter);
- hpnow = GetCycleCountFast();
+ TThreadCtx& threadCtx = Threads[workerId];
+ ThreadQueue.Push(workerId + 1, revolvingCounter);
+ hpnow = GetCycleCountFast();
elapsed += hpnow - hpstart;
if (threadCtx.Pad.Park())
return 0;
- hpstart = GetCycleCountFast();
+ hpstart = GetCycleCountFast();
parked += hpstart - hpnow;
}
while (!RelaxedLoad(&StopFlag)) {
if (const ui32 activation = Activations.Pop(++revolvingCounter)) {
- hpnow = GetCycleCountFast();
+ hpnow = GetCycleCountFast();
elapsed += hpnow - hpstart;
- wctx.AddElapsedCycles(IActor::ACTOR_SYSTEM, elapsed);
+ wctx.AddElapsedCycles(IActor::ACTOR_SYSTEM, elapsed);
if (parked > 0) {
- wctx.AddParkedCycles(parked);
+ wctx.AddParkedCycles(parked);
}
return activation;
}
@@ -63,12 +63,12 @@ namespace NActors {
return 0;
}
- void TIOExecutorPool::Schedule(TInstant deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) {
- Schedule(deadline - ActorSystem->Timestamp(), ev, cookie, workerId);
+ 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);
+ void TIOExecutorPool::Schedule(TMonotonic deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) {
+ Y_UNUSED(workerId);
const auto current = ActorSystem->Monotonic();
if (deadline < current)
@@ -78,8 +78,8 @@ namespace NActors {
ScheduleQueue->Writer.Push(deadline.MicroSeconds(), ev.Release(), cookie);
}
- void TIOExecutorPool::Schedule(TDuration delta, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie, TWorkerId workerId) {
- Y_UNUSED(workerId);
+ 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);
@@ -101,7 +101,7 @@ namespace NActors {
}
}
- void TIOExecutorPool::Prepare(TActorSystem* actorSystem, NSchedulerQueue::TReader** scheduleReaders, ui32* scheduleSz) {
+ void TIOExecutorPool::Prepare(TActorSystem* actorSystem, NSchedulerQueue::TReader** scheduleReaders, ui32* scheduleSz) {
TAffinityGuard affinityGuard(Affinity());
ActorSystem = actorSystem;
@@ -109,7 +109,7 @@ namespace NActors {
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));
+ Threads[i].Thread.Reset(new TExecutorThread(i, 0, actorSystem, this, MailboxTable.Get(), PoolName));
}
*scheduleReaders = &ScheduleQueue->Reader;
diff --git a/library/cpp/actors/core/executor_pool_io.h b/library/cpp/actors/core/executor_pool_io.h
index e576d642a1..00de73bacd 100644
--- a/library/cpp/actors/core/executor_pool_io.h
+++ b/library/cpp/actors/core/executor_pool_io.h
@@ -25,20 +25,20 @@ namespace NActors {
const TString PoolName;
public:
- TIOExecutorPool(ui32 poolId, ui32 threads, const TString& poolName = "", TAffinity* affinity = nullptr,
+ TIOExecutorPool(ui32 poolId, ui32 threads, const TString& poolName = "", TAffinity* affinity = nullptr,
ui32 maxActivityType = 1);
- explicit TIOExecutorPool(const TIOExecutorPoolConfig& cfg);
+ explicit TIOExecutorPool(const TIOExecutorPoolConfig& cfg);
~TIOExecutorPool();
- ui32 GetReadyActivation(TWorkerContext& wctx, ui64 revolvingCounter) override;
+ 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 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 Prepare(TActorSystem* actorSystem, NSchedulerQueue::TReader** scheduleReaders, ui32* scheduleSz) override;
void Start() override;
void PrepareStop() override;
void Shutdown() override;
diff --git a/library/cpp/actors/core/executor_pool_united.cpp b/library/cpp/actors/core/executor_pool_united.cpp
index dac6245635..d932714aa9 100644
--- a/library/cpp/actors/core/executor_pool_united.cpp
+++ b/library/cpp/actors/core/executor_pool_united.cpp
@@ -1,1428 +1,1428 @@
-#include "executor_pool_united.h"
-
-#include "balancer.h"
-#include "cpu_state.h"
-#include "executor_thread.h"
-#include "probes.h"
+#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 "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 <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 <algorithm>
+#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);
+ 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);
}
- 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();
+ 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();
}
- }
- } 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();
+ SetPriority(0, IdlePriority);
+ idleTimer->Wait();
break;
- case Stopped: return false;
- default: Y_FAIL();
+ 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();
+ } 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;
+ case BecameIdle:
+ wctx.Lease = wctx.Lease.NeverExpire();
+ wctx.PoolId = MaxPools;
+ idleTimer = nullptr;
break;
- case Stopped: return false;
- default: Y_FAIL();
+ 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
+ }
+ }
+
+ 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
+ }
}
- }
- }
- }
-
- 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 { // 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);
+ // 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
}
- }
- }
-
- 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
+ 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
}
- }
- }
-
- 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
+ }
+ }
+
+ 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 { // 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 {
+ 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
}
- } 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());
- }
+ }
+ }
+ 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);
+ }
+ }
+ }
}
- inline TThreadId TUnitedWorkers::GetWorkerThreadId(TWorkerId workerId) const {
- volatile TThreadId* threadId = &Workers[workerId].ThreadId;
-#ifdef _linux_
- while (AtomicLoad(threadId) == UnknownThreadId) {
- NanoSleep(1000);
+ 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);
+ }
+ }
}
-#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() {
+ 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);
+ for (TPoolId pool = 0; pool < PoolCount; pool++) {
+ Pools[pool].Stop();
}
+ for (TCpuId cpuId = 0; cpuId < CpuCount; cpuId++) {
+ Cpus[cpuId].Stop();
+ }
}
- inline bool TUnitedWorkers::TryAcquireToken(TPoolId pool) {
- return Pools[pool].TryAcquireToken();
+ void TUnitedWorkers::Shutdown() {
+ for (TWorkerId workerId = 0; workerId < WorkerCount; workerId++) {
+ Workers[workerId].Thread->Join();
+ }
}
- 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
- }
- }
- }
+ inline void TUnitedWorkers::PushActivation(TPoolId pool, ui32 activation, ui64 revolvingCounter) {
+ if (Pools[pool].PushActivation(activation, revolvingCounter)) { // token generated
+ TryWake(pool);
+ }
+ }
- // Cpu has not been woken up
+ inline bool TUnitedWorkers::TryAcquireToken(TPoolId pool) {
+ return Pools[pool].TryAcquireToken();
}
- inline void TUnitedWorkers::BeginExecution(TPoolId pool, ui32& activation, ui64 revolvingCounter) {
- Pools[pool].BeginExecution(activation, revolvingCounter);
+ 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 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::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 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;
}
- inline TPoolId TUnitedWorkers::AssignedPool(TWorkerContext& wctx) {
- return Cpus[wctx.CpuId].State.AssignedPool();
+ 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;
}
- 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);
+ 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
index a090ba2466..8f5cabe365 100644
--- a/library/cpp/actors/core/executor_pool_united.h
+++ b/library/cpp/actors/core/executor_pool_united.h
@@ -1,133 +1,133 @@
#pragma once
#include "actorsystem.h"
-#include "balancer.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>
-
+#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;
+ 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;
-
+ 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
index d4df17f1b8..97e1f91557 100644
--- a/library/cpp/actors/core/executor_pool_united_ut.cpp
+++ b/library/cpp/actors/core/executor_pool_united_ut.cpp
@@ -18,18 +18,18 @@ struct TEvMsg : public NActors::TEventBase<TEvMsg, 10347> {
////////////////////////////////////////////////////////////////////////////////
-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;
-}
-
+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 ;
@@ -48,7 +48,7 @@ public:
, Action(action)
{}
- void Start(TActorId receiver, size_t count) {
+ void Start(TActorId receiver, size_t count) {
AtomicSet(Counter, count);
Receiver = receiver;
}
@@ -68,14 +68,14 @@ public:
}
private:
- STFUNC(Execute) {
+ STFUNC(Execute) {
Y_UNUSED(ctx);
switch (ev->GetTypeRewrite()) {
hFunc(TEvMsg, Handle);
}
}
- void Handle(TEvMsg::TPtr &ev) {
+ void Handle(TEvMsg::TPtr &ev) {
Y_UNUSED(ev);
Action();
TAtomicBase count = AtomicDecrement(Counter);
@@ -86,57 +86,57 @@ private:
}
};
-// 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) {
+// 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->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) {
+Y_UNIT_TEST_SUITE(UnitedExecutorPool) {
-#ifdef _linux_
+#ifdef _linux_
- Y_UNIT_TEST(OnePoolManyCpus) {
+ Y_UNIT_TEST(OnePoolManyCpus) {
const size_t msgCount = 1e4;
- auto setup = GetActorSystemSetup(4);
- AddUnitedPool(setup);
+ auto setup = GetActorSystemSetup(4);
+ AddUnitedPool(setup);
TActorSystem actorSystem(setup);
actorSystem.Start();
@@ -153,186 +153,186 @@ Y_UNIT_TEST_SUITE(UnitedExecutorPool) {
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);
+
+ 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) {
+ 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);
- }
+ 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);
+ 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();
+ size_t left = 0;
+ for (auto actor : actors) {
+ left += actor->GetCounter();
}
- if (left == 0) {
+ if (left == 0) {
break;
}
auto now = TInstant::Now();
- UNIT_ASSERT_C(now - begin < TDuration::Seconds(5), "left " << left);
+ 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);
- }
+
+ 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);
- }
+ 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();
- auto actorId = actorSystem.Register(actor, TMailboxType::HTSwap, pool);
- actor->Start(actor->SelfId(), msgCount);
- actorSystem.Send(actorId, new TEvMsg());
- actors.push_back(actor);
+ 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();
+ size_t left = 0;
+ for (auto actor : actors) {
+ left += actor->GetCounter();
}
- if (left == 0) {
+ if (left == 0) {
break;
}
auto now = TInstant::Now();
- UNIT_ASSERT_C(now - begin < TDuration::Seconds(5), "left " << left);
+ 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].PoolActorRegistrations, 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);
}
- }
-
- 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
+#endif
}
diff --git a/library/cpp/actors/core/executor_thread.cpp b/library/cpp/actors/core/executor_thread.cpp
index 446b651efd..1fd93f1a42 100644
--- a/library/cpp/actors/core/executor_thread.cpp
+++ b/library/cpp/actors/core/executor_thread.cpp
@@ -7,18 +7,18 @@
#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/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
-
+#ifdef _linux_
+#include <sys/syscall.h>
+#include <unistd.h>
+#endif
+
#include <util/system/type_name.h>
#include <util/system/datetime.h>
@@ -28,42 +28,42 @@ 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)
+ 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)
+ , 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);
- }
+ {
+ 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);
+ 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);
+ 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());
+ Ctx.DecrementActorsAliveByActivity(actor->GetActivityType());
DyingActors.push_back(THolder(actor));
}
@@ -73,17 +73,17 @@ namespace NActors {
void TExecutorThread::Schedule(TInstant deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie* cookie) {
++CurrentActorScheduledEventsCounter;
- Ctx.Executor->Schedule(deadline, ev, cookie, Ctx.WorkerId);
+ 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);
+ 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);
+ Ctx.Executor->Schedule(delta, ev, cookie, Ctx.WorkerId);
}
template <class T>
@@ -96,7 +96,7 @@ namespace NActors {
} catch (...) {
return "unknown-type";
}
- }
+ }
inline TString ActorTypeName(const IActor* actor, ui32 activityType) {
return actor ? SafeTypeName(actor) : ("activityType_" + ToString(activityType) + " (destroyed)");
@@ -113,8 +113,8 @@ namespace NActors {
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();
@@ -126,25 +126,25 @@ namespace NActors {
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 hpstart = GetCycleCountFast();
NHPTimer::STime hpprev = hpstart;
- IActor* actor = nullptr;
+ IActor* actor = nullptr;
ui32 prevActivityType = std::numeric_limits<ui32>::max();
- TActorId recipient;
- for (ui32 executed = 0; executed < Ctx.EventsPerMailbox; ++executed) {
+ 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())) {
+ recipient = ev->GetRecipientRewrite();
+ if (actor = mailbox->FindActor(recipient.LocalId())) {
TActorContext ctx(*mailbox, *this, hpprev, recipient);
TlsActivationContext = &ctx;
@@ -156,16 +156,16 @@ namespace NActors {
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);
+ 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);
+ 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);
+ LwTraceSlowDelivery(ev.Get(), actor, Ctx.PoolId, CurrentRecipient, NHPTimer::GetSeconds(hpprev - ev->SendTime) * 1000.0, sinceActivationMs, executed);
}
ui32 evTypeForTracing = ev->Type;
@@ -179,7 +179,7 @@ namespace NActors {
actor->Receive(ev, ctx);
size_t dyingActorsCnt = DyingActors.size();
- Ctx.UpdateActorsStats(dyingActorsCnt);
+ Ctx.UpdateActorsStats(dyingActorsCnt);
if (dyingActorsCnt) {
DropUnregistered();
actor = nullptr;
@@ -188,11 +188,11 @@ namespace NActors {
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);
- }
+ 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)
@@ -204,99 +204,99 @@ namespace NActors {
if (nonDelivered.Get()) {
ActorSystem->Send(nonDelivered);
} else {
- Ctx.IncrementNonDeliveredEvents();
+ Ctx.IncrementNonDeliveredEvents();
}
- hpnow = GetCycleCountFast();
+ 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;
- }
-
+ // 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));
+ 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));
+ 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));
+ 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;
+ 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 _linux_
+ pid_t tid = syscall(SYS_gettid);
+ AtomicSet(ThreadId, (ui64)tid);
+#endif
+
#ifdef BALLOC
ThreadDisableBalloc();
#endif
-
+
if (ThreadName) {
::SetCurrentThreadName(ThreadName);
}
@@ -304,20 +304,20 @@ namespace NActors {
ExecutorPool->SetRealTimeMode();
TAffinityGuard affinity(ExecutorPool->Affinity());
- NHPTimer::STime hpnow = GetCycleCountFast();
+ 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);
+ 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 (TMailboxHeader* header = Ctx.MailboxTable->Get(activation)) {
if (header->LockForExecution()) {
- hpnow = GetCycleCountFast();
+ hpnow = GetCycleCountFast();
nonExecCycles += hpnow - hpprev;
hpprev = hpnow;
switch (header->Type) {
@@ -337,27 +337,27 @@ namespace NActors {
Execute(static_cast<TMailboxTable::TTinyReadAsFilledMailbox*>(header), activation);
break;
}
- hpnow = GetCycleCountFast();
+ 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,
+ 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();
+ 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
+ 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;
}
diff --git a/library/cpp/actors/core/executor_thread.h b/library/cpp/actors/core/executor_thread.h
index 9d3c573f0d..5599c82e58 100644
--- a/library/cpp/actors/core/executor_thread.h
+++ b/library/cpp/actors/core/executor_thread.h
@@ -5,23 +5,23 @@
#include "actor.h"
#include "actorsystem.h"
#include "callstack.h"
-#include "probes.h"
-#include "worker_context.h"
-
-#include <library/cpp/actors/util/datetime.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,
+ TExecutorThread(TWorkerId workerId,
+ TWorkerId cpuId,
TActorSystem* actorSystem,
IExecutorPool* executorPool,
TMailboxTable* mailboxTable,
@@ -29,16 +29,16 @@ namespace NActors {
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)
- {}
-
+ 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());
@@ -50,22 +50,22 @@ namespace NActors {
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();
+ bool Send(TAutoPtr<IEventHandle> ev) {
+#ifdef USE_ACTOR_CALLSTACK
+ ev->Callstack = TCallstack::GetTlsCallstack();
+ ev->Callstack.Trace();
#endif
- Ctx.IncrementSentEvents();
- return ActorSystem->Send(ev);
+ Ctx.IncrementSentEvents();
+ return ActorSystem->Send(ev);
}
- void GetCurrentStats(TExecutorThreadStats& statsCopy) const {
- Ctx.GetCurrentStats(statsCopy);
+ void GetCurrentStats(TExecutorThreadStats& statsCopy) const {
+ Ctx.GetCurrentStats(statsCopy);
}
- TThreadId GetThreadId() const; // blocks, must be called after Start()
- TWorkerId GetWorkerId() const { return Ctx.WorkerId; }
-
+ TThreadId GetThreadId() const; // blocks, must be called after Start()
+ TWorkerId GetWorkerId() const { return Ctx.WorkerId; }
+
private:
void* ThreadProc();
@@ -76,37 +76,37 @@ namespace NActors {
TActorSystem* const ActorSystem;
private:
- // Pool-specific
+ // Pool-specific
IExecutorPool* const ExecutorPool;
- // Event-specific (currently executing)
+ // Event-specific (currently executing)
TVector<THolder<IActor>> DyingActors;
TActorId CurrentRecipient;
- ui64 CurrentActorScheduledEventsCounter = 0;
+ ui64 CurrentActorScheduledEventsCounter = 0;
- // Thread-specific
- TWorkerContext Ctx;
- ui64 RevolvingReadCounter = 0;
- ui64 RevolvingWriteCounter = 0;
+ // Thread-specific
+ TWorkerContext Ctx;
+ ui64 RevolvingReadCounter = 0;
+ ui64 RevolvingWriteCounter = 0;
const TString ThreadName;
- volatile TThreadId ThreadId = UnknownThreadId;
+ volatile TThreadId ThreadId = UnknownThreadId;
};
template <typename TMailbox>
- void UnlockFromExecution(TMailbox* mailbox, IExecutorPool* executorPool, bool asFree, ui32 hint, TWorkerId workerId, ui64& revolvingWriteCounter) {
+ 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());
+ RelaxedStore<NHPTimer::STime>(&mailbox->ScheduleMoment, GetCycleCountFast());
executorPool->ScheduleActivationEx(hint, ++revolvingWriteCounter);
}
} else {
if (mailbox->UnlockAsFree(needReschedule1)) {
- RelaxedStore<NHPTimer::STime>(&mailbox->ScheduleMoment, GetCycleCountFast());
+ RelaxedStore<NHPTimer::STime>(&mailbox->ScheduleMoment, GetCycleCountFast());
executorPool->ScheduleActivationEx(hint, ++revolvingWriteCounter);
}
- executorPool->ReclaimMailbox(TMailbox::MailboxType, hint, workerId, ++revolvingWriteCounter);
+ executorPool->ReclaimMailbox(TMailbox::MailboxType, hint, workerId, ++revolvingWriteCounter);
}
}
}
diff --git a/library/cpp/actors/core/lease.h b/library/cpp/actors/core/lease.h
index 650ae7b122..3bde281960 100644
--- a/library/cpp/actors/core/lease.h
+++ b/library/cpp/actors/core/lease.h
@@ -1,56 +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);
-}
+#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
index 5f63b5af58..c05a32f020 100644
--- a/library/cpp/actors/core/log.cpp
+++ b/library/cpp/actors/core/log.cpp
@@ -311,13 +311,13 @@ namespace NActors {
void TLoggerActor::RenderComponentPriorities(IOutputStream& str) {
using namespace NLog;
- HTML(str) {
+ HTML(str) {
H4() {
str << "Priority Settings for the Components";
}
- TABLE_SORTABLE_CLASS("table") {
- TABLEHEAD() {
- TABLER() {
+ TABLE_SORTABLE_CLASS("table") {
+ TABLEHEAD() {
+ TABLER() {
TABLEH() {
str << "Component";
}
@@ -330,16 +330,16 @@ namespace NActors {
TABLEH() {
str << "Sampling Rate";
}
- }
- }
- TABLEBODY() {
+ }
+ }
+ 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() {
+ TABLER() {
TABLED() {
str << "<a href='logger?c=" << i << "'>" << name << "</a>";
}
@@ -352,11 +352,11 @@ namespace NActors {
TABLED() {
str << componentSettings.Raw.X.SamplingRate;
}
- }
+ }
}
- }
- }
- }
+ }
+ }
+ }
}
/*
@@ -412,53 +412,53 @@ namespace NActors {
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") {
+ HTML(str) {
+ DIV_CLASS("row") {
+ DIV_CLASS("col-md-12") {
H4() {
str << "Current log settings for " << Settings->ComponentName(component) << Endl;
}
- UL() {
- LI() {
+ UL() {
+ LI() {
str << "Priority: "
<< NLog::PriorityToString(NLog::EPrio(componentSettings.Raw.X.Level));
- }
- LI() {
+ }
+ LI() {
str << "Sampling priority: "
<< NLog::PriorityToString(NLog::EPrio(componentSettings.Raw.X.SamplingLevel));
- }
- LI() {
+ }
+ LI() {
str << "Sampling rate: "
<< samplingRate;
- }
- }
- }
- }
+ }
+ }
+ }
+ }
- DIV_CLASS("row") {
- DIV_CLASS("col-md-12") {
+ DIV_CLASS("row") {
+ DIV_CLASS("col-md-12") {
H4() {
str << "Change priority" << Endl;
}
- UL() {
+ UL() {
for (int p = NLog::PRI_EMERG; p <= NLog::PRI_TRACE; ++p) {
- LI() {
+ LI() {
str << "<a href='logger?c=" << component << "&p=" << p << "'>"
<< NLog::PriorityToString(NLog::EPrio(p)) << "</a>";
- }
+ }
}
- }
+ }
H4() {
str << "Change sampling priority" << Endl;
}
- UL() {
+ UL() {
for (int p = NLog::PRI_EMERG; p <= NLog::PRI_TRACE; ++p) {
- LI() {
+ LI() {
str << "<a href='logger?c=" << component << "&sp=" << p << "'>"
<< NLog::PriorityToString(NLog::EPrio(p)) << "</a>";
- }
+ }
}
- }
+ }
H4() {
str << "Change sampling rate" << Endl;
}
@@ -470,9 +470,9 @@ namespace NActors {
H4() {
str << "<a href='logger'>Cancel</a>" << Endl;
}
- }
- }
- }
+ }
+ }
+ }
} else {
TString explanation;
@@ -489,64 +489,64 @@ namespace NActors {
Settings->SetAllowDrop(allowDrop);
}
- HTML(str) {
+ HTML(str) {
if (!explanation.empty()) {
- DIV_CLASS("row") {
+ DIV_CLASS("row") {
DIV_CLASS("col-md-12 alert alert-info") {
str << explanation;
}
- }
+ }
}
- DIV_CLASS("row") {
- DIV_CLASS("col-md-6") {
+ 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() {
+ TABLE_CLASS("table table-condensed") {
+ TABLEHEAD() {
+ TABLER() {
TABLEH() {
str << "Priority";
}
- }
- }
- TABLEBODY() {
+ }
+ }
+ TABLEBODY() {
for (int p = NLog::PRI_EMERG; p <= NLog::PRI_TRACE; ++p) {
- TABLER() {
+ 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() {
+ TABLE_CLASS("table table-condensed") {
+ TABLEHEAD() {
+ TABLER() {
TABLEH() {
str << "Priority";
}
- }
- }
- TABLEBODY() {
+ }
+ }
+ TABLEBODY() {
for (int p = NLog::PRI_EMERG; p <= NLog::PRI_TRACE; ++p) {
- TABLER() {
+ TABLER() {
TABLED() {
str << "<a href = 'logger?c=-1&sp=" << p << "'>"
<< NLog::PriorityToString(NLog::EPrio(p)) << "</a>";
- }
- }
+ }
+ }
}
- }
- }
+ }
+ }
H4() {
str << "Change sampling rate for all components";
}
@@ -563,27 +563,27 @@ namespace NActors {
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;
-
+ 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];
+ char buf[TimeBufSize];
switch (Settings->Format) {
case NActors::NLog::TSettings::PLAIN_FULL_FORMAT: {
TStringBuilder logRecord;
if (Settings->UseLocalTimestamps) {
- logRecord << FormatLocalTimestamp(time, buf);
+ logRecord << FormatLocalTimestamp(time, buf);
} else {
logRecord << time;
}
@@ -649,12 +649,12 @@ namespace NActors {
Become(&TThis::StateFunc);
}
- const char* TLoggerActor::FormatLocalTimestamp(TInstant time, char* buf) {
+ 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;
+ int r = strftime(buf, TimeBufSize, "%Y-%m-%d-%H-%M-%S", &localTime);
+ Y_VERIFY(r != 0);
+ return buf;
}
TAutoPtr<TLogBackend> CreateSysLogBackend(const TString& ident,
diff --git a/library/cpp/actors/core/log.h b/library/cpp/actors/core/log.h
index c11a7cf3c1..32a5abfed2 100644
--- a/library/cpp/actors/core/log.h
+++ b/library/cpp/actors/core/log.h
@@ -255,7 +255,7 @@ namespace NActors {
void RenderComponentPriorities(IOutputStream& str);
void LogIgnoredCount(TInstant now);
void WriteMessageStat(const NLog::TEvLog& ev);
- static const char* FormatLocalTimestamp(TInstant time, char* buf);
+ static const char* FormatLocalTimestamp(TInstant time, char* buf);
};
////////////////////////////////////////////////////////////////////////////////
diff --git a/library/cpp/actors/core/mailbox.cpp b/library/cpp/actors/core/mailbox.cpp
index d84b4f9e46..92763dbb1e 100644
--- a/library/cpp/actors/core/mailbox.cpp
+++ b/library/cpp/actors/core/mailbox.cpp
@@ -1,8 +1,8 @@
#include "mailbox.h"
#include "actorsystem.h"
-#include <library/cpp/actors/util/datetime.h>
-
+#include <library/cpp/actors/util/datetime.h>
+
#include <util/system/sanitizers.h>
namespace NActors {
@@ -183,7 +183,7 @@ namespace NActors {
#endif
mailbox->Queue.Push(ev.Release());
if (mailbox->MarkForSchedule()) {
- RelaxedStore<NHPTimer::STime>(&mailbox->ScheduleMoment, GetCycleCountFast());
+ RelaxedStore<NHPTimer::STime>(&mailbox->ScheduleMoment, GetCycleCountFast());
executorPool->ScheduleActivation(hint);
}
}
@@ -207,7 +207,7 @@ namespace NActors {
#endif
mailbox->QueueWriter.Push(ev.Release());
if (mailbox->MarkForSchedule()) {
- RelaxedStore<NHPTimer::STime>(&mailbox->ScheduleMoment, GetCycleCountFast());
+ RelaxedStore<NHPTimer::STime>(&mailbox->ScheduleMoment, GetCycleCountFast());
executorPool->ScheduleActivation(hint);
}
}
@@ -219,7 +219,7 @@ namespace NActors {
#endif
mailbox->Queue.Push(ev.Release());
if (mailbox->MarkForSchedule()) {
- RelaxedStore<NHPTimer::STime>(&mailbox->ScheduleMoment, GetCycleCountFast());
+ RelaxedStore<NHPTimer::STime>(&mailbox->ScheduleMoment, GetCycleCountFast());
executorPool->ScheduleActivation(hint);
}
}
@@ -234,7 +234,7 @@ namespace NActors {
#endif
mailbox->Queue.Push(ev.Release());
if (mailbox->MarkForSchedule()) {
- RelaxedStore<NHPTimer::STime>(&mailbox->ScheduleMoment, GetCycleCountFast());
+ RelaxedStore<NHPTimer::STime>(&mailbox->ScheduleMoment, GetCycleCountFast());
executorPool->ScheduleActivation(hint);
}
}
@@ -249,7 +249,7 @@ namespace NActors {
#endif
mailbox->Queue.Push(ev.Release());
if (mailbox->MarkForSchedule()) {
- RelaxedStore<NHPTimer::STime>(&mailbox->ScheduleMoment, GetCycleCountFast());
+ RelaxedStore<NHPTimer::STime>(&mailbox->ScheduleMoment, GetCycleCountFast());
executorPool->ScheduleActivation(hint);
}
}
@@ -347,7 +347,7 @@ namespace NActors {
TMailboxHeader::TMailboxHeader(TMailboxType::EType type)
: ExecutionState(TExecutionState::Free)
- , Reserved(0)
+ , Reserved(0)
, Type(type)
, ActorPack(TMailboxActorPack::Simple)
, Knobs(0)
diff --git a/library/cpp/actors/core/mailbox.h b/library/cpp/actors/core/mailbox.h
index 0bd9c4d314..4022915e1d 100644
--- a/library/cpp/actors/core/mailbox.h
+++ b/library/cpp/actors/core/mailbox.h
@@ -54,7 +54,7 @@ namespace NActors {
};
volatile ui32 ExecutionState;
- ui32 Reserved : 4; // never changes, always zero
+ ui32 Reserved : 4; // never changes, always zero
ui32 Type : 4; // never changes
ui32 ActorPack : 2;
ui32 Knobs : 22;
diff --git a/library/cpp/actors/core/mon_stats.h b/library/cpp/actors/core/mon_stats.h
index d55552af0c..e2db1e32e6 100644
--- a/library/cpp/actors/core/mon_stats.h
+++ b/library/cpp/actors/core/mon_stats.h
@@ -63,15 +63,15 @@ namespace NActors {
};
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;
+ 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;
@@ -80,19 +80,19 @@ namespace NActors {
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;
+ 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)
+ : ElapsedTicksByActivity(activityVecSize)
, ReceivedEventsByActivity(activityVecSize)
, ActorsAliveByActivity(activityVecSize)
, ScheduledEventsByActivity(activityVecSize)
- {}
+ {}
template <typename T>
static void AggregateOne(TVector<T>& self, const TVector<T>& other) {
@@ -107,14 +107,14 @@ namespace NActors {
void Aggregate(const TExecutorThreadStats& other) {
SentEvents += RelaxedLoad(&other.SentEvents);
ReceivedEvents += RelaxedLoad(&other.ReceivedEvents);
- PreemptedEvents += RelaxedLoad(&other.PreemptedEvents);
+ PreemptedEvents += RelaxedLoad(&other.PreemptedEvents);
NonDeliveredEvents += RelaxedLoad(&other.NonDeliveredEvents);
EmptyMailboxActivation += RelaxedLoad(&other.EmptyMailboxActivation);
- CpuNs += RelaxedLoad(&other.CpuNs);
+ CpuNs += RelaxedLoad(&other.CpuNs);
ElapsedTicks += RelaxedLoad(&other.ElapsedTicks);
ParkedTicks += RelaxedLoad(&other.ParkedTicks);
BlockedTicks += RelaxedLoad(&other.BlockedTicks);
- MailboxPushedOutBySoftPreemption += RelaxedLoad(&other.MailboxPushedOutBySoftPreemption);
+ MailboxPushedOutBySoftPreemption += RelaxedLoad(&other.MailboxPushedOutBySoftPreemption);
MailboxPushedOutByTime += RelaxedLoad(&other.MailboxPushedOutByTime);
MailboxPushedOutByEventCount += RelaxedLoad(&other.MailboxPushedOutByEventCount);
diff --git a/library/cpp/actors/core/probes.cpp b/library/cpp/actors/core/probes.cpp
index 7ace83e102..0a9b256f70 100644
--- a/library/cpp/actors/core/probes.cpp
+++ b/library/cpp/actors/core/probes.cpp
@@ -1,28 +1,28 @@
#include "probes.h"
-#include "actorsystem.h"
-
-#include <util/string/builder.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;
- }
-}
+
+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
index 4912d6dd26..0733cd010d 100644
--- a/library/cpp/actors/core/probes.h
+++ b/library/cpp/actors/core/probes.h
@@ -1,12 +1,12 @@
#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"
+#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), \
@@ -23,27 +23,27 @@
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(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")) \
+ TYPES(ui32, TString, ui64, ui64, ui64, double, double), \
+ NAMES("poolId", "pool", "workerId", "execCount", "readyActivationCount", "execMs", "nonExecMs")) \
PROBE(SlowICReadLoopAdjustSize, GROUPS("ActorLibSlowIC"), \
TYPES(double), \
NAMES("icReadLoopAdjustSizeMs")) \
@@ -75,102 +75,102 @@
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")) \
+ 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);
-}
+
+namespace NActors {
+ struct TActorSystemSetup;
+ TVector<NLWTrace::TDashboard> LWTraceDashboards(TActorSystemSetup* setup);
+}
diff --git a/library/cpp/actors/core/scheduler_actor.cpp b/library/cpp/actors/core/scheduler_actor.cpp
index febc5e40dd..7e38a41e30 100644
--- a/library/cpp/actors/core/scheduler_actor.cpp
+++ b/library/cpp/actors/core/scheduler_actor.cpp
@@ -116,11 +116,11 @@ namespace NActors {
void TryUpdateTime(NHPTimer::STime* lastTimeUpdate) {
NHPTimer::STime hpnow;
- GetTimeFast(&hpnow);
+ GetTimeFast(&hpnow);
const ui64 elapsedCycles = hpnow > *lastTimeUpdate ? hpnow - *lastTimeUpdate : 0;
if (elapsedCycles > Cfg.ResolutionMicroseconds * (NHPTimer::GetCyclesPerSecond() / IntrasecondThreshold)) {
UpdateTime();
- GetTimeFast(lastTimeUpdate);
+ GetTimeFast(lastTimeUpdate);
}
}
diff --git a/library/cpp/actors/core/scheduler_basic.cpp b/library/cpp/actors/core/scheduler_basic.cpp
index fba200e16b..715cfb787d 100644
--- a/library/cpp/actors/core/scheduler_basic.cpp
+++ b/library/cpp/actors/core/scheduler_basic.cpp
@@ -1,9 +1,9 @@
#include "scheduler_basic.h"
#include "scheduler_queue.h"
-#include <library/cpp/actors/util/datetime.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
@@ -61,7 +61,7 @@ namespace NActors {
ui64 activeTick = AlignUp<ui64>(throttledMonotonic, IntrasecondThreshold);
TAutoPtr<TMomentMap> activeSec;
- NHPTimer::STime hpprev = GetCycleCountFast();
+ NHPTimer::STime hpprev = GetCycleCountFast();
ui64 nextTimestamp = TInstant::Now().MicroSeconds();
ui64 nextMonotonic = Max(currentMonotonic, GetMonotonicMicroSeconds());
@@ -170,7 +170,7 @@ namespace NActors {
}
}
- NHPTimer::STime hpnow = GetCycleCountFast();
+ NHPTimer::STime hpnow = GetCycleCountFast();
if (MonCounters) {
*MonCounters->QueueSize -= eventsSent + eventsDropped;
@@ -202,7 +202,7 @@ namespace NActors {
NanoSleep(delta * 1000); // ok, looks like we should sleep a bit.
// Don't count sleep in elapsed microseconds
- hpprev = GetCycleCountFast();
+ hpprev = GetCycleCountFast();
nextTimestamp = TInstant::Now().MicroSeconds();
nextMonotonic = Max(currentMonotonic, GetMonotonicMicroSeconds());
}
@@ -218,7 +218,7 @@ namespace NActors {
*CurrentMonotonic = GetMonotonicMicroSeconds();
}
- void TBasicSchedulerThread::PrepareSchedules(NSchedulerQueue::TReader** readers, ui32 scheduleReadersCount) {
+ void TBasicSchedulerThread::PrepareSchedules(NSchedulerQueue::TReader** readers, ui32 scheduleReadersCount) {
Y_VERIFY(scheduleReadersCount > 0);
TotalReaders = scheduleReadersCount;
Readers.Reset(new NSchedulerQueue::TReader*[scheduleReadersCount]);
diff --git a/library/cpp/actors/core/scheduler_basic.h b/library/cpp/actors/core/scheduler_basic.h
index 2ccde39235..16a5b005bd 100644
--- a/library/cpp/actors/core/scheduler_basic.h
+++ b/library/cpp/actors/core/scheduler_basic.h
@@ -42,7 +42,7 @@ namespace NActors {
~TBasicSchedulerThread();
void Prepare(TActorSystem* actorSystem, volatile ui64* currentTimestamp, volatile ui64* currentMonotonic) override;
- void PrepareSchedules(NSchedulerQueue::TReader** readers, ui32 scheduleReadersCount) override;
+ void PrepareSchedules(NSchedulerQueue::TReader** readers, ui32 scheduleReadersCount) override;
void PrepareStart() override;
void Start() override;
@@ -61,7 +61,7 @@ namespace NActors {
*currentMonotonic = GetMonotonicMicroSeconds();
}
- void PrepareSchedules(NSchedulerQueue::TReader** readers, ui32 scheduleReadersCount) override {
+ void PrepareSchedules(NSchedulerQueue::TReader** readers, ui32 scheduleReadersCount) override {
Y_UNUSED(readers);
Y_UNUSED(scheduleReadersCount);
}
diff --git a/library/cpp/actors/core/ut/ya.make b/library/cpp/actors/core/ut/ya.make
index 3ee28d5850..64ab864b4a 100644
--- a/library/cpp/actors/core/ut/ya.make
+++ b/library/cpp/actors/core/ut/ya.make
@@ -33,11 +33,11 @@ SRCS(
actor_ut.cpp
actorsystem_ut.cpp
ask_ut.cpp
- balancer_ut.cpp
+ balancer_ut.cpp
event_pb_payload_ut.cpp
event_pb_ut.cpp
executor_pool_basic_ut.cpp
- executor_pool_united_ut.cpp
+ executor_pool_united_ut.cpp
log_ut.cpp
memory_tracker_ut.cpp
scheduler_actor_ut.cpp
diff --git a/library/cpp/actors/core/worker_context.cpp b/library/cpp/actors/core/worker_context.cpp
index ada6c997d4..55b50f4708 100644
--- a/library/cpp/actors/core/worker_context.cpp
+++ b/library/cpp/actors/core/worker_context.cpp
@@ -1,7 +1,7 @@
-#include "worker_context.h"
-#include "probes.h"
-
-namespace NActors {
- LWTRACE_USING(ACTORLIB_PROVIDER);
-
-}
+#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
index b4c37a7629..d4a9a5e6ee 100644
--- a/library/cpp/actors/core/worker_context.h
+++ b/library/cpp/actors/core/worker_context.h
@@ -1,175 +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;
- }
- };
-}
+#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
index 880a9d00db..9cb2e2d396 100644
--- a/library/cpp/actors/core/ya.make
+++ b/library/cpp/actors/core/ya.make
@@ -30,16 +30,16 @@ SRCS(
actorsystem.h
ask.cpp
ask.h
- balancer.h
- balancer.cpp
+ balancer.h
+ balancer.cpp
buffer.cpp
buffer.h
callstack.cpp
callstack.h
- config.h
- cpu_manager.cpp
- cpu_manager.h
- cpu_state.h
+ config.h
+ cpu_manager.cpp
+ cpu_manager.h
+ cpu_state.h
defs.h
event.cpp
event.h
@@ -56,8 +56,8 @@ SRCS(
executor_pool_basic.h
executor_pool_io.cpp
executor_pool_io.h
- executor_pool_united.cpp
- executor_pool_united.h
+ executor_pool_united.cpp
+ executor_pool_united.h
executor_thread.cpp
executor_thread.h
hfunc.h
@@ -66,7 +66,7 @@ SRCS(
invoke.h
io_dispatcher.cpp
io_dispatcher.h
- lease.h
+ lease.h
log.cpp
log.h
log_settings.cpp
@@ -82,9 +82,9 @@ SRCS(
mon.h
mon_stats.h
monotonic.cpp
- monotonic.h
- worker_context.cpp
- worker_context.h
+ monotonic.h
+ worker_context.cpp
+ worker_context.h
probes.cpp
probes.h
process_stats.cpp
diff --git a/library/cpp/actors/dnscachelib/dnscache.cpp b/library/cpp/actors/dnscachelib/dnscache.cpp
index 649339ddb2..026491bcf1 100644
--- a/library/cpp/actors/dnscachelib/dnscache.cpp
+++ b/library/cpp/actors/dnscachelib/dnscache.cpp
@@ -1,5 +1,5 @@
#include "dnscache.h"
-#include "probes.h"
+#include "probes.h"
#include "timekeeper.h"
#include <contrib/libs/c-ares/ares.h>
@@ -8,8 +8,8 @@
const TDnsCache::THost TDnsCache::NullHost;
-LWTRACE_USING(DNSCACHELIB_PROVIDER);
-
+LWTRACE_USING(DNSCACHELIB_PROVIDER);
+
static_assert(sizeof(ares_channel) == sizeof(void*), "expect sizeof(ares_channel) == sizeof(void *)");
TDnsCache::TDnsCache(bool allowIpv4, bool allowIpv6, time_t lifetime, time_t neg, ui32 timeout)
@@ -33,11 +33,11 @@ TDnsCache::TDnsCache(bool allowIpv4, bool allowIpv6, time_t lifetime, time_t neg
ares_channel chan;
if (ares_init(&chan) != ARES_SUCCESS) {
- LWPROBE(AresInitFailed);
+ LWPROBE(AresInitFailed);
ythrow yexception() << "ares_init() failed";
}
Channel = chan;
- LWPROBE(Created);
+ LWPROBE(Created);
}
TDnsCache::~TDnsCache(void) {
@@ -45,7 +45,7 @@ TDnsCache::~TDnsCache(void) {
ares_cancel(chan);
ares_destroy(chan);
- LWPROBE(Destroyed);
+ LWPROBE(Destroyed);
#ifdef _win_
ares_library_cleanup();
@@ -112,7 +112,7 @@ NAddr::IRemoteAddrPtr TDnsCache::GetAddr(
}
}
- LWPROBE(FamilyMismatch, family, AllowIpV4, AllowIpV6);
+ LWPROBE(FamilyMismatch, family, AllowIpV4, AllowIpV6);
return nullptr;
}
@@ -170,7 +170,7 @@ bool TDnsCache::THost::IsStale(int family, const TDnsCache* ctx) const noexcept
const TDnsCache::THost&
TDnsCache::Resolve(const TString& hostname, int family, bool cacheOnly) {
if (!ValidateHName(hostname)) {
- LWPROBE(ResolveNullHost, hostname, family);
+ LWPROBE(ResolveNullHost, hostname, family);
return NullHost;
}
@@ -185,15 +185,15 @@ TDnsCache::Resolve(const TString& hostname, int family, bool cacheOnly) {
if (!p->second.IsStale(family, this)) {
/* Recently resolved, just return cached value */
ACacheHits += 1;
- THost& host = p->second;
- LWPROBE(ResolveFromCache, hostname, family, host.AddrsV4ToString(), host.AddrsV6ToString(), ACacheHits);
- return host;
- } else {
- LWPROBE(ResolveCacheTimeout, hostname);
+ THost& host = p->second;
+ LWPROBE(ResolveFromCache, hostname, family, host.AddrsV4ToString(), host.AddrsV6ToString(), ACacheHits);
+ return host;
+ } else {
+ LWPROBE(ResolveCacheTimeout, hostname);
}
} else {
/* Never resolved, create cache entry */
- LWPROBE(ResolveCacheNew, hostname);
+ LWPROBE(ResolveCacheNew, hostname);
p = HostCache.insert(std::make_pair(hostname, THost())).first;
}
ACacheMisses += 1;
@@ -227,7 +227,7 @@ TDnsCache::Resolve(const TString& hostname, int family, bool cacheOnly) {
WaitTask(inprogress);
- LWPROBE(ResolveDone, hostname, family, p->second.AddrsV4ToString(), p->second.AddrsV6ToString());
+ LWPROBE(ResolveDone, hostname, family, p->second.AddrsV4ToString(), p->second.AddrsV6ToString());
return p->second;
}
@@ -415,31 +415,31 @@ void TDnsCache::GHBACallback(void* arg, int status, int, struct hostent* info) {
}
AtomicSet(p->second.InProgress, 0);
}
-
+
TString TDnsCache::THost::AddrsV4ToString() const {
- TStringStream ss;
- bool first = false;
- for (TIpHost addr : AddrsV4) {
+ TStringStream ss;
+ bool first = false;
+ for (TIpHost addr : AddrsV4) {
ss << (first ? "" : " ") << IpToString(addr);
- first = false;
- }
- return ss.Str();
-}
-
+ first = false;
+ }
+ return ss.Str();
+}
+
TString TDnsCache::THost::AddrsV6ToString() const {
- TStringStream ss;
- bool first = false;
- for (in6_addr addr : AddrsV6) {
- struct sockaddr_in6 sin6;
- Zero(sin6);
- sin6.sin6_family = AF_INET6;
- sin6.sin6_addr = addr;
-
- NAddr::TIPv6Addr addr6(sin6);
+ TStringStream ss;
+ bool first = false;
+ for (in6_addr addr : AddrsV6) {
+ struct sockaddr_in6 sin6;
+ Zero(sin6);
+ sin6.sin6_family = AF_INET6;
+ sin6.sin6_addr = addr;
+
+ NAddr::TIPv6Addr addr6(sin6);
ss << (first ? "" : " ") << NAddr::PrintHost(addr6);
- first = false;
- }
- return ss.Str();
-}
+ first = false;
+ }
+ return ss.Str();
+}
TDnsCache::TAresLibInit TDnsCache::InitAresLib;
diff --git a/library/cpp/actors/dnscachelib/dnscache.h b/library/cpp/actors/dnscachelib/dnscache.h
index 3313a251a1..03eec356d2 100644
--- a/library/cpp/actors/dnscachelib/dnscache.h
+++ b/library/cpp/actors/dnscachelib/dnscache.h
@@ -65,7 +65,7 @@ private:
time_t ResolvedV6 = 0;
time_t NotFoundV6 = 0;
TAtomic InProgressV6 = 0;
-
+
TString AddrsV4ToString() const;
TString AddrsV6ToString() const;
diff --git a/library/cpp/actors/dnscachelib/probes.cpp b/library/cpp/actors/dnscachelib/probes.cpp
index 07734ab20f..441e0609f9 100644
--- a/library/cpp/actors/dnscachelib/probes.cpp
+++ b/library/cpp/actors/dnscachelib/probes.cpp
@@ -1,3 +1,3 @@
-#include "probes.h"
-
-LWTRACE_DEFINE_PROVIDER(DNSCACHELIB_PROVIDER)
+#include "probes.h"
+
+LWTRACE_DEFINE_PROVIDER(DNSCACHELIB_PROVIDER)
diff --git a/library/cpp/actors/dnscachelib/probes.h b/library/cpp/actors/dnscachelib/probes.h
index 313b7b8712..6416e997b7 100644
--- a/library/cpp/actors/dnscachelib/probes.h
+++ b/library/cpp/actors/dnscachelib/probes.h
@@ -1,7 +1,7 @@
-#pragma once
-
+#pragma once
+
#include <library/cpp/lwtrace/all.h>
-
+
#define DNSCACHELIB_PROVIDER(PROBE, EVENT, GROUPS, TYPES, NAMES) \
PROBE(Created, GROUPS(), TYPES(), NAMES()) \
PROBE(Destroyed, GROUPS(), TYPES(), NAMES()) \
@@ -30,6 +30,6 @@
GROUPS(), \
TYPES(TString), \
NAMES("hostname")) \
- /**/
-
-LWTRACE_DECLARE_PROVIDER(DNSCACHELIB_PROVIDER)
+ /**/
+
+LWTRACE_DECLARE_PROVIDER(DNSCACHELIB_PROVIDER)
diff --git a/library/cpp/actors/dnscachelib/ya.make b/library/cpp/actors/dnscachelib/ya.make
index e3a6ad6202..3ee25102c1 100644
--- a/library/cpp/actors/dnscachelib/ya.make
+++ b/library/cpp/actors/dnscachelib/ya.make
@@ -11,8 +11,8 @@ OWNER(
SRCS(
dnscache.cpp
dnscache.h
- probes.cpp
- probes.h
+ probes.cpp
+ probes.h
timekeeper.h
)
diff --git a/library/cpp/actors/helpers/flow_controlled_queue.cpp b/library/cpp/actors/helpers/flow_controlled_queue.cpp
index d75cc54023..46addfa708 100644
--- a/library/cpp/actors/helpers/flow_controlled_queue.cpp
+++ b/library/cpp/actors/helpers/flow_controlled_queue.cpp
@@ -2,7 +2,7 @@
#include <library/cpp/actors/core/interconnect.h>
#include <library/cpp/actors/core/hfunc.h>
-#include <library/cpp/actors/util/datetime.h>
+#include <library/cpp/actors/util/datetime.h>
#include <util/generic/deque.h>
#include <util/datetime/cputimer.h>
@@ -29,7 +29,7 @@ public:
, Source(source)
, Cookie(cookie)
, Flags(flags)
- , StartCounter(GetCycleCountFast())
+ , StartCounter(GetCycleCountFast())
{}
STATEFN(StateWait) {
@@ -41,7 +41,7 @@ public:
}
TDuration AccumulatedLatency() const {
- const ui64 cc = GetCycleCountFast() - StartCounter;
+ const ui64 cc = GetCycleCountFast() - StartCounter;
return CyclesToDuration(cc);
}
diff --git a/library/cpp/actors/interconnect/interconnect_channel.cpp b/library/cpp/actors/interconnect/interconnect_channel.cpp
index a66ba2a154..9aee8fd5a2 100644
--- a/library/cpp/actors/interconnect/interconnect_channel.cpp
+++ b/library/cpp/actors/interconnect/interconnect_channel.cpp
@@ -3,13 +3,13 @@
#include <library/cpp/actors/core/events.h>
#include <library/cpp/actors/core/executor_thread.h>
#include <library/cpp/actors/core/log.h>
-#include <library/cpp/actors/core/probes.h>
+#include <library/cpp/actors/core/probes.h>
#include <library/cpp/actors/protos/services_common.pb.h>
#include <library/cpp/actors/prof/tag.h>
#include <library/cpp/digest/crc32c/crc32c.h>
-LWTRACE_USING(ACTORLIB_PROVIDER);
-
+LWTRACE_USING(ACTORLIB_PROVIDER);
+
namespace NActors {
DECLARE_WILSON_EVENT(EventSentToSocket);
DECLARE_WILSON_EVENT(EventReceivedFromSocket);
@@ -25,8 +25,8 @@ namespace NActors {
// WILSON_TRACE(*ctx, &traceId, EventSentToSocket);
// }
traceId.Serialize(&event.Descr.TraceId);
- LWTRACK(SerializeToPacketEnd, event.Orbit, PeerNodeId, ChannelId, OutputQueueSize, task.GetDataSize());
- task.Orbit.Take(event.Orbit);
+ LWTRACK(SerializeToPacketEnd, event.Orbit, PeerNodeId, ChannelId, OutputQueueSize, task.GetDataSize());
+ task.Orbit.Take(event.Orbit);
event.Descr.Flags = (event.Descr.Flags & ~IEventHandle::FlagForwardOnNondelivery) |
(ExtendedFormat ? IEventHandle::FlagExtendedFormat : 0);
@@ -58,7 +58,7 @@ namespace NActors {
switch (State) {
case EState::INITIAL:
event.InitChecksum();
- LWTRACK(SerializeToPacketBegin, event.Orbit, PeerNodeId, ChannelId, OutputQueueSize);
+ LWTRACK(SerializeToPacketBegin, event.Orbit, PeerNodeId, ChannelId, OutputQueueSize);
if (event.Event) {
State = EState::CHUNKER;
IEventBase *base = event.Event.Get();
diff --git a/library/cpp/actors/interconnect/interconnect_channel.h b/library/cpp/actors/interconnect/interconnect_channel.h
index e4a0ae3cda..a26cb7a92c 100644
--- a/library/cpp/actors/interconnect/interconnect_channel.h
+++ b/library/cpp/actors/interconnect/interconnect_channel.h
@@ -42,11 +42,11 @@ namespace NActors {
class TEventOutputChannel : public TInterconnectLoggingBase {
public:
- TEventOutputChannel(TEventHolderPool& pool, ui16 id, ui32 peerNodeId, ui32 maxSerializedEventSize,
+ TEventOutputChannel(TEventHolderPool& pool, ui16 id, ui32 peerNodeId, ui32 maxSerializedEventSize,
std::shared_ptr<IInterconnectMetrics> metrics, TSessionParams params)
- : TInterconnectLoggingBase(Sprintf("OutputChannel %" PRIu16 " [node %" PRIu32 "]", id, peerNodeId))
+ : TInterconnectLoggingBase(Sprintf("OutputChannel %" PRIu16 " [node %" PRIu32 "]", id, peerNodeId))
, Pool(pool)
- , PeerNodeId(peerNodeId)
+ , PeerNodeId(peerNodeId)
, ChannelId(id)
, Metrics(std::move(metrics))
, Params(std::move(params))
@@ -56,11 +56,11 @@ namespace NActors {
~TEventOutputChannel() {
}
- std::pair<ui32, TEventHolder*> Push(IEventHandle& ev) {
+ std::pair<ui32, TEventHolder*> Push(IEventHandle& ev) {
TEventHolder& event = Pool.Allocate(Queue);
const ui32 bytes = event.Fill(ev) + sizeof(TEventDescr);
OutputQueueSize += bytes;
- return std::make_pair(bytes, &event);
+ return std::make_pair(bytes, &event);
}
void DropConfirmed(ui64 confirm);
@@ -86,7 +86,7 @@ namespace NActors {
void NotifyUndelivered();
TEventHolderPool& Pool;
- const ui32 PeerNodeId;
+ const ui32 PeerNodeId;
const ui16 ChannelId;
std::shared_ptr<IInterconnectMetrics> Metrics;
const TSessionParams Params;
diff --git a/library/cpp/actors/interconnect/interconnect_common.h b/library/cpp/actors/interconnect/interconnect_common.h
index 285709a00c..104573daaa 100644
--- a/library/cpp/actors/interconnect/interconnect_common.h
+++ b/library/cpp/actors/interconnect/interconnect_common.h
@@ -2,7 +2,7 @@
#include <library/cpp/actors/core/actorid.h>
#include <library/cpp/actors/core/actorsystem.h>
-#include <library/cpp/actors/util/datetime.h>
+#include <library/cpp/actors/util/datetime.h>
#include <library/cpp/monlib/dynamic_counters/counters.h>
#include <library/cpp/monlib/metrics/metric_registry.h>
#include <util/generic/map.h>
@@ -82,7 +82,7 @@ namespace NActors {
TAtomicBase MaxDestructorQueueSize = 1024 * 1024 * 1024;
TString ClusterUUID;
TVector<TString> AcceptUUID;
- ui64 StartTime = GetCycleCountFast();
+ ui64 StartTime = GetCycleCountFast();
TString TechnicalSelfHostName;
TInitWhiteboardCallback InitWhiteboard;
TUpdateWhiteboardCallback UpdateWhiteboard;
diff --git a/library/cpp/actors/interconnect/interconnect_tcp_input_session.cpp b/library/cpp/actors/interconnect/interconnect_tcp_input_session.cpp
index 0abe9fe659..e445c42f70 100644
--- a/library/cpp/actors/interconnect/interconnect_tcp_input_session.cpp
+++ b/library/cpp/actors/interconnect/interconnect_tcp_input_session.cpp
@@ -1,7 +1,7 @@
#include "interconnect_tcp_session.h"
#include "interconnect_tcp_proxy.h"
#include <library/cpp/actors/core/probes.h>
-#include <library/cpp/actors/util/datetime.h>
+#include <library/cpp/actors/util/datetime.h>
namespace NActors {
LWTRACE_USING(ACTORLIB_PROVIDER);
@@ -195,7 +195,7 @@ namespace NActors {
ConfirmedByInput = HeaderConfirm;
if (AtomicGet(Context->ControlPacketId) <= HeaderConfirm && !NewPingProtocol) {
ui64 sendTime = AtomicGet(Context->ControlPacketSendTimer);
- TDuration duration = CyclesToDuration(GetCycleCountFast() - sendTime);
+ TDuration duration = CyclesToDuration(GetCycleCountFast() - sendTime);
const auto durationUs = duration.MicroSeconds();
Metrics->UpdateLegacyPingTimeHist(durationUs);
PingQ.push_back(duration);
@@ -217,7 +217,7 @@ namespace NActors {
Send(SessionId, new TEvProcessPingRequest(HeaderSerial & ~TTcpPacketBuf::PingRequestMask));
} else if (HeaderSerial & TTcpPacketBuf::PingResponseMask) {
const ui64 sent = HeaderSerial & ~TTcpPacketBuf::PingResponseMask;
- const ui64 received = GetCycleCountFast();
+ const ui64 received = GetCycleCountFast();
HandlePingResponse(CyclesToDuration(received - sent));
} else if (HeaderSerial & TTcpPacketBuf::ClockMask) {
HandleClock(TInstant::MicroSeconds(HeaderSerial & ~TTcpPacketBuf::ClockMask));
diff --git a/library/cpp/actors/interconnect/interconnect_tcp_session.cpp b/library/cpp/actors/interconnect/interconnect_tcp_session.cpp
index 2ded7f9f53..8da77bcd1d 100644
--- a/library/cpp/actors/interconnect/interconnect_tcp_session.cpp
+++ b/library/cpp/actors/interconnect/interconnect_tcp_session.cpp
@@ -5,7 +5,7 @@
#include <library/cpp/actors/core/probes.h>
#include <library/cpp/actors/core/log.h>
#include <library/cpp/actors/core/interconnect.h>
-#include <library/cpp/actors/util/datetime.h>
+#include <library/cpp/actors/util/datetime.h>
#include <library/cpp/actors/protos/services_common.pb.h>
#include <library/cpp/monlib/service/pages/templates.h>
@@ -128,20 +128,20 @@ namespace NActors {
auto& oChannel = ChannelScheduler->GetOutputChannel(evChannel);
const bool wasWorking = oChannel.IsWorking();
- const auto [dataSize, event] = oChannel.Push(*ev);
- LWTRACK(ForwardEvent, event->Orbit, Proxy->PeerNodeId, event->Descr.Type, event->Descr.Flags, LWACTORID(event->Descr.Recipient), LWACTORID(event->Descr.Sender), event->Descr.Cookie, event->EventSerializedSize);
-
- TotalOutputQueueSize += dataSize;
+ const auto [dataSize, event] = oChannel.Push(*ev);
+ LWTRACK(ForwardEvent, event->Orbit, Proxy->PeerNodeId, event->Descr.Type, event->Descr.Flags, LWACTORID(event->Descr.Recipient), LWACTORID(event->Descr.Sender), event->Descr.Cookie, event->EventSerializedSize);
+
+ TotalOutputQueueSize += dataSize;
Proxy->Metrics->AddOutputBuffersTotalSize(dataSize);
- if (!wasWorking) {
- // this channel has returned to work -- it was empty and this we have just put first event in the queue
- ChannelScheduler->AddToHeap(oChannel, EqualizeCounter);
+ if (!wasWorking) {
+ // this channel has returned to work -- it was empty and this we have just put first event in the queue
+ ChannelScheduler->AddToHeap(oChannel, EqualizeCounter);
}
SetOutputStuckFlag(true);
++NumEventsInReadyChannels;
- LWTRACK(EnqueueEvent, event->Orbit, Proxy->PeerNodeId, NumEventsInReadyChannels, GetWriteBlockedTotal(), evChannel, oChannel.GetQueueSize(), oChannel.GetBufferedAmountOfData());
+ LWTRACK(EnqueueEvent, event->Orbit, Proxy->PeerNodeId, NumEventsInReadyChannels, GetWriteBlockedTotal(), evChannel, oChannel.GetQueueSize(), oChannel.GetBufferedAmountOfData());
WILSON_TRACE(*TlsActivationContext, &ev->TraceId, OutputQueuePush,
QueueSizeInEvents = oChannel.GetQueueSize(),
QueueSizeInBytes = oChannel.GetBufferedAmountOfData());
@@ -179,7 +179,7 @@ namespace NActors {
} else {
TActivationContext::Send(ev);
}
- LWPROBE(StartBatching, Proxy->PeerNodeId, batchPeriod.MillisecondsFloat());
+ LWPROBE(StartBatching, Proxy->PeerNodeId, batchPeriod.MillisecondsFloat());
LOG_DEBUG_IC_SESSION("ICS17", "batching started");
}
}
@@ -315,7 +315,7 @@ namespace NActors {
// update ping time
Ping = msg.Ping;
- LWPROBE(UpdateFromInputSession, Proxy->PeerNodeId, Ping.MillisecondsFloat());
+ LWPROBE(UpdateFromInputSession, Proxy->PeerNodeId, Ping.MillisecondsFloat());
bool needConfirm = false;
@@ -342,7 +342,7 @@ namespace NActors {
// generate more traffic if we have unblocked state now
if (unblockedSomething) {
- LWPROBE(UnblockByDropConfirmed, Proxy->PeerNodeId, NHPTimer::GetSeconds(GetCycleCountFast() - ev->SendTime) * 1000.0);
+ LWPROBE(UnblockByDropConfirmed, Proxy->PeerNodeId, NHPTimer::GetSeconds(GetCycleCountFast() - ev->SendTime) * 1000.0);
GenerateTraffic();
}
@@ -380,7 +380,7 @@ namespace NActors {
void TInterconnectSessionTCP::HandleRam(TEvRam::TPtr& ev) {
if (ev->Get() == RamInQueue) {
- LWPROBE(FinishRam, Proxy->PeerNodeId, NHPTimer::GetSeconds(GetCycleCountFast() - ev->SendTime) * 1000.0);
+ LWPROBE(FinishRam, Proxy->PeerNodeId, NHPTimer::GetSeconds(GetCycleCountFast() - ev->SendTime) * 1000.0);
RamInQueue = nullptr;
GenerateTraffic();
}
@@ -391,7 +391,7 @@ namespace NActors {
IssuePingRequest();
if (RamInQueue && !RamInQueue->Batching) {
- LWPROBE(SkipGenerateTraffic, Proxy->PeerNodeId, NHPTimer::GetSeconds(GetCycleCountFast() - RamStartedCycles) * 1000.0);
+ LWPROBE(SkipGenerateTraffic, Proxy->PeerNodeId, NHPTimer::GetSeconds(GetCycleCountFast() - RamStartedCycles) * 1000.0);
return; // we'll do it a bit later
} else {
RamInQueue = nullptr;
@@ -399,51 +399,51 @@ namespace NActors {
LOG_DEBUG_IC_SESSION("ICS19", "GenerateTraffic");
- // There is a tradeoff between fairness and efficiency.
- // The less traffic is generated here, the less buffering is after fair scheduler,
- // the more fair system is, the less latency is present.
- // The more traffic is generated here, the less syscalls and actor-system overhead occurs,
- // the less cpu is consumed.
- static const ui64 generateLimit = 64 * 1024;
-
+ // There is a tradeoff between fairness and efficiency.
+ // The less traffic is generated here, the less buffering is after fair scheduler,
+ // the more fair system is, the less latency is present.
+ // The more traffic is generated here, the less syscalls and actor-system overhead occurs,
+ // the less cpu is consumed.
+ static const ui64 generateLimit = 64 * 1024;
+
const ui64 sizeBefore = TotalOutputQueueSize;
- ui32 generatedPackets = 0;
- ui64 generatedBytes = 0;
- ui64 generateStarted = GetCycleCountFast();
+ ui32 generatedPackets = 0;
+ ui64 generatedBytes = 0;
+ ui64 generateStarted = GetCycleCountFast();
// apply traffic changes
auto accountTraffic = [&] { ChannelScheduler->ForEach([](TEventOutputChannel& channel) { channel.AccountTraffic(); }); };
- // first, we create as many data packets as we can generate under certain conditions; they include presence
- // of events in channels queues and in flight fitting into requested limit; after we hit one of these conditions
- // we exit cycle
- while (Socket && NumEventsInReadyChannels && InflightDataAmount < GetTotalInflightAmountOfData() && !ReceiveContext->WriteBlockedByFullSendBuffer) {
- if (generatedBytes >= generateLimit) {
- // resume later but ensure that we have issued at least one packet
+ // first, we create as many data packets as we can generate under certain conditions; they include presence
+ // of events in channels queues and in flight fitting into requested limit; after we hit one of these conditions
+ // we exit cycle
+ while (Socket && NumEventsInReadyChannels && InflightDataAmount < GetTotalInflightAmountOfData() && !ReceiveContext->WriteBlockedByFullSendBuffer) {
+ if (generatedBytes >= generateLimit) {
+ // resume later but ensure that we have issued at least one packet
RamInQueue = new TEvRam(false);
Send(SelfId(), RamInQueue);
- RamStartedCycles = GetCycleCountFast();
- LWPROBE(StartRam, Proxy->PeerNodeId);
- break;
+ RamStartedCycles = GetCycleCountFast();
+ LWPROBE(StartRam, Proxy->PeerNodeId);
+ break;
}
- try {
- generatedBytes += MakePacket(true);
- ++generatedPackets;
- } catch (const TExSerializedEventTooLarge& ex) {
- // terminate session if the event can't be serialized properly
- accountTraffic();
- LOG_CRIT_IC("ICS31", "serialized event Type# 0x%08" PRIx32 " is too large", ex.Type);
- return Terminate(TDisconnectReason::EventTooLarge());
+ try {
+ generatedBytes += MakePacket(true);
+ ++generatedPackets;
+ } catch (const TExSerializedEventTooLarge& ex) {
+ // terminate session if the event can't be serialized properly
+ accountTraffic();
+ LOG_CRIT_IC("ICS31", "serialized event Type# 0x%08" PRIx32 " is too large", ex.Type);
+ return Terminate(TDisconnectReason::EventTooLarge());
}
}
- if (Socket) {
- WriteData();
- }
-
- LWPROBE(GenerateTraffic, Proxy->PeerNodeId, NHPTimer::GetSeconds(GetCycleCountFast() - generateStarted) * 1000.0, sizeBefore - TotalOutputQueueSize, generatedPackets, generatedBytes);
-
+ if (Socket) {
+ WriteData();
+ }
+
+ LWPROBE(GenerateTraffic, Proxy->PeerNodeId, NHPTimer::GetSeconds(GetCycleCountFast() - generateStarted) * 1000.0, sizeBefore - TotalOutputQueueSize, generatedPackets, generatedBytes);
+
accountTraffic();
EqualizeCounter += ChannelScheduler->Equalize();
}
@@ -520,10 +520,10 @@ namespace NActors {
ReceiveContext->WriteBlockedByFullSendBuffer ? "true" : "false");
if (std::exchange(ReceiveContext->WriteBlockedByFullSendBuffer, false)) {
Proxy->Metrics->IncUsefulWriteWakeups();
- ui64 nowCycles = GetCycleCountFast();
- double blockedUs = NHPTimer::GetSeconds(nowCycles - WriteBlockedCycles) * 1000000.0;
- LWPROBE(ReadyWrite, Proxy->PeerNodeId, NHPTimer::GetSeconds(nowCycles - ev->SendTime) * 1000.0, blockedUs / 1000.0);
- WriteBlockedTotal += TDuration::MicroSeconds(blockedUs);
+ ui64 nowCycles = GetCycleCountFast();
+ double blockedUs = NHPTimer::GetSeconds(nowCycles - WriteBlockedCycles) * 1000000.0;
+ LWPROBE(ReadyWrite, Proxy->PeerNodeId, NHPTimer::GetSeconds(nowCycles - ev->SendTime) * 1000.0, blockedUs / 1000.0);
+ WriteBlockedTotal += TDuration::MicroSeconds(blockedUs);
GenerateTraffic();
} else if (!ev->Cookie) {
Proxy->Metrics->IncSpuriousWriteWakeups();
@@ -602,7 +602,7 @@ namespace NActors {
Y_VERIFY(static_cast<size_t>(r) <= BytesUnwritten);
BytesUnwritten -= r;
written += r;
- ui64 packets = 0;
+ ui64 packets = 0;
// advance SendQueuePos to eat all processed items
for (size_t amount = r; amount && SendQueuePos->DropBufs(amount); ++SendQueuePos) {
@@ -610,11 +610,11 @@ namespace NActors {
LastSentSerial = Max(LastSentSerial, SendQueuePos->GetSerial());
}
++PacketsWrittenToSocket;
- ++packets;
- LWTRACK(PacketWrittenToSocket, SendQueuePos->Orbit, Proxy->PeerNodeId, PacketsWrittenToSocket, SendQueuePos->TriedWriting, SendQueuePos->GetDataSize(), BytesUnwritten, GetWriteBlockedTotal(), (SOCKET)*Socket);
+ ++packets;
+ LWTRACK(PacketWrittenToSocket, SendQueuePos->Orbit, Proxy->PeerNodeId, PacketsWrittenToSocket, SendQueuePos->TriedWriting, SendQueuePos->GetDataSize(), BytesUnwritten, GetWriteBlockedTotal(), (SOCKET)*Socket);
}
-
- LWPROBE(WriteToSocket, Proxy->PeerNodeId, r, packets, PacketsWrittenToSocket, BytesUnwritten, GetWriteBlockedTotal(), (SOCKET)*Socket);
+
+ LWPROBE(WriteToSocket, Proxy->PeerNodeId, r, packets, PacketsWrittenToSocket, BytesUnwritten, GetWriteBlockedTotal(), (SOCKET)*Socket);
} else if (-r != EAGAIN && -r != EWOULDBLOCK) {
const TString message = r == 0 ? "connection closed by peer"
: err ? err
@@ -635,8 +635,8 @@ namespace NActors {
// TEvPollerReadyWrite event from poller; set up flag meaning this and wait for that event
Y_VERIFY(!ReceiveContext->WriteBlockedByFullSendBuffer);
ReceiveContext->WriteBlockedByFullSendBuffer = true;
- WriteBlockedCycles = GetCycleCountFast();
- LWPROBE(BlockedWrite, Proxy->PeerNodeId, SendQueue.size(), written);
+ WriteBlockedCycles = GetCycleCountFast();
+ LWPROBE(BlockedWrite, Proxy->PeerNodeId, SendQueue.size(), written);
LOG_DEBUG_IC_SESSION("ICS18", "hit send buffer limit");
if (PollerToken) {
@@ -718,7 +718,7 @@ namespace NActors {
}
}
- ui64 TInterconnectSessionTCP::MakePacket(bool data, TMaybe<ui64> pingMask) {
+ ui64 TInterconnectSessionTCP::MakePacket(bool data, TMaybe<ui64> pingMask) {
Y_VERIFY(Socket);
TSendQueue::iterator packet;
@@ -759,7 +759,7 @@ namespace NActors {
}
if (AtomicGet(ReceiveContext->ControlPacketId) == 0) {
- AtomicSet(ReceiveContext->ControlPacketSendTimer, GetCycleCountFast());
+ AtomicSet(ReceiveContext->ControlPacketSendTimer, GetCycleCountFast());
AtomicSet(ReceiveContext->ControlPacketId, OutputCounter);
}
@@ -788,8 +788,8 @@ namespace NActors {
packet->Sign();
// count number of bytes pending for write
- ui64 packetSize = (Params.UseModernFrame ? sizeof(TTcpPacketHeader_v2) : sizeof(TTcpPacketHeader_v1)) + packet->GetDataSize();
- BytesUnwritten += packetSize;
+ ui64 packetSize = (Params.UseModernFrame ? sizeof(TTcpPacketHeader_v2) : sizeof(TTcpPacketHeader_v1)) + packet->GetDataSize();
+ BytesUnwritten += packetSize;
LOG_DEBUG_IC_SESSION("ICS22", "outgoing packet Serial# %" PRIu64 " Confirm# %" PRIu64 " DataSize# %zu"
" InflightDataAmount# %" PRIu64 " BytesUnwritten# %" PRIu64, serial, lastInputSerial, packet->GetDataSize(),
@@ -799,13 +799,13 @@ namespace NActors {
ResetFlushLogic();
++PacketsGenerated;
- LWTRACK(PacketGenerated, packet->Orbit, Proxy->PeerNodeId, BytesUnwritten, InflightDataAmount, PacketsGenerated, packetSize);
+ LWTRACK(PacketGenerated, packet->Orbit, Proxy->PeerNodeId, BytesUnwritten, InflightDataAmount, PacketsGenerated, packetSize);
if (!data) {
WriteData();
}
-
- return packetSize;
+
+ return packetSize;
}
bool TInterconnectSessionTCP::DropConfirmed(ui64 confirm) {
@@ -836,14 +836,14 @@ namespace NActors {
channel.DropConfirmed(lastDroppedSerial);
});
- const ui64 current = InflightDataAmount;
- const ui64 limit = GetTotalInflightAmountOfData();
- const bool unblockedSomething = current >= limit && current < limit + droppedDataAmount;
+ const ui64 current = InflightDataAmount;
+ const ui64 limit = GetTotalInflightAmountOfData();
+ const bool unblockedSomething = current >= limit && current < limit + droppedDataAmount;
PacketsConfirmed += numDropped;
InflightDataAmount -= droppedDataAmount;
Proxy->Metrics->SubInflightDataAmount(droppedDataAmount);
- LWPROBE(DropConfirmed, Proxy->PeerNodeId, droppedDataAmount, InflightDataAmount);
+ LWPROBE(DropConfirmed, Proxy->PeerNodeId, droppedDataAmount, InflightDataAmount);
LOG_DEBUG_IC_SESSION("ICS24", "exit InflightDataAmount: %" PRIu64 " bytes droppedDataAmount: %" PRIu64 " bytes"
" dropped %" PRIu32 " packets", InflightDataAmount, droppedDataAmount, numDropped);
@@ -896,7 +896,7 @@ namespace NActors {
}
}
- LWTRACK(FillSendingBuffer, task.Orbit, Proxy->PeerNodeId, bytesGenerated, NumEventsInReadyChannels, WriteBlockedTotal);
+ LWTRACK(FillSendingBuffer, task.Orbit, Proxy->PeerNodeId, bytesGenerated, NumEventsInReadyChannels, WriteBlockedTotal);
Y_VERIFY(bytesGenerated); // ensure we are not stalled in serialization
}
@@ -967,15 +967,15 @@ namespace NActors {
auto& lastpair = OutputQueueUtilization.Last();
if (state)
- lastpair.first -= GetCycleCountFast();
+ lastpair.first -= GetCycleCountFast();
else
- lastpair.first += GetCycleCountFast();
+ lastpair.first += GetCycleCountFast();
OutputStuckFlag = state;
}
void TInterconnectSessionTCP::SwitchStuckPeriod() {
- auto now = GetCycleCountFast();
+ auto now = GetCycleCountFast();
if (OutputQueueUtilization.Size() != 0) {
auto& lastpair = OutputQueueUtilization.Last();
lastpair.second = now - lastpair.second;
@@ -1005,7 +1005,7 @@ namespace NActors {
}
ui64 TInterconnectSessionTCP::GetMaxCyclesPerEvent() const {
- return DurationToCycles(TDuration::MicroSeconds(50));
+ return DurationToCycles(TDuration::MicroSeconds(50));
}
void TInterconnectSessionTCP::IssuePingRequest() {
@@ -1013,7 +1013,7 @@ namespace NActors {
if (now >= LastPingTimestamp + PingPeriodicity) {
LOG_DEBUG_IC_SESSION("ICS22", "Issuing ping request");
if (Socket) {
- MakePacket(false, GetCycleCountFast() | TTcpPacketBuf::PingRequestMask);
+ MakePacket(false, GetCycleCountFast() | TTcpPacketBuf::PingRequestMask);
}
if (Socket) {
MakePacket(false, TInstant::Now().MicroSeconds() | TTcpPacketBuf::ClockMask);
diff --git a/library/cpp/actors/interconnect/interconnect_tcp_session.h b/library/cpp/actors/interconnect/interconnect_tcp_session.h
index 7fc00dbcc5..e347d9b799 100644
--- a/library/cpp/actors/interconnect/interconnect_tcp_session.h
+++ b/library/cpp/actors/interconnect/interconnect_tcp_session.h
@@ -6,7 +6,7 @@
#include <library/cpp/actors/core/log.h>
#include <library/cpp/actors/helpers/mon_histogram_helper.h>
#include <library/cpp/actors/protos/services_common.pb.h>
-#include <library/cpp/actors/util/datetime.h>
+#include <library/cpp/actors/util/datetime.h>
#include <library/cpp/actors/util/rope.h>
#include <library/cpp/actors/util/funnel_queue.h>
#include <library/cpp/actors/util/recentwnd.h>
@@ -38,12 +38,12 @@ namespace NActors {
public:
TSlowPathChecker(TTraceCallback&& callback)
: Callback(std::move(callback))
- , Start(GetCycleCountFast())
+ , Start(GetCycleCountFast())
{
}
~TSlowPathChecker() {
- const NHPTimer::STime end = GetCycleCountFast();
+ const NHPTimer::STime end = GetCycleCountFast();
const NHPTimer::STime elapsed = end - Start;
if (elapsed > 1000000) {
Callback(NHPTimer::GetSeconds(elapsed) * 1000);
@@ -63,7 +63,7 @@ namespace NActors {
class TTimeLimit {
public:
TTimeLimit(ui64 limitInCycles)
- : UpperLimit(limitInCycles == 0 ? 0 : GetCycleCountFast() + limitInCycles)
+ : UpperLimit(limitInCycles == 0 ? 0 : GetCycleCountFast() + limitInCycles)
{
}
@@ -73,7 +73,7 @@ namespace NActors {
}
bool CheckExceeded() {
- return UpperLimit != 0 && GetCycleCountFast() > UpperLimit;
+ return UpperLimit != 0 && GetCycleCountFast() > UpperLimit;
}
const ui64 UpperLimit;
@@ -125,7 +125,7 @@ namespace NActors {
std::unordered_map<ui16, TRope> ChannelMap;
TReceiveContext() {
- GetTimeFast(&StartTime);
+ GetTimeFast(&StartTime);
}
// returns false if sessions needs to be terminated and packet not to be processed
@@ -378,7 +378,7 @@ namespace NActors {
void SetNewConnection(TEvHandshakeDone::TPtr& ev);
TEvRam* RamInQueue = nullptr;
- ui64 RamStartedCycles = 0;
+ ui64 RamStartedCycles = 0;
void HandleRam(TEvRam::TPtr& ev);
void GenerateTraffic();
@@ -389,7 +389,7 @@ namespace NActors {
void Handle(TEvPollerRegisterResult::TPtr ev);
void WriteData();
- ui64 MakePacket(bool data, TMaybe<ui64> pingMask = {});
+ ui64 MakePacket(bool data, TMaybe<ui64> pingMask = {});
void FillSendingBuffer(TTcpPacketOutTask& packet, ui64 serial);
bool DropConfirmed(ui64 confirm);
void ShutdownSocket(TDisconnectReason reason);
@@ -452,21 +452,21 @@ namespace NActors {
TSendQueue SendQueue;
TSendQueue SendQueueCache;
TSendQueue::iterator SendQueuePos;
- ui64 WriteBlockedCycles = 0; // start of current block period
- TDuration WriteBlockedTotal; // total incremental duration that session has been blocked
+ ui64 WriteBlockedCycles = 0; // start of current block period
+ TDuration WriteBlockedTotal; // total incremental duration that session has been blocked
ui64 BytesUnwritten = 0;
void TrimSendQueueCache();
- TDuration GetWriteBlockedTotal() const {
+ TDuration GetWriteBlockedTotal() const {
if (ReceiveContext->WriteBlockedByFullSendBuffer) {
- double blockedUs = NHPTimer::GetSeconds(GetCycleCountFast() - WriteBlockedCycles) * 1000000.0;
- return WriteBlockedTotal + TDuration::MicroSeconds(blockedUs); // append current blocking period if any
- } else {
- return WriteBlockedTotal;
- }
- }
-
+ double blockedUs = NHPTimer::GetSeconds(GetCycleCountFast() - WriteBlockedCycles) * 1000000.0;
+ return WriteBlockedTotal + TDuration::MicroSeconds(blockedUs); // append current blocking period if any
+ } else {
+ return WriteBlockedTotal;
+ }
+ }
+
ui64 OutputCounter;
ui64 LastSentSerial = 0;
diff --git a/library/cpp/actors/interconnect/packet.cpp b/library/cpp/actors/interconnect/packet.cpp
index e2c289ed59..55c9679414 100644
--- a/library/cpp/actors/interconnect/packet.cpp
+++ b/library/cpp/actors/interconnect/packet.cpp
@@ -1,11 +1,11 @@
#include "packet.h"
-#include <library/cpp/actors/core/probes.h>
-
+#include <library/cpp/actors/core/probes.h>
+
#include <util/system/datetime.h>
-LWTRACE_USING(ACTORLIB_PROVIDER);
-
+LWTRACE_USING(ACTORLIB_PROVIDER);
+
ui32 TEventHolder::Fill(IEventHandle& ev) {
Serial = 0;
Descr.Type = ev.Type;
@@ -27,6 +27,6 @@ ui32 TEventHolder::Fill(IEventHandle& ev) {
} else {
EventSerializedSize = 0;
}
-
+
return EventSerializedSize;
}
diff --git a/library/cpp/actors/interconnect/packet.h b/library/cpp/actors/interconnect/packet.h
index 4ba50a2b5f..f062633b0b 100644
--- a/library/cpp/actors/interconnect/packet.h
+++ b/library/cpp/actors/interconnect/packet.h
@@ -8,7 +8,7 @@
#include <library/cpp/actors/util/rope.h>
#include <library/cpp/actors/prof/tag.h>
#include <library/cpp/digest/crc32c/crc32c.h>
-#include <library/cpp/lwtrace/shuttle.h>
+#include <library/cpp/lwtrace/shuttle.h>
#include <util/generic/string.h>
#include <util/generic/list.h>
@@ -108,7 +108,7 @@ struct TEventHolder : TNonCopyable {
ui64 Serial;
ui32 EventSerializedSize;
ui32 EventActuallySerialized;
- mutable NLWTrace::TOrbit Orbit;
+ mutable NLWTrace::TOrbit Orbit;
ui32 Fill(IEventHandle& ev);
@@ -136,7 +136,7 @@ struct TEventHolder : TNonCopyable {
void Clear() {
Event.Reset();
Buffer.Reset();
- Orbit.Reset();
+ Orbit.Reset();
}
};
@@ -154,7 +154,7 @@ struct TTcpPacketOutTask : TNonCopyable {
bool TriedWriting;
char *FreeArea;
char *End;
- mutable NLWTrace::TOrbit Orbit;
+ mutable NLWTrace::TOrbit Orbit;
public:
TTcpPacketOutTask(const TSessionParams& params)
@@ -189,7 +189,7 @@ public:
TriedWriting = false;
FreeArea = Params.UseModernFrame ? Packet.v2.Data : Packet.v1.Data;
End = FreeArea + TTcpPacketBuf::PacketDataLen;
- Orbit.Reset();
+ Orbit.Reset();
}
bool IsEmpty() const {
diff --git a/library/cpp/actors/interconnect/poller_actor.cpp b/library/cpp/actors/interconnect/poller_actor.cpp
index e75cbcaef4..ef363b208e 100644
--- a/library/cpp/actors/interconnect/poller_actor.cpp
+++ b/library/cpp/actors/interconnect/poller_actor.cpp
@@ -5,7 +5,7 @@
#include <library/cpp/actors/core/actorsystem.h>
#include <library/cpp/actors/core/hfunc.h>
#include <library/cpp/actors/core/log.h>
-#include <library/cpp/actors/core/probes.h>
+#include <library/cpp/actors/core/probes.h>
#include <library/cpp/actors/protos/services_common.pb.h>
#include <library/cpp/actors/util/funnel_queue.h>
@@ -17,9 +17,9 @@
#include <variant>
namespace NActors {
-
- LWTRACE_USING(ACTORLIB_PROVIDER);
-
+
+ LWTRACE_USING(ACTORLIB_PROVIDER);
+
namespace {
int LastSocketError() {
#if defined(_win_)
diff --git a/library/cpp/actors/interconnect/profiler.h b/library/cpp/actors/interconnect/profiler.h
index 77a59e3179..d9192e1559 100644
--- a/library/cpp/actors/interconnect/profiler.h
+++ b/library/cpp/actors/interconnect/profiler.h
@@ -1,7 +1,7 @@
#pragma once
-#include <library/cpp/actors/util/datetime.h>
-
+#include <library/cpp/actors/util/datetime.h>
+
namespace NActors {
class TProfiled {
@@ -14,7 +14,7 @@ namespace NActors {
EType Type; // entry kind
int Line;
const char *Marker; // name of the profiled function/part
- ui64 Timestamp; // cycles
+ ui64 Timestamp; // cycles
};
bool Enable = false;
@@ -44,7 +44,7 @@ namespace NActors {
type,
line,
marker,
- GetCycleCountFast()
+ GetCycleCountFast()
});
}
}
diff --git a/library/cpp/actors/memory_log/memlog.cpp b/library/cpp/actors/memory_log/memlog.cpp
index 8e6b46727d..a7ffa24623 100644
--- a/library/cpp/actors/memory_log/memlog.cpp
+++ b/library/cpp/actors/memory_log/memlog.cpp
@@ -1,7 +1,7 @@
#include "memlog.h"
-#include <library/cpp/actors/util/datetime.h>
-
+#include <library/cpp/actors/util/datetime.h>
+
#include <util/system/info.h>
#include <util/system/atomic.h>
#include <util/system/align.h>
@@ -270,7 +270,7 @@ bool MemLogWrite(const char* begin, size_t msgSize, bool addLF) noexcept {
Y_VERIFY(AlignDown(&prolog, TMemoryLog::MemcpyAlignment) == &prolog);
int snprintfResult = snprintf(prolog, prologSize + 1,
- "TS %020" PRIu64 " TI %020" PRIu64 " ", GetCycleCountFast(), threadId);
+ "TS %020" PRIu64 " TI %020" PRIu64 " ", GetCycleCountFast(), threadId);
if (snprintfResult < 0) {
return false;
@@ -341,7 +341,7 @@ bool MemLogVPrintF(const char* format, va_list params) noexcept {
int prologSize = snprintf(buf,
TMemoryLog::MAX_MESSAGE_SIZE - 2,
"TS %020" PRIu64 " TI %020" PRIu64 " ",
- GetCycleCountFast(),
+ GetCycleCountFast(),
threadId);
if (Y_UNLIKELY(prologSize < 0)) {
diff --git a/library/cpp/actors/testlib/test_runtime.cpp b/library/cpp/actors/testlib/test_runtime.cpp
index 6fa25b9965..008e25b416 100644
--- a/library/cpp/actors/testlib/test_runtime.cpp
+++ b/library/cpp/actors/testlib/test_runtime.cpp
@@ -6,7 +6,7 @@
#include <library/cpp/actors/core/executor_pool_io.h>
#include <library/cpp/actors/core/log.h>
#include <library/cpp/actors/core/scheduler_basic.h>
-#include <library/cpp/actors/util/datetime.h>
+#include <library/cpp/actors/util/datetime.h>
#include <library/cpp/actors/protos/services_common.pb.h>
#include <library/cpp/random_provider/random_provider.h>
#include <library/cpp/actors/interconnect/interconnect.h>
@@ -248,7 +248,7 @@ namespace NActors {
Node->ActorSystemMonotonic = currentMonotonic;
}
- void PrepareSchedules(NSchedulerQueue::TReader **readers, ui32 scheduleReadersCount) override {
+ void PrepareSchedules(NSchedulerQueue::TReader **readers, ui32 scheduleReadersCount) override {
Y_UNUSED(readers);
Y_UNUSED(scheduleReadersCount);
}
@@ -282,33 +282,33 @@ namespace NActors {
}
// for threads
- ui32 GetReadyActivation(TWorkerContext& wctx, ui64 revolvingCounter) override {
- Y_UNUSED(wctx);
+ ui32 GetReadyActivation(TWorkerContext& wctx, ui64 revolvingCounter) override {
+ Y_UNUSED(wctx);
Y_UNUSED(revolvingCounter);
Y_FAIL();
}
- void ReclaimMailbox(TMailboxType::EType mailboxType, ui32 hint, TWorkerId workerId, ui64 revolvingCounter) override {
- Y_UNUSED(workerId);
+ void ReclaimMailbox(TMailboxType::EType mailboxType, ui32 hint, TWorkerId workerId, ui64 revolvingCounter) override {
+ Y_UNUSED(workerId);
Node->MailboxTable->ReclaimMailbox(mailboxType, hint, revolvingCounter);
}
- void Schedule(TInstant deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie *cookie, TWorkerId workerId) override {
- DoSchedule(deadline, ev, cookie, workerId);
+ void Schedule(TInstant deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie *cookie, TWorkerId workerId) override {
+ DoSchedule(deadline, ev, cookie, workerId);
}
- void Schedule(TMonotonic deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie *cookie, TWorkerId workerId) override {
- DoSchedule(TInstant::FromValue(deadline.GetValue()), ev, cookie, workerId);
+ void Schedule(TMonotonic deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie *cookie, TWorkerId workerId) override {
+ DoSchedule(TInstant::FromValue(deadline.GetValue()), ev, cookie, workerId);
}
- void Schedule(TDuration delay, TAutoPtr<IEventHandle> ev, ISchedulerCookie *cookie, TWorkerId workerId) override {
+ void Schedule(TDuration delay, TAutoPtr<IEventHandle> ev, ISchedulerCookie *cookie, TWorkerId workerId) override {
TInstant deadline = Runtime->GetTimeProvider()->Now() + delay;
- DoSchedule(deadline, ev, cookie, workerId);
+ DoSchedule(deadline, ev, cookie, workerId);
}
- void DoSchedule(TInstant deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie *cookie, TWorkerId workerId) {
- Y_UNUSED(workerId);
-
+ void DoSchedule(TInstant deadline, TAutoPtr<IEventHandle> ev, ISchedulerCookie *cookie, TWorkerId workerId) {
+ Y_UNUSED(workerId);
+
TGuard<TMutex> guard(Runtime->Mutex);
bool verbose = (Runtime->CurrentDispatchContext ? !Runtime->CurrentDispatchContext->Options->Quiet : true) && VERBOSE;
if (Runtime->BlockedOutput.find(ev->Sender) != Runtime->BlockedOutput.end()) {
@@ -372,7 +372,7 @@ namespace NActors {
TMailboxHeader* mailbox = node->MailboxTable->Get(mailboxHint);
IActor* recipientActor = mailbox->FindActor(ev->GetRecipientRewrite().LocalId());
if (recipientActor) {
- TActorContext ctx(*mailbox, *node->ExecutorThread, GetCycleCountFast(), ev->GetRecipientRewrite());
+ TActorContext ctx(*mailbox, *node->ExecutorThread, GetCycleCountFast(), ev->GetRecipientRewrite());
TActivationContext *prevTlsActivationContext = TlsActivationContext;
TlsActivationContext = &ctx;
recipientActor->Receive(ev, ctx);
@@ -412,7 +412,7 @@ namespace NActors {
}
// lifecycle stuff
- void Prepare(TActorSystem *actorSystem, NSchedulerQueue::TReader **scheduleReaders, ui32 *scheduleSz) override {
+ void Prepare(TActorSystem *actorSystem, NSchedulerQueue::TReader **scheduleReaders, ui32 *scheduleSz) override {
Y_UNUSED(actorSystem);
Y_UNUSED(scheduleReaders);
Y_UNUSED(scheduleSz);
@@ -507,7 +507,7 @@ namespace NActors {
node->SchedulerPool.Reset(CreateExecutorPoolStub(this, nodeIndex, node, 0));
node->MailboxTable.Reset(new TMailboxTable());
node->ActorSystem = MakeActorSystem(nodeIndex, node);
- node->ExecutorThread.Reset(new TExecutorThread(0, 0, node->ActorSystem.Get(), node->SchedulerPool.Get(), node->MailboxTable.Get(), "TestExecutor"));
+ node->ExecutorThread.Reset(new TExecutorThread(0, 0, node->ActorSystem.Get(), node->SchedulerPool.Get(), node->MailboxTable.Get(), "TestExecutor"));
} else {
node->ActorSystem = MakeActorSystem(nodeIndex, node);
}
@@ -904,19 +904,19 @@ namespace NActors {
switch (mailboxType) {
case TMailboxType::Simple:
- UnlockFromExecution((TMailboxTable::TSimpleMailbox *)mailbox, node->ExecutorPools[0], false, hint, MaxWorkers, ++revolvingCounter);
+ UnlockFromExecution((TMailboxTable::TSimpleMailbox *)mailbox, node->ExecutorPools[0], false, hint, MaxWorkers, ++revolvingCounter);
break;
case TMailboxType::Revolving:
- UnlockFromExecution((TMailboxTable::TRevolvingMailbox *)mailbox, node->ExecutorPools[0], false, hint, MaxWorkers, ++revolvingCounter);
+ UnlockFromExecution((TMailboxTable::TRevolvingMailbox *)mailbox, node->ExecutorPools[0], false, hint, MaxWorkers, ++revolvingCounter);
break;
case TMailboxType::HTSwap:
- UnlockFromExecution((TMailboxTable::THTSwapMailbox *)mailbox, node->ExecutorPools[0], false, hint, MaxWorkers, ++revolvingCounter);
+ UnlockFromExecution((TMailboxTable::THTSwapMailbox *)mailbox, node->ExecutorPools[0], false, hint, MaxWorkers, ++revolvingCounter);
break;
case TMailboxType::ReadAsFilled:
- UnlockFromExecution((TMailboxTable::TReadAsFilledMailbox *)mailbox, node->ExecutorPools[0], false, hint, MaxWorkers, ++revolvingCounter);
+ UnlockFromExecution((TMailboxTable::TReadAsFilledMailbox *)mailbox, node->ExecutorPools[0], false, hint, MaxWorkers, ++revolvingCounter);
break;
case TMailboxType::TinyReadAsFilled:
- UnlockFromExecution((TMailboxTable::TTinyReadAsFilledMailbox *)mailbox, node->ExecutorPools[0], false, hint, MaxWorkers, ++revolvingCounter);
+ UnlockFromExecution((TMailboxTable::TTinyReadAsFilledMailbox *)mailbox, node->ExecutorPools[0], false, hint, MaxWorkers, ++revolvingCounter);
break;
default:
Y_FAIL("Unsupported mailbox type");
@@ -1029,13 +1029,13 @@ namespace NActors {
}
bool TTestActorRuntimeBase::DispatchEvents(const TDispatchOptions& options) {
- return DispatchEvents(options, TInstant::Max());
- }
-
+ return DispatchEvents(options, TInstant::Max());
+ }
+
bool TTestActorRuntimeBase::DispatchEvents(const TDispatchOptions& options, TDuration simTimeout) {
- return DispatchEvents(options, TInstant::MicroSeconds(CurrentTimestamp) + simTimeout);
- }
-
+ return DispatchEvents(options, TInstant::MicroSeconds(CurrentTimestamp) + simTimeout);
+ }
+
bool TTestActorRuntimeBase::DispatchEvents(const TDispatchOptions& options, TInstant simDeadline) {
TGuard<TMutex> guard(Mutex);
return DispatchEventsInternal(options, simDeadline);
@@ -1251,7 +1251,7 @@ namespace NActors {
}
if (!localContext.FoundNonEmptyMailboxes.empty())
- return true;
+ return true;
if (options.CustomFinalCondition && options.CustomFinalCondition())
return true;
@@ -1266,15 +1266,15 @@ namespace NActors {
Cerr << "Dispatch complete with non-empty queue at " << TInstant::MicroSeconds(CurrentTimestamp) << "\n";
}
- return true;
+ return true;
}
}
}
- if (TInstant::MicroSeconds(CurrentTimestamp) > simDeadline) {
- return false;
- }
-
+ if (TInstant::MicroSeconds(CurrentTimestamp) > simDeadline) {
+ return false;
+ }
+
if (dispatchTime >= deadline) {
if (verbose) {
Cerr << "Reach deadline at " << TInstant::MicroSeconds(CurrentTimestamp) << "\n";
@@ -1333,7 +1333,7 @@ namespace NActors {
Cerr << "Process selected events at " << TInstant::MicroSeconds(CurrentTimestamp) << "\n";
}
- deadline = dispatchTime + DispatchTimeout;
+ deadline = dispatchTime + DispatchTimeout;
continue;
}
@@ -1351,7 +1351,7 @@ namespace NActors {
dispatchTime += waitDelay;
MailboxesHasEvents.WaitT(Mutex, waitDelay);
}
- return false;
+ return false;
}
void TTestActorRuntimeBase::HandleNonEmptyMailboxesForEachContext(TEventMailboxId mboxId) {
@@ -1452,8 +1452,8 @@ namespace NActors {
++dispatchCount;
{
if (!DispatchEventsInternal(TDispatchOptions(), deadline)) {
- return; // Timed out; event was not found
- }
+ return; // Timed out; event was not found
+ }
}
Y_VERIFY(dispatchCount < 1000, "Hard limit to prevent endless loop");
@@ -1572,7 +1572,7 @@ namespace NActors {
// Save actorId by value in order to prevent ctx from being invalidated during another Send call.
TActorId actorId = ev->GetRecipientRewrite();
node->ActorToActorId[recipientActor] = ev->GetRecipientRewrite();
- TActorContext ctx(*mailbox, *node->ExecutorThread, GetCycleCountFast(), actorId);
+ TActorContext ctx(*mailbox, *node->ExecutorThread, GetCycleCountFast(), actorId);
TActivationContext *prevTlsActivationContext = TlsActivationContext;
TlsActivationContext = &ctx;
CurrentRecipient = actorId;
diff --git a/library/cpp/actors/testlib/test_runtime.h b/library/cpp/actors/testlib/test_runtime.h
index 26e3b45c98..863e2cdb49 100644
--- a/library/cpp/actors/testlib/test_runtime.h
+++ b/library/cpp/actors/testlib/test_runtime.h
@@ -253,9 +253,9 @@ namespace NActors {
void PushEventsFront(TEventsList& events);
void PushMailboxEventsFront(ui32 hint, ui32 nodeId, TEventsList& events);
// doesn't dispatch events for edge actors
- bool DispatchEvents(const TDispatchOptions& options = TDispatchOptions());
- bool DispatchEvents(const TDispatchOptions& options, TDuration simTimeout);
- bool DispatchEvents(const TDispatchOptions& options, TInstant simDeadline);
+ bool DispatchEvents(const TDispatchOptions& options = TDispatchOptions());
+ bool DispatchEvents(const TDispatchOptions& options, TDuration simTimeout);
+ bool DispatchEvents(const TDispatchOptions& options, TInstant simDeadline);
void Send(IEventHandle* ev, ui32 senderNodeIndex = 0, bool viaActorSystem = false);
void Schedule(IEventHandle* ev, const TDuration& duration, ui32 nodeIndex = 0);
void ClearCounters();
@@ -288,7 +288,7 @@ namespace NActors {
TActorSystem* GetAnyNodeActorSystem();
TActorSystem* GetActorSystem(ui32 nodeId);
template <typename TEvent>
- TEvent* GrabEdgeEventIf(TAutoPtr<IEventHandle>& handle, std::function<bool(const TEvent&)> predicate, TDuration simTimeout = TDuration::Max()) {
+ TEvent* GrabEdgeEventIf(TAutoPtr<IEventHandle>& handle, std::function<bool(const TEvent&)> predicate, TDuration simTimeout = TDuration::Max()) {
handle.Destroy();
const ui32 eventType = TEvent::EventType;
WaitForEdgeEvents([&](TTestActorRuntimeBase& runtime, TAutoPtr<IEventHandle>& event) {
@@ -305,14 +305,14 @@ namespace NActors {
return false;
}, {}, simTimeout);
- if (simTimeout == TDuration::Max())
+ if (simTimeout == TDuration::Max())
Y_VERIFY(handle);
-
- if (handle) {
- return reinterpret_cast<TAutoPtr<TEventHandle<TEvent>>&>(handle)->Get();
- } else {
- return nullptr;
- }
+
+ if (handle) {
+ return reinterpret_cast<TAutoPtr<TEventHandle<TEvent>>&>(handle)->Get();
+ } else {
+ return nullptr;
+ }
}
template<class TEvent>
@@ -354,9 +354,9 @@ namespace NActors {
}
template <typename TEvent>
- TEvent* GrabEdgeEvent(TAutoPtr<IEventHandle>& handle, TDuration simTimeout = TDuration::Max()) {
+ TEvent* GrabEdgeEvent(TAutoPtr<IEventHandle>& handle, TDuration simTimeout = TDuration::Max()) {
std::function<bool(const TEvent&)> truth = [](const TEvent&) { return true; };
- return GrabEdgeEventIf(handle, truth, simTimeout);
+ return GrabEdgeEventIf(handle, truth, simTimeout);
}
template <typename TEvent>
@@ -400,9 +400,9 @@ namespace NActors {
}
template <typename TEvent>
- TEvent* GrabEdgeEventRethrow(TAutoPtr<IEventHandle>& handle, TDuration simTimeout = TDuration::Max()) {
+ TEvent* GrabEdgeEventRethrow(TAutoPtr<IEventHandle>& handle, TDuration simTimeout = TDuration::Max()) {
try {
- return GrabEdgeEvent<TEvent>(handle, simTimeout);
+ return GrabEdgeEvent<TEvent>(handle, simTimeout);
} catch (...) {
ythrow TWithBackTrace<yexception>() << "Exception occured while waiting for " << TypeName<TEvent>() << ": " << CurrentExceptionMessage();
}
diff --git a/library/cpp/actors/util/affinity.cpp b/library/cpp/actors/util/affinity.cpp
index cc1b6e70ec..69f9048ec1 100644
--- a/library/cpp/actors/util/affinity.cpp
+++ b/library/cpp/actors/util/affinity.cpp
@@ -16,17 +16,17 @@ public:
#endif
}
- explicit TImpl(const ui8* cpus, ui32 size) {
+ explicit TImpl(const ui8* cpus, ui32 size) {
#ifdef _linux_
CPU_ZERO(&Mask);
- for (ui32 i = 0; i != size; ++i) {
- if (cpus[i]) {
- CPU_SET(i, &Mask);
- }
- }
+ for (ui32 i = 0; i != size; ++i) {
+ if (cpus[i]) {
+ CPU_SET(i, &Mask);
+ }
+ }
#else
- Y_UNUSED(cpus);
- Y_UNUSED(size);
+ Y_UNUSED(cpus);
+ Y_UNUSED(size);
#endif
}
@@ -36,18 +36,18 @@ public:
Y_VERIFY_DEBUG(ar == 0);
#endif
}
-
- operator TCpuMask() const {
- TCpuMask result;
-#ifdef _linux_
- for (ui32 i = 0; i != CPU_SETSIZE; ++i) {
- result.Cpus.emplace_back(CPU_ISSET(i, &Mask));
- }
- result.RemoveTrailingZeros();
-#endif
- return result;
- }
-
+
+ operator TCpuMask() const {
+ TCpuMask result;
+#ifdef _linux_
+ for (ui32 i = 0; i != CPU_SETSIZE; ++i) {
+ result.Cpus.emplace_back(CPU_ISSET(i, &Mask));
+ }
+ result.RemoveTrailingZeros();
+#endif
+ return result;
+ }
+
};
TAffinity::TAffinity() {
@@ -57,37 +57,37 @@ TAffinity::~TAffinity() {
}
TAffinity::TAffinity(const ui8* x, ui32 sz) {
- if (x && sz) {
- Impl.Reset(new TImpl(x, sz));
- }
-}
-
-TAffinity::TAffinity(const TCpuMask& mask) {
- if (!mask.IsEmpty()) {
- static_assert(sizeof(ui8) == sizeof(mask.Cpus[0]));
- const ui8* x = reinterpret_cast<const ui8*>(&mask.Cpus[0]);
- const ui32 sz = mask.Size();
+ if (x && sz) {
Impl.Reset(new TImpl(x, sz));
- }
+ }
}
+TAffinity::TAffinity(const TCpuMask& mask) {
+ if (!mask.IsEmpty()) {
+ static_assert(sizeof(ui8) == sizeof(mask.Cpus[0]));
+ const ui8* x = reinterpret_cast<const ui8*>(&mask.Cpus[0]);
+ const ui32 sz = mask.Size();
+ Impl.Reset(new TImpl(x, sz));
+ }
+}
+
void TAffinity::Current() {
Impl.Reset(new TImpl());
}
void TAffinity::Set() const {
- if (!!Impl) {
+ if (!!Impl) {
Impl->Set();
- }
+ }
}
bool TAffinity::Empty() const {
- return !Impl;
-}
-
-TAffinity::operator TCpuMask() const {
- if (!!Impl) {
- return *Impl;
- }
- return TCpuMask();
+ return !Impl;
}
+
+TAffinity::operator TCpuMask() const {
+ if (!!Impl) {
+ return *Impl;
+ }
+ return TCpuMask();
+}
diff --git a/library/cpp/actors/util/affinity.h b/library/cpp/actors/util/affinity.h
index ae106ed180..c807d03b76 100644
--- a/library/cpp/actors/util/affinity.h
+++ b/library/cpp/actors/util/affinity.h
@@ -1,27 +1,27 @@
#pragma once
#include "defs.h"
-#include "cpumask.h"
+#include "cpumask.h"
-// Platform-specific class to set or get thread affinity
+// Platform-specific class to set or get thread affinity
class TAffinity: public TThrRefBase, TNonCopyable {
class TImpl;
THolder<TImpl> Impl;
public:
TAffinity();
- TAffinity(const ui8* cpus, ui32 size);
- explicit TAffinity(const TCpuMask& mask);
+ TAffinity(const ui8* cpus, ui32 size);
+ explicit TAffinity(const TCpuMask& mask);
~TAffinity();
void Current();
void Set() const;
bool Empty() const;
-
- operator TCpuMask() const;
+
+ operator TCpuMask() const;
};
-// Scoped affinity setter
+// Scoped affinity setter
class TAffinityGuard : TNonCopyable {
bool Stacked;
TAffinity OldAffinity;
diff --git a/library/cpp/actors/util/cpumask.h b/library/cpp/actors/util/cpumask.h
index 29741aa1d6..1f72ae3895 100644
--- a/library/cpp/actors/util/cpumask.h
+++ b/library/cpp/actors/util/cpumask.h
@@ -1,133 +1,133 @@
-#pragma once
-
-#include "defs.h"
-
-#include <library/cpp/containers/stack_vector/stack_vec.h>
-
-#include <util/string/split.h>
-#include <util/generic/yexception.h>
-
-using TCpuId = ui32;
-
-// Simple data structure to operate with set of cpus
-struct TCpuMask {
- TStackVec<bool, 1024> Cpus;
-
- // Creates empty mask
- TCpuMask() {}
-
- // Creates mask with single cpu set
- explicit TCpuMask(TCpuId cpuId) {
- Set(cpuId);
- }
-
- // Initialize mask from raw boolean array
- template <class T>
- TCpuMask(const T* cpus, TCpuId size) {
- Cpus.reserve(size);
- for (TCpuId i = 0; i != size; ++i) {
- Cpus.emplace_back(bool(cpus[i]));
- }
- }
-
- // Parse a numerical list of processors. The numbers are separated by commas and may include ranges. For example: 0,5,7,9-11
- explicit TCpuMask(const TString& cpuList) {
- try {
- for (TStringBuf s : StringSplitter(cpuList).Split(',')) {
- TCpuId l, r;
- if (s.find('-') != TString::npos) {
- StringSplitter(s).Split('-').CollectInto(&l, &r);
- } else {
- l = r = FromString<TCpuId>(s);
- }
- if (r >= Cpus.size()) {
- Cpus.resize(r + 1, false);
- }
- for (TCpuId cpu = l; cpu <= r; cpu++) {
- Cpus[cpu] = true;
- }
- }
- } catch (...) {
- ythrow TWithBackTrace<yexception>() << "Exception occured while parsing cpu list '" << cpuList << "': " << CurrentExceptionMessage();
- }
- }
-
- // Returns size of underlying vector
- TCpuId Size() const {
- return Cpus.size();
- }
-
- // Returns number of set bits in mask
- TCpuId CpuCount() const {
- TCpuId result = 0;
- for (bool value : Cpus) {
- result += value;
- }
- return result;
- }
-
- bool IsEmpty() const {
- for (bool value : Cpus) {
- if (value) {
- return false;
- }
- }
- return true;
- }
-
- bool IsSet(TCpuId cpu) const {
- return cpu < Cpus.size() && Cpus[cpu];
- }
-
- void Set(TCpuId cpu) {
- if (cpu >= Cpus.size()) {
- Cpus.resize(cpu + 1, false);
- }
- Cpus[cpu] = true;
- }
-
- void Reset(TCpuId cpu) {
- if (cpu < Cpus.size()) {
- Cpus[cpu] = false;
- }
- }
-
- void RemoveTrailingZeros() {
- while (!Cpus.empty() && !Cpus.back()) {
- Cpus.pop_back();
- }
- }
-
- explicit operator bool() const {
- return !IsEmpty();
- }
-
- TCpuMask operator &(const TCpuMask& rhs) const {
- TCpuMask result;
- TCpuId size = Max(Size(), rhs.Size());
- result.Cpus.reserve(size);
- for (TCpuId cpu = 0; cpu < size; cpu++) {
- result.Cpus.emplace_back(IsSet(cpu) && rhs.IsSet(cpu));
- }
- return result;
- }
-
- TCpuMask operator |(const TCpuMask& rhs) const {
- TCpuMask result;
- TCpuId size = Max(Size(), rhs.Size());
- result.Cpus.reserve(size);
- for (TCpuId cpu = 0; cpu < size; cpu++) {
- result.Cpus.emplace_back(IsSet(cpu) || rhs.IsSet(cpu));
- }
- return result;
- }
-
- TCpuMask operator -(const TCpuMask& rhs) const {
- TCpuMask result;
- result.Cpus.reserve(Size());
- for (TCpuId cpu = 0; cpu < Size(); cpu++) {
- result.Cpus.emplace_back(IsSet(cpu) && !rhs.IsSet(cpu));
- }
- return result;
- }
-};
+#pragma once
+
+#include "defs.h"
+
+#include <library/cpp/containers/stack_vector/stack_vec.h>
+
+#include <util/string/split.h>
+#include <util/generic/yexception.h>
+
+using TCpuId = ui32;
+
+// Simple data structure to operate with set of cpus
+struct TCpuMask {
+ TStackVec<bool, 1024> Cpus;
+
+ // Creates empty mask
+ TCpuMask() {}
+
+ // Creates mask with single cpu set
+ explicit TCpuMask(TCpuId cpuId) {
+ Set(cpuId);
+ }
+
+ // Initialize mask from raw boolean array
+ template <class T>
+ TCpuMask(const T* cpus, TCpuId size) {
+ Cpus.reserve(size);
+ for (TCpuId i = 0; i != size; ++i) {
+ Cpus.emplace_back(bool(cpus[i]));
+ }
+ }
+
+ // Parse a numerical list of processors. The numbers are separated by commas and may include ranges. For example: 0,5,7,9-11
+ explicit TCpuMask(const TString& cpuList) {
+ try {
+ for (TStringBuf s : StringSplitter(cpuList).Split(',')) {
+ TCpuId l, r;
+ if (s.find('-') != TString::npos) {
+ StringSplitter(s).Split('-').CollectInto(&l, &r);
+ } else {
+ l = r = FromString<TCpuId>(s);
+ }
+ if (r >= Cpus.size()) {
+ Cpus.resize(r + 1, false);
+ }
+ for (TCpuId cpu = l; cpu <= r; cpu++) {
+ Cpus[cpu] = true;
+ }
+ }
+ } catch (...) {
+ ythrow TWithBackTrace<yexception>() << "Exception occured while parsing cpu list '" << cpuList << "': " << CurrentExceptionMessage();
+ }
+ }
+
+ // Returns size of underlying vector
+ TCpuId Size() const {
+ return Cpus.size();
+ }
+
+ // Returns number of set bits in mask
+ TCpuId CpuCount() const {
+ TCpuId result = 0;
+ for (bool value : Cpus) {
+ result += value;
+ }
+ return result;
+ }
+
+ bool IsEmpty() const {
+ for (bool value : Cpus) {
+ if (value) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ bool IsSet(TCpuId cpu) const {
+ return cpu < Cpus.size() && Cpus[cpu];
+ }
+
+ void Set(TCpuId cpu) {
+ if (cpu >= Cpus.size()) {
+ Cpus.resize(cpu + 1, false);
+ }
+ Cpus[cpu] = true;
+ }
+
+ void Reset(TCpuId cpu) {
+ if (cpu < Cpus.size()) {
+ Cpus[cpu] = false;
+ }
+ }
+
+ void RemoveTrailingZeros() {
+ while (!Cpus.empty() && !Cpus.back()) {
+ Cpus.pop_back();
+ }
+ }
+
+ explicit operator bool() const {
+ return !IsEmpty();
+ }
+
+ TCpuMask operator &(const TCpuMask& rhs) const {
+ TCpuMask result;
+ TCpuId size = Max(Size(), rhs.Size());
+ result.Cpus.reserve(size);
+ for (TCpuId cpu = 0; cpu < size; cpu++) {
+ result.Cpus.emplace_back(IsSet(cpu) && rhs.IsSet(cpu));
+ }
+ return result;
+ }
+
+ TCpuMask operator |(const TCpuMask& rhs) const {
+ TCpuMask result;
+ TCpuId size = Max(Size(), rhs.Size());
+ result.Cpus.reserve(size);
+ for (TCpuId cpu = 0; cpu < size; cpu++) {
+ result.Cpus.emplace_back(IsSet(cpu) || rhs.IsSet(cpu));
+ }
+ return result;
+ }
+
+ TCpuMask operator -(const TCpuMask& rhs) const {
+ TCpuMask result;
+ result.Cpus.reserve(Size());
+ for (TCpuId cpu = 0; cpu < Size(); cpu++) {
+ result.Cpus.emplace_back(IsSet(cpu) && !rhs.IsSet(cpu));
+ }
+ return result;
+ }
+};
diff --git a/library/cpp/actors/util/datetime.h b/library/cpp/actors/util/datetime.h
index cbec5965d6..4bdf57bbde 100644
--- a/library/cpp/actors/util/datetime.h
+++ b/library/cpp/actors/util/datetime.h
@@ -1,82 +1,82 @@
-#pragma once
-
-#include <util/system/defaults.h>
-#include <util/system/hp_timer.h>
-#include <util/system/platform.h>
-
-#if defined(_win_)
-#include <intrin.h>
-#pragma intrinsic(__rdtsc)
-#endif // _win_
-
-#if defined(_darwin_) && !defined(_x86_)
-#include <mach/mach_time.h>
-#endif
-
-// GetCycleCount() from util/system/datetime.h uses rdtscp, which is more accurate than rdtsc,
-// but rdtscp disables processor's out-of-order execution, so it can be slow
-Y_FORCE_INLINE ui64 GetCycleCountFast() {
-#if defined(_MSC_VER)
- // Generates the rdtsc instruction, which returns the processor time stamp.
- // The processor time stamp records the number of clock cycles since the last reset.
- return __rdtsc();
-#elif defined(__clang__) && !defined(_arm64_)
- return __builtin_readcyclecounter();
-#elif defined(_x86_64_)
- unsigned hi, lo;
- __asm__ __volatile__("rdtsc"
- : "=a"(lo), "=d"(hi));
- return ((unsigned long long)lo) | (((unsigned long long)hi) << 32);
-#elif defined(_i386_)
- ui64 x;
- __asm__ volatile("rdtsc\n\t"
- : "=A"(x));
- return x;
-#elif defined(_darwin_)
- return mach_absolute_time();
-#elif defined(_arm32_)
- return MicroSeconds();
-#elif defined(_arm64_)
- ui64 x;
-
- __asm__ __volatile__("isb; mrs %0, cntvct_el0"
- : "=r"(x));
-
- return x;
-#else
-#error "unsupported arch"
-#endif
-}
-
-// NHPTimer::GetTime fast analog
-Y_FORCE_INLINE void GetTimeFast(NHPTimer::STime* pTime) noexcept {
- *pTime = GetCycleCountFast();
-}
-
-namespace NActors {
- inline double Ts2Ns(ui64 ts) {
- return NHPTimer::GetSeconds(ts) * 1e9;
- }
-
- inline double Ts2Us(ui64 ts) {
- return NHPTimer::GetSeconds(ts) * 1e6;
- }
-
- inline double Ts2Ms(ui64 ts) {
- return NHPTimer::GetSeconds(ts) * 1e3;
- }
-
- inline ui64 Us2Ts(double us) {
- return ui64(NHPTimer::GetClockRate() * us / 1e6);
- }
-
- struct TTimeTracker {
- ui64 Ts;
- TTimeTracker(): Ts(GetCycleCountFast()) {}
- ui64 Elapsed() {
- ui64 ts = GetCycleCountFast();
- std::swap(Ts, ts);
- return Ts - ts;
- }
- };
-}
+#pragma once
+
+#include <util/system/defaults.h>
+#include <util/system/hp_timer.h>
+#include <util/system/platform.h>
+
+#if defined(_win_)
+#include <intrin.h>
+#pragma intrinsic(__rdtsc)
+#endif // _win_
+
+#if defined(_darwin_) && !defined(_x86_)
+#include <mach/mach_time.h>
+#endif
+
+// GetCycleCount() from util/system/datetime.h uses rdtscp, which is more accurate than rdtsc,
+// but rdtscp disables processor's out-of-order execution, so it can be slow
+Y_FORCE_INLINE ui64 GetCycleCountFast() {
+#if defined(_MSC_VER)
+ // Generates the rdtsc instruction, which returns the processor time stamp.
+ // The processor time stamp records the number of clock cycles since the last reset.
+ return __rdtsc();
+#elif defined(__clang__) && !defined(_arm64_)
+ return __builtin_readcyclecounter();
+#elif defined(_x86_64_)
+ unsigned hi, lo;
+ __asm__ __volatile__("rdtsc"
+ : "=a"(lo), "=d"(hi));
+ return ((unsigned long long)lo) | (((unsigned long long)hi) << 32);
+#elif defined(_i386_)
+ ui64 x;
+ __asm__ volatile("rdtsc\n\t"
+ : "=A"(x));
+ return x;
+#elif defined(_darwin_)
+ return mach_absolute_time();
+#elif defined(_arm32_)
+ return MicroSeconds();
+#elif defined(_arm64_)
+ ui64 x;
+
+ __asm__ __volatile__("isb; mrs %0, cntvct_el0"
+ : "=r"(x));
+
+ return x;
+#else
+#error "unsupported arch"
+#endif
+}
+
+// NHPTimer::GetTime fast analog
+Y_FORCE_INLINE void GetTimeFast(NHPTimer::STime* pTime) noexcept {
+ *pTime = GetCycleCountFast();
+}
+
+namespace NActors {
+ inline double Ts2Ns(ui64 ts) {
+ return NHPTimer::GetSeconds(ts) * 1e9;
+ }
+
+ inline double Ts2Us(ui64 ts) {
+ return NHPTimer::GetSeconds(ts) * 1e6;
+ }
+
+ inline double Ts2Ms(ui64 ts) {
+ return NHPTimer::GetSeconds(ts) * 1e3;
+ }
+
+ inline ui64 Us2Ts(double us) {
+ return ui64(NHPTimer::GetClockRate() * us / 1e6);
+ }
+
+ struct TTimeTracker {
+ ui64 Ts;
+ TTimeTracker(): Ts(GetCycleCountFast()) {}
+ ui64 Elapsed() {
+ ui64 ts = GetCycleCountFast();
+ std::swap(Ts, ts);
+ return Ts - ts;
+ }
+ };
+}
diff --git a/library/cpp/actors/util/thread.h b/library/cpp/actors/util/thread.h
index d742c8c585..4cab922647 100644
--- a/library/cpp/actors/util/thread.h
+++ b/library/cpp/actors/util/thread.h
@@ -5,7 +5,7 @@
#include <util/system/execpath.h>
#include <util/system/thread.h>
#include <util/system/thread.h>
-#include <time.h>
+#include <time.h>
inline void SetCurrentThreadName(const TString& name,
const ui32 maxCharsFromProcessName = 8) {
diff --git a/library/cpp/actors/util/timerfd.h b/library/cpp/actors/util/timerfd.h
index 3189e2a672..a9b35ff9e7 100644
--- a/library/cpp/actors/util/timerfd.h
+++ b/library/cpp/actors/util/timerfd.h
@@ -1,65 +1,65 @@
-#pragma once
-
-#include "datetime.h"
-
-#include <util/generic/noncopyable.h>
-
-#ifdef _linux_
-
-#include <util/system/yassert.h>
-#include <errno.h>
-#include <sys/timerfd.h>
-
-struct TTimerFd: public TNonCopyable {
- int Fd;
-
- TTimerFd() {
- Fd = timerfd_create(CLOCK_MONOTONIC, 0);
- Y_VERIFY(Fd != -1, "timerfd_create(CLOCK_MONOTONIC, 0) -> -1; errno:%d: %s", int(errno), strerror(errno));
- }
-
- ~TTimerFd() {
- close(Fd);
- }
-
- void Set(ui64 ts) {
- ui64 now = GetCycleCountFast();
- Arm(now >= ts? 1: NHPTimer::GetSeconds(ts - now) * 1e9);
- }
-
- void Reset() {
- Arm(0); // disarm timer
- }
-
- void Wait() {
- ui64 expirations;
- ssize_t s = read(Fd, &expirations, sizeof(ui64));
- Y_UNUSED(s); // Y_VERIFY(s == sizeof(ui64));
- }
-
- void Wake() {
- Arm(1);
- }
-private:
- void Arm(ui64 ns) {
- struct itimerspec spec;
- spec.it_value.tv_sec = ns / 1'000'000'000;
- spec.it_value.tv_nsec = ns % 1'000'000'000;
- spec.it_interval.tv_sec = 0;
- spec.it_interval.tv_nsec = 0;
- int ret = timerfd_settime(Fd, 0, &spec, nullptr);
- Y_VERIFY(ret != -1, "timerfd_settime(%d, 0, %" PRIu64 "ns, 0) -> %d; errno:%d: %s", Fd, ns, ret, int(errno), strerror(errno));
- }
-};
-
-#else
-
-struct TTimerFd: public TNonCopyable {
- int Fd = 0;
- void Set(ui64) {}
- void Reset() {}
- void Wait() {}
- void Wake() {}
-};
-
-#endif
+#pragma once
+
+#include "datetime.h"
+
+#include <util/generic/noncopyable.h>
+
+#ifdef _linux_
+
+#include <util/system/yassert.h>
+#include <errno.h>
+#include <sys/timerfd.h>
+
+struct TTimerFd: public TNonCopyable {
+ int Fd;
+
+ TTimerFd() {
+ Fd = timerfd_create(CLOCK_MONOTONIC, 0);
+ Y_VERIFY(Fd != -1, "timerfd_create(CLOCK_MONOTONIC, 0) -> -1; errno:%d: %s", int(errno), strerror(errno));
+ }
+
+ ~TTimerFd() {
+ close(Fd);
+ }
+
+ void Set(ui64 ts) {
+ ui64 now = GetCycleCountFast();
+ Arm(now >= ts? 1: NHPTimer::GetSeconds(ts - now) * 1e9);
+ }
+
+ void Reset() {
+ Arm(0); // disarm timer
+ }
+
+ void Wait() {
+ ui64 expirations;
+ ssize_t s = read(Fd, &expirations, sizeof(ui64));
+ Y_UNUSED(s); // Y_VERIFY(s == sizeof(ui64));
+ }
+
+ void Wake() {
+ Arm(1);
+ }
+private:
+ void Arm(ui64 ns) {
+ struct itimerspec spec;
+ spec.it_value.tv_sec = ns / 1'000'000'000;
+ spec.it_value.tv_nsec = ns % 1'000'000'000;
+ spec.it_interval.tv_sec = 0;
+ spec.it_interval.tv_nsec = 0;
+ int ret = timerfd_settime(Fd, 0, &spec, nullptr);
+ Y_VERIFY(ret != -1, "timerfd_settime(%d, 0, %" PRIu64 "ns, 0) -> %d; errno:%d: %s", Fd, ns, ret, int(errno), strerror(errno));
+ }
+};
+
+#else
+
+struct TTimerFd: public TNonCopyable {
+ int Fd = 0;
+ void Set(ui64) {}
+ void Reset() {}
+ void Wait() {}
+ void Wake() {}
+};
+
+#endif
diff --git a/library/cpp/actors/util/ya.make b/library/cpp/actors/util/ya.make
index 37488c3962..a8bfd05d5a 100644
--- a/library/cpp/actors/util/ya.make
+++ b/library/cpp/actors/util/ya.make
@@ -8,8 +8,8 @@ OWNER(
SRCS(
affinity.cpp
affinity.h
- cpumask.h
- datetime.h
+ cpumask.h
+ datetime.h
defs.h
funnel_queue.h
futex.h
@@ -26,7 +26,7 @@ SRCS(
threadparkpad.cpp
threadparkpad.h
ticket_lock.h
- timerfd.h
+ timerfd.h
unordered_cache.h
)