diff options
author | ivanmorozov <ivanmorozov@yandex-team.com> | 2023-07-26 19:33:36 +0300 |
---|---|---|
committer | ivanmorozov <ivanmorozov@yandex-team.com> | 2023-07-26 19:33:36 +0300 |
commit | c1a0a6dc9351bcf42a4388db5f643525eda156bc (patch) | |
tree | 4d00feb23208459b78246fe6e7c1dbffbe7920fd | |
parent | f112995ac962911827ed4ff8b1f0fdbac27c6072 (diff) | |
download | ydb-c1a0a6dc9351bcf42a4388db5f643525eda156bc.tar.gz |
KIKIMR-18853: startttl. fix cpu usage
12 files changed, 141 insertions, 74 deletions
diff --git a/ydb/core/tx/columnshard/columnshard_impl.cpp b/ydb/core/tx/columnshard/columnshard_impl.cpp index 7ae7309501f..e33fd497c91 100644 --- a/ydb/core/tx/columnshard/columnshard_impl.cpp +++ b/ydb/core/tx/columnshard/columnshard_impl.cpp @@ -794,11 +794,13 @@ std::unique_ptr<TEvPrivate::TEvEviction> TColumnShard::SetupTtl(const THashMap<u LOG_S_DEBUG("Do not start TTL while eviction is in progress at tablet " << TabletID()); return {}; } - + if (force) { + TablesManager.MutablePrimaryIndex().OnTieringModified(Tiers); + } THashMap<ui64, NOlap::TTiering> eviction = pathTtls; if (eviction.empty()) { if (Tiers) { - eviction = Tiers->GetTiering(); // TODO: pathIds + eviction = Tiers->GetTiering(); } TablesManager.AddTtls(eviction, AppData()->TimeProvider->Now(), force); } @@ -816,8 +818,7 @@ std::unique_ptr<TEvPrivate::TEvEviction> TColumnShard::SetupTtl(const THashMap<u } auto actualIndexInfo = TablesManager.GetPrimaryIndex()->GetVersionedIndex(); - std::shared_ptr<NOlap::TTTLColumnEngineChanges> indexChanges = - TablesManager.MutablePrimaryIndex().StartTtl(eviction, actualIndexInfo.GetLastSchema()->GetIndexInfo().ArrowSchema()); + std::shared_ptr<NOlap::TTTLColumnEngineChanges> indexChanges = TablesManager.MutablePrimaryIndex().StartTtl(eviction); if (!indexChanges) { LOG_S_INFO("Cannot prepare TTL at tablet " << TabletID()); @@ -1080,6 +1081,7 @@ void TColumnShard::ActivateTiering(const ui64 pathId, const TString& useTiering) if (!Tiers) { Tiers = std::make_shared<TTiersManager>(TabletID(), SelfId(), [this](const TActorContext& ctx){ + TablesManager.MutablePrimaryIndex().OnTieringModified(Tiers); CleanForgottenBlobs(ctx); Reexport(ctx); }); diff --git a/ydb/core/tx/columnshard/engines/changes/abstract.cpp b/ydb/core/tx/columnshard/engines/changes/abstract.cpp index 7f1dc1b35fb..b12b1cdec0c 100644 --- a/ydb/core/tx/columnshard/engines/changes/abstract.cpp +++ b/ydb/core/tx/columnshard/engines/changes/abstract.cpp @@ -91,7 +91,7 @@ void TColumnEngineChanges::Compile(TFinalizationContext& context) noexcept { } TColumnEngineChanges::~TColumnEngineChanges() { - Y_VERIFY(!NActors::TlsActivationContext || Stage == EStage::Created || Stage == EStage::Finished || Stage == EStage::Aborted); + Y_VERIFY_DEBUG(!NActors::TlsActivationContext || Stage == EStage::Created || Stage == EStage::Finished || Stage == EStage::Aborted); } void TColumnEngineChanges::Abort(NColumnShard::TColumnShard& self, TChangesFinishContext& context) { diff --git a/ydb/core/tx/columnshard/engines/column_engine.h b/ydb/core/tx/columnshard/engines/column_engine.h index 32e9e1b9f04..9fa4859ddbe 100644 --- a/ydb/core/tx/columnshard/engines/column_engine.h +++ b/ydb/core/tx/columnshard/engines/column_engine.h @@ -7,6 +7,10 @@ #include "changes/compaction_info.h" #include <ydb/core/tx/columnshard/common/reverse_accessor.h> +namespace NKikimr::NColumnShard { +class TTiersManager; +} + namespace NKikimr::NOlap { class TInsertColumnEngineChanges; class TCompactColumnEngineChanges; @@ -360,7 +364,7 @@ public: const TCompactionLimits& limits) = 0; virtual std::shared_ptr<TCleanupColumnEngineChanges> StartCleanup(const TSnapshot& snapshot, const TCompactionLimits& limits, THashSet<ui64>& pathsToDrop, ui32 maxRecords) = 0; - virtual std::shared_ptr<TTTLColumnEngineChanges> StartTtl(const THashMap<ui64, TTiering>& pathEviction, const std::shared_ptr<arrow::Schema>& schema, + virtual std::shared_ptr<TTTLColumnEngineChanges> StartTtl(const THashMap<ui64, TTiering>& pathEviction, ui64 maxBytesToEvict = TCompactionLimits::DEFAULT_EVICTION_BYTES) = 0; virtual bool ApplyChanges(IDbWrapper& db, std::shared_ptr<TColumnEngineChanges> changes, const TSnapshot& snapshot) noexcept = 0; virtual void UpdateDefaultSchema(const TSnapshot& snapshot, TIndexInfo&& info) = 0; @@ -369,6 +373,7 @@ public: virtual const TColumnEngineStats& GetTotalStats() = 0; virtual ui64 MemoryUsage() const { return 0; } virtual TSnapshot LastUpdate() const { return TSnapshot::Zero(); } + virtual void OnTieringModified(std::shared_ptr<NColumnShard::TTiersManager> manager) = 0; }; } diff --git a/ydb/core/tx/columnshard/engines/column_engine_logs.cpp b/ydb/core/tx/columnshard/engines/column_engine_logs.cpp index 20ba4fb8852..f9a4667f7f3 100644 --- a/ydb/core/tx/columnshard/engines/column_engine_logs.cpp +++ b/ydb/core/tx/columnshard/engines/column_engine_logs.cpp @@ -400,32 +400,38 @@ std::shared_ptr<TCleanupColumnEngineChanges> TColumnEngineForLogs::StartCleanup( return changes; } -std::shared_ptr<TTTLColumnEngineChanges> TColumnEngineForLogs::StartTtl(const THashMap<ui64, TTiering>& pathEviction, const std::shared_ptr<arrow::Schema>& schema, - ui64 maxEvictBytes) { +std::shared_ptr<TTTLColumnEngineChanges> TColumnEngineForLogs::StartTtl(const THashMap<ui64, TTiering>& pathEviction, ui64 maxEvictBytes) { if (pathEviction.empty()) { return {}; } - auto changes = TChangesConstructor::BuildTtlChanges(); - ui64 evicttionSize = 0; + ui64 evictionSize = 0; bool allowEviction = true; ui64 dropBlobs = 0; bool allowDrop = true; auto& indexInfo = GetIndexInfo(); + const TMonotonic nowMonotonic = TlsActivationContext ? AppData()->MonotonicTimeProvider->Now() : TMonotonic::Now(); for (const auto& [pathId, ttl] : pathEviction) { - if (!PathGranules.contains(pathId)) { + auto it = NextCheckInstantForTTL.find(pathId); + if (it != NextCheckInstantForTTL.end() && nowMonotonic < it->second) { + AFL_DEBUG(NKikimrServices::TX_COLUMNSHARD)("event", "skip_path_id_ttl")("path_id", pathId)("now", nowMonotonic)("expected", it->second); + continue; + } + auto itGranules = PathGranules.find(pathId); + if (itGranules == PathGranules.end()) { continue; // It's not an error: allow TTL over multiple shards with different pathIds presented } - auto expireTimestamp = ttl.EvictScalar(schema); - Y_VERIFY(expireTimestamp); + auto expireTimestampOpt = ttl.GetEvictBorder(); + Y_VERIFY(expireTimestampOpt); + auto expireTimestamp = *expireTimestampOpt; auto ttlColumnNames = ttl.GetTtlColumns(); Y_VERIFY(ttlColumnNames.size() == 1); // TODO: support different ttl columns ui32 ttlColumnId = indexInfo.GetColumnId(*ttlColumnNames.begin()); - - for (const auto& [ts, granule] : PathGranules[pathId]) { + std::optional<TDuration> dWaiting; + for (const auto& [ts, granule] : itGranules->second) { auto spg = Granules[granule]; Y_VERIFY(spg); @@ -434,33 +440,52 @@ std::shared_ptr<TTTLColumnEngineChanges> TColumnEngineForLogs::StartTtl(const TH continue; } - allowEviction = (evicttionSize <= maxEvictBytes); + allowEviction = (evictionSize <= maxEvictBytes); allowDrop = (dropBlobs <= TCompactionLimits::MAX_BLOBS_TO_DELETE); - bool tryEvictPortion = allowEviction && ttl.HasTiers() - && info.EvictReady(TCompactionLimits::EVICT_HOT_PORTION_BYTES); + const bool tryEvictPortion = allowEviction && ttl.HasTiers() && info.EvictReady(TCompactionLimits::EVICT_HOT_PORTION_BYTES); if (auto max = info.MaxValue(ttlColumnId)) { - bool keep = NArrow::ScalarLess(expireTimestamp, max); + bool keep = false; + { + auto mpiOpt = ttl.ScalarToInstant(max); + Y_VERIFY(mpiOpt); + const TInstant maxTtlPortionInstant = *mpiOpt; + const TDuration d = maxTtlPortionInstant - expireTimestamp; + keep = !!d; + AFL_DEBUG(NKikimrServices::TX_COLUMNSHARD)("event", "keep_detect")("max", maxTtlPortionInstant.Seconds())("expire", expireTimestamp.Seconds()); + if (d && (!dWaiting || *dWaiting > d)) { + dWaiting = d; + } + } + AFL_DEBUG(NKikimrServices::TX_COLUMNSHARD)("event", "scalar_less_result")("keep", keep)("tryEvictPortion", tryEvictPortion)("allowDrop", allowDrop); if (keep && tryEvictPortion) { TString tierName; - for (auto& tierRef : ttl.GetOrderedTiers()) { // TODO: lower/upper_bound + move into TEviction + for (auto& tierRef : ttl.GetOrderedTiers()) { auto& tierInfo = tierRef.Get(); if (!indexInfo.AllowTtlOverColumn(tierInfo.GetEvictColumnName())) { SignalCounters.OnPortionNoTtlColumn(info.BlobsBytes()); - continue; // Ignore tiers with bad ttl column + continue; } - if (NArrow::ScalarLess(tierInfo.EvictScalar(schema), max)) { + auto mpiOpt = tierInfo.ScalarToInstant(max); + Y_VERIFY(mpiOpt); + const TInstant maxTieringPortionInstant = *mpiOpt; + + const TDuration d = maxTieringPortionInstant - tierInfo.GetEvictBorder(); + AFL_DEBUG(NKikimrServices::TX_COLUMNSHARD)("event", "tiering")("max", maxTieringPortionInstant.Seconds())("evict", tierInfo.GetEvictBorder().Seconds()); + if (d) { + if (!dWaiting || *dWaiting > d) { + dWaiting = d; + } tierName = tierInfo.GetName(); } else { break; } } if (info.TierName != tierName) { - evicttionSize += info.BlobsSizes().first; - bool needExport = ttl.NeedExport(tierName); - changes->PortionsToEvict.emplace_back( - info, TPortionEvictionFeatures(tierName, pathId, needExport)); + evictionSize += info.BlobsSizes().first; + const bool needExport = ttl.NeedExport(tierName); + changes->PortionsToEvict.emplace_back(info, TPortionEvictionFeatures(tierName, pathId, needExport)); SignalCounters.OnPortionToEvict(info.BlobsBytes()); } } @@ -475,6 +500,11 @@ std::shared_ptr<TTTLColumnEngineChanges> TColumnEngineForLogs::StartTtl(const TH } } } + if (dWaiting) { + NextCheckInstantForTTL[pathId] = nowMonotonic + std::min<TDuration>(*dWaiting, TDuration::Minutes(5)); + } else { + NextCheckInstantForTTL.erase(pathId); + } } if (changes->PortionsToDrop.empty() && @@ -780,4 +810,9 @@ std::unique_ptr<TCompactionInfo> TColumnEngineForLogs::Compact(const TCompaction } } +void TColumnEngineForLogs::OnTieringModified(std::shared_ptr<NColumnShard::TTiersManager> /*manager*/) { + AFL_DEBUG(NKikimrServices::TX_COLUMNSHARD)("event", "OnTieringModified"); + NextCheckInstantForTTL.clear(); +} + } // namespace NKikimr::NOlap diff --git a/ydb/core/tx/columnshard/engines/column_engine_logs.h b/ydb/core/tx/columnshard/engines/column_engine_logs.h index 0f9ab568bdf..f33f1a398cd 100644 --- a/ydb/core/tx/columnshard/engines/column_engine_logs.h +++ b/ydb/core/tx/columnshard/engines/column_engine_logs.h @@ -35,6 +35,7 @@ class TColumnEngineForLogs : public IColumnEngine { private: const NColumnShard::TEngineLogsCounters SignalCounters; std::shared_ptr<TGranulesStorage> GranulesStorage; + THashMap<ui64, TMonotonic> NextCheckInstantForTTL; public: class TChangesConstructor : public TColumnEngineChanges { public: @@ -68,6 +69,8 @@ public: TColumnEngineForLogs(ui64 tabletId, const TCompactionLimits& limits = {}); + virtual void OnTieringModified(std::shared_ptr<NColumnShard::TTiersManager> manager) override; + const TIndexInfo& GetIndexInfo() const override { return VersionedIndex.GetLastSchema()->GetIndexInfo(); } @@ -113,7 +116,7 @@ public: std::shared_ptr<TCompactColumnEngineChanges> StartCompaction(std::unique_ptr<TCompactionInfo>&& compactionInfo, const TCompactionLimits& limits) override; std::shared_ptr<TCleanupColumnEngineChanges> StartCleanup(const TSnapshot& snapshot, const TCompactionLimits& limits, THashSet<ui64>& pathsToDrop, ui32 maxRecords) override; - std::shared_ptr<TTTLColumnEngineChanges> StartTtl(const THashMap<ui64, TTiering>& pathEviction, const std::shared_ptr<arrow::Schema>& schema, + std::shared_ptr<TTTLColumnEngineChanges> StartTtl(const THashMap<ui64, TTiering>& pathEviction, ui64 maxEvictBytes = TCompactionLimits::DEFAULT_EVICTION_BYTES) override; bool ApplyChanges(IDbWrapper& db, std::shared_ptr<TColumnEngineChanges> indexChanges, diff --git a/ydb/core/tx/columnshard/engines/scheme/tier_info.cpp b/ydb/core/tx/columnshard/engines/scheme/tier_info.cpp index 8114b5f11af..1672d7a97ac 100644 --- a/ydb/core/tx/columnshard/engines/scheme/tier_info.cpp +++ b/ydb/core/tx/columnshard/engines/scheme/tier_info.cpp @@ -2,4 +2,20 @@ namespace NKikimr::NOlap { +std::optional<TInstant> TTierInfo::ScalarToInstant(const std::shared_ptr<arrow::Scalar>& scalar) const { + const ui64 unitsInSeconds = TtlUnitsInSecond ? TtlUnitsInSecond : 1; + switch (scalar->type->id()) { + case arrow::Type::TIMESTAMP: + return TInstant::MicroSeconds(std::static_pointer_cast<arrow::TimestampScalar>(scalar)->value); + case arrow::Type::UINT16: // YQL Date + return TInstant::Days(std::static_pointer_cast<arrow::UInt16Scalar>(scalar)->value); + case arrow::Type::UINT32: // YQL Datetime or Uint32 + return TInstant::MicroSeconds(std::static_pointer_cast<arrow::UInt32Scalar>(scalar)->value / (1.0 * unitsInSeconds / 1000000)); + case arrow::Type::UINT64: + return TInstant::MicroSeconds(std::static_pointer_cast<arrow::UInt64Scalar>(scalar)->value / (1.0 * unitsInSeconds / 1000000)); + default: + return {}; + } +} + } diff --git a/ydb/core/tx/columnshard/engines/scheme/tier_info.h b/ydb/core/tx/columnshard/engines/scheme/tier_info.h index 2271e4c87c6..bf5e8980cb3 100644 --- a/ydb/core/tx/columnshard/engines/scheme/tier_info.h +++ b/ydb/core/tx/columnshard/engines/scheme/tier_info.h @@ -20,8 +20,6 @@ private: ui32 TtlUnitsInSecond; std::optional<NArrow::TCompression> Compression; - mutable std::shared_ptr<arrow::Scalar> Scalar; - public: TTierInfo(const TString& tierName, TInstant evictBorder, const TString& column, ui32 unitsInSecond = 0) : Name(tierName) @@ -70,34 +68,7 @@ public: return schema->GetFieldByName(EvictColumnName); } - std::shared_ptr<arrow::Scalar> EvictScalar(const std::shared_ptr<arrow::Schema>& schema) const { - if (Scalar) { - return Scalar; - } - auto evictColumn = GetEvictColumn(schema); - Y_VERIFY(evictColumn); - - ui32 multiplier = TtlUnitsInSecond ? TtlUnitsInSecond : 1; - switch (evictColumn->type()->id()) { - case arrow::Type::TIMESTAMP: - Scalar = std::make_shared<arrow::TimestampScalar>( - EvictBorder.MicroSeconds(), arrow::timestamp(arrow::TimeUnit::MICRO)); - break; - case arrow::Type::UINT16: // YQL Date - Scalar = std::make_shared<arrow::UInt16Scalar>(EvictBorder.Days()); - break; - case arrow::Type::UINT32: // YQL Datetime or Uint32 - Scalar = std::make_shared<arrow::UInt32Scalar>(EvictBorder.Seconds() * multiplier); - break; - case arrow::Type::UINT64: - Scalar = std::make_shared<arrow::UInt64Scalar>(EvictBorder.Seconds() * multiplier); - break; - default: - break; - } - - return Scalar; - } + std::optional<TInstant> ScalarToInstant(const std::shared_ptr<arrow::Scalar>& scalar) const; static std::shared_ptr<TTierInfo> MakeTtl(TInstant ttlBorder, const TString& ttlColumn, ui32 unitsInSecond = 0) { return std::make_shared<TTierInfo>("TTL", ttlBorder, ttlColumn, unitsInSecond); @@ -141,6 +112,10 @@ public: return *Info; } + std::shared_ptr<TTierInfo> GetPtr() const { + return Info; + } + private: std::shared_ptr<TTierInfo> Info; }; @@ -150,6 +125,27 @@ class TTiering { TTiersMap TierByName; TSet<TTierRef> OrderedTiers; public: + + std::shared_ptr<TTierInfo> GetMainTierInfo() const { + auto ttl = Ttl; + auto tier = OrderedTiers.size() ? OrderedTiers.begin()->GetPtr() : nullptr; + if (!ttl && !tier) { + return nullptr; + } else if (!tier) { + return ttl; + } else if (!ttl) { + return tier; + } else { + const TInstant ttlInstant = ttl->GetEvictBorder(); + const TInstant tierInstant = tier->GetEvictBorder(); + if (ttlInstant < tierInstant) { + return tier; + } else { + return ttl; + } + } + } + std::shared_ptr<TTierInfo> Ttl; const TTiersMap& GetTierByName() const { @@ -181,15 +177,22 @@ public: return {}; } - std::shared_ptr<arrow::Scalar> EvictScalar(const std::shared_ptr<arrow::Schema>& schema) const { - auto ttlTs = Ttl ? Ttl->EvictScalar(schema) : nullptr; - auto tierTs = OrderedTiers.empty() ? nullptr : OrderedTiers.begin()->Get().EvictScalar(schema); - if (!ttlTs) { - return tierTs; - } else if (!tierTs) { - return ttlTs; + std::optional<TInstant> ScalarToInstant(const std::shared_ptr<arrow::Scalar>& scalar) const { + auto mainTier = GetMainTierInfo(); + if (!mainTier) { + return {}; + } else { + return mainTier->ScalarToInstant(scalar); + } + } + + std::optional<TInstant> GetEvictBorder() const { + auto mainTier = GetMainTierInfo(); + if (!mainTier) { + return {}; + } else { + return mainTier->GetEvictBorder(); } - return NArrow::ScalarLess(ttlTs, tierTs) ? tierTs : ttlTs; // either TTL or tier border appear } std::optional<NArrow::TCompression> GetCompression(const TString& name) const { diff --git a/ydb/core/tx/columnshard/engines/storage/granule.cpp b/ydb/core/tx/columnshard/engines/storage/granule.cpp index 5d0a614ec58..61e9a959eb2 100644 --- a/ydb/core/tx/columnshard/engines/storage/granule.cpp +++ b/ydb/core/tx/columnshard/engines/storage/granule.cpp @@ -41,9 +41,11 @@ void TGranuleMeta::UpsertPortion(const TPortionInfo& info) { bool TGranuleMeta::ErasePortion(const ui64 portion) { auto it = Portions.find(portion); if (it == Portions.end()) { + AFL_DEBUG(NKikimrServices::TX_COLUMNSHARD)("event", "portion_erased_already")("portion_id", portion)("pathId", Record.PathId); return false; + } else { + AFL_DEBUG(NKikimrServices::TX_COLUMNSHARD)("event", "portion_erased")("portion_info", it->second)("pathId", Record.PathId); } - AFL_DEBUG(NKikimrServices::TX_COLUMNSHARD)("event", "portion_erased")("portion_info", it->second)("pathId", Record.PathId); OnBeforeChangePortion(&it->second, nullptr); Portions.erase(it); OnAfterChangePortion(); diff --git a/ydb/core/tx/columnshard/engines/storage/granule.h b/ydb/core/tx/columnshard/engines/storage/granule.h index 84b24ec82ef..0ba027708e2 100644 --- a/ydb/core/tx/columnshard/engines/storage/granule.h +++ b/ydb/core/tx/columnshard/engines/storage/granule.h @@ -276,7 +276,7 @@ public: private: TMonotonic ModificationLastTime = TMonotonic::Now(); - THashMap<ui64, TPortionInfo> Portions; // portion -> portionInfo + THashMap<ui64, TPortionInfo> Portions; mutable std::optional<TGranuleAdditiveSummary> AdditiveSummaryCache; mutable std::optional<TGranuleHardSummary> HardSummaryCache; diff --git a/ydb/core/tx/columnshard/engines/ut_logs_engine.cpp b/ydb/core/tx/columnshard/engines/ut_logs_engine.cpp index 39098c91a41..bbc20cc3946 100644 --- a/ydb/core/tx/columnshard/engines/ut_logs_engine.cpp +++ b/ydb/core/tx/columnshard/engines/ut_logs_engine.cpp @@ -325,7 +325,7 @@ bool Cleanup(TColumnEngineForLogs& engine, TTestDbWrapper& db, TSnapshot snap, u bool Ttl(TColumnEngineForLogs& engine, TTestDbWrapper& db, const THashMap<ui64, NOlap::TTiering>& pathEviction, ui32 expectedToDrop) { - std::shared_ptr<TTTLColumnEngineChanges> changes = engine.StartTtl(pathEviction, engine.GetIndexInfo().ArrowSchema()); + std::shared_ptr<TTTLColumnEngineChanges> changes = engine.StartTtl(pathEviction); UNIT_ASSERT(changes); UNIT_ASSERT_VALUES_EQUAL(changes->PortionsToDrop.size(), expectedToDrop); diff --git a/ydb/core/tx/columnshard/ut_schema/ut_columnshard_schema.cpp b/ydb/core/tx/columnshard/ut_schema/ut_columnshard_schema.cpp index aea9ddf1249..1b388028c28 100644 --- a/ydb/core/tx/columnshard/ut_schema/ut_columnshard_schema.cpp +++ b/ydb/core/tx/columnshard/ut_schema/ut_columnshard_schema.cpp @@ -697,7 +697,7 @@ std::vector<std::pair<ui32, ui64>> TestTiers(bool reboots, const std::vector<TSt // Read crossed with eviction (start) if (!misconfig) { - auto read = std::make_unique<TEvColumnShard::TEvRead>(sender, metaShard, planStep-1, Max<ui64>(), tableId); + auto read = std::make_unique<TEvColumnShard::TEvRead>(sender, metaShard, planStep - 1, Max<ui64>(), tableId); Proto(read.get()).AddColumnNames(specs[i].TtlColumn); counter.CaptureReadEvents = 1; // TODO: we need affected by tiering blob here diff --git a/ydb/core/tx/tiering/ut/ut_tiers.cpp b/ydb/core/tx/tiering/ut/ut_tiers.cpp index cd21295bc9a..a743adc90b1 100644 --- a/ydb/core/tx/tiering/ut/ut_tiers.cpp +++ b/ydb/core/tx/tiering/ut/ut_tiers.cpp @@ -493,13 +493,13 @@ Y_UNIT_TEST_SUITE(ColumnShardTiers) { Tests::NCommon::TLoggerInit(server->GetRuntime()).SetComponents({ NKikimrServices::TX_COLUMNSHARD }).Initialize(); auto& runtime = *server->GetRuntime(); - runtime.SetLogPriority(NKikimrServices::TX_PROXY, NLog::PRI_TRACE); - runtime.SetLogPriority(NKikimrServices::KQP_YQL, NLog::PRI_TRACE); +// runtime.SetLogPriority(NKikimrServices::TX_PROXY, NLog::PRI_TRACE); +// runtime.SetLogPriority(NKikimrServices::KQP_YQL, NLog::PRI_TRACE); auto sender = runtime.AllocateEdgeActor(); server->SetupRootStoragePools(sender); - runtime.SetLogPriority(NKikimrServices::TX_DATASHARD, NLog::PRI_NOTICE); +// runtime.SetLogPriority(NKikimrServices::TX_DATASHARD, NLog::PRI_NOTICE); runtime.SetLogPriority(NKikimrServices::TX_COLUMNSHARD, NLog::PRI_DEBUG); runtime.SetLogPriority(NKikimrServices::BG_TASKS, NLog::PRI_DEBUG); // runtime.SetLogPriority(NKikimrServices::TX_PROXY_SCHEME_CACHE, NLog::PRI_DEBUG); @@ -534,9 +534,8 @@ Y_UNIT_TEST_SUITE(ColumnShardTiers) { runtime.SimulateSleep(TDuration::Seconds(20)); Cerr << "Initialization tables" << Endl; const TInstant pkStart = Now() - TDuration::Days(15); - ui32 idx = 0; - auto batch = lHelper.TestArrowBatch(0, (pkStart + TDuration::Seconds(2 * idx++)).GetValue(), 6000); + auto batch = lHelper.TestArrowBatch(0, pkStart.GetValue(), 6000); auto batchSize = NArrow::GetBatchDataSize(batch); Cerr << "Inserting " << batchSize << " bytes..." << Endl; UNIT_ASSERT(batchSize > 4 * 1024 * 1024); // NColumnShard::TLimits::MIN_BYTES_TO_INSERT @@ -563,6 +562,7 @@ Y_UNIT_TEST_SUITE(ColumnShardTiers) { #else check = true; #endif + runtime.AdvanceCurrentTime(TDuration::Minutes(6)); runtime.SimulateSleep(TDuration::Seconds(1)); } UNIT_ASSERT(check); @@ -586,6 +586,7 @@ Y_UNIT_TEST_SUITE(ColumnShardTiers) { #else check = true; #endif + runtime.AdvanceCurrentTime(TDuration::Minutes(6)); runtime.SimulateSleep(TDuration::Seconds(1)); } UNIT_ASSERT(check); |