Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
36 changes: 36 additions & 0 deletions clients/src/main/resources/common/message/AbortedTxn.json
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
// Licensed to the Apache Software Foundation (ASF) under one or more
// contributor license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright ownership.
// The ASF licenses this file to You under the Apache License, Version 2.0
// (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

{
"type": "data",
"name": "AbortedTxn",
"validVersions": "0",
"flexibleVersions": "none",
"fields": [
{ "name": "ProducerId", "type": "int64", "versions": "0+",
"about": "The producer id associated with the aborted transaction"
},
{ "name": "FirstOffset", "type": "int64", "versions": "0+",
"about": "The first offset in the aborted transaction"
},
{ "name": "LastOffset", "type": "int64", "versions": "0+",
"about": "The last offset in the aborted transaction"
},
{ "name": "LastStableOffset", "type": "int64", "versions": "0+",
"about": "The last stable offset at the time the transaction was aborted"
}
// Note: adding new fields may require TransactionIndex to be refactored to read version-per-record.
]
}
7 changes: 4 additions & 3 deletions core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,8 @@ import org.apache.kafka.coordinator.transaction.TransactionLogConfig
import org.apache.kafka.server.common.{RequestLocal, TransactionVersion}
import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics}
import org.apache.kafka.server.util.MockTime
import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, CleanedTransactionMetadata, Cleaner, CleanerConfig, CleanerStats, LocalLog, LogAppendInfo, LogCleaner, LogCleanerManager, LogCleaningAbortedException, LogConfig, LogDirFailureChannel, LogFileUtils, LogLoader, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogToClean, OffsetMap, ProducerStateManager, ProducerStateManagerConfig, UnifiedLog, VerificationGuard}
import org.apache.kafka.common.message.AbortedTxn
import org.apache.kafka.storage.internals.log.{AppendOrigin, CleanedTransactionMetadata, Cleaner, CleanerConfig, CleanerStats, LocalLog, LogAppendInfo, LogCleaner, LogCleanerManager, LogCleaningAbortedException, LogConfig, LogDirFailureChannel, LogFileUtils, LogLoader, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, LogToClean, OffsetMap, ProducerStateManager, ProducerStateManagerConfig, UnifiedLog, VerificationGuard}
import org.apache.kafka.storage.internals.utils.Throttler
import org.apache.kafka.storage.log.metrics.BrokerTopicStats
import org.junit.jupiter.api.Assertions._
Expand Down Expand Up @@ -417,8 +418,8 @@ class LogCleanerTest extends Logging {
assertEquals(20L, log.logEndOffset)

val expectedAbortedTxns = util.List.of(
new AbortedTxn(producerId1, 8, 10, 11),
new AbortedTxn(producerId2, 11, 16, 17)
new AbortedTxn().setProducerId(producerId1).setFirstOffset(8).setLastOffset(10).setLastStableOffset(11),
new AbortedTxn().setProducerId(producerId2).setFirstOffset(11).setLastOffset(16).setLastStableOffset(17)
)

assertAllTransactionsComplete(log)
Expand Down
9 changes: 5 additions & 4 deletions core/src/test/scala/unit/kafka/log/LogLoaderTest.scala
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,8 @@ import org.apache.kafka.metadata.MockConfigRepository
import org.apache.kafka.server.common.TransactionVersion
import org.apache.kafka.server.util.{MockTime, Scheduler}
import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache
import org.apache.kafka.storage.internals.log.{AbortedTxn, CleanerConfig, EpochEntry, LocalLog, LogConfig, LogDirFailureChannel, LogFileUtils, LogLoader, LogOffsetMetadata, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, OffsetIndex, ProducerStateManager, ProducerStateManagerConfig, SnapshotFile, UnifiedLog}
import org.apache.kafka.common.message.AbortedTxn
import org.apache.kafka.storage.internals.log.{CleanerConfig, EpochEntry, LocalLog, LogConfig, LogDirFailureChannel, LogFileUtils, LogLoader, LogOffsetMetadata, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, OffsetIndex, ProducerStateManager, ProducerStateManagerConfig, SnapshotFile, UnifiedLog}
import org.apache.kafka.storage.internals.checkpoint.CleanShutdownFileHandler
import org.apache.kafka.storage.log.metrics.BrokerTopicStats
import org.junit.jupiter.api.Assertions.{assertDoesNotThrow, assertEquals, assertFalse, assertNotEquals, assertThrows, assertTrue}
Expand Down Expand Up @@ -1281,7 +1282,7 @@ class LogLoaderTest {
val reloadedLogConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 5)
val reloadedLog = createLog(logDir, reloadedLogConfig, lastShutdownClean = false)
val abortedTransactions = LogTestUtils.allAbortedTransactions(reloadedLog)
assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions)
assertEquals(List(new AbortedTxn().setProducerId(pid1).setFirstOffset(0L).setLastOffset(29L).setLastStableOffset(8L), new AbortedTxn().setProducerId(pid2).setFirstOffset(8L).setLastOffset(74L).setLastStableOffset(36L)), abortedTransactions)
}

