diff --git a/ydb/core/kqp/ut/olap/kqp_olap_stats_ut.cpp b/ydb/core/kqp/ut/olap/kqp_olap_stats_ut.cpp index 635b8ea745d2..2d34fb23496a 100644 --- a/ydb/core/kqp/ut/olap/kqp_olap_stats_ut.cpp +++ b/ydb/core/kqp/ut/olap/kqp_olap_stats_ut.cpp @@ -1,7 +1,7 @@ -#include #include -#include +#include #include +#include namespace NKikimr { namespace NKqp { @@ -15,12 +15,11 @@ Y_UNIT_TEST_SUITE(KqpOlapStats) { constexpr size_t size_single_table = 13152; const TVector schema = { - TTestHelper::TColumnSchema().SetName("id").SetType(NScheme::NTypeIds::Int32).SetNullable(false), - TTestHelper::TColumnSchema().SetName("resource_id").SetType(NScheme::NTypeIds::Utf8), - TTestHelper::TColumnSchema().SetName("level").SetType(NScheme::NTypeIds::Int32) - }; + TTestHelper::TColumnSchema().SetName("id").SetType(NScheme::NTypeIds::Int32).SetNullable(false), + TTestHelper::TColumnSchema().SetName("resource_id").SetType(NScheme::NTypeIds::Utf8), + TTestHelper::TColumnSchema().SetName("level").SetType(NScheme::NTypeIds::Int32)}; - class TOlapStatsController: public NYDBTest::NColumnShard::TController { + class TOlapStatsController : public NYDBTest::NColumnShard::TController { public: TDuration GetPeriodicWakeupActivationPeriod(const TDuration /*defaultValue*/) const override { return TDuration::MilliSeconds(10); @@ -32,7 +31,7 @@ Y_UNIT_TEST_SUITE(KqpOlapStats) { Y_UNIT_TEST(AddRowsTableStandalone) { auto csController = NYDBTest::TControllers::RegisterCSControllerGuard(); - + TKikimrSettings runnerSettings; runnerSettings.WithSampleTables = false; @@ -44,8 +43,8 @@ Y_UNIT_TEST_SUITE(KqpOlapStats) { testHelper.CreateTable(testTable); { TTestHelper::TUpdatesBuilder tableInserter(testTable.GetArrowSchema(schema)); - - for(size_t i=0; iTryUpdate(ctx); } -void TColumnShard::ConfigureStats(const NOlap::TColumnEngineStats& indexStats, ::NKikimrTableStats::TTableStats * tabletStats) { +void TColumnShard::ConfigureStats(const NOlap::TColumnEngineStats& indexStats, + ::NKikimrTableStats::TTableStats* tabletStats) { NOlap::TSnapshot lastIndexUpdate = TablesManager.GetPrimaryIndexSafe().LastUpdate(); - auto activeIndexStats = indexStats.Active(); // data stats excluding inactive and evicted + auto activeIndexStats = indexStats.Active(); // data stats excluding inactive and evicted if (activeIndexStats.Rows < 0 || activeIndexStats.Bytes < 0) { - LOG_S_WARN("Negative stats counter. Rows: " << activeIndexStats.Rows - << " Bytes: " << activeIndexStats.Bytes << TabletID()); + LOG_S_WARN("Negative stats counter. Rows: " << activeIndexStats.Rows << " Bytes: " << activeIndexStats.Bytes + << TabletID()); activeIndexStats.Rows = (activeIndexStats.Rows < 0) ? 0 : activeIndexStats.Rows; activeIndexStats.Bytes = (activeIndexStats.Bytes < 0) ? 0 : activeIndexStats.Bytes; @@ -309,7 +310,7 @@ void TColumnShard::ConfigureStats(const NOlap::TColumnEngineStats& indexStats, : tabletStats->SetDataSize(activeIndexStats.Bytes + TabletCounters->Simple()[COUNTER_COMMITTED_BYTES].Get()); // TODO: we need row/dataSize counters for evicted data (managed by tablet but stored outside) - //tabletStats->SetIndexSize(); // TODO: calc size of internal tables + // tabletStats->SetIndexSize(); // TODO: calc size of internal tables tabletStats->SetLastAccessTime(LastAccessTime.MilliSeconds()); tabletStats->SetLastUpdateTime(lastIndexUpdate.GetPlanStep()); @@ -330,7 +331,7 @@ void TColumnShard::FillTxTableStats(::NKikimrTableStats::TTableStats* tableStats } void TColumnShard::FillOlapStats(const TActorContext& ctx, std::unique_ptr& ev) { - ev->Record.SetShardState(2); // NKikimrTxDataShard.EDatashardState.Ready + ev->Record.SetShardState(2); // NKikimrTxDataShard.EDatashardState.Ready ev->Record.SetGeneration(Executor()->Generation()); ev->Record.SetRound(StatsReportRound++); ev->Record.SetNodeId(ctx.ExecutorThread.ActorSystem->NodeId); @@ -346,13 +347,14 @@ void TColumnShard::FillOlapStats(const TActorContext& ctx, std::unique_ptr& ev) { +void TColumnShard::FillColumnTableStats(const TActorContext& ctx, + std::unique_ptr& ev) { if (!TablesManager.HasPrimaryIndex()) { return; } const auto& tablesIndexStats = TablesManager.MutablePrimaryIndex().GetStats(); LOG_S_DEBUG("There are stats for " << tablesIndexStats.size() << " tables"); - for(const auto& [tableLocalID, columnStats] : tablesIndexStats) { + for (const auto& [tableLocalID, columnStats] : tablesIndexStats) { if (!columnStats) { LOG_S_ERROR("SendPeriodicStats: empty stats"); continue; @@ -362,7 +364,7 @@ void TColumnShard::FillColumnTableStats(const TActorContext& ctx, std::unique_pt periodicTableStats->SetDatashardId(TabletID()); periodicTableStats->SetTableLocalId(tableLocalID); - periodicTableStats->SetShardState(2); // NKikimrTxDataShard.EDatashardState.Ready + periodicTableStats->SetShardState(2); // NKikimrTxDataShard.EDatashardState.Ready periodicTableStats->SetGeneration(Executor()->Generation()); periodicTableStats->SetRound(StatsReportRound++); periodicTableStats->SetNodeId(ctx.ExecutorThread.ActorSystem->NodeId); @@ -411,4 +413,4 @@ void TColumnShard::SendPeriodicStats() { NTabletPipe::SendData(ctx, StatsReportPipe, ev.release()); } -} +} // namespace NKikimr::NColumnShard diff --git a/ydb/core/tx/columnshard/columnshard_impl.h b/ydb/core/tx/columnshard/columnshard_impl.h index cbd33a169c6d..10d3e177e8e5 100644 --- a/ydb/core/tx/columnshard/columnshard_impl.h +++ b/ydb/core/tx/columnshard/columnshard_impl.h @@ -488,11 +488,12 @@ class TColumnShard void SendPeriodicStats(); void FillOlapStats(const TActorContext& ctx, std::unique_ptr& ev); void FillColumnTableStats(const TActorContext& ctx, std::unique_ptr& ev); - void ConfigureStats(const NOlap::TColumnEngineStats& indexStats, ::NKikimrTableStats::TTableStats * tabletStats); + void ConfigureStats(const NOlap::TColumnEngineStats& indexStats, ::NKikimrTableStats::TTableStats* tabletStats); void FillTxTableStats(::NKikimrTableStats::TTableStats* tableStats) const; static TDuration GetControllerPeriodicWakeupActivationPeriod(); static TDuration GetControllerStatsReportInterval(); + public: const std::shared_ptr& GetStoragesManager() const { return StoragesManager; diff --git a/ydb/core/tx/schemeshard/schemeshard__table_stats.cpp b/ydb/core/tx/schemeshard/schemeshard__table_stats.cpp index aebba18ca42a..0434df8d07cf 100644 --- a/ydb/core/tx/schemeshard/schemeshard__table_stats.cpp +++ b/ydb/core/tx/schemeshard/schemeshard__table_stats.cpp @@ -92,7 +92,7 @@ class TTxStoreTableStats: public TTxStoreStats::TItem& item, TTransactionContext& txc, const TActorContext& ctx) override; void ScheduleNextBatch(const TActorContext& ctx) override; - template + template TPartitionStats PrepareStats(const TActorContext& ctx, const T& rec) const; }; @@ -124,10 +124,8 @@ THolder MergeRequest( return std::move(request); } -template -TPartitionStats TTxStoreTableStats::PrepareStats( - const TActorContext& ctx, const T& rec) const { - +template +TPartitionStats TTxStoreTableStats::PrepareStats(const TActorContext& ctx, const T& rec) const { const auto& tableStats = rec.GetTableStats(); const auto& tabletMetrics = rec.GetTabletMetrics(); @@ -183,7 +181,8 @@ TPartitionStats TTxStoreTableStats::PrepareStats( } bool TTxStoreTableStats::PersistSingleStats(const TPathId& pathId, - const TStatsQueueItem& item, NTabletFlatExecutor::TTransactionContext& txc, const TActorContext& ctx) { + const TStatsQueueItem& item, + NTabletFlatExecutor::TTransactionContext& txc, const TActorContext& ctx) { const auto& rec = item.Ev->Get()->Record; const auto datashardId = TTabletId(rec.GetDatashardId()); @@ -208,19 +207,17 @@ bool TTxStoreTableStats::PersistSingleStats(const TPathId& pathId, TShardIdx shardIdx = Self->TabletIdToShardIdx[datashardId]; LOG_DEBUG_S(ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, - "TTxStoreTableStats.PersistSingleStats: main stats from" - << " datashardId(TabletID)=" << datashardId - << " maps to shardIdx: " << shardIdx - << ", pathId: " << pathId - << ", pathId map=" << Self->PathsById[pathId]->Name - << ", is column=" << isColumnTable - << ", is olap=" << isOlapStore); + "TTxStoreTableStats.PersistSingleStats: main stats from" + << " datashardId(TabletID)=" << datashardId << " maps to shardIdx: " << shardIdx + << ", pathId: " << pathId << ", pathId map=" << Self->PathsById[pathId]->Name + << ", is column=" << isColumnTable << ", is olap=" << isOlapStore); const TPartitionStats newStats = PrepareStats(ctx, rec); LOG_INFO_S(ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, - "Add stats from shard with datashardId(TabletID)=" << datashardId << ", pathId " << pathId.LocalPathId - << ": RowCount " << newStats.RowCount << ", DataSize " << newStats.DataSize); + "Add stats from shard with datashardId(TabletID)=" << datashardId << ", pathId " << pathId.LocalPathId + << ": RowCount " << newStats.RowCount << ", DataSize " + << newStats.DataSize); NIceDb::TNiceDb db(txc.DB); @@ -261,29 +258,28 @@ bool TTxStoreTableStats::PersistSingleStats(const TPathId& pathId, updateSubdomainInfo = true; const auto tables = rec.GetTables(); - LOG_DEBUG_S(ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, - "OLAP store contains " << tables.size() << " tables."); + LOG_DEBUG_S(ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, "OLAP store contains " << tables.size() << " tables."); - for(const auto& table : tables) { + for (const auto& table : tables) { const TPartitionStats newTableStats = PrepareStats(ctx, table); const TPathId tablePathId = TPathId(TOwnerId(pathId.OwnerId), TLocalPathId(table.GetTableLocalId())); if (Self->ColumnTables.contains(tablePathId)) { LOG_DEBUG_S(ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, - "add stats for exists table with pathId=" << tablePathId); + "add stats for exists table with pathId=" << tablePathId); auto columnTable = Self->ColumnTables.TakeVerified(tablePathId); columnTable->UpdateTableStats(tablePathId, newTableStats); } else { LOG_WARN_S(ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, - "failed add stats for table with pathId=" << tablePathId); + "failed add stats for table with pathId=" << tablePathId); } } } else if (isColumnTable) { - LOG_INFO_S(ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, "PersistSingleStats: ColumnTable rec.GetColumnTables() size=" - << rec.GetTables().size()); + LOG_INFO_S(ctx, NKikimrServices::FLAT_TX_SCHEMESHARD, + "PersistSingleStats: ColumnTable rec.GetColumnTables() size=" << rec.GetTables().size()); auto columnTable = Self->ColumnTables.TakeVerified(pathId); oldAggrStats = columnTable->GetStats().Aggregated; diff --git a/ydb/core/tx/schemeshard/schemeshard_path_describer.cpp b/ydb/core/tx/schemeshard/schemeshard_path_describer.cpp index 7e36ac8d1eb9..876fe8a7296b 100644 --- a/ydb/core/tx/schemeshard/schemeshard_path_describer.cpp +++ b/ydb/core/tx/schemeshard/schemeshard_path_describer.cpp @@ -393,7 +393,7 @@ void TPathDescriber::DescribeOlapStore(TPathId pathId, TPathElement::TPtr pathEl } void TPathDescriber::DescribeColumnTable(TPathId pathId, TPathElement::TPtr pathEl) { - const auto tableInfo = Self->ColumnTables.GetVerified(pathId); + const auto tableInfo = Self->ColumnTables.GetVerified(pathId); Y_UNUSED(pathEl); auto* pathDescription = Result->Record.MutablePathDescription();