From 5a00bae377526e6ab4d9b284e2ac1276e77033c1 Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Tue, 2 Dec 2025 21:25:28 +0300 Subject: [PATCH 01/27] update offset bugfix --- .../topicreaderinternal/stream_reader_impl.go | 74 ++++--------------- tests/integration/helpers_test.go | 19 +++++ tests/integration/topic_transactions_test.go | 29 ++++++++ 3 files changed, 63 insertions(+), 59 deletions(-) diff --git a/internal/topic/topicreaderinternal/stream_reader_impl.go b/internal/topic/topicreaderinternal/stream_reader_impl.go index e379849db..921ae225c 100644 --- a/internal/topic/topicreaderinternal/stream_reader_impl.go +++ b/internal/topic/topicreaderinternal/stream_reader_impl.go @@ -25,7 +25,6 @@ import ( "github.com/ydb-platform/ydb-go-sdk/v3/internal/xcontext" "github.com/ydb-platform/ydb-go-sdk/v3/internal/xerrors" "github.com/ydb-platform/ydb-go-sdk/v3/internal/xsync" - "github.com/ydb-platform/ydb-go-sdk/v3/retry" "github.com/ydb-platform/ydb-go-sdk/v3/trace" ) @@ -228,12 +227,14 @@ func (r *topicStreamReaderImpl) commitWithTransaction( tx tx.Transaction, batch *topicreadercommon.PublicBatch, ) error { - if err := tx.UnLazy(ctx); err != nil { + err := tx.UnLazy(ctx) + if err != nil { return fmt.Errorf("ydb: failed to materialize transaction: %w", err) } req := r.createUpdateOffsetRequest(ctx, batch, tx) - updateOffsetInTransactionErr := retry.Retry(ctx, func(ctx context.Context) (err error) { + + tx.OnBeforeCommit(func(ctx context.Context) error { logCtx := r.cfg.BaseContext onDone := trace.TopicOnReaderUpdateOffsetsInTransaction( r.cfg.Trace, @@ -243,6 +244,7 @@ func (r *topicStreamReaderImpl) commitWithTransaction( tx.SessionID(), tx, ) + defer func() { onDone(err) }() @@ -251,69 +253,23 @@ func (r *topicStreamReaderImpl) commitWithTransaction( // Otherwise, errors such as `Database coordinators are unavailable` may occur. ctx = endpoint.WithNodeID(ctx, tx.NodeID()) - err = r.topicClient.UpdateOffsetsInTransaction(ctx, req) - - return err - }) - if updateOffsetInTransactionErr == nil { - r.addOnTransactionCompletedHandler(ctx, tx, batch) - } else { - _ = retry.Retry(ctx, func(ctx context.Context) (err error) { - logCtx := r.cfg.BaseContext - onDone := trace.TopicOnReaderTransactionRollback( - r.cfg.Trace, - &logCtx, - r.readerID, - r.readConnectionID, - tx.SessionID(), - tx, - ) - defer func() { - onDone(err) - }() + err := r.topicClient.UpdateOffsetsInTransaction(ctx, req) + if err != nil { + err = fmt.Errorf("updating offsets in transaction: %w", err) + _ = r.CloseWithError(xcontext.ValueOnly(ctx), xerrors.WithStackTrace(xerrors.Retryable(err))) - return tx.Rollback(ctx) - }) + return xerrors.WithStackTrace(err) + } - _ = r.CloseWithError(xcontext.ValueOnly(ctx), xerrors.WithStackTrace(xerrors.Retryable( - fmt.Errorf("ydb: failed add topic offsets in transaction: %w", updateOffsetInTransactionErr), - ))) + commitRange := topicreadercommon.GetCommitRange(batch) + topicreadercommon.BatchGetPartitionSession(batch).SetCommittedOffsetForward(commitRange.CommitOffsetEnd) - return updateOffsetInTransactionErr - } + return nil + }) return nil } -func (r *topicStreamReaderImpl) addOnTransactionCompletedHandler( - ctx context.Context, - tx tx.Transaction, - batch *topicreadercommon.PublicBatch, -) { - commitRange := topicreadercommon.GetCommitRange(batch) - tx.OnCompleted(func(transactionResult error) { - logCtx := r.cfg.BaseContext - onDone := trace.TopicOnReaderTransactionCompleted( - r.cfg.Trace, - &logCtx, - r.readerID, - r.readConnectionID, - tx.SessionID(), - tx, - transactionResult, - ) - defer onDone() - - if transactionResult == nil { - topicreadercommon.BatchGetPartitionSession(batch).SetCommittedOffsetForward(commitRange.CommitOffsetEnd) - } else { - _ = r.CloseWithError(xcontext.ValueOnly(ctx), xerrors.WithStackTrace(xerrors.Retryable( - fmt.Errorf("ydb: failed batch commit because transaction doesn't committed: %w", transactionResult), - ))) - } - }) -} - func (r *topicStreamReaderImpl) createUpdateOffsetRequest( ctx context.Context, batch *topicreadercommon.PublicBatch, diff --git a/tests/integration/helpers_test.go b/tests/integration/helpers_test.go index b6e22bc1b..8f26969dc 100644 --- a/tests/integration/helpers_test.go +++ b/tests/integration/helpers_test.go @@ -19,8 +19,12 @@ import ( "github.com/stretchr/testify/require" "google.golang.org/grpc" + "github.com/ydb-platform/ydb-go-genproto/Ydb_Query_V1" + "github.com/ydb-platform/ydb-go-genproto/protos/Ydb_Query" "github.com/ydb-platform/ydb-go-sdk/v3" "github.com/ydb-platform/ydb-go-sdk/v3/config" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/balancer" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/conn" "github.com/ydb-platform/ydb-go-sdk/v3/internal/xsql" "github.com/ydb-platform/ydb-go-sdk/v3/internal/xsync" "github.com/ydb-platform/ydb-go-sdk/v3/log" @@ -413,6 +417,21 @@ func (scope *scopeT) TablePath(opts ...func(t *tableNameParams)) string { return path.Join(scope.Folder(), scope.TableName(opts...)) } +func (scope *scopeT) DeleteSession(ctx context.Context, sessionID string) { + baConfig := config.New( + config.WithEndpoint("localhost:2136"), // TODO: + config.WithDatabase("/local")) + + ba, err := balancer.New(ctx, baConfig, conn.NewPool(ctx, baConfig)) + scope.Require.NoError(err) + + c := Ydb_Query_V1.NewQueryServiceClient(ba) + _, err = c.DeleteSession(ctx, &Ydb_Query.DeleteSessionRequest{ + SessionId: sessionID, + }) + scope.Require.NoError(err) +} + // logger for tests type testLogger struct { test *xtest.SyncedTest diff --git a/tests/integration/topic_transactions_test.go b/tests/integration/topic_transactions_test.go index a3be5b670..6d5bc0a57 100644 --- a/tests/integration/topic_transactions_test.go +++ b/tests/integration/topic_transactions_test.go @@ -119,6 +119,35 @@ func TestTopicReaderTLIIssue1797(t *testing.T) { })) } +func TestTopicReaderUpdateOffsetsIssue(t *testing.T) { + scope := newScope(t) + ctx, cancel := context.WithTimeout(scope.Ctx, 10*time.Second) + defer cancel() + + db := scope.Driver() + + writer := scope.TopicWriter() + reader := scope.TopicReader() + + scope.Require.NoError(writer.Write(ctx, topicwriter.Message{Data: strings.NewReader("1")})) + + sessionIDAtStart := "" + scope.Require.NoError(db.Query().DoTx(ctx, func(ctx context.Context, tx query.TxActor) error { + sss := tx.(interface{ SessionID() string }) + if sessionIDAtStart == "" { + sessionIDAtStart = sss.SessionID() + + scope.DeleteSession(ctx, sessionIDAtStart) + } + + batch, err := reader.PopMessagesBatchTx(ctx, tx) + scope.Require.NoError(err) + scope.Require.Len(batch.Messages, 1) + + return nil + })) +} + func TestTopicWriterTLI(t *testing.T) { scope := newScope(t) ctx := scope.Ctx From 805e4af2d41c10336b9c1dab8b1ebe8fc43df567 Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Wed, 3 Dec 2025 18:03:42 +0300 Subject: [PATCH 02/27] fix and refactoring --- .../topicreaderinternal/stream_reader_impl.go | 112 +++------ .../stream_reader_impl_test.go | 70 +++--- .../transaction_commits.go | 215 ++++++++++++++++++ .../transaction_mock_test.go | 2 + 4 files changed, 288 insertions(+), 111 deletions(-) create mode 100644 internal/topic/topicreaderinternal/transaction_commits.go diff --git a/internal/topic/topicreaderinternal/stream_reader_impl.go b/internal/topic/topicreaderinternal/stream_reader_impl.go index 921ae225c..cf5fdbdef 100644 --- a/internal/topic/topicreaderinternal/stream_reader_impl.go +++ b/internal/topic/topicreaderinternal/stream_reader_impl.go @@ -14,12 +14,8 @@ import ( "github.com/ydb-platform/ydb-go-sdk/v3/credentials" "github.com/ydb-platform/ydb-go-sdk/v3/internal/background" - "github.com/ydb-platform/ydb-go-sdk/v3/internal/endpoint" - "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic" "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic/rawtopiccommon" "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic/rawtopicreader" - "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawydb" - "github.com/ydb-platform/ydb-go-sdk/v3/internal/operation" "github.com/ydb-platform/ydb-go-sdk/v3/internal/topic/topicreadercommon" "github.com/ydb-platform/ydb-go-sdk/v3/internal/tx" "github.com/ydb-platform/ydb-go-sdk/v3/internal/xcontext" @@ -60,10 +56,11 @@ type topicStreamReaderImpl struct { readConnectionID string readerID int64 - m xsync.RWMutex - err error - started bool - closed bool + m xsync.RWMutex + err error + started bool + closed bool + transactionCommits *TransactionCommitsStorage } type topicStreamReaderConfig struct { @@ -169,6 +166,11 @@ func newTopicStreamReaderStopped( rawMessagesFromBuffer: make(chan rawtopicreader.ServerMessage, 1), } + res.transactionCommits = &TransactionCommitsStorage{ + reader: res, + commits: make(map[string]*transactionCommits), + } + res.backgroundWorkers = *background.NewWorker(stopPump, "topic-reader-stream-background") res.committer = topicreadercommon.NewCommitterStopped(cfg.Trace, labeledContext, cfg.CommitMode, res.send) @@ -222,87 +224,43 @@ func (r *topicStreamReaderImpl) PopMessagesBatchTx( return nil, err } +// commitWithTransaction prepares a batch of messages to be committed within a YDB transaction. +// +// This method does not immediately commit the batch. Instead, it registers hooks that will be +// executed when the transaction is committed: +// 1. Materializes the transaction +// 2. Registers a pre-commit hook to update topic offsets +// 3. Registers a completion hook to update local offsets or close the reader on failure +// +// The actual batch commit happens when the transaction is committed and the registered hooks are executed. +// The offset update is performed on the same node where the transaction was initiated to avoid +// coordination errors. All errors from UpdateOffsetsInTransaction are treated as retryable. +// +// Parameters: +// - ctx: context for the operation +// - tx: the YDB transaction to associate the batch commit with +// - batch: the batch of messages to be committed when the transaction commits +// +// Returns an error if transaction materialization fails. Offset update errors are handled +// within the transaction's pre-commit hook and will cause the transaction to fail. func (r *topicStreamReaderImpl) commitWithTransaction( ctx context.Context, - tx tx.Transaction, + transaction tx.Transaction, batch *topicreadercommon.PublicBatch, ) error { - err := tx.UnLazy(ctx) + err := transaction.UnLazy(ctx) if err != nil { - return fmt.Errorf("ydb: failed to materialize transaction: %w", err) + return fmt.Errorf("failed to materialize transaction: %w", err) } - req := r.createUpdateOffsetRequest(ctx, batch, tx) + r.transactionCommits.Append(transaction, batch) - tx.OnBeforeCommit(func(ctx context.Context) error { - logCtx := r.cfg.BaseContext - onDone := trace.TopicOnReaderUpdateOffsetsInTransaction( - r.cfg.Trace, - &logCtx, - r.readerID, - r.readConnectionID, - tx.SessionID(), - tx, - ) - - defer func() { - onDone(err) - }() - - // UpdateOffsetsInTransaction operation must be executed on the same Node where the transaction was initiated. - // Otherwise, errors such as `Database coordinators are unavailable` may occur. - ctx = endpoint.WithNodeID(ctx, tx.NodeID()) - - err := r.topicClient.UpdateOffsetsInTransaction(ctx, req) - if err != nil { - err = fmt.Errorf("updating offsets in transaction: %w", err) - _ = r.CloseWithError(xcontext.ValueOnly(ctx), xerrors.WithStackTrace(xerrors.Retryable(err))) - - return xerrors.WithStackTrace(err) - } - - commitRange := topicreadercommon.GetCommitRange(batch) - topicreadercommon.BatchGetPartitionSession(batch).SetCommittedOffsetForward(commitRange.CommitOffsetEnd) - - return nil - }) + transaction.OnBeforeCommit(r.transactionCommits.BeforeCommitFn(transaction)) + transaction.OnCompleted(r.transactionCommits.CompletedFn(transaction)) return nil } -func (r *topicStreamReaderImpl) createUpdateOffsetRequest( - ctx context.Context, - batch *topicreadercommon.PublicBatch, - tx tx.Transaction, -) *rawtopic.UpdateOffsetsInTransactionRequest { - commitRange := topicreadercommon.GetCommitRange(batch) - - return &rawtopic.UpdateOffsetsInTransactionRequest{ - OperationParams: rawydb.NewRawOperationParamsFromProto(operation.Params(ctx, 0, 0, operation.ModeSync)), - Tx: rawtopiccommon.TransactionIdentity{ - ID: tx.ID(), - Session: tx.SessionID(), - }, - Topics: []rawtopic.UpdateOffsetsInTransactionRequest_TopicOffsets{ - { - Path: batch.Topic(), - Partitions: []rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets{ - { - PartitionID: batch.PartitionID(), - PartitionOffsets: []rawtopiccommon.OffsetRange{ - { - Start: commitRange.CommitOffsetStart, - End: commitRange.CommitOffsetEnd, - }, - }, - }, - }, - }, - }, - Consumer: r.cfg.Consumer, - } -} - func (r *topicStreamReaderImpl) ReadMessageBatch( ctx context.Context, opts ReadMessageBatchOptions, diff --git a/internal/topic/topicreaderinternal/stream_reader_impl_test.go b/internal/topic/topicreaderinternal/stream_reader_impl_test.go index ab8ff03ee..592816748 100644 --- a/internal/topic/topicreaderinternal/stream_reader_impl_test.go +++ b/internal/topic/topicreaderinternal/stream_reader_impl_test.go @@ -1336,33 +1336,6 @@ func TestUpdateCommitInTransaction(t *testing.T) { txID := "test-tx-id" sessionID := "test-session-id" - e.TopicClient.EXPECT().UpdateOffsetsInTransaction(gomock.Any(), &rawtopic.UpdateOffsetsInTransactionRequest{ - OperationParams: rawydb.OperationParams{ - OperationMode: rawydb.OperationParamsModeSync, - }, - Tx: rawtopiccommon.TransactionIdentity{ - ID: txID, - Session: sessionID, - }, - Topics: []rawtopic.UpdateOffsetsInTransactionRequest_TopicOffsets{ - { - Path: e.partitionSession.Topic, - Partitions: []rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets{ - { - PartitionID: e.partitionSession.PartitionID, - PartitionOffsets: []rawtopiccommon.OffsetRange{ - { - Start: initialCommitOffset, - End: initialCommitOffset + 1, - }, - }, - }, - }, - }, - }, - Consumer: e.reader.cfg.Consumer, - }) - txMock := newMockTransactionWrapper(sessionID, txID) batch, err := topicreadercommon.NewBatch(e.partitionSession, []*topicreadercommon.PublicMessage{ @@ -1375,6 +1348,28 @@ func TestUpdateCommitInTransaction(t *testing.T) { err = e.reader.commitWithTransaction(e.ctx, txMock, batch) require.NoError(t, err) + e.TopicClient.EXPECT().UpdateOffsetsInTransaction(gomock.Any(), gomock.Cond(func(x any) bool { + req, ok := x.(*rawtopic.UpdateOffsetsInTransactionRequest) + if !ok { + return false + } + + return req.OperationParams.OperationMode == rawydb.OperationParamsModeSync && + req.Tx.ID == txID && + req.Tx.Session == sessionID && + req.Consumer == e.reader.cfg.Consumer && + len(req.Topics) == 1 && + req.Topics[0].Path == e.partitionSession.Topic && + len(req.Topics[0].Partitions) == 1 && + req.Topics[0].Partitions[0].PartitionID == e.partitionSession.PartitionID && + len(req.Topics[0].Partitions[0].PartitionOffsets) == 1 && + req.Topics[0].Partitions[0].PartitionOffsets[0].Start == initialCommitOffset && + req.Topics[0].Partitions[0].PartitionOffsets[0].End == initialCommitOffset+1 + })) + + err = txMock.onBeforeCommit[0](e.ctx) + require.NoError(t, err) + require.Len(t, txMock.onCompleted, 1) txMock.onCompleted[0](nil) require.True(t, txMock.materialized) @@ -1400,14 +1395,18 @@ func TestUpdateCommitInTransaction(t *testing.T) { }) require.NoError(t, err) err = e.reader.commitWithTransaction(e.ctx, txMock, batch) + require.NoError(t, err) + + require.NotEmpty(t, txMock.onBeforeCommit) + err = txMock.onBeforeCommit[0](e.ctx) require.ErrorIs(t, err, testError) - require.NoError(t, xerrors.RetryableError(err)) - require.Empty(t, txMock.onCompleted) + + txMock.onCompleted[0](err) require.True(t, e.reader.closed) require.ErrorIs(t, e.reader.err, testError) require.Error(t, xerrors.RetryableError(e.reader.err)) - require.True(t, txMock.RolledBack) + require.False(t, txMock.RolledBack, "Rollback is an responsibility of tx itself") require.True(t, txMock.materialized) }) t.Run("UpdateOffsetsInTransaction must be executed on the tx Node", func(t *testing.T) { @@ -1417,6 +1416,12 @@ func TestUpdateCommitInTransaction(t *testing.T) { txMock := newMockTransactionWrapper("test-session-id", "test-tx-id") txMock.nodeID = 123 + batch, err := topicreadercommon.NewBatch(e.partitionSession, nil) + require.NoError(t, err) + + err = e.reader.commitWithTransaction(e.ctx, txMock, batch) + require.NoError(t, err) + e.TopicClient.EXPECT().UpdateOffsetsInTransaction(gomock.Any(), gomock.Any()).DoAndReturn( func(ctx context.Context, _ *rawtopic.UpdateOffsetsInTransactionRequest) error { nodeID, ok := endpoint.ContextNodeID(ctx) @@ -1427,10 +1432,7 @@ func TestUpdateCommitInTransaction(t *testing.T) { return nil }) - batch, err := topicreadercommon.NewBatch(e.partitionSession, nil) - require.NoError(t, err) - - err = e.reader.commitWithTransaction(e.ctx, txMock, batch) + err = txMock.onBeforeCommit[0](e.ctx) require.NoError(t, err) }) } diff --git a/internal/topic/topicreaderinternal/transaction_commits.go b/internal/topic/topicreaderinternal/transaction_commits.go new file mode 100644 index 000000000..d91e00795 --- /dev/null +++ b/internal/topic/topicreaderinternal/transaction_commits.go @@ -0,0 +1,215 @@ +package topicreaderinternal + +import ( + "context" + "fmt" + "sync" + + "github.com/ydb-platform/ydb-go-sdk/v3/internal/endpoint" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic/rawtopiccommon" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawydb" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/operation" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/topic/topicreadercommon" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/tx" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/xcontext" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/xerrors" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/xsync" + "github.com/ydb-platform/ydb-go-sdk/v3/trace" +) + +type TransactionCommitsStorage struct { + commits map[string]*transactionCommits // by `tx.ID` + + m xsync.RWMutex + + reader *topicStreamReaderImpl +} + +func (s *TransactionCommitsStorage) Append(tx tx.Transaction, batch *topicreadercommon.PublicBatch) { + tc := s.getTransactionCommits(tx) + tc.AppendBatch(batch) +} + +func (s *TransactionCommitsStorage) getTransactionCommits(transaction tx.Transaction) *transactionCommits { + s.m.Lock() + defer s.m.Unlock() + + tc, ok := s.commits[transaction.ID()] + if !ok { + tc = s.newTransactionCommits(transaction) + s.commits[transaction.ID()] = tc + } + + return tc +} + +func (s *TransactionCommitsStorage) newTransactionCommits(transaction tx.Transaction) *transactionCommits { + ranges := topicreadercommon.NewCommitRangesWithCapacity(1) + return &transactionCommits{ + tx: transaction, + commitRanges: &ranges, + reader: s.reader, + } +} + +// transactionCommits manages commit ranges for transactions +type transactionCommits struct { + tx tx.Transaction + commitRanges *topicreadercommon.CommitRanges + + beforeCommitOnce sync.Once + completedOnce sync.Once + + reader *topicStreamReaderImpl +} + +// AppendBatch adds a batch to the commit ranges +func (tc *transactionCommits) AppendBatch(batch *topicreadercommon.PublicBatch) { + tc.commitRanges.Append(batch) +} + +// BeforeCommitFn returns a function that will be called before transaction commit +// This function is guaranteed to be called only once per transaction using sync.Once +func (s *TransactionCommitsStorage) BeforeCommitFn(tx tx.Transaction) tx.OnTransactionBeforeCommit { + return func(ctx context.Context) (err error) { + s.m.RLock() + tc, ok := s.commits[tx.ID()] + s.m.RUnlock() + + if !ok { + return + } + + // Ensure this function is called only once + var callErr error + tc.beforeCommitOnce.Do(func() { + callErr = tc.updateOffsetsInTransaction(ctx) + }) + + return callErr + } +} + +func (s *TransactionCommitsStorage) CompletedFn(tx tx.Transaction) tx.OnTransactionCompletedFunc { + return func(err error) { + s.m.RLock() + tc, ok := s.commits[tx.ID()] + s.m.RUnlock() + + if !ok { + return + } + + tc.completedOnce.Do(func() { + defer func() { + s.m.Lock() + defer s.m.Unlock() + delete(s.commits, tx.ID()) + }() + + tc.onTransactionCompleted(err) + }) + } +} + +// updateOffsetsInTransaction updates offsets in the transaction +func (tc *transactionCommits) updateOffsetsInTransaction(ctx context.Context) (err error) { + cfg := tc.reader.cfg + logCtx := cfg.BaseContext + onDone := trace.TopicOnReaderUpdateOffsetsInTransaction( + cfg.Trace, + &logCtx, + tc.reader.readerID, + tc.reader.readConnectionID, + tc.tx.SessionID(), + tc.tx, + ) + + defer func() { + onDone(err) + }() + + // UpdateOffsetsInTransaction operation must be executed on the same Node where the transaction was initiated. + // Otherwise, errors such as `Database coordinators are unavailable` may occur. + ctx = endpoint.WithNodeID(ctx, tc.tx.NodeID()) + + req := tc.commitRangesToUpdateOffsetsRequest() + + err = tc.reader.topicClient.UpdateOffsetsInTransaction(ctx, req) + if err != nil { + // mark error as retryable - all `UpdateOffsetsInTransaction` failures can be retried + err = xerrors.Retryable(err) + + return xerrors.WithStackTrace(fmt.Errorf("updating offsets in transaction: %w", err)) + } + + return nil +} + +// onTransactionCompleted handles transaction completion +func (tc *transactionCommits) onTransactionCompleted(err error) { + if err != nil { + _ = tc.reader.CloseWithError(xcontext.ValueOnly(tc.reader.cfg.BaseContext), fmt.Errorf("transaction failed: %w", err)) + + return + } + + tc.commitRanges.Optimize() + + // Update committed offsets for all partition sessions + for i := range tc.commitRanges.Ranges { + tc.commitRanges.Optimize() + commitRange := &tc.commitRanges.Ranges[i] + commitRange.PartitionSession.SetCommittedOffsetForward(commitRange.CommitOffsetEnd) + } +} + +// commitRangesToUpdateOffsetsRequest converts commit ranges to UpdateOffsetsInTransaction request +func (tc *transactionCommits) commitRangesToUpdateOffsetsRequest() *rawtopic.UpdateOffsetsInTransactionRequest { + req := &rawtopic.UpdateOffsetsInTransactionRequest{ + OperationParams: rawydb.NewRawOperationParamsFromProto( + operation.Params(context.Background(), 0, 0, operation.ModeSync), + ), + Tx: rawtopiccommon.TransactionIdentity{ + ID: tc.tx.ID(), + Session: tc.tx.SessionID(), + }, + Consumer: tc.reader.cfg.Consumer, + } + + // Convert commit ranges to topics structure + partitionOffsets := tc.commitRanges.ToPartitionsOffsets() + + // Group partition offsets by topic + topicMap := make(map[string][]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets) + for i := range partitionOffsets { + po := &partitionOffsets[i] + // Find the corresponding partition session to get topic and partition ID + var topic string + var partitionID int64 + for j := range tc.commitRanges.Ranges { + if tc.commitRanges.Ranges[j].PartitionSession.StreamPartitionSessionID == po.PartitionSessionID { + topic = tc.commitRanges.Ranges[j].PartitionSession.Topic + partitionID = tc.commitRanges.Ranges[j].PartitionSession.PartitionID + break + } + } + + topicMap[topic] = append(topicMap[topic], rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets{ + PartitionID: partitionID, + PartitionOffsets: po.Offsets, + }) + } + + // Convert map to slice + req.Topics = make([]rawtopic.UpdateOffsetsInTransactionRequest_TopicOffsets, 0, len(topicMap)) + for path, partitions := range topicMap { + req.Topics = append(req.Topics, rawtopic.UpdateOffsetsInTransactionRequest_TopicOffsets{ + Path: path, + Partitions: partitions, + }) + } + + return req +} diff --git a/internal/topic/topicreaderinternal/transaction_mock_test.go b/internal/topic/topicreaderinternal/transaction_mock_test.go index a4bbe5e3a..687b80440 100644 --- a/internal/topic/topicreaderinternal/transaction_mock_test.go +++ b/internal/topic/topicreaderinternal/transaction_mock_test.go @@ -24,6 +24,7 @@ type mockTransaction struct { sessionID string nodeID uint32 onCompleted []tx.OnTransactionCompletedFunc + onBeforeCommit []tx.OnTransactionBeforeCommit RolledBack bool } @@ -43,6 +44,7 @@ func (m *mockTransaction) NodeID() uint32 { } func (m *mockTransaction) OnBeforeCommit(f tx.OnTransactionBeforeCommit) { + m.onBeforeCommit = append(m.onBeforeCommit, f) } func (m *mockTransaction) OnCompleted(f tx.OnTransactionCompletedFunc) { From 8be60feac88133ceae19fead76f917c2b195a2fe Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Wed, 3 Dec 2025 19:57:41 +0300 Subject: [PATCH 03/27] fix linter --- internal/topic/topicreaderinternal/transaction_commits.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/internal/topic/topicreaderinternal/transaction_commits.go b/internal/topic/topicreaderinternal/transaction_commits.go index d91e00795..f0f8d42e5 100644 --- a/internal/topic/topicreaderinternal/transaction_commits.go +++ b/internal/topic/topicreaderinternal/transaction_commits.go @@ -46,6 +46,7 @@ func (s *TransactionCommitsStorage) getTransactionCommits(transaction tx.Transac func (s *TransactionCommitsStorage) newTransactionCommits(transaction tx.Transaction) *transactionCommits { ranges := topicreadercommon.NewCommitRangesWithCapacity(1) + return &transactionCommits{ tx: transaction, commitRanges: &ranges, @@ -192,6 +193,7 @@ func (tc *transactionCommits) commitRangesToUpdateOffsetsRequest() *rawtopic.Upd if tc.commitRanges.Ranges[j].PartitionSession.StreamPartitionSessionID == po.PartitionSessionID { topic = tc.commitRanges.Ranges[j].PartitionSession.Topic partitionID = tc.commitRanges.Ranges[j].PartitionSession.PartitionID + break } } From 1925d3c43a1cdf0212b9485705fe3d1cc352908d Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Thu, 4 Dec 2025 16:53:49 +0300 Subject: [PATCH 04/27] fixes --- .../topicreaderinternal/stream_reader_impl.go | 92 +++++++++++++++---- .../topicreaderinternal/topics_whatever.go | 14 +++ tests/integration/topic_transactions_test.go | 41 +++++++++ 3 files changed, 130 insertions(+), 17 deletions(-) create mode 100644 internal/topic/topicreaderinternal/topics_whatever.go diff --git a/internal/topic/topicreaderinternal/stream_reader_impl.go b/internal/topic/topicreaderinternal/stream_reader_impl.go index cf5fdbdef..931418afd 100644 --- a/internal/topic/topicreaderinternal/stream_reader_impl.go +++ b/internal/topic/topicreaderinternal/stream_reader_impl.go @@ -14,6 +14,7 @@ import ( "github.com/ydb-platform/ydb-go-sdk/v3/credentials" "github.com/ydb-platform/ydb-go-sdk/v3/internal/background" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/endpoint" "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic/rawtopiccommon" "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic/rawtopicreader" "github.com/ydb-platform/ydb-go-sdk/v3/internal/topic/topicreadercommon" @@ -56,11 +57,12 @@ type topicStreamReaderImpl struct { readConnectionID string readerID int64 - m xsync.RWMutex - err error - started bool - closed bool - transactionCommits *TransactionCommitsStorage + m xsync.RWMutex + err error + started bool + closed bool + + batchTxStorage BatchTxStorage } type topicStreamReaderConfig struct { @@ -166,11 +168,6 @@ func newTopicStreamReaderStopped( rawMessagesFromBuffer: make(chan rawtopicreader.ServerMessage, 1), } - res.transactionCommits = &TransactionCommitsStorage{ - reader: res, - commits: make(map[string]*transactionCommits), - } - res.backgroundWorkers = *background.NewWorker(stopPump, "topic-reader-stream-background") res.committer = topicreadercommon.NewCommitterStopped(cfg.Trace, labeledContext, cfg.CommitMode, res.send) @@ -245,22 +242,83 @@ func (r *topicStreamReaderImpl) PopMessagesBatchTx( // within the transaction's pre-commit hook and will cause the transaction to fail. func (r *topicStreamReaderImpl) commitWithTransaction( ctx context.Context, - transaction tx.Transaction, + tx tx.Transaction, batch *topicreadercommon.PublicBatch, ) error { - err := transaction.UnLazy(ctx) - if err != nil { - return fmt.Errorf("failed to materialize transaction: %w", err) + if err := tx.UnLazy(ctx); err != nil { + return fmt.Errorf("ydb: failed to materialize transaction: %w", err) } - r.transactionCommits.Append(transaction, batch) + if r.batchTxStorage.Add(tx, batch) { + // tx hooks already configured - exiting + return nil + } + + tx.OnBeforeCommit(r.txBeforeCommitFn(tx)) + + tx.OnCompleted(func(err error) { + logCtx := r.cfg.BaseContext + onDone := trace.TopicOnReaderTransactionCompleted( + r.cfg.Trace, + &logCtx, + r.readerID, + r.readConnectionID, + tx.SessionID(), + tx, + err, + ) + defer onDone() + + defer r.batchTxStorage.Clear(tx) + + if err != nil { + // mark error as retryable - for proper reconnector working + err = xerrors.Retryable(err) + + _ = r.CloseWithError(xcontext.ValueOnly(ctx), fmt.Errorf("transaction failed: %w", err)) + + return + } - transaction.OnBeforeCommit(r.transactionCommits.BeforeCommitFn(transaction)) - transaction.OnCompleted(r.transactionCommits.CompletedFn(transaction)) + for _, batch := range r.batchTxStorage.GetBatches(tx) { + commitRange := topicreadercommon.GetCommitRange(batch) + topicreadercommon.BatchGetPartitionSession(batch).SetCommittedOffsetForward(commitRange.CommitOffsetEnd) + } + }) return nil } +func (r *topicStreamReaderImpl) txBeforeCommitFn(tx tx.Transaction) tx.OnTransactionBeforeCommit { + return func(ctx context.Context) (err error) { + logCtx := r.cfg.BaseContext + onDone := trace.TopicOnReaderUpdateOffsetsInTransaction( + r.cfg.Trace, + &logCtx, + r.readerID, + r.readConnectionID, + tx.SessionID(), + tx, + ) + defer func() { + onDone(err) + }() + + // UpdateOffsetsInTransaction operation must be executed on the same Node where the transaction was initiated. + // Otherwise, errors such as `Database coordinators are unavailable` may occur. + ctx = endpoint.WithNodeID(ctx, tx.NodeID()) + + req := r.batchTxStorage.GetUpdateOffsetsInTransactionRequest(tx) + + err = r.topicClient.UpdateOffsetsInTransaction(ctx, req) + if err != nil { + return xerrors.WithStackTrace(fmt.Errorf("updating offsets in transaction: %w", err)) + } + + return nil + } +} + func (r *topicStreamReaderImpl) ReadMessageBatch( ctx context.Context, opts ReadMessageBatchOptions, diff --git a/internal/topic/topicreaderinternal/topics_whatever.go b/internal/topic/topicreaderinternal/topics_whatever.go new file mode 100644 index 000000000..871c7413b --- /dev/null +++ b/internal/topic/topicreaderinternal/topics_whatever.go @@ -0,0 +1,14 @@ +package topicreaderinternal + +import ( + "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/topic/topicreadercommon" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/tx" +) + +type BatchTxStorage interface { + Add(transaction tx.Transaction, batch *topicreadercommon.PublicBatch) (txAlreadyExists bool) + GetUpdateOffsetsInTransactionRequest(transaction tx.Transaction) *rawtopic.UpdateOffsetsInTransactionRequest + GetBatches(transaction tx.Transaction) []*topicreadercommon.PublicBatch + Clear(transaction tx.Transaction) +} diff --git a/tests/integration/topic_transactions_test.go b/tests/integration/topic_transactions_test.go index 6d5bc0a57..bb1cafa51 100644 --- a/tests/integration/topic_transactions_test.go +++ b/tests/integration/topic_transactions_test.go @@ -8,6 +8,7 @@ import ( "errors" "fmt" "io" + "log" "os" "strconv" "strings" @@ -146,6 +147,46 @@ func TestTopicReaderUpdateOffsetsIssue(t *testing.T) { return nil })) + + t.Run("several reads", func(t *testing.T) { + scope.Require.NoError(db.Query().DoTx(ctx, func(ctx context.Context, tx query.TxActor) error { + sss := tx.(interface{ SessionID() string }) + if sessionIDAtStart == "" { + sessionIDAtStart = sss.SessionID() + + scope.DeleteSession(ctx, sessionIDAtStart) + } + + batch, err := reader.PopMessagesBatchTx(ctx, tx) + scope.Require.NoError(err) + scope.Require.Len(batch.Messages, 1) + + scope.Require.NoError(writer.Write(ctx, topicwriter.Message{Data: strings.NewReader("2")})) + + return nil + })) + + scope.Require.NoError(writer.Write(ctx, topicwriter.Message{Data: strings.NewReader("3")})) + + scope.Require.NoError(db.Query().DoTx(ctx, func(ctx context.Context, tx query.TxActor) error { + sss := tx.(interface{ SessionID() string }) + if sessionIDAtStart == "" { + sessionIDAtStart = sss.SessionID() + + scope.DeleteSession(ctx, sessionIDAtStart) + } + + batch, err := reader.PopMessagesBatchTx(ctx, tx) + scope.Require.NoError(err) + scope.Require.Len(batch.Messages, 1) + + log.Println(batch.Messages[0].Offset) + + //scope.Require.NoError(writer.Write(ctx, topicwriter.Message{Data: strings.NewReader("2")})) + + return nil + })) + }) } func TestTopicWriterTLI(t *testing.T) { From 5d62f42a75b0bfe997c71317e02037f4a6b5f8e7 Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Thu, 4 Dec 2025 17:23:14 +0300 Subject: [PATCH 05/27] vibecoding) --- .../topicreaderinternal/batch_tx_storage.go | 151 +++++++ .../batch_tx_storage_test.go | 401 ++++++++++++++++++ .../topicreaderinternal/stream_reader_impl.go | 4 +- .../topicreaderinternal/topics_whatever.go | 13 - 4 files changed, 555 insertions(+), 14 deletions(-) create mode 100644 internal/topic/topicreaderinternal/batch_tx_storage.go create mode 100644 internal/topic/topicreaderinternal/batch_tx_storage_test.go diff --git a/internal/topic/topicreaderinternal/batch_tx_storage.go b/internal/topic/topicreaderinternal/batch_tx_storage.go new file mode 100644 index 000000000..2898ab727 --- /dev/null +++ b/internal/topic/topicreaderinternal/batch_tx_storage.go @@ -0,0 +1,151 @@ +package topicreaderinternal + +import ( + "context" + + "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic/rawtopiccommon" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawydb" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/operation" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/topic/topicreadercommon" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/tx" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/xsync" +) + +// batchTxStorage stores batches associated with transactions for commit within transaction. +// It is thread-safe and allows multiple transactions to be managed concurrently. +type batchTxStorage struct { + batches map[string][]*topicreadercommon.PublicBatch + consumer string + m xsync.RWMutex +} + +// newBatchTxStorage creates a new batch transaction storage with the given consumer name. +// The consumer name is used when building UpdateOffsetsInTransactionRequest. +func newBatchTxStorage(consumer string) *batchTxStorage { + return &batchTxStorage{ + batches: make(map[string][]*topicreadercommon.PublicBatch), + consumer: consumer, + } +} + +// Add adds a batch to the transaction storage. +// It returns true if the transaction already exists (has been added before), false otherwise. +// This method is thread-safe. +func (s *batchTxStorage) Add(transaction tx.Transaction, batch *topicreadercommon.PublicBatch) (txAlreadyExists bool) { + s.m.Lock() + defer s.m.Unlock() + + txID := transaction.ID() + _, exists := s.batches[txID] + s.batches[txID] = append(s.batches[txID], batch) + + return exists +} + +// GetBatches returns all batches stored for the given transaction. +// Returns an empty slice (nil) if no batches are stored for the transaction. +// The returned slice is a copy to prevent external modifications. +// This method is thread-safe. +func (s *batchTxStorage) GetBatches(transaction tx.Transaction) []*topicreadercommon.PublicBatch { + s.m.RLock() + defer s.m.RUnlock() + + batches, ok := s.batches[transaction.ID()] + if !ok { + return nil + } + + // Return a copy to prevent external modifications + result := make([]*topicreadercommon.PublicBatch, len(batches)) + copy(result, batches) + + return result +} + +// GetUpdateOffsetsInTransactionRequest builds an UpdateOffsetsInTransactionRequest +// from all batches stored for the given transaction. +// The batches are converted to commit ranges, optimized (adjacent ranges are merged), +// and grouped by topic and partition. +// Returns nil if no batches are stored for the transaction. +// This method is thread-safe. +func (s *batchTxStorage) GetUpdateOffsetsInTransactionRequest(transaction tx.Transaction) *rawtopic.UpdateOffsetsInTransactionRequest { + s.m.RLock() + batches, ok := s.batches[transaction.ID()] + s.m.RUnlock() + + if !ok || len(batches) == 0 { + return nil + } + + // Convert batches to CommitRanges + commitRanges := topicreadercommon.NewCommitRangesWithCapacity(len(batches)) + for _, batch := range batches { + commitRange := topicreadercommon.GetCommitRange(batch) + commitRanges.AppendCommitRange(commitRange) + } + + // Optimize ranges (merge adjacent ranges) + commitRanges.Optimize() + + // Convert to partition offsets + partitionOffsets := commitRanges.ToPartitionsOffsets() + if len(partitionOffsets) == 0 { + return nil + } + + // Group partition offsets by topic + topicMap := make(map[string][]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets) + for i := range partitionOffsets { + po := &partitionOffsets[i] + // Find the corresponding partition session to get topic and partition ID + var topic string + var partitionID int64 + for _, batch := range batches { + commitRange := topicreadercommon.GetCommitRange(batch) + if commitRange.PartitionSession.StreamPartitionSessionID == po.PartitionSessionID { + topic = commitRange.PartitionSession.Topic + partitionID = commitRange.PartitionSession.PartitionID + break + } + } + + topicMap[topic] = append(topicMap[topic], rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets{ + PartitionID: partitionID, + PartitionOffsets: po.Offsets, + }) + } + + // Build request + req := &rawtopic.UpdateOffsetsInTransactionRequest{ + OperationParams: rawydb.NewRawOperationParamsFromProto( + operation.Params(context.Background(), 0, 0, operation.ModeSync), + ), + Tx: rawtopiccommon.TransactionIdentity{ + ID: transaction.ID(), + Session: transaction.SessionID(), + }, + Consumer: s.consumer, + Topics: make([]rawtopic.UpdateOffsetsInTransactionRequest_TopicOffsets, 0, len(topicMap)), + } + + for path, partitions := range topicMap { + req.Topics = append(req.Topics, rawtopic.UpdateOffsetsInTransactionRequest_TopicOffsets{ + Path: path, + Partitions: partitions, + }) + } + + return req +} + +// Clear removes all batches stored for the given transaction. +// After calling Clear, GetBatches and GetUpdateOffsetsInTransactionRequest +// will return empty results for this transaction. +// This method is thread-safe. +func (s *batchTxStorage) Clear(transaction tx.Transaction) { + s.m.Lock() + defer s.m.Unlock() + + delete(s.batches, transaction.ID()) +} diff --git a/internal/topic/topicreaderinternal/batch_tx_storage_test.go b/internal/topic/topicreaderinternal/batch_tx_storage_test.go new file mode 100644 index 000000000..c6bf63bd6 --- /dev/null +++ b/internal/topic/topicreaderinternal/batch_tx_storage_test.go @@ -0,0 +1,401 @@ +package topicreaderinternal + +import ( + "context" + "testing" + + "github.com/stretchr/testify/suite" + + "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic/rawtopiccommon" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic/rawtopicreader" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/topic/topicreadercommon" +) + +type BatchTxStorageTestSuite struct { + suite.Suite + storage *batchTxStorage +} + +func (s *BatchTxStorageTestSuite) SetupTest() { + s.storage = newBatchTxStorage("test-consumer") +} + +func (s *BatchTxStorageTestSuite) materializeTx(tx *mockTransaction) { + _ = tx.UnLazy(context.Background()) +} + +func TestBatchTxStorage(t *testing.T) { + suite.Run(t, new(BatchTxStorageTestSuite)) +} + +func (s *BatchTxStorageTestSuite) TestAdd_NewTransaction() { + tx := newMockTransactionWrapper("session-1", "tx-1") + s.materializeTx(tx) + batch := s.createTestBatch("topic-1", 1, 10, 20, 1) + + exists := s.storage.Add(tx, batch) + + s.False(exists) +} + +func (s *BatchTxStorageTestSuite) TestAdd_ExistingTransaction() { + tx := newMockTransactionWrapper("session-1", "tx-1") + s.materializeTx(tx) + batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) + batch2 := s.createTestBatch("topic-1", 1, 20, 30, 1) + + _ = s.storage.Add(tx, batch1) + exists := s.storage.Add(tx, batch2) + + s.True(exists) +} + +func (s *BatchTxStorageTestSuite) TestGetBatches_Empty() { + tx := newMockTransactionWrapper("session-1", "tx-1") + s.materializeTx(tx) + + batches := s.storage.GetBatches(tx) + + s.Empty(batches) +} + +func (s *BatchTxStorageTestSuite) TestGetBatches_WithBatches() { + tx := newMockTransactionWrapper("session-1", "tx-1") + s.materializeTx(tx) + batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) + batch2 := s.createTestBatch("topic-1", 1, 20, 30, 1) + + _ = s.storage.Add(tx, batch1) + _ = s.storage.Add(tx, batch2) + + batches := s.storage.GetBatches(tx) + + s.Len(batches, 2) + s.Equal(batch1, batches[0]) + s.Equal(batch2, batches[1]) +} + +func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_Empty() { + tx := newMockTransactionWrapper("session-1", "tx-1") + s.materializeTx(tx) + + req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) + + s.Nil(req) +} + +func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_SingleBatch() { + tx := newMockTransactionWrapper("session-1", "tx-1") + s.materializeTx(tx) + batch := s.createTestBatch("topic-1", 1, 10, 20, 1) + + _ = s.storage.Add(tx, batch) + req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) + + s.Require().NotNil(req) + s.Equal("test-consumer", req.Consumer) + s.Equal("tx-1", req.Tx.ID) + s.Equal("session-1", req.Tx.Session) + s.Len(req.Topics, 1) + s.Equal("topic-1", req.Topics[0].Path) + s.Len(req.Topics[0].Partitions, 1) + s.Equal(int64(1), req.Topics[0].Partitions[0].PartitionID) + s.Len(req.Topics[0].Partitions[0].PartitionOffsets, 1) + s.Equal(rawtopiccommon.Offset(10), req.Topics[0].Partitions[0].PartitionOffsets[0].Start) + s.Equal(rawtopiccommon.Offset(20), req.Topics[0].Partitions[0].PartitionOffsets[0].End) +} + +func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_MultipleBatches() { + tx := newMockTransactionWrapper("session-1", "tx-1") + s.materializeTx(tx) + batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) + batch2 := s.createTestBatch("topic-1", 1, 20, 30, 1) + + _ = s.storage.Add(tx, batch1) + _ = s.storage.Add(tx, batch2) + req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) + + s.Require().NotNil(req) + s.Len(req.Topics, 1) + s.Len(req.Topics[0].Partitions, 1) + s.Len(req.Topics[0].Partitions[0].PartitionOffsets, 1) + // Should be optimized to single range + s.Equal(rawtopiccommon.Offset(10), req.Topics[0].Partitions[0].PartitionOffsets[0].Start) + s.Equal(rawtopiccommon.Offset(30), req.Topics[0].Partitions[0].PartitionOffsets[0].End) +} + +func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_MultipleTopics() { + tx := newMockTransactionWrapper("session-1", "tx-1") + s.materializeTx(tx) + batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) + batch2 := s.createTestBatch("topic-2", 2, 30, 40, 2) + + _ = s.storage.Add(tx, batch1) + _ = s.storage.Add(tx, batch2) + req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) + + s.Require().NotNil(req) + s.Len(req.Topics, 2) + // Topics should be grouped by path + topicMap := make(map[string]bool) + for _, topic := range req.Topics { + topicMap[topic.Path] = true + } + s.True(topicMap["topic-1"]) + s.True(topicMap["topic-2"]) +} + +func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_MultiplePartitionsSameTopic() { + tx := newMockTransactionWrapper("session-1", "tx-1") + s.materializeTx(tx) + batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) + batch2 := s.createTestBatch("topic-1", 2, 30, 40, 2) + + _ = s.storage.Add(tx, batch1) + _ = s.storage.Add(tx, batch2) + req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) + + s.Require().NotNil(req) + s.Len(req.Topics, 1) + s.Equal("topic-1", req.Topics[0].Path) + s.Len(req.Topics[0].Partitions, 2) + + // Find partitions + partitionMap := make(map[int64]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets) + for _, partition := range req.Topics[0].Partitions { + partitionMap[partition.PartitionID] = partition + } + + s.Contains(partitionMap, int64(1)) + s.Contains(partitionMap, int64(2)) + s.Len(partitionMap[1].PartitionOffsets, 1) + s.Len(partitionMap[2].PartitionOffsets, 1) + s.Equal(rawtopiccommon.Offset(10), partitionMap[1].PartitionOffsets[0].Start) + s.Equal(rawtopiccommon.Offset(20), partitionMap[1].PartitionOffsets[0].End) + s.Equal(rawtopiccommon.Offset(30), partitionMap[2].PartitionOffsets[0].Start) + s.Equal(rawtopiccommon.Offset(40), partitionMap[2].PartitionOffsets[0].End) +} + +func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_NonAdjacentBatches() { + tx := newMockTransactionWrapper("session-1", "tx-1") + s.materializeTx(tx) + batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) + batch2 := s.createTestBatch("topic-1", 1, 30, 40, 1) // Gap between 20 and 30 + + _ = s.storage.Add(tx, batch1) + _ = s.storage.Add(tx, batch2) + req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) + + s.Require().NotNil(req) + s.Len(req.Topics, 1) + s.Len(req.Topics[0].Partitions, 1) + s.Len(req.Topics[0].Partitions[0].PartitionOffsets, 2) // Should not be merged + s.Equal(rawtopiccommon.Offset(10), req.Topics[0].Partitions[0].PartitionOffsets[0].Start) + s.Equal(rawtopiccommon.Offset(20), req.Topics[0].Partitions[0].PartitionOffsets[0].End) + s.Equal(rawtopiccommon.Offset(30), req.Topics[0].Partitions[0].PartitionOffsets[1].Start) + s.Equal(rawtopiccommon.Offset(40), req.Topics[0].Partitions[0].PartitionOffsets[1].End) +} + +func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_MultiplePartitionsMultipleTopics() { + tx := newMockTransactionWrapper("session-1", "tx-1") + s.materializeTx(tx) + batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) + batch2 := s.createTestBatch("topic-1", 2, 30, 40, 2) + batch3 := s.createTestBatch("topic-2", 1, 50, 60, 3) + batch4 := s.createTestBatch("topic-2", 2, 70, 80, 4) + + _ = s.storage.Add(tx, batch1) + _ = s.storage.Add(tx, batch2) + _ = s.storage.Add(tx, batch3) + _ = s.storage.Add(tx, batch4) + req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) + + s.Require().NotNil(req) + s.Len(req.Topics, 2) + + // Find topics + topicMap := make(map[string]rawtopic.UpdateOffsetsInTransactionRequest_TopicOffsets) + for _, topic := range req.Topics { + topicMap[topic.Path] = topic + } + + s.Contains(topicMap, "topic-1") + s.Contains(topicMap, "topic-2") + s.Len(topicMap["topic-1"].Partitions, 2) + s.Len(topicMap["topic-2"].Partitions, 2) + + // Check topic-1 partitions + partition1Map := make(map[int64]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets) + for _, partition := range topicMap["topic-1"].Partitions { + partition1Map[partition.PartitionID] = partition + } + s.Contains(partition1Map, int64(1)) + s.Contains(partition1Map, int64(2)) + + // Check topic-2 partitions + partition2Map := make(map[int64]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets) + for _, partition := range topicMap["topic-2"].Partitions { + partition2Map[partition.PartitionID] = partition + } + s.Contains(partition2Map, int64(1)) + s.Contains(partition2Map, int64(2)) +} + +func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_ComplexOptimization() { + tx := newMockTransactionWrapper("session-1", "tx-1") + s.materializeTx(tx) + // Adjacent batches that should be merged + batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) + batch2 := s.createTestBatch("topic-1", 1, 20, 30, 1) + batch3 := s.createTestBatch("topic-1", 1, 30, 40, 1) + // Non-adjacent batch (should not merge) + batch4 := s.createTestBatch("topic-1", 1, 50, 60, 1) + + _ = s.storage.Add(tx, batch1) + _ = s.storage.Add(tx, batch2) + _ = s.storage.Add(tx, batch3) + _ = s.storage.Add(tx, batch4) + req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) + + s.Require().NotNil(req) + s.Len(req.Topics, 1) + s.Len(req.Topics[0].Partitions, 1) + // First three should be merged into one range, fourth is separate + s.Len(req.Topics[0].Partitions[0].PartitionOffsets, 2) + s.Equal(rawtopiccommon.Offset(10), req.Topics[0].Partitions[0].PartitionOffsets[0].Start) + s.Equal(rawtopiccommon.Offset(40), req.Topics[0].Partitions[0].PartitionOffsets[0].End) + s.Equal(rawtopiccommon.Offset(50), req.Topics[0].Partitions[0].PartitionOffsets[1].Start) + s.Equal(rawtopiccommon.Offset(60), req.Topics[0].Partitions[0].PartitionOffsets[1].End) +} + +func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_MixedPartitionsAndTopics() { + tx := newMockTransactionWrapper("session-1", "tx-1") + s.materializeTx(tx) + // Topic 1, partition 1 - adjacent batches (should merge) + batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) + batch2 := s.createTestBatch("topic-1", 1, 20, 30, 1) + // Topic 1, partition 2 - non-adjacent batches (should not merge) + batch3 := s.createTestBatch("topic-1", 2, 40, 50, 2) + batch4 := s.createTestBatch("topic-1", 2, 60, 70, 2) + // Topic 2, partition 1 - single batch + batch5 := s.createTestBatch("topic-2", 1, 80, 90, 3) + + _ = s.storage.Add(tx, batch1) + _ = s.storage.Add(tx, batch2) + _ = s.storage.Add(tx, batch3) + _ = s.storage.Add(tx, batch4) + _ = s.storage.Add(tx, batch5) + req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) + + s.Require().NotNil(req) + s.Len(req.Topics, 2) + + // Find topics + topicMap := make(map[string]rawtopic.UpdateOffsetsInTransactionRequest_TopicOffsets) + for _, topic := range req.Topics { + topicMap[topic.Path] = topic + } + + // Check topic-1: should have 2 partitions + s.Contains(topicMap, "topic-1") + s.Len(topicMap["topic-1"].Partitions, 2) + + // Check topic-2: should have 1 partition + s.Contains(topicMap, "topic-2") + s.Len(topicMap["topic-2"].Partitions, 1) + + // Verify topic-1 partition 1 (merged) + partition1Map := make(map[int64]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets) + for _, partition := range topicMap["topic-1"].Partitions { + partition1Map[partition.PartitionID] = partition + } + s.Len(partition1Map[1].PartitionOffsets, 1) // Merged + s.Equal(rawtopiccommon.Offset(10), partition1Map[1].PartitionOffsets[0].Start) + s.Equal(rawtopiccommon.Offset(30), partition1Map[1].PartitionOffsets[0].End) + + // Verify topic-1 partition 2 (not merged) + s.Len(partition1Map[2].PartitionOffsets, 2) // Not merged + s.Equal(rawtopiccommon.Offset(40), partition1Map[2].PartitionOffsets[0].Start) + s.Equal(rawtopiccommon.Offset(50), partition1Map[2].PartitionOffsets[0].End) + s.Equal(rawtopiccommon.Offset(60), partition1Map[2].PartitionOffsets[1].Start) + s.Equal(rawtopiccommon.Offset(70), partition1Map[2].PartitionOffsets[1].End) + + // Verify topic-2 partition 1 + s.Len(topicMap["topic-2"].Partitions[0].PartitionOffsets, 1) + s.Equal(rawtopiccommon.Offset(80), topicMap["topic-2"].Partitions[0].PartitionOffsets[0].Start) + s.Equal(rawtopiccommon.Offset(90), topicMap["topic-2"].Partitions[0].PartitionOffsets[0].End) +} + +func (s *BatchTxStorageTestSuite) TestClear() { + tx := newMockTransactionWrapper("session-1", "tx-1") + s.materializeTx(tx) + batch := s.createTestBatch("topic-1", 1, 10, 20, 1) + + _ = s.storage.Add(tx, batch) + s.storage.Clear(tx) + + batches := s.storage.GetBatches(tx) + s.Empty(batches) + + req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) + s.Nil(req) +} + +func (s *BatchTxStorageTestSuite) TestMultipleTransactions() { + tx1 := newMockTransactionWrapper("session-1", "tx-1") + tx2 := newMockTransactionWrapper("session-2", "tx-2") + s.materializeTx(tx1) + s.materializeTx(tx2) + batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) + batch2 := s.createTestBatch("topic-2", 2, 30, 40, 2) + + _ = s.storage.Add(tx1, batch1) + _ = s.storage.Add(tx2, batch2) + + batches1 := s.storage.GetBatches(tx1) + s.Len(batches1, 1) + s.Equal(batch1, batches1[0]) + + batches2 := s.storage.GetBatches(tx2) + s.Len(batches2, 1) + s.Equal(batch2, batches2[0]) + + s.storage.Clear(tx1) + + batches1 = s.storage.GetBatches(tx1) + s.Empty(batches1) + + batches2 = s.storage.GetBatches(tx2) + s.Len(batches2, 1) + s.Equal(batch2, batches2[0]) +} + +// Helper methods + +func (s *BatchTxStorageTestSuite) createTestBatch(topic string, partitionID int64, startOffset, endOffset int64, sessionID int) *topicreadercommon.PublicBatch { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + session := topicreadercommon.NewPartitionSession( + ctx, + topic, + partitionID, + 1, + "connection-1", + rawtopicreader.PartitionSessionID(sessionID), + int64(sessionID), + rawtopiccommon.Offset(0), + ) + + batch, err := topicreadercommon.NewBatch(session, nil) + s.Require().NoError(err) + + commitRange := topicreadercommon.CommitRange{ + CommitOffsetStart: rawtopiccommon.Offset(startOffset), + CommitOffsetEnd: rawtopiccommon.Offset(endOffset), + PartitionSession: session, + } + + return topicreadercommon.BatchSetCommitRangeForTest(batch, commitRange) +} diff --git a/internal/topic/topicreaderinternal/stream_reader_impl.go b/internal/topic/topicreaderinternal/stream_reader_impl.go index 931418afd..8eee838d2 100644 --- a/internal/topic/topicreaderinternal/stream_reader_impl.go +++ b/internal/topic/topicreaderinternal/stream_reader_impl.go @@ -62,7 +62,7 @@ type topicStreamReaderImpl struct { started bool closed bool - batchTxStorage BatchTxStorage + batchTxStorage *batchTxStorage } type topicStreamReaderConfig struct { @@ -175,6 +175,8 @@ func newTopicStreamReaderStopped( res.committer.BufferCountTrigger = cfg.CommitterBatchCounterTrigger res.freeBytes <- cfg.BufferSizeProtoBytes + res.batchTxStorage = newBatchTxStorage(cfg.Consumer) + return res } diff --git a/internal/topic/topicreaderinternal/topics_whatever.go b/internal/topic/topicreaderinternal/topics_whatever.go index 871c7413b..0ae07c76e 100644 --- a/internal/topic/topicreaderinternal/topics_whatever.go +++ b/internal/topic/topicreaderinternal/topics_whatever.go @@ -1,14 +1 @@ package topicreaderinternal - -import ( - "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic" - "github.com/ydb-platform/ydb-go-sdk/v3/internal/topic/topicreadercommon" - "github.com/ydb-platform/ydb-go-sdk/v3/internal/tx" -) - -type BatchTxStorage interface { - Add(transaction tx.Transaction, batch *topicreadercommon.PublicBatch) (txAlreadyExists bool) - GetUpdateOffsetsInTransactionRequest(transaction tx.Transaction) *rawtopic.UpdateOffsetsInTransactionRequest - GetBatches(transaction tx.Transaction) []*topicreadercommon.PublicBatch - Clear(transaction tx.Transaction) -} From b3ecc8ad9db3b6751d9e337c7fa55a9b06fdcc4d Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Thu, 4 Dec 2025 17:25:30 +0300 Subject: [PATCH 06/27] Delete internal/topic/topicreaderinternal/topics_whatever.go --- internal/topic/topicreaderinternal/topics_whatever.go | 1 - 1 file changed, 1 deletion(-) delete mode 100644 internal/topic/topicreaderinternal/topics_whatever.go diff --git a/internal/topic/topicreaderinternal/topics_whatever.go b/internal/topic/topicreaderinternal/topics_whatever.go deleted file mode 100644 index 0ae07c76e..000000000 --- a/internal/topic/topicreaderinternal/topics_whatever.go +++ /dev/null @@ -1 +0,0 @@ -package topicreaderinternal From 992c19820721b5067f761f0c606bd8c00d356ba9 Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Thu, 4 Dec 2025 17:26:01 +0300 Subject: [PATCH 07/27] Delete internal/topic/topicreaderinternal/transaction_commits.go --- .../transaction_commits.go | 217 ------------------ 1 file changed, 217 deletions(-) delete mode 100644 internal/topic/topicreaderinternal/transaction_commits.go diff --git a/internal/topic/topicreaderinternal/transaction_commits.go b/internal/topic/topicreaderinternal/transaction_commits.go deleted file mode 100644 index f0f8d42e5..000000000 --- a/internal/topic/topicreaderinternal/transaction_commits.go +++ /dev/null @@ -1,217 +0,0 @@ -package topicreaderinternal - -import ( - "context" - "fmt" - "sync" - - "github.com/ydb-platform/ydb-go-sdk/v3/internal/endpoint" - "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic" - "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic/rawtopiccommon" - "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawydb" - "github.com/ydb-platform/ydb-go-sdk/v3/internal/operation" - "github.com/ydb-platform/ydb-go-sdk/v3/internal/topic/topicreadercommon" - "github.com/ydb-platform/ydb-go-sdk/v3/internal/tx" - "github.com/ydb-platform/ydb-go-sdk/v3/internal/xcontext" - "github.com/ydb-platform/ydb-go-sdk/v3/internal/xerrors" - "github.com/ydb-platform/ydb-go-sdk/v3/internal/xsync" - "github.com/ydb-platform/ydb-go-sdk/v3/trace" -) - -type TransactionCommitsStorage struct { - commits map[string]*transactionCommits // by `tx.ID` - - m xsync.RWMutex - - reader *topicStreamReaderImpl -} - -func (s *TransactionCommitsStorage) Append(tx tx.Transaction, batch *topicreadercommon.PublicBatch) { - tc := s.getTransactionCommits(tx) - tc.AppendBatch(batch) -} - -func (s *TransactionCommitsStorage) getTransactionCommits(transaction tx.Transaction) *transactionCommits { - s.m.Lock() - defer s.m.Unlock() - - tc, ok := s.commits[transaction.ID()] - if !ok { - tc = s.newTransactionCommits(transaction) - s.commits[transaction.ID()] = tc - } - - return tc -} - -func (s *TransactionCommitsStorage) newTransactionCommits(transaction tx.Transaction) *transactionCommits { - ranges := topicreadercommon.NewCommitRangesWithCapacity(1) - - return &transactionCommits{ - tx: transaction, - commitRanges: &ranges, - reader: s.reader, - } -} - -// transactionCommits manages commit ranges for transactions -type transactionCommits struct { - tx tx.Transaction - commitRanges *topicreadercommon.CommitRanges - - beforeCommitOnce sync.Once - completedOnce sync.Once - - reader *topicStreamReaderImpl -} - -// AppendBatch adds a batch to the commit ranges -func (tc *transactionCommits) AppendBatch(batch *topicreadercommon.PublicBatch) { - tc.commitRanges.Append(batch) -} - -// BeforeCommitFn returns a function that will be called before transaction commit -// This function is guaranteed to be called only once per transaction using sync.Once -func (s *TransactionCommitsStorage) BeforeCommitFn(tx tx.Transaction) tx.OnTransactionBeforeCommit { - return func(ctx context.Context) (err error) { - s.m.RLock() - tc, ok := s.commits[tx.ID()] - s.m.RUnlock() - - if !ok { - return - } - - // Ensure this function is called only once - var callErr error - tc.beforeCommitOnce.Do(func() { - callErr = tc.updateOffsetsInTransaction(ctx) - }) - - return callErr - } -} - -func (s *TransactionCommitsStorage) CompletedFn(tx tx.Transaction) tx.OnTransactionCompletedFunc { - return func(err error) { - s.m.RLock() - tc, ok := s.commits[tx.ID()] - s.m.RUnlock() - - if !ok { - return - } - - tc.completedOnce.Do(func() { - defer func() { - s.m.Lock() - defer s.m.Unlock() - delete(s.commits, tx.ID()) - }() - - tc.onTransactionCompleted(err) - }) - } -} - -// updateOffsetsInTransaction updates offsets in the transaction -func (tc *transactionCommits) updateOffsetsInTransaction(ctx context.Context) (err error) { - cfg := tc.reader.cfg - logCtx := cfg.BaseContext - onDone := trace.TopicOnReaderUpdateOffsetsInTransaction( - cfg.Trace, - &logCtx, - tc.reader.readerID, - tc.reader.readConnectionID, - tc.tx.SessionID(), - tc.tx, - ) - - defer func() { - onDone(err) - }() - - // UpdateOffsetsInTransaction operation must be executed on the same Node where the transaction was initiated. - // Otherwise, errors such as `Database coordinators are unavailable` may occur. - ctx = endpoint.WithNodeID(ctx, tc.tx.NodeID()) - - req := tc.commitRangesToUpdateOffsetsRequest() - - err = tc.reader.topicClient.UpdateOffsetsInTransaction(ctx, req) - if err != nil { - // mark error as retryable - all `UpdateOffsetsInTransaction` failures can be retried - err = xerrors.Retryable(err) - - return xerrors.WithStackTrace(fmt.Errorf("updating offsets in transaction: %w", err)) - } - - return nil -} - -// onTransactionCompleted handles transaction completion -func (tc *transactionCommits) onTransactionCompleted(err error) { - if err != nil { - _ = tc.reader.CloseWithError(xcontext.ValueOnly(tc.reader.cfg.BaseContext), fmt.Errorf("transaction failed: %w", err)) - - return - } - - tc.commitRanges.Optimize() - - // Update committed offsets for all partition sessions - for i := range tc.commitRanges.Ranges { - tc.commitRanges.Optimize() - commitRange := &tc.commitRanges.Ranges[i] - commitRange.PartitionSession.SetCommittedOffsetForward(commitRange.CommitOffsetEnd) - } -} - -// commitRangesToUpdateOffsetsRequest converts commit ranges to UpdateOffsetsInTransaction request -func (tc *transactionCommits) commitRangesToUpdateOffsetsRequest() *rawtopic.UpdateOffsetsInTransactionRequest { - req := &rawtopic.UpdateOffsetsInTransactionRequest{ - OperationParams: rawydb.NewRawOperationParamsFromProto( - operation.Params(context.Background(), 0, 0, operation.ModeSync), - ), - Tx: rawtopiccommon.TransactionIdentity{ - ID: tc.tx.ID(), - Session: tc.tx.SessionID(), - }, - Consumer: tc.reader.cfg.Consumer, - } - - // Convert commit ranges to topics structure - partitionOffsets := tc.commitRanges.ToPartitionsOffsets() - - // Group partition offsets by topic - topicMap := make(map[string][]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets) - for i := range partitionOffsets { - po := &partitionOffsets[i] - // Find the corresponding partition session to get topic and partition ID - var topic string - var partitionID int64 - for j := range tc.commitRanges.Ranges { - if tc.commitRanges.Ranges[j].PartitionSession.StreamPartitionSessionID == po.PartitionSessionID { - topic = tc.commitRanges.Ranges[j].PartitionSession.Topic - partitionID = tc.commitRanges.Ranges[j].PartitionSession.PartitionID - - break - } - } - - topicMap[topic] = append(topicMap[topic], rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets{ - PartitionID: partitionID, - PartitionOffsets: po.Offsets, - }) - } - - // Convert map to slice - req.Topics = make([]rawtopic.UpdateOffsetsInTransactionRequest_TopicOffsets, 0, len(topicMap)) - for path, partitions := range topicMap { - req.Topics = append(req.Topics, rawtopic.UpdateOffsetsInTransactionRequest_TopicOffsets{ - Path: path, - Partitions: partitions, - }) - } - - return req -} From 51d8346b706169c2216d0ed369e714ddeb72cf76 Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Thu, 4 Dec 2025 17:29:56 +0300 Subject: [PATCH 08/27] add Changelog --- CHANGELOG.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index aae34facd..544f8f4c5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,5 @@ +* Fixed issue with topic offsets update in transactions + ## v3.121.0 * Changed internal pprof label to pyroscope supported format * Added `query.ImplicitTxControl()` transaction control (the same as `query.NoTx()` and `query.EmptyTxControl()`). See more about implicit transactions on [ydb.tech](https://ydb.tech/docs/en/concepts/transactions?version=v25.2#implicit) From 8e5f319f3f0c1f173b24db6a0a32f9f23e1e957d Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Thu, 4 Dec 2025 17:37:04 +0300 Subject: [PATCH 09/27] refactor(batch_tx_storage): remove unnecessary slice copy in GetBatches method --- internal/topic/topicreaderinternal/batch_tx_storage.go | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/internal/topic/topicreaderinternal/batch_tx_storage.go b/internal/topic/topicreaderinternal/batch_tx_storage.go index 2898ab727..702505665 100644 --- a/internal/topic/topicreaderinternal/batch_tx_storage.go +++ b/internal/topic/topicreaderinternal/batch_tx_storage.go @@ -45,7 +45,6 @@ func (s *batchTxStorage) Add(transaction tx.Transaction, batch *topicreadercommo // GetBatches returns all batches stored for the given transaction. // Returns an empty slice (nil) if no batches are stored for the transaction. -// The returned slice is a copy to prevent external modifications. // This method is thread-safe. func (s *batchTxStorage) GetBatches(transaction tx.Transaction) []*topicreadercommon.PublicBatch { s.m.RLock() @@ -56,11 +55,7 @@ func (s *batchTxStorage) GetBatches(transaction tx.Transaction) []*topicreaderco return nil } - // Return a copy to prevent external modifications - result := make([]*topicreadercommon.PublicBatch, len(batches)) - copy(result, batches) - - return result + return batches } // GetUpdateOffsetsInTransactionRequest builds an UpdateOffsetsInTransactionRequest From ab00f95daf08db654351e1c7105e8d480cdf1d07 Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Fri, 5 Dec 2025 14:03:23 +0300 Subject: [PATCH 10/27] add tests --- tests/integration/helpers_test.go | 17 -- .../topic_read_update_offsets_test.go | 161 ++++++++++++++++++ tests/integration/topic_transactions_test.go | 70 -------- 3 files changed, 161 insertions(+), 87 deletions(-) create mode 100644 tests/integration/topic_read_update_offsets_test.go diff --git a/tests/integration/helpers_test.go b/tests/integration/helpers_test.go index 8f26969dc..fdccd8f07 100644 --- a/tests/integration/helpers_test.go +++ b/tests/integration/helpers_test.go @@ -19,12 +19,8 @@ import ( "github.com/stretchr/testify/require" "google.golang.org/grpc" - "github.com/ydb-platform/ydb-go-genproto/Ydb_Query_V1" - "github.com/ydb-platform/ydb-go-genproto/protos/Ydb_Query" "github.com/ydb-platform/ydb-go-sdk/v3" "github.com/ydb-platform/ydb-go-sdk/v3/config" - "github.com/ydb-platform/ydb-go-sdk/v3/internal/balancer" - "github.com/ydb-platform/ydb-go-sdk/v3/internal/conn" "github.com/ydb-platform/ydb-go-sdk/v3/internal/xsql" "github.com/ydb-platform/ydb-go-sdk/v3/internal/xsync" "github.com/ydb-platform/ydb-go-sdk/v3/log" @@ -419,19 +415,6 @@ func (scope *scopeT) TablePath(opts ...func(t *tableNameParams)) string { func (scope *scopeT) DeleteSession(ctx context.Context, sessionID string) { baConfig := config.New( - config.WithEndpoint("localhost:2136"), // TODO: - config.WithDatabase("/local")) - - ba, err := balancer.New(ctx, baConfig, conn.NewPool(ctx, baConfig)) - scope.Require.NoError(err) - - c := Ydb_Query_V1.NewQueryServiceClient(ba) - _, err = c.DeleteSession(ctx, &Ydb_Query.DeleteSessionRequest{ - SessionId: sessionID, - }) - scope.Require.NoError(err) -} - // logger for tests type testLogger struct { test *xtest.SyncedTest diff --git a/tests/integration/topic_read_update_offsets_test.go b/tests/integration/topic_read_update_offsets_test.go new file mode 100644 index 000000000..6300ac400 --- /dev/null +++ b/tests/integration/topic_read_update_offsets_test.go @@ -0,0 +1,161 @@ +//go:build integration +// +build integration + +package integration + +import ( + "context" + "strings" + "sync" + "testing" + "time" + + "github.com/stretchr/testify/suite" + "github.com/ydb-platform/ydb-go-genproto/Ydb_Query_V1" + "github.com/ydb-platform/ydb-go-genproto/protos/Ydb_Query" + "github.com/ydb-platform/ydb-go-sdk/v3" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/tx" + "github.com/ydb-platform/ydb-go-sdk/v3/query" + "github.com/ydb-platform/ydb-go-sdk/v3/topic/topicreader" + "github.com/ydb-platform/ydb-go-sdk/v3/topic/topicsugar" + "github.com/ydb-platform/ydb-go-sdk/v3/topic/topicwriter" +) + +type TopicReaderUpdateOffsetsSuite struct { + suite.Suite + + scope *scopeT + writer *topicwriter.Writer + reader *topicreader.Reader + driver *ydb.Driver +} + +func (t *TopicReaderUpdateOffsetsSuite) SetupTest() { + t.scope = newScope(t.T()) + t.writer = t.scope.TopicWriter() + t.reader = t.scope.TopicReader() + t.driver = t.scope.DriverWithGRPCLogging() +} + +func TestBatchTxStorage(t *testing.T) { + suite.Run(t, new(TopicReaderUpdateOffsetsSuite)) +} + +func (t *TopicReaderUpdateOffsetsSuite) TestSingleTransaction() { + var ( + once sync.Once + batch *topicreader.Batch + ) + + ctx, cancel := context.WithTimeout(t.scope.Ctx, 10*time.Second) + defer cancel() + + t.writeMessage(ctx, "1") + + err := t.driver.Query().DoTx(ctx, func(ctx context.Context, tr query.TxActor) (err error) { + once.Do(func() { + t.deleteTxSession(ctx, tr.(tx.Transaction)) + }) + + batch, err = t.reader.PopMessagesBatchTx(ctx, tr) + return err + }) + t.NoError(err) + t.Len(batch.Messages, 1) + t.MsgEqualString("1", batch.Messages[0]) +} + +func (t *TopicReaderUpdateOffsetsSuite) TestSeveralReads() { + ctx, cancel := context.WithTimeout(t.scope.Ctx, 10*time.Second) + defer cancel() + + t.writeMessage(ctx, "1") + + err := t.driver.Query().DoTx(ctx, func(ctx context.Context, tr query.TxActor) error { + _, err := t.reader.PopMessagesBatchTx(ctx, tr) + t.Require().NoError(err) + + t.writeMessage(ctx, "2") + + _, err = t.reader.PopMessagesBatchTx(ctx, tr) + t.Require().NoError(err) + + t.writeMessage(ctx, "3") + + return nil + }) + + msg, err := t.reader.ReadMessage(ctx) + t.Require().NoError(err) + + t.MsgEqualString("3", msg) +} + +func (t *TopicReaderUpdateOffsetsSuite) TestSeveralTransactions() { + var ( + onceTx1 sync.Once + onceTx2 sync.Once + batch *topicreader.Batch + ) + + ctx, cancel := context.WithTimeout(t.scope.Ctx, 10*time.Second) + defer cancel() + + t.writeMessage(ctx, "1") + + err := t.driver.Query().DoTx(ctx, func(ctx context.Context, tr query.TxActor) (err error) { + onceTx1.Do(func() { + t.deleteTxSession(ctx, tr.(tx.Transaction)) + }) + + _, err = t.reader.PopMessagesBatchTx(ctx, tr) + return err + }) + t.NoError(err) + + t.writeMessage(ctx, "2") + + err = t.driver.Query().DoTx(ctx, func(ctx context.Context, tr query.TxActor) (err error) { + onceTx2.Do(func() { + t.deleteTxSession(ctx, tr.(tx.Transaction)) + }) + + batch, err = t.reader.PopMessagesBatchTx(ctx, tr) + return err + }) + t.NoError(err) + + t.MsgEqualString("2", batch.Messages[0]) +} + +// Helper methods + +func (t *TopicReaderUpdateOffsetsSuite) writeMessage(ctx context.Context, msg string) { + err := t.writer.Write(ctx, topicwriter.Message{Data: strings.NewReader(msg)}) + t.NoError(err) +} + +func (t *TopicReaderUpdateOffsetsSuite) MsgEqualString(expected string, msg *topicreader.Message) { + t.T().Helper() + + var actual string + + topicsugar.ReadMessageDataWithCallback(msg, func(data []byte) error { + actual = string(data) + return nil + }) + + t.Equal(expected, actual) +} + +func (t *TopicReaderUpdateOffsetsSuite) deleteTxSession(ctx context.Context, tx tx.Transaction) { + t.deleteSession(ctx, tx.SessionID()) +} + +func (t *TopicReaderUpdateOffsetsSuite) deleteSession(ctx context.Context, sessionID string) { + _, err := Ydb_Query_V1.NewQueryServiceClient(ydb.GRPCConn(t.driver)). + DeleteSession(ctx, &Ydb_Query.DeleteSessionRequest{ + SessionId: sessionID, + }) + t.NoError(err) +} diff --git a/tests/integration/topic_transactions_test.go b/tests/integration/topic_transactions_test.go index bb1cafa51..a3be5b670 100644 --- a/tests/integration/topic_transactions_test.go +++ b/tests/integration/topic_transactions_test.go @@ -8,7 +8,6 @@ import ( "errors" "fmt" "io" - "log" "os" "strconv" "strings" @@ -120,75 +119,6 @@ func TestTopicReaderTLIIssue1797(t *testing.T) { })) } -func TestTopicReaderUpdateOffsetsIssue(t *testing.T) { - scope := newScope(t) - ctx, cancel := context.WithTimeout(scope.Ctx, 10*time.Second) - defer cancel() - - db := scope.Driver() - - writer := scope.TopicWriter() - reader := scope.TopicReader() - - scope.Require.NoError(writer.Write(ctx, topicwriter.Message{Data: strings.NewReader("1")})) - - sessionIDAtStart := "" - scope.Require.NoError(db.Query().DoTx(ctx, func(ctx context.Context, tx query.TxActor) error { - sss := tx.(interface{ SessionID() string }) - if sessionIDAtStart == "" { - sessionIDAtStart = sss.SessionID() - - scope.DeleteSession(ctx, sessionIDAtStart) - } - - batch, err := reader.PopMessagesBatchTx(ctx, tx) - scope.Require.NoError(err) - scope.Require.Len(batch.Messages, 1) - - return nil - })) - - t.Run("several reads", func(t *testing.T) { - scope.Require.NoError(db.Query().DoTx(ctx, func(ctx context.Context, tx query.TxActor) error { - sss := tx.(interface{ SessionID() string }) - if sessionIDAtStart == "" { - sessionIDAtStart = sss.SessionID() - - scope.DeleteSession(ctx, sessionIDAtStart) - } - - batch, err := reader.PopMessagesBatchTx(ctx, tx) - scope.Require.NoError(err) - scope.Require.Len(batch.Messages, 1) - - scope.Require.NoError(writer.Write(ctx, topicwriter.Message{Data: strings.NewReader("2")})) - - return nil - })) - - scope.Require.NoError(writer.Write(ctx, topicwriter.Message{Data: strings.NewReader("3")})) - - scope.Require.NoError(db.Query().DoTx(ctx, func(ctx context.Context, tx query.TxActor) error { - sss := tx.(interface{ SessionID() string }) - if sessionIDAtStart == "" { - sessionIDAtStart = sss.SessionID() - - scope.DeleteSession(ctx, sessionIDAtStart) - } - - batch, err := reader.PopMessagesBatchTx(ctx, tx) - scope.Require.NoError(err) - scope.Require.Len(batch.Messages, 1) - - log.Println(batch.Messages[0].Offset) - - //scope.Require.NoError(writer.Write(ctx, topicwriter.Message{Data: strings.NewReader("2")})) - - return nil - })) - }) -} - func TestTopicWriterTLI(t *testing.T) { scope := newScope(t) ctx := scope.Ctx From 735b5dbcf9096545b754aa47e5ac6687afb8b2ff Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Fri, 5 Dec 2025 14:42:07 +0300 Subject: [PATCH 11/27] fix lint --- internal/topic/topicreaderinternal/batch_tx_storage.go | 1 + internal/topic/topicreaderinternal/batch_tx_storage_test.go | 1 + 2 files changed, 2 insertions(+) diff --git a/internal/topic/topicreaderinternal/batch_tx_storage.go b/internal/topic/topicreaderinternal/batch_tx_storage.go index 702505665..fec760126 100644 --- a/internal/topic/topicreaderinternal/batch_tx_storage.go +++ b/internal/topic/topicreaderinternal/batch_tx_storage.go @@ -101,6 +101,7 @@ func (s *batchTxStorage) GetUpdateOffsetsInTransactionRequest(transaction tx.Tra if commitRange.PartitionSession.StreamPartitionSessionID == po.PartitionSessionID { topic = commitRange.PartitionSession.Topic partitionID = commitRange.PartitionSession.PartitionID + break } } diff --git a/internal/topic/topicreaderinternal/batch_tx_storage_test.go b/internal/topic/topicreaderinternal/batch_tx_storage_test.go index c6bf63bd6..13035836a 100644 --- a/internal/topic/topicreaderinternal/batch_tx_storage_test.go +++ b/internal/topic/topicreaderinternal/batch_tx_storage_test.go @@ -14,6 +14,7 @@ import ( type BatchTxStorageTestSuite struct { suite.Suite + storage *batchTxStorage } From af24e39085870881f947863c3e003e31ccfeb8e7 Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Fri, 5 Dec 2025 14:44:52 +0300 Subject: [PATCH 12/27] fix --- tests/integration/helpers_test.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/integration/helpers_test.go b/tests/integration/helpers_test.go index fdccd8f07..b6e22bc1b 100644 --- a/tests/integration/helpers_test.go +++ b/tests/integration/helpers_test.go @@ -413,8 +413,6 @@ func (scope *scopeT) TablePath(opts ...func(t *tableNameParams)) string { return path.Join(scope.Folder(), scope.TableName(opts...)) } -func (scope *scopeT) DeleteSession(ctx context.Context, sessionID string) { - baConfig := config.New( // logger for tests type testLogger struct { test *xtest.SyncedTest From 0b64e625cfe6e00d936c936eca416515cf4c2e83 Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Fri, 5 Dec 2025 15:47:51 +0300 Subject: [PATCH 13/27] =?UTF-8?q?fix:=20=D0=B8=D1=81=D0=BF=D1=80=D0=B0?= =?UTF-8?q?=D0=B2=D0=BB=D0=B5=D0=BD=D1=8B=20=D0=BE=D1=88=D0=B8=D0=B1=D0=BA?= =?UTF-8?q?=D0=B8=20=D0=BB=D0=B8=D0=BD=D1=82=D0=B5=D1=80=D0=B0=20(funlen,?= =?UTF-8?q?=20lll)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../topicreaderinternal/batch_tx_storage.go | 28 +++++++++++++++++-- .../batch_tx_storage_test.go | 7 ++++- 2 files changed, 32 insertions(+), 3 deletions(-) diff --git a/internal/topic/topicreaderinternal/batch_tx_storage.go b/internal/topic/topicreaderinternal/batch_tx_storage.go index fec760126..9956a3db4 100644 --- a/internal/topic/topicreaderinternal/batch_tx_storage.go +++ b/internal/topic/topicreaderinternal/batch_tx_storage.go @@ -5,6 +5,7 @@ import ( "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic" "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic/rawtopiccommon" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic/rawtopicreader" "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawydb" "github.com/ydb-platform/ydb-go-sdk/v3/internal/operation" "github.com/ydb-platform/ydb-go-sdk/v3/internal/topic/topicreadercommon" @@ -64,7 +65,9 @@ func (s *batchTxStorage) GetBatches(transaction tx.Transaction) []*topicreaderco // and grouped by topic and partition. // Returns nil if no batches are stored for the transaction. // This method is thread-safe. -func (s *batchTxStorage) GetUpdateOffsetsInTransactionRequest(transaction tx.Transaction) *rawtopic.UpdateOffsetsInTransactionRequest { +func (s *batchTxStorage) GetUpdateOffsetsInTransactionRequest( + transaction tx.Transaction, +) *rawtopic.UpdateOffsetsInTransactionRequest { s.m.RLock() batches, ok := s.batches[transaction.ID()] s.m.RUnlock() @@ -90,6 +93,20 @@ func (s *batchTxStorage) GetUpdateOffsetsInTransactionRequest(transaction tx.Tra } // Group partition offsets by topic + topicMap := s.buildPartitionOffsetsMap(partitionOffsets, batches) + if len(topicMap) == 0 { + return nil + } + + // Build request + return s.buildUpdateOffsetsRequest(transaction, topicMap) +} + +// buildPartitionOffsetsMap groups partition offsets by topic. +func (s *batchTxStorage) buildPartitionOffsetsMap( + partitionOffsets []rawtopicreader.PartitionCommitOffset, + batches []*topicreadercommon.PublicBatch, +) map[string][]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets { topicMap := make(map[string][]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets) for i := range partitionOffsets { po := &partitionOffsets[i] @@ -112,7 +129,14 @@ func (s *batchTxStorage) GetUpdateOffsetsInTransactionRequest(transaction tx.Tra }) } - // Build request + return topicMap +} + +// buildUpdateOffsetsRequest creates the final UpdateOffsetsInTransactionRequest. +func (s *batchTxStorage) buildUpdateOffsetsRequest( + transaction tx.Transaction, + topicMap map[string][]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets, +) *rawtopic.UpdateOffsetsInTransactionRequest { req := &rawtopic.UpdateOffsetsInTransactionRequest{ OperationParams: rawydb.NewRawOperationParamsFromProto( operation.Params(context.Background(), 0, 0, operation.ModeSync), diff --git a/internal/topic/topicreaderinternal/batch_tx_storage_test.go b/internal/topic/topicreaderinternal/batch_tx_storage_test.go index 13035836a..c0750601e 100644 --- a/internal/topic/topicreaderinternal/batch_tx_storage_test.go +++ b/internal/topic/topicreaderinternal/batch_tx_storage_test.go @@ -374,7 +374,12 @@ func (s *BatchTxStorageTestSuite) TestMultipleTransactions() { // Helper methods -func (s *BatchTxStorageTestSuite) createTestBatch(topic string, partitionID int64, startOffset, endOffset int64, sessionID int) *topicreadercommon.PublicBatch { +func (s *BatchTxStorageTestSuite) createTestBatch( + topic string, + partitionID int64, + startOffset, endOffset int64, + sessionID int, +) *topicreadercommon.PublicBatch { ctx, cancel := context.WithCancel(context.Background()) defer cancel() From b2d3bf104c0501efb52580a613bffca28e5a0f1c Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Fri, 5 Dec 2025 18:11:46 +0300 Subject: [PATCH 14/27] refactor: optimize batch_tx_storage and improve tests - Optimize buildPartitionOffsetsMap: use sessionInfo map for O(1) lookup instead of O(n*m) search - Refactor tests: add helper methods, remove duplication, improve structure - Remove Given-When-Then comments as requested - Fix linter issues: line length, return statements, formatting --- .../topicreaderinternal/batch_tx_storage.go | 50 +++-- .../batch_tx_storage_test.go | 211 ++++++++---------- .../stream_reader_impl_test.go | 6 +- .../topic_read_update_offsets_test.go | 61 ++--- 4 files changed, 158 insertions(+), 170 deletions(-) diff --git a/internal/topic/topicreaderinternal/batch_tx_storage.go b/internal/topic/topicreaderinternal/batch_tx_storage.go index 9956a3db4..770b589e0 100644 --- a/internal/topic/topicreaderinternal/batch_tx_storage.go +++ b/internal/topic/topicreaderinternal/batch_tx_storage.go @@ -86,6 +86,9 @@ func (s *batchTxStorage) GetUpdateOffsetsInTransactionRequest( // Optimize ranges (merge adjacent ranges) commitRanges.Optimize() + // Build sessionID -> (topic, partitionID) map for efficient lookup + sessionInfoMap := s.buildSessionInfoMap(batches) + // Convert to partition offsets partitionOffsets := commitRanges.ToPartitionsOffsets() if len(partitionOffsets) == 0 { @@ -93,7 +96,7 @@ func (s *batchTxStorage) GetUpdateOffsetsInTransactionRequest( } // Group partition offsets by topic - topicMap := s.buildPartitionOffsetsMap(partitionOffsets, batches) + topicMap := s.buildPartitionOffsetsMap(partitionOffsets, sessionInfoMap) if len(topicMap) == 0 { return nil } @@ -102,29 +105,46 @@ func (s *batchTxStorage) GetUpdateOffsetsInTransactionRequest( return s.buildUpdateOffsetsRequest(transaction, topicMap) } +type sessionInfo struct { + topic string + partitionID int64 +} + +// buildSessionInfoMap creates a map from partition session ID to topic and partition ID. +func (s *batchTxStorage) buildSessionInfoMap( + batches []*topicreadercommon.PublicBatch, +) map[rawtopicreader.PartitionSessionID]sessionInfo { + sessionInfoMap := make(map[rawtopicreader.PartitionSessionID]sessionInfo) + for _, batch := range batches { + commitRange := topicreadercommon.GetCommitRange(batch) + sessionID := commitRange.PartitionSession.StreamPartitionSessionID + if _, exists := sessionInfoMap[sessionID]; !exists { + sessionInfoMap[sessionID] = sessionInfo{ + topic: commitRange.PartitionSession.Topic, + partitionID: commitRange.PartitionSession.PartitionID, + } + } + } + + return sessionInfoMap +} + // buildPartitionOffsetsMap groups partition offsets by topic. func (s *batchTxStorage) buildPartitionOffsetsMap( partitionOffsets []rawtopicreader.PartitionCommitOffset, - batches []*topicreadercommon.PublicBatch, + sessionInfoMap map[rawtopicreader.PartitionSessionID]sessionInfo, ) map[string][]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets { topicMap := make(map[string][]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets) for i := range partitionOffsets { po := &partitionOffsets[i] - // Find the corresponding partition session to get topic and partition ID - var topic string - var partitionID int64 - for _, batch := range batches { - commitRange := topicreadercommon.GetCommitRange(batch) - if commitRange.PartitionSession.StreamPartitionSessionID == po.PartitionSessionID { - topic = commitRange.PartitionSession.Topic - partitionID = commitRange.PartitionSession.PartitionID - - break - } + info, ok := sessionInfoMap[po.PartitionSessionID] + if !ok { + // Skip if session info not found (should not happen in normal flow) + continue } - topicMap[topic] = append(topicMap[topic], rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets{ - PartitionID: partitionID, + topicMap[info.topic] = append(topicMap[info.topic], rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets{ + PartitionID: info.partitionID, PartitionOffsets: po.Offsets, }) } diff --git a/internal/topic/topicreaderinternal/batch_tx_storage_test.go b/internal/topic/topicreaderinternal/batch_tx_storage_test.go index c0750601e..77776d5be 100644 --- a/internal/topic/topicreaderinternal/batch_tx_storage_test.go +++ b/internal/topic/topicreaderinternal/batch_tx_storage_test.go @@ -22,8 +22,19 @@ func (s *BatchTxStorageTestSuite) SetupTest() { s.storage = newBatchTxStorage("test-consumer") } -func (s *BatchTxStorageTestSuite) materializeTx(tx *mockTransaction) { +// Helper methods for test setup + +func (s *BatchTxStorageTestSuite) newTransaction(sessionID, txID string) *mockTransaction { + tx := newMockTransactionWrapper(sessionID, txID) _ = tx.UnLazy(context.Background()) + + return tx +} + +func (s *BatchTxStorageTestSuite) addBatches(tx *mockTransaction, batches ...*topicreadercommon.PublicBatch) { + for _, batch := range batches { + _ = s.storage.Add(tx, batch) + } } func TestBatchTxStorage(t *testing.T) { @@ -31,8 +42,7 @@ func TestBatchTxStorage(t *testing.T) { } func (s *BatchTxStorageTestSuite) TestAdd_NewTransaction() { - tx := newMockTransactionWrapper("session-1", "tx-1") - s.materializeTx(tx) + tx := s.newTransaction("session-1", "tx-1") batch := s.createTestBatch("topic-1", 1, 10, 20, 1) exists := s.storage.Add(tx, batch) @@ -41,20 +51,18 @@ func (s *BatchTxStorageTestSuite) TestAdd_NewTransaction() { } func (s *BatchTxStorageTestSuite) TestAdd_ExistingTransaction() { - tx := newMockTransactionWrapper("session-1", "tx-1") - s.materializeTx(tx) + tx := s.newTransaction("session-1", "tx-1") batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) batch2 := s.createTestBatch("topic-1", 1, 20, 30, 1) + s.addBatches(tx, batch1) - _ = s.storage.Add(tx, batch1) exists := s.storage.Add(tx, batch2) s.True(exists) } func (s *BatchTxStorageTestSuite) TestGetBatches_Empty() { - tx := newMockTransactionWrapper("session-1", "tx-1") - s.materializeTx(tx) + tx := s.newTransaction("session-1", "tx-1") batches := s.storage.GetBatches(tx) @@ -62,13 +70,10 @@ func (s *BatchTxStorageTestSuite) TestGetBatches_Empty() { } func (s *BatchTxStorageTestSuite) TestGetBatches_WithBatches() { - tx := newMockTransactionWrapper("session-1", "tx-1") - s.materializeTx(tx) + tx := s.newTransaction("session-1", "tx-1") batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) batch2 := s.createTestBatch("topic-1", 1, 20, 30, 1) - - _ = s.storage.Add(tx, batch1) - _ = s.storage.Add(tx, batch2) + s.addBatches(tx, batch1, batch2) batches := s.storage.GetBatches(tx) @@ -78,8 +83,7 @@ func (s *BatchTxStorageTestSuite) TestGetBatches_WithBatches() { } func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_Empty() { - tx := newMockTransactionWrapper("session-1", "tx-1") - s.materializeTx(tx) + tx := s.newTransaction("session-1", "tx-1") req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) @@ -87,11 +91,10 @@ func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_Empty } func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_SingleBatch() { - tx := newMockTransactionWrapper("session-1", "tx-1") - s.materializeTx(tx) + tx := s.newTransaction("session-1", "tx-1") batch := s.createTestBatch("topic-1", 1, 10, 20, 1) + s.addBatches(tx, batch) - _ = s.storage.Add(tx, batch) req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) s.Require().NotNil(req) @@ -108,53 +111,42 @@ func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_Singl } func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_MultipleBatches() { - tx := newMockTransactionWrapper("session-1", "tx-1") - s.materializeTx(tx) + tx := s.newTransaction("session-1", "tx-1") batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) batch2 := s.createTestBatch("topic-1", 1, 20, 30, 1) + s.addBatches(tx, batch1, batch2) - _ = s.storage.Add(tx, batch1) - _ = s.storage.Add(tx, batch2) req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) s.Require().NotNil(req) s.Len(req.Topics, 1) s.Len(req.Topics[0].Partitions, 1) s.Len(req.Topics[0].Partitions[0].PartitionOffsets, 1) - // Should be optimized to single range s.Equal(rawtopiccommon.Offset(10), req.Topics[0].Partitions[0].PartitionOffsets[0].Start) s.Equal(rawtopiccommon.Offset(30), req.Topics[0].Partitions[0].PartitionOffsets[0].End) } func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_MultipleTopics() { - tx := newMockTransactionWrapper("session-1", "tx-1") - s.materializeTx(tx) + tx := s.newTransaction("session-1", "tx-1") batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) batch2 := s.createTestBatch("topic-2", 2, 30, 40, 2) + s.addBatches(tx, batch1, batch2) - _ = s.storage.Add(tx, batch1) - _ = s.storage.Add(tx, batch2) req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) s.Require().NotNil(req) s.Len(req.Topics, 2) - // Topics should be grouped by path - topicMap := make(map[string]bool) - for _, topic := range req.Topics { - topicMap[topic.Path] = true - } - s.True(topicMap["topic-1"]) - s.True(topicMap["topic-2"]) + topicMap := s.buildTopicMap(req.Topics) + s.Contains(topicMap, "topic-1") + s.Contains(topicMap, "topic-2") } func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_MultiplePartitionsSameTopic() { - tx := newMockTransactionWrapper("session-1", "tx-1") - s.materializeTx(tx) + tx := s.newTransaction("session-1", "tx-1") batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) batch2 := s.createTestBatch("topic-1", 2, 30, 40, 2) + s.addBatches(tx, batch1, batch2) - _ = s.storage.Add(tx, batch1) - _ = s.storage.Add(tx, batch2) req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) s.Require().NotNil(req) @@ -162,12 +154,7 @@ func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_Multi s.Equal("topic-1", req.Topics[0].Path) s.Len(req.Topics[0].Partitions, 2) - // Find partitions - partitionMap := make(map[int64]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets) - for _, partition := range req.Topics[0].Partitions { - partitionMap[partition.PartitionID] = partition - } - + partitionMap := s.buildPartitionMap(req.Topics[0].Partitions) s.Contains(partitionMap, int64(1)) s.Contains(partitionMap, int64(2)) s.Len(partitionMap[1].PartitionOffsets, 1) @@ -179,19 +166,17 @@ func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_Multi } func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_NonAdjacentBatches() { - tx := newMockTransactionWrapper("session-1", "tx-1") - s.materializeTx(tx) + tx := s.newTransaction("session-1", "tx-1") batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) - batch2 := s.createTestBatch("topic-1", 1, 30, 40, 1) // Gap between 20 and 30 + batch2 := s.createTestBatch("topic-1", 1, 30, 40, 1) + s.addBatches(tx, batch1, batch2) - _ = s.storage.Add(tx, batch1) - _ = s.storage.Add(tx, batch2) req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) s.Require().NotNil(req) s.Len(req.Topics, 1) s.Len(req.Topics[0].Partitions, 1) - s.Len(req.Topics[0].Partitions[0].PartitionOffsets, 2) // Should not be merged + s.Len(req.Topics[0].Partitions[0].PartitionOffsets, 2) s.Equal(rawtopiccommon.Offset(10), req.Topics[0].Partitions[0].PartitionOffsets[0].Start) s.Equal(rawtopiccommon.Offset(20), req.Topics[0].Partitions[0].PartitionOffsets[0].End) s.Equal(rawtopiccommon.Offset(30), req.Topics[0].Partitions[0].PartitionOffsets[1].Start) @@ -199,70 +184,46 @@ func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_NonAd } func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_MultiplePartitionsMultipleTopics() { - tx := newMockTransactionWrapper("session-1", "tx-1") - s.materializeTx(tx) + tx := s.newTransaction("session-1", "tx-1") batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) batch2 := s.createTestBatch("topic-1", 2, 30, 40, 2) batch3 := s.createTestBatch("topic-2", 1, 50, 60, 3) batch4 := s.createTestBatch("topic-2", 2, 70, 80, 4) + s.addBatches(tx, batch1, batch2, batch3, batch4) - _ = s.storage.Add(tx, batch1) - _ = s.storage.Add(tx, batch2) - _ = s.storage.Add(tx, batch3) - _ = s.storage.Add(tx, batch4) req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) s.Require().NotNil(req) s.Len(req.Topics, 2) - // Find topics - topicMap := make(map[string]rawtopic.UpdateOffsetsInTransactionRequest_TopicOffsets) - for _, topic := range req.Topics { - topicMap[topic.Path] = topic - } - + topicMap := s.buildTopicOffsetsMap(req.Topics) s.Contains(topicMap, "topic-1") s.Contains(topicMap, "topic-2") s.Len(topicMap["topic-1"].Partitions, 2) s.Len(topicMap["topic-2"].Partitions, 2) - // Check topic-1 partitions - partition1Map := make(map[int64]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets) - for _, partition := range topicMap["topic-1"].Partitions { - partition1Map[partition.PartitionID] = partition - } + partition1Map := s.buildPartitionMap(topicMap["topic-1"].Partitions) s.Contains(partition1Map, int64(1)) s.Contains(partition1Map, int64(2)) - // Check topic-2 partitions - partition2Map := make(map[int64]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets) - for _, partition := range topicMap["topic-2"].Partitions { - partition2Map[partition.PartitionID] = partition - } + partition2Map := s.buildPartitionMap(topicMap["topic-2"].Partitions) s.Contains(partition2Map, int64(1)) s.Contains(partition2Map, int64(2)) } func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_ComplexOptimization() { - tx := newMockTransactionWrapper("session-1", "tx-1") - s.materializeTx(tx) - // Adjacent batches that should be merged + tx := s.newTransaction("session-1", "tx-1") batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) batch2 := s.createTestBatch("topic-1", 1, 20, 30, 1) batch3 := s.createTestBatch("topic-1", 1, 30, 40, 1) - // Non-adjacent batch (should not merge) batch4 := s.createTestBatch("topic-1", 1, 50, 60, 1) + s.addBatches(tx, batch1, batch2, batch3, batch4) - _ = s.storage.Add(tx, batch1) - _ = s.storage.Add(tx, batch2) - _ = s.storage.Add(tx, batch3) - _ = s.storage.Add(tx, batch4) req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) s.Require().NotNil(req) s.Len(req.Topics, 1) s.Len(req.Topics[0].Partitions, 1) - // First three should be merged into one range, fourth is separate s.Len(req.Topics[0].Partitions[0].PartitionOffsets, 2) s.Equal(rawtopiccommon.Offset(10), req.Topics[0].Partitions[0].PartitionOffsets[0].Start) s.Equal(rawtopiccommon.Offset(40), req.Topics[0].Partitions[0].PartitionOffsets[0].End) @@ -271,69 +232,46 @@ func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_Compl } func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_MixedPartitionsAndTopics() { - tx := newMockTransactionWrapper("session-1", "tx-1") - s.materializeTx(tx) - // Topic 1, partition 1 - adjacent batches (should merge) + tx := s.newTransaction("session-1", "tx-1") batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) batch2 := s.createTestBatch("topic-1", 1, 20, 30, 1) - // Topic 1, partition 2 - non-adjacent batches (should not merge) batch3 := s.createTestBatch("topic-1", 2, 40, 50, 2) batch4 := s.createTestBatch("topic-1", 2, 60, 70, 2) - // Topic 2, partition 1 - single batch batch5 := s.createTestBatch("topic-2", 1, 80, 90, 3) + s.addBatches(tx, batch1, batch2, batch3, batch4, batch5) - _ = s.storage.Add(tx, batch1) - _ = s.storage.Add(tx, batch2) - _ = s.storage.Add(tx, batch3) - _ = s.storage.Add(tx, batch4) - _ = s.storage.Add(tx, batch5) req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) s.Require().NotNil(req) s.Len(req.Topics, 2) - // Find topics - topicMap := make(map[string]rawtopic.UpdateOffsetsInTransactionRequest_TopicOffsets) - for _, topic := range req.Topics { - topicMap[topic.Path] = topic - } - - // Check topic-1: should have 2 partitions + topicMap := s.buildTopicOffsetsMap(req.Topics) s.Contains(topicMap, "topic-1") s.Len(topicMap["topic-1"].Partitions, 2) - - // Check topic-2: should have 1 partition s.Contains(topicMap, "topic-2") s.Len(topicMap["topic-2"].Partitions, 1) - // Verify topic-1 partition 1 (merged) - partition1Map := make(map[int64]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets) - for _, partition := range topicMap["topic-1"].Partitions { - partition1Map[partition.PartitionID] = partition - } + partition1Map := s.buildPartitionMap(topicMap["topic-1"].Partitions) s.Len(partition1Map[1].PartitionOffsets, 1) // Merged s.Equal(rawtopiccommon.Offset(10), partition1Map[1].PartitionOffsets[0].Start) s.Equal(rawtopiccommon.Offset(30), partition1Map[1].PartitionOffsets[0].End) - // Verify topic-1 partition 2 (not merged) s.Len(partition1Map[2].PartitionOffsets, 2) // Not merged s.Equal(rawtopiccommon.Offset(40), partition1Map[2].PartitionOffsets[0].Start) s.Equal(rawtopiccommon.Offset(50), partition1Map[2].PartitionOffsets[0].End) s.Equal(rawtopiccommon.Offset(60), partition1Map[2].PartitionOffsets[1].Start) s.Equal(rawtopiccommon.Offset(70), partition1Map[2].PartitionOffsets[1].End) - // Verify topic-2 partition 1 s.Len(topicMap["topic-2"].Partitions[0].PartitionOffsets, 1) s.Equal(rawtopiccommon.Offset(80), topicMap["topic-2"].Partitions[0].PartitionOffsets[0].Start) s.Equal(rawtopiccommon.Offset(90), topicMap["topic-2"].Partitions[0].PartitionOffsets[0].End) } func (s *BatchTxStorageTestSuite) TestClear() { - tx := newMockTransactionWrapper("session-1", "tx-1") - s.materializeTx(tx) + tx := s.newTransaction("session-1", "tx-1") batch := s.createTestBatch("topic-1", 1, 10, 20, 1) + s.addBatches(tx, batch) - _ = s.storage.Add(tx, batch) s.storage.Clear(tx) batches := s.storage.GetBatches(tx) @@ -344,21 +282,18 @@ func (s *BatchTxStorageTestSuite) TestClear() { } func (s *BatchTxStorageTestSuite) TestMultipleTransactions() { - tx1 := newMockTransactionWrapper("session-1", "tx-1") - tx2 := newMockTransactionWrapper("session-2", "tx-2") - s.materializeTx(tx1) - s.materializeTx(tx2) + tx1 := s.newTransaction("session-1", "tx-1") + tx2 := s.newTransaction("session-2", "tx-2") batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) batch2 := s.createTestBatch("topic-2", 2, 30, 40, 2) - - _ = s.storage.Add(tx1, batch1) - _ = s.storage.Add(tx2, batch2) + s.addBatches(tx1, batch1) + s.addBatches(tx2, batch2) batches1 := s.storage.GetBatches(tx1) + batches2 := s.storage.GetBatches(tx2) + s.Len(batches1, 1) s.Equal(batch1, batches1[0]) - - batches2 := s.storage.GetBatches(tx2) s.Len(batches2, 1) s.Equal(batch2, batches2[0]) @@ -366,13 +301,47 @@ func (s *BatchTxStorageTestSuite) TestMultipleTransactions() { batches1 = s.storage.GetBatches(tx1) s.Empty(batches1) - batches2 = s.storage.GetBatches(tx2) s.Len(batches2, 1) s.Equal(batch2, batches2[0]) } -// Helper methods +// Helper methods for assertions + +func (s *BatchTxStorageTestSuite) buildTopicMap( + topics []rawtopic.UpdateOffsetsInTransactionRequest_TopicOffsets, +) map[string]bool { + topicMap := make(map[string]bool) + for _, topic := range topics { + topicMap[topic.Path] = true + } + + return topicMap +} + +func (s *BatchTxStorageTestSuite) buildTopicOffsetsMap( + topics []rawtopic.UpdateOffsetsInTransactionRequest_TopicOffsets, +) map[string]rawtopic.UpdateOffsetsInTransactionRequest_TopicOffsets { + topicMap := make(map[string]rawtopic.UpdateOffsetsInTransactionRequest_TopicOffsets) + for _, topic := range topics { + topicMap[topic.Path] = topic + } + + return topicMap +} + +func (s *BatchTxStorageTestSuite) buildPartitionMap( + partitions []rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets, +) map[int64]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets { + partitionMap := make(map[int64]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets) + for _, partition := range partitions { + partitionMap[partition.PartitionID] = partition + } + + return partitionMap +} + +// Helper methods for test data creation func (s *BatchTxStorageTestSuite) createTestBatch( topic string, diff --git a/internal/topic/topicreaderinternal/stream_reader_impl_test.go b/internal/topic/topicreaderinternal/stream_reader_impl_test.go index 592816748..263df37f6 100644 --- a/internal/topic/topicreaderinternal/stream_reader_impl_test.go +++ b/internal/topic/topicreaderinternal/stream_reader_impl_test.go @@ -1335,7 +1335,6 @@ func TestUpdateCommitInTransaction(t *testing.T) { initialCommitOffset := e.partitionSession.CommittedOffset() txID := "test-tx-id" sessionID := "test-session-id" - txMock := newMockTransactionWrapper(sessionID, txID) batch, err := topicreadercommon.NewBatch(e.partitionSession, []*topicreadercommon.PublicMessage{ @@ -1345,6 +1344,7 @@ func TestUpdateCommitInTransaction(t *testing.T) { Build(), }) require.NoError(t, err) + err = e.reader.commitWithTransaction(e.ctx, txMock, batch) require.NoError(t, err) @@ -1381,12 +1381,10 @@ func TestUpdateCommitInTransaction(t *testing.T) { txID := "test-tx-id" sessionID := "test-session-id" - testError := errors.New("test error") e.TopicClient.EXPECT().UpdateOffsetsInTransaction(gomock.Any(), gomock.Any()).Return(testError) txMock := newMockTransactionWrapper(sessionID, txID) - batch, err := topicreadercommon.NewBatch(e.partitionSession, []*topicreadercommon.PublicMessage{ topicreadercommon.NewPublicMessageBuilder(). Offset(e.partitionSession.CommittedOffset().ToInt64()). @@ -1394,6 +1392,7 @@ func TestUpdateCommitInTransaction(t *testing.T) { Build(), }) require.NoError(t, err) + err = e.reader.commitWithTransaction(e.ctx, txMock, batch) require.NoError(t, err) @@ -1425,7 +1424,6 @@ func TestUpdateCommitInTransaction(t *testing.T) { e.TopicClient.EXPECT().UpdateOffsetsInTransaction(gomock.Any(), gomock.Any()).DoAndReturn( func(ctx context.Context, _ *rawtopic.UpdateOffsetsInTransactionRequest) error { nodeID, ok := endpoint.ContextNodeID(ctx) - require.True(t, ok) require.Equal(t, uint32(123), nodeID) diff --git a/tests/integration/topic_read_update_offsets_test.go b/tests/integration/topic_read_update_offsets_test.go index 6300ac400..968fc0df0 100644 --- a/tests/integration/topic_read_update_offsets_test.go +++ b/tests/integration/topic_read_update_offsets_test.go @@ -41,32 +41,45 @@ func TestBatchTxStorage(t *testing.T) { suite.Run(t, new(TopicReaderUpdateOffsetsSuite)) } -func (t *TopicReaderUpdateOffsetsSuite) TestSingleTransaction() { - var ( - once sync.Once - batch *topicreader.Batch - ) - - ctx, cancel := context.WithTimeout(t.scope.Ctx, 10*time.Second) - defer cancel() +// Helper methods for test setup - t.writeMessage(ctx, "1") +func (t *TopicReaderUpdateOffsetsSuite) testContext() (context.Context, context.CancelFunc) { + return context.WithTimeout(t.scope.Ctx, 10*time.Second) +} - err := t.driver.Query().DoTx(ctx, func(ctx context.Context, tr query.TxActor) (err error) { +func (t *TopicReaderUpdateOffsetsSuite) doTransactionWithDeletedSession( + ctx context.Context, + fn func(ctx context.Context, tr query.TxActor) error, +) error { + var once sync.Once + return t.driver.Query().DoTx(ctx, func(ctx context.Context, tr query.TxActor) error { once.Do(func() { t.deleteTxSession(ctx, tr.(tx.Transaction)) }) + return fn(ctx, tr) + }) +} + +func (t *TopicReaderUpdateOffsetsSuite) TestSingleTransaction() { + ctx, cancel := t.testContext() + defer cancel() + + t.writeMessage(ctx, "1") + var batch *topicreader.Batch + err := t.doTransactionWithDeletedSession(ctx, func(ctx context.Context, tr query.TxActor) error { + var err error batch, err = t.reader.PopMessagesBatchTx(ctx, tr) return err }) + t.NoError(err) t.Len(batch.Messages, 1) t.MsgEqualString("1", batch.Messages[0]) } func (t *TopicReaderUpdateOffsetsSuite) TestSeveralReads() { - ctx, cancel := context.WithTimeout(t.scope.Ctx, 10*time.Second) + ctx, cancel := t.testContext() defer cancel() t.writeMessage(ctx, "1") @@ -84,42 +97,30 @@ func (t *TopicReaderUpdateOffsetsSuite) TestSeveralReads() { return nil }) + t.Require().NoError(err) msg, err := t.reader.ReadMessage(ctx) t.Require().NoError(err) - t.MsgEqualString("3", msg) } func (t *TopicReaderUpdateOffsetsSuite) TestSeveralTransactions() { - var ( - onceTx1 sync.Once - onceTx2 sync.Once - batch *topicreader.Batch - ) - - ctx, cancel := context.WithTimeout(t.scope.Ctx, 10*time.Second) + ctx, cancel := t.testContext() defer cancel() t.writeMessage(ctx, "1") - err := t.driver.Query().DoTx(ctx, func(ctx context.Context, tr query.TxActor) (err error) { - onceTx1.Do(func() { - t.deleteTxSession(ctx, tr.(tx.Transaction)) - }) - - _, err = t.reader.PopMessagesBatchTx(ctx, tr) + err := t.doTransactionWithDeletedSession(ctx, func(ctx context.Context, tr query.TxActor) error { + _, err := t.reader.PopMessagesBatchTx(ctx, tr) return err }) t.NoError(err) t.writeMessage(ctx, "2") - err = t.driver.Query().DoTx(ctx, func(ctx context.Context, tr query.TxActor) (err error) { - onceTx2.Do(func() { - t.deleteTxSession(ctx, tr.(tx.Transaction)) - }) - + var batch *topicreader.Batch + err = t.doTransactionWithDeletedSession(ctx, func(ctx context.Context, tr query.TxActor) error { + var err error batch, err = t.reader.PopMessagesBatchTx(ctx, tr) return err }) From 17fc8a55b7d8a1da30d7a6f22812221fae9558fb Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Mon, 8 Dec 2025 13:26:00 +0300 Subject: [PATCH 15/27] fix linter --- tests/integration/topic_read_update_offsets_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/topic_read_update_offsets_test.go b/tests/integration/topic_read_update_offsets_test.go index 968fc0df0..eedac958c 100644 --- a/tests/integration/topic_read_update_offsets_test.go +++ b/tests/integration/topic_read_update_offsets_test.go @@ -13,6 +13,7 @@ import ( "github.com/stretchr/testify/suite" "github.com/ydb-platform/ydb-go-genproto/Ydb_Query_V1" "github.com/ydb-platform/ydb-go-genproto/protos/Ydb_Query" + "github.com/ydb-platform/ydb-go-sdk/v3" "github.com/ydb-platform/ydb-go-sdk/v3/internal/tx" "github.com/ydb-platform/ydb-go-sdk/v3/query" From 7b45dbef18f1b19454bdadcbf5a389e8374dd9f0 Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Mon, 8 Dec 2025 13:37:53 +0300 Subject: [PATCH 16/27] fix test --- .../stream_reader_impl_test.go | 66 +++++++++++-------- 1 file changed, 37 insertions(+), 29 deletions(-) diff --git a/internal/topic/topicreaderinternal/stream_reader_impl_test.go b/internal/topic/topicreaderinternal/stream_reader_impl_test.go index 263df37f6..f56b4d16b 100644 --- a/internal/topic/topicreaderinternal/stream_reader_impl_test.go +++ b/internal/topic/topicreaderinternal/stream_reader_impl_test.go @@ -1335,6 +1335,34 @@ func TestUpdateCommitInTransaction(t *testing.T) { initialCommitOffset := e.partitionSession.CommittedOffset() txID := "test-tx-id" sessionID := "test-session-id" + + e.TopicClient.EXPECT().UpdateOffsetsInTransaction(gomock.Any(), &rawtopic.UpdateOffsetsInTransactionRequest{ + OperationParams: rawydb.OperationParams{ + OperationMode: rawydb.OperationParamsModeSync, + }, + Tx: rawtopiccommon.TransactionIdentity{ + ID: txID, + Session: sessionID, + }, + Topics: []rawtopic.UpdateOffsetsInTransactionRequest_TopicOffsets{ + { + Path: e.partitionSession.Topic, + Partitions: []rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets{ + { + PartitionID: e.partitionSession.PartitionID, + PartitionOffsets: []rawtopiccommon.OffsetRange{ + { + Start: initialCommitOffset, + End: initialCommitOffset + 1, + }, + }, + }, + }, + }, + }, + Consumer: e.reader.cfg.Consumer, + }) + txMock := newMockTransactionWrapper(sessionID, txID) batch, err := topicreadercommon.NewBatch(e.partitionSession, []*topicreadercommon.PublicMessage{ @@ -1344,29 +1372,8 @@ func TestUpdateCommitInTransaction(t *testing.T) { Build(), }) require.NoError(t, err) - err = e.reader.commitWithTransaction(e.ctx, txMock, batch) require.NoError(t, err) - - e.TopicClient.EXPECT().UpdateOffsetsInTransaction(gomock.Any(), gomock.Cond(func(x any) bool { - req, ok := x.(*rawtopic.UpdateOffsetsInTransactionRequest) - if !ok { - return false - } - - return req.OperationParams.OperationMode == rawydb.OperationParamsModeSync && - req.Tx.ID == txID && - req.Tx.Session == sessionID && - req.Consumer == e.reader.cfg.Consumer && - len(req.Topics) == 1 && - req.Topics[0].Path == e.partitionSession.Topic && - len(req.Topics[0].Partitions) == 1 && - req.Topics[0].Partitions[0].PartitionID == e.partitionSession.PartitionID && - len(req.Topics[0].Partitions[0].PartitionOffsets) == 1 && - req.Topics[0].Partitions[0].PartitionOffsets[0].Start == initialCommitOffset && - req.Topics[0].Partitions[0].PartitionOffsets[0].End == initialCommitOffset+1 - })) - err = txMock.onBeforeCommit[0](e.ctx) require.NoError(t, err) @@ -1381,10 +1388,12 @@ func TestUpdateCommitInTransaction(t *testing.T) { txID := "test-tx-id" sessionID := "test-session-id" + testError := errors.New("test error") e.TopicClient.EXPECT().UpdateOffsetsInTransaction(gomock.Any(), gomock.Any()).Return(testError) txMock := newMockTransactionWrapper(sessionID, txID) + batch, err := topicreadercommon.NewBatch(e.partitionSession, []*topicreadercommon.PublicMessage{ topicreadercommon.NewPublicMessageBuilder(). Offset(e.partitionSession.CommittedOffset().ToInt64()). @@ -1392,14 +1401,12 @@ func TestUpdateCommitInTransaction(t *testing.T) { Build(), }) require.NoError(t, err) - err = e.reader.commitWithTransaction(e.ctx, txMock, batch) require.NoError(t, err) require.NotEmpty(t, txMock.onBeforeCommit) err = txMock.onBeforeCommit[0](e.ctx) require.ErrorIs(t, err, testError) - txMock.onCompleted[0](err) require.True(t, e.reader.closed) @@ -1415,21 +1422,22 @@ func TestUpdateCommitInTransaction(t *testing.T) { txMock := newMockTransactionWrapper("test-session-id", "test-tx-id") txMock.nodeID = 123 - batch, err := topicreadercommon.NewBatch(e.partitionSession, nil) - require.NoError(t, err) - - err = e.reader.commitWithTransaction(e.ctx, txMock, batch) - require.NoError(t, err) - e.TopicClient.EXPECT().UpdateOffsetsInTransaction(gomock.Any(), gomock.Any()).DoAndReturn( func(ctx context.Context, _ *rawtopic.UpdateOffsetsInTransactionRequest) error { nodeID, ok := endpoint.ContextNodeID(ctx) + require.True(t, ok) require.Equal(t, uint32(123), nodeID) return nil }) + batch, err := topicreadercommon.NewBatch(e.partitionSession, nil) + require.NoError(t, err) + + err = e.reader.commitWithTransaction(e.ctx, txMock, batch) + require.NoError(t, err) + err = txMock.onBeforeCommit[0](e.ctx) require.NoError(t, err) }) From 492f487f38a56b149495c4f0fda93438f22c3e6f Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Mon, 8 Dec 2025 14:39:45 +0300 Subject: [PATCH 17/27] fix integration test --- .../topic_read_update_offsets_test.go | 60 ++++++++++--------- 1 file changed, 31 insertions(+), 29 deletions(-) diff --git a/tests/integration/topic_read_update_offsets_test.go b/tests/integration/topic_read_update_offsets_test.go index eedac958c..88925f574 100644 --- a/tests/integration/topic_read_update_offsets_test.go +++ b/tests/integration/topic_read_update_offsets_test.go @@ -8,7 +8,6 @@ import ( "strings" "sync" "testing" - "time" "github.com/stretchr/testify/suite" "github.com/ydb-platform/ydb-go-genproto/Ydb_Query_V1" @@ -35,7 +34,7 @@ func (t *TopicReaderUpdateOffsetsSuite) SetupTest() { t.scope = newScope(t.T()) t.writer = t.scope.TopicWriter() t.reader = t.scope.TopicReader() - t.driver = t.scope.DriverWithGRPCLogging() + t.driver = t.scope.Driver() } func TestBatchTxStorage(t *testing.T) { @@ -44,44 +43,35 @@ func TestBatchTxStorage(t *testing.T) { // Helper methods for test setup -func (t *TopicReaderUpdateOffsetsSuite) testContext() (context.Context, context.CancelFunc) { - return context.WithTimeout(t.scope.Ctx, 10*time.Second) -} - -func (t *TopicReaderUpdateOffsetsSuite) doTransactionWithDeletedSession( - ctx context.Context, - fn func(ctx context.Context, tr query.TxActor) error, -) error { - var once sync.Once - return t.driver.Query().DoTx(ctx, func(ctx context.Context, tr query.TxActor) error { - once.Do(func() { - t.deleteTxSession(ctx, tr.(tx.Transaction)) - }) - return fn(ctx, tr) - }) +func (t *TopicReaderUpdateOffsetsSuite) testContext() context.Context { + return t.scope.Ctx } func (t *TopicReaderUpdateOffsetsSuite) TestSingleTransaction() { - ctx, cancel := t.testContext() - defer cancel() + ctx := t.testContext() t.writeMessage(ctx, "1") var batch *topicreader.Batch - err := t.doTransactionWithDeletedSession(ctx, func(ctx context.Context, tr query.TxActor) error { + + var once sync.Once + err := t.driver.Query().DoTx(ctx, func(ctx context.Context, tr query.TxActor) error { + once.Do(func() { + t.deleteTxSession(ctx, tr.(tx.Transaction)) + }) + var err error batch, err = t.reader.PopMessagesBatchTx(ctx, tr) return err }) - t.NoError(err) - t.Len(batch.Messages, 1) + t.Require().NoError(err) + t.Require().Len(batch.Messages, 1) t.MsgEqualString("1", batch.Messages[0]) } func (t *TopicReaderUpdateOffsetsSuite) TestSeveralReads() { - ctx, cancel := t.testContext() - defer cancel() + ctx := t.testContext() t.writeMessage(ctx, "1") @@ -106,12 +96,20 @@ func (t *TopicReaderUpdateOffsetsSuite) TestSeveralReads() { } func (t *TopicReaderUpdateOffsetsSuite) TestSeveralTransactions() { - ctx, cancel := t.testContext() - defer cancel() + ctx := t.testContext() t.writeMessage(ctx, "1") - err := t.doTransactionWithDeletedSession(ctx, func(ctx context.Context, tr query.TxActor) error { + var ( + once1 sync.Once + once2 sync.Once + ) + + err := t.driver.Query().DoTx(ctx, func(ctx context.Context, tr query.TxActor) error { + once1.Do(func() { + t.deleteTxSession(ctx, tr.(tx.Transaction)) + }) + _, err := t.reader.PopMessagesBatchTx(ctx, tr) return err }) @@ -120,12 +118,16 @@ func (t *TopicReaderUpdateOffsetsSuite) TestSeveralTransactions() { t.writeMessage(ctx, "2") var batch *topicreader.Batch - err = t.doTransactionWithDeletedSession(ctx, func(ctx context.Context, tr query.TxActor) error { + err = t.driver.Query().DoTx(ctx, func(ctx context.Context, tr query.TxActor) error { + once2.Do(func() { + t.deleteTxSession(ctx, tr.(tx.Transaction)) + }) + var err error batch, err = t.reader.PopMessagesBatchTx(ctx, tr) return err }) - t.NoError(err) + t.Require().NoError(err) t.MsgEqualString("2", batch.Messages[0]) } From a5929526bb4544026e46e2aa1573cf2be737a995 Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Wed, 10 Dec 2025 19:26:53 +0300 Subject: [PATCH 18/27] fix review --- .../topicreaderinternal/batch_tx_storage.go | 10 +- .../batch_tx_storage_test.go | 279 ++++++++++-------- 2 files changed, 157 insertions(+), 132 deletions(-) diff --git a/internal/topic/topicreaderinternal/batch_tx_storage.go b/internal/topic/topicreaderinternal/batch_tx_storage.go index 770b589e0..3ae96cc03 100644 --- a/internal/topic/topicreaderinternal/batch_tx_storage.go +++ b/internal/topic/topicreaderinternal/batch_tx_storage.go @@ -18,7 +18,7 @@ import ( type batchTxStorage struct { batches map[string][]*topicreadercommon.PublicBatch consumer string - m xsync.RWMutex + m xsync.Mutex } // newBatchTxStorage creates a new batch transaction storage with the given consumer name. @@ -48,8 +48,8 @@ func (s *batchTxStorage) Add(transaction tx.Transaction, batch *topicreadercommo // Returns an empty slice (nil) if no batches are stored for the transaction. // This method is thread-safe. func (s *batchTxStorage) GetBatches(transaction tx.Transaction) []*topicreadercommon.PublicBatch { - s.m.RLock() - defer s.m.RUnlock() + s.m.Lock() + defer s.m.Unlock() batches, ok := s.batches[transaction.ID()] if !ok { @@ -68,9 +68,9 @@ func (s *batchTxStorage) GetBatches(transaction tx.Transaction) []*topicreaderco func (s *batchTxStorage) GetUpdateOffsetsInTransactionRequest( transaction tx.Transaction, ) *rawtopic.UpdateOffsetsInTransactionRequest { - s.m.RLock() + s.m.Lock() batches, ok := s.batches[transaction.ID()] - s.m.RUnlock() + s.m.Unlock() if !ok || len(batches) == 0 { return nil diff --git a/internal/topic/topicreaderinternal/batch_tx_storage_test.go b/internal/topic/topicreaderinternal/batch_tx_storage_test.go index 77776d5be..bbd424c25 100644 --- a/internal/topic/topicreaderinternal/batch_tx_storage_test.go +++ b/internal/topic/topicreaderinternal/batch_tx_storage_test.go @@ -4,6 +4,8 @@ import ( "context" "testing" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" "github.com/stretchr/testify/suite" "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic" @@ -14,109 +16,102 @@ import ( type BatchTxStorageTestSuite struct { suite.Suite - - storage *batchTxStorage -} - -func (s *BatchTxStorageTestSuite) SetupTest() { - s.storage = newBatchTxStorage("test-consumer") } // Helper methods for test setup -func (s *BatchTxStorageTestSuite) newTransaction(sessionID, txID string) *mockTransaction { - tx := newMockTransactionWrapper(sessionID, txID) - _ = tx.UnLazy(context.Background()) - - return tx -} - -func (s *BatchTxStorageTestSuite) addBatches(tx *mockTransaction, batches ...*topicreadercommon.PublicBatch) { - for _, batch := range batches { - _ = s.storage.Add(tx, batch) - } -} - func TestBatchTxStorage(t *testing.T) { suite.Run(t, new(BatchTxStorageTestSuite)) } -func (s *BatchTxStorageTestSuite) TestAdd_NewTransaction() { - tx := s.newTransaction("session-1", "tx-1") - batch := s.createTestBatch("topic-1", 1, 10, 20, 1) +func TestBatchTxStorageAdd_NewTransaction(t *testing.T) { + tx := newMockTransactionWrapper("session-1", "tx-1") + batch := createTestBatch("topic-1", 1, 10, 20, 1) - exists := s.storage.Add(tx, batch) + exists := newBatchTxStorage("test-consumer").Add(tx, batch) - s.False(exists) + assert.False(t, exists) } -func (s *BatchTxStorageTestSuite) TestAdd_ExistingTransaction() { - tx := s.newTransaction("session-1", "tx-1") - batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) - batch2 := s.createTestBatch("topic-1", 1, 20, 30, 1) - s.addBatches(tx, batch1) +func TestBatchTxStorageAdd_ExistingTransaction(t *testing.T) { + tx := newMockTransactionWrapper("session-1", "tx-1") + batch1 := createTestBatch("topic-1", 1, 10, 20, 1) + batch2 := createTestBatch("topic-1", 1, 20, 30, 1) + + storage := newBatchTxStorage("test-consumer") - exists := s.storage.Add(tx, batch2) + storage.Add(tx, batch1) + exists := storage.Add(tx, batch2) - s.True(exists) + assert.True(t, exists) } -func (s *BatchTxStorageTestSuite) TestGetBatches_Empty() { - tx := s.newTransaction("session-1", "tx-1") +func TestBatchTxStorageGetBatches_Empty(t *testing.T) { + tx := newMockTransactionWrapper("session-1", "tx-1") - batches := s.storage.GetBatches(tx) + batches := newBatchTxStorage("test-consumer").GetBatches(tx) - s.Empty(batches) + assert.Empty(t, batches) } -func (s *BatchTxStorageTestSuite) TestGetBatches_WithBatches() { - tx := s.newTransaction("session-1", "tx-1") - batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) - batch2 := s.createTestBatch("topic-1", 1, 20, 30, 1) - s.addBatches(tx, batch1, batch2) +func TestBatchTxStorageGetBatches_WithBatches(t *testing.T) { + tx := newMockTransactionWrapper("session-1", "tx-1") + batch1 := createTestBatch("topic-1", 1, 10, 20, 1) + batch2 := createTestBatch("topic-1", 1, 20, 30, 1) + + storage := newBatchTxStorage("test-consumer") + storage.Add(tx, batch1) + storage.Add(tx, batch2) - batches := s.storage.GetBatches(tx) + batches := storage.GetBatches(tx) - s.Len(batches, 2) - s.Equal(batch1, batches[0]) - s.Equal(batch2, batches[1]) + require.Len(t, batches, 2) + assert.Equal(t, batch1, batches[0]) + assert.Equal(t, batch2, batches[1]) } -func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_Empty() { - tx := s.newTransaction("session-1", "tx-1") +func TestBatchTxStorageGetUpdateOffsetsInTransactionRequest_Empty(t *testing.T) { + tx := newMockTransactionWrapper("session-1", "tx-1") - req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) + req := newBatchTxStorage("test-consumer").GetUpdateOffsetsInTransactionRequest(tx) - s.Nil(req) + assert.Nil(t, req) } -func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_SingleBatch() { - tx := s.newTransaction("session-1", "tx-1") - batch := s.createTestBatch("topic-1", 1, 10, 20, 1) - s.addBatches(tx, batch) +func TestBatchTxStorageGetUpdateOffsetsInTransactionRequest_SingleBatch(t *testing.T) { + tx := newMockTransactionWrapper("session-1", "tx-1") + _ = tx.UnLazy(context.TODO()) + batch := createTestBatch("topic-1", 1, 10, 20, 1) - req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) + storage := newBatchTxStorage("test-consumer") - s.Require().NotNil(req) - s.Equal("test-consumer", req.Consumer) - s.Equal("tx-1", req.Tx.ID) - s.Equal("session-1", req.Tx.Session) - s.Len(req.Topics, 1) - s.Equal("topic-1", req.Topics[0].Path) - s.Len(req.Topics[0].Partitions, 1) - s.Equal(int64(1), req.Topics[0].Partitions[0].PartitionID) - s.Len(req.Topics[0].Partitions[0].PartitionOffsets, 1) - s.Equal(rawtopiccommon.Offset(10), req.Topics[0].Partitions[0].PartitionOffsets[0].Start) - s.Equal(rawtopiccommon.Offset(20), req.Topics[0].Partitions[0].PartitionOffsets[0].End) + storage.Add(tx, batch) + + req := storage.GetUpdateOffsetsInTransactionRequest(tx) + + require.NotNil(t, req) + assert.Equal(t, "test-consumer", req.Consumer) + assert.Equal(t, "tx-1", req.Tx.ID) + assert.Equal(t, "session-1", req.Tx.Session) + require.Len(t, req.Topics, 1) + assert.Equal(t, "topic-1", req.Topics[0].Path) + require.Len(t, req.Topics[0].Partitions, 1) + assert.Equal(t, int64(1), req.Topics[0].Partitions[0].PartitionID) + require.Len(t, req.Topics[0].Partitions[0].PartitionOffsets, 1) + assert.Equal(t, rawtopiccommon.Offset(10), req.Topics[0].Partitions[0].PartitionOffsets[0].Start) + assert.Equal(t, rawtopiccommon.Offset(20), req.Topics[0].Partitions[0].PartitionOffsets[0].End) } func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_MultipleBatches() { - tx := s.newTransaction("session-1", "tx-1") - batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) - batch2 := s.createTestBatch("topic-1", 1, 20, 30, 1) - s.addBatches(tx, batch1, batch2) + tx := newMockTransactionWrapper("session-1", "tx-1") + batch1 := createTestBatch("topic-1", 1, 10, 20, 1) + batch2 := createTestBatch("topic-1", 1, 20, 30, 1) - req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) + storage := newBatchTxStorage("test-consumer") + storage.Add(tx, batch1) + storage.Add(tx, batch2) + + req := storage.GetUpdateOffsetsInTransactionRequest(tx) s.Require().NotNil(req) s.Len(req.Topics, 1) @@ -127,12 +122,15 @@ func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_Multi } func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_MultipleTopics() { - tx := s.newTransaction("session-1", "tx-1") - batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) - batch2 := s.createTestBatch("topic-2", 2, 30, 40, 2) - s.addBatches(tx, batch1, batch2) + tx := newMockTransactionWrapper("session-1", "tx-1") + batch1 := createTestBatch("topic-1", 1, 10, 20, 1) + batch2 := createTestBatch("topic-2", 2, 30, 40, 2) + + storage := newBatchTxStorage("test-consumer") + storage.Add(tx, batch1) + storage.Add(tx, batch2) - req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) + req := storage.GetUpdateOffsetsInTransactionRequest(tx) s.Require().NotNil(req) s.Len(req.Topics, 2) @@ -142,12 +140,15 @@ func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_Multi } func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_MultiplePartitionsSameTopic() { - tx := s.newTransaction("session-1", "tx-1") - batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) - batch2 := s.createTestBatch("topic-1", 2, 30, 40, 2) - s.addBatches(tx, batch1, batch2) + tx := newMockTransactionWrapper("session-1", "tx-1") + batch1 := createTestBatch("topic-1", 1, 10, 20, 1) + batch2 := createTestBatch("topic-1", 2, 30, 40, 2) - req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) + storage := newBatchTxStorage("test-consumer") + storage.Add(tx, batch1) + storage.Add(tx, batch2) + + req := storage.GetUpdateOffsetsInTransactionRequest(tx) s.Require().NotNil(req) s.Len(req.Topics, 1) @@ -166,12 +167,15 @@ func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_Multi } func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_NonAdjacentBatches() { - tx := s.newTransaction("session-1", "tx-1") - batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) - batch2 := s.createTestBatch("topic-1", 1, 30, 40, 1) - s.addBatches(tx, batch1, batch2) + tx := newMockTransactionWrapper("session-1", "tx-1") + batch1 := createTestBatch("topic-1", 1, 10, 20, 1) + batch2 := createTestBatch("topic-1", 1, 30, 40, 1) + + storage := newBatchTxStorage("test-consumer") + storage.Add(tx, batch1) + storage.Add(tx, batch2) - req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) + req := storage.GetUpdateOffsetsInTransactionRequest(tx) s.Require().NotNil(req) s.Len(req.Topics, 1) @@ -184,14 +188,19 @@ func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_NonAd } func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_MultiplePartitionsMultipleTopics() { - tx := s.newTransaction("session-1", "tx-1") - batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) - batch2 := s.createTestBatch("topic-1", 2, 30, 40, 2) - batch3 := s.createTestBatch("topic-2", 1, 50, 60, 3) - batch4 := s.createTestBatch("topic-2", 2, 70, 80, 4) - s.addBatches(tx, batch1, batch2, batch3, batch4) + tx := newMockTransactionWrapper("session-1", "tx-1") + batch1 := createTestBatch("topic-1", 1, 10, 20, 1) + batch2 := createTestBatch("topic-1", 2, 30, 40, 2) + batch3 := createTestBatch("topic-2", 1, 50, 60, 3) + batch4 := createTestBatch("topic-2", 2, 70, 80, 4) - req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) + storage := newBatchTxStorage("test-consumer") + storage.Add(tx, batch1) + storage.Add(tx, batch2) + storage.Add(tx, batch3) + storage.Add(tx, batch4) + + req := storage.GetUpdateOffsetsInTransactionRequest(tx) s.Require().NotNil(req) s.Len(req.Topics, 2) @@ -212,14 +221,19 @@ func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_Multi } func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_ComplexOptimization() { - tx := s.newTransaction("session-1", "tx-1") - batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) - batch2 := s.createTestBatch("topic-1", 1, 20, 30, 1) - batch3 := s.createTestBatch("topic-1", 1, 30, 40, 1) - batch4 := s.createTestBatch("topic-1", 1, 50, 60, 1) - s.addBatches(tx, batch1, batch2, batch3, batch4) + tx := newMockTransactionWrapper("session-1", "tx-1") + batch1 := createTestBatch("topic-1", 1, 10, 20, 1) + batch2 := createTestBatch("topic-1", 1, 20, 30, 1) + batch3 := createTestBatch("topic-1", 1, 30, 40, 1) + batch4 := createTestBatch("topic-1", 1, 50, 60, 1) + + storage := newBatchTxStorage("test-consumer") + storage.Add(tx, batch1) + storage.Add(tx, batch2) + storage.Add(tx, batch3) + storage.Add(tx, batch4) - req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) + req := storage.GetUpdateOffsetsInTransactionRequest(tx) s.Require().NotNil(req) s.Len(req.Topics, 1) @@ -232,15 +246,21 @@ func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_Compl } func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_MixedPartitionsAndTopics() { - tx := s.newTransaction("session-1", "tx-1") - batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) - batch2 := s.createTestBatch("topic-1", 1, 20, 30, 1) - batch3 := s.createTestBatch("topic-1", 2, 40, 50, 2) - batch4 := s.createTestBatch("topic-1", 2, 60, 70, 2) - batch5 := s.createTestBatch("topic-2", 1, 80, 90, 3) - s.addBatches(tx, batch1, batch2, batch3, batch4, batch5) - - req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) + tx := newMockTransactionWrapper("session-1", "tx-1") + batch1 := createTestBatch("topic-1", 1, 10, 20, 1) + batch2 := createTestBatch("topic-1", 1, 20, 30, 1) + batch3 := createTestBatch("topic-1", 2, 40, 50, 2) + batch4 := createTestBatch("topic-1", 2, 60, 70, 2) + batch5 := createTestBatch("topic-2", 1, 80, 90, 3) + + storage := newBatchTxStorage("test-consumer") + storage.Add(tx, batch1) + storage.Add(tx, batch2) + storage.Add(tx, batch3) + storage.Add(tx, batch4) + storage.Add(tx, batch5) + + req := storage.GetUpdateOffsetsInTransactionRequest(tx) s.Require().NotNil(req) s.Len(req.Topics, 2) @@ -268,40 +288,46 @@ func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_Mixed } func (s *BatchTxStorageTestSuite) TestClear() { - tx := s.newTransaction("session-1", "tx-1") - batch := s.createTestBatch("topic-1", 1, 10, 20, 1) - s.addBatches(tx, batch) + tx := newMockTransactionWrapper("session-1", "tx-1") + batch := createTestBatch("topic-1", 1, 10, 20, 1) + + storage := newBatchTxStorage("test-consumer") + storage.Add(tx, batch) - s.storage.Clear(tx) + storage.Clear(tx) - batches := s.storage.GetBatches(tx) + batches := storage.GetBatches(tx) s.Empty(batches) - req := s.storage.GetUpdateOffsetsInTransactionRequest(tx) + req := storage.GetUpdateOffsetsInTransactionRequest(tx) s.Nil(req) } func (s *BatchTxStorageTestSuite) TestMultipleTransactions() { - tx1 := s.newTransaction("session-1", "tx-1") - tx2 := s.newTransaction("session-2", "tx-2") - batch1 := s.createTestBatch("topic-1", 1, 10, 20, 1) - batch2 := s.createTestBatch("topic-2", 2, 30, 40, 2) - s.addBatches(tx1, batch1) - s.addBatches(tx2, batch2) + tx1 := newMockTransactionWrapper("session-1", "tx-1") + _ = tx1.UnLazy(context.TODO()) + tx2 := newMockTransactionWrapper("session-2", "tx-2") + _ = tx1.UnLazy(context.TODO()) + batch1 := createTestBatch("topic-1", 1, 10, 20, 1) + batch2 := createTestBatch("topic-2", 2, 30, 40, 2) + + storage := newBatchTxStorage("test-consumer") + storage.Add(tx1, batch1) + storage.Add(tx2, batch2) - batches1 := s.storage.GetBatches(tx1) - batches2 := s.storage.GetBatches(tx2) + batches1 := storage.GetBatches(tx1) + batches2 := storage.GetBatches(tx2) s.Len(batches1, 1) s.Equal(batch1, batches1[0]) s.Len(batches2, 1) s.Equal(batch2, batches2[0]) - s.storage.Clear(tx1) + storage.Clear(tx1) - batches1 = s.storage.GetBatches(tx1) + batches1 = storage.GetBatches(tx1) s.Empty(batches1) - batches2 = s.storage.GetBatches(tx2) + batches2 = storage.GetBatches(tx2) s.Len(batches2, 1) s.Equal(batch2, batches2[0]) } @@ -343,7 +369,7 @@ func (s *BatchTxStorageTestSuite) buildPartitionMap( // Helper methods for test data creation -func (s *BatchTxStorageTestSuite) createTestBatch( +func createTestBatch( topic string, partitionID int64, startOffset, endOffset int64, @@ -363,8 +389,7 @@ func (s *BatchTxStorageTestSuite) createTestBatch( rawtopiccommon.Offset(0), ) - batch, err := topicreadercommon.NewBatch(session, nil) - s.Require().NoError(err) + batch, _ := topicreadercommon.NewBatch(session, nil) commitRange := topicreadercommon.CommitRange{ CommitOffsetStart: rawtopiccommon.Offset(startOffset), From d7834dfb2ed9dbaf9703a42826e4f0b84b99db3f Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Wed, 10 Dec 2025 20:13:29 +0300 Subject: [PATCH 19/27] fix --- .../topicreaderinternal/batch_tx_storage.go | 26 +- .../batch_tx_storage_test.go | 239 +++++++++--------- .../topicreaderinternal/stream_reader_impl.go | 72 +++--- 3 files changed, 175 insertions(+), 162 deletions(-) diff --git a/internal/topic/topicreaderinternal/batch_tx_storage.go b/internal/topic/topicreaderinternal/batch_tx_storage.go index 3ae96cc03..2b99d423c 100644 --- a/internal/topic/topicreaderinternal/batch_tx_storage.go +++ b/internal/topic/topicreaderinternal/batch_tx_storage.go @@ -2,6 +2,7 @@ package topicreaderinternal import ( "context" + "fmt" "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic" "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic/rawtopiccommon" @@ -63,17 +64,18 @@ func (s *batchTxStorage) GetBatches(transaction tx.Transaction) []*topicreaderco // from all batches stored for the given transaction. // The batches are converted to commit ranges, optimized (adjacent ranges are merged), // and grouped by topic and partition. -// Returns nil if no batches are stored for the transaction. +// Returns nil, nil if no batches are stored for the transaction. +// Returns an error if session info is missing for any partition offset. // This method is thread-safe. func (s *batchTxStorage) GetUpdateOffsetsInTransactionRequest( transaction tx.Transaction, -) *rawtopic.UpdateOffsetsInTransactionRequest { +) (*rawtopic.UpdateOffsetsInTransactionRequest, error) { s.m.Lock() batches, ok := s.batches[transaction.ID()] s.m.Unlock() if !ok || len(batches) == 0 { - return nil + return nil, nil } // Convert batches to CommitRanges @@ -92,17 +94,20 @@ func (s *batchTxStorage) GetUpdateOffsetsInTransactionRequest( // Convert to partition offsets partitionOffsets := commitRanges.ToPartitionsOffsets() if len(partitionOffsets) == 0 { - return nil + return nil, nil } // Group partition offsets by topic - topicMap := s.buildPartitionOffsetsMap(partitionOffsets, sessionInfoMap) + topicMap, err := s.buildPartitionOffsetsMap(partitionOffsets, sessionInfoMap) + if err != nil { + return nil, err + } if len(topicMap) == 0 { - return nil + return nil, nil } // Build request - return s.buildUpdateOffsetsRequest(transaction, topicMap) + return s.buildUpdateOffsetsRequest(transaction, topicMap), nil } type sessionInfo struct { @@ -133,14 +138,13 @@ func (s *batchTxStorage) buildSessionInfoMap( func (s *batchTxStorage) buildPartitionOffsetsMap( partitionOffsets []rawtopicreader.PartitionCommitOffset, sessionInfoMap map[rawtopicreader.PartitionSessionID]sessionInfo, -) map[string][]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets { +) (map[string][]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets, error) { topicMap := make(map[string][]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets) for i := range partitionOffsets { po := &partitionOffsets[i] info, ok := sessionInfoMap[po.PartitionSessionID] if !ok { - // Skip if session info not found (should not happen in normal flow) - continue + return nil, fmt.Errorf("session info not found for partition session ID %d", po.PartitionSessionID) } topicMap[info.topic] = append(topicMap[info.topic], rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets{ @@ -149,7 +153,7 @@ func (s *batchTxStorage) buildPartitionOffsetsMap( }) } - return topicMap + return topicMap, nil } // buildUpdateOffsetsRequest creates the final UpdateOffsetsInTransactionRequest. diff --git a/internal/topic/topicreaderinternal/batch_tx_storage_test.go b/internal/topic/topicreaderinternal/batch_tx_storage_test.go index bbd424c25..b169fd98c 100644 --- a/internal/topic/topicreaderinternal/batch_tx_storage_test.go +++ b/internal/topic/topicreaderinternal/batch_tx_storage_test.go @@ -6,7 +6,6 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "github.com/stretchr/testify/suite" "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic" "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic/rawtopiccommon" @@ -14,16 +13,6 @@ import ( "github.com/ydb-platform/ydb-go-sdk/v3/internal/topic/topicreadercommon" ) -type BatchTxStorageTestSuite struct { - suite.Suite -} - -// Helper methods for test setup - -func TestBatchTxStorage(t *testing.T) { - suite.Run(t, new(BatchTxStorageTestSuite)) -} - func TestBatchTxStorageAdd_NewTransaction(t *testing.T) { tx := newMockTransactionWrapper("session-1", "tx-1") batch := createTestBatch("topic-1", 1, 10, 20, 1) @@ -73,8 +62,9 @@ func TestBatchTxStorageGetBatches_WithBatches(t *testing.T) { func TestBatchTxStorageGetUpdateOffsetsInTransactionRequest_Empty(t *testing.T) { tx := newMockTransactionWrapper("session-1", "tx-1") - req := newBatchTxStorage("test-consumer").GetUpdateOffsetsInTransactionRequest(tx) + req, err := newBatchTxStorage("test-consumer").GetUpdateOffsetsInTransactionRequest(tx) + assert.NoError(t, err) assert.Nil(t, req) } @@ -87,8 +77,9 @@ func TestBatchTxStorageGetUpdateOffsetsInTransactionRequest_SingleBatch(t *testi storage.Add(tx, batch) - req := storage.GetUpdateOffsetsInTransactionRequest(tx) + req, err := storage.GetUpdateOffsetsInTransactionRequest(tx) + require.NoError(t, err) require.NotNil(t, req) assert.Equal(t, "test-consumer", req.Consumer) assert.Equal(t, "tx-1", req.Tx.ID) @@ -102,7 +93,7 @@ func TestBatchTxStorageGetUpdateOffsetsInTransactionRequest_SingleBatch(t *testi assert.Equal(t, rawtopiccommon.Offset(20), req.Topics[0].Partitions[0].PartitionOffsets[0].End) } -func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_MultipleBatches() { +func TestBatchTxStorageGetUpdateOffsetsInTransactionRequest_MultipleBatches(t *testing.T) { tx := newMockTransactionWrapper("session-1", "tx-1") batch1 := createTestBatch("topic-1", 1, 10, 20, 1) batch2 := createTestBatch("topic-1", 1, 20, 30, 1) @@ -111,17 +102,18 @@ func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_Multi storage.Add(tx, batch1) storage.Add(tx, batch2) - req := storage.GetUpdateOffsetsInTransactionRequest(tx) + req, err := storage.GetUpdateOffsetsInTransactionRequest(tx) - s.Require().NotNil(req) - s.Len(req.Topics, 1) - s.Len(req.Topics[0].Partitions, 1) - s.Len(req.Topics[0].Partitions[0].PartitionOffsets, 1) - s.Equal(rawtopiccommon.Offset(10), req.Topics[0].Partitions[0].PartitionOffsets[0].Start) - s.Equal(rawtopiccommon.Offset(30), req.Topics[0].Partitions[0].PartitionOffsets[0].End) + require.NoError(t, err) + require.NotNil(t, req) + require.Len(t, req.Topics, 1) + require.Len(t, req.Topics[0].Partitions, 1) + require.Len(t, req.Topics[0].Partitions[0].PartitionOffsets, 1) + assert.Equal(t, rawtopiccommon.Offset(10), req.Topics[0].Partitions[0].PartitionOffsets[0].Start) + assert.Equal(t, rawtopiccommon.Offset(30), req.Topics[0].Partitions[0].PartitionOffsets[0].End) } -func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_MultipleTopics() { +func TestBatchTxStorageGetUpdateOffsetsInTransactionRequest_MultipleTopics(t *testing.T) { tx := newMockTransactionWrapper("session-1", "tx-1") batch1 := createTestBatch("topic-1", 1, 10, 20, 1) batch2 := createTestBatch("topic-2", 2, 30, 40, 2) @@ -130,16 +122,17 @@ func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_Multi storage.Add(tx, batch1) storage.Add(tx, batch2) - req := storage.GetUpdateOffsetsInTransactionRequest(tx) + req, err := storage.GetUpdateOffsetsInTransactionRequest(tx) - s.Require().NotNil(req) - s.Len(req.Topics, 2) - topicMap := s.buildTopicMap(req.Topics) - s.Contains(topicMap, "topic-1") - s.Contains(topicMap, "topic-2") + require.NoError(t, err) + require.NotNil(t, req) + require.Len(t, req.Topics, 2) + topicMap := buildTopicMap(req.Topics) + assert.Contains(t, topicMap, "topic-1") + assert.Contains(t, topicMap, "topic-2") } -func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_MultiplePartitionsSameTopic() { +func TestBatchTxStorageGetUpdateOffsetsInTransactionRequest_MultiplePartitionsSameTopic(t *testing.T) { tx := newMockTransactionWrapper("session-1", "tx-1") batch1 := createTestBatch("topic-1", 1, 10, 20, 1) batch2 := createTestBatch("topic-1", 2, 30, 40, 2) @@ -148,25 +141,26 @@ func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_Multi storage.Add(tx, batch1) storage.Add(tx, batch2) - req := storage.GetUpdateOffsetsInTransactionRequest(tx) - - s.Require().NotNil(req) - s.Len(req.Topics, 1) - s.Equal("topic-1", req.Topics[0].Path) - s.Len(req.Topics[0].Partitions, 2) - - partitionMap := s.buildPartitionMap(req.Topics[0].Partitions) - s.Contains(partitionMap, int64(1)) - s.Contains(partitionMap, int64(2)) - s.Len(partitionMap[1].PartitionOffsets, 1) - s.Len(partitionMap[2].PartitionOffsets, 1) - s.Equal(rawtopiccommon.Offset(10), partitionMap[1].PartitionOffsets[0].Start) - s.Equal(rawtopiccommon.Offset(20), partitionMap[1].PartitionOffsets[0].End) - s.Equal(rawtopiccommon.Offset(30), partitionMap[2].PartitionOffsets[0].Start) - s.Equal(rawtopiccommon.Offset(40), partitionMap[2].PartitionOffsets[0].End) + req, err := storage.GetUpdateOffsetsInTransactionRequest(tx) + + require.NoError(t, err) + require.NotNil(t, req) + require.Len(t, req.Topics, 1) + assert.Equal(t, "topic-1", req.Topics[0].Path) + require.Len(t, req.Topics[0].Partitions, 2) + + partitionMap := buildPartitionMap(req.Topics[0].Partitions) + assert.Contains(t, partitionMap, int64(1)) + assert.Contains(t, partitionMap, int64(2)) + require.Len(t, partitionMap[1].PartitionOffsets, 1) + require.Len(t, partitionMap[2].PartitionOffsets, 1) + assert.Equal(t, rawtopiccommon.Offset(10), partitionMap[1].PartitionOffsets[0].Start) + assert.Equal(t, rawtopiccommon.Offset(20), partitionMap[1].PartitionOffsets[0].End) + assert.Equal(t, rawtopiccommon.Offset(30), partitionMap[2].PartitionOffsets[0].Start) + assert.Equal(t, rawtopiccommon.Offset(40), partitionMap[2].PartitionOffsets[0].End) } -func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_NonAdjacentBatches() { +func TestBatchTxStorageGetUpdateOffsetsInTransactionRequest_NonAdjacentBatches(t *testing.T) { tx := newMockTransactionWrapper("session-1", "tx-1") batch1 := createTestBatch("topic-1", 1, 10, 20, 1) batch2 := createTestBatch("topic-1", 1, 30, 40, 1) @@ -175,19 +169,20 @@ func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_NonAd storage.Add(tx, batch1) storage.Add(tx, batch2) - req := storage.GetUpdateOffsetsInTransactionRequest(tx) + req, err := storage.GetUpdateOffsetsInTransactionRequest(tx) - s.Require().NotNil(req) - s.Len(req.Topics, 1) - s.Len(req.Topics[0].Partitions, 1) - s.Len(req.Topics[0].Partitions[0].PartitionOffsets, 2) - s.Equal(rawtopiccommon.Offset(10), req.Topics[0].Partitions[0].PartitionOffsets[0].Start) - s.Equal(rawtopiccommon.Offset(20), req.Topics[0].Partitions[0].PartitionOffsets[0].End) - s.Equal(rawtopiccommon.Offset(30), req.Topics[0].Partitions[0].PartitionOffsets[1].Start) - s.Equal(rawtopiccommon.Offset(40), req.Topics[0].Partitions[0].PartitionOffsets[1].End) + require.NoError(t, err) + require.NotNil(t, req) + require.Len(t, req.Topics, 1) + require.Len(t, req.Topics[0].Partitions, 1) + require.Len(t, req.Topics[0].Partitions[0].PartitionOffsets, 2) + assert.Equal(t, rawtopiccommon.Offset(10), req.Topics[0].Partitions[0].PartitionOffsets[0].Start) + assert.Equal(t, rawtopiccommon.Offset(20), req.Topics[0].Partitions[0].PartitionOffsets[0].End) + assert.Equal(t, rawtopiccommon.Offset(30), req.Topics[0].Partitions[0].PartitionOffsets[1].Start) + assert.Equal(t, rawtopiccommon.Offset(40), req.Topics[0].Partitions[0].PartitionOffsets[1].End) } -func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_MultiplePartitionsMultipleTopics() { +func TestBatchTxStorageGetUpdateOffsetsInTransactionRequest_MultiplePartitionsMultipleTopics(t *testing.T) { tx := newMockTransactionWrapper("session-1", "tx-1") batch1 := createTestBatch("topic-1", 1, 10, 20, 1) batch2 := createTestBatch("topic-1", 2, 30, 40, 2) @@ -200,27 +195,28 @@ func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_Multi storage.Add(tx, batch3) storage.Add(tx, batch4) - req := storage.GetUpdateOffsetsInTransactionRequest(tx) + req, err := storage.GetUpdateOffsetsInTransactionRequest(tx) - s.Require().NotNil(req) - s.Len(req.Topics, 2) + require.NoError(t, err) + require.NotNil(t, req) + require.Len(t, req.Topics, 2) - topicMap := s.buildTopicOffsetsMap(req.Topics) - s.Contains(topicMap, "topic-1") - s.Contains(topicMap, "topic-2") - s.Len(topicMap["topic-1"].Partitions, 2) - s.Len(topicMap["topic-2"].Partitions, 2) + topicMap := buildTopicOffsetsMap(req.Topics) + assert.Contains(t, topicMap, "topic-1") + assert.Contains(t, topicMap, "topic-2") + require.Len(t, topicMap["topic-1"].Partitions, 2) + require.Len(t, topicMap["topic-2"].Partitions, 2) - partition1Map := s.buildPartitionMap(topicMap["topic-1"].Partitions) - s.Contains(partition1Map, int64(1)) - s.Contains(partition1Map, int64(2)) + partition1Map := buildPartitionMap(topicMap["topic-1"].Partitions) + assert.Contains(t, partition1Map, int64(1)) + assert.Contains(t, partition1Map, int64(2)) - partition2Map := s.buildPartitionMap(topicMap["topic-2"].Partitions) - s.Contains(partition2Map, int64(1)) - s.Contains(partition2Map, int64(2)) + partition2Map := buildPartitionMap(topicMap["topic-2"].Partitions) + assert.Contains(t, partition2Map, int64(1)) + assert.Contains(t, partition2Map, int64(2)) } -func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_ComplexOptimization() { +func TestBatchTxStorageGetUpdateOffsetsInTransactionRequest_ComplexOptimization(t *testing.T) { tx := newMockTransactionWrapper("session-1", "tx-1") batch1 := createTestBatch("topic-1", 1, 10, 20, 1) batch2 := createTestBatch("topic-1", 1, 20, 30, 1) @@ -233,19 +229,20 @@ func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_Compl storage.Add(tx, batch3) storage.Add(tx, batch4) - req := storage.GetUpdateOffsetsInTransactionRequest(tx) + req, err := storage.GetUpdateOffsetsInTransactionRequest(tx) - s.Require().NotNil(req) - s.Len(req.Topics, 1) - s.Len(req.Topics[0].Partitions, 1) - s.Len(req.Topics[0].Partitions[0].PartitionOffsets, 2) - s.Equal(rawtopiccommon.Offset(10), req.Topics[0].Partitions[0].PartitionOffsets[0].Start) - s.Equal(rawtopiccommon.Offset(40), req.Topics[0].Partitions[0].PartitionOffsets[0].End) - s.Equal(rawtopiccommon.Offset(50), req.Topics[0].Partitions[0].PartitionOffsets[1].Start) - s.Equal(rawtopiccommon.Offset(60), req.Topics[0].Partitions[0].PartitionOffsets[1].End) + require.NoError(t, err) + require.NotNil(t, req) + require.Len(t, req.Topics, 1) + require.Len(t, req.Topics[0].Partitions, 1) + require.Len(t, req.Topics[0].Partitions[0].PartitionOffsets, 2) + assert.Equal(t, rawtopiccommon.Offset(10), req.Topics[0].Partitions[0].PartitionOffsets[0].Start) + assert.Equal(t, rawtopiccommon.Offset(40), req.Topics[0].Partitions[0].PartitionOffsets[0].End) + assert.Equal(t, rawtopiccommon.Offset(50), req.Topics[0].Partitions[0].PartitionOffsets[1].Start) + assert.Equal(t, rawtopiccommon.Offset(60), req.Topics[0].Partitions[0].PartitionOffsets[1].End) } -func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_MixedPartitionsAndTopics() { +func TestBatchTxStorageGetUpdateOffsetsInTransactionRequest_MixedPartitionsAndTopics(t *testing.T) { tx := newMockTransactionWrapper("session-1", "tx-1") batch1 := createTestBatch("topic-1", 1, 10, 20, 1) batch2 := createTestBatch("topic-1", 1, 20, 30, 1) @@ -260,34 +257,35 @@ func (s *BatchTxStorageTestSuite) TestGetUpdateOffsetsInTransactionRequest_Mixed storage.Add(tx, batch4) storage.Add(tx, batch5) - req := storage.GetUpdateOffsetsInTransactionRequest(tx) - - s.Require().NotNil(req) - s.Len(req.Topics, 2) - - topicMap := s.buildTopicOffsetsMap(req.Topics) - s.Contains(topicMap, "topic-1") - s.Len(topicMap["topic-1"].Partitions, 2) - s.Contains(topicMap, "topic-2") - s.Len(topicMap["topic-2"].Partitions, 1) - - partition1Map := s.buildPartitionMap(topicMap["topic-1"].Partitions) - s.Len(partition1Map[1].PartitionOffsets, 1) // Merged - s.Equal(rawtopiccommon.Offset(10), partition1Map[1].PartitionOffsets[0].Start) - s.Equal(rawtopiccommon.Offset(30), partition1Map[1].PartitionOffsets[0].End) + req, err := storage.GetUpdateOffsetsInTransactionRequest(tx) - s.Len(partition1Map[2].PartitionOffsets, 2) // Not merged - s.Equal(rawtopiccommon.Offset(40), partition1Map[2].PartitionOffsets[0].Start) - s.Equal(rawtopiccommon.Offset(50), partition1Map[2].PartitionOffsets[0].End) - s.Equal(rawtopiccommon.Offset(60), partition1Map[2].PartitionOffsets[1].Start) - s.Equal(rawtopiccommon.Offset(70), partition1Map[2].PartitionOffsets[1].End) - - s.Len(topicMap["topic-2"].Partitions[0].PartitionOffsets, 1) - s.Equal(rawtopiccommon.Offset(80), topicMap["topic-2"].Partitions[0].PartitionOffsets[0].Start) - s.Equal(rawtopiccommon.Offset(90), topicMap["topic-2"].Partitions[0].PartitionOffsets[0].End) + require.NoError(t, err) + require.NotNil(t, req) + require.Len(t, req.Topics, 2) + + topicMap := buildTopicOffsetsMap(req.Topics) + assert.Contains(t, topicMap, "topic-1") + require.Len(t, topicMap["topic-1"].Partitions, 2) + assert.Contains(t, topicMap, "topic-2") + require.Len(t, topicMap["topic-2"].Partitions, 1) + + partition1Map := buildPartitionMap(topicMap["topic-1"].Partitions) + require.Len(t, partition1Map[1].PartitionOffsets, 1) // Merged + assert.Equal(t, rawtopiccommon.Offset(10), partition1Map[1].PartitionOffsets[0].Start) + assert.Equal(t, rawtopiccommon.Offset(30), partition1Map[1].PartitionOffsets[0].End) + + require.Len(t, partition1Map[2].PartitionOffsets, 2) // Not merged + assert.Equal(t, rawtopiccommon.Offset(40), partition1Map[2].PartitionOffsets[0].Start) + assert.Equal(t, rawtopiccommon.Offset(50), partition1Map[2].PartitionOffsets[0].End) + assert.Equal(t, rawtopiccommon.Offset(60), partition1Map[2].PartitionOffsets[1].Start) + assert.Equal(t, rawtopiccommon.Offset(70), partition1Map[2].PartitionOffsets[1].End) + + require.Len(t, topicMap["topic-2"].Partitions[0].PartitionOffsets, 1) + assert.Equal(t, rawtopiccommon.Offset(80), topicMap["topic-2"].Partitions[0].PartitionOffsets[0].Start) + assert.Equal(t, rawtopiccommon.Offset(90), topicMap["topic-2"].Partitions[0].PartitionOffsets[0].End) } -func (s *BatchTxStorageTestSuite) TestClear() { +func TestBatchTxStorageClear(t *testing.T) { tx := newMockTransactionWrapper("session-1", "tx-1") batch := createTestBatch("topic-1", 1, 10, 20, 1) @@ -297,13 +295,14 @@ func (s *BatchTxStorageTestSuite) TestClear() { storage.Clear(tx) batches := storage.GetBatches(tx) - s.Empty(batches) + assert.Empty(t, batches) - req := storage.GetUpdateOffsetsInTransactionRequest(tx) - s.Nil(req) + req, err := storage.GetUpdateOffsetsInTransactionRequest(tx) + assert.NoError(t, err) + assert.Nil(t, req) } -func (s *BatchTxStorageTestSuite) TestMultipleTransactions() { +func TestBatchTxStorageMultipleTransactions(t *testing.T) { tx1 := newMockTransactionWrapper("session-1", "tx-1") _ = tx1.UnLazy(context.TODO()) tx2 := newMockTransactionWrapper("session-2", "tx-2") @@ -318,23 +317,23 @@ func (s *BatchTxStorageTestSuite) TestMultipleTransactions() { batches1 := storage.GetBatches(tx1) batches2 := storage.GetBatches(tx2) - s.Len(batches1, 1) - s.Equal(batch1, batches1[0]) - s.Len(batches2, 1) - s.Equal(batch2, batches2[0]) + require.Len(t, batches1, 1) + assert.Equal(t, batch1, batches1[0]) + require.Len(t, batches2, 1) + assert.Equal(t, batch2, batches2[0]) storage.Clear(tx1) batches1 = storage.GetBatches(tx1) - s.Empty(batches1) + assert.Empty(t, batches1) batches2 = storage.GetBatches(tx2) - s.Len(batches2, 1) - s.Equal(batch2, batches2[0]) + require.Len(t, batches2, 1) + assert.Equal(t, batch2, batches2[0]) } // Helper methods for assertions -func (s *BatchTxStorageTestSuite) buildTopicMap( +func buildTopicMap( topics []rawtopic.UpdateOffsetsInTransactionRequest_TopicOffsets, ) map[string]bool { topicMap := make(map[string]bool) @@ -345,7 +344,7 @@ func (s *BatchTxStorageTestSuite) buildTopicMap( return topicMap } -func (s *BatchTxStorageTestSuite) buildTopicOffsetsMap( +func buildTopicOffsetsMap( topics []rawtopic.UpdateOffsetsInTransactionRequest_TopicOffsets, ) map[string]rawtopic.UpdateOffsetsInTransactionRequest_TopicOffsets { topicMap := make(map[string]rawtopic.UpdateOffsetsInTransactionRequest_TopicOffsets) @@ -356,7 +355,7 @@ func (s *BatchTxStorageTestSuite) buildTopicOffsetsMap( return topicMap } -func (s *BatchTxStorageTestSuite) buildPartitionMap( +func buildPartitionMap( partitions []rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets, ) map[int64]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets { partitionMap := make(map[int64]rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets) diff --git a/internal/topic/topicreaderinternal/stream_reader_impl.go b/internal/topic/topicreaderinternal/stream_reader_impl.go index 8eee838d2..fec792cc1 100644 --- a/internal/topic/topicreaderinternal/stream_reader_impl.go +++ b/internal/topic/topicreaderinternal/stream_reader_impl.go @@ -251,73 +251,83 @@ func (r *topicStreamReaderImpl) commitWithTransaction( return fmt.Errorf("ydb: failed to materialize transaction: %w", err) } - if r.batchTxStorage.Add(tx, batch) { + txAlreadyExists := r.batchTxStorage.Add(tx, batch) + if txAlreadyExists { // tx hooks already configured - exiting return nil } tx.OnBeforeCommit(r.txBeforeCommitFn(tx)) + tx.OnCompleted(r.txOnCompletedFn(ctx, tx)) - tx.OnCompleted(func(err error) { + return nil +} + +func (r *topicStreamReaderImpl) txBeforeCommitFn(tx tx.Transaction) tx.OnTransactionBeforeCommit { + return func(ctx context.Context) (err error) { logCtx := r.cfg.BaseContext - onDone := trace.TopicOnReaderTransactionCompleted( + onDone := trace.TopicOnReaderUpdateOffsetsInTransaction( r.cfg.Trace, &logCtx, r.readerID, r.readConnectionID, tx.SessionID(), tx, - err, ) - defer onDone() + defer func() { + onDone(err) + }() - defer r.batchTxStorage.Clear(tx) + // UpdateOffsetsInTransaction operation must be executed on the same Node where the transaction was initiated. + // Otherwise, errors such as `Database coordinators are unavailable` may occur. + ctx = endpoint.WithNodeID(ctx, tx.NodeID()) + req, err := r.batchTxStorage.GetUpdateOffsetsInTransactionRequest(tx) if err != nil { - // mark error as retryable - for proper reconnector working - err = xerrors.Retryable(err) - - _ = r.CloseWithError(xcontext.ValueOnly(ctx), fmt.Errorf("transaction failed: %w", err)) - - return + return xerrors.WithStackTrace(fmt.Errorf("building update offsets request: %w", err)) + } + if req == nil { + return nil } - for _, batch := range r.batchTxStorage.GetBatches(tx) { - commitRange := topicreadercommon.GetCommitRange(batch) - topicreadercommon.BatchGetPartitionSession(batch).SetCommittedOffsetForward(commitRange.CommitOffsetEnd) + err = r.topicClient.UpdateOffsetsInTransaction(ctx, req) + if err != nil { + return xerrors.WithStackTrace(fmt.Errorf("updating offsets in transaction: %w", err)) } - }) - return nil + return nil + } } -func (r *topicStreamReaderImpl) txBeforeCommitFn(tx tx.Transaction) tx.OnTransactionBeforeCommit { - return func(ctx context.Context) (err error) { +func (r *topicStreamReaderImpl) txOnCompletedFn(ctx context.Context, tx tx.Transaction) tx.OnTransactionCompletedFunc { + return func(err error) { logCtx := r.cfg.BaseContext - onDone := trace.TopicOnReaderUpdateOffsetsInTransaction( + onDone := trace.TopicOnReaderTransactionCompleted( r.cfg.Trace, &logCtx, r.readerID, r.readConnectionID, tx.SessionID(), tx, + err, ) - defer func() { - onDone(err) - }() - - // UpdateOffsetsInTransaction operation must be executed on the same Node where the transaction was initiated. - // Otherwise, errors such as `Database coordinators are unavailable` may occur. - ctx = endpoint.WithNodeID(ctx, tx.NodeID()) + defer onDone() - req := r.batchTxStorage.GetUpdateOffsetsInTransactionRequest(tx) + defer r.batchTxStorage.Clear(tx) - err = r.topicClient.UpdateOffsetsInTransaction(ctx, req) if err != nil { - return xerrors.WithStackTrace(fmt.Errorf("updating offsets in transaction: %w", err)) + // mark error as retryable - for proper reconnector working + err = xerrors.Retryable(err) + + _ = r.CloseWithError(xcontext.ValueOnly(ctx), fmt.Errorf("transaction failed: %w", err)) + + return } - return nil + for _, batch := range r.batchTxStorage.GetBatches(tx) { + commitRange := topicreadercommon.GetCommitRange(batch) + topicreadercommon.BatchGetPartitionSession(batch).SetCommittedOffsetForward(commitRange.CommitOffsetEnd) + } } } From 3cee5b952ff33c66510ab4b75c1fb481e3790e18 Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Wed, 10 Dec 2025 22:10:01 +0300 Subject: [PATCH 20/27] fix linter --- internal/topic/topicreaderinternal/batch_tx_storage.go | 10 +++++++--- .../topic/topicreaderinternal/batch_tx_storage_test.go | 7 +++++-- .../topic/topicreaderinternal/stream_reader_impl.go | 7 ++++--- 3 files changed, 16 insertions(+), 8 deletions(-) diff --git a/internal/topic/topicreaderinternal/batch_tx_storage.go b/internal/topic/topicreaderinternal/batch_tx_storage.go index 2b99d423c..1d037bc34 100644 --- a/internal/topic/topicreaderinternal/batch_tx_storage.go +++ b/internal/topic/topicreaderinternal/batch_tx_storage.go @@ -2,6 +2,7 @@ package topicreaderinternal import ( "context" + "errors" "fmt" "github.com/ydb-platform/ydb-go-sdk/v3/internal/grpcwrapper/rawtopic" @@ -14,6 +15,9 @@ import ( "github.com/ydb-platform/ydb-go-sdk/v3/internal/xsync" ) +// errNoBatches is returned when there are no batches to process for a transaction. +var errNoBatches = errors.New("no batches for transaction") + // batchTxStorage stores batches associated with transactions for commit within transaction. // It is thread-safe and allows multiple transactions to be managed concurrently. type batchTxStorage struct { @@ -75,7 +79,7 @@ func (s *batchTxStorage) GetUpdateOffsetsInTransactionRequest( s.m.Unlock() if !ok || len(batches) == 0 { - return nil, nil + return nil, errNoBatches } // Convert batches to CommitRanges @@ -94,7 +98,7 @@ func (s *batchTxStorage) GetUpdateOffsetsInTransactionRequest( // Convert to partition offsets partitionOffsets := commitRanges.ToPartitionsOffsets() if len(partitionOffsets) == 0 { - return nil, nil + return nil, errNoBatches } // Group partition offsets by topic @@ -103,7 +107,7 @@ func (s *batchTxStorage) GetUpdateOffsetsInTransactionRequest( return nil, err } if len(topicMap) == 0 { - return nil, nil + return nil, errNoBatches } // Build request diff --git a/internal/topic/topicreaderinternal/batch_tx_storage_test.go b/internal/topic/topicreaderinternal/batch_tx_storage_test.go index b169fd98c..1c9d546c4 100644 --- a/internal/topic/topicreaderinternal/batch_tx_storage_test.go +++ b/internal/topic/topicreaderinternal/batch_tx_storage_test.go @@ -2,6 +2,7 @@ package topicreaderinternal import ( "context" + "errors" "testing" "github.com/stretchr/testify/assert" @@ -64,7 +65,8 @@ func TestBatchTxStorageGetUpdateOffsetsInTransactionRequest_Empty(t *testing.T) req, err := newBatchTxStorage("test-consumer").GetUpdateOffsetsInTransactionRequest(tx) - assert.NoError(t, err) + assert.Error(t, err) + assert.True(t, errors.Is(err, errNoBatches)) assert.Nil(t, req) } @@ -298,7 +300,8 @@ func TestBatchTxStorageClear(t *testing.T) { assert.Empty(t, batches) req, err := storage.GetUpdateOffsetsInTransactionRequest(tx) - assert.NoError(t, err) + assert.Error(t, err) + assert.True(t, errors.Is(err, errNoBatches)) assert.Nil(t, req) } diff --git a/internal/topic/topicreaderinternal/stream_reader_impl.go b/internal/topic/topicreaderinternal/stream_reader_impl.go index fec792cc1..c92296e83 100644 --- a/internal/topic/topicreaderinternal/stream_reader_impl.go +++ b/internal/topic/topicreaderinternal/stream_reader_impl.go @@ -284,11 +284,12 @@ func (r *topicStreamReaderImpl) txBeforeCommitFn(tx tx.Transaction) tx.OnTransac req, err := r.batchTxStorage.GetUpdateOffsetsInTransactionRequest(tx) if err != nil { + if errors.Is(err, errNoBatches) { + return nil + } + return xerrors.WithStackTrace(fmt.Errorf("building update offsets request: %w", err)) } - if req == nil { - return nil - } err = r.topicClient.UpdateOffsetsInTransaction(ctx, req) if err != nil { From 965e6a07ebb5b5fa7aa5b5a40ada1cc5d34fef70 Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Wed, 10 Dec 2025 22:10:12 +0300 Subject: [PATCH 21/27] fix lint --- Agents.md | 3 +++ .../topic/topicreaderinternal/stream_reader_impl_test.go | 9 ++++----- 2 files changed, 7 insertions(+), 5 deletions(-) create mode 100644 Agents.md diff --git a/Agents.md b/Agents.md new file mode 100644 index 000000000..075c3d0a9 --- /dev/null +++ b/Agents.md @@ -0,0 +1,3 @@ +## Testing instructions + +Run `golangci-lint run --fix` to be sure code style rules pass. diff --git a/internal/topic/topicreaderinternal/stream_reader_impl_test.go b/internal/topic/topicreaderinternal/stream_reader_impl_test.go index f56b4d16b..2d9eaeeb4 100644 --- a/internal/topic/topicreaderinternal/stream_reader_impl_test.go +++ b/internal/topic/topicreaderinternal/stream_reader_impl_test.go @@ -1049,7 +1049,7 @@ func TestTopicStreamReadImpl_BatchReaderWantMoreMessagesThenBufferCanHold(t *tes e := newTopicReaderTestEnv(t) e.Start() - nextDataRequested := sendMessageWithFullBuffer(&e) + nextDataRequested := sendMessageWithFullBuffer(e) // wait message received to internal buffer xtest.SpinWaitCondition(t, &e.reader.batcher.m, func() bool { @@ -1098,7 +1098,7 @@ func TestTopicStreamReadImpl_BatchReaderWantMoreMessagesThenBufferCanHold(t *tes return len(e.reader.batcher.hasNewMessages) == 0 }) - nextDataRequested := sendMessageWithFullBuffer(&e) + nextDataRequested := sendMessageWithFullBuffer(e) <-readCompleted require.NoError(t, readErr) @@ -1183,7 +1183,7 @@ type testStreamResult struct { waitOnly bool } -func newTopicReaderTestEnv(t testing.TB) streamEnv { +func newTopicReaderTestEnv(t testing.TB) *streamEnv { ctx := xtest.Context(t) mc := gomock.NewController(t) @@ -1218,7 +1218,7 @@ func newTopicReaderTestEnv(t testing.TB) streamEnv { ) require.NoError(t, reader.sessionController.Add(session)) - env := streamEnv{ + env := &streamEnv{ TopicClient: topicClientMock, ctx: ctx, t: t, @@ -1260,7 +1260,6 @@ func newTopicReaderTestEnv(t testing.TB) streamEnv { } }) - //nolint:govet return env } From 3c73baf261bb171104cfbc06f1bc8c3288d268ac Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Wed, 10 Dec 2025 22:14:57 +0300 Subject: [PATCH 22/27] remove agents.md --- Agents.md | 3 --- 1 file changed, 3 deletions(-) delete mode 100644 Agents.md diff --git a/Agents.md b/Agents.md deleted file mode 100644 index 075c3d0a9..000000000 --- a/Agents.md +++ /dev/null @@ -1,3 +0,0 @@ -## Testing instructions - -Run `golangci-lint run --fix` to be sure code style rules pass. From d9fdf5e4205fe9fbd7187012cb727169d424510a Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Wed, 10 Dec 2025 22:19:25 +0300 Subject: [PATCH 23/27] Refactor integration tests for topic read and update offsets: replace testify suite with require, streamline test functions, and enhance helper methods for better readability and maintainability. --- .../topic_read_update_offsets_test.go | 137 ++++++++---------- 1 file changed, 63 insertions(+), 74 deletions(-) diff --git a/tests/integration/topic_read_update_offsets_test.go b/tests/integration/topic_read_update_offsets_test.go index 88925f574..a43ddb8e8 100644 --- a/tests/integration/topic_read_update_offsets_test.go +++ b/tests/integration/topic_read_update_offsets_test.go @@ -9,7 +9,7 @@ import ( "sync" "testing" - "github.com/stretchr/testify/suite" + "github.com/stretchr/testify/require" "github.com/ydb-platform/ydb-go-genproto/Ydb_Query_V1" "github.com/ydb-platform/ydb-go-genproto/protos/Ydb_Query" @@ -21,126 +21,113 @@ import ( "github.com/ydb-platform/ydb-go-sdk/v3/topic/topicwriter" ) -type TopicReaderUpdateOffsetsSuite struct { - suite.Suite +func TestSingleTransaction(t *testing.T) { + scope := newScope(t) + writer := scope.TopicWriter() + reader := scope.TopicReader() + driver := scope.Driver() + ctx := scope.Ctx - scope *scopeT - writer *topicwriter.Writer - reader *topicreader.Reader - driver *ydb.Driver -} - -func (t *TopicReaderUpdateOffsetsSuite) SetupTest() { - t.scope = newScope(t.T()) - t.writer = t.scope.TopicWriter() - t.reader = t.scope.TopicReader() - t.driver = t.scope.Driver() -} - -func TestBatchTxStorage(t *testing.T) { - suite.Run(t, new(TopicReaderUpdateOffsetsSuite)) -} - -// Helper methods for test setup - -func (t *TopicReaderUpdateOffsetsSuite) testContext() context.Context { - return t.scope.Ctx -} - -func (t *TopicReaderUpdateOffsetsSuite) TestSingleTransaction() { - ctx := t.testContext() - - t.writeMessage(ctx, "1") + writeMessage(t, ctx, writer, "1") var batch *topicreader.Batch var once sync.Once - err := t.driver.Query().DoTx(ctx, func(ctx context.Context, tr query.TxActor) error { + err := driver.Query().DoTx(ctx, func(ctx context.Context, tr query.TxActor) error { once.Do(func() { - t.deleteTxSession(ctx, tr.(tx.Transaction)) + deleteTxSession(t, ctx, driver, tr.(tx.Transaction)) }) var err error - batch, err = t.reader.PopMessagesBatchTx(ctx, tr) + batch, err = reader.PopMessagesBatchTx(ctx, tr) return err }) - t.Require().NoError(err) - t.Require().Len(batch.Messages, 1) - t.MsgEqualString("1", batch.Messages[0]) + require.NoError(t, err) + require.Len(t, batch.Messages, 1) + msgEqualString(t, "1", batch.Messages[0]) } -func (t *TopicReaderUpdateOffsetsSuite) TestSeveralReads() { - ctx := t.testContext() +func TestSeveralReads(t *testing.T) { + scope := newScope(t) + writer := scope.TopicWriter() + reader := scope.TopicReader() + driver := scope.Driver() + ctx := scope.Ctx - t.writeMessage(ctx, "1") + writeMessage(t, ctx, writer, "1") - err := t.driver.Query().DoTx(ctx, func(ctx context.Context, tr query.TxActor) error { - _, err := t.reader.PopMessagesBatchTx(ctx, tr) - t.Require().NoError(err) + err := driver.Query().DoTx(ctx, func(ctx context.Context, tr query.TxActor) error { + _, err := reader.PopMessagesBatchTx(ctx, tr) + require.NoError(t, err) - t.writeMessage(ctx, "2") + writeMessage(t, ctx, writer, "2") - _, err = t.reader.PopMessagesBatchTx(ctx, tr) - t.Require().NoError(err) + _, err = reader.PopMessagesBatchTx(ctx, tr) + require.NoError(t, err) - t.writeMessage(ctx, "3") + writeMessage(t, ctx, writer, "3") return nil }) - t.Require().NoError(err) + require.NoError(t, err) - msg, err := t.reader.ReadMessage(ctx) - t.Require().NoError(err) - t.MsgEqualString("3", msg) + msg, err := reader.ReadMessage(ctx) + require.NoError(t, err) + msgEqualString(t, "3", msg) } -func (t *TopicReaderUpdateOffsetsSuite) TestSeveralTransactions() { - ctx := t.testContext() +func TestSeveralTransactions(t *testing.T) { + scope := newScope(t) + writer := scope.TopicWriter() + reader := scope.TopicReader() + driver := scope.Driver() + ctx := scope.Ctx - t.writeMessage(ctx, "1") + writeMessage(t, ctx, writer, "1") var ( once1 sync.Once once2 sync.Once ) - err := t.driver.Query().DoTx(ctx, func(ctx context.Context, tr query.TxActor) error { + err := driver.Query().DoTx(ctx, func(ctx context.Context, tr query.TxActor) error { once1.Do(func() { - t.deleteTxSession(ctx, tr.(tx.Transaction)) + deleteTxSession(t, ctx, driver, tr.(tx.Transaction)) }) - _, err := t.reader.PopMessagesBatchTx(ctx, tr) + _, err := reader.PopMessagesBatchTx(ctx, tr) return err }) - t.NoError(err) + require.NoError(t, err) - t.writeMessage(ctx, "2") + writeMessage(t, ctx, writer, "2") var batch *topicreader.Batch - err = t.driver.Query().DoTx(ctx, func(ctx context.Context, tr query.TxActor) error { + err = driver.Query().DoTx(ctx, func(ctx context.Context, tr query.TxActor) error { once2.Do(func() { - t.deleteTxSession(ctx, tr.(tx.Transaction)) + deleteTxSession(t, ctx, driver, tr.(tx.Transaction)) }) var err error - batch, err = t.reader.PopMessagesBatchTx(ctx, tr) + batch, err = reader.PopMessagesBatchTx(ctx, tr) return err }) - t.Require().NoError(err) + require.NoError(t, err) - t.MsgEqualString("2", batch.Messages[0]) + msgEqualString(t, "2", batch.Messages[0]) } // Helper methods -func (t *TopicReaderUpdateOffsetsSuite) writeMessage(ctx context.Context, msg string) { - err := t.writer.Write(ctx, topicwriter.Message{Data: strings.NewReader(msg)}) - t.NoError(err) +func writeMessage(t *testing.T, ctx context.Context, writer *topicwriter.Writer, msg string) { + t.Helper() + err := writer.Write(ctx, topicwriter.Message{Data: strings.NewReader(msg)}) + require.NoError(t, err) } -func (t *TopicReaderUpdateOffsetsSuite) MsgEqualString(expected string, msg *topicreader.Message) { - t.T().Helper() +func msgEqualString(t *testing.T, expected string, msg *topicreader.Message) { + t.Helper() var actual string @@ -149,17 +136,19 @@ func (t *TopicReaderUpdateOffsetsSuite) MsgEqualString(expected string, msg *top return nil }) - t.Equal(expected, actual) + require.Equal(t, expected, actual) } -func (t *TopicReaderUpdateOffsetsSuite) deleteTxSession(ctx context.Context, tx tx.Transaction) { - t.deleteSession(ctx, tx.SessionID()) +func deleteTxSession(t *testing.T, ctx context.Context, driver *ydb.Driver, tx tx.Transaction) { + t.Helper() + deleteSession(t, ctx, driver, tx.SessionID()) } -func (t *TopicReaderUpdateOffsetsSuite) deleteSession(ctx context.Context, sessionID string) { - _, err := Ydb_Query_V1.NewQueryServiceClient(ydb.GRPCConn(t.driver)). +func deleteSession(t *testing.T, ctx context.Context, driver *ydb.Driver, sessionID string) { + t.Helper() + _, err := Ydb_Query_V1.NewQueryServiceClient(ydb.GRPCConn(driver)). DeleteSession(ctx, &Ydb_Query.DeleteSessionRequest{ SessionId: sessionID, }) - t.NoError(err) + require.NoError(t, err) } From 1754e865eb7d793264aa75cc55026ecc4b063771 Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Wed, 10 Dec 2025 22:23:23 +0300 Subject: [PATCH 24/27] fix --- .../topic/topicreaderinternal/stream_reader_impl_test.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/internal/topic/topicreaderinternal/stream_reader_impl_test.go b/internal/topic/topicreaderinternal/stream_reader_impl_test.go index 2d9eaeeb4..f56b4d16b 100644 --- a/internal/topic/topicreaderinternal/stream_reader_impl_test.go +++ b/internal/topic/topicreaderinternal/stream_reader_impl_test.go @@ -1049,7 +1049,7 @@ func TestTopicStreamReadImpl_BatchReaderWantMoreMessagesThenBufferCanHold(t *tes e := newTopicReaderTestEnv(t) e.Start() - nextDataRequested := sendMessageWithFullBuffer(e) + nextDataRequested := sendMessageWithFullBuffer(&e) // wait message received to internal buffer xtest.SpinWaitCondition(t, &e.reader.batcher.m, func() bool { @@ -1098,7 +1098,7 @@ func TestTopicStreamReadImpl_BatchReaderWantMoreMessagesThenBufferCanHold(t *tes return len(e.reader.batcher.hasNewMessages) == 0 }) - nextDataRequested := sendMessageWithFullBuffer(e) + nextDataRequested := sendMessageWithFullBuffer(&e) <-readCompleted require.NoError(t, readErr) @@ -1183,7 +1183,7 @@ type testStreamResult struct { waitOnly bool } -func newTopicReaderTestEnv(t testing.TB) *streamEnv { +func newTopicReaderTestEnv(t testing.TB) streamEnv { ctx := xtest.Context(t) mc := gomock.NewController(t) @@ -1218,7 +1218,7 @@ func newTopicReaderTestEnv(t testing.TB) *streamEnv { ) require.NoError(t, reader.sessionController.Add(session)) - env := &streamEnv{ + env := streamEnv{ TopicClient: topicClientMock, ctx: ctx, t: t, @@ -1260,6 +1260,7 @@ func newTopicReaderTestEnv(t testing.TB) *streamEnv { } }) + //nolint:govet return env } From 8fa7421e80423b896792dcfb518e89f516a69776 Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Wed, 10 Dec 2025 23:09:41 +0300 Subject: [PATCH 25/27] fix --- .../topicreaderinternal/batch_tx_storage.go | 58 ++++++++++---- .../batch_tx_storage_test.go | 80 +++++++++++-------- .../topicreaderinternal/stream_reader_impl.go | 5 +- 3 files changed, 93 insertions(+), 50 deletions(-) diff --git a/internal/topic/topicreaderinternal/batch_tx_storage.go b/internal/topic/topicreaderinternal/batch_tx_storage.go index 1d037bc34..bd6870efc 100644 --- a/internal/topic/topicreaderinternal/batch_tx_storage.go +++ b/internal/topic/topicreaderinternal/batch_tx_storage.go @@ -18,35 +18,56 @@ import ( // errNoBatches is returned when there are no batches to process for a transaction. var errNoBatches = errors.New("no batches for transaction") +// transactionBatches stores batches for a single transaction. +// It is not thread-safe and should be accessed only through batchTxStorage methods. +type transactionBatches struct { + batches []*topicreadercommon.PublicBatch +} + +// AddBatch adds a batch to the transaction. +func (tb *transactionBatches) AddBatch(batch *topicreadercommon.PublicBatch) { + tb.batches = append(tb.batches, batch) +} + +// GetBatches returns all batches stored for this transaction. +func (tb *transactionBatches) GetBatches() []*topicreadercommon.PublicBatch { + return tb.batches +} + // batchTxStorage stores batches associated with transactions for commit within transaction. // It is thread-safe and allows multiple transactions to be managed concurrently. type batchTxStorage struct { - batches map[string][]*topicreadercommon.PublicBatch - consumer string - m xsync.Mutex + transactions map[string]*transactionBatches + consumer string + m xsync.Mutex } // newBatchTxStorage creates a new batch transaction storage with the given consumer name. // The consumer name is used when building UpdateOffsetsInTransactionRequest. func newBatchTxStorage(consumer string) *batchTxStorage { return &batchTxStorage{ - batches: make(map[string][]*topicreadercommon.PublicBatch), - consumer: consumer, + transactions: make(map[string]*transactionBatches), + consumer: consumer, } } -// Add adds a batch to the transaction storage. -// It returns true if the transaction already exists (has been added before), false otherwise. +// GetOrCreateTransactionBatches gets or creates a transaction batches handler for the given transaction. +// It returns the handler and a flag indicating whether the transaction is new (true) or already existed (false). // This method is thread-safe. -func (s *batchTxStorage) Add(transaction tx.Transaction, batch *topicreadercommon.PublicBatch) (txAlreadyExists bool) { +func (s *batchTxStorage) GetOrCreateTransactionBatches(transaction tx.Transaction) (*transactionBatches, bool) { s.m.Lock() defer s.m.Unlock() txID := transaction.ID() - _, exists := s.batches[txID] - s.batches[txID] = append(s.batches[txID], batch) + txBatches, exists := s.transactions[txID] + if !exists { + txBatches = &transactionBatches{ + batches: make([]*topicreadercommon.PublicBatch, 0), + } + s.transactions[txID] = txBatches + } - return exists + return txBatches, !exists } // GetBatches returns all batches stored for the given transaction. @@ -56,12 +77,12 @@ func (s *batchTxStorage) GetBatches(transaction tx.Transaction) []*topicreaderco s.m.Lock() defer s.m.Unlock() - batches, ok := s.batches[transaction.ID()] + txBatches, ok := s.transactions[transaction.ID()] if !ok { return nil } - return batches + return txBatches.GetBatches() } // GetUpdateOffsetsInTransactionRequest builds an UpdateOffsetsInTransactionRequest @@ -75,10 +96,15 @@ func (s *batchTxStorage) GetUpdateOffsetsInTransactionRequest( transaction tx.Transaction, ) (*rawtopic.UpdateOffsetsInTransactionRequest, error) { s.m.Lock() - batches, ok := s.batches[transaction.ID()] + txBatches, ok := s.transactions[transaction.ID()] s.m.Unlock() - if !ok || len(batches) == 0 { + if !ok { + return nil, errNoBatches + } + + batches := txBatches.GetBatches() + if len(batches) == 0 { return nil, errNoBatches } @@ -195,5 +221,5 @@ func (s *batchTxStorage) Clear(transaction tx.Transaction) { s.m.Lock() defer s.m.Unlock() - delete(s.batches, transaction.ID()) + delete(s.transactions, transaction.ID()) } diff --git a/internal/topic/topicreaderinternal/batch_tx_storage_test.go b/internal/topic/topicreaderinternal/batch_tx_storage_test.go index 1c9d546c4..66265de65 100644 --- a/internal/topic/topicreaderinternal/batch_tx_storage_test.go +++ b/internal/topic/topicreaderinternal/batch_tx_storage_test.go @@ -18,9 +18,11 @@ func TestBatchTxStorageAdd_NewTransaction(t *testing.T) { tx := newMockTransactionWrapper("session-1", "tx-1") batch := createTestBatch("topic-1", 1, 10, 20, 1) - exists := newBatchTxStorage("test-consumer").Add(tx, batch) + storage := newBatchTxStorage("test-consumer") + txBatches, isNew := storage.GetOrCreateTransactionBatches(tx) + txBatches.AddBatch(batch) - assert.False(t, exists) + assert.True(t, isNew) } func TestBatchTxStorageAdd_ExistingTransaction(t *testing.T) { @@ -30,10 +32,12 @@ func TestBatchTxStorageAdd_ExistingTransaction(t *testing.T) { storage := newBatchTxStorage("test-consumer") - storage.Add(tx, batch1) - exists := storage.Add(tx, batch2) + txBatches1, _ := storage.GetOrCreateTransactionBatches(tx) + txBatches1.AddBatch(batch1) + _, isNew := storage.GetOrCreateTransactionBatches(tx) + txBatches1.AddBatch(batch2) - assert.True(t, exists) + assert.False(t, isNew) } func TestBatchTxStorageGetBatches_Empty(t *testing.T) { @@ -50,8 +54,9 @@ func TestBatchTxStorageGetBatches_WithBatches(t *testing.T) { batch2 := createTestBatch("topic-1", 1, 20, 30, 1) storage := newBatchTxStorage("test-consumer") - storage.Add(tx, batch1) - storage.Add(tx, batch2) + txBatches, _ := storage.GetOrCreateTransactionBatches(tx) + txBatches.AddBatch(batch1) + txBatches.AddBatch(batch2) batches := storage.GetBatches(tx) @@ -77,7 +82,8 @@ func TestBatchTxStorageGetUpdateOffsetsInTransactionRequest_SingleBatch(t *testi storage := newBatchTxStorage("test-consumer") - storage.Add(tx, batch) + txBatches, _ := storage.GetOrCreateTransactionBatches(tx) + txBatches.AddBatch(batch) req, err := storage.GetUpdateOffsetsInTransactionRequest(tx) @@ -101,8 +107,9 @@ func TestBatchTxStorageGetUpdateOffsetsInTransactionRequest_MultipleBatches(t *t batch2 := createTestBatch("topic-1", 1, 20, 30, 1) storage := newBatchTxStorage("test-consumer") - storage.Add(tx, batch1) - storage.Add(tx, batch2) + txBatches, _ := storage.GetOrCreateTransactionBatches(tx) + txBatches.AddBatch(batch1) + txBatches.AddBatch(batch2) req, err := storage.GetUpdateOffsetsInTransactionRequest(tx) @@ -121,8 +128,9 @@ func TestBatchTxStorageGetUpdateOffsetsInTransactionRequest_MultipleTopics(t *te batch2 := createTestBatch("topic-2", 2, 30, 40, 2) storage := newBatchTxStorage("test-consumer") - storage.Add(tx, batch1) - storage.Add(tx, batch2) + txBatches, _ := storage.GetOrCreateTransactionBatches(tx) + txBatches.AddBatch(batch1) + txBatches.AddBatch(batch2) req, err := storage.GetUpdateOffsetsInTransactionRequest(tx) @@ -140,8 +148,9 @@ func TestBatchTxStorageGetUpdateOffsetsInTransactionRequest_MultiplePartitionsSa batch2 := createTestBatch("topic-1", 2, 30, 40, 2) storage := newBatchTxStorage("test-consumer") - storage.Add(tx, batch1) - storage.Add(tx, batch2) + txBatches, _ := storage.GetOrCreateTransactionBatches(tx) + txBatches.AddBatch(batch1) + txBatches.AddBatch(batch2) req, err := storage.GetUpdateOffsetsInTransactionRequest(tx) @@ -168,8 +177,9 @@ func TestBatchTxStorageGetUpdateOffsetsInTransactionRequest_NonAdjacentBatches(t batch2 := createTestBatch("topic-1", 1, 30, 40, 1) storage := newBatchTxStorage("test-consumer") - storage.Add(tx, batch1) - storage.Add(tx, batch2) + txBatches, _ := storage.GetOrCreateTransactionBatches(tx) + txBatches.AddBatch(batch1) + txBatches.AddBatch(batch2) req, err := storage.GetUpdateOffsetsInTransactionRequest(tx) @@ -192,10 +202,11 @@ func TestBatchTxStorageGetUpdateOffsetsInTransactionRequest_MultiplePartitionsMu batch4 := createTestBatch("topic-2", 2, 70, 80, 4) storage := newBatchTxStorage("test-consumer") - storage.Add(tx, batch1) - storage.Add(tx, batch2) - storage.Add(tx, batch3) - storage.Add(tx, batch4) + txBatches, _ := storage.GetOrCreateTransactionBatches(tx) + txBatches.AddBatch(batch1) + txBatches.AddBatch(batch2) + txBatches.AddBatch(batch3) + txBatches.AddBatch(batch4) req, err := storage.GetUpdateOffsetsInTransactionRequest(tx) @@ -226,10 +237,11 @@ func TestBatchTxStorageGetUpdateOffsetsInTransactionRequest_ComplexOptimization( batch4 := createTestBatch("topic-1", 1, 50, 60, 1) storage := newBatchTxStorage("test-consumer") - storage.Add(tx, batch1) - storage.Add(tx, batch2) - storage.Add(tx, batch3) - storage.Add(tx, batch4) + txBatches, _ := storage.GetOrCreateTransactionBatches(tx) + txBatches.AddBatch(batch1) + txBatches.AddBatch(batch2) + txBatches.AddBatch(batch3) + txBatches.AddBatch(batch4) req, err := storage.GetUpdateOffsetsInTransactionRequest(tx) @@ -253,11 +265,12 @@ func TestBatchTxStorageGetUpdateOffsetsInTransactionRequest_MixedPartitionsAndTo batch5 := createTestBatch("topic-2", 1, 80, 90, 3) storage := newBatchTxStorage("test-consumer") - storage.Add(tx, batch1) - storage.Add(tx, batch2) - storage.Add(tx, batch3) - storage.Add(tx, batch4) - storage.Add(tx, batch5) + txBatches, _ := storage.GetOrCreateTransactionBatches(tx) + txBatches.AddBatch(batch1) + txBatches.AddBatch(batch2) + txBatches.AddBatch(batch3) + txBatches.AddBatch(batch4) + txBatches.AddBatch(batch5) req, err := storage.GetUpdateOffsetsInTransactionRequest(tx) @@ -292,7 +305,8 @@ func TestBatchTxStorageClear(t *testing.T) { batch := createTestBatch("topic-1", 1, 10, 20, 1) storage := newBatchTxStorage("test-consumer") - storage.Add(tx, batch) + txBatches, _ := storage.GetOrCreateTransactionBatches(tx) + txBatches.AddBatch(batch) storage.Clear(tx) @@ -314,8 +328,10 @@ func TestBatchTxStorageMultipleTransactions(t *testing.T) { batch2 := createTestBatch("topic-2", 2, 30, 40, 2) storage := newBatchTxStorage("test-consumer") - storage.Add(tx1, batch1) - storage.Add(tx2, batch2) + txBatches1, _ := storage.GetOrCreateTransactionBatches(tx1) + txBatches1.AddBatch(batch1) + txBatches2, _ := storage.GetOrCreateTransactionBatches(tx2) + txBatches2.AddBatch(batch2) batches1 := storage.GetBatches(tx1) batches2 := storage.GetBatches(tx2) diff --git a/internal/topic/topicreaderinternal/stream_reader_impl.go b/internal/topic/topicreaderinternal/stream_reader_impl.go index c92296e83..6f7de9d7c 100644 --- a/internal/topic/topicreaderinternal/stream_reader_impl.go +++ b/internal/topic/topicreaderinternal/stream_reader_impl.go @@ -251,8 +251,9 @@ func (r *topicStreamReaderImpl) commitWithTransaction( return fmt.Errorf("ydb: failed to materialize transaction: %w", err) } - txAlreadyExists := r.batchTxStorage.Add(tx, batch) - if txAlreadyExists { + txBatches, isNew := r.batchTxStorage.GetOrCreateTransactionBatches(tx) + txBatches.AddBatch(batch) + if !isNew { // tx hooks already configured - exiting return nil } From afcb483fa480479f83eef676ec4ac7aed2bf9580 Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Thu, 11 Dec 2025 20:20:09 +0300 Subject: [PATCH 26/27] Enhance error handling in batch transaction storage by wrapping errors with stack traces and improve return values for transaction batch creation. --- .../topicreaderinternal/batch_tx_storage.go | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/internal/topic/topicreaderinternal/batch_tx_storage.go b/internal/topic/topicreaderinternal/batch_tx_storage.go index bd6870efc..46203f56f 100644 --- a/internal/topic/topicreaderinternal/batch_tx_storage.go +++ b/internal/topic/topicreaderinternal/batch_tx_storage.go @@ -12,6 +12,7 @@ import ( "github.com/ydb-platform/ydb-go-sdk/v3/internal/operation" "github.com/ydb-platform/ydb-go-sdk/v3/internal/topic/topicreadercommon" "github.com/ydb-platform/ydb-go-sdk/v3/internal/tx" + "github.com/ydb-platform/ydb-go-sdk/v3/internal/xerrors" "github.com/ydb-platform/ydb-go-sdk/v3/internal/xsync" ) @@ -52,9 +53,8 @@ func newBatchTxStorage(consumer string) *batchTxStorage { } // GetOrCreateTransactionBatches gets or creates a transaction batches handler for the given transaction. -// It returns the handler and a flag indicating whether the transaction is new (true) or already existed (false). // This method is thread-safe. -func (s *batchTxStorage) GetOrCreateTransactionBatches(transaction tx.Transaction) (*transactionBatches, bool) { +func (s *batchTxStorage) GetOrCreateTransactionBatches(transaction tx.Transaction) (batches *transactionBatches, created bool) { s.m.Lock() defer s.m.Unlock() @@ -65,9 +65,10 @@ func (s *batchTxStorage) GetOrCreateTransactionBatches(transaction tx.Transactio batches: make([]*topicreadercommon.PublicBatch, 0), } s.transactions[txID] = txBatches + created = true } - return txBatches, !exists + return txBatches, created } // GetBatches returns all batches stored for the given transaction. @@ -100,12 +101,12 @@ func (s *batchTxStorage) GetUpdateOffsetsInTransactionRequest( s.m.Unlock() if !ok { - return nil, errNoBatches + return nil, xerrors.WithStackTrace(errNoBatches) } batches := txBatches.GetBatches() if len(batches) == 0 { - return nil, errNoBatches + return nil, xerrors.WithStackTrace(errNoBatches) } // Convert batches to CommitRanges @@ -124,7 +125,7 @@ func (s *batchTxStorage) GetUpdateOffsetsInTransactionRequest( // Convert to partition offsets partitionOffsets := commitRanges.ToPartitionsOffsets() if len(partitionOffsets) == 0 { - return nil, errNoBatches + return nil, xerrors.WithStackTrace(errNoBatches) } // Group partition offsets by topic @@ -133,7 +134,7 @@ func (s *batchTxStorage) GetUpdateOffsetsInTransactionRequest( return nil, err } if len(topicMap) == 0 { - return nil, errNoBatches + return nil, xerrors.WithStackTrace(errNoBatches) } // Build request @@ -174,7 +175,7 @@ func (s *batchTxStorage) buildPartitionOffsetsMap( po := &partitionOffsets[i] info, ok := sessionInfoMap[po.PartitionSessionID] if !ok { - return nil, fmt.Errorf("session info not found for partition session ID %d", po.PartitionSessionID) + return nil, xerrors.WithStackTrace(fmt.Errorf("session info not found for partition session ID %d", po.PartitionSessionID)) } topicMap[info.topic] = append(topicMap[info.topic], rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets{ From e9d74ec838b8800aacbe161131e4791cd5a32ee4 Mon Sep 17 00:00:00 2001 From: Konstantin Prokopenko Date: Thu, 11 Dec 2025 20:21:44 +0300 Subject: [PATCH 27/27] Refactor GetOrCreateTransactionBatches and buildPartitionOffsetsMap for improved readability by formatting function parameters and error messages across multiple lines. --- internal/topic/topicreaderinternal/batch_tx_storage.go | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/internal/topic/topicreaderinternal/batch_tx_storage.go b/internal/topic/topicreaderinternal/batch_tx_storage.go index 46203f56f..021937108 100644 --- a/internal/topic/topicreaderinternal/batch_tx_storage.go +++ b/internal/topic/topicreaderinternal/batch_tx_storage.go @@ -54,7 +54,9 @@ func newBatchTxStorage(consumer string) *batchTxStorage { // GetOrCreateTransactionBatches gets or creates a transaction batches handler for the given transaction. // This method is thread-safe. -func (s *batchTxStorage) GetOrCreateTransactionBatches(transaction tx.Transaction) (batches *transactionBatches, created bool) { +func (s *batchTxStorage) GetOrCreateTransactionBatches( + transaction tx.Transaction, +) (batches *transactionBatches, created bool) { s.m.Lock() defer s.m.Unlock() @@ -175,7 +177,9 @@ func (s *batchTxStorage) buildPartitionOffsetsMap( po := &partitionOffsets[i] info, ok := sessionInfoMap[po.PartitionSessionID] if !ok { - return nil, xerrors.WithStackTrace(fmt.Errorf("session info not found for partition session ID %d", po.PartitionSessionID)) + return nil, xerrors.WithStackTrace( + fmt.Errorf("session info not found for partition session ID %d", po.PartitionSessionID), + ) } topicMap[info.topic] = append(topicMap[info.topic], rawtopic.UpdateOffsetsInTransactionRequest_PartitionOffsets{