diff options
| author | Alexey Zatelepin <[email protected]> | 2025-10-15 21:29:38 +0300 |
|---|---|---|
| committer | GitHub <[email protected]> | 2025-10-15 21:29:38 +0300 |
| commit | efa0852b55a4b51de45d7706163bbe79e56ef61b (patch) | |
| tree | f523a728a4189a7b6eccc02d7a98388c81d8ffb3 | |
| parent | 78aa6fa75eba124b91b200bfe76c67e94ee968a4 (diff) | |
Datashard configurable stats report intervals (#26862)
23 files changed, 141 insertions, 64 deletions
diff --git a/ydb/core/client/flat_ut.cpp b/ydb/core/client/flat_ut.cpp index 419d98ef385..d1a6fae50cb 100644 --- a/ydb/core/client/flat_ut.cpp +++ b/ydb/core/client/flat_ut.cpp @@ -2627,7 +2627,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); @@ -2800,7 +2801,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); @@ -2906,7 +2908,8 @@ Y_UNIT_TEST_SUITE(TFlatTest) { TVector<ui64> 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 c69f1c6583c..445a6cc7ded 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 86ae27239f8..852ee1b148b 100644 --- a/ydb/core/sys_view/ut_common.cpp +++ b/ydb/core/sys_view/ut_common.cpp @@ -78,6 +78,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 595a5da1309..3071fdb0be8 100644 --- a/ydb/core/sys_view/ut_common.h +++ b/ydb/core/sys_view/ut_common.h @@ -29,6 +29,7 @@ struct TTestEnvSettings { bool EnableTableCacheModes = false; TMaybe<bool> EnableRealSystemViewPaths; NKikimrProto::TAuthConfig AuthConfig = {}; + TMaybe<ui32> DataShardStatsReportIntervalSeconds; }; class TTestEnv { diff --git a/ydb/core/sys_view/ut_kqp.cpp b/ydb/core/sys_view/ut_kqp.cpp index 4f3f2c6c425..ae0bbf721d6 100644 --- a/ydb/core/sys_view/ut_kqp.cpp +++ b/ydb/core/sys_view/ut_kqp.cpp @@ -2834,9 +2834,7 @@ R"(CREATE TABLE `test_show_create` ( } 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()); @@ -2864,11 +2862,9 @@ R"(CREATE TABLE `test_show_create` ( } Y_UNIT_TEST(PartitionStatsFields) { - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); - auto nowUs = TInstant::Now().MicroSeconds(); - TTestEnv env; + TTestEnv env({.DataShardStatsReportIntervalSeconds = 0}); CreateRootTable(env); const auto describeResult = env.GetClient().Describe(env.GetServer().GetRuntime(), "Root/Table0"); const auto tablePathId = describeResult.GetPathId(); @@ -3359,11 +3355,9 @@ R"(CREATE TABLE `test_show_create` ( } 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()); @@ -3411,11 +3405,9 @@ R"(CREATE TABLE `test_show_create` ( } 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()); @@ -3441,11 +3433,9 @@ R"(CREATE TABLE `test_show_create` ( } 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()); @@ -3522,11 +3512,13 @@ R"(CREATE TABLE `test_show_create` ( } 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); @@ -3697,9 +3689,7 @@ R"(CREATE TABLE `test_show_create` ( } 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 c70b785954f..592db4926f2 100644 --- a/ydb/core/testlib/test_client.cpp +++ b/ydb/core/testlib/test_client.cpp @@ -555,6 +555,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 5c266e8b370..c7ecc2f1c27 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 03c171b535b..b136dd45cda 100644 --- a/ydb/core/tx/datashard/datashard.h +++ b/ydb/core/tx/datashard/datashard.h @@ -153,7 +153,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 792ea0bae51..5046d4cc954 100644 --- a/ydb/core/tx/datashard/datashard__stats.cpp +++ b/ydb/core/tx/datashard/datashard__stats.cpp @@ -578,13 +578,31 @@ public: } }; +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 201389fe2e4..c90c319d3a2 100644 --- a/ydb/core/tx/datashard/datashard_impl.h +++ b/ydb/core/tx/datashard/datashard_impl.h @@ -3348,13 +3348,16 @@ protected: 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 76465e00c66..0e45dcf45fd 100644 --- a/ydb/core/tx/datashard/datashard_ut_stats.cpp +++ b/ydb/core/tx/datashard/datashard_ut_stats.cpp @@ -391,6 +391,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(); @@ -402,8 +404,6 @@ Y_UNIT_TEST_SUITE(DataShardStats) { InitRoot(server, sender); - NDataShard::gDbStatsReportInterval = TDuration::Seconds(0); - NLocalDb::TCompactionPolicyPtr policy = NLocalDb::CreateDefaultUserTablePolicy(); policy->InMemForceStepsToSnapshot = 1; @@ -834,6 +834,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<TLocalPathId, size_t> tableToStatsCount; + auto observerFunc = [&](auto& ev) { + const NKikimrTxDataShard::TEvPeriodicTableStats& record = ev->Get()->Record; + ++tableToStatsCount[record.GetTableLocalId()]; + }; + auto observer = runtime.AddObserver<TEvDataShard::TEvPeriodicTableStats>(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 6f7da92ff85..6738edc1f1f 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 bf72d051047..4b0e9d743ed 100644 --- a/ydb/core/tx/datashard/ut_common/datashard_ut_common.h +++ b/ydb/core/tx/datashard/ut_common/datashard_ut_common.h @@ -531,7 +531,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 7358e71a21a..206adbb163b 100644 --- a/ydb/core/tx/schemeshard/ut_base/ut_base.cpp +++ b/ydb/core/tx/schemeshard/ut_base/ut_base.cpp @@ -10900,12 +10900,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 9ad60205648..c4b2d89583a 100644 --- a/ydb/core/tx/schemeshard/ut_compaction/ut_compaction.cpp +++ b/ydb/core/tx/schemeshard/ut_compaction/ut_compaction.cpp @@ -771,12 +771,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); @@ -902,12 +904,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); @@ -976,12 +980,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 ab528d3565a..d82ef700c62 100644 --- a/ydb/core/tx/schemeshard/ut_helpers/test_env.cpp +++ b/ydb/core/tx/schemeshard/ut_helpers/test_env.cpp @@ -657,6 +657,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, opts.NStoragePools_); 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 8517793aff4..48cc5d083cf 100644 --- a/ydb/core/tx/schemeshard/ut_helpers/test_env.h +++ b/ydb/core/tx/schemeshard/ut_helpers/test_env.h @@ -81,6 +81,7 @@ namespace NSchemeShardUT_Private { OPTION(std::optional<bool>, EnableSystemNamesProtection, std::nullopt); OPTION(std::optional<bool>, EnableRealSystemViewPaths, std::nullopt); OPTION(ui32, NStoragePools, 2); + OPTION(std::optional<ui32>, 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 e065812160c..eb2eaf4cd9a 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 @@ -446,7 +446,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<IEventHandle>& ev) -> auto { @@ -840,10 +840,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 0bdb4e1d9e9..989c5ebb07e 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); @@ -1074,10 +1073,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 f5e63e27fb4..9008b1f2505 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 0e98e11cbbb..d69eaec1542 100644 --- a/ydb/core/tx/schemeshard/ut_subdomain/ut_subdomain.cpp +++ b/ydb/core/tx/schemeshard/ut_subdomain/ut_subdomain.cpp @@ -3526,11 +3526,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; @@ -3699,9 +3699,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 653808eb3a2..e2dbbeb5337 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); TKikimrWithGrpcAndRootSchema 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 a89b1379a18..52efcf60308 100644 --- a/ydb/services/ydb/ydb_ut.cpp +++ b/ydb/services/ydb/ydb_ut.cpp @@ -5917,6 +5917,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); @@ -5925,7 +5926,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; |