@Test
Expand Down Expand Up @@ -1332,7 +1333,7 @@ class LogLoaderTest {
val reloadedLogConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 5)
val reloadedLog = createLog(logDir, reloadedLogConfig, recoveryPoint = recoveryPoint, lastShutdownClean = false)
val abortedTransactions = LogTestUtils.allAbortedTransactions(reloadedLog)
assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions)
assertEquals(List(new AbortedTxn().setProducerId(pid1).setFirstOffset(0L).setLastOffset(29L).setLastStableOffset(8L), new AbortedTxn().setProducerId(pid2).setFirstOffset(8L).setLastOffset(74L).setLastStableOffset(36L)), abortedTransactions)
}

@Test
Expand Down Expand Up @@ -1386,7 +1387,7 @@ class LogLoaderTest {
val reloadedLogConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 5)
val reloadedLog = createLog(logDir, reloadedLogConfig, recoveryPoint = recoveryPoint, lastShutdownClean = false)
val abortedTransactions = LogTestUtils.allAbortedTransactions(reloadedLog)
assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions)
assertEquals(List(new AbortedTxn().setProducerId(pid1).setFirstOffset(0L).setLastOffset(29L).setLastStableOffset(8L), new AbortedTxn().setProducerId(pid2).setFirstOffset(8L).setLastOffset(74L).setLastStableOffset(36L)), abortedTransactions)
}

@Test
Expand Down
3 changes: 2 additions & 1 deletion core/src/test/scala/unit/kafka/log/LogTestUtils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,8 @@ import org.apache.kafka.server.log.remote.storage.RemoteLogManager
import org.apache.kafka.server.storage.log.FetchIsolation
import org.apache.kafka.server.util.Scheduler
import org.apache.kafka.storage.internals.log.LogConfig.{DEFAULT_REMOTE_LOG_COPY_DISABLE_CONFIG, DEFAULT_REMOTE_LOG_DELETE_ON_DISABLE_CONFIG}
import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, FetchDataInfo, LazyIndex, LogAppendInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogOffsetsListener, LogSegment, ProducerStateManager, ProducerStateManagerConfig, TransactionIndex, VerificationGuard, UnifiedLog}
import org.apache.kafka.common.message.AbortedTxn
import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchDataInfo, LazyIndex, LogAppendInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogOffsetsListener, LogSegment, ProducerStateManager, ProducerStateManagerConfig, TransactionIndex, VerificationGuard, UnifiedLog}
import org.apache.kafka.storage.log.metrics.BrokerTopicStats

import scala.jdk.CollectionConverters._
Expand Down
7 changes: 4 additions & 3 deletions core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,8 @@ import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.apache.kafka.server.storage.log.FetchIsolation
import org.apache.kafka.server.util.{MockTime, Scheduler}

import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, LogConfig, LogFileUtils, LogOffsetMetadata, LogOffsetSnapshot, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, OffsetResultHolder, ProducerStateManagerConfig, UnifiedLog, VerificationGuard}
import org.apache.kafka.common.message.AbortedTxn
import org.apache.kafka.storage.internals.log.{AppendOrigin, LogConfig, LogFileUtils, LogOffsetMetadata, LogOffsetSnapshot, LogOffsetsListener, LogSegment, LogSegments, LogStartOffsetIncrementReason, OffsetResultHolder, ProducerStateManagerConfig, UnifiedLog, VerificationGuard}
import org.apache.kafka.storage.log.metrics.BrokerTopicStats
import org.junit.jupiter.api.Assertions.{assertDoesNotThrow, _}
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
Expand Down Expand Up @@ -128,8 +129,8 @@ class UnifiedLogTest {

val abortedTransactions = LogTestUtils.allAbortedTransactions(log)
val expectedTransactions = List(
new AbortedTxn(pid1, 0L, 29L, 8L),
new AbortedTxn(pid2, 8L, 74L, 36L)
new AbortedTxn().setProducerId(pid1).setFirstOffset(0L).setLastOffset(29L).setLastStableOffset(8L),
new AbortedTxn().setProducerId(pid2).setFirstOffset(8L).setLastOffset(74L).setLastStableOffset(36L)
)

assertEquals(expectedTransactions, abortedTransactions)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.kafka.common.errors.RetriableException;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.internals.SecurityManagerCompatibility;
import org.apache.kafka.common.message.AbortedTxn;
import org.apache.kafka.common.message.FetchResponseData;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.metrics.Quota;
Expand Down Expand Up @@ -60,7 +61,6 @@
import org.apache.kafka.server.storage.log.FetchIsolation;
import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile;
import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache;
import org.apache.kafka.storage.internals.log.AbortedTxn;
import org.apache.kafka.storage.internals.log.AsyncOffsetReadFutureHolder;
import org.apache.kafka.storage.internals.log.AsyncOffsetReader;
import org.apache.kafka.storage.internals.log.EpochEntry;
Expand Down Expand Up @@ -1893,7 +1893,9 @@ private FetchDataInfo addAbortedTransactions(long startOffset,

Consumer<List<AbortedTxn>> accumulator =
abortedTxns -> abortedTransactions.addAll(abortedTxns.stream()
.map(AbortedTxn::asAbortedTransaction).toList());
.map(txn -> new FetchResponseData.AbortedTransaction()
.setProducerId(txn.producerId())
.setFirstOffset(txn.firstOffset())).toList());

long startTimeNs = time.nanoseconds();
collectAbortedTransactions(startOffset, upperBoundOffset, segmentMetadata, accumulator, log);
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
*/
package org.apache.kafka.storage.internals.log;

import org.apache.kafka.common.message.AbortedTxn;
import org.apache.kafka.common.record.internal.ControlRecordType;
import org.apache.kafka.common.record.internal.Record;
import org.apache.kafka.common.record.internal.RecordBatch;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.CorruptRecordException;
import org.apache.kafka.common.message.AbortedTxn;
import org.apache.kafka.common.record.internal.FileRecords;
import org.apache.kafka.common.record.internal.MemoryRecords;
import org.apache.kafka.common.record.internal.MutableRecordBatch;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.KafkaStorageException;
import org.apache.kafka.common.errors.OffsetOutOfRangeException;
import org.apache.kafka.common.message.AbortedTxn;
import org.apache.kafka.common.message.FetchResponseData;
import org.apache.kafka.common.record.internal.FileLogInputStream;
import org.apache.kafka.common.record.internal.FileRecords;
Expand Down Expand Up @@ -540,7 +541,9 @@ FetchDataInfo addAbortedTransactions(long startOffset, LogSegment segment, Fetch
List<FetchResponseData.AbortedTransaction> abortedTransactions = new ArrayList<>();
Consumer<List<AbortedTxn>> accumulator = abortedTxns -> {
for (AbortedTxn abortedTxn : abortedTxns)
abortedTransactions.add(abortedTxn.asAbortedTransaction());
abortedTransactions.add(new FetchResponseData.AbortedTransaction()
.setProducerId(abortedTxn.producerId())
.setFirstOffset(abortedTxn.firstOffset()));
};
collectAbortedTransactions(startOffset, upperBoundOffset, segment, accumulator);
return new FetchDataInfo(fetchInfo.fetchOffsetMetadata,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

import org.apache.kafka.common.InvalidRecordException;
import org.apache.kafka.common.errors.CorruptRecordException;
import org.apache.kafka.common.message.AbortedTxn;
import org.apache.kafka.common.record.internal.FileLogInputStream.FileChannelRecordBatch;
import org.apache.kafka.common.record.internal.FileRecords;
import org.apache.kafka.common.record.internal.FileRecords.LogOffsetPosition;
Expand Down Expand Up @@ -348,7 +349,11 @@ public int appendFromFile(FileRecords records, int start) throws IOException {
public void updateTxnIndex(CompletedTxn completedTxn, long lastStableOffset) throws IOException {
if (completedTxn.isAborted()) {
LOGGER.trace("Writing aborted transaction {} to transaction index, last stable offset is {}", completedTxn, lastStableOffset);
txnIndex.append(new AbortedTxn(completedTxn, lastStableOffset));
txnIndex.append(new AbortedTxn()
.setProducerId(completedTxn.producerId())
.setFirstOffset(completedTxn.firstOffset())
.setLastOffset(completedTxn.lastOffset())
.setLastStableOffset(lastStableOffset));
}
}

Expand Down
Loading
Loading