Skip to content

Commit

Permalink
Add column shard Controller parameters to AppData (ydb-platform#8000)
Browse files Browse the repository at this point in the history
  • Loading branch information
swalrus1 authored and zverevgeny committed Sep 15, 2024
1 parent 0ed6fd9 commit e70c070
Show file tree
Hide file tree
Showing 31 changed files with 223 additions and 165 deletions.
16 changes: 8 additions & 8 deletions ydb/core/kqp/ut/olap/blobs_sharing_ut.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -93,8 +93,8 @@ Y_UNIT_TEST_SUITE(KqpOlapBlobsSharing) {
, Controller(NYDBTest::TControllers::RegisterCSControllerGuard<NYDBTest::NColumnShard::TController>()) {
Controller->SetCompactionControl(NYDBTest::EOptimizerCompactionWeightControl::Disable);
Controller->SetExpectedShardsCount(ShardsCount);
Controller->SetPeriodicWakeupActivationPeriod(TDuration::Seconds(1));
Controller->SetReadTimeoutClean(TDuration::Seconds(1));
Controller->SetOverridePeriodicWakeupActivationPeriod(TDuration::Seconds(1));
Controller->SetOverrideReadTimeoutClean(TDuration::Seconds(1));

Tests::NCommon::TLoggerInit(Kikimr).SetComponents({ NKikimrServices::TX_COLUMNSHARD }, "CS").Initialize();

Expand All @@ -111,7 +111,7 @@ Y_UNIT_TEST_SUITE(KqpOlapBlobsSharing) {
}

void WaitNormalization() {
Controller->SetReadTimeoutClean(TDuration::Seconds(1));
Controller->SetOverrideReadTimeoutClean(TDuration::Seconds(1));
Controller->SetCompactionControl(NYDBTest::EOptimizerCompactionWeightControl::Force);
const auto start = TInstant::Now();
while (!Controller->IsTrivialLinks() && TInstant::Now() - start < TDuration::Seconds(30)) {
Expand All @@ -120,11 +120,11 @@ Y_UNIT_TEST_SUITE(KqpOlapBlobsSharing) {
}
AFL_VERIFY(Controller->IsTrivialLinks());
Controller->CheckInvariants();
Controller->SetReadTimeoutClean(TDuration::Minutes(5));
Controller->SetOverrideReadTimeoutClean(TDuration::Minutes(5));
}

void Execute(const ui64 destinationIdx, const std::vector<ui64>& sourceIdxs, const bool move, const NOlap::TSnapshot& snapshot, const std::set<ui64>& pathIdxs) {
Controller->SetReadTimeoutClean(TDuration::Seconds(1));
Controller->SetOverrideReadTimeoutClean(TDuration::Seconds(1));
AFL_VERIFY(destinationIdx < ShardIds.size());
const ui64 destination = ShardIds[destinationIdx];
std::vector<ui64> sources;
Expand Down Expand Up @@ -192,7 +192,7 @@ Y_UNIT_TEST_SUITE(KqpOlapBlobsSharing) {
CSTransferStatus->Reset();
AFL_VERIFY(!Controller->IsTrivialLinks());
Controller->CheckInvariants();
Controller->SetReadTimeoutClean(TDuration::Minutes(5));
Controller->SetOverrideReadTimeoutClean(TDuration::Minutes(5));
}
};
Y_UNIT_TEST(BlobsSharingSplit1_1) {
Expand Down Expand Up @@ -318,8 +318,8 @@ Y_UNIT_TEST_SUITE(KqpOlapBlobsSharing) {

void Execute() {
auto csController = NYDBTest::TControllers::RegisterCSControllerGuard<NYDBTest::NColumnShard::TController>();
csController->SetPeriodicWakeupActivationPeriod(TDuration::Seconds(1));
csController->SetLagForCompactionBeforeTierings(TDuration::Seconds(1));
csController->SetOverridePeriodicWakeupActivationPeriod(TDuration::Seconds(1));
csController->SetOverrideLagForCompactionBeforeTierings(TDuration::Seconds(1));
csController->SetOverrideReduceMemoryIntervalLimit(1LLU << 30);

TLocalHelper(Kikimr).SetShardingMethod(ShardingType).CreateTestOlapTable("olapTable", "olapStore", 24, 4);
Expand Down
6 changes: 3 additions & 3 deletions ydb/core/kqp/ut/olap/indexes_ut.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,8 @@ Y_UNIT_TEST_SUITE(KqpOlapIndexes) {
TKikimrRunner kikimr(settings);

auto csController = NYDBTest::TControllers::RegisterCSControllerGuard<NYDBTest::NColumnShard::TController>();
csController->SetPeriodicWakeupActivationPeriod(TDuration::Seconds(1));
csController->SetLagForCompactionBeforeTierings(TDuration::Seconds(1));
csController->SetOverridePeriodicWakeupActivationPeriod(TDuration::Seconds(1));
csController->SetOverrideLagForCompactionBeforeTierings(TDuration::Seconds(1));
csController->SetOverrideReduceMemoryIntervalLimit(1LLU << 30);

TLocalHelper(kikimr).CreateTestOlapTable();
Expand Down Expand Up @@ -111,7 +111,7 @@ Y_UNIT_TEST_SUITE(KqpOlapIndexes) {
TKikimrRunner kikimr(settings);

auto csController = NYDBTest::TControllers::RegisterCSControllerGuard<NYDBTest::NColumnShard::TController>();
csController->SetPeriodicWakeupActivationPeriod(TDuration::Seconds(1));
csController->SetOverridePeriodicWakeupActivationPeriod(TDuration::Seconds(1));

TLocalHelper(kikimr).CreateTestOlapTable();
auto tableClient = kikimr.GetTableClient();
Expand Down
4 changes: 2 additions & 2 deletions ydb/core/kqp/ut/olap/kqp_olap_stats_ut.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -21,10 +21,10 @@ Y_UNIT_TEST_SUITE(KqpOlapStats) {

class TOlapStatsController : public NYDBTest::NColumnShard::TController {
public:
TDuration GetPeriodicWakeupActivationPeriod(const TDuration /*defaultValue*/) const override {
TDuration DoGetPeriodicWakeupActivationPeriod(const TDuration /*defaultValue*/) const override {
return TDuration::MilliSeconds(10);
}
TDuration GetStatsReportInterval(const TDuration /*defaultValue*/) const override {
TDuration DoGetStatsReportInterval(const TDuration /*defaultValue*/) const override {
return TDuration::MilliSeconds(10);
}
};
Expand Down
4 changes: 2 additions & 2 deletions ydb/core/kqp/ut/olap/kqp_olap_ut.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -2631,8 +2631,8 @@ Y_UNIT_TEST_SUITE(KqpOlap) {
TLocalHelper testHelper(kikimr);

auto csController = NYDBTest::TControllers::RegisterCSControllerGuard<NYDBTest::NColumnShard::TController>();
csController->SetPeriodicWakeupActivationPeriod(TDuration::Seconds(1));
csController->SetLagForCompactionBeforeTierings(TDuration::Seconds(1));
csController->SetOverridePeriodicWakeupActivationPeriod(TDuration::Seconds(1));
csController->SetOverrideLagForCompactionBeforeTierings(TDuration::Seconds(1));
csController->SetOverrideReduceMemoryIntervalLimit(1LLU << 30);
csController->DisableBackground(NKikimr::NYDBTest::ICSController::EBackground::Indexation);

Expand Down
2 changes: 1 addition & 1 deletion ydb/core/kqp/ut/olap/sparsed_ut.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -95,7 +95,7 @@ Y_UNIT_TEST_SUITE(KqpOlapSparsed) {
void Execute() {
CSController->DisableBackground(NKikimr::NYDBTest::ICSController::EBackground::Indexation);
CSController->DisableBackground(NKikimr::NYDBTest::ICSController::EBackground::Compaction);
CSController->SetPeriodicWakeupActivationPeriod(TDuration::MilliSeconds(100));
CSController->SetOverridePeriodicWakeupActivationPeriod(TDuration::MilliSeconds(100));

Tests::NCommon::TLoggerInit(Kikimr).Initialize();
TTypedLocalHelper helper("Utf8", Kikimr);
Expand Down
8 changes: 4 additions & 4 deletions ydb/core/kqp/ut/olap/write_ut.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@ Y_UNIT_TEST_SUITE(KqpOlapWrite) {
Y_UNIT_TEST(TierDraftsGC) {
auto csController = NKikimr::NYDBTest::TControllers::RegisterCSControllerGuard<NKikimr::NYDBTest::NColumnShard::TController>();
csController->SetIndexWriteControllerEnabled(false);
csController->SetPeriodicWakeupActivationPeriod(TDuration::Seconds(1));
csController->SetOverridePeriodicWakeupActivationPeriod(TDuration::Seconds(1));
Singleton<NKikimr::NWrappers::NExternalStorage::TFakeExternalStorage>()->ResetWriteCounters();

auto settings = TKikimrSettings()
Expand Down Expand Up @@ -50,7 +50,7 @@ Y_UNIT_TEST_SUITE(KqpOlapWrite) {
Y_UNIT_TEST(TierDraftsGCWithRestart) {
auto csController = NKikimr::NYDBTest::TControllers::RegisterCSControllerGuard<NKikimr::NYDBTest::NColumnShard::TController>();
csController->SetIndexWriteControllerEnabled(false);
csController->SetPeriodicWakeupActivationPeriod(TDuration::Seconds(1000));
csController->SetOverridePeriodicWakeupActivationPeriod(TDuration::Seconds(1000));
csController->DisableBackground(NKikimr::NYDBTest::ICSController::EBackground::GC);
Singleton<NKikimr::NWrappers::NExternalStorage::TFakeExternalStorage>()->ResetWriteCounters();

Expand Down Expand Up @@ -133,7 +133,7 @@ Y_UNIT_TEST_SUITE(KqpOlapWrite) {

Y_UNIT_TEST(WriteDeleteCleanGC) {
auto csController = NKikimr::NYDBTest::TControllers::RegisterCSControllerGuard<NKikimr::NYDBTest::NColumnShard::TController>();
csController->SetPeriodicWakeupActivationPeriod(TDuration::MilliSeconds(100));
csController->SetOverridePeriodicWakeupActivationPeriod(TDuration::MilliSeconds(100));
csController->DisableBackground(NKikimr::NYDBTest::ICSController::EBackground::GC);
Singleton<NKikimr::NWrappers::NExternalStorage::TFakeExternalStorage>()->ResetWriteCounters();

Expand Down Expand Up @@ -176,7 +176,7 @@ Y_UNIT_TEST_SUITE(KqpOlapWrite) {
)", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync();
UNIT_ASSERT_C(it.IsSuccess(), it.GetIssues().ToString());
}
csController->SetReadTimeoutClean(TDuration::Zero());
csController->SetOverrideReadTimeoutClean(TDuration::Zero());
csController->EnableBackground(NKikimr::NYDBTest::ICSController::EBackground::GC);
{
const TInstant start = TInstant::Now();
Expand Down
4 changes: 2 additions & 2 deletions ydb/core/kqp/ut/service/kqp_qs_queries_ut.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -3042,8 +3042,8 @@ Y_UNIT_TEST_SUITE(KqpQueryService) {
auto session = Kikimr->GetTableClient().CreateSession().GetValueSync().GetSession();

auto csController = NYDBTest::TControllers::RegisterCSControllerGuard<NYDBTest::NColumnShard::TController>();
csController->SetPeriodicWakeupActivationPeriod(TDuration::Seconds(1));
csController->SetLagForCompactionBeforeTierings(TDuration::Seconds(1));
csController->SetOverridePeriodicWakeupActivationPeriod(TDuration::Seconds(1));
csController->SetOverrideLagForCompactionBeforeTierings(TDuration::Seconds(1));
csController->DisableBackground(NKikimr::NYDBTest::ICSController::EBackground::Indexation);

const TString query = Sprintf(R"(
Expand Down
6 changes: 6 additions & 0 deletions ydb/core/protos/config.proto
Original file line number Diff line number Diff line change
Expand Up @@ -1535,6 +1535,12 @@ message TColumnShardConfig {

optional uint32 MaxInFlightIntervalsOnRequest = 16;
optional uint32 MaxReadStaleness_ms = 18 [default = 300000];
optional uint32 GCIntervalMs = 19 [default = 30000];
optional uint32 CompactionActualizationLagMs = 20 [default = 1000];
optional uint32 ActualizationTasksLagMs = 21 [default = 1000];
optional uint32 LagForCompactionBeforeTieringsMs = 22 [default = 3600000];
optional uint32 OptimizerFreshnessCheckDurationMs = 23 [default = 300000];
optional uint32 SmallPortionDetectSizeLimit = 24 [default = 1048576]; // 1 << 20
}

message TSchemeShardConfig {
Expand Down
2 changes: 1 addition & 1 deletion ydb/core/tx/columnshard/blobs_action/bs/blob_manager.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -309,7 +309,7 @@ std::shared_ptr<NBlobOperations::NBlobStorage::TGCTask> TBlobManager::BuildGCTas
return nullptr;
}

if (AppData()->TimeProvider->Now() - PreviousGCTime < NYDBTest::TControllers::GetColumnShardController()->GetOverridenGCPeriod(TDuration::Seconds(GC_INTERVAL_SECONDS))) {
if (AppData()->TimeProvider->Now() - PreviousGCTime < NYDBTest::TControllers::GetColumnShardController()->GetOverridenGCPeriod()) {
ACFL_DEBUG("event", "TBlobManager::BuildGCTask skip")("current_gen", CurrentGen)("current_step", CurrentStep)("reason", "too_often");
BlobsManagerCounters.GCCounters.SkipCollectionThrottling->Add(1);
return nullptr;
Expand Down
3 changes: 0 additions & 3 deletions ydb/core/tx/columnshard/blobs_action/bs/blob_manager.h
Original file line number Diff line number Diff line change
Expand Up @@ -133,9 +133,6 @@ struct TBlobManagerCounters {

// The implementation of BlobManager that hides all GC-related details
class TBlobManager : public IBlobManager, public TCommonBlobsTracker {
private:
static constexpr ui64 GC_INTERVAL_SECONDS = 30;

private:
using TBlobAddress = NBlobOperations::NBlobStorage::TBlobAddress;
class TGCContext;
Expand Down
2 changes: 1 addition & 1 deletion ydb/core/tx/columnshard/columnshard.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -178,7 +178,7 @@ void TColumnShard::Handle(TEvPrivate::TEvReadFinished::TPtr& ev, const TActorCon

void TColumnShard::Handle(TEvPrivate::TEvPingSnapshotsUsage::TPtr& /*ev*/, const TActorContext& ctx) {
if (auto writeTx = InFlightReadsTracker.Ping(
this, NYDBTest::TControllers::GetColumnShardController()->GetPingCheckPeriod(0.6 * GetMaxReadStaleness()), TInstant::Now())) {
this, NYDBTest::TControllers::GetColumnShardController()->GetPingCheckPeriod(), TInstant::Now())) {
Execute(writeTx.release(), ctx);
}
ctx.Schedule(0.3 * GetMaxReadStaleness(), new TEvPrivate::TEvPingSnapshotsUsage());
Expand Down
12 changes: 5 additions & 7 deletions ydb/core/tx/columnshard/columnshard_impl.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -71,9 +71,8 @@ TColumnShard::TColumnShard(TTabletStorageInfo* info, const TActorId& tablet)
, ProgressTxController(std::make_unique<TTxController>(*this))
, StoragesManager(std::make_shared<NOlap::TStoragesManager>(*this))
, DataLocksManager(std::make_shared<NOlap::NDataLocks::TManager>())
, PeriodicWakeupActivationPeriod(NYDBTest::TControllers::GetColumnShardController()->GetPeriodicWakeupActivationPeriod(
TSettings::DefaultPeriodicWakeupActivationPeriod))
, StatsReportInterval(NYDBTest::TControllers::GetColumnShardController()->GetStatsReportInterval(TSettings::DefaultStatsReportInterval))
, PeriodicWakeupActivationPeriod(NYDBTest::TControllers::GetColumnShardController()->GetPeriodicWakeupActivationPeriod())
, StatsReportInterval(NYDBTest::TControllers::GetColumnShardController()->GetStatsReportInterval())
, InFlightReadsTracker(StoragesManager, Counters.GetRequestsTracingCounters())
, TablesManager(StoragesManager, info->TabletID)
, Subscribers(std::make_shared<NSubscriber::TManager>(*this))
Expand Down Expand Up @@ -680,8 +679,8 @@ void TColumnShard::SetupIndexation() {
if (InsertTable->GetPathPriorities().size() && InsertTable->GetPathPriorities().rbegin()->first.GetCategory() == NOlap::TPathInfoIndexPriority::EIndexationPriority::PreventOverload) {
force = true;
}
const ui64 bytesLimit = NYDBTest::TControllers::GetColumnShardController()->GetGuaranteeIndexationStartBytesLimit(TSettings::GuaranteeIndexationStartBytesLimit);
const TDuration durationLimit = NYDBTest::TControllers::GetColumnShardController()->GetGuaranteeIndexationInterval(TSettings::GuaranteeIndexationInterval);
const ui64 bytesLimit = NYDBTest::TControllers::GetColumnShardController()->GetGuaranteeIndexationStartBytesLimit();
const TDuration durationLimit = NYDBTest::TControllers::GetColumnShardController()->GetGuaranteeIndexationInterval();
if (!force && InsertTable->GetCountersCommitted().Bytes < bytesLimit &&
TMonotonic::Now() < BackgroundController.GetLastIndexationInstant() + durationLimit) {
AFL_DEBUG(NKikimrServices::TX_COLUMNSHARD)("event", "skip_indexation")("reason", "not_enough_data_and_too_frequency")
Expand Down Expand Up @@ -1145,8 +1144,7 @@ const NKikimr::NColumnShard::NTiers::TManager* TColumnShard::GetTierManagerPoint
}

TDuration TColumnShard::GetMaxReadStaleness() {
return NYDBTest::TControllers::GetColumnShardController()->GetReadTimeoutClean(
TDuration::MilliSeconds(AppDataVerified().ColumnShardConfig.GetMaxReadStaleness_ms()));
return NYDBTest::TControllers::GetColumnShardController()->GetReadTimeoutClean();
}

}
2 changes: 1 addition & 1 deletion ydb/core/tx/columnshard/engines/column_engine_logs.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -421,7 +421,7 @@ std::vector<std::shared_ptr<TTTLColumnEngineChanges>> TColumnEngineForLogs::Star

TSaverContext saverContext(StoragesManager);
NActualizer::TTieringProcessContext context(memoryUsageLimit, saverContext, dataLocksManager, SignalCounters, ActualizationController);
const TDuration actualizationLag = NYDBTest::TControllers::GetColumnShardController()->GetActualizationTasksLag(TDuration::Seconds(1));
const TDuration actualizationLag = NYDBTest::TControllers::GetColumnShardController()->GetActualizationTasksLag();
for (auto&& i : pathEviction) {
auto g = GetGranuleOptional(i.first);
if (g) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,13 +41,9 @@ class TSpecialReadContext {
CacheFetchingScripts;

public:
static const inline ui64 DefaultRejectMemoryIntervalLimit = TGlobalLimits::DefaultRejectMemoryIntervalLimit;
static const inline ui64 DefaultReduceMemoryIntervalLimit = TGlobalLimits::DefaultReduceMemoryIntervalLimit;
static const inline ui64 DefaultReadSequentiallyBufferSize = TGlobalLimits::DefaultReadSequentiallyBufferSize;

const ui64 ReduceMemoryIntervalLimit = NYDBTest::TControllers::GetColumnShardController()->GetReduceMemoryIntervalLimit(DefaultReduceMemoryIntervalLimit);
const ui64 RejectMemoryIntervalLimit = NYDBTest::TControllers::GetColumnShardController()->GetRejectMemoryIntervalLimit(DefaultRejectMemoryIntervalLimit);
const ui64 ReadSequentiallyBufferSize = DefaultReadSequentiallyBufferSize;
const ui64 ReduceMemoryIntervalLimit = NYDBTest::TControllers::GetColumnShardController()->GetReduceMemoryIntervalLimit();
const ui64 RejectMemoryIntervalLimit = NYDBTest::TControllers::GetColumnShardController()->GetRejectMemoryIntervalLimit();
const ui64 ReadSequentiallyBufferSize = TGlobalLimits::DefaultReadSequentiallyBufferSize;

ui64 GetProcessMemoryControlId() const {
AFL_VERIFY(ProcessMemoryGuard);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@ std::shared_ptr<NKikimr::NOlap::TGranuleMeta> TGranulesStorage::GetGranuleForCom
std::map<NStorageOptimizer::TOptimizationPriority, std::shared_ptr<TGranuleMeta>> granulesSorted;
ui32 countChecker = 0;
std::optional<NStorageOptimizer::TOptimizationPriority> priorityChecker;
const TDuration actualizationLag = NYDBTest::TControllers::GetColumnShardController()->GetCompactionActualizationLag(TDuration::Seconds(1));
const TDuration actualizationLag = NYDBTest::TControllers::GetColumnShardController()->GetCompactionActualizationLag();
for (auto&& i : Tables) {
NActors::TLogContextGuard lGuard = NActors::TLogContextBuilder::Build()("path_id", i.first);
i.second->ActualizeOptimizer(now, actualizationLag);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3,8 +3,7 @@
namespace NKikimr::NOlap::NStorageOptimizer::NLBuckets {

TDuration GetCommonFreshnessCheckDuration() {
static const TDuration CommonFreshnessCheckDuration = TDuration::Seconds(300);
return NYDBTest::TControllers::GetColumnShardController()->GetOptimizerFreshnessCheckDuration(CommonFreshnessCheckDuration);
return NYDBTest::TControllers::GetColumnShardController()->GetOptimizerFreshnessCheckDuration();
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,6 @@

namespace NKikimr::NOlap::NStorageOptimizer::NLBuckets {

static const ui64 SmallPortionDetectSizeLimit = 1 << 20;

TDuration GetCommonFreshnessCheckDuration();

class TSimplePortionsGroupInfo {
Expand Down Expand Up @@ -683,7 +681,7 @@ class TPortionsBucket: public TMoveOnly {
return;
}
MainPortion->InitRuntimeFeature(TPortionInfo::ERuntimeFeature::Optimized, Others.IsEmpty() && currentInstant > MainPortion->RecordSnapshotMax().GetPlanInstant() +
NYDBTest::TControllers::GetColumnShardController()->GetLagForCompactionBeforeTierings(TDuration::Minutes(60)));
NYDBTest::TControllers::GetColumnShardController()->GetLagForCompactionBeforeTierings());
}
public:
TTaskDescription GetTaskDescription() const {
Expand Down Expand Up @@ -1104,7 +1102,7 @@ class TPortionBuckets {
}

void RemovePortion(const std::shared_ptr<TPortionInfo>& portion) {
if (portion->GetTotalBlobBytes() < NYDBTest::TControllers::GetColumnShardController()->GetSmallPortionSizeDetector(SmallPortionDetectSizeLimit)) {
if (portion->GetTotalBlobBytes() < NYDBTest::TControllers::GetColumnShardController()->GetSmallPortionSizeDetector()) {
Counters->SmallPortions->RemovePortion(portion);
}
if (!RemoveBucket(portion)) {
Expand Down Expand Up @@ -1146,7 +1144,7 @@ class TPortionBuckets {
}

void AddPortion(const std::shared_ptr<TPortionInfo>& portion, const TInstant now) {
if (portion->GetTotalBlobBytes() < NYDBTest::TControllers::GetColumnShardController()->GetSmallPortionSizeDetector(SmallPortionDetectSizeLimit)) {
if (portion->GetTotalBlobBytes() < NYDBTest::TControllers::GetColumnShardController()->GetSmallPortionSizeDetector()) {
Counters->SmallPortions->AddPortion(portion);
AddOther(portion, now);
return;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4,8 +4,7 @@
namespace NKikimr::NOlap::NStorageOptimizer::NSBuckets {

TDuration GetCommonFreshnessCheckDuration() {
static const TDuration CommonFreshnessCheckDuration = TDuration::Seconds(300);
return NYDBTest::TControllers::GetColumnShardController()->GetOptimizerFreshnessCheckDuration(CommonFreshnessCheckDuration);
return NYDBTest::TControllers::GetColumnShardController()->GetOptimizerFreshnessCheckDuration();
}

}
} // namespace NKikimr::NOlap::NStorageOptimizer::NSBuckets
Loading

0 comments on commit e70c070

Please sign in to comment.