diff --git a/ydb/core/base/blobstorage.h b/ydb/core/base/blobstorage.h index 2413eb2c4235..7d018a9e1e66 100644 --- a/ydb/core/base/blobstorage.h +++ b/ydb/core/base/blobstorage.h @@ -488,6 +488,8 @@ struct TEvBlobStorage { EvInplacePatch, EvAssimilate, + EvGetQueuesInfo, // for debugging purposes + // EvPutResult = EvPut + 512, /// 268 632 576 EvGetResult, @@ -502,6 +504,8 @@ struct TEvBlobStorage { EvInplacePatchResult, EvAssimilateResult, + EvQueuesInfo, // for debugging purposes + // proxy <-> vdisk interface EvVPut = EvPut + 2 * 512, /// 268 633 088 EvVGet, @@ -869,6 +873,7 @@ struct TEvBlobStorage { EvRunActor = EvPut + 15 * 512, EvVMockCtlRequest, EvVMockCtlResponse, + EvDelayedMessageWrapper, // incremental huge blob keeper EvIncrHugeInit = EvPut + 17 * 512, diff --git a/ydb/core/blobstorage/backpressure/common.h b/ydb/core/blobstorage/backpressure/common.h index 6539a5db84e8..fc94ac6b1fce 100644 --- a/ydb/core/blobstorage/backpressure/common.h +++ b/ydb/core/blobstorage/backpressure/common.h @@ -15,3 +15,39 @@ #define QLOG_DEBUG_S(marker, arg) QLOG_LOG_S(marker, NActors::NLog::PRI_DEBUG , arg) LWTRACE_USING(BLOBSTORAGE_PROVIDER); + +namespace NKikimr::NBsQueue { + +// Special timer for debug purposes, which works with virtual time of TTestActorSystem +struct TActivationContextTimer { + TActivationContextTimer() + : CreationTimestamp(NActors::TActivationContext::Monotonic()) + {} + + double Passed() const { + return (NActors::TActivationContext::Monotonic() - CreationTimestamp).SecondsFloat(); + } + + TMonotonic CreationTimestamp; +}; + +struct TBSQueueTimer { + TBSQueueTimer(bool useActorSystemTime) + { + if (useActorSystemTime) { + Timer.emplace(); + } else { + Timer.emplace(); + } + } + + std::variant Timer; + + double Passed() const { + return std::visit([](const auto& timer) -> double { + return timer.Passed(); + }, Timer); + } +}; + +} // namespace NKikimr::NBsQueue \ No newline at end of file diff --git a/ydb/core/blobstorage/backpressure/event.cpp b/ydb/core/blobstorage/backpressure/event.cpp index 4426704fe6bd..db9bf4078cd7 100644 --- a/ydb/core/blobstorage/backpressure/event.cpp +++ b/ydb/core/blobstorage/backpressure/event.cpp @@ -27,7 +27,7 @@ IEventBase *TEventHolder::MakeErrorReply(NKikimrProto::EReplyStatus status, cons void TEventHolder::SendToVDisk(const TActorContext& ctx, const TActorId& remoteVDisk, ui64 queueCookie, ui64 msgId, ui64 sequenceId, bool sendMeCostSettings, NWilson::TTraceId traceId, const NBackpressure::TQueueClientId& clientId, - const THPTimer& processingTimer) { + const TBSQueueTimer& processingTimer) { // check that we are not discarded yet Y_ABORT_UNLESS(Type != 0); diff --git a/ydb/core/blobstorage/backpressure/event.h b/ydb/core/blobstorage/backpressure/event.h index 0c77f3c4a188..7400a9ce8850 100644 --- a/ydb/core/blobstorage/backpressure/event.h +++ b/ydb/core/blobstorage/backpressure/event.h @@ -142,7 +142,7 @@ class TEventHolder { void SendToVDisk(const TActorContext& ctx, const TActorId& remoteVDisk, ui64 queueCookie, ui64 msgId, ui64 sequenceId, bool sendMeCostSettings, NWilson::TTraceId traceId, const NBackpressure::TQueueClientId& clientId, - const THPTimer& processingTimer); + const TBSQueueTimer& processingTimer); void Discard(); }; diff --git a/ydb/core/blobstorage/backpressure/queue.cpp b/ydb/core/blobstorage/backpressure/queue.cpp index b2402a32c671..da7433ff8cc3 100644 --- a/ydb/core/blobstorage/backpressure/queue.cpp +++ b/ydb/core/blobstorage/backpressure/queue.cpp @@ -4,7 +4,7 @@ namespace NKikimr::NBsQueue { TBlobStorageQueue::TBlobStorageQueue(const TIntrusivePtr<::NMonitoring::TDynamicCounters>& counters, TString& logPrefix, const TBSProxyContextPtr& bspctx, const NBackpressure::TQueueClientId& clientId, ui32 interconnectChannel, - const TBlobStorageGroupType& gType, NMonitoring::TCountableBase::EVisibility visibility) + const TBlobStorageGroupType& gType, NMonitoring::TCountableBase::EVisibility visibility, bool useActorSystemTime) : Queues(bspctx) , WindowSize(0) , InFlightCost(0) @@ -16,6 +16,7 @@ TBlobStorageQueue::TBlobStorageQueue(const TIntrusivePtr<::NMonitoring::TDynamic , ClientId(clientId) , BytesWaiting(0) , InterconnectChannel(interconnectChannel) + , UseActorSystemTime(useActorSystemTime) // use parent group visibility , QueueWaitingItems(counters->GetCounter("QueueWaitingItems", false, visibility)) , QueueWaitingBytes(counters->GetCounter("QueueWaitingBytes", false, visibility)) diff --git a/ydb/core/blobstorage/backpressure/queue.h b/ydb/core/blobstorage/backpressure/queue.h index 61d9843f1451..b0acd5383758 100644 --- a/ydb/core/blobstorage/backpressure/queue.h +++ b/ydb/core/blobstorage/backpressure/queue.h @@ -51,7 +51,8 @@ class TBlobStorageQueue { const ui64 QueueCookie; ui64 Cost; bool DirtyCost; - THPTimer ProcessingTimer; + TBSQueueTimer ProcessingTimer; + TTrackableList::iterator Iterator; template @@ -59,7 +60,7 @@ class TBlobStorageQueue { const ::NMonitoring::TDynamicCounters::TCounterPtr& serItems, const ::NMonitoring::TDynamicCounters::TCounterPtr& serBytes, const TBSProxyContextPtr& bspctx, ui32 interconnectChannel, - bool local) + bool local, bool useActorSystemTime) : Queue(EItemQueue::NotSet) , CostEssence(*event->Get()) , Span(TWilson::VDiskTopLevel, std::move(event->TraceId), "Backpressure.InFlight") @@ -70,6 +71,7 @@ class TBlobStorageQueue { , QueueCookie(RandomNumber()) , Cost(0) , DirtyCost(true) + , ProcessingTimer(useActorSystemTime) { if (Span) { Span @@ -129,6 +131,8 @@ class TBlobStorageQueue { const ui32 InterconnectChannel; + const bool UseActorSystemTime; + public: ::NMonitoring::TDynamicCounters::TCounterPtr QueueWaitingItems; ::NMonitoring::TDynamicCounters::TCounterPtr QueueWaitingBytes; @@ -156,7 +160,8 @@ class TBlobStorageQueue { TBlobStorageQueue(const TIntrusivePtr<::NMonitoring::TDynamicCounters>& counters, TString& logPrefix, const TBSProxyContextPtr& bspctx, const NBackpressure::TQueueClientId& clientId, ui32 interconnectChannel, const TBlobStorageGroupType &gType, - NMonitoring::TCountableBase::EVisibility visibility = NMonitoring::TCountableBase::EVisibility::Public); + NMonitoring::TCountableBase::EVisibility visibility = NMonitoring::TCountableBase::EVisibility::Public, + bool useActorSystemTime = false); ~TBlobStorageQueue(); @@ -213,7 +218,8 @@ class TBlobStorageQueue { TItemList::iterator newIt; if (Queues.Unused.empty()) { newIt = Queues.Waiting.emplace(Queues.Waiting.end(), event, deadline, - QueueSerializedItems, QueueSerializedBytes, BSProxyCtx, InterconnectChannel, local); + QueueSerializedItems, QueueSerializedBytes, BSProxyCtx, InterconnectChannel, local, + UseActorSystemTime); ++*QueueSize; } else { newIt = Queues.Unused.begin(); @@ -222,7 +228,7 @@ class TBlobStorageQueue { TItem& item = *newIt; item.~TItem(); new(&item) TItem(event, deadline, QueueSerializedItems, QueueSerializedBytes, BSProxyCtx, - InterconnectChannel, local); + InterconnectChannel, local, UseActorSystemTime); } newIt->Iterator = newIt; diff --git a/ydb/core/blobstorage/backpressure/queue_backpressure_client.cpp b/ydb/core/blobstorage/backpressure/queue_backpressure_client.cpp index f361d6a80782..d8c2c6db742c 100644 --- a/ydb/core/blobstorage/backpressure/queue_backpressure_client.cpp +++ b/ydb/core/blobstorage/backpressure/queue_backpressure_client.cpp @@ -77,12 +77,13 @@ class TVDiskBackpressureClientActor : public TActorBootstrapped& counters, const TBSProxyContextPtr& bspctx, const NBackpressure::TQueueClientId& clientId, const TString& queueName, ui32 interconnectChannel, bool /*local*/, TDuration watchdogTimeout, - TIntrusivePtr &flowRecord, NMonitoring::TCountableBase::EVisibility visibility) + TIntrusivePtr &flowRecord, NMonitoring::TCountableBase::EVisibility visibility, + bool useActorSystemTime) : BSProxyCtx(bspctx) , QueueName(queueName) , Counters(counters->GetSubgroup("queue", queueName)) , Queue(Counters, LogPrefix, bspctx, clientId, interconnectChannel, - (info ? info->Type : TErasureType::ErasureNone), visibility) + (info ? info->Type : TErasureType::ErasureNone), visibility, useActorSystemTime) , VDiskIdShort(vdiskId) , QueueId(queueId) , QueueWatchdogTimeout(watchdogTimeout) @@ -975,9 +976,10 @@ IActor* CreateVDiskBackpressureClient(const TIntrusivePtr NKikimrBlobStorage::EVDiskQueueId queueId,const TIntrusivePtr<::NMonitoring::TDynamicCounters>& counters, const TBSProxyContextPtr& bspctx, const NBackpressure::TQueueClientId& clientId, const TString& queueName, ui32 interconnectChannel, bool local, TDuration watchdogTimeout, - TIntrusivePtr &flowRecord, NMonitoring::TCountableBase::EVisibility visibility) { + TIntrusivePtr &flowRecord, NMonitoring::TCountableBase::EVisibility visibility, + bool useActorSystemTime) { return new NBsQueue::TVDiskBackpressureClientActor(info, vdiskId, queueId, counters, bspctx, clientId, queueName, - interconnectChannel, local, watchdogTimeout, flowRecord, visibility); + interconnectChannel, local, watchdogTimeout, flowRecord, visibility, useActorSystemTime); } } // NKikimr diff --git a/ydb/core/blobstorage/backpressure/queue_backpressure_client.h b/ydb/core/blobstorage/backpressure/queue_backpressure_client.h index 0b80053d6133..722874818838 100644 --- a/ydb/core/blobstorage/backpressure/queue_backpressure_client.h +++ b/ydb/core/blobstorage/backpressure/queue_backpressure_client.h @@ -50,6 +50,7 @@ namespace NKikimr { NKikimrBlobStorage::EVDiskQueueId queueId,const TIntrusivePtr<::NMonitoring::TDynamicCounters>& counters, const TBSProxyContextPtr& bspctx, const NBackpressure::TQueueClientId& clientId, const TString& queueName, ui32 interconnectChannel, bool local, TDuration watchdogTimeout, - TIntrusivePtr &flowRecord, NMonitoring::TCountableBase::EVisibility visibility); + TIntrusivePtr &flowRecord, NMonitoring::TCountableBase::EVisibility visibility, + bool useActorSystemTime = false); } // NKikimr diff --git a/ydb/core/blobstorage/common/defs.h b/ydb/core/blobstorage/common/defs.h new file mode 100644 index 000000000000..fe09e00a103d --- /dev/null +++ b/ydb/core/blobstorage/common/defs.h @@ -0,0 +1,4 @@ +#pragma once + +#include +#include diff --git a/ydb/core/blobstorage/common/immediate_control_defaults.cpp b/ydb/core/blobstorage/common/immediate_control_defaults.cpp new file mode 100644 index 000000000000..03749ff97bed --- /dev/null +++ b/ydb/core/blobstorage/common/immediate_control_defaults.cpp @@ -0,0 +1,14 @@ +#include "immediate_control_defaults.h" + +namespace NKikimr { + +TControlWrapper SlowDiskThresholdDefaultControl = + TControlWrapper(std::round(DefaultSlowDiskThreshold * 1000), 1, 1'000'000); + +TControlWrapper PredictedDelayMultiplierDefaultControl = + TControlWrapper(std::round(DefaultPredictedDelayMultiplier * 1000), 0, 1'000'000); + +TControlWrapper MaxNumOfSlowDisksDefaultControl = + TControlWrapper(DefaultMaxNumOfSlowDisks, 1, 2); + +} // namespace NKikimr diff --git a/ydb/core/blobstorage/common/immediate_control_defaults.h b/ydb/core/blobstorage/common/immediate_control_defaults.h new file mode 100644 index 000000000000..98377ffb84cf --- /dev/null +++ b/ydb/core/blobstorage/common/immediate_control_defaults.h @@ -0,0 +1,18 @@ +#pragma once + +#include "defs.h" +#include + +namespace NKikimr { + +constexpr bool DefaultEnablePutBatching = true; +constexpr bool DefaultEnableVPatch = false; + +constexpr float DefaultSlowDiskThreshold = 2; +constexpr float DefaultPredictedDelayMultiplier = 1; +constexpr ui32 DefaultMaxNumOfSlowDisks = 2; + +extern TControlWrapper SlowDiskThresholdDefaultControl; +extern TControlWrapper PredictedDelayMultiplierDefaultControl; +extern TControlWrapper MaxNumOfSlowDisksDefaultControl; +} diff --git a/ydb/core/blobstorage/common/ya.make b/ydb/core/blobstorage/common/ya.make new file mode 100644 index 000000000000..a257ed4e8607 --- /dev/null +++ b/ydb/core/blobstorage/common/ya.make @@ -0,0 +1,11 @@ +LIBRARY() + +PEERDIR( + ydb/core/base +) + +SRCS( + immediate_control_defaults.cpp +) + +END() diff --git a/ydb/core/blobstorage/dsproxy/dsproxy.h b/ydb/core/blobstorage/dsproxy/dsproxy.h index 55bcc513dcc6..1d839f3a48e4 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy.h +++ b/ydb/core/blobstorage/dsproxy/dsproxy.h @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -52,9 +53,6 @@ const ui32 MaxRequestSize = 1000; const ui32 MaskSizeBits = 32; -constexpr bool DefaultEnablePutBatching = true; -constexpr bool DefaultEnableVPatch = false; - constexpr bool WithMovingPatchRequestToStaticNode = true; //////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// @@ -168,6 +166,12 @@ inline void SetExecutionRelay(IEventBase& ev, std::shared_ptr class TBlobStorageGroupRequestActor : public TActor { public: @@ -175,26 +179,47 @@ class TBlobStorageGroupRequestActor : public TActor { return NKikimrServices::TActivity::BS_GROUP_REQUEST; } - TBlobStorageGroupRequestActor(TIntrusivePtr info, TIntrusivePtr groupQueues, - TIntrusivePtr mon, const TActorId& source, ui64 cookie, - NKikimrServices::EServiceKikimr logComponent, bool logAccEnabled, TMaybe latencyQueueKind, - TInstant now, TIntrusivePtr &storagePoolCounters, ui32 restartCounter, - NWilson::TSpan&& span, std::shared_ptr executionRelay) - : TActor(&TThis::InitialStateFunc, TDerived::ActorActivityType()) - , Info(std::move(info)) - , GroupQueues(std::move(groupQueues)) - , Mon(std::move(mon)) - , PoolCounters(storagePoolCounters) - , LogCtx(logComponent, logAccEnabled) - , Span(std::move(span)) - , RestartCounter(restartCounter) + struct TCommonParameters { + TIntrusivePtr GroupInfo; + TIntrusivePtr GroupQueues; + TIntrusivePtr Mon; + TActorId Source = TActorId{}; + ui64 Cookie = 0; + TInstant Now; + TIntrusivePtr& StoragePoolCounters; + ui32 RestartCounter; + NWilson::TSpan Span; + TDerived* Event = nullptr; + std::shared_ptr ExecutionRelay = nullptr; + + bool LogAccEnabled = false; + TMaybe LatencyQueueKind = {}; + }; + + struct TTypeSpecificParameters { + NKikimrServices::EServiceKikimr LogComponent; + const char* Name; + NKikimrServices::TActivity::EType Activity; + }; + +public: + template + TBlobStorageGroupRequestActor(TGroupRequestParameters& params) + : TActor(&TThis::InitialStateFunc, params.TypeSpecific.Activity) + , Info(std::move(params.Common.GroupInfo)) + , GroupQueues(std::move(params.Common.GroupQueues)) + , Mon(std::move(params.Common.Mon)) + , PoolCounters(params.Common.StoragePoolCounters) + , LogCtx(params.TypeSpecific.LogComponent, params.Common.LogAccEnabled) + , Span(std::move(params.Common.Span)) + , RestartCounter(params.Common.RestartCounter) , CostModel(GroupQueues->CostModel) - , Source(source) - , Cookie(cookie) - , LatencyQueueKind(latencyQueueKind) - , RequestStartTime(now) + , Source(params.Common.Source) + , Cookie(params.Common.Cookie) + , LatencyQueueKind(params.Common.LatencyQueueKind) + , RequestStartTime(params.Common.Now) , RacingDomains(&Info->GetTopology()) - , ExecutionRelay(std::move(executionRelay)) + , ExecutionRelay(std::move(params.Common.ExecutionRelay)) { TDerived::ActiveCounter(Mon)->Inc(); Span @@ -644,101 +669,196 @@ void Decrypt(char *destination, const char *source, size_t shift, size_t sizeByt const TBlobStorageGroupInfo &info); void DecryptInplace(TRope& rope, ui32 offset, ui32 shift, ui32 size, const TLogoBlobID& id, const TBlobStorageGroupInfo& info); -IActor* CreateBlobStorageGroupRangeRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvRange *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, TIntrusivePtr &storagePoolCounters); - -IActor* CreateBlobStorageGroupPutRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvPut *ev, - ui64 cookie, NWilson::TTraceId traceId, bool timeStatsEnabled, - TDiskResponsivenessTracker::TPerDiskStatsPtr stats, - TMaybe latencyQueueKind, TInstant now, TIntrusivePtr &storagePoolCounters, - bool enableRequestMod3x3ForMinLatecy); - -IActor* CreateBlobStorageGroupPutRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, - const TIntrusivePtr &mon, - TBatchedVec &ev, - bool timeStatsEnabled, - TDiskResponsivenessTracker::TPerDiskStatsPtr stats, - TMaybe latencyQueueKind, TInstant now, TIntrusivePtr &storagePoolCounters, - NKikimrBlobStorage::EPutHandleClass handleClass, TEvBlobStorage::TEvPut::ETactic tactic, - bool enableRequestMod3x3ForMinLatecy); - -IActor* CreateBlobStorageGroupGetRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvGet *ev, - ui64 cookie, NWilson::TTraceId traceId, TNodeLayoutInfoPtr&& nodeLayout, - TMaybe latencyQueueKind, TInstant now, TIntrusivePtr &storagePoolCounters); - -IActor* CreateBlobStorageGroupPatchRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvPatch *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, TIntrusivePtr &storagePoolCounters, - bool useVPatch); - -IActor* CreateBlobStorageGroupMultiGetRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvGet *ev, - ui64 cookie, NWilson::TTraceId traceId, TMaybe latencyQueueKind, - TInstant now, TIntrusivePtr &storagePoolCounters); - -IActor* CreateBlobStorageGroupIndexRestoreGetRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvGet *ev, - ui64 cookie, NWilson::TTraceId traceId, TMaybe latencyQueueKind, - TInstant now, TIntrusivePtr &storagePoolCounters); - -IActor* CreateBlobStorageGroupDiscoverRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvDiscover *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, TIntrusivePtr &storagePoolCounters); - -IActor* CreateBlobStorageGroupMirror3dcDiscoverRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvDiscover *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, TIntrusivePtr &storagePoolCounters); - -IActor* CreateBlobStorageGroupMirror3of4DiscoverRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvDiscover *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, TIntrusivePtr &storagePoolCounters); - -IActor* CreateBlobStorageGroupCollectGarbageRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvCollectGarbage *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, TIntrusivePtr &storagePoolCounters); - -IActor* CreateBlobStorageGroupMultiCollectRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvCollectGarbage *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, TIntrusivePtr &storagePoolCounters); - -IActor* CreateBlobStorageGroupBlockRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvBlock *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, TIntrusivePtr &storagePoolCounters); - -IActor* CreateBlobStorageGroupStatusRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvStatus *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, TIntrusivePtr &storagePoolCounters); - -IActor* CreateBlobStorageGroupAssimilateRequest(const TIntrusivePtr& info, - const TIntrusivePtr& state, const TActorId& source, - const TIntrusivePtr& mon, TEvBlobStorage::TEvAssimilate *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, TIntrusivePtr& storagePoolCounters); +struct TBlobStorageGroupRangeParameters { + TBlobStorageGroupRequestActor::TCommonParameters Common; + TBlobStorageGroupRequestActor::TTypeSpecificParameters TypeSpecific = { + .LogComponent = NKikimrServices::BS_PROXY_RANGE, + .Name = "DSProxy.Range", + .Activity = NKikimrServices::TActivity::BS_GROUP_RANGE + , + }; +}; +IActor* CreateBlobStorageGroupRangeRequest(TBlobStorageGroupRangeParameters params, NWilson::TTraceId traceId); + +struct TBlobStorageGroupPutParameters { + TBlobStorageGroupRequestActor::TCommonParameters Common; + TBlobStorageGroupRequestActor::TTypeSpecificParameters TypeSpecific = { + .LogComponent = NKikimrServices::BS_PROXY_PUT, + .Name = "DSProxy.Put", + .Activity = NKikimrServices::TActivity::BS_PROXY_PUT_ACTOR, + }; + bool TimeStatsEnabled; + TDiskResponsivenessTracker::TPerDiskStatsPtr Stats; + bool EnableRequestMod3x3ForMinLatency; + TAccelerationParams AccelerationParams; +}; +IActor* CreateBlobStorageGroupPutRequest(TBlobStorageGroupPutParameters params, NWilson::TTraceId traceId); + +struct TBlobStorageGroupMultiPutParameters { + TBlobStorageGroupRequestActor::TCommonParameters Common; + TBlobStorageGroupRequestActor::TTypeSpecificParameters TypeSpecific = { + .LogComponent = NKikimrServices::BS_PROXY_PUT, + .Name = "DSProxy.Put", + .Activity = NKikimrServices::TActivity::BS_PROXY_PUT_ACTOR, + }; + + TBatchedVec& Events; + bool TimeStatsEnabled; + TDiskResponsivenessTracker::TPerDiskStatsPtr Stats; + NKikimrBlobStorage::EPutHandleClass HandleClass; + TEvBlobStorage::TEvPut::ETactic Tactic; + bool EnableRequestMod3x3ForMinLatency; + TAccelerationParams AccelerationParams; + + static ui32 CalculateRestartCounter(TBatchedVec& events) { + ui32 maxRestarts = 0; + for (const auto& ev : events) { + maxRestarts = std::max(maxRestarts, ev->Get()->RestartCounter); + } + return maxRestarts; + } +}; +IActor* CreateBlobStorageGroupPutRequest(TBlobStorageGroupMultiPutParameters params); + +struct TBlobStorageGroupGetParameters { + TBlobStorageGroupRequestActor::TCommonParameters Common; + TBlobStorageGroupRequestActor::TTypeSpecificParameters TypeSpecific = { + .LogComponent = NKikimrServices::BS_PROXY_GET, + .Name = "DSProxy.Get", + .Activity = NKikimrServices::TActivity::BS_PROXY_GET_ACTOR, + }; + TNodeLayoutInfoPtr NodeLayout; + TAccelerationParams AccelerationParams; +}; +IActor* CreateBlobStorageGroupGetRequest(TBlobStorageGroupGetParameters params, NWilson::TTraceId traceId); + +struct TBlobStorageGroupPatchParameters { + TBlobStorageGroupRequestActor::TCommonParameters Common; + TBlobStorageGroupRequestActor::TTypeSpecificParameters TypeSpecific = { + .LogComponent = NKikimrServices::BS_PROXY_PATCH, + .Name = "DSProxy.Patch", + .Activity = NKikimrServices::TActivity::BS_PROXY_PATCH_ACTOR, + }; + + bool UseVPatch = false; +}; +IActor* CreateBlobStorageGroupPatchRequest(TBlobStorageGroupPatchParameters params, NWilson::TTraceId traceId); + +struct TBlobStorageGroupMultiGetParameters { + TBlobStorageGroupRequestActor::TCommonParameters Common; + TBlobStorageGroupRequestActor::TTypeSpecificParameters TypeSpecific = { + .LogComponent = NKikimrServices::BS_PROXY_MULTIGET, + .Name = "DSProxy.MultiGet", + .Activity = NKikimrServices::TActivity::BS_PROXY_MULTIGET_ACTOR, + }; + bool UseVPatch = false; +}; +IActor* CreateBlobStorageGroupMultiGetRequest(TBlobStorageGroupMultiGetParameters params, NWilson::TTraceId traceId); + +struct TBlobStorageGroupRestoreGetParameters { + TBlobStorageGroupRequestActor::TCommonParameters Common; + TBlobStorageGroupRequestActor::TTypeSpecificParameters TypeSpecific = { + .LogComponent = NKikimrServices::BS_PROXY_INDEXRESTOREGET, + .Name = "DSProxy.IndexRestoreGet", + .Activity = NKikimrServices::TActivity::BS_PROXY_INDEXRESTOREGET_ACTOR, + }; +}; +IActor* CreateBlobStorageGroupIndexRestoreGetRequest(TBlobStorageGroupRestoreGetParameters params, NWilson::TTraceId traceId); + +struct TBlobStorageGroupDiscoverParameters { + TBlobStorageGroupRequestActor::TCommonParameters Common; + TBlobStorageGroupRequestActor::TTypeSpecificParameters TypeSpecific = { + .LogComponent = NKikimrServices::BS_PROXY_DISCOVER, + .Name = "DSProxy.Discover", + .Activity = NKikimrServices::TActivity::BS_GROUP_DISCOVER, + }; +}; +IActor* CreateBlobStorageGroupDiscoverRequest(TBlobStorageGroupDiscoverParameters params, NWilson::TTraceId traceId); +IActor* CreateBlobStorageGroupMirror3dcDiscoverRequest(TBlobStorageGroupDiscoverParameters params, NWilson::TTraceId traceId); +IActor* CreateBlobStorageGroupMirror3of4DiscoverRequest(TBlobStorageGroupDiscoverParameters params, NWilson::TTraceId traceId); + +struct TBlobStorageGroupCollectGarbageParameters { + TBlobStorageGroupRequestActor::TCommonParameters Common; + TBlobStorageGroupRequestActor::TTypeSpecificParameters TypeSpecific = { + .LogComponent = NKikimrServices::BS_PROXY_COLLECT, + .Name = "DSProxy.CollectGarbage", + .Activity = NKikimrServices::TActivity::BS_GROUP_COLLECT_GARBAGE, + }; +}; +IActor* CreateBlobStorageGroupCollectGarbageRequest(TBlobStorageGroupCollectGarbageParameters params, NWilson::TTraceId traceId); + +struct TBlobStorageGroupMultiCollectParameters { + TBlobStorageGroupRequestActor::TCommonParameters Common; + TBlobStorageGroupRequestActor::TTypeSpecificParameters TypeSpecific = { + .LogComponent = NKikimrServices::BS_PROXY_MULTICOLLECT, + .Name = "DSProxy.MultiCollect", + .Activity = NKikimrServices::TActivity::BS_PROXY_MULTICOLLECT_ACTOR, + }; +}; +IActor* CreateBlobStorageGroupMultiCollectRequest(TBlobStorageGroupMultiCollectParameters params, NWilson::TTraceId traceId); + +struct TBlobStorageGroupBlockParameters { + TBlobStorageGroupRequestActor::TCommonParameters Common; + TBlobStorageGroupRequestActor::TTypeSpecificParameters TypeSpecific = { + .LogComponent = NKikimrServices::BS_PROXY_BLOCK, + .Name = "DSProxy.Block", + .Activity = NKikimrServices::TActivity::BS_GROUP_BLOCK, + }; +}; +IActor* CreateBlobStorageGroupBlockRequest(TBlobStorageGroupBlockParameters params, NWilson::TTraceId traceId); + +struct TBlobStorageGroupStatusParameters { + TBlobStorageGroupRequestActor::TCommonParameters Common; + TBlobStorageGroupRequestActor::TTypeSpecificParameters TypeSpecific = { + .LogComponent = NKikimrServices::BS_PROXY_STATUS, + .Name = "DSProxy.Status", + .Activity = NKikimrServices::TActivity::BS_PROXY_STATUS_ACTOR, + }; +}; +IActor* CreateBlobStorageGroupStatusRequest(TBlobStorageGroupStatusParameters params, NWilson::TTraceId traceId); + +struct TBlobStorageGroupAssimilateParameters { + TBlobStorageGroupRequestActor::TCommonParameters Common; + TBlobStorageGroupRequestActor + ::TTypeSpecificParameters TypeSpecific = { + .LogComponent = NKikimrServices::BS_PROXY_ASSIMILATE, + .Name = "DSProxy.Assimilate", + .Activity = NKikimrServices::TActivity::BS_GROUP_ASSIMILATE, + }; +}; +IActor* CreateBlobStorageGroupAssimilateRequest(TBlobStorageGroupAssimilateParameters params, NWilson::TTraceId traceId); IActor* CreateBlobStorageGroupEjectedProxy(ui32 groupId, TIntrusivePtr &nodeMon); +struct TBlobStorageProxyControlWrappers { + TMemorizableControlWrapper EnablePutBatching; + TMemorizableControlWrapper EnableVPatch; + +#define DEVICE_TYPE_SEPECIFIC_MEMORIZABLE_CONTROLS(prefix) \ + TMemorizableControlWrapper prefix = prefix##DefaultControl; \ + TMemorizableControlWrapper prefix##HDD = prefix##DefaultControl; \ + TMemorizableControlWrapper prefix##SSD = prefix##DefaultControl + + // Acceleration parameters + DEVICE_TYPE_SEPECIFIC_MEMORIZABLE_CONTROLS(SlowDiskThreshold); + DEVICE_TYPE_SEPECIFIC_MEMORIZABLE_CONTROLS(PredictedDelayMultiplier); + DEVICE_TYPE_SEPECIFIC_MEMORIZABLE_CONTROLS(MaxNumOfSlowDisks); + +#undef DEVICE_TYPE_SEPECIFIC_MEMORIZABLE_CONTROLS + +}; + +struct TBlobStorageProxyParameters { + bool UseActorSystemTimeInBSQueue = false; + + TBlobStorageProxyControlWrappers Controls; +}; + IActor* CreateBlobStorageGroupProxyConfigured(TIntrusivePtr&& info, bool forceWaitAllDrives, TIntrusivePtr &nodeMon, - TIntrusivePtr&& storagePoolCounters, const TControlWrapper &enablePutBatching, - const TControlWrapper &enableVPatch); + TIntrusivePtr&& storagePoolCounters, const TBlobStorageProxyParameters& params); IActor* CreateBlobStorageGroupProxyUnconfigured(ui32 groupId, TIntrusivePtr &nodeMon, - const TControlWrapper &enablePutBatching, const TControlWrapper &enableVPatch); + const TBlobStorageProxyParameters& params); }//NKikimr diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_assimilate.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_assimilate.cpp index f9000ce6cc66..07d6fa488b73 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_assimilate.cpp +++ b/ydb/core/blobstorage/dsproxy/dsproxy_assimilate.cpp @@ -264,17 +264,12 @@ class TBlobStorageGroupAssimilateRequest : public TBlobStorageGroupRequestActor< return mon->ActiveAssimilate; } - TBlobStorageGroupAssimilateRequest(const TIntrusivePtr& info, - const TIntrusivePtr& state, const TActorId& source, - const TIntrusivePtr& mon, TEvBlobStorage::TEvAssimilate *ev, ui64 cookie, - NWilson::TTraceId traceId, TInstant now, TIntrusivePtr& storagePoolCounters) - : TBlobStorageGroupRequestActor(info, state, mon, source, cookie, - NKikimrServices::BS_PROXY_ASSIMILATE, false, {}, now, storagePoolCounters, ev->RestartCounter, - NWilson::TSpan(TWilson::BlobStorage, std::move(traceId), "DSProxy.Assimilate"), std::move(ev->ExecutionRelay)) - , SkipBlocksUpTo(ev->SkipBlocksUpTo) - , SkipBarriersUpTo(ev->SkipBarriersUpTo) - , SkipBlobsUpTo(ev->SkipBlobsUpTo) - , PerVDiskInfo(info->GetTotalVDisksNum()) + TBlobStorageGroupAssimilateRequest(TBlobStorageGroupAssimilateParameters& params) + : TBlobStorageGroupRequestActor(params) + , SkipBlocksUpTo(params.Common.Event->SkipBlocksUpTo) + , SkipBarriersUpTo(params.Common.Event->SkipBarriersUpTo) + , SkipBlobsUpTo(params.Common.Event->SkipBlobsUpTo) + , PerVDiskInfo(Info->GetTotalVDisksNum()) , Result(new TEvBlobStorage::TEvAssimilateResult(NKikimrProto::OK, {})) { Heap.reserve(PerVDiskInfo.size()); @@ -464,11 +459,9 @@ class TBlobStorageGroupAssimilateRequest : public TBlobStorageGroupRequestActor< } }; -IActor* CreateBlobStorageGroupAssimilateRequest(const TIntrusivePtr& info, - const TIntrusivePtr& state, const TActorId& source, - const TIntrusivePtr& mon, TEvBlobStorage::TEvAssimilate *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, TIntrusivePtr& storagePoolCounters) { - return new TBlobStorageGroupAssimilateRequest(info, state, source, mon, ev, cookie, std::move(traceId), now, storagePoolCounters); +IActor* CreateBlobStorageGroupAssimilateRequest(TBlobStorageGroupAssimilateParameters params, NWilson::TTraceId traceId) { + params.Common.Span = NWilson::TSpan(TWilson::BlobStorage, std::move(traceId), "DSProxy.Assimilate"); + return new TBlobStorageGroupAssimilateRequest(params); } } // NKikimr diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_blackboard.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_blackboard.cpp index 397a2cbd3de3..c705c9a04e2a 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_blackboard.cpp +++ b/ydb/core/blobstorage/dsproxy/dsproxy_blackboard.cpp @@ -178,12 +178,18 @@ ui64 TBlobState::GetPredictedDelayNs(const TBlobStorageGroupInfo &info, TGroupQu } void TBlobState::GetWorstPredictedDelaysNs(const TBlobStorageGroupInfo &info, TGroupQueues &groupQueues, - NKikimrBlobStorage::EVDiskQueueId queueId, ui32 nWorst, TDiskDelayPredictions *outNWorst) const { + NKikimrBlobStorage::EVDiskQueueId queueId, TDiskDelayPredictions *outNWorst, + const TAccelerationParams& accelerationParams) const { outNWorst->resize(Disks.size()); for (ui32 diskIdx = 0; diskIdx < Disks.size(); ++diskIdx) { - (*outNWorst)[diskIdx] = { GetPredictedDelayNs(info, groupQueues, diskIdx, queueId), diskIdx }; + ui64 predictedDelayNs = GetPredictedDelayNs(info, groupQueues, diskIdx, queueId); + (*outNWorst)[diskIdx] = { + static_cast(predictedDelayNs * accelerationParams.PredictedDelayMultiplier), + diskIdx + }; } - std::partial_sort(outNWorst->begin(), outNWorst->begin() + std::min(nWorst, (ui32)Disks.size()), outNWorst->end()); + ui32 sortedPrefixSize = std::min(accelerationParams.MaxNumOfSlowDisks + 1, (ui32)Disks.size()); + std::partial_sort(outNWorst->begin(), outNWorst->begin() + sortedPrefixSize, outNWorst->end()); } bool TBlobState::HasWrittenQuorum(const TBlobStorageGroupInfo& info, const TBlobStorageGroupInfo::TGroupVDisks& expired) const { @@ -361,7 +367,8 @@ void TBlackboard::AddErrorResponse(const TLogoBlobID &id, ui32 orderNumber) { } EStrategyOutcome TBlackboard::RunStrategies(TLogContext &logCtx, const TStackVec& s, - TBatchedVec *finished, const TBlobStorageGroupInfo::TGroupVDisks *expired) { + const TAccelerationParams& accelerationParams, TBatchedVec *finished, + const TBlobStorageGroupInfo::TGroupVDisks *expired) { for (auto it = BlobStates.begin(); it != BlobStates.end(); ) { auto& blob = it->second; if (!std::exchange(blob.IsChanged, false)) { @@ -373,7 +380,7 @@ EStrategyOutcome TBlackboard::RunStrategies(TLogContext &logCtx, const TStackVec NKikimrProto::EReplyStatus status = NKikimrProto::OK; TString errorReason; for (IStrategy *strategy : s) { - switch (auto res = strategy->Process(logCtx, blob, *Info, *this, GroupDiskRequests)) { + switch (auto res = strategy->Process(logCtx, blob, *Info, *this, GroupDiskRequests, accelerationParams)) { case EStrategyOutcome::IN_PROGRESS: status = NKikimrProto::UNKNOWN; break; @@ -415,8 +422,9 @@ EStrategyOutcome TBlackboard::RunStrategies(TLogContext &logCtx, const TStackVec } EStrategyOutcome TBlackboard::RunStrategy(TLogContext &logCtx, const IStrategy& s, - TBatchedVec *finished, const TBlobStorageGroupInfo::TGroupVDisks *expired) { - return RunStrategies(logCtx, {const_cast(&s)}, finished, expired); + const TAccelerationParams& accelerationParams, TBatchedVec *finished, + const TBlobStorageGroupInfo::TGroupVDisks *expired) { + return RunStrategies(logCtx, {const_cast(&s)}, accelerationParams, finished, expired); } TBlobState& TBlackboard::GetState(const TLogoBlobID &id) { @@ -458,13 +466,19 @@ void TBlackboard::ReportPartMapStatus(const TLogoBlobID &id, ssize_t partMapInde } void TBlackboard::GetWorstPredictedDelaysNs(const TBlobStorageGroupInfo &info, TGroupQueues &groupQueues, - NKikimrBlobStorage::EVDiskQueueId queueId, ui32 nWorst, TDiskDelayPredictions *outNWorst) const { + NKikimrBlobStorage::EVDiskQueueId queueId, TDiskDelayPredictions *outNWorst, + const TAccelerationParams& accelerationParams) const { ui32 totalVDisks = info.GetTotalVDisksNum(); outNWorst->resize(totalVDisks); for (ui32 orderNumber = 0; orderNumber < totalVDisks; ++orderNumber) { - (*outNWorst)[orderNumber] = { groupQueues.GetPredictedDelayNsByOrderNumber(orderNumber, queueId), orderNumber }; + ui64 predictedDelayNs = groupQueues.GetPredictedDelayNsByOrderNumber(orderNumber, queueId); + (*outNWorst)[orderNumber] = { + static_cast(predictedDelayNs * accelerationParams.PredictedDelayMultiplier), + orderNumber + }; } - std::partial_sort(outNWorst->begin(), outNWorst->begin() + std::min(nWorst, totalVDisks), outNWorst->end()); + ui32 sortedPrefixSize = std::min(accelerationParams.MaxNumOfSlowDisks + 1, totalVDisks); + std::partial_sort(outNWorst->begin(), outNWorst->begin() + sortedPrefixSize, outNWorst->end()); } void TBlackboard::RegisterBlobForPut(const TLogoBlobID& id, size_t blobIdx) { @@ -531,4 +545,35 @@ void TBlackboard::InvalidatePartStates(ui32 orderNumber) { } } +void TBlackboard::MarkSlowDisks(TBlobState& state, bool isPut, const TAccelerationParams& accelerationParams) { + // by default all disks are considered fast + for (TBlobState::TDisk& disk : state.Disks) { + disk.IsSlow = false; + } + + ui32 maxNumSlow = accelerationParams.MaxNumOfSlowDisks; + if (Info->GetTotalVDisksNum() <= maxNumSlow) { + // all disks cannot be slow + return; + } + + TDiskDelayPredictions worstDisks; + state.GetWorstPredictedDelaysNs(*Info, *GroupQueues, + (isPut ? HandleClassToQueueId(PutHandleClass) : HandleClassToQueueId(GetHandleClass)), + &worstDisks, accelerationParams); + + ui64 slowThreshold = worstDisks[maxNumSlow].PredictedNs * accelerationParams.SlowDiskThreshold; + if (slowThreshold == 0) { + // invalid or non-initialized predicted ns, consider all disks not slow + return; + } + + for (ui32 idx = 0; idx < maxNumSlow; ++idx) { + if (worstDisks[idx].PredictedNs > slowThreshold) { + ui32 orderNumber = worstDisks[idx].DiskIdx; + state.Disks[orderNumber].IsSlow = true; + } + } +} + }//NKikimr diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_blackboard.h b/ydb/core/blobstorage/dsproxy/dsproxy_blackboard.h index 54d53302cdf5..947dad80446c 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_blackboard.h +++ b/ydb/core/blobstorage/dsproxy/dsproxy_blackboard.h @@ -97,10 +97,10 @@ struct TBlobState { ui64 GetPredictedDelayNs(const TBlobStorageGroupInfo &info, TGroupQueues &groupQueues, ui32 diskIdxInSubring, NKikimrBlobStorage::EVDiskQueueId queueId) const; void GetWorstPredictedDelaysNs(const TBlobStorageGroupInfo &info, TGroupQueues &groupQueues, - NKikimrBlobStorage::EVDiskQueueId queueId, ui32 nWorst, TDiskDelayPredictions *outNWorst) const; + NKikimrBlobStorage::EVDiskQueueId queueId, TDiskDelayPredictions *outNWorst, + const TAccelerationParams& accelerationParams) const; TString ToString() const; bool HasWrittenQuorum(const TBlobStorageGroupInfo& info, const TBlobStorageGroupInfo::TGroupVDisks& expired) const; - static TString SituationToString(ESituation situation); }; @@ -158,12 +158,13 @@ class IStrategy { public: virtual ~IStrategy() = default; virtual EStrategyOutcome Process(TLogContext &logCtx, TBlobState &state, const TBlobStorageGroupInfo &info, - TBlackboard &blackboard, TGroupDiskRequests &groupDiskRequests) = 0; + TBlackboard &blackboard, TGroupDiskRequests &groupDiskRequests, + const TAccelerationParams& accelerationParams) = 0; }; struct TBlackboard { enum EAccelerationMode { - AccelerationModeSkipOneSlowest, + AccelerationModeSkipNSlowest, AccelerationModeSkipMarked }; @@ -187,7 +188,7 @@ struct TBlackboard { NKikimrBlobStorage::EPutHandleClass putHandleClass, NKikimrBlobStorage::EGetHandleClass getHandleClass) : Info(info) , GroupQueues(groupQueues) - , AccelerationMode(AccelerationModeSkipOneSlowest) + , AccelerationMode(AccelerationModeSkipNSlowest) , PutHandleClass(putHandleClass) , GetHandleClass(getHandleClass) {} @@ -201,14 +202,16 @@ struct TBlackboard { void AddNotYetResponse(const TLogoBlobID &id, ui32 orderNumber); EStrategyOutcome RunStrategies(TLogContext& logCtx, const TStackVec& strategies, - TBatchedVec *finished = nullptr, const TBlobStorageGroupInfo::TGroupVDisks *expired = nullptr); - EStrategyOutcome RunStrategy(TLogContext &logCtx, const IStrategy& s, TBatchedVec *finished = nullptr, + const TAccelerationParams& accelerationParams, TBatchedVec *finished = nullptr, const TBlobStorageGroupInfo::TGroupVDisks *expired = nullptr); + EStrategyOutcome RunStrategy(TLogContext &logCtx, const IStrategy& s, const TAccelerationParams& accelerationParams, + TBatchedVec *finished = nullptr, const TBlobStorageGroupInfo::TGroupVDisks *expired = nullptr); TBlobState& GetState(const TLogoBlobID &id); ssize_t AddPartMap(const TLogoBlobID &id, ui32 diskOrderNumber, ui32 requestIndex); void ReportPartMapStatus(const TLogoBlobID &id, ssize_t partMapIndex, ui32 responseIndex, NKikimrProto::EReplyStatus status); void GetWorstPredictedDelaysNs(const TBlobStorageGroupInfo &info, TGroupQueues &groupQueues, - NKikimrBlobStorage::EVDiskQueueId queueId, ui32 nWorst, TDiskDelayPredictions *outNWorst) const; + NKikimrBlobStorage::EVDiskQueueId queueId, TDiskDelayPredictions *outNWorst, + const TAccelerationParams& accelerationParams) const; TString ToString() const; void ChangeAll() { @@ -221,6 +224,8 @@ struct TBlackboard { void RegisterBlobForPut(const TLogoBlobID& id, size_t blobIdx); + void MarkSlowDisks(TBlobState& state, bool isPut, const TAccelerationParams& accelerationParams); + TBlobState& operator [](const TLogoBlobID& id); }; diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_block.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_block.cpp index e3aecd3b8bbe..1a5da7b27629 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_block.cpp +++ b/ydb/core/blobstorage/dsproxy/dsproxy_block.cpp @@ -132,19 +132,13 @@ class TBlobStorageGroupBlockRequest : public TBlobStorageGroupRequestActorActiveBlock; } - TBlobStorageGroupBlockRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvBlock *ev, - ui64 cookie, NWilson::TSpan&& span, TInstant now, - TIntrusivePtr &storagePoolCounters) - : TBlobStorageGroupRequestActor(info, state, mon, source, cookie, - NKikimrServices::BS_PROXY_BLOCK, false, {}, now, storagePoolCounters, ev->RestartCounter, - std::move(span), std::move(ev->ExecutionRelay)) - , TabletId(ev->TabletId) - , Generation(ev->Generation) - , Deadline(ev->Deadline) - , IssuerGuid(ev->IssuerGuid) - , StartTime(now) + TBlobStorageGroupBlockRequest(TBlobStorageGroupBlockParameters& params) + : TBlobStorageGroupRequestActor(params) + , TabletId(params.Common.Event->TabletId) + , Generation(params.Common.Event->Generation) + , Deadline(params.Common.Event->Deadline) + , IssuerGuid(params.Common.Event->IssuerGuid) + , StartTime(params.Common.Now) , QuorumTracker(Info.Get()) {} @@ -175,16 +169,13 @@ class TBlobStorageGroupBlockRequest : public TBlobStorageGroupRequestActor &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvBlock *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, TIntrusivePtr &storagePoolCounters) { +IActor* CreateBlobStorageGroupBlockRequest(TBlobStorageGroupBlockParameters params, NWilson::TTraceId traceId) { NWilson::TSpan span(TWilson::BlobStorage, std::move(traceId), "DSProxy.Block"); if (span) { - span.Attribute("event", ev->ToString()); + span.Attribute("event", params.Common.Event->ToString()); } - - return new TBlobStorageGroupBlockRequest(info, state, source, mon, ev, cookie, std::move(span), now, storagePoolCounters); + params.Common.Span = std::move(span); + return new TBlobStorageGroupBlockRequest(params); } } // NKikimr diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_collect.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_collect.cpp index 9a091ae2e91c..ee18d2fc3e34 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_collect.cpp +++ b/ydb/core/blobstorage/dsproxy/dsproxy_collect.cpp @@ -141,27 +141,22 @@ class TBlobStorageGroupCollectGarbageRequest : public TBlobStorageGroupRequestAc return mon->ActiveCollectGarbage; } - TBlobStorageGroupCollectGarbageRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvCollectGarbage *ev, ui64 cookie, - NWilson::TTraceId traceId, TInstant now, TIntrusivePtr &storagePoolCounters) - : TBlobStorageGroupRequestActor(info, state, mon, source, cookie, - NKikimrServices::BS_PROXY_COLLECT, false, {}, now, storagePoolCounters, ev->RestartCounter, - NWilson::TSpan(TWilson::BlobStorage, std::move(traceId), "DSProxy.CollectGarbage"), std::move(ev->ExecutionRelay)) - , TabletId(ev->TabletId) - , RecordGeneration(ev->RecordGeneration) - , PerGenerationCounter(ev->PerGenerationCounter) - , Channel(ev->Channel) - , Deadline(ev->Deadline) - , Keep(ev->Keep.Release()) - , DoNotKeep(ev->DoNotKeep.Release()) - , CollectGeneration(ev->CollectGeneration) - , CollectStep(ev->CollectStep) - , Hard(ev->Hard) - , Collect(ev->Collect) - , Decommission(ev->Decommission) + TBlobStorageGroupCollectGarbageRequest(TBlobStorageGroupCollectGarbageParameters& params) + : TBlobStorageGroupRequestActor(params) + , TabletId(params.Common.Event->TabletId) + , RecordGeneration(params.Common.Event->RecordGeneration) + , PerGenerationCounter(params.Common.Event->PerGenerationCounter) + , Channel(params.Common.Event->Channel) + , Deadline(params.Common.Event->Deadline) + , Keep(params.Common.Event->Keep.Release()) + , DoNotKeep(params.Common.Event->DoNotKeep.Release()) + , CollectGeneration(params.Common.Event->CollectGeneration) + , CollectStep(params.Common.Event->CollectStep) + , Hard(params.Common.Event->Hard) + , Collect(params.Common.Event->Collect) + , Decommission(params.Common.Event->Decommission) , QuorumTracker(Info.Get()) - , StartTime(now) + , StartTime(params.Common.Now) {} void Bootstrap() { @@ -205,12 +200,9 @@ class TBlobStorageGroupCollectGarbageRequest : public TBlobStorageGroupRequestAc } }; -IActor* CreateBlobStorageGroupCollectGarbageRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvCollectGarbage *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, TIntrusivePtr &storagePoolCounters) { - return new TBlobStorageGroupCollectGarbageRequest(info, state, source, mon, ev, cookie, std::move(traceId), now, - storagePoolCounters); +IActor* CreateBlobStorageGroupCollectGarbageRequest(TBlobStorageGroupCollectGarbageParameters params, NWilson::TTraceId traceId) { + params.Common.Span = NWilson::TSpan(TWilson::BlobStorage, std::move(traceId), "DSProxy.CollectGarbage"); + return new TBlobStorageGroupCollectGarbageRequest(params); } } // NKikimr diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_discover.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_discover.cpp index 72624aba6c49..000a70105a2a 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_discover.cpp +++ b/ydb/core/blobstorage/dsproxy/dsproxy_discover.cpp @@ -877,25 +877,18 @@ class TBlobStorageGroupDiscoverRequest : public TBlobStorageGroupRequestActor &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr mon, TEvBlobStorage::TEvDiscover *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, - TIntrusivePtr &storagePoolCounters) - : TBlobStorageGroupRequestActor(info, state, mon, source, cookie, - NKikimrServices::BS_PROXY_DISCOVER, true, {}, now, storagePoolCounters, ev->RestartCounter, - NWilson::TSpan(TWilson::BlobStorage, std::move(traceId), "DSProxy.Discover"), - std::move(ev->ExecutionRelay)) - , TabletId(ev->TabletId) - , MinGeneration(ev->MinGeneration) - , ReadBody(ev->ReadBody) - , DiscoverBlockedGeneration(ev->DiscoverBlockedGeneration) - , Deadline(ev->Deadline) - , StartTime(now) + TBlobStorageGroupDiscoverRequest(TBlobStorageGroupDiscoverParameters& params) + : TBlobStorageGroupRequestActor(params) + , TabletId(params.Common.Event->TabletId) + , MinGeneration(params.Common.Event->MinGeneration) + , ReadBody(params.Common.Event->ReadBody) + , DiscoverBlockedGeneration(params.Common.Event->DiscoverBlockedGeneration) + , Deadline(params.Common.Event->Deadline) + , StartTime(params.Common.Now) , GroupResponseTracker(Info) , IsGetBlockDone(!DiscoverBlockedGeneration) - , ForceBlockedGeneration(ev->ForceBlockedGeneration) - , FromLeader(ev->FromLeader) + , ForceBlockedGeneration(params.Common.Event->ForceBlockedGeneration) + , FromLeader(params.Common.Event->FromLeader) {} void Bootstrap() { @@ -974,13 +967,9 @@ class TBlobStorageGroupDiscoverRequest : public TBlobStorageGroupRequestActor &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvDiscover *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, - TIntrusivePtr &storagePoolCounters) { - return new TBlobStorageGroupDiscoverRequest(info, state, source, mon, ev, cookie, std::move(traceId), now, - storagePoolCounters); +IActor* CreateBlobStorageGroupDiscoverRequest(TBlobStorageGroupDiscoverParameters params, NWilson::TTraceId traceId) { + params.Common.Span = NWilson::TSpan(TWilson::BlobStorage, std::move(traceId), "DSProxy.Discover"); + return new TBlobStorageGroupDiscoverRequest(params); } }//NKikimr diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_discover_m3dc.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_discover_m3dc.cpp index aaf5b8224e7b..1b10ba950c09 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_discover_m3dc.cpp +++ b/ydb/core/blobstorage/dsproxy/dsproxy_discover_m3dc.cpp @@ -457,23 +457,16 @@ class TBlobStorageGroupMirror3dcDiscoverRequest : public TBlobStorageGroupReques return ERequestType::Discover; } - TBlobStorageGroupMirror3dcDiscoverRequest(TIntrusivePtr info, - TIntrusivePtr state, const TActorId& source, - TIntrusivePtr mon, TEvBlobStorage::TEvDiscover *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, - TIntrusivePtr &storagePoolCounters) - : TBlobStorageGroupRequestActor(std::move(info), std::move(state), std::move(mon), source, cookie, - NKikimrServices::BS_PROXY_DISCOVER, false, {}, now, storagePoolCounters, ev->RestartCounter, - NWilson::TSpan(TWilson::BlobStorage, std::move(traceId), "DSProxy.Discover(mirror-3-dc)"), - std::move(ev->ExecutionRelay)) - , TabletId(ev->TabletId) - , MinGeneration(ev->MinGeneration) - , StartTime(now) - , Deadline(ev->Deadline) - , ReadBody(ev->ReadBody) - , DiscoverBlockedGeneration(ev->DiscoverBlockedGeneration) - , ForceBlockedGeneration(ev->ForceBlockedGeneration) - , FromLeader(ev->FromLeader) + TBlobStorageGroupMirror3dcDiscoverRequest(TBlobStorageGroupDiscoverParameters& params) + : TBlobStorageGroupRequestActor(params) + , TabletId(params.Common.Event->TabletId) + , MinGeneration(params.Common.Event->MinGeneration) + , StartTime(params.Common.Now) + , Deadline(params.Common.Event->Deadline) + , ReadBody(params.Common.Event->ReadBody) + , DiscoverBlockedGeneration(params.Common.Event->DiscoverBlockedGeneration) + , ForceBlockedGeneration(params.Common.Event->ForceBlockedGeneration) + , FromLeader(params.Common.Event->FromLeader) , GetBlockTracker(Info.Get()) {} @@ -738,13 +731,9 @@ class TBlobStorageGroupMirror3dcDiscoverRequest : public TBlobStorageGroupReques } }; -IActor* CreateBlobStorageGroupMirror3dcDiscoverRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvDiscover *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, - TIntrusivePtr &storagePoolCounters) { - return new TBlobStorageGroupMirror3dcDiscoverRequest(info, state, source, mon, ev, cookie, std::move(traceId), now, - storagePoolCounters); +IActor* CreateBlobStorageGroupMirror3dcDiscoverRequest(TBlobStorageGroupDiscoverParameters params, NWilson::TTraceId traceId) { + params.Common.Span = NWilson::TSpan(TWilson::BlobStorage, std::move(traceId), "DSProxy.Discover(mirror-3-dc)"); + return new TBlobStorageGroupMirror3dcDiscoverRequest(params); } }//NKikimr diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_discover_m3of4.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_discover_m3of4.cpp index 4f31a1d83311..c6432e9f9728 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_discover_m3of4.cpp +++ b/ydb/core/blobstorage/dsproxy/dsproxy_discover_m3of4.cpp @@ -29,23 +29,16 @@ class TBlobStorageGroupMirror3of4DiscoverRequest return ERequestType::Discover; } - TBlobStorageGroupMirror3of4DiscoverRequest(TIntrusivePtr info, - TIntrusivePtr state, const TActorId& source, - TIntrusivePtr mon, TEvBlobStorage::TEvDiscover *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, - TIntrusivePtr &storagePoolCounters) - : TBlobStorageGroupRequestActor(std::move(info), std::move(state), std::move(mon), source, cookie, - NKikimrServices::BS_PROXY_DISCOVER, false, {}, now, storagePoolCounters, ev->RestartCounter, - NWilson::TSpan(TWilson::BlobStorage, std::move(traceId), "DSProxy.Discover(mirror-3of4)"), - std::move(ev->ExecutionRelay)) - , TabletId(ev->TabletId) - , MinGeneration(ev->MinGeneration) - , StartTime(now) - , Deadline(ev->Deadline) - , ReadBody(ev->ReadBody) - , DiscoverBlockedGeneration(ev->DiscoverBlockedGeneration) - , ForceBlockedGeneration(ev->ForceBlockedGeneration) - , FromLeader(ev->FromLeader) + TBlobStorageGroupMirror3of4DiscoverRequest(TBlobStorageGroupDiscoverParameters& params) + : TBlobStorageGroupRequestActor(params) + , TabletId(params.Common.Event->TabletId) + , MinGeneration(params.Common.Event->MinGeneration) + , StartTime(params.Common.Now) + , Deadline(params.Common.Event->Deadline) + , ReadBody(params.Common.Event->ReadBody) + , DiscoverBlockedGeneration(params.Common.Event->DiscoverBlockedGeneration) + , ForceBlockedGeneration(params.Common.Event->ForceBlockedGeneration) + , FromLeader(params.Common.Event->FromLeader) { for (size_t i = 0; i < DiskState.size(); ++i) { TDiskState& disk = DiskState[i]; @@ -361,13 +354,9 @@ class TBlobStorageGroupMirror3of4DiscoverRequest } }; -IActor* CreateBlobStorageGroupMirror3of4DiscoverRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvDiscover *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, - TIntrusivePtr &storagePoolCounters) { - return new TBlobStorageGroupMirror3of4DiscoverRequest(info, state, source, mon, ev, cookie, std::move(traceId), now, - storagePoolCounters); +IActor* CreateBlobStorageGroupMirror3of4DiscoverRequest(TBlobStorageGroupDiscoverParameters params, NWilson::TTraceId traceId) { + params.Common.Span = NWilson::TSpan(TWilson::BlobStorage, std::move(traceId), "DSProxy.Discover(mirror-3of4)"); + return new TBlobStorageGroupMirror3of4DiscoverRequest(params); } }//NKikimr diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_get.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_get.cpp index 5f09fdee38a0..eb3a0a7bb482 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_get.cpp +++ b/ydb/core/blobstorage/dsproxy/dsproxy_get.cpp @@ -57,6 +57,8 @@ class TBlobStorageGroupGetRequest : public TBlobStorageGroupRequestActorGet()->CauseIdx); @@ -312,16 +314,16 @@ class TBlobStorageGroupGetRequest : public TBlobStorageGroupRequestActor StartTime) ? (now - StartTime) : TDuration::MilliSeconds(0); - if (timeSinceStart.MicroSeconds() < timeToAccelerateUs) { + TInstant nextAcceleration = StartTime + timeToAccelerate; + if (nextAcceleration > now) { ui64 causeIdx = RootCauseTrack.RegisterAccelerate(); - Schedule(TDuration::MicroSeconds(timeToAccelerateUs - timeSinceStart.MicroSeconds()), - new TEvAccelerateGet(causeIdx)); + Schedule(nextAcceleration - now, new TEvAccelerateGet(causeIdx)); IsGetAccelerateScheduled = true; } else { AccelerateGet(); @@ -331,16 +333,16 @@ class TBlobStorageGroupGetRequest : public TBlobStorageGroupRequestActor StartTimePut) ? (now - StartTimePut) : TDuration::MilliSeconds(0); - if (timeSinceStart.MicroSeconds() < timeToAccelerateUs) { + TInstant nextAcceleration = StartTime + timeToAccelerate; + if (nextAcceleration > now) { ui64 causeIdx = RootCauseTrack.RegisterAccelerate(); - Schedule(TDuration::MicroSeconds(timeToAccelerateUs - timeSinceStart.MicroSeconds()), - new TEvAcceleratePut(causeIdx)); + Schedule(nextAcceleration - now, new TEvAcceleratePut(causeIdx)); IsPutAccelerateScheduled = true; } else { AcceleratePut(); @@ -384,36 +386,33 @@ class TBlobStorageGroupGetRequest : public TBlobStorageGroupRequestActor& mon) { + return mon->ActiveGet; + } + static constexpr ERequestType RequestType() { return ERequestType::Get; } - static const auto& ActiveCounter(const TIntrusivePtr& mon) { - return mon->ActiveGet; - } - TBlobStorageGroupGetRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvGet *ev, ui64 cookie, - NWilson::TSpan&& span, TNodeLayoutInfoPtr&& nodeLayout, TMaybe latencyQueueKind, - TInstant now, TIntrusivePtr &storagePoolCounters) - : TBlobStorageGroupRequestActor(info, state, mon, source, cookie, - NKikimrServices::BS_PROXY_GET, ev->IsVerboseNoDataEnabled || ev->CollectDebugInfo, - latencyQueueKind, now, storagePoolCounters, ev->RestartCounter, std::move(span), - std::move(ev->ExecutionRelay)) - , GetImpl(info, state, ev, std::move(nodeLayout), LogCtx.RequestPrefix) - , Orbit(std::move(ev->Orbit)) - , Deadline(ev->Deadline) - , StartTime(now) + TBlobStorageGroupGetRequest(TBlobStorageGroupGetParameters& params) + : TBlobStorageGroupRequestActor(params) + , GetImpl(Info, GroupQueues, params.Common.Event, std::move(params.NodeLayout), + params.AccelerationParams, LogCtx.RequestPrefix) + , Orbit(std::move(params.Common.Event->Orbit)) + , Deadline(params.Common.Event->Deadline) + , StartTime(params.Common.Now) , StartTimePut(StartTime) - , GroupSize(info->Type.BlobSubgroupSize()) + , GroupSize(Info->Type.BlobSubgroupSize()) , ReportedBytes(0) + , AccelerationParams(params.AccelerationParams) { ReportBytes(sizeof(*this)); - MaxSaneRequests = ev->QuerySize * info->Type.TotalPartCount() * (1 + info->Type.Handoff()) * 3; + MaxSaneRequests = params.Common.Event->QuerySize * Info->Type.TotalPartCount() * + (1 + Info->Type.Handoff()) * 3; RequestBytes = GetImpl.CountRequestBytes(); - RequestHandleClass = HandleClassToHandleClass(ev->GetHandleClass); + RequestHandleClass = HandleClassToHandleClass(params.Common.Event->GetHandleClass); if (Orbit.HasShuttles()) { RootCauseTrack.IsOn = true; } @@ -466,19 +465,13 @@ class TBlobStorageGroupGetRequest : public TBlobStorageGroupRequestActor &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvGet *ev, - ui64 cookie, NWilson::TTraceId traceId, TNodeLayoutInfoPtr&& nodeLayout, - TMaybe latencyQueueKind, TInstant now, - TIntrusivePtr &storagePoolCounters) { +IActor* CreateBlobStorageGroupGetRequest(TBlobStorageGroupGetParameters params, NWilson::TTraceId traceId) { NWilson::TSpan span(TWilson::BlobStorage, std::move(traceId), "DSProxy.Get"); if (span) { - span.Attribute("event", ev->ToString()); + span.Attribute("event", params.Common.Event->ToString()); } - - return new TBlobStorageGroupGetRequest(info, state, source, mon, ev, cookie, std::move(span), - std::move(nodeLayout), latencyQueueKind, now, storagePoolCounters); + params.Common.Span = std::move(span); + return new TBlobStorageGroupGetRequest(params); } }//NKikimr diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_get_impl.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_get_impl.cpp index 18c63e66de73..9b10ead05f97 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_get_impl.cpp +++ b/ydb/core/blobstorage/dsproxy/dsproxy_get_impl.cpp @@ -159,29 +159,26 @@ void TGetImpl::PrepareReply(NKikimrProto::EReplyStatus status, TString errorReas } -ui64 TGetImpl::GetTimeToAccelerateNs(TLogContext &logCtx, NKikimrBlobStorage::EVDiskQueueId queueId, ui32 nthWorst) { +ui64 TGetImpl::GetTimeToAccelerateNs(TLogContext &logCtx, NKikimrBlobStorage::EVDiskQueueId queueId) { Y_UNUSED(logCtx); // Find the slowest disk TDiskDelayPredictions worstDisks; if (Blackboard.BlobStates.size() == 1) { - Blackboard.BlobStates.begin()->second.GetWorstPredictedDelaysNs( - *Info, *Blackboard.GroupQueues, queueId, nthWorst, &worstDisks); + Blackboard.BlobStates.begin()->second.GetWorstPredictedDelaysNs(*Info, *Blackboard.GroupQueues, + queueId, &worstDisks, AccelerationParams); } else { - Blackboard.GetWorstPredictedDelaysNs( - *Info, *Blackboard.GroupQueues, queueId, nthWorst, &worstDisks); + Blackboard.GetWorstPredictedDelaysNs(*Info, *Blackboard.GroupQueues, queueId, &worstDisks, + AccelerationParams); } - nthWorst = std::min(nthWorst, (ui32)worstDisks.size() - 1); - return worstDisks[nthWorst].PredictedNs; + return worstDisks[std::min(AccelerationParams.MaxNumOfSlowDisks, (ui32)worstDisks.size() - 1)].PredictedNs; } -ui64 TGetImpl::GetTimeToAccelerateGetNs(TLogContext &logCtx, ui32 acceleratesSent) { - Y_DEBUG_ABORT_UNLESS(acceleratesSent < 2); - return GetTimeToAccelerateNs(logCtx, HandleClassToQueueId(Blackboard.GetHandleClass), 2 - acceleratesSent); +ui64 TGetImpl::GetTimeToAccelerateGetNs(TLogContext &logCtx) { + return GetTimeToAccelerateNs(logCtx, HandleClassToQueueId(Blackboard.GetHandleClass)); } -ui64 TGetImpl::GetTimeToAcceleratePutNs(TLogContext &logCtx, ui32 acceleratesSent) { - Y_DEBUG_ABORT_UNLESS(acceleratesSent < 2); - return GetTimeToAccelerateNs(logCtx, HandleClassToQueueId(Blackboard.PutHandleClass), 2 - acceleratesSent); +ui64 TGetImpl::GetTimeToAcceleratePutNs(TLogContext &logCtx) { + return GetTimeToAccelerateNs(logCtx, HandleClassToQueueId(Blackboard.PutHandleClass)); } TString TGetImpl::DumpFullState() const { @@ -328,13 +325,13 @@ EStrategyOutcome TGetImpl::RunBoldStrategy(TLogContext &logCtx) { if (MustRestoreFirst) { strategies.push_back(&s2); } - return Blackboard.RunStrategies(logCtx, strategies); + return Blackboard.RunStrategies(logCtx, strategies, AccelerationParams); } EStrategyOutcome TGetImpl::RunMirror3dcStrategy(TLogContext &logCtx) { return MustRestoreFirst - ? Blackboard.RunStrategy(logCtx, TMirror3dcGetWithRestoreStrategy()) - : Blackboard.RunStrategy(logCtx, TMirror3dcBasicGetStrategy(NodeLayout, PhantomCheck)); + ? Blackboard.RunStrategy(logCtx, TMirror3dcGetWithRestoreStrategy(), AccelerationParams) + : Blackboard.RunStrategy(logCtx, TMirror3dcBasicGetStrategy(NodeLayout, PhantomCheck), AccelerationParams); } EStrategyOutcome TGetImpl::RunMirror3of4Strategy(TLogContext &logCtx) { @@ -345,7 +342,7 @@ EStrategyOutcome TGetImpl::RunMirror3of4Strategy(TLogContext &logCtx) { if (MustRestoreFirst) { strategies.push_back(&s2); } - return Blackboard.RunStrategies(logCtx, strategies); + return Blackboard.RunStrategies(logCtx, strategies, AccelerationParams); } EStrategyOutcome TGetImpl::RunStrategies(TLogContext &logCtx) { @@ -356,9 +353,9 @@ EStrategyOutcome TGetImpl::RunStrategies(TLogContext &logCtx) { } else if (MustRestoreFirst || PhantomCheck) { return RunBoldStrategy(logCtx); } else if (Info->Type.ErasureFamily() == TErasureType::ErasureParityBlock) { - return Blackboard.RunStrategy(logCtx, TMinIopsBlockStrategy()); + return Blackboard.RunStrategy(logCtx, TMinIopsBlockStrategy(), AccelerationParams); } else if (Info->Type.ErasureFamily() == TErasureType::ErasureMirror) { - return Blackboard.RunStrategy(logCtx, TMinIopsMirrorStrategy()); + return Blackboard.RunStrategy(logCtx, TMinIopsMirrorStrategy(), AccelerationParams); } else { return RunBoldStrategy(logCtx); } diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_get_impl.h b/ydb/core/blobstorage/dsproxy/dsproxy_get_impl.h index 2b7623c63f93..53d7433651b1 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_get_impl.h +++ b/ydb/core/blobstorage/dsproxy/dsproxy_get_impl.h @@ -49,9 +49,12 @@ class TGetImpl { std::unordered_map> BlobFlags; // keep, doNotKeep per blob + TAccelerationParams AccelerationParams; + public: TGetImpl(const TIntrusivePtr &info, const TIntrusivePtr &groupQueues, - TEvBlobStorage::TEvGet *ev, TNodeLayoutInfoPtr&& nodeLayout, const TString& requestPrefix = {}) + TEvBlobStorage::TEvGet *ev, TNodeLayoutInfoPtr&& nodeLayout, + const TAccelerationParams& accelerationParams, const TString& requestPrefix = {}) : Deadline(ev->Deadline) , Info(info) , Queries(ev->Queries.Release()) @@ -68,6 +71,7 @@ class TGetImpl { , PhantomCheck(ev->PhantomCheck) , Decommission(ev->Decommission) , ReaderTabletData(ev->ReaderTabletData) + , AccelerationParams(accelerationParams) { Y_ABORT_UNLESS(QuerySize > 0); } @@ -275,8 +279,8 @@ class TGetImpl { AccelerateGet(logCtx, slowDisksMask, outVGets, outVPuts); } - ui64 GetTimeToAccelerateGetNs(TLogContext &logCtx, ui32 acceleratesSent); - ui64 GetTimeToAcceleratePutNs(TLogContext &logCtx, ui32 acceleratesSent); + ui64 GetTimeToAccelerateGetNs(TLogContext &logCtx); + ui64 GetTimeToAcceleratePutNs(TLogContext &logCtx); TString DumpFullState() const; @@ -313,7 +317,7 @@ class TGetImpl { void PrepareVPuts(TLogContext &logCtx, TDeque> &outVPuts); - ui64 GetTimeToAccelerateNs(TLogContext &logCtx, NKikimrBlobStorage::EVDiskQueueId queueId, ui32 nthWorst); + ui64 GetTimeToAccelerateNs(TLogContext &logCtx, NKikimrBlobStorage::EVDiskQueueId queueId); }; //TGetImpl }//NKikimr diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_impl.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_impl.cpp index 0dbf35619f6b..9975916fdcdb 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_impl.cpp +++ b/ydb/core/blobstorage/dsproxy/dsproxy_impl.cpp @@ -5,8 +5,8 @@ namespace NKikimr { std::atomic TBlobStorageGroupProxy::ThrottlingTimestamp; TBlobStorageGroupProxy::TBlobStorageGroupProxy(TIntrusivePtr&& info, bool forceWaitAllDrives, - TIntrusivePtr &nodeMon, TIntrusivePtr&& storagePoolCounters, - const TControlWrapper &enablePutBatching, const TControlWrapper &enableVPatch) + TIntrusivePtr &nodeMon, TIntrusivePtr&& storagePoolCounters, + const TBlobStorageProxyParameters& params) : GroupId(info->GroupID) , Info(std::move(info)) , Topology(Info->PickTopology()) @@ -14,36 +14,42 @@ namespace NKikimr { , StoragePoolCounters(std::move(storagePoolCounters)) , IsEjected(false) , ForceWaitAllDrives(forceWaitAllDrives) - , EnablePutBatching(enablePutBatching) - , EnableVPatch(enableVPatch) + , UseActorSystemTimeInBSQueue(params.UseActorSystemTimeInBSQueue) + , Controls(std::move(params.Controls)) {} - TBlobStorageGroupProxy::TBlobStorageGroupProxy(ui32 groupId, bool isEjected, TIntrusivePtr &nodeMon, - const TControlWrapper &enablePutBatching, const TControlWrapper &enableVPatch) + TBlobStorageGroupProxy::TBlobStorageGroupProxy(ui32 groupId, bool isEjected,TIntrusivePtr &nodeMon, + const TBlobStorageProxyParameters& params) : GroupId(TGroupId::FromValue(groupId)) , NodeMon(nodeMon) , IsEjected(isEjected) , ForceWaitAllDrives(false) - , EnablePutBatching(enablePutBatching) - , EnableVPatch(enableVPatch) + , UseActorSystemTimeInBSQueue(params.UseActorSystemTimeInBSQueue) + , Controls(std::move(params.Controls)) {} IActor* CreateBlobStorageGroupEjectedProxy(ui32 groupId, TIntrusivePtr &nodeMon) { - return new TBlobStorageGroupProxy(groupId, true, nodeMon, TControlWrapper(false, false, true), - TControlWrapper(false, false, true)); + return new TBlobStorageGroupProxy(groupId, true, nodeMon, + TBlobStorageProxyParameters{ + .Controls = TBlobStorageProxyControlWrappers{ + .EnablePutBatching = TControlWrapper(false, false, true), + .EnableVPatch = TControlWrapper(false, false, true), + } + } + ); } IActor* CreateBlobStorageGroupProxyConfigured(TIntrusivePtr&& info, bool forceWaitAllDrives, TIntrusivePtr &nodeMon, TIntrusivePtr&& storagePoolCounters, - const TControlWrapper &enablePutBatching, const TControlWrapper &enableVPatch) { + const TBlobStorageProxyParameters& params) { Y_ABORT_UNLESS(info); - return new TBlobStorageGroupProxy(std::move(info), forceWaitAllDrives, nodeMon, std::move(storagePoolCounters), - enablePutBatching, enableVPatch); + return new TBlobStorageGroupProxy(std::move(info), forceWaitAllDrives, nodeMon, + std::move(storagePoolCounters), params); } IActor* CreateBlobStorageGroupProxyUnconfigured(ui32 groupId, TIntrusivePtr &nodeMon, - const TControlWrapper &enablePutBatching, const TControlWrapper &enableVPatch) { - return new TBlobStorageGroupProxy(groupId, false, nodeMon, enablePutBatching, enableVPatch); + const TBlobStorageProxyParameters& params) { + return new TBlobStorageGroupProxy(groupId, false, nodeMon, params); } NActors::NLog::EPriority PriorityForStatusOutbound(NKikimrProto::EReplyStatus status) { diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_impl.h b/ydb/core/blobstorage/dsproxy/dsproxy_impl.h index dd61054fffae..e1b8e10f6229 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_impl.h +++ b/ydb/core/blobstorage/dsproxy/dsproxy_impl.h @@ -66,6 +66,7 @@ class TBlobStorageGroupProxy : public TActorBootstrapped ui64 UnconfiguredBufferSize = 0; const bool IsEjected; bool ForceWaitAllDrives; + bool UseActorSystemTimeInBSQueue; bool IsLimitedKeyless = false; bool IsFullMonitoring = false; // current state of monitoring ui32 MinREALHugeBlobInBytes = 0; @@ -105,9 +106,6 @@ class TBlobStorageGroupProxy : public TActorBootstrapped TBatchedQueue BatchedGets[GetHandleClassCount]; TStackVec GetBatchedBucketQueue; - TMemorizableControlWrapper EnablePutBatching; - TMemorizableControlWrapper EnableVPatch; - TInstant EstablishingSessionStartTime; const TDuration MuteDuration = TDuration::Seconds(5); @@ -118,6 +116,10 @@ class TBlobStorageGroupProxy : public TActorBootstrapped bool HasInvalidGroupId() const { return GroupId.GetRawId() == Max(); } void ProcessInitQueue(); + TBlobStorageProxyControlWrappers Controls; + + TAccelerationParams GetAccelerationParams(); + //////////////////////////////////////////////////////////////////////////////////////////////////////////////////// // Enable monitoring @@ -264,6 +266,7 @@ class TBlobStorageGroupProxy : public TActorBootstrapped void HandleNormal(TEvBlobStorage::TEvAssimilate::TPtr &ev); void Handle(TEvBlobStorage::TEvBunchOfEvents::TPtr ev); void Handle(TEvDeathNote::TPtr ev); + void Handle(TEvGetQueuesInfo::TPtr ev); //////////////////////////////////////////////////////////////////////////////////////////////////////////////////// // Error state @@ -313,10 +316,10 @@ class TBlobStorageGroupProxy : public TActorBootstrapped TBlobStorageGroupProxy(TIntrusivePtr&& info, bool forceWaitAllDrives, TIntrusivePtr &nodeMon, TIntrusivePtr&& storagePoolCounters, - const TControlWrapper &enablePutBatching, const TControlWrapper &enableVPatch); + const TBlobStorageProxyParameters& params); TBlobStorageGroupProxy(ui32 groupId, bool isEjected, TIntrusivePtr &nodeMon, - const TControlWrapper &enablePutBatching, const TControlWrapper &enableVPatch); + const TBlobStorageProxyParameters& params); void Bootstrap(); @@ -360,6 +363,7 @@ class TBlobStorageGroupProxy : public TActorBootstrapped IgnoreFunc(TEvEstablishingSessionTimeout); fFunc(Ev5min, Handle5min); cFunc(EvCheckDeadlines, CheckDeadlines); + hFunc(TEvGetQueuesInfo, Handle); ) #define HANDLE_EVENTS(HANDLER) \ diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_indexrestoreget.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_indexrestoreget.cpp index c86ae72db269..89e36c3c814f 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_indexrestoreget.cpp +++ b/ydb/core/blobstorage/dsproxy/dsproxy_indexrestoreget.cpp @@ -266,23 +266,17 @@ class TBlobStorageGroupIndexRestoreGetRequest return ERequestType::Get; } - TBlobStorageGroupIndexRestoreGetRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvGet *ev, ui64 cookie, - NWilson::TSpan&& span, TMaybe latencyQueueKind, TInstant now, - TIntrusivePtr &storagePoolCounters) - : TBlobStorageGroupRequestActor(info, state, mon, source, cookie, - NKikimrServices::BS_PROXY_INDEXRESTOREGET, false, latencyQueueKind, now, storagePoolCounters, - ev->RestartCounter, std::move(span), std::move(ev->ExecutionRelay)) - , QuerySize(ev->QuerySize) - , Queries(ev->Queries.Release()) - , Deadline(ev->Deadline) - , IsInternal(ev->IsInternal) - , Decommission(ev->Decommission) - , ForceBlockTabletData(ev->ForceBlockTabletData) + TBlobStorageGroupIndexRestoreGetRequest(TBlobStorageGroupRestoreGetParameters& params) + : TBlobStorageGroupRequestActor(params) + , QuerySize(params.Common.Event->QuerySize) + , Queries(params.Common.Event->Queries.Release()) + , Deadline(params.Common.Event->Deadline) + , IsInternal(params.Common.Event->IsInternal) + , Decommission(params.Common.Event->Decommission) + , ForceBlockTabletData(params.Common.Event->ForceBlockTabletData) , VGetsInFlight(0) - , StartTime(now) - , GetHandleClass(ev->GetHandleClass) + , StartTime(params.Common.Now) + , GetHandleClass(params.Common.Event->GetHandleClass) , RestoreQueriesStarted(0) , RestoreQueriesFinished(0) { @@ -298,7 +292,7 @@ class TBlobStorageGroupIndexRestoreGetRequest } // phantom checks are for non-index queries only - Y_ABORT_UNLESS(!ev->PhantomCheck); + Y_ABORT_UNLESS(!params.Common.Event->PhantomCheck); } void Bootstrap() { @@ -394,18 +388,13 @@ class TBlobStorageGroupIndexRestoreGetRequest } }; -IActor* CreateBlobStorageGroupIndexRestoreGetRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvGet *ev, - ui64 cookie, NWilson::TTraceId traceId, TMaybe latencyQueueKind, TInstant now, - TIntrusivePtr &storagePoolCounters) { +IActor* CreateBlobStorageGroupIndexRestoreGetRequest(TBlobStorageGroupRestoreGetParameters params, NWilson::TTraceId traceId) { NWilson::TSpan span(TWilson::BlobStorage, std::move(traceId), "DSProxy.IndexRestoreGet"); if (span) { - span.Attribute("event", ev->ToString()); + span.Attribute("event", params.Common.Event->ToString()); } - - return new TBlobStorageGroupIndexRestoreGetRequest(info, state, source, mon, ev, cookie, std::move(span), - latencyQueueKind, now, storagePoolCounters); + params.Common.Span = std::move(span); + return new TBlobStorageGroupIndexRestoreGetRequest(params); } }//NKikimr diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_multicollect.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_multicollect.cpp index 5440f44b30ba..dc10e454030e 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_multicollect.cpp +++ b/ydb/core/blobstorage/dsproxy/dsproxy_multicollect.cpp @@ -93,29 +93,24 @@ class TBlobStorageGroupMultiCollectRequest return mon->ActiveMultiCollect; } - TBlobStorageGroupMultiCollectRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvCollectGarbage *ev, ui64 cookie, - NWilson::TTraceId traceId, TInstant now, TIntrusivePtr &storagePoolCounters) - : TBlobStorageGroupRequestActor(info, state, mon, source, cookie, - NKikimrServices::BS_PROXY_MULTICOLLECT, false, {}, now, storagePoolCounters, 0, - NWilson::TSpan(TWilson::BlobStorage, std::move(traceId), "DSProxy.MultiCollect"), std::move(ev->ExecutionRelay)) - , Iterations(ev->PerGenerationCounterStepSize()) - , TabletId(ev->TabletId) - , RecordGeneration(ev->RecordGeneration) - , PerGenerationCounter(ev->PerGenerationCounter) - , Channel(ev->Channel) - , Keep(ev->Keep.Release()) - , DoNotKeep(ev->DoNotKeep.Release()) - , Deadline(ev->Deadline) - , CollectGeneration(ev->CollectGeneration) - , CollectStep(ev->CollectStep) - , Hard(ev->Hard) - , Collect(ev->Collect) - , Decommission(ev->Decommission) + TBlobStorageGroupMultiCollectRequest(TBlobStorageGroupMultiCollectParameters& params) + : TBlobStorageGroupRequestActor(params) + , Iterations(params.Common.Event->PerGenerationCounterStepSize()) + , TabletId(params.Common.Event->TabletId) + , RecordGeneration(params.Common.Event->RecordGeneration) + , PerGenerationCounter(params.Common.Event->PerGenerationCounter) + , Channel(params.Common.Event->Channel) + , Keep(params.Common.Event->Keep.Release()) + , DoNotKeep(params.Common.Event->DoNotKeep.Release()) + , Deadline(params.Common.Event->Deadline) + , CollectGeneration(params.Common.Event->CollectGeneration) + , CollectStep(params.Common.Event->CollectStep) + , Hard(params.Common.Event->Hard) + , Collect(params.Common.Event->Collect) + , Decommission(params.Common.Event->Decommission) , FlagRequestsInFlight(0) , CollectRequestsInFlight(0) - , StartTime(now) + , StartTime(params.Common.Now) { Y_ABORT_UNLESS(Iterations > 1); } @@ -214,12 +209,9 @@ class TBlobStorageGroupMultiCollectRequest } }; -IActor* CreateBlobStorageGroupMultiCollectRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvCollectGarbage *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, TIntrusivePtr &storagePoolCounters) { - return new TBlobStorageGroupMultiCollectRequest(info, state, source, mon, ev, cookie, std::move(traceId), now, - storagePoolCounters); +IActor* CreateBlobStorageGroupMultiCollectRequest(TBlobStorageGroupMultiCollectParameters params, NWilson::TTraceId traceId) { + params.Common.Span = NWilson::TSpan(TWilson::BlobStorage, std::move(traceId), "DSProxy.MultiCollect"); + return new TBlobStorageGroupMultiCollectRequest(params); } }//NKikimr diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_multiget.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_multiget.cpp index 52483b486312..cc882b41d36c 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_multiget.cpp +++ b/ydb/core/blobstorage/dsproxy/dsproxy_multiget.cpp @@ -95,25 +95,19 @@ class TBlobStorageGroupMultiGetRequest : public TBlobStorageGroupRequestActorActiveMultiGet; } - TBlobStorageGroupMultiGetRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvGet *ev, ui64 cookie, - NWilson::TSpan&& span, TMaybe latencyQueueKind, TInstant now, - TIntrusivePtr &storagePoolCounters) - : TBlobStorageGroupRequestActor(info, state, mon, source, cookie, - NKikimrServices::BS_PROXY_MULTIGET, false, latencyQueueKind, now, storagePoolCounters, 0, - std::move(span), std::move(ev->ExecutionRelay)) - , QuerySize(ev->QuerySize) - , Queries(ev->Queries.Release()) - , Deadline(ev->Deadline) - , IsInternal(ev->IsInternal) - , PhantomCheck(ev->PhantomCheck) - , Decommission(ev->Decommission) + TBlobStorageGroupMultiGetRequest(TBlobStorageGroupMultiGetParameters& params) + : TBlobStorageGroupRequestActor(params) + , QuerySize(params.Common.Event->QuerySize) + , Queries(params.Common.Event->Queries.Release()) + , Deadline(params.Common.Event->Deadline) + , IsInternal(params.Common.Event->IsInternal) + , PhantomCheck(params.Common.Event->PhantomCheck) + , Decommission(params.Common.Event->Decommission) , Responses(new TEvBlobStorage::TEvGetResult::TResponse[QuerySize]) - , StartTime(now) - , MustRestoreFirst(ev->MustRestoreFirst) - , GetHandleClass(ev->GetHandleClass) - , ForceBlockTabletData(ev->ForceBlockTabletData) + , StartTime(params.Common.Now) + , MustRestoreFirst(params.Common.Event->MustRestoreFirst) + , GetHandleClass(params.Common.Event->GetHandleClass) + , ForceBlockTabletData(params.Common.Event->ForceBlockTabletData) {} void PrepareRequest(ui32 beginIdx, ui32 endIdx) { @@ -208,18 +202,9 @@ class TBlobStorageGroupMultiGetRequest : public TBlobStorageGroupRequestActor &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvGet *ev, - ui64 cookie, NWilson::TTraceId traceId, TMaybe latencyQueueKind, - TInstant now, TIntrusivePtr &storagePoolCounters) { - NWilson::TSpan span(TWilson::BlobStorage, std::move(traceId), "DSProxy.MultiGet"); - if (span) { - span.Attribute("event", ev->ToString()); - } - - return new TBlobStorageGroupMultiGetRequest(info, state, source, mon, ev, cookie, std::move(span), - latencyQueueKind, now, storagePoolCounters); +IActor* CreateBlobStorageGroupMultiGetRequest(TBlobStorageGroupMultiGetParameters params, NWilson::TTraceId traceId) { + params.Common.Span = NWilson::TSpan(TWilson::BlobStorage, std::move(traceId), "DSProxy.MultiGet"); + return new TBlobStorageGroupMultiGetRequest(params); } }//NKikimr diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_patch.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_patch.cpp index fbb545409a89..642ec68d1475 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_patch.cpp +++ b/ydb/core/blobstorage/dsproxy/dsproxy_patch.cpp @@ -139,25 +139,18 @@ class TBlobStorageGroupPatchRequest : public TBlobStorageGroupRequestActor &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvPatch *ev, - ui64 cookie, NWilson::TSpan&& span, TInstant now, - TIntrusivePtr &storagePoolCounters, - bool useVPatch = false) - : TBlobStorageGroupRequestActor(info, state, mon, source, cookie, - NKikimrServices::BS_PROXY_PATCH, false, {}, now, storagePoolCounters, - ev->RestartCounter, std::move(span), std::move(ev->ExecutionRelay)) - , OriginalGroupId(TGroupId::FromValue(ev->OriginalGroupId)) - , OriginalId(ev->OriginalId) - , PatchedId(ev->PatchedId) - , MaskForCookieBruteForcing(ev->MaskForCookieBruteForcing) - , DiffCount(ev->DiffCount) - , Diffs(ev->Diffs.Release()) - , StartTime(now) - , Deadline(ev->Deadline) - , Orbit(std::move(ev->Orbit)) - , UseVPatch(useVPatch) + TBlobStorageGroupPatchRequest(TBlobStorageGroupPatchParameters& params) + : TBlobStorageGroupRequestActor(params) + , OriginalGroupId(TGroupId::FromValue(params.Common.Event->OriginalGroupId)) + , OriginalId(params.Common.Event->OriginalId) + , PatchedId(params.Common.Event->PatchedId) + , MaskForCookieBruteForcing(params.Common.Event->MaskForCookieBruteForcing) + , DiffCount(params.Common.Event->DiffCount) + , Diffs(params.Common.Event->Diffs.Release()) + , StartTime(params.Common.Now) + , Deadline(params.Common.Event->Deadline) + , Orbit(std::move(params.Common.Event->Orbit)) + , UseVPatch(params.UseVPatch) {} void ReplyAndDie(NKikimrProto::EReplyStatus status) { @@ -1045,19 +1038,13 @@ class TBlobStorageGroupPatchRequest : public TBlobStorageGroupRequestActor &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvPatch *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, - TIntrusivePtr &storagePoolCounters, - bool useVPatch) { +IActor* CreateBlobStorageGroupPatchRequest(TBlobStorageGroupPatchParameters params, NWilson::TTraceId traceId) { NWilson::TSpan span(TWilson::BlobStorage, std::move(traceId), "DSProxy.Patch"); if (span) { - span.Attribute("event", ev->ToString()); + span.Attribute("event", params.Common.Event->ToString()); } - - return new TBlobStorageGroupPatchRequest(info, state, source, mon, ev, cookie, std::move(span), now, - storagePoolCounters, useVPatch); + params.Common.Span = std::move(span); + return new TBlobStorageGroupPatchRequest(params); } }//NKikimr diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_put.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_put.cpp index 1fc110a6d17d..dcd0b53829e1 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_put.cpp +++ b/ydb/core/blobstorage/dsproxy/dsproxy_put.cpp @@ -47,6 +47,7 @@ class TBlobStorageGroupPutRequest : public TBlobStorageGroupRequestActor 0); + RequestsPendingBeforeAcceleration--; + } const ui64 cyclesPerUs = NHPTimer::GetCyclesPerSecond() / 1000000; ev->Get()->Record.MutableTimestamps()->SetReceivedByDSProxyUs(GetCycleCountFast() / cyclesPerUs); @@ -282,6 +291,10 @@ class TBlobStorageGroupPutRequest : public TBlobStorageGroupRequestActor 0); + RequestsPendingBeforeAcceleration--; + } const ui64 cyclesPerUs = NHPTimer::GetCyclesPerSecond() / 1000000; ev->Get()->Record.MutableTimestamps()->SetReceivedByDSProxyUs(GetCycleCountFast() / cyclesPerUs); @@ -369,14 +382,19 @@ class TBlobStorageGroupPutRequest : public TBlobStorageGroupRequestActor 0 && WaitingVDiskCount <= 2 && RequestsSent > 1) { - ui64 timeToAccelerateUs = Max(1, PutImpl.GetTimeToAccelerateNs(LogCtx, 2 - AccelerateRequestsSent) / 1000); - TDuration timeSinceStart = TActivationContext::Monotonic() - StartTime; - if (timeSinceStart.MicroSeconds() < timeToAccelerateUs) { + if (!IsAccelerateScheduled && AccelerateRequestsSent < AccelerationParams.MaxNumOfSlowDisks) { + if (WaitingVDiskCount > 0 && WaitingVDiskCount <= AccelerationParams.MaxNumOfSlowDisks && RequestsSent > 1) { + ui64 timeToAccelerateUs = Max(1, PutImpl.GetTimeToAccelerateNs(LogCtx) / 1000); + if (RequestsPendingBeforeAcceleration == 1 && AccelerateRequestsSent == 1) { + // if there is only one request pending, but first accelerate is unsuccessful, make a pause + timeToAccelerateUs *= 2; + } + TDuration timeToAccelerate = TDuration::MicroSeconds(timeToAccelerateUs); + TMonotonic now = TActivationContext::Monotonic(); + TMonotonic nextAcceleration = StartTime + timeToAccelerate; + if (nextAcceleration > now) { ui64 causeIdx = RootCauseTrack.RegisterAccelerate(); - Schedule(TDuration::MicroSeconds(timeToAccelerateUs - timeSinceStart.MicroSeconds()), - new TEvAccelerate(causeIdx)); + Schedule(nextAcceleration - now, new TEvAccelerate(causeIdx)); IsAccelerateScheduled = true; } else { Accelerate(); @@ -499,71 +517,50 @@ class TBlobStorageGroupPutRequest : public TBlobStorageGroupRequestActor &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvPut *ev, - ui64 cookie, NWilson::TSpan&& span, bool timeStatsEnabled, - TDiskResponsivenessTracker::TPerDiskStatsPtr stats, - TMaybe latencyQueueKind, TInstant now, - TIntrusivePtr &storagePoolCounters, - bool enableRequestMod3x3ForMinLatecy) - : TBlobStorageGroupRequestActor(info, state, mon, source, cookie, - NKikimrServices::BS_PROXY_PUT, false, latencyQueueKind, now, storagePoolCounters, - ev->RestartCounter, std::move(span), nullptr) - , PutImpl(info, state, ev, mon, enableRequestMod3x3ForMinLatecy, source, cookie, Span.GetTraceId()) - , WaitingVDiskResponseCount(info->GetTotalVDisksNum()) - , HandleClass(ev->HandleClass) + TBlobStorageGroupPutRequest(TBlobStorageGroupPutParameters& params) + : TBlobStorageGroupRequestActor(params) + , PutImpl(Info, GroupQueues, params.Common.Event, Mon, + params.EnableRequestMod3x3ForMinLatency, params.Common.Source, + params.Common.Cookie, Span.GetTraceId(), params.AccelerationParams) + , WaitingVDiskResponseCount(Info->GetTotalVDisksNum()) + , HandleClass(params.Common.Event->HandleClass) , ReportedBytes(0) - , TimeStatsEnabled(timeStatsEnabled) - , Tactic(ev->Tactic) - , Stats(std::move(stats)) + , TimeStatsEnabled(params.TimeStatsEnabled) + , Tactic(params.Common.Event->Tactic) + , Stats(std::move(params.Stats)) + , AccelerationParams(params.AccelerationParams) , IsMultiPutMode(false) - , IncarnationRecords(info->GetTotalVDisksNum()) - , ExpiredVDiskSet(&info->GetTopology()) + , IncarnationRecords(Info->GetTotalVDisksNum()) + , ExpiredVDiskSet(&Info->GetTopology()) { - if (ev->Orbit.HasShuttles()) { + if (params.Common.Event->Orbit.HasShuttles()) { RootCauseTrack.IsOn = true; } ReportBytes(PutImpl.Blobs[0].Buffer.capacity() + sizeof(*this)); - RequestBytes = ev->Buffer.size(); + RequestBytes = params.Common.Event->Buffer.size(); RequestHandleClass = HandleClassToHandleClass(HandleClass); - MaxSaneRequests = info->Type.TotalPartCount() * (1ull + info->Type.Handoff()) * 2; + MaxSaneRequests = Info->Type.TotalPartCount() * (1ull + Info->Type.Handoff()) * 2; } - ui32 MaxRestartCounter(const TBatchedVec& events) { - ui32 res = 0; - for (const auto& ev : events) { - res = Max(res, ev->Get()->RestartCounter); - } - return res; - } - - TBlobStorageGroupPutRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, - const TIntrusivePtr &mon, TBatchedVec &events, - bool timeStatsEnabled, TDiskResponsivenessTracker::TPerDiskStatsPtr stats, - TMaybe latencyQueueKind, TInstant now, - TIntrusivePtr &storagePoolCounters, - NKikimrBlobStorage::EPutHandleClass handleClass, TEvBlobStorage::TEvPut::ETactic tactic, - bool enableRequestMod3x3ForMinLatecy) - : TBlobStorageGroupRequestActor(info, state, mon, TActorId(), 0, - NKikimrServices::BS_PROXY_PUT, false, latencyQueueKind, now, storagePoolCounters, - MaxRestartCounter(events), NWilson::TSpan(), nullptr) - , PutImpl(info, state, events, mon, handleClass, tactic, enableRequestMod3x3ForMinLatecy) - , WaitingVDiskResponseCount(info->GetTotalVDisksNum()) + TBlobStorageGroupPutRequest(TBlobStorageGroupMultiPutParameters& params) + : TBlobStorageGroupRequestActor(params) + , PutImpl(Info, GroupQueues, params.Events, Mon, params.HandleClass, params.Tactic, + params.EnableRequestMod3x3ForMinLatency, params.AccelerationParams) + , WaitingVDiskResponseCount(Info->GetTotalVDisksNum()) , IsManyPuts(true) - , HandleClass(handleClass) + , HandleClass(params.HandleClass) , ReportedBytes(0) - , TimeStatsEnabled(timeStatsEnabled) - , Tactic(tactic) - , Stats(std::move(stats)) + , TimeStatsEnabled(params.TimeStatsEnabled) + , Tactic(params.Tactic) + , Stats(std::move(params.Stats)) + , AccelerationParams(params.AccelerationParams) , IsMultiPutMode(true) - , IncarnationRecords(info->GetTotalVDisksNum()) - , ExpiredVDiskSet(&info->GetTopology()) + , IncarnationRecords(Info->GetTotalVDisksNum()) + , ExpiredVDiskSet(&Info->GetTopology()) { - Y_DEBUG_ABORT_UNLESS(events.size() <= MaxBatchedPutRequests); - for (auto &ev : events) { + Y_DEBUG_ABORT_UNLESS(params.Events.size() <= MaxBatchedPutRequests); + for (auto &ev : params.Events) { auto& msg = *ev->Get(); if (msg.Orbit.HasShuttles()) { RootCauseTrack.IsOn = true; @@ -577,7 +574,7 @@ class TBlobStorageGroupPutRequest : public TBlobStorageGroupRequestActorType.TotalPartCount() * (1ull + info->Type.Handoff()) * 2; + MaxSaneRequests = Info->Type.TotalPartCount() * (1ull + Info->Type.Handoff()) * 2; } void ReportBytes(i64 bytes) { @@ -752,35 +749,17 @@ class TBlobStorageGroupPutRequest : public TBlobStorageGroupRequestActor &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvPut *ev, - ui64 cookie, NWilson::TTraceId traceId, bool timeStatsEnabled, - TDiskResponsivenessTracker::TPerDiskStatsPtr stats, - TMaybe latencyQueueKind, TInstant now, - TIntrusivePtr &storagePoolCounters, - bool enableRequestMod3x3ForMinLatecy) { +IActor* CreateBlobStorageGroupPutRequest(TBlobStorageGroupPutParameters params, NWilson::TTraceId traceId) { NWilson::TSpan span(TWilson::BlobStorage, std::move(traceId), "DSProxy.Put"); if (span) { - span.Attribute("event", ev->ToString()); + span.Attribute("event", params.Common.Event->ToString()); } - - return new TBlobStorageGroupPutRequest(info, state, source, mon, ev, cookie, std::move(span), timeStatsEnabled, - std::move(stats), latencyQueueKind, now, storagePoolCounters, enableRequestMod3x3ForMinLatecy); + params.Common.Span = std::move(span); + return new TBlobStorageGroupPutRequest(params); } -IActor* CreateBlobStorageGroupPutRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, - const TIntrusivePtr &mon, TBatchedVec &ev, - bool timeStatsEnabled, - TDiskResponsivenessTracker::TPerDiskStatsPtr stats, - TMaybe latencyQueueKind, TInstant now, - TIntrusivePtr &storagePoolCounters, - NKikimrBlobStorage::EPutHandleClass handleClass, TEvBlobStorage::TEvPut::ETactic tactic, - bool enableRequestMod3x3ForMinLatecy) { - return new TBlobStorageGroupPutRequest(info, state, mon, ev, timeStatsEnabled, - std::move(stats), latencyQueueKind, now, storagePoolCounters, handleClass, tactic, - enableRequestMod3x3ForMinLatecy); +IActor* CreateBlobStorageGroupPutRequest(TBlobStorageGroupMultiPutParameters params) { + return new TBlobStorageGroupPutRequest(params); } }//NKikimr diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_put_impl.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_put_impl.cpp index 710fbdf4cfb6..67e9a2bd6412 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_put_impl.cpp +++ b/ydb/core/blobstorage/dsproxy/dsproxy_put_impl.cpp @@ -37,7 +37,7 @@ void TPutImpl::RunStrategy(TLogContext &logCtx, const IStrategy& strategy, TPutR const TBlobStorageGroupInfo::TGroupVDisks& expired) { Y_VERIFY_S(Blackboard.BlobStates.size(), "State# " << DumpFullState()); TBatchedVec finished; - const EStrategyOutcome outcome = Blackboard.RunStrategy(logCtx, strategy, &finished, &expired); + const EStrategyOutcome outcome = Blackboard.RunStrategy(logCtx, strategy, AccelerationParams, &finished, &expired); for (const TBlackboard::TFinishedBlob& item : finished) { Y_ABORT_UNLESS(item.BlobIdx < Blobs.size()); Y_ABORT_UNLESS(!IsDone[item.BlobIdx]); @@ -82,7 +82,7 @@ void TPutImpl::PrepareReply(NKikimrProto::EReplyStatus status, TLogContext &logC } } -ui64 TPutImpl::GetTimeToAccelerateNs(TLogContext &logCtx, ui32 nthWorst) { +ui64 TPutImpl::GetTimeToAccelerateNs(TLogContext &logCtx) { Y_UNUSED(logCtx); Y_ABORT_UNLESS(!Blackboard.BlobStates.empty()); TBatchedVec nthWorstPredictedNsVec(Blackboard.BlobStates.size()); @@ -90,9 +90,9 @@ ui64 TPutImpl::GetTimeToAccelerateNs(TLogContext &logCtx, ui32 nthWorst) { for (auto &[_, state] : Blackboard.BlobStates) { // Find the n'th slowest disk TDiskDelayPredictions worstDisks; - state.GetWorstPredictedDelaysNs(*Info, *Blackboard.GroupQueues, HandleClassToQueueId(Blackboard.PutHandleClass), nthWorst, - &worstDisks); - nthWorstPredictedNsVec[idx++] = worstDisks[nthWorst].PredictedNs; + state.GetWorstPredictedDelaysNs(*Info, *Blackboard.GroupQueues, HandleClassToQueueId(Blackboard.PutHandleClass), + &worstDisks, AccelerationParams); + nthWorstPredictedNsVec[idx++] = worstDisks[AccelerationParams.MaxNumOfSlowDisks].PredictedNs; } return *MaxElement(nthWorstPredictedNsVec.begin(), nthWorstPredictedNsVec.end()); } diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_put_impl.h b/ydb/core/blobstorage/dsproxy/dsproxy_put_impl.h index 48300adfd2e2..c292cf44368c 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_put_impl.h +++ b/ydb/core/blobstorage/dsproxy/dsproxy_put_impl.h @@ -44,6 +44,8 @@ class TPutImpl { const TEvBlobStorage::TEvPut::ETactic Tactic; + const TAccelerationParams AccelerationParams; + struct TBlobInfo { TLogoBlobID BlobId; TRope Buffer; @@ -103,7 +105,8 @@ class TPutImpl { public: TPutImpl(const TIntrusivePtr &info, const TIntrusivePtr &state, TEvBlobStorage::TEvPut *ev, const TIntrusivePtr &mon, - bool enableRequestMod3x3ForMinLatecy, TActorId recipient, ui64 cookie, NWilson::TTraceId traceId) + bool enableRequestMod3x3ForMinLatecy, TActorId recipient, ui64 cookie, NWilson::TTraceId traceId, + const TAccelerationParams& accelerationParams) : Info(info) , Blackboard(info, state, ev->HandleClass, NKikimrBlobStorage::EGetHandleClass::AsyncRead) , IsDone(1) @@ -113,6 +116,7 @@ class TPutImpl { , Mon(mon) , EnableRequestMod3x3ForMinLatecy(enableRequestMod3x3ForMinLatecy) , Tactic(ev->Tactic) + , AccelerationParams(accelerationParams) { BlobMap.emplace(ev->Id, Blobs.size()); Blobs.emplace_back(ev->Id, TRope(ev->Buffer), recipient, cookie, std::move(traceId), std::move(ev->Orbit), @@ -126,7 +130,7 @@ class TPutImpl { TPutImpl(const TIntrusivePtr &info, const TIntrusivePtr &state, TBatchedVec &events, const TIntrusivePtr &mon, NKikimrBlobStorage::EPutHandleClass putHandleClass, TEvBlobStorage::TEvPut::ETactic tactic, - bool enableRequestMod3x3ForMinLatecy) + bool enableRequestMod3x3ForMinLatecy, const TAccelerationParams& accelerationParams) : Info(info) , Blackboard(info, state, putHandleClass, NKikimrBlobStorage::EGetHandleClass::AsyncRead) , IsDone(events.size()) @@ -136,6 +140,7 @@ class TPutImpl { , Mon(mon) , EnableRequestMod3x3ForMinLatecy(enableRequestMod3x3ForMinLatecy) , Tactic(tactic) + , AccelerationParams(accelerationParams) { Y_ABORT_UNLESS(events.size(), "TEvPut vector is empty"); @@ -186,7 +191,7 @@ class TPutImpl { void PrepareOneReply(NKikimrProto::EReplyStatus status, size_t blobIdx, TLogContext &logCtx, TString errorReason, TPutResultVec &outPutResults); - ui64 GetTimeToAccelerateNs(TLogContext &logCtx, ui32 nthWorst); + ui64 GetTimeToAccelerateNs(TLogContext &logCtx); TString DumpFullState() const; diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_range.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_range.cpp index 246ee8417719..6f4843037ec1 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_range.cpp +++ b/ydb/core/blobstorage/dsproxy/dsproxy_range.cpp @@ -339,23 +339,17 @@ class TBlobStorageGroupRangeRequest : public TBlobStorageGroupRequestActor &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvRange *ev, - ui64 cookie, NWilson::TSpan&& span, TInstant now, - TIntrusivePtr &storagePoolCounters) - : TBlobStorageGroupRequestActor(info, state, mon, source, cookie, - NKikimrServices::BS_PROXY_RANGE, false, {}, now, storagePoolCounters, - ev->RestartCounter, std::move(span), std::move(ev->ExecutionRelay)) - , TabletId(ev->TabletId) - , From(ev->From) - , To(ev->To) - , Deadline(ev->Deadline) - , MustRestoreFirst(ev->MustRestoreFirst) - , IsIndexOnly(ev->IsIndexOnly) - , ForceBlockedGeneration(ev->ForceBlockedGeneration) - , Decommission(ev->Decommission) - , StartTime(now) + TBlobStorageGroupRangeRequest(TBlobStorageGroupRangeParameters& params) + : TBlobStorageGroupRequestActor(params) + , TabletId(params.Common.Event->TabletId) + , From(params.Common.Event->From) + , To(params.Common.Event->To) + , Deadline(params.Common.Event->Deadline) + , MustRestoreFirst(params.Common.Event->MustRestoreFirst) + , IsIndexOnly(params.Common.Event->IsIndexOnly) + , ForceBlockedGeneration(params.Common.Event->ForceBlockedGeneration) + , Decommission(params.Common.Event->Decommission) + , StartTime(params.Common.Now) , FailedDisks(&Info->GetTopology()) {} @@ -402,14 +396,9 @@ class TBlobStorageGroupRangeRequest : public TBlobStorageGroupRequestActor &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvRange *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, - TIntrusivePtr &storagePoolCounters) { - NWilson::TSpan span(TWilson::BlobStorage, std::move(traceId), "DSProxy.Range"); - return new TBlobStorageGroupRangeRequest(info, state, source, mon, ev, cookie, std::move(span), now, - storagePoolCounters); +IActor* CreateBlobStorageGroupRangeRequest(TBlobStorageGroupRangeParameters params, NWilson::TTraceId traceId) { + params.Common.Span = NWilson::TSpan(TWilson::BlobStorage, std::move(traceId), "DSProxy.Range"); + return new TBlobStorageGroupRangeRequest(params); } };//NKikimr diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_request.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_request.cpp index fd8b1c52dc93..3b806162e7a3 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_request.cpp +++ b/ydb/core/blobstorage/dsproxy/dsproxy_request.cpp @@ -44,9 +44,23 @@ namespace NKikimr { EnableWilsonTracing(ev, Mon->GetSamplePPM); if (ev->Get()->IsIndexOnly) { Mon->EventIndexRestoreGet->Inc(); - PushRequest(CreateBlobStorageGroupIndexRestoreGetRequest(Info, Sessions->GroupQueues, ev->Sender, Mon, - ev->Get(), ev->Cookie, std::move(ev->TraceId), {}, TActivationContext::Now(), StoragePoolCounters), - ev->Get()->Deadline); + PushRequest(CreateBlobStorageGroupIndexRestoreGetRequest( + TBlobStorageGroupRestoreGetParameters{ + .Common = { + .GroupInfo = Info, + .GroupQueues = Sessions->GroupQueues, + .Mon = Mon, + .Source = ev->Sender, + .Cookie = ev->Cookie, + .Now = TActivationContext::Now(), + .StoragePoolCounters = StoragePoolCounters, + .RestartCounter = ev->Get()->RestartCounter, + .Event = ev->Get(), + .ExecutionRelay = ev->Get()->ExecutionRelay, + } + }, std::move(ev->TraceId)), + ev->Get()->Deadline + ); } else { TLogoBlobID lastBlobId; const ui32 querySize = ev->Get()->QuerySize; @@ -78,14 +92,47 @@ namespace NKikimr { if (differentBlobCount == 1 || isSmall) { Mon->EventGet->Inc(); - PushRequest(CreateBlobStorageGroupGetRequest(Info, Sessions->GroupQueues, ev->Sender, Mon, - ev->Get(), ev->Cookie, std::move(ev->TraceId), TNodeLayoutInfoPtr(NodeLayoutInfo), - kind, TActivationContext::Now(), StoragePoolCounters), ev->Get()->Deadline); + PushRequest(CreateBlobStorageGroupGetRequest( + TBlobStorageGroupGetParameters{ + .Common = { + .GroupInfo = Info, + .GroupQueues = Sessions->GroupQueues, + .Mon = Mon, + .Source = ev->Sender, + .Cookie = ev->Cookie, + .Now = TActivationContext::Now(), + .StoragePoolCounters = StoragePoolCounters, + .RestartCounter = ev->Get()->RestartCounter, + .Event = ev->Get(), + .ExecutionRelay = ev->Get()->ExecutionRelay, + .LogAccEnabled = ev->Get()->IsVerboseNoDataEnabled || ev->Get()->CollectDebugInfo, + .LatencyQueueKind = kind, + }, + .NodeLayout = TNodeLayoutInfoPtr(NodeLayoutInfo), + .AccelerationParams = GetAccelerationParams(), + }, std::move(ev->TraceId)), + ev->Get()->Deadline + ); } else { Mon->EventMultiGet->Inc(); - PushRequest(CreateBlobStorageGroupMultiGetRequest(Info, Sessions->GroupQueues, ev->Sender, Mon, - ev->Get(), ev->Cookie, std::move(ev->TraceId), kind, TActivationContext::Now(), StoragePoolCounters), - ev->Get()->Deadline); + PushRequest(CreateBlobStorageGroupMultiGetRequest( + TBlobStorageGroupMultiGetParameters{ + .Common = { + .GroupInfo = Info, + .GroupQueues = Sessions->GroupQueues, + .Mon = Mon, + .Source = ev->Sender, + .Cookie = ev->Cookie, + .Now = TActivationContext::Now(), + .StoragePoolCounters = StoragePoolCounters, + .RestartCounter = ev->Get()->RestartCounter, + .Event = ev->Get(), + .ExecutionRelay = ev->Get()->ExecutionRelay, + .LatencyQueueKind = kind, + }, + }, std::move(ev->TraceId)), + ev->Get()->Deadline + ); } } } @@ -130,7 +177,10 @@ namespace NKikimr { Y_DEBUG_ABORT_UNLESS(MinREALHugeBlobInBytes); const ui32 partSize = Info->Type.PartSize(ev->Get()->Id); - if (EnablePutBatching && partSize < MinREALHugeBlobInBytes && partSize <= MaxBatchedPutSize) { + TInstant now = TActivationContext::Now(); + + if (Controls.EnablePutBatching.Update(now) && partSize < MinREALHugeBlobInBytes && + partSize <= MaxBatchedPutSize) { NKikimrBlobStorage::EPutHandleClass handleClass = ev->Get()->HandleClass; TEvBlobStorage::TEvPut::ETactic tactic = ev->Get()->Tactic; Y_ABORT_UNLESS((ui64)handleClass <= PutHandleClassCount); @@ -155,19 +205,51 @@ namespace NKikimr { TAppData *app = NKikimr::AppData(TActivationContext::AsActorContext()); bool enableRequestMod3x3ForMinLatency = app->FeatureFlags.GetEnable3x3RequestsForMirror3DCMinLatencyPut(); // TODO(alexvru): MinLatency support - PushRequest(CreateBlobStorageGroupPutRequest(Info, Sessions->GroupQueues, ev->Sender, Mon, - ev->Get(), ev->Cookie, std::move(ev->TraceId), Mon->TimeStats.IsEnabled(), - PerDiskStats, kind, TActivationContext::Now(), StoragePoolCounters, - enableRequestMod3x3ForMinLatency), ev->Get()->Deadline); + PushRequest(CreateBlobStorageGroupPutRequest( + TBlobStorageGroupPutParameters{ + .Common = { + .GroupInfo = Info, + .GroupQueues = Sessions->GroupQueues, + .Mon = Mon, + .Source = ev->Sender, + .Cookie = ev->Cookie, + .Now = TActivationContext::Now(), + .StoragePoolCounters = StoragePoolCounters, + .RestartCounter = ev->Get()->RestartCounter, + .Event = ev->Get(), + .ExecutionRelay = ev->Get()->ExecutionRelay, + .LatencyQueueKind = kind + }, + .TimeStatsEnabled = Mon->TimeStats.IsEnabled(), + .Stats = PerDiskStats, + .EnableRequestMod3x3ForMinLatency = enableRequestMod3x3ForMinLatency, + .AccelerationParams = GetAccelerationParams(), + }, std::move(ev->TraceId)), + ev->Get()->Deadline + ); } } void TBlobStorageGroupProxy::HandleNormal(TEvBlobStorage::TEvBlock::TPtr &ev) { EnsureMonitoring(ev->Get()->IsMonitored); Mon->EventBlock->Inc(); - PushRequest(CreateBlobStorageGroupBlockRequest(Info, Sessions->GroupQueues, ev->Sender, Mon, - ev->Get(), ev->Cookie, std::move(ev->TraceId), TActivationContext::Now(), StoragePoolCounters), - ev->Get()->Deadline); + PushRequest(CreateBlobStorageGroupBlockRequest( + TBlobStorageGroupBlockParameters{ + .Common = { + .GroupInfo = Info, + .GroupQueues = Sessions->GroupQueues, + .Mon = Mon, + .Source = ev->Sender, + .Cookie = ev->Cookie, + .Now = TActivationContext::Now(), + .StoragePoolCounters = StoragePoolCounters, + .RestartCounter = ev->Get()->RestartCounter, + .Event = ev->Get(), + .ExecutionRelay = ev->Get()->ExecutionRelay + } + }, std::move(ev->TraceId)), + ev->Get()->Deadline + ); } void TBlobStorageGroupProxy::HandleNormal(TEvBlobStorage::TEvPatch::TPtr &ev) { @@ -179,9 +261,24 @@ namespace NKikimr { EnsureMonitoring(true); Mon->EventPatch->Inc(); TInstant now = TActivationContext::Now(); - PushRequest(CreateBlobStorageGroupPatchRequest(Info, Sessions->GroupQueues, ev->Sender, Mon, - ev->Get(), ev->Cookie, std::move(ev->TraceId), now, StoragePoolCounters, EnableVPatch.Update(now)), - ev->Get()->Deadline); + PushRequest(CreateBlobStorageGroupPatchRequest( + TBlobStorageGroupPatchParameters{ + .Common = { + .GroupInfo = Info, + .GroupQueues = Sessions->GroupQueues, + .Mon = Mon, + .Source = ev->Sender, + .Cookie = ev->Cookie, + .Now = now, + .StoragePoolCounters = StoragePoolCounters, + .RestartCounter = ev->Get()->RestartCounter, + .Event = ev->Get(), + .ExecutionRelay = ev->Get()->ExecutionRelay + }, + .UseVPatch = static_cast(Controls.EnableVPatch.Update(now)) + }, std::move(ev->TraceId)), + ev->Get()->Deadline + ); } void TBlobStorageGroupProxy::HandleNormal(TEvBlobStorage::TEvDiscover::TPtr &ev) { @@ -195,13 +292,31 @@ namespace NKikimr { EnsureMonitoring(true); Mon->EventDiscover->Inc(); EnableWilsonTracing(ev, Mon->DiscoverSamplePPM); - auto&& callback = Info->Type.GetErasure() == TBlobStorageGroupType::ErasureMirror3dc + TErasureType::EErasureSpecies erasure = Info->Type.GetErasure(); + auto&& callback = erasure == TBlobStorageGroupType::ErasureMirror3dc ? CreateBlobStorageGroupMirror3dcDiscoverRequest - : Info->Type.GetErasure() == TBlobStorageGroupType::ErasureMirror3of4 + : erasure == TBlobStorageGroupType::ErasureMirror3of4 ? CreateBlobStorageGroupMirror3of4DiscoverRequest : CreateBlobStorageGroupDiscoverRequest; - PushRequest(callback(Info, Sessions->GroupQueues, ev->Sender, Mon, ev->Get(), ev->Cookie, std::move(ev->TraceId), - TActivationContext::Now(), StoragePoolCounters), ev->Get()->Deadline); + PushRequest(callback( + TBlobStorageGroupDiscoverParameters{ + .Common = { + .GroupInfo = Info, + .GroupQueues = Sessions->GroupQueues, + .Mon = Mon, + .Source = ev->Sender, + .Cookie = ev->Cookie, + .Now = TActivationContext::Now(), + .StoragePoolCounters = StoragePoolCounters, + .RestartCounter = ev->Get()->RestartCounter, + .Event = ev->Get(), + .ExecutionRelay = ev->Get()->ExecutionRelay, + .LogAccEnabled = (erasure != TBlobStorageGroupType::ErasureMirror3dc) && + (erasure != TBlobStorageGroupType::ErasureMirror3of4) + } + }, std::move(ev->TraceId)), + ev->Get()->Deadline + ); } void TBlobStorageGroupProxy::HandleNormal(TEvBlobStorage::TEvRange::TPtr &ev) { @@ -212,9 +327,23 @@ namespace NKikimr { } EnsureMonitoring(true); Mon->EventRange->Inc(); - PushRequest(CreateBlobStorageGroupRangeRequest(Info, Sessions->GroupQueues, ev->Sender, Mon, - ev->Get(), ev->Cookie, std::move(ev->TraceId), TActivationContext::Now(), StoragePoolCounters), - ev->Get()->Deadline); + PushRequest(CreateBlobStorageGroupRangeRequest( + TBlobStorageGroupRangeParameters{ + .Common = { + .GroupInfo = Info, + .GroupQueues = Sessions->GroupQueues, + .Mon = Mon, + .Source = ev->Sender, + .Cookie = ev->Cookie, + .Now = TActivationContext::Now(), + .StoragePoolCounters = StoragePoolCounters, + .RestartCounter = ev->Get()->RestartCounter, + .Event = ev->Get(), + .ExecutionRelay = ev->Get()->ExecutionRelay + } + }, std::move(ev->TraceId)), + ev->Get()->Deadline + ); } void TBlobStorageGroupProxy::HandleNormal(TEvBlobStorage::TEvCollectGarbage::TPtr &ev) { @@ -222,14 +351,42 @@ namespace NKikimr { if (!ev->Get()->IsMultiCollectAllowed || ev->Get()->PerGenerationCounterStepSize() == 1) { Mon->EventCollectGarbage->Inc(); - PushRequest(CreateBlobStorageGroupCollectGarbageRequest(Info, Sessions->GroupQueues, - ev->Sender, Mon, ev->Get(), ev->Cookie, std::move(ev->TraceId), TActivationContext::Now(), - StoragePoolCounters), ev->Get()->Deadline); + PushRequest(CreateBlobStorageGroupCollectGarbageRequest( + TBlobStorageGroupCollectGarbageParameters{ + .Common = { + .GroupInfo = Info, + .GroupQueues = Sessions->GroupQueues, + .Mon = Mon, + .Source = ev->Sender, + .Cookie = ev->Cookie, + .Now = TActivationContext::Now(), + .StoragePoolCounters = StoragePoolCounters, + .RestartCounter = ev->Get()->RestartCounter, + .Event = ev->Get(), + .ExecutionRelay = ev->Get()->ExecutionRelay + } + }, std::move(ev->TraceId)), + ev->Get()->Deadline + ); } else { Mon->EventMultiCollect->Inc(); - PushRequest(CreateBlobStorageGroupMultiCollectRequest(Info, Sessions->GroupQueues, - ev->Sender, Mon, ev->Get(), ev->Cookie, std::move(ev->TraceId), TActivationContext::Now(), - StoragePoolCounters), ev->Get()->Deadline); + PushRequest(CreateBlobStorageGroupMultiCollectRequest( + TBlobStorageGroupMultiCollectParameters{ + .Common = { + .GroupInfo = Info, + .GroupQueues = Sessions->GroupQueues, + .Mon = Mon, + .Source = ev->Sender, + .Cookie = ev->Cookie, + .Now = TActivationContext::Now(), + .StoragePoolCounters = StoragePoolCounters, + .RestartCounter = ev->Get()->RestartCounter, + .Event = ev->Get(), + .ExecutionRelay = ev->Get()->ExecutionRelay + } + }, std::move(ev->TraceId)), + ev->Get()->Deadline + ); } } @@ -241,17 +398,45 @@ namespace NKikimr { } EnsureMonitoring(true); Mon->EventStatus->Inc(); - PushRequest(CreateBlobStorageGroupStatusRequest(Info, Sessions->GroupQueues, ev->Sender, Mon, - ev->Get(), ev->Cookie, std::move(ev->TraceId), TActivationContext::Now(), StoragePoolCounters), - ev->Get()->Deadline); + PushRequest(CreateBlobStorageGroupStatusRequest( + TBlobStorageGroupStatusParameters{ + .Common = { + .GroupInfo = Info, + .GroupQueues = Sessions->GroupQueues, + .Mon = Mon, + .Source = ev->Sender, + .Cookie = ev->Cookie, + .Now = TActivationContext::Now(), + .StoragePoolCounters = StoragePoolCounters, + .RestartCounter = ev->Get()->RestartCounter, + .Event = ev->Get(), + .ExecutionRelay = ev->Get()->ExecutionRelay + } + }, std::move(ev->TraceId)), + TInstant::Max() + ); } void TBlobStorageGroupProxy::HandleNormal(TEvBlobStorage::TEvAssimilate::TPtr &ev) { EnsureMonitoring(true); Mon->EventAssimilate->Inc(); - PushRequest(CreateBlobStorageGroupAssimilateRequest(Info, Sessions->GroupQueues, ev->Sender, - Mon, ev->Get(), ev->Cookie, std::move(ev->TraceId), TActivationContext::Now(), StoragePoolCounters), - TInstant::Max()); + PushRequest(CreateBlobStorageGroupAssimilateRequest( + TBlobStorageGroupAssimilateParameters{ + .Common = { + .GroupInfo = Info, + .GroupQueues = Sessions->GroupQueues, + .Mon = Mon, + .Source = ev->Sender, + .Cookie = ev->Cookie, + .Now = TActivationContext::Now(), + .StoragePoolCounters = StoragePoolCounters, + .RestartCounter = ev->Get()->RestartCounter, + .Event = ev->Get(), + .ExecutionRelay = ev->Get()->ExecutionRelay + } + }, std::move(ev->TraceId)), + TInstant::Max() + ); } void TBlobStorageGroupProxy::Handle(TEvDeathNote::TPtr ev) { @@ -285,14 +470,50 @@ namespace NKikimr { // TODO(alexvru): MinLatency support if (batchedPuts.Queue.size() == 1) { auto& ev = batchedPuts.Queue.front(); - PushRequest(CreateBlobStorageGroupPutRequest(Info, Sessions->GroupQueues, ev->Sender, - Mon, ev->Get(), ev->Cookie, std::move(ev->TraceId), Mon->TimeStats.IsEnabled(), PerDiskStats, - kind, TActivationContext::Now(), StoragePoolCounters, enableRequestMod3x3ForMinLatency), - ev->Get()->Deadline); + PushRequest(CreateBlobStorageGroupPutRequest( + TBlobStorageGroupPutParameters{ + .Common = { + .GroupInfo = Info, + .GroupQueues = Sessions->GroupQueues, + .Mon = Mon, + .Source = ev->Sender, + .Cookie = ev->Cookie, + .Now = TActivationContext::Now(), + .StoragePoolCounters = StoragePoolCounters, + .RestartCounter = ev->Get()->RestartCounter, + .Event = ev->Get(), + .ExecutionRelay = ev->Get()->ExecutionRelay, + .LatencyQueueKind = kind, + }, + .TimeStatsEnabled = Mon->TimeStats.IsEnabled(), + .Stats = PerDiskStats, + .EnableRequestMod3x3ForMinLatency = enableRequestMod3x3ForMinLatency, + .AccelerationParams = GetAccelerationParams(), + }, std::move(ev->TraceId)), + ev->Get()->Deadline + ); } else { - PushRequest(CreateBlobStorageGroupPutRequest(Info, Sessions->GroupQueues, - Mon, batchedPuts.Queue, Mon->TimeStats.IsEnabled(), PerDiskStats, kind, TActivationContext::Now(), - StoragePoolCounters, handleClass, tactic, enableRequestMod3x3ForMinLatency), TInstant::Max()); + PushRequest(CreateBlobStorageGroupPutRequest( + TBlobStorageGroupMultiPutParameters{ + .Common = { + .GroupInfo = Info, + .GroupQueues = Sessions->GroupQueues, + .Mon = Mon, + .Now = TActivationContext::Now(), + .StoragePoolCounters = StoragePoolCounters, + .RestartCounter = TBlobStorageGroupMultiPutParameters::CalculateRestartCounter(batchedPuts.Queue), + .LatencyQueueKind = kind, + }, + .Events = batchedPuts.Queue, + .TimeStatsEnabled = Mon->TimeStats.IsEnabled(), + .Stats = PerDiskStats, + .HandleClass = handleClass, + .Tactic = tactic, + .EnableRequestMod3x3ForMinLatency = enableRequestMod3x3ForMinLatency, + .AccelerationParams = GetAccelerationParams(), + }), + TInstant::Max() + ); } } else { for (auto it = batchedPuts.Queue.begin(); it != batchedPuts.Queue.end(); ++it) { @@ -323,7 +544,7 @@ namespace NKikimr { ++*Mon->EventStopPutBatching; LWPROBE(DSProxyBatchedPutRequest, BatchedPutRequestCount, GroupId.GetRawId()); BatchedPutRequestCount = 0; - EnablePutBatching.Update(TActivationContext::Now()); + Controls.EnablePutBatching.Update(TActivationContext::Now()); } void TBlobStorageGroupProxy::Handle(TEvStopBatchingGetRequests::TPtr& ev) { @@ -333,4 +554,18 @@ namespace NKikimr { BatchedGetRequestCount = 0; } + void TBlobStorageGroupProxy::Handle(TEvGetQueuesInfo::TPtr ev) { + ui32 groupSize = Info->GetTotalVDisksNum(); + std::unique_ptr res = std::make_unique(groupSize); + if (Sessions && Sessions->GroupQueues) { + for (ui32 orderNum = 0; orderNum < groupSize; ++orderNum) { + TGroupQueues::TVDisk* vdisk = Sessions->GroupQueues->DisksByOrderNumber[orderNum]; + if (vdisk) { + const TGroupQueues::TVDisk::TQueues::TQueue& queue = vdisk->Queues.GetQueue(ev->Get()->QueueId); + res->AddInfoForQueue(orderNum, queue.ActorId, queue.FlowRecord); + } + } + } + TActivationContext::Send(ev->Sender, std::move(res)); + } } // NKikimr diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_state.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_state.cpp index 079ce5e2d5df..3cb1b3fd448b 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_state.cpp +++ b/ydb/core/blobstorage/dsproxy/dsproxy_state.cpp @@ -150,7 +150,8 @@ namespace NKikimr { } } else { // this is the first time configuration arrives -- no queues are created yet EnsureMonitoring(false); - Sessions = MakeIntrusive(Info, BSProxyCtx, MonActor, SelfId()); + Sessions = MakeIntrusive(Info, BSProxyCtx, MonActor, SelfId(), + UseActorSystemTimeInBSQueue); NumUnconnectedDisks = Sessions->GetNumUnconnectedDisks(); NodeMon->IncNumUnconnected(NumUnconnectedDisks); } @@ -322,4 +323,28 @@ namespace NKikimr { Send(ev->Sender, new TEvProxySessionsState(Sessions ? Sessions->GroupQueues : nullptr)); } +#define SELECT_CONTROL_BY_DEVICE_TYPE(prefix, info) \ +([&](NPDisk::EDeviceType deviceType) -> i64 { \ + TInstant now = TActivationContext::Now(); \ + switch (deviceType) { \ + case NPDisk::DEVICE_TYPE_ROT: \ + return Controls.prefix##HDD.Update(now); \ + case NPDisk::DEVICE_TYPE_SSD: \ + case NPDisk::DEVICE_TYPE_NVME: \ + return Controls.prefix##SSD.Update(now); \ + default: \ + return Controls.prefix.Update(now); \ + } \ +})(info ? info->GetDeviceType() : NPDisk::DEVICE_TYPE_UNKNOWN) + + TAccelerationParams TBlobStorageGroupProxy::GetAccelerationParams() { + return TAccelerationParams{ + .SlowDiskThreshold = .001f * SELECT_CONTROL_BY_DEVICE_TYPE(SlowDiskThreshold, Info), + .PredictedDelayMultiplier = .001f * SELECT_CONTROL_BY_DEVICE_TYPE(PredictedDelayMultiplier, Info), + .MaxNumOfSlowDisks = static_cast(SELECT_CONTROL_BY_DEVICE_TYPE(MaxNumOfSlowDisks, Info)), + }; + } + +#undef SELECT_CONTROL_BY_DEVICE_TYPE + } // NKikimr diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_status.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_status.cpp index 293c7db6d3f2..2eb56e8c78e0 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_status.cpp +++ b/ydb/core/blobstorage/dsproxy/dsproxy_status.cpp @@ -84,14 +84,9 @@ class TBlobStorageGroupStatusRequest : public TBlobStorageGroupRequestActorActiveStatus; } - TBlobStorageGroupStatusRequest(const TIntrusivePtr &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvStatus *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, TIntrusivePtr &storagePoolCounters) - : TBlobStorageGroupRequestActor(info, state, mon, source, cookie, - NKikimrServices::BS_PROXY_STATUS, false, {}, now, storagePoolCounters, ev->RestartCounter, - NWilson::TSpan(TWilson::BlobStorage, std::move(traceId), "DSProxy.Status"), std::move(ev->ExecutionRelay)) - , Deadline(ev->Deadline) + TBlobStorageGroupStatusRequest(TBlobStorageGroupStatusParameters& params) + : TBlobStorageGroupRequestActor(params) + , Deadline(params.Common.Event->Deadline) , Requests(0) , Responses(0) , QuorumTracker(Info.Get()) @@ -134,11 +129,9 @@ class TBlobStorageGroupStatusRequest : public TBlobStorageGroupRequestActor &info, - const TIntrusivePtr &state, const TActorId &source, - const TIntrusivePtr &mon, TEvBlobStorage::TEvStatus *ev, - ui64 cookie, NWilson::TTraceId traceId, TInstant now, TIntrusivePtr &storagePoolCounters) { - return new TBlobStorageGroupStatusRequest(info, state, source, mon, ev, cookie, std::move(traceId), now, storagePoolCounters); +IActor* CreateBlobStorageGroupStatusRequest(TBlobStorageGroupStatusParameters params, NWilson::TTraceId traceId) { + params.Common.Span = NWilson::TSpan(TWilson::BlobStorage, std::move(traceId), "DSProxy.Status"); + return new TBlobStorageGroupStatusRequest(params); } } // NKikimr diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_accelerate_put.h b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_accelerate_put.h index 3b40f5374a4c..9e78e5c73248 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_accelerate_put.h +++ b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_accelerate_put.h @@ -11,24 +11,26 @@ namespace NKikimr { class TAcceleratePutStrategy : public TStrategyBase { public: EStrategyOutcome Process(TLogContext &logCtx, TBlobState &state, const TBlobStorageGroupInfo &info, - TBlackboard& /*blackboard*/, TGroupDiskRequests &groupDiskRequests) override { + TBlackboard& /*blackboard*/, TGroupDiskRequests &groupDiskRequests, + const TAccelerationParams& accelerationParams) override { + Y_UNUSED(accelerationParams); // Find the unput part and disk - TStackVec badDiskIdxs; + ui32 badDisksMask = 0; for (size_t diskIdx = 0; diskIdx < state.Disks.size(); ++diskIdx) { TBlobState::TDisk &disk = state.Disks[diskIdx]; for (size_t partIdx = 0; partIdx < disk.DiskParts.size(); ++partIdx) { TBlobState::TDiskPart &diskPart = disk.DiskParts[partIdx]; if (diskPart.Situation == TBlobState::ESituation::Sent) { - badDiskIdxs.push_back(diskIdx); + badDisksMask |= (1 << diskIdx); } } } - if (!badDiskIdxs.empty()) { + if (badDisksMask > 0) { // Mark the corresponding disks 'bad' // Prepare part layout if possible TBlobStorageGroupType::TPartLayout layout; - PreparePartLayout(state, info, &layout, badDiskIdxs); + PreparePartLayout(state, info, &layout, badDisksMask); TBlobStorageGroupType::TPartPlacement partPlacement; bool isCorrectable = info.Type.CorrectLayout(layout, partPlacement); diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_accelerate_put_m3dc.h b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_accelerate_put_m3dc.h index 03ba79f7b0db..806155f54d9e 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_accelerate_put_m3dc.h +++ b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_accelerate_put_m3dc.h @@ -30,50 +30,56 @@ class TAcceleratePut3dcStrategy : public TStrategyBase { } EStrategyOutcome Process(TLogContext &logCtx, TBlobState &state, const TBlobStorageGroupInfo &info, - TBlackboard& /*blackboard*/, TGroupDiskRequests &groupDiskRequests) override { - // Find the unput part and disk - i32 badDiskIdx = -1; - for (size_t diskIdx = 0; diskIdx < state.Disks.size(); ++diskIdx) { + TBlackboard& blackboard, TGroupDiskRequests &groupDiskRequests, + const TAccelerationParams& accelerationParams) override { + Y_UNUSED(accelerationParams); + // Find the unput parts and disks + bool unresponsiveDisk = false; + for (size_t diskIdx = 0; diskIdx < state.Disks.size() && !unresponsiveDisk; ++diskIdx) { TBlobState::TDisk &disk = state.Disks[diskIdx]; - for (size_t partIdx = 0; partIdx < disk.DiskParts.size(); ++partIdx) { - TBlobState::TDiskPart &diskPart = disk.DiskParts[partIdx]; + for (TBlobState::TDiskPart &diskPart : disk.DiskParts) { if (diskPart.Situation == TBlobState::ESituation::Sent) { - badDiskIdx = diskIdx; + unresponsiveDisk = true; + break; } } } - - if (badDiskIdx >= 0) { - // Mark the 'bad' disk as the single slow disk - for (size_t diskIdx = 0; diskIdx < state.Disks.size(); ++diskIdx) { - state.Disks[diskIdx].IsSlow = false; - } - state.Disks[badDiskIdx].IsSlow = true; - - // Prepare part placement if possible - TBlobStorageGroupType::TPartPlacement partPlacement; - bool degraded = false; - - // check if we are in degraded mode -- that means that we have one fully failed realm - TBlobStorageGroupInfo::TSubgroupVDisks success(&info.GetTopology()); - TBlobStorageGroupInfo::TSubgroupVDisks error(&info.GetTopology()); - Evaluate3dcSituation(state, NumFailRealms, NumFailDomainsPerFailRealm, info, true, success, error, degraded); - - // check for failure tolerance; we issue ERROR in case when it is not possible to achieve success condition in - // any way; also check if we have already finished writing replicas - const auto& checker = info.GetQuorumChecker(); - if (checker.CheckFailModelForSubgroup(error)) { - if (checker.CheckQuorumForSubgroup(success)) { - // OK - return EStrategyOutcome::DONE; - } - - // now check every realm and check if we have to issue some write requests to it - Prepare3dcPartPlacement(state, NumFailRealms, NumFailDomainsPerFailRealm, - PreferredReplicasPerRealm(degraded), true, partPlacement); - - if (IsPutNeeded(state, partPlacement)) { - PreparePutsForPartPlacement(logCtx, state, info, groupDiskRequests, partPlacement); + if (unresponsiveDisk) { + blackboard.MarkSlowDisks(state, true, accelerationParams); + + for (bool considerSlowAsError : {true, false}) { + // Prepare part placement if possible + TBlobStorageGroupType::TPartPlacement partPlacement; + bool degraded = false; + + // check if we are in degraded mode -- that means that we have one fully failed realm + TBlobStorageGroupInfo::TSubgroupVDisks success(&info.GetTopology()); + TBlobStorageGroupInfo::TSubgroupVDisks error(&info.GetTopology()); + Evaluate3dcSituation(state, NumFailRealms, NumFailDomainsPerFailRealm, info, considerSlowAsError, + success, error, degraded); + // check for failure tolerance; we issue ERROR in case when it is not possible to achieve success condition in + // any way; also check if we have already finished writing replicas + const auto& checker = info.GetQuorumChecker(); + if (checker.CheckFailModelForSubgroup(error)) { + if (checker.CheckQuorumForSubgroup(success)) { + // OK + return EStrategyOutcome::DONE; + } + + // now check every realm and check if we have to issue some write requests to it + bool fullPlacement; + Prepare3dcPartPlacement(state, NumFailRealms, NumFailDomainsPerFailRealm, + PreferredReplicasPerRealm(degraded), considerSlowAsError, true, partPlacement, fullPlacement); + + if (considerSlowAsError && !fullPlacement) { + // unable to place all parts to fast disks, retry + continue; + } + + if (IsPutNeeded(state, partPlacement)) { + PreparePutsForPartPlacement(logCtx, state, info, groupDiskRequests, partPlacement); + } + break; } } } diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_base.cpp b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_base.cpp index 9243f693990c..f2b3dfc8b151 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_base.cpp +++ b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_base.cpp @@ -194,7 +194,7 @@ void TStrategyBase::AddGetRequest(TLogContext &logCtx, TGroupDiskRequests &group } void TStrategyBase::PreparePartLayout(const TBlobState &state, const TBlobStorageGroupInfo &info, - TBlobStorageGroupType::TPartLayout *layout, const TStackVec& slowDiskIdxs) { + TBlobStorageGroupType::TPartLayout *layout, ui32 slowDiskSubgroupMask) { Y_ABORT_UNLESS(layout); const ui32 totalPartCount = info.Type.TotalPartCount(); const ui32 blobSubringSize = info.Type.BlobSubgroupSize(); @@ -216,7 +216,7 @@ void TStrategyBase::PreparePartLayout(const TBlobState &state, const TBlobStorag if (!isErrorDisk) { for (ui32 partIdx = beginPartIdx; partIdx < endPartIdx; ++partIdx) { TBlobState::ESituation partSituation = disk.DiskParts[partIdx].Situation; - bool isOnSlowDisk = (std::find(slowDiskIdxs.begin(), slowDiskIdxs.end(), diskIdx) != slowDiskIdxs.end()); + bool isOnSlowDisk = (slowDiskSubgroupMask & (1 << diskIdx)); if (partSituation == TBlobState::ESituation::Present || (!isOnSlowDisk && partSituation == TBlobState::ESituation::Sent)) { layout->VDiskPartMask[diskIdx] |= (1ul << partIdx); @@ -225,15 +225,7 @@ void TStrategyBase::PreparePartLayout(const TBlobState &state, const TBlobStorag } } } - if (slowDiskIdxs.empty()) { - layout->SlowVDiskMask = 0; - } else { - layout->SlowVDiskMask = 0; - for (ui32 slowDiskIdx : slowDiskIdxs) { - Y_DEBUG_ABORT_UNLESS(slowDiskIdx < sizeof(layout->SlowVDiskMask) * 8); - layout->SlowVDiskMask |= (1ull << slowDiskIdx); - } - } + layout->SlowVDiskMask = slowDiskSubgroupMask; } bool TStrategyBase::IsPutNeeded(const TBlobState &state, const TBlobStorageGroupType::TPartPlacement &partPlacement) { @@ -363,10 +355,10 @@ void TStrategyBase::Evaluate3dcSituation(const TBlobState &state, } } -void TStrategyBase::Prepare3dcPartPlacement(const TBlobState &state, - size_t numFailRealms, size_t numFailDomainsPerFailRealm, - ui8 preferredReplicasPerRealm, bool considerSlowAsError, - TBlobStorageGroupType::TPartPlacement &outPartPlacement) { +void TStrategyBase::Prepare3dcPartPlacement(const TBlobState& state, size_t numFailRealms, size_t numFailDomainsPerFailRealm, + ui8 preferredReplicasPerRealm, bool considerSlowAsError, bool replaceUnresponsive, + TBlobStorageGroupType::TPartPlacement& outPartPlacement, bool& fullPlacement) { + fullPlacement = true; for (size_t realm = 0; realm < numFailRealms; ++realm) { ui8 placed = 0; for (size_t domain = 0; placed < preferredReplicasPerRealm @@ -377,6 +369,10 @@ void TStrategyBase::Prepare3dcPartPlacement(const TBlobState &state, if (situation != TBlobState::ESituation::Error) { if (situation == TBlobState::ESituation::Present) { placed++; + } else if (situation == TBlobState::ESituation::Sent) { + if (!replaceUnresponsive) { + placed++; + } } else if (!considerSlowAsError || !disk.IsSlow) { if (situation != TBlobState::ESituation::Sent) { outPartPlacement.Records.emplace_back(subgroupIdx, realm); @@ -385,51 +381,29 @@ void TStrategyBase::Prepare3dcPartPlacement(const TBlobState &state, } } } + if (placed < preferredReplicasPerRealm) { + fullPlacement = false; + } } } -ui32 TStrategyBase::MakeSlowSubgroupDiskMask(TBlobState &state, const TBlobStorageGroupInfo &info, TBlackboard &blackboard, - bool isPut) { - if (info.GetTotalVDisksNum() == 1) { - // when there is only one disk, we consider it not slow - return 0; - } - // Find the slowest disk +ui32 TStrategyBase::MakeSlowSubgroupDiskMask(TBlobState &state, TBlackboard &blackboard, bool isPut, + const TAccelerationParams& accelerationParams) { + // Find slow disks switch (blackboard.AccelerationMode) { - case TBlackboard::AccelerationModeSkipOneSlowest: { - TDiskDelayPredictions worstDisks; - state.GetWorstPredictedDelaysNs(info, *blackboard.GroupQueues, - (isPut ? HandleClassToQueueId(blackboard.PutHandleClass) : - HandleClassToQueueId(blackboard.GetHandleClass)), 1, - &worstDisks); - - // Check if the slowest disk exceptionally slow, or just not very fast - ui32 slowDiskSubgroupMask = 0; - if (worstDisks[1].PredictedNs > 0 && worstDisks[0].PredictedNs > worstDisks[1].PredictedNs * 2) { - slowDiskSubgroupMask = 1 << worstDisks[0].DiskIdx; - } - - // Mark single slow disk - for (size_t diskIdx = 0; diskIdx < state.Disks.size(); ++diskIdx) { - state.Disks[diskIdx].IsSlow = false; - } - if (slowDiskSubgroupMask > 0) { - state.Disks[worstDisks[0].DiskIdx].IsSlow = true; - } - - return slowDiskSubgroupMask; - } - case TBlackboard::AccelerationModeSkipMarked: { - ui32 slowDiskSubgroupMask = 0; - for (size_t diskIdx = 0; diskIdx < state.Disks.size(); ++diskIdx) { - if (state.Disks[diskIdx].IsSlow) { - slowDiskSubgroupMask |= 1 << diskIdx; - } - } - return slowDiskSubgroupMask; + case TBlackboard::AccelerationModeSkipNSlowest: + blackboard.MarkSlowDisks(state, isPut, accelerationParams); + break; + case TBlackboard::AccelerationModeSkipMarked: + break; + } + ui32 slowDiskSubgroupMask = 0; + for (size_t diskIdx = 0; diskIdx < state.Disks.size(); ++diskIdx) { + if (state.Disks[diskIdx].IsSlow) { + slowDiskSubgroupMask |= 1 << diskIdx; } } - return 0; + return slowDiskSubgroupMask; } }//NKikimr diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_base.h b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_base.h index 67da2e3fef3e..3e90c6641aa5 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_base.h +++ b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_base.h @@ -34,7 +34,7 @@ class TStrategyBase : public IStrategy { void AddGetRequest(TLogContext &logCtx, TGroupDiskRequests &groupDiskRequests, TLogoBlobID &fullId, ui32 partIdx, TBlobState::TDisk &disk, TIntervalSet &intervalSet, const char *logMarker); void PreparePartLayout(const TBlobState &state, const TBlobStorageGroupInfo &info, - TBlobStorageGroupType::TPartLayout *layout, const TStackVec& slowDiskIdxs); + TBlobStorageGroupType::TPartLayout *layout, ui32 slowDiskSubgroupMask); bool IsPutNeeded(const TBlobState &state, const TBlobStorageGroupType::TPartPlacement &partPlacement); void PreparePutsForPartPlacement(TLogContext &logCtx, TBlobState &state, const TBlobStorageGroupInfo &info, TGroupDiskRequests &groupDiskRequests, @@ -47,13 +47,12 @@ class TStrategyBase : public IStrategy { TBlobStorageGroupInfo::TSubgroupVDisks &inOutSuccess, TBlobStorageGroupInfo::TSubgroupVDisks &inOutError, bool &outIsDegraded); - void Prepare3dcPartPlacement(const TBlobState &state, size_t numFailRealms, size_t numFailDomainsPerFailRealm, - ui8 preferredReplicasPerRealm, bool considerSlowAsError, - TBlobStorageGroupType::TPartPlacement &outPartPlacement); + void Prepare3dcPartPlacement(const TBlobState& state, size_t numFailRealms, size_t numFailDomainsPerFailRealm, + ui8 preferredReplicasPerRealm, bool considerSlowAsError, bool replaceUnresponsive, + TBlobStorageGroupType::TPartPlacement& outPartPlacement, bool& fullPlacement); // Sets IsSlow for the slow disk, resets for other disks. - // Returns -1 if there is no slow disk, or subgroupIdx of the slow disk. - ui32 MakeSlowSubgroupDiskMask(TBlobState &state, const TBlobStorageGroupInfo &info, TBlackboard &blackboard, bool isPut); + // returns bit mask with 1 on positions of slow disks + ui32 MakeSlowSubgroupDiskMask(TBlobState &state, TBlackboard &blackboard, bool isPut, const TAccelerationParams& accelerationParams); }; - }//NKikimr diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_bold.h b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_bold.h index 5788df96cd0a..134df866a233 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_bold.h +++ b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_bold.h @@ -13,7 +13,9 @@ class TBoldStrategy : public TStrategyBase { }; EStrategyOutcome Process(TLogContext &logCtx, TBlobState &state, const TBlobStorageGroupInfo &info, - TBlackboard& /*blackboard*/, TGroupDiskRequests &groupDiskRequests) override { + TBlackboard& /*blackboard*/, TGroupDiskRequests &groupDiskRequests, + const TAccelerationParams& accelerationParams) override { + Y_UNUSED(accelerationParams); // Look at the current layout and set the status if possible const ui32 totalPartCount = info.Type.TotalPartCount(); bool doLook = true; diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_m3dc_basic.h b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_m3dc_basic.h index 1e4c9acfb584..c0e9c8178949 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_m3dc_basic.h +++ b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_m3dc_basic.h @@ -55,7 +55,8 @@ namespace NKikimr { } EStrategyOutcome Process(TLogContext& logCtx, TBlobState& state, const TBlobStorageGroupInfo& info, - TBlackboard &blackboard, TGroupDiskRequests& groupDiskRequests) override { + TBlackboard &blackboard, TGroupDiskRequests& groupDiskRequests, + const TAccelerationParams& accelerationParams) override { if (state.WholeSituation == TBlobState::ESituation::Present) { return EStrategyOutcome::DONE; } @@ -70,34 +71,14 @@ namespace NKikimr { // issue request for a specific disk; returns true if the request was issued and not yet completed, otherwise // false - if (info.GetTotalVDisksNum() > 1) { - // find the slowest disk and mark it - switch (blackboard.AccelerationMode) { - case TBlackboard::AccelerationModeSkipOneSlowest: { - TDiskDelayPredictions worstDisks; - state.GetWorstPredictedDelaysNs(info, *blackboard.GroupQueues, - HandleClassToQueueId(blackboard.GetHandleClass), 1, - &worstDisks); - - // Check if the slowest disk exceptionally slow, or just not very fast - i32 slowDiskSubgroupIdx = -1; - if (worstDisks[1].PredictedNs > 0 && worstDisks[0].PredictedNs > worstDisks[1].PredictedNs * 2) { - slowDiskSubgroupIdx = worstDisks[1].DiskIdx; - } - - // Mark single slow disk - for (size_t diskIdx = 0; diskIdx < state.Disks.size(); ++diskIdx) { - state.Disks[diskIdx].IsSlow = false; - } - if (slowDiskSubgroupIdx >= 0) { - state.Disks[slowDiskSubgroupIdx].IsSlow = true; - } - break; - } - case TBlackboard::AccelerationModeSkipMarked: - // The slowest disk is already marked! - break; - } + // mark slow disks + switch (blackboard.AccelerationMode) { + case TBlackboard::AccelerationModeSkipNSlowest: + blackboard.MarkSlowDisks(state, false, accelerationParams); + break; + case TBlackboard::AccelerationModeSkipMarked: + // Slow disks are already marked! + break; } // create an array defining order in which we traverse the disks diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_m3dc_restore.h b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_m3dc_restore.h index 84c11bc47474..9ed0256a41ed 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_m3dc_restore.h +++ b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_m3dc_restore.h @@ -14,7 +14,8 @@ namespace NKikimr { public: EStrategyOutcome Process(TLogContext& logCtx, TBlobState& state, const TBlobStorageGroupInfo& info, - TBlackboard& blackboard, TGroupDiskRequests& groupDiskRequests) override { + TBlackboard& blackboard, TGroupDiskRequests& groupDiskRequests, + const TAccelerationParams& accelerationParams) override { if (state.WholeSituation == TBlobState::ESituation::Present) { return EStrategyOutcome::DONE; } @@ -55,7 +56,7 @@ namespace NKikimr { state.Id.ToString().c_str(), ui32(state.WholeSituation)); state.WholeSituation = TBlobState::ESituation::Present; const EStrategyOutcome outcome = TPut3dcStrategy(TEvBlobStorage::TEvPut::TacticMaxThroughput, false).Process(logCtx, - state, info, blackboard, groupDiskRequests); + state, info, blackboard, groupDiskRequests, accelerationParams); switch (outcome) { case EStrategyOutcome::IN_PROGRESS: state.WholeSituation = TBlobState::ESituation::Unknown; diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_m3of4.h b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_m3of4.h index aec4efb8592a..4bb43f816ef9 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_m3of4.h +++ b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_m3of4.h @@ -10,7 +10,9 @@ namespace NKikimr { class TMirror3of4GetStrategy : public TMirror3of4StrategyBase { public: EStrategyOutcome Process(TLogContext& /*logCtx*/, TBlobState& state, const TBlobStorageGroupInfo& info, - TBlackboard& /*blackboard*/, TGroupDiskRequests& groupDiskRequests) override { + TBlackboard& /*blackboard*/, TGroupDiskRequests& groupDiskRequests, + const TAccelerationParams& accelerationParams) override { + Y_UNUSED(accelerationParams); if (!CheckFailModel(state, info)) { state.WholeSituation = TBlobState::ESituation::Error; return EStrategyOutcome::Error("failure model exceeded"); diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_min_iops_block.h b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_min_iops_block.h index 5829bd7f35b6..bf61ce78044f 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_min_iops_block.h +++ b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_min_iops_block.h @@ -288,7 +288,8 @@ class TMinIopsBlockStrategy : public TStrategyBase { } EStrategyOutcome Process(TLogContext &logCtx, TBlobState &state, const TBlobStorageGroupInfo &info, - TBlackboard& blackboard, TGroupDiskRequests &groupDiskRequests) override { + TBlackboard& blackboard, TGroupDiskRequests &groupDiskRequests, + const TAccelerationParams& accelerationParams) override { if (auto res = RestoreWholeFromDataParts(logCtx, state, info)) { return *res; } else if (auto res = RestoreWholeWithErasure(logCtx, state, info)) { @@ -311,7 +312,7 @@ class TMinIopsBlockStrategy : public TStrategyBase { // Try excluding the slow disk bool isDone = false; // TODO: Mark disk that does not answer when accelerating requests - ui32 slowDiskSubgroupMask = MakeSlowSubgroupDiskMask(state, info, blackboard, false); + ui32 slowDiskSubgroupMask = MakeSlowSubgroupDiskMask(state, blackboard, false, accelerationParams); if (slowDiskSubgroupMask >= 0) { TBlobStorageGroupInfo::EBlobState fastPessimisticState = TBlobStorageGroupInfo::EBS_DISINTEGRATED; TBlobStorageGroupInfo::EBlobState fastOptimisticState = TBlobStorageGroupInfo::EBS_DISINTEGRATED; diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_min_iops_mirror.h b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_min_iops_mirror.h index 38a7f0c9e78a..8da71d39ee9f 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_min_iops_mirror.h +++ b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_get_min_iops_mirror.h @@ -18,7 +18,9 @@ class TMinIopsMirrorStrategy : public TStrategyBase { } EStrategyOutcome Process(TLogContext &logCtx, TBlobState &state, const TBlobStorageGroupInfo &info, - TBlackboard& /*blackboard*/, TGroupDiskRequests &groupDiskRequests) override { + TBlackboard& /*blackboard*/, TGroupDiskRequests &groupDiskRequests, + const TAccelerationParams& accelerationParams) override { + Y_UNUSED(accelerationParams); if (auto res = RestoreWholeFromDataParts(logCtx, state, info)) { return *res; } diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_put_m3dc.h b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_put_m3dc.h index ed07074072fe..b7b52fe37422 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_put_m3dc.h +++ b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_put_m3dc.h @@ -31,18 +31,19 @@ class TPut3dcStrategy : public TStrategyBase { } EStrategyOutcome Process(TLogContext &logCtx, TBlobState &state, const TBlobStorageGroupInfo &info, - TBlackboard& blackboard, TGroupDiskRequests &groupDiskRequests) override { + TBlackboard& blackboard, TGroupDiskRequests &groupDiskRequests, + const TAccelerationParams& accelerationParams) override { TBlobStorageGroupType::TPartPlacement partPlacement; bool degraded = false; bool isDone = false; - ui32 slowDiskSubgroupMask = MakeSlowSubgroupDiskMask(state, info, blackboard, true); + ui32 slowDiskSubgroupMask = MakeSlowSubgroupDiskMask(state, blackboard, true, accelerationParams); do { if (slowDiskSubgroupMask == 0) { break; // ignore this case } TBlobStorageGroupInfo::TSubgroupVDisks success(&info.GetTopology()); TBlobStorageGroupInfo::TSubgroupVDisks error(&info.GetTopology()); - Evaluate3dcSituation(state, NumFailRealms, NumFailDomainsPerFailRealm, info, true, success, error, degraded); + Evaluate3dcSituation(state, NumFailRealms, NumFailDomainsPerFailRealm, info, false, success, error, degraded); TBlobStorageGroupInfo::TSubgroupVDisks slow = TBlobStorageGroupInfo::TSubgroupVDisks::CreateFromMask( &info.GetTopology(), slowDiskSubgroupMask); if ((success | error) & slow) { @@ -60,9 +61,7 @@ class TPut3dcStrategy : public TStrategyBase { // now check every realm and check if we have to issue some write requests to it Prepare3dcPartPlacement(state, NumFailRealms, NumFailDomainsPerFailRealm, - PreferredReplicasPerRealm(degraded), - true, partPlacement); - isDone = true; + PreferredReplicasPerRealm(degraded), true, false, partPlacement, isDone); } } while (false); if (!isDone) { @@ -80,9 +79,10 @@ class TPut3dcStrategy : public TStrategyBase { } // now check every realm and check if we have to issue some write requests to it + partPlacement.Records.clear(); + bool fullPlacement; Prepare3dcPartPlacement(state, NumFailRealms, NumFailDomainsPerFailRealm, - PreferredReplicasPerRealm(degraded), - false, partPlacement); + PreferredReplicasPerRealm(degraded), false, false, partPlacement, fullPlacement); } if (IsPutNeeded(state, partPlacement)) { PreparePutsForPartPlacement(logCtx, state, info, groupDiskRequests, partPlacement); diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_put_m3of4.h b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_put_m3of4.h index 3a3bfbefcd6b..457ae37f6a62 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_put_m3of4.h +++ b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_put_m3of4.h @@ -15,7 +15,9 @@ class TPut3of4Strategy : public TMirror3of4StrategyBase { {} EStrategyOutcome Process(TLogContext& /*logCtx*/, TBlobState& state, const TBlobStorageGroupInfo& info, - TBlackboard& /*blackboard*/, TGroupDiskRequests& groupDiskRequests) override { + TBlackboard& /*blackboard*/, TGroupDiskRequests& groupDiskRequests, + const TAccelerationParams& accelerationParams) override { + Y_UNUSED(accelerationParams); if (!CheckFailModel(state, info)) { state.WholeSituation = TBlobState::ESituation::Error; return EStrategyOutcome::Error("failure model exceeded"); diff --git a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_restore.h b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_restore.h index 5be8e482a663..c31ed1e71b0a 100644 --- a/ydb/core/blobstorage/dsproxy/dsproxy_strategy_restore.h +++ b/ydb/core/blobstorage/dsproxy/dsproxy_strategy_restore.h @@ -80,7 +80,8 @@ class TRestoreStrategy : public TStrategyBase { } EStrategyOutcome Process(TLogContext &logCtx, TBlobState &state, const TBlobStorageGroupInfo &info, - TBlackboard &blackboard, TGroupDiskRequests &groupDiskRequests) override { + TBlackboard &blackboard, TGroupDiskRequests &groupDiskRequests, + const TAccelerationParams& accelerationParams) override { // Check if the work is already done. if (state.WholeSituation == TBlobState::ESituation::Absent) { return EStrategyOutcome::DONE; // nothing to restore @@ -125,25 +126,13 @@ class TRestoreStrategy : public TStrategyBase { return *res; } - TStackVec slowDiskSubgroupIdxs; - if (info.GetTotalVDisksNum() > 1) { - // Find the slowest disk, if there are more than 1 - TDiskDelayPredictions worstDisks; - state.GetWorstPredictedDelaysNs(info, *blackboard.GroupQueues, - HandleClassToQueueId(blackboard.PutHandleClass), 1, - &worstDisks); - - // Check if the slowest disk exceptionally slow, or just not very fast - if (worstDisks[1].PredictedNs > 0 && worstDisks[0].PredictedNs > worstDisks[1].PredictedNs * 2) { - slowDiskSubgroupIdxs.push_back(worstDisks[0].DiskIdx); - } - } + ui32 slowDiskSubgroupMask = MakeSlowSubgroupDiskMask(state, blackboard, true, accelerationParams); bool isDone = false; - if (!slowDiskSubgroupIdxs.empty()) { + if (slowDiskSubgroupMask != 0) { // If there is an exceptionally slow disk, try not touching it, mark isDone TBlobStorageGroupType::TPartLayout layout; - PreparePartLayout(state, info, &layout, slowDiskSubgroupIdxs); + PreparePartLayout(state, info, &layout, slowDiskSubgroupMask); TBlobStorageGroupType::TPartPlacement partPlacement; bool isCorrectable = info.Type.CorrectLayout(layout, partPlacement); @@ -157,7 +146,7 @@ class TRestoreStrategy : public TStrategyBase { if (!isDone) { // Fill in the part layout TBlobStorageGroupType::TPartLayout layout; - PreparePartLayout(state, info, &layout, {}); + PreparePartLayout(state, info, &layout, 0); TBlobStorageGroupType::TPartPlacement partPlacement; bool isCorrectable = info.Type.CorrectLayout(layout, partPlacement); Y_ABORT_UNLESS(isCorrectable); diff --git a/ydb/core/blobstorage/dsproxy/group_sessions.cpp b/ydb/core/blobstorage/dsproxy/group_sessions.cpp index 2623218cadd3..8569a94ad948 100644 --- a/ydb/core/blobstorage/dsproxy/group_sessions.cpp +++ b/ydb/core/blobstorage/dsproxy/group_sessions.cpp @@ -28,7 +28,7 @@ TString QueueIdName(NKikimrBlobStorage::EVDiskQueueId queueId) { } TGroupSessions::TGroupSessions(const TIntrusivePtr& info, const TBSProxyContextPtr& bspctx, - const TActorId& monActor, const TActorId& proxyActor) + const TActorId& monActor, const TActorId& proxyActor, bool useActorSystemTimeInBSQueue) : GroupQueues(MakeIntrusive(info->GetTopology())) , ConnectedQueuesMask(info->GetTotalVDisksNum(), 0) , MonActor(monActor) @@ -71,7 +71,7 @@ TGroupSessions::TGroupSessions(const TIntrusivePtr& info, std::unique_ptr queueActor(CreateVDiskBackpressureClient(info, vd, queueId, counters, bspctx, NBackpressure::TQueueClientId(NBackpressure::EQueueClientType::DSProxy, nodeId), QueueIdName(queueId), interconnectChannel, nodeId == targetNodeId, TDuration::Minutes(1), flowRecord, - NMonitoring::TCountableBase::EVisibility::Public)); + NMonitoring::TCountableBase::EVisibility::Public, useActorSystemTimeInBSQueue)); TActorId queue = TActivationContext::Register(queueActor.release(), ProxyActor, TMailboxType::ReadAsFilled, AppData()->SystemPoolId); diff --git a/ydb/core/blobstorage/dsproxy/group_sessions.h b/ydb/core/blobstorage/dsproxy/group_sessions.h index 1cd731698304..16d9b804421d 100644 --- a/ydb/core/blobstorage/dsproxy/group_sessions.h +++ b/ydb/core/blobstorage/dsproxy/group_sessions.h @@ -242,7 +242,7 @@ namespace NKikimr { TActorId ProxyActor; TGroupSessions(const TIntrusivePtr& info, const TBSProxyContextPtr& bspctx, - const TActorId& monActor, const TActorId& proxyActor); + const TActorId& monActor, const TActorId& proxyActor, bool useActorSystemTimeInBSQueue); void Poison(); bool GoodToGo(const TBlobStorageGroupInfo::TTopology& topology, bool waitForAllVDisks); void QueueConnectUpdate(ui32 orderNumber, NKikimrBlobStorage::EVDiskQueueId queueId, bool connected, @@ -262,4 +262,49 @@ namespace NKikimr { {} }; + struct TEvGetQueuesInfo : public TEventLocal { + NKikimrBlobStorage::EVDiskQueueId QueueId; + + TEvGetQueuesInfo(NKikimrBlobStorage::EVDiskQueueId queueId) + : QueueId(queueId) + {} + }; + + struct TEvQueuesInfo : public TEventLocal { + struct TQueueInfo { + TActorId ActorId; + TIntrusivePtr FlowRecord; + }; + + TEvQueuesInfo(ui32 groupSize) { + Queues.resize(groupSize); + } + + void AddInfoForQueue(ui32 orderNumber, TActorId actorId, const TIntrusivePtr& flowRecord) { + Queues[orderNumber].emplace(TQueueInfo{ + .ActorId = actorId, + .FlowRecord = flowRecord + }); + } + + TString ToString() const override { + TStringStream str; + str << "{ TEvQueuesInfo"; + str << " Queues ["; + for (ui32 orderNum = 0; orderNum < Queues.size(); ++orderNum) { + const std::optional& queue = Queues[orderNum]; + if (queue) { + str << " { OrderNumber# " << orderNum + << " ActorId# " << queue->ActorId.ToString() << " },"; + } else { + str << " {}"; + } + } + str << " ] }"; + return str.Str(); + } + + TStackVec, TypicalDisksInGroup> Queues; + }; + } // NKikimr diff --git a/ydb/core/blobstorage/dsproxy/ut/dsproxy_env_mock_ut.h b/ydb/core/blobstorage/dsproxy/ut/dsproxy_env_mock_ut.h index c74ea2d3c32e..74fcfe01ca57 100644 --- a/ydb/core/blobstorage/dsproxy/ut/dsproxy_env_mock_ut.h +++ b/ydb/core/blobstorage/dsproxy/ut/dsproxy_env_mock_ut.h @@ -82,7 +82,13 @@ struct TDSProxyEnv { TControlWrapper enablePutBatching(DefaultEnablePutBatching, false, true); TControlWrapper enableVPatch(DefaultEnableVPatch, false, true); IActor *dsproxy = CreateBlobStorageGroupProxyConfigured(TIntrusivePtr(Info), true, nodeMon, - std::move(storagePoolCounters), enablePutBatching, enableVPatch); + std::move(storagePoolCounters), TBlobStorageProxyParameters{ + .Controls = TBlobStorageProxyControlWrappers{ + .EnablePutBatching = enablePutBatching, + .EnableVPatch = enableVPatch, + } + } + ); TActorId actorId = runtime.Register(dsproxy, nodeIndex); runtime.RegisterService(RealProxyActorId, actorId, nodeIndex); @@ -103,33 +109,91 @@ struct TDSProxyEnv { std::unique_ptr CreatePutRequestActor(TEvBlobStorage::TEvPut::TPtr &ev) { TMaybe kind = PutHandleClassToGroupStatKind(ev->Get()->HandleClass); - return std::unique_ptr(CreateBlobStorageGroupPutRequest(Info, GroupQueues, ev->Sender, Mon, ev->Get(), - ev->Cookie, std::move(ev->TraceId), Mon->TimeStats.IsEnabled(), PerDiskStatsPtr, kind, - TInstant::Now(), StoragePoolCounters, false)); + return std::unique_ptr(CreateBlobStorageGroupPutRequest( + TBlobStorageGroupPutParameters{ + .Common = { + .GroupInfo = Info, + .GroupQueues = GroupQueues, + .Mon = Mon, + .Source = ev->Sender, + .Cookie = ev->Cookie, + .Now = TInstant::Now(), + .StoragePoolCounters = StoragePoolCounters, + .RestartCounter = ev->Get()->RestartCounter, + .Event = ev->Get(), + .ExecutionRelay = ev->Get()->ExecutionRelay, + .LatencyQueueKind = kind, + }, + .TimeStatsEnabled = Mon->TimeStats.IsEnabled(), + .Stats = PerDiskStatsPtr, + .EnableRequestMod3x3ForMinLatency = false, + }, std::move(ev->TraceId))); } std::unique_ptr CreatePutRequestActor(TBatchedVec &batched, TEvBlobStorage::TEvPut::ETactic tactic, NKikimrBlobStorage::EPutHandleClass handleClass) { TMaybe kind = PutHandleClassToGroupStatKind(handleClass); - return std::unique_ptr(CreateBlobStorageGroupPutRequest(Info, GroupQueues, - Mon, batched, Mon->TimeStats.IsEnabled(), PerDiskStatsPtr, kind,TInstant::Now(), - StoragePoolCounters, handleClass, tactic, false)); + return std::unique_ptr(CreateBlobStorageGroupPutRequest( + TBlobStorageGroupMultiPutParameters{ + .Common = { + .GroupInfo = Info, + .GroupQueues = GroupQueues, + .Mon = Mon, + .Now = TInstant::Now(), + .StoragePoolCounters = StoragePoolCounters, + .RestartCounter = TBlobStorageGroupMultiPutParameters::CalculateRestartCounter(batched), + .LatencyQueueKind = kind, + }, + .Events = batched, + .TimeStatsEnabled = Mon->TimeStats.IsEnabled(), + .Stats = PerDiskStatsPtr, + .HandleClass = handleClass, + .Tactic = tactic, + .EnableRequestMod3x3ForMinLatency = false, + })); } std::unique_ptr CreateGetRequestActor(TEvBlobStorage::TEvGet::TPtr &ev, NKikimrBlobStorage::EPutHandleClass handleClass) { TMaybe kind = PutHandleClassToGroupStatKind(handleClass); - return std::unique_ptr(CreateBlobStorageGroupGetRequest(Info, GroupQueues, ev->Sender, Mon, - ev->Get(), ev->Cookie, std::move(ev->TraceId), TNodeLayoutInfoPtr(NodeLayoutInfo), - kind, TInstant::Now(), StoragePoolCounters)); + return std::unique_ptr(CreateBlobStorageGroupGetRequest( + TBlobStorageGroupGetParameters{ + .Common = { + .GroupInfo = Info, + .GroupQueues = GroupQueues, + .Mon = Mon, + .Source = ev->Sender, + .Cookie = ev->Cookie, + .Now = TInstant::Now(), + .StoragePoolCounters = StoragePoolCounters, + .RestartCounter = ev->Get()->RestartCounter, + .Event = ev->Get(), + .ExecutionRelay = ev->Get()->ExecutionRelay, + .LatencyQueueKind = kind, + }, + .NodeLayout = TNodeLayoutInfoPtr(NodeLayoutInfo) + }, std::move(ev->TraceId))); } std::unique_ptr CreatePatchRequestActor(TEvBlobStorage::TEvPatch::TPtr &ev, bool useVPatch = false) { - return std::unique_ptr(CreateBlobStorageGroupPatchRequest(Info, GroupQueues, ev->Sender, Mon, - ev->Get(), ev->Cookie, std::move(ev->TraceId), TInstant::Now(), StoragePoolCounters, - useVPatch)); + return std::unique_ptr(CreateBlobStorageGroupPatchRequest( + TBlobStorageGroupPatchParameters{ + .Common = { + .GroupInfo = Info, + .GroupQueues = GroupQueues, + .Mon = Mon, + .Source = ev->Sender, + .Cookie = ev->Cookie, + .Now = TInstant::Now(), + .StoragePoolCounters = StoragePoolCounters, + .RestartCounter = ev->Get()->RestartCounter, + .Event = ev->Get(), + .ExecutionRelay = ev->Get()->ExecutionRelay + }, + .UseVPatch = useVPatch + }, std::move(ev->TraceId))); } }; diff --git a/ydb/core/blobstorage/dsproxy/ut/dsproxy_get_ut.cpp b/ydb/core/blobstorage/dsproxy/ut/dsproxy_get_ut.cpp index 730ef996b5ff..adf673bff63e 100644 --- a/ydb/core/blobstorage/dsproxy/ut/dsproxy_get_ut.cpp +++ b/ydb/core/blobstorage/dsproxy/ut/dsproxy_get_ut.cpp @@ -71,7 +71,7 @@ void TestIntervalsAndCrcAllOk(TErasureType::EErasureSpecies erasureSpecies, bool TEvBlobStorage::TEvGet ev(queriesA, queryCount, TInstant::Max(), NKikimrBlobStorage::EGetHandleClass::FastRead, false, false); ev.IsVerboseNoDataEnabled = isVerboseNoDataEnabled; - TGetImpl getImpl(group.GetInfo(), groupQueues, &ev, nullptr); + TGetImpl getImpl(group.GetInfo(), groupQueues, &ev, nullptr, TAccelerationParams{}); TDeque> vGets; TLogContext logCtx(NKikimrServices::BS_PROXY_GET, false); logCtx.LogAcc.IsLogEnabled = false; @@ -324,7 +324,7 @@ class TTestWipedAllOkStep { TEvBlobStorage::TEvGet ev(queriesA, queryCount, TInstant::Max(), NKikimrBlobStorage::EGetHandleClass::FastRead, IsRestore, false); ev.IsVerboseNoDataEnabled = IsVerboseNoDataEnabled; - TGetImpl getImpl(Group->GetInfo(), GroupQueues, &ev, nullptr); + TGetImpl getImpl(Group->GetInfo(), GroupQueues, &ev, nullptr, TAccelerationParams{}); ClearCounters(); TDeque> vGets; TDeque> vPuts; @@ -452,7 +452,7 @@ class TGetSimulator { TAutoPtr Simulate(TEvBlobStorage::TEvGet *ev) { TAutoPtr getResult; - TGetImpl getImpl(Group.GetInfo(), GroupQueues, ev, nullptr); + TGetImpl getImpl(Group.GetInfo(), GroupQueues, ev, nullptr, TAccelerationParams{}); TDeque> vGets; TDeque> vPuts; TLogContext logCtx(NKikimrServices::BS_PROXY_GET, false); @@ -563,7 +563,7 @@ Y_UNIT_TEST(TestBlock42VGetCountWithErasure) { TAutoPtr getResult; - TGetImpl getImpl(group.GetInfo(), groupQueues, &ev, nullptr); + TGetImpl getImpl(group.GetInfo(), groupQueues, &ev, nullptr, TAccelerationParams{}); TDeque> vGets; TDeque> vPuts; TLogContext logCtx(NKikimrServices::BS_PROXY_GET, false); @@ -703,7 +703,7 @@ Y_UNIT_TEST(TestBlock42WipedOneDiskAndErrorDurringGet) { TAutoPtr getResult; - TGetImpl getImpl(group.GetInfo(), groupQueues, &ev, nullptr); + TGetImpl getImpl(group.GetInfo(), groupQueues, &ev, nullptr, TAccelerationParams{}); TDeque> vGets; TDeque> vPuts; TLogContext logCtx(NKikimrServices::BS_PROXY_GET, false); @@ -974,7 +974,7 @@ void TestWipedErrorWithTwoBlobs(TErasureType::EErasureSpecies erasureSpecies, bo TAutoPtr getResult; - TGetImpl getImpl(group.GetInfo(), groupQueues, &ev, nullptr); + TGetImpl getImpl(group.GetInfo(), groupQueues, &ev, nullptr, TAccelerationParams{}); TDeque> vGets; TDeque> vPuts; TLogContext logCtx(NKikimrServices::BS_PROXY_GET, false); @@ -1192,7 +1192,7 @@ class TTestPossibleBlobLost { TEvBlobStorage::TEvGet ev(queriesA, MaxQueryCount, TInstant::Max(), NKikimrBlobStorage::EGetHandleClass::Discover, true, false); ev.IsVerboseNoDataEnabled = false; - TGetImpl getImpl(Group.GetInfo(), groupQueues, &ev, nullptr); + TGetImpl getImpl(Group.GetInfo(), groupQueues, &ev, nullptr, TAccelerationParams{}); TDeque> vGets; getImpl.GenerateInitialRequests(logCtx, vGets); return vGets.size(); @@ -1257,7 +1257,7 @@ class TTestPossibleBlobLost { TEvBlobStorage::TEvGet ev(queriesA, MaxQueryCount, TInstant::Max(), NKikimrBlobStorage::EGetHandleClass::Discover, true, false); ev.IsVerboseNoDataEnabled = false; - TGetImpl getImpl(Group.GetInfo(), groupQueues, &ev, nullptr); + TGetImpl getImpl(Group.GetInfo(), groupQueues, &ev, nullptr, TAccelerationParams{}); TDeque> vGets; TDeque> vPuts; getImpl.GenerateInitialRequests(logCtx, vGets); @@ -1404,7 +1404,7 @@ class TTestNoDataRegression { TEvBlobStorage::TEvGet ev(queriesA, MaxQueryCount, TInstant::Max(), NKikimrBlobStorage::EGetHandleClass::Discover, true, false); ev.IsVerboseNoDataEnabled = false; - TGetImpl getImpl(Group.GetInfo(), groupQueues, &ev, nullptr); + TGetImpl getImpl(Group.GetInfo(), groupQueues, &ev, nullptr, TAccelerationParams{}); TDeque> vGets; getImpl.GenerateInitialRequests(logCtx, vGets); return vGets.size(); @@ -1469,7 +1469,7 @@ class TTestNoDataRegression { TEvBlobStorage::TEvGet ev(queriesA, MaxQueryCount, TInstant::Max(), NKikimrBlobStorage::EGetHandleClass::Discover, true, false); ev.IsVerboseNoDataEnabled = false; - TGetImpl getImpl(Group.GetInfo(), groupQueues, &ev, nullptr); + TGetImpl getImpl(Group.GetInfo(), groupQueues, &ev, nullptr, TAccelerationParams{}); TDeque> vGets; TDeque> vPuts; getImpl.GenerateInitialRequests(logCtx, vGets); diff --git a/ydb/core/blobstorage/dsproxy/ut/dsproxy_put_ut.cpp b/ydb/core/blobstorage/dsproxy/ut/dsproxy_put_ut.cpp index 2af3baccace5..fc6390a32ca4 100644 --- a/ydb/core/blobstorage/dsproxy/ut/dsproxy_put_ut.cpp +++ b/ydb/core/blobstorage/dsproxy/ut/dsproxy_put_ut.cpp @@ -66,7 +66,7 @@ void TestPutMaxPartCountOnHandoff(TErasureType::EErasureSpecies erasureSpecies) TEvBlobStorage::TEvPut ev(blobId, data, TInstant::Max(), NKikimrBlobStorage::TabletLog, TEvBlobStorage::TEvPut::TacticDefault); - TPutImpl putImpl(group.GetInfo(), groupQueues, &ev, mon, false, TActorId(), 0, NWilson::TTraceId()); + TPutImpl putImpl(group.GetInfo(), groupQueues, &ev, mon, false, TActorId(), 0, NWilson::TTraceId(), TAccelerationParams{}); for (ui32 idx = 0; idx < domainCount; ++idx) { group.SetPredictedDelayNs(idx, 1); @@ -302,10 +302,11 @@ struct TTestPutAllOk { TMaybe putImpl; TPutImpl::TPutResultVec putResults; if constexpr (IsVPut) { - putImpl.ConstructInPlace(Group.GetInfo(), GroupQueues, events[0]->Get(), Mon, false, TActorId(), 0, NWilson::TTraceId()); + putImpl.ConstructInPlace(Group.GetInfo(), GroupQueues, events[0]->Get(), Mon, false, TActorId(), 0, NWilson::TTraceId(), + TAccelerationParams{}); } else { putImpl.ConstructInPlace(Group.GetInfo(), GroupQueues, events, Mon, - NKikimrBlobStorage::TabletLog, TEvBlobStorage::TEvPut::TacticDefault, false); + NKikimrBlobStorage::TabletLog, TEvBlobStorage::TEvPut::TacticDefault, false, TAccelerationParams{}); } putImpl->GenerateInitialRequests(LogCtx, PartSets); @@ -352,7 +353,7 @@ Y_UNIT_TEST(TestMirror3dcWith3x3MinLatencyMod) { TString data = AlphaData(size); TEvBlobStorage::TEvPut ev(blobId, data, TInstant::Max(), NKikimrBlobStorage::TabletLog, TEvBlobStorage::TEvPut::TacticMinLatency); - TPutImpl putImpl(env.Info, env.GroupQueues, &ev, env.Mon, true, TActorId(), 0, NWilson::TTraceId()); + TPutImpl putImpl(env.Info, env.GroupQueues, &ev, env.Mon, true, TActorId(), 0, NWilson::TTraceId(), TAccelerationParams{}); TLogContext logCtx(NKikimrServices::BS_PROXY_PUT, false); logCtx.LogAcc.IsLogEnabled = false; diff --git a/ydb/core/blobstorage/dsproxy/ut_fat/dsproxy_ut.cpp b/ydb/core/blobstorage/dsproxy/ut_fat/dsproxy_ut.cpp index c8e0d01a41b9..9b4730164dc5 100644 --- a/ydb/core/blobstorage/dsproxy/ut_fat/dsproxy_ut.cpp +++ b/ydb/core/blobstorage/dsproxy/ut_fat/dsproxy_ut.cpp @@ -3403,9 +3403,25 @@ class TTestBlobStorageProxyBatchedPutRequestDoesNotContainAHugeBlob : public TTe batched[1] = GetPut(blobIds[1], Data2); TMaybe kind = PutHandleClassToGroupStatKind(HandleClass); - IActor *reqActor = CreateBlobStorageGroupPutRequest(BsInfo, GroupQueues, - Mon, batched, false, PerDiskStatsPtr, kind,TInstant::Now(), - StoragePoolCounters, HandleClass, Tactic, false); + IActor *reqActor = CreateBlobStorageGroupPutRequest( + TBlobStorageGroupMultiPutParameters{ + .Common = { + .GroupInfo = BsInfo, + .GroupQueues = GroupQueues, + .Mon = Mon, + .Now = TInstant::Now(), + .StoragePoolCounters = StoragePoolCounters, + .RestartCounter = TBlobStorageGroupMultiPutParameters::CalculateRestartCounter(batched), + .LatencyQueueKind = kind, + }, + .Events = batched, + .TimeStatsEnabled = false, + .Stats = PerDiskStatsPtr, + .HandleClass = HandleClass, + .Tactic = Tactic, + .EnableRequestMod3x3ForMinLatency = false, + .AccelerationParams = TAccelerationParams{}, + }); ctx.Register(reqActor); break; @@ -4188,8 +4204,18 @@ class TBlobStorageProxyTest: public TTestBase { TIntrusivePtr dsProxyNodeMon(new TDsProxyNodeMon(counters, true)); TDsProxyPerPoolCounters perPoolCounters(counters); TIntrusivePtr storagePoolCounters = perPoolCounters.GetPoolCounters("pool_name"); + TControlWrapper enablePutBatching(args.EnablePutBatching, false, true); + TControlWrapper enableVPatch(DefaultEnableVPatch, false, true); std::unique_ptr proxyActor{CreateBlobStorageGroupProxyConfigured(TIntrusivePtr(bsInfo), false, - dsProxyNodeMon, TIntrusivePtr(storagePoolCounters), args.EnablePutBatching, DefaultEnableVPatch)}; + dsProxyNodeMon, TIntrusivePtr(storagePoolCounters), + TBlobStorageProxyParameters{ + .Controls = TBlobStorageProxyControlWrappers{ + .EnablePutBatching = enablePutBatching, + .EnableVPatch = enableVPatch, + } + } + ) + }; TActorSetupCmd bsproxySetup(proxyActor.release(), TMailboxType::Revolving, 3); setup1->LocalServices.push_back(std::pair(env->ProxyId, std::move(bsproxySetup))); diff --git a/ydb/core/blobstorage/dsproxy/ut_ftol/dsproxy_fault_tolerance_ut_runtime.h b/ydb/core/blobstorage/dsproxy/ut_ftol/dsproxy_fault_tolerance_ut_runtime.h index d6220c72972a..b724bd073f3f 100644 --- a/ydb/core/blobstorage/dsproxy/ut_ftol/dsproxy_fault_tolerance_ut_runtime.h +++ b/ydb/core/blobstorage/dsproxy/ut_ftol/dsproxy_fault_tolerance_ut_runtime.h @@ -88,7 +88,13 @@ class TFaultToleranceTestRuntime { TControlWrapper enablePutBatching(DefaultEnablePutBatching, false, true); TControlWrapper enableVPatch(DefaultEnableVPatch, false, true); IActor *dsproxy = CreateBlobStorageGroupProxyConfigured(TIntrusivePtr(GroupInfo), false, nodeMon, - std::move(storagePoolCounters), enablePutBatching, enableVPatch); + std::move(storagePoolCounters), TBlobStorageProxyParameters{ + .Controls = TBlobStorageProxyControlWrappers{ + .EnablePutBatching = enablePutBatching, + .EnableVPatch = enableVPatch, + } + } + ); setup->LocalServices.emplace_back(MakeBlobStorageProxyID(GroupInfo->GroupID), TActorSetupCmd(dsproxy, TMailboxType::Simple, 0)); diff --git a/ydb/core/blobstorage/dsproxy/ut_strategy/strategy_ut.cpp b/ydb/core/blobstorage/dsproxy/ut_strategy/strategy_ut.cpp index 64ace509dd63..8b69da407acc 100644 --- a/ydb/core/blobstorage/dsproxy/ut_strategy/strategy_ut.cpp +++ b/ydb/core/blobstorage/dsproxy/ut_strategy/strategy_ut.cpp @@ -165,7 +165,7 @@ void RunStrategyTest(TBlobStorageGroupType type) { TString state = blackboard[id].ToString(); - auto outcome = blackboard.RunStrategy(logCtx, strategy); + auto outcome = blackboard.RunStrategy(logCtx, strategy, TAccelerationParams{}); TString nextState = blackboard[id].ToString(); if (const auto [it, inserted] = transitions.try_emplace(state, std::make_tuple(outcome, nextState)); !inserted) { @@ -173,7 +173,7 @@ void RunStrategyTest(TBlobStorageGroupType type) { } if (outcome == EStrategyOutcome::IN_PROGRESS) { - auto temp = blackboard.RunStrategy(logCtx, strategy); + auto temp = blackboard.RunStrategy(logCtx, strategy, TAccelerationParams{}); UNIT_ASSERT_EQUAL(temp, outcome); UNIT_ASSERT_VALUES_EQUAL(blackboard[id].ToString(), nextState); } @@ -328,7 +328,7 @@ Y_UNIT_TEST_SUITE(DSProxyStrategyTest) { logCtx.SuppressLog = true; auto runStrategies = [&](TBlackboard& blackboard) { - return blackboard.RunStrategy(logCtx, TMirror3dcGetWithRestoreStrategy()); + return blackboard.RunStrategy(logCtx, TMirror3dcGetWithRestoreStrategy(), TAccelerationParams{}); }; const ui32 base = RandomNumber(512u); diff --git a/ydb/core/blobstorage/dsproxy/ya.make b/ydb/core/blobstorage/dsproxy/ya.make index c4096cbdd681..0dd9503a727b 100644 --- a/ydb/core/blobstorage/dsproxy/ya.make +++ b/ydb/core/blobstorage/dsproxy/ya.make @@ -62,6 +62,7 @@ PEERDIR( ydb/core/base ydb/core/blobstorage/backpressure ydb/core/blobstorage/base + ydb/core/blobstorage/common ydb/core/blobstorage/groupinfo ydb/core/blobstorage/storagepoolmon ydb/core/blobstorage/vdisk/ingress diff --git a/ydb/core/blobstorage/nodewarden/node_warden.h b/ydb/core/blobstorage/nodewarden/node_warden.h index 561a8fdc0348..24509a0f0300 100644 --- a/ydb/core/blobstorage/nodewarden/node_warden.h +++ b/ydb/core/blobstorage/nodewarden/node_warden.h @@ -37,6 +37,7 @@ namespace NKikimr { // debugging options bool VDiskReplPausedAtStart = false; + bool UseActorSystemTimeInBSQueue = false; TNodeWardenConfig(const TIntrusivePtr &pDiskServiceFactory) : PDiskServiceFactory(pDiskServiceFactory) diff --git a/ydb/core/blobstorage/nodewarden/node_warden_impl.cpp b/ydb/core/blobstorage/nodewarden/node_warden_impl.cpp index f969813d5b57..024639e4ab7d 100644 --- a/ydb/core/blobstorage/nodewarden/node_warden_impl.cpp +++ b/ydb/core/blobstorage/nodewarden/node_warden_impl.cpp @@ -1,5 +1,6 @@ #include "node_warden_impl.h" +#include #include #include #include @@ -197,6 +198,18 @@ void TNodeWarden::Bootstrap() { "VDiskControls.DiskTimeAvailableScaleSSD"); icb->RegisterSharedControl(CostMetricsParametersByMedia[NPDisk::DEVICE_TYPE_NVME].DiskTimeAvailableScale, "VDiskControls.DiskTimeAvailableScaleNVME"); + + icb->RegisterSharedControl(SlowDiskThreshold, "DSProxyControls.SlowDiskThreshold"); + icb->RegisterSharedControl(SlowDiskThresholdHDD, "DSProxyControls.SlowDiskThresholdHDD"); + icb->RegisterSharedControl(SlowDiskThresholdSSD, "DSProxyControls.SlowDiskThresholdSSD"); + + icb->RegisterSharedControl(PredictedDelayMultiplier, "DSProxyControls.PredictedDelayMultiplier"); + icb->RegisterSharedControl(PredictedDelayMultiplierHDD, "DSProxyControls.PredictedDelayMultiplierHDD"); + icb->RegisterSharedControl(PredictedDelayMultiplierSSD, "DSProxyControls.PredictedDelayMultiplierSSD"); + + icb->RegisterSharedControl(MaxNumOfSlowDisks, "DSProxyControls.MaxNumOfSlowDisks"); + icb->RegisterSharedControl(MaxNumOfSlowDisksHDD, "DSProxyControls.MaxNumOfSlowDisksHDD"); + icb->RegisterSharedControl(MaxNumOfSlowDisksSSD, "DSProxyControls.MaxNumOfSlowDisksSSD"); } // start replication broker diff --git a/ydb/core/blobstorage/nodewarden/node_warden_impl.h b/ydb/core/blobstorage/nodewarden/node_warden_impl.h index 3a141f3b113c..0388879d87f2 100644 --- a/ydb/core/blobstorage/nodewarden/node_warden_impl.h +++ b/ydb/core/blobstorage/nodewarden/node_warden_impl.h @@ -146,6 +146,18 @@ namespace NKikimr::NStorage { TCostMetricsParametersByMedia CostMetricsParametersByMedia; + TControlWrapper SlowDiskThreshold; + TControlWrapper SlowDiskThresholdHDD; + TControlWrapper SlowDiskThresholdSSD; + + TControlWrapper PredictedDelayMultiplier; + TControlWrapper PredictedDelayMultiplierHDD; + TControlWrapper PredictedDelayMultiplierSSD; + + TControlWrapper MaxNumOfSlowDisks; + TControlWrapper MaxNumOfSlowDisksHDD; + TControlWrapper MaxNumOfSlowDisksSSD; + public: struct TGroupRecord; @@ -169,6 +181,15 @@ namespace NKikimr::NStorage { TCostMetricsParameters{50}, TCostMetricsParameters{32}, }) + , SlowDiskThreshold(std::round(DefaultSlowDiskThreshold * 1000), 1, 1'000'000) + , SlowDiskThresholdHDD(std::round(DefaultSlowDiskThreshold * 1000), 1, 1'000'000) + , SlowDiskThresholdSSD(std::round(DefaultSlowDiskThreshold * 1000), 1, 1'000'000) + , PredictedDelayMultiplier(std::round(DefaultPredictedDelayMultiplier * 1000), 0, 1'000'000) + , PredictedDelayMultiplierHDD(std::round(DefaultPredictedDelayMultiplier * 1000), 0, 1'000'000) + , PredictedDelayMultiplierSSD(std::round(DefaultPredictedDelayMultiplier * 1000), 0, 1'000'000) + , MaxNumOfSlowDisks(DefaultMaxNumOfSlowDisks, 1, 2) + , MaxNumOfSlowDisksHDD(DefaultMaxNumOfSlowDisks, 1, 2) + , MaxNumOfSlowDisksSSD(DefaultMaxNumOfSlowDisks, 1, 2) { Y_ABORT_UNLESS(Cfg->BlobStorageConfig.GetServiceSet().AvailabilityDomainsSize() <= 1); AvailDomainId = 1; diff --git a/ydb/core/blobstorage/nodewarden/node_warden_proxy.cpp b/ydb/core/blobstorage/nodewarden/node_warden_proxy.cpp index 599bb5861c84..6ef49e31ea83 100644 --- a/ydb/core/blobstorage/nodewarden/node_warden_proxy.cpp +++ b/ydb/core/blobstorage/nodewarden/node_warden_proxy.cpp @@ -8,6 +8,11 @@ TActorId TNodeWarden::StartEjectedProxy(ui32 groupId) { return Register(CreateBlobStorageGroupEjectedProxy(groupId, DsProxyNodeMon), TMailboxType::ReadAsFilled, AppData()->SystemPoolId); } +#define ADD_CONTROLS_FOR_DEVICE_TYPES(prefix) \ + .prefix = prefix, \ + .prefix##HDD = prefix##HDD, \ + .prefix##SSD = prefix##SSD + void TNodeWarden::StartLocalProxy(ui32 groupId) { STLOG(PRI_DEBUG, BS_NODE, NW12, "StartLocalProxy", (GroupId, groupId)); @@ -35,9 +40,17 @@ void TNodeWarden::StartLocalProxy(ui32 groupId) { case NKikimrBlobStorage::TGroupDecommitStatus::IN_PROGRESS: // create proxy that will be used by blob depot agent to fetch underlying data proxyActorId = as->Register(CreateBlobStorageGroupProxyConfigured( - TIntrusivePtr(info), false, DsProxyNodeMon, - getCounters(info), EnablePutBatching, EnableVPatch), TMailboxType::ReadAsFilled, - AppData()->SystemPoolId); + TIntrusivePtr(info), false, DsProxyNodeMon, getCounters(info), + TBlobStorageProxyParameters{ + .UseActorSystemTimeInBSQueue = Cfg->UseActorSystemTimeInBSQueue, + .Controls = TBlobStorageProxyControlWrappers{ + .EnablePutBatching = EnablePutBatching, + .EnableVPatch = EnableVPatch, + ADD_CONTROLS_FOR_DEVICE_TYPES(SlowDiskThreshold), + ADD_CONTROLS_FOR_DEVICE_TYPES(PredictedDelayMultiplier), + ADD_CONTROLS_FOR_DEVICE_TYPES(MaxNumOfSlowDisks), + } + }), TMailboxType::ReadAsFilled, AppData()->SystemPoolId); [[fallthrough]]; case NKikimrBlobStorage::TGroupDecommitStatus::DONE: proxy.reset(NBlobDepot::CreateBlobDepotAgent(groupId, info, proxyActorId)); @@ -50,12 +63,32 @@ void TNodeWarden::StartLocalProxy(ui32 groupId) { } } else { // create proxy with configuration - proxy.reset(CreateBlobStorageGroupProxyConfigured(TIntrusivePtr(info), false, DsProxyNodeMon, getCounters(info), - EnablePutBatching, EnableVPatch)); + proxy.reset(CreateBlobStorageGroupProxyConfigured(TIntrusivePtr(info), false, + DsProxyNodeMon, getCounters(info), TBlobStorageProxyParameters{ + .UseActorSystemTimeInBSQueue = Cfg->UseActorSystemTimeInBSQueue, + .Controls = TBlobStorageProxyControlWrappers{ + .EnablePutBatching = EnablePutBatching, + .EnableVPatch = EnableVPatch, + ADD_CONTROLS_FOR_DEVICE_TYPES(SlowDiskThreshold), + ADD_CONTROLS_FOR_DEVICE_TYPES(PredictedDelayMultiplier), + ADD_CONTROLS_FOR_DEVICE_TYPES(MaxNumOfSlowDisks), + } + } + ) + ); } } else { // create proxy without configuration - proxy.reset(CreateBlobStorageGroupProxyUnconfigured(groupId, DsProxyNodeMon, EnablePutBatching, EnableVPatch)); + proxy.reset(CreateBlobStorageGroupProxyUnconfigured(groupId, DsProxyNodeMon, TBlobStorageProxyParameters{ + .UseActorSystemTimeInBSQueue = Cfg->UseActorSystemTimeInBSQueue, + .Controls = TBlobStorageProxyControlWrappers{ + .EnablePutBatching = EnablePutBatching, + .EnableVPatch = EnableVPatch, + ADD_CONTROLS_FOR_DEVICE_TYPES(SlowDiskThreshold), + ADD_CONTROLS_FOR_DEVICE_TYPES(PredictedDelayMultiplier), + ADD_CONTROLS_FOR_DEVICE_TYPES(MaxNumOfSlowDisks), + } + })); } group.ProxyId = as->Register(proxy.release(), TMailboxType::ReadAsFilled, AppData()->SystemPoolId); @@ -158,3 +191,5 @@ void TNodeWarden::Handle(NNodeWhiteboard::TEvWhiteboard::TEvBSGroupStateUpdate:: TActivationContext::Send(ev->Forward(WhiteboardId)); } } + +#undef ADD_CONTROLS_FOR_DEVICE_TYPES diff --git a/ydb/core/blobstorage/nodewarden/ya.make b/ydb/core/blobstorage/nodewarden/ya.make index 0f0790d4bca2..bfb4e2cd5227 100644 --- a/ydb/core/blobstorage/nodewarden/ya.make +++ b/ydb/core/blobstorage/nodewarden/ya.make @@ -37,6 +37,7 @@ PEERDIR( library/cpp/openssl/crypto ydb/core/base ydb/core/blob_depot/agent + ydb/core/blobstorage/common ydb/core/blobstorage/crypto ydb/core/blobstorage/groupinfo ydb/core/blobstorage/pdisk diff --git a/ydb/core/blobstorage/ut_blobstorage/acceleration.cpp b/ydb/core/blobstorage/ut_blobstorage/acceleration.cpp index 69ae209bf1e6..7af07d01efe3 100644 --- a/ydb/core/blobstorage/ut_blobstorage/acceleration.cpp +++ b/ydb/core/blobstorage/ut_blobstorage/acceleration.cpp @@ -1,4 +1,6 @@ #include +#include +#include #include #include @@ -8,164 +10,575 @@ #define Ctest Cnull Y_UNIT_TEST_SUITE(Acceleration) { + using TFlowRecord = TIntrusivePtr; + using TQueueId = NKikimrBlobStorage::EVDiskQueueId; + + struct TDiskDelay { + TWeightedRandom Delays; + TDuration Max; + TString Tag; + + TDiskDelay(TDuration delay = TDuration::Zero(), TString tag = "") + : Max(delay) + , Tag(tag) + { + Delays.AddValue(delay, 1); + } - void SetupEnv(const TBlobStorageGroupType& erasure, std::unique_ptr& env, - ui32& nodeCount, ui32& groupId) { - nodeCount = erasure.BlobSubgroupSize(); + TDiskDelay(TDuration min, ui64 minWeight, TDuration max, ui64 maxWeight, TString tag = "") + : Max(max) + , Tag(tag) + { + Delays.AddValue(min, minWeight); + Delays.AddValue(max, maxWeight); + } - env.reset(new TEnvironmentSetup{{ - .NodeCount = nodeCount, - .Erasure = erasure, - }}); + TDiskDelay(const TDiskDelay&) = default; + TDiskDelay(TDiskDelay&&) = default; + TDiskDelay& operator=(const TDiskDelay&) = default; + TDiskDelay& operator=(TDiskDelay&&) = default; + TDuration GetRandom() { + return Delays.GetRandom(); + } + }; - env->CreateBoxAndPool(1, 1); - env->Sim(TDuration::Minutes(1)); + struct TEvDelayedMessageWrapper : public TEventLocal { + public: + std::unique_ptr Event; - NKikimrBlobStorage::TBaseConfig base = env->FetchBaseConfig(); - UNIT_ASSERT_VALUES_EQUAL(base.GroupSize(), 1); - groupId = base.GetGroup(0).GetGroupId(); + TEvDelayedMessageWrapper(std::unique_ptr& ev) + : Event(ev.release()) + {} + }; - TActorId edge = env->Runtime->AllocateEdgeActor(1); + struct TVDiskDelayEmulator { + TVDiskDelayEmulator(const std::shared_ptr& env) + : Env(env) + {} - env->Runtime->WrapInActorContext(edge, [&] { - SendToBSProxy(edge, groupId, new TEvBlobStorage::TEvStatus(TInstant::Max())); - }); - auto res = env->WaitForEdgeActorEvent(edge, true, TInstant::Max()); - UNIT_ASSERT_VALUES_EQUAL(res->Get()->Status, NKikimrProto::OK); - } + using TFlowKey = std::pair; // { nodeId, queueId } - void TestAcceleratePut(const TBlobStorageGroupType& erasure, ui32 slowDisksNum, - NKikimrBlobStorage::EPutHandleClass handleClass) { - for (ui32 fastDisksNum = 0; fastDisksNum < erasure.BlobSubgroupSize() - 2; ++fastDisksNum) { - std::unique_ptr env; - ui32 nodeCount; - ui32 groupId; - SetupEnv(erasure, env, nodeCount, groupId); + std::shared_ptr Env; + TActorId Edge; + // assuming there is only one disk per node + std::unordered_map FlowRecords; - constexpr TDuration delay = TDuration::Seconds(2); - constexpr TDuration waitFor = TDuration::Seconds(1); + std::unordered_map DelayByNode; + std::deque DelayByResponseOrder; + TDiskDelay DefaultDelay = TDuration::Seconds(1); + bool LogUnwrap = false; - Ctest << "fastDisksNum# " << fastDisksNum << Endl; + using TEventHandler = std::function&)>; - TActorId edge = env->Runtime->AllocateEdgeActor(1); - TString data = "Test"; - TLogoBlobID blobId = TLogoBlobID(1, 1, 1, 1, data.size(), 1); + std::unordered_map EventHandlers; + + void AddHandler(ui32 eventType, TEventHandler handler) { + EventHandlers[eventType] = handler; + } + + bool Filter(ui32/* nodeId*/, std::unique_ptr& ev) { + if (ev->GetTypeRewrite() == TEvDelayedMessageWrapper::EventType) { + std::unique_ptr delayedMsg(std::move(ev)); + ev.reset(delayedMsg->Get()->Event.release()); + if (LogUnwrap) { + Ctest << TAppData::TimeProvider->Now() << " Unwrap " << ev->ToString() << Endl; + } + return true; + } + + ui32 type = ev->GetTypeRewrite(); + auto it = EventHandlers.find(type); + if (it != EventHandlers.end() && it->second) { + return (it->second)(ev); + } + return true; + } + + TDuration GetMsgDelay(ui32 vdiskNodeId) { + TDiskDelay& delay = DefaultDelay; + auto it = DelayByNode.find(vdiskNodeId); + if (it == DelayByNode.end()) { + if (!DelayByResponseOrder.empty()) { + delay = DelayByResponseOrder.front(); + DelayByResponseOrder.pop_front(); + } + DelayByNode[vdiskNodeId] = delay; + } else { + delay = it->second; + } + TDuration rand = delay.GetRandom(); + return rand; + } + + TDuration DelayMsg(std::unique_ptr& ev) { + TDuration delay = GetMsgDelay(ev->Sender.NodeId()); - env->Runtime->WrapInActorContext(edge, [&] { - SendToBSProxy(edge, groupId, new TEvBlobStorage::TEvPut(blobId, data, TInstant::Max()), handleClass); + Env->Runtime->WrapInActorContext(Edge, [&] { + TActivationContext::Schedule(delay, new IEventHandle( + ev->Sender, + ev->Recipient, + new TEvDelayedMessageWrapper(ev)) + ); }); + return delay; + } + + void SetDelayByResponseOrder(const std::deque& delays) { + DelayByResponseOrder = delays; + DelayByNode = {}; + } + }; + + struct TDelayer { + std::shared_ptr VDiskDelayEmulator; + + bool operator()(ui32 nodeId, std::unique_ptr& ev) { + return VDiskDelayEmulator->Filter(nodeId, ev); + } + }; + + struct TestCtx { + TestCtx(const TBlobStorageGroupType& erasure, float slowDiskThreshold, float delayMultiplier, + ui32 maxSlowCount = 2) + : NodeCount(erasure.BlobSubgroupSize() + 1) + , Erasure(erasure) + , Env(new TEnvironmentSetup({ + .NodeCount = NodeCount, + .Erasure = erasure, + .LocationGenerator = [this](ui32 nodeId) { return LocationGenerator(nodeId); }, + .SlowDiskThreshold = slowDiskThreshold, + .VDiskPredictedDelayMultiplier = delayMultiplier, + .MaxNumOfSlowDisks = maxSlowCount, + })) + , VDiskDelayEmulator(new TVDiskDelayEmulator(Env)) + {} + + TNodeLocation LocationGenerator(ui32 nodeId) { + if (Erasure.BlobSubgroupSize() == 9) { + if (nodeId == NodeCount) { + return TNodeLocation{"4", "1", "1", "1"}; + } + return TNodeLocation{ + std::to_string((nodeId - 1) / 3), + "1", + std::to_string((nodeId - 1) % 3), + "0" + }; + } else { + if (nodeId == NodeCount) { + return TNodeLocation{"2", "1", "1", "1"}; + } + return TNodeLocation{"1", "1", std::to_string(nodeId), "0"}; + } + } - THashSet fastDisks; - THashSet slowDisks; + void Initialize() { + Env->CreateBoxAndPool(1, 1); + Env->Sim(TDuration::Minutes(1)); - env->Runtime->FilterFunction = [&](ui32/* nodeId*/, std::unique_ptr& ev) { - if (ev->GetTypeRewrite() == TEvBlobStorage::TEvVPutResult::EventType) { + NKikimrBlobStorage::TBaseConfig base = Env->FetchBaseConfig(); + UNIT_ASSERT_VALUES_EQUAL(base.GroupSize(), 1); + const auto& group = base.GetGroup(0); + GroupId = group.GetGroupId(); + + Edge = Env->Runtime->AllocateEdgeActor(NodeCount); + VDiskDelayEmulator->Edge = Edge; + + std::unordered_map OrderNumberToNodeId; + + for (ui32 orderNum = 0; orderNum < group.VSlotIdSize(); ++orderNum) { + OrderNumberToNodeId[orderNum] = group.GetVSlotId(orderNum).GetNodeId(); + } + + Env->Runtime->WrapInActorContext(Edge, [&] { + SendToBSProxy(Edge, GroupId, new TEvBlobStorage::TEvStatus(TInstant::Max())); + }); + auto res = Env->WaitForEdgeActorEvent(Edge, false, TInstant::Max()); + + Env->Runtime->FilterFunction = TDelayer{ .VDiskDelayEmulator = VDiskDelayEmulator }; + + for (const TQueueId& queueId : { TQueueId::PutTabletLog, TQueueId::GetFastRead, TQueueId::PutAsyncBlob, + TQueueId::GetAsyncRead }) { + Ctest << "Send TEvGetQueuesInfo " << queueId << Endl; + Env->Runtime->WrapInActorContext(Edge, [&] { + SendToBSProxy(Edge, GroupId, new TEvGetQueuesInfo(queueId)); + }); + auto res = Env->WaitForEdgeActorEvent(Edge, false, TInstant::Max()); + Ctest << "Get TEvQueuesInfo " << res->Get()->ToString() << Endl; + + for (ui32 orderNum = 0; orderNum < res->Get()->Queues.size(); ++orderNum) { + const std::optional& queue = res->Get()->Queues[orderNum]; + if (queue) { + Y_ABORT_UNLESS(queue->FlowRecord); + queue->FlowRecord->SetPredictedDelayNs(VDiskDelayEmulator->DefaultDelay.Max.NanoSeconds()); + VDiskDelayEmulator->FlowRecords[{ OrderNumberToNodeId[orderNum], queueId }] = queue->FlowRecord; + } + } + } + } + + ~TestCtx() { + Env->Runtime->FilterFunction = {}; + } + + ui32 NodeCount; + TBlobStorageGroupType Erasure; + std::shared_ptr Env; + + ui32 GroupId; + TActorId Edge; + std::shared_ptr VDiskDelayEmulator; + }; + + #define ADD_DSPROXY_MESSAGE_PRINTER(MsgType) \ + ctx.VDiskDelayEmulator->AddHandler(MsgType::EventType, [&](std::unique_ptr& ev) { \ + if (ev->Recipient.NodeId() == ctx.NodeCount) { \ + Ctest << TAppData::TimeProvider->Now() << " Send "#MsgType": " << ev->Sender.ToString() << " " << \ + ev->Recipient.ToString() << ev->Get()->ToString() << Endl; \ + } \ + return true; \ + } \ + ) + + void TestAcceleratePut(const TBlobStorageGroupType& erasure, ui32 slowDisksNum, + NKikimrBlobStorage::EPutHandleClass handleClass, TDuration fastDelay, + TDuration slowDelay, TDuration initDelay, TDuration waitTime, + float delayMultiplier, ui32 maxSlowCount = 2) { + ui32 initialRequests = 100; + float slowDiskThreshold = 2; + TDiskDelay fastDiskDelay = TDiskDelay(fastDelay); + TDiskDelay slowDiskDelay = TDiskDelay(slowDelay); + TDiskDelay initDiskDelay = TDiskDelay(initDelay); + + ui32 requests = (erasure.GetErasure() == TBlobStorageGroupType::ErasureMirror3dc) ? 3 : 6; + + for (ui32 fastDisksNum = 0; fastDisksNum < requests - 1; ++fastDisksNum) { + Ctest << "fastDisksNum# " << fastDisksNum << Endl; + TestCtx ctx(erasure, slowDiskThreshold, delayMultiplier, maxSlowCount); + ctx.VDiskDelayEmulator->DefaultDelay = initDiskDelay; + ctx.Initialize(); + bool verboseHandlers = false; + + TString data = MakeData(1024); + auto put = [&](TLogoBlobID blobId, bool timeout) { + ctx.Env->Runtime->WrapInActorContext(ctx.Edge, [&] { + TEvBlobStorage::TEvPut* ev = new TEvBlobStorage::TEvPut(blobId, data, TInstant::Max()); + if (verboseHandlers) { + Ctest << TAppData::TimeProvider->Now() << " Send TEvPut# " << ev->ToString() << Endl; + } + SendToBSProxy(ctx.Edge, ctx.GroupId, ev, handleClass); + }); + auto res = ctx.Env->WaitForEdgeActorEvent( + ctx.Edge, false, timeout ? (TAppData::TimeProvider->Now() + waitTime) : TInstant::Max()); + if (timeout) { + if (slowDisksNum <= maxSlowCount) { + UNIT_ASSERT_C(res, "fastDisksNum# " << fastDisksNum); + UNIT_ASSERT_VALUES_EQUAL(res->Get()->Status, NKikimrProto::OK); + } else { + UNIT_ASSERT_C(!res, "fastDisksNum# " << fastDisksNum); + } + } + }; + + ctx.VDiskDelayEmulator->AddHandler(TEvBlobStorage::TEvVPutResult::EventType, [&](std::unique_ptr& ev) { + ui32 nodeId = ev->Sender.NodeId(); + if (nodeId < ctx.NodeCount) { TVDiskID vdiskId = VDiskIDFromVDiskID(ev->Get()->Record.GetVDiskID()); TLogoBlobID partId = LogoBlobIDFromLogoBlobID(ev->Get()->Record.GetBlobID()); - Ctest << TAppData::TimeProvider->Now() << " TEvVPutResult: vdiskId# " << vdiskId.ToString() << - " partId# " << partId.ToString() << ", "; - if (fastDisks.size() < fastDisksNum || fastDisks.count(vdiskId)) { - fastDisks.insert(vdiskId); - Ctest << "pass message" << Endl; - return true; - } else if (!slowDisks.count(vdiskId) && slowDisks.size() >= slowDisksNum) { - Ctest << "pass message" << Endl; - return true; - } else { - Ctest << "delay message for " << delay.ToString() << Endl; - slowDisks.insert(vdiskId); - env->Runtime->WrapInActorContext(edge, [&] { - TActivationContext::Schedule(delay, ev.release()); - }); + TDuration delay = ctx.VDiskDelayEmulator->DelayMsg(ev); + if (verboseHandlers) { + Ctest << TAppData::TimeProvider->Now() << " TEvVPutResult: vdiskId# " << vdiskId.ToString() << + " partId# " << partId.ToString() << " nodeId# " << nodeId << ", delay " << delay << Endl; + } + return false; + } + return true; + }); + + for (ui32 i = 0; i < initialRequests; ++i) { + put(TLogoBlobID(1, 1, 1, 1, data.size(), 123 + i), false); + } + + ctx.Env->Sim(slowDelay); - return false; + std::deque delayByResponseOrder; + for (ui32 i = 0; i < erasure.BlobSubgroupSize(); ++i) { + if (i >= fastDisksNum && i < fastDisksNum + slowDisksNum) { + delayByResponseOrder.push_back(slowDiskDelay); + } else { + delayByResponseOrder.push_back(fastDiskDelay); + } + } + ctx.VDiskDelayEmulator->SetDelayByResponseOrder(delayByResponseOrder); + + ctx.VDiskDelayEmulator->LogUnwrap = true; + verboseHandlers = true; + ADD_DSPROXY_MESSAGE_PRINTER(TEvBlobStorage::TEvVPut); + ADD_DSPROXY_MESSAGE_PRINTER(TEvBlobStorage::TEvPutResult); + put(TLogoBlobID(1, 1, 1, 1, data.size(), 1), true); + + } + } + + void TestAccelerateGet(const TBlobStorageGroupType& erasure, ui32 slowDisksNum, + NKikimrBlobStorage::EGetHandleClass handleClass, TDuration fastDelay, + TDuration slowDelay, TDuration initDelay, TDuration waitTime, + float delayMultiplier, ui32 maxSlowCount = 2) { + ui32 initialRequests = 100; + float slowDiskThreshold = 2; + TDiskDelay fastDiskDelay = TDiskDelay(fastDelay); + TDiskDelay slowDiskDelay = TDiskDelay(slowDelay); + TDiskDelay initDiskDelay = TDiskDelay(initDelay); + + ui32 requests = 3; + + for (ui32 fastDisksNum = 0; fastDisksNum < requests - 1; ++fastDisksNum) { + Ctest << "fastDisksNum# " << fastDisksNum << Endl; + TestCtx ctx(erasure, slowDiskThreshold, delayMultiplier, maxSlowCount); + ctx.VDiskDelayEmulator->DefaultDelay = initDiskDelay; + ctx.Initialize(); + + bool verboseHandlers = false; + ctx.VDiskDelayEmulator->AddHandler(TEvBlobStorage::TEvVGetResult::EventType, [&](std::unique_ptr& ev) { + ui32 nodeId = ev->Sender.NodeId(); + if (nodeId < ctx.NodeCount) { + TVDiskID vdiskId = VDiskIDFromVDiskID(ev->Get()->Record.GetVDiskID()); + TLogoBlobID partId = LogoBlobIDFromLogoBlobID( + ev->Get()->Record.GetResult(0).GetBlobID()); + TDuration delay = ctx.VDiskDelayEmulator->DelayMsg(ev); + if (verboseHandlers) { + Ctest << TAppData::TimeProvider->Now() << " TEvVGetResult: vdiskId# " << vdiskId.ToString() << + " partId# " << partId.ToString() << " nodeId# " << nodeId << ", delay " << delay << Endl; } + return false; } return true; + }); + + TString data = MakeData(1024); + auto putAndGet = [&](TLogoBlobID blobId, bool timeout) { + ctx.Env->Runtime->WrapInActorContext(ctx.Edge, [&] { + SendToBSProxy(ctx.Edge, ctx.GroupId, new TEvBlobStorage::TEvPut(blobId, data, TInstant::Max())); + }); + auto putRes = ctx.Env->WaitForEdgeActorEvent(ctx.Edge, false, TInstant::Max()); + UNIT_ASSERT_VALUES_EQUAL(putRes->Get()->Status, NKikimrProto::OK); + + ctx.Env->Runtime->WrapInActorContext(ctx.Edge, [&] { + SendToBSProxy(ctx.Edge, ctx.GroupId, new TEvBlobStorage::TEvGet(blobId, 0, data.size(), TInstant::Max(), handleClass)); + }); + auto getRes = ctx.Env->WaitForEdgeActorEvent(ctx.Edge, false, + timeout ? (TAppData::TimeProvider->Now() + waitTime) : TInstant::Max()); + if (timeout) { + if (slowDisksNum <= maxSlowCount) { + UNIT_ASSERT_C(getRes, "fastDisksNum# " << fastDisksNum); + UNIT_ASSERT_VALUES_EQUAL(getRes->Get()->Status, NKikimrProto::OK); + UNIT_ASSERT_VALUES_EQUAL(getRes->Get()->Responses[0].Status, NKikimrProto::OK); + } else { + UNIT_ASSERT_C(!getRes, "fastDisksNum# " << fastDisksNum); + } + } }; - auto res = env->WaitForEdgeActorEvent(edge, false, TAppData::TimeProvider->Now() + waitFor); - UNIT_ASSERT_C(res, "fastDisksNum# " << fastDisksNum); - UNIT_ASSERT_VALUES_EQUAL(res->Get()->Status, NKikimrProto::OK); + for (ui32 i = 0; i < initialRequests; ++i) { + putAndGet(TLogoBlobID(1, 1, 1, 1, data.size(), 123 + i), false); + } + ctx.Env->Sim(slowDelay); + + std::deque delayByResponseOrder; + for (ui32 i = 0; i < erasure.BlobSubgroupSize(); ++i) { + if (i >= fastDisksNum && i < fastDisksNum + slowDisksNum) { + delayByResponseOrder.push_back(slowDiskDelay); + } else { + delayByResponseOrder.push_back(fastDiskDelay); + } + } + ctx.VDiskDelayEmulator->SetDelayByResponseOrder(delayByResponseOrder); + + ctx.VDiskDelayEmulator->LogUnwrap = true; + verboseHandlers = true; + ADD_DSPROXY_MESSAGE_PRINTER(TEvBlobStorage::TEvVGet); + ADD_DSPROXY_MESSAGE_PRINTER(TEvBlobStorage::TEvGetResult); + putAndGet(TLogoBlobID(1, 1, 1, 1, data.size(), 2), true); } } - void TestAccelerateGet(const TBlobStorageGroupType& erasure, ui32 slowDisksNum, - NKikimrBlobStorage::EGetHandleClass handleClass) { - for (ui32 fastDisksNum = 0; fastDisksNum < erasure.BlobSubgroupSize() - 2; ++fastDisksNum) { - std::unique_ptr env; - ui32 nodeCount; - ui32 groupId; - SetupEnv(erasure, env, nodeCount, groupId); + using TTestThresholdRequestSender = std::function; - constexpr TDuration delay = TDuration::Seconds(2); - constexpr TDuration waitFor = TDuration::Seconds(1); + void TestThresholdSendPutRequests(TestCtx& ctx, ui32 requests) { + ui64 cookie = 1; - Ctest << "fastDisksNum# " << fastDisksNum << Endl; + for (ui32 i = 0; i < requests; ++i) { + TString data = "Test"; + TLogoBlobID blobId = TLogoBlobID(1, 1, 1, 1, data.size(), ++cookie); - TActorId edge = env->Runtime->AllocateEdgeActor(1); - TString data = MakeData(1024); - TLogoBlobID blobId = TLogoBlobID(1, 1, 1, 1, data.size(), 1); + Ctest << " ------------------- Send TEvPut# " << i << " ------------------- " << Endl; + ctx.Env->Runtime->WrapInActorContext(ctx.Edge, [&] { + SendToBSProxy(ctx.Edge, ctx.GroupId, new TEvBlobStorage::TEvPut(blobId, data, TInstant::Max())); + }); + auto res = ctx.Env->WaitForEdgeActorEvent(ctx.Edge, false, TInstant::Max()); + UNIT_ASSERT_VALUES_EQUAL(res->Get()->Status, NKikimrProto::OK); + } + } + + void TestThresholdSendGetRequests(TestCtx& ctx, ui32 requests) { + ui64 cookie = 1; + std::vector blobs; + TString data = MakeData(1024); - env->Runtime->WrapInActorContext(edge, [&] { - SendToBSProxy(edge, groupId, new TEvBlobStorage::TEvPut(blobId, data, TInstant::Max())); + for (ui32 i = 0; i < requests; ++i) { + TLogoBlobID blobId = TLogoBlobID(1, 1, 1, 1, data.size(), ++cookie); + ctx.Env->Runtime->WrapInActorContext(ctx.Edge, [&] { + SendToBSProxy(ctx.Edge, ctx.GroupId, new TEvBlobStorage::TEvPut(blobId, data, TInstant::Max())); }); - - env->WaitForEdgeActorEvent(edge, false, TInstant::Max()); - env->Runtime->WrapInActorContext(edge, [&] { - SendToBSProxy(edge, groupId, new TEvBlobStorage::TEvGet(blobId, 0, data.size(), TInstant::Max(), handleClass)); + blobs.push_back(blobId); + auto res = ctx.Env->WaitForEdgeActorEvent(ctx.Edge, false, TInstant::Max()); + UNIT_ASSERT_VALUES_EQUAL(res->Get()->Status, NKikimrProto::OK); + } + + for (const auto& blobId : blobs) { + ctx.Env->Runtime->WrapInActorContext(ctx.Edge, [&] { + SendToBSProxy(ctx.Edge, ctx.GroupId, new TEvBlobStorage::TEvGet(blobId, 0, data.size(), TInstant::Max(), + NKikimrBlobStorage::AsyncRead)); }); + auto res = ctx.Env->WaitForEdgeActorEvent(ctx.Edge, false, TInstant::Max()); + UNIT_ASSERT_VALUES_EQUAL(res->Get()->Status, NKikimrProto::OK); + UNIT_ASSERT_VALUES_EQUAL(res->Get()->Responses[0].Status, NKikimrProto::OK); + } + } - THashSet slowDisks; - THashSet fastDisks; + void TestThreshold(const TBlobStorageGroupType& erasure, ui32 slowDisks, bool delayPuts, bool delayGets, + TTestThresholdRequestSender sendRequests, float maxRatio) { + float delayMultiplier = 1; + float slowDiskThreshold = 1.2; + TDiskDelay fastDiskDelay = TDiskDelay(TDuration::Seconds(0.1), 10, TDuration::Seconds(1), 1, "fast"); + TDiskDelay slowDiskDelay = TDiskDelay(TDuration::Seconds(1.5), "slow"); + + ui32 requests = 1000; + + TestCtx ctx(erasure, slowDiskThreshold, delayMultiplier); + ctx.VDiskDelayEmulator->DefaultDelay = fastDiskDelay; + ui32 groupSize = erasure.BlobSubgroupSize(); + + std::vector nodeIsSlow(groupSize, true); + std::vector vrequestsByNode(groupSize, 0); + + for (ui32 i = 0; i < groupSize; ++i) { + bool isSlow = (i % 3 == 0 && i / 3 < slowDisks); + ctx.VDiskDelayEmulator->DelayByNode[i + 1] = isSlow ? slowDiskDelay : fastDiskDelay; + nodeIsSlow[i] = isSlow; + } - env->Runtime->FilterFunction = [&](ui32/* nodeId*/, std::unique_ptr& ev) { - if (ev->GetTypeRewrite() == TEvBlobStorage::TEvVGetResult::EventType) { + ctx.Initialize(); + ctx.VDiskDelayEmulator->LogUnwrap = true; + + if (delayPuts) { + ctx.VDiskDelayEmulator->AddHandler(TEvBlobStorage::TEvVPutResult::EventType, [&](std::unique_ptr& ev) { + ui32 nodeId = ev->Sender.NodeId(); + if (nodeId < ctx.NodeCount) { + TVDiskID vdiskId = VDiskIDFromVDiskID(ev->Get()->Record.GetVDiskID()); + TLogoBlobID partId = LogoBlobIDFromLogoBlobID(ev->Get()->Record.GetBlobID()); + TDuration delay = ctx.VDiskDelayEmulator->DelayMsg(ev); + Ctest << TAppData::TimeProvider->Now() << " TEvVPutResult: vdiskId# " << vdiskId.ToString() << + " partId# " << partId.ToString() << " nodeId# " << nodeId << ", delay " << delay << Endl; + ++vrequestsByNode[nodeId - 1]; + return false; + } + return true; + }); + ADD_DSPROXY_MESSAGE_PRINTER(TEvBlobStorage::TEvVPut); + } + + if (delayGets) { + ctx.VDiskDelayEmulator->AddHandler(TEvBlobStorage::TEvVGetResult::EventType, [&](std::unique_ptr& ev) { + ui32 nodeId = ev->Sender.NodeId(); + if (nodeId < ctx.NodeCount) { TVDiskID vdiskId = VDiskIDFromVDiskID(ev->Get()->Record.GetVDiskID()); TLogoBlobID partId = LogoBlobIDFromLogoBlobID( ev->Get()->Record.GetResult(0).GetBlobID()); - Ctest << TAppData::TimeProvider->Now() << " TEvVGetResult: " << vdiskId.ToString() << - " partId# " << partId.ToString() << ", "; - if (fastDisks.size() < fastDisksNum || fastDisks.count(vdiskId)) { - fastDisks.insert(vdiskId); - Ctest << "pass message" << Endl; - return true; - } else if (!slowDisks.count(vdiskId) && slowDisks.size() >= slowDisksNum) { - Ctest << "pass message" << Endl; - return true; - } else { - Ctest << "delay message for " << delay.ToString() << Endl; - slowDisks.insert(vdiskId); - env->Runtime->WrapInActorContext(edge, [&] { - TActivationContext::Schedule(delay, ev.release()); - }); - - return false; - } + TDuration delay = ctx.VDiskDelayEmulator->DelayMsg(ev); + Ctest << TAppData::TimeProvider->Now() << " TEvVGetResult: vdiskId# " << vdiskId.ToString() << + " partId# " << partId.ToString() << " nodeId# " << nodeId << ", delay " << delay << Endl; + ++vrequestsByNode[nodeId - 1]; + return false; } return true; - }; + }); + ADD_DSPROXY_MESSAGE_PRINTER(TEvBlobStorage::TEvVGet); + } - auto res = env->WaitForEdgeActorEvent(edge, false, TAppData::TimeProvider->Now() + waitFor); - UNIT_ASSERT_C(res, "fastDisksNum# " << fastDisksNum); - UNIT_ASSERT_VALUES_EQUAL(res->Get()->Status, NKikimrProto::OK); - UNIT_ASSERT_VALUES_EQUAL(res->Get()->Responses[0].Status, NKikimrProto::OK); - Ctest << "TEvGetResult# " << res->Get()->ToString() << Endl; + sendRequests(ctx, requests); + + ui32 slowNodesCount = 0; + ui32 slowNodesRequests = 0; + ui32 fastNodesCount = 0; + ui32 fastNodesRequests = 0; + + TStringStream str; + + str << "VRequests by node: "; + for (ui32 i = 0; i < groupSize; ++i) { + str << "{ nodeId# " << i << " isSlow# " << nodeIsSlow[i] << ' ' << vrequestsByNode[i] << "}, "; + if (nodeIsSlow[i]) { + ++slowNodesCount; + slowNodesRequests += vrequestsByNode[i]; + } else { + ++fastNodesCount; + fastNodesRequests += vrequestsByNode[i]; + } } + Ctest << str.Str() << Endl; + + double slowNodeRequestsAvg = 1. * slowNodesRequests / slowNodesCount; + double fastNodeRequestsAvg = 1. * fastNodesRequests / fastNodesCount; + + double ratio = fastNodeRequestsAvg / slowNodeRequestsAvg; + Ctest << "Fast to slow ratio# " << ratio << Endl; + UNIT_ASSERT_GE_C(ratio, maxRatio, "ratio# " << ratio << " " << str.Str()); + } + + void TestThresholdPut(const TBlobStorageGroupType& erasure, ui32 slowDisks) { + TestThreshold(erasure, slowDisks, true, false, TestThresholdSendPutRequests, 5); + } + + void TestThresholdGet(const TBlobStorageGroupType& erasure, ui32 slowDisks) { + TestThreshold(erasure, slowDisks, false, true, TestThresholdSendGetRequests, 1.5); + } + + void TestDelayMultiplierPut(const TBlobStorageGroupType& erasure, ui32 slowDisks) { + TestAcceleratePut(erasure, slowDisks, NKikimrBlobStorage::AsyncBlob, TDuration::Seconds(0.9), + TDuration::Seconds(2), TDuration::Seconds(1), TDuration::Seconds(1.95), 0.8); + } + + void TestDelayMultiplierGet(const TBlobStorageGroupType& erasure, ui32 slowDisks) { + TestAccelerateGet(erasure, slowDisks, NKikimrBlobStorage::AsyncRead, TDuration::Seconds(0.9), + TDuration::Seconds(2), TDuration::Seconds(1), TDuration::Seconds(1.95), 0.8); + } + + void TestMaxNumOfSlowDisksPut(const TBlobStorageGroupType& erasure, ui32 slowDisks) { + TestAcceleratePut(erasure, slowDisks, NKikimrBlobStorage::AsyncBlob, TDuration::Seconds(1), + TDuration::Seconds(5), TDuration::Seconds(1), TDuration::Seconds(4), 1, 1); + } + + void TestMaxNumOfSlowDisksGet(const TBlobStorageGroupType& erasure, ui32 slowDisks) { + TestAccelerateGet(erasure, slowDisks, NKikimrBlobStorage::AsyncRead, TDuration::Seconds(1), + TDuration::Seconds(5), TDuration::Seconds(1), TDuration::Seconds(4), 1, 1); } #define TEST_ACCELERATE(erasure, method, handleClass, slowDisks) \ - Y_UNIT_TEST(Test##erasure##method##handleClass##slowDisks##Slow) { \ - TestAccelerate##method(TBlobStorageGroupType::Erasure##erasure, slowDisks, NKikimrBlobStorage::handleClass); \ + Y_UNIT_TEST(TestAcceleration##erasure##method##handleClass##slowDisks##Slow) { \ + TestAccelerate##method(TBlobStorageGroupType::Erasure##erasure, slowDisks, NKikimrBlobStorage::handleClass, \ + TDuration::Seconds(1), TDuration::Seconds(5), TDuration::Seconds(1), TDuration::Seconds(4), 1); \ } + // TODO fix Acceleration in mirror-3-of-4 + TEST_ACCELERATE(Mirror3dc, Put, AsyncBlob, 1); // TEST_ACCELERATE(Mirror3of4, Put, AsyncBlob, 1); TEST_ACCELERATE(4Plus2Block, Put, AsyncBlob, 1); -// TEST_ACCELERATE(Mirror3dc, Put, AsyncBlob, 2); + TEST_ACCELERATE(Mirror3dc, Put, AsyncBlob, 2); // TEST_ACCELERATE(Mirror3of4, Put, AsyncBlob, 2); TEST_ACCELERATE(4Plus2Block, Put, AsyncBlob, 2); @@ -173,9 +586,49 @@ Y_UNIT_TEST_SUITE(Acceleration) { // TEST_ACCELERATE(Mirror3of4, Get, AsyncRead, 1); TEST_ACCELERATE(4Plus2Block, Get, AsyncRead, 1); -// TEST_ACCELERATE(Mirror3dc, Get, AsyncRead, 2); + TEST_ACCELERATE(Mirror3dc, Get, AsyncRead, 2); // TEST_ACCELERATE(Mirror3of4, Get, AsyncRead, 2); TEST_ACCELERATE(4Plus2Block, Get, AsyncRead, 2); + #define TEST_ACCELERATE_PARAMS(param, method, erasure, slowDisks) \ + Y_UNIT_TEST(Test##param##method##erasure##slowDisks##Slow) { \ + Test##param##method(TBlobStorageGroupType::Erasure##erasure, slowDisks); \ + } + + TEST_ACCELERATE_PARAMS(Threshold, Put, Mirror3dc, 1); + TEST_ACCELERATE_PARAMS(Threshold, Put, 4Plus2Block, 1); + + TEST_ACCELERATE_PARAMS(Threshold, Put, Mirror3dc, 2); + TEST_ACCELERATE_PARAMS(Threshold, Put, 4Plus2Block, 2); + + TEST_ACCELERATE_PARAMS(Threshold, Get, Mirror3dc, 1); + TEST_ACCELERATE_PARAMS(Threshold, Get, 4Plus2Block, 1); + + TEST_ACCELERATE_PARAMS(Threshold, Get, Mirror3dc, 2); + TEST_ACCELERATE_PARAMS(Threshold, Get, 4Plus2Block, 2); + + TEST_ACCELERATE_PARAMS(DelayMultiplier, Put, Mirror3dc, 1); + TEST_ACCELERATE_PARAMS(DelayMultiplier, Put, 4Plus2Block, 1); + + TEST_ACCELERATE_PARAMS(DelayMultiplier, Put, Mirror3dc, 2); + TEST_ACCELERATE_PARAMS(DelayMultiplier, Put, 4Plus2Block, 2); + + TEST_ACCELERATE_PARAMS(DelayMultiplier, Get, Mirror3dc, 1); + TEST_ACCELERATE_PARAMS(DelayMultiplier, Get, 4Plus2Block, 1); + + TEST_ACCELERATE_PARAMS(DelayMultiplier, Get, Mirror3dc, 2); + TEST_ACCELERATE_PARAMS(DelayMultiplier, Get, 4Plus2Block, 2); + + TEST_ACCELERATE_PARAMS(MaxNumOfSlowDisks, Get, Mirror3dc, 1); + TEST_ACCELERATE_PARAMS(MaxNumOfSlowDisks, Get, 4Plus2Block, 1); + + TEST_ACCELERATE_PARAMS(MaxNumOfSlowDisks, Put, Mirror3dc, 1); + TEST_ACCELERATE_PARAMS(MaxNumOfSlowDisks, Put, 4Plus2Block, 1); + + TEST_ACCELERATE_PARAMS(MaxNumOfSlowDisks, Put, Mirror3dc, 2); + TEST_ACCELERATE_PARAMS(MaxNumOfSlowDisks, Put, 4Plus2Block, 2); + #undef TEST_ACCELERATE + #undef TEST_ACCELERATE_PARAMS + #undef PRINT_DSPROXY_MESSAGE } diff --git a/ydb/core/blobstorage/ut_blobstorage/blob_depot_test_env.h b/ydb/core/blobstorage/ut_blobstorage/blob_depot_test_env.h index f7b66f589a75..bdc6b0e0a534 100644 --- a/ydb/core/blobstorage/ut_blobstorage/blob_depot_test_env.h +++ b/ydb/core/blobstorage/ut_blobstorage/blob_depot_test_env.h @@ -75,29 +75,6 @@ struct TTabletInfo { using TBSState = std::map; -struct TIntervals { - std::vector Borders; // [0; x_1) [x_1; x_2) ... [x_n-1; x_n) - - TIntervals(std::vector borders) { - Borders = borders; - for (ui32 i = 1; i < Borders.size(); ++i) { - Borders[i] += Borders[i - 1]; - } - } - - ui32 GetInterval(ui32 x) { - for (ui32 i = 0; i < Borders.size(); ++i) { - if (x < Borders[i]) { - return i; - } - } - return Borders.size(); - } - ui32 UpperLimit() { - return Borders[Borders.size() - 1]; - } -}; - struct TEvArgs { enum EEventType : ui32 { PUT, @@ -181,6 +158,7 @@ struct TEvRangeArgs : public TEvArgs { }; struct TBlobDepotTestEnvironment { + ui64 RandomSeed; TMersenne Mt; TMersenne Mt64; @@ -191,7 +169,8 @@ struct TBlobDepotTestEnvironment { TBlobDepotTestEnvironment(ui32 seed = 0, ui32 numGroups = 1, ui32 nodeCount = 8, TBlobStorageGroupType erasure = TBlobStorageGroupType::ErasureMirror3of4) - : Mt(seed) + : RandomSeed(seed) + , Mt(seed) , Mt64(seed) { Cerr << "Mersenne random seed " << seed << Endl; ConfigureEnvironment(numGroups, Env, RegularGroups, BlobDepot, nodeCount, erasure); diff --git a/ydb/core/blobstorage/ut_blobstorage/blob_depot_test_functions.cpp b/ydb/core/blobstorage/ut_blobstorage/blob_depot_test_functions.cpp index 1cdbeb9b7cd3..ec56f7a91d6a 100644 --- a/ydb/core/blobstorage/ut_blobstorage/blob_depot_test_functions.cpp +++ b/ydb/core/blobstorage/ut_blobstorage/blob_depot_test_functions.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -460,14 +461,18 @@ void TestVerifiedRandom(TBlobDepotTestEnvironment& tenv, ui32 nodeCount, ui64 ta COLLECT_GARBAGE_HARD, COLLECT_GARBAGE_SOFT, RESTART_BLOB_DEPOT, + __COUNT__, }; std::vector actionName = { "ALTER", "PUT", "GET", "MULTIGET", "RANGE", "BLOCK", "DISCOVER", "COLLECT_GARBAGE_HARD", "COLLECT_GARBAGE_SOFT", "RESTART_BLOB_DEPOT" }; - std::vector probs = probabilities; - TIntervals act(probs); + TWeightedRandom act(tenv.RandomSeed + 0xABCD); + Y_ABORT_UNLESS(probabilities.size() == EActions::__COUNT__); + for (ui32 i = 0; i < probabilities.size(); ++i) { + act.AddValue(i, probabilities[i]); + } std::vector tablets = {tabletId0, tabletId0 + 1, tabletId0 + 2}; std::vector tabletGen = {1, 1, 1}; @@ -508,7 +513,7 @@ void TestVerifiedRandom(TBlobDepotTestEnvironment& tenv, ui32 nodeCount, ui64 ta ui32 hardCollectGen = state[tabletId].Channels[channel].HardCollectGen; ui32 hardCollectStep = state[tabletId].Channels[channel].HardCollectStep; - ui32 action = act.GetInterval(tenv.Rand(act.UpperLimit())); + ui32 action = act.GetRandom(); // Cerr << "iteration# " << iteration << " action# " << actionName[action] << " timer# " << timer.Passed() << Endl; switch (action) { case EActions::ALTER: @@ -689,13 +694,13 @@ void TestVerifiedRandom(TBlobDepotTestEnvironment& tenv, ui32 nodeCount, ui64 ta break; default: - UNIT_FAIL("TIntervals failed"); + UNIT_FAIL("Unknown action# " << action); } } } void TestLoadPutAndGet(TBlobDepotTestEnvironment& tenv, ui64 tabletId, ui32 groupId, ui32 blobsNum, ui32 maxBlobSize, - ui32 readsNum, bool decommit, ui32 timeLimitSec, std::vector probablities) { + ui32 readsNum, bool decommit, ui32 timeLimitSec, std::vector probabilities) { enum EActions { GET, MULTIGET, @@ -703,9 +708,14 @@ void TestLoadPutAndGet(TBlobDepotTestEnvironment& tenv, ui64 tabletId, ui32 grou DISCOVER, CATCH_ALL, RESTART_BLOB_DEPOT, + __COUNT__, }; - std::vector probs = probablities; - TIntervals act(probs); + + TWeightedRandom act(tenv.RandomSeed + 0xABCD); + Y_ABORT_UNLESS(probabilities.size() == EActions::__COUNT__); + for (ui32 i = 0; i < probabilities.size(); ++i) { + act.AddValue(i, probabilities[i]); + } std::vector blobs; std::map mappedBlobs; @@ -748,7 +758,7 @@ void TestLoadPutAndGet(TBlobDepotTestEnvironment& tenv, ui64 tabletId, ui32 grou THPTimer timer; for (ui32 iteration = 0; iteration < readsNum; ++iteration) { - ui32 action = act.GetInterval(tenv.Rand(act.UpperLimit())); + ui32 action = act.GetRandom(); if (iteration == readsNum - 1) { // Catch all results on the last iteration action = EActions::CATCH_ALL; } @@ -875,7 +885,7 @@ void TestLoadPutAndGet(TBlobDepotTestEnvironment& tenv, ui64 tabletId, ui32 grou break; default: - UNIT_FAIL("TIntervals failed"); + UNIT_FAIL("Unknown action# " << action); } } } diff --git a/ydb/core/blobstorage/ut_blobstorage/counting_events.cpp b/ydb/core/blobstorage/ut_blobstorage/counting_events.cpp index e9de1c0fab97..e5b112e445b4 100644 --- a/ydb/core/blobstorage/ut_blobstorage/counting_events.cpp +++ b/ydb/core/blobstorage/ut_blobstorage/counting_events.cpp @@ -86,7 +86,11 @@ Y_UNIT_TEST_SUITE(CountingEvents) { void CountingEventsTest(TString typeOperation, ui32 eventsCount, TBlobStorageGroupType groupType) { - TEnvironmentSetup env(true, groupType); + TEnvironmentSetup env({ + .VDiskReplPausedAtStart = true, + .Erasure = groupType, + .UseActorSystemTimeInBSQueue = false, + }); auto& runtime = env.Runtime; env.CreateBoxAndPool(); diff --git a/ydb/core/blobstorage/ut_blobstorage/lib/common.h b/ydb/core/blobstorage/ut_blobstorage/lib/common.h index e581eeb206f8..3ed6d98cbc61 100644 --- a/ydb/core/blobstorage/ut_blobstorage/lib/common.h +++ b/ydb/core/blobstorage/ut_blobstorage/lib/common.h @@ -2,6 +2,7 @@ #include "env.h" #include +#include inline TBlobStorageGroupType GetErasureTypeByString(const TString& erasure) { @@ -55,3 +56,44 @@ inline std::vector MakePDiskLayout(const NKikimrBlobStorage::TBaseConfig& } return pdiskLayout; } + +template +class TWeightedRandom { +public: + TWeightedRandom(ui64 seed = 0) + : PrefixSum({ 0 }) + , Mt64(new TMersenne(seed)) + {} + + TWeightedRandom(const TWeightedRandom&) = default; + TWeightedRandom(TWeightedRandom&&) = default; + TWeightedRandom& operator=(const TWeightedRandom&) = default; + TWeightedRandom& operator=(TWeightedRandom&&) = default; + + void AddValue(T value, ui64 weight) { + PrefixSum.push_back(weight + PrefixSum.back()); + Values.push_back(value); + } + + T GetRandom() { + Y_ABORT_UNLESS(WeightSum() != 0); + return Get((*Mt64)() % WeightSum()); + } + + T Get(ui64 w) { + Y_ABORT_UNLESS(PrefixSum.size() > 1); + auto it = std::upper_bound(PrefixSum.begin(), PrefixSum.end(), w); + Y_ABORT_UNLESS(it > PrefixSum.begin()); + ui32 idx = it - PrefixSum.begin() - 1; + return Values[idx]; + } + + ui32 WeightSum() { + return PrefixSum.back(); + } + +private: + std::vector Values; + std::vector PrefixSum; + std::shared_ptr> Mt64; +}; diff --git a/ydb/core/blobstorage/ut_blobstorage/lib/env.h b/ydb/core/blobstorage/ut_blobstorage/lib/env.h index f079b571104e..ad829ba64eaf 100644 --- a/ydb/core/blobstorage/ut_blobstorage/lib/env.h +++ b/ydb/core/blobstorage/ut_blobstorage/lib/env.h @@ -48,6 +48,10 @@ struct TEnvironmentSetup { const ui32 MinHugeBlobInBytes = 0; const float DiskTimeAvailableScale = 1; const bool UseFakeConfigDispatcher = false; + const float SlowDiskThreshold = 2; + const float VDiskPredictedDelayMultiplier = 1; + const ui32 MaxNumOfSlowDisks = 2; + const bool UseActorSystemTimeInBSQueue = true; }; const TSettings Settings; @@ -363,6 +367,7 @@ struct TEnvironmentSetup { auto config = MakeIntrusive(new TMockPDiskServiceFactory(*this)); config->BlobStorageConfig.MutableServiceSet()->AddAvailabilityDomains(DomainId); config->VDiskReplPausedAtStart = Settings.VDiskReplPausedAtStart; + config->UseActorSystemTimeInBSQueue = Settings.UseActorSystemTimeInBSQueue; if (Settings.ConfigPreprocessor) { Settings.ConfigPreprocessor(nodeId, *config); } @@ -396,6 +401,17 @@ struct TEnvironmentSetup { ADD_ICB_CONTROL("VDiskControls.DiskTimeAvailableScaleHDD", 1'000, 1, 1'000'000, std::round(Settings.DiskTimeAvailableScale * 1'000)); ADD_ICB_CONTROL("VDiskControls.DiskTimeAvailableScaleSSD", 1'000, 1, 1'000'000, std::round(Settings.DiskTimeAvailableScale * 1'000)); ADD_ICB_CONTROL("VDiskControls.DiskTimeAvailableScaleNVME", 1'000, 1, 1'000'000, std::round(Settings.DiskTimeAvailableScale * 1'000)); + + ADD_ICB_CONTROL("DSProxyControls.SlowDiskThreshold", 2'000, 1, 1'000'000, std::round(Settings.SlowDiskThreshold * 1'000)); + ADD_ICB_CONTROL("DSProxyControls.SlowDiskThresholdHDD", 2'000, 1, 1'000'000, std::round(Settings.SlowDiskThreshold * 1'000)); + ADD_ICB_CONTROL("DSProxyControls.SlowDiskThresholdSSD", 2'000, 1, 1'000'000, std::round(Settings.SlowDiskThreshold * 1'000)); + ADD_ICB_CONTROL("DSProxyControls.PredictedDelayMultiplier", 1'000, 1, 1'000'000, std::round(Settings.VDiskPredictedDelayMultiplier * 1'000)); + ADD_ICB_CONTROL("DSProxyControls.PredictedDelayMultiplierHDD", 1'000, 1, 1'000'000, std::round(Settings.VDiskPredictedDelayMultiplier * 1'000)); + ADD_ICB_CONTROL("DSProxyControls.PredictedDelayMultiplierSSD", 1'000, 1, 1'000'000, std::round(Settings.VDiskPredictedDelayMultiplier * 1'000)); + ADD_ICB_CONTROL("DSProxyControls.MaxNumOfSlowDisks", 2, 1, 2, Settings.MaxNumOfSlowDisks); + ADD_ICB_CONTROL("DSProxyControls.MaxNumOfSlowDisksHDD", 2, 1, 2, Settings.MaxNumOfSlowDisks); + ADD_ICB_CONTROL("DSProxyControls.MaxNumOfSlowDisksSSD", 2, 1, 2, Settings.MaxNumOfSlowDisks); + #undef ADD_ICB_CONTROL { diff --git a/ydb/core/blobstorage/ut_group/main.cpp b/ydb/core/blobstorage/ut_group/main.cpp index 4debcce43505..0f8c6a5a5db3 100644 --- a/ydb/core/blobstorage/ut_group/main.cpp +++ b/ydb/core/blobstorage/ut_group/main.cpp @@ -407,8 +407,15 @@ class TTestEnv { auto proxy = Counters->GetSubgroup("subsystem", "proxy"); TIntrusivePtr mon = MakeIntrusive(proxy, true); StoragePoolCounters = MakeIntrusive(proxy, TString(), NPDisk::DEVICE_TYPE_SSD); + TControlWrapper enablePutBatching(DefaultEnablePutBatching, false, true); + TControlWrapper enableVPatch(DefaultEnableVPatch, false, true); std::unique_ptr proxyActor{CreateBlobStorageGroupProxyConfigured(TIntrusivePtr(Info), false, mon, - TIntrusivePtr(StoragePoolCounters), DefaultEnablePutBatching, DefaultEnableVPatch)}; + TIntrusivePtr(StoragePoolCounters), TBlobStorageProxyParameters{ + .Controls = TBlobStorageProxyControlWrappers{ + .EnablePutBatching = enablePutBatching, + .EnableVPatch = enableVPatch, + } + })}; const TActorId& actorId = runtime.Register(proxyActor.release(), TActorId(), 0, std::nullopt, 1); runtime.RegisterService(MakeBlobStorageProxyID(GroupId), actorId); } diff --git a/ydb/core/blobstorage/vdisk/balance/balancing_actor.cpp b/ydb/core/blobstorage/vdisk/balance/balancing_actor.cpp index ff14300f9b84..9fcea654e21a 100644 --- a/ydb/core/blobstorage/vdisk/balance/balancing_actor.cpp +++ b/ydb/core/blobstorage/vdisk/balance/balancing_actor.cpp @@ -157,7 +157,7 @@ namespace NBalancing { CreateQueuesForVDisks(*QueueActorMapPtr, SelfId(), GInfo, Ctx->VCtx, GInfo->GetVDisks(), Ctx->MonGroup.GetGroup(), queueClientId, NKikimrBlobStorage::EVDiskQueueId::GetAsyncRead, - "DisksBalancing", interconnectChannel); + "DisksBalancing", interconnectChannel, false); } void Handle(NActors::TEvents::TEvUndelivered::TPtr ev) { diff --git a/ydb/core/blobstorage/vdisk/common/vdisk_config.h b/ydb/core/blobstorage/vdisk/common/vdisk_config.h index 73b1d27daaa1..84f791ae2a48 100644 --- a/ydb/core/blobstorage/vdisk/common/vdisk_config.h +++ b/ydb/core/blobstorage/vdisk/common/vdisk_config.h @@ -219,6 +219,7 @@ namespace NKikimr { bool EnableVDiskCooldownTimeout; TControlWrapper EnableVPatch = true; TControlWrapper DefaultHugeGarbagePerMille; + bool UseActorSystemTimeInBSQueue = false; ///////////// COST METRICS SETTINGS //////////////// bool UseCostTracker = true; diff --git a/ydb/core/blobstorage/vdisk/common/vdisk_queues.h b/ydb/core/blobstorage/vdisk/common/vdisk_queues.h index 7f2c3e8af107..5ccbac0278c2 100644 --- a/ydb/core/blobstorage/vdisk/common/vdisk_queues.h +++ b/ydb/core/blobstorage/vdisk/common/vdisk_queues.h @@ -51,7 +51,7 @@ namespace NKikimr { const TIntrusivePtr<::NMonitoring::TDynamicCounters> &groupCounters, const NBackpressure::TQueueClientId &queueClientId, NKikimrBlobStorage::EVDiskQueueId vDiskQueueId, const TString &queueName, TInterconnectChannels::EInterconnectChannels interconnectChannel, - TWrapper wrapper = {}) + const bool useActorSystemTimeInBSQueue, TWrapper wrapper = {}) { for (auto &vdiskInfo : disks) { auto vdisk = GetVDiskID(vdiskInfo); @@ -62,7 +62,8 @@ namespace NKikimr { queue.reset(CreateVDiskBackpressureClient(gInfo, vdisk, vDiskQueueId, groupCounters, vCtx, queueClientId, queueName, interconnectChannel, vdiskActorId.NodeId() == parent.NodeId(), - TDuration::Minutes(1), flowRecord, NMonitoring::TCountableBase::EVisibility::Private)); + TDuration::Minutes(1), flowRecord, NMonitoring::TCountableBase::EVisibility::Private, + useActorSystemTimeInBSQueue)); TActorId serviceId = TActivationContext::Register(queue.release(), parent); EmplaceToContainer(cont, vdisk, wrapper.Wrap(std::move(serviceId))); } diff --git a/ydb/core/blobstorage/vdisk/repl/blobstorage_repl.cpp b/ydb/core/blobstorage/vdisk/repl/blobstorage_repl.cpp index b1b4e794e86e..c2daa082b723 100644 --- a/ydb/core/blobstorage/vdisk/repl/blobstorage_repl.cpp +++ b/ydb/core/blobstorage/vdisk/repl/blobstorage_repl.cpp @@ -209,7 +209,7 @@ namespace NKikimr { CreateQueuesForVDisks(*QueueActorMapPtr, SelfId(), ReplCtx->GInfo, ReplCtx->VCtx, ReplCtx->GInfo->GetVDisks(), ReplCtx->MonGroup.GetGroup(), replQueueClientId, NKikimrBlobStorage::EVDiskQueueId::GetAsyncRead, - "PeerRepl", replInterconnectChannel); + "PeerRepl", replInterconnectChannel, false); for (const auto& [vdiskId, vdiskActorId] : ReplCtx->VDiskCfg->BaseInfo.DonorDiskIds) { TIntrusivePtr flowRecord(new NBackpressure::TFlowRecord); diff --git a/ydb/core/blobstorage/vdisk/scrub/blob_recovery_queue.cpp b/ydb/core/blobstorage/vdisk/scrub/blob_recovery_queue.cpp index f1c0d7052dc6..5d274fbdf261 100644 --- a/ydb/core/blobstorage/vdisk/scrub/blob_recovery_queue.cpp +++ b/ydb/core/blobstorage/vdisk/scrub/blob_recovery_queue.cpp @@ -14,7 +14,7 @@ namespace NKikimr { Info->GetTotalVDisksNum() + Info->GetOrderNumber(VCtx->ShortSelfVDisk)); // distinct queue client id CreateQueuesForVDisks(Queues, SelfId(), Info, VCtx, Info->GetVDisks(), Counters, clientId, NKikimrBlobStorage::EVDiskQueueId::GetLowRead, "PeerScrub", - TInterconnectChannels::IC_BLOBSTORAGE_ASYNC_DATA, TQueueActorIdWrapper()); + TInterconnectChannels::IC_BLOBSTORAGE_ASYNC_DATA, false, TQueueActorIdWrapper()); } void TBlobRecoveryActor::StopQueues() { diff --git a/ydb/core/blobstorage/vdisk/skeleton/blobstorage_skeleton.cpp b/ydb/core/blobstorage/vdisk/skeleton/blobstorage_skeleton.cpp index c7a7ff970a4c..1feffca3aaa5 100644 --- a/ydb/core/blobstorage/vdisk/skeleton/blobstorage_skeleton.cpp +++ b/ydb/core/blobstorage/vdisk/skeleton/blobstorage_skeleton.cpp @@ -258,7 +258,8 @@ namespace NKikimr { VCtx->Top->GetOrderNumber(VCtx->ShortSelfVDisk)); CreateQueuesForVDisks(VPatchCtx->AsyncBlobQueues, SelfId(), GInfo, VCtx, GInfo->GetVDisks(), patchGroup, patchQueueClientId, NKikimrBlobStorage::EVDiskQueueId::PutAsyncBlob, - "PeerVPatch", TInterconnectChannels::IC_BLOBSTORAGE_ASYNC_DATA); + "PeerVPatch", TInterconnectChannels::IC_BLOBSTORAGE_ASYNC_DATA, + Config->UseActorSystemTimeInBSQueue); } } diff --git a/ydb/core/cms/json_proxy_proto.h b/ydb/core/cms/json_proxy_proto.h index d65d419f4495..ef9a98c380ae 100644 --- a/ydb/core/cms/json_proxy_proto.h +++ b/ydb/core/cms/json_proxy_proto.h @@ -80,6 +80,8 @@ class TJsonProxyProto : public TActorBootstrapped { return ReplyWithTypeDescription(*NKikimrConfig::TImmediateControlsConfig::TVDiskControls::descriptor(), ctx); else if (name == ".NKikimrConfig.TImmediateControlsConfig.TTabletControls") return ReplyWithTypeDescription(*NKikimrConfig::TImmediateControlsConfig::TTabletControls::descriptor(), ctx); + else if (name == ".NKikimrConfig.TImmediateControlsConfig.TDSProxyControls") + return ReplyWithTypeDescription(*NKikimrConfig::TImmediateControlsConfig::TDSProxyControls::descriptor(), ctx); else if (name == ".NKikimrConfig.TImmediateControlsConfig.TBlobStorageControllerControls") return ReplyWithTypeDescription(*NKikimrConfig::TImmediateControlsConfig::TBlobStorageControllerControls::descriptor(), ctx); } diff --git a/ydb/core/protos/config.proto b/ydb/core/protos/config.proto index fe7003783f34..bf6b0ddc6216 100644 --- a/ydb/core/protos/config.proto +++ b/ydb/core/protos/config.proto @@ -1316,6 +1316,58 @@ message TImmediateControlsConfig { DefaultValue: 256 }]; } + message TDSProxyControls { + optional uint64 SlowDiskThreshold = 1 [(ControlOptions) = { + Description: "The minimum ratio of slowest and second slowest disks, required to accelerate, promille", + MinValue: 1, + MaxValue: 1000000, + DefaultValue: 2000 }]; + optional uint64 PredictedDelayMultiplier = 2 [(ControlOptions) = { + Description: "Predicted time of VDisk's response multiplier, promille", + MinValue: 0, + MaxValue: 1000000, + DefaultValue: 1000 }]; + reserved 3; + reserved 4; + optional uint64 MaxNumOfSlowDisks = 5 [(ControlOptions) = { + Description: "Maximum number of slow disks, which DSProxy can skip with Accelerations", + MinValue: 1, + MaxValue: 2, + DefaultValue: 2 }]; + + optional uint64 SlowDiskThresholdHDD = 6 [(ControlOptions) = { + Description: "The minimum ratio of slowest and second slowest disks, required to accelerate, promille, option for HDD", + MinValue: 1, + MaxValue: 1000000, + DefaultValue: 2000 }]; + optional uint64 PredictedDelayMultiplierHDD = 7 [(ControlOptions) = { + Description: "Predicted time of VDisk's response multiplier, promille, option for HDD", + MinValue: 0, + MaxValue: 1000000, + DefaultValue: 1000 }]; + optional uint64 MaxNumOfSlowDisksHDD = 8 [(ControlOptions) = { + Description: "Maximum number of slow disks, which DSProxy can skip with Accelerations, option for HDD", + MinValue: 1, + MaxValue: 2, + DefaultValue: 2 }]; + + optional uint64 SlowDiskThresholdSSD = 9 [(ControlOptions) = { + Description: "The minimum ratio of slowest and second slowest disks, required to accelerate, promille, option for SSD", + MinValue: 1, + MaxValue: 1000000, + DefaultValue: 2000 }]; + optional uint64 PredictedDelayMultiplierSSD = 10 [(ControlOptions) = { + Description: "Predicted time of VDisk's response multiplier, promille, option for SSD", + MinValue: 0, + MaxValue: 1000000, + DefaultValue: 1000 }]; + optional uint64 MaxNumOfSlowDisksSSD = 11 [(ControlOptions) = { + Description: "Maximum number of slow disks, which DSProxy can skip with Accelerations, option for SSD", + MinValue: 1, + MaxValue: 2, + DefaultValue: 2 }]; + } + message TBlobStorageControllerControls { optional uint64 EnableSelfHealWithDegraded = 1 [(ControlOptions) = { Description: "Should SelfHeal automatically process groups that are in DEGRADED status (one step from nonworking)", @@ -1332,6 +1384,7 @@ message TImmediateControlsConfig { reserved 6; optional TVDiskControls VDiskControls = 7; optional TTabletControls TabletControls = 8; + optional TDSProxyControls DSProxyControls = 9; optional TBlobStorageControllerControls BlobStorageControllerControls = 11; };