diff --git a/cloud/filestore/config/diagnostics.proto b/cloud/filestore/config/diagnostics.proto index 38596d19aa5..1e5d689b33a 100644 --- a/cloud/filestore/config/diagnostics.proto +++ b/cloud/filestore/config/diagnostics.proto @@ -19,6 +19,37 @@ message TMonitoringUrlData optional string MonitoringProject = 3; }; +//////////////////////////////////////////////////////////////////////////////// +// Performance profile settings for performance diagnostics (e.g. Suffer metric +// calculation). + +message TRequestPerformanceProfile +{ + // Requests per second. + optional uint64 RPS = 1; + // Throughput in any kind of units - for Read/Write it should be bytes per + // second, for ListNodes it can be entries per second, for other requests + // it can be 0 until we come up with something more useful. + optional uint64 Throughput = 2; +}; + +//////////////////////////////////////////////////////////////////////////////// +// Performance profile for filesystems / tablets / etc. + +message TFileSystemPerformanceProfile +{ + optional TRequestPerformanceProfile Read = 1; + optional TRequestPerformanceProfile Write = 2; + optional TRequestPerformanceProfile ListNodes = 3; + optional TRequestPerformanceProfile GetNodeAttr = 4; + optional TRequestPerformanceProfile CreateHandle = 5; + optional TRequestPerformanceProfile DestroyHandle = 6; + optional TRequestPerformanceProfile CreateNode = 7; + optional TRequestPerformanceProfile RenameNode = 8; + optional TRequestPerformanceProfile UnlinkNode = 9; + optional TRequestPerformanceProfile StatFileStore = 10; +}; + //////////////////////////////////////////////////////////////////////////////// message TDiagnosticsConfig @@ -27,15 +58,12 @@ message TDiagnosticsConfig optional string BastionNameSuffix = 1; // [obsolete] - // Solomon cluster name (e.g. yandexcloud_prod_vla). // optional string SolomonClusterName = 2; // [obsolete] - // Solomon host url. // optional string SolomonUrl = 3; // [obsolete] - // Project name in solomon. // optional string SolomonProject = 4; // Monserver will listen on this port. @@ -103,4 +131,10 @@ message TDiagnosticsConfig // Report histogram as a single counter (THistogramCounter) optional bool ReportHistogramAsSingleCounter = 25; + + // Performance profile for HDD filesystems. + optional TFileSystemPerformanceProfile HDDFileSystemPerformanceProfile = 26; + + // Performance profile for SSD filesystems. + optional TFileSystemPerformanceProfile SSDFileSystemPerformanceProfile = 27; } diff --git a/cloud/filestore/libs/diagnostics/config.cpp b/cloud/filestore/libs/diagnostics/config.cpp index e3ffc4670fd..1751e74b00d 100644 --- a/cloud/filestore/libs/diagnostics/config.cpp +++ b/cloud/filestore/libs/diagnostics/config.cpp @@ -37,6 +37,9 @@ namespace { xxx(MonitoringUrlData, TMonitoringUrlData, {} )\ xxx(ReportHistogramAsMultipleCounters, bool, true )\ xxx(ReportHistogramAsSingleCounter, bool, false )\ + \ + xxx(HDDFileSystemPerformanceProfile, TFileSystemPerformanceProfile, {} )\ + xxx(SSDFileSystemPerformanceProfile, TFileSystemPerformanceProfile, {} )\ // FILESTORE_DIAGNOSTICS_CONFIG #define FILESTORE_DIAGNOSTICS_DECLARE_CONFIG(name, type, value) \ @@ -77,6 +80,32 @@ ConvertValue( return TMonitoringUrlData(value); } +TRequestPerformanceProfile ConvertValue( + const NProto::TRequestPerformanceProfile& value) +{ + return {value.GetRPS(), value.GetThroughput()}; +} + +template <> +TFileSystemPerformanceProfile ConvertValue< + TFileSystemPerformanceProfile, + NProto::TFileSystemPerformanceProfile> +( + const NProto::TFileSystemPerformanceProfile& value) +{ + return { + ConvertValue(value.GetRead()), + ConvertValue(value.GetWrite()), + ConvertValue(value.GetListNodes()), + ConvertValue(value.GetGetNodeAttr()), + ConvertValue(value.GetCreateHandle()), + ConvertValue(value.GetDestroyHandle()), + ConvertValue(value.GetCreateNode()), + ConvertValue(value.GetRenameNode()), + ConvertValue(value.GetUnlinkNode()), + ConvertValue(value.GetStatFileStore())}; +} + } // namespace //////////////////////////////////////////////////////////////////////////////// @@ -170,3 +199,31 @@ void Out( v.SetMonitoringProject(value.MonitoringProject); SerializeToTextFormat(v, out); } + +void ConvertFromValue( + const NCloud::NFileStore::TRequestPerformanceProfile& v, + NCloud::NFileStore::NProto::TRequestPerformanceProfile* p) +{ + p->SetRPS(v.RPS); + p->SetThroughput(v.Throughput); +} + +template <> +void Out( + IOutputStream& out, + const NCloud::NFileStore::TFileSystemPerformanceProfile& value) +{ + NCloud::NFileStore::NProto::TFileSystemPerformanceProfile v; + ConvertFromValue(value.Read, v.MutableRead()); + ConvertFromValue(value.Write, v.MutableWrite()); + ConvertFromValue(value.ListNodes, v.MutableListNodes()); + ConvertFromValue(value.GetNodeAttr, v.MutableGetNodeAttr()); + ConvertFromValue(value.CreateHandle, v.MutableCreateHandle()); + ConvertFromValue(value.DestroyHandle, v.MutableDestroyHandle()); + ConvertFromValue(value.CreateNode, v.MutableCreateNode()); + ConvertFromValue(value.RenameNode, v.MutableRenameNode()); + ConvertFromValue(value.UnlinkNode, v.MutableUnlinkNode()); + ConvertFromValue(value.StatFileStore, v.MutableStatFileStore()); + + SerializeToTextFormat(v, out); +} diff --git a/cloud/filestore/libs/diagnostics/config.h b/cloud/filestore/libs/diagnostics/config.h index a774690a3dc..fe473ac4d30 100644 --- a/cloud/filestore/libs/diagnostics/config.h +++ b/cloud/filestore/libs/diagnostics/config.h @@ -14,6 +14,69 @@ namespace NCloud::NFileStore { //////////////////////////////////////////////////////////////////////////////// +struct TRequestPerformanceProfile +{ + ui64 RPS = 0; + ui64 Throughput = 0; + + TRequestPerformanceProfile() = default; + + TRequestPerformanceProfile(ui64 rps, ui64 throughput) + : RPS(rps) + , Throughput(throughput) + {} + + TRequestPerformanceProfile(const TRequestPerformanceProfile& rhs) = default; + TRequestPerformanceProfile& operator=( + const TRequestPerformanceProfile& rhs) = default; +}; + +struct TFileSystemPerformanceProfile +{ + TRequestPerformanceProfile Read; + TRequestPerformanceProfile Write; + TRequestPerformanceProfile ListNodes; + TRequestPerformanceProfile GetNodeAttr; + TRequestPerformanceProfile CreateHandle; + TRequestPerformanceProfile DestroyHandle; + TRequestPerformanceProfile CreateNode; + TRequestPerformanceProfile RenameNode; + TRequestPerformanceProfile UnlinkNode; + TRequestPerformanceProfile StatFileStore; + + TFileSystemPerformanceProfile() = default; + + TFileSystemPerformanceProfile( + TRequestPerformanceProfile read, + TRequestPerformanceProfile write, + TRequestPerformanceProfile listNodes, + TRequestPerformanceProfile getNodeAttr, + TRequestPerformanceProfile createHandle, + TRequestPerformanceProfile destroyHandle, + TRequestPerformanceProfile createNode, + TRequestPerformanceProfile renameNode, + TRequestPerformanceProfile unlinkNode, + TRequestPerformanceProfile statFileStore) + : Read(read) + , Write(write) + , ListNodes(listNodes) + , GetNodeAttr(getNodeAttr) + , CreateHandle(createHandle) + , DestroyHandle(destroyHandle) + , CreateNode(createNode) + , RenameNode(renameNode) + , UnlinkNode(unlinkNode) + , StatFileStore(statFileStore) + {} + + TFileSystemPerformanceProfile( + const TFileSystemPerformanceProfile& rhs) = default; + TFileSystemPerformanceProfile& operator=( + const TFileSystemPerformanceProfile& rhs) = default; +}; + +//////////////////////////////////////////////////////////////////////////////// + struct TMonitoringUrlData: public TAtomicRefCount { TString MonitoringClusterName; @@ -71,6 +134,9 @@ class TDiagnosticsConfig bool GetReportHistogramAsSingleCounter() const; EHistogramCounterOptions GetHistogramCounterOptions() const; + TFileSystemPerformanceProfile GetHDDFileSystemPerformanceProfile() const; + TFileSystemPerformanceProfile GetSSDFileSystemPerformanceProfile() const; + void Dump(IOutputStream& out) const; void DumpHtml(IOutputStream& out) const; }; diff --git a/cloud/filestore/libs/storage/init/actorsystem.cpp b/cloud/filestore/libs/storage/init/actorsystem.cpp index c29cd93c766..e465f0f9437 100644 --- a/cloud/filestore/libs/storage/init/actorsystem.cpp +++ b/cloud/filestore/libs/storage/init/actorsystem.cpp @@ -201,9 +201,11 @@ class TCustomLocalServiceInitializer final const TAppData* appData) override { auto config = Args.StorageConfig; + auto diagConfig = Args.DiagnosticsConfig; auto tabletFactory = [config, + diagConfig, profileLog = Args.ProfileLog, traceSerializer = Args.TraceSerializer, metricsRegistry = MetricsRegistry] ( @@ -219,6 +221,7 @@ class TCustomLocalServiceInitializer final owner, storage, config, + diagConfig, std::move(profileLog), std::move(traceSerializer), std::move(metricsRegistry), diff --git a/cloud/filestore/libs/storage/tablet/tablet.cpp b/cloud/filestore/libs/storage/tablet/tablet.cpp index 8d060bd1099..796e13314e0 100644 --- a/cloud/filestore/libs/storage/tablet/tablet.cpp +++ b/cloud/filestore/libs/storage/tablet/tablet.cpp @@ -14,6 +14,7 @@ IActorPtr CreateIndexTablet( const TActorId& owner, TTabletStorageInfoPtr storage, TStorageConfigPtr config, + TDiagnosticsConfigPtr diagConfig, IProfileLogPtr profileLog, ITraceSerializerPtr traceSerializer, NMetrics::IMetricsRegistryPtr metricsRegistry, @@ -23,6 +24,7 @@ IActorPtr CreateIndexTablet( owner, std::move(storage), std::move(config), + std::move(diagConfig), std::move(profileLog), std::move(traceSerializer), std::move(metricsRegistry), diff --git a/cloud/filestore/libs/storage/tablet/tablet.h b/cloud/filestore/libs/storage/tablet/tablet.h index bdd96255e4c..05848b2a227 100644 --- a/cloud/filestore/libs/storage/tablet/tablet.h +++ b/cloud/filestore/libs/storage/tablet/tablet.h @@ -18,6 +18,7 @@ NActors::IActorPtr CreateIndexTablet( const NActors::TActorId& owner, NKikimr::TTabletStorageInfoPtr storage, TStorageConfigPtr config, + TDiagnosticsConfigPtr diagConfig, IProfileLogPtr profileLog, ITraceSerializerPtr traceSerializer, NMetrics::IMetricsRegistryPtr metricsRegistry, diff --git a/cloud/filestore/libs/storage/tablet/tablet_actor.cpp b/cloud/filestore/libs/storage/tablet/tablet_actor.cpp index 0c41787bd63..478793f5c72 100644 --- a/cloud/filestore/libs/storage/tablet/tablet_actor.cpp +++ b/cloud/filestore/libs/storage/tablet/tablet_actor.cpp @@ -37,6 +37,7 @@ TIndexTabletActor::TIndexTabletActor( const TActorId& owner, TTabletStorageInfoPtr storage, TStorageConfigPtr config, + TDiagnosticsConfigPtr diagConfig, IProfileLogPtr profileLog, ITraceSerializerPtr traceSerializer, NMetrics::IMetricsRegistryPtr metricsRegistry, @@ -54,6 +55,7 @@ TIndexTabletActor::TIndexTabletActor( } ) , Config(std::move(config)) + , DiagConfig(std::move(diagConfig)) , UseNoneCompactionPolicy(useNoneCompactionPolicy) , BlobCodec(NBlockCodecs::Codec(Config->GetBlobCompressionCodec())) { @@ -921,6 +923,9 @@ STFUNC(TIndexTabletActor::StateInit) HFunc( TEvIndexTabletPrivate::TEvNodeUnlinkedInShard, HandleNodeUnlinkedInShard); + HFunc( + TEvIndexTabletPrivate::TEvGetShardStatsCompleted, + HandleGetShardStatsCompleted); FILESTORE_HANDLE_REQUEST(WaitReady, TEvIndexTablet) @@ -962,6 +967,9 @@ STFUNC(TIndexTabletActor::StateWork) HFunc( TEvIndexTabletPrivate::TEvNodeUnlinkedInShard, HandleNodeUnlinkedInShard); + HFunc( + TEvIndexTabletPrivate::TEvGetShardStatsCompleted, + HandleGetShardStatsCompleted); HFunc(TEvents::TEvWakeup, HandleWakeup); HFunc(TEvents::TEvPoisonPill, HandlePoisonPill); @@ -1020,6 +1028,9 @@ STFUNC(TIndexTabletActor::StateZombie) HFunc( TEvIndexTabletPrivate::TEvNodeUnlinkedInShard, HandleNodeUnlinkedInShard); + HFunc( + TEvIndexTabletPrivate::TEvGetShardStatsCompleted, + HandleGetShardStatsCompleted); default: HandleUnexpectedEvent(ev, TFileStoreComponents::TABLET); @@ -1060,6 +1071,9 @@ STFUNC(TIndexTabletActor::StateBroken) HFunc( TEvIndexTabletPrivate::TEvNodeUnlinkedInShard, HandleNodeUnlinkedInShard); + HFunc( + TEvIndexTabletPrivate::TEvGetShardStatsCompleted, + HandleGetShardStatsCompleted); default: HandleUnexpectedEvent(ev, TFileStoreComponents::TABLET); diff --git a/cloud/filestore/libs/storage/tablet/tablet_actor.h b/cloud/filestore/libs/storage/tablet/tablet_actor.h index 9749648df9b..0b66f70a56c 100644 --- a/cloud/filestore/libs/storage/tablet/tablet_actor.h +++ b/cloud/filestore/libs/storage/tablet/tablet_actor.h @@ -170,16 +170,89 @@ class TIndexTabletActor final struct TRequestMetrics { - std::atomic Count{0}; - std::atomic RequestBytes{0}; + std::atomic Count = 0; + std::atomic RequestBytes = 0; + std::atomic TimeSumUs = 0; TLatHistogram Time; + ui64 PrevCount = 0; + ui64 PrevRequestBytes = 0; + ui64 PrevTimeSumUs = 0; + TInstant PrevTs; + void Update(ui64 requestCount, ui64 requestBytes, TDuration d) { Count.fetch_add(requestCount, std::memory_order_relaxed); RequestBytes.fetch_add(requestBytes, std::memory_order_relaxed); + TimeSumUs.fetch_add( + d.MicroSeconds(), + std::memory_order_relaxed); Time.Record(d); } + + void UpdatePrev(TInstant now) + { + PrevCount = Count.load(std::memory_order_relaxed); + PrevRequestBytes = RequestBytes.load(std::memory_order_relaxed); + PrevTimeSumUs = TimeSumUs.load(std::memory_order_relaxed); + PrevTs = now; + } + + double RPS(TInstant now) const + { + return Rate(now, Count, PrevCount); + } + + double Throughput(TInstant now) const + { + return Rate(now, RequestBytes, PrevRequestBytes); + } + + ui64 AverageRequestSize() const + { + const auto requestCount = + Count.load(std::memory_order_relaxed) - PrevCount; + if (!requestCount) { + return 0; + } + + const auto requestBytes = + RequestBytes.load(std::memory_order_relaxed) + - PrevRequestBytes; + return requestBytes / requestCount; + } + + TDuration AverageLatency() const + { + const auto requestCount = + Count.load(std::memory_order_relaxed) - PrevCount; + if (!requestCount) { + return TDuration::Zero(); + } + + const auto timeSumUs = + TimeSumUs.load(std::memory_order_relaxed) - PrevTimeSumUs; + return TDuration::MicroSeconds(timeSumUs / requestCount); + } + + private: + double Rate( + TInstant now, + const std::atomic& counter, + ui64 prevCounter) const + { + if (!PrevTs) { + return 0; + } + + auto micros = (now - PrevTs).MicroSeconds(); + if (!micros) { + return 0; + } + + auto cur = counter.load(std::memory_order_relaxed); + return (cur - prevCounter) * 1'000'000. / micros; + } }; struct TCompactionMetrics: TRequestMetrics @@ -187,14 +260,27 @@ class TIndexTabletActor final std::atomic DudCount{0}; }; + // private requests TRequestMetrics ReadBlob; TRequestMetrics WriteBlob; TRequestMetrics PatchBlob; + + // public requests TRequestMetrics ReadData; TRequestMetrics DescribeData; TRequestMetrics WriteData; TRequestMetrics AddData; TRequestMetrics GenerateBlobIds; + TRequestMetrics ListNodes; + TRequestMetrics GetNodeAttr; + TRequestMetrics CreateHandle; + TRequestMetrics DestroyHandle; + TRequestMetrics CreateNode; + TRequestMetrics RenameNode; + TRequestMetrics UnlinkNode; + TRequestMetrics StatFileStore; + + // background requests TCompactionMetrics Compaction; TRequestMetrics Cleanup; TRequestMetrics Flush; @@ -205,11 +291,15 @@ class TIndexTabletActor final i64 LastNetworkMetric = 0; i64 CalculateNetworkRequestBytes(ui32 nonNetworkMetricsBalancingFactor); - // Compaction/cleanup stats + // Compaction/Cleanup stats std::atomic MaxBlobsInRange{0}; std::atomic MaxDeletionsInRange{0}; std::atomic MaxGarbageBlocksInRange{0}; + // performance evaluation + std::atomic CurrentLoad{0}; + std::atomic Suffer{0}; + const NMetrics::IMetricsRegistryPtr StorageRegistry; const NMetrics::IMetricsRegistryPtr StorageFsRegistry; @@ -220,6 +310,8 @@ class TIndexTabletActor final void Register(const TString& fsId, const TString& mediaKind); void Update( + TInstant now, + const TDiagnosticsConfig& diagConfig, const NProto::TFileSystem& fileSystem, const NProto::TFileSystemStats& stats, const NProto::TFileStorePerformanceProfile& performanceProfile, @@ -231,6 +323,10 @@ class TIndexTabletActor final const TNodeToSessionCounters& nodeToSessionCounters, const TMiscNodeStats& miscNodeStats, const TInMemoryIndexStateStats& inMemoryIndexStateStats); + void UpdatePerformanceMetrics( + TInstant now, + const TDiagnosticsConfig& diagConfig, + const NProto::TFileSystem& fileSystem); } Metrics; const IProfileLogPtr ProfileLog; @@ -256,6 +352,7 @@ class TIndexTabletActor final ITabletThrottlerPtr Throttler; TStorageConfigPtr Config; + TDiagnosticsConfigPtr DiagConfig; const bool UseNoneCompactionPolicy; @@ -283,6 +380,7 @@ class TIndexTabletActor final const NActors::TActorId& owner, NKikimr::TTabletStorageInfoPtr storage, TStorageConfigPtr config, + TDiagnosticsConfigPtr diagConfig, IProfileLogPtr profileLog, ITraceSerializerPtr traceSerializer, NMetrics::IMetricsRegistryPtr metricsRegistry, @@ -322,7 +420,7 @@ class TIndexTabletActor final void BecomeAux(const NActors::TActorContext& ctx, EState state); void ReportTabletState(const NActors::TActorContext& ctx); - void RegisterStatCounters(); + void RegisterStatCounters(TInstant now); void RegisterCounters(const NActors::TActorContext& ctx); void ScheduleUpdateCounters(const NActors::TActorContext& ctx); void UpdateCounters(); @@ -624,6 +722,10 @@ class TIndexTabletActor final const TEvIndexTabletPrivate::TEvNodeUnlinkedInShard::TPtr& ev, const NActors::TActorContext& ctx); + void HandleGetShardStatsCompleted( + const TEvIndexTabletPrivate::TEvGetShardStatsCompleted::TPtr& ev, + const NActors::TActorContext& ctx); + void HandleLoadCompactionMapChunkResponse( const TEvIndexTabletPrivate::TEvLoadCompactionMapChunkResponse::TPtr& ev, const NActors::TActorContext& ctx); diff --git a/cloud/filestore/libs/storage/tablet/tablet_actor_counters.cpp b/cloud/filestore/libs/storage/tablet/tablet_actor_counters.cpp index 7491292b97b..7400998812e 100644 --- a/cloud/filestore/libs/storage/tablet/tablet_actor_counters.cpp +++ b/cloud/filestore/libs/storage/tablet/tablet_actor_counters.cpp @@ -1,5 +1,6 @@ #include "tablet_actor.h" +#include #include #include #include @@ -21,6 +22,7 @@ class TGetShardStatsActor final { private: const TString LogTag; + const TActorId Tablet; const TRequestInfoPtr RequestInfo; const NProtoPrivate::TGetStorageStatsRequest Request; const google::protobuf::RepeatedPtrField ShardIds; @@ -30,6 +32,7 @@ class TGetShardStatsActor final public: TGetShardStatsActor( TString logTag, + TActorId tablet, TRequestInfoPtr requestInfo, NProtoPrivate::TGetStorageStatsRequest request, google::protobuf::RepeatedPtrField shardIds, @@ -59,11 +62,13 @@ class TGetShardStatsActor final TGetShardStatsActor::TGetShardStatsActor( TString logTag, + TActorId tablet, TRequestInfoPtr requestInfo, NProtoPrivate::TGetStorageStatsRequest request, google::protobuf::RepeatedPtrField shardIds, std::unique_ptr response) : LogTag(std::move(logTag)) + , Tablet(tablet) , RequestInfo(std::move(requestInfo)) , Request(std::move(request)) , ShardIds(std::move(shardIds)) @@ -158,6 +163,11 @@ void TGetShardStatsActor::ReplyAndDie( Response = std::make_unique(error); } + using TCompletion = TEvIndexTabletPrivate::TEvGetShardStatsCompleted; + auto response = std::make_unique( + error, + RequestInfo->StartedTs); + NCloud::Send(ctx, Tablet, std::move(response)); NCloud::Reply(ctx, *RequestInfo, std::move(Response)); Die(ctx); @@ -381,6 +391,10 @@ void TIndexTabletActor::TMetrics::Register( name.RequestBytes, \ EMetricType::MT_DERIVATIVE); \ \ + REGISTER_AGGREGATABLE_SUM( \ + name.TimeSumUs, \ + EMetricType::MT_DERIVATIVE); \ + \ name.Time.Register( \ AggregatableFsRegistry, \ {CreateLabel("request", #name), CreateLabel("histogram", "Time")}); \ @@ -389,11 +403,21 @@ void TIndexTabletActor::TMetrics::Register( REGISTER_REQUEST(ReadBlob); REGISTER_REQUEST(WriteBlob); REGISTER_REQUEST(PatchBlob); + REGISTER_REQUEST(ReadData); REGISTER_REQUEST(DescribeData); REGISTER_REQUEST(WriteData); REGISTER_REQUEST(AddData); REGISTER_REQUEST(GenerateBlobIds); + REGISTER_REQUEST(ListNodes); + REGISTER_REQUEST(GetNodeAttr); + REGISTER_REQUEST(CreateHandle); + REGISTER_REQUEST(DestroyHandle); + REGISTER_REQUEST(CreateNode); + REGISTER_REQUEST(RenameNode); + REGISTER_REQUEST(UnlinkNode); + REGISTER_REQUEST(StatFileStore); + REGISTER_REQUEST(Compaction); REGISTER_AGGREGATABLE_SUM(Compaction.DudCount, EMetricType::MT_DERIVATIVE); REGISTER_REQUEST(Cleanup); @@ -406,6 +430,9 @@ void TIndexTabletActor::TMetrics::Register( REGISTER_LOCAL(MaxDeletionsInRange, EMetricType::MT_ABSOLUTE); REGISTER_LOCAL(MaxGarbageBlocksInRange, EMetricType::MT_ABSOLUTE); + REGISTER_LOCAL(CurrentLoad, EMetricType::MT_ABSOLUTE); + REGISTER_LOCAL(Suffer, EMetricType::MT_ABSOLUTE); + #undef REGISTER_REQUEST #undef REGISTER_LOCAL #undef REGISTER_AGGREGATABLE_SUM @@ -417,6 +444,8 @@ void TIndexTabletActor::TMetrics::Register( } void TIndexTabletActor::TMetrics::Update( + TInstant now, + const TDiagnosticsConfig& diagConfig, const NProto::TFileSystem& fileSystem, const NProto::TFileSystemStats& stats, const NProto::TFileStorePerformanceProfile& performanceProfile, @@ -519,6 +548,82 @@ void TIndexTabletActor::TMetrics::Update( Store(OrphanNodesCount, miscNodeStats.OrphanNodesCount); BusyIdleCalc.OnUpdateStats(); + UpdatePerformanceMetrics(now, diagConfig, fileSystem); + + ReadBlob.UpdatePrev(now); + WriteBlob.UpdatePrev(now); + PatchBlob.UpdatePrev(now); + + ReadData.UpdatePrev(now); + DescribeData.UpdatePrev(now); + WriteData.UpdatePrev(now); + AddData.UpdatePrev(now); + GenerateBlobIds.UpdatePrev(now); + ListNodes.UpdatePrev(now); + GetNodeAttr.UpdatePrev(now); + CreateHandle.UpdatePrev(now); + DestroyHandle.UpdatePrev(now); + CreateNode.UpdatePrev(now); + RenameNode.UpdatePrev(now); + UnlinkNode.UpdatePrev(now); + StatFileStore.UpdatePrev(now); + + Cleanup.UpdatePrev(now); + Flush.UpdatePrev(now); + FlushBytes.UpdatePrev(now); + TrimBytes.UpdatePrev(now); + CollectGarbage.UpdatePrev(now); +} + +void TIndexTabletActor::TMetrics::UpdatePerformanceMetrics( + TInstant now, + const TDiagnosticsConfig& diagConfig, + const NProto::TFileSystem& fileSystem) +{ + const ui32 expectedParallelism = 32; + double load = 0; + bool suffer = false; + auto calcSufferAndLoad = [&] ( + const TRequestPerformanceProfile& rpp, + const TRequestMetrics& rm) + { + if (!rpp.RPS) { + return; + } + + load += rm.RPS(now) / rpp.RPS; + ui64 expectedLatencyUs = 1'000'000 / rpp.RPS; + if (rpp.Throughput) { + expectedLatencyUs += + 1'000'000 * rm.AverageRequestSize() / rpp.Throughput; + load += rm.Throughput(now) / rpp.Throughput; + } + + const auto averageLatency = rm.AverageLatency(); + suffer |= TDuration::MicroSeconds(expectedLatencyUs) + < averageLatency / expectedParallelism; + }; + + const auto& pp = + fileSystem.GetStorageMediaKind() == NProto::STORAGE_MEDIA_SSD + ? diagConfig.GetSSDFileSystemPerformanceProfile() + : diagConfig.GetHDDFileSystemPerformanceProfile(); + + calcSufferAndLoad(pp.Read, ReadData); + calcSufferAndLoad(pp.Read, DescribeData); + calcSufferAndLoad(pp.Write, WriteData); + calcSufferAndLoad(pp.Write, AddData); + calcSufferAndLoad(pp.ListNodes, ListNodes); + calcSufferAndLoad(pp.GetNodeAttr, GetNodeAttr); + calcSufferAndLoad(pp.CreateHandle, CreateHandle); + calcSufferAndLoad(pp.DestroyHandle, DestroyHandle); + calcSufferAndLoad(pp.CreateNode, CreateNode); + calcSufferAndLoad(pp.RenameNode, RenameNode); + calcSufferAndLoad(pp.UnlinkNode, UnlinkNode); + calcSufferAndLoad(pp.StatFileStore, StatFileStore); + + Store(CurrentLoad, load * 1000); + Store(Suffer, load < 1 ? suffer : 0); } //////////////////////////////////////////////////////////////////////////////// @@ -540,7 +645,7 @@ void TIndexTabletActor::RegisterCounters(const TActorContext& ctx) } } -void TIndexTabletActor::RegisterStatCounters() +void TIndexTabletActor::RegisterStatCounters(TInstant now) { const auto& fsId = GetFileSystemId(); if (!fsId) { @@ -558,6 +663,8 @@ void TIndexTabletActor::RegisterStatCounters() // aggregate zero values, in the middle of the registration (or right after // registration, before update). Metrics.Update( + now, + *DiagConfig, fs, GetFileSystemStats(), GetPerformanceProfile(), @@ -606,6 +713,8 @@ void TIndexTabletActor::HandleUpdateCounters( UpdateCounters(); Metrics.Update( + ctx.Now(), + *DiagConfig, GetFileSystem(), GetFileSystemStats(), GetPerformanceProfile(), @@ -711,6 +820,7 @@ void TIndexTabletActor::HandleGetStorageStats( const auto& shardIds = GetFileSystem().GetShardFileSystemIds(); if (shardIds.empty()) { + Metrics.StatFileStore.Update(1, 0, TDuration::Zero()); NCloud::Reply(ctx, *ev, std::move(response)); return; } @@ -722,15 +832,26 @@ void TIndexTabletActor::HandleGetStorageStats( auto actor = std::make_unique( LogTag, + SelfId(), std::move(requestInfo), std::move(req), shardIds, std::move(response)); auto actorId = NCloud::Register(ctx, std::move(actor)); + WorkerActors.insert(actorId); +} + +//////////////////////////////////////////////////////////////////////////////// - Y_UNUSED(actorId); - // TODO(#1350): register actorId in WorkerActors, erase upon completion +void TIndexTabletActor::HandleGetShardStatsCompleted( + const TEvIndexTabletPrivate::TEvGetShardStatsCompleted::TPtr& ev, + const TActorContext& ctx) +{ + if (!HasError(ev->Get()->Error)) { + Metrics.StatFileStore.Update(1, 0, ctx.Now() - ev->Get()->StartedTs); + } + WorkerActors.erase(ev->Sender); } } // namespace NCloud::NFileStore::NStorage diff --git a/cloud/filestore/libs/storage/tablet/tablet_actor_createhandle.cpp b/cloud/filestore/libs/storage/tablet/tablet_actor_createhandle.cpp index 1d2b891fafc..8d6825eb4f2 100644 --- a/cloud/filestore/libs/storage/tablet/tablet_actor_createhandle.cpp +++ b/cloud/filestore/libs/storage/tablet/tablet_actor_createhandle.cpp @@ -444,6 +444,11 @@ void TIndexTabletActor::CompleteTx_CreateHandle( if (!HasError(args.Error)) { CommitDupCacheEntry(args.SessionId, args.RequestId); response->Record = std::move(args.Response); + + Metrics.CreateHandle.Update( + 1, + 0, + ctx.Now() - args.RequestInfo->StartedTs); } CompleteResponse( diff --git a/cloud/filestore/libs/storage/tablet/tablet_actor_createnode.cpp b/cloud/filestore/libs/storage/tablet/tablet_actor_createnode.cpp index 0184b021cff..fc9ea00e42b 100644 --- a/cloud/filestore/libs/storage/tablet/tablet_actor_createnode.cpp +++ b/cloud/filestore/libs/storage/tablet/tablet_actor_createnode.cpp @@ -625,6 +625,11 @@ void TIndexTabletActor::CompleteTx_CreateNode( args.RequestInfo->CallContext, ctx); + Metrics.CreateNode.Update( + 1, + 0, + ctx.Now() - args.RequestInfo->StartedTs); + NCloud::Reply(ctx, *args.RequestInfo, std::move(response)); } @@ -654,6 +659,11 @@ void TIndexTabletActor::HandleNodeCreatedInShard( msg->RequestInfo->CallContext, ctx); + Metrics.CreateNode.Update( + 1, + 0, + ctx.Now() - msg->RequestInfo->StartedTs); + // replying before DupCacheEntry is committed to reduce response // latency NCloud::Reply(ctx, *msg->RequestInfo, std::move(response)); @@ -678,6 +688,11 @@ void TIndexTabletActor::HandleNodeCreatedInShard( msg->RequestInfo->CallContext, ctx); + Metrics.CreateHandle.Update( + 1, + 0, + ctx.Now() - msg->RequestInfo->StartedTs); + NCloud::Reply(ctx, *msg->RequestInfo, std::move(response)); } diff --git a/cloud/filestore/libs/storage/tablet/tablet_actor_destroyhandle.cpp b/cloud/filestore/libs/storage/tablet/tablet_actor_destroyhandle.cpp index a4395fca245..9cc666ead8b 100644 --- a/cloud/filestore/libs/storage/tablet/tablet_actor_destroyhandle.cpp +++ b/cloud/filestore/libs/storage/tablet/tablet_actor_destroyhandle.cpp @@ -128,7 +128,15 @@ void TIndexTabletActor::CompleteTx_DestroyHandle( { RemoveTransaction(*args.RequestInfo); - auto response = std::make_unique(args.Error); + if (!HasError(args.Error)) { + Metrics.DestroyHandle.Update( + 1, + 0, + ctx.Now() - args.RequestInfo->StartedTs); + } + + auto response = + std::make_unique(args.Error); CompleteResponse( response->Record, args.RequestInfo->CallContext, diff --git a/cloud/filestore/libs/storage/tablet/tablet_actor_getnodeattr.cpp b/cloud/filestore/libs/storage/tablet/tablet_actor_getnodeattr.cpp index 35c4c5fea8b..67e40f22fc4 100644 --- a/cloud/filestore/libs/storage/tablet/tablet_actor_getnodeattr.cpp +++ b/cloud/filestore/libs/storage/tablet/tablet_actor_getnodeattr.cpp @@ -84,6 +84,7 @@ void TIndexTabletActor::HandleGetNodeAttr( Metrics.NodeIndexCacheHitCount.fetch_add( 1, std::memory_order_relaxed); + Metrics.GetNodeAttr.Update(1, 0, TDuration::Zero()); NCloud::Reply(ctx, *requestInfo, std::move(response)); return; @@ -216,6 +217,11 @@ void TIndexTabletActor::CompleteTx_GetNodeAttr( args.Name, *node); } + + Metrics.GetNodeAttr.Update( + 1, + 0, + ctx.Now() - args.RequestInfo->StartedTs); } CompleteResponse( @@ -274,6 +280,8 @@ void TIndexTabletActor::HandleGetNodeAttrBatch( msg->CallContext, ctx); + Metrics.GetNodeAttr.Update(cacheHits, 0, TDuration::Zero()); + NCloud::Reply(ctx, *requestInfo, std::move(response)); return; } @@ -437,6 +445,11 @@ void TIndexTabletActor::CompleteTx_GetNodeAttrBatch( } response->Record = std::move(args.Response); + + Metrics.GetNodeAttr.Update( + args.Request.NamesSize(), + 0, + ctx.Now() - args.RequestInfo->StartedTs); } CompleteResponse( diff --git a/cloud/filestore/libs/storage/tablet/tablet_actor_listnodes.cpp b/cloud/filestore/libs/storage/tablet/tablet_actor_listnodes.cpp index 3d381d97e5c..675fcaf04fb 100644 --- a/cloud/filestore/libs/storage/tablet/tablet_actor_listnodes.cpp +++ b/cloud/filestore/libs/storage/tablet/tablet_actor_listnodes.cpp @@ -187,9 +187,12 @@ void TIndexTabletActor::CompleteTx_ListNodes( record.MutableNames()->Reserve(args.ChildRefs.size()); record.MutableNodes()->Reserve(args.ChildRefs.size()); + ui64 requestBytes = 0; + size_t j = 0; for (size_t i = 0; i < args.ChildRefs.size(); ++i) { const auto& ref = args.ChildRefs[i]; + requestBytes += ref.Name.size(); if (ref.ShardId) { AddExternalNode( record, @@ -211,6 +214,11 @@ void TIndexTabletActor::CompleteTx_ListNodes( if (args.Next) { record.SetCookie(args.Next); } + + Metrics.ListNodes.Update( + 1, + requestBytes, + ctx.Now() - args.RequestInfo->StartedTs); } CompleteResponse( diff --git a/cloud/filestore/libs/storage/tablet/tablet_actor_loadstate.cpp b/cloud/filestore/libs/storage/tablet/tablet_actor_loadstate.cpp index 943d969fd45..13626a0bf40 100644 --- a/cloud/filestore/libs/storage/tablet/tablet_actor_loadstate.cpp +++ b/cloud/filestore/libs/storage/tablet/tablet_actor_loadstate.cpp @@ -321,7 +321,7 @@ void TIndexTabletActor::CompleteTx_LoadState( EnqueueTruncateIfNeeded(ctx); RegisterFileStore(ctx); - RegisterStatCounters(); + RegisterStatCounters(ctx.Now()); ResetThrottlingPolicy(); LOG_INFO_S(ctx, TFileStoreComponents::TABLET, diff --git a/cloud/filestore/libs/storage/tablet/tablet_actor_readdata.cpp b/cloud/filestore/libs/storage/tablet/tablet_actor_readdata.cpp index b1a03f74e37..e08814501a1 100644 --- a/cloud/filestore/libs/storage/tablet/tablet_actor_readdata.cpp +++ b/cloud/filestore/libs/storage/tablet/tablet_actor_readdata.cpp @@ -660,11 +660,9 @@ void TIndexTabletActor::HandleDescribeData( NCloud::Reply(ctx, *requestInfo, std::move(response)); Metrics.ReadAheadCacheHitCount.fetch_add(1, std::memory_order_relaxed); - Metrics.DescribeData.Count.fetch_add(1, std::memory_order_relaxed); - Metrics.DescribeData.RequestBytes.fetch_add( + Metrics.DescribeData.Update( + 1, byteRange.Length, - std::memory_order_relaxed); - Metrics.DescribeData.Time.Record( ctx.Now() - requestInfo->StartedTs); return; @@ -870,11 +868,9 @@ void TIndexTabletActor::CompleteTx_ReadData( NCloud::Reply(ctx, *args.RequestInfo, std::move(response)); - Metrics.DescribeData.Count.fetch_add(1, std::memory_order_relaxed); - Metrics.DescribeData.RequestBytes.fetch_add( + Metrics.DescribeData.Update( + 1, args.OriginByteRange.Length, - std::memory_order_relaxed); - Metrics.DescribeData.Time.Record( ctx.Now() - args.RequestInfo->StartedTs); return; diff --git a/cloud/filestore/libs/storage/tablet/tablet_actor_renamenode.cpp b/cloud/filestore/libs/storage/tablet/tablet_actor_renamenode.cpp index 00ca1413e56..a785f98c13f 100644 --- a/cloud/filestore/libs/storage/tablet/tablet_actor_renamenode.cpp +++ b/cloud/filestore/libs/storage/tablet/tablet_actor_renamenode.cpp @@ -462,6 +462,11 @@ void TIndexTabletActor::CompleteTx_RenameNode( RemoveTransaction(*args.RequestInfo); + Metrics.RenameNode.Update( + 1, + 0, + ctx.Now() - args.RequestInfo->StartedTs); + auto response = std::make_unique(args.Error); CompleteResponse( response->Record, diff --git a/cloud/filestore/libs/storage/tablet/tablet_actor_unlinknode.cpp b/cloud/filestore/libs/storage/tablet/tablet_actor_unlinknode.cpp index 2cfbffc6f6c..ff8d12739dc 100644 --- a/cloud/filestore/libs/storage/tablet/tablet_actor_unlinknode.cpp +++ b/cloud/filestore/libs/storage/tablet/tablet_actor_unlinknode.cpp @@ -482,6 +482,11 @@ void TIndexTabletActor::CompleteTx_UnlinkNode( RemoveTransaction(*args.RequestInfo); EnqueueBlobIndexOpIfNeeded(ctx); + Metrics.UnlinkNode.Update( + 1, + 0, + ctx.Now() - args.RequestInfo->StartedTs); + auto response = std::make_unique(args.Error); CompleteResponse( @@ -515,6 +520,11 @@ void TIndexTabletActor::HandleNodeUnlinkedInShard( msg->RequestInfo->CallContext, ctx); + Metrics.RenameNode.Update( + 1, + 0, + ctx.Now() - msg->RequestInfo->StartedTs); + NCloud::Reply(ctx, *msg->RequestInfo, std::move(response)); } else if (auto* x = std::get_if(&res)) { auto response = @@ -526,6 +536,11 @@ void TIndexTabletActor::HandleNodeUnlinkedInShard( msg->RequestInfo->CallContext, ctx); + Metrics.UnlinkNode.Update( + 1, + 0, + ctx.Now() - msg->RequestInfo->StartedTs); + NCloud::Reply(ctx, *msg->RequestInfo, std::move(response)); } else { TABLET_VERIFY_C( diff --git a/cloud/filestore/libs/storage/tablet/tablet_actor_updateconfig.cpp b/cloud/filestore/libs/storage/tablet/tablet_actor_updateconfig.cpp index dcbe25449a0..c4198a17d9f 100644 --- a/cloud/filestore/libs/storage/tablet/tablet_actor_updateconfig.cpp +++ b/cloud/filestore/libs/storage/tablet/tablet_actor_updateconfig.cpp @@ -238,7 +238,7 @@ void TIndexTabletActor::CompleteTx_UpdateConfig( // update tablet id and stat counters w proper volume information UpdateLogTag(); RegisterFileStore(ctx); - RegisterStatCounters(); + RegisterStatCounters(ctx.Now()); ResetThrottlingPolicy(); LOG_DEBUG(ctx, TFileStoreComponents::TABLET, diff --git a/cloud/filestore/libs/storage/tablet/tablet_private.h b/cloud/filestore/libs/storage/tablet/tablet_private.h index 9b590c4828c..acde08f1dba 100644 --- a/cloud/filestore/libs/storage/tablet/tablet_private.h +++ b/cloud/filestore/libs/storage/tablet/tablet_private.h @@ -814,6 +814,15 @@ struct TEvIndexTabletPrivate ui64 CommitId = InvalidCommitId; }; + // + // GetShardStats + // + + struct TGetShardStatsCompleted + { + TInstant StartedTs; + }; + // // Events declaration // @@ -839,6 +848,8 @@ struct TEvIndexTabletPrivate EvNodeCreatedInShard, EvNodeUnlinkedInShard, + EvGetShardStatsCompleted, + EvEnd }; @@ -866,6 +877,9 @@ struct TEvIndexTabletPrivate using TEvNodeUnlinkedInShard = TRequestEvent; + + using TEvGetShardStatsCompleted = + TResponseEvent; }; } // namespace NCloud::NFileStore::NStorage diff --git a/cloud/filestore/libs/storage/tablet/tablet_ut_counters.cpp b/cloud/filestore/libs/storage/tablet/tablet_ut_counters.cpp index 021d1f78c18..f3961d38735 100644 --- a/cloud/filestore/libs/storage/tablet/tablet_ut_counters.cpp +++ b/cloud/filestore/libs/storage/tablet/tablet_ut_counters.cpp @@ -53,6 +53,30 @@ TFileSystemConfig MakeThrottlerConfig( //////////////////////////////////////////////////////////////////////////////// +NProto::TDiagnosticsConfig MakeDiagConfig() +{ + NProto::TDiagnosticsConfig config; + NProto::TFileSystemPerformanceProfile pp; + pp.MutableRead()->SetRPS(1000); + pp.MutableRead()->SetThroughput(100_MB); + pp.MutableWrite()->SetRPS(2000); + pp.MutableWrite()->SetThroughput(200_MB); + pp.MutableListNodes()->SetRPS(10); + pp.MutableGetNodeAttr()->SetRPS(5000); + pp.MutableCreateHandle()->SetRPS(100); + pp.MutableDestroyHandle()->SetRPS(40); + pp.MutableCreateNode()->SetRPS(50); + pp.MutableRenameNode()->SetRPS(70); + pp.MutableUnlinkNode()->SetRPS(80); + pp.MutableStatFileStore()->SetRPS(15); + + *config.MutableSSDFileSystemPerformanceProfile() = pp; + *config.MutableHDDFileSystemPerformanceProfile() = pp; + return config; +} + +//////////////////////////////////////////////////////////////////////////////// + struct TEnv : public NUnitTest::TBaseFixture { @@ -61,6 +85,15 @@ struct TEnv TTestRegistryVisitor Visitor; + TEnv() + : Env( + TTestEnvConfig{}, + NProto::TStorageConfig{}, + NKikimr::NFake::TCaches{}, + CreateProfileLogStub(), + MakeDiagConfig()) + {} + void SetUp(NUnitTest::TTestContext& /*context*/) override { Env.CreateSubDomain("nfs"); @@ -390,7 +423,8 @@ Y_UNIT_TEST_SUITE(TIndexTabletTest_Counters) Tablet->InitSession("client", "session"); - auto id = CreateNode(*Tablet, TCreateNodeArgs::File(RootNodeId, "test")); + auto id = + CreateNode(*Tablet, TCreateNodeArgs::File(RootNodeId, "test")); auto handle = CreateHandle(*Tablet, id); const auto sz = 256_KB; @@ -515,9 +549,70 @@ Y_UNIT_TEST_SUITE(TIndexTabletTest_Counters) {{ {"sensor", "AddData.RequestBytes"}, {"filesystem", "test"}}, sz}, + {{ + {"sensor", "CurrentLoad"}, + {"filesystem", "test"}}, 0}, }); + Tablet->AdvanceTime(TDuration::Seconds(15)); + Env.GetRuntime().DispatchEvents({}, TDuration::Seconds(5)); + registry->Visit(TInstant::Zero(), Visitor); + Visitor.ValidateExpectedCounters({ + {{ + {"sensor", "CurrentLoad"}, + {"filesystem", "test"}}, 7}, + {{ + {"sensor", "Suffer"}, + {"filesystem", "test"}}, 0}, + }); + + Tablet->DestroyHandle(handle); + } + + Y_UNIT_TEST_F(ShouldCalculateLoadAndSuffer, TEnv) + { + auto registry = Env.GetRegistry(); + + Tablet->InitSession("client", "session"); + + auto id = + CreateNode(*Tablet, TCreateNodeArgs::File(RootNodeId, "test")); + auto handle = CreateHandle(*Tablet, id); + + const auto sz = 256_KB; + + Tablet->WriteData(handle, 0, sz, 'a'); + Tablet->ListNodes(RootNodeId); + Tablet->GetNodeAttr(id); + Tablet->RenameNode(RootNodeId, "test", RootNodeId, "test2"); + Tablet->UnlinkNode(RootNodeId, "test2", false); + Tablet->GetStorageStats(); + + { + auto response = Tablet->ReadData(handle, 0, sz); + const auto& buffer = response->Record.GetBuffer(); + UNIT_ASSERT(CompareBuffer(buffer, sz, 'a')); + } + + { + auto response = Tablet->DescribeData(handle, 0, sz); + const auto& blobs = response->Record.GetBlobPieces(); + UNIT_ASSERT_VALUES_EQUAL(1, blobs.size()); + } + Tablet->DestroyHandle(handle); + + Tablet->AdvanceTime(TDuration::Seconds(15)); + Env.GetRuntime().DispatchEvents({}, TDuration::Seconds(5)); + registry->Visit(TInstant::Zero(), Visitor); + Visitor.ValidateExpectedCounters({ + {{ + {"sensor", "CurrentLoad"}, + {"filesystem", "test"}}, 17}, + {{ + {"sensor", "Suffer"}, + {"filesystem", "test"}}, 1}, + }); } Y_UNIT_TEST(ShouldCorrectlyWriteCompactionStats) diff --git a/cloud/filestore/libs/storage/testlib/test_env.cpp b/cloud/filestore/libs/storage/testlib/test_env.cpp index ed17813544b..8ff9737af06 100644 --- a/cloud/filestore/libs/storage/testlib/test_env.cpp +++ b/cloud/filestore/libs/storage/testlib/test_env.cpp @@ -75,7 +75,8 @@ TTestEnv::TTestEnv( const TTestEnvConfig& config, NProto::TStorageConfig storageConfig, NKikimr::NFake::TCaches cachesConfig, - IProfileLogPtr profileLog) + IProfileLogPtr profileLog, + NProto::TDiagnosticsConfig diagConfig) : Config(config) , Logging(CreateLoggingService("console", { TLOG_DEBUG })) , ProfileLog(std::move(profileLog)) @@ -92,6 +93,8 @@ TTestEnv::TTestEnv( StorageConfig = CreateTestStorageConfig(std::move(storageConfig)); + DiagConfig = std::make_shared(std::move(diagConfig)); + TAppPrepare app; // app.SetEnableSchemeBoard(true); SetupLogging(); @@ -108,7 +111,7 @@ TTestEnv::TTestEnv( StatsRegistry = CreateRequestStatsRegistry( "service", - std::make_shared(), + DiagConfig, Counters, CreateWallClockTimer(), NCloud::NStorage::NUserStats::CreateUserCounterSupplierStub()); @@ -458,6 +461,7 @@ ui64 TTestEnv::BootIndexTablet(ui32 nodeIdx) owner, storage, StorageConfig, + DiagConfig, ProfileLog, TraceSerializer, Registry, @@ -619,6 +623,7 @@ void TTestEnv::SetupLocalServiceConfig( owner, storage, StorageConfig, + DiagConfig, ProfileLog, TraceSerializer, Registry, diff --git a/cloud/filestore/libs/storage/testlib/test_env.h b/cloud/filestore/libs/storage/testlib/test_env.h index 5c1000225e6..720ffb1a9f3 100644 --- a/cloud/filestore/libs/storage/testlib/test_env.h +++ b/cloud/filestore/libs/storage/testlib/test_env.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -82,6 +83,7 @@ class TTestEnv TTestEnvConfig Config; ILoggingServicePtr Logging; TStorageConfigPtr StorageConfig; + TDiagnosticsConfigPtr DiagConfig; IProfileLogPtr ProfileLog; ITraceSerializerPtr TraceSerializer; @@ -102,7 +104,8 @@ class TTestEnv const TTestEnvConfig& config = {}, NProto::TStorageConfig storageConfig = {}, NKikimr::NFake::TCaches cachesConfig = {}, - IProfileLogPtr profileLog = CreateProfileLogStub()); + IProfileLogPtr profileLog = CreateProfileLogStub(), + NProto::TDiagnosticsConfig diagConfig = {}); NActors::TTestActorRuntime& GetRuntime() {