diff --git a/ydb/core/blobstorage/nodewarden/distconf.cpp b/ydb/core/blobstorage/nodewarden/distconf.cpp index 29476d9ff1ce..9221631d6fa7 100644 --- a/ydb/core/blobstorage/nodewarden/distconf.cpp +++ b/ydb/core/blobstorage/nodewarden/distconf.cpp @@ -19,10 +19,11 @@ namespace NKikimr::NStorage { STLOG(PRI_DEBUG, BS_NODE, NWDC00, "Bootstrap"); auto ns = NNodeBroker::BuildNameserverTable(Cfg->NameserviceConfig); - auto ev = std::make_unique(); + auto nodes = MakeIntrusive>(); for (const auto& [nodeId, item] : ns->StaticNodeTable) { - ev->Nodes.emplace_back(nodeId, item.Address, item.Host, item.ResolveHost, item.Port, item.Location); + nodes->emplace_back(nodeId, item.Address, item.Host, item.ResolveHost, item.Port, item.Location); } + auto ev = std::make_unique(nodes); Send(SelfId(), ev.release()); // and subscribe for the node list too diff --git a/ydb/core/fq/libs/actors/nodes_manager.cpp b/ydb/core/fq/libs/actors/nodes_manager.cpp index b422fc77c470..9646da107187 100644 --- a/ydb/core/fq/libs/actors/nodes_manager.cpp +++ b/ydb/core/fq/libs/actors/nodes_manager.cpp @@ -259,14 +259,13 @@ class TNodesManagerActor : public NActors::TActorBootstrappedGet()->Status.GetStatus(); - THolder nameServiceUpdateReq(new TEvInterconnect::TEvNodesInfo()); if (!ev->Get()->Status.IsSuccess()) { ythrow yexception() << status << '\n' << ev->Get()->Status.GetIssues().ToString(); } const auto& res = ev->Get()->Result; - auto& nodesInfo = nameServiceUpdateReq->Nodes; - nodesInfo.reserve(res.nodes().size()); + auto nodesInfo = MakeIntrusive>(); + nodesInfo->reserve(res.nodes().size()); Peers.clear(); std::set nodeIds; // may be not unique @@ -281,7 +280,7 @@ class TNodesManagerActor : public NActors::TActorBootstrappedemplace_back(TEvInterconnect::TNodeInfo{ node.node_id(), node.node_address(), node.hostname(), // host @@ -297,8 +296,9 @@ class TNodesManagerActor : public NActors::TActorBootstrappedGetCounter("PeerCount", false)->Set(Peers.size()); ServiceCounters.Counters->GetCounter("NodesHealthCheckOk", true)->Inc(); - LOG_T("Send NodeInfo with size: " << nodesInfo.size() << " to DynamicNameserver"); - if (!nodesInfo.empty()) { + LOG_T("Send NodeInfo with size: " << nodesInfo->size() << " to DynamicNameserver"); + if (!nodesInfo->empty()) { + THolder nameServiceUpdateReq(new TEvInterconnect::TEvNodesInfo(nodesInfo)); Send(GetNameserviceActorId(), nameServiceUpdateReq.Release()); } } catch (yexception &e) { diff --git a/ydb/core/health_check/health_check_ut.cpp b/ydb/core/health_check/health_check_ut.cpp index eefa30345e57..2d4c7212ff9d 100644 --- a/ydb/core/health_check/health_check_ut.cpp +++ b/ydb/core/health_check/health_check_ut.cpp @@ -337,11 +337,15 @@ Y_UNIT_TEST_SUITE(THealthCheckTest) { } void SetLongHostValue(TEvInterconnect::TEvNodesInfo::TPtr* ev) { + auto nodes = MakeIntrusive>((*ev)->Get()->Nodes); TString host(1000000, 'a'); - auto& pbRecord = (*ev)->Get()->Nodes; - for (auto itIssue = pbRecord.begin(); itIssue != pbRecord.end(); ++itIssue) { - itIssue->Host = host; + for (auto it = nodes->begin(); it != nodes->end(); ++it) { + it->Host = host; } + auto newEv = IEventHandle::Downcast( + new IEventHandle((*ev)->Recipient, (*ev)->Sender, new TEvInterconnect::TEvNodesInfo(nodes)) + ); + ev->Swap(newEv); } Ydb::Monitoring::SelfCheckResult RequestHc(size_t const groupNumber, size_t const vdiscPerGroupNumber, bool const isMergeRecords = false, bool const largeSizeVdisksIssues = false) { diff --git a/ydb/core/mind/bscontroller/impl.h b/ydb/core/mind/bscontroller/impl.h index c8e24f664880..46391caef516 100644 --- a/ydb/core/mind/bscontroller/impl.h +++ b/ydb/core/mind/bscontroller/impl.h @@ -1412,9 +1412,9 @@ class TBlobStorageController : public TActor, public TTa TNodeId NodeId; TNodeLocation Location; - THostRecord(TEvInterconnect::TNodeInfo&& nodeInfo) + THostRecord(const TEvInterconnect::TNodeInfo& nodeInfo) : NodeId(nodeInfo.NodeId) - , Location(std::move(nodeInfo.Location)) + , Location(nodeInfo.Location) {} }; @@ -1423,11 +1423,13 @@ class TBlobStorageController : public TActor, public TTa THashMap NodeIdToHostId; public: + THostRecordMapImpl() = default; + THostRecordMapImpl(TEvInterconnect::TEvNodesInfo *msg) { - for (TEvInterconnect::TNodeInfo& nodeInfo : msg->Nodes) { + for (const TEvInterconnect::TNodeInfo& nodeInfo : msg->Nodes) { const THostId hostId(nodeInfo.Host, nodeInfo.Port); NodeIdToHostId.emplace(nodeInfo.NodeId, hostId); - HostIdToRecord.emplace(hostId, std::move(nodeInfo)); + HostIdToRecord.emplace(hostId, nodeInfo); } } @@ -1805,8 +1807,7 @@ class TBlobStorageController : public TActor, public TTa // For test purposes, required for self heal actor void CreateEmptyHostRecordsMap() { - TEvInterconnect::TEvNodesInfo nodes; - HostRecords = std::make_shared(&nodes); + HostRecords = std::make_shared(); } ui64 NextConfigTxSeqNo = 1; diff --git a/ydb/core/mind/dynamic_nameserver.cpp b/ydb/core/mind/dynamic_nameserver.cpp index d6d135286404..7dd3964174b8 100644 --- a/ydb/core/mind/dynamic_nameserver.cpp +++ b/ydb/core/mind/dynamic_nameserver.cpp @@ -66,13 +66,18 @@ void TDynamicNodeResolverBase::Handle(TEvNodeBroker::TEvResolvedNode::TPtr &ev, if (rec.GetStatus().GetCode() != NKikimrNodeBroker::TStatus::OK) { // Reset proxy if node expired. - if (exists) + if (exists) { ResetInterconnectProxyConfig(NodeId, ctx); + ListNodesCache->Invalidate(); // node was erased + } ReplyWithErrorAndDie(ctx); return; } TDynamicConfig::TDynamicNodeInfo node(rec.GetNode()); + if (!exists || !oldNode.EqualExceptExpire(node)) { + ListNodesCache->Invalidate(); + } // If ID is re-used by another node then proxy has to be reset. if (exists && !oldNode.EqualExceptExpire(node)) @@ -202,31 +207,40 @@ void TDynamicNameserver::ResolveDynamicNode(ui32 nodeId, reply->NodeId = nodeId; ctx.Send(ev->Sender, reply); } else { - ctx.RegisterWithSameMailbox(new TDynamicNodeResolver(SelfId(), nodeId, DynamicConfigs[domain], ev, deadline)); + ctx.RegisterWithSameMailbox(new TDynamicNodeResolver(SelfId(), nodeId, DynamicConfigs[domain], + ListNodesCache, ev, deadline)); } } void TDynamicNameserver::SendNodesList(const TActorContext &ctx) -{ +{ auto now = ctx.Now(); - for (auto &sender : ListNodesQueue) { - THolder reply(new TEvInterconnect::TEvNodesInfo); + if (ListNodesCache->NeedUpdate(now)) { + auto newNodes = MakeIntrusive>(); + auto newExpire = now; + for (const auto &pr : StaticConfig->StaticNodeTable) { - reply->Nodes.emplace_back(pr.first, - pr.second.Address, pr.second.Host, pr.second.ResolveHost, - pr.second.Port, pr.second.Location, true); + newNodes->emplace_back(pr.first, + pr.second.Address, pr.second.Host, pr.second.ResolveHost, + pr.second.Port, pr.second.Location, true); } for (auto &config : DynamicConfigs) { for (auto &pr : config->DynamicNodes) { - if (pr.second.Expire > now) - reply->Nodes.emplace_back(pr.first, pr.second.Address, - pr.second.Host, pr.second.ResolveHost, - pr.second.Port, pr.second.Location, false); + if (pr.second.Expire > now) { + newNodes->emplace_back(pr.first, pr.second.Address, + pr.second.Host, pr.second.ResolveHost, + pr.second.Port, pr.second.Location, false); + newExpire = std::min(newExpire, pr.second.Expire); + } } } - ctx.Send(sender, reply.Release()); + ListNodesCache->Update(newNodes, newExpire); + } + + for (auto &sender : ListNodesQueue) { + ctx.Send(sender, new TEvInterconnect::TEvNodesInfo(ListNodesCache->GetNodes())); } ListNodesQueue.clear(); } @@ -282,6 +296,7 @@ void TDynamicNameserver::UpdateState(const NKikimrNodeBroker::TNodesInfo &rec, config->ExpiredNodes.emplace(node.GetNodeId(), info); } + ListNodesCache->Invalidate(); config->Epoch = rec.GetEpoch(); ctx.Schedule(config->Epoch.End - ctx.Now(), new TEvPrivate::TEvUpdateEpoch(domain, config->Epoch.Id + 1)); @@ -289,8 +304,10 @@ void TDynamicNameserver::UpdateState(const NKikimrNodeBroker::TNodesInfo &rec, // Note: this update may be optimized to only include new nodes for (auto &node : rec.GetNodes()) { auto nodeId = node.GetNodeId(); - if (!config->DynamicNodes.contains(nodeId)) + if (!config->DynamicNodes.contains(nodeId)) { config->DynamicNodes.emplace(nodeId, node); + ListNodesCache->Invalidate(); + } } config->Epoch = rec.GetEpoch(); } @@ -377,8 +394,8 @@ void TDynamicNameserver::Handle(TEvInterconnect::TEvGetNode::TPtr &ev, const TAc ctx.Send(ev->Sender, reply.Release()); } else { const TInstant deadline = ev->Get()->Deadline; - ctx.RegisterWithSameMailbox(new TDynamicNodeSearcher(SelfId(), nodeId, DynamicConfigs[domain], ev.Release(), - deadline)); + ctx.RegisterWithSameMailbox(new TDynamicNodeSearcher(SelfId(), nodeId, DynamicConfigs[domain], + ListNodesCache, ev.Release(), deadline)); } } } @@ -437,6 +454,7 @@ void TDynamicNameserver::Handle(NConsole::TEvConsole::TEvConfigNotificationReque auto newStaticConfig = BuildNameserverTable(config.GetNameserviceConfig()); if (StaticConfig->StaticNodeTable != newStaticConfig->StaticNodeTable) { StaticConfig = std::move(newStaticConfig); + ListNodesCache->Invalidate(); for (const auto& subscriber : StaticNodeChangeSubscribers) { TActivationContext::Send(new IEventHandle(SelfId(), subscriber, new TEvInterconnect::TEvListNodes)); } @@ -478,5 +496,29 @@ TIntrusivePtr BuildNameserverTable(const NKikimrConfig::T return table; } +TListNodesCache::TListNodesCache() + : Nodes(nullptr) + , Expire(TInstant::Zero()) +{} + + +void TListNodesCache::Update(TIntrusiveVector::TConstPtr newNodes, TInstant newExpire) { + Nodes = newNodes; + Expire = newExpire; +} + +void TListNodesCache::Invalidate() { + Nodes = nullptr; + Expire = TInstant::Zero(); +} + +bool TListNodesCache::NeedUpdate(TInstant now) const { + return Nodes == nullptr || now > Expire; +} + +TIntrusiveVector::TConstPtr TListNodesCache::GetNodes() const { + return Nodes; +} + } // NNodeBroker } // NKikimr diff --git a/ydb/core/mind/dynamic_nameserver_impl.h b/ydb/core/mind/dynamic_nameserver_impl.h index 68c0b26f4615..fd82d1ea4798 100644 --- a/ydb/core/mind/dynamic_nameserver_impl.h +++ b/ydb/core/mind/dynamic_nameserver_impl.h @@ -68,6 +68,19 @@ struct TDynamicConfig : public TThrRefBase { using TDynamicConfigPtr = TIntrusivePtr; +class TListNodesCache : public TSimpleRefCount { +public: + TListNodesCache(); + + void Update(TIntrusiveVector::TConstPtr newNodes, TInstant newExpire); + void Invalidate(); + bool NeedUpdate(TInstant now) const; + TIntrusiveVector::TConstPtr GetNodes() const; +private: + TIntrusiveVector::TConstPtr Nodes; + TInstant Expire; +}; + class TDynamicNodeResolverBase : public TActorBootstrapped { public: using TBase = TActorBootstrapped; @@ -77,10 +90,12 @@ class TDynamicNodeResolverBase : public TActorBootstrapped listNodesCache, TAutoPtr origRequest, TInstant deadline) : Owner(owner) , NodeId(nodeId) , Config(config) + , ListNodesCache(listNodesCache) , OrigRequest(origRequest) , Deadline(deadline) { @@ -117,6 +132,7 @@ class TDynamicNodeResolverBase : public TActorBootstrapped ListNodesCache; TAutoPtr OrigRequest; const TInstant Deadline; @@ -127,8 +143,9 @@ class TDynamicNodeResolverBase : public TActorBootstrapped listNodesCache, TAutoPtr origRequest, TInstant deadline) - : TDynamicNodeResolverBase(owner, nodeId, config, origRequest, deadline) + : TDynamicNodeResolverBase(owner, nodeId, config, listNodesCache, origRequest, deadline) { } @@ -139,8 +156,9 @@ class TDynamicNodeResolver : public TDynamicNodeResolverBase { class TDynamicNodeSearcher : public TDynamicNodeResolverBase { public: TDynamicNodeSearcher(TActorId owner, ui32 nodeId, TDynamicConfigPtr config, + TIntrusivePtr listNodesCache, TAutoPtr origRequest, TInstant deadline) - : TDynamicNodeResolverBase(owner, nodeId, config, origRequest, deadline) + : TDynamicNodeResolverBase(owner, nodeId, config, listNodesCache, origRequest, deadline) { } @@ -179,6 +197,7 @@ class TDynamicNameserver : public TActorBootstrapped { TDynamicNameserver(const TIntrusivePtr &setup, ui32 resolvePoolId) : StaticConfig(setup) + , ListNodesCache(MakeIntrusive()) , ResolvePoolId(resolvePoolId) { Y_ABORT_UNLESS(StaticConfig->IsEntriesUnique()); @@ -254,6 +273,8 @@ class TDynamicNameserver : public TActorBootstrapped { TIntrusivePtr StaticConfig; std::array DynamicConfigs; TVector ListNodesQueue; + TIntrusivePtr ListNodesCache; + std::array NodeBrokerPipes; // When ListNodes requests are sent to NodeBroker tablets this // bitmap indicates domains which didn't answer yet. diff --git a/ydb/core/tablet/node_tablet_monitor.cpp b/ydb/core/tablet/node_tablet_monitor.cpp index 831aa9dfdfdb..82fb30a6869e 100644 --- a/ydb/core/tablet/node_tablet_monitor.cpp +++ b/ydb/core/tablet/node_tablet_monitor.cpp @@ -44,8 +44,8 @@ class TNodeList : public TActorBootstrapped { } void Handle(TEvInterconnect::TEvNodesInfo::TPtr &ev, const TActorContext &ctx) { - NodesInfo = ev->Release(); - if (!NodesInfo->Nodes.empty()) { + Nodes = ev->Get()->Nodes; + if (!Nodes.empty()) { RenderResponse(ctx); } else { NoData(ctx); @@ -53,7 +53,7 @@ class TNodeList : public TActorBootstrapped { } void RenderResponse(const TActorContext &ctx) { - Sort(NodesInfo->Nodes.begin(), NodesInfo->Nodes.end()); + Sort(Nodes.begin(), Nodes.end()); TStringStream str; HTML(str) { TAG(TH3) { @@ -69,7 +69,7 @@ class TNodeList : public TActorBootstrapped { } } TABLEBODY() { - for (const auto& ni : NodesInfo->Nodes) { + for (const auto& ni : Nodes) { const TEvInterconnect::TNodeInfo &nodeInfo = ni; TABLER() { TABLED() {str << "" @@ -104,7 +104,7 @@ class TNodeList : public TActorBootstrapped { protected: TActorId Sender; - TAutoPtr NodesInfo; + TVector Nodes; }; class TTabletList : public TActorBootstrapped { @@ -147,14 +147,14 @@ class TTabletList : public TActorBootstrapped { } void Handle(TEvInterconnect::TEvNodesInfo::TPtr &ev, const TActorContext &ctx) { - NodesInfo = ev->Release(); - if (!NodesInfo->Nodes.empty()) { + Nodes = ev->Get()->Nodes; + if (!Nodes.empty()) { if (FilterNodeId) { TActorId tabletStateActorId = NNodeWhiteboard::MakeNodeWhiteboardServiceId(FilterNodeId); ctx.Send(tabletStateActorId, new TEvWhiteboard::TEvTabletStateRequest(), IEventHandle::FlagTrackDelivery, FilterNodeId); ++NodesRequested; } else { - for (const auto& ni : NodesInfo->Nodes) { + for (const auto& ni : Nodes) { TActorId tabletStateActorId = NNodeWhiteboard::MakeNodeWhiteboardServiceId(ni.NodeId); ctx.Send(tabletStateActorId, new TEvWhiteboard::TEvTabletStateRequest(), IEventHandle::FlagTrackDelivery, ni.NodeId); ++NodesRequested; @@ -193,8 +193,8 @@ class TTabletList : public TActorBootstrapped { for (const auto& ni : PerNodeTabletInfo) { if (FilterNodeId != 0 && FilterNodeId != ni.first) continue; - auto eq_it = EqualRange(NodesInfo->Nodes.begin(), NodesInfo->Nodes.end(), ni.first); - if (eq_it.first != NodesInfo->Nodes.end() && ni.second) { + auto eq_it = EqualRange(Nodes.begin(), Nodes.end(), ni.first); + if (eq_it.first != Nodes.end() && ni.second) { for (const auto& ti : ni.second->Record.GetTabletStateInfo()) { if (filter(ti)) { tabletIdIndex.push_back(ti.GetTabletId()); @@ -208,8 +208,8 @@ class TTabletList : public TActorBootstrapped { for (const auto& ni : PerNodeTabletInfo) { if (FilterNodeId != 0 && FilterNodeId != ni.first) continue; - auto eq_it = EqualRange(NodesInfo->Nodes.begin(), NodesInfo->Nodes.end(), ni.first); - if (eq_it.first != NodesInfo->Nodes.end() && ni.second) { + auto eq_it = EqualRange(Nodes.begin(), Nodes.end(), ni.first); + if (eq_it.first != Nodes.end() && ni.second) { const TEvInterconnect::TNodeInfo& nodeInfo = *eq_it.first; for (const auto& ti : ni.second->Record.GetTabletStateInfo()) { if (filter(ti)) { @@ -222,11 +222,11 @@ class TTabletList : public TActorBootstrapped { } void RenderResponse(const TActorContext &ctx) { - Sort(NodesInfo->Nodes.begin(), NodesInfo->Nodes.end()); + Sort(Nodes.begin(), Nodes.end()); TString filterNodeHost; if (FilterNodeId != 0) { - auto eq_it = EqualRange(NodesInfo->Nodes.begin(), NodesInfo->Nodes.end(), FilterNodeId); - if (eq_it.first != NodesInfo->Nodes.end()) { + auto eq_it = EqualRange(Nodes.begin(), Nodes.end(), FilterNodeId); + if (eq_it.first != Nodes.end()) { filterNodeHost = eq_it.first->Host; } } @@ -262,7 +262,7 @@ class TTabletList : public TActorBootstrapped { protected: TActorId Sender; - TAutoPtr NodesInfo; + TVector Nodes; TMap> PerNodeTabletInfo; size_t NodesRequested; size_t NodesReceived; @@ -298,13 +298,13 @@ class TStateStorageTabletList : public TActorBootstrappedRelease(); + Nodes = ev->Get()->Nodes; const TActorId proxyActorID = MakeStateStorageProxyID(); ctx.Send(proxyActorID, new TEvStateStorage::TEvRequestReplicasDumps()); } void Handle(TEvStateStorage::TEvResponseReplicasDumps::TPtr &ev, const TActorContext &ctx) { - Sort(NodesInfo->Nodes.begin(), NodesInfo->Nodes.end()); + Sort(Nodes.begin(), Nodes.end()); TEvStateStorage::TEvResponseReplicasDumps &event = *ev->Get(); TMap>> indexByTabletId; for (const auto& rdi : event.ReplicasDumps) { @@ -342,16 +342,16 @@ class TStateStorageTabletList : public TActorBootstrappedNodes.begin(), NodesInfo->Nodes.end(), replicaNodeId); - if (eq_it.first != NodesInfo->Nodes.end() && eq_it.first->Host) str << " / " << eq_it.first->Host; + auto eq_it = EqualRange(Nodes.begin(), Nodes.end(), replicaNodeId); + if (eq_it.first != Nodes.end() && eq_it.first->Host) str << " / " << eq_it.first->Host; } TABLED() {str << ei.GetCurrentGeneration();} TABLED() {if (ei.HasLockedFor()) str << TDuration::MicroSeconds(ei.GetLockedFor()).Seconds();} TABLED() { ui32 nodeId = ActorIdFromProto(ei.GetCurrentLeader()).NodeId(); str << nodeId; - auto eq_it = EqualRange(NodesInfo->Nodes.begin(), NodesInfo->Nodes.end(), nodeId); - if (eq_it.first != NodesInfo->Nodes.end() && eq_it.first->Host) str << " / " << eq_it.first->Host; + auto eq_it = EqualRange(Nodes.begin(), Nodes.end(), nodeId); + if (eq_it.first != Nodes.end() && eq_it.first->Host) str << " / " << eq_it.first->Host; } TABLED() {if (ActorIdFromProto(ei.GetCurrentLeaderTablet())) str << "";} } @@ -381,7 +381,7 @@ class TStateStorageTabletList : public TActorBootstrapped NodesInfo; + TVector Nodes; }; class TNodeTabletMonitor : public TActorBootstrapped { diff --git a/ydb/core/tablet/tablet_counters_aggregator_ut.cpp b/ydb/core/tablet/tablet_counters_aggregator_ut.cpp index 844c91320208..c842f39c87be 100644 --- a/ydb/core/tablet/tablet_counters_aggregator_ut.cpp +++ b/ydb/core/tablet/tablet_counters_aggregator_ut.cpp @@ -47,10 +47,11 @@ void TestHeavy(const ui32 v, ui32 numWorkers) { options.FinalEvents.emplace_back(TEvents::TSystem::Bootstrap, numWorkers); runtime.DispatchEvents(options); for (const auto& a : cc) { - THolder nodesInfo = MakeHolder(); + auto nodes = MakeIntrusive>(); for (auto i = 1; i <= NODES; ++i) { - nodesInfo->Nodes.emplace_back(TEvInterconnect::TNodeInfo(i, "::", "localhost", "localhost", 1234, TNodeLocation())); + nodes->emplace_back(TEvInterconnect::TNodeInfo(i, "::", "localhost", "localhost", 1234, TNodeLocation())); } + THolder nodesInfo = MakeHolder(nodes); runtime.Send(new NActors::IEventHandle(a, edge, nodesInfo.Release()), 0, true); } @@ -727,10 +728,11 @@ Y_UNIT_TEST_SUITE(TTabletLabeledCountersAggregator) { options.FinalEvents.emplace_back(TEvents::TSystem::Bootstrap, 1); runtime.DispatchEvents(options); for (const auto& a : cc) { - THolder nodesInfo = MakeHolder(); - nodesInfo->Nodes.emplace_back(TEvInterconnect::TNodeInfo(1, "::", "localhost", "localhost", 1234, TNodeLocation())); - nodesInfo->Nodes.emplace_back(TEvInterconnect::TNodeInfo(2, "::", "localhost", "localhost", 1234, TNodeLocation())); - nodesInfo->Nodes.emplace_back(TEvInterconnect::TNodeInfo(3, "::", "localhost", "localhost", 1234, TNodeLocation())); + auto nodes = MakeIntrusive>(); + nodes->emplace_back(TEvInterconnect::TNodeInfo(1, "::", "localhost", "localhost", 1234, TNodeLocation())); + nodes->emplace_back(TEvInterconnect::TNodeInfo(2, "::", "localhost", "localhost", 1234, TNodeLocation())); + nodes->emplace_back(TEvInterconnect::TNodeInfo(3, "::", "localhost", "localhost", 1234, TNodeLocation())); + THolder nodesInfo = MakeHolder(nodes); runtime.Send(new NActors::IEventHandle(a, edge, nodesInfo.Release()), 0, true); } @@ -827,10 +829,11 @@ Y_UNIT_TEST_SUITE(TTabletLabeledCountersAggregator) { options.FinalEvents.emplace_back(TEvents::TSystem::Bootstrap, 1); runtime.DispatchEvents(options); for (const auto& a : cc) { - THolder nodesInfo = MakeHolder(); - nodesInfo->Nodes.emplace_back(TEvInterconnect::TNodeInfo(1, "::", "localhost", "localhost", 1234, TNodeLocation())); - nodesInfo->Nodes.emplace_back(TEvInterconnect::TNodeInfo(2, "::", "localhost", "localhost", 1234, TNodeLocation())); - nodesInfo->Nodes.emplace_back(TEvInterconnect::TNodeInfo(3, "::", "localhost", "localhost", 1234, TNodeLocation())); + auto nodes = MakeIntrusive>(); + nodes->emplace_back(TEvInterconnect::TNodeInfo(1, "::", "localhost", "localhost", 1234, TNodeLocation())); + nodes->emplace_back(TEvInterconnect::TNodeInfo(2, "::", "localhost", "localhost", 1234, TNodeLocation())); + nodes->emplace_back(TEvInterconnect::TNodeInfo(3, "::", "localhost", "localhost", 1234, TNodeLocation())); + THolder nodesInfo = MakeHolder(nodes); runtime.Send(new NActors::IEventHandle(a, edge, nodesInfo.Release()), 0, true); } @@ -903,10 +906,11 @@ Y_UNIT_TEST_SUITE(TTabletLabeledCountersAggregator) { options.FinalEvents.emplace_back(TEvents::TSystem::Bootstrap, 1); runtime.DispatchEvents(options); for (const auto& a : cc) { - THolder nodesInfo = MakeHolder(); - nodesInfo->Nodes.emplace_back(TEvInterconnect::TNodeInfo(1, "::", "localhost", "localhost", 1234, TNodeLocation())); - nodesInfo->Nodes.emplace_back(TEvInterconnect::TNodeInfo(2, "::", "localhost", "localhost", 1234, TNodeLocation())); - nodesInfo->Nodes.emplace_back(TEvInterconnect::TNodeInfo(3, "::", "localhost", "localhost", 1234, TNodeLocation())); + auto nodes = MakeIntrusive>(); + nodes->emplace_back(TEvInterconnect::TNodeInfo(1, "::", "localhost", "localhost", 1234, TNodeLocation())); + nodes->emplace_back(TEvInterconnect::TNodeInfo(2, "::", "localhost", "localhost", 1234, TNodeLocation())); + nodes->emplace_back(TEvInterconnect::TNodeInfo(3, "::", "localhost", "localhost", 1234, TNodeLocation())); + THolder nodesInfo = MakeHolder(nodes); runtime.Send(new NActors::IEventHandle(a, edge, nodesInfo.Release()), 0, true); } diff --git a/ydb/core/viewer/viewer_counters.h b/ydb/core/viewer/viewer_counters.h index 6bb10f92d5ce..adbb6233f0f6 100644 --- a/ydb/core/viewer/viewer_counters.h +++ b/ydb/core/viewer/viewer_counters.h @@ -18,7 +18,7 @@ class TJsonCounters : public TActorBootstrapped { NMon::TEvHttpInfo::TPtr Event; ui32 Requested; ui32 Received; - THolder NodesInfo; + TVector Nodes; TMap VDiskInfo; TMap PDiskInfo; TMap TabletInfo; @@ -44,10 +44,8 @@ class TJsonCounters : public TActorBootstrapped { } void Die(const TActorContext& ctx) override { - if (NodesInfo != nullptr) { - for (const auto& ni : NodesInfo->Nodes) { - ctx.Send(TActivationContext::InterconnectProxy(ni.NodeId), new TEvents::TEvUnsubscribe()); - } + for (const auto& ni : Nodes) { + ctx.Send(TActivationContext::InterconnectProxy(ni.NodeId), new TEvents::TEvUnsubscribe()); } TBase::Die(ctx); } @@ -65,8 +63,8 @@ class TJsonCounters : public TActorBootstrapped { } void HandleBrowse(TEvInterconnect::TEvNodesInfo::TPtr& ev, const TActorContext& ctx) { - NodesInfo = ev->Release(); - for (const auto& ni : NodesInfo->Nodes) { + Nodes = ev->Get()->Nodes; + for (const auto& ni : Nodes) { SendRequest(ni.NodeId, ctx); } if (Requested > 0) { @@ -238,7 +236,7 @@ class TJsonCounters : public TActorBootstrapped { json << '{'; json << "\"sensors\":["; - Sort(NodesInfo->Nodes, []( + Sort(Nodes, []( const TEvInterconnect::TNodeInfo& a, const TEvInterconnect::TNodeInfo& b) -> bool { return a.NodeId < b.NodeId; @@ -278,7 +276,7 @@ class TJsonCounters : public TActorBootstrapped { auto itPDiskInfo = PDiskInfo.begin(); auto itTabletInfo = TabletInfo.begin(); - for (const auto& nodeInfo : NodesInfo->Nodes) { + for (const auto& nodeInfo : Nodes) { while (itVDiskInfo != VDiskInfo.end() && itVDiskInfo->first < nodeInfo.NodeId) ++itVDiskInfo; if (itVDiskInfo != VDiskInfo.end() && itVDiskInfo->first == nodeInfo.NodeId) { diff --git a/ydb/core/viewer/viewer_ut.cpp b/ydb/core/viewer/viewer_ut.cpp index 20d7c731b81b..9c764cc5b77d 100644 --- a/ydb/core/viewer/viewer_ut.cpp +++ b/ydb/core/viewer/viewer_ut.cpp @@ -251,14 +251,19 @@ Y_UNIT_TEST_SUITE(Viewer) { }; void ChangeListNodes(TEvInterconnect::TEvNodesInfo::TPtr* ev, int nodesTotal) { - auto& nodes = (*ev)->Get()->Nodes; + auto nodes = MakeIntrusive>((*ev)->Get()->Nodes); - auto sample = nodes[0]; - nodes.clear(); + auto sample = *nodes->begin(); + nodes->clear(); for (int nodeId = 0; nodeId < nodesTotal; nodeId++) { - nodes.emplace_back(sample); + nodes->emplace_back(sample); } + + auto newEv = IEventHandle::Downcast( + new IEventHandle((*ev)->Recipient, (*ev)->Sender, new TEvInterconnect::TEvNodesInfo(nodes)) + ); + ev->Swap(newEv); } void ChangeTabletStateResponse(TEvWhiteboard::TEvTabletStateResponse::TPtr* ev, int tabletsTotal, int& tabletId, int& nodeId) { @@ -722,7 +727,7 @@ Y_UNIT_TEST_SUITE(Viewer) { } case TEvInterconnect::EvNodesInfo: { auto *x = reinterpret_cast(&ev); - TVector &nodes = (*x)->Get()->Nodes; + const TVector &nodes = (*x)->Get()->Nodes; UNIT_ASSERT_EQUAL(nodes.size(), 2); staticNodeId = nodes[0]; sharedDynNodeId = nodes[1]; @@ -801,7 +806,7 @@ Y_UNIT_TEST_SUITE(Viewer) { } case TEvInterconnect::EvNodesInfo: { auto *x = reinterpret_cast(&ev); - TVector &nodes = (*x)->Get()->Nodes; + const TVector &nodes = (*x)->Get()->Nodes; UNIT_ASSERT_EQUAL(nodes.size(), 3); staticNodeId = nodes[0]; sharedDynNodeId = nodes[1]; @@ -884,7 +889,7 @@ Y_UNIT_TEST_SUITE(Viewer) { } case TEvInterconnect::EvNodesInfo: { auto *x = reinterpret_cast(&ev); - TVector &nodes = (*x)->Get()->Nodes; + const TVector &nodes = (*x)->Get()->Nodes; UNIT_ASSERT_EQUAL(nodes.size(), 3); staticNodeId = nodes[0]; sharedDynNodeId = nodes[1]; @@ -970,7 +975,7 @@ Y_UNIT_TEST_SUITE(Viewer) { } case TEvInterconnect::EvNodesInfo: { auto *x = reinterpret_cast(&ev); - TVector &nodes = (*x)->Get()->Nodes; + const TVector &nodes = (*x)->Get()->Nodes; UNIT_ASSERT_EQUAL(nodes.size(), 4); staticNodeId = nodes[0]; sharedDynNodeId = nodes[1]; diff --git a/ydb/library/actors/core/interconnect.h b/ydb/library/actors/core/interconnect.h index 33fecca6ccd6..55c0fc7bcbf4 100644 --- a/ydb/library/actors/core/interconnect.h +++ b/ydb/library/actors/core/interconnect.h @@ -2,6 +2,7 @@ #include "events.h" #include "event_local.h" +#include #include #include #include @@ -224,7 +225,13 @@ namespace NActors { }; struct TEvNodesInfo: public TEventLocal { - TVector Nodes; + TIntrusiveVector::TConstPtr NodesPtr; + const TVector& Nodes; + + TEvNodesInfo(TIntrusiveVector::TConstPtr nodesPtr) + : NodesPtr(nodesPtr) + , Nodes(*nodesPtr) + {} const TNodeInfo* GetNodeInfo(ui32 nodeId) const { for (const auto& x : Nodes) { diff --git a/ydb/library/actors/interconnect/interconnect_nameserver_base.h b/ydb/library/actors/interconnect/interconnect_nameserver_base.h index 7bffbefc4c40..9c0ef46aedde 100644 --- a/ydb/library/actors/interconnect/interconnect_nameserver_base.h +++ b/ydb/library/actors/interconnect/interconnect_nameserver_base.h @@ -55,15 +55,14 @@ namespace NActors { void Handle(TEvInterconnect::TEvListNodes::TPtr& ev, const TActorContext& ctx) { - THolder - reply(new TEvInterconnect::TEvNodesInfo()); - reply->Nodes.reserve(NodeTable.size()); + auto nodes = MakeIntrusive>(); + nodes->reserve(NodeTable.size()); for (const auto& pr : NodeTable) { - reply->Nodes.emplace_back(pr.first, - pr.second.Address, pr.second.Host, pr.second.ResolveHost, - pr.second.Port, pr.second.Location); + nodes->emplace_back(pr.first, + pr.second.Address, pr.second.Host, pr.second.ResolveHost, + pr.second.Port, pr.second.Location); } - ctx.Send(ev->Sender, reply.Release()); + ctx.Send(ev->Sender, new TEvInterconnect::TEvNodesInfo(nodes)); } void Handle(TEvInterconnect::TEvGetNode::TPtr& ev, diff --git a/ydb/library/actors/util/intrusive_vector.h b/ydb/library/actors/util/intrusive_vector.h new file mode 100644 index 000000000000..9b6ef8ad1535 --- /dev/null +++ b/ydb/library/actors/util/intrusive_vector.h @@ -0,0 +1,16 @@ +#pragma once + +#include +#include + +template +class TIntrusiveVector : public TVector, public TThrRefBase { +public: + using TVector::TVector; + + using TPtr = TIntrusivePtr>; + using TConstPtr = TIntrusiveConstPtr>; + + TIntrusiveVector(const TVector& other) : TVector(other) {} + TIntrusiveVector(TVector&& other) : TVector(std::move(other)) {} +}; diff --git a/ydb/library/yql/providers/dq/actors/dynamic_nameserver.cpp b/ydb/library/yql/providers/dq/actors/dynamic_nameserver.cpp index f632ca89d79c..1c32ac1a5510 100644 --- a/ydb/library/yql/providers/dq/actors/dynamic_nameserver.cpp +++ b/ydb/library/yql/providers/dq/actors/dynamic_nameserver.cpp @@ -166,10 +166,8 @@ namespace NYql::NDqs { void ReplyListNodes(const NActors::TActorId sender, const TActorContext& ctx) { - THolder - reply(new TEvInterconnect::TEvNodesInfo()); - reply->Nodes = GetNodesInfo(); - ctx.Send(sender, reply.Release()); + auto nodes = MakeIntrusive>(GetNodesInfo()); + ctx.Send(sender, new TEvInterconnect::TEvNodesInfo(nodes)); } void Handle(TEvInterconnect::TEvListNodes::TPtr& ev, diff --git a/ydb/library/yql/providers/dq/global_worker_manager/service_node_pinger.cpp b/ydb/library/yql/providers/dq/global_worker_manager/service_node_pinger.cpp index c8ea724e4fee..1ebe3e1a3396 100644 --- a/ydb/library/yql/providers/dq/global_worker_manager/service_node_pinger.cpp +++ b/ydb/library/yql/providers/dq/global_worker_manager/service_node_pinger.cpp @@ -154,15 +154,14 @@ class TServiceNodePinger: public TActor { auto& resp = ev->Get()->Response; - THolder - reply(new TEvInterconnect::TEvNodesInfo()); - reply->Nodes.reserve(resp.GetNodes().size()); if (resp.GetEpoch()) { RuntimeData->Epoch = resp.GetEpoch(); } + auto nodes = MakeIntrusive>(); + nodes->reserve(resp.GetNodes().size()); for (auto& node : resp.GetNodes()) { - reply->Nodes.emplace_back( + nodes->emplace_back( node.GetNodeId(), node.GetAddress(), node.GetAddress(), @@ -170,6 +169,7 @@ class TServiceNodePinger: public TActor { node.GetPort(), NActors::TNodeLocation()); } + THolder reply(new TEvInterconnect::TEvNodesInfo(nodes)); TVector downloadList; for (auto& file : resp.GetDownloadList()) {