From ae44a9ac34784cbd64828c6092c71fe6b5acbc8f Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 15 Oct 2025 21:29:38 +0300 Subject: [PATCH] Datashard configurable stats report intervals (#26862) (cherry picked from commit efa0852b55a4b51de45d7706163bbe79e56ef61b) Conflicts: ydb/core/sys_view/ut_common.h ydb/core/tx/schemeshard/ut_helpers/test_env.cpp ydb/core/tx/schemeshard/ut_helpers/test_env.h --- ydb/core/client/flat_ut.cpp | 12 +++-- ydb/core/protos/datashard_config.proto | 2 + ydb/core/sys_view/ut_common.cpp | 5 +++ ydb/core/sys_view/ut_common.h | 5 +++ ydb/core/sys_view/ut_kqp.cpp | 32 +++++-------- ydb/core/testlib/test_client.cpp | 1 + ydb/core/tx/datashard/datashard.cpp | 1 - ydb/core/tx/datashard/datashard.h | 1 - ydb/core/tx/datashard/datashard__stats.cpp | 22 ++++++++- ydb/core/tx/datashard/datashard_impl.h | 7 ++- ydb/core/tx/datashard/datashard_ut_stats.cpp | 45 ++++++++++++++++++- .../ut_common/datashard_ut_common.cpp | 4 +- .../datashard/ut_common/datashard_ut_common.h | 3 +- ydb/core/tx/schemeshard/ut_base/ut_base.cpp | 2 +- .../ut_compaction/ut_compaction.cpp | 18 +++++--- .../tx/schemeshard/ut_helpers/test_env.cpp | 5 +++ ydb/core/tx/schemeshard/ut_helpers/test_env.h | 1 + .../ut_index_build/ut_index_build.cpp | 5 +-- .../ut_split_merge/ut_split_merge.cpp | 11 +++-- ydb/core/tx/schemeshard/ut_stats/ut_stats.cpp | 18 ++++---- .../schemeshard/ut_subdomain/ut_subdomain.cpp | 4 +- ydb/services/ydb/ydb_table_split_ut.cpp | 6 +-- ydb/services/ydb/ydb_ut.cpp | 2 +- 23 files changed, 147 insertions(+), 65 deletions(-) diff --git a/ydb/core/client/flat_ut.cpp b/ydb/core/client/flat_ut.cpp index b4d9a807edd4..baf486269f9c 100644 --- a/ydb/core/client/flat_ut.cpp +++ b/ydb/core/client/flat_ut.cpp @@ -2631,7 +2631,8 @@ Y_UNIT_TEST_SUITE(TFlatTest) { UNIT_ASSERT_VALUES_EQUAL(partitions.size(), 1); // Force stats reporting without delays - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); + cleverServer.GetRuntime()->GetAppData() + .DataShardConfig.SetStatsReportIntervalSeconds(0); cleverServer.GetRuntime()->SetLogPriority(NKikimrServices::FLAT_TX_SCHEMESHARD, NActors::NLog::PRI_DEBUG); // cleverServer.GetRuntime()->SetLogPriority(NKikimrServices::TABLET_EXECUTOR, NActors::NLog::PRI_DEBUG); @@ -2748,7 +2749,8 @@ Y_UNIT_TEST_SUITE(TFlatTest) { UNIT_ASSERT_VALUES_EQUAL(partitions.size(), 1); // Force stats reporting without delays - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); + cleverServer.GetRuntime()->GetAppData() + .DataShardConfig.SetStatsReportIntervalSeconds(0); cleverServer.GetRuntime()->SetLogPriority(NKikimrServices::FLAT_TX_SCHEMESHARD, NActors::NLog::PRI_DEBUG); // cleverServer.GetRuntime()->SetLogPriority(NKikimrServices::TABLET_EXECUTOR, NActors::NLog::PRI_DEBUG); @@ -2921,7 +2923,8 @@ Y_UNIT_TEST_SUITE(TFlatTest) { UNIT_ASSERT_VALUES_EQUAL(partitions.size(), 4); // Force stats reporting without delays - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); + cleverServer.GetRuntime()->GetAppData() + .DataShardConfig.SetStatsReportIntervalSeconds(0); cleverServer.GetRuntime()->SetLogPriority(NKikimrServices::FLAT_TX_SCHEMESHARD, NActors::NLog::PRI_DEBUG); // cleverServer.GetRuntime()->SetLogPriority(NKikimrServices::TABLET_EXECUTOR, NActors::NLog::PRI_DEBUG); @@ -3027,7 +3030,8 @@ Y_UNIT_TEST_SUITE(TFlatTest) { TVector initialPartitions = annoyingClient.GetTablePartitions("/dc-1/Dir/T1"); // Force stats reporting without delays - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); + cleverServer.GetRuntime()->GetAppData() + .DataShardConfig.SetStatsReportIntervalSeconds(0); NDataShard::gDbStatsDataSizeResolution = 80000; TString bigValue(100*1024, '0'); diff --git a/ydb/core/protos/datashard_config.proto b/ydb/core/protos/datashard_config.proto index c69f1c6583ce..445a6cc7dede 100644 --- a/ydb/core/protos/datashard_config.proto +++ b/ydb/core/protos/datashard_config.proto @@ -26,4 +26,6 @@ message TDataShardConfig { optional uint64 IncrementalRestoreReadAheadLo = 20 [default = 524288]; optional uint64 IncrementalRestoreReadAheadHi = 21 [default = 1048576]; optional uint64 InMemoryStateMigrationTimeoutMs = 24 [default = 1000]; + optional uint32 StatsReportIntervalSeconds = 25 [default = 10]; + optional uint32 BackupTableStatsReportIntervalSeconds = 26 [default = 30]; } diff --git a/ydb/core/sys_view/ut_common.cpp b/ydb/core/sys_view/ut_common.cpp index e2f39ef33ee2..5ca693e0c372 100644 --- a/ydb/core/sys_view/ut_common.cpp +++ b/ydb/core/sys_view/ut_common.cpp @@ -63,6 +63,11 @@ TTestEnv::TTestEnv(ui32 staticNodes, ui32 dynamicNodes, const TTestEnvSettings& Settings->AppConfig->MutableHiveConfig()->AddBalancerIgnoreTabletTypes(NKikimrTabletBase::TTabletTypes::SysViewProcessor); + if (settings.DataShardStatsReportIntervalSeconds) { + Settings->AppConfig->MutableDataShardConfig() + ->SetStatsReportIntervalSeconds(*settings.DataShardStatsReportIntervalSeconds); + } + Server = new Tests::TServer(*Settings); Server->EnableGRpc(grpcPort); diff --git a/ydb/core/sys_view/ut_common.h b/ydb/core/sys_view/ut_common.h index b0930f30f8cb..ff930f0625a7 100644 --- a/ydb/core/sys_view/ut_common.h +++ b/ydb/core/sys_view/ut_common.h @@ -21,6 +21,7 @@ struct TTestEnvSettings { ui32 PqTabletsN = 0; bool EnableSVP = false; bool EnableForceFollowers = false; + TMaybe DataShardStatsReportIntervalSeconds; }; class TTestEnv { @@ -31,6 +32,10 @@ class TTestEnv { public: TTestEnv(ui32 staticNodes = 1, ui32 dynamicNodes = 4, const TTestEnvSettings& settings = {}); + TTestEnv(const TTestEnvSettings& settings) : TTestEnv(1, 4, settings) + { + } + ~TTestEnv(); Tests::TServer& GetServer() const { diff --git a/ydb/core/sys_view/ut_kqp.cpp b/ydb/core/sys_view/ut_kqp.cpp index b42080244aff..5b5cd04dcc1b 100644 --- a/ydb/core/sys_view/ut_kqp.cpp +++ b/ydb/core/sys_view/ut_kqp.cpp @@ -846,9 +846,7 @@ Y_UNIT_TEST_SUITE(SystemView) { } Y_UNIT_TEST(PartitionStatsLocksFields) { - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); - - TTestEnv env; + TTestEnv env({.DataShardStatsReportIntervalSeconds = 0}); CreateRootTable(env, /* partitionCount */ 1, /* fillTable */ true); TTableClient client(env.GetDriver()); @@ -876,11 +874,9 @@ Y_UNIT_TEST_SUITE(SystemView) { } Y_UNIT_TEST(PartitionStatsFields) { - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); - auto nowUs = TInstant::Now().MicroSeconds(); - TTestEnv env; + TTestEnv env({.DataShardStatsReportIntervalSeconds = 0}); CreateRootTable(env); TTableClient client(env.GetDriver()); @@ -1357,11 +1353,9 @@ Y_UNIT_TEST_SUITE(SystemView) { } Y_UNIT_TEST(TopPartitionsByCpuFields) { - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); - auto nowUs = TInstant::Now().MicroSeconds(); - TTestEnv env(1, 4, {.EnableSVP = true}); + TTestEnv env(1, 4, {.EnableSVP = true, .DataShardStatsReportIntervalSeconds = 0}); CreateTenantsAndTables(env); TTableClient client(env.GetDriver()); @@ -1409,11 +1403,9 @@ Y_UNIT_TEST_SUITE(SystemView) { } Y_UNIT_TEST(TopPartitionsByCpuTables) { - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); - constexpr ui64 partitionCount = 5; - TTestEnv env(1, 4, {.EnableSVP = true}); + TTestEnv env(1, 4, {.EnableSVP = true, .DataShardStatsReportIntervalSeconds = 0}); CreateTenantsAndTables(env, true, partitionCount); TTableClient client(env.GetDriver()); @@ -1439,11 +1431,9 @@ Y_UNIT_TEST_SUITE(SystemView) { } Y_UNIT_TEST(TopPartitionsByCpuRanges) { - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); - constexpr ui64 partitionCount = 5; - TTestEnv env(1, 4, {.EnableSVP = true}); + TTestEnv env(1, 4, {.EnableSVP = true, .DataShardStatsReportIntervalSeconds = 0}); CreateTenantsAndTables(env, true, partitionCount); TTableClient client(env.GetDriver()); @@ -1520,11 +1510,13 @@ Y_UNIT_TEST_SUITE(SystemView) { } Y_UNIT_TEST(TopPartitionsByCpuFollowers) { - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); - auto nowUs = TInstant::Now().MicroSeconds(); - TTestEnv env(1, 4, {.EnableSVP = true, .EnableForceFollowers = true}); + TTestEnv env(1, 4, { + .EnableSVP = true, + .EnableForceFollowers = true, + .DataShardStatsReportIntervalSeconds = 0, + }); auto& runtime = *env.GetServer().GetRuntime(); runtime.SetLogPriority(NKikimrServices::TX_DATASHARD, NLog::PRI_TRACE); @@ -1695,9 +1687,7 @@ Y_UNIT_TEST_SUITE(SystemView) { } Y_UNIT_TEST(TopPartitionsByTliFields) { - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); - - TTestEnv env(1, 4, {.EnableSVP = true}); + TTestEnv env(1, 4, {.EnableSVP = true, .DataShardStatsReportIntervalSeconds = 0}); CreateTenantsAndTables(env); TTableClient client(env.GetDriver()); diff --git a/ydb/core/testlib/test_client.cpp b/ydb/core/testlib/test_client.cpp index 566b302e3c43..1a89cb7b175a 100644 --- a/ydb/core/testlib/test_client.cpp +++ b/ydb/core/testlib/test_client.cpp @@ -540,6 +540,7 @@ namespace Tests { appData.DataStreamsAuthFactory = Settings->DataStreamsAuthFactory.get(); appData.PersQueueMirrorReaderFactory = Settings->PersQueueMirrorReaderFactory.get(); appData.HiveConfig.MergeFrom(Settings->AppConfig->GetHiveConfig()); + appData.DataShardConfig.MergeFrom(Settings->AppConfig->GetDataShardConfig()); appData.GraphConfig.MergeFrom(Settings->AppConfig->GetGraphConfig()); appData.SqsConfig.MergeFrom(Settings->AppConfig->GetSqsConfig()); appData.SharedCacheConfig.MergeFrom(Settings->AppConfig->GetSharedCacheConfig()); diff --git a/ydb/core/tx/datashard/datashard.cpp b/ydb/core/tx/datashard/datashard.cpp index d08b8521e2d3..82b9dccda5d1 100644 --- a/ydb/core/tx/datashard/datashard.cpp +++ b/ydb/core/tx/datashard/datashard.cpp @@ -35,7 +35,6 @@ using namespace NTabletFlatExecutor; // But in unit tests we want to test both scenarios bool gAllowLogBatchingDefaultValue = true; -TDuration gDbStatsReportInterval = TDuration::Seconds(10); ui64 gDbStatsDataSizeResolution = 10*1024*1024; ui64 gDbStatsRowCountResolution = 100000; ui32 gDbStatsHistogramBucketsCount = 10; diff --git a/ydb/core/tx/datashard/datashard.h b/ydb/core/tx/datashard/datashard.h index 976124a10eca..4edde526585d 100644 --- a/ydb/core/tx/datashard/datashard.h +++ b/ydb/core/tx/datashard/datashard.h @@ -148,7 +148,6 @@ namespace NDataShard { // NOTE: this switch should be modified only in tests !!! extern bool gAllowLogBatchingDefaultValue; - extern TDuration gDbStatsReportInterval; extern ui64 gDbStatsDataSizeResolution; extern ui64 gDbStatsRowCountResolution; extern ui32 gDbStatsHistogramBucketsCount; diff --git a/ydb/core/tx/datashard/datashard__stats.cpp b/ydb/core/tx/datashard/datashard__stats.cpp index 01e971cd9002..e0554c70c9f6 100644 --- a/ydb/core/tx/datashard/datashard__stats.cpp +++ b/ydb/core/tx/datashard/datashard__stats.cpp @@ -597,13 +597,31 @@ class TDataShard::TTxInitiateStatsUpdate : public NTabletFlatExecutor::TTransact } }; +TDuration TDataShard::GetStatsReportInterval(const TAppData& appData) const { + const auto& userTables = GetUserTables(); + const bool isBackup = !userTables.empty() && std::all_of(userTables.begin(), userTables.end(), + [](const auto& kv) { return kv.second->IsBackup; }); + + if (isBackup) { + // Clamp the interval for backup tables to the value for ordinary tables, as it + // makes no sense for the latter to be longer than the former. + auto interval = std::max( + appData.DataShardConfig.GetBackupTableStatsReportIntervalSeconds(), + appData.DataShardConfig.GetStatsReportIntervalSeconds()); + return TDuration::Seconds(interval); + } else { + return TDuration::Seconds(appData.DataShardConfig.GetStatsReportIntervalSeconds()); + } +} + void TDataShard::UpdateTableStats(const TActorContext &ctx) { if (StatisticsDisabled) return; - TInstant now = AppData(ctx)->TimeProvider->Now(); + auto* appData = AppData(ctx); + TInstant now = appData->TimeProvider->Now(); - if (LastDbStatsUpdateTime + gDbStatsReportInterval > now) + if (LastDbStatsUpdateTime + GetStatsReportInterval(*appData) > now) return; if (State != TShardState::Ready && State != TShardState::Readonly) diff --git a/ydb/core/tx/datashard/datashard_impl.h b/ydb/core/tx/datashard/datashard_impl.h index 967e629e39e0..52226aeaa4a7 100644 --- a/ydb/core/tx/datashard/datashard_impl.h +++ b/ydb/core/tx/datashard/datashard_impl.h @@ -3294,13 +3294,16 @@ class TDataShard NTabletPipe::SendData(ctx, StateReportPipe, ev.Release()); } + TDuration GetStatsReportInterval(const TAppData&) const; + void SendPeriodicTableStats(const TActorContext &ctx) { if (StatisticsDisabled) return; - TInstant now = AppData(ctx)->TimeProvider->Now(); + auto* appData = AppData(ctx); + TInstant now = appData->TimeProvider->Now(); - if (LastDbStatsReportTime + gDbStatsReportInterval > now) + if (LastDbStatsReportTime + GetStatsReportInterval(*appData) > now) return; auto* resourceMetrics = Executor()->GetResourceMetrics(); diff --git a/ydb/core/tx/datashard/datashard_ut_stats.cpp b/ydb/core/tx/datashard/datashard_ut_stats.cpp index aab103d03cf9..ced1842eaeda 100644 --- a/ydb/core/tx/datashard/datashard_ut_stats.cpp +++ b/ydb/core/tx/datashard/datashard_ut_stats.cpp @@ -389,6 +389,8 @@ Y_UNIT_TEST_SUITE(DataShardStats) { TServerSettings serverSettings(pm.GetPort(2134)); serverSettings.SetDomainName("Root") .SetUseRealThreads(false); + serverSettings.AppConfig->MutableDataShardConfig() + ->SetStatsReportIntervalSeconds(0); TServer::TPtr server = new TServer(serverSettings); auto& runtime = *server->GetRuntime(); @@ -400,8 +402,6 @@ Y_UNIT_TEST_SUITE(DataShardStats) { InitRoot(server, sender); - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); - NLocalDb::TCompactionPolicyPtr policy = NLocalDb::CreateDefaultUserTablePolicy(); policy->InMemForceStepsToSnapshot = 1; @@ -832,6 +832,47 @@ Y_UNIT_TEST_SUITE(DataShardStats) { WaitTableStats(runtime, shard1, DoesNotHaveSchemaChangesCondition()); } + Y_UNIT_TEST(BackupTableStatsReportInterval) { + TPortManager pm; + TServerSettings serverSettings(pm.GetPort(2134)); + serverSettings.SetDomainName("Root") + .SetUseRealThreads(false); + + TServer::TPtr server = new TServer(serverSettings); + auto& runtime = *server->GetRuntime(); + auto sender = runtime.AllocateEdgeActor(); + + runtime.SetLogPriority(NKikimrServices::TX_DATASHARD, NLog::PRI_TRACE); + + InitRoot(server, sender); + + auto [shards, tableId1] = CreateShardedTable(server, sender, "/Root", "table-1", 1); + + // Create a backup copy table + { + auto senderCopy = runtime.AllocateEdgeActor(); + ui64 txId = AsyncCreateCopyTable( + server, senderCopy, "/Root", "table-2", "/Root/table-1", /*isBackup=*/true); + WaitTxNotification(server, senderCopy, txId); + } + auto tableId2 = ResolveTableId(server, sender, "/Root/table-2"); + + std::unordered_map tableToStatsCount; + auto observerFunc = [&](auto& ev) { + const NKikimrTxDataShard::TEvPeriodicTableStats& record = ev->Get()->Record; + ++tableToStatsCount[record.GetTableLocalId()]; + }; + auto observer = runtime.AddObserver(observerFunc); + + runtime.WaitFor("First stats event", [&]{ return !tableToStatsCount.empty(); }); + runtime.SimulateSleep(TDuration::Seconds(45)); + // Once every 10 seconds + UNIT_ASSERT_GE(tableToStatsCount[tableId1.PathId.LocalPathId], 4); + UNIT_ASSERT_LE(tableToStatsCount[tableId1.PathId.LocalPathId], 5); + // Once every 30 seconds + UNIT_ASSERT_GE(tableToStatsCount[tableId2.PathId.LocalPathId], 1); + UNIT_ASSERT_LE(tableToStatsCount[tableId2.PathId.LocalPathId], 2); + } } } diff --git a/ydb/core/tx/datashard/ut_common/datashard_ut_common.cpp b/ydb/core/tx/datashard/ut_common/datashard_ut_common.cpp index af72e2e6d89a..685d28cb563d 100644 --- a/ydb/core/tx/datashard/ut_common/datashard_ut_common.cpp +++ b/ydb/core/tx/datashard/ut_common/datashard_ut_common.cpp @@ -1319,12 +1319,14 @@ ui64 AsyncCreateCopyTable( TActorId sender, const TString &root, const TString &name, - const TString &from) + const TString &from, + bool isBackup) { auto request = SchemeTxTemplate(NKikimrSchemeOp::ESchemeOpCreateTable, root); auto& desc = *request->Record.MutableTransaction()->MutableModifyScheme()->MutableCreateTable(); desc.SetName(name); desc.SetCopyFromTable(from); + desc.SetIsBackup(isBackup); return RunSchemeTx(*server->GetRuntime(), std::move(request), sender); } diff --git a/ydb/core/tx/datashard/ut_common/datashard_ut_common.h b/ydb/core/tx/datashard/ut_common/datashard_ut_common.h index b5503ad85adc..b61faddf41b1 100644 --- a/ydb/core/tx/datashard/ut_common/datashard_ut_common.h +++ b/ydb/core/tx/datashard/ut_common/datashard_ut_common.h @@ -530,7 +530,8 @@ ui64 AsyncCreateCopyTable(Tests::TServer::TPtr server, TActorId sender, const TString &root, const TString &name, - const TString &from); + const TString &from, + bool isBackup = false); NKikimrTxDataShard::TEvCompactTableResult CompactTable( TTestActorRuntime& runtime, ui64 shardId, const TTableId& tableId, bool compactBorrowed = false); diff --git a/ydb/core/tx/schemeshard/ut_base/ut_base.cpp b/ydb/core/tx/schemeshard/ut_base/ut_base.cpp index 20aca91e5c00..15bebf4a9079 100644 --- a/ydb/core/tx/schemeshard/ut_base/ut_base.cpp +++ b/ydb/core/tx/schemeshard/ut_base/ut_base.cpp @@ -10770,12 +10770,12 @@ Y_UNIT_TEST_SUITE(TSchemeShardTest) { TTestEnvOptions opts; opts.EnableBackgroundCompaction(false); + opts.DataShardStatsReportIntervalSeconds(1); TTestEnv env(runtime, opts); ui64 txId = 100; - NDataShard::gDbStatsReportInterval = TDuration::Seconds(1); NDataShard::gDbStatsDataSizeResolution = 10; NDataShard::gDbStatsRowCountResolution = 10; diff --git a/ydb/core/tx/schemeshard/ut_compaction/ut_compaction.cpp b/ydb/core/tx/schemeshard/ut_compaction/ut_compaction.cpp index f5884ec8433b..0f83aa5e7e88 100644 --- a/ydb/core/tx/schemeshard/ut_compaction/ut_compaction.cpp +++ b/ydb/core/tx/schemeshard/ut_compaction/ut_compaction.cpp @@ -768,12 +768,14 @@ Y_UNIT_TEST_SUITE(TSchemeshardBorrowedCompactionTest) { // 1. Copy table is not compacted until we want to split it // 2. After borrow compaction both src and dst tables are background compacted - NDataShard::gDbStatsReportInterval = TDuration::Seconds(1); NDataShard::gDbStatsDataSizeResolution = 10; NDataShard::gDbStatsRowCountResolution = 10; + TTestEnvOptions opts; + opts.DataShardStatsReportIntervalSeconds(1); + TTestBasicRuntime runtime; - TTestEnv env(runtime); + TTestEnv env(runtime, opts); runtime.SetLogPriority(NKikimrServices::TX_DATASHARD, NLog::PRI_DEBUG); runtime.SetLogPriority(NKikimrServices::FLAT_TX_SCHEMESHARD, NActors::NLog::PRI_TRACE); @@ -899,12 +901,14 @@ Y_UNIT_TEST_SUITE(TSchemeshardBorrowedCompactionTest) { // I.e. we compact borrowed data ASAP except copy table case, when // we don't want to compact at all. - NDataShard::gDbStatsReportInterval = TDuration::Seconds(1); NDataShard::gDbStatsDataSizeResolution = 10; NDataShard::gDbStatsRowCountResolution = 10; + TTestEnvOptions opts; + opts.DataShardStatsReportIntervalSeconds(1); + TTestBasicRuntime runtime; - TTestEnv env(runtime); + TTestEnv env(runtime, opts); runtime.SetLogPriority(NKikimrServices::TX_DATASHARD, NLog::PRI_DEBUG); runtime.SetLogPriority(NKikimrServices::FLAT_TX_SCHEMESHARD, NActors::NLog::PRI_TRACE); @@ -973,12 +977,14 @@ Y_UNIT_TEST_SUITE(TSchemeshardBorrowedCompactionTest) { Y_UNIT_TEST(SchemeshardShouldNotCompactBorrowedAfterSplitMergeWhenDisabled) { - NDataShard::gDbStatsReportInterval = TDuration::Seconds(1); NDataShard::gDbStatsDataSizeResolution = 10; NDataShard::gDbStatsRowCountResolution = 10; + TTestEnvOptions opts; + opts.DataShardStatsReportIntervalSeconds(1); + TTestBasicRuntime runtime; - TTestEnv env(runtime); + TTestEnv env(runtime, opts); runtime.SetLogPriority(NKikimrServices::TX_DATASHARD, NLog::PRI_DEBUG); runtime.SetLogPriority(NKikimrServices::FLAT_TX_SCHEMESHARD, NActors::NLog::PRI_TRACE); diff --git a/ydb/core/tx/schemeshard/ut_helpers/test_env.cpp b/ydb/core/tx/schemeshard/ut_helpers/test_env.cpp index 304f65d5c7d4..332017ef5850 100644 --- a/ydb/core/tx/schemeshard/ut_helpers/test_env.cpp +++ b/ydb/core/tx/schemeshard/ut_helpers/test_env.cpp @@ -652,6 +652,11 @@ NSchemeShardUT_Private::TTestEnv::TTestEnv(TTestActorRuntime& runtime, const TTe app.AddSystemBackupSID(sid); } + if (opts.DataShardStatsReportIntervalSeconds_) { + app.DataShardConfig.SetStatsReportIntervalSeconds( + *opts.DataShardStatsReportIntervalSeconds_); + } + AddDomain(runtime, app, TTestTxConfig::DomainUid, hive, schemeRoot); SetupLogging(runtime); diff --git a/ydb/core/tx/schemeshard/ut_helpers/test_env.h b/ydb/core/tx/schemeshard/ut_helpers/test_env.h index c77aed42885a..35dab2697f17 100644 --- a/ydb/core/tx/schemeshard/ut_helpers/test_env.h +++ b/ydb/core/tx/schemeshard/ut_helpers/test_env.h @@ -79,6 +79,7 @@ namespace NSchemeShardUT_Private { OPTION(std::optional, EnableChecksumsExport, std::nullopt); OPTION(std::optional, EnableLocalDBBtreeIndex, std::nullopt); OPTION(TVector>, DSProxies, {}); + OPTION(std::optional, DataShardStatsReportIntervalSeconds, std::nullopt); #undef OPTION }; diff --git a/ydb/core/tx/schemeshard/ut_index_build/ut_index_build.cpp b/ydb/core/tx/schemeshard/ut_index_build/ut_index_build.cpp index 5730c3bf5f0a..9ea88e9e2260 100644 --- a/ydb/core/tx/schemeshard/ut_index_build/ut_index_build.cpp +++ b/ydb/core/tx/schemeshard/ut_index_build/ut_index_build.cpp @@ -417,7 +417,7 @@ Y_UNIT_TEST_SUITE(IndexBuildTest) { NLs::PathVersionEqual(3)}); // Force stats reporting without delays - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); + runtime.GetAppData().DataShardConfig.SetStatsReportIntervalSeconds(0); NDataShard::gDbStatsDataSizeResolution = 80000; auto upgradeEvent = [&](TAutoPtr& ev) -> auto { @@ -741,10 +741,9 @@ Y_UNIT_TEST_SUITE(IndexBuildTest) { TTestEnvOptions opts; opts.EnableBackgroundCompaction(false); opts.DisableStatsBatching(true); + opts.DataShardStatsReportIntervalSeconds(0); TTestEnv env(runtime, opts); - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); - ui64 txId = 100; TestCreateTable(runtime, ++txId, "/MyRoot", R"( diff --git a/ydb/core/tx/schemeshard/ut_split_merge/ut_split_merge.cpp b/ydb/core/tx/schemeshard/ut_split_merge/ut_split_merge.cpp index 5cbef442592d..4a81c8f3f19d 100644 --- a/ydb/core/tx/schemeshard/ut_split_merge/ut_split_merge.cpp +++ b/ydb/core/tx/schemeshard/ut_split_merge/ut_split_merge.cpp @@ -139,12 +139,12 @@ Y_UNIT_TEST_SUITE(TSchemeShardSplitBySizeTest) { TTestEnvOptions opts; opts.EnableBackgroundCompaction(false); + opts.DataShardStatsReportIntervalSeconds(1); TTestEnv env(runtime, opts); ui64 txId = 100; - NDataShard::gDbStatsReportInterval = TDuration::Seconds(1); NDataShard::gDbStatsDataSizeResolution = 10; NDataShard::gDbStatsRowCountResolution = 10; @@ -208,12 +208,12 @@ Y_UNIT_TEST_SUITE(TSchemeShardSplitBySizeTest) { TTestEnvOptions opts; opts.EnableBackgroundCompaction(false); opts.EnableParameterizedDecimal(true); + opts.DataShardStatsReportIntervalSeconds(1); TTestEnv env(runtime, opts); ui64 txId = 100; - NDataShard::gDbStatsReportInterval = TDuration::Seconds(1); NDataShard::gDbStatsDataSizeResolution = 10; NDataShard::gDbStatsRowCountResolution = 10; @@ -265,12 +265,12 @@ Y_UNIT_TEST_SUITE(TSchemeShardSplitBySizeTest) { TTestEnvOptions opts; opts.EnableBackgroundCompaction(false); opts.EnableTablePgTypes(true); + opts.DataShardStatsReportIntervalSeconds(1); TTestEnv env(runtime, opts); ui64 txId = 100; - NDataShard::gDbStatsReportInterval = TDuration::Seconds(1); NDataShard::gDbStatsDataSizeResolution = 10; NDataShard::gDbStatsRowCountResolution = 10; @@ -315,14 +315,13 @@ Y_UNIT_TEST_SUITE(TSchemeShardSplitBySizeTest) { TTestEnvOptions opts; opts.EnableBackgroundCompaction(false); opts.DisableStatsBatching(true); + opts.DataShardStatsReportIntervalSeconds(0); TTestEnv env(runtime, opts); ui64 txId = 100; runtime.SetDispatchedEventsLimit(10'000'000); - NDataShard::gDbStatsReportInterval = TDuration::MilliSeconds(1); - //runtime.SetLogPriority(NKikimrServices::FLAT_TX_SCHEMESHARD, NActors::NLog::PRI_CRIT); runtime.SetLogPriority(NKikimrServices::TX_DATASHARD, NActors::NLog::PRI_CRIT); @@ -815,10 +814,10 @@ struct TLoadAndSplitSimulator { TTestEnv SetupEnv(TTestBasicRuntime &runtime) { TTestEnvOptions opts; opts.EnableBackgroundCompaction(false); + opts.DataShardStatsReportIntervalSeconds(0); TTestEnv env(runtime, opts); - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); NDataShard::gDbStatsDataSizeResolution = 10; NDataShard::gDbStatsRowCountResolution = 10; diff --git a/ydb/core/tx/schemeshard/ut_stats/ut_stats.cpp b/ydb/core/tx/schemeshard/ut_stats/ut_stats.cpp index d74a846feacd..038cd8532264 100644 --- a/ydb/core/tx/schemeshard/ut_stats/ut_stats.cpp +++ b/ydb/core/tx/schemeshard/ut_stats/ut_stats.cpp @@ -151,13 +151,13 @@ TTableId ResolveTableId(TTestActorRuntime& runtime, const TString& path) { Y_UNIT_TEST_SUITE(TSchemeshardStatsBatchingTest) { Y_UNIT_TEST(ShouldNotBatchWhenDisabled) { TTestBasicRuntime runtime; - TTestEnv env(runtime); + TTestEnvOptions opts; + opts.DataShardStatsReportIntervalSeconds(1); + TTestEnv env(runtime, opts); runtime.SetLogPriority(NKikimrServices::TX_DATASHARD, NLog::PRI_TRACE); runtime.SetLogPriority(NKikimrServices::FLAT_TX_SCHEMESHARD, NActors::NLog::PRI_TRACE); - NDataShard::gDbStatsReportInterval = TDuration::Seconds(1); - auto& appData = runtime.GetAppData(); appData.FeatureFlags.SetEnablePersistentPartitionStats(true); @@ -186,12 +186,13 @@ Y_UNIT_TEST_SUITE(TSchemeshardStatsBatchingTest) { Y_UNIT_TEST(ShouldPersistByBatchSize) { TTestBasicRuntime runtime; - TTestEnv env(runtime); + TTestEnvOptions opts; + opts.DataShardStatsReportIntervalSeconds(1); + TTestEnv env(runtime, opts); runtime.SetLogPriority(NKikimrServices::TX_DATASHARD, NLog::PRI_TRACE); runtime.SetLogPriority(NKikimrServices::FLAT_TX_SCHEMESHARD, NActors::NLog::PRI_TRACE); - NDataShard::gDbStatsReportInterval = TDuration::Seconds(1); const ui32 batchSize = 2; auto& appData = runtime.GetAppData(); @@ -247,12 +248,13 @@ Y_UNIT_TEST_SUITE(TSchemeshardStatsBatchingTest) { Y_UNIT_TEST(ShouldPersistByBatchTimeout) { TTestBasicRuntime runtime; - TTestEnv env(runtime); + TTestEnvOptions opts; + opts.DataShardStatsReportIntervalSeconds(1); + TTestEnv env(runtime, opts); runtime.SetLogPriority(NKikimrServices::TX_DATASHARD, NLog::PRI_TRACE); runtime.SetLogPriority(NKikimrServices::FLAT_TX_SCHEMESHARD, NActors::NLog::PRI_TRACE); - NDataShard::gDbStatsReportInterval = TDuration::Seconds(1); TDuration dsWakeupInterval = TDuration::Seconds(5); // hardcoded in DS TDuration batchTimeout = dsWakeupInterval; @@ -591,9 +593,9 @@ Y_UNIT_TEST_SUITE(TStoragePoolsStatsPersistence) { opts.DisableStatsBatching(true); opts.EnablePersistentPartitionStats(true); opts.EnableBackgroundCompaction(false); + opts.DataShardStatsReportIntervalSeconds(0); TTestEnv env(runtime, opts); - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); NDataShard::gDbStatsDataSizeResolution = 1; NDataShard::gDbStatsRowCountResolution = 1; diff --git a/ydb/core/tx/schemeshard/ut_subdomain/ut_subdomain.cpp b/ydb/core/tx/schemeshard/ut_subdomain/ut_subdomain.cpp index fd526f41da6e..39ebb337215f 100644 --- a/ydb/core/tx/schemeshard/ut_subdomain/ut_subdomain.cpp +++ b/ydb/core/tx/schemeshard/ut_subdomain/ut_subdomain.cpp @@ -3522,11 +3522,11 @@ Y_UNIT_TEST_SUITE(TStoragePoolsQuotasTest) { opts.DisableStatsBatching(true); opts.EnablePersistentPartitionStats(true); opts.EnableBackgroundCompaction(false); + opts.DataShardStatsReportIntervalSeconds(0); TTestEnv env(runtime, opts); runtime.GetAppData().FeatureFlags.SetEnableSeparateDiskSpaceQuotas(true); - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); NDataShard::gDbStatsDataSizeResolution = 1; NDataShard::gDbStatsRowCountResolution = 1; @@ -3695,9 +3695,9 @@ Y_UNIT_TEST_SUITE(TStoragePoolsQuotasTest) { opts.DisableStatsBatching(true); opts.EnablePersistentPartitionStats(true); opts.EnableBackgroundCompaction(false); + opts.DataShardStatsReportIntervalSeconds(0); TTestEnv env(runtime, opts); - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); NDataShard::gDbStatsDataSizeResolution = 1; NDataShard::gDbStatsRowCountResolution = 1; diff --git a/ydb/services/ydb/ydb_table_split_ut.cpp b/ydb/services/ydb/ydb_table_split_ut.cpp index dcac04f28707..0c50083ab2c2 100644 --- a/ydb/services/ydb/ydb_table_split_ut.cpp +++ b/ydb/services/ydb/ydb_table_split_ut.cpp @@ -144,7 +144,7 @@ Y_UNIT_TEST_SUITE(YdbTableSplit) { Cerr << "Table has " << shardsBefore << " shards" << Endl; UNIT_ASSERT_VALUES_EQUAL(shardsBefore, 1); - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); + server.Server_->GetRuntime()->GetAppData().DataShardConfig.SetStatsReportIntervalSeconds(0); server.Server_->GetRuntime()->SetLogPriority(NKikimrServices::FLAT_TX_SCHEMESHARD, NActors::NLog::PRI_INFO); server.Server_->GetRuntime()->SetLogPriority(NKikimrServices::TX_DATASHARD, NActors::NLog::PRI_INFO); @@ -281,7 +281,7 @@ Y_UNIT_TEST_SUITE(YdbTableSplit) { Cerr << "Table has " << shardsBefore << " shards" << Endl; UNIT_ASSERT_VALUES_EQUAL(shardsBefore, 1); - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); + server.Server_->GetRuntime()->GetAppData().DataShardConfig.SetStatsReportIntervalSeconds(0); server.Server_->GetRuntime()->SetLogPriority(NKikimrServices::FLAT_TX_SCHEMESHARD, NActors::NLog::PRI_INFO); server.Server_->GetRuntime()->SetLogPriority(NKikimrServices::TX_DATASHARD, NActors::NLog::PRI_INFO); @@ -427,7 +427,6 @@ Y_UNIT_TEST_SUITE(YdbTableSplit) { Y_UNIT_TEST(RenameTablesAndSplit) { // KIKIMR-14636 - NDataShard::gDbStatsReportInterval = TDuration::Seconds(2); NDataShard::gDbStatsDataSizeResolution = 10; NDataShard::gDbStatsRowCountResolution = 10; @@ -437,6 +436,7 @@ Y_UNIT_TEST_SUITE(YdbTableSplit) { NKikimrConfig::TAppConfig appConfig; appConfig.MutableFeatureFlags()->SetEnableResourcePools(true); + appConfig.MutableDataShardConfig()->SetStatsReportIntervalSeconds(2); TKikimrWithGrpcAndRootSchemaNoSystemViews server(appConfig); server.Server_->GetRuntime()->SetLogPriority(NKikimrServices::FLAT_TX_SCHEMESHARD, NActors::NLog::PRI_NOTICE); diff --git a/ydb/services/ydb/ydb_ut.cpp b/ydb/services/ydb/ydb_ut.cpp index b6bfd312b4b4..e704e25d3735 100644 --- a/ydb/services/ydb/ydb_ut.cpp +++ b/ydb/services/ydb/ydb_ut.cpp @@ -5903,6 +5903,7 @@ Y_UNIT_TEST(DisableWritesToDatabase) { NKikimrConfig::TAppConfig appConfig; // default table profile with a storage policy is needed to be able to create a table with families *appConfig.MutableTableProfilesConfig() = CreateDefaultTableProfilesConfig(storagePools[0].GetKind()); + appConfig.MutableDataShardConfig()->SetStatsReportIntervalSeconds(0); serverSettings.SetAppConfig(appConfig); TServer::TPtr server = new TServer(serverSettings); @@ -5911,7 +5912,6 @@ Y_UNIT_TEST(DisableWritesToDatabase) { InitRoot(server, sender); runtime.SetLogPriority(NKikimrServices::FLAT_TX_SCHEMESHARD, NLog::PRI_TRACE); - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); NDataShard::gDbStatsDataSizeResolution = 1; NDataShard::gDbStatsRowCountResolution = 1;