diff --git a/ydb/core/persqueue/events/global.h b/ydb/core/persqueue/events/global.h index 4db023fee74c..f37411a14a76 100644 --- a/ydb/core/persqueue/events/global.h +++ b/ydb/core/persqueue/events/global.h @@ -36,7 +36,7 @@ struct TEvPersQueue { EvDescribeResponse, EvGetReadSessionsInfo, EvReadSessionsInfoResponse, - EvWakeupClient, + EvWakeupClient, // deprecated EvUpdateACL, EvCheckACL, EvCheckACLResponse, @@ -198,16 +198,6 @@ struct TEvPersQueue { TEvPartitionClientInfoResponse() = default; }; - struct TEvWakeupClient : TEventLocal { - TEvWakeupClient(const TString& client, const ui32 group) - : Client(client) - , Group(group) - {} - - TString Client; - ui32 Group; - }; - struct TEvDescribe : public TEventPB { TEvDescribe() {} diff --git a/ydb/core/persqueue/events/internal.h b/ydb/core/persqueue/events/internal.h index 978cc12da7ed..1c82abaf764c 100644 --- a/ydb/core/persqueue/events/internal.h +++ b/ydb/core/persqueue/events/internal.h @@ -188,6 +188,7 @@ struct TEvPQ { EvWakeupReleasePartition, EvPartitionScaleStatusChanged, EvPartitionScaleRequestDone, + EvBalanceConsumer, EvEnd }; @@ -1123,6 +1124,14 @@ struct TEvPQ { Record.SetScaleStatus(scaleStatus); } }; + + struct TEvBalanceConsumer : TEventLocal { + TEvBalanceConsumer(const TString& consumerName) + : ConsumerName(consumerName) + {} + + TString ConsumerName; + }; }; } //NKikimr diff --git a/ydb/core/persqueue/read_balancer.cpp b/ydb/core/persqueue/read_balancer.cpp index 4c77f50d49b8..d4964a8865e7 100644 --- a/ydb/core/persqueue/read_balancer.cpp +++ b/ydb/core/persqueue/read_balancer.cpp @@ -1,4 +1,5 @@ #include "read_balancer.h" +#include "read_balancer__balancing.h" #include "read_balancer__txpreinit.h" #include "read_balancer__txwrite.h" @@ -13,6 +14,8 @@ namespace NKikimr { namespace NPQ { +using namespace NBalancing; + static constexpr TDuration ACL_SUCCESS_RETRY_TIMEOUT = TDuration::Seconds(30); static constexpr TDuration ACL_ERROR_RETRY_TIMEOUT = TDuration::Seconds(5); @@ -41,11 +44,11 @@ TPersQueueReadBalancer::TPersQueueReadBalancer(const TActorId &tablet, TTabletSt , NextPartitionIdForWrite(0) , StartPartitionIdForWrite(0) , TotalGroups(0) - , NoGroupsInBase(true) , ResourceMetrics(nullptr) , WaitingForACL(false) , StatsReportRound(0) { + Balancer = std::make_unique(*this); } struct TPersQueueReadBalancer::TTxWritePartitionStats : public ITransaction { @@ -130,11 +133,6 @@ void TPersQueueReadBalancer::InitDone(const TActorContext &ctx) { s << "(" << p.first << ", " << p.second.TabletId << ") "; } LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, s); - for (auto& [_, clientInfo] : ClientsInfo) { - for (auto& [_, groupInfo] : clientInfo.ClientGroupsInfo) { - groupInfo.Balance(ctx); - } - } for (auto &ev : UpdateEvents) { ctx.Send(ctx.SelfID, ev.Release()); @@ -187,13 +185,15 @@ bool TPersQueueReadBalancer::OnRenderAppHtmlPage(NMon::TEvRemoteHttpInfo::TPtr e TString TPersQueueReadBalancer::GenerateStat() { auto& metrics = AggregatedStats.Metrics; + auto balancerStatistcs = Balancer->GetStatistics(); + TStringStream str; HTML(str) { TAG(TH2) {str << "PersQueueReadBalancer Tablet";} TAG(TH3) {str << "Topic: " << Topic;} TAG(TH3) {str << "Generation: " << Generation;} TAG(TH3) {str << "Inited: " << Inited;} - TAG(TH3) {str << "ActivePipes: " << PipesInfo.size();} + TAG(TH3) {str << "ActivePipes: " << balancerStatistcs.Sessions.size();} if (Inited) { TAG(TH3) {str << "Active partitions: " << NumActiveParts;} TAG(TH3) {str << "[Total/Max/Avg]WriteSpeedSec: " << metrics.TotalAvgWriteSpeedPerSec << "/" << metrics.MaxAvgWriteSpeedPerSec << "/" << metrics.TotalAvgWriteSpeedPerSec / NumActiveParts;} @@ -209,9 +209,9 @@ TString TPersQueueReadBalancer::GenerateStat() { LI_CLASS("active") { str << "partitions"; } - for (auto& pp : ClientsInfo) { + for (auto& consumer : balancerStatistcs.Consumers) { LI() { - str << "" << NPersQueue::ConvertOldConsumerName(pp.first) << ""; + str << "" << NPersQueue::ConvertOldConsumerName(consumer.ConsumerName) << ""; } } } @@ -221,7 +221,6 @@ TString TPersQueueReadBalancer::GenerateStat() { TABLEHEAD() { TABLER() { TABLEH() {str << "partition";} - TABLEH() {str << "group";} TABLEH() {str << "tabletId";} } } @@ -229,37 +228,30 @@ TString TPersQueueReadBalancer::GenerateStat() { for (auto& p : PartitionsInfo) { TABLER() { TABLED() { str << p.first;} - TABLED() { str << p.second.GroupId;} TABLED() { str << p.second.TabletId;} } } } } } - for (auto& p : ClientsInfo) { - DIV_CLASS_ID("tab-pane fade", "client_" + Base64Encode(p.first)) { + for (auto& consumer : balancerStatistcs.Consumers) { + DIV_CLASS_ID("tab-pane fade", "client_" + Base64Encode(consumer.ConsumerName)) { TABLE_SORTABLE_CLASS("table") { TABLEHEAD() { TABLER() { TABLEH() {str << "partition";} - TABLEH() {str << "group";} TABLEH() {str << "tabletId";} TABLEH() {str << "state";} TABLEH() {str << "session";} } } TABLEBODY() { - for (auto& ci : p.second.ClientGroupsInfo) { - for (auto& pp : ci.second.PartitionsInfo) { - TABLER() { - TABLED() { str << pp.first;} - TABLED() { str << ci.second.Group;} - TABLED() { str << pp.second.TabletId;} - TABLED() { str << (ui32)pp.second.State;} - auto* session = ci.second.FindSession(pp.second.Session); - Y_ABORT_UNLESS((session == nullptr) == (pp.second.State == EPS_FREE)); - TABLED() { str << (pp.second.State != EPS_FREE ? session->Session : "");} - } + for (auto& partition : consumer.Partitions) { + TABLER() { + TABLED() { str << partition.PartitionId;} + TABLED() { str << partition.TabletId;} + TABLED() { str << partition.State;} + TABLED() { str << partition.Session;} } } } @@ -269,32 +261,30 @@ TString TPersQueueReadBalancer::GenerateStat() { TABLEHEAD() { TABLER() { TABLEH() {str << "session";} - TABLEH() {str << "group";} TABLEH() {str << "suspended partitions";} TABLEH() {str << "active partitions";} + TABLEH() {str << "inactive partitions";} TABLEH() {str << "total partitions";} } } TABLEBODY() { - for (auto& ci : p.second.ClientGroupsInfo) { - for (auto& pp : ci.second.SessionsInfo) { - TABLER() { - TABLED() { str << pp.second.Session;} - TABLED() { str << ci.second.Group;} - TABLED() { str << pp.second.NumSuspended;} - TABLED() { str << pp.second.NumActive - pp.second.NumSuspended;} - TABLED() { str << (pp.second.NumActive);} - } - } + for (auto& session : balancerStatistcs.Sessions) { TABLER() { - TABLED() { str << "FREE";} - TABLED() { str << ci.second.Group;} - TABLED() { str << 0;} - TABLED() { str << ci.second.FreePartitions.size();} - TABLED() { str << ci.second.FreePartitions.size();} + TABLED() { str << session.Session;} + TABLED() { str << session.SuspendedPartitionCount;} + TABLED() { str << session.ActivePartitionCount;} + TABLED() { str << session.InactivePartitionCount;} + TABLED() { str << session.TotalPartitionCount;} } } + + TABLER() { + TABLED() { str << "FREE";} + TABLED() { str << 0;} + TABLED() { str << balancerStatistcs.FreePartitions;} + TABLED() { str << balancerStatistcs.FreePartitions;} + } } } } @@ -393,20 +383,6 @@ void TPersQueueReadBalancer::CheckACL(const TEvPersQueue::TEvCheckACL::TPtr &req } } -void TPersQueueReadBalancer::Handle(TEvPersQueue::TEvWakeupClient::TPtr &ev, const TActorContext& ctx) { - auto jt = ClientsInfo.find(ev->Get()->Client); - if (jt == ClientsInfo.end()) - return; - - auto& clientInfo = jt->second; - auto it = clientInfo.ClientGroupsInfo.find(ev->Get()->Group); - if (it != clientInfo.ClientGroupsInfo.end()) { - auto& groupInfo = it->second; - groupInfo.WakeupScheduled = false; - groupInfo.Balance(ctx); - } -} - void TPersQueueReadBalancer::Handle(TEvPersQueue::TEvDescribe::TPtr &ev, const TActorContext& ctx) { if (ctx.Now() > LastACLUpdate + ACL_EXPIRATION_TIMEOUT || Topic.empty()) { //Topic.empty is only for tests WaitingDescribeRequests.push_back(ev); @@ -478,31 +454,26 @@ void TPersQueueReadBalancer::Handle(TEvPersQueue::TEvUpdateBalancerConfig::TPtr SchemeShardId = record.GetSchemeShardId(); TotalGroups = record.HasTotalGroupCount() ? record.GetTotalGroupCount() : 0; + ui32 prevNextPartitionId = NextPartitionId; NextPartitionId = record.HasNextPartitionId() ? record.GetNextPartitionId() : 0; - std::map partitionsInfo; + if (record.HasSubDomainPathId()) { SubDomainPathId.emplace(record.GetSchemeShardId(), record.GetSubDomainPathId()); } + PartitionGraph = MakePartitionGraph(record); + auto oldConsumers = std::move(Consumers); Consumers.clear(); for (auto& consumer : TabletConfig.GetConsumers()) { - auto scalingSupport = consumer.HasScalingSupport() ? consumer.GetScalingSupport() : DefaultScalingSupport(); - auto it = oldConsumers.find(consumer.GetName()); if (it != oldConsumers.end()) { - auto& c = Consumers[consumer.GetName()] = std::move(it->second); - c.ScalingSupport = scalingSupport; - } else { - Consumers[consumer.GetName()].ScalingSupport = scalingSupport; + Consumers[consumer.GetName()] = std::move(it->second); } } - PartitionGraph = MakePartitionGraph(record); - std::vector newPartitions; - std::vector deletedPartitions; std::vector> newTablets; std::vector> newGroups; std::vector> reallocatedTablets; @@ -531,43 +502,36 @@ void TPersQueueReadBalancer::Handle(TEvPersQueue::TEvUpdateBalancerConfig::TPtr } - ui32 prevGroups = GroupsInfo.size(); - + std::map partitionsInfo; + std::vector newPartitions; + std::vector newPartitionsIds; for (auto& p : record.GetPartitions()) { auto it = PartitionsInfo.find(p.GetPartition()); - ui32 group = p.HasGroup() ? p.GetGroup() : p.GetPartition() + 1; - Y_ABORT_UNLESS(group > 0); - - if (NoGroupsInBase) { - Y_ABORT_UNLESS(group <= TotalGroups || TotalGroups == 0); - newGroups.push_back(std::make_pair(group, p.GetPartition())); - } if (it == PartitionsInfo.end()) { - Y_ABORT_UNLESS(group <= TotalGroups && group > prevGroups || TotalGroups == 0); Y_ABORT_UNLESS(p.GetPartition() >= prevNextPartitionId && p.GetPartition() < NextPartitionId || NextPartitionId == 0); - partitionsInfo[p.GetPartition()] = {p.GetTabletId(), EPS_FREE, TActorId(), group, {}}; + + partitionsInfo[p.GetPartition()] = {p.GetTabletId(), {}}; if (SplitMergeEnabled(TabletConfig)) { partitionsInfo[p.GetPartition()].KeyRange.DeserializeFromProto(p.GetKeyRange()); } - newPartitions.push_back(TPartInfo{p.GetPartition(), p.GetTabletId(), group, partitionsInfo[p.GetPartition()].KeyRange}); + newPartitionsIds.push_back(p.GetPartition()); + newPartitions.push_back(TPartInfo{p.GetPartition(), p.GetTabletId(), 0, partitionsInfo[p.GetPartition()].KeyRange}); - if (!NoGroupsInBase) - newGroups.push_back(std::make_pair(group, p.GetPartition())); - GroupsInfo[group].push_back(p.GetPartition()); ++NumActiveParts; + + // for back compatibility. Remove it after 24-3 + newGroups.push_back({p.GetGroup(), p.GetPartition()}); } else { //group is already defined - Y_ABORT_UNLESS(it->second.GroupId == group); partitionsInfo[p.GetPartition()] = it->second; } } if (TotalGroups == 0) { - NextPartitionId = TotalGroups = GroupsInfo.size(); + NextPartitionId = TotalGroups = partitionsInfo.size(); // this will not work when we support the deletion of the partition } - Y_ABORT_UNLESS(GroupsInfo.size() == TotalGroups); - + std::vector deletedPartitions; for (auto& p : PartitionsInfo) { if (partitionsInfo.find(p.first) == partitionsInfo.end()) { Y_ABORT("deleting of partitions is not fully supported yet"); @@ -576,29 +540,13 @@ void TPersQueueReadBalancer::Handle(TEvPersQueue::TEvUpdateBalancerConfig::TPtr } PartitionsInfo = std::unordered_map(partitionsInfo.rbegin(), partitionsInfo.rend()); - for (auto& [_, clientInfo] : ClientsInfo) { - auto mainGroup = clientInfo.ClientGroupsInfo.find(TClientInfo::MAIN_GROUP); - for (auto& newPartition : newPartitions) { - ui32 groupId = newPartition.Group; - auto it = clientInfo.SessionsWithGroup ? clientInfo.ClientGroupsInfo.find(groupId) : mainGroup; - if (it == clientInfo.ClientGroupsInfo.end()) { - Y_ABORT_UNLESS(clientInfo.SessionsWithGroup); - clientInfo.AddGroup(groupId); - it = clientInfo.ClientGroupsInfo.find(groupId); - } - auto& group = it->second; - group.FreePartition(newPartition.PartitionId); - group.PartitionsInfo[newPartition.PartitionId] = {newPartition.TabletId, EPS_FREE, TActorId(), groupId, newPartition.KeyRange}; - group.ScheduleBalance(ctx); - } - } - RebuildStructs(); - Execute(new TTxWrite(this, std::move(deletedPartitions), std::move(newPartitions), std::move(newTablets), std::move(newGroups), std::move(reallocatedTablets)), ctx); if (SubDomainPathId && (!WatchingSubDomainPathId || *WatchingSubDomainPathId != *SubDomainPathId)) { StartWatchingSubDomainPathId(); } + + Balancer->UpdateConfig(newPartitionsIds, deletedPartitions, ctx); } @@ -606,33 +554,6 @@ TStringBuilder TPersQueueReadBalancer::GetPrefix() const { return TStringBuilder() << "tablet " << TabletID() << " topic " << Topic << " "; } -TStringBuilder TPersQueueReadBalancer::TClientGroupInfo::GetPrefix() const { - return TStringBuilder() << "tablet " << TabletId << " topic " << Topic << " "; -} - -TStringBuilder TPersQueueReadBalancer::TClientInfo::GetPrefix() const { - return TStringBuilder() << "tablet " << TabletId << " topic " << Topic << " "; -} - -void TPersQueueReadBalancer::Handle(TEvTabletPipe::TEvServerDisconnected::TPtr& ev, const TActorContext& ctx) -{ - auto it = PipesInfo.find(ev->Get()->ClientId); - - LOG_INFO_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, GetPrefix() << "pipe " << ev->Get()->ClientId << " disconnected; active server actors: " - << (it != PipesInfo.end() ? it->second.ServerActors : -1)); - - if (it != PipesInfo.end()) { - if (--(it->second.ServerActors) > 0) - return; - if (!it->second.Session.empty()) { - LOG_NOTICE_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, GetPrefix() << "pipe " << ev->Get()->ClientId << " client " << it->second.ClientId << " disconnected session " << it->second.Session); - UnregisterSession(it->first, ctx); - } else { - LOG_INFO_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, GetPrefix() << "pipe " << ev->Get()->ClientId << " disconnected no session"); - PipesInfo.erase(it); - } - } -} void TPersQueueReadBalancer::Handle(TEvTabletPipe::TEvClientDestroyed::TPtr& ev, const TActorContext& ctx) { @@ -742,31 +663,25 @@ void TPersQueueReadBalancer::Handle(TEvPersQueue::TEvStatusResponse::TPtr& ev, c for (const auto& partRes : record.GetPartResult()) { ui32 partitionId = partRes.GetPartition(); + if (!PartitionsInfo.contains(partitionId)) { + continue; + } + auto generation = partRes.GetGeneration(); auto cookie = partRes.GetCookie(); for (const auto& consumer : partRes.GetConsumerResult()) { if (consumer.GetReadingFinished()) { - auto it = ClientsInfo.find(consumer.GetConsumer()); - if (it != ClientsInfo.end()) { - auto& clientInfo = it->second; - if (clientInfo.IsReadeable(partitionId) && clientInfo.SetCommittedState(partitionId, generation, cookie)) { - clientInfo.ProccessReadingFinished(partRes.GetPartition(), ctx); - } - } + Balancer->SetCommittedState(consumer.GetConsumer(), partitionId, generation, cookie, ctx); } } - if (!PartitionsInfo.contains(partRes.GetPartition())) { - continue; - } if (SplitMergeEnabled(TabletConfig) && PartitionsScaleManager) { TPartitionScaleManager::TPartitionInfo scalePartitionInfo = { .Id = partitionId, - .KeyRange = PartitionsInfo[partRes.GetPartition()].KeyRange + .KeyRange = PartitionsInfo[partitionId].KeyRange }; PartitionsScaleManager->HandleScaleStatusChange(scalePartitionInfo, partRes.GetScaleStatus(), ctx); } - partRes.GetScaleStatus(); AggregatedStats.AggrStats(partitionId, partRes.GetPartitionSize(), partRes.GetUsedReserveSize()); AggregatedStats.AggrStats(partRes.GetAvgWriteSpeedPerSec(), partRes.GetAvgWriteSpeedPerMin(), @@ -1044,476 +959,6 @@ void TPersQueueReadBalancer::GetACL(const TActorContext& ctx) { } } -void TPersQueueReadBalancer::Handle(TEvTabletPipe::TEvServerConnected::TPtr& ev, const TActorContext& ctx) -{ - const TActorId& sender = ev->Get()->ClientId; - auto& pipe = PipesInfo[sender]; - ++pipe.ServerActors; - - LOG_INFO_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, - GetPrefix() << "pipe " << sender << " connected; active server actors: " << pipe.ServerActors); -} - -TPersQueueReadBalancer::TClientGroupInfo& TPersQueueReadBalancer::TClientInfo::AddGroup(const ui32 group) { - auto r = ClientGroupsInfo.insert({group, TClientGroupInfo{ *this }}); - - TClientGroupInfo& clientInfo = r.first->second; - clientInfo.Group = group; - clientInfo.ClientId = ClientId; - clientInfo.Topic = Topic; - clientInfo.TabletId = TabletId; - clientInfo.Path = Path; - clientInfo.Generation = Generation; - clientInfo.Step = &Step; - - clientInfo.SessionKeySalt = TAppData::RandomProvider->GenRand64(); - return clientInfo; -} - -void TPersQueueReadBalancer::TClientGroupInfo::ActivatePartition(ui32 partitionId) { - auto* session = FindSession(partitionId); - if (session) { - --session->NumInactive; - } -} - -void TPersQueueReadBalancer::TClientGroupInfo::InactivatePartition(ui32 partitionId) { - auto* session = FindSession(partitionId); - if (session) { - ++session->NumInactive; - } -} - -void TPersQueueReadBalancer::TClientGroupInfo::FreePartition(ui32 partitionId) { - if (Group != TClientInfo::MAIN_GROUP || ClientInfo.IsReadeable(partitionId)) { - FreePartitions.push_back(partitionId); - } -} - -void TPersQueueReadBalancer::TClientInfo::FillEmptyGroup(const ui32 group, const std::unordered_map& partitionsInfo) { - auto& groupInfo = AddGroup(group); - - for (auto& [partitionId, partitionInfo] : partitionsInfo) { - if (partitionInfo.GroupId == group || group == MAIN_GROUP) { //check group - groupInfo.PartitionsInfo.insert({partitionId, partitionInfo}); - groupInfo.FreePartition(partitionId); - } - } -} - -void TPersQueueReadBalancer::TClientInfo::AddSession(const ui32 groupId, const std::unordered_map& partitionsInfo, - const TActorId& sender, const NKikimrPQ::TRegisterReadSession& record) { - - TActorId pipe = ActorIdFromProto(record.GetPipeClient()); - - Y_ABORT_UNLESS(pipe); - - if (ClientGroupsInfo.find(groupId) == ClientGroupsInfo.end()) { - FillEmptyGroup(groupId, partitionsInfo); - } - - auto it = ClientGroupsInfo.find(groupId); - auto& group = it->second; - group.SessionsInfo.insert({ - group.SessionKey(pipe), - TSessionInfo( - record.GetSession(), - sender, - record.HasClientNode() ? record.GetClientNode() : "none", - sender.NodeId(), - TAppData::TimeProvider->Now() - ) - }); -} - -TPersQueueReadBalancer::TReadingPartitionStatus& TPersQueueReadBalancer::TClientInfo::GetPartitionReadingStatus(ui32 partitionId) { - return ReadingPartitionStatus[partitionId]; -} - -bool TPersQueueReadBalancer::TClientInfo::IsReadeable(ui32 partitionId) const { - if (!ScalingSupport()) { - return true; - } - - auto* node = Balancer.PartitionGraph.GetPartition(partitionId); - if (!node) { - return false; - } - - if (ReadingPartitionStatus.empty()) { - return node->Parents.empty(); - } - - for(auto* parent : node->HierarhicalParents) { - if (!IsFinished(parent->Id)) { - return false; - } - } - - return true; -} - -bool TPersQueueReadBalancer::TClientInfo::IsFinished(ui32 partitionId) const { - auto it = ReadingPartitionStatus.find(partitionId); - if (it == ReadingPartitionStatus.end()) { - return false; - } - return it->second.IsFinished(); -} - -bool TPersQueueReadBalancer::TClientInfo::SetCommittedState(ui32 partitionId, ui32 generation, ui64 cookie) { - return ReadingPartitionStatus[partitionId].SetCommittedState(generation, cookie); -} - -TPersQueueReadBalancer::TClientGroupInfo* TPersQueueReadBalancer::TClientInfo::FindGroup(ui32 partitionId) { - auto it = ClientGroupsInfo.find(partitionId + 1); - if (it != ClientGroupsInfo.end()) { - return &it->second; - } - - it = ClientGroupsInfo.find(MAIN_GROUP); - if (it == ClientGroupsInfo.end()) { - return nullptr; - } - - auto& group = it->second; - if (group.PartitionsInfo.contains(partitionId)) { - return &group; - } - - return nullptr; -} - -bool TPersQueueReadBalancer::TClientInfo::ProccessReadingFinished(ui32 partitionId, const TActorContext& ctx) { - if (!ScalingSupport()) { - return false; - } - - auto* groupInfo = FindGroup(partitionId); - if (!groupInfo) { - return false; // TODO is it correct? - } - groupInfo->InactivatePartition(partitionId); - - bool hasChanges = false; - - Balancer.PartitionGraph.Travers(partitionId, [&](ui32 id) { - if (IsReadeable(id)) { - auto* groupInfo = FindGroup(id); - if (!groupInfo) { - return false; // TODO is it correct? - } - auto it = groupInfo->PartitionsInfo.find(id); - if (it == groupInfo->PartitionsInfo.end()) { - return false; // TODO is it correct? - } - auto& partitionInfo = it->second; - - if (partitionInfo.State == EPS_FREE) { - groupInfo->FreePartitions.push_back(id); - groupInfo->ScheduleBalance(ctx); - hasChanges = true; - } - return true; - } - return false; - }); - - return hasChanges; -} - -void TPersQueueReadBalancer::HandleOnInit(TEvPersQueue::TEvRegisterReadSession::TPtr& ev, const TActorContext&) -{ - Y_ABORT(""); // TODO why? - RegisterEvents.push_back(ev->Release().Release()); -} - - -void TPersQueueReadBalancer::Handle(TEvPersQueue::TEvRegisterReadSession::TPtr& ev, const TActorContext& ctx) -{ - const auto& record = ev->Get()->Record; - - TActorId pipe = ActorIdFromProto(record.GetPipeClient()); - LOG_NOTICE_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, - "client " << record.GetClientId() << " register session for pipe " << pipe << " session " << record.GetSession()); - - Y_ABORT_UNLESS(!record.GetSession().empty()); - Y_ABORT_UNLESS(!record.GetClientId().empty()); - - Y_ABORT_UNLESS(pipe); - - //TODO: check here that pipe with clientPipe=sender is still connected - - auto jt = PipesInfo.find(pipe); - if (jt == PipesInfo.end()) { - LOG_CRIT_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, - GetPrefix() << "client " << record.GetClientId() << " pipe " << pipe - << " is not connected and got register session request for session " << record.GetSession()); - return; - } - - std::vector groups; - groups.reserve(record.GroupsSize()); - for (auto& group : record.GetGroups()) { - if (group == 0 || group > TotalGroups) { - THolder response(new TEvPersQueue::TEvError); - response->Record.SetCode(NPersQueue::NErrorCode::BAD_REQUEST); - response->Record.SetDescription(TStringBuilder() << "no group " << group << " in topic " << Topic); - ctx.Send(ev->Sender, response.Release()); - return; - } - groups.push_back(group); - } - - auto& pipeInfo = jt->second; - pipeInfo.Init(record.GetClientId(), record.GetSession(), ev->Sender, groups); - - auto cit = Consumers.find(record.GetClientId()); - NKikimrPQ::EConsumerScalingSupport scalingSupport = cit == Consumers.end() ? DefaultScalingSupport() : cit->second.ScalingSupport; - - auto it = ClientsInfo.find(record.GetClientId()); - if (it == ClientsInfo.end()) { - auto p = ClientsInfo.insert({record.GetClientId(), TClientInfo{ *this, scalingSupport }}); - Y_ABORT_UNLESS(p.second); - it = p.first; - it->second.ClientId = record.GetClientId(); - it->second.Topic = Topic; - it->second.TabletId = TabletID(); - it->second.Path = Path; - it->second.Generation = Generation; - it->second.Step = 0; - } - - auto& clientInfo = it->second; - if (!groups.empty()) { - ++clientInfo.SessionsWithGroup; - } - - if (clientInfo.SessionsWithGroup > 0 && groups.empty()) { - groups.reserve(TotalGroups); - for (ui32 i = 1; i <= TotalGroups; ++i) { - groups.push_back(i); - } - } - - if (!groups.empty()) { - auto jt = clientInfo.ClientGroupsInfo.find(0); - if (jt != clientInfo.ClientGroupsInfo.end()) { - clientInfo.KillSessionsWithoutGroup(ctx); - } - for (auto g : groups) { - clientInfo.AddSession(g, PartitionsInfo, ev->Sender, record); - } - for (ui32 group = 1; group <= TotalGroups; ++group) { - if (clientInfo.ClientGroupsInfo.find(group) == clientInfo.ClientGroupsInfo.end()) { - clientInfo.FillEmptyGroup(group, PartitionsInfo); - } - } - } else { - clientInfo.AddSession(0, PartitionsInfo, ev->Sender, record); - Y_ABORT_UNLESS(clientInfo.ClientGroupsInfo.size() == 1); - } - - RegisterSession(pipe, ctx); -} - - -void TPersQueueReadBalancer::Handle(TEvPersQueue::TEvGetReadSessionsInfo::TPtr& ev, const TActorContext& ctx) -{ - const auto& record = ev->Get()->Record; - auto it = ClientsInfo.find(record.GetClientId()); - THolder response(new TEvPersQueue::TEvReadSessionsInfoResponse()); - - std::unordered_set partitionsRequested; - for (auto p : record.GetPartitions()) { - partitionsRequested.insert(p); - } - response->Record.SetTabletId(TabletID()); - - if (it != ClientsInfo.end()) { - for (auto& c : it->second.ClientGroupsInfo) { - for (auto& p : c.second.PartitionsInfo) { - if (!partitionsRequested.empty() && !partitionsRequested.contains(p.first)) { - continue; - } - auto pi = response->Record.AddPartitionInfo(); - pi->SetPartition(p.first); - if (p.second.State == EPS_ACTIVE) { - auto* session = c.second.FindSession(p.second.Session); - Y_ABORT_UNLESS(session != nullptr); - pi->SetClientNode(session->ClientNode); - pi->SetProxyNodeId(session->ProxyNodeId); - pi->SetSession(session->Session); - pi->SetTimestamp(session->Timestamp.Seconds()); - pi->SetTimestampMs(session->Timestamp.MilliSeconds()); - } else { - pi->SetClientNode(""); - pi->SetProxyNodeId(0); - pi->SetSession(""); - pi->SetTimestamp(0); - pi->SetTimestampMs(0); - } - } - for (auto& s : c.second.SessionsInfo) { - auto si = response->Record.AddReadSessions(); - si->SetSession(s.second.Session); - - ActorIdToProto(s.second.Sender, si->MutableSessionActor()); - } - } - } - ctx.Send(ev->Sender, response.Release()); -} - - -bool TPersQueueReadBalancer::TClientInfo::ScalingSupport() const { - return NKikimrPQ::EConsumerScalingSupport::FULL_SUPPORT == ScalingSupport_; -} - -void TPersQueueReadBalancer::TClientInfo::KillSessionsWithoutGroup(const TActorContext& ctx) { - auto it = ClientGroupsInfo.find(MAIN_GROUP); - Y_ABORT_UNLESS(it != ClientGroupsInfo.end()); - for (auto& s : it->second.SessionsInfo) { - THolder response(new TEvPersQueue::TEvError); - response->Record.SetCode(NPersQueue::NErrorCode::ERROR); - response->Record.SetDescription(TStringBuilder() << "there are new sessions with group, old session without group will be killed - recreate it, please"); - ctx.Send(s.second.Sender, response.Release()); - LOG_NOTICE_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, GetPrefix() <<"client " << ClientId << " kill session pipe " << s.first.first << " session " << s.second.Session); - } - ClientGroupsInfo.erase(it); -} - -void TPersQueueReadBalancer::TClientInfo::MergeGroups(const TActorContext& ctx) { - Y_ABORT_UNLESS(ClientGroupsInfo.find(0) == ClientGroupsInfo.end()); - - LOG_INFO_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, GetPrefix() << "client " << ClientId << " merge groups"); - - auto& mainGroupInfo = AddGroup(MAIN_GROUP); - - ui32 numSessions = 0; - ui32 numGroups = 0; - - for (auto it = ClientGroupsInfo.begin(); it != ClientGroupsInfo.end();) { - auto jt = it++; - if (jt->first == MAIN_GROUP) { - continue; - } - ++numGroups; - - auto& groupInfo = jt->second; - for (auto& pi : groupInfo.PartitionsInfo) { - bool res = mainGroupInfo.PartitionsInfo.insert(pi).second; - Y_ABORT_UNLESS(res); - } - - for (auto& si : groupInfo.SessionsInfo) { - auto key = si.first; - key.second = mainGroupInfo.SessionKeySalt; - auto it = mainGroupInfo.SessionsInfo.find(key); - if (it == mainGroupInfo.SessionsInfo.end()) { - mainGroupInfo.SessionsInfo.insert(std::make_pair(key, si.second)); //there must be all sessions in all groups - } else { - auto& session = it->second; - session.NumActive += si.second.NumActive; - session.NumSuspended += si.second.NumSuspended; - session.NumInactive += si.second.NumInactive; - } - ++numSessions; - } - - for (auto& fp : groupInfo.FreePartitions) { - mainGroupInfo.FreePartition(fp); - } - - ClientGroupsInfo.erase(jt); - } - Y_ABORT_UNLESS(mainGroupInfo.SessionsInfo.size() * numGroups == numSessions); - Y_ABORT_UNLESS(ClientGroupsInfo.size() == 1); - mainGroupInfo.ScheduleBalance(ctx); - -} - -void TPersQueueReadBalancer::Handle(TEvPersQueue::TEvPartitionReleased::TPtr& ev, const TActorContext& ctx) -{ - const auto& record = ev->Get()->Record; - auto partitionId = record.GetPartition(); - TActorId sender = ActorIdFromProto(record.GetPipeClient()); - const TString& clientId = record.GetClientId(); - - auto pit = PartitionsInfo.find(partitionId); - if (pit == PartitionsInfo.end()) { - LOG_CRIT_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, GetPrefix() << "client " << record.GetClientId() << " pipe " << sender << " got deleted partition " << record); - return; - } - - ui32 group = pit->second.GroupId; - Y_ABORT_UNLESS(group > 0); - - LOG_INFO_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, GetPrefix() << "client " << record.GetClientId() << " released partition from pipe " << sender - << " session " << record.GetSession() << " partition " << partitionId << " group " << group); - - auto it = ClientsInfo.find(clientId); - if (it == ClientsInfo.end()) { - LOG_CRIT_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, GetPrefix() << "client " << record.GetClientId() << " pipe " << sender - << " is not connected adn got release partitions request for session " << record.GetSession()); - return; - } - - auto& clientInfo = it->second; - if (!clientInfo.SessionsWithGroup) { - group = TClientInfo::MAIN_GROUP; - } - auto cit = clientInfo.ClientGroupsInfo.find(group); - if (cit == clientInfo.ClientGroupsInfo.end()) { - LOG_CRIT_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, GetPrefix() << "client " << record.GetClientId() << " pipe " << sender - << " is not connected and got release partitions request for session " << record.GetSession()); - return; - } - - auto& clientGroupsInfo = cit->second; - auto jt = clientGroupsInfo.PartitionsInfo.find(partitionId); - - auto* session = clientGroupsInfo.FindSession(sender); - if (session == nullptr) { //already dead session - return; - } - Y_ABORT_UNLESS(jt != clientGroupsInfo.PartitionsInfo.end()); - auto& partitionInfo = jt->second; - partitionInfo.Unlock(); - - clientGroupsInfo.FreePartition(partitionId); - - session->Unlock(!clientInfo.IsReadeable(partitionId)); // TODO тут точно должно быть IsReadable без условия что прочитана? - clientInfo.UnlockPartition(partitionId, ctx); - - clientGroupsInfo.ScheduleBalance(ctx); -} - -void TPersQueueReadBalancer::TClientInfo::UnlockPartition(ui32 partitionId, const TActorContext& ctx) { - if (GetPartitionReadingStatus(partitionId).StopReading()) { - // Release all children partitions if required - - auto* n = Balancer.PartitionGraph.GetPartition(partitionId); - if (!n) { - return; - } - - std::deque queue; - queue.insert(queue.end(), n->Children.begin(), n->Children.end()); - - while (!queue.empty()) { - auto* node = queue.front(); - queue.pop_front(); - queue.insert(queue.end(), node->Children.begin(), node->Children.end()); - - auto* group = FindGroup(node->Id); - if (!group) { - continue; - } - group->ReleasePartition(node->Id, ctx); - } - } -} void TPersQueueReadBalancer::HandleOnInit(TEvPersQueue::TEvGetPartitionsLocation::TPtr& ev, const TActorContext& ctx) { auto* evResponse = new TEvPersQueue::TEvGetPartitionsLocationResponse(); @@ -1521,7 +966,6 @@ void TPersQueueReadBalancer::HandleOnInit(TEvPersQueue::TEvGetPartitionsLocation ctx.Send(ev->Sender, evResponse); } - void TPersQueueReadBalancer::Handle(TEvPersQueue::TEvGetPartitionsLocation::TPtr& ev, const TActorContext& ctx) { auto* evResponse = new TEvPersQueue::TEvGetPartitionsLocationResponse(); const auto& request = ev->Get()->Record; @@ -1570,353 +1014,11 @@ void TPersQueueReadBalancer::Handle(TEvPersQueue::TEvGetPartitionsLocation::TPtr } -void TPersQueueReadBalancer::RebuildStructs() { - //TODO : bug here in case of deleting number of partitions - //TODO : track session with smallest and biggest number of (active but not suspended partitions -} - -void TPersQueueReadBalancer::RegisterSession(const TActorId& pipe, const TActorContext& ctx) -{ - //TODO : change structs for only this session, not all client - auto it = PipesInfo.find(pipe); - Y_ABORT_UNLESS(it != PipesInfo.end()); - auto jt = ClientsInfo.find(it->second.ClientId); - Y_ABORT_UNLESS(jt != ClientsInfo.end()); - for (auto& c : jt->second.ClientGroupsInfo) { - c.second.ScheduleBalance(ctx); - } -} - -void TPersQueueReadBalancer::UnregisterSession(const TActorId& pipe, const TActorContext& ctx) -{ - //TODO : change structs for only this session - auto it = PipesInfo.find(pipe); - Y_ABORT_UNLESS(it != PipesInfo.end()); - auto& pipeInfo = it->second; - - auto jt = ClientsInfo.find(pipeInfo.ClientId); - Y_ABORT_UNLESS(jt != ClientsInfo.end()); - TClientInfo& clientInfo = jt->second; - - for (auto& [groupKey, groupInfo] : clientInfo.ClientGroupsInfo) { - for (auto& [partitionId, partitionInfo] : groupInfo.PartitionsInfo) { //TODO: reverse map - if (partitionInfo.Session == pipe) { - partitionInfo.Unlock(); - groupInfo.FreePartition(partitionId); - } - } - if (groupInfo.EraseSession(pipe)) { - groupInfo.ScheduleBalance(ctx); - } - } - if (pipeInfo.WithGroups() && --clientInfo.SessionsWithGroup == 0) { - clientInfo.MergeGroups(ctx); - } - - PipesInfo.erase(it); -} - - -std::pair TPersQueueReadBalancer::TClientGroupInfo::SessionKey(const TActorId pipe) const { - return std::make_pair(pipe, SessionKeySalt); -} - -bool TPersQueueReadBalancer::TClientGroupInfo::EraseSession(const TActorId pipe) { - return SessionsInfo.erase(SessionKey(pipe)); -} - -TPersQueueReadBalancer::TSessionInfo* TPersQueueReadBalancer::TClientGroupInfo::FindSession(const TActorId pipe) { - auto it = SessionsInfo.find(SessionKey(pipe)); - if (it == SessionsInfo.end()) { - return nullptr; - } - return &(it->second); -} - -TPersQueueReadBalancer::TSessionInfo* TPersQueueReadBalancer::TClientGroupInfo::FindSession(ui32 partitionId) { - auto partitionIt = PartitionsInfo.find(partitionId); - if (partitionIt != PartitionsInfo.end()) { - auto& partitionInfo = partitionIt->second; - if (partitionInfo.Session) { - return FindSession(partitionInfo.Session); - } - } - - return nullptr; -} - -void TPersQueueReadBalancer::TClientGroupInfo::ScheduleBalance(const TActorContext& ctx) { - if (WakeupScheduled) { - return; - } - WakeupScheduled = true; - ctx.Send(ctx.SelfID, new TEvPersQueue::TEvWakeupClient(ClientId, Group)); -} - -std::tuple TPersQueueReadBalancer::TClientGroupInfo::TotalPartitions() const { - ui32 totalActive = 0; - ui32 totalInactive = 0; - ui32 totalUnreadable = 0; - - if (ClientInfo.ReadingPartitionStatus.empty()) { - totalActive = FreePartitions.size(); - } else { - for (auto p : FreePartitions) { - if (ClientInfo.IsReadeable(p)) { - if (ClientInfo.IsFinished(p)) { - ++totalInactive; - } else { - ++totalActive; - } - } else { - ++totalUnreadable; - } - } - } - for(auto& [_, session] : SessionsInfo) { - totalActive += session.NumActive - session.NumInactive; - totalInactive += session.NumInactive; - } - - return {totalActive, totalInactive, totalUnreadable}; -} - -void TPersQueueReadBalancer::TClientGroupInfo::ReleaseExtraPartitions(ui32 desired, ui32 allowPlusOne, const TActorContext& ctx) { - // request partitions from sessions if needed - for (auto& [sessionKey, sessionInfo] : SessionsInfo) { - ui32 realDesired = (allowPlusOne > 0) ? desired + 1 : desired; - if (allowPlusOne > 0) { - --allowPlusOne; - } - - i64 canRequest = ((i64)sessionInfo.NumActive) - sessionInfo.NumInactive - sessionInfo.NumSuspended - realDesired; - if (canRequest > 0) { - ReleasePartition(sessionKey.first, sessionInfo, canRequest, ctx); - } - } -} - -void TPersQueueReadBalancer::TClientGroupInfo::LockMissingPartitions( - ui32 desired, - ui32 allowPlusOne, - const std::function partitionPredicate, - const std::function actualExtractor, - const TActorContext& ctx) { - - std::deque freePartitions = std::move(FreePartitions); - std::deque toOtherPartitions; - - for (auto& [sessionKey, sessionInfo] : SessionsInfo) { - auto& pipe = sessionKey.first; - - ui32 realDesired = (allowPlusOne > 0) ? desired + 1 : desired; - if (allowPlusOne > 0) { - --allowPlusOne; - } - - ssize_t actual = actualExtractor(sessionInfo); - if (actual >= realDesired) { - continue; - } - - i64 req = ((i64)realDesired) - actual; - while (req > 0 && !freePartitions.empty()) { - auto partitionId = freePartitions.front(); - if (partitionPredicate(partitionId)) { - auto& status = ClientInfo.GetPartitionReadingStatus(partitionId); - if (status.BalanceToOtherPipe() && status.LastPipe != pipe || SessionsInfo.size() == 1) { - --req; - LockPartition(pipe, sessionInfo, partitionId, ctx); - } else { - toOtherPartitions.push_back(partitionId); - } - } else { - FreePartitions.push_back(partitionId); - } - freePartitions.pop_front(); - } - - if (!freePartitions.empty()) { - Y_ABORT_UNLESS(actualExtractor(sessionInfo) >= desired && actualExtractor(sessionInfo) <= desired + 1); - } - } - - if (!toOtherPartitions.empty()) { - LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, - GetPrefix() << "client: "<< ClientId << " balance group " << Group << " partitions " << JoinRange(", ", toOtherPartitions.begin(), toOtherPartitions.end()) << " to other sessions"); - - for (auto& [sessionKey, sessionInfo] : SessionsInfo) { - auto& pipe = sessionKey.first; - ui32 realDesired = desired + 1; - - ssize_t actual = actualExtractor(sessionInfo); - if (actual >= realDesired) { - continue; - } - - ssize_t req = ((ssize_t)realDesired) - actual; - size_t possibleIterations = toOtherPartitions.size(); - while (req > 0 && !toOtherPartitions.empty() && possibleIterations) { - auto partitionId = toOtherPartitions.front(); - toOtherPartitions.pop_front(); - - auto& status = ClientInfo.GetPartitionReadingStatus(partitionId); - if (status.LastPipe != pipe) { - --req; - --possibleIterations; - LockPartition(pipe, sessionInfo, partitionId, ctx); - } else { - --possibleIterations; - toOtherPartitions.push_back(partitionId); - } - } - } - } - - FreePartitions.insert(FreePartitions.end(), freePartitions.begin(), freePartitions.end()); -} - -void TPersQueueReadBalancer::TClientGroupInfo::Balance(const TActorContext& ctx) { - ui32 sessionsCount = SessionsInfo.size(); - - if (!sessionsCount) { - return; - } - - auto [totalActive, totalInactive, totalUnreadable] = TotalPartitions(); - - LOG_INFO_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, GetPrefix() << "client " << ClientId << " balance group " << Group << ": " - << " TotalActive=" << totalActive << ", TotalInactive=" << totalInactive << ", TotalUnreadable=" << totalUnreadable); - - - //FreePartitions and PipeInfo[].NumActive are consistent - ui32 desiredActive = totalActive / sessionsCount; - ui32 allowPlusOne = totalActive % sessionsCount; - ui32 desiredInactive = totalInactive / sessionsCount + 1; - - ReleaseExtraPartitions(desiredActive, allowPlusOne, ctx); - - //give free partitions to starving sessions - if (FreePartitions.empty()) { - return; - } - - LockMissingPartitions(desiredActive, allowPlusOne, - [&](ui32 partitionId) { return !ClientInfo.IsFinished(partitionId) && ClientInfo.IsReadeable(partitionId); }, - [](const TSessionInfo& sessionInfo) {return ((ssize_t)sessionInfo.NumActive) - sessionInfo.NumInactive; }, - ctx); - - LockMissingPartitions(desiredInactive, 0, - [&](ui32 partitionId) { return ClientInfo.IsFinished(partitionId) && ClientInfo.IsReadeable(partitionId); }, - [](const TSessionInfo& sessionInfo) {return (ssize_t)sessionInfo.NumInactive; }, - ctx); - - Y_ABORT_UNLESS(FreePartitions.size() == totalUnreadable); - FreePartitions.clear(); -} - -void TPersQueueReadBalancer::TClientGroupInfo::LockPartition(const TActorId pipe, TSessionInfo& sessionInfo, ui32 partition, const TActorContext& ctx) { - auto it = PartitionsInfo.find(partition); - Y_ABORT_UNLESS(it != PartitionsInfo.end()); - - auto& partitionInfo = it->second; - partitionInfo.Lock(pipe); - - ++sessionInfo.NumActive; - if (ClientInfo.IsFinished(partition)) { - ++sessionInfo.NumInactive; - } - //TODO:rebuild structs - - THolder res{new TEvPersQueue::TEvLockPartition}; - res->Record.SetSession(sessionInfo.Session); - res->Record.SetPartition(partition); - res->Record.SetTopic(Topic); - res->Record.SetPath(Path); - res->Record.SetGeneration(Generation); - res->Record.SetStep(++(*Step)); - res->Record.SetClientId(ClientId); - ActorIdToProto(pipe, res->Record.MutablePipeClient()); - res->Record.SetTabletId(PartitionsInfo[partition].TabletId); - - LOG_INFO_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, GetPrefix() << "client " << ClientId << " lock partition for pipe " - << pipe << " session " << sessionInfo.Session << " partition " << partition << " generation " << Generation << " step " << *Step); - - ctx.Send(sessionInfo.Sender, res.Release()); -} - -THolder TPersQueueReadBalancer::TClientGroupInfo::MakeEvReleasePartition( - const TActorId pipe, - const TSessionInfo& sessionInfo, - const ui32 count, - const std::set& partitions) { - THolder res{new TEvPersQueue::TEvReleasePartition}; - auto& r = res->Record; - - r.SetSession(sessionInfo.Session); - r.SetTopic(Topic); - r.SetPath(Path); - r.SetGeneration(Generation); - if (count) { - r.SetCount(count); - } - for (auto& p : partitions) { - r.AddPartition(p); - } - r.SetClientId(ClientId); - r.SetGroup(Group); - ActorIdToProto(pipe, r.MutablePipeClient()); - - return res; -} - -void TPersQueueReadBalancer::TClientGroupInfo::ReleasePartition(const ui32 partitionId, const TActorContext& ctx) { - auto it = PartitionsInfo.find(partitionId); - if (it == PartitionsInfo.end()) { - // TODO inconsistent status? - return; - } - - auto& partitionInfo = it->second; - - if (partitionInfo.Session) { - auto* session = FindSession(partitionInfo.Session); - if (session) { - ReleasePartition(partitionInfo.Session, *session, std::set{partitionId}, ctx); - } - } -} - -void TPersQueueReadBalancer::TClientGroupInfo::ReleasePartition(const TActorId pipe, TSessionInfo& sessionInfo, const ui32 count, const TActorContext& ctx) { - sessionInfo.NumSuspended += count; - - LOG_INFO_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, GetPrefix() << "client " << ClientId << " release partition group " << Group - << " for pipe " << pipe << " session " << sessionInfo.Session << " count " << count); - - ctx.Send(sessionInfo.Sender, MakeEvReleasePartition(pipe, sessionInfo, count, {}).Release()); -} - -void TPersQueueReadBalancer::TClientGroupInfo::ReleasePartition(const TActorId pipe, TSessionInfo& sessionInfo, const std::set& partitions, const TActorContext& ctx) { - sessionInfo.NumSuspended += partitions.size(); - - LOG_INFO_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, GetPrefix() << "client " << ClientId << " release partition group " << Group - << " for pipe " << pipe << " session " << sessionInfo.Session); - - ctx.Send(sessionInfo.Sender, MakeEvReleasePartition(pipe, sessionInfo, 0, partitions).Release()); -} - - -static constexpr TDuration MaxFindSubDomainPathIdDelay = TDuration::Minutes(1); - - -void TPersQueueReadBalancer::StopFindSubDomainPathId() { - if (FindSubDomainPathIdActor) { - Send(FindSubDomainPathIdActor, new TEvents::TEvPoison); - FindSubDomainPathIdActor = { }; - } -} +// +// Watching PQConfig +// struct TTxWriteSubDomainPathId : public ITransaction { TPersQueueReadBalancer* const Self; @@ -1936,6 +1038,15 @@ struct TTxWriteSubDomainPathId : public ITransaction { } }; +static constexpr TDuration MaxFindSubDomainPathIdDelay = TDuration::Minutes(1); + +void TPersQueueReadBalancer::StopFindSubDomainPathId() { + if (FindSubDomainPathIdActor) { + Send(FindSubDomainPathIdActor, new TEvents::TEvPoison); + FindSubDomainPathIdActor = { }; + } +} + void TPersQueueReadBalancer::StartFindSubDomainPathId(bool delayFirstRequest) { if (!FindSubDomainPathIdActor && SchemeShardId != 0 && @@ -2021,137 +1132,64 @@ void TPersQueueReadBalancer::Handle(TEvTxProxySchemeCache::TEvWatchNotifyUpdated } } -void TPersQueueReadBalancer::Handle(TEvPQ::TEvReadingPartitionStatusRequest::TPtr& ev, const TActorContext& ctx) { - auto& r = ev->Get()->Record; - auto partitionId = r.GetPartitionId(); - - auto it = ClientsInfo.find(r.GetConsumer()); - if (it != ClientsInfo.end()) { - auto& clientInfo = it->second; - - if (!clientInfo.IsReadeable(partitionId)) { - LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, - "The offset of the partition " << partitionId << " was commited by " << r.GetConsumer() - << " but the partition isn't readable"); - return; - } - if (clientInfo.SetCommittedState(partitionId, r.GetGeneration(), r.GetCookie())) { - LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, - "The offset of the partition " << partitionId << " was commited by " << r.GetConsumer()); +// +// Balancing +// - clientInfo.ProccessReadingFinished(partitionId, ctx); - } - } +void TPersQueueReadBalancer::Handle(TEvPQ::TEvReadingPartitionStatusRequest::TPtr& ev, const TActorContext& ctx) { + Balancer->Handle(ev, ctx); } void TPersQueueReadBalancer::Handle(TEvPersQueue::TEvReadingPartitionStartedRequest::TPtr& ev, const TActorContext& ctx) { - auto& r = ev->Get()->Record; - auto partitionId = r.GetPartitionId(); - - auto it = ClientsInfo.find(r.GetConsumer()); - if (it == ClientsInfo.end()) { - LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, - "Received TEvReadingPartitionStartedRequest from unknown consumer " << r.GetConsumer()); - } - - auto& clientInfo = it->second; - auto& status = clientInfo.GetPartitionReadingStatus(partitionId); - - if (status.StartReading()) { - LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, - "Reading of the partition " << partitionId << " was started by " << r.GetConsumer() << ". We stop reading from child partitions."); - - auto* groupInfo = clientInfo.FindGroup(partitionId); - if (groupInfo) { - groupInfo->ActivatePartition(partitionId); - } - - // We releasing all children's partitions because we don't start reading the partition from EndOffset - PartitionGraph.Travers(partitionId, [&](ui32 partitionId) { - auto& status = clientInfo.GetPartitionReadingStatus(partitionId); - auto* group = clientInfo.FindGroup(partitionId); - - if (group) { - if (status.Reset()) { - group->ActivatePartition(partitionId); - } - group->ReleasePartition(partitionId, ctx); - } - - return true; - }); - } else { - LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, - "Reading of the partition " << partitionId << " was started by " << r.GetConsumer() << "."); - - } -} - -TString GetSdkDebugString(bool scaleAwareSDK) { - return scaleAwareSDK ? "ScaleAwareSDK" : "old SDK"; + Balancer->Handle(ev, ctx); } void TPersQueueReadBalancer::Handle(TEvPersQueue::TEvReadingPartitionFinishedRequest::TPtr& ev, const TActorContext& ctx) { - auto& r = ev->Get()->Record; - auto partitionId = r.GetPartitionId(); - - auto it = ClientsInfo.find(r.GetConsumer()); - if (it == ClientsInfo.end()) { - LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, - "Received TEvReadingPartitionFinishedRequest from unknown consumer " << r.GetConsumer()); - } + Balancer->Handle(ev, ctx); +} - auto& clientInfo = it->second; - auto& status = clientInfo.GetPartitionReadingStatus(partitionId); +void TPersQueueReadBalancer::Handle(TEvPersQueue::TEvPartitionReleased::TPtr& ev, const TActorContext& ctx) { + Balancer->Handle(ev, ctx); +} - if (!clientInfo.IsReadeable(partitionId)) { - LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, - "Reading of the partition " << partitionId << " was finished by " << r.GetConsumer() - << " but the partition isn't readable"); - return; - } +void TPersQueueReadBalancer::Handle(TEvPQ::TEvWakeupReleasePartition::TPtr &ev, const TActorContext& ctx) { + Balancer->Handle(ev, ctx); +} - if (status.SetFinishedState(r.GetScaleAwareSDK(), r.GetStartedReadingFromEndOffset())) { - LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, - "Reading of the partition " << partitionId << " was finished by " << r.GetConsumer() - << ", firstMessage=" << r.GetStartedReadingFromEndOffset() << ", " << GetSdkDebugString(r.GetScaleAwareSDK())); +void TPersQueueReadBalancer::Handle(TEvPQ::TEvBalanceConsumer::TPtr& ev, const TActorContext& ctx) { + Balancer->Handle(ev, ctx); +} - clientInfo.ProccessReadingFinished(partitionId, ctx); - } else if (!status.IsFinished()) { - auto delay = std::min(1ul << status.Iteration, TabletConfig.GetPartitionConfig().GetLifetimeSeconds()); // TODO Учесть время закрытия партиции на запись +void TPersQueueReadBalancer::Handle(TEvTabletPipe::TEvServerConnected::TPtr& ev, const TActorContext& ctx) +{ + Balancer->Handle(ev, ctx); +} - LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, - "Reading of the partition " << partitionId << " was finished by " << r.GetConsumer() - << ". Scheduled release of the partition for re-reading. Delay=" << delay << " seconds," - << " firstMessage=" << r.GetStartedReadingFromEndOffset() << ", " << GetSdkDebugString(r.GetScaleAwareSDK())); +void TPersQueueReadBalancer::Handle(TEvTabletPipe::TEvServerDisconnected::TPtr& ev, const TActorContext& ctx) +{ + Balancer->Handle(ev, ctx); +} - status.LastPipe = ev->Sender; - ctx.Schedule(TDuration::Seconds(delay), new TEvPQ::TEvWakeupReleasePartition(r.GetConsumer(), partitionId, status.Cookie)); - } +void TPersQueueReadBalancer::HandleOnInit(TEvPersQueue::TEvRegisterReadSession::TPtr& ev, const TActorContext&) +{ + RegisterEvents.push_back(ev->Release().Release()); } -void TPersQueueReadBalancer::Handle(TEvPQ::TEvWakeupReleasePartition::TPtr &ev, const TActorContext& ctx) { - auto* msg = ev->Get(); - auto it = ClientsInfo.find(msg->Consumer); - if (it == ClientsInfo.end()) { - return; - } +void TPersQueueReadBalancer::Handle(TEvPersQueue::TEvRegisterReadSession::TPtr& ev, const TActorContext& ctx) +{ + Balancer->Handle(ev, ctx); +} - auto& clientInfo = it->second; - auto& readingStatus = clientInfo.GetPartitionReadingStatus(msg->PartitionId); - if (readingStatus.Cookie != msg->Cookie) { - return; - } +void TPersQueueReadBalancer::Handle(TEvPersQueue::TEvGetReadSessionsInfo::TPtr& ev, const TActorContext& ctx) +{ + Balancer->Handle(ev, ctx); +} - auto* group = clientInfo.FindGroup(msg->PartitionId); - if (!group) { - // TODO inconsistent status? must be filtered by cookie? - return; - } - group->ReleasePartition(msg->PartitionId, ctx); -} +// +// Autoscaling +// void TPersQueueReadBalancer::Handle(TEvPQ::TEvPartitionScaleStatusChanged::TPtr& ev, const TActorContext& ctx) { if (!SplitMergeEnabled(TabletConfig)) { diff --git a/ydb/core/persqueue/read_balancer.h b/ydb/core/persqueue/read_balancer.h index 92fcf44e1b95..bc14629a63a8 100644 --- a/ydb/core/persqueue/read_balancer.h +++ b/ydb/core/persqueue/read_balancer.h @@ -26,6 +26,17 @@ namespace NPQ { using namespace NTabletFlatExecutor; +namespace NBalancing { +class TBalancer; +} + + +struct TPartitionInfo { + ui64 TabletId; + NSchemeShard::TTopicTabletInfo::TKeyRange KeyRange; +}; + + class TMetricsTimeKeeper { public: TMetricsTimeKeeper(NMetrics::TResourceMetrics* metrics, const TActorContext& ctx) @@ -67,35 +78,39 @@ class TPersQueueReadBalancer : public TActor, public TTa bool OnRenderAppHtmlPage(NMon::TEvRemoteHttpInfo::TPtr ev, const TActorContext& ctx) override; TString GenerateStat(); - void Handle(TEvPersQueue::TEvWakeupClient::TPtr &ev, const TActorContext& ctx); - void Handle(TEvPQ::TEvWakeupReleasePartition::TPtr &ev, const TActorContext& ctx); void Handle(TEvPersQueue::TEvDescribe::TPtr &ev, const TActorContext& ctx); void HandleOnInit(TEvPersQueue::TEvUpdateBalancerConfig::TPtr &ev, const TActorContext& ctx); void Handle(TEvPersQueue::TEvUpdateBalancerConfig::TPtr &ev, const TActorContext& ctx); - void HandleOnInit(TEvPersQueue::TEvRegisterReadSession::TPtr &ev, const TActorContext& ctx); - void Handle(TEvPersQueue::TEvRegisterReadSession::TPtr &ev, const TActorContext& ctx); - void HandleOnInit(TEvPersQueue::TEvGetPartitionsLocation::TPtr& ev, const TActorContext& ctx); void Handle(TEvPersQueue::TEvGetPartitionsLocation::TPtr& ev, const TActorContext& ctx); - void Handle(TEvPersQueue::TEvGetReadSessionsInfo::TPtr &ev, const TActorContext& ctx); void Handle(TEvPersQueue::TEvCheckACL::TPtr&, const TActorContext&); void Handle(TEvPersQueue::TEvGetPartitionIdForWrite::TPtr&, const TActorContext&); - void Handle(TEvTabletPipe::TEvServerConnected::TPtr& ev, const TActorContext&); - void Handle(TEvTabletPipe::TEvServerDisconnected::TPtr& ev, const TActorContext&); - void Handle(TEvTabletPipe::TEvClientConnected::TPtr& ev, const TActorContext&); void Handle(TEvTabletPipe::TEvClientDestroyed::TPtr& ev, const TActorContext&); void Handle(NSchemeShard::TEvSchemeShard::TEvSubDomainPathIdFound::TPtr& ev, const TActorContext& ctx); void Handle(TEvTxProxySchemeCache::TEvWatchNotifyUpdated::TPtr& ev, const TActorContext& ctx); + // Begin balancing + void Handle(TEvPQ::TEvWakeupReleasePartition::TPtr &ev, const TActorContext& ctx); // from self + void Handle(TEvPQ::TEvBalanceConsumer::TPtr& ev, const TActorContext& ctx); // from self + void Handle(TEvPQ::TEvReadingPartitionStatusRequest::TPtr& ev, const TActorContext& ctx); // from Partition/PQ void Handle(TEvPersQueue::TEvReadingPartitionStartedRequest::TPtr& ev, const TActorContext& ctx); // from ReadSession void Handle(TEvPersQueue::TEvReadingPartitionFinishedRequest::TPtr& ev, const TActorContext& ctx); // from ReadSession + void HandleOnInit(TEvPersQueue::TEvRegisterReadSession::TPtr &ev, const TActorContext& ctx); // from ReadSession + void Handle(TEvPersQueue::TEvRegisterReadSession::TPtr &ev, const TActorContext& ctx); // from ReadSession + void Handle(TEvPersQueue::TEvPartitionReleased::TPtr& ev, const TActorContext& ctx); // from ReadSession + + void Handle(TEvTabletPipe::TEvServerConnected::TPtr& ev, const TActorContext&); + void Handle(TEvTabletPipe::TEvServerDisconnected::TPtr& ev, const TActorContext&); + + void Handle(TEvPersQueue::TEvGetReadSessionsInfo::TPtr &ev, const TActorContext& ctx); + // End balancing TStringBuilder GetPrefix() const; @@ -116,8 +131,6 @@ class TPersQueueReadBalancer : public TActor, public TTa void GetACL(const TActorContext&); void AnswerWaitingRequests(const TActorContext& ctx); - void Handle(TEvPersQueue::TEvPartitionReleased::TPtr& ev, const TActorContext& ctx); - void Handle(TEvPersQueue::TEvStatusResponse::TPtr& ev, const TActorContext& ctx); void Handle(TEvPQ::TEvStatsWakeup::TPtr& ev, const TActorContext& ctx); void Handle(NSchemeShard::TEvSchemeShard::TEvDescribeSchemeResult::TPtr& ev, const TActorContext& ctx); @@ -126,9 +139,6 @@ class TPersQueueReadBalancer : public TActor, public TTa void Handle(TEvPQ::TEvPartitionScaleStatusChanged::TPtr& ev, const TActorContext& ctx); void Handle(TPartitionScaleRequest::TEvPartitionScaleRequestDone::TPtr& ev, const TActorContext& ctx); - void RegisterSession(const TActorId& pipe, const TActorContext& ctx); - void UnregisterSession(const TActorId& pipe, const TActorContext& ctx); - void RebuildStructs(); ui64 PartitionReserveSize() { return TopicPartitionReserveSize(TabletConfig); } @@ -154,8 +164,6 @@ class TPersQueueReadBalancer : public TActor, public TTa struct TConsumerInfo { - NKikimrPQ::EConsumerScalingSupport ScalingSupport; - std::vector<::NMonitoring::TDynamicCounters::TCounterPtr> AggregatedCounters; THolder Aggr; }; @@ -169,24 +177,7 @@ class TPersQueueReadBalancer : public TActor, public TTa std::vector WaitingACLRequests; std::vector WaitingDescribeRequests; - enum EPartitionState { - EPS_FREE = 0, - EPS_ACTIVE = 1 - }; - - struct TPartitionInfo { - ui64 TabletId; - EPartitionState State; - TActorId Session; - ui32 GroupId; - NSchemeShard::TTopicTabletInfo::TKeyRange KeyRange; - - void Unlock() { Session = TActorId(); State = EPS_FREE; }; - void Lock(const TActorId& session) { Session = session; State = EPS_ACTIVE; } - }; - std::unordered_map PartitionsInfo; - std::unordered_map> GroupsInfo; struct TTabletInfo { ui64 Owner; @@ -200,204 +191,15 @@ class TPersQueueReadBalancer : public TActor, public TTa ui32 NextPartitionIdForWrite; ui32 StartPartitionIdForWrite; ui32 TotalGroups; - bool NoGroupsInBase; private: - struct TClientInfo; - - struct TReadingPartitionStatus { - // Client had commited rad offset equals EndOffset of the partition - bool Commited = false; - // ReadSession reach EndOffset of the partition - bool ReadingFinished = false; - // ReadSession connected with new SDK with garantee of read order - bool ScaleAwareSDK = false; - // ReadSession reach EndOffset of the partition by first request - bool StartedReadingFromEndOffset = false; - - size_t Iteration = 0; - ui64 Cookie = 0; - - TActorId LastPipe; - - // Generation of PQ-tablet and cookie for synchronization of commit information. - ui32 PartitionGeneration; - ui64 PartitionCookie; - - // Return true if the reading of the partition has been finished and children's partitions are readable. - bool IsFinished() const; - // Return true if children's partitions can't be balance separately. - bool NeedReleaseChildren() const; - bool BalanceToOtherPipe() const; - - // Called when reading from a partition is started. - // Return true if the reading of the partition has been finished before. - bool StartReading(); - // Called when reading from a partition is stopped. - // Return true if children's partitions can't be balance separately. - bool StopReading(); - - // Called when the partition is inactive and commited offset is equal to EndOffset. - // Return true if the commited status changed. - bool SetCommittedState(ui32 generation, ui64 cookie); - // Called when the partition reading finished. - // Return true if the reading status changed. - bool SetFinishedState(bool scaleAwareSDK, bool startedReadingFromEndOffset); - // Called when the parent partition is reading. - bool Reset(); - }; - - struct TSessionInfo { - TSessionInfo(const TString& session, const TActorId sender, const TString& clientNode, ui32 proxyNodeId, TInstant ts) - : Session(session) - , Sender(sender) - , NumSuspended(0) - , NumActive(0) - , NumInactive(0) - , ClientNode(clientNode) - , ProxyNodeId(proxyNodeId) - , Timestamp(ts) - {} - - TString Session; - TActorId Sender; - ui32 NumSuspended; - ui32 NumActive; - ui32 NumInactive; - - TString ClientNode; - ui32 ProxyNodeId; - TInstant Timestamp; - - void Unlock(bool inactive); - }; - - struct TClientGroupInfo { - TClientGroupInfo(TClientInfo& clientInfo) - : ClientInfo(clientInfo) {} - - TClientInfo& ClientInfo; - - TString ClientId; - TString Topic; - ui64 TabletId; - TString Path; - ui32 Generation = 0; - ui64 SessionKeySalt = 0; - ui32* Step = nullptr; - ui32 Group = 0; - - std::unordered_map PartitionsInfo; // partitionId -> info - std::deque FreePartitions; - std::unordered_map, TSessionInfo> SessionsInfo; //map from ActorID and random value - need for reordering sessions in different topics (groups?) - - std::pair SessionKey(const TActorId pipe) const; - bool EraseSession(const TActorId pipe); - TSessionInfo* FindSession(const TActorId pipe); - TSessionInfo* FindSession(ui32 partitionId); - - void ScheduleBalance(const TActorContext& ctx); - void Balance(const TActorContext& ctx); - - void LockPartition(const TActorId pipe, TSessionInfo& sessionInfo, ui32 partition, const TActorContext& ctx); - void ReleasePartition(const ui32 partitionId, const TActorContext& ctx); - void ReleasePartition(const TActorId pipe, TSessionInfo& sessionInfo, const ui32 count, const TActorContext& ctx); - void ReleasePartition(const TActorId pipe, TSessionInfo& sessionInfo, const std::set& partitions, const TActorContext& ctx); - THolder MakeEvReleasePartition(const TActorId pipe, const TSessionInfo& sessionInfo, const ui32 count, const std::set& partitions); - - void FreePartition(ui32 partitionId); - void ActivatePartition(ui32 partitionId); - void InactivatePartition(ui32 partitionId); - - TStringBuilder GetPrefix() const; - - std::tuple TotalPartitions() const; - void ReleaseExtraPartitions(ui32 desired, ui32 allowPlusOne, const TActorContext& ctx); - void LockMissingPartitions(ui32 desired, - ui32 allowPlusOne, - const std::function partitionPredicate, - const std::function actualExtractor, - const TActorContext& ctx); - - bool WakeupScheduled = false; - }; + friend class NBalancing::TBalancer; + std::unique_ptr Balancer; std::unique_ptr PartitionsScaleManager; - struct TClientInfo { - constexpr static ui32 MAIN_GROUP = 0; - - TClientInfo(const TPersQueueReadBalancer& balancer, NKikimrPQ::EConsumerScalingSupport scalingSupport) - : Balancer(balancer) - , ScalingSupport_(scalingSupport) { - } - - const TPersQueueReadBalancer& Balancer; - const NKikimrPQ::EConsumerScalingSupport ScalingSupport_; - - std::unordered_map ClientGroupsInfo; //map from group to info - std::unordered_map ReadingPartitionStatus; // partitionId->status - - size_t SessionsWithGroup = 0; - - TString ClientId; - TString Topic; - ui64 TabletId; - TString Path; - ui32 Generation = 0; - ui32 Step = 0; - - bool ScalingSupport() const; - - void KillSessionsWithoutGroup(const TActorContext& ctx); - void MergeGroups(const TActorContext& ctx); - TClientGroupInfo& AddGroup(const ui32 group); - void FillEmptyGroup(const ui32 group, const std::unordered_map& partitionsInfo); - void AddSession(const ui32 group, const std::unordered_map& partitionsInfo, - const TActorId& sender, const NKikimrPQ::TRegisterReadSession& record); - - bool ProccessReadingFinished(ui32 partitionId, const TActorContext& ctx); - - TStringBuilder GetPrefix() const; - - void UnlockPartition(ui32 partitionId, const TActorContext& ctx); - - TReadingPartitionStatus& GetPartitionReadingStatus(ui32 partitionId); - - bool IsReadeable(ui32 partitionId) const; - bool IsFinished(ui32 partitionId) const; - bool SetCommittedState(ui32 partitionId, ui32 generation, ui64 cookie); - - TClientGroupInfo* FindGroup(ui32 partitionId); - }; - - std::unordered_map ClientsInfo; //map from userId -> to info - private: - struct TPipeInfo { - TPipeInfo() - : ServerActors(0) - {} - - TString ClientId; // The consumer name - TString Session; - TActorId Sender; - std::vector Groups; // groups which are reading - ui32 ServerActors; // the number of pipes connected from SessionActor to ReadBalancer - - // true if client connected to read from concret partitions - bool WithGroups() { return !Groups.empty(); } - - void Init(const TString& clientId, const TString& session, const TActorId& sender, const std::vector& groups) { - ClientId = clientId; - Session = session; - Sender = sender; - Groups = groups; - } - }; - - std::unordered_map PipesInfo; NMetrics::TResourceMetrics *ResourceMetrics; @@ -486,7 +288,6 @@ class TPersQueueReadBalancer : public TActor, public TTa switch (ev->GetTypeRewrite()) { HFunc(TEvPersQueue::TEvUpdateBalancerConfig, HandleOnInit); - HFunc(TEvPersQueue::TEvWakeupClient, Handle); HFunc(TEvPersQueue::TEvDescribe, Handle); HFunc(TEvPersQueue::TEvRegisterReadSession, HandleOnInit); HFunc(TEvPersQueue::TEvGetReadSessionsInfo, Handle); @@ -513,7 +314,6 @@ class TPersQueueReadBalancer : public TActor, public TTa HFunc(TEvPersQueue::TEvCheckACL, Handle); HFunc(TEvPersQueue::TEvGetPartitionIdForWrite, Handle); HFunc(TEvPersQueue::TEvUpdateBalancerConfig, Handle); - HFunc(TEvPersQueue::TEvWakeupClient, Handle); HFunc(TEvPersQueue::TEvDescribe, Handle); HFunc(TEvPersQueue::TEvRegisterReadSession, Handle); HFunc(TEvPersQueue::TEvGetReadSessionsInfo, Handle); @@ -533,6 +333,7 @@ class TPersQueueReadBalancer : public TActor, public TTa HFunc(TEvPersQueue::TEvReadingPartitionStartedRequest, Handle); HFunc(TEvPersQueue::TEvReadingPartitionFinishedRequest, Handle); HFunc(TEvPQ::TEvWakeupReleasePartition, Handle); + HFunc(TEvPQ::TEvBalanceConsumer, Handle); // from PQ HFunc(TEvPQ::TEvPartitionScaleStatusChanged, Handle); // from TPartitionScaleRequest diff --git a/ydb/core/persqueue/read_balancer__balancing.cpp b/ydb/core/persqueue/read_balancer__balancing.cpp new file mode 100644 index 000000000000..be9d14b2726d --- /dev/null +++ b/ydb/core/persqueue/read_balancer__balancing.cpp @@ -0,0 +1,1783 @@ +#include "read_balancer__balancing.h" + +#define DEBUG(message) + + +namespace NKikimr::NPQ::NBalancing { + + +// +// TPartition +// + +bool TPartition::IsInactive() const { + return Commited || (ReadingFinished && (StartedReadingFromEndOffset || ScaleAwareSDK)); +} + +bool TPartition::NeedReleaseChildren() const { + return !(Commited || (ReadingFinished && !ScaleAwareSDK)); +} + +bool TPartition::BalanceToOtherPipe() const { + return !Commited && ReadingFinished && !ScaleAwareSDK; +} + +bool TPartition::StartReading() { + return std::exchange(ReadingFinished, false); +} + +bool TPartition::StopReading() { + ReadingFinished = false; + ++Cookie; + return NeedReleaseChildren(); +} + +bool TPartition::SetCommittedState(ui32 generation, ui64 cookie) { + if (PartitionGeneration < generation || (PartitionGeneration == generation && PartitionCookie < cookie)) { + Iteration = 0; + PartitionGeneration = generation; + PartitionCookie = cookie; + + return !std::exchange(Commited, true); + } + + return false; +} + +bool TPartition::SetFinishedState(bool scaleAwareSDK, bool startedReadingFromEndOffset) { + bool previousStatus = IsInactive(); + + ScaleAwareSDK = scaleAwareSDK; + StartedReadingFromEndOffset = startedReadingFromEndOffset; + ReadingFinished = true; + ++Cookie; + + bool currentStatus = IsInactive(); + if (currentStatus) { + Iteration = 0; + } else { + ++Iteration; + } + return currentStatus && !previousStatus; +} + +bool TPartition::Reset() { + bool result = IsInactive(); + + ScaleAwareSDK = false; + ReadingFinished = false; + Commited = false; + ++Cookie; + + return result; +}; + + +// +// TPartitionFamily +// + +TPartitionFamily::TPartitionFamily(TConsumer& consumerInfo, size_t id, std::vector&& partitions) + : Consumer(consumerInfo) + , Id(id) + , Status(EStatus::Free) + , TargetStatus(ETargetStatus::Free) + , RootPartitions(partitions) + , Partitions(std::move(partitions)) + , Session(nullptr) + , MergeTo(0) +{ + ClassifyPartitions(); + UpdatePartitionMapping(Partitions); + UpdateSpecialSessions(); +} + +bool TPartitionFamily::IsActive() const { + return Status == EStatus::Active; +} + +bool TPartitionFamily::IsFree() const { + return Status == EStatus::Free; +} + +bool TPartitionFamily::IsRelesing() const { + return Status == EStatus::Releasing; +} + +bool TPartitionFamily::IsLonely() const { + return Partitions.size() == 1; +} + +bool TPartitionFamily::HasActivePartitions() const { + return ActivePartitionCount; +} + +const TString& TPartitionFamily::Topic() const { + return Consumer.Topic(); +} + +const TString& TPartitionFamily::TopicPath() const { + return Consumer.TopicPath(); +} + +ui32 TPartitionFamily::TabletGeneration() const { + return Consumer.TabletGeneration(); +} + +const TPartitionInfo* TPartitionFamily::GetPartitionInfo(ui32 partitionId) const { + return Consumer.GetPartitionInfo(partitionId); +} +bool TPartitionFamily::IsReadable(ui32 partitionId) const { + return Consumer.IsReadable(partitionId); +} + +ui32 TPartitionFamily::NextStep() { + return Consumer.NextStep(); +} + +TString TPartitionFamily::GetPrefix() const { + TStringBuilder sb; + sb << Consumer.GetPrefix() << "family " << Id << " status " << Status + << " partitions [" << JoinRange(", ", Partitions.begin(), Partitions.end()) << "] "; + if (Session) { + sb << "session \"" << Session->SessionName << "\" sender " << Session->Sender << " "; + } + return sb; +} + + +void TPartitionFamily::Release(const TActorContext& ctx, ETargetStatus targetStatus) { + if (Status != EStatus::Active) { + LOG_CRIT_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "releasing the family " << DebugStr() << " that isn't active"); + return; + } + + if (!Session) { + LOG_CRIT_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "releasing the family " << DebugStr() << " that does not have a session"); + return; + } + + LOG_INFO_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << " release partitions [" << JoinRange(", ", LockedPartitions.begin(), LockedPartitions.end()) + << "]. Target status " << targetStatus); + + Status = EStatus::Releasing; + TargetStatus = targetStatus; + + Session->ActivePartitionCount -= ActivePartitionCount; + Session->InactivePartitionCount -= InactivePartitionCount; + Session->ReleasingPartitionCount += LockedPartitions.size(); + + --Session->ActiveFamilyCount; + ++Session->ReleasingFamilyCount; + --Consumer.ActiveFamilyCount; + + for (auto partitionId : LockedPartitions) { + ctx.Send(Session->Sender, MakeEvReleasePartition(partitionId).release()); + } +} + +bool TPartitionFamily::Unlock(const TActorId& sender, ui32 partitionId, const TActorContext& ctx) { + if (!Session || Session->Pipe != sender) { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "try unlock the partition " << partitionId << " from other sender"); + return false; + } + + if (Status != EStatus::Releasing) { + LOG_CRIT_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "try unlock partition " << partitionId << " but family status is " << Status); + return false; + } + + if (!LockedPartitions.erase(partitionId)) { + LOG_CRIT_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "try unlock partition " << partitionId << " but partition isn't locked." + << " Locked partitions are [" << JoinRange(", ", LockedPartitions.begin(), LockedPartitions.end()) << "]"); + return false; + } + + --Session->ReleasingPartitionCount; + + if (!LockedPartitions.empty()) { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "partition " << partitionId << " was unlocked but wait else [" << JoinRange(", ", LockedPartitions.begin(), LockedPartitions.end()) << "]"); + return false; + } + + --Session->ReleasingFamilyCount; + + Reset(ctx); + + return true; +} + +bool TPartitionFamily::Reset(const TActorContext& ctx) { + return Reset(TargetStatus, ctx); +} + +bool TPartitionFamily::Reset(ETargetStatus targetStatus, const TActorContext& ctx) { + if (IsActive()) { + --Consumer.ActiveFamilyCount; + } + + Session->Families.erase(this->Id); + Session = nullptr; + + TargetStatus = ETargetStatus::Free; + + switch (targetStatus) { + case ETargetStatus::Destroy: + Destroy(ctx); + return false; + + case ETargetStatus::Free: + LOG_TRACE_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << " is free."); + + Status = EStatus::Free; + AfterRelease(); + + return true; + + case ETargetStatus::Merge: + Status = EStatus::Free; + AfterRelease(); + + auto it = Consumer.Families.find(MergeTo); + if (it == Consumer.Families.end()) { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << " has been released for merge but target family is not exists."); + return true; + } + Consumer.MergeFamilies(it->second.get(), this, ctx); + + return true; + } +} + +void TPartitionFamily::Destroy(const TActorContext& ctx) { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << " destroyed."); + + for (auto partitionId : Partitions) { + Consumer.PartitionMapping.erase(partitionId); + } + Consumer.UnreadableFamilies.erase(Id); + Consumer.FamiliesRequireBalancing.erase(Id); + Consumer.Families.erase(Id); +} + +void TPartitionFamily::AfterRelease() { + Consumer.UnreadableFamilies[Id] = this; + Consumer.FamiliesRequireBalancing.erase(Id); + + for (auto partitionId : Partitions) { + Consumer.PartitionMapping.erase(partitionId); + } + + Partitions.clear(); + Partitions.insert(Partitions.end(), RootPartitions.begin(), RootPartitions.end()); + + ClassifyPartitions(); + UpdatePartitionMapping(Partitions); + // After reducing the number of partitions in the family, the list of reading sessions that can read this family may expand. + UpdateSpecialSessions(); +} + +void TPartitionFamily::StartReading(TSession& session, const TActorContext& ctx) { + if (Status != EStatus::Free) { + LOG_CRIT_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "try start reading but the family status is " << Status); + return; + } + + LOG_TRACE_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "start reading"); + + Status = EStatus::Active; + + Session = &session; + Session->Families.try_emplace(this->Id, this); + + Session->ActivePartitionCount += ActivePartitionCount; + Session->InactivePartitionCount += InactivePartitionCount; + + ++Session->ActiveFamilyCount; + ++Consumer.ActiveFamilyCount; + + LastPipe = Session->Pipe; + + for (auto partitionId : Partitions) { + LockPartition(partitionId, ctx); + } + + LockedPartitions.insert(Partitions.begin(), Partitions.end()); +} + +void TPartitionFamily::AttachePartitions(const std::vector& partitions, const TActorContext& ctx) { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "attaching partitions [" << JoinRange(", ", partitions.begin(), partitions.end()) << "]"); + + std::unordered_set existedPartitions; + existedPartitions.insert(Partitions.begin(), Partitions.end()); + + std::vector newPartitions; + newPartitions.reserve(partitions.size()); + for (auto partitionId : partitions) { + if (existedPartitions.contains(partitionId)) { + continue; + } + + newPartitions.push_back(partitionId); + existedPartitions.insert(partitionId); + } + + auto [activePartitionCount, inactivePartitionCount] = ClassifyPartitions(newPartitions); + ChangePartitionCounters(activePartitionCount, activePartitionCount); + + if (IsActive()) { + if (!Session->AllPartitionsReadable(newPartitions)) { + WantedPartitions.insert(newPartitions.begin(), newPartitions.end()); + UpdateSpecialSessions(); + Release(ctx); + return; + } + + for (auto partitionId : newPartitions) { + LockPartition(partitionId, ctx); + WantedPartitions.erase(partitionId); + } + + Partitions.insert(Partitions.end(), newPartitions.begin(), newPartitions.end()); + UpdatePartitionMapping(newPartitions); + + LockedPartitions.insert(newPartitions.begin(), newPartitions.end()); + } + + // Removing sessions wich can't read the family now + for (auto it = SpecialSessions.begin(); it != SpecialSessions.end();) { + auto& session = it->second; + if (session->AllPartitionsReadable(newPartitions)) { + ++it; + } else { + it = SpecialSessions.erase(it); + } + } +} + +void TPartitionFamily::ActivatePartition(ui32 partitionId) { + ALOG_DEBUG(NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "activating partition " << partitionId); + + ChangePartitionCounters(1, -1); +} + +void TPartitionFamily::InactivatePartition(ui32 partitionId) { + ALOG_DEBUG(NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "inactivating partition " << partitionId); + + ChangePartitionCounters(-1, 1); +} + + void TPartitionFamily::ChangePartitionCounters(ssize_t active, ssize_t inactive) { + ActivePartitionCount += active; + InactivePartitionCount += inactive; + + if (IsActive()) { + Session->ActivePartitionCount += active; + Session->InactivePartitionCount += inactive; + } + } + +void TPartitionFamily::Merge(TPartitionFamily* other) { + Partitions.insert(Partitions.end(), other->Partitions.begin(), other->Partitions.end()); + UpdatePartitionMapping(other->Partitions); + other->Partitions.clear(); + + RootPartitions.insert(RootPartitions.end(), other->RootPartitions.begin(), other->RootPartitions.end()); + other->RootPartitions.clear(); + + WantedPartitions.insert(other->WantedPartitions.begin(), other->WantedPartitions.end()); + WantedPartitions.clear(); + + ChangePartitionCounters(other->ActivePartitionCount, other->InactivePartitionCount); + other->ChangePartitionCounters(-other->ActivePartitionCount, -other->InactivePartitionCount); + + UpdateSpecialSessions(); +} + +TString TPartitionFamily::DebugStr() const { + return TStringBuilder() << "family=" << Id << " (Status=" << Status + << ", Partitions=[" << JoinRange(", ", Partitions.begin(), Partitions.end()) << "], SpecialSessions=" << SpecialSessions.size() << ")"; +} + +TPartition* TPartitionFamily::GetPartition(ui32 partitionId) { + return Consumer.GetPartition(partitionId); +} + +bool TPartitionFamily::PossibleForBalance(TSession* session) { + if (!IsLonely()) { + return true; + } + + auto partitionId = Partitions.front(); + auto* partition = GetPartition(partitionId); + if (!partition) { + return true; + } + + if (!partition->BalanceToOtherPipe()) { + return true; + } + + return session->Pipe != LastPipe; +} + + +void TPartitionFamily::ClassifyPartitions() { + auto [activePartitionCount, inactivePartitionCount] = ClassifyPartitions(Partitions); + ChangePartitionCounters(activePartitionCount, inactivePartitionCount); +} + +template +std::pair TPartitionFamily::ClassifyPartitions(const TPartitions& partitions) { + size_t activePartitionCount = 0; + size_t inactivePartitionCount = 0; + + for (auto partitionId : partitions) { + auto* partition = GetPartition(partitionId); + if (IsReadable(partitionId)) { + if (partition && partition->IsInactive()) { + ++inactivePartitionCount; + } else { + ++activePartitionCount; + } + } + } + + return {activePartitionCount, inactivePartitionCount}; +} + +template +std::pair TPartitionFamily::ClassifyPartitions(const std::set& partitions); + +template +std::pair TPartitionFamily::ClassifyPartitions(const std::vector& partitions); + +void TPartitionFamily::UpdatePartitionMapping(const std::vector& partitions) { + for (auto partitionId: partitions) { + Consumer.PartitionMapping[partitionId] = this; + } +} + +void TPartitionFamily::UpdateSpecialSessions() { + bool hasChanges = false; + + for (auto& [_, session] : Consumer.Sessions) { + if (session->WithGroups() && session->AllPartitionsReadable(Partitions) && session->AllPartitionsReadable(WantedPartitions)) { + auto [_, inserted] = SpecialSessions.try_emplace(session->Pipe, session); + if (inserted) { + hasChanges = true; + } + } + } + + if (hasChanges) { + Consumer.FamiliesRequireBalancing[Id] = this; + } +} + +void TPartitionFamily::LockPartition(ui32 partitionId, const TActorContext& ctx) { + auto step = NextStep(); + + LOG_INFO_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "lock partition " << partitionId << " for " << Session->DebugStr() + << " generation " << TabletGeneration() << " step " << step); + + ctx.Send(Session->Sender, MakeEvLockPartition(partitionId, step).release()); +} + +std::unique_ptr TPartitionFamily::MakeEvReleasePartition(ui32 partitionId) const { + auto res = std::make_unique(); + auto& r = res->Record; + + r.SetSession(Session->SessionName); + r.SetTopic(Topic()); + r.SetPath(TopicPath()); + r.SetGeneration(TabletGeneration()); + r.SetClientId(Session->ClientId); + r.SetGroup(partitionId + 1); + ActorIdToProto(Session->Pipe, r.MutablePipeClient()); + + return res; +} + +std::unique_ptr TPartitionFamily::MakeEvLockPartition(ui32 partitionId, ui32 step) const { + auto res = std::make_unique(); + auto& r = res->Record; + + r.SetSession(Session->SessionName); + r.SetPartition(partitionId); + r.SetTopic(Topic()); + r.SetPath(TopicPath()); + r.SetGeneration(TabletGeneration()); + r.SetStep(step); + r.SetClientId(Session->ClientId); + ActorIdToProto(Session->Pipe, res->Record.MutablePipeClient()); + + auto* partitionInfo = GetPartitionInfo(partitionId); + if (partitionInfo) { + r.SetTabletId(partitionInfo->TabletId); + } + + return res; +} + + +// +// TConsumer +// + +TConsumer::TConsumer(TBalancer& balancer, const TString& consumerName) + : Balancer(balancer) + , ConsumerName(consumerName) + , NextFamilyId(0) + , ActiveFamilyCount(0) + , BalanceScheduled(false) +{ +} + +const TString& TConsumer::Topic() const { + return Balancer.Topic(); +} + +const TString& TConsumer::TopicPath() const { + return Balancer.TopicPath(); +} + +ui32 TConsumer::TabletGeneration() const { + return Balancer.TabletGeneration(); +} + +const TPartitionInfo* TConsumer::GetPartitionInfo(ui32 partitionId) const { + return Balancer.GetPartitionInfo(partitionId); +} + +TPartition* TConsumer::GetPartition(ui32 partitionId) { + auto it = Partitions.find(partitionId); + if (it == Partitions.end()) { + return nullptr; + } + return &it->second; +} + +const TPartitionGraph& TConsumer::GetPartitionGraph() const { + return Balancer.GetPartitionGraph(); +} + +ui32 TConsumer::NextStep() { + return Balancer.NextStep(); +} + +void TConsumer::RegisterPartition(ui32 partitionId, const TActorContext& ctx) { + auto [_, inserted] = Partitions.try_emplace(partitionId, TPartition()); + if (inserted && IsReadable(partitionId)) { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "register readable partition " << partitionId); + + CreateFamily({partitionId}, ctx); + } +} + +void TConsumer::UnregisterPartition(ui32 partitionId, const TActorContext& ctx) { + BreakUpFamily(partitionId, true, ctx); +} + +void TConsumer::InitPartitions(const TActorContext& ctx) { + for (auto& [partitionId,_] : Balancer.GetPartitionsInfo()) { + RegisterPartition(partitionId, ctx); + } +} + +TPartitionFamily* TConsumer::CreateFamily(std::vector&& partitions, const TActorContext& ctx) { + return CreateFamily(std::move(partitions), TPartitionFamily::EStatus::Free, ctx); +} + +TPartitionFamily* TConsumer::CreateFamily(std::vector&& partitions, TPartitionFamily::EStatus status, const TActorContext& ctx) { + auto id = ++NextFamilyId; + auto [it, _] = Families.emplace(id, std::make_unique(*this, id, std::move(partitions))); + auto* family = it->second.get(); + + if (status == TPartitionFamily::EStatus::Free) { + UnreadableFamilies[id] = family; + } + + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "family created " << family->DebugStr()); + + return family; +} + +std::unordered_set Intercept(std::unordered_set values, std::vector members) { + std::unordered_set result; + for (auto m : members) { + if (values.contains(m)) { + result.insert(m); + } + } + return result; +} + +bool IsRoot(const TPartitionGraph::Node* node, const std::unordered_set& partitions) { + if (node->IsRoot()) { + return true; + } + for (auto* p : node->Parents) { + if (partitions.contains(p->Id)) { + return false; + } + } + return true; +} + +bool TConsumer::BreakUpFamily(ui32 partitionId, bool destroy, const TActorContext& ctx) { + auto* family = FindFamily(partitionId); + if (!family) { + return false; + } + + return BreakUpFamily(family, partitionId, destroy, ctx); +} + +bool TConsumer::BreakUpFamily(TPartitionFamily* family, ui32 partitionId, bool destroy, const TActorContext& ctx) { + std::vector newFamilies; + + if (!family->IsLonely()) { + std::unordered_set partitions; + partitions.insert(family->Partitions.begin(), family->Partitions.end()); + + if (IsRoot(GetPartitionGraph().GetPartition(partitionId), partitions)) { + partitions.erase(partitionId); + + std::unordered_set processedPartitions; + // There are partitions that are contained in two families at once + bool familiesIntersect = false; + + for (auto id : family->Partitions) { + if (id == partitionId) { + continue; + } + + if (!IsRoot(GetPartitionGraph().GetPartition(id), partitions)) { + continue; + } + + std::vector members; + + GetPartitionGraph().Travers(id, [&](auto childId) { + if (partitions.contains(childId)) { + members.push_back(childId); + auto [_, i] = processedPartitions.insert(childId); + if (!i) { + familiesIntersect = true; + } + + return true; + } + return false; + }); + + auto* f = CreateFamily({id}, family->Status, ctx); + f->Partitions.insert(f->Partitions.end(), members.begin(), members.end()); + f->TargetStatus = family->TargetStatus; + f->Session = family->Session; + f->LockedPartitions = Intercept(family->LockedPartitions, f->Partitions); + f->LastPipe = family->LastPipe; + if (f->Session) { + f->Session->Families.try_emplace(f->Id, f); + } + + newFamilies.push_back(f); + } + + family->Partitions.clear(); + family->Partitions.push_back(partitionId); + + auto locked = family->LockedPartitions.contains(partitionId); + family->LockedPartitions.clear(); + if (locked) { + family->LockedPartitions.insert(partitionId); + } + + family->ClassifyPartitions(); + + if (familiesIntersect) { + for (auto* f : newFamilies) { + if (f->IsActive()) { + f->Release(ctx); + } + } + } + } + } else { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "can't break up " << family->DebugStr() << " because partition is not root of family " << family->DebugStr()); + } + + family->WantedPartitions.clear(); + + if (destroy) { + DestroyFamily(family, ctx); + } else { + family->UpdateSpecialSessions(); + } + + return !newFamilies.empty(); +} + +bool TConsumer::MergeFamilies(TPartitionFamily* lhs, TPartitionFamily* rhs, const TActorContext& ctx) { + if (lhs->IsFree() && rhs->IsFree() || + lhs->IsActive() && rhs->IsActive() && lhs->Session == rhs->Session || + lhs->IsRelesing() && rhs->IsRelesing() && lhs->Session == rhs->Session && lhs->TargetStatus == rhs->TargetStatus) { + + lhs->Merge(rhs); + rhs->Destroy(ctx); + + return true; + } + + if (lhs->IsFree() && (rhs->IsActive() || rhs->IsRelesing())) { + std::swap(lhs, rhs); + } + if ((lhs->IsActive() || lhs->IsRelesing()) && rhs->IsFree()) { + lhs->AttachePartitions(rhs->Partitions, ctx); + lhs->RootPartitions.insert(lhs->RootPartitions.end(), rhs->Partitions.begin(), rhs->Partitions.end()); + + rhs->Partitions.clear(); + rhs->Destroy(ctx); + + return false; + } + + if (lhs->IsActive() && rhs->IsActive()) { // lhs->Session != rhs->Session + rhs->Release(ctx); + } + if (lhs->IsRelesing() && rhs->IsActive()) { + std::swap(rhs, lhs); + } + if (lhs->IsActive() && rhs->IsRelesing() && rhs->TargetStatus == TPartitionFamily::ETargetStatus::Free) { + lhs->TargetStatus = TPartitionFamily::ETargetStatus::Merge; + lhs->MergeTo = rhs->Id; + + return false; + } + + // In this case, one of the families is either already being merged or is being destroyed. In any case, they cannot be merged. + + return false; +} + +void TConsumer::DestroyFamily(TPartitionFamily* family, const TActorContext& ctx) { + switch(family->Status) { + case TPartitionFamily::EStatus::Active: + family->Release(ctx, TPartitionFamily::ETargetStatus::Destroy); + break; + case TPartitionFamily::EStatus::Releasing: + family->TargetStatus = TPartitionFamily::ETargetStatus::Destroy; + break; + case TPartitionFamily::EStatus::Free: + family->Reset(TPartitionFamily::ETargetStatus::Destroy, ctx); + break; + } +} + +TPartitionFamily* TConsumer::FindFamily(ui32 partitionId) { + auto it = PartitionMapping.find(partitionId); + if (it == PartitionMapping.end()) { + return nullptr; + } + return it->second; +} + +void TConsumer::RegisterReadingSession(TSession* session, const TActorContext& ctx) { + LOG_INFO_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "register reading session " << session->DebugStr()); + + Sessions[session->Pipe] = session; + + if (session->WithGroups()) { + for (auto& [_, family] : Families) { + if (session->AllPartitionsReadable(family->Partitions)) { + family->SpecialSessions[session->Pipe] = session; + FamiliesRequireBalancing[family->Id] = family.get(); + } + } + } +} + + +std::vector Snapshot(const std::unordered_map>& families) { + std::vector result; + result.reserve(families.size()); + + for (auto& [_, family] : families) { + result.push_back(family.get()); + } + + return result; +} + +void TConsumer::UnregisterReadingSession(TSession* session, const TActorContext& ctx) { + for (auto* family : Snapshot(Families)) { + if (session == family->Session) { + if (family->Reset(ctx)) { + UnreadableFamilies[family->Id] = family; + FamiliesRequireBalancing.erase(family->Id); + } + } + + family->SpecialSessions.erase(session->Pipe); + } + + Sessions.erase(session->Pipe); +} + +bool TConsumer::Unlock(const TActorId& sender, ui32 partitionId, const TActorContext& ctx) { + auto* family = FindFamily(partitionId); + if (!family) { + LOG_CRIT_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "unlocking the partition " << partitionId << " from unknown family."); + return false; + } + + return family->Unlock(sender, partitionId, ctx); +} + +bool TConsumer::IsReadable(ui32 partitionId) { + if (!ScalingSupport()) { + return true; + } + + auto* node = GetPartitionGraph().GetPartition(partitionId); + if (!node) { + return false; + } + + if (Partitions.empty()) { + return node->Parents.empty(); + } + + for(auto* parent : node->HierarhicalParents) { + if (!IsInactive(parent->Id)) { + return false; + } + } + + return true; +} + +bool TConsumer::IsInactive(ui32 partitionId) { + auto* partition = GetPartition(partitionId); + if (partition) { + return partition->IsInactive(); + } + return false; +} + +bool TConsumer::ScalingSupport() const { + return Balancer.ScalingSupport(); +} + +TString TConsumer::GetPrefix() const { + return TStringBuilder() << Balancer.GetPrefix() << "consumer " << ConsumerName << " "; +} + +bool TConsumer::SetCommittedState(ui32 partitionId, ui32 generation, ui64 cookie) { + return Partitions[partitionId].SetCommittedState(generation, cookie); +} + +bool TConsumer::ProccessReadingFinished(ui32 partitionId, const TActorContext& ctx) { + if (!ScalingSupport()) { + return false; + } + + auto& partition = Partitions[partitionId]; + + auto* family = FindFamily(partitionId); + if (!family) { + return false; + } + family->InactivatePartition(partitionId); + + if (!family->IsLonely() && partition.Commited) { + if (BreakUpFamily(family, partitionId, false, ctx)) { + return true; + } + } + + std::vector newPartitions; + GetPartitionGraph().Travers(partitionId, [&](ui32 id) { + if (!IsReadable(id)) { + return false; + } + + newPartitions.push_back(id); + return true; + }); + + if (partition.NeedReleaseChildren()) { + for (auto id : newPartitions) { + auto* node = GetPartitionGraph().GetPartition(id); + if (node->Children.size() > 1) { + // The partition was obtained as a result of the merge. + for (auto* c : node->Children) { + if (c->Id == family->Id) { + continue; + } + auto* other = FindFamily(c->Id); + if (other) { + MergeFamilies(family, other, ctx); + } + } + } else { + family->AttachePartitions(newPartitions, ctx); + } + } + } else { + for (auto p : newPartitions) { + auto* f = FindFamily(p); + if (!f) { + CreateFamily({p}, ctx); + } + } + } + + return !newPartitions.empty(); +} + +void TConsumer::StartReading(ui32 partitionId, const TActorContext& ctx) { + if (!GetPartitionInfo(partitionId)) { + LOG_CRIT_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "start reading for deleted partition " << partitionId); + return; + } + + auto* partition = GetPartition(partitionId); + + if (partition && partition->StartReading()) { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "Reading of the partition " << partitionId << " was started by " << ConsumerName << ". We stop reading from child partitions."); + + auto* family = FindFamily(partitionId); + if (!family) { + return; + } + + if (!family->IsLonely()) { + family->Release(ctx); + return; + } + + family->ActivatePartition(partitionId); + + // We releasing all children's partitions because we don't start reading the partition from EndOffset + GetPartitionGraph().Travers(partitionId, [&](ui32 partitionId) { + auto* partition = GetPartition(partitionId); + auto* f = FindFamily(partitionId); + + if (f) { + if (partition && partition->Reset()) { + f->ActivatePartition(partitionId); + } + DestroyFamily(f, ctx); + } + + return true; + }); + } else { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "Reading of the partition " << partitionId << " was started by " << ConsumerName << "."); + } +} + +TString GetSdkDebugString0(bool scaleAwareSDK) { + return scaleAwareSDK ? "ScaleAwareSDK" : "old SDK"; +} + +void TConsumer::FinishReading(TEvPersQueue::TEvReadingPartitionFinishedRequest::TPtr& ev, const TActorContext& ctx) { + auto& r = ev->Get()->Record; + auto partitionId = r.GetPartitionId(); + + if (!IsReadable(partitionId)) { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "Reading of the partition " << partitionId << " was finished by " << ConsumerName + << " but the partition isn't readable"); + return; + } + + auto* family = FindFamily(partitionId); + if (!family) { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "Reading of the partition " << partitionId << " was finished by " << ConsumerName + << " but the partition hasn't family"); + } + + if (!family->Session) { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "Reading of the partition " << partitionId << " was finished by " << ConsumerName + << " but the partition hasn't reading session"); + } + + auto& partition = Partitions[partitionId]; + + if (partition.SetFinishedState(r.GetScaleAwareSDK(), r.GetStartedReadingFromEndOffset())) { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "Reading of the partition " << partitionId << " was finished by " << r.GetConsumer() + << ", firstMessage=" << r.GetStartedReadingFromEndOffset() << ", " << GetSdkDebugString0(r.GetScaleAwareSDK())); + + if (ProccessReadingFinished(partitionId, ctx)) { + ScheduleBalance(ctx); + } + } else if (!partition.IsInactive()) { + auto delay = std::min(1ul << partition.Iteration, Balancer.GetLifetimeSeconds()); // TODO use split/merge time + + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "Reading of the partition " << partitionId << " was finished by " << r.GetConsumer() + << ". Scheduled release of the partition for re-reading. Delay=" << delay << " seconds," + << " firstMessage=" << r.GetStartedReadingFromEndOffset() << ", " << GetSdkDebugString0(r.GetScaleAwareSDK())); + + ctx.Schedule(TDuration::Seconds(delay), new TEvPQ::TEvWakeupReleasePartition(ConsumerName, partitionId, partition.Cookie)); + } +} + +void TConsumer::ScheduleBalance(const TActorContext& ctx) { + if (BalanceScheduled) { + LOG_TRACE_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "rebalancing already was scheduled"); + return; + } + + BalanceScheduled = true; + + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "rebalancing was scheduled"); + + ctx.Send(Balancer.TopicActor.SelfId(), new TEvPQ::TEvBalanceConsumer(ConsumerName)); +} + +TOrderedSessions OrderSessions( + const std::unordered_map& values, + std::function predicate = [](const TSession*) { return true; } +) { + TOrderedSessions result; + for (auto& [_, v] : values) { + if (predicate(v)) { + result.insert(v); + } + } + + return result; +} + +TString DebugStr(const std::unordered_map& values) { + TStringBuilder sb; + for (auto& [id, family] : values) { + sb << id << " (" << JoinRange(", ", family->Partitions.begin(), family->Partitions.end()) << "), "; + } + return sb; +} + +TString DebugStr(const TOrderedPartitionFamilies& values) { + TStringBuilder sb; + for (auto* family : values) { + sb << family->DebugStr() << ", "; + } + return sb; +} + +TOrderedPartitionFamilies OrderFamilies( + const std::unordered_map& values +) { + TOrderedPartitionFamilies result; + for (auto& [_, v] : values) { + result.insert(v); + } + + return result; +} + +size_t GetStatistics( + const std::unordered_map>& values, + std::function predicate = [](const TPartitionFamily*) { return true; } +) { + size_t count = 0; + + for (auto& [_, family] : values) { + if (predicate(family.get())) { + ++count; + } + } + + return count; +} + +size_t GetMaxFamilySize(const std::unordered_map>& values) { + size_t result = 1; + for (auto& [_, v] : values) { + result = std::max(result, v->ActivePartitionCount); + } + return result; +} + +void TConsumer::Balance(const TActorContext& ctx) { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "balancing. Sessions=" << Sessions.size() << ", Families=" << Families.size() + << ", UnradableFamilies=" << UnreadableFamilies.size() << " [" << DebugStr(UnreadableFamilies) + << "], RequireBalancing=" << FamiliesRequireBalancing.size() << " [" << DebugStr(FamiliesRequireBalancing) << "]"); + + if (Sessions.empty()) { + return; + } + + auto startTime = TInstant::Now(); + + // We try to balance the partitions by sessions that clearly want to read them, even if the distribution is not uniform. + for (auto& [_, family] : Families) { + if (family->Status != TPartitionFamily::EStatus::Active || family->SpecialSessions.empty()) { + continue; + } + if (!family->SpecialSessions.contains(family->Session->Pipe)) { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "rebalance " << family->DebugStr() << " because exists the special session for it"); + family->Release(ctx); + } + } + + TOrderedSessions commonSessions = OrderSessions(Sessions, [](auto* session) { + return !session->WithGroups(); + }); + + // Balance unredable families. + if (!UnreadableFamilies.empty()) { + auto families = OrderFamilies(UnreadableFamilies); + for (auto it = families.rbegin(); it != families.rend(); ++it) { + auto* family = *it; + TOrderedSessions specialSessions; + auto& sessions = (family->SpecialSessions.empty()) ? commonSessions : (specialSessions = OrderSessions(family->SpecialSessions)); + + auto sit = sessions.begin(); + for (;sit != sessions.end() && sessions.size() > 1 && !family->PossibleForBalance(*sit); ++sit) { + // Skip unpossible session. If there is only one session, then we always balance in it. + } + + if (sit == sessions.end()) { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "balancing of the " << family->DebugStr() << " failed because there are no suitable reading sessions."); + continue; + } + + auto* session = *sit; + + // Reorder sessions + sessions.erase(sit); + + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "balancing " << family->DebugStr() << " for " << session->DebugStr()); + family->StartReading(*session, ctx); + + // Reorder sessions + sessions.insert(session); + + UnreadableFamilies.erase(family->Id); + } + } + + // Rebalancing reading sessions with a large number of readable partitions. + if (!commonSessions.empty()) { + auto familyCount = GetStatistics(Families, [](auto* family) { + return family->SpecialSessions.empty(); + }); + + auto desiredFamilyCount = familyCount / commonSessions.size(); + auto allowPlusOne = familyCount % commonSessions.size(); + + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "start rebalancing. familyCount=" << familyCount << ", sessionCount=" << commonSessions.size() + << ", desiredFamilyCount=" << desiredFamilyCount << ", allowPlusOne=" << allowPlusOne); + + for (auto it = commonSessions.rbegin(); it != commonSessions.rend(); ++it) { + auto* session = *it; + auto targerFamilyCount = desiredFamilyCount + (allowPlusOne ? 1 : 0); + auto families = OrderFamilies(session->Families); + for (auto it = session->Families.begin(); it != session->Families.end() && session->ActiveFamilyCount > targerFamilyCount; ++it) { + auto* f = it->second; + if (f->IsActive()) { + f->Release(ctx); + } + } + + if (session->ActiveFamilyCount > desiredFamilyCount) { + --allowPlusOne; + } + } + } + + // Rebalancing special sessions + if (!FamiliesRequireBalancing.empty()) { + for (auto it = FamiliesRequireBalancing.begin(); it != FamiliesRequireBalancing.end();) { + auto* family = it->second; + + if (!family->IsActive()) { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "skip balancing " << family->DebugStr() << " because it is not active."); + + it = FamiliesRequireBalancing.erase(it); + continue; + } + + if (!family->SpecialSessions.contains(family->Session->Pipe)) { + family->Release(ctx); + continue; + } + + if (family->Session->ActiveFamilyCount == 1) { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "skip balancing " << family->DebugStr() << " because it is considered a session that does not read anything else."); + + it = FamiliesRequireBalancing.erase(it); + continue; + } + + if (family->SpecialSessions.size() <= 1) { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "skip balancing " << family->DebugStr() << " because there are no other suitable reading sessions."); + + it = FamiliesRequireBalancing.erase(it); + continue; + } + + bool hasGoodestSession = false; + size_t targetPartitionCount = family->Session->ActivePartitionCount - family->ActivePartitionCount; + for (auto [_, s] : family->SpecialSessions) { + if (s == family->Session) { + continue; + } + if (s->ActivePartitionCount < targetPartitionCount) { + hasGoodestSession = true; + break; + } + } + + if (hasGoodestSession) { + family->Release(ctx); + it = FamiliesRequireBalancing.erase(it); + } else { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "skip balancing " << family->DebugStr() << " because it is already being read by the best session."); + } + } + } + + auto duration = TInstant::Now() - startTime; + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "balancing duration: " << duration); +} + +void TConsumer::Release(ui32 partitionId, const TActorContext& ctx) { + auto* family = FindFamily(partitionId); + if (!family) { + return; + } + + family->Release(ctx); +} + + +// +// TSession +// + +TSession::TSession(const TActorId& pipe) + : Pipe(pipe) + , ServerActors(0) + , ActivePartitionCount(0) + , InactivePartitionCount(0) + , ReleasingPartitionCount(0) + , ActiveFamilyCount(0) + , ReleasingFamilyCount(0) { +} + +bool TSession::WithGroups() const { return !Partitions.empty(); } + +template +bool TSession::AllPartitionsReadable(const TCollection& partitions) const { + if (WithGroups()) { + for (auto p : partitions) { + if (!Partitions.contains(p)) { + return false; + } + } + } + + return true; +} + +template bool TSession::AllPartitionsReadable(const std::vector& partitions) const; +template bool TSession::AllPartitionsReadable(const std::unordered_set& partitions) const; + +TString TSession::DebugStr() const { + return TStringBuilder() << "ReadingSession \"" << SessionName << "\" (Sender=" << Sender << ", Pipe=" << Pipe + << ", Partitions=[" << JoinRange(", ", Partitions.begin(), Partitions.end()) + << "], ActiveFamilyCount=" << ActiveFamilyCount << ")"; +} + + +// +// TBalancer +// + +TBalancer::TBalancer(TPersQueueReadBalancer& topicActor) + : TopicActor(topicActor) + , Step(0) { +} + +const TString& TBalancer::Topic() const { + return TopicActor.Topic; +} + +const TString& TBalancer::TopicPath() const { + return TopicActor.Path; +} + +ui32 TBalancer::TabletGeneration() const { + return TopicActor.Generation; +} + +const TPartitionInfo* TBalancer::GetPartitionInfo(ui32 partitionId) const { + auto it = GetPartitionsInfo().find(partitionId); + if (it == GetPartitionsInfo().end()) { + return nullptr; + } + return &it->second; +} + +const std::unordered_map& TBalancer::GetPartitionsInfo() const { + return TopicActor.PartitionsInfo; +} + +const TPartitionGraph& TBalancer::GetPartitionGraph() const { + return TopicActor.PartitionGraph; +} + +bool TBalancer::ScalingSupport() const { + return SplitMergeEnabled(TopicActor.TabletConfig); +} + +i32 TBalancer::GetLifetimeSeconds() const { + return TopicActor.TabletConfig.GetPartitionConfig().GetLifetimeSeconds(); +} + +TConsumer* TBalancer::GetConsumer(const TString& consumerName) { + auto it = Consumers.find(consumerName); + if (it == Consumers.end()) { + return nullptr; + } + return it->second.get(); +} + +const TStatistics TBalancer::GetStatistics() const { + TStatistics result; + + result.Consumers.reserve(Consumers.size()); + for (auto& [_, consumer] : Consumers) { + result.Consumers.push_back(TStatistics::TConsumerStatistics()); + auto& c = result.Consumers.back(); + + c.ConsumerName = consumer->ConsumerName; + c.Partitions.reserve(GetPartitionsInfo().size()); + for (auto [partitionId, partitionInfo] : GetPartitionsInfo()) { + c.Partitions.push_back(TStatistics::TConsumerStatistics::TPartitionStatistics()); + auto& p = c.Partitions.back(); + p.PartitionId = partitionId; + p.TabletId = partitionInfo.TabletId; + + auto* family = consumer->FindFamily(partitionId); + if (family && family->Session && family->LockedPartitions.contains(partitionId)) { + p.Session = family->Session->SessionName; + p.State = 1; + } + } + } + + size_t readablePartitionCount = 0; + + result.Sessions.reserve(Sessions.size()); + for (auto& [_, session] : Sessions) { + result.Sessions.push_back(TStatistics::TSessionStatistics()); + auto& s = result.Sessions.back(); + s.Session = session->SessionName; + s.ActivePartitionCount = session->ActivePartitionCount; + s.InactivePartitionCount = session->InactivePartitionCount; + s.SuspendedPartitionCount = session->ReleasingPartitionCount; + s.TotalPartitionCount = s.ActivePartitionCount + s.InactivePartitionCount; + + readablePartitionCount += s.TotalPartitionCount; + } + + result.FreePartitions = GetPartitionsInfo().size() - readablePartitionCount; + + return result; +} + +void TBalancer::UpdateConfig(std::vector addedPartitions, std::vector deletedPartitions, const TActorContext& ctx) { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "updating configuration. Deleted partitions [" << JoinRange(", ", deletedPartitions.begin(), deletedPartitions.end()) + << "]. Added partitions [" << JoinRange(", ", addedPartitions.begin(), addedPartitions.end()) << "]"); + + for (auto partitionId : deletedPartitions) { + for (auto& [_, consumer] : Consumers) { + consumer->UnregisterPartition(partitionId, ctx); + } + } + + for (auto& partitionId : addedPartitions) { + for (auto& [_, balancingConsumer] : Consumers) { + balancingConsumer->RegisterPartition(partitionId, ctx); + } + } + + for (auto& [_, consumer] : Consumers) { + consumer->ScheduleBalance(ctx); + } +} + +bool TBalancer::SetCommittedState(const TString& consumerName, ui32 partitionId, ui32 generation, ui64 cookie, const TActorContext& ctx) { + auto* consumer = GetConsumer(consumerName); + if (!consumer) { + return false; + } + + if (!consumer->IsReadable(partitionId)) { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "The offset of the partition " << partitionId << " was commited by " << consumerName + << " but the partition isn't readable"); + return false; + } + + if (consumer->SetCommittedState(partitionId, generation, cookie)) { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "The offset of the partition " << partitionId << " was commited by " << consumerName); + + if (consumer->ProccessReadingFinished(partitionId, ctx)) { + consumer->ScheduleBalance(ctx); + } + + return true; + } + + return false; +} + +void TBalancer::Handle(TEvPQ::TEvReadingPartitionStatusRequest::TPtr& ev, const TActorContext& ctx) { + auto& r = ev->Get()->Record; + + SetCommittedState(r.GetConsumer(), r.GetPartitionId(), r.GetGeneration(), r.GetCookie(), ctx); +} + +void TBalancer::Handle(TEvPersQueue::TEvReadingPartitionStartedRequest::TPtr& ev, const TActorContext& ctx) { + auto& r = ev->Get()->Record; + auto partitionId = r.GetPartitionId(); + + auto consumer = GetConsumer(r.GetConsumer()); + if (!consumer) { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "Received TEvReadingPartitionStartedRequest from unknown consumer " << r.GetConsumer()); + return; + } + + consumer->StartReading(partitionId, ctx); +} + +void TBalancer::Handle(TEvPersQueue::TEvReadingPartitionFinishedRequest::TPtr& ev, const TActorContext& ctx) { + auto& r = ev->Get()->Record; + + auto consumer = GetConsumer(r.GetConsumer()); + if (!consumer) { + LOG_DEBUG_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "Received TEvReadingPartitionFinishedRequest from unknown consumer " << r.GetConsumer()); + return; + } + + consumer->FinishReading(ev, ctx); +} + +void TBalancer::Handle(TEvPersQueue::TEvPartitionReleased::TPtr& ev, const TActorContext& ctx) { + const auto& r = ev->Get()->Record; + const TString& consumerName = r.GetClientId(); + auto partitionId = r.GetPartition(); + TActorId sender = ActorIdFromProto(r.GetPipeClient()); + + auto* partitionInfo = GetPartitionInfo(partitionId); + if (!partitionInfo) { + LOG_CRIT_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "client " << r.GetClientId() << " pipe " << sender << " got deleted partition " << r); + return; + } + + LOG_INFO_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "client " << r.GetClientId() << " released partition from pipe " << sender + << " session " << r.GetSession() << " partition " << partitionId); + + auto* consumer = GetConsumer(consumerName); + if (!consumer) { + LOG_CRIT_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "client " << r.GetClientId() << " pipe " << sender + << " is not connected and got release partitions request for session " << r.GetSession()); + return; + } + + if (consumer->Unlock(sender, partitionId, ctx)) { + consumer->ScheduleBalance(ctx); + } +} + +void TBalancer::Handle(TEvPQ::TEvWakeupReleasePartition::TPtr &ev, const TActorContext& ctx) { + auto* msg = ev->Get(); + auto* consumer = GetConsumer(msg->Consumer); + if (!consumer) { + return; + } + + auto* partition = consumer->GetPartition(msg->PartitionId); + if (!partition || partition->Cookie != msg->Cookie) { + return; + } + + LOG_INFO_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "releasing partition " << msg->PartitionId << " of consumer \"" << msg->Consumer << "\" by reading finished timeout"); + + consumer->Release(msg->PartitionId, ctx); +} + +void TBalancer::Handle(TEvTabletPipe::TEvServerConnected::TPtr& ev, const TActorContext& ctx) { + const TActorId& sender = ev->Get()->ClientId; + + auto it = Sessions.find(sender); + if (it == Sessions.end()) { + auto [i, _] = Sessions.emplace(sender, std::make_unique(sender)); + it = i; + } + auto& session = it->second; + ++session->ServerActors; + + LOG_INFO_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "pipe " << sender << " connected; active server actors: " << session->ServerActors); +} + +void TBalancer::Handle(TEvTabletPipe::TEvServerDisconnected::TPtr& ev, const TActorContext& ctx) { + auto it = Sessions.find(ev->Get()->ClientId); + + if (it == Sessions.end()) { + LOG_ERROR_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "pipe " << ev->Get()->ClientId << " disconnected but there aren't sessions exists."); + return; + } + + LOG_INFO_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "pipe " << ev->Get()->ClientId << " disconnected; active server actors: " + << (it != Sessions.end() ? it->second->ServerActors : -1)); + + auto& session = it->second; + if (--(session->ServerActors) > 0) { + return; + } + + if (!session->SessionName.empty()) { + LOG_NOTICE_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "pipe " << ev->Get()->ClientId << " client " + << session->ClientId << " disconnected session " << session->SessionName); + + auto* consumer = GetConsumer(session->ClientId); + if (consumer) { + consumer->UnregisterReadingSession(session.get(), ctx); + + if (consumer->Sessions.empty()) { + Consumers.erase(consumer->ConsumerName); + } else { + consumer->ScheduleBalance(ctx); + } + } + + Sessions.erase(it); + } else { + LOG_INFO_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "pipe " << ev->Get()->ClientId << " disconnected no session"); + + Sessions.erase(it); + } +} + +void TBalancer::Handle(TEvPersQueue::TEvRegisterReadSession::TPtr& ev, const TActorContext& ctx) { + const auto& r = ev->Get()->Record; + auto& consumerName = r.GetClientId(); + + TActorId pipe = ActorIdFromProto(r.GetPipeClient()); + LOG_NOTICE_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "consumer \"" << consumerName << "\" register session for pipe " << pipe << " session " << r.GetSession()); + + if (consumerName.empty()) { + LOG_CRIT_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "ignored the session registration with empty consumer name."); + return; + } + + if (r.GetSession().empty()) { + LOG_CRIT_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "ignored the session registration with empty session name."); + return; + } + + if (!pipe) { + LOG_CRIT_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "ignored the session registration with empty Pipe."); + return; + } + + auto jt = Sessions.find(pipe); + if (jt == Sessions.end()) { + LOG_CRIT_S(ctx, NKikimrServices::PERSQUEUE_READ_BALANCER, + GetPrefix() << "client \"" << consumerName << "\" pipe " << pipe + << " is not connected and got register session request for session " << r.GetSession()); + return; + } + + std::vector partitions; + partitions.reserve(r.GroupsSize()); + for (auto& group : r.GetGroups()) { + auto partitionId = group - 1; + if (group == 0 || !GetPartitionInfo(partitionId)) { + THolder response(new TEvPersQueue::TEvError); + response->Record.SetCode(NPersQueue::NErrorCode::BAD_REQUEST); + response->Record.SetDescription(TStringBuilder() << "no group " << group << " in topic " << Topic()); + ctx.Send(ev->Sender, response.Release()); + return; + } + partitions.push_back(partitionId); + } + + auto* session = jt->second.get(); + session->ClientId = r.GetClientId(); + session->SessionName = r.GetSession(); + session->Sender = ev->Sender; + session->Partitions.insert(partitions.begin(), partitions.end()); + session->ClientNode = r.HasClientNode() ? r.GetClientNode() : "none"; + session->ProxyNodeId = ev->Sender.NodeId(); + session->CreateTimestamp = TAppData::TimeProvider->Now(); + + auto it = Consumers.find(consumerName); + if (it == Consumers.end()) { + auto [i, _] = Consumers.emplace(consumerName, std::make_unique(*this, consumerName)); + i->second->InitPartitions(ctx); + it = i; + } + + auto* consumer = it->second.get(); + consumer->RegisterReadingSession(session, ctx); + consumer->ScheduleBalance(ctx); +} + +void TBalancer::Handle(TEvPersQueue::TEvGetReadSessionsInfo::TPtr& ev, const TActorContext& ctx) { + const auto& r = ev->Get()->Record; + + std::unordered_set partitionsRequested; + partitionsRequested.insert(r.GetPartitions().begin(), r.GetPartitions().end()); + + auto response = std::make_unique(); + response->Record.SetTabletId(TopicActor.TabletID()); + + auto consumer = GetConsumer(r.GetClientId()); + if (consumer) { + for (auto& [partitionId, _] : GetPartitionsInfo()) { + if (!partitionsRequested.empty() && !partitionsRequested.contains(partitionId)) { + continue; + } + + auto pi = response->Record.AddPartitionInfo(); + pi->SetPartition(partitionId); + + auto* family = consumer->FindFamily(partitionId); + if (family && family->LockedPartitions.contains(partitionId)) { + auto* session = family->Session; + + Y_ABORT_UNLESS(session != nullptr); + pi->SetClientNode(session->ClientNode); + pi->SetProxyNodeId(session->ProxyNodeId); + pi->SetSession(session->SessionName); + pi->SetTimestamp(session->CreateTimestamp.Seconds()); + pi->SetTimestampMs(session->CreateTimestamp.MilliSeconds()); + } else { + pi->SetClientNode(""); + pi->SetProxyNodeId(0); + pi->SetSession(""); + pi->SetTimestamp(0); + pi->SetTimestampMs(0); + } + } + + for (auto& [_, session] : consumer->Sessions) { + auto si = response->Record.AddReadSessions(); + si->SetSession(session->SessionName); + + ActorIdToProto(session->Sender, si->MutableSessionActor()); + } + } + ctx.Send(ev->Sender, response.release()); +} + +void TBalancer::Handle(TEvPQ::TEvBalanceConsumer::TPtr& ev, const TActorContext& ctx) { + auto* consumer = GetConsumer(ev->Get()->ConsumerName); + if (consumer) { + consumer->BalanceScheduled = false; + consumer->Balance(ctx); + } +} + +TString TBalancer::GetPrefix() const { + return TStringBuilder() << "balancer: [" << TopicActor.TabletID() << "] topic " << Topic() << " "; +} + +ui32 TBalancer::NextStep() { + return ++Step; +} + + +bool TPartitionFamilyComparator::operator()(const TPartitionFamily* lhs, const TPartitionFamily* rhs) const { + if (lhs->ActivePartitionCount != rhs->ActivePartitionCount) { + return lhs->ActivePartitionCount < rhs->ActivePartitionCount; + } + if (lhs->InactivePartitionCount != rhs->InactivePartitionCount) { + return lhs->InactivePartitionCount < rhs->InactivePartitionCount; + } + return lhs->Id < rhs->Id; +} + +bool SessionComparator::operator()(const TSession* lhs, const TSession* rhs) const { + if (lhs->ActiveFamilyCount != rhs->ActiveFamilyCount) { + return lhs->ActiveFamilyCount < rhs->ActiveFamilyCount; + } + if (lhs->ActivePartitionCount != rhs->ActivePartitionCount) { + return lhs->ActivePartitionCount < rhs->ActivePartitionCount; + } + if (lhs->InactivePartitionCount != rhs->InactivePartitionCount) { + return lhs->InactivePartitionCount < rhs->InactivePartitionCount; + } + if (lhs->Partitions.size() != rhs->Partitions.size()) { + return lhs->Partitions.size() < rhs->Partitions.size(); + } + return lhs->SessionName < rhs->SessionName; +} + +} diff --git a/ydb/core/persqueue/read_balancer__balancing.h b/ydb/core/persqueue/read_balancer__balancing.h new file mode 100644 index 000000000000..781c73a2e19a --- /dev/null +++ b/ydb/core/persqueue/read_balancer__balancing.h @@ -0,0 +1,364 @@ +#pragma once + +#include "read_balancer.h" + +namespace NKikimr::NPQ::NBalancing { + +using namespace NTabletFlatExecutor; + +struct TSession; +struct TConsumer; +class TBalancer; + +struct TPartition { + // Client had commited rad offset equals EndOffset of the partition + bool Commited = false; + // ReadSession reach EndOffset of the partition + bool ReadingFinished = false; + // ReadSession connected with new SDK with garantee of read order + bool ScaleAwareSDK = false; + // ReadSession reach EndOffset of the partition by first request + bool StartedReadingFromEndOffset = false; + + size_t Iteration = 0; + ui64 Cookie = 0; + + // Generation of PQ-tablet and cookie for synchronization of commit information. + ui32 PartitionGeneration; + ui64 PartitionCookie; + + // Return true if the reading of the partition has been finished and children's partitions are readable. + bool IsInactive() const; + // Return true if children's partitions can't be balance separately. + bool NeedReleaseChildren() const; + bool BalanceToOtherPipe() const; + + // Called when reading from a partition is started. + // Return true if the reading of the partition has been finished before. + bool StartReading(); + // Called when reading from a partition is stopped. + // Return true if children's partitions can't be balance separately. + bool StopReading(); + + // Called when the partition is inactive and commited offset is equal to EndOffset. + // Return true if the commited status changed. + bool SetCommittedState(ui32 generation, ui64 cookie); + // Called when the partition reading finished. + // Return true if the reading status changed. + bool SetFinishedState(bool scaleAwareSDK, bool startedReadingFromEndOffset); + // Called when the parent partition is reading. + bool Reset(); +}; + +// Multiple partitions balancing together always in one reading session +struct TPartitionFamily { + friend struct TConsumer; + + enum class EStatus { + Active, // The family are reading + Releasing, // The family is waiting for partition to be released + Free + }; + + enum class ETargetStatus { + Free, // The family will be free for balancing. + Destroy, // The family will be destroyed after releasing. + Merge // The family will be merged with other family. + }; + + TConsumer& Consumer; + + size_t Id; + EStatus Status; + ETargetStatus TargetStatus; + + std::vector RootPartitions; + // Partitions that are in the family + std::vector Partitions; + + std::unordered_set WantedPartitions; + + // The reading session in which the family is currently being read. + TSession* Session; + // Partitions that are in the family + std::unordered_set LockedPartitions; + + // The number of active partitions in the family. + size_t ActivePartitionCount; + // The number of inactive partitions in the family. + size_t InactivePartitionCount; + + // Reading sessions that have a list of partitions to read and these sessions can read this family + std::unordered_map SpecialSessions; + + TActorId LastPipe; + size_t MergeTo; + + TPartitionFamily(TConsumer& consumerInfo, size_t id, std::vector&& partitions); + ~TPartitionFamily() = default; + + bool IsActive() const; + bool IsFree() const; + bool IsRelesing() const; + + bool IsLonely() const; + bool HasActivePartitions() const; + + // Releases all partitions of the family. + void Release(const TActorContext& ctx, ETargetStatus targetStatus = ETargetStatus::Free); + // Processes the signal from the reading session that the partition has been released. + // Return true if all partitions has been unlocked. + bool Unlock(const TActorId& sender, ui32 partitionId, const TActorContext& ctx); + // Processes the signal that the reading session has ended. + bool Reset(const TActorContext& ctx); + bool Reset(ETargetStatus targetStatus, const TActorContext& ctx); + // Starts reading the family in the specified reading session. + void StartReading(TSession& session, const TActorContext& ctx); + // Add partitions to the family. + void AttachePartitions(const std::vector& partitions, const TActorContext& ctx); + void Merge(TPartitionFamily* other); + + // The partition became active + void ActivatePartition(ui32 partitionId); + // The partition became inactive + void InactivatePartition(ui32 partitionId); + + bool PossibleForBalance(TSession* session); + + TString DebugStr() const; + +private: + void Destroy(const TActorContext& ctx); + void AfterRelease(); + +private: + const TString& Topic() const; + const TString& TopicPath() const; + ui32 TabletGeneration() const; + + const TPartitionInfo* GetPartitionInfo(ui32 partitionId) const; + TPartition* GetPartition(ui32 partitionId); + bool IsReadable(ui32 partitionId) const; + ui32 NextStep(); + +private: + void ClassifyPartitions(); + template + std::pair ClassifyPartitions(const TPartitions& partitions); + void UpdatePartitionMapping(const std::vector& partitions); + void UpdateSpecialSessions(); + void ChangePartitionCounters(ssize_t activeDiff, ssize_t inactiveDiff); + void LockPartition(ui32 partitionId, const TActorContext& ctx); + std::unique_ptr MakeEvReleasePartition(ui32 partitionId) const; + std::unique_ptr MakeEvLockPartition(ui32 partitionId, ui32 step) const; + TString GetPrefix() const; +}; + +struct TPartitionFamilyComparator { + bool operator()(const TPartitionFamily* lhs, const TPartitionFamily* rhs) const; +}; + +using TOrderedPartitionFamilies = std::set; + +struct SessionComparator { + bool operator()(const TSession* lhs, const TSession* rhs) const; +}; + +using TOrderedSessions = std::set; + +// It contains all the logic of balancing the reading sessions of a single consumer: the distribution of partitions +// across reading sessions, the uniformity of the load. +struct TConsumer { + friend struct TPartitionFamily; + + TBalancer& Balancer; + + TString ConsumerName; + + size_t NextFamilyId; + std::unordered_map> Families; + + // Mapping the IDs of the partitions to the families they belong to + std::unordered_map PartitionMapping; + // All reading sessions in which the family is currently being read. + std::unordered_map Sessions; + + // Families is not reading now. + std::unordered_map UnreadableFamilies; + // Families that require balancing. Only families are included here if there are reading + // sessions that want to read the partitions of this family. + std::unordered_map FamiliesRequireBalancing; + + std::unordered_map Partitions; + + size_t ActiveFamilyCount; + bool BalanceScheduled; + + TConsumer(TBalancer& balancer, const TString& consumerName); + ~TConsumer() = default; + + const TString& Topic() const; + const TString& TopicPath() const; + ui32 TabletGeneration() const; + const TPartitionInfo* GetPartitionInfo(ui32 partitionId) const; + TPartition* GetPartition(ui32 partitionId); + const TPartitionGraph& GetPartitionGraph() const; + ui32 NextStep(); + + void RegisterPartition(ui32 partitionId, const TActorContext& ctx); + void UnregisterPartition(ui32 partitionId, const TActorContext& ctx); + void InitPartitions(const TActorContext& ctx); + + TPartitionFamily* CreateFamily(std::vector&& partitions, const TActorContext& ctx); + TPartitionFamily* CreateFamily(std::vector&& partitions, TPartitionFamily::EStatus status, const TActorContext& ctx); + bool BreakUpFamily(ui32 partitionId, bool destroy, const TActorContext& ctx); + bool BreakUpFamily(TPartitionFamily* family, ui32 partitionId, bool destroy, const TActorContext& ctx); + bool MergeFamilies(TPartitionFamily* lhs, TPartitionFamily* rhs, const TActorContext& ctx); + void DestroyFamily(TPartitionFamily* family, const TActorContext& ctx); + TPartitionFamily* FindFamily(ui32 partitionId); + + void RegisterReadingSession(TSession* session, const TActorContext& ctx); + void UnregisterReadingSession(TSession* session, const TActorContext& ctx); + + bool Unlock(const TActorId& sender, ui32 partitionId, const TActorContext& ctx); + + bool SetCommittedState(ui32 partitionId, ui32 generation, ui64 cookie); + bool ProccessReadingFinished(ui32 partitionId, const TActorContext& ctx); + void StartReading(ui32 partitionId, const TActorContext& ctx); + void FinishReading(TEvPersQueue::TEvReadingPartitionFinishedRequest::TPtr& ev, const TActorContext& ctx); + + void ScheduleBalance(const TActorContext& ctx); + void Balance(const TActorContext& ctx); + void Release(ui32 partitionId, const TActorContext& ctx); + + bool IsReadable(ui32 partitionId); + bool IsInactive(ui32 partitionId); + + bool ScalingSupport() const; + +private: + TString GetPrefix() const; +}; + +struct TSession { + TSession(const TActorId& pipe); + + const TActorId Pipe; + + // The consumer name + TString ClientId; + TString SessionName; + TActorId Sender; + + TString ClientNode; + ui32 ProxyNodeId; + TInstant CreateTimestamp; + + // partitions which are reading + std::unordered_set Partitions; + // the number of pipes connected from SessionActor to ReadBalancer + ui32 ServerActors; + + // The number of active partitions + size_t ActivePartitionCount; + // The number of inactive partitions + size_t InactivePartitionCount; + // The number of releasing partitions (active and inactive) + size_t ReleasingPartitionCount; + + // The number of active families (the status equal Active) + size_t ActiveFamilyCount; + // The number of releasing families (the status equal Releasing) + size_t ReleasingFamilyCount; + + // The partition families that are being read by this session. + std::unordered_map Families; + + // true if client connected to read from concret partitions + bool WithGroups() const; + + template + bool AllPartitionsReadable(const TCollection& partitions) const; + + TString DebugStr() const; +}; + +struct TStatistics { + struct TConsumerStatistics { + struct TPartitionStatistics { + ui32 PartitionId; + ui64 TabletId = 0; + ui32 State = 0; + TString Session; + }; + + TString ConsumerName; + std::vector Partitions; + }; + + struct TSessionStatistics { + TString Session; + size_t ActivePartitionCount; + size_t InactivePartitionCount; + size_t SuspendedPartitionCount; + size_t TotalPartitionCount; + }; + + std::vector Consumers; + std::vector Sessions; + + size_t FreePartitions; +}; + +class TBalancer { + friend struct TConsumer; +public: + TBalancer(TPersQueueReadBalancer& topicActor); + + const TString& Topic() const; + const TString& TopicPath() const; + ui32 TabletGeneration() const; + + const TPartitionInfo* GetPartitionInfo(ui32 partitionId) const; + const std::unordered_map& GetPartitionsInfo() const; + const TPartitionGraph& GetPartitionGraph() const; + bool ScalingSupport() const; + i32 GetLifetimeSeconds() const; + + TConsumer* GetConsumer(const TString& consumerName); + const TStatistics GetStatistics() const; + + void UpdateConfig(std::vector addedPartitions, std::vector deletedPartitions, const TActorContext& ctx); + bool SetCommittedState(const TString& consumer, ui32 partitionId, ui32 generation, ui64 cookie, const TActorContext& ctx); + + void Handle(TEvPQ::TEvReadingPartitionStatusRequest::TPtr& ev, const TActorContext& ctx); + void Handle(TEvPersQueue::TEvReadingPartitionStartedRequest::TPtr& ev, const TActorContext& ctx); + void Handle(TEvPersQueue::TEvReadingPartitionFinishedRequest::TPtr& ev, const TActorContext& ctx); + + void Handle(TEvPersQueue::TEvPartitionReleased::TPtr& ev, const TActorContext& ctx); + + void Handle(TEvPQ::TEvWakeupReleasePartition::TPtr &ev, const TActorContext& ctx); + + void Handle(TEvTabletPipe::TEvServerConnected::TPtr& ev, const TActorContext&); + void Handle(TEvTabletPipe::TEvServerDisconnected::TPtr& ev, const TActorContext&); + + void Handle(TEvPersQueue::TEvRegisterReadSession::TPtr& ev, const TActorContext& ctx); + + void Handle(TEvPersQueue::TEvGetReadSessionsInfo::TPtr& ev, const TActorContext& ctx); + + void Handle(TEvPQ::TEvBalanceConsumer::TPtr& ev, const TActorContext& ctx); + +private: + TString GetPrefix() const; + ui32 NextStep(); + +private: + TPersQueueReadBalancer& TopicActor; + + std::unordered_map> Sessions; + std::unordered_map> Consumers; + + ui32 Step; +}; + +} diff --git a/ydb/core/persqueue/read_balancer__txinit.h b/ydb/core/persqueue/read_balancer__txinit.h index 9f33aa120eb3..aed928dd707e 100644 --- a/ydb/core/persqueue/read_balancer__txinit.h +++ b/ydb/core/persqueue/read_balancer__txinit.h @@ -54,11 +54,9 @@ struct TPersQueueReadBalancer::TTxInit : public ITransaction { Migrate(Self->TabletConfig); Self->Consumers.clear(); - for (auto& consumer : Self->TabletConfig.GetConsumers()) { - Self->Consumers[consumer.GetName()].ScalingSupport = consumer.HasScalingSupport() ? consumer.GetScalingSupport() : DefaultScalingSupport(); + Self->Consumers[consumer.GetName()]; } - Self->PartitionGraph = MakePartitionGraph(Self->TabletConfig); } Self->Inited = true; @@ -72,7 +70,7 @@ struct TPersQueueReadBalancer::TTxInit : public ITransaction { ui32 part = partsRowset.GetValue(); ui64 tabletId = partsRowset.GetValue(); - partitionsInfo[part] = {tabletId, EPartitionState::EPS_FREE, TActorId(), part + 1, {}}; + partitionsInfo = {tabletId, {}}; Self->AggregatedStats.AggrStats(part, partsRowset.GetValue(), partsRowset.GetValue()); @@ -81,29 +79,7 @@ struct TPersQueueReadBalancer::TTxInit : public ITransaction { } Self->PartitionsInfo.insert(partitionsInfo.rbegin(), partitionsInfo.rend()); - while (!groupsRowset.EndOfSet()) { //found out tablets for partitions - ui32 groupId = groupsRowset.GetValue(); - ui32 partition = groupsRowset.GetValue(); - Y_ABORT_UNLESS(groupId > 0); - auto jt = Self->PartitionsInfo.find(partition); - Y_ABORT_UNLESS(jt != Self->PartitionsInfo.end()); - jt->second.GroupId = groupId; - - Self->NoGroupsInBase = false; - - if (!groupsRowset.Next()) - return false; - } - - Y_ABORT_UNLESS(Self->ClientsInfo.empty()); - - for (auto& p : Self->PartitionsInfo) { - ui32 groupId = p.second.GroupId; - Self->GroupsInfo[groupId].push_back(p.first); - - } - Self->TotalGroups = Self->GroupsInfo.size(); - + Self->TotalGroups = Self->PartitionsInfo.size(); while (!tabletsRowset.EndOfSet()) { //found out tablets for partitions ui64 tabletId = tabletsRowset.GetValue(); diff --git a/ydb/core/persqueue/read_balancer__txwrite.h b/ydb/core/persqueue/read_balancer__txwrite.h index e1312af9631d..29eb5243a619 100644 --- a/ydb/core/persqueue/read_balancer__txwrite.h +++ b/ydb/core/persqueue/read_balancer__txwrite.h @@ -93,7 +93,6 @@ struct TPersQueueReadBalancer::TTxWrite : public ITransaction { } Self->WaitingResponse.clear(); - Self->NoGroupsInBase = false; if (!Self->Inited) { Self->Inited = true; Self->InitDone(ctx); diff --git a/ydb/core/persqueue/read_balancer__types.cpp b/ydb/core/persqueue/read_balancer__types.cpp deleted file mode 100644 index 803d933ede4c..000000000000 --- a/ydb/core/persqueue/read_balancer__types.cpp +++ /dev/null @@ -1,89 +0,0 @@ -#include "read_balancer.h" - - -namespace NKikimr::NPQ { - -// -// TReadingPartitionStatus -// - -bool TPersQueueReadBalancer::TReadingPartitionStatus::IsFinished() const { - return Commited || (ReadingFinished && (StartedReadingFromEndOffset || ScaleAwareSDK)); -} - -bool TPersQueueReadBalancer::TReadingPartitionStatus::NeedReleaseChildren() const { - return !(Commited || (ReadingFinished && !ScaleAwareSDK)); -} - -bool TPersQueueReadBalancer::TReadingPartitionStatus::BalanceToOtherPipe() const { - return LastPipe && !Commited && ReadingFinished && !ScaleAwareSDK; -} - -bool TPersQueueReadBalancer::TReadingPartitionStatus::StartReading() { - return std::exchange(ReadingFinished, false); -} - -bool TPersQueueReadBalancer::TReadingPartitionStatus::StopReading() { - ReadingFinished = false; - ++Cookie; - return NeedReleaseChildren(); -} - -bool TPersQueueReadBalancer::TReadingPartitionStatus::SetCommittedState(ui32 generation, ui64 cookie) { - if (PartitionGeneration < generation || (PartitionGeneration == generation && PartitionCookie < cookie)) { - Iteration = 0; - PartitionGeneration = generation; - PartitionCookie = cookie; - - return !std::exchange(Commited, true); - } - - return false; -} - -bool TPersQueueReadBalancer::TReadingPartitionStatus::SetFinishedState(bool scaleAwareSDK, bool startedReadingFromEndOffset) { - bool previousStatus = IsFinished(); - - ScaleAwareSDK = scaleAwareSDK; - StartedReadingFromEndOffset = startedReadingFromEndOffset; - ReadingFinished = true; - ++Cookie; - - bool currentStatus = IsFinished(); - if (currentStatus) { - Iteration = 0; - } else { - ++Iteration; - } - if (scaleAwareSDK || currentStatus) { - LastPipe = TActorId(); - } - return currentStatus && !previousStatus; -} - -bool TPersQueueReadBalancer::TReadingPartitionStatus::Reset() { - bool result = IsFinished(); - - ScaleAwareSDK = false; - ReadingFinished = false; - Commited = false; - ++Cookie; - LastPipe = TActorId(); - - return result; -}; - - -// -// TSessionInfo -// - -void TPersQueueReadBalancer::TSessionInfo::Unlock(bool inactive) { - --NumActive; - --NumSuspended; - if (inactive) { - -- NumInactive; - } -} - -} diff --git a/ydb/core/persqueue/ut/autoscaling_ut.cpp b/ydb/core/persqueue/ut/autoscaling_ut.cpp index 6d033a07daf3..4255ec43f9f1 100644 --- a/ydb/core/persqueue/ut/autoscaling_ut.cpp +++ b/ydb/core/persqueue/ut/autoscaling_ut.cpp @@ -26,7 +26,7 @@ using namespace NYdb::NTopic; using namespace NYdb::NTopic::NTests; using namespace NSchemeShardUT_Private; -Y_UNIT_TEST_SUITE(TopicSplitMerge) { +Y_UNIT_TEST_SUITE(TopicAutoscaling) { Y_UNIT_TEST(Simple) { TTopicSdkTestSetup setup = CreateSetup(); @@ -280,28 +280,20 @@ Y_UNIT_TEST_SUITE(TopicSplitMerge) { ui64 txId = 1023; SplitPartition(setup, ++txId, 0, "a"); - TTestReadSession readSession1("Session-0", client, Max(), false); + TTestReadSession readSession1("Session-0", client, Max(), false, {0, 1, 2}); readSession1.Offsets[0] = 1; readSession1.WaitAndAssertPartitions({0, 1, 2}, "Must read all exists partitions because read the partition 0 from offset 1"); readSession1.Offsets[0] = 0; - TTestReadSession readSession2("Session-1", client, Max(), false, 0); + TTestReadSession readSession2("Session-1", client, Max(), false, {0}); readSession2.Offsets[0] = 0; - auto p1 = readSession1.Wait({}, "Must release all partitions becase readSession2 read not from EndOffset"); - auto p2 = readSession2.Wait({0}, "Must read partition 0 because it defined in the readSession"); + readSession2.WaitAndAssertPartitions({0}, "Must read partition 0 because it defined in the readSession"); + readSession2.Run(); - p1.Wait(TDuration::Seconds(5)); - readSession1.Assert({}, p1, ""); + readSession1.WaitAndAssertPartitions({}, "Must release all partitions becase readSession2 read not from EndOffset"); readSession1.Run(); - p2.Wait(TDuration::Seconds(5)); - readSession2.Assert({0}, p2, ""); - - readSession2.WaitAndAssertPartitions({}, "Partition must be released because reding finished"); - readSession2.Run(); - - readSession1.WaitAndAssertPartitions({}, "Partitions must be read only from Session-1"); readSession1.WaitAndAssertPartitions({0}, "Partition 0 must rebalance to other sessions (Session-0)"); readSession1.Close(); diff --git a/ydb/core/persqueue/ut/balancing_ut.cpp b/ydb/core/persqueue/ut/balancing_ut.cpp new file mode 100644 index 000000000000..9caf286a0aeb --- /dev/null +++ b/ydb/core/persqueue/ut/balancing_ut.cpp @@ -0,0 +1,155 @@ +#include + +#include + +#include +#include +#include + +#include +#include + + +static inline IOutputStream& operator<<(IOutputStream& o, std::set t) { + o << "[" << JoinRange(", ", t.begin(), t.end()) << "]"; + return o; +} + +namespace NKikimr { + +using namespace NYdb::NTopic; +using namespace NYdb::NTopic::NTests; +using namespace NSchemeShardUT_Private; + +Y_UNIT_TEST_SUITE(Balancing) { + + Y_UNIT_TEST(Simple) { + TTopicSdkTestSetup setup = CreateSetup(); + setup.CreateTopic(TEST_TOPIC, TEST_CONSUMER, 10); + + TTopicClient client = setup.MakeClient(); + + TTestReadSession readSession0("Session-0", client); + { + readSession0.WaitAndAssertPartitions({0, 1, 2, 3, 4, 5, 6, 7, 8, 9}, "Single reading session must read all partitions"); + readSession0.Run(); + } + + TTestReadSession readSession1("Session-1", client); + { + readSession1.Run(); + + Sleep(TDuration::Seconds(1)); + + auto p0 = readSession0.GetPartitions(); + auto p1 = readSession1.GetPartitions(); + + UNIT_ASSERT_VALUES_EQUAL_C(5, p0.size(), "After the appearance of the second reading session, the partitions should be distributed evenly among them (p0, " << p0 << ")"); + UNIT_ASSERT_VALUES_EQUAL_C(5, p1.size(), "After the appearance of the second reading session, the partitions should be distributed evenly among them (p1, " << p1 << ")"); + p0.insert(p1.begin(), p1.end()); + UNIT_ASSERT_VALUES_EQUAL_C(10, p0.size(), "Must read all partitions but " << p0); + } + + TTestReadSession readSession2("Session-2", client, Max(), true, {0, 1}); + { + readSession2.WaitAndAssertPartitions({0, 1}, "The reading session should read partitions 0 and 1 because it clearly required them to be read."); + readSession2.Run(); + + Sleep(TDuration::Seconds(1)); + + auto p0 = readSession0.GetPartitions(); + auto p1 = readSession1.GetPartitions(); + auto pa = p0; + pa.insert(p1.begin(), p1.end()); + UNIT_ASSERT_VALUES_EQUAL_C(4, p0.size(), "There should be an even distribution of partitions " << p0); + UNIT_ASSERT_VALUES_EQUAL_C(4, p1.size(), "There should be an even distribution of partitions " << p1); + UNIT_ASSERT_VALUES_EQUAL_C(8, pa.size(), "Must read all partitions but " << pa); + } + + TTestReadSession readSession3("Session-3", client, Max(), true, {0}); + { + readSession3.WaitAndAssertPartitions({0}, "The reading session should read partitions 0 and 1 because it clearly required them to be read."); + readSession2.WaitAndAssertPartitions({1}, "The reading session should read partitions 0 and 1 because it clearly required them to be read."); + + auto p0 = readSession0.Partitions; + p0.insert(readSession1.Partitions.begin(), readSession1.Partitions.end()); + UNIT_ASSERT_VALUES_EQUAL_C(8, p0.size(), "Must read all partitions but " << p0); + } + + { + readSession3.Run(); + readSession3.Close(); + + readSession2.WaitAndAssertPartitions({0, 1}, "The reading session should read partitions 0 and 1 because it clearly required them to be read. (after release Session-3)"); + readSession2.Run(); + } + + { + readSession2.Run(); + readSession2.Close(); + + Sleep(TDuration::Seconds(1)); + + auto p0 = readSession0.GetPartitions(); + auto p1 = readSession1.GetPartitions(); + + UNIT_ASSERT_VALUES_EQUAL_C(5, p0.size(), "After the appearance of the second reading session, the partitions should be distributed evenly among them (p0, " << p0 << ")"); + UNIT_ASSERT_VALUES_EQUAL_C(5, p1.size(), "After the appearance of the second reading session, the partitions should be distributed evenly among them (p1, " << p1 << ")"); + p0.insert(p1.begin(), p1.end()); + UNIT_ASSERT_VALUES_EQUAL_C(10, p0.size(), "Must read all partitions but " << p0); + } + + { + readSession1.Run(); + readSession1.Close(); + + readSession0.WaitAndAssertPartitions({0, 1, 2, 3, 4, 5, 6, 7, 8, 9}, "Single reading session must read all partitions"); + readSession0.Run(); + } + + + readSession0.Close(); + } + +/* + Y_UNIT_TEST(BalanceManySession) { + TTopicSdkTestSetup setup = CreateSetup(); + setup.CreateTopic(TEST_TOPIC, TEST_CONSUMER, 1000); + + TTopicClient client = setup.MakeClient(); + + auto CreateClient = [&](size_t i) { + auto readSettings = TReadSessionSettings() + .ConsumerName(TEST_CONSUMER) + .AppendTopics(TEST_TOPIC); + readSettings.Topics_[0].AppendPartitionIds(i % 1000); + + return client.CreateReadSession(readSettings); + }; + + Cerr << ">>>>> " << TInstant::Now() << " Begin create sessions" << Endl << Flush; + + std::deque> sessions; + for (int i = 0; i < 2000; ++i) { + sessions.push_back(CreateClient(i)); + } + + for (int i = 0 ; i < 1000 ; ++i) { + Cerr << ">>>>> " << TInstant::Now() << " Close session " << i << Endl << Flush; + + auto s = sessions.front(); + s->Close(); + sessions.pop_front(); + + Sleep(TDuration::MilliSeconds(50)); + + sessions.push_back(CreateClient(i * 7)); + } + + Cerr << ">>>>> " << TInstant::Now() << " Finished" << Endl << Flush; + Sleep(TDuration::Seconds(10)); + } +*/ + } + +} // namespace NKikimr diff --git a/ydb/core/persqueue/ut/common/autoscaling_ut_common.cpp b/ydb/core/persqueue/ut/common/autoscaling_ut_common.cpp index c6f26a9c2953..ad3d47157c0f 100644 --- a/ydb/core/persqueue/ut/common/autoscaling_ut_common.cpp +++ b/ydb/core/persqueue/ut/common/autoscaling_ut_common.cpp @@ -119,7 +119,7 @@ std::shared_ptr CreateWriteSession(TTopicClient& cl } -TTestReadSession::TTestReadSession(const TString& name, TTopicClient& client, size_t expectedMessagesCount, bool autoCommit, std::optional partition) +TTestReadSession::TTestReadSession(const TString& name, TTopicClient& client, size_t expectedMessagesCount, bool autoCommit, std::set partitions) : Name(name) , AutoCommit(autoCommit) , Semaphore(name.c_str(), SemCount) { @@ -129,8 +129,8 @@ TTestReadSession::TTestReadSession(const TString& name, TTopicClient& client, si auto readSettings = TReadSessionSettings() .ConsumerName(TEST_CONSUMER) .AppendTopics(TEST_TOPIC); - if (partition) { - readSettings.Topics_[0].AppendPartitionIds(partition.value()); + for (auto partitionId : partitions) { + readSettings.Topics_[0].AppendPartitionIds(partitionId); } readSettings.EventHandlers_.SimpleDataHandlers( @@ -206,7 +206,7 @@ TTestReadSession::TTestReadSession(const TString& name, TTopicClient& client, si } void TTestReadSession::WaitAllMessages() { - DataPromise.GetFuture().GetValueSync(); + DataPromise.GetFuture().GetValue(TDuration::Seconds(5)); } void TTestReadSession::Commit() { @@ -244,8 +244,9 @@ NThreading::TFuture> TTestReadSession::Wait(std::set pa } void TTestReadSession::Assert(const std::set& expected, NThreading::TFuture> f, const TString& message) { - Cerr << ">>>>> " << Name << " Partitions " << Partitions << " received #2" << Endl << Flush; - UNIT_ASSERT_VALUES_EQUAL_C(expected, f.HasValue() ? f.GetValueSync() : Partitions, message); + auto actual = f.HasValue() ? f.GetValueSync() : GetPartitions(); + Cerr << ">>>>> " << Name << " Partitions " << actual << " received #2" << Endl << Flush; + UNIT_ASSERT_VALUES_EQUAL_C(expected, actual, message); Release(); } @@ -263,10 +264,17 @@ void TTestReadSession::Run() { void TTestReadSession::Close() { Run(); + Cerr << ">>>>> " << Name << " Closing reading session " << Endl << Flush; Session->Close(); Session.reset(); } +std::set TTestReadSession::GetPartitions() { + with_lock (Lock) { + return Partitions; + } +} + void TTestReadSession::Modify(std::function&)> modifier) { bool found = false; diff --git a/ydb/core/persqueue/ut/common/autoscaling_ut_common.h b/ydb/core/persqueue/ut/common/autoscaling_ut_common.h index 83ba86c79e6e..f60e60428427 100644 --- a/ydb/core/persqueue/ut/common/autoscaling_ut_common.h +++ b/ydb/core/persqueue/ut/common/autoscaling_ut_common.h @@ -59,7 +59,7 @@ struct TTestReadSession { static constexpr size_t SemCount = 1; - TTestReadSession(const TString& name, TTopicClient& client, size_t expectedMessagesCount = Max(), bool autoCommit = true, std::optional partition = std::nullopt); + TTestReadSession(const TString& name, TTopicClient& client, size_t expectedMessagesCount = Max(), bool autoCommit = true, std::set partitions = {}); void WaitAllMessages(); NThreading::TFuture> Wait(std::set partitions, const TString& message); @@ -72,6 +72,8 @@ struct TTestReadSession { void Close(); + std::set GetPartitions(); + private: void Acquire(); void Release(); diff --git a/ydb/core/persqueue/ut/common/pq_ut_common.h b/ydb/core/persqueue/ut/common/pq_ut_common.h index 81ba6399b0d8..ce2336cd66c2 100644 --- a/ydb/core/persqueue/ut/common/pq_ut_common.h +++ b/ydb/core/persqueue/ut/common/pq_ut_common.h @@ -104,6 +104,7 @@ struct TTestContext { NActors::NLog::EPriority otherPriority = NLog::PRI_INFO; runtime.SetLogPriority(NKikimrServices::PERSQUEUE, pqPriority); + runtime.SetLogPriority(NKikimrServices::PERSQUEUE_READ_BALANCER, pqPriority); runtime.SetLogPriority(NKikimrServices::SYSTEM_VIEWS, pqPriority); runtime.SetLogPriority(NKikimrServices::KEYVALUE, priority); diff --git a/ydb/core/persqueue/ut/pq_ut.cpp b/ydb/core/persqueue/ut/pq_ut.cpp index 149bfca6c90a..a8883af070aa 100644 --- a/ydb/core/persqueue/ut/pq_ut.cpp +++ b/ydb/core/persqueue/ut/pq_ut.cpp @@ -316,115 +316,6 @@ Y_UNIT_TEST(TestPartitionWriteQuota) { }); } -Y_UNIT_TEST(TestGroupsBalancer) { - TTestContext tc; - TFinalizer finalizer(tc); - tc.Prepare(); - - tc.Runtime->SetScheduledLimit(50); - tc.Runtime->SetDispatchTimeout(TDuration::Seconds(1)); - tc.Runtime->SetLogPriority(NKikimrServices::PERSQUEUE, NLog::PRI_DEBUG); - TFakeSchemeShardState::TPtr state{new TFakeSchemeShardState()}; - ui64 ssId = 325; - BootFakeSchemeShard(*tc.Runtime, ssId, state); - - PQBalancerPrepare(TOPIC_NAME, {{0,{1, 1}}, {11,{1, 1}}, {1,{1, 2}}, {2,{1, 2}}}, ssId, tc); - - TActorId pipe = RegisterReadSession("session1", tc); - Y_UNUSED(pipe); - WaitPartition("session1", tc, 0, "", "", TActorId()); - WaitPartition("session1", tc, 0, "", "", TActorId()); - WaitPartition("session1", tc, 0, "", "", TActorId()); - WaitPartition("session1", tc, 0, "", "", TActorId()); - WaitPartition("", tc, 0, "", "", TActorId(), false);//no partitions - return error - - TActorId pipe2 = RegisterReadSession("session2", tc, {1}); - - WaitPartition("session2", tc, 0, "", "", TActorId()); - WaitPartition("session2", tc, 0, "", "", TActorId()); - WaitPartition("", tc, 0, "", "", TActorId(), false);//no partitions to balance - - TActorId pipe4 = RegisterReadSession("session8", tc, {1}); - Y_UNUSED(pipe4); - - WaitPartition("session8", tc, 0, "session2", "topic1", pipe2); - WaitPartition("", tc, 0, "", "", TActorId(), false);//no partitions to balance - - tc.Runtime->Send(new IEventHandle(pipe2, tc.Edge, new TEvents::TEvPoisonPill()), 0, true); //will cause dying of pipe and first session - - WaitPartition("session8", tc, 0, "", "", TActorId()); - WaitPartition("", tc, 0, "", "", TActorId(), false);//no partitions to balance - - RegisterReadSession("session3", tc); - WaitPartition("session3", tc, 0, "", "", TActorId()); - WaitPartition("session3", tc, 0, "", "", TActorId()); - WaitPartition("session3", tc, 0, "session8", "topic1", pipe4); - WaitPartition("", tc, 0, "", "", TActorId(), false);//no partitions to balance - - -} - -Y_UNIT_TEST(TestGroupsBalancer2) { - TTestContext tc; - TFinalizer finalizer(tc); - tc.Prepare(); - - tc.Runtime->SetScheduledLimit(50); - tc.Runtime->SetDispatchTimeout(TDuration::Seconds(1)); - tc.Runtime->SetLogPriority(NKikimrServices::PERSQUEUE, NLog::PRI_DEBUG); - TFakeSchemeShardState::TPtr state{new TFakeSchemeShardState()}; - ui64 ssId = 325; - BootFakeSchemeShard(*tc.Runtime, ssId, state); - - PQBalancerPrepare(TOPIC_NAME, {{0, {1, 1}}, {1, {1, 2}}, {2, {1, 3}}, {3, {1, 4}}}, ssId, tc); - - TActorId pipe = RegisterReadSession("session1", tc, {1,2}); - Y_UNUSED(pipe); - - WaitPartition("session1", tc, 0, "", "", TActorId()); - WaitPartition("session1", tc, 0, "", "", TActorId()); - WaitPartition("", tc, 0, "", "", TActorId(), false);//no partitions - return error - TActorId pipe2 = RegisterReadSession("session2", tc, {3,4}); - Y_UNUSED(pipe2); - - WaitPartition("session2", tc, 0, "", "", TActorId()); - WaitPartition("session2", tc, 0, "", "", TActorId()); - WaitPartition("", tc, 0, "", "", TActorId(), false);//no partitions - return error -} - -Y_UNIT_TEST(TestGroupsBalancer3) { - TTestContext tc; - TFinalizer finalizer(tc); - tc.Prepare(); - - tc.Runtime->SetScheduledLimit(50); - tc.Runtime->SetDispatchTimeout(TDuration::Seconds(1)); - tc.Runtime->SetLogPriority(NKikimrServices::PERSQUEUE, NLog::PRI_DEBUG); - TFakeSchemeShardState::TPtr state{new TFakeSchemeShardState()}; - ui64 ssId = 325; - BootFakeSchemeShard(*tc.Runtime, ssId, state); - - PQBalancerPrepare(TOPIC_NAME, {{0, {1, 1}}, {1, {1, 2}} }, ssId, tc); - - TActorId pipe = RegisterReadSession("session", tc, {2}); - - WaitPartition("session", tc, 0, "", "", TActorId()); - WaitPartition("", tc, 0, "", "", TActorId(), false);//no partitions - return error - - tc.Runtime->Send(new IEventHandle(pipe, tc.Edge, new TEvents::TEvPoisonPill()), 0, true); //will cause dying of pipe and first session - - TActorId pipe2 = RegisterReadSession("session1", tc); - Y_UNUSED(pipe2); - - WaitPartition("session1", tc, 0, "", "", TActorId()); - WaitPartition("session1", tc, 0, "", "", TActorId()); - WaitPartition("", tc, 0, "", "", TActorId(), false);//no partitions - return error - - pipe = RegisterReadSession("session2", tc, {2}); - WaitReadSessionKill(tc); //session 1 will die -} - - Y_UNIT_TEST(TestUserInfoCompatibility) { TTestContext tc; RunTestWithReboots(tc.TabletIds, [&]() { diff --git a/ydb/core/persqueue/ut/ya.make b/ydb/core/persqueue/ut/ya.make index 156b48944f9b..2b5cd72f9a1a 100644 --- a/ydb/core/persqueue/ut/ya.make +++ b/ydb/core/persqueue/ut/ya.make @@ -29,6 +29,8 @@ PEERDIR( YQL_LAST_ABI_VERSION() SRCS( + autoscaling_ut.cpp + balancing_ut.cpp counters_ut.cpp pqtablet_mock.cpp internals_ut.cpp @@ -42,7 +44,6 @@ SRCS( pqtablet_ut.cpp quota_tracker_ut.cpp sourceid_ut.cpp - autoscaling_ut.cpp type_codecs_ut.cpp user_info_ut.cpp pqrb_describes_ut.cpp diff --git a/ydb/core/persqueue/utils.cpp b/ydb/core/persqueue/utils.cpp index 3a3c940d0eb3..124c56b4d5d3 100644 --- a/ydb/core/persqueue/utils.cpp +++ b/ydb/core/persqueue/utils.cpp @@ -253,6 +253,10 @@ TPartitionGraph::Node::Node(ui32 id, ui64 tabletId) , TabletId(tabletId) { } +bool TPartitionGraph::Node::IsRoot() const { + return Parents.empty(); +} + TPartitionGraph MakePartitionGraph(const NKikimrPQ::TPQTabletConfig& config) { return TPartitionGraph(BuildGraph(config.GetAllPartitions())); } diff --git a/ydb/core/persqueue/utils.h b/ydb/core/persqueue/utils.h index e22c1a53902c..390b7bbcdb8d 100644 --- a/ydb/core/persqueue/utils.h +++ b/ydb/core/persqueue/utils.h @@ -43,6 +43,8 @@ class TPartitionGraph { std::vector Children; // All parents include parents of parents and so on std::set HierarhicalParents; + + bool IsRoot() const; }; TPartitionGraph(); diff --git a/ydb/core/persqueue/ya.make b/ydb/core/persqueue/ya.make index 2b1d42cc08d2..12777e54cadf 100644 --- a/ydb/core/persqueue/ya.make +++ b/ydb/core/persqueue/ya.make @@ -33,7 +33,7 @@ SRCS( pq_l2_cache.cpp pq_rl_helpers.cpp quota_tracker.cpp - read_balancer__types.cpp + read_balancer__balancing.cpp read_balancer.cpp account_read_quoter.cpp read_quoter.cpp @@ -49,6 +49,7 @@ SRCS( dread_cache_service/caching_service.cpp ) +GENERATE_ENUM_SERIALIZATION(read_balancer__balancing.h) GENERATE_ENUM_SERIALIZATION(sourceid_info.h) PEERDIR( diff --git a/ydb/services/persqueue_v1/persqueue_new_schemecache_ut.cpp b/ydb/services/persqueue_v1/persqueue_new_schemecache_ut.cpp index 7d4a33371514..9df02407fe60 100644 --- a/ydb/services/persqueue_v1/persqueue_new_schemecache_ut.cpp +++ b/ydb/services/persqueue_v1/persqueue_new_schemecache_ut.cpp @@ -320,15 +320,22 @@ namespace NKikimr::NPersQueueTests { } } - Y_UNIT_TEST(TestReadAtTimestamp) { + Y_UNIT_TEST(TestReadAtTimestamp_3) { auto generate = [](ui32 messageId) { return TStringBuilder() << "Hello___" << messageId << "___" << CreateGuidAsString() << TString(1_MB, 'a'); }; - TestReadAtTimestampImpl(10, generate); TestReadAtTimestampImpl(3, generate); } + Y_UNIT_TEST(TestReadAtTimestamp_10) { + auto generate = [](ui32 messageId) { + return TStringBuilder() << "Hello___" << messageId << "___" << CreateGuidAsString() << TString(1_MB, 'a'); + }; + + TestReadAtTimestampImpl(10, generate); + } + Y_UNIT_TEST(TestWriteStat1stClass) { auto testWriteStat1stClass = [](const TString& consumerName) { TTestServer server(false);