From 75fce8161412437f64bd91f3cc0861201bf822a5 Mon Sep 17 00:00:00 2001 From: Nikolay Shestakov Date: Sun, 9 Nov 2025 16:33:59 +0000 Subject: [PATCH 01/19] WIP --- .../persqueue/pqtablet/partition/mlp/mlp.h | 1 + .../pqtablet/partition/mlp/mlp_common.h | 12 +++++ .../pqtablet/partition/mlp/mlp_consumer.cpp | 26 ++++++++-- .../pqtablet/partition/mlp/mlp_consumer.h | 6 ++- .../pqtablet/partition/mlp/mlp_dlq_mover.cpp | 21 ++++++++ .../pqtablet/partition/mlp/mlp_dlq_mover.h | 48 +++++++++++++++++++ .../partition/mlp/mlp_message_enricher.cpp | 7 +++ .../pqtablet/partition/mlp/mlp_storage.cpp | 48 ++++++++++++------- .../pqtablet/partition/mlp/mlp_storage.h | 29 ++++++----- .../mlp/mlp_storage__serialization.cpp | 12 ++--- .../pqtablet/partition/mlp/mlp_storage_ut.cpp | 10 +++- .../persqueue/pqtablet/partition/mlp/ya.make | 2 +- .../pqtablet/partition/partition_mlp.cpp | 1 + 13 files changed, 177 insertions(+), 46 deletions(-) create mode 100644 ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp create mode 100644 ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp.h index 853262cf6969..33269a468d2b 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp.h @@ -14,6 +14,7 @@ namespace NKikimr::NPQ::NMLP { // MLP не работает если включена компактифкация по ключу!!! (иначе не понятно как прореживать скомпакченные значения) NActors::IActor* CreateConsumerActor( + const TString& database, ui64 tabletId, const NActors::TActorId& tabletActorId, ui32 partitionId, diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h index 2a93770cc23c..8b18c4b426ee 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h @@ -44,4 +44,16 @@ bool IsSucess(const TEvPQ::TEvProxyResponse::TPtr& ev); bool IsSucess(const TEvPersQueue::TEvResponse::TPtr& ev); ui64 GetCookie(const TEvPQ::TEvProxyResponse::TPtr& ev); +NActors::IActor* CreateMessageEnricher(const NActors::TActorId& tabletActorId, + const ui32 partitionId, + const TString& consumerName, + std::deque&& replies); + +NActors::IActor* CreateDLQMover(const TString& database, + const ui64 tabletId, + const ui32 partitionId, + const TString& consumerName, + const TString& destinationTopic, + std::deque&& offsets); + } // namespace NKikimr::NPQ::NMLP diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp index 33ddc9e2c433..7bca7698e6ed 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp @@ -1,5 +1,4 @@ #include "mlp_consumer.h" -#include "mlp_message_enricher.h" #include "mlp_storage.h" #include @@ -82,10 +81,11 @@ void AddReadWAL(std::unique_ptr& request, ui32 partitio readWAL->SetIncludeData(true); } -TConsumerActor::TConsumerActor(ui64 tabletId, const TActorId& tabletActorId, ui32 partitionId, +TConsumerActor::TConsumerActor(const TString& database,ui64 tabletId, const TActorId& tabletActorId, ui32 partitionId, const TActorId& partitionActorId, const NKikimrPQ::TPQTabletConfig_TConsumer& config, std::optional retentionPeriod) : TBaseTabletActor(tabletId, tabletActorId, NKikimrServices::EServiceKikimr::PQ_MLP_CONSUMER) + , Database(database) , PartitionId(partitionId) , PartitionActorId(partitionActorId) , Config(config) @@ -311,7 +311,7 @@ void TConsumerActor::Handle(TEvKeyValue::TEvResponse::TPtr& ev) { if (!PendingReadQueue.empty()) { auto msgs = std::exchange(PendingReadQueue, {}); - RegisterWithSameMailbox(new TMessageEnricherActor(TabletActorId, PartitionId, Config.GetName(), std::move(msgs))); + RegisterWithSameMailbox(CreateMessageEnricher(TabletActorId, PartitionId, Config.GetName(), std::move(msgs))); } ReplyOk(SelfId(), PendingCommitQueue); ReplyOk(SelfId(), PendingUnlockQueue); @@ -334,6 +334,11 @@ void TConsumerActor::UpdateStorageConfig() { Storage->SetKeepMessageOrder(Config.GetKeepMessageOrder()); Storage->SetMaxMessageProcessingCount(Config.GetMaxProcessingAttempts()); Storage->SetRetentionPeriod(RetentionPeriod); + if (Config.GetDeadLetterPolicyEnabled() && Config.GetDeadLetterPolicy() != NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_UNSPECIFIED) { + Storage->SetDeadLetterPolicy(Config.GetDeadLetterPolicy()); + } else { + Storage->SetDeadLetterPolicy(std::nullopt); + } } void TConsumerActor::Handle(TEvPQ::TEvMLPConsumerUpdateConfig::TPtr& ev) { @@ -672,6 +677,18 @@ void TConsumerActor::Handle(TEvPQ::TEvError::TPtr& ev) { void TConsumerActor::HandleOnWork(TEvents::TEvWakeup::TPtr&) { FetchMessagesIfNeeded(); ProcessEventQueue(); + + if (!DLQMoverActorId && !Storage->GetDLQMessages().empty()) { + std::deque messages; + DLQMoverActorId = RegisterWithSameMailbox(CreateDLQMover( + Database, + TabletId, + PartitionId, + Config.GetName(), + Config.GetDeadLetterQueue(), + std::move(messages) + )); + } Schedule(WakeupInterval, new TEvents::TEvWakeup()); } @@ -681,13 +698,14 @@ void TConsumerActor::Handle(TEvents::TEvWakeup::TPtr&) { } NActors::IActor* CreateConsumerActor( + const TString& database, ui64 tabletId, const NActors::TActorId& tabletActorId, ui32 partitionId, const NActors::TActorId& partitionActorId, const NKikimrPQ::TPQTabletConfig_TConsumer& config, const std::optional reteintion) { - return new TConsumerActor(tabletId, tabletActorId, partitionId, partitionActorId, config, reteintion); + return new TConsumerActor(database, tabletId, tabletActorId, partitionId, partitionActorId, config, reteintion); } } diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h index b96b6898dedd..5cc027978ecf 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h @@ -8,7 +8,6 @@ #include #include -// TODO MLP DLQ namespace NKikimr::NPQ::NMLP { class TBatch; @@ -21,7 +20,7 @@ class TConsumerActor : public TBaseTabletActor static constexpr TDuration WakeupInterval = TDuration::Seconds(1); public: - TConsumerActor(ui64 tabletId, const TActorId& tabletActorId, ui32 partitionId, + TConsumerActor(const TString& database, ui64 tabletId, const TActorId& tabletActorId, ui32 partitionId, const TActorId& partitionActorId, const NKikimrPQ::TPQTabletConfig::TConsumer& config, std::optional retentionPeriod); @@ -70,6 +69,7 @@ class TConsumerActor : public TBaseTabletActor void UpdateStorageConfig(); private: + const TString Database; const ui32 PartitionId; const TActorId PartitionActorId; NKikimrPQ::TPQTabletConfig::TConsumer Config; @@ -79,6 +79,8 @@ class TConsumerActor : public TBaseTabletActor ui64 FetchCookie = 0; ui64 LastCommittedOffset = 0; + TActorId DLQMoverActorId; + std::unique_ptr Storage; std::deque ReadRequestsQueue; diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp new file mode 100644 index 000000000000..fdbaccaf4863 --- /dev/null +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp @@ -0,0 +1,21 @@ +#include "mlp_dlq_mover.h" + +namespace NKikimr::NPQ::NMLP { + +NActors::IActor* CreateDLQMover(const TString& database, + const ui64 tabletId, + const ui32 partitionId, + const TString& consumerName, + const TString& destinationTopic, + std::deque&& offsets) { + Y_UNUSED(database); + Y_UNUSED(tabletId); + Y_UNUSED(partitionId); + Y_UNUSED(consumerName); + Y_UNUSED(destinationTopic); + Y_UNUSED(offsets); + + return nullptr; +} + +} diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h new file mode 100644 index 000000000000..652447a65a31 --- /dev/null +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h @@ -0,0 +1,48 @@ +#pragma once + +#include "mlp.h" +#include "mlp_common.h" + +#include +#include +#include +#include + +namespace NKikimr::NPQ::NMLP { + +class TDLQMoverActor : public TBaseActor + , public TConstantLogPrefix { + + static constexpr TDuration Timeout = TDuration::Seconds(5); + +public: + TDLQMoverActor(const TString& database, + const ui64 tabletId, + const ui32 partitionId, + const TString& consumerName, + const TString& destinationTopic, + std::deque&& offsets); + + void Bootstrap(); + void PassAway() override; + +private: + void Handle(TEvPersQueue::TEvResponse::TPtr&); + void Handle(TEvPQ::TEvError::TPtr&); + void Handle(TEvents::TEvWakeup::TPtr&); + void Handle(TEvPipeCache::TEvDeliveryProblem::TPtr&); + + STFUNC(StateWork); + + void ProcessQueue(); + +private: + const TString Database; + const ui64 TabletId; + const ui32 PartitionId; + const TString ConsumerName; + const TString DestinationTopic; + std::deque Queue; +}; + +} // namespace NKikimr::NPQ::NMLP diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_message_enricher.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_message_enricher.cpp index 450db4fac839..126dc8e2ee59 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_message_enricher.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_message_enricher.cpp @@ -136,4 +136,11 @@ void TMessageEnricherActor::ProcessQueue() { } } +NActors::IActor* CreateMessageEnricher(const NActors::TActorId& tabletActorId, + const ui32 partitionId, + const TString& consumerName, + std::deque&& replies) { + return new TMessageEnricherActor(tabletActorId, partitionId, consumerName, std::move(replies)); +} + } // namespace NKikimr::NPQ::NMLP diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp index e080086837a3..38697c72462e 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp @@ -38,6 +38,10 @@ void TStorage::SetRetentionPeriod(std::optional retentionPeriod) { RetentionPeriod = retentionPeriod; } +void TStorage::SetDeadLetterPolicy(std::optional deadLetterPolicy) { + DeadLetterPolicy = deadLetterPolicy; +} + std::optional TStorage::Next(TInstant deadline, TPosition& position) { auto dieDelta = Max(); if (RetentionPeriod) { @@ -276,11 +280,11 @@ bool TStorage::AddMessage(ui64 offset, bool hasMessagegroup, ui32 messageGroupId Messages.push_back({ .Status = EMessageStatus::Unprocessed, - .ReceiveCount = 0, + .ProcessingCount = 0, .DeadlineDelta = 0, .HasMessageGroupId = hasMessagegroup, .MessageGroupIdHash = messageGroupIdHash, - .WriteTimestampDelta = (TrimToSeconds(writeTimestamp) - BaseWriteTimestamp).Seconds() + .WriteTimestampDelta = static_cast((TrimToSeconds(writeTimestamp) - BaseWriteTimestamp).Seconds()) }); Batch.AddNewMessage(offset); @@ -375,7 +379,9 @@ ui64 TStorage::DoLock(ui64 offset, TMessage& message, TInstant& deadline) { AFL_VERIFY(message.Status == EMessageStatus::Unprocessed)("status", message.Status); message.Status = EMessageStatus::Locked; message.DeadlineDelta = NormalizeDeadline(deadline); - ++message.ReceiveCount; + if (message.ProcessingCount < MAX_PROCESSING_COUNT) { + ++message.ProcessingCount; + } Batch.AddChange(offset); @@ -470,20 +476,30 @@ void TStorage::DoUnlock( ui64 offset, TMessage& message) { --Metrics.LockedMessageCount; - if (message.ReceiveCount >= MaxMessageProcessingCount) { - // TODO Move to DLQ or remove message - message.Status = EMessageStatus::DLQ; - DLQQueue.push_back(offset); - Batch.AddDLQ(offset); - - ++Metrics.DLQMessageCount; - } else { - if (offset >= FirstOffset) { - FirstUnlockedOffset = std::min(FirstUnlockedOffset, offset); + if (message.ProcessingCount >= MaxMessageProcessingCount) { + if (DeadLetterPolicy) { + switch (DeadLetterPolicy.value()) { + case NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_MOVE: + message.Status = EMessageStatus::DLQ; + DLQQueue.push_back(offset); + Batch.AddDLQ(offset); + + ++Metrics.DLQMessageCount; + return; + case NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_DELETE: + DoCommit(offset); + return; + case NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_UNSPECIFIED: + break; + } } - - ++Metrics.UnprocessedMessageCount; } + + if (offset >= FirstOffset) { + FirstUnlockedOffset = std::min(FirstUnlockedOffset, offset); + } + + ++Metrics.UnprocessedMessageCount; } void TStorage::MoveBaseDeadline() { @@ -704,7 +720,7 @@ TStorage::TMessageWrapper TStorage::TMessageIterator::operator*() const { .SlowZone = Iterator != Storage.SlowMessages.end(), .Offset = offset, .Status = static_cast(message->Status), - .ProcessingCount = message->ReceiveCount, + .ProcessingCount = message->ProcessingCount, .ProcessingDeadline = static_cast(message->Status) == EMessageStatus::Locked ? Storage.BaseDeadline + TDuration::Seconds(message->DeadlineDelta) : TInstant::Zero(), .WriteTimestamp = Storage.BaseWriteTimestamp + TDuration::Seconds(message->WriteTimestampDelta), diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.h index d0422b70488b..48962becd329 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.h @@ -2,21 +2,22 @@ #include "mlp.h" +#include + #include #include #include #include -#include #include namespace NKikimr::NPQ::NMLP { -// TODO MLP Slow zone class TStorage { static constexpr size_t MAX_MESSAGES = 48000; static constexpr size_t MIN_MESSAGES = 100; + static constexpr size_t MAX_PROCESSING_COUNT = 1023; public: // The maximum number of messages per flight. If a larger number is required, then you need @@ -46,22 +47,23 @@ class TStorage { }; struct TMessage { - ui64 Status: 3 = EMessageStatus::Unprocessed; - ui64 Reserve: 3; + ui32 Status: 3 = EMessageStatus::Unprocessed; + ui32 Reserve: 3; // It stores how many times the message was submitted to work. // If the value is large, then the message has been processed several times, // but it has never been processed successfully. - ui64 ReceiveCount: 10 = 0; + ui32 ProcessingCount: 10 = 0; // For locked messages, the time after which the message should be returned to the queue by timeout. - ui64 DeadlineDelta: 16 = 0; - ui64 HasMessageGroupId: 1 = false; + ui32 DeadlineDelta: 16 = 0; + ui32 HasMessageGroupId: 1 = false; // Hash of the message group. For consumers who keep the order of messages, it is guaranteed that // messages within the same group are processed sequentially in the order in which they were recorded // in the topic. - ui64 MessageGroupIdHash: 31 = 0; - ui64 WriteTimestampDelta: 26 = 0; + ui32 MessageGroupIdHash: 31 = 0; + ui32 WriteTimestampDelta: 26 = 0; + ui32 Reserve2: 6; }; - static_assert(sizeof(TMessage) == sizeof(ui64) * 2); + static_assert(sizeof(TMessage) == sizeof(ui32) * 3); struct TMessageWrapper { bool SlowZone; @@ -140,6 +142,7 @@ class TStorage { void SetKeepMessageOrder(bool keepMessageOrder); void SetMaxMessageProcessingCount(ui32 MaxMessageProcessingCount); void SetRetentionPeriod(std::optional retentionPeriod); + void SetDeadLetterPolicy(std::optional deadLetterPolicy); ui64 GetFirstOffset() const; size_t GetMessageCount() const; @@ -162,10 +165,6 @@ class TStorage { // deadline - time for processing visibility // fromOffset indicates from which offset it is necessary to continue searching for the next free message. // it is an optimization for the case when the method is called several times in a row. - struct NextResult { - ui64 Message; - TPosition Position; - }; std::optional Next(TInstant deadline, TPosition& position); bool Commit(ui64 message); bool Unlock(ui64 message); @@ -175,7 +174,6 @@ class TStorage { bool AddMessage(ui64 offset, bool hasMessagegroup, ui32 messageGroupIdHash, TInstant writeTimestamp); size_t ProccessDeadlines(); - // TODO MLP удалять сообщения если в партиции сместился StartOffset size_t Compact(); void MoveBaseDeadline(); @@ -216,6 +214,7 @@ class TStorage { bool KeepMessageOrder = false; ui32 MaxMessageProcessingCount = 1000; std::optional RetentionPeriod = TDuration::Days(365); + std::optional DeadLetterPolicy; // Offset of the first message loaded for processing. All messages with a smaller offset // have either already been committed or deleted from the partition. diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage__serialization.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage__serialization.cpp index fd9a34feca6b..ad9df6483cc5 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage__serialization.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage__serialization.cpp @@ -14,7 +14,7 @@ struct TSnapshotMessage { struct { ui64 Status: 3; ui64 Reserve: 3; - ui64 ReceiveCount: 10; + ui64 ProcessingCount: 10; ui64 DeadlineDelta: 16; ui64 HasMessageGroupId: 1; ui64 MessageGroupIdHash: 31; @@ -46,7 +46,7 @@ struct TMessageChange { struct { ui32 Status: 3; ui32 Reserve: 3; - ui32 ReceiveCount: 10; + ui32 ProcessingCount: 10; ui32 DeadlineDelta: 16; } Fields; ui32 Value; @@ -362,7 +362,7 @@ bool TStorage::ApplyWAL(const NKikimrPQ::TMLPStorageWAL& wal) { auto& msg = it->second; SlowMessages[offset] = TMessage{ .Status = EMessageStatus::Unprocessed, - .ReceiveCount = 0, + .ProcessingCount = 0, .DeadlineDelta = 0, .HasMessageGroupId = msg.MessageGroup.Fields.HasMessageGroupId, .MessageGroupIdHash = msg.MessageGroup.Fields.MessageGroupIdHash, @@ -394,7 +394,7 @@ bool TStorage::ApplyWAL(const NKikimrPQ::TMLPStorageWAL& wal) { if (offset >= GetLastOffset()) { Messages.push_back({ .Status = EMessageStatus::Unprocessed, - .ReceiveCount = 0, + .ProcessingCount = 0, .DeadlineDelta = 0, .HasMessageGroupId = msg.MessageGroup.Fields.HasMessageGroupId, .MessageGroupIdHash = msg.MessageGroup.Fields.MessageGroupIdHash, @@ -426,7 +426,7 @@ bool TStorage::ApplyWAL(const NKikimrPQ::TMLPStorageWAL& wal) { message->Status = msg.Common.Fields.Status; message->DeadlineDelta = msg.Common.Fields.DeadlineDelta; - message->ReceiveCount = msg.Common.Fields.ReceiveCount; + message->ProcessingCount = msg.Common.Fields.ProcessingCount; if (statusChanged) { switch(message->Status) { @@ -566,7 +566,7 @@ bool TStorage::TBatch::SerializeTo(NKikimrPQ::TMLPStorageWAL& wal) { if (message) { TMessageChange msg; msg.Common.Fields.Status = message->Status; - msg.Common.Fields.ReceiveCount = message->ReceiveCount; + msg.Common.Fields.ProcessingCount = message->ProcessingCount; msg.Common.Fields.DeadlineDelta = message->DeadlineDelta; serializer.Add(offset, msg); } diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage_ut.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage_ut.cpp index 5a60ad72ccf1..ad37df2ffef7 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage_ut.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage_ut.cpp @@ -34,6 +34,7 @@ struct TUtils { Storage.SetKeepMessageOrder(true); Storage.SetMaxMessageProcessingCount(1); Storage.SetRetentionPeriod(TDuration::Seconds(10)); + Storage.SetDeadLetterPolicy(NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_MOVE); } TIntrusivePtr TimeProvider; @@ -1119,6 +1120,8 @@ Y_UNIT_TEST(StorageSerialization_WAL_DLQ) { TStorage storage(timeProvider); storage.SetKeepMessageOrder(true); storage.SetMaxMessageProcessingCount(1); + storage.SetDeadLetterPolicy(NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_MOVE); + storage.SerializeTo(snapshot); storage.AddMessage(3, true, 5, writeTimestamp); @@ -1158,6 +1161,7 @@ Y_UNIT_TEST(StorageSerialization_WAL_DLQ) { { TStorage storage(timeProvider); storage.SetKeepMessageOrder(true); + storage.SetDeadLetterPolicy(NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_MOVE); storage.Initialize(snapshot); storage.ApplyWAL(wal); @@ -1450,6 +1454,8 @@ Y_UNIT_TEST(CompactStorage_ByRetention) { Y_UNIT_TEST(CompactStorage_WithDLQ) { TStorage storage(CreateDefaultTimeProvider()); storage.SetMaxMessageProcessingCount(1); + storage.SetDeadLetterPolicy(NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_MOVE); + storage.AddMessage(3, true, 5, TInstant::Now()); storage.AddMessage(4, true, 7, TInstant::Now()); @@ -1519,12 +1525,12 @@ Y_UNIT_TEST(ProccessDeadlines) { { auto [message, _] = storage.GetMessage(3); UNIT_ASSERT_VALUES_EQUAL(message->Status, TStorage::EMessageStatus::Unprocessed); - UNIT_ASSERT_VALUES_EQUAL(message->ReceiveCount, 1); + UNIT_ASSERT_VALUES_EQUAL(message->ProcessingCount, 1); } { auto [message, _] = storage.GetMessage(4); UNIT_ASSERT_VALUES_EQUAL(message->Status, TStorage::EMessageStatus::Locked); - UNIT_ASSERT_VALUES_EQUAL(message->ReceiveCount, 1); + UNIT_ASSERT_VALUES_EQUAL(message->ProcessingCount, 1); } auto& metrics = storage.GetMetrics(); diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/ya.make b/ydb/core/persqueue/pqtablet/partition/mlp/ya.make index 9f31b690a78a..0ec37722dda3 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/ya.make +++ b/ydb/core/persqueue/pqtablet/partition/mlp/ya.make @@ -3,6 +3,7 @@ LIBRARY() SRCS( mlp_common.cpp mlp_consumer.cpp + mlp_dlq_mover.cpp mlp_message_enricher.cpp mlp_storage.cpp mlp_storage__serialization.cpp @@ -18,7 +19,6 @@ PEERDIR( ydb/core/persqueue/public/write_meta ) - GENERATE_ENUM_SERIALIZATION(mlp_storage.h) END() diff --git a/ydb/core/persqueue/pqtablet/partition/partition_mlp.cpp b/ydb/core/persqueue/pqtablet/partition/partition_mlp.cpp index b68d84f436ff..3896b4ca38ce 100644 --- a/ydb/core/persqueue/pqtablet/partition/partition_mlp.cpp +++ b/ydb/core/persqueue/pqtablet/partition/partition_mlp.cpp @@ -127,6 +127,7 @@ void TPartition::InitializeMLPConsumers() { LOG_I("Creating MLP consumer '" << name << "'"); auto actorId = RegisterWithSameMailbox(NMLP::CreateConsumerActor( + DbPath, TabletId, TabletActorId, Partition.OriginalPartitionId, From b640782a8546fdbe7089d012a71160df9b6e1f26 Mon Sep 17 00:00:00 2001 From: Nikolay Shestakov Date: Sun, 9 Nov 2025 17:50:12 +0000 Subject: [PATCH 02/19] more tests --- .../pqtablet/partition/mlp/mlp_storage.cpp | 24 +++- .../pqtablet/partition/mlp/mlp_storage_ut.cpp | 122 ++++++++---------- 2 files changed, 74 insertions(+), 72 deletions(-) diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp index 38697c72462e..b7813e0a9b03 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp @@ -212,21 +212,27 @@ size_t TStorage::Compact() { } void TStorage::RemoveMessage(const TMessage& message) { + AFL_ENSURE(Metrics.InflyMessageCount > 0); --Metrics.InflyMessageCount; switch(message.Status) { case EMessageStatus::Unprocessed: + AFL_ENSURE(Metrics.UnprocessedMessageCount > 0); --Metrics.UnprocessedMessageCount; break; case EMessageStatus::Locked: + AFL_ENSURE(Metrics.LockedMessageCount > 0); --Metrics.LockedMessageCount; if (KeepMessageOrder && message.HasMessageGroupId && LockedMessageGroupsId.erase(message.MessageGroupIdHash)) { + AFL_ENSURE(Metrics.LockedMessageGroupCount > 0); --Metrics.LockedMessageGroupCount; } break; case EMessageStatus::Committed: + AFL_ENSURE(Metrics.CommittedMessageCount > 0); --Metrics.CommittedMessageCount; break; case EMessageStatus::DLQ: + AFL_ENSURE(Metrics.DLQMessageCount > 0); --Metrics.DLQMessageCount; break; } @@ -392,6 +398,7 @@ ui64 TStorage::DoLock(ui64 offset, TMessage& message, TInstant& deadline) { } ++Metrics.LockedMessageCount; + AFL_ENSURE(Metrics.UnprocessedMessageCount > 0)("o", offset); --Metrics.UnprocessedMessageCount; return offset; @@ -409,6 +416,7 @@ bool TStorage::DoCommit(ui64 offset) { Batch.AddChange(offset); ++Metrics.CommittedMessageCount; } + AFL_ENSURE(Metrics.UnprocessedMessageCount > 0)("o", offset); --Metrics.UnprocessedMessageCount; break; case EMessageStatus::Locked: @@ -417,9 +425,11 @@ bool TStorage::DoCommit(ui64 offset) { ++Metrics.CommittedMessageCount; } + AFL_ENSURE(-Metrics.LockedMessageCount > 0)("o", offset); --Metrics.LockedMessageCount; if (KeepMessageOrder && message->HasMessageGroupId) { if (LockedMessageGroupsId.erase(message->MessageGroupIdHash)) { + AFL_ENSURE(Metrics.LockedMessageGroupCount > 0)("o", offset); --Metrics.LockedMessageGroupCount; } } @@ -433,6 +443,7 @@ bool TStorage::DoCommit(ui64 offset) { ++Metrics.CommittedMessageCount; } + AFL_ENSURE(Metrics.DLQMessageCount > 0)("o", offset); --Metrics.DLQMessageCount; break; } @@ -440,6 +451,7 @@ bool TStorage::DoCommit(ui64 offset) { if (slowZone) { SlowMessages.erase(offset); Batch.DeleteFromSlow(offset); + AFL_ENSURE(Metrics.InflyMessageCount > 0)("o", offset); --Metrics.InflyMessageCount; } else { message->Status = EMessageStatus::Committed; @@ -466,14 +478,18 @@ void TStorage::DoUnlock( ui64 offset, TMessage& message) { message.Status = EMessageStatus::Unprocessed; message.DeadlineDelta = 0; + Batch.AddChange(offset); + + ++Metrics.UnprocessedMessageCount; + if (KeepMessageOrder && message.HasMessageGroupId) { if (LockedMessageGroupsId.erase(message.MessageGroupIdHash)) { + AFL_ENSURE(Metrics.LockedMessageGroupCount > 0)("o", offset); --Metrics.LockedMessageGroupCount; } } - Batch.AddChange(offset); - + AFL_ENSURE(Metrics.LockedMessageCount > 0)("o", offset); --Metrics.LockedMessageCount; if (message.ProcessingCount >= MaxMessageProcessingCount) { @@ -484,6 +500,8 @@ void TStorage::DoUnlock( ui64 offset, TMessage& message) { DLQQueue.push_back(offset); Batch.AddDLQ(offset); + AFL_ENSURE(Metrics.UnprocessedMessageCount > 0)("o", offset); + --Metrics.UnprocessedMessageCount; ++Metrics.DLQMessageCount; return; case NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_DELETE: @@ -498,8 +516,6 @@ void TStorage::DoUnlock( ui64 offset, TMessage& message) { if (offset >= FirstOffset) { FirstUnlockedOffset = std::min(FirstUnlockedOffset, offset); } - - ++Metrics.UnprocessedMessageCount; } void TStorage::MoveBaseDeadline() { diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage_ut.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage_ut.cpp index ad37df2ffef7..f36d44d4d4a1 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage_ut.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage_ut.cpp @@ -144,7 +144,9 @@ struct TUtils { ui64 Next(TDuration timeout = TDuration::Seconds(8)) { TStorage::TPosition position; - return Storage.Next(TimeProvider->Now() + timeout, position).value(); + auto result = Storage.Next(TimeProvider->Now() + timeout, position); + UNIT_ASSERT(result); + return result.value(); } bool Commit(ui64 offset) { @@ -1109,89 +1111,73 @@ Y_UNIT_TEST(StorageSerialization_WAL_Committed) { } Y_UNIT_TEST(StorageSerialization_WAL_DLQ) { - auto timeProvider = TIntrusivePtr(new MockTimeProvider()); + TUtils utils; + auto writeTimestamp = utils.BaseWriteTimestamp + TDuration::Seconds(7); - auto writeTimestamp = timeProvider->Now() - TDuration::Seconds(13); + utils.Begin(); + utils.Storage.AddMessage(3, true, 5, writeTimestamp); - NKikimrPQ::TMLPStorageSnapshot snapshot; - NKikimrPQ::TMLPStorageWAL wal; + auto r = utils.Next(); + UNIT_ASSERT(r); + utils.Storage.Unlock(3); + utils.End(); + + auto it = utils.Storage.begin(); { - TStorage storage(timeProvider); - storage.SetKeepMessageOrder(true); - storage.SetMaxMessageProcessingCount(1); - storage.SetDeadLetterPolicy(NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_MOVE); - - storage.SerializeTo(snapshot); + UNIT_ASSERT(it != utils.Storage.end()); + auto message = *it; + UNIT_ASSERT_VALUES_EQUAL(message.Offset, 3); + UNIT_ASSERT_VALUES_EQUAL(message.Status, TStorage::EMessageStatus::DLQ); + UNIT_ASSERT_VALUES_EQUAL(message.ProcessingCount, 1); + UNIT_ASSERT_VALUES_EQUAL(message.ProcessingDeadline, TInstant::Zero()); + UNIT_ASSERT_VALUES_EQUAL(message.WriteTimestamp, writeTimestamp); + } + ++it; + UNIT_ASSERT(it == utils.Storage.end()); - storage.AddMessage(3, true, 5, writeTimestamp); + const auto& dlq = utils.Storage.GetDLQMessages(); + UNIT_ASSERT_VALUES_EQUAL(dlq.size(), 1); + UNIT_ASSERT_VALUES_EQUAL(dlq.front(), 3); - TStorage::TPosition position; - auto r = storage.Next(timeProvider->Now() + TDuration::Seconds(7), position); - UNIT_ASSERT(r); + utils.TimeProvider->Tick(TDuration::Seconds(5)); - storage.Unlock(3); + utils.AssertLoad(); +} - auto it = storage.begin(); - { - UNIT_ASSERT(it != storage.end()); - auto message = *it; - UNIT_ASSERT_VALUES_EQUAL(message.Offset, 3); - UNIT_ASSERT_VALUES_EQUAL(message.Status, TStorage::EMessageStatus::DLQ); - UNIT_ASSERT_VALUES_EQUAL(message.ProcessingCount, 1); - UNIT_ASSERT_VALUES_EQUAL(message.ProcessingDeadline, TInstant::Zero()); - UNIT_ASSERT_VALUES_EQUAL(message.WriteTimestamp, writeTimestamp); - } - ++it; - UNIT_ASSERT(it == storage.end()); +Y_UNIT_TEST(StorageSerialization_WAL_DeadLetterPolicy_Delete) { + TUtils utils; + utils.Storage.SetDeadLetterPolicy(NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_DELETE); + auto writeTimestamp = utils.BaseWriteTimestamp + TDuration::Seconds(7); - const auto& dlq = storage.GetDLQMessages(); - UNIT_ASSERT_VALUES_EQUAL(dlq.size(), 1); - UNIT_ASSERT_VALUES_EQUAL(dlq.front(), 3); + utils.Begin(); + utils.Storage.AddMessage(3, true, 5, writeTimestamp); - auto batch = storage.GetBatch(); - UNIT_ASSERT_VALUES_EQUAL(batch.AddedMessageCount(), 1); - UNIT_ASSERT_VALUES_EQUAL(batch.ChangedMessageCount(), 2); - UNIT_ASSERT_VALUES_EQUAL(batch.DLQMessageCount(), 1); - batch.SerializeTo(wal); - } + auto r = utils.Next(); + UNIT_ASSERT(r); + utils.Storage.Unlock(3); - timeProvider->Tick(TDuration::Seconds(5)); + utils.End(); + auto it = utils.Storage.begin(); { - TStorage storage(timeProvider); - storage.SetKeepMessageOrder(true); - storage.SetDeadLetterPolicy(NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_MOVE); + UNIT_ASSERT(it != utils.Storage.end()); + auto message = *it; + UNIT_ASSERT_VALUES_EQUAL(message.Offset, 3); + UNIT_ASSERT_VALUES_EQUAL(message.Status, TStorage::EMessageStatus::Committed); + UNIT_ASSERT_VALUES_EQUAL(message.ProcessingCount, 1); + UNIT_ASSERT_VALUES_EQUAL(message.ProcessingDeadline, TInstant::Zero()); + UNIT_ASSERT_VALUES_EQUAL(message.WriteTimestamp, writeTimestamp); + } + ++it; + UNIT_ASSERT(it == utils.Storage.end()); - storage.Initialize(snapshot); - storage.ApplyWAL(wal); + const auto& dlq = utils.Storage.GetDLQMessages(); + UNIT_ASSERT_VALUES_EQUAL(dlq.size(), 0); - auto it = storage.begin(); - { - UNIT_ASSERT(it != storage.end()); - auto message = *it; - UNIT_ASSERT_VALUES_EQUAL(message.Offset, 3); - UNIT_ASSERT_VALUES_EQUAL(message.Status, TStorage::EMessageStatus::DLQ); - UNIT_ASSERT_VALUES_EQUAL(message.ProcessingCount, 1); - UNIT_ASSERT_VALUES_EQUAL(message.ProcessingDeadline, TInstant::Zero()); - UNIT_ASSERT_VALUES_EQUAL(message.WriteTimestamp, writeTimestamp); - } - ++it; - UNIT_ASSERT(it == storage.end()); - - auto& metrics = storage.GetMetrics(); - UNIT_ASSERT_VALUES_EQUAL(metrics.InflyMessageCount, 1); - UNIT_ASSERT_VALUES_EQUAL(metrics.UnprocessedMessageCount, 0); - UNIT_ASSERT_VALUES_EQUAL(metrics.LockedMessageCount, 0); - UNIT_ASSERT_VALUES_EQUAL(metrics.LockedMessageGroupCount, 0); - UNIT_ASSERT_VALUES_EQUAL(metrics.CommittedMessageCount, 0); - UNIT_ASSERT_VALUES_EQUAL(metrics.DeadlineExpiredMessageCount, 0); - UNIT_ASSERT_VALUES_EQUAL(metrics.DLQMessageCount, 1); + utils.TimeProvider->Tick(TDuration::Seconds(5)); - const auto& dlq = storage.GetDLQMessages(); - UNIT_ASSERT_VALUES_EQUAL(dlq.size(), 1); - UNIT_ASSERT_VALUES_EQUAL(dlq.front(), 3); - } + utils.AssertLoad(); } Y_UNIT_TEST(StorageSerialization_WAL_WithHole) { From 92d75968eeba2ea1bcc1e284992defcbdcda00dc Mon Sep 17 00:00:00 2001 From: Nikolay Shestakov Date: Sun, 9 Nov 2025 18:51:09 +0000 Subject: [PATCH 03/19] WIP --- .../pqtablet/partition/mlp/mlp_storage.cpp | 29 +++++++++++++++++++ .../pqtablet/partition/mlp/mlp_storage.h | 2 ++ .../mlp/mlp_storage__serialization.cpp | 5 ++++ ydb/core/protos/pqconfig.proto | 1 + 4 files changed, 37 insertions(+) diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp index b7813e0a9b03..4e1e05bb7f50 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp @@ -40,6 +40,30 @@ void TStorage::SetRetentionPeriod(std::optional retentionPeriod) { void TStorage::SetDeadLetterPolicy(std::optional deadLetterPolicy) { DeadLetterPolicy = deadLetterPolicy; + + if (DeadLetterPolicy && DeadLetterPolicy.value() == NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_MOVE) { + return; + } + + auto policy = DeadLetterPolicy.value_or(NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_UNSPECIFIED); + while (!DLQQueue.empty()) { + auto offset = DLQQueue.front(); + DLQQueue.pop_front(); + + Batch.AddChange(offset); + + switch (policy) { + case NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_MOVE: + // unreachable + break; + case NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_DELETE: + Commit(offset); + break; + case NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_UNSPECIFIED: + Unlock(offset); + break; + } + } } std::optional TStorage::Next(TInstant deadline, TPosition& position) { @@ -657,6 +681,11 @@ void TStorage::TBatch::AddDLQ(ui64 offset) { DLQ.push_back(offset); } +void TStorage::TBatch::DeleteFromDLQ(ui64 offset) { + Y_UNUSED(offset); + ++DeletedFromDLQ; +} + void TStorage::TBatch::AddNewMessage(ui64 offset) { if (!FirstNewMessage) { FirstNewMessage = offset; diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.h index 48962becd329..b17233e3f10b 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.h @@ -106,6 +106,7 @@ class TStorage { void AddNewMessage(ui64 offset); void AddChange(ui64 offset); void AddDLQ(ui64 offset); + void DeleteFromDLQ(ui64 offset); void MoveToSlow(ui64 offset); void DeleteFromSlow(ui64 offset); @@ -119,6 +120,7 @@ class TStorage { std::optional FirstNewMessage; size_t NewMessageCount = 0; std::vector DLQ; + size_t DeletedFromDLQ = 0; std::vector MovedToSlowZone; std::vector DeletedFromSlowZone; size_t CompactedMessages = 0; diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage__serialization.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage__serialization.cpp index ad9df6483cc5..182de2a6db38 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage__serialization.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage__serialization.cpp @@ -474,6 +474,10 @@ bool TStorage::ApplyWAL(const NKikimrPQ::TMLPStorageWAL& wal) { } { + AFL_ENSURE(wal.GetDeletedFromDLQ() <= DLQQueue.size())("d", wal.GetDeletedFromDLQ())("q", DLQQueue.size()); + for (size_t i = 0; i < wal.GetDeletedFromDLQ(); ++i) { + DLQQueue.pop_front(); + } for (auto offset : wal.GetDLQ()) { DLQQueue.push_back(offset); } @@ -579,6 +583,7 @@ bool TStorage::TBatch::SerializeTo(NKikimrPQ::TMLPStorageWAL& wal) { for (auto offset : DLQ) { wal.AddDLQ(offset); } + wal.SetDeletedFromDLQ(DeletedFromDLQ); } { diff --git a/ydb/core/protos/pqconfig.proto b/ydb/core/protos/pqconfig.proto index ae1f543aca47..e3061348158a 100644 --- a/ydb/core/protos/pqconfig.proto +++ b/ydb/core/protos/pqconfig.proto @@ -1406,5 +1406,6 @@ message TMLPStorageWAL { optional uint64 SlowFirstOffset = 12; repeated uint64 MovedToSlowZone = 13; repeated uint64 DeletedFromSlowZone = 14; + optional uint64 DeletedFromDLQ = 15; } From 8ddb7654d0a3be00ee9d75ce86b61fabc9274a7b Mon Sep 17 00:00:00 2001 From: Nikolay Shestakov Date: Sun, 9 Nov 2025 19:13:59 +0000 Subject: [PATCH 04/19] more tests --- .../pqtablet/partition/mlp/mlp_storage.cpp | 13 ++++-- .../pqtablet/partition/mlp/mlp_storage_ut.cpp | 40 +++++++++++++++++++ 2 files changed, 50 insertions(+), 3 deletions(-) diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp index 4e1e05bb7f50..34f87e23413b 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp @@ -51,6 +51,7 @@ void TStorage::SetDeadLetterPolicy(std::optionalStatus = EMessageStatus::Unprocessed; + + --Metrics.DLQMessageCount; + ++Metrics.UnprocessedMessageCount; + break; + } } } } @@ -498,7 +505,7 @@ bool TStorage::DoUnlock(ui64 offset) { return true; } -void TStorage::DoUnlock( ui64 offset, TMessage& message) { +void TStorage::DoUnlock(ui64 offset, TMessage& message) { message.Status = EMessageStatus::Unprocessed; message.DeadlineDelta = 0; diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage_ut.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage_ut.cpp index f36d44d4d4a1..0b6a9d5869a3 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage_ut.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage_ut.cpp @@ -1827,6 +1827,46 @@ Y_UNIT_TEST(SlowZone_Retention_3message) { utils.AssertLoad(); } +Y_UNIT_TEST(ChangeDeadLettePolicy_Delete) { + TUtils utils; + utils.Storage.SetDeadLetterPolicy(NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_MOVE); + utils.Storage.SetMaxMessageProcessingCount(1); + + utils.AddMessage(1); + utils.Next(); + utils.Storage.Unlock(0); + UNIT_ASSERT_VALUES_EQUAL(utils.Storage.GetDLQMessages().size(), 1); + + utils.Begin(); + utils.Storage.SetDeadLetterPolicy(NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_DELETE); + UNIT_ASSERT_VALUES_EQUAL(utils.Storage.GetDLQMessages().size(), 0); + UNIT_ASSERT_VALUES_EQUAL(utils.Storage.GetMetrics().InflyMessageCount, 1); + UNIT_ASSERT_VALUES_EQUAL(utils.Storage.GetMetrics().CommittedMessageCount, 1); + utils.End(); + + utils.AssertLoad(); +} + +Y_UNIT_TEST(ChangeDeadLettePolicy_Unspecified) { + TUtils utils; + utils.Storage.SetDeadLetterPolicy(NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_MOVE); + utils.Storage.SetMaxMessageProcessingCount(1); + + utils.AddMessage(1); + utils.Next(); + utils.Storage.Unlock(0); + UNIT_ASSERT_VALUES_EQUAL(utils.Storage.GetDLQMessages().size(), 1); + + utils.Begin(); + utils.Storage.SetDeadLetterPolicy(NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_UNSPECIFIED); + UNIT_ASSERT_VALUES_EQUAL(utils.Storage.GetDLQMessages().size(), 0); + UNIT_ASSERT_VALUES_EQUAL(utils.Storage.GetMetrics().InflyMessageCount, 1); + UNIT_ASSERT_VALUES_EQUAL(utils.Storage.GetMetrics().UnprocessedMessageCount, 1); + utils.End(); + + utils.AssertLoad(); +} + } } // namespace NKikimr::NPQ::NMLP From c30c22e5e82622deb84423ac5a084d41adc9519e Mon Sep 17 00:00:00 2001 From: Nikolay Shestakov Date: Mon, 10 Nov 2025 05:23:06 +0000 Subject: [PATCH 05/19] WIP --- .../pqtablet/partition/mlp/mlp_common.h | 18 ++++++++----- .../pqtablet/partition/mlp/mlp_consumer.cpp | 26 +++++++++++++------ .../pqtablet/partition/mlp/mlp_consumer.h | 1 + .../pqtablet/partition/mlp/mlp_dlq_mover.cpp | 14 ++-------- .../pqtablet/partition/mlp/mlp_dlq_mover.h | 2 ++ 5 files changed, 35 insertions(+), 26 deletions(-) diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h index 8b18c4b426ee..c1bcc3024bff 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h @@ -49,11 +49,17 @@ NActors::IActor* CreateMessageEnricher(const NActors::TActorId& tabletActorId, const TString& consumerName, std::deque&& replies); -NActors::IActor* CreateDLQMover(const TString& database, - const ui64 tabletId, - const ui32 partitionId, - const TString& consumerName, - const TString& destinationTopic, - std::deque&& offsets); +struct TDLQMoverSettings { + TString Database; + ui64 TabletId; + ui32 PartitionId; + TString ConsumerName; + ui64 ConsumerGeneration; + TString DestinationTopic; + ui64 FirstMessageSeqNo; + std::deque Messages; +}; + +NActors::IActor* CreateDLQMover(TDLQMoverSettings&& settings); } // namespace NKikimr::NPQ::NMLP diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp index 7bca7698e6ed..f5842c59a7a6 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp @@ -122,6 +122,10 @@ void TConsumerActor::PassAway() { ReplyErrorAll(SelfId(), UnlockRequestsQueue); ReplyErrorAll(SelfId(), ChangeMessageDeadlineRequestsQueue); + if (DLQMoverActorId) { + Send(DLQMoverActorId, new TEvents::TEvPoison()); + } + TBase::PassAway(); } @@ -202,6 +206,7 @@ void TConsumerActor::HandleOnInit(TEvKeyValue::TEvResponse::TPtr& ev) { LOG_D("Read snapshot"); HasSnapshot = true; LastWALIndex = snapshot.GetWALIndex(); + DLQMovedMessageCount = snapshot.GetMeta().GetDLQMovedMessages(); Storage->Initialize(snapshot); } else { LOG_W("Received snapshot from old consumer generation: " << Config.GetGeneration() << " vs " << snapshot.GetConfiguration().GetGeneration()); @@ -238,6 +243,7 @@ void TConsumerActor::HandleOnInit(TEvKeyValue::TEvResponse::TPtr& ev) { if (Config.GetGeneration() == wal.GetGeneration()) { LOG_D("Read WAL " << w.key()); LastWALIndex = wal.GetWALIndex(); + DLQMovedMessageCount = wal.GetDLQMovedMessages(); Storage->ApplyWAL(wal); } else { LOG_W("Received snapshot from old consumer generation: " << Config.GetGeneration() << " vs " << wal.GetGeneration()); @@ -535,6 +541,7 @@ void TConsumerActor::Persist() { NKikimrPQ::TMLPStorageWAL wal; wal.SetWALIndex(LastWALIndex); + wal.SetDLQMovedMessages(DLQMovedMessageCount); batch.SerializeTo(wal); auto data = wal.SerializeAsString(); @@ -561,6 +568,7 @@ void TConsumerActor::Persist() { Storage->SerializeTo(snapshot); snapshot.SetWALIndex(LastWALIndex); + snapshot.MutableMeta()->SetDLQMovedMessages(DLQMovedMessageCount); auto request = std::make_unique(); @@ -680,14 +688,16 @@ void TConsumerActor::HandleOnWork(TEvents::TEvWakeup::TPtr&) { if (!DLQMoverActorId && !Storage->GetDLQMessages().empty()) { std::deque messages; - DLQMoverActorId = RegisterWithSameMailbox(CreateDLQMover( - Database, - TabletId, - PartitionId, - Config.GetName(), - Config.GetDeadLetterQueue(), - std::move(messages) - )); + DLQMoverActorId = RegisterWithSameMailbox(CreateDLQMover({ + .Database = Database, + .TabletId = TabletId, + .PartitionId = PartitionId, + .ConsumerName = Config.GetName(), + .ConsumerGeneration = Config.GetGeneration(), + .DestinationTopic = Config.GetDeadLetterQueue(), + .FirstMessageSeqNo = DLQMovedMessageCount, + .Messages = std::move(messages) + })); } Schedule(WakeupInterval, new TEvents::TEvWakeup()); } diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h index 5cc027978ecf..b420cdd31bde 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h @@ -79,6 +79,7 @@ class TConsumerActor : public TBaseTabletActor ui64 FetchCookie = 0; ui64 LastCommittedOffset = 0; + ui64 DLQMovedMessageCount = 0; TActorId DLQMoverActorId; std::unique_ptr Storage; diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp index fdbaccaf4863..afbe66d9b8de 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp @@ -2,18 +2,8 @@ namespace NKikimr::NPQ::NMLP { -NActors::IActor* CreateDLQMover(const TString& database, - const ui64 tabletId, - const ui32 partitionId, - const TString& consumerName, - const TString& destinationTopic, - std::deque&& offsets) { - Y_UNUSED(database); - Y_UNUSED(tabletId); - Y_UNUSED(partitionId); - Y_UNUSED(consumerName); - Y_UNUSED(destinationTopic); - Y_UNUSED(offsets); +NActors::IActor* CreateDLQMover(TDLQMoverSettings&& settings) { + Y_UNUSED(settings); return nullptr; } diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h index 652447a65a31..6f005bbef284 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h @@ -41,7 +41,9 @@ class TDLQMoverActor : public TBaseActor const ui64 TabletId; const ui32 PartitionId; const TString ConsumerName; + const ui64 ConsumerGeneration; const TString DestinationTopic; + ui64 FirstMessageSeqNo; std::deque Queue; }; From a6115c3dea948d7507d1f196af2acd55c1273860 Mon Sep 17 00:00:00 2001 From: Nikolay Shestakov Date: Mon, 10 Nov 2025 07:46:19 +0000 Subject: [PATCH 06/19] WIP --- ydb/core/persqueue/events/internal.h | 13 +++++++ .../pqtablet/partition/mlp/mlp_dlq_mover.cpp | 36 +++++++++++++++++-- .../pqtablet/partition/mlp/mlp_dlq_mover.h | 25 +++++++------ ydb/library/services/services.proto | 1 + 4 files changed, 59 insertions(+), 16 deletions(-) diff --git a/ydb/core/persqueue/events/internal.h b/ydb/core/persqueue/events/internal.h index 809bb7952d3d..7c009eaf3b58 100644 --- a/ydb/core/persqueue/events/internal.h +++ b/ydb/core/persqueue/events/internal.h @@ -213,6 +213,7 @@ struct TEvPQ { EvGetMLPConsumerStateRequest, EvGetMLPConsumerStateResponse, EvMLPConsumerUpdateConfig, + EvMLPDLQMoverResponse, EvEnd }; @@ -1593,6 +1594,18 @@ struct TEvPQ { NKikimrPQ::TPQTabletConfig::TConsumer Config; std::optional RetentionPeriod; }; + + struct TEvMLPDLQMoverResponse : TEventLocal { + + TEvMLPDLQMoverResponse(Ydb::StatusIds::StatusCode status, std::vector&& movedMessages) + : Status(status) + , MovedMessages(std::move(movedMessages)) + { + } + + Ydb::StatusIds::StatusCode Status; + std::vector MovedMessages; + }; }; } //NKikimr diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp index afbe66d9b8de..e0fd645ea45e 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp @@ -2,10 +2,40 @@ namespace NKikimr::NPQ::NMLP { -NActors::IActor* CreateDLQMover(TDLQMoverSettings&& settings) { - Y_UNUSED(settings); +TDLQMoverActor::TDLQMoverActor(TDLQMoverSettings&& settings) + : TBaseActor(NKikimrServices::EServiceKikimr::PQ_MLP_DLQ_MOVER) + , Settings(std::move(settings)) + , Queue(Settings.Messages) +{ +} + +void TDLQMoverActor::Bootstrap() { + Become(&TDLQMoverActor::StateDescribe); + RegisterWithSameMailbox(NDescriber::CreateDescriberActor(SelfId(), Settings.Database, { Settings.DestinationTopic })); +} + +void TDLQMoverActor::Handle(NDescriber::TEvDescribeTopicsResponse::TPtr& ev) { + LOG_D("Handle NDescriber::TEvDescribeTopicsResponse"); + + auto& topics = ev->Get()->Topics; + if (topics.size() != 1) { + return ReplyError("Unexpected describe result"); + } - return nullptr; + auto& topic = topics[Settings.DestinationTopic]; + + switch (topic.Status) { + case NDescriber::EStatus::SUCCESS: + TopicInfo = std::move(topic); + return CreateWriter(); + + default: + return ReplyError(NDescriber::Description(Settings.DestinationTopic, topic.Status)); + } +} + +NActors::IActor* CreateDLQMover(TDLQMoverSettings&& settings) { + return new TDLQMoverActor(std::move(settings)); } } diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h index 6f005bbef284..c0aaf9f10576 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h @@ -5,6 +5,7 @@ #include #include +#include #include #include @@ -16,17 +17,17 @@ class TDLQMoverActor : public TBaseActor static constexpr TDuration Timeout = TDuration::Seconds(5); public: - TDLQMoverActor(const TString& database, - const ui64 tabletId, - const ui32 partitionId, - const TString& consumerName, - const TString& destinationTopic, - std::deque&& offsets); + TDLQMoverActor(TDLQMoverSettings&& settings); void Bootstrap(); void PassAway() override; private: + void Handle(NDescriber::TEvDescribeTopicsResponse::TPtr&); + STFUNC(StateDescribe); + + void CreateWriter(); + void Handle(TEvPersQueue::TEvResponse::TPtr&); void Handle(TEvPQ::TEvError::TPtr&); void Handle(TEvents::TEvWakeup::TPtr&); @@ -36,15 +37,13 @@ class TDLQMoverActor : public TBaseActor void ProcessQueue(); + void ReplyError(TString&& error); + private: - const TString Database; - const ui64 TabletId; - const ui32 PartitionId; - const TString ConsumerName; - const ui64 ConsumerGeneration; - const TString DestinationTopic; - ui64 FirstMessageSeqNo; + TDLQMoverSettings Settings; std::deque Queue; + + NDescriber::TTopicInfo TopicInfo; }; } // namespace NKikimr::NPQ::NMLP diff --git a/ydb/library/services/services.proto b/ydb/library/services/services.proto index 111a3c1feb8b..528c39025569 100644 --- a/ydb/library/services/services.proto +++ b/ydb/library/services/services.proto @@ -197,6 +197,7 @@ enum EServiceKikimr { PQ_MLP_COMMITTER = 457; PQ_MLP_UNLOCKER = 458; PQ_MLP_DEADLINER = 459; + PQ_MLP_DLQ_MOVER = 464; CHOOSE_PROXY = 444; LB_CONFIG_MANAGER = 445; PQ_PARTITION_CHOOSER = 462; From eec1ad2b98d91c676e83b48da638939c6a5f4b04 Mon Sep 17 00:00:00 2001 From: Nikolay Shestakov Date: Mon, 10 Nov 2025 12:46:05 +0000 Subject: [PATCH 07/19] compiled --- ydb/core/persqueue/events/internal.h | 4 +- .../pqtablet/partition/mlp/mlp_common.h | 1 + .../pqtablet/partition/mlp/mlp_consumer.cpp | 17 ++ .../pqtablet/partition/mlp/mlp_consumer.h | 2 + .../pqtablet/partition/mlp/mlp_dlq_mover.cpp | 225 ++++++++++++++++++ .../pqtablet/partition/mlp/mlp_dlq_mover.h | 37 ++- .../pqtablet/partition/mlp/mlp_storage.cpp | 14 ++ .../pqtablet/partition/mlp/mlp_storage.h | 1 + 8 files changed, 292 insertions(+), 9 deletions(-) diff --git a/ydb/core/persqueue/events/internal.h b/ydb/core/persqueue/events/internal.h index 7c009eaf3b58..6d77c0442374 100644 --- a/ydb/core/persqueue/events/internal.h +++ b/ydb/core/persqueue/events/internal.h @@ -1597,14 +1597,16 @@ struct TEvPQ { struct TEvMLPDLQMoverResponse : TEventLocal { - TEvMLPDLQMoverResponse(Ydb::StatusIds::StatusCode status, std::vector&& movedMessages) + TEvMLPDLQMoverResponse(Ydb::StatusIds::StatusCode status, std::vector&& movedMessages, TString&& errorDescription = "") : Status(status) , MovedMessages(std::move(movedMessages)) + , ErrorDescription(std::move(errorDescription)) { } Ydb::StatusIds::StatusCode Status; std::vector MovedMessages; + TString ErrorDescription; }; }; diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h index c1bcc3024bff..503e24f16bb3 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h @@ -50,6 +50,7 @@ NActors::IActor* CreateMessageEnricher(const NActors::TActorId& tabletActorId, std::deque&& replies); struct TDLQMoverSettings { + TActorId ParentActorId; TString Database; ui64 TabletId; ui32 PartitionId; diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp index d02d8cc6664c..ba7d0a929d8f 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp @@ -407,6 +407,7 @@ STFUNC(TConsumerActor::StateWork) { hFunc(TEvKeyValue::TEvResponse, Handle); hFunc(TEvPQ::TEvProxyResponse, Handle); hFunc(TEvPQ::TEvError, Handle); + hFunc(TEvPQ::TEvMLPDLQMoverResponse, Handle); hFunc(TEvents::TEvWakeup, HandleOnWork); sFunc(TEvents::TEvPoison, PassAway); default: @@ -426,6 +427,7 @@ STFUNC(TConsumerActor::StateWrite) { hFunc(TEvKeyValue::TEvResponse, Handle); hFunc(TEvPQ::TEvProxyResponse, Handle); hFunc(TEvPQ::TEvError, Handle); + hFunc(TEvPQ::TEvMLPDLQMoverResponse, Handle); hFunc(TEvents::TEvWakeup, Handle); sFunc(TEvents::TEvPoison, PassAway); default: @@ -705,6 +707,21 @@ void TConsumerActor::HandleOnWork(TEvents::TEvWakeup::TPtr&) { Schedule(WakeupInterval, new TEvents::TEvWakeup()); } +void TConsumerActor::Handle(TEvPQ::TEvMLPDLQMoverResponse::TPtr& ev) { + LOG_D("Handle TEvPQ::TEvMLPDLQMoverResponse"); + + if (ev->Get()->Status != Ydb::StatusIds::SUCCESS) { + LOG_W("Error moving messages to the DLQ queue: " << ev->Get()->ErrorDescription); + } + + DLQMoverActorId = {}; + for (auto offset : ev->Get()->MovedMessages) { + AFL_VERIFY(Storage->MarkDLQMoved(offset))("o", offset); + } + + DLQMovedMessageCount += ev->Get()->MovedMessages.size(); +} + void TConsumerActor::Handle(TEvents::TEvWakeup::TPtr&) { LOG_D("Handle TEvents::TEvWakeup"); Schedule(WakeupInterval, new TEvents::TEvWakeup()); diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h index b420cdd31bde..9118245c1867 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h @@ -54,6 +54,8 @@ class TConsumerActor : public TBaseTabletActor void HandleOnWork(TEvents::TEvWakeup::TPtr&); void Handle(TEvents::TEvWakeup::TPtr&); + void Handle(TEvPQ::TEvMLPDLQMoverResponse::TPtr&); + STFUNC(StateInit); STFUNC(StateWork); STFUNC(StateWrite); diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp index e0fd645ea45e..8ab3161bdb0e 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp @@ -2,6 +2,12 @@ namespace NKikimr::NPQ::NMLP { +namespace { + +static constexpr ui64 CacheSubscribeCookie = 1; + +} + TDLQMoverActor::TDLQMoverActor(TDLQMoverSettings&& settings) : TBaseActor(NKikimrServices::EServiceKikimr::PQ_MLP_DLQ_MOVER) , Settings(std::move(settings)) @@ -14,6 +20,24 @@ void TDLQMoverActor::Bootstrap() { RegisterWithSameMailbox(NDescriber::CreateDescriberActor(SelfId(), Settings.Database, { Settings.DestinationTopic })); } +void TDLQMoverActor::PassAway() { + if (PartitionWriterActorId) { + Send(PartitionWriterActorId, new TEvents::TEvPoison()); + } + + if (Error) { + Send(Settings.ParentActorId, new TEvPQ::TEvMLPDLQMoverResponse(Ydb::StatusIds::INTERNAL_ERROR, std::move(Processed), std::move(Error))); + } else { + Send(Settings.ParentActorId, new TEvPQ::TEvMLPDLQMoverResponse(Ydb::StatusIds::SUCCESS, std::move(Processed))); + } + + TActor::PassAway(); +} + +TString TDLQMoverActor::BuildLogPrefix() const { + return TStringBuilder() << "[" << Settings.TabletId<< "][" << Settings.PartitionId << "][DLQ][" << Settings.ConsumerName << "] "; +} + void TDLQMoverActor::Handle(NDescriber::TEvDescribeTopicsResponse::TPtr& ev) { LOG_D("Handle NDescriber::TEvDescribeTopicsResponse"); @@ -34,6 +58,207 @@ void TDLQMoverActor::Handle(NDescriber::TEvDescribeTopicsResponse::TPtr& ev) { } } +void TDLQMoverActor::CreateWriter() { + Become(&TDLQMoverActor::StateInit); + + ProducerId = TStringBuilder() << "DLQMover/" << Settings.TabletId << "/" << Settings.PartitionId << "/" << Settings.ConsumerGeneration << "/" << Settings.ConsumerName; + TString sessionId = TStringBuilder() << "DLQMover/" << SelfId(); + + auto& chooser = TopicInfo.Info->PartitionChooser; + TargetPartition = chooser->GetPartition(ProducerId); + AFL_ENSURE(TargetPartition)("p", ProducerId); + + TPartitionWriterOpts opts; // TODO request units + opts.WithDeduplication(true) + .WithSourceId(ProducerId) + .WithAutoRegister(true) + .WithDatabase(Settings.Database) + .WithTopicPath(TopicInfo.RealPath) + .WithSessionId(sessionId) + .WithInitialSeqNo(Settings.FirstMessageSeqNo); + + PartitionWriterActorId = RegisterWithSameMailbox(CreatePartitionWriter(SelfId(), TargetPartition->TabletId, TargetPartition->PartitionId, opts)); +} + +void TDLQMoverActor::Handle(TEvPartitionWriter::TEvInitResult::TPtr& ev) { + const auto* result = ev->Get(); + + if (!result->IsSuccess()) { + return ReplyError(TStringBuilder() << "The error of creating a writer: " << result->GetError().Reason); + } + + ui64 targetSeqNo = result->GetResult().SourceIdInfo.GetSeqNo(); + for (SeqNo = Settings.FirstMessageSeqNo; SeqNo < targetSeqNo && !Queue.empty(); ++SeqNo) { + Processed.push_back(Queue.front()); + Queue.pop_front(); + } + + AFL_ENSURE(targetSeqNo == SeqNo)("t", targetSeqNo)("s", SeqNo); + + ProcessQueue(); +} + +void TDLQMoverActor::Handle(TEvPartitionWriter::TEvDisconnected::TPtr&) { + ReplyError("The writer disconnected"); +} + +void TDLQMoverActor::ProcessQueue() { + Become(&TDLQMoverActor::StateRead); + + if (Queue.empty()) { + return ReplySuccess(); + } + + SendToTablet(MakeEvRead(SelfId(), "", Queue.front(), 1, ++FetchCookie, NextPartNo)); +} + +void TDLQMoverActor::Handle(TEvPQ::TEvProxyResponse::TPtr& ev) { + LOG_D("Handle TEvPQ::TEvProxyResponse"); + if (FetchCookie != GetCookie(ev)) { + // TODO MLP + LOG_D("Cookie mismatch: " << FetchCookie << " != " << GetCookie(ev)); + //return; + } + + if (!IsSucess(ev)) { + ReplyError(TStringBuilder() << "Fetch message failed: " << ev->Get()->Response->DebugString()); + return; + } + + auto& response = ev->Get()->Response; + AFL_ENSURE(response->GetPartitionResponse().HasCmdReadResult())("t", Settings.TabletId)("p", Settings.PartitionId)("c", Settings.ConsumerName); + + auto write = std::make_unique(++WriteCookie); + auto* request = write->Record.MutablePartitionRequest(); + request->SetTopic(Settings.DestinationTopic); + request->SetPartition(TargetPartition->PartitionId); + + auto writeTimeMs = TInstant::Now().MilliSeconds(); + + auto currentOffset = Queue.front(); + for (auto& result : *response->MutablePartitionResponse()->MutableCmdReadResult()->MutableResult()) { + if (currentOffset > result.GetOffset()) { + continue; + } + AFL_ENSURE(currentOffset == result.GetOffset())("l", currentOffset)("r", result.GetOffset()); + + if (NextPartNo > result.GetPartNo()) { + continue; + } + AFL_ENSURE(NextPartNo == result.GetPartNo())("l", NextPartNo)("r", result.GetPartNo()); + + auto* write = request->AddCmdWrite(); + write->SetSourceId(ProducerId); + write->SetSeqNo(SeqNo); + write->SetData(std::move(*result.MutableData())); + write->SetPartNo(result.GetPartNo()); + write->SetTotalParts(result.GetTotalParts()); + write->SetTotalSize(result.GetTotalSize()); + write->SetCreateTimeMS(result.GetCreateTimestampMS()); + write->SetDisableDeduplication(false); + write->SetWriteTimeMS(writeTimeMs); + write->SetUncompressedSize(result.GetUncompressedSize()); + //write->SetClientDC(result.GetC); + if (result.HasPartitionKey()) { + write->SetPartitionKey(std::move(*result.MutablePartitionKey())); + write->SetExplicitHash(std::move(*result.MutableExplicitHash())); + } + + ++NextPartNo; + TotalPartNo = result.GetTotalParts(); + } + + Send(PartitionWriterActorId, std::move(write)); + + WaitWrite(); +} + +void TDLQMoverActor::Handle(TEvPipeCache::TEvDeliveryProblem::TPtr&) { + ReplyError("Source topic unavailable"); +} + +void TDLQMoverActor::WaitWrite() { + Become(&TDLQMoverActor::StateWrite); +} + +void TDLQMoverActor::Handle(TEvPartitionWriter::TEvWriteResponse::TPtr& ev) { + LOG_D("Handle TEvPartitionWriter::TEvWriteResponse"); + + auto* result = ev->Get(); + if (!result->IsSuccess()) { + return ReplyError(TStringBuilder() << "Write error: " << result->GetError().Reason); + } + + if (NextPartNo == TotalPartNo) { + Processed.push_back(Queue.front()); + Queue.pop_front(); + + NextPartNo = 0; + } + + ProcessQueue(); +} + +void TDLQMoverActor::ReplySuccess() { + PassAway(); +} + +void TDLQMoverActor::ReplyError(TString&& error) { + Error = std::move(error); + PassAway(); +} + +void TDLQMoverActor::SendToTablet(std::unique_ptr ev) { + auto forward = std::make_unique(ev.release(), Settings.TabletId, FirstRequest, CacheSubscribeCookie); + Send(MakePipePerNodeCacheID(false), forward.release(), IEventHandle::FlagTrackDelivery); + FirstRequest = false; +} + +STFUNC(TDLQMoverActor::StateDescribe) { + switch (ev->GetTypeRewrite()) { + hFunc(NDescriber::TEvDescribeTopicsResponse, Handle); + sFunc(TEvents::TEvPoison, PassAway); + default: + LOG_E("Unexpected " << EventStr("StateDescribe", ev)); + AFL_VERIFY_DEBUG(false)("Unexpected", EventStr("StateDescribe", ev)); + } +} + +STFUNC(TDLQMoverActor::StateInit) { + switch (ev->GetTypeRewrite()) { + hFunc(TEvPartitionWriter::TEvInitResult, Handle); + hFunc(TEvPartitionWriter::TEvDisconnected, Handle); + sFunc(TEvents::TEvPoison, PassAway); + default: + LOG_E("Unexpected " << EventStr("StateDescribe", ev)); + AFL_VERIFY_DEBUG(false)("Unexpected", EventStr("StateDescribe", ev)); + } +} + +STFUNC(TDLQMoverActor::StateRead) { + switch (ev->GetTypeRewrite()) { + hFunc(TEvPQ::TEvProxyResponse, Handle); + hFunc(TEvPipeCache::TEvDeliveryProblem, Handle); + hFunc(TEvPartitionWriter::TEvDisconnected, Handle); + sFunc(TEvents::TEvPoison, PassAway); + default: + LOG_E("Unexpected " << EventStr("StateDescribe", ev)); + AFL_VERIFY_DEBUG(false)("Unexpected", EventStr("StateDescribe", ev)); + } +} + +STFUNC(TDLQMoverActor::StateWrite) { + switch (ev->GetTypeRewrite()) { + hFunc(TEvPartitionWriter::TEvWriteResponse, Handle); + hFunc(TEvPipeCache::TEvDeliveryProblem, Handle); + hFunc(TEvPartitionWriter::TEvDisconnected, Handle); + sFunc(TEvents::TEvPoison, PassAway); + default: + LOG_E("Unexpected " << EventStr("StateDescribe", ev)); + AFL_VERIFY_DEBUG(false)("Unexpected", EventStr("StateDescribe", ev)); + } +} + NActors::IActor* CreateDLQMover(TDLQMoverSettings&& settings) { return new TDLQMoverActor(std::move(settings)); } diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h index c0aaf9f10576..704d3c79e0ed 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -14,36 +15,56 @@ namespace NKikimr::NPQ::NMLP { class TDLQMoverActor : public TBaseActor , public TConstantLogPrefix { - static constexpr TDuration Timeout = TDuration::Seconds(5); - public: TDLQMoverActor(TDLQMoverSettings&& settings); void Bootstrap(); void PassAway() override; + TString BuildLogPrefix() const override; private: void Handle(NDescriber::TEvDescribeTopicsResponse::TPtr&); STFUNC(StateDescribe); void CreateWriter(); + void Handle(TEvPartitionWriter::TEvInitResult::TPtr&); + void Handle(TEvPartitionWriter::TEvDisconnected::TPtr&); + STFUNC(StateInit); - void Handle(TEvPersQueue::TEvResponse::TPtr&); - void Handle(TEvPQ::TEvError::TPtr&); - void Handle(TEvents::TEvWakeup::TPtr&); + void ProcessQueue(); + void Handle(TEvPQ::TEvProxyResponse::TPtr&); void Handle(TEvPipeCache::TEvDeliveryProblem::TPtr&); + STFUNC(StateRead); - STFUNC(StateWork); - - void ProcessQueue(); + void WaitWrite(); + void Handle(TEvPartitionWriter::TEvWriteResponse::TPtr&); + STFUNC(StateWrite); + void ReplySuccess(); void ReplyError(TString&& error); + void SendToTablet(std::unique_ptr ev); + private: TDLQMoverSettings Settings; + + TString ProducerId; + ui64 SeqNo; std::deque Queue; + TString Error; + std::vector Processed; + NDescriber::TTopicInfo TopicInfo; + const IPartitionChooser::TPartitionInfo* TargetPartition; + TActorId PartitionWriterActorId; + + bool FirstRequest = true; + ui64 FetchCookie = 0; + + ui64 NextPartNo = 0; + ui64 TotalPartNo = 0; + ui64 WriteCookie = 0; }; } // namespace NKikimr::NPQ::NMLP diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp index ae042214c512..05cb34d132f8 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp @@ -341,6 +341,19 @@ bool TStorage::AddMessage(ui64 offset, bool hasMessagegroup, ui32 messageGroupId return true; } +bool TStorage::MarkDLQMoved(ui64 offset) { + if (DLQQueue.empty() || DLQQueue.front() != offset) { + return false; + } + + Commit(offset); + + DLQQueue.pop_front(); + Batch.DeleteFromDLQ(offset); + + return true; +} + std::pair TStorage::GetMessageInt(ui64 offset) const { if (auto it = SlowMessages.find(offset); it != SlowMessages.end()) { return {&it->second, true}; @@ -730,6 +743,7 @@ bool TStorage::TBatch::Empty() const { return ChangedMessages.empty() && !FirstNewMessage.has_value() && DLQ.empty() + && DeletedFromDLQ == 0 && !BaseDeadline.has_value() && !BaseWriteTimestamp.has_value() && MovedToSlowZone.empty() diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.h index af2084751b0e..5d006328718e 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.h @@ -174,6 +174,7 @@ class TStorage { // https://docs.amazonaws.cn/en_us/AWSSimpleQueueService/latest/APIReference/API_ChangeMessageVisibility.html bool ChangeMessageDeadline(ui64 message, TInstant deadline); bool AddMessage(ui64 offset, bool hasMessagegroup, ui32 messageGroupIdHash, TInstant writeTimestamp); + bool MarkDLQMoved(ui64 offset); size_t ProccessDeadlines(); size_t Compact(); From 7780210c3055a744b99e150f39179ce28d864c3c Mon Sep 17 00:00:00 2001 From: Nikolay Shestakov Date: Mon, 10 Nov 2025 14:31:42 +0000 Subject: [PATCH 08/19] test passed --- .../pqtablet/partition/mlp/mlp_common.h | 1 + .../pqtablet/partition/mlp/mlp_consumer.cpp | 12 ++- .../pqtablet/partition/mlp/mlp_consumer.h | 1 + .../partition/mlp/mlp_consumer_ut.cpp | 76 +++++++++++++++++++ .../pqtablet/partition/mlp/mlp_dlq_mover.cpp | 64 ++++++++++------ .../pqtablet/partition/mlp/mlp_dlq_mover.h | 1 + .../pqtablet/partition/partition_write.cpp | 1 + .../persqueue/public/mlp/ut/common/common.cpp | 10 +++ .../persqueue/public/mlp/ut/common/common.h | 1 + 9 files changed, 141 insertions(+), 26 deletions(-) diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h index 503e24f16bb3..1977e9af8117 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h @@ -54,6 +54,7 @@ struct TDLQMoverSettings { TString Database; ui64 TabletId; ui32 PartitionId; + TActorId PartitionActorId; TString ConsumerName; ui64 ConsumerGeneration; TString DestinationTopic; diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp index ba7d0a929d8f..74126076e0d4 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp @@ -690,21 +690,26 @@ void TConsumerActor::Handle(TEvPQ::TEvError::TPtr& ev) { void TConsumerActor::HandleOnWork(TEvents::TEvWakeup::TPtr&) { FetchMessagesIfNeeded(); ProcessEventQueue(); + MoveToDLQIfPossible(); + Schedule(WakeupInterval, new TEvents::TEvWakeup()); +} +void TConsumerActor::MoveToDLQIfPossible() { if (!DLQMoverActorId && !Storage->GetDLQMessages().empty()) { - std::deque messages; + std::deque messages(Storage->GetDLQMessages()); DLQMoverActorId = RegisterWithSameMailbox(CreateDLQMover({ + .ParentActorId = SelfId(), .Database = Database, .TabletId = TabletId, .PartitionId = PartitionId, + .PartitionActorId = PartitionActorId, .ConsumerName = Config.GetName(), .ConsumerGeneration = Config.GetGeneration(), .DestinationTopic = Config.GetDeadLetterQueue(), - .FirstMessageSeqNo = DLQMovedMessageCount, + .FirstMessageSeqNo = DLQMovedMessageCount + 1, .Messages = std::move(messages) })); } - Schedule(WakeupInterval, new TEvents::TEvWakeup()); } void TConsumerActor::Handle(TEvPQ::TEvMLPDLQMoverResponse::TPtr& ev) { @@ -724,6 +729,7 @@ void TConsumerActor::Handle(TEvPQ::TEvMLPDLQMoverResponse::TPtr& ev) { void TConsumerActor::Handle(TEvents::TEvWakeup::TPtr&) { LOG_D("Handle TEvents::TEvWakeup"); + MoveToDLQIfPossible(); Schedule(WakeupInterval, new TEvents::TEvWakeup()); } diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h index 9118245c1867..883ad0db10a0 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h @@ -66,6 +66,7 @@ class TConsumerActor : public TBaseTabletActor bool FetchMessagesIfNeeded(); void ReadSnapshot(); void Persist(); + void MoveToDLQIfPossible(); void CommitIfNeeded(); void UpdateStorageConfig(); diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp index 705fa037f22a..4e5e83b5906b 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp @@ -217,6 +217,82 @@ Y_UNIT_TEST(RetentionStorageAfterReload) { } } +Y_UNIT_TEST(MoveToDLQ) { + auto setup = CreateSetup(); + auto& runtime = setup->GetRuntime(); + + auto driver = TDriver(setup->MakeDriverConfig()); + auto client = TTopicClient(driver); + + client.CreateTopic("/Root/topic1-dlq", NYdb::NTopic::TCreateTopicSettings() + .BeginAddSharedConsumer("mlp-consumer") + .EndAddConsumer()); + + client.CreateTopic("/Root/topic1", NYdb::NTopic::TCreateTopicSettings() + .BeginAddSharedConsumer("mlp-consumer") + .BeginDeadLetterPolicy() + .Enable() + .BeginCondition() + .MaxProcessingAttempts(1) + .EndCondition() + .MoveAction("/Root/topic1-dlq") + .EndDeadLetterPolicy() + .EndAddConsumer()); + + Sleep(TDuration::Seconds(1)); + + WriteMany(setup, "/Root/topic1", 0, 1_KB, 2); + + Sleep(TDuration::Seconds(2)); + + { + CreateReaderActor(runtime, TReaderSettings{ + .DatabasePath = "/Root", + .TopicName = "/Root/topic1", + .Consumer = "mlp-consumer", + }); + auto response = GetReadResponse(runtime); + UNIT_ASSERT_VALUES_EQUAL_C(response->Status, Ydb::StatusIds::SUCCESS, response->ErrorDescription); + UNIT_ASSERT_VALUES_EQUAL(response->Messages.size(), 1); + UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.PartitionId, 0); + UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.PartitionId, 0); + } + + { + CreateUnlockerActor(runtime, TUnlockerSettings{ + .DatabasePath = "/Root", + .TopicName = "/Root/topic1", + .Consumer = "mlp-consumer", + .Messages = {{0, 0}} + }); + + auto result = GetChangeResponse(runtime); + + UNIT_ASSERT_VALUES_EQUAL(result->Status, Ydb::StatusIds::SUCCESS); + UNIT_ASSERT_VALUES_EQUAL(result->Messages.size(), 1); + UNIT_ASSERT_VALUES_EQUAL(result->Messages[0].MessageId.PartitionId, 0); + UNIT_ASSERT_VALUES_EQUAL(result->Messages[0].MessageId.Offset, 0); + UNIT_ASSERT_VALUES_EQUAL(result->Messages[0].Success, true); + + } + + Sleep(TDuration::Seconds(2)); + + { + // The message should appear in DQL + CreateReaderActor(runtime, TReaderSettings{ + .DatabasePath = "/Root", + .TopicName = "/Root/topic1-dlq", + .Consumer = "mlp-consumer", + }); + auto response = GetReadResponse(runtime); + UNIT_ASSERT_VALUES_EQUAL_C(response->Status, Ydb::StatusIds::SUCCESS, response->ErrorDescription); + UNIT_ASSERT_VALUES_EQUAL(response->Messages.size(), 1); + UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.PartitionId, 0); + UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.PartitionId, 0); + } +} + } } // namespace NKikimr::NPQ::NMLP diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp index 8ab3161bdb0e..b919da41fb2e 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp @@ -1,5 +1,7 @@ #include "mlp_dlq_mover.h" +#include + namespace NKikimr::NPQ::NMLP { namespace { @@ -18,6 +20,8 @@ TDLQMoverActor::TDLQMoverActor(TDLQMoverSettings&& settings) void TDLQMoverActor::Bootstrap() { Become(&TDLQMoverActor::StateDescribe); RegisterWithSameMailbox(NDescriber::CreateDescriberActor(SelfId(), Settings.Database, { Settings.DestinationTopic })); + + LOG_E("QUEUE: " << Queue.size()); } void TDLQMoverActor::PassAway() { @@ -59,6 +63,7 @@ void TDLQMoverActor::Handle(NDescriber::TEvDescribeTopicsResponse::TPtr& ev) { } void TDLQMoverActor::CreateWriter() { + LOG_D("Writer creating"); Become(&TDLQMoverActor::StateInit); ProducerId = TStringBuilder() << "DLQMover/" << Settings.TabletId << "/" << Settings.PartitionId << "/" << Settings.ConsumerGeneration << "/" << Settings.ConsumerName; @@ -74,42 +79,47 @@ void TDLQMoverActor::CreateWriter() { .WithAutoRegister(true) .WithDatabase(Settings.Database) .WithTopicPath(TopicInfo.RealPath) - .WithSessionId(sessionId) - .WithInitialSeqNo(Settings.FirstMessageSeqNo); + .WithSessionId(sessionId); + //.WithInitialSeqNo(Settings.FirstMessageSeqNo); PartitionWriterActorId = RegisterWithSameMailbox(CreatePartitionWriter(SelfId(), TargetPartition->TabletId, TargetPartition->PartitionId, opts)); } void TDLQMoverActor::Handle(TEvPartitionWriter::TEvInitResult::TPtr& ev) { - const auto* result = ev->Get(); + LOG_D("Handle TEvPartitionWriter::TEvInitResult"); + const auto* result = ev->Get(); if (!result->IsSuccess()) { + LOG_E(TStringBuilder() << "The error of creating a writer: " << result->GetError().Reason); return ReplyError(TStringBuilder() << "The error of creating a writer: " << result->GetError().Reason); } - ui64 targetSeqNo = result->GetResult().SourceIdInfo.GetSeqNo(); + ui64 targetSeqNo = result->GetResult().SourceIdInfo.GetSeqNo() + 1; for (SeqNo = Settings.FirstMessageSeqNo; SeqNo < targetSeqNo && !Queue.empty(); ++SeqNo) { Processed.push_back(Queue.front()); Queue.pop_front(); } + LOG_E("targetSeqNo == SeqNo " << targetSeqNo << " == " << SeqNo); AFL_ENSURE(targetSeqNo == SeqNo)("t", targetSeqNo)("s", SeqNo); ProcessQueue(); } void TDLQMoverActor::Handle(TEvPartitionWriter::TEvDisconnected::TPtr&) { + LOG_D("Handle TEvPartitionWriter::TEvDisconnected"); ReplyError("The writer disconnected"); } void TDLQMoverActor::ProcessQueue() { + LOG_D("ProcessQueue"); Become(&TDLQMoverActor::StateRead); if (Queue.empty()) { return ReplySuccess(); } - SendToTablet(MakeEvRead(SelfId(), "", Queue.front(), 1, ++FetchCookie, NextPartNo)); + Send(Settings.PartitionActorId,MakeEvRead(SelfId(), CLIENTID_WITHOUT_CONSUMER, Queue.front(), 1, ++FetchCookie, NextPartNo)); } void TDLQMoverActor::Handle(TEvPQ::TEvProxyResponse::TPtr& ev) { @@ -121,19 +131,16 @@ void TDLQMoverActor::Handle(TEvPQ::TEvProxyResponse::TPtr& ev) { } if (!IsSucess(ev)) { - ReplyError(TStringBuilder() << "Fetch message failed: " << ev->Get()->Response->DebugString()); - return; + return ReplyError(TStringBuilder() << "Fetch message failed: " << ev->Get()->Response->DebugString()); } auto& response = ev->Get()->Response; AFL_ENSURE(response->GetPartitionResponse().HasCmdReadResult())("t", Settings.TabletId)("p", Settings.PartitionId)("c", Settings.ConsumerName); - auto write = std::make_unique(++WriteCookie); - auto* request = write->Record.MutablePartitionRequest(); + auto writeRequest = std::make_unique(++WriteCookie); + auto* request = writeRequest->Record.MutablePartitionRequest(); request->SetTopic(Settings.DestinationTopic); - request->SetPartition(TargetPartition->PartitionId); - - auto writeTimeMs = TInstant::Now().MilliSeconds(); + //request->SetPartition(TargetPartition->PartitionId); auto currentOffset = Queue.front(); for (auto& result : *response->MutablePartitionResponse()->MutableCmdReadResult()->MutableResult()) { @@ -150,42 +157,52 @@ void TDLQMoverActor::Handle(TEvPQ::TEvProxyResponse::TPtr& ev) { auto* write = request->AddCmdWrite(); write->SetSourceId(ProducerId); write->SetSeqNo(SeqNo); - write->SetData(std::move(*result.MutableData())); - write->SetPartNo(result.GetPartNo()); - write->SetTotalParts(result.GetTotalParts()); - write->SetTotalSize(result.GetTotalSize()); + if (result.GetTotalParts() > 0) { + write->SetPartNo(result.GetPartNo()); + write->SetTotalParts(result.GetTotalParts()); + write->SetTotalSize(result.GetTotalSize()); + } write->SetCreateTimeMS(result.GetCreateTimestampMS()); write->SetDisableDeduplication(false); - write->SetWriteTimeMS(writeTimeMs); write->SetUncompressedSize(result.GetUncompressedSize()); - //write->SetClientDC(result.GetC); if (result.HasPartitionKey()) { write->SetPartitionKey(std::move(*result.MutablePartitionKey())); write->SetExplicitHash(std::move(*result.MutableExplicitHash())); } + write->SetData(std::move(*result.MutableData())); + ++NextPartNo; TotalPartNo = result.GetTotalParts(); } - Send(PartitionWriterActorId, std::move(write)); + LOG_D("Write message: " << writeRequest->Record.ShortDebugString()); + + Send(PartitionWriterActorId, std::move(writeRequest)); WaitWrite(); } void TDLQMoverActor::Handle(TEvPipeCache::TEvDeliveryProblem::TPtr&) { + LOG_D("Handle TEvPipeCache::TEvDeliveryProblem"); ReplyError("Source topic unavailable"); } void TDLQMoverActor::WaitWrite() { + LOG_D("WaitWrite"); Become(&TDLQMoverActor::StateWrite); } +void TDLQMoverActor::Handle(TEvPartitionWriter::TEvWriteAccepted::TPtr&) { + LOG_D("Handle TEvPartitionWriter::TEvWriteAccepted"); +} + void TDLQMoverActor::Handle(TEvPartitionWriter::TEvWriteResponse::TPtr& ev) { LOG_D("Handle TEvPartitionWriter::TEvWriteResponse"); auto* result = ev->Get(); if (!result->IsSuccess()) { + LOG_E("Write error: " << result->GetError().Reason); return ReplyError(TStringBuilder() << "Write error: " << result->GetError().Reason); } @@ -242,20 +259,21 @@ STFUNC(TDLQMoverActor::StateRead) { hFunc(TEvPartitionWriter::TEvDisconnected, Handle); sFunc(TEvents::TEvPoison, PassAway); default: - LOG_E("Unexpected " << EventStr("StateDescribe", ev)); - AFL_VERIFY_DEBUG(false)("Unexpected", EventStr("StateDescribe", ev)); + LOG_E("Unexpected " << EventStr("StateRead", ev)); + AFL_VERIFY_DEBUG(false)("Unexpected", EventStr("StateRead", ev)); } } STFUNC(TDLQMoverActor::StateWrite) { switch (ev->GetTypeRewrite()) { + hFunc(TEvPartitionWriter::TEvWriteAccepted, Handle); hFunc(TEvPartitionWriter::TEvWriteResponse, Handle); hFunc(TEvPipeCache::TEvDeliveryProblem, Handle); hFunc(TEvPartitionWriter::TEvDisconnected, Handle); sFunc(TEvents::TEvPoison, PassAway); default: - LOG_E("Unexpected " << EventStr("StateDescribe", ev)); - AFL_VERIFY_DEBUG(false)("Unexpected", EventStr("StateDescribe", ev)); + LOG_E("Unexpected " << EventStr("StateWrite", ev)); + AFL_VERIFY_DEBUG(false)("Unexpected", EventStr("StateWrite", ev)); } } diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h index 704d3c79e0ed..6f415583b0ee 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h @@ -37,6 +37,7 @@ class TDLQMoverActor : public TBaseActor STFUNC(StateRead); void WaitWrite(); + void Handle(TEvPartitionWriter::TEvWriteAccepted::TPtr&); void Handle(TEvPartitionWriter::TEvWriteResponse::TPtr&); STFUNC(StateWrite); diff --git a/ydb/core/persqueue/pqtablet/partition/partition_write.cpp b/ydb/core/persqueue/pqtablet/partition/partition_write.cpp index 60f8118b6f2b..241ae98a9a9a 100644 --- a/ydb/core/persqueue/pqtablet/partition/partition_write.cpp +++ b/ydb/core/persqueue/pqtablet/partition/partition_write.cpp @@ -1211,6 +1211,7 @@ bool TPartition::ExecRequest(TWriteMsg& p, ProcessParameters& parameters, TEvKey LOG_D("Already written message. Topic: '" << TopicName() << "' Partition: " << Partition << " SourceId: '" << EscapeC(p.Msg.SourceId) << "'. Message seqNo: " << p.Msg.SeqNo + << ". InitialSeqNo: " << p.InitialSeqNo << ". Committed seqNo: " << sourceId.CommittedSeqNo() << ". Writing seqNo: " << sourceId.UpdatedSeqNo() << ". EndOffset: " << BlobEncoder.EndOffset << ". CurOffset: " << curOffset << ". Offset: " << poffset diff --git a/ydb/core/persqueue/public/mlp/ut/common/common.cpp b/ydb/core/persqueue/public/mlp/ut/common/common.cpp index eaf564c60347..4506f3806a98 100644 --- a/ydb/core/persqueue/public/mlp/ut/common/common.cpp +++ b/ydb/core/persqueue/public/mlp/ut/common/common.cpp @@ -12,6 +12,7 @@ std::shared_ptr CreateSetup() { NKikimrServices::PQ_MLP_DEADLINER, NKikimrServices::PQ_MLP_CONSUMER, NKikimrServices::PQ_MLP_ENRICHER, + NKikimrServices::PQ_MLP_DLQ_MOVER, NKikimrServices::PERSQUEUE, NKikimrServices::PERSQUEUE_READ_BALANCER, }, @@ -75,6 +76,15 @@ TActorId CreateCommitterActor(NActors::TTestActorRuntime& runtime, TCommitterSet return readerId; } +TActorId CreateUnlockerActor(NActors::TTestActorRuntime& runtime, TUnlockerSettings&& settings) { + auto edgeId = runtime.AllocateEdgeActor(); + auto readerId = runtime.Register(CreateUnlocker(edgeId, std::move(settings))); + runtime.EnableScheduleForActor(readerId); + runtime.DispatchEvents(); + + return readerId; +} + TActorId CreateMessageDeadlineChangerActor(NActors::TTestActorRuntime& runtime, TMessageDeadlineChangerSettings&& settings) { auto edgeId = runtime.AllocateEdgeActor(); auto readerId = runtime.Register(CreateMessageDeadlineChanger(edgeId, std::move(settings))); diff --git a/ydb/core/persqueue/public/mlp/ut/common/common.h b/ydb/core/persqueue/public/mlp/ut/common/common.h index bb05dcad8176..70ada5bda2da 100644 --- a/ydb/core/persqueue/public/mlp/ut/common/common.h +++ b/ydb/core/persqueue/public/mlp/ut/common/common.h @@ -26,6 +26,7 @@ void CreateTopic(std::shared_ptr& setup, const TString& topi void CreateTopic(std::shared_ptr& setup, const TString& topicName, const TString& consumerName); TActorId CreateReaderActor(NActors::TTestActorRuntime& runtime, TReaderSettings&& settings); TActorId CreateCommitterActor(NActors::TTestActorRuntime& runtime, TCommitterSettings&& settings); +TActorId CreateUnlockerActor(NActors::TTestActorRuntime& runtime, TUnlockerSettings&& settings); TActorId CreateMessageDeadlineChangerActor(NActors::TTestActorRuntime& runtime, TMessageDeadlineChangerSettings&& settings); TActorId CreateDescriberActor(NActors::TTestActorRuntime& runtime,const TString& databasePath, const TString& topicPath); THolder WaitResult(NActors::TTestActorRuntime& runtime); From 000d6f17ebd9309a8d7b54faef57bcb8c36eeffc Mon Sep 17 00:00:00 2001 From: Nikolay Shestakov Date: Mon, 10 Nov 2025 16:23:07 +0000 Subject: [PATCH 09/19] WIP --- .../pqtablet/partition/mlp/mlp_dlq_mover.cpp | 13 ++++++------- .../persqueue/pqtablet/partition/mlp/ut/ya.make | 2 ++ ydb/core/persqueue/writer/common.h | 3 ++- 3 files changed, 10 insertions(+), 8 deletions(-) diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp index b919da41fb2e..13d0975ba6ba 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp @@ -20,8 +20,7 @@ TDLQMoverActor::TDLQMoverActor(TDLQMoverSettings&& settings) void TDLQMoverActor::Bootstrap() { Become(&TDLQMoverActor::StateDescribe); RegisterWithSameMailbox(NDescriber::CreateDescriberActor(SelfId(), Settings.Database, { Settings.DestinationTopic })); - - LOG_E("QUEUE: " << Queue.size()); + LOG_D("QUEUE: " << Queue.size()); } void TDLQMoverActor::PassAway() { @@ -80,7 +79,6 @@ void TDLQMoverActor::CreateWriter() { .WithDatabase(Settings.Database) .WithTopicPath(TopicInfo.RealPath) .WithSessionId(sessionId); - //.WithInitialSeqNo(Settings.FirstMessageSeqNo); PartitionWriterActorId = RegisterWithSameMailbox(CreatePartitionWriter(SelfId(), TargetPartition->TabletId, TargetPartition->PartitionId, opts)); } @@ -100,8 +98,8 @@ void TDLQMoverActor::Handle(TEvPartitionWriter::TEvInitResult::TPtr& ev) { Queue.pop_front(); } - LOG_E("targetSeqNo == SeqNo " << targetSeqNo << " == " << SeqNo); - AFL_ENSURE(targetSeqNo == SeqNo)("t", targetSeqNo)("s", SeqNo); + // targetSeqNo can be eq to 0 if the topic has been recreated + AFL_ENSURE(targetSeqNo <= SeqNo)("t", targetSeqNo)("s", SeqNo); ProcessQueue(); } @@ -140,12 +138,13 @@ void TDLQMoverActor::Handle(TEvPQ::TEvProxyResponse::TPtr& ev) { auto writeRequest = std::make_unique(++WriteCookie); auto* request = writeRequest->Record.MutablePartitionRequest(); request->SetTopic(Settings.DestinationTopic); - //request->SetPartition(TargetPartition->PartitionId); auto currentOffset = Queue.front(); for (auto& result : *response->MutablePartitionResponse()->MutableCmdReadResult()->MutableResult()) { if (currentOffset > result.GetOffset()) { continue; + } else if (currentOffset < result.GetOffset()) { + break; } AFL_ENSURE(currentOffset == result.GetOffset())("l", currentOffset)("r", result.GetOffset()); @@ -206,7 +205,7 @@ void TDLQMoverActor::Handle(TEvPartitionWriter::TEvWriteResponse::TPtr& ev) { return ReplyError(TStringBuilder() << "Write error: " << result->GetError().Reason); } - if (NextPartNo == TotalPartNo) { + if (NextPartNo >= TotalPartNo) { Processed.push_back(Queue.front()); Queue.pop_front(); diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/ut/ya.make b/ydb/core/persqueue/pqtablet/partition/mlp/ut/ya.make index 39d52cddb2d3..a7045d43bf3c 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/ut/ya.make +++ b/ydb/core/persqueue/pqtablet/partition/mlp/ut/ya.make @@ -2,6 +2,8 @@ UNITTEST_FOR(ydb/core/persqueue/pqtablet/partition/mlp) YQL_LAST_ABI_VERSION() +SIZE(MEDIUM) + SRCS( mlp_storage_ut.cpp mlp_consumer_ut.cpp diff --git a/ydb/core/persqueue/writer/common.h b/ydb/core/persqueue/writer/common.h index 92b4f5ff56ca..825a0f559941 100644 --- a/ydb/core/persqueue/writer/common.h +++ b/ydb/core/persqueue/writer/common.h @@ -12,7 +12,8 @@ namespace NKikimr::NPQ { inline bool BasicCheck(const NKikimrClient::TResponse& response, TString& error, bool mustHaveResponse = true) { if (response.GetStatus() != NMsgBusProxy::MSTATUS_OK) { error = TStringBuilder() << "Status is not ok" - << ": status# " << static_cast(response.GetStatus()); + << ": status# " << static_cast(response.GetStatus()) + << " " << response.GetErrorReason(); return false; } From 5a9aa135d3fb9e63afafad923a1becec83bd5a0e Mon Sep 17 00:00:00 2001 From: Nikolay Shestakov Date: Mon, 10 Nov 2025 16:34:00 +0000 Subject: [PATCH 10/19] fix --- ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp index 4e5e83b5906b..7b706a1c03a3 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp @@ -241,7 +241,7 @@ Y_UNIT_TEST(MoveToDLQ) { Sleep(TDuration::Seconds(1)); - WriteMany(setup, "/Root/topic1", 0, 1_KB, 2); + setup->Write("/Root/topic1", "short-message", 0); Sleep(TDuration::Seconds(2)); @@ -284,12 +284,14 @@ Y_UNIT_TEST(MoveToDLQ) { .DatabasePath = "/Root", .TopicName = "/Root/topic1-dlq", .Consumer = "mlp-consumer", + .UncompressMessages = true }); auto response = GetReadResponse(runtime); UNIT_ASSERT_VALUES_EQUAL_C(response->Status, Ydb::StatusIds::SUCCESS, response->ErrorDescription); UNIT_ASSERT_VALUES_EQUAL(response->Messages.size(), 1); UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.PartitionId, 0); UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.PartitionId, 0); + UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].Data, "short-message"); } } From 22e79633b4e5798a75830de2227d715c2c2afde7 Mon Sep 17 00:00:00 2001 From: Nikolay Shestakov Date: Mon, 10 Nov 2025 17:23:07 +0000 Subject: [PATCH 11/19] fix --- .../pqtablet/partition/mlp/mlp_common.cpp | 20 ++++++++++++- .../pqtablet/partition/mlp/mlp_common.h | 9 +++++- .../pqtablet/partition/mlp/mlp_consumer.cpp | 30 +++++++++++++++---- .../pqtablet/partition/mlp/mlp_consumer.h | 4 +++ 4 files changed, 55 insertions(+), 8 deletions(-) diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.cpp index 6008a798c42c..e516487ef22d 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.cpp @@ -32,7 +32,7 @@ std::unique_ptr MakeEvRead( } std::unique_ptr MakeEvCommit( - const NKikimrPQ::TPQTabletConfig::TConsumer consumer, + const NKikimrPQ::TPQTabletConfig::TConsumer& consumer, ui64 offset, ui64 cookie ) { @@ -48,6 +48,24 @@ std::unique_ptr MakeEvCommit( ); } +std::unique_ptr MakeEvHasData( + const TActorId& selfId, + ui32 partitionId, + ui64 offset, + const NKikimrPQ::TPQTabletConfig::TConsumer& consumer +) { + + auto result = std::make_unique(); + auto& record = result->Record; + record.SetPartition(partitionId); + record.SetOffset(offset); + record.SetDeadline(TDuration::Seconds(5).MilliSeconds()); + ActorIdToProto(selfId, record.MutableSender()); + record.SetClientId(consumer.GetName()); + + return result; +} + bool IsSucess(const TEvPQ::TEvProxyResponse::TPtr& ev) { return ev->Get()->Response->GetStatus() == NMsgBusProxy::MSTATUS_OK && ev->Get()->Response->GetErrorCode() == NPersQueue::NErrorCode::OK; diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h index 1977e9af8117..3ff57935314b 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h @@ -35,11 +35,18 @@ std::unique_ptr MakeEvRead( ); std::unique_ptr MakeEvCommit( - const NKikimrPQ::TPQTabletConfig::TConsumer consumer, + const NKikimrPQ::TPQTabletConfig::TConsumer& consumer, ui64 offset, ui64 cookie = 0 ); +std::unique_ptr MakeEvHasData( + const TActorId& selfId, + ui32 partitionId, + ui64 offset, + const NKikimrPQ::TPQTabletConfig::TConsumer& consumer +); + bool IsSucess(const TEvPQ::TEvProxyResponse::TPtr& ev); bool IsSucess(const TEvPersQueue::TEvResponse::TPtr& ev); ui64 GetCookie(const TEvPQ::TEvProxyResponse::TPtr& ev); diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp index 74126076e0d4..23796eeea3f0 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp @@ -406,6 +406,7 @@ STFUNC(TConsumerActor::StateWork) { hFunc(TEvPQ::TEvGetMLPConsumerStateRequest, Handle); hFunc(TEvKeyValue::TEvResponse, Handle); hFunc(TEvPQ::TEvProxyResponse, Handle); + hFunc(TEvPersQueue::TEvHasDataInfoResponse, Handle); hFunc(TEvPQ::TEvError, Handle); hFunc(TEvPQ::TEvMLPDLQMoverResponse, Handle); hFunc(TEvents::TEvWakeup, HandleOnWork); @@ -426,6 +427,7 @@ STFUNC(TConsumerActor::StateWrite) { hFunc(TEvPQ::TEvGetMLPConsumerStateRequest, Handle); hFunc(TEvKeyValue::TEvResponse, Handle); hFunc(TEvPQ::TEvProxyResponse, Handle); + hFunc(TEvPersQueue::TEvHasDataInfoResponse, Handle); hFunc(TEvPQ::TEvError, Handle); hFunc(TEvPQ::TEvMLPDLQMoverResponse, Handle); hFunc(TEvents::TEvWakeup, Handle); @@ -598,6 +600,17 @@ void TConsumerActor::Persist() { } } +size_t TConsumerActor::RequireInflyMessageCount() const { + auto& metrics = Storage->GetMetrics(); + + auto maxMessages = Storage->MinMessages; + if (metrics.LockedMessageCount * 2 > metrics.UnprocessedMessageCount) { + maxMessages = std::max(maxMessages, metrics.LockedMessageCount * 2 - metrics.UnprocessedMessageCount); + } + + return std::min(maxMessages, Storage->MaxMessages - metrics.InflyMessageCount); +} + bool TConsumerActor::FetchMessagesIfNeeded() { if (FetchInProgress) { return false; @@ -617,12 +630,7 @@ bool TConsumerActor::FetchMessagesIfNeeded() { FetchInProgress = true; - auto maxMessages = Storage->MinMessages; - if (metrics.LockedMessageCount * 2 > metrics.UnprocessedMessageCount) { - maxMessages = std::max(maxMessages, metrics.LockedMessageCount * 2 - metrics.UnprocessedMessageCount); - } - maxMessages = std::min(maxMessages, Storage->MaxMessages - metrics.InflyMessageCount); - + auto maxMessages = RequireInflyMessageCount(); LOG_D("Fetching " << maxMessages << " messages from offset " << Storage->GetLastOffset() << " from " << PartitionActorId); Send(PartitionActorId, MakeEvRead(SelfId(), Config.GetName(), Storage->GetLastOffset(), maxMessages, ++FetchCookie)); @@ -680,9 +688,19 @@ void TConsumerActor::Handle(TEvPQ::TEvProxyResponse::TPtr& ev) { if (CurrentStateFunc() == &TConsumerActor::StateWork) { ProcessEventQueue(); } + + if (!HasDataInProgress && RequireInflyMessageCount()) { + HasDataInProgress = true; + Send(TabletActorId, MakeEvHasData(SelfId(), PartitionId,Storage->GetLastOffset(), Config)); + } } } +void TConsumerActor::Handle(TEvPersQueue::TEvHasDataInfoResponse::TPtr&) { + LOG_D("Handle TEvPersQueue::TEvHasDataInfo"); + FetchMessagesIfNeeded(); +} + void TConsumerActor::Handle(TEvPQ::TEvError::TPtr& ev) { Restart(TStringBuilder() << "Received error: " << ev->Get()->Error); } diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h index 883ad0db10a0..efe040214c8f 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h @@ -49,6 +49,7 @@ class TConsumerActor : public TBaseTabletActor void HandleOnInit(TEvPQ::TEvProxyResponse::TPtr&); void Handle(TEvPQ::TEvProxyResponse::TPtr&); + void Handle(TEvPersQueue::TEvHasDataInfoResponse::TPtr&); void Handle(TEvPQ::TEvError::TPtr&); void HandleOnWork(TEvents::TEvWakeup::TPtr&); @@ -71,6 +72,8 @@ class TConsumerActor : public TBaseTabletActor void CommitIfNeeded(); void UpdateStorageConfig(); + size_t RequireInflyMessageCount() const; + private: const TString Database; const ui32 PartitionId; @@ -78,6 +81,7 @@ class TConsumerActor : public TBaseTabletActor NKikimrPQ::TPQTabletConfig::TConsumer Config; std::optional RetentionPeriod; + bool HasDataInProgress = false; bool FetchInProgress = false; ui64 FetchCookie = 0; ui64 LastCommittedOffset = 0; From 4ffa24ff860565eac17bc28cedafc54fb09b52c9 Mon Sep 17 00:00:00 2001 From: Nikolay Shestakov Date: Mon, 10 Nov 2025 17:27:57 +0000 Subject: [PATCH 12/19] fix --- .../persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp index 7b706a1c03a3..ef1dfd7fd5f4 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp @@ -276,9 +276,9 @@ Y_UNIT_TEST(MoveToDLQ) { } - Sleep(TDuration::Seconds(2)); - { + for (size_t i = 0; i < 10; ++i) { + Sleep(TDuration::Seconds(1)); // The message should appear in DQL CreateReaderActor(runtime, TReaderSettings{ .DatabasePath = "/Root", @@ -287,6 +287,10 @@ Y_UNIT_TEST(MoveToDLQ) { .UncompressMessages = true }); auto response = GetReadResponse(runtime); + if (i < 10 && (response->Status != Ydb::StatusIds::SUCCESS || response->Messages.empty())) { + continue; + } + UNIT_ASSERT_VALUES_EQUAL_C(response->Status, Ydb::StatusIds::SUCCESS, response->ErrorDescription); UNIT_ASSERT_VALUES_EQUAL(response->Messages.size(), 1); UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.PartitionId, 0); From e0228e6e9cf61c349831f5dac8a85b387628efe3 Mon Sep 17 00:00:00 2001 From: Nikolay Shestakov Date: Mon, 10 Nov 2025 17:44:46 +0000 Subject: [PATCH 13/19] fix --- .../partition/mlp/mlp_consumer_ut.cpp | 23 +++++++++++++------ 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp index ef1dfd7fd5f4..d29949c07c1f 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp @@ -78,14 +78,21 @@ Y_UNIT_TEST(Reload) { Cerr << ">>>>> BEGIN REBOOT " << Endl; ReloadPQTablet(setup, "/Root", "/Root/topic1", 0); - Sleep(TDuration::Seconds(2)); + for (size_t i = 0; i < 10; ++i) { + Sleep(TDuration::Seconds(1)); - auto result = GetConsumerState(setup, "/Root", "/Root/topic1", "mlp-consumer"); + auto result = GetConsumerState(setup, "/Root", "/Root/topic1", "mlp-consumer"); + if (i < 9 && result->Messages.size() != 2) { + continue; + } + + UNIT_ASSERT_VALUES_EQUAL(result->Messages[0].Offset, 1); + UNIT_ASSERT_VALUES_EQUAL(result->Messages[0].Status, TStorage::EMessageStatus::Locked); + UNIT_ASSERT_VALUES_EQUAL(result->Messages[1].Offset, 2); + UNIT_ASSERT_VALUES_EQUAL(result->Messages[1].Status, TStorage::EMessageStatus::Unprocessed); - UNIT_ASSERT_VALUES_EQUAL(result->Messages[0].Offset, 1); - UNIT_ASSERT_VALUES_EQUAL(result->Messages[0].Status, TStorage::EMessageStatus::Locked); - UNIT_ASSERT_VALUES_EQUAL(result->Messages[1].Offset, 2); - UNIT_ASSERT_VALUES_EQUAL(result->Messages[1].Status, TStorage::EMessageStatus::Unprocessed); + break; + } } Y_UNIT_TEST(AlterConsumer) { @@ -287,7 +294,7 @@ Y_UNIT_TEST(MoveToDLQ) { .UncompressMessages = true }); auto response = GetReadResponse(runtime); - if (i < 10 && (response->Status != Ydb::StatusIds::SUCCESS || response->Messages.empty())) { + if (i < 9 && response->Messages.empty()) { continue; } @@ -296,6 +303,8 @@ Y_UNIT_TEST(MoveToDLQ) { UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.PartitionId, 0); UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.PartitionId, 0); UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].Data, "short-message"); + + break; } } From c2bda7a48744aababd08d376c19d40e30411b1c2 Mon Sep 17 00:00:00 2001 From: Nikolay Shestakov Date: Mon, 10 Nov 2025 19:07:29 +0000 Subject: [PATCH 14/19] WIP --- .../pqtablet/partition/mlp/mlp_consumer.h | 3 + .../partition/mlp/mlp_consumer_ut.cpp | 84 ------------- .../pqtablet/partition/mlp/mlp_dlq_mover.cpp | 79 +++++------- .../pqtablet/partition/mlp/mlp_dlq_mover.h | 5 +- .../partition/mlp/mlp_dlq_mover_ut.cpp | 115 ++++++++++++++++++ .../pqtablet/partition/mlp/ut/ya.make | 3 +- 6 files changed, 151 insertions(+), 138 deletions(-) create mode 100644 ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover_ut.cpp diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h index efe040214c8f..9453d8605192 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h @@ -103,6 +103,9 @@ class TConsumerActor : public TBaseTabletActor ui64 LastWALIndex = 0; bool HasSnapshot = false; + + ui64 LastFetchedOffset = 0; + ui64 LastFetchedPartNo = 0; }; } diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp index d29949c07c1f..1a8c2b2c290a 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp @@ -224,90 +224,6 @@ Y_UNIT_TEST(RetentionStorageAfterReload) { } } -Y_UNIT_TEST(MoveToDLQ) { - auto setup = CreateSetup(); - auto& runtime = setup->GetRuntime(); - - auto driver = TDriver(setup->MakeDriverConfig()); - auto client = TTopicClient(driver); - - client.CreateTopic("/Root/topic1-dlq", NYdb::NTopic::TCreateTopicSettings() - .BeginAddSharedConsumer("mlp-consumer") - .EndAddConsumer()); - - client.CreateTopic("/Root/topic1", NYdb::NTopic::TCreateTopicSettings() - .BeginAddSharedConsumer("mlp-consumer") - .BeginDeadLetterPolicy() - .Enable() - .BeginCondition() - .MaxProcessingAttempts(1) - .EndCondition() - .MoveAction("/Root/topic1-dlq") - .EndDeadLetterPolicy() - .EndAddConsumer()); - - Sleep(TDuration::Seconds(1)); - - setup->Write("/Root/topic1", "short-message", 0); - - Sleep(TDuration::Seconds(2)); - - { - CreateReaderActor(runtime, TReaderSettings{ - .DatabasePath = "/Root", - .TopicName = "/Root/topic1", - .Consumer = "mlp-consumer", - }); - auto response = GetReadResponse(runtime); - UNIT_ASSERT_VALUES_EQUAL_C(response->Status, Ydb::StatusIds::SUCCESS, response->ErrorDescription); - UNIT_ASSERT_VALUES_EQUAL(response->Messages.size(), 1); - UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.PartitionId, 0); - UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.PartitionId, 0); - } - - { - CreateUnlockerActor(runtime, TUnlockerSettings{ - .DatabasePath = "/Root", - .TopicName = "/Root/topic1", - .Consumer = "mlp-consumer", - .Messages = {{0, 0}} - }); - - auto result = GetChangeResponse(runtime); - - UNIT_ASSERT_VALUES_EQUAL(result->Status, Ydb::StatusIds::SUCCESS); - UNIT_ASSERT_VALUES_EQUAL(result->Messages.size(), 1); - UNIT_ASSERT_VALUES_EQUAL(result->Messages[0].MessageId.PartitionId, 0); - UNIT_ASSERT_VALUES_EQUAL(result->Messages[0].MessageId.Offset, 0); - UNIT_ASSERT_VALUES_EQUAL(result->Messages[0].Success, true); - - } - - - for (size_t i = 0; i < 10; ++i) { - Sleep(TDuration::Seconds(1)); - // The message should appear in DQL - CreateReaderActor(runtime, TReaderSettings{ - .DatabasePath = "/Root", - .TopicName = "/Root/topic1-dlq", - .Consumer = "mlp-consumer", - .UncompressMessages = true - }); - auto response = GetReadResponse(runtime); - if (i < 9 && response->Messages.empty()) { - continue; - } - - UNIT_ASSERT_VALUES_EQUAL_C(response->Status, Ydb::StatusIds::SUCCESS, response->ErrorDescription); - UNIT_ASSERT_VALUES_EQUAL(response->Messages.size(), 1); - UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.PartitionId, 0); - UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.PartitionId, 0); - UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].Data, "short-message"); - - break; - } -} - } } // namespace NKikimr::NPQ::NMLP diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp index 13d0975ba6ba..58096243abe7 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp @@ -34,6 +34,8 @@ void TDLQMoverActor::PassAway() { Send(Settings.ParentActorId, new TEvPQ::TEvMLPDLQMoverResponse(Ydb::StatusIds::SUCCESS, std::move(Processed))); } + Send(MakePipePerNodeCacheID(false), new TEvPipeCache::TEvUnlink(0)); + TActor::PassAway(); } @@ -117,68 +119,45 @@ void TDLQMoverActor::ProcessQueue() { return ReplySuccess(); } - Send(Settings.PartitionActorId,MakeEvRead(SelfId(), CLIENTID_WITHOUT_CONSUMER, Queue.front(), 1, ++FetchCookie, NextPartNo)); + auto request = std::make_unique(); + auto* partitionRequest = request->Record.MutablePartitionRequest(); + partitionRequest->SetPartition(Settings.PartitionId); + auto* read = partitionRequest->MutableCmdRead(); + read->SetClientId(CLIENTID_WITHOUT_CONSUMER); + read->SetOffset(Queue.front()); + read->SetTimeoutMs(0); + read->SetCount(1); + + SendToPQTablet(std::move(request)); } -void TDLQMoverActor::Handle(TEvPQ::TEvProxyResponse::TPtr& ev) { - LOG_D("Handle TEvPQ::TEvProxyResponse"); - if (FetchCookie != GetCookie(ev)) { - // TODO MLP - LOG_D("Cookie mismatch: " << FetchCookie << " != " << GetCookie(ev)); - //return; - } +void TDLQMoverActor::Handle(TEvPersQueue::TEvResponse::TPtr& ev) { + LOG_D("Handle TEvPersQueue::TEvResponse"); if (!IsSucess(ev)) { - return ReplyError(TStringBuilder() << "Fetch message failed: " << ev->Get()->Response->DebugString()); + return ReplyError(TStringBuilder() << "Fetch message failed: " << ev->Get()->Record.DebugString()); } - auto& response = ev->Get()->Response; - AFL_ENSURE(response->GetPartitionResponse().HasCmdReadResult())("t", Settings.TabletId)("p", Settings.PartitionId)("c", Settings.ConsumerName); + auto& response = ev->Get()->Record; + AFL_ENSURE(response.GetPartitionResponse().HasCmdReadResult()); + auto* result = response.MutablePartitionResponse()->MutableCmdReadResult()->MutableResult(0); auto writeRequest = std::make_unique(++WriteCookie); auto* request = writeRequest->Record.MutablePartitionRequest(); request->SetTopic(Settings.DestinationTopic); - auto currentOffset = Queue.front(); - for (auto& result : *response->MutablePartitionResponse()->MutableCmdReadResult()->MutableResult()) { - if (currentOffset > result.GetOffset()) { - continue; - } else if (currentOffset < result.GetOffset()) { - break; - } - AFL_ENSURE(currentOffset == result.GetOffset())("l", currentOffset)("r", result.GetOffset()); - - if (NextPartNo > result.GetPartNo()) { - continue; - } - AFL_ENSURE(NextPartNo == result.GetPartNo())("l", NextPartNo)("r", result.GetPartNo()); - - auto* write = request->AddCmdWrite(); - write->SetSourceId(ProducerId); - write->SetSeqNo(SeqNo); - if (result.GetTotalParts() > 0) { - write->SetPartNo(result.GetPartNo()); - write->SetTotalParts(result.GetTotalParts()); - write->SetTotalSize(result.GetTotalSize()); - } - write->SetCreateTimeMS(result.GetCreateTimestampMS()); - write->SetDisableDeduplication(false); - write->SetUncompressedSize(result.GetUncompressedSize()); - if (result.HasPartitionKey()) { - write->SetPartitionKey(std::move(*result.MutablePartitionKey())); - write->SetExplicitHash(std::move(*result.MutableExplicitHash())); - } - - write->SetData(std::move(*result.MutableData())); - - ++NextPartNo; - TotalPartNo = result.GetTotalParts(); + auto* write = request->AddCmdWrite(); + write->SetSourceId(ProducerId); + write->SetSeqNo(SeqNo); + write->SetData(std::move(*result->MutableData())); + write->SetCreateTimeMS(result->GetCreateTimestampMS()); + write->SetUncompressedSize(result->GetUncompressedSize()); + if (result->HasPartitionKey()) { + write->SetPartitionKey(std::move(*result->MutablePartitionKey())); + write->SetExplicitHash(std::move(*result->MutableExplicitHash())); } - LOG_D("Write message: " << writeRequest->Record.ShortDebugString()); - Send(PartitionWriterActorId, std::move(writeRequest)); - WaitWrite(); } @@ -224,7 +203,7 @@ void TDLQMoverActor::ReplyError(TString&& error) { PassAway(); } -void TDLQMoverActor::SendToTablet(std::unique_ptr ev) { +void TDLQMoverActor::SendToPQTablet(std::unique_ptr ev) { auto forward = std::make_unique(ev.release(), Settings.TabletId, FirstRequest, CacheSubscribeCookie); Send(MakePipePerNodeCacheID(false), forward.release(), IEventHandle::FlagTrackDelivery); FirstRequest = false; @@ -253,7 +232,7 @@ STFUNC(TDLQMoverActor::StateInit) { STFUNC(TDLQMoverActor::StateRead) { switch (ev->GetTypeRewrite()) { - hFunc(TEvPQ::TEvProxyResponse, Handle); + hFunc(TEvPersQueue::TEvResponse, Handle); hFunc(TEvPipeCache::TEvDeliveryProblem, Handle); hFunc(TEvPartitionWriter::TEvDisconnected, Handle); sFunc(TEvents::TEvPoison, PassAway); diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h index 6f415583b0ee..ed4abe2343b2 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h @@ -32,7 +32,7 @@ class TDLQMoverActor : public TBaseActor STFUNC(StateInit); void ProcessQueue(); - void Handle(TEvPQ::TEvProxyResponse::TPtr&); + void Handle(TEvPersQueue::TEvResponse::TPtr&); void Handle(TEvPipeCache::TEvDeliveryProblem::TPtr&); STFUNC(StateRead); @@ -44,7 +44,7 @@ class TDLQMoverActor : public TBaseActor void ReplySuccess(); void ReplyError(TString&& error); - void SendToTablet(std::unique_ptr ev); + void SendToPQTablet(std::unique_ptr ev); private: TDLQMoverSettings Settings; @@ -61,7 +61,6 @@ class TDLQMoverActor : public TBaseActor TActorId PartitionWriterActorId; bool FirstRequest = true; - ui64 FetchCookie = 0; ui64 NextPartNo = 0; ui64 TotalPartNo = 0; diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover_ut.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover_ut.cpp new file mode 100644 index 000000000000..ac49884a7601 --- /dev/null +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover_ut.cpp @@ -0,0 +1,115 @@ +#include +#include +#include + +namespace NKikimr::NPQ::NMLP { + +Y_UNIT_TEST_SUITE(TMLPDLQMoverTests) { + + +void MoveToDLQ(const TString& msg) { + auto setup = CreateSetup(); + auto& runtime = setup->GetRuntime(); + + auto driver = TDriver(setup->MakeDriverConfig()); + auto client = TTopicClient(driver); + + client.CreateTopic("/Root/topic1-dlq", NYdb::NTopic::TCreateTopicSettings() + .BeginAddSharedConsumer("mlp-consumer") + .EndAddConsumer()); + + client.CreateTopic("/Root/topic1", NYdb::NTopic::TCreateTopicSettings() + .BeginAddSharedConsumer("mlp-consumer") + .BeginDeadLetterPolicy() + .Enable() + .BeginCondition() + .MaxProcessingAttempts(1) + .EndCondition() + .MoveAction("/Root/topic1-dlq") + .EndDeadLetterPolicy() + .EndAddConsumer()); + + Sleep(TDuration::Seconds(1)); + + setup->Write("/Root/topic1", msg, 0); + + Sleep(TDuration::Seconds(2)); + + { + CreateReaderActor(runtime, TReaderSettings{ + .DatabasePath = "/Root", + .TopicName = "/Root/topic1", + .Consumer = "mlp-consumer", + }); + auto response = GetReadResponse(runtime); + UNIT_ASSERT_VALUES_EQUAL_C(response->Status, Ydb::StatusIds::SUCCESS, response->ErrorDescription); + UNIT_ASSERT_VALUES_EQUAL(response->Messages.size(), 1); + UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.PartitionId, 0); + UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.PartitionId, 0); + } + + { + CreateUnlockerActor(runtime, TUnlockerSettings{ + .DatabasePath = "/Root", + .TopicName = "/Root/topic1", + .Consumer = "mlp-consumer", + .Messages = {{0, 0}} + }); + + auto result = GetChangeResponse(runtime); + + UNIT_ASSERT_VALUES_EQUAL(result->Status, Ydb::StatusIds::SUCCESS); + UNIT_ASSERT_VALUES_EQUAL(result->Messages.size(), 1); + UNIT_ASSERT_VALUES_EQUAL(result->Messages[0].MessageId.PartitionId, 0); + UNIT_ASSERT_VALUES_EQUAL(result->Messages[0].MessageId.Offset, 0); + UNIT_ASSERT_VALUES_EQUAL(result->Messages[0].Success, true); + } + + + for (size_t i = 0; i < 10; ++i) { + Sleep(TDuration::Seconds(1)); + // The message should appear in DQL + CreateReaderActor(runtime, TReaderSettings{ + .DatabasePath = "/Root", + .TopicName = "/Root/topic1-dlq", + .Consumer = "mlp-consumer", + .UncompressMessages = true + }); + auto response = GetReadResponse(runtime); + if (i < 9 && response->Messages.empty()) { + continue; + } + + UNIT_ASSERT_VALUES_EQUAL_C(response->Status, Ydb::StatusIds::SUCCESS, response->ErrorDescription); + UNIT_ASSERT_VALUES_EQUAL(response->Messages.size(), 1); + UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.PartitionId, 0); + UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.PartitionId, 0); + UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].Data, msg); + + break; + } + + for (size_t i = 0; i < 10; ++i) { + auto result = GetConsumerState(setup, "/Root", "/Root/topic1", "mlp-consumer"); + if (i < 9 && result->Messages.size() != 0) { + Sleep(TDuration::Seconds(1)); + continue; + } + + UNIT_ASSERT_VALUES_EQUAL(result->Messages.size(), 0); + + break; + } +} + +Y_UNIT_TEST(MoveToDLQ_ShortMessage) { + MoveToDLQ(NUnitTest::RandomString(1_KB)); +} + +Y_UNIT_TEST(MoveToDLQ_BigMessageMessage) { + MoveToDLQ(NUnitTest::RandomString(31_MB)); +} + +} + +} // namespace NKikimr::NPQ::NMLP diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/ut/ya.make b/ydb/core/persqueue/pqtablet/partition/mlp/ut/ya.make index a7045d43bf3c..071e998efb83 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/ut/ya.make +++ b/ydb/core/persqueue/pqtablet/partition/mlp/ut/ya.make @@ -5,8 +5,9 @@ YQL_LAST_ABI_VERSION() SIZE(MEDIUM) SRCS( - mlp_storage_ut.cpp mlp_consumer_ut.cpp + mlp_dlq_mover_ut.cpp + mlp_storage_ut.cpp ) PEERDIR( From 6e33dcaebb96bc891509428ed50ce81be8f15ed0 Mon Sep 17 00:00:00 2001 From: Nikolay Shestakov Date: Tue, 11 Nov 2025 06:18:02 +0000 Subject: [PATCH 15/19] more tests --- .../pqtablet/partition/mlp/mlp_common.h | 3 +- .../pqtablet/partition/mlp/mlp_consumer.cpp | 12 +- .../pqtablet/partition/mlp/mlp_dlq_mover.cpp | 4 +- .../partition/mlp/mlp_dlq_mover_ut.cpp | 122 +++++++++++++++++- .../partition/mlp/mlp_message_enricher.cpp | 40 +++--- .../partition/mlp/mlp_message_enricher.h | 16 +-- 6 files changed, 155 insertions(+), 42 deletions(-) diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h index 3ff57935314b..728378379af0 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_common.h @@ -51,7 +51,7 @@ bool IsSucess(const TEvPQ::TEvProxyResponse::TPtr& ev); bool IsSucess(const TEvPersQueue::TEvResponse::TPtr& ev); ui64 GetCookie(const TEvPQ::TEvProxyResponse::TPtr& ev); -NActors::IActor* CreateMessageEnricher(const NActors::TActorId& tabletActorId, +NActors::IActor* CreateMessageEnricher(ui64 tabletId, const ui32 partitionId, const TString& consumerName, std::deque&& replies); @@ -61,7 +61,6 @@ struct TDLQMoverSettings { TString Database; ui64 TabletId; ui32 PartitionId; - TActorId PartitionActorId; TString ConsumerName; ui64 ConsumerGeneration; TString DestinationTopic; diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp index 23796eeea3f0..971304f82769 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp @@ -317,7 +317,7 @@ void TConsumerActor::Handle(TEvKeyValue::TEvResponse::TPtr& ev) { if (!PendingReadQueue.empty()) { auto msgs = std::exchange(PendingReadQueue, {}); - RegisterWithSameMailbox(CreateMessageEnricher(TabletActorId, PartitionId, Config.GetName(), std::move(msgs))); + RegisterWithSameMailbox(CreateMessageEnricher(TabletId, PartitionId, Config.GetName(), std::move(msgs))); } ReplyOk(SelfId(), PendingCommitQueue); ReplyOk(SelfId(), PendingUnlockQueue); @@ -720,7 +720,6 @@ void TConsumerActor::MoveToDLQIfPossible() { .Database = Database, .TabletId = TabletId, .PartitionId = PartitionId, - .PartitionActorId = PartitionActorId, .ConsumerName = Config.GetName(), .ConsumerGeneration = Config.GetGeneration(), .DestinationTopic = Config.GetDeadLetterQueue(), @@ -734,15 +733,18 @@ void TConsumerActor::Handle(TEvPQ::TEvMLPDLQMoverResponse::TPtr& ev) { LOG_D("Handle TEvPQ::TEvMLPDLQMoverResponse"); if (ev->Get()->Status != Ydb::StatusIds::SUCCESS) { - LOG_W("Error moving messages to the DLQ queue: " << ev->Get()->ErrorDescription); + LOG_W("Error moving messages to the DLQ: " << ev->Get()->ErrorDescription); } + auto& moved = ev->Get()->MovedMessages; + LOG_D("Moved to the DLQ: " << JoinRange(", ", moved.begin(), moved.end())); + DLQMoverActorId = {}; - for (auto offset : ev->Get()->MovedMessages) { + for (auto offset : moved) { AFL_VERIFY(Storage->MarkDLQMoved(offset))("o", offset); } - DLQMovedMessageCount += ev->Get()->MovedMessages.size(); + DLQMovedMessageCount += moved.size(); } void TConsumerActor::Handle(TEvents::TEvWakeup::TPtr&) { diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp index 58096243abe7..814b7442030e 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp @@ -142,13 +142,15 @@ void TDLQMoverActor::Handle(TEvPersQueue::TEvResponse::TPtr& ev) { AFL_ENSURE(response.GetPartitionResponse().HasCmdReadResult()); auto* result = response.MutablePartitionResponse()->MutableCmdReadResult()->MutableResult(0); + LOG_D("Move message with offset " << result->GetOffset() << " seqNo " << SeqNo); + auto writeRequest = std::make_unique(++WriteCookie); auto* request = writeRequest->Record.MutablePartitionRequest(); request->SetTopic(Settings.DestinationTopic); auto* write = request->AddCmdWrite(); write->SetSourceId(ProducerId); - write->SetSeqNo(SeqNo); + write->SetSeqNo(SeqNo++); write->SetData(std::move(*result->MutableData())); write->SetCreateTimeMS(result->GetCreateTimestampMS()); write->SetUncompressedSize(result->GetUncompressedSize()); diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover_ut.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover_ut.cpp index ac49884a7601..16f901d25596 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover_ut.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover_ut.cpp @@ -45,7 +45,7 @@ void MoveToDLQ(const TString& msg) { UNIT_ASSERT_VALUES_EQUAL_C(response->Status, Ydb::StatusIds::SUCCESS, response->ErrorDescription); UNIT_ASSERT_VALUES_EQUAL(response->Messages.size(), 1); UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.PartitionId, 0); - UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.PartitionId, 0); + UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.Offset, 0); } { @@ -83,7 +83,7 @@ void MoveToDLQ(const TString& msg) { UNIT_ASSERT_VALUES_EQUAL_C(response->Status, Ydb::StatusIds::SUCCESS, response->ErrorDescription); UNIT_ASSERT_VALUES_EQUAL(response->Messages.size(), 1); UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.PartitionId, 0); - UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.PartitionId, 0); + UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.Offset, 0); UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].Data, msg); break; @@ -110,6 +110,124 @@ Y_UNIT_TEST(MoveToDLQ_BigMessageMessage) { MoveToDLQ(NUnitTest::RandomString(31_MB)); } +Y_UNIT_TEST(MoveToDLQ_ManyMessages) { + auto setup = CreateSetup(); + auto& runtime = setup->GetRuntime(); + + auto driver = TDriver(setup->MakeDriverConfig()); + auto client = TTopicClient(driver); + + client.CreateTopic("/Root/topic1-dlq", NYdb::NTopic::TCreateTopicSettings() + .BeginAddSharedConsumer("mlp-consumer") + .EndAddConsumer()); + + client.CreateTopic("/Root/topic1", NYdb::NTopic::TCreateTopicSettings() + .BeginAddSharedConsumer("mlp-consumer") + .BeginDeadLetterPolicy() + .Enable() + .BeginCondition() + .MaxProcessingAttempts(1) + .EndCondition() + .MoveAction("/Root/topic1-dlq") + .EndDeadLetterPolicy() + .EndAddConsumer()); + + Sleep(TDuration::Seconds(1)); + + auto msg0 = NUnitTest::RandomString(1_KB); + auto msg1 = NUnitTest::RandomString(1_KB); + auto msg2 = NUnitTest::RandomString(1_KB); + + setup->Write("/Root/topic1", msg0, 0); + setup->Write("/Root/topic1", msg1, 0); + setup->Write("/Root/topic1", msg2, 0); + + Sleep(TDuration::Seconds(2)); + + { + CreateReaderActor(runtime, TReaderSettings{ + .DatabasePath = "/Root", + .TopicName = "/Root/topic1", + .Consumer = "mlp-consumer", + .MaxNumberOfMessage = 3, + .UncompressMessages = true + }); + auto response = GetReadResponse(runtime); + UNIT_ASSERT_VALUES_EQUAL_C(response->Status, Ydb::StatusIds::SUCCESS, response->ErrorDescription); + UNIT_ASSERT_VALUES_EQUAL(response->Messages.size(), 3); + } + + auto unlock = [&](const TString& topic, std::vector offsets) { + auto settings = TUnlockerSettings{ + .DatabasePath = "/Root", + .TopicName = topic, + .Consumer = "mlp-consumer", + }; + + for (auto& o : offsets) { + settings.Messages.push_back({0, o}); + } + + CreateUnlockerActor(runtime, std::move(settings)); + + auto result = GetChangeResponse(runtime); + + UNIT_ASSERT_VALUES_EQUAL(result->Status, Ydb::StatusIds::SUCCESS); + UNIT_ASSERT_VALUES_EQUAL(result->Messages.size(), offsets.size()); + }; + + unlock("/Root/topic1", {2, 0, 1}); + + for (size_t i = 0; i < 10; ++i) { + Sleep(TDuration::Seconds(1)); + // The message should appear in DQL + CreateReaderActor(runtime, TReaderSettings{ + .DatabasePath = "/Root", + .TopicName = "/Root/topic1-dlq", + .Consumer = "mlp-consumer", + .MaxNumberOfMessage = 10, + .UncompressMessages = true + }); + auto response = GetReadResponse(runtime); + if (i < 9 && response->Messages.size() != 3) { + Cerr << (TStringBuilder() << ">>>>> i: " << i << " response->Messages.size(): " << response->Messages.size() << Endl); + for (auto& m : response->Messages) { + unlock("/Root/topic1-dlq", {m.MessageId.Offset}); + } + continue; + } + + UNIT_ASSERT_VALUES_EQUAL_C(response->Status, Ydb::StatusIds::SUCCESS, response->ErrorDescription); + UNIT_ASSERT_VALUES_EQUAL(response->Messages.size(), 3); + + UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.PartitionId, 0); + UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].MessageId.Offset, 0); + UNIT_ASSERT_VALUES_EQUAL(response->Messages[0].Data, msg2); + + UNIT_ASSERT_VALUES_EQUAL(response->Messages[1].MessageId.PartitionId, 0); + UNIT_ASSERT_VALUES_EQUAL(response->Messages[1].MessageId.Offset, 1); + UNIT_ASSERT_VALUES_EQUAL(response->Messages[1].Data, msg0); + + UNIT_ASSERT_VALUES_EQUAL(response->Messages[2].MessageId.PartitionId, 0); + UNIT_ASSERT_VALUES_EQUAL(response->Messages[2].MessageId.Offset, 2); + UNIT_ASSERT_VALUES_EQUAL(response->Messages[2].Data, msg1); + + break; + } + + for (size_t i = 0; i < 10; ++i) { + auto result = GetConsumerState(setup, "/Root", "/Root/topic1", "mlp-consumer"); + if (i < 9 && result->Messages.size() != 0) { + Sleep(TDuration::Seconds(1)); + continue; + } + + UNIT_ASSERT_VALUES_EQUAL(result->Messages.size(), 0); + + break; + } +} + } } // namespace NKikimr::NPQ::NMLP diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_message_enricher.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_message_enricher.cpp index dd44011a91a4..fa47e100a090 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_message_enricher.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_message_enricher.cpp @@ -2,13 +2,12 @@ namespace NKikimr::NPQ::NMLP { -TMessageEnricherActor::TMessageEnricherActor(const TActorId& tabletActorId, ui32 partitionId, const TString& consumerName, std::deque&& replies) +TMessageEnricherActor::TMessageEnricherActor(ui64 tabletId, ui32 partitionId, const TString& consumerName, std::deque&& replies) : TBaseActor(NKikimrServices::EServiceKikimr::PQ_MLP_ENRICHER) - , TabletActorId(tabletActorId) + , TabletId(tabletId) , PartitionId(partitionId) , ConsumerName(consumerName) , Queue(std::move(replies)) - , Backoff(5, TDuration::MilliSeconds(50)) , PendingResponse(std::make_unique()) { } @@ -16,7 +15,6 @@ TMessageEnricherActor::TMessageEnricherActor(const TActorId& tabletActorId, ui32 void TMessageEnricherActor::Bootstrap() { Become(&TThis::StateWork); ProcessQueue(); - Schedule(Timeout, new TEvents::TEvWakeup()); } void TMessageEnricherActor::PassAway() { @@ -25,6 +23,8 @@ void TMessageEnricherActor::PassAway() { Send(reply.Sender, new TEvPQ::TEvMLPErrorResponse(Ydb::StatusIds::SCHEME_ERROR, "Shutdown"), 0, reply.Cookie); } + Send(MakePipePerNodeCacheID(false), new TEvPipeCache::TEvUnlink(0)); + TBase::PassAway(); } @@ -81,27 +81,15 @@ void TMessageEnricherActor::Handle(TEvPersQueue::TEvResponse::TPtr& ev) { ProcessQueue(); } -void TMessageEnricherActor::Handle(TEvPQ::TEvError::TPtr&) { - LOG_D("Handle TEvPQ::TEvError"); - ProcessQueue(); -} - -void TMessageEnricherActor::Handle(TEvents::TEvWakeup::TPtr&) { - LOG_D("TEvents::TEvWakeup"); - - for (auto& reply : Queue) { - Send(reply.Sender, new TEvPQ::TEvMLPErrorResponse(Ydb::StatusIds::TIMEOUT, "Enrich timeout"), 0, reply.Cookie); - } - Queue.clear(); - +void TMessageEnricherActor::Handle(TEvPipeCache::TEvDeliveryProblem::TPtr&) { + LOG_D("Handle TEvPipeCache::TEvDeliveryProblem"); PassAway(); } STFUNC(TMessageEnricherActor::StateWork) { switch (ev->GetTypeRewrite()) { hFunc(TEvPersQueue::TEvResponse, Handle); - hFunc(TEvPQ::TEvError, Handle); - hFunc(TEvents::TEvWakeup, Handle); + hFunc(TEvPipeCache::TEvDeliveryProblem, Handle); sFunc(TEvents::TEvPoison, PassAway); default: LOG_E("Unexpected " << EventStr("StateWork", ev)); @@ -121,7 +109,7 @@ void TMessageEnricherActor::ProcessQueue() { auto firstOffset = reply.Offsets.front(); auto lastOffset = Queue.back().Offsets.back(); auto count = lastOffset - firstOffset + 1; - LOG_D("Fetching from offset " << firstOffset << " count " << count << " from " << TabletActorId); + LOG_D("Fetching from offset " << firstOffset << " count " << count << " from " << TabletId); auto request = std::make_unique(); auto* partitionRequest = request->Record.MutablePartitionRequest(); @@ -131,7 +119,7 @@ void TMessageEnricherActor::ProcessQueue() { read->SetOffset(firstOffset); read->SetTimeoutMs(0); - Send(TabletActorId, std::move(request), 0, ++Cookie); + SendToPQTablet(std::move(request)); return; } @@ -141,11 +129,17 @@ void TMessageEnricherActor::ProcessQueue() { } } -NActors::IActor* CreateMessageEnricher(const NActors::TActorId& tabletActorId, +void TMessageEnricherActor::SendToPQTablet(std::unique_ptr ev) { + auto forward = std::make_unique(ev.release(), TabletId, FirstRequest, 1); + Send(MakePipePerNodeCacheID(false), forward.release(), IEventHandle::FlagTrackDelivery); + FirstRequest = false; +} + +NActors::IActor* CreateMessageEnricher(ui64 tabletId, const ui32 partitionId, const TString& consumerName, std::deque&& replies) { - return new TMessageEnricherActor(tabletActorId, partitionId, consumerName, std::move(replies)); + return new TMessageEnricherActor(tabletId, partitionId, consumerName, std::move(replies)); } } // namespace NKikimr::NPQ::NMLP diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_message_enricher.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_message_enricher.h index cfa6be14cbe8..f9d88097e5b5 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_message_enricher.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_message_enricher.h @@ -3,6 +3,7 @@ #include "mlp.h" #include "mlp_common.h" +#include #include #include #include @@ -12,10 +13,8 @@ namespace NKikimr::NPQ::NMLP { class TMessageEnricherActor : public TBaseActor , public TConstantLogPrefix { - static constexpr TDuration Timeout = TDuration::Seconds(5); - public: - TMessageEnricherActor(const TActorId& tabletActorId, + TMessageEnricherActor(ui64 tabletId, const ui32 partitionId, const TString& consumerName, std::deque&& replies); @@ -25,22 +24,21 @@ class TMessageEnricherActor : public TBaseActor private: void Handle(TEvPersQueue::TEvResponse::TPtr&); - void Handle(TEvPQ::TEvError::TPtr&); - void Handle(TEvents::TEvWakeup::TPtr&); + void Handle(TEvPipeCache::TEvDeliveryProblem::TPtr&); STFUNC(StateWork); void ProcessQueue(); + void SendToPQTablet(std::unique_ptr ev); private: - const TActorId TabletActorId; + const ui64 TabletId; const ui32 PartitionId; const TString ConsumerName; std::deque Queue; - TBackoff Backoff; - ui64 Cookie = 0; - std::unique_ptr PendingResponse; + + bool FirstRequest = true; }; } // namespace NKikimr::NPQ::NMLP From 43ea96df6b4d8b59d3e224475f5f94e630212273 Mon Sep 17 00:00:00 2001 From: Nikolay Shestakov Date: Tue, 11 Nov 2025 06:54:24 +0000 Subject: [PATCH 16/19] fix --- .../pqtablet/partition/mlp/mlp_consumer.cpp | 22 +++++++++++++++---- .../pqtablet/partition/mlp/mlp_consumer.h | 7 +++++- .../partition/mlp/mlp_consumer_ut.cpp | 2 +- .../pqtablet/partition/mlp/ut/ya.make | 1 + 4 files changed, 26 insertions(+), 6 deletions(-) diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp index 971304f82769..82264a11d56f 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp @@ -377,6 +377,10 @@ void TConsumerActor::Handle(TEvPQ::TEvGetMLPConsumerStateRequest::TPtr& ev) { Send(ev->Sender, std::move(response), 0, ev->Cookie); } +void TConsumerActor::Handle(TEvPipeCache::TEvDeliveryProblem::TPtr&) { + FirstPipeCacheRequest = true; +} + STFUNC(TConsumerActor::StateInit) { switch (ev->GetTypeRewrite()) { hFunc(TEvPQ::TEvMLPReadRequest, Queue); @@ -408,6 +412,7 @@ STFUNC(TConsumerActor::StateWork) { hFunc(TEvPQ::TEvProxyResponse, Handle); hFunc(TEvPersQueue::TEvHasDataInfoResponse, Handle); hFunc(TEvPQ::TEvError, Handle); + hFunc(TEvPipeCache::TEvDeliveryProblem, Handle); hFunc(TEvPQ::TEvMLPDLQMoverResponse, Handle); hFunc(TEvents::TEvWakeup, HandleOnWork); sFunc(TEvents::TEvPoison, PassAway); @@ -429,6 +434,7 @@ STFUNC(TConsumerActor::StateWrite) { hFunc(TEvPQ::TEvProxyResponse, Handle); hFunc(TEvPersQueue::TEvHasDataInfoResponse, Handle); hFunc(TEvPQ::TEvError, Handle); + hFunc(TEvPipeCache::TEvDeliveryProblem, Handle); hFunc(TEvPQ::TEvMLPDLQMoverResponse, Handle); hFunc(TEvents::TEvWakeup, Handle); sFunc(TEvents::TEvPoison, PassAway); @@ -600,7 +606,7 @@ void TConsumerActor::Persist() { } } -size_t TConsumerActor::RequireInflyMessageCount() const { +size_t TConsumerActor::RequiredToFetchMessageCount() const { auto& metrics = Storage->GetMetrics(); auto maxMessages = Storage->MinMessages; @@ -630,7 +636,7 @@ bool TConsumerActor::FetchMessagesIfNeeded() { FetchInProgress = true; - auto maxMessages = RequireInflyMessageCount(); + auto maxMessages = RequiredToFetchMessageCount(); LOG_D("Fetching " << maxMessages << " messages from offset " << Storage->GetLastOffset() << " from " << PartitionActorId); Send(PartitionActorId, MakeEvRead(SelfId(), Config.GetName(), Storage->GetLastOffset(), maxMessages, ++FetchCookie)); @@ -689,9 +695,11 @@ void TConsumerActor::Handle(TEvPQ::TEvProxyResponse::TPtr& ev) { ProcessEventQueue(); } - if (!HasDataInProgress && RequireInflyMessageCount()) { + if (!HasDataInProgress && RequiredToFetchMessageCount()) { HasDataInProgress = true; - Send(TabletActorId, MakeEvHasData(SelfId(), PartitionId,Storage->GetLastOffset(), Config)); + auto request = MakeEvHasData(SelfId(), PartitionId, Storage->GetLastOffset(), Config); + LOG_D("Subscribing to data: " << request->Record.ShortDebugString()); + SendToPQTablet(std::move(request)); } } } @@ -753,6 +761,12 @@ void TConsumerActor::Handle(TEvents::TEvWakeup::TPtr&) { Schedule(WakeupInterval, new TEvents::TEvWakeup()); } +void TConsumerActor::SendToPQTablet(std::unique_ptr ev) { + auto forward = std::make_unique(ev.release(), TabletId, FirstPipeCacheRequest, 1); + Send(MakePipePerNodeCacheID(false), forward.release(), IEventHandle::FlagTrackDelivery); + FirstPipeCacheRequest = false; +} + NActors::IActor* CreateConsumerActor( const TString& database, ui64 tabletId, diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h index 9453d8605192..f41953add64b 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h @@ -3,6 +3,7 @@ #include "mlp.h" #include "mlp_common.h" +#include #include #include #include @@ -52,6 +53,8 @@ class TConsumerActor : public TBaseTabletActor void Handle(TEvPersQueue::TEvHasDataInfoResponse::TPtr&); void Handle(TEvPQ::TEvError::TPtr&); + void Handle(TEvPipeCache::TEvDeliveryProblem::TPtr&); + void HandleOnWork(TEvents::TEvWakeup::TPtr&); void Handle(TEvents::TEvWakeup::TPtr&); @@ -72,7 +75,8 @@ class TConsumerActor : public TBaseTabletActor void CommitIfNeeded(); void UpdateStorageConfig(); - size_t RequireInflyMessageCount() const; + size_t RequiredToFetchMessageCount() const; + void SendToPQTablet(std::unique_ptr ev); private: const TString Database; @@ -106,6 +110,7 @@ class TConsumerActor : public TBaseTabletActor ui64 LastFetchedOffset = 0; ui64 LastFetchedPartNo = 0; + bool FirstPipeCacheRequest = true; }; } diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp index 1a8c2b2c290a..45ad1bf2ddf0 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer_ut.cpp @@ -8,7 +8,7 @@ namespace NKikimr::NPQ::NMLP { Y_UNIT_TEST_SUITE(TMLPConsumerTests) { -Y_UNIT_TEST(Reload) { +Y_UNIT_TEST(ReloadPQTablet) { auto setup = CreateSetup(); auto& runtime = setup->GetRuntime(); diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/ut/ya.make b/ydb/core/persqueue/pqtablet/partition/mlp/ut/ya.make index 071e998efb83..3bf4340bf49a 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/ut/ya.make +++ b/ydb/core/persqueue/pqtablet/partition/mlp/ut/ya.make @@ -3,6 +3,7 @@ UNITTEST_FOR(ydb/core/persqueue/pqtablet/partition/mlp) YQL_LAST_ABI_VERSION() SIZE(MEDIUM) +#TIMEOUT(30) SRCS( mlp_consumer_ut.cpp From cc4939793f22bf57cc118f8ef632ea09e2a0451a Mon Sep 17 00:00:00 2001 From: Nikolay Shestakov Date: Tue, 11 Nov 2025 08:03:42 +0000 Subject: [PATCH 17/19] typing --- .../persqueue/pqtablet/partition/mlp/mlp_dlq_mover_ut.cpp | 4 ++-- ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp | 2 +- ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage_ut.cpp | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover_ut.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover_ut.cpp index 16f901d25596..a7d3ed6d8fbb 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover_ut.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover_ut.cpp @@ -68,7 +68,7 @@ void MoveToDLQ(const TString& msg) { for (size_t i = 0; i < 10; ++i) { Sleep(TDuration::Seconds(1)); - // The message should appear in DQL + // The message should appear in DLQ CreateReaderActor(runtime, TReaderSettings{ .DatabasePath = "/Root", .TopicName = "/Root/topic1-dlq", @@ -180,7 +180,7 @@ Y_UNIT_TEST(MoveToDLQ_ManyMessages) { for (size_t i = 0; i < 10; ++i) { Sleep(TDuration::Seconds(1)); - // The message should appear in DQL + // The message should appear in DLQ CreateReaderActor(runtime, TReaderSettings{ .DatabasePath = "/Root", .TopicName = "/Root/topic1-dlq", diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp index 05cb34d132f8..5c85acbd906f 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage.cpp @@ -478,7 +478,7 @@ bool TStorage::DoCommit(ui64 offset) { ++Metrics.CommittedMessageCount; } - AFL_ENSURE(-Metrics.LockedMessageCount > 0)("o", offset); + AFL_ENSURE(Metrics.LockedMessageCount > 0)("o", offset); --Metrics.LockedMessageCount; if (KeepMessageOrder && message->HasMessageGroupId) { if (LockedMessageGroupsId.erase(message->MessageGroupIdHash)) { diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage_ut.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage_ut.cpp index cd9b11ee3709..767ee10f03ca 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage_ut.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_storage_ut.cpp @@ -1851,7 +1851,7 @@ Y_UNIT_TEST(SlowZone_Retention_3message) { utils.AssertLoad(); } -Y_UNIT_TEST(ChangeDeadLettePolicy_Delete) { +Y_UNIT_TEST(ChangeDeadLetterPolicy_Delete) { TUtils utils; utils.Storage.SetDeadLetterPolicy(NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_MOVE); utils.Storage.SetMaxMessageProcessingCount(1); @@ -1871,7 +1871,7 @@ Y_UNIT_TEST(ChangeDeadLettePolicy_Delete) { utils.AssertLoad(); } -Y_UNIT_TEST(ChangeDeadLettePolicy_Unspecified) { +Y_UNIT_TEST(ChangeDeadLetterPolicy_Unspecified) { TUtils utils; utils.Storage.SetDeadLetterPolicy(NKikimrPQ::TPQTabletConfig::DEAD_LETTER_POLICY_MOVE); utils.Storage.SetMaxMessageProcessingCount(1); From 5dc9765e817b20449c7cb286135e0cb577745470 Mon Sep 17 00:00:00 2001 From: Nikolay Shestakov Date: Tue, 11 Nov 2025 08:20:43 +0000 Subject: [PATCH 18/19] fix --- .../persqueue/pqtablet/partition/mlp/mlp_consumer.cpp | 2 +- ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h | 2 -- .../persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp | 8 ++------ ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h | 2 -- .../persqueue/pqtablet/partition/mlp/mlp_dlq_mover_ut.cpp | 2 +- 5 files changed, 4 insertions(+), 12 deletions(-) diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp index 82264a11d56f..7f1ab97275df 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.cpp @@ -749,7 +749,7 @@ void TConsumerActor::Handle(TEvPQ::TEvMLPDLQMoverResponse::TPtr& ev) { DLQMoverActorId = {}; for (auto offset : moved) { - AFL_VERIFY(Storage->MarkDLQMoved(offset))("o", offset); + AFL_ENSURE(Storage->MarkDLQMoved(offset))("o", offset); } DLQMovedMessageCount += moved.size(); diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h index f41953add64b..bf2da63718d9 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_consumer.h @@ -108,8 +108,6 @@ class TConsumerActor : public TBaseTabletActor ui64 LastWALIndex = 0; bool HasSnapshot = false; - ui64 LastFetchedOffset = 0; - ui64 LastFetchedPartNo = 0; bool FirstPipeCacheRequest = true; }; diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp index 814b7442030e..5c83b871e7c0 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp @@ -186,12 +186,8 @@ void TDLQMoverActor::Handle(TEvPartitionWriter::TEvWriteResponse::TPtr& ev) { return ReplyError(TStringBuilder() << "Write error: " << result->GetError().Reason); } - if (NextPartNo >= TotalPartNo) { - Processed.push_back(Queue.front()); - Queue.pop_front(); - - NextPartNo = 0; - } + Processed.push_back(Queue.front()); + Queue.pop_front(); ProcessQueue(); } diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h index ed4abe2343b2..bbba0a024688 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.h @@ -62,8 +62,6 @@ class TDLQMoverActor : public TBaseActor bool FirstRequest = true; - ui64 NextPartNo = 0; - ui64 TotalPartNo = 0; ui64 WriteCookie = 0; }; diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover_ut.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover_ut.cpp index a7d3ed6d8fbb..6c71abdf4552 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover_ut.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover_ut.cpp @@ -106,7 +106,7 @@ Y_UNIT_TEST(MoveToDLQ_ShortMessage) { MoveToDLQ(NUnitTest::RandomString(1_KB)); } -Y_UNIT_TEST(MoveToDLQ_BigMessageMessage) { +Y_UNIT_TEST(MoveToDLQ_BigMessage) { MoveToDLQ(NUnitTest::RandomString(31_MB)); } From 50d8af9ebc780945b43536873ee711ed2559af4b Mon Sep 17 00:00:00 2001 From: Nikolay Shestakov Date: Tue, 11 Nov 2025 08:23:08 +0000 Subject: [PATCH 19/19] fix --- ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp index 5c83b871e7c0..33e82d2a2cbb 100644 --- a/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp +++ b/ydb/core/persqueue/pqtablet/partition/mlp/mlp_dlq_mover.cpp @@ -40,7 +40,7 @@ void TDLQMoverActor::PassAway() { } TString TDLQMoverActor::BuildLogPrefix() const { - return TStringBuilder() << "[" << Settings.TabletId<< "][" << Settings.PartitionId << "][DLQ][" << Settings.ConsumerName << "] "; + return TStringBuilder() << "[" << Settings.TabletId << "][" << Settings.PartitionId << "][DLQ][" << Settings.ConsumerName << "] "; } void TDLQMoverActor::Handle(NDescriber::TEvDescribeTopicsResponse::TPtr& ev) { @@ -223,8 +223,8 @@ STFUNC(TDLQMoverActor::StateInit) { hFunc(TEvPartitionWriter::TEvDisconnected, Handle); sFunc(TEvents::TEvPoison, PassAway); default: - LOG_E("Unexpected " << EventStr("StateDescribe", ev)); - AFL_VERIFY_DEBUG(false)("Unexpected", EventStr("StateDescribe", ev)); + LOG_E("Unexpected " << EventStr("StateInit", ev)); + AFL_VERIFY_DEBUG(false)("Unexpected", EventStr("StateInit", ev)); } }