summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorAlexey Zatelepin <[email protected]>2025-10-15 21:29:38 +0300
committerGitHub <[email protected]>2025-10-15 21:29:38 +0300
commitefa0852b55a4b51de45d7706163bbe79e56ef61b (patch)
treef523a728a4189a7b6eccc02d7a98388c81d8ffb3
parent78aa6fa75eba124b91b200bfe76c67e94ee968a4 (diff)
Datashard configurable stats report intervals (#26862)
-rw-r--r--ydb/core/client/flat_ut.cpp9
-rw-r--r--ydb/core/protos/datashard_config.proto2
-rw-r--r--ydb/core/sys_view/ut_common.cpp5
-rw-r--r--ydb/core/sys_view/ut_common.h1
-rw-r--r--ydb/core/sys_view/ut_kqp.cpp32
-rw-r--r--ydb/core/testlib/test_client.cpp1
-rw-r--r--ydb/core/tx/datashard/datashard.cpp1
-rw-r--r--ydb/core/tx/datashard/datashard.h1
-rw-r--r--ydb/core/tx/datashard/datashard__stats.cpp22
-rw-r--r--ydb/core/tx/datashard/datashard_impl.h7
-rw-r--r--ydb/core/tx/datashard/datashard_ut_stats.cpp45
-rw-r--r--ydb/core/tx/datashard/ut_common/datashard_ut_common.cpp4
-rw-r--r--ydb/core/tx/datashard/ut_common/datashard_ut_common.h3
-rw-r--r--ydb/core/tx/schemeshard/ut_base/ut_base.cpp2
-rw-r--r--ydb/core/tx/schemeshard/ut_compaction/ut_compaction.cpp18
-rw-r--r--ydb/core/tx/schemeshard/ut_helpers/test_env.cpp5
-rw-r--r--ydb/core/tx/schemeshard/ut_helpers/test_env.h1
-rw-r--r--ydb/core/tx/schemeshard/ut_index_build/ut_index_build.cpp5
-rw-r--r--ydb/core/tx/schemeshard/ut_split_merge/ut_split_merge.cpp11
-rw-r--r--ydb/core/tx/schemeshard/ut_stats/ut_stats.cpp18
-rw-r--r--ydb/core/tx/schemeshard/ut_subdomain/ut_subdomain.cpp4
-rw-r--r--ydb/services/ydb/ydb_table_split_ut.cpp6
-rw-r--r--ydb/services/ydb/ydb_ut.cpp2
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;