aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorserg-belyakov <serg-belyakov@yandex-team.com>2023-08-24 15:54:36 +0300
committerserg-belyakov <serg-belyakov@yandex-team.com>2023-08-24 16:09:41 +0300
commit83f0168df7c60e323f0b6ebea8ac12012fca5f03 (patch)
tree062360ba01215377a9e41893655adf6b7a738713
parent364d024c03a5fbe3b261299f6444fc4c8909fa31 (diff)
downloadydb-83f0168df7c60e323f0b6ebea8ac12012fca5f03.tar.gz
Add disk time consumption metrics to dsproxy node, fix -11 crash in hive tests, KIKIMR-17759
Fix -11 crash code Add estimatedDiskTimeConsumption metrics to dsproxy on node, KIKIMR-17759 Add estimatedDiskTimeConsumption to dsproxy on node
-rw-r--r--ydb/core/blobstorage/backpressure/queue.cpp12
-rw-r--r--ydb/core/blobstorage/backpressure/queue.h4
-rw-r--r--ydb/core/blobstorage/backpressure/queue_backpressure_client.cpp6
-rw-r--r--ydb/core/blobstorage/backpressure/queue_backpressure_client.h10
-rw-r--r--ydb/core/blobstorage/dsproxy/dsproxy.h13
-rw-r--r--ydb/core/blobstorage/dsproxy/dsproxy_monactor.cpp9
-rw-r--r--ydb/core/blobstorage/dsproxy/dsproxy_state.cpp2
-rw-r--r--ydb/core/blobstorage/dsproxy/group_sessions.cpp75
-rw-r--r--ydb/core/blobstorage/dsproxy/group_sessions.h10
-rw-r--r--ydb/core/blobstorage/storagepoolmon/storagepool_counters.h20
-rw-r--r--ydb/core/blobstorage/ut_vdisk/lib/helpers.cpp2
-rw-r--r--ydb/core/blobstorage/vdisk/common/vdisk_costmodel.cpp22
-rw-r--r--ydb/core/blobstorage/vdisk/common/vdisk_costmodel.h5
13 files changed, 140 insertions, 50 deletions
diff --git a/ydb/core/blobstorage/backpressure/queue.cpp b/ydb/core/blobstorage/backpressure/queue.cpp
index f16a73bcfc..0977eed6bf 100644
--- a/ydb/core/blobstorage/backpressure/queue.cpp
+++ b/ydb/core/blobstorage/backpressure/queue.cpp
@@ -11,7 +11,7 @@ TBlobStorageQueue::TBlobStorageQueue(const TIntrusivePtr<::NMonitoring::TDynamic
, NextMsgId(0)
, CurrentSequenceId(1)
, LogPrefix(logPrefix)
- , CostModel(2000, 100000000, 50000000, 540000, 540000, 500000, gType) // default cost model
+ , CostModel(std::make_shared<const TCostModel>(2000, 100000000, 50000000, 540000, 540000, 500000, gType)) // default cost model
, BSProxyCtx(bspctx)
, ClientId(clientId)
, BytesWaiting(0)
@@ -52,8 +52,8 @@ TBlobStorageQueue::~TBlobStorageQueue() {
void TBlobStorageQueue::UpdateCostModel(TInstant now, const NKikimrBlobStorage::TVDiskCostSettings& settings,
const TBlobStorageGroupType& type) {
TCostModel newCostModel(settings, type);
- if (newCostModel != CostModel) {
- CostModel = std::move(newCostModel);
+ if (newCostModel != *CostModel) {
+ CostModel = std::make_shared<const TCostModel>(std::move(newCostModel));
InvalidateCosts();
}
CostSettingsUpdate = now + TDuration::Minutes(1);
@@ -78,8 +78,8 @@ bool TBlobStorageQueue::SetMaxWindowSize(ui64 maxWindowSize) {
}
}
-ui32 TBlobStorageQueue::GetMinREALHugeBlobInBytes() const {
- return CostModel.MinREALHugeBlobInBytes;
+std::shared_ptr<const TCostModel> TBlobStorageQueue::GetCostModel() const {
+ return CostModel;
}
void TBlobStorageQueue::SetItemQueue(TItem& item, EItemQueue newQueue) {
@@ -141,7 +141,7 @@ void TBlobStorageQueue::SendToVDisk(const TActorContext& ctx, const TActorId& re
// update item's cost if it is dirty
if (item.DirtyCost) {
- item.Cost = CostModel.CalculateCost(item.CostEssence);
+ item.Cost = CostModel->CalculateCost(item.CostEssence);
item.DirtyCost = false;
}
diff --git a/ydb/core/blobstorage/backpressure/queue.h b/ydb/core/blobstorage/backpressure/queue.h
index c5d8387e2c..f6667aef89 100644
--- a/ydb/core/blobstorage/backpressure/queue.h
+++ b/ydb/core/blobstorage/backpressure/queue.h
@@ -116,7 +116,7 @@ class TBlobStorageQueue {
TString& LogPrefix;
- TCostModel CostModel;
+ std::shared_ptr<const TCostModel> CostModel;
TInstant CostSettingsUpdate;
TBSProxyContextPtr BSProxyCtx;
@@ -184,7 +184,7 @@ public:
const TBlobStorageGroupType& type);
void InvalidateCosts();
bool SetMaxWindowSize(ui64 maxWindowSize);
- ui32 GetMinREALHugeBlobInBytes() const;
+ std::shared_ptr<const TCostModel> GetCostModel() const;
void SetItemQueue(TItem& item, EItemQueue newQueue);
diff --git a/ydb/core/blobstorage/backpressure/queue_backpressure_client.cpp b/ydb/core/blobstorage/backpressure/queue_backpressure_client.cpp
index 9f4aa973f6..0fcaab8426 100644
--- a/ydb/core/blobstorage/backpressure/queue_backpressure_client.cpp
+++ b/ydb/core/blobstorage/backpressure/queue_backpressure_client.cpp
@@ -456,7 +456,7 @@ private:
case EState::READY:
QLOG_NOTICE_S("BSQ96", "connection lost status# " << NKikimrProto::EReplyStatus_Name(status)
<< " errorReason# " << errorReason << " timeout# " << timeout);
- ctx.Send(BlobStorageProxy, new TEvProxyQueueState(VDiskId, QueueId, false, false, 0));
+ ctx.Send(BlobStorageProxy, new TEvProxyQueueState(VDiskId, QueueId, false, false, nullptr));
Queue.DrainQueue(status, TStringBuilder() << "BS_QUEUE: " << errorReason, ctx);
DrainStatus(status, ctx);
DrainAssimilate(status, errorReason, ctx);
@@ -563,7 +563,7 @@ private:
Queue.UpdateCostModel(ctx.Now(), record.GetCostSettings(), GType);
}
ctx.Send(BlobStorageProxy, new TEvProxyQueueState(VDiskId, QueueId, true, ExtraBlockChecksSupport,
- Queue.GetMinREALHugeBlobInBytes()));
+ Queue.GetCostModel()));
Queue.OnConnect();
State = EState::READY;
} else {
@@ -799,7 +799,7 @@ private:
<< " VDiskId# " << VDiskId
<< " IsConnected# " << isConnected);
ctx.Send(ev->Sender, new TEvProxyQueueState(VDiskId, QueueId, isConnected, isConnected && ExtraBlockChecksSupport,
- Queue.GetMinREALHugeBlobInBytes()));
+ Queue.GetCostModel()));
}
#define QueueRequestHFunc(TEvType) \
diff --git a/ydb/core/blobstorage/backpressure/queue_backpressure_client.h b/ydb/core/blobstorage/backpressure/queue_backpressure_client.h
index 971b8e00fa..0b80053d61 100644
--- a/ydb/core/blobstorage/backpressure/queue_backpressure_client.h
+++ b/ydb/core/blobstorage/backpressure/queue_backpressure_client.h
@@ -17,15 +17,15 @@ namespace NKikimr {
NKikimrBlobStorage::EVDiskQueueId QueueId;
bool IsConnected;
bool ExtraBlockChecksSupport;
- ui32 MinREALHugeBlobInBytes;
+ std::shared_ptr<const TCostModel> CostModel;
TEvProxyQueueState(const TVDiskID &vDiskId, NKikimrBlobStorage::EVDiskQueueId queueId, bool isConnected,
- bool extraBlockChecksSupport, ui32 minREALHugeBlobInBytes)
+ bool extraBlockChecksSupport, std::shared_ptr<const TCostModel> costModel)
: VDiskId(vDiskId)
, QueueId(queueId)
, IsConnected(isConnected)
, ExtraBlockChecksSupport(extraBlockChecksSupport)
- , MinREALHugeBlobInBytes(minREALHugeBlobInBytes)
+ , CostModel(std::move(costModel))
{}
TString ToString() const {
@@ -34,7 +34,9 @@ namespace NKikimr {
str << " QueueId# " << static_cast<ui32>(QueueId);
str << " IsConnected# " << (IsConnected ? "true" : "false");
str << " ExtraBlockChecksSupport# " << (ExtraBlockChecksSupport ? "true" : "false");
- str << " MinREALHugeBlobInBytes# " << MinREALHugeBlobInBytes;
+ if (CostModel) {
+ str << " CostModel# " << CostModel->ToString();
+ }
str << "}";
return str.Str();
}
diff --git a/ydb/core/blobstorage/dsproxy/dsproxy.h b/ydb/core/blobstorage/dsproxy/dsproxy.h
index ce7f1a2478..2a1f64a24c 100644
--- a/ydb/core/blobstorage/dsproxy/dsproxy.h
+++ b/ydb/core/blobstorage/dsproxy/dsproxy.h
@@ -178,6 +178,7 @@ public:
, LogCtx(logComponent, logAccEnabled)
, Span(TWilson::BlobStorage, std::move(traceId), std::move(name))
, RestartCounter(restartCounter)
+ , CostModel(GroupQueues->CostModel)
, Source(source)
, Cookie(cookie)
, LatencyQueueKind(latencyQueueKind)
@@ -189,6 +190,8 @@ public:
Span
.Attribute("GroupId", Info->GroupID)
.Attribute("RestartCounter", RestartCounter);
+
+ Y_VERIFY(CostModel);
}
void Registered(TActorSystem *as, const TActorId& parentId) override {
@@ -366,7 +369,14 @@ public:
void SendToQueue(std::unique_ptr<T> event, ui64 cookie, bool timeStatsEnabled = false) {
if constexpr (!std::is_same_v<T, TEvBlobStorage::TEvVStatus> && !std::is_same_v<T, TEvBlobStorage::TEvVAssimilate>) {
event->MessageRelevanceTracker = MessageRelevanceTracker;
+ if constexpr (std::is_same_v<T, TEvBlobStorage::TEvVMultiPut>) {
+ bool internalQueue;
+ SentSubrequestCost += CostModel->GetCost(*event, &internalQueue);
+ } else {
+ SentSubrequestCost += CostModel->GetCost(*event);
+ }
}
+
const TActorId queueId = GroupQueues->Send(*this, Info->GetTopology(), std::move(event), cookie, Span.GetTraceId(),
timeStatsEnabled);
++RequestsInFlight;
@@ -499,6 +509,7 @@ public:
if (RequestHandleClass && PoolCounters) {
PoolCounters->GetItem(*RequestHandleClass, RequestBytes).Register(
RequestBytes, GeneratedSubrequests, GeneratedSubrequestBytes, Timer.Passed());
+ *PoolCounters->DSProxyDiskCostCounter += SentSubrequestCost;
}
if (timeStats) {
@@ -570,8 +581,10 @@ protected:
ui32 RequestBytes = 0;
ui32 GeneratedSubrequests = 0;
ui32 GeneratedSubrequestBytes = 0;
+ ui64 SentSubrequestCost = 0;
bool Dead = false;
const ui32 RestartCounter = 0;
+ std::shared_ptr<const TCostModel> CostModel;
private:
const TActorId Source;
diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_monactor.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_monactor.cpp
index 453b31506e..09da2018d5 100644
--- a/ydb/core/blobstorage/dsproxy/dsproxy_monactor.cpp
+++ b/ydb/core/blobstorage/dsproxy/dsproxy_monactor.cpp
@@ -87,8 +87,13 @@ public:
str << "GroupID: " << Info->GroupID << "<br/>" << "Generation: " << Info->GroupGeneration;
}
DIV() {
- str << "MinREALHugeBlobInBytes: ";
- str << GroupQueues->MinREALHugeBlobInBytes;
+ str << "CostModel: ";
+ auto costModel = GroupQueues->CostModel; // acquire owning pointer
+ if (costModel) {
+ str << costModel->ToString();
+ } else {
+ str << "None";
+ }
}
DIV() TABLE_CLASS("table table-bordered table-condensed") TABLEBODY() {
ui32 maxFailDomain = 0;
diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_state.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_state.cpp
index 0092b7d436..c74b0c09ef 100644
--- a/ydb/core/blobstorage/dsproxy/dsproxy_state.cpp
+++ b/ydb/core/blobstorage/dsproxy/dsproxy_state.cpp
@@ -206,7 +206,7 @@ namespace NKikimr {
auto *msg = ev->Get();
Y_VERIFY(Topology);
Sessions->QueueConnectUpdate(Topology->GetOrderNumber(msg->VDiskId), msg->QueueId, msg->IsConnected,
- msg->ExtraBlockChecksSupport, msg->MinREALHugeBlobInBytes, *Topology);
+ msg->ExtraBlockChecksSupport, msg->CostModel, *Topology);
MinREALHugeBlobInBytes = Sessions->GetMinREALHugeBlobInBytes();
if (msg->IsConnected && (CurrentStateFunc() == &TThis::StateEstablishingSessions ||
CurrentStateFunc() == &TThis::StateEstablishingSessionsTimeout)) {
diff --git a/ydb/core/blobstorage/dsproxy/group_sessions.cpp b/ydb/core/blobstorage/dsproxy/group_sessions.cpp
index 0ffe410e66..b56ae76421 100644
--- a/ydb/core/blobstorage/dsproxy/group_sessions.cpp
+++ b/ydb/core/blobstorage/dsproxy/group_sessions.cpp
@@ -116,49 +116,80 @@ bool TGroupSessions::GoodToGo(const TBlobStorageGroupInfo::TTopology& topology,
}
void TGroupSessions::QueueConnectUpdate(ui32 orderNumber, NKikimrBlobStorage::EVDiskQueueId queueId, bool connected,
- bool extraGroupChecksSupport, ui32 minREALHugeBlobInBytes, const TBlobStorageGroupInfo::TTopology& topology) {
+ bool extraGroupChecksSupport, std::shared_ptr<const TCostModel> costModel, const TBlobStorageGroupInfo::TTopology& topology) {
const auto v = topology.GetVDiskId(orderNumber);
const ui32 fdom = topology.GetFailDomainOrderNumber(v);
auto& f = GroupQueues->FailDomains[fdom];
auto& vdisk = f.VDisks[v.VDisk];
auto& q = vdisk.Queues.GetQueue(queueId);
+ bool updated = false;
+
if (connected) {
ConnectedQueuesMask[orderNumber] |= 1 << queueId;
q.ExtraBlockChecksSupport = extraGroupChecksSupport;
- q.MinREALHugeBlobInBytes = minREALHugeBlobInBytes;
+ Y_VERIFY(costModel);
+ if (!q.CostModel || *q.CostModel != *costModel) {
+ updated = true;
+ q.CostModel = costModel;
+ }
} else {
ConnectedQueuesMask[orderNumber] &= ~(1 << queueId);
q.ExtraBlockChecksSupport.reset();
- q.MinREALHugeBlobInBytes = 0;
+ if (q.CostModel) {
+ updated = true;
+ q.CostModel = nullptr;
+ }
}
q.IsConnected = connected;
- auto update = [](auto& current, const auto& next) {
- if (next.MinREALHugeBlobInBytes && (!current.MinREALHugeBlobInBytes || next.MinREALHugeBlobInBytes < current.MinREALHugeBlobInBytes)) {
- current.MinREALHugeBlobInBytes = next.MinREALHugeBlobInBytes;
+ if (updated) {
+ auto iterate = [](auto& currentCostModel, const auto& next) {
+ if (next.CostModel) {
+ if (!currentCostModel) {
+ currentCostModel.emplace(*next.CostModel);
+ } else {
+ currentCostModel->PessimisticComposition(*next.CostModel);
+ }
+ }
+ };
+
+ auto update = [](std::shared_ptr<const TCostModel>& current, const std::optional<TCostModel>& recalculated) {
+ if (!recalculated) {
+ current.reset();
+ } else {
+ if (!current || *current != *recalculated) {
+ current = std::make_shared<const TCostModel>(*recalculated);
+ }
+ }
+ };
+
+ // recalculate CostModel for the whole VDisk
+ std::optional<TCostModel> pessimistic;
+ vdisk.CostModel.reset();
+ vdisk.Queues.ForEachQueue([&](auto& q) { iterate(pessimistic, q); });
+ update(vdisk.CostModel, pessimistic);
+
+ // do the same for the fail domain
+ f.CostModel.reset();
+ pessimistic.reset();
+ for (const auto& vdisk : f.VDisks) {
+ iterate(pessimistic, vdisk);
}
- };
+ update(f.CostModel, pessimistic);
- // recalculate MinREALHugeBlobInBytes for the whole VDisk
- vdisk.MinREALHugeBlobInBytes = 0;
- vdisk.Queues.ForEachQueue([&](auto& q) { update(vdisk, q); });
-
- // do the same for the fail domain
- f.MinREALHugeBlobInBytes = 0;
- for (const auto& vdisk : f.VDisks) {
- update(f, vdisk);
- }
-
- // and for the whole group
- GroupQueues->MinREALHugeBlobInBytes = 0;
- for (const auto& fdom : GroupQueues->FailDomains) {
- update(*GroupQueues, fdom);
+ // and for the whole group
+ GroupQueues->CostModel.reset();
+ pessimistic.reset();
+ for (const auto& fdom : GroupQueues->FailDomains) {
+ iterate(pessimistic, fdom);
+ }
+ update(GroupQueues->CostModel, pessimistic);
}
}
ui32 TGroupSessions::GetMinREALHugeBlobInBytes() const {
- return GroupQueues->MinREALHugeBlobInBytes;
+ return GroupQueues->CostModel ? GroupQueues->CostModel->MinREALHugeBlobInBytes : 0;
}
ui32 TGroupSessions::GetNumUnconnectedDisks() {
diff --git a/ydb/core/blobstorage/dsproxy/group_sessions.h b/ydb/core/blobstorage/dsproxy/group_sessions.h
index 714159f90c..c8ce50fca8 100644
--- a/ydb/core/blobstorage/dsproxy/group_sessions.h
+++ b/ydb/core/blobstorage/dsproxy/group_sessions.h
@@ -22,7 +22,7 @@ namespace NKikimr {
TActorId ActorId;
TIntrusivePtr<NBackpressure::TFlowRecord> FlowRecord;
std::optional<bool> ExtraBlockChecksSupport;
- ui32 MinREALHugeBlobInBytes = 0;
+ std::shared_ptr<const TCostModel> CostModel = nullptr;
volatile bool IsConnected = false;
};
TQueue PutTabletLog;
@@ -133,7 +133,7 @@ namespace NKikimr {
};
TQueues Queues;
- ui32 MinREALHugeBlobInBytes = 0;
+ std::shared_ptr<const TCostModel> CostModel;
TString ToString() const {
return TStringBuilder() << "{Queues# " << Queues.ToString() << "}";
@@ -142,7 +142,7 @@ namespace NKikimr {
struct TFailDomain {
TStackVec<TVDisk, TypicalDisksInFailDomain> VDisks;
- ui32 MinREALHugeBlobInBytes = 0;
+ std::shared_ptr<const TCostModel> CostModel;
// Ill-formed because TVDisk is not assignable.
TFailDomain(const TFailDomain& other) = default;
@@ -162,7 +162,7 @@ namespace NKikimr {
TStackVec<TFailDomain, TypicalFailDomainsInGroup> FailDomains;
TStackVec<TVDisk*, TypicalDisksInGroup> DisksByOrderNumber;
- ui32 MinREALHugeBlobInBytes = 0;
+ std::shared_ptr<const TCostModel> CostModel;
TGroupQueues(const TBlobStorageGroupInfo::TTopology& topology)
: FailDomains(topology.GetTotalFailDomainsNum())
@@ -246,7 +246,7 @@ namespace NKikimr {
void Poison();
bool GoodToGo(const TBlobStorageGroupInfo::TTopology& topology, bool waitForAllVDisks);
void QueueConnectUpdate(ui32 orderNumber, NKikimrBlobStorage::EVDiskQueueId queueId, bool connected,
- bool extraBlockChecksSupport, ui32 minREALHugeBlobInBytes, const TBlobStorageGroupInfo::TTopology& topology);
+ bool extraBlockChecksSupport, std::shared_ptr<const TCostModel> costModel, const TBlobStorageGroupInfo::TTopology& topology);
ui32 GetNumUnconnectedDisks();
ui32 GetMinREALHugeBlobInBytes() const;
};
diff --git a/ydb/core/blobstorage/storagepoolmon/storagepool_counters.h b/ydb/core/blobstorage/storagepoolmon/storagepool_counters.h
index 0eb004ec16..de923b6bb7 100644
--- a/ydb/core/blobstorage/storagepoolmon/storagepool_counters.h
+++ b/ydb/core/blobstorage/storagepoolmon/storagepool_counters.h
@@ -149,6 +149,8 @@ private:
TRequestMonItem RequestMon[HcCount][MaxSizeClassBucketIdx + 1];
TString StoragePoolName;
+ TIntrusivePtr<::NMonitoring::TDynamicCounters> PoolGroup;
+
public:
TRequestMonItem& GetItem(EHandleClass handleClass, ui32 requestBytes) {
Y_VERIFY((ui32)handleClass < (ui32)HcCount);
@@ -164,12 +166,13 @@ public:
}
TStoragePoolCounters(TIntrusivePtr<::NMonitoring::TDynamicCounters> &counters, const TString &storagePoolName,
- NPDisk::EDeviceType type) {
- StoragePoolName = storagePoolName;
- TIntrusivePtr<::NMonitoring::TDynamicCounters> poolGroup = counters->GetSubgroup("storagePool", storagePoolName);
+ NPDisk::EDeviceType type)
+ : StoragePoolName(storagePoolName)
+ , PoolGroup(counters->GetSubgroup("storagePool", storagePoolName))
+ {
for (ui32 handleClass = 0; handleClass < (ui32)HcCount; ++handleClass) {
TString handleClassName = GetHandleClassName((EHandleClass)handleClass);
- TIntrusivePtr<::NMonitoring::TDynamicCounters> hcGroup = poolGroup->GetSubgroup("handleClass", handleClassName);
+ TIntrusivePtr<::NMonitoring::TDynamicCounters> hcGroup = PoolGroup->GetSubgroup("handleClass", handleClassName);
if (IsReducedHandleClass((EHandleClass)handleClass)) {
for (ui32 sizeClassIdx = 0; sizeClassIdx <= MaxReducedSizeClassBucketIdx; ++sizeClassIdx) {
TString sizeClassName = ReducedSizeClassName(sizeClassIdx);
@@ -182,8 +185,17 @@ public:
}
}
}
+
+ // request cost counters
+ {
+ auto group = PoolGroup->GetSubgroup("subsystem", "cost");
+ DSProxyDiskCostCounter = group->GetCounter("DSProxyDiskCostNs", true);
+ }
}
+public:
+ // request cost counters
+ ::NMonitoring::TDynamicCounters::TCounterPtr DSProxyDiskCostCounter;
};
class TDsProxyPerPoolCounters : public TThrRefBase {
diff --git a/ydb/core/blobstorage/ut_vdisk/lib/helpers.cpp b/ydb/core/blobstorage/ut_vdisk/lib/helpers.cpp
index 884673b2ed..48ea78f3bd 100644
--- a/ydb/core/blobstorage/ut_vdisk/lib/helpers.cpp
+++ b/ydb/core/blobstorage/ut_vdisk/lib/helpers.cpp
@@ -413,7 +413,7 @@ class TManyMultiPuts : public TActorBootstrapped<TManyMultiPuts> {
void Handle(TEvProxyQueueState::TPtr& ev, const TActorContext& ctx) {
if (ev->Get()->IsConnected && !Started) {
// put logo blob
- MinREALHugeBlobInBytes = ev->Get()->MinREALHugeBlobInBytes;
+ MinREALHugeBlobInBytes = ev->Get()->CostModel->MinREALHugeBlobInBytes;
Y_VERIFY(MinREALHugeBlobInBytes);
SendPut(ctx);
Started = true;
diff --git a/ydb/core/blobstorage/vdisk/common/vdisk_costmodel.cpp b/ydb/core/blobstorage/vdisk/common/vdisk_costmodel.cpp
index 3acfe26ea3..4a6921a34e 100644
--- a/ydb/core/blobstorage/vdisk/common/vdisk_costmodel.cpp
+++ b/ydb/core/blobstorage/vdisk/common/vdisk_costmodel.cpp
@@ -153,4 +153,26 @@ namespace NKikimr {
return cost;
}
+ TString TCostModel::ToString() const {
+ TStringStream str;
+ str << "{SeekTimeUs# " << SeekTimeUs;
+ str << " ReadSpeedBps# " << ReadSpeedBps;
+ str << " WriteSpeedBps# " << WriteSpeedBps;
+ str << " ReadBlockSize# " << ReadBlockSize;
+ str << " WriteBlockSize# " << WriteBlockSize;
+ str << " MinREALHugeBlobInBytes# " << MinREALHugeBlobInBytes;
+ str << " GType# " << GType.ToString();
+ str << "}";
+ return str.Str();
+ }
+
+ void TCostModel::PessimisticComposition(const TCostModel& other) {
+ SeekTimeUs = std::max(SeekTimeUs, other.SeekTimeUs);
+ ReadSpeedBps = std::min(ReadSpeedBps, other.ReadSpeedBps);
+ WriteSpeedBps = std::min(WriteSpeedBps, other.WriteSpeedBps);
+ ReadBlockSize = std::min(ReadBlockSize, other.ReadBlockSize);
+ WriteBlockSize = std::min(WriteBlockSize, other.WriteBlockSize);
+ MinREALHugeBlobInBytes = std::max(MinREALHugeBlobInBytes, other.MinREALHugeBlobInBytes);
+ }
+
} // NKikimr
diff --git a/ydb/core/blobstorage/vdisk/common/vdisk_costmodel.h b/ydb/core/blobstorage/vdisk/common/vdisk_costmodel.h
index eb16996c30..d072757471 100644
--- a/ydb/core/blobstorage/vdisk/common/vdisk_costmodel.h
+++ b/ydb/core/blobstorage/vdisk/common/vdisk_costmodel.h
@@ -216,6 +216,11 @@ namespace NKikimr {
ui64 MovedPatchCostBySize(ui32 blobSize) const;
ui64 ReadCostBySize(ui64 size) const;
ui64 ReadCost(const TEvBlobStorage::TEvVGet &ev) const;
+
+ public:
+ TString ToString() const;
+ void PessimisticComposition(const TCostModel& other);
+
};
} // NKikimr