aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authortesseract <tesseract@yandex-team.com>2023-02-28 19:40:15 +0300
committertesseract <tesseract@yandex-team.com>2023-02-28 19:40:15 +0300
commite35b11c3153c04a5a53accd88fadf5433a2261e9 (patch)
tree0be57e2a72bc4537dbbeabfebd689cdc86bcecf1
parent79b8c36df4fd7ec9ce425cf48371d7ef64f7d100 (diff)
downloadydb-e35b11c3153c04a5a53accd88fadf5433a2261e9.tar.gz
В schemeshard-е поддерживать статистику об используемом месте топиками пользователя (выделение методов)
-rw-r--r--ydb/core/tx/schemeshard/schemeshard__stats.h132
-rw-r--r--ydb/core/tx/schemeshard/schemeshard__stats_impl.h142
-rw-r--r--ydb/core/tx/schemeshard/schemeshard__table_stats.cpp162
-rw-r--r--ydb/core/tx/schemeshard/schemeshard_impl.cpp6
-rw-r--r--ydb/core/tx/schemeshard/schemeshard_impl.h50
-rw-r--r--ydb/core/tx/schemeshard/schemeshard_private.h6
6 files changed, 331 insertions, 167 deletions
diff --git a/ydb/core/tx/schemeshard/schemeshard__stats.h b/ydb/core/tx/schemeshard/schemeshard__stats.h
new file mode 100644
index 0000000000..659e57f087
--- /dev/null
+++ b/ydb/core/tx/schemeshard/schemeshard__stats.h
@@ -0,0 +1,132 @@
+#pragma once
+
+#include "schemeshard.h"
+#include "schemeshard_private.h"
+#include "schemeshard_tx_infly.h"
+#include "schemeshard__operation.h"
+
+#include <ydb/core/tablet_flat/tablet_flat_executed.h>
+
+namespace NKikimr {
+namespace NSchemeShard {
+
+struct TStatsId {
+ TPathId PathId;
+ TTabletId Datashard;
+
+ TStatsId(const TPathId& pathId, const TTabletId& datashard)
+ : PathId(pathId)
+ , Datashard(datashard)
+ {
+ }
+
+ bool operator==(const TStatsId& rhs) const {
+ return PathId == rhs.PathId && Datashard == rhs.Datashard;
+ }
+
+ struct THash {
+ inline size_t operator()(const TStatsId& obj) const {
+ return MultiHash(obj.PathId.Hash(), obj.Datashard);
+ }
+ };
+};
+
+template<typename TEvPeriodicStats>
+struct TStatsQueueItem {
+ typename TEvPeriodicStats::TPtr Ev;
+ TStatsId Id;
+ TMonotonic Ts;
+
+ TStatsQueueItem(typename TEvPeriodicStats::TPtr ev, const TStatsId& id)
+ : Ev(ev)
+ , Id(id)
+ , Ts(AppData()->MonotonicTimeProvider->Now())
+ {}
+
+ TPathId PathId() {
+ return Id.PathId;
+ }
+};
+
+enum EStatsQueueStatus {
+ READY,
+ NOT_READY
+};
+
+template<typename TEvent>
+class TStatsQueue {
+public:
+ using TEventPtr = typename TEvent::TPtr;
+ using TItem = TStatsQueueItem<TEvent>;
+ using TStatsMap = THashMap<TStatsId, TItem*, TStatsId::THash>;
+ using TStatsQ = TStatsQueue<TEvent>;
+
+ TStatsQueue(TSchemeShard* ss, ESimpleCounters queueSizeCounter, ECumulativeCounters writtenCounter, EPercentileCounters latencyCounter)
+ : QueueSizeCounter(queueSizeCounter)
+ , WrittenCounter(writtenCounter)
+ , LatencyCounter(latencyCounter)
+ , SS(ss)
+ {}
+
+ EStatsQueueStatus Add(TStatsId statsId, TEventPtr ev);
+ TItem Next();
+
+ TDuration Age() const;
+ TDuration Delay() const;
+
+ bool Empty() const;
+ size_t Size() const;
+
+ bool BatchingEnabled() const;
+ TDuration BatchTimeout() const;
+ ui32 MaxBatchSize() const;
+ TDuration MaxExecuteTime() const;
+
+ void WriteLatencyMetric(const TItem& item);
+ void WriteQueueSizeMetric();
+
+ const ESimpleCounters QueueSizeCounter;
+ const ECumulativeCounters WrittenCounter;
+ const EPercentileCounters LatencyCounter;
+
+private:
+ EStatsQueueStatus Status() const;
+
+ TSchemeShard* SS;
+
+ TStatsMap Map;
+ TDeque<TItem> Queue;
+};
+
+
+template<typename TEvent>
+class TTxStoreStats: public NTabletFlatExecutor::TTransactionBase<TSchemeShard> {
+ using TStatsQ = TStatsQueue<TEvent>;
+ using TItem = TStatsQueueItem<TEvent>;
+
+protected:
+ TStatsQ& Queue;
+ bool& PersistStatsPending;
+
+public:
+ TTxStoreStats(TSchemeShard* ss, TStatsQ& queue, bool& persistStatsPending)
+ : TBase(ss)
+ , Queue(queue)
+ , PersistStatsPending(persistStatsPending)
+ {
+ }
+
+ virtual ~TTxStoreStats() = default;
+
+ TTxType GetTxType() const override {
+ return TXTYPE_STORE_PARTITION_STATS;
+ }
+
+ bool Execute(TTransactionContext& txc, const TActorContext& ctx) override;
+
+ // returns true to continue batching
+ virtual bool PersistSingleStats(const TPathId& pathId, const TItem& item, TTransactionContext& txc, const TActorContext& ctx) = 0;
+};
+
+} // NSchemeShard
+} // NKikimr
diff --git a/ydb/core/tx/schemeshard/schemeshard__stats_impl.h b/ydb/core/tx/schemeshard/schemeshard__stats_impl.h
new file mode 100644
index 0000000000..338de84899
--- /dev/null
+++ b/ydb/core/tx/schemeshard/schemeshard__stats_impl.h
@@ -0,0 +1,142 @@
+#pragma once
+
+#include "schemeshard_impl.h"
+
+namespace NKikimr {
+namespace NSchemeShard {
+
+template<typename TEvent>
+EStatsQueueStatus TStatsQueue<TEvent>::Add(TStatsId statsId, TEventPtr ev) {
+ typename TStatsMap::insert_ctx insertCtx;
+ auto it = Map.find(statsId, insertCtx);
+ if (it == Map.end()) {
+ Queue.emplace_back(ev, statsId);
+ Map.emplace_direct(insertCtx, statsId, &Queue.back());
+ } else {
+ // already in queue, just update
+ it->second->Ev = ev;
+ }
+
+ WriteQueueSizeMetric();
+
+ return Status();
+}
+
+template<typename TEvent>
+TDuration TStatsQueue<TEvent>::Age() const {
+ if (Empty()) {
+ return TDuration::Zero();
+ }
+ const auto& oldestItem = Queue.front();
+ return AppData()->MonotonicTimeProvider->Now() - oldestItem.Ts;
+}
+
+template<typename TEvent>
+bool TStatsQueue<TEvent>::BatchingEnabled() const {
+ return SS->StatsMaxBatchSize > 0;
+}
+
+template<typename TEvent>
+TDuration TStatsQueue<TEvent>::BatchTimeout() const {
+ return SS->StatsBatchTimeout;
+}
+
+template<typename TEvent>
+TDuration TStatsQueue<TEvent>::Delay() const {
+ return BatchTimeout() - Age();
+}
+
+template<typename TEvent>
+bool TStatsQueue<TEvent>::Empty() const {
+ return Queue.empty();
+}
+
+template<typename TEvent>
+ui32 TStatsQueue<TEvent>::MaxBatchSize() const {
+ return std::max<ui32>(SS->StatsMaxBatchSize, 1);
+}
+
+template<typename TEvent>
+TDuration TStatsQueue<TEvent>::MaxExecuteTime() const {
+ return SS->StatsMaxExecuteTime;
+}
+
+template<typename TEvent>
+TStatsQueueItem<TEvent> TStatsQueue<TEvent>::Next() {
+ auto item = Queue.front();
+ Queue.pop_front();
+
+ Map.erase(item.Id);
+
+ return item;
+}
+
+template<typename TEvent>
+EStatsQueueStatus TStatsQueue<TEvent>::Status() const {
+ if (Empty()) {
+ return NOT_READY;
+ }
+
+ if (!BatchingEnabled()) {
+ return READY;
+ }
+
+ if (Size() >= MaxBatchSize()) {
+ return READY;
+ }
+
+ if (!BatchTimeout() || Age() >= BatchTimeout()) {
+ return READY;
+ }
+
+ return NOT_READY;
+}
+
+template<typename TEvent>
+size_t TStatsQueue<TEvent>::Size() const {
+ return Queue.size();
+}
+
+template<typename TEvent>
+void TStatsQueue<TEvent>::WriteLatencyMetric(const TItem& item) {
+ auto timeInQueue = AppData()->MonotonicTimeProvider->Now() - item.Ts;
+ SS->TabletCounters->Percentile()[LatencyCounter].IncrementFor(timeInQueue.MicroSeconds());
+}
+
+template<typename TEvent>
+void TStatsQueue<TEvent>::WriteQueueSizeMetric() {
+ SS->TabletCounters->Simple()[QueueSizeCounter].Set(Size());
+}
+
+
+template<typename TEvent>
+bool TTxStoreStats<TEvent>::Execute(NTabletFlatExecutor::TTransactionContext& txc, const TActorContext& ctx) {
+ PersistStatsPending = false;
+
+ if (Queue.Empty()) {
+ LOG_DEBUG_S(ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, "TTxStoreStats::Execute empty");
+ return true;
+ }
+
+ TMonotonic start = TMonotonic::Now();
+ const ui32 maxBatchSize = Queue.MaxBatchSize();
+ ui32 batchSize = 0;
+ for (; batchSize < maxBatchSize && !Queue.Empty(); ++batchSize) {
+ auto item = Queue.Next();
+ Queue.WriteLatencyMetric(item);
+ if (!PersistSingleStats(item.PathId(), item, txc, ctx)) {
+ break;
+ }
+
+ if ((TMonotonic::Now() - start) >= Queue.MaxExecuteTime()) {
+ break;
+ }
+ }
+
+ Self->TabletCounters->Cumulative()[Queue.WrittenCounter].Increment(batchSize);
+
+ return true;
+}
+
+} // NSchemeShard
+} // NKikimr
diff --git a/ydb/core/tx/schemeshard/schemeshard__table_stats.cpp b/ydb/core/tx/schemeshard/schemeshard__table_stats.cpp
index 0929d8bb89..30a6a0e114 100644
--- a/ydb/core/tx/schemeshard/schemeshard__table_stats.cpp
+++ b/ydb/core/tx/schemeshard/schemeshard__table_stats.cpp
@@ -1,4 +1,6 @@
#include "schemeshard_impl.h"
+#include "schemeshard__stats_impl.h"
+
#include <ydb/core/base/appdata.h>
#include <ydb/core/base/cputime.h>
#include <ydb/core/protos/sys_view.pb.h>
@@ -61,7 +63,7 @@ auto TSchemeShard::BuildStatsForCollector(TPathId pathId, TShardIdx shardIdx, TT
return ev;
}
-class TTxStorePartitionStats: public NTabletFlatExecutor::TTransactionBase<TSchemeShard> {
+class TTxStoreTableStats: public TTxStoreStats<TEvDataShard::TEvPeriodicTableStats> {
TSideEffects MergeOpSideEffects;
struct TMessage {
@@ -77,24 +79,20 @@ class TTxStorePartitionStats: public NTabletFlatExecutor::TTransactionBase<TSche
TVector<TMessage> PendingMessages;
public:
- TTxStorePartitionStats(TSelf* self)
- : TBase(self)
+ TTxStoreTableStats(TSchemeShard* ss, TStatsQueue<TEvDataShard::TEvPeriodicTableStats>& queue, bool& persistStatsPending)
+ : TTxStoreStats(ss, queue, persistStatsPending)
{
}
- virtual ~TTxStorePartitionStats() = default;
-
- TTxType GetTxType() const override {
- return TXTYPE_STORE_PARTITION_STATS;
- }
+ virtual ~TTxStoreTableStats() = default;
- bool Execute(TTransactionContext& txc, const TActorContext& ctx) override;
void Complete(const TActorContext& ctx) override;
// returns true to continue batching
- bool PersistSingleStats(TTransactionContext& txc, const TActorContext& ctx);
+ bool PersistSingleStats(const TPathId& pathId, const TStatsQueue<TEvDataShard::TEvPeriodicTableStats>::TItem& item, TTransactionContext& txc, const TActorContext& ctx) override;
};
+
THolder<TProposeRequest> MergeRequest(
TSchemeShard* ss, TTxId& txId, TPathId& pathId, const TVector<TShardIdx>& shardsToMerge)
{
@@ -122,53 +120,10 @@ THolder<TProposeRequest> MergeRequest(
return std::move(request);
}
-bool TTxStorePartitionStats::Execute(TTransactionContext& txc, const TActorContext& ctx) {
- Self->PersistStatsPending = false;
-
- if (Self->StatsQueue.empty())
- return true;
-
- TMonotonic start = TMonotonic::Now();
-
- const ui32 maxBatchSize = Self->StatsMaxBatchSize ? Self->StatsMaxBatchSize : 1;
- ui32 batchSize = 0;
- while (batchSize < maxBatchSize && !Self->StatsQueue.empty()) {
- ++batchSize;
- if (!PersistSingleStats(txc, ctx))
- break;
-
- if ((TMonotonic::Now() - start) >= Self->StatsMaxExecuteTime)
- break;
- }
-
- Self->TabletCounters->Cumulative()[COUNTER_STATS_WRITTEN].Increment(batchSize);
-
- bool isBatchingDisabled = Self->StatsMaxBatchSize == 0;
- if (isBatchingDisabled) {
- // there will be per stat transaction, don't need to schedule additional one
- return true;
- }
-
- if (!Self->StatsQueue.empty()) {
- Self->ScheduleStatsBatch(ctx);
- }
-
- return true;
-}
-
-bool TTxStorePartitionStats::PersistSingleStats(TTransactionContext& txc, const TActorContext& ctx) {
- auto item = Self->StatsQueue.front();
- Self->StatsQueue.pop_front();
-
- auto timeInQueue = AppData()->MonotonicTimeProvider->Now() - item.Ts;
- Self->TabletCounters->Percentile()[COUNTER_STATS_BATCH_LATENCY].IncrementFor(timeInQueue.MicroSeconds());
-
+bool TTxStoreTableStats::PersistSingleStats(const TPathId& pathId,
+ const TStatsQueueItem<TEvDataShard::TEvPeriodicTableStats>& item, NTabletFlatExecutor::TTransactionContext& txc, const TActorContext& ctx) {
const auto& rec = item.Ev->Get()->Record;
auto datashardId = TTabletId(rec.GetDatashardId());
- const TPathId& pathId = item.PathId;
-
- TSchemeShard::TStatsId statsId(pathId, datashardId);
- Self->StatsMap.erase(statsId);
const auto& tableStats = rec.GetTableStats();
const auto& tabletMetrics = rec.GetTabletMetrics();
@@ -434,7 +389,7 @@ bool TTxStorePartitionStats::PersistSingleStats(TTransactionContext& txc, const
return true;
}
-void TTxStorePartitionStats::Complete(const TActorContext& ctx) {
+void TTxStoreTableStats::Complete(const TActorContext& ctx) {
MergeOpSideEffects.ApplyOnComplete(Self, ctx);
for (auto& m: PendingMessages) {
@@ -442,7 +397,7 @@ void TTxStorePartitionStats::Complete(const TActorContext& ctx) {
ctx.Send(m.Actor, m.Event.Release());
}
- Self->TabletCounters->Simple()[COUNTER_STATS_QUEUE_SIZE].Set(Self->StatsQueue.size());
+ Queue.WriteQueueSizeMetric();
}
void TSchemeShard::Handle(TEvDataShard::TEvPeriodicTableStats::TPtr& ev, const TActorContext& ctx) {
@@ -468,81 +423,48 @@ void TSchemeShard::Handle(TEvDataShard::TEvPeriodicTableStats::TPtr& ev, const T
<< " cpuUsage " << tabletMetrics.GetCPU()/10000.0);
TStatsId statsId(pathId, datashardId);
- TStatsMap::insert_ctx insertCtx;
- auto it = StatsMap.find(statsId, insertCtx);
- if (it == StatsMap.end()) {
- StatsQueue.emplace_back(ev.Release(), pathId);
- StatsMap.emplace_direct(insertCtx, statsId, &StatsQueue.back());
- } else {
- // already in queue, just update
- it->second->Ev = ev.Release();
- }
-
- TabletCounters->Simple()[COUNTER_STATS_QUEUE_SIZE].Set(StatsQueue.size());
- ScheduleStatsBatch(ctx);
-}
+
+ switch(TableStatsQueue.Add(statsId, ev.Release())) {
+ case READY:
+ ExecuteTableStatsBatch(ctx);
+ break;
-void TSchemeShard::ScheduleStatsBatch(const TActorContext& ctx) {
- if (StatsQueue.empty())
- return;
+ case NOT_READY:
+ ScheduleTableStatsBatch(ctx);
+ break;
- bool isBatchingDisabled = StatsMaxBatchSize == 0;
- if (isBatchingDisabled) {
- PersistStatsPending = true;
- Execute(new TTxStorePartitionStats(this), ctx);
- LOG_TRACE_S(ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, "Will execute TTxStorePartitionStats without batch");
- return;
+ default:
+ Y_FAIL("Unknown batch status");
}
+}
- if (PersistStatsPending)
- return;
+void TSchemeShard::Handle(TEvPrivate::TEvPersistTableStats::TPtr&, const TActorContext& ctx) {
+ LOG_INFO_S(ctx, NKikimrServices::FLAT_TX_SCHEMESHARD,
+ "Started TEvPersistStats at tablet " << TabletID() << ", queue size# " << TableStatsQueue.Size());
- if (StatsQueue.size() >= StatsMaxBatchSize || !StatsBatchTimeout) {
- // note that we don't care if already scheduled
- PersistStatsPending = true;
- Execute(new TTxStorePartitionStats(this), ctx);
- LOG_TRACE_S(ctx, NKikimrServices::FLAT_TX_SCHEMESHARD,
- "Will execute TTxStorePartitionStats, queue# " << StatsQueue.size());
- return;
- }
+ TableStatsBatchScheduled = false;
+ ExecuteTableStatsBatch(ctx);
+}
- const auto& oldestItem = StatsQueue.front();
- auto age = AppData()->MonotonicTimeProvider->Now() - oldestItem.Ts;
- if (age >= StatsBatchTimeout) {
- PersistStatsPending = true;
- Execute(new TTxStorePartitionStats(this), ctx);
+void TSchemeShard::ExecuteTableStatsBatch(const TActorContext& ctx) {
+ if (!TablePersistStatsPending && !TableStatsQueue.Empty()) {
+ TablePersistStatsPending = true;
+ Execute(new TTxStoreTableStats(this, TableStatsQueue, TablePersistStatsPending), ctx);
LOG_TRACE_S(ctx, NKikimrServices::FLAT_TX_SCHEMESHARD,
- "Will execute TTxStorePartitionStats because of age, queue# " << StatsQueue.size());
- return;
+ "Will execute TTxStoreStats, queue# " << TableStatsQueue.Size());
+ ScheduleTableStatsBatch(ctx);
}
-
- if (StatsBatchScheduled)
- return;
-
- auto delay = StatsBatchTimeout - age;
- LOG_TRACE_S(ctx, NKikimrServices::FLAT_TX_SCHEMESHARD,
- "Will delay TTxStorePartitionStats on# " << delay << ", queue# " << StatsQueue.size());
-
- ctx.Schedule(delay, new TEvPrivate::TEvPersistStats());
- StatsBatchScheduled = true;
}
-void TSchemeShard::Handle(TEvPrivate::TEvPersistStats::TPtr&, const TActorContext& ctx) {
- LOG_INFO_S(ctx, NKikimrServices::FLAT_TX_SCHEMESHARD,
- "Started TEvPersistStats at tablet " << TabletID() << ", queue size# " << StatsQueue.size());
-
- StatsBatchScheduled = false;
-
- if (PersistStatsPending) {
- return;
- }
+void TSchemeShard::ScheduleTableStatsBatch(const TActorContext& ctx) {
+ if (!TableStatsBatchScheduled && !TableStatsQueue.Empty()) {
+ TDuration delay = TableStatsQueue.Delay();
+ LOG_TRACE_S(ctx, NKikimrServices::FLAT_TX_SCHEMESHARD,
+ "Will delay TTxStoreTableStats on# " << delay << ", queue# " << TableStatsQueue.Size());
- if (StatsQueue.empty()) {
- return;
+ ctx.Schedule(delay, new TEvPrivate::TEvPersistTableStats());
+ TableStatsBatchScheduled = true;
}
-
- PersistStatsPending = true;
- Execute(new TTxStorePartitionStats(this), ctx);
}
}}
diff --git a/ydb/core/tx/schemeshard/schemeshard_impl.cpp b/ydb/core/tx/schemeshard/schemeshard_impl.cpp
index 8cd87155d1..ab7f1a0c77 100644
--- a/ydb/core/tx/schemeshard/schemeshard_impl.cpp
+++ b/ydb/core/tx/schemeshard/schemeshard_impl.cpp
@@ -3963,6 +3963,10 @@ TSchemeShard::TSchemeShard(const TActorId &tablet, TTabletStorageInfo *info)
, CompactionStarter(this)
, BorrowedCompactionStarter(this)
, ShardDeleter(info->TabletID)
+ , TableStatsQueue(this,
+ COUNTER_STATS_QUEUE_SIZE,
+ COUNTER_STATS_WRITTEN,
+ COUNTER_STATS_BATCH_LATENCY)
, AllowDataColumnForIndexTable(0, 0, 1)
{
TabletCountersPtr.Reset(new TProtobufTabletCounters<
@@ -4350,7 +4354,7 @@ void TSchemeShard::StateWork(STFUNC_SIG) {
HFuncTraced(TEvPrivate::TEvCleanDroppedSubDomains, Handle);
HFuncTraced(TEvPrivate::TEvSubscribeToShardDeletion, Handle);
- HFuncTraced(TEvPrivate::TEvPersistStats, Handle);
+ HFuncTraced(TEvPrivate::TEvPersistTableStats, Handle);
HFuncTraced(TEvSchemeShard::TEvLogin, Handle);
diff --git a/ydb/core/tx/schemeshard/schemeshard_impl.h b/ydb/core/tx/schemeshard/schemeshard_impl.h
index 9e753a55bd..ae0cf92367 100644
--- a/ydb/core/tx/schemeshard/schemeshard_impl.h
+++ b/ydb/core/tx/schemeshard/schemeshard_impl.h
@@ -17,6 +17,7 @@
#include "schemeshard_utils.h"
#include "schemeshard_schema.h"
#include "schemeshard__operation.h"
+#include "schemeshard__stats.h"
#include <ydb/core/base/hive.h>
#include <ydb/core/base/storage_pools.h>
@@ -281,47 +282,9 @@ public:
THashMap<TTxState::ETxType, ui32> InFlightLimits;
// time when we opened the batch
- TMonotonic StatsBatchStartTs;
- bool StatsBatchScheduled = false;
- bool PersistStatsPending = false;
-
- struct TStatsQueueItem {
- TEvDataShard::TEvPeriodicTableStats::TPtr Ev;
- TPathId PathId;
- TMonotonic Ts;
-
- TStatsQueueItem(TEvDataShard::TEvPeriodicTableStats::TPtr ev, const TPathId& pathId)
- : Ev(ev)
- , PathId(pathId)
- , Ts(AppData()->MonotonicTimeProvider->Now())
- {}
- };
-
- struct TStatsId {
- TPathId PathId;
- TTabletId Datashard;
-
- TStatsId(const TPathId& pathId, const TTabletId& datashard)
- : PathId(pathId)
- , Datashard(datashard)
- {
- }
-
- bool operator==(const TStatsId& rhs) const {
- return PathId == rhs.PathId && Datashard == rhs.Datashard;
- }
-
- struct THash {
- inline size_t operator()(const TStatsId& obj) const {
- return MultiHash(obj.PathId.Hash(), obj.Datashard);
- }
- };
- };
-
- using TStatsMap = THashMap<TStatsId, TStatsQueueItem*, TStatsId::THash>;
-
- TStatsMap StatsMap;
- TDeque<TStatsQueueItem> StatsQueue;
+ bool TableStatsBatchScheduled = false;
+ bool TablePersistStatsPending = false;
+ TStatsQueue<TEvDataShard::TEvPeriodicTableStats> TableStatsQueue;
TSet<TPathId> CleanDroppedPathsCandidates;
TSet<TPathId> CleanDroppedSubDomainsCandidates;
@@ -1020,8 +983,9 @@ public:
void Handle(TEvDataShard::TEvSplitAck::TPtr& ev, const TActorContext& ctx);
void Handle(TEvDataShard::TEvSplitPartitioningChangedAck::TPtr& ev, const TActorContext& ctx);
- void ScheduleStatsBatch(const TActorContext& ctx);
- void Handle(TEvPrivate::TEvPersistStats::TPtr& ev, const TActorContext& ctx);
+ void ExecuteTableStatsBatch(const TActorContext& ctx);
+ void ScheduleTableStatsBatch(const TActorContext& ctx);
+ void Handle(TEvPrivate::TEvPersistTableStats::TPtr& ev, const TActorContext& ctx);
void Handle(TEvDataShard::TEvPeriodicTableStats::TPtr& ev, const TActorContext& ctx);
void Handle(TEvDataShard::TEvGetTableStatsResult::TPtr& ev, const TActorContext& ctx);
diff --git a/ydb/core/tx/schemeshard/schemeshard_private.h b/ydb/core/tx/schemeshard/schemeshard_private.h
index 6a2171865c..fbe6c1e04d 100644
--- a/ydb/core/tx/schemeshard/schemeshard_private.h
+++ b/ydb/core/tx/schemeshard/schemeshard_private.h
@@ -24,7 +24,7 @@ struct TEvPrivate {
EvRunBorrowedCompaction,
EvCompletePublication,
EvCompleteBarrier,
- EvPersistStats,
+ EvPersistTableStats,
EvConsoleConfigsTimeout,
EvRunCdcStreamScan,
EvEnd
@@ -159,8 +159,8 @@ struct TEvPrivate {
}
};
- struct TEvPersistStats: public TEventLocal<TEvPersistStats, EvPersistStats> {
- TEvPersistStats() = default;
+ struct TEvPersistTableStats: public TEventLocal<TEvPersistTableStats, EvPersistTableStats> {
+ TEvPersistTableStats() = default;
};
struct TEvConsoleConfigsTimeout: public TEventLocal<TEvConsoleConfigsTimeout, EvConsoleConfigsTimeout> {