You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by jg...@apache.org on 2017/05/06 18:51:10 UTC
[1/6] kafka git commit: KAFKA-5121;
Implement transaction index for KIP-98
Repository: kafka
Updated Branches:
refs/heads/trunk 29994dd10 -> e71dce89c
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala b/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala
new file mode 100644
index 0000000..e8c918d
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala
@@ -0,0 +1,562 @@
+/**
+ * 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.
+ */
+
+package kafka.log
+
+import java.io.File
+
+import kafka.server.LogOffsetMetadata
+import kafka.utils.TestUtils
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.errors._
+import org.apache.kafka.common.record.{ControlRecordType, EndTransactionMarker, RecordBatch}
+import org.apache.kafka.common.utils.{MockTime, Utils}
+import org.junit.Assert._
+import org.junit.{After, Before, Test}
+import org.scalatest.junit.JUnitSuite
+
+class ProducerStateManagerTest extends JUnitSuite {
+ var idMappingDir: File = null
+ var idMapping: ProducerStateManager = null
+ val partition = new TopicPartition("test", 0)
+ val pid = 1L
+ val maxPidExpirationMs = 60 * 1000
+ val time = new MockTime
+
+ @Before
+ def setUp(): Unit = {
+ idMappingDir = TestUtils.tempDir()
+ idMapping = new ProducerStateManager(partition, idMappingDir, maxPidExpirationMs)
+ }
+
+ @After
+ def tearDown(): Unit = {
+ Utils.delete(idMappingDir)
+ }
+
+ @Test
+ def testBasicIdMapping(): Unit = {
+ val epoch = 0.toShort
+
+ // First entry for id 0 added
+ append(idMapping, pid, 0, epoch, 0L, 0L)
+
+ // Second entry for id 0 added
+ append(idMapping, pid, 1, epoch, 0L, 1L)
+
+ // Duplicate sequence number (matches previous sequence number)
+ assertThrows[DuplicateSequenceNumberException] {
+ append(idMapping, pid, 1, epoch, 0L, 1L)
+ }
+
+ // Invalid sequence number (greater than next expected sequence number)
+ assertThrows[OutOfOrderSequenceException] {
+ append(idMapping, pid, 5, epoch, 0L, 2L)
+ }
+
+ // Change epoch
+ append(idMapping, pid, 0, (epoch + 1).toShort, 0L, 3L)
+
+ // Incorrect epoch
+ assertThrows[ProducerFencedException] {
+ append(idMapping, pid, 0, epoch, 0L, 4L)
+ }
+ }
+
+ @Test
+ def testNoValidationOnFirstEntryWhenLoadingLog(): Unit = {
+ val epoch = 5.toShort
+ val sequence = 16
+ val offset = 735L
+ append(idMapping, pid, sequence, epoch, offset, isLoadingFromLog = true)
+
+ val maybeLastEntry = idMapping.lastEntry(pid)
+ assertTrue(maybeLastEntry.isDefined)
+
+ val lastEntry = maybeLastEntry.get
+ assertEquals(epoch, lastEntry.producerEpoch)
+ assertEquals(sequence, lastEntry.firstSeq)
+ assertEquals(sequence, lastEntry.lastSeq)
+ assertEquals(offset, lastEntry.lastOffset)
+ assertEquals(offset, lastEntry.firstOffset)
+ }
+
+ @Test
+ def testControlRecordBumpsEpoch(): Unit = {
+ val epoch = 0.toShort
+ append(idMapping, pid, 0, epoch, 0L)
+
+ val bumpedEpoch = 1.toShort
+ val (completedTxn, lastStableOffset) = appendEndTxnMarker(idMapping, pid, bumpedEpoch, ControlRecordType.ABORT, 1L)
+ assertEquals(1L, completedTxn.firstOffset)
+ assertEquals(1L, completedTxn.lastOffset)
+ assertEquals(2L, lastStableOffset)
+ assertTrue(completedTxn.isAborted)
+ assertEquals(pid, completedTxn.producerId)
+
+ val maybeLastEntry = idMapping.lastEntry(pid)
+ assertTrue(maybeLastEntry.isDefined)
+
+ val lastEntry = maybeLastEntry.get
+ assertEquals(bumpedEpoch, lastEntry.producerEpoch)
+ assertEquals(None, lastEntry.currentTxnFirstOffset)
+ assertEquals(RecordBatch.NO_SEQUENCE, lastEntry.firstSeq)
+ assertEquals(RecordBatch.NO_SEQUENCE, lastEntry.lastSeq)
+
+ // should be able to append with the new epoch if we start at sequence 0
+ append(idMapping, pid, 0, bumpedEpoch, 2L)
+ assertEquals(Some(0), idMapping.lastEntry(pid).map(_.firstSeq))
+ }
+
+ @Test
+ def testTxnFirstOffsetMetadataCached(): Unit = {
+ val producerEpoch = 0.toShort
+ val offset = 992342L
+ val seq = 0
+ val producerAppendInfo = new ProducerAppendInfo(pid, None, false)
+ producerAppendInfo.append(producerEpoch, seq, seq, time.milliseconds(), offset, isTransactional = true)
+
+ val logOffsetMetadata = new LogOffsetMetadata(messageOffset = offset, segmentBaseOffset = 990000L,
+ relativePositionInSegment = 234224)
+ producerAppendInfo.maybeCacheTxnFirstOffsetMetadata(logOffsetMetadata)
+ idMapping.update(producerAppendInfo)
+
+ assertEquals(Some(logOffsetMetadata), idMapping.firstUnstableOffset)
+ }
+
+ @Test
+ def testNonMatchingTxnFirstOffsetMetadataNotCached(): Unit = {
+ val producerEpoch = 0.toShort
+ val offset = 992342L
+ val seq = 0
+ val producerAppendInfo = new ProducerAppendInfo(pid, None, false)
+ producerAppendInfo.append(producerEpoch, seq, seq, time.milliseconds(), offset, isTransactional = true)
+
+ // use some other offset to simulate a follower append where the log offset metadata won't typically
+ // match any of the transaction first offsets
+ val logOffsetMetadata = new LogOffsetMetadata(messageOffset = offset - 23429, segmentBaseOffset = 990000L,
+ relativePositionInSegment = 234224)
+ producerAppendInfo.maybeCacheTxnFirstOffsetMetadata(logOffsetMetadata)
+ idMapping.update(producerAppendInfo)
+
+ assertEquals(Some(LogOffsetMetadata(offset)), idMapping.firstUnstableOffset)
+ }
+
+ @Test
+ def updateProducerTransactionState(): Unit = {
+ val producerEpoch = 0.toShort
+ val coordinatorEpoch = 15
+ val offset = 9L
+ append(idMapping, pid, 0, producerEpoch, offset)
+
+ val appendInfo = new ProducerAppendInfo(pid, idMapping.lastEntry(pid), loadingFromLog = false)
+ appendInfo.append(producerEpoch, 1, 5, time.milliseconds(), 20L, isTransactional = true)
+ var lastEntry = appendInfo.lastEntry
+ assertEquals(producerEpoch, lastEntry.producerEpoch)
+ assertEquals(1, lastEntry.firstSeq)
+ assertEquals(5, lastEntry.lastSeq)
+ assertEquals(16L, lastEntry.firstOffset)
+ assertEquals(20L, lastEntry.lastOffset)
+ assertEquals(Some(16L), lastEntry.currentTxnFirstOffset)
+ assertEquals(List(new TxnMetadata(pid, 16L)), appendInfo.startedTransactions)
+
+ appendInfo.append(producerEpoch, 6, 10, time.milliseconds(), 30L, isTransactional = true)
+ lastEntry = appendInfo.lastEntry
+ assertEquals(producerEpoch, lastEntry.producerEpoch)
+ assertEquals(6, lastEntry.firstSeq)
+ assertEquals(10, lastEntry.lastSeq)
+ assertEquals(26L, lastEntry.firstOffset)
+ assertEquals(30L, lastEntry.lastOffset)
+ assertEquals(Some(16L), lastEntry.currentTxnFirstOffset)
+ assertEquals(List(new TxnMetadata(pid, 16L)), appendInfo.startedTransactions)
+
+ val endTxnMarker = new EndTransactionMarker(ControlRecordType.COMMIT, coordinatorEpoch)
+ val completedTxn = appendInfo.appendEndTxnMarker(endTxnMarker, producerEpoch, 40L, time.milliseconds())
+ assertEquals(pid, completedTxn.producerId)
+ assertEquals(16L, completedTxn.firstOffset)
+ assertEquals(40L, completedTxn.lastOffset)
+ assertFalse(completedTxn.isAborted)
+
+ lastEntry = appendInfo.lastEntry
+ assertEquals(producerEpoch, lastEntry.producerEpoch)
+ assertEquals(10, lastEntry.firstSeq)
+ assertEquals(10, lastEntry.lastSeq)
+ assertEquals(40L, lastEntry.firstOffset)
+ assertEquals(40L, lastEntry.lastOffset)
+ assertEquals(coordinatorEpoch, lastEntry.coordinatorEpoch)
+ assertEquals(None, lastEntry.currentTxnFirstOffset)
+ assertEquals(List(new TxnMetadata(pid, 16L)), appendInfo.startedTransactions)
+ }
+
+ @Test(expected = classOf[OutOfOrderSequenceException])
+ def testOutOfSequenceAfterControlRecordEpochBump(): Unit = {
+ val epoch = 0.toShort
+ append(idMapping, pid, 0, epoch, 0L)
+ append(idMapping, pid, 1, epoch, 1L)
+
+ val bumpedEpoch = 1.toShort
+ appendEndTxnMarker(idMapping, pid, bumpedEpoch, ControlRecordType.ABORT, 1L)
+
+ // next append is invalid since we expect the sequence to be reset
+ append(idMapping, pid, 2, bumpedEpoch, 2L)
+ }
+
+ @Test(expected = classOf[InvalidTxnStateException])
+ def testNonTransactionalAppendWithOngoingTransaction(): Unit = {
+ val epoch = 0.toShort
+ append(idMapping, pid, 0, epoch, 0L, isTransactional = true)
+ append(idMapping, pid, 1, epoch, 1L, isTransactional = false)
+ }
+
+ @Test
+ def testTakeSnapshot(): Unit = {
+ val epoch = 0.toShort
+ append(idMapping, pid, 0, epoch, 0L, 0L)
+ append(idMapping, pid, 1, epoch, 1L, 1L)
+
+ // Take snapshot
+ idMapping.takeSnapshot()
+
+ // Check that file exists and it is not empty
+ assertEquals("Directory doesn't contain a single file as expected", 1, idMappingDir.list().length)
+ assertTrue("Snapshot file is empty", idMappingDir.list().head.length > 0)
+ }
+
+ @Test
+ def testRecoverFromSnapshot(): Unit = {
+ val epoch = 0.toShort
+ append(idMapping, pid, 0, epoch, 0L)
+ append(idMapping, pid, 1, epoch, 1L)
+
+ idMapping.takeSnapshot()
+ val recoveredMapping = new ProducerStateManager(partition, idMappingDir, maxPidExpirationMs)
+ recoveredMapping.truncateAndReload(0L, 3L, time.milliseconds)
+
+ // entry added after recovery
+ append(recoveredMapping, pid, 2, epoch, 2L)
+ }
+
+ @Test(expected = classOf[OutOfOrderSequenceException])
+ def testRemoveExpiredPidsOnReload(): Unit = {
+ val epoch = 0.toShort
+ append(idMapping, pid, 0, epoch, 0L, 0)
+ append(idMapping, pid, 1, epoch, 1L, 1)
+
+ idMapping.takeSnapshot()
+ val recoveredMapping = new ProducerStateManager(partition, idMappingDir, maxPidExpirationMs)
+ recoveredMapping.truncateAndReload(0L, 1L, 70000)
+
+ // entry added after recovery. The pid should be expired now, and would not exist in the pid mapping. Hence
+ // we should get an out of order sequence exception.
+ append(recoveredMapping, pid, 2, epoch, 2L, 70001)
+ }
+
+ @Test
+ def testDeleteSnapshotsBefore(): Unit = {
+ val epoch = 0.toShort
+ append(idMapping, pid, 0, epoch, 0L)
+ append(idMapping, pid, 1, epoch, 1L)
+ idMapping.takeSnapshot()
+ assertEquals(1, idMappingDir.listFiles().length)
+ assertEquals(Set(2), currentSnapshotOffsets)
+
+ append(idMapping, pid, 2, epoch, 2L)
+ idMapping.takeSnapshot()
+ assertEquals(2, idMappingDir.listFiles().length)
+ assertEquals(Set(2, 3), currentSnapshotOffsets)
+
+ idMapping.deleteSnapshotsBefore(3L)
+ assertEquals(1, idMappingDir.listFiles().length)
+ assertEquals(Set(3), currentSnapshotOffsets)
+
+ idMapping.deleteSnapshotsBefore(4L)
+ assertEquals(0, idMappingDir.listFiles().length)
+ assertEquals(Set(), currentSnapshotOffsets)
+ }
+
+ @Test
+ def testTruncate(): Unit = {
+ val epoch = 0.toShort
+
+ append(idMapping, pid, 0, epoch, 0L)
+ append(idMapping, pid, 1, epoch, 1L)
+ idMapping.takeSnapshot()
+ assertEquals(1, idMappingDir.listFiles().length)
+ assertEquals(Set(2), currentSnapshotOffsets)
+
+ append(idMapping, pid, 2, epoch, 2L)
+ idMapping.takeSnapshot()
+ assertEquals(2, idMappingDir.listFiles().length)
+ assertEquals(Set(2, 3), currentSnapshotOffsets)
+
+ idMapping.truncate()
+
+ assertEquals(0, idMappingDir.listFiles().length)
+ assertEquals(Set(), currentSnapshotOffsets)
+
+ append(idMapping, pid, 0, epoch, 0L)
+ idMapping.takeSnapshot()
+ assertEquals(1, idMappingDir.listFiles().length)
+ assertEquals(Set(1), currentSnapshotOffsets)
+ }
+
+ @Test
+ def testFirstUnstableOffsetAfterTruncation(): Unit = {
+ val epoch = 0.toShort
+ val sequence = 0
+
+ append(idMapping, pid, sequence, epoch, offset = 99, isTransactional = true)
+ assertEquals(Some(99), idMapping.firstUnstableOffset.map(_.messageOffset))
+ idMapping.takeSnapshot()
+
+ appendEndTxnMarker(idMapping, pid, epoch, ControlRecordType.COMMIT, offset = 105)
+ idMapping.onHighWatermarkUpdated(106)
+ assertEquals(None, idMapping.firstUnstableOffset.map(_.messageOffset))
+ idMapping.takeSnapshot()
+
+ append(idMapping, pid, sequence + 1, epoch, offset = 106)
+ idMapping.truncateAndReload(0L, 106, time.milliseconds())
+ assertEquals(None, idMapping.firstUnstableOffset.map(_.messageOffset))
+
+ idMapping.truncateAndReload(0L, 100L, time.milliseconds())
+ assertEquals(Some(99), idMapping.firstUnstableOffset.map(_.messageOffset))
+ }
+
+ @Test
+ def testFirstUnstableOffsetAfterEviction(): Unit = {
+ val epoch = 0.toShort
+ val sequence = 0
+ append(idMapping, pid, sequence, epoch, offset = 99, isTransactional = true)
+ assertEquals(Some(99), idMapping.firstUnstableOffset.map(_.messageOffset))
+ append(idMapping, 2L, 0, epoch, offset = 106, isTransactional = true)
+ idMapping.evictUnretainedProducers(100)
+ assertEquals(Some(106), idMapping.firstUnstableOffset.map(_.messageOffset))
+ }
+
+ @Test
+ def testEvictUnretainedPids(): Unit = {
+ val epoch = 0.toShort
+
+ append(idMapping, pid, 0, epoch, 0L)
+ append(idMapping, pid, 1, epoch, 1L)
+ idMapping.takeSnapshot()
+
+ val anotherPid = 2L
+ append(idMapping, anotherPid, 0, epoch, 2L)
+ append(idMapping, anotherPid, 1, epoch, 3L)
+ idMapping.takeSnapshot()
+ assertEquals(Set(2, 4), currentSnapshotOffsets)
+
+ idMapping.evictUnretainedProducers(2)
+ assertEquals(Set(4), currentSnapshotOffsets)
+ assertEquals(Set(anotherPid), idMapping.activeProducers.keySet)
+ assertEquals(None, idMapping.lastEntry(pid))
+
+ val maybeEntry = idMapping.lastEntry(anotherPid)
+ assertTrue(maybeEntry.isDefined)
+ assertEquals(3L, maybeEntry.get.lastOffset)
+
+ idMapping.evictUnretainedProducers(3)
+ assertEquals(Set(anotherPid), idMapping.activeProducers.keySet)
+ assertEquals(Set(4), currentSnapshotOffsets)
+ assertEquals(4, idMapping.mapEndOffset)
+
+ idMapping.evictUnretainedProducers(5)
+ assertEquals(Set(), idMapping.activeProducers.keySet)
+ assertEquals(Set(), currentSnapshotOffsets)
+ assertEquals(5, idMapping.mapEndOffset)
+ }
+
+ @Test
+ def testSkipSnapshotIfOffsetUnchanged(): Unit = {
+ val epoch = 0.toShort
+ append(idMapping, pid, 0, epoch, 0L, 0L)
+
+ idMapping.takeSnapshot()
+ assertEquals(1, idMappingDir.listFiles().length)
+ assertEquals(Set(1), currentSnapshotOffsets)
+
+ // nothing changed so there should be no new snapshot
+ idMapping.takeSnapshot()
+ assertEquals(1, idMappingDir.listFiles().length)
+ assertEquals(Set(1), currentSnapshotOffsets)
+ }
+
+ @Test
+ def testStartOffset(): Unit = {
+ val epoch = 0.toShort
+ val pid2 = 2L
+ append(idMapping, pid2, 0, epoch, 0L, 1L)
+ append(idMapping, pid, 0, epoch, 1L, 2L)
+ append(idMapping, pid, 1, epoch, 2L, 3L)
+ append(idMapping, pid, 2, epoch, 3L, 4L)
+ idMapping.takeSnapshot()
+
+ intercept[OutOfOrderSequenceException] {
+ val recoveredMapping = new ProducerStateManager(partition, idMappingDir, maxPidExpirationMs)
+ recoveredMapping.truncateAndReload(0L, 1L, time.milliseconds)
+ append(recoveredMapping, pid2, 1, epoch, 4L, 5L)
+ }
+ }
+
+ @Test(expected = classOf[OutOfOrderSequenceException])
+ def testPidExpirationTimeout() {
+ val epoch = 5.toShort
+ val sequence = 37
+ append(idMapping, pid, sequence, epoch, 1L)
+ time.sleep(maxPidExpirationMs + 1)
+ idMapping.removeExpiredProducers(time.milliseconds)
+ append(idMapping, pid, sequence + 1, epoch, 1L)
+ }
+
+ @Test
+ def testFirstUnstableOffset() {
+ val epoch = 5.toShort
+ val sequence = 0
+
+ assertEquals(None, idMapping.firstUndecidedOffset)
+
+ append(idMapping, pid, sequence, epoch, offset = 99, isTransactional = true)
+ assertEquals(Some(99L), idMapping.firstUndecidedOffset)
+ assertEquals(Some(99L), idMapping.firstUnstableOffset.map(_.messageOffset))
+
+ val anotherPid = 2L
+ append(idMapping, anotherPid, sequence, epoch, offset = 105, isTransactional = true)
+ assertEquals(Some(99L), idMapping.firstUndecidedOffset)
+ assertEquals(Some(99L), idMapping.firstUnstableOffset.map(_.messageOffset))
+
+ appendEndTxnMarker(idMapping, pid, epoch, ControlRecordType.COMMIT, offset = 109)
+ assertEquals(Some(105L), idMapping.firstUndecidedOffset)
+ assertEquals(Some(99L), idMapping.firstUnstableOffset.map(_.messageOffset))
+
+ idMapping.onHighWatermarkUpdated(100L)
+ assertEquals(Some(99L), idMapping.firstUnstableOffset.map(_.messageOffset))
+
+ idMapping.onHighWatermarkUpdated(110L)
+ assertEquals(Some(105L), idMapping.firstUnstableOffset.map(_.messageOffset))
+
+ appendEndTxnMarker(idMapping, anotherPid, epoch, ControlRecordType.ABORT, offset = 112)
+ assertEquals(None, idMapping.firstUndecidedOffset)
+ assertEquals(Some(105L), idMapping.firstUnstableOffset.map(_.messageOffset))
+
+ idMapping.onHighWatermarkUpdated(113L)
+ assertEquals(None, idMapping.firstUnstableOffset.map(_.messageOffset))
+ }
+
+ @Test
+ def testProducersWithOngoingTransactionsDontExpire() {
+ val epoch = 5.toShort
+ val sequence = 0
+
+ append(idMapping, pid, sequence, epoch, offset = 99, isTransactional = true)
+ assertEquals(Some(99L), idMapping.firstUndecidedOffset)
+
+ time.sleep(maxPidExpirationMs + 1)
+ idMapping.removeExpiredProducers(time.milliseconds)
+
+ assertTrue(idMapping.lastEntry(pid).isDefined)
+ assertEquals(Some(99L), idMapping.firstUndecidedOffset)
+
+ idMapping.removeExpiredProducers(time.milliseconds)
+ assertTrue(idMapping.lastEntry(pid).isDefined)
+ }
+
+ @Test(expected = classOf[ProducerFencedException])
+ def testOldEpochForControlRecord(): Unit = {
+ val epoch = 5.toShort
+ val sequence = 0
+
+ assertEquals(None, idMapping.firstUndecidedOffset)
+
+ append(idMapping, pid, sequence, epoch, offset = 99, isTransactional = true)
+ appendEndTxnMarker(idMapping, pid, 3.toShort, ControlRecordType.COMMIT, offset=100)
+ }
+
+ @Test
+ def testCoordinatorFencing(): Unit = {
+ val epoch = 5.toShort
+ val sequence = 0
+
+ append(idMapping, pid, sequence, epoch, offset = 99, isTransactional = true)
+ appendEndTxnMarker(idMapping, pid, epoch, ControlRecordType.COMMIT, offset = 100, coordinatorEpoch = 1)
+
+ val lastEntry = idMapping.lastEntry(pid)
+ assertEquals(Some(1), lastEntry.map(_.coordinatorEpoch))
+
+ // writing with the current epoch is allowed
+ appendEndTxnMarker(idMapping, pid, epoch, ControlRecordType.COMMIT, offset = 101, coordinatorEpoch = 1)
+
+ // bumping the epoch is allowed
+ appendEndTxnMarker(idMapping, pid, epoch, ControlRecordType.COMMIT, offset = 102, coordinatorEpoch = 2)
+
+ // old epochs are not allowed
+ try {
+ appendEndTxnMarker(idMapping, pid, epoch, ControlRecordType.COMMIT, offset = 103, coordinatorEpoch = 1)
+ fail("Expected coordinator to be fenced")
+ } catch {
+ case e: TransactionCoordinatorFencedException =>
+ }
+ }
+
+ @Test(expected = classOf[TransactionCoordinatorFencedException])
+ def testCoordinatorFencedAfterReload(): Unit = {
+ val epoch = 0.toShort
+ append(idMapping, pid, 0, epoch, offset = 99, isTransactional = true)
+ appendEndTxnMarker(idMapping, pid, epoch, ControlRecordType.COMMIT, offset = 100, coordinatorEpoch = 1)
+ idMapping.takeSnapshot()
+
+ val recoveredMapping = new ProducerStateManager(partition, idMappingDir, maxPidExpirationMs)
+ recoveredMapping.truncateAndReload(0L, 2L, 70000)
+
+ // append from old coordinator should be rejected
+ appendEndTxnMarker(idMapping, pid, epoch, ControlRecordType.COMMIT, offset = 100, coordinatorEpoch = 0)
+ }
+
+ private def appendEndTxnMarker(mapping: ProducerStateManager,
+ pid: Long,
+ epoch: Short,
+ controlType: ControlRecordType,
+ offset: Long,
+ coordinatorEpoch: Int = 0,
+ timestamp: Long = time.milliseconds()): (CompletedTxn, Long) = {
+ val producerAppendInfo = new ProducerAppendInfo(pid, mapping.lastEntry(pid).getOrElse(ProducerIdEntry.Empty))
+ val endTxnMarker = new EndTransactionMarker(controlType, coordinatorEpoch)
+ val completedTxn = producerAppendInfo.appendEndTxnMarker(endTxnMarker, epoch, offset, timestamp)
+ mapping.update(producerAppendInfo)
+ val lastStableOffset = mapping.completeTxn(completedTxn)
+ mapping.updateMapEndOffset(offset + 1)
+ (completedTxn, lastStableOffset)
+ }
+
+ private def append(mapping: ProducerStateManager,
+ pid: Long,
+ seq: Int,
+ epoch: Short,
+ offset: Long,
+ timestamp: Long = time.milliseconds(),
+ isTransactional: Boolean = false,
+ isLoadingFromLog: Boolean = false): Unit = {
+ val producerAppendInfo = new ProducerAppendInfo(pid, mapping.lastEntry(pid), isLoadingFromLog)
+ producerAppendInfo.append(epoch, seq, seq, timestamp, offset, isTransactional)
+ mapping.update(producerAppendInfo)
+ mapping.updateMapEndOffset(offset + 1)
+ }
+
+ private def currentSnapshotOffsets =
+ idMappingDir.listFiles().map(file => Log.offsetFromFilename(file.getName)).toSet
+
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/log/TransactionIndexTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/TransactionIndexTest.scala b/core/src/test/scala/unit/kafka/log/TransactionIndexTest.scala
new file mode 100644
index 0000000..4546818
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/log/TransactionIndexTest.scala
@@ -0,0 +1,173 @@
+/*
+ * 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.
+ */
+package kafka.log
+
+import java.io.File
+
+import kafka.utils.TestUtils
+import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction
+import org.junit.Assert._
+import org.junit.{After, Before, Test}
+import org.scalatest.junit.JUnitSuite
+
+class TransactionIndexTest extends JUnitSuite {
+ var file: File = _
+ var index: TransactionIndex = _
+ val offset = 0L
+
+ @Before
+ def setup: Unit = {
+ file = TestUtils.tempFile()
+ index = new TransactionIndex(offset, file)
+ }
+
+ @After
+ def teardown: Unit = {
+ index.close()
+ }
+
+ @Test
+ def testPositionSetCorrectlyWhenOpened(): Unit = {
+ val abortedTxns = List(
+ new AbortedTxn(producerId = 0L, firstOffset = 0, lastOffset = 10, lastStableOffset = 11),
+ new AbortedTxn(producerId = 1L, firstOffset = 5, lastOffset = 15, lastStableOffset = 13),
+ new AbortedTxn(producerId = 2L, firstOffset = 18, lastOffset = 35, lastStableOffset = 25),
+ new AbortedTxn(producerId = 3L, firstOffset = 32, lastOffset = 50, lastStableOffset = 40))
+ abortedTxns.foreach(index.append)
+ index.close()
+
+ val reopenedIndex = new TransactionIndex(0L, file)
+ val anotherAbortedTxn = new AbortedTxn(producerId = 3L, firstOffset = 50, lastOffset = 60, lastStableOffset = 55)
+ reopenedIndex.append(anotherAbortedTxn)
+ assertEquals(abortedTxns ++ List(anotherAbortedTxn), reopenedIndex.allAbortedTxns)
+ }
+
+ @Test(expected = classOf[IllegalArgumentException])
+ def testSanityCheck(): Unit = {
+ val abortedTxns = List(
+ new AbortedTxn(producerId = 0L, firstOffset = 0, lastOffset = 10, lastStableOffset = 11),
+ new AbortedTxn(producerId = 1L, firstOffset = 5, lastOffset = 15, lastStableOffset = 13),
+ new AbortedTxn(producerId = 2L, firstOffset = 18, lastOffset = 35, lastStableOffset = 25),
+ new AbortedTxn(producerId = 3L, firstOffset = 32, lastOffset = 50, lastStableOffset = 40))
+ abortedTxns.foreach(index.append)
+ index.close()
+
+ // open the index with a different starting offset to fake invalid data
+ val reopenedIndex = new TransactionIndex(100L, file)
+ reopenedIndex.sanityCheck()
+ }
+
+ @Test(expected = classOf[IllegalArgumentException])
+ def testLastOffsetMustIncrease(): Unit = {
+ index.append(new AbortedTxn(producerId = 1L, firstOffset = 5, lastOffset = 15, lastStableOffset = 13))
+ index.append(new AbortedTxn(producerId = 0L, firstOffset = 0, lastOffset = 15, lastStableOffset = 11))
+ }
+
+ @Test(expected = classOf[IllegalArgumentException])
+ def testLastOffsetCannotDecrease(): Unit = {
+ index.append(new AbortedTxn(producerId = 1L, firstOffset = 5, lastOffset = 15, lastStableOffset = 13))
+ index.append(new AbortedTxn(producerId = 0L, firstOffset = 0, lastOffset = 10, lastStableOffset = 11))
+ }
+
+ @Test
+ def testCollectAbortedTransactions(): Unit = {
+ val abortedTxns = List(
+ new AbortedTxn(producerId = 0L, firstOffset = 0, lastOffset = 10, lastStableOffset = 11),
+ new AbortedTxn(producerId = 1L, firstOffset = 5, lastOffset = 15, lastStableOffset = 13),
+ new AbortedTxn(producerId = 2L, firstOffset = 18, lastOffset = 35, lastStableOffset = 25),
+ new AbortedTxn(producerId = 3L, firstOffset = 32, lastOffset = 50, lastStableOffset = 40))
+
+ abortedTxns.foreach(index.append)
+
+ val abortedTransactions = abortedTxns.map(_.asAbortedTransaction)
+
+ var result = index.collectAbortedTxns(0L, 100L)
+ assertEquals(abortedTransactions, result.abortedTransactions)
+ assertFalse(result.isComplete)
+
+ result = index.collectAbortedTxns(0L, 32)
+ assertEquals(abortedTransactions.take(3), result.abortedTransactions)
+ assertTrue(result.isComplete)
+
+ result = index.collectAbortedTxns(0L, 35)
+ assertEquals(abortedTransactions, result.abortedTransactions)
+ assertTrue(result.isComplete)
+
+ result = index.collectAbortedTxns(10, 35)
+ assertEquals(abortedTransactions, result.abortedTransactions)
+ assertTrue(result.isComplete)
+
+ result = index.collectAbortedTxns(11, 35)
+ assertEquals(abortedTransactions.slice(1, 4), result.abortedTransactions)
+ assertTrue(result.isComplete)
+
+ result = index.collectAbortedTxns(20, 41)
+ assertEquals(abortedTransactions.slice(2, 4), result.abortedTransactions)
+ assertFalse(result.isComplete)
+ }
+
+ @Test
+ def testTruncate(): Unit = {
+ val abortedTxns = List(
+ new AbortedTxn(producerId = 0L, firstOffset = 0, lastOffset = 10, lastStableOffset = 2),
+ new AbortedTxn(producerId = 1L, firstOffset = 5, lastOffset = 15, lastStableOffset = 16),
+ new AbortedTxn(producerId = 2L, firstOffset = 18, lastOffset = 35, lastStableOffset = 25),
+ new AbortedTxn(producerId = 3L, firstOffset = 32, lastOffset = 50, lastStableOffset = 40))
+ val abortedTransactions = abortedTxns.map(_.asAbortedTransaction)
+
+ abortedTxns.foreach(index.append)
+
+ index.truncateTo(51)
+ assertEquals(abortedTransactions, index.collectAbortedTxns(0L, 100L).abortedTransactions)
+
+ index.truncateTo(50)
+ assertEquals(abortedTransactions.take(3), index.collectAbortedTxns(0L, 100L).abortedTransactions)
+
+ index.truncate()
+ assertEquals(List.empty[AbortedTransaction], index.collectAbortedTxns(0L, 100L).abortedTransactions)
+ }
+
+ @Test
+ def testAbortedTxnSerde(): Unit = {
+ val pid = 983493L
+ val firstOffset = 137L
+ val lastOffset = 299L
+ val lastStableOffset = 200L
+
+ val abortedTxn = new AbortedTxn(pid, firstOffset, lastOffset, lastStableOffset)
+ assertEquals(AbortedTxn.CurrentVersion, abortedTxn.version)
+ assertEquals(pid, abortedTxn.producerId)
+ assertEquals(firstOffset, abortedTxn.firstOffset)
+ assertEquals(lastOffset, abortedTxn.lastOffset)
+ assertEquals(lastStableOffset, abortedTxn.lastStableOffset)
+ }
+
+ @Test
+ def testRenameIndex(): Unit = {
+ val renamed = TestUtils.tempFile()
+ index.append(new AbortedTxn(producerId = 0L, firstOffset = 0, lastOffset = 10, lastStableOffset = 2))
+
+ index.renameTo(renamed)
+ index.append(new AbortedTxn(producerId = 1L, firstOffset = 5, lastOffset = 15, lastStableOffset = 16))
+
+ val abortedTxns = index.collectAbortedTxns(0L, 100L).abortedTransactions
+ assertEquals(2, abortedTxns.size)
+ assertEquals(0, abortedTxns(0).firstOffset)
+ assertEquals(5, abortedTxns(1).firstOffset)
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
index 5dfcb63..415027c 100755
--- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
+++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
@@ -18,9 +18,8 @@
package kafka.server
import java.io.File
-import java.util.concurrent.atomic.AtomicLong
+import java.util.concurrent.atomic.AtomicInteger
import java.util.{Properties, Random}
-import java.lang.{Long => JLong}
import kafka.admin.AdminUtils
import kafka.api.{FetchRequestBuilder, OffsetRequest, PartitionOffsetRequestInfo}
@@ -32,13 +31,10 @@ import kafka.utils._
import kafka.zk.ZooKeeperTestHarness
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.protocol.Errors
-import org.apache.kafka.common.record.{MemoryRecords, Record}
-import org.apache.kafka.common.requests.DeleteRecordsRequest
import org.apache.kafka.common.utils.{Time, Utils}
import org.easymock.{EasyMock, IAnswer}
import org.junit.Assert._
import org.junit.{After, Before, Test}
-import scala.collection.JavaConverters._
class LogOffsetTest extends ZooKeeperTestHarness {
val random = new Random()
@@ -239,9 +235,9 @@ class LogOffsetTest extends ZooKeeperTestHarness {
def testFetchOffsetsBeforeWithChangingSegmentSize() {
val log = EasyMock.niceMock(classOf[Log])
val logSegment = EasyMock.niceMock(classOf[LogSegment])
- EasyMock.expect(logSegment.size).andStubAnswer(new IAnswer[Long] {
- private val value = new AtomicLong(0)
- def answer: Long = value.getAndIncrement()
+ EasyMock.expect(logSegment.size).andStubAnswer(new IAnswer[Int] {
+ private val value = new AtomicInteger(0)
+ def answer: Int = value.getAndIncrement()
})
EasyMock.replay(logSegment)
val logSegments = Seq(logSegment)
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
index d6b1649..9f7a47a 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
@@ -24,10 +24,11 @@ import kafka.log.Log
import kafka.utils._
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.metrics.Metrics
-import org.apache.kafka.common.record.{CompressionType, SimpleRecord, MemoryRecords}
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord}
import org.apache.kafka.common.requests.FetchRequest.PartitionData
import org.easymock.EasyMock
import EasyMock._
+import org.apache.kafka.common.requests.IsolationLevel
import org.junit.Assert._
import org.junit.{After, Test}
@@ -152,14 +153,14 @@ class ReplicaManagerQuotasTest {
expect(log.logEndOffsetMetadata).andReturn(new LogOffsetMetadata(20L)).anyTimes()
//if we ask for len 1 return a message
- expect(log.read(anyObject(), geq(1), anyObject(), anyObject())).andReturn(
+ expect(log.read(anyObject(), geq(1), anyObject(), anyObject(), anyObject())).andReturn(
FetchDataInfo(
new LogOffsetMetadata(0L, 0L, 0),
MemoryRecords.withRecords(CompressionType.NONE, record)
)).anyTimes()
//if we ask for len = 0, return 0 messages
- expect(log.read(anyObject(), EasyMock.eq(0), anyObject(), anyObject())).andReturn(
+ expect(log.read(anyObject(), EasyMock.eq(0), anyObject(), anyObject(), anyObject())).andReturn(
FetchDataInfo(
new LogOffsetMetadata(0L, 0L, 0),
MemoryRecords.EMPTY
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index e00c142..4886b94 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -21,7 +21,6 @@ import java.io.File
import java.util.Properties
import java.util.concurrent.atomic.AtomicBoolean
-import kafka.cluster.Broker
import kafka.log.LogConfig
import kafka.utils.{MockScheduler, MockTime, TestUtils, ZkUtils}
import TestUtils.createBroker
@@ -29,7 +28,7 @@ import org.I0Itec.zkclient.ZkClient
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record._
-import org.apache.kafka.common.requests.{LeaderAndIsrRequest, PartitionState}
+import org.apache.kafka.common.requests.{IsolationLevel, LeaderAndIsrRequest, PartitionState}
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.common.requests.FetchRequest.PartitionData
import org.apache.kafka.common.{Node, TopicPartition}
@@ -109,6 +108,7 @@ class ReplicaManagerTest {
timeout = 0,
requiredAcks = 3,
internalTopicsAllowed = false,
+ isFromClient = true,
entriesPerPartition = Map(new TopicPartition("test1", 0) -> MemoryRecords.withRecords(CompressionType.NONE,
new SimpleRecord("first message".getBytes))),
responseCallback = callback)
@@ -166,6 +166,7 @@ class ReplicaManagerTest {
timeout = 1000,
requiredAcks = -1,
internalTopicsAllowed = false,
+ isFromClient = true,
entriesPerPartition = Map(new TopicPartition(topic, 0) -> MemoryRecords.withRecords(CompressionType.NONE,
new SimpleRecord("first message".getBytes()))),
responseCallback = produceCallback)
@@ -178,7 +179,8 @@ class ReplicaManagerTest {
fetchMaxBytes = Int.MaxValue,
hardMaxBytesLimit = false,
fetchInfos = Seq(new TopicPartition(topic, 0) -> new PartitionData(0, 0, 100000)),
- responseCallback = fetchCallback)
+ responseCallback = fetchCallback,
+ isolationLevel = IsolationLevel.READ_UNCOMMITTED)
// Make this replica the follower
val leaderAndIsrRequest2 = new LeaderAndIsrRequest.Builder(0, 0,
@@ -192,7 +194,133 @@ class ReplicaManagerTest {
rm.shutdown(checkpointHW = false)
}
}
-
+
+ @Test
+ def testReadCommittedFetchLimitedAtLSO(): Unit = {
+ val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect)
+ props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath)
+ props.put("broker.id", Int.box(0))
+ val config = KafkaConfig.fromProps(props)
+ val logProps = new Properties()
+ logProps.put(LogConfig.MessageTimestampDifferenceMaxMsProp, Long.MaxValue.toString)
+ val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray, LogConfig(logProps))
+ val aliveBrokers = Seq(createBroker(0, "host0", 0), createBroker(1, "host1", 1))
+ val metadataCache = EasyMock.createMock(classOf[MetadataCache])
+ EasyMock.expect(metadataCache.getAliveBrokers).andReturn(aliveBrokers).anyTimes()
+ EasyMock.expect(metadataCache.isBrokerAlive(EasyMock.eq(0))).andReturn(true).anyTimes()
+ EasyMock.expect(metadataCache.isBrokerAlive(EasyMock.eq(1))).andReturn(true).anyTimes()
+ EasyMock.replay(metadataCache)
+ val rm = new ReplicaManager(config, metrics, time, zkUtils, new MockScheduler(time), mockLogMgr,
+ new AtomicBoolean(false), QuotaFactory.instantiate(config, metrics, time).follower, metadataCache, Option(this.getClass.getName))
+
+ try {
+ val brokerList: java.util.List[Integer] = Seq[Integer](0, 1).asJava
+ val brokerSet: java.util.Set[Integer] = Set[Integer](0, 1).asJava
+
+ val partition = rm.getOrCreatePartition(new TopicPartition(topic, 0))
+ partition.getOrCreateReplica(0)
+
+ // Make this replica the leader.
+ val leaderAndIsrRequest1 = new LeaderAndIsrRequest.Builder(0, 0,
+ collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerSet)).asJava,
+ Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava).build()
+ rm.becomeLeaderOrFollower(0, leaderAndIsrRequest1, (_, _) => {})
+ rm.getLeaderReplicaIfLocal(new TopicPartition(topic, 0))
+
+ def produceCallback(responseStatus: Map[TopicPartition, PartitionResponse]) =
+ responseStatus.values.foreach { status =>
+ assertEquals(Errors.NONE, status.error)
+ }
+
+ val producerId = 234L
+ val epoch = 5.toShort
+
+ // write a few batches as part of a transaction
+ val numRecords = 3
+ for (sequence <- 0 until numRecords) {
+ val records = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, epoch, sequence,
+ new SimpleRecord(s"message $sequence".getBytes))
+ rm.appendRecords(
+ timeout = 1000,
+ requiredAcks = -1,
+ internalTopicsAllowed = false,
+ isFromClient = true,
+ entriesPerPartition = Map(new TopicPartition(topic, 0) -> records),
+ responseCallback = produceCallback)
+ }
+
+ var fetchCallbackFired = false
+ var fetchError = Errors.NONE
+ var fetchedRecords: Records = null
+ def fetchCallback(responseStatus: Seq[(TopicPartition, FetchPartitionData)]) = {
+ fetchError = responseStatus.map(_._2).head.error
+ fetchedRecords = responseStatus.map(_._2).head.records
+ fetchCallbackFired = true
+ }
+
+ def fetchMessages(fetchInfos: Seq[(TopicPartition, PartitionData)],
+ isolationLevel: IsolationLevel = IsolationLevel.READ_UNCOMMITTED): Unit = {
+ rm.fetchMessages(
+ timeout = 1000,
+ replicaId = 1,
+ fetchMinBytes = 0,
+ fetchMaxBytes = Int.MaxValue,
+ hardMaxBytesLimit = false,
+ fetchInfos = fetchInfos,
+ responseCallback = fetchCallback,
+ isolationLevel = isolationLevel)
+ }
+
+ // fetch as follower to advance the high watermark
+ fetchMessages(fetchInfos = Seq(new TopicPartition(topic, 0) -> new PartitionData(numRecords, 0, 100000)),
+ isolationLevel = IsolationLevel.READ_UNCOMMITTED)
+
+ // fetch should return empty since LSO should be stuck at 0
+ fetchMessages(fetchInfos = Seq(new TopicPartition(topic, 0) -> new PartitionData(0, 0, 100000)),
+ isolationLevel = IsolationLevel.READ_COMMITTED)
+
+ assertTrue(fetchCallbackFired)
+ assertEquals(Errors.NONE, fetchError)
+ assertTrue(fetchedRecords.batches.asScala.isEmpty)
+ fetchCallbackFired = false
+
+ // now commit the transaction
+ val endTxnMarker = new EndTransactionMarker(ControlRecordType.COMMIT, 0)
+ val commitRecordBatch = MemoryRecords.withEndTransactionMarker(producerId, epoch, endTxnMarker)
+ rm.appendRecords(
+ timeout = 1000,
+ requiredAcks = -1,
+ internalTopicsAllowed = false,
+ isFromClient = false,
+ entriesPerPartition = Map(new TopicPartition(topic, 0) -> commitRecordBatch),
+ responseCallback = produceCallback)
+
+ // the LSO has advanced, but the appended commit marker has not been replicated, so
+ // none of the data from the transaction should be visible yet
+ fetchMessages(fetchInfos = Seq(new TopicPartition(topic, 0) -> new PartitionData(0, 0, 100000)),
+ isolationLevel = IsolationLevel.READ_COMMITTED)
+
+ assertTrue(fetchCallbackFired)
+ assertEquals(Errors.NONE, fetchError)
+ assertTrue(fetchedRecords.batches.asScala.isEmpty)
+ fetchCallbackFired = false
+
+ // fetch as follower to advance the high watermark
+ fetchMessages(fetchInfos = Seq(new TopicPartition(topic, 0) -> new PartitionData(numRecords + 1, 0, 100000)),
+ isolationLevel = IsolationLevel.READ_UNCOMMITTED)
+
+ // now all of the records should be fetchable
+ fetchMessages(fetchInfos = Seq(new TopicPartition(topic, 0) -> new PartitionData(0, 0, 100000)),
+ isolationLevel = IsolationLevel.READ_COMMITTED)
+
+ assertTrue(fetchCallbackFired)
+ assertEquals(Errors.NONE, fetchError)
+ assertEquals(numRecords + 1, fetchedRecords.batches.asScala.size)
+ } finally {
+ rm.shutdown(checkpointHW = false)
+ }
+ }
+
@Test
def testFetchBeyondHighWatermarkReturnEmptyResponse() {
val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect)
@@ -211,8 +339,8 @@ class ReplicaManagerTest {
EasyMock.replay(metadataCache)
val rm = new ReplicaManager(config, metrics, time, zkUtils, new MockScheduler(time), mockLogMgr,
new AtomicBoolean(false), QuotaFactory.instantiate(config, metrics, time).follower, metadataCache, Option(this.getClass.getName))
+
try {
-
val brokerList: java.util.List[Integer] = Seq[Integer](0, 1, 2).asJava
val brokerSet: java.util.Set[Integer] = Set[Integer](0, 1, 2).asJava
@@ -234,6 +362,7 @@ class ReplicaManagerTest {
timeout = 1000,
requiredAcks = -1,
internalTopicsAllowed = false,
+ isFromClient = true,
entriesPerPartition = Map(new TopicPartition(topic, 0) -> TestUtils.singletonRecords("message %d".format(i).getBytes)),
responseCallback = produceCallback)
@@ -254,7 +383,8 @@ class ReplicaManagerTest {
fetchMaxBytes = Int.MaxValue,
hardMaxBytesLimit = false,
fetchInfos = Seq(new TopicPartition(topic, 0) -> new PartitionData(1, 0, 100000)),
- responseCallback = fetchCallback)
+ responseCallback = fetchCallback,
+ isolationLevel = IsolationLevel.READ_UNCOMMITTED)
assertTrue(fetchCallbackFired)
@@ -270,11 +400,12 @@ class ReplicaManagerTest {
fetchMaxBytes = Int.MaxValue,
hardMaxBytesLimit = false,
fetchInfos = Seq(new TopicPartition(topic, 0) -> new PartitionData(1, 0, 100000)),
- responseCallback = fetchCallback)
+ responseCallback = fetchCallback,
+ isolationLevel = IsolationLevel.READ_UNCOMMITTED)
- assertTrue(fetchCallbackFired)
- assertEquals("Should not give an exception", Errors.NONE, fetchError)
- assertEquals("Should return empty response", MemoryRecords.EMPTY, fetchedRecords)
+ assertTrue(fetchCallbackFired)
+ assertEquals("Should not give an exception", Errors.NONE, fetchError)
+ assertEquals("Should return empty response", MemoryRecords.EMPTY, fetchedRecords)
} finally {
rm.shutdown(checkpointHW = false)
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
index 5e91c9b..9270544 100644
--- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
+++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
@@ -49,7 +49,7 @@ class RequestQuotaTest extends BaseRequestTest {
private var leaderNode: KafkaServer = null
// Run tests concurrently since a throttle could be up to 1 second because quota percentage allocated is very low
- case class Task(val apiKey: ApiKeys, val future: Future[_])
+ case class Task(apiKey: ApiKeys, future: Future[_])
private val executor = Executors.newCachedThreadPool
private val tasks = new ListBuffer[Task]
@@ -183,7 +183,8 @@ class RequestQuotaTest extends BaseRequestTest {
new requests.MetadataRequest.Builder(List(topic).asJava)
case ApiKeys.LIST_OFFSETS =>
- requests.ListOffsetRequest.Builder.forConsumer(false).setTargetTimes(Map(tp -> (0L: java.lang.Long)).asJava)
+ requests.ListOffsetRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED)
+ .setTargetTimes(Map(tp -> (0L: java.lang.Long)).asJava)
case ApiKeys.LEADER_AND_ISR =>
new LeaderAndIsrRequest.Builder(brokerId, Int.MaxValue,
@@ -285,7 +286,7 @@ class RequestQuotaTest extends BaseRequestTest {
apiKey.parseResponse(request.version, responseBuffer)
}
- case class Client(val clientId: String, val apiKey: ApiKeys) {
+ case class Client(clientId: String, apiKey: ApiKeys) {
var correlationId: Int = 0
val builder = requestBuilder(apiKey)
def runUntil(until: (Struct) => Boolean): Boolean = {
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
index ba17db6..d7822c1 100644
--- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
+++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
@@ -28,7 +28,8 @@ import java.util.Properties
import java.util.concurrent.atomic.AtomicBoolean
import org.apache.kafka.common.TopicPartition
-import org.apache.kafka.common.record.{CompressionType, SimpleRecord, MemoryRecords}
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord}
+import org.apache.kafka.common.requests.IsolationLevel
import org.easymock.EasyMock
import org.junit.Assert._
@@ -79,12 +80,12 @@ class SimpleFetchTest {
EasyMock.expect(log.logEndOffset).andReturn(leaderLEO).anyTimes()
EasyMock.expect(log.dir).andReturn(TestUtils.tempDir()).anyTimes()
EasyMock.expect(log.logEndOffsetMetadata).andReturn(new LogOffsetMetadata(leaderLEO)).anyTimes()
- EasyMock.expect(log.read(0, fetchSize, Some(partitionHW), true)).andReturn(
+ EasyMock.expect(log.read(0, fetchSize, Some(partitionHW), true, IsolationLevel.READ_UNCOMMITTED)).andReturn(
FetchDataInfo(
new LogOffsetMetadata(0L, 0L, 0),
MemoryRecords.withRecords(CompressionType.NONE, recordToHW)
)).anyTimes()
- EasyMock.expect(log.read(0, fetchSize, None, true)).andReturn(
+ EasyMock.expect(log.read(0, fetchSize, None, true, IsolationLevel.READ_UNCOMMITTED)).andReturn(
FetchDataInfo(
new LogOffsetMetadata(0L, 0L, 0),
MemoryRecords.withRecords(CompressionType.NONE, recordToLEO)
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/utils/TestUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
index bedc7bc..5d9e7c1 100755
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -735,7 +735,7 @@ object TestUtils extends Logging {
* @return The new leader or assertion failure if timeout is reached.
*/
def waitUntilLeaderIsElectedOrChanged(zkUtils: ZkUtils, topic: String, partition: Int,
- timeoutMs: Long = JTestUtils.DEFAULT_MAX_WAIT_MS,
+ timeoutMs: Long = 30000,
oldLeaderOpt: Option[Int] = None, newLeaderOpt: Option[Int] = None): Option[Int] = {
require(!(oldLeaderOpt.isDefined && newLeaderOpt.isDefined), "Can't define both the old and the new leader")
val startTime = System.currentTimeMillis()
[5/6] kafka git commit: KAFKA-5121;
Implement transaction index for KIP-98
Posted by jg...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
index e38e583..0a0f3d9 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
@@ -44,6 +44,7 @@ import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.ControlRecordType;
import org.apache.kafka.common.header.internals.RecordHeader;
+import org.apache.kafka.common.record.EndTransactionMarker;
import org.apache.kafka.common.record.LegacyRecord;
import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.MemoryRecordsBuilder;
@@ -181,16 +182,18 @@ public class FetcherTest {
assertEquals(1, fetcher.sendFetches());
assertFalse(fetcher.hasCompletedFetches());
+ long producerId = 1;
+ short epoch = 0;
+ int baseSequence = 0;
+
ByteBuffer buffer = ByteBuffer.allocate(1024);
- MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.CREATE_TIME, 0L);
- builder.append(0L, "key".getBytes(), null);
- builder.appendControlRecord(0L, ControlRecordType.COMMIT, null);
+ MemoryRecordsBuilder builder = MemoryRecords.idempotentBuilder(buffer, CompressionType.NONE, 0L, producerId,
+ epoch, baseSequence);
builder.append(0L, "key".getBytes(), null);
builder.close();
- builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.CREATE_TIME, 3L);
- builder.appendControlRecord(0L, ControlRecordType.ABORT, null);
- builder.close();
+ MemoryRecords.writeEndTransactionalMarker(buffer, 1L, producerId, epoch, new EndTransactionMarker(ControlRecordType.ABORT, 0)
+ );
buffer.flip();
@@ -202,10 +205,11 @@ public class FetcherTest {
assertTrue(partitionRecords.containsKey(tp1));
List<ConsumerRecord<byte[], byte[]>> records = partitionRecords.get(tp1);
- assertEquals(2, records.size());
- assertEquals(4L, subscriptions.position(tp1).longValue());
- for (ConsumerRecord<byte[], byte[]> record : records)
- assertArrayEquals("key".getBytes(), record.key());
+ assertEquals(1, records.size());
+ assertEquals(2L, subscriptions.position(tp1).longValue());
+
+ ConsumerRecord<byte[], byte[]> record = records.get(0);
+ assertArrayEquals("key".getBytes(), record.key());
}
@Test
@@ -814,6 +818,29 @@ public class FetcherTest {
}
@Test
+ public void testListOffsetsSendsIsolationLevel() {
+ for (final IsolationLevel isolationLevel : IsolationLevel.values()) {
+ Fetcher<byte[], byte[]> fetcher = createFetcher(subscriptions, new Metrics(), new ByteArrayDeserializer(),
+ new ByteArrayDeserializer(), Integer.MAX_VALUE, isolationLevel);
+
+ subscriptions.assignFromUser(singleton(tp1));
+ subscriptions.needOffsetReset(tp1, OffsetResetStrategy.LATEST);
+
+ client.prepareResponse(new MockClient.RequestMatcher() {
+ @Override
+ public boolean matches(AbstractRequest body) {
+ ListOffsetRequest request = (ListOffsetRequest) body;
+ return request.isolationLevel() == isolationLevel;
+ }
+ }, listOffsetResponse(Errors.NONE, 1L, 5L));
+ fetcher.updateFetchPositions(singleton(tp1));
+ assertFalse(subscriptions.isOffsetResetNeeded(tp1));
+ assertTrue(subscriptions.isFetchable(tp1));
+ assertEquals(5, subscriptions.position(tp1).longValue());
+ }
+ }
+
+ @Test
public void testUpdateFetchPositionResetToEarliestOffset() {
subscriptions.assignFromUser(singleton(tp1));
subscriptions.needOffsetReset(tp1, OffsetResetStrategy.EARLIEST);
@@ -1206,7 +1233,7 @@ public class FetcherTest {
new SimpleRecord(time.milliseconds(), "key".getBytes(), "value".getBytes()),
new SimpleRecord(time.milliseconds(), "key".getBytes(), "value".getBytes()));
- currentOffset += abortTransaction(buffer, 1L, currentOffset, time.milliseconds());
+ currentOffset += abortTransaction(buffer, 1L, currentOffset);
buffer.flip();
@@ -1240,7 +1267,7 @@ public class FetcherTest {
new SimpleRecord(time.milliseconds(), "key".getBytes(), "value".getBytes()),
new SimpleRecord(time.milliseconds(), "key".getBytes(), "value".getBytes()));
- currentOffset += commitTransaction(buffer, 1L, currentOffset, time.milliseconds());
+ currentOffset += commitTransaction(buffer, 1L, currentOffset);
buffer.flip();
List<FetchResponse.AbortedTransaction> abortedTransactions = new ArrayList<>();
@@ -1278,7 +1305,7 @@ public class FetcherTest {
List<FetchResponse.AbortedTransaction> abortedTransactions = new ArrayList<>();
int currOffset = 0;
- // Appends for producer 1 (evetually committed)
+ // Appends for producer 1 (eventually committed)
currOffset += appendTransactionalRecords(buffer, 1L, currOffset,
new SimpleRecord(time.milliseconds(), "commit1-1".getBytes(), "value".getBytes()),
new SimpleRecord(time.milliseconds(), "commit1-2".getBytes(), "value".getBytes()));
@@ -1288,13 +1315,13 @@ public class FetcherTest {
new SimpleRecord(time.milliseconds(), "abort2-1".getBytes(), "value".getBytes()));
// commit producer 1
- currOffset += commitTransaction(buffer, 1L, currOffset, time.milliseconds());
+ currOffset += commitTransaction(buffer, 1L, currOffset);
// append more for producer 2 (eventually aborted)
currOffset += appendTransactionalRecords(buffer, 2L, currOffset,
new SimpleRecord(time.milliseconds(), "abort2-2".getBytes(), "value".getBytes()));
// abort producer 2
- currOffset += abortTransaction(buffer, 2L, currOffset, time.milliseconds());
+ currOffset += abortTransaction(buffer, 2L, currOffset);
abortedTransactions.add(new FetchResponse.AbortedTransaction(2, 2));
// New transaction for producer 1 (eventually aborted)
@@ -1310,11 +1337,11 @@ public class FetcherTest {
new SimpleRecord(time.milliseconds(), "abort1-2".getBytes(), "value".getBytes()));
// abort producer 1
- currOffset += abortTransaction(buffer, 1L, currOffset, time.milliseconds());
+ currOffset += abortTransaction(buffer, 1L, currOffset);
abortedTransactions.add(new FetchResponse.AbortedTransaction(1, 6));
// commit producer 2
- currOffset += commitTransaction(buffer, 2L, currOffset, time.milliseconds());
+ currOffset += commitTransaction(buffer, 2L, currOffset);
buffer.flip();
@@ -1335,12 +1362,11 @@ public class FetcherTest {
assertTrue(fetchedRecords.containsKey(tp1));
// There are only 3 committed records
List<ConsumerRecord<byte[], byte[]>> fetchedConsumerRecords = fetchedRecords.get(tp1);
- Set<String> committedKeys = new HashSet<>(Arrays.asList("commit1-1", "commit1-2", "commit2-1"));
- Set<String> actuallyCommittedKeys = new HashSet<>();
+ Set<String> fetchedKeys = new HashSet<>();
for (ConsumerRecord<byte[], byte[]> consumerRecord : fetchedConsumerRecords) {
- actuallyCommittedKeys.add(new String(consumerRecord.key(), StandardCharsets.UTF_8));
+ fetchedKeys.add(new String(consumerRecord.key(), StandardCharsets.UTF_8));
}
- assertTrue(actuallyCommittedKeys.equals(committedKeys));
+ assertEquals(Utils.mkSet("commit1-1", "commit1-2", "commit2-1"), fetchedKeys);
}
@Test
@@ -1354,14 +1380,14 @@ public class FetcherTest {
new SimpleRecord(time.milliseconds(), "abort1-1".getBytes(), "value".getBytes()),
new SimpleRecord(time.milliseconds(), "abort1-2".getBytes(), "value".getBytes()));
- currentOffset += abortTransaction(buffer, 1L, currentOffset, time.milliseconds());
+ currentOffset += abortTransaction(buffer, 1L, currentOffset);
// Duplicate abort -- should be ignored.
- currentOffset += abortTransaction(buffer, 1L, currentOffset, time.milliseconds());
+ currentOffset += abortTransaction(buffer, 1L, currentOffset);
// Now commit a transaction.
currentOffset += appendTransactionalRecords(buffer, 1L, currentOffset,
new SimpleRecord(time.milliseconds(), "commit1-1".getBytes(), "value".getBytes()),
new SimpleRecord(time.milliseconds(), "commit1-2".getBytes(), "value".getBytes()));
- currentOffset += commitTransaction(buffer, 1L, currentOffset, time.milliseconds());
+ currentOffset += commitTransaction(buffer, 1L, currentOffset);
buffer.flip();
List<FetchResponse.AbortedTransaction> abortedTransactions = new ArrayList<>();
@@ -1402,7 +1428,7 @@ public class FetcherTest {
new SimpleRecord(time.milliseconds(), "key".getBytes(), "value".getBytes()),
new SimpleRecord(time.milliseconds(), "key".getBytes(), "value".getBytes()));
- currentOffset += abortTransaction(buffer, 1L, currentOffset, time.milliseconds());
+ currentOffset += abortTransaction(buffer, 1L, currentOffset);
buffer.flip();
@@ -1436,7 +1462,7 @@ public class FetcherTest {
new SimpleRecord(time.milliseconds(), "abort1-1".getBytes(), "value".getBytes()),
new SimpleRecord(time.milliseconds(), "abort1-2".getBytes(), "value".getBytes()));
- currentOffset += abortTransaction(buffer, 1L, currentOffset, time.milliseconds());
+ currentOffset += abortTransaction(buffer, 1L, currentOffset);
buffer.flip();
List<FetchResponse.AbortedTransaction> abortedTransactions = new ArrayList<>();
@@ -1463,7 +1489,8 @@ public class FetcherTest {
private int appendTransactionalRecords(ByteBuffer buffer, long pid, long baseOffset, SimpleRecord... records) {
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE,
- TimestampType.LOG_APPEND_TIME, baseOffset, time.milliseconds(), pid, (short) 0, (int) baseOffset, true, RecordBatch.NO_PARTITION_LEADER_EPOCH);
+ TimestampType.LOG_APPEND_TIME, baseOffset, time.milliseconds(), pid, (short) 0, (int) baseOffset, true,
+ RecordBatch.NO_PARTITION_LEADER_EPOCH);
for (SimpleRecord record : records) {
builder.append(record);
@@ -1472,19 +1499,15 @@ public class FetcherTest {
return records.length;
}
- private int commitTransaction(ByteBuffer buffer, long pid, int baseOffset, long timestamp) {
- MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE,
- TimestampType.LOG_APPEND_TIME, baseOffset, time.milliseconds(), pid, (short) 0, baseOffset, true, RecordBatch.NO_PARTITION_LEADER_EPOCH);
- builder.appendControlRecord(timestamp, ControlRecordType.COMMIT, null);
- builder.build();
+ private int commitTransaction(ByteBuffer buffer, long producerId, int baseOffset) {
+ MemoryRecords.writeEndTransactionalMarker(buffer, baseOffset, producerId, (short) 0,
+ new EndTransactionMarker(ControlRecordType.COMMIT, 0));
return 1;
}
- private int abortTransaction(ByteBuffer buffer, long pid, long baseOffset, long timestamp) {
- MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE,
- TimestampType.LOG_APPEND_TIME, baseOffset, time.milliseconds(), pid, (short) 0, (int) baseOffset, true, RecordBatch.NO_PARTITION_LEADER_EPOCH);
- builder.appendControlRecord(timestamp, ControlRecordType.ABORT, null);
- builder.build();
+ private int abortTransaction(ByteBuffer buffer, long producerId, long baseOffset) {
+ MemoryRecords.writeEndTransactionalMarker(buffer, baseOffset, producerId, (short) 0,
+ new EndTransactionMarker(ControlRecordType.ABORT, 0));
return 1;
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordBatchTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordBatchTest.java b/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordBatchTest.java
index 57f4663..ec858aa 100644
--- a/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordBatchTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordBatchTest.java
@@ -224,25 +224,32 @@ public class DefaultRecordBatchTest {
}
@Test
- public void testReadAndWriteControlRecord() {
+ public void testReadAndWriteControlBatch() {
+ long producerId = 1L;
+ short producerEpoch = 0;
+ int coordinatorEpoch = 15;
+
ByteBuffer buffer = ByteBuffer.allocate(128);
- MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE,
- TimestampType.CREATE_TIME, 0L);
+ MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.CURRENT_MAGIC_VALUE,
+ CompressionType.NONE, TimestampType.CREATE_TIME, 0L, RecordBatch.NO_TIMESTAMP, producerId,
+ producerEpoch, RecordBatch.NO_SEQUENCE, true, true, RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ buffer.remaining());
- builder.appendControlRecord(System.currentTimeMillis(), ControlRecordType.COMMIT, null);
- builder.appendControlRecord(System.currentTimeMillis(), ControlRecordType.ABORT, null);
+ EndTransactionMarker marker = new EndTransactionMarker(ControlRecordType.COMMIT, coordinatorEpoch);
+ builder.appendEndTxnMarker(System.currentTimeMillis(), marker);
MemoryRecords records = builder.build();
+ List<MutableRecordBatch> batches = TestUtils.toList(records.batches());
+ assertEquals(1, batches.size());
+
+ MutableRecordBatch batch = batches.get(0);
+ assertTrue(batch.isControlBatch());
+
List<Record> logRecords = TestUtils.toList(records.records());
- assertEquals(2, logRecords.size());
+ assertEquals(1, logRecords.size());
Record commitRecord = logRecords.get(0);
- assertTrue(commitRecord.isControlRecord());
- assertEquals(ControlRecordType.COMMIT, ControlRecordType.parse(commitRecord.key()));
-
- Record abortRecord = logRecords.get(1);
- assertTrue(abortRecord.isControlRecord());
- assertEquals(ControlRecordType.ABORT, ControlRecordType.parse(abortRecord.key()));
+ assertEquals(marker, EndTransactionMarker.deserialize(commitRecord));
}
@Test
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordTest.java b/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordTest.java
index 251db15..61b7b00 100644
--- a/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordTest.java
@@ -21,7 +21,6 @@ import org.apache.kafka.common.header.internals.RecordHeader;
import org.junit.Test;
import java.nio.ByteBuffer;
-import java.util.Arrays;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
@@ -45,31 +44,27 @@ public class DefaultRecordTest {
new SimpleRecord(15L, "hi".getBytes(), "there".getBytes(), headers)
};
- for (boolean isControlRecord : Arrays.asList(true, false)) {
- for (SimpleRecord record : records) {
- int baseSequence = 723;
- long baseOffset = 37;
- int offsetDelta = 10;
- long baseTimestamp = System.currentTimeMillis();
- long timestampDelta = 323;
+ for (SimpleRecord record : records) {
+ int baseSequence = 723;
+ long baseOffset = 37;
+ int offsetDelta = 10;
+ long baseTimestamp = System.currentTimeMillis();
+ long timestampDelta = 323;
- ByteBuffer buffer = ByteBuffer.allocate(1024);
- DefaultRecord.writeTo(buffer, isControlRecord, offsetDelta, timestampDelta, record.key(),
- record.value(), record.headers());
- buffer.flip();
+ ByteBuffer buffer = ByteBuffer.allocate(1024);
+ DefaultRecord.writeTo(buffer, offsetDelta, timestampDelta, record.key(), record.value(), record.headers());
+ buffer.flip();
- DefaultRecord logRecord = DefaultRecord.readFrom(buffer, baseOffset, baseTimestamp, baseSequence, null);
- assertNotNull(logRecord);
- assertEquals(baseOffset + offsetDelta, logRecord.offset());
- assertEquals(baseSequence + offsetDelta, logRecord.sequence());
- assertEquals(baseTimestamp + timestampDelta, logRecord.timestamp());
- assertEquals(record.key(), logRecord.key());
- assertEquals(record.value(), logRecord.value());
- assertEquals(isControlRecord, logRecord.isControlRecord());
- assertArrayEquals(record.headers(), logRecord.headers());
- assertEquals(DefaultRecord.sizeInBytes(offsetDelta, timestampDelta, record.key(), record.value(),
- record.headers()), logRecord.sizeInBytes());
- }
+ DefaultRecord logRecord = DefaultRecord.readFrom(buffer, baseOffset, baseTimestamp, baseSequence, null);
+ assertNotNull(logRecord);
+ assertEquals(baseOffset + offsetDelta, logRecord.offset());
+ assertEquals(baseSequence + offsetDelta, logRecord.sequence());
+ assertEquals(baseTimestamp + timestampDelta, logRecord.timestamp());
+ assertEquals(record.key(), logRecord.key());
+ assertEquals(record.value(), logRecord.value());
+ assertArrayEquals(record.headers(), logRecord.headers());
+ assertEquals(DefaultRecord.sizeInBytes(offsetDelta, timestampDelta, record.key(), record.value(),
+ record.headers()), logRecord.sizeInBytes());
}
}
@@ -83,7 +78,7 @@ public class DefaultRecordTest {
long timestampDelta = 323;
ByteBuffer buffer = ByteBuffer.allocate(1024);
- DefaultRecord.writeTo(buffer, false, offsetDelta, timestampDelta, key, value, new Header[0]);
+ DefaultRecord.writeTo(buffer, offsetDelta, timestampDelta, key, value, new Header[0]);
buffer.flip();
DefaultRecord record = DefaultRecord.readFrom(buffer, baseOffset, baseTimestamp, RecordBatch.NO_SEQUENCE, null);
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/test/java/org/apache/kafka/common/record/EndTransactionMarkerTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/record/EndTransactionMarkerTest.java b/clients/src/test/java/org/apache/kafka/common/record/EndTransactionMarkerTest.java
new file mode 100644
index 0000000..903f674
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/record/EndTransactionMarkerTest.java
@@ -0,0 +1,70 @@
+/*
+ * 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.
+ */
+package org.apache.kafka.common.record;
+
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertEquals;
+
+public class EndTransactionMarkerTest {
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testUnknownControlTypeNotAllowed() {
+ new EndTransactionMarker(ControlRecordType.UNKNOWN, 24);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testCannotDeserializeUnknownControlType() {
+ EndTransactionMarker.deserializeValue(ControlRecordType.UNKNOWN, ByteBuffer.wrap(new byte[0]));
+ }
+
+ @Test(expected = InvalidRecordException.class)
+ public void testIllegalNegativeVersion() {
+ ByteBuffer buffer = ByteBuffer.allocate(2);
+ buffer.putShort((short) -1);
+ buffer.flip();
+ EndTransactionMarker.deserializeValue(ControlRecordType.ABORT, buffer);
+ }
+
+ @Test(expected = InvalidRecordException.class)
+ public void testNotEnoughBytes() {
+ EndTransactionMarker.deserializeValue(ControlRecordType.COMMIT, ByteBuffer.wrap(new byte[0]));
+ }
+
+ @Test
+ public void testSerde() {
+ int coordinatorEpoch = 79;
+ EndTransactionMarker marker = new EndTransactionMarker(ControlRecordType.COMMIT, coordinatorEpoch);
+ ByteBuffer buffer = marker.serializeValue();
+ EndTransactionMarker deserialized = EndTransactionMarker.deserializeValue(ControlRecordType.COMMIT, buffer);
+ assertEquals(coordinatorEpoch, deserialized.coordinatorEpoch());
+ }
+
+ @Test
+ public void testDeserializeNewerVersion() {
+ int coordinatorEpoch = 79;
+ ByteBuffer buffer = ByteBuffer.allocate(8);
+ buffer.putShort((short) 5);
+ buffer.putInt(coordinatorEpoch);
+ buffer.putShort((short) 0); // unexpected data
+ buffer.flip();
+ EndTransactionMarker deserialized = EndTransactionMarker.deserializeValue(ControlRecordType.COMMIT, buffer);
+ assertEquals(coordinatorEpoch, deserialized.coordinatorEpoch());
+ }
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java
index 294f2f8..11ee419 100644
--- a/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java
@@ -140,25 +140,25 @@ public class FileRecordsTest {
int message1Size = batches.get(0).sizeInBytes();
assertEquals("Should be able to find the first message by its offset",
- new FileRecords.LogEntryPosition(0L, position, message1Size),
+ new FileRecords.LogOffsetPosition(0L, position, message1Size),
fileRecords.searchForOffsetWithSize(0, 0));
position += message1Size;
int message2Size = batches.get(1).sizeInBytes();
assertEquals("Should be able to find second message when starting from 0",
- new FileRecords.LogEntryPosition(1L, position, message2Size),
+ new FileRecords.LogOffsetPosition(1L, position, message2Size),
fileRecords.searchForOffsetWithSize(1, 0));
assertEquals("Should be able to find second message starting from its offset",
- new FileRecords.LogEntryPosition(1L, position, message2Size),
+ new FileRecords.LogOffsetPosition(1L, position, message2Size),
fileRecords.searchForOffsetWithSize(1, position));
position += message2Size + batches.get(2).sizeInBytes();
int message4Size = batches.get(3).sizeInBytes();
assertEquals("Should be able to find fourth message from a non-existant offset",
- new FileRecords.LogEntryPosition(50L, position, message4Size),
+ new FileRecords.LogOffsetPosition(50L, position, message4Size),
fileRecords.searchForOffsetWithSize(3, position));
assertEquals("Should be able to find fourth message by correct offset",
- new FileRecords.LogEntryPosition(50L, position, message4Size),
+ new FileRecords.LogOffsetPosition(50L, position, message4Size),
fileRecords.searchForOffsetWithSize(50, position));
}
@@ -241,7 +241,6 @@ public class FileRecordsTest {
EasyMock.expect(channelMock.size()).andReturn(42L).atLeastOnce();
EasyMock.expect(channelMock.position(42L)).andReturn(null).once();
EasyMock.expect(channelMock.truncate(23L)).andReturn(null).once();
- EasyMock.expect(channelMock.position(23L)).andReturn(null).once();
EasyMock.replay(channelMock);
FileRecords fileRecords = new FileRecords(tempFile(), channelMock, 0, Integer.MAX_VALUE, false);
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java
index 330879f..0467522 100644
--- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java
@@ -50,7 +50,7 @@ public class MemoryRecordsBuilderTest {
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.MAGIC_VALUE_V0, compressionType,
TimestampType.CREATE_TIME, 0L, 0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
- false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
+ false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
MemoryRecords records = builder.build();
assertEquals(0, records.sizeInBytes());
assertEquals(bufferOffset, buffer.position());
@@ -66,8 +66,8 @@ public class MemoryRecordsBuilderTest {
int sequence = 2342;
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType,
- TimestampType.CREATE_TIME, 0L, 0L, pid, epoch, sequence, true, RecordBatch.NO_PARTITION_LEADER_EPOCH,
- buffer.capacity());
+ TimestampType.CREATE_TIME, 0L, 0L, pid, epoch, sequence, true, false,
+ RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
builder.append(System.currentTimeMillis(), "foo".getBytes(), "bar".getBytes());
MemoryRecords records = builder.build();
@@ -86,7 +86,7 @@ public class MemoryRecordsBuilderTest {
int sequence = 2342;
new MemoryRecordsBuilder(buffer, RecordBatch.MAGIC_VALUE_V0, compressionType, TimestampType.CREATE_TIME,
- 0L, 0L, pid, epoch, sequence, true, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
+ 0L, 0L, pid, epoch, sequence, true, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
}
@Test(expected = IllegalArgumentException.class)
@@ -99,7 +99,33 @@ public class MemoryRecordsBuilderTest {
int sequence = 2342;
new MemoryRecordsBuilder(buffer, RecordBatch.MAGIC_VALUE_V1, compressionType, TimestampType.CREATE_TIME,
- 0L, 0L, pid, epoch, sequence, true, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
+ 0L, 0L, pid, epoch, sequence, true, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testWriteControlBatchNotAllowedMagicV0() {
+ ByteBuffer buffer = ByteBuffer.allocate(128);
+ buffer.position(bufferOffset);
+
+ long pid = 9809;
+ short epoch = 15;
+ int sequence = 2342;
+
+ new MemoryRecordsBuilder(buffer, RecordBatch.MAGIC_VALUE_V0, compressionType, TimestampType.CREATE_TIME,
+ 0L, 0L, pid, epoch, sequence, false, true, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testWriteControlBatchNotAllowedMagicV1() {
+ ByteBuffer buffer = ByteBuffer.allocate(128);
+ buffer.position(bufferOffset);
+
+ long pid = 9809;
+ short epoch = 15;
+ int sequence = 2342;
+
+ new MemoryRecordsBuilder(buffer, RecordBatch.MAGIC_VALUE_V1, compressionType, TimestampType.CREATE_TIME,
+ 0L, 0L, pid, epoch, sequence, false, true, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
}
@Test(expected = IllegalArgumentException.class)
@@ -112,7 +138,7 @@ public class MemoryRecordsBuilderTest {
int sequence = 2342;
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, TimestampType.CREATE_TIME,
- 0L, 0L, pid, epoch, sequence, true, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
+ 0L, 0L, pid, epoch, sequence, true, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
builder.close();
}
@@ -126,7 +152,7 @@ public class MemoryRecordsBuilderTest {
int sequence = 2342;
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, TimestampType.CREATE_TIME,
- 0L, 0L, pid, epoch, sequence, true, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
+ 0L, 0L, pid, epoch, sequence, true, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
builder.close();
}
@@ -140,10 +166,38 @@ public class MemoryRecordsBuilderTest {
int sequence = RecordBatch.NO_SEQUENCE;
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, TimestampType.CREATE_TIME,
- 0L, 0L, pid, epoch, sequence, true, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
+ 0L, 0L, pid, epoch, sequence, true, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
builder.close();
}
+ @Test(expected = IllegalArgumentException.class)
+ public void testWriteEndTxnMarkerNonTransactionalBatch() {
+ ByteBuffer buffer = ByteBuffer.allocate(128);
+ buffer.position(bufferOffset);
+
+ long pid = 9809;
+ short epoch = 15;
+ int sequence = RecordBatch.NO_SEQUENCE;
+
+ MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, TimestampType.CREATE_TIME,
+ 0L, 0L, pid, epoch, sequence, false, true, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
+ builder.appendEndTxnMarker(RecordBatch.NO_TIMESTAMP, new EndTransactionMarker(ControlRecordType.ABORT, 0));
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testWriteEndTxnMarkerNonControlBatch() {
+ ByteBuffer buffer = ByteBuffer.allocate(128);
+ buffer.position(bufferOffset);
+
+ long pid = 9809;
+ short epoch = 15;
+ int sequence = RecordBatch.NO_SEQUENCE;
+
+ MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, TimestampType.CREATE_TIME,
+ 0L, 0L, pid, epoch, sequence, true, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
+ builder.appendEndTxnMarker(RecordBatch.NO_TIMESTAMP, new EndTransactionMarker(ControlRecordType.ABORT, 0));
+ }
+
@Test
public void testCompressionRateV0() {
ByteBuffer buffer = ByteBuffer.allocate(1024);
@@ -157,7 +211,7 @@ public class MemoryRecordsBuilderTest {
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.MAGIC_VALUE_V0, compressionType,
TimestampType.CREATE_TIME, 0L, 0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
- false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
+ false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
int uncompressedSize = 0;
for (LegacyRecord record : records) {
@@ -188,7 +242,7 @@ public class MemoryRecordsBuilderTest {
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.MAGIC_VALUE_V1, compressionType,
TimestampType.CREATE_TIME, 0L, 0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
- false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
+ false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
int uncompressedSize = 0;
for (LegacyRecord record : records) {
@@ -214,7 +268,7 @@ public class MemoryRecordsBuilderTest {
long logAppendTime = System.currentTimeMillis();
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.MAGIC_VALUE_V1, compressionType,
TimestampType.LOG_APPEND_TIME, 0L, logAppendTime, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH,
- RecordBatch.NO_SEQUENCE, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
+ RecordBatch.NO_SEQUENCE, false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
builder.append(0L, "a".getBytes(), "1".getBytes());
builder.append(0L, "b".getBytes(), "2".getBytes());
builder.append(0L, "c".getBytes(), "3".getBytes());
@@ -243,7 +297,7 @@ public class MemoryRecordsBuilderTest {
long logAppendTime = System.currentTimeMillis();
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.MAGIC_VALUE_V1, compressionType,
TimestampType.CREATE_TIME, 0L, logAppendTime, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
- false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
+ false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
builder.append(0L, "a".getBytes(), "1".getBytes());
builder.append(2L, "b".getBytes(), "2".getBytes());
builder.append(1L, "c".getBytes(), "3".getBytes());
@@ -276,7 +330,7 @@ public class MemoryRecordsBuilderTest {
ByteBuffer buffer = ByteBuffer.allocate(512);
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType,
TimestampType.CREATE_TIME, 0L, LegacyRecord.NO_TIMESTAMP, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH,
- RecordBatch.NO_SEQUENCE, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, writeLimit);
+ RecordBatch.NO_SEQUENCE, false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, writeLimit);
assertFalse(builder.isFull());
assertTrue(builder.hasRoomFor(0L, key, value));
@@ -302,7 +356,7 @@ public class MemoryRecordsBuilderTest {
long logAppendTime = System.currentTimeMillis();
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.MAGIC_VALUE_V1, compressionType,
TimestampType.CREATE_TIME, 0L, logAppendTime, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
- false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
+ false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
builder.append(0L, "a".getBytes(), "1".getBytes());
builder.append(1L, "b".getBytes(), "2".getBytes());
@@ -330,7 +384,7 @@ public class MemoryRecordsBuilderTest {
long logAppendTime = System.currentTimeMillis();
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.MAGIC_VALUE_V1, compressionType,
TimestampType.CREATE_TIME, 0L, logAppendTime, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
- false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
+ false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
builder.appendWithOffset(0L, System.currentTimeMillis(), "a".getBytes(), null);
@@ -346,13 +400,18 @@ public class MemoryRecordsBuilderTest {
builder.append(10L, "1".getBytes(), "a".getBytes());
builder.close();
+ MemoryRecords.writeEndTransactionalMarker(buffer, 1L, 15L, (short) 0,
+ new EndTransactionMarker(ControlRecordType.ABORT, 0));
+
builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, compressionType,
TimestampType.CREATE_TIME, 1L);
- builder.append(11L, "2".getBytes(), "b".getBytes());
- builder.appendControlRecord(12L, ControlRecordType.COMMIT, null);
+ builder.append(12L, "2".getBytes(), "b".getBytes());
builder.append(13L, "3".getBytes(), "c".getBytes());
builder.close();
+ MemoryRecords.writeEndTransactionalMarker(buffer, 14L, 1L, (short) 0,
+ new EndTransactionMarker(ControlRecordType.COMMIT, 0));
+
buffer.flip();
Records records = MemoryRecords.readableRecords(buffer).downConvert(RecordBatch.MAGIC_VALUE_V1);
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java
index 49e1429..014a5bd 100644
--- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java
@@ -65,7 +65,7 @@ public class MemoryRecordsTest {
ByteBuffer buffer = ByteBuffer.allocate(1024);
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, magic, compression,
- TimestampType.CREATE_TIME, firstOffset, logAppendTime, pid, epoch, firstSequence, false,
+ TimestampType.CREATE_TIME, firstOffset, logAppendTime, pid, epoch, firstSequence, false, false,
partitionLeaderEpoch, buffer.limit());
SimpleRecord[] records = new SimpleRecord[] {
@@ -216,9 +216,44 @@ public class MemoryRecordsTest {
}
@Test
+ public void testBuildEndTxnMarker() {
+ if (magic >= RecordBatch.MAGIC_VALUE_V2) {
+ long producerId = 73;
+ short producerEpoch = 13;
+ long initialOffset = 983L;
+ int coordinatorEpoch = 347;
+ EndTransactionMarker marker = new EndTransactionMarker(ControlRecordType.COMMIT, coordinatorEpoch);
+ MemoryRecords records = MemoryRecords.withEndTransactionMarker(initialOffset, producerId, producerEpoch, marker);
+ // verify that buffer allocation was precise
+ assertEquals(records.buffer().remaining(), records.buffer().capacity());
+
+ List<MutableRecordBatch> batches = TestUtils.toList(records.batches());
+ assertEquals(1, batches.size());
+
+ RecordBatch batch = batches.get(0);
+ assertTrue(batch.isControlBatch());
+ assertEquals(producerId, batch.producerId());
+ assertEquals(producerEpoch, batch.producerEpoch());
+ assertEquals(initialOffset, batch.baseOffset());
+ assertTrue(batch.isValid());
+
+ List<Record> createdRecords = TestUtils.toList(batch);
+ assertEquals(1, createdRecords.size());
+
+ Record record = createdRecords.get(0);
+ assertTrue(record.isValid());
+ EndTransactionMarker deserializedMarker = EndTransactionMarker.deserialize(record);
+ assertEquals(ControlRecordType.COMMIT, deserializedMarker.controlType());
+ assertEquals(coordinatorEpoch, deserializedMarker.coordinatorEpoch());
+ }
+ }
+
+ @Test
public void testFilterToPreservesProducerInfo() {
if (magic >= RecordBatch.MAGIC_VALUE_V2) {
ByteBuffer buffer = ByteBuffer.allocate(2048);
+
+ // non-idempotent, non-transactional
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, 0L);
builder.append(10L, null, "a".getBytes());
builder.append(11L, "1".getBytes(), "b".getBytes());
@@ -226,17 +261,28 @@ public class MemoryRecordsTest {
builder.close();
- long pid = 23L;
- short epoch = 5;
- int baseSequence = 10;
-
+ // idempotent
+ long pid1 = 23L;
+ short epoch1 = 5;
+ int baseSequence1 = 10;
builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, 3L,
- RecordBatch.NO_TIMESTAMP, pid, epoch, baseSequence);
+ RecordBatch.NO_TIMESTAMP, pid1, epoch1, baseSequence1);
builder.append(13L, null, "d".getBytes());
builder.append(14L, "4".getBytes(), "e".getBytes());
builder.append(15L, "5".getBytes(), "f".getBytes());
builder.close();
+ // transactional
+ long pid2 = 99384L;
+ short epoch2 = 234;
+ int baseSequence2 = 15;
+ builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, 3L,
+ RecordBatch.NO_TIMESTAMP, pid2, epoch2, baseSequence2, true, RecordBatch.NO_PARTITION_LEADER_EPOCH);
+ builder.append(16L, "6".getBytes(), "g".getBytes());
+ builder.append(17L, null, "h".getBytes());
+ builder.append(18L, "8".getBytes(), "i".getBytes());
+ builder.close();
+
buffer.flip();
ByteBuffer filtered = ByteBuffer.allocate(2048);
@@ -246,7 +292,7 @@ public class MemoryRecordsTest {
MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered);
List<MutableRecordBatch> batches = TestUtils.toList(filteredRecords.batches());
- assertEquals(2, batches.size());
+ assertEquals(3, batches.size());
MutableRecordBatch firstBatch = batches.get(0);
assertEquals(1, firstBatch.countOrNull().intValue());
@@ -256,15 +302,27 @@ public class MemoryRecordsTest {
assertEquals(RecordBatch.NO_PRODUCER_EPOCH, firstBatch.producerEpoch());
assertEquals(RecordBatch.NO_SEQUENCE, firstBatch.baseSequence());
assertEquals(RecordBatch.NO_SEQUENCE, firstBatch.lastSequence());
+ assertFalse(firstBatch.isTransactional());
MutableRecordBatch secondBatch = batches.get(1);
assertEquals(2, secondBatch.countOrNull().intValue());
assertEquals(3L, secondBatch.baseOffset());
assertEquals(5L, secondBatch.lastOffset());
- assertEquals(pid, secondBatch.producerId());
- assertEquals(epoch, secondBatch.producerEpoch());
- assertEquals(baseSequence, secondBatch.baseSequence());
- assertEquals(baseSequence + 2, secondBatch.lastSequence());
+ assertEquals(pid1, secondBatch.producerId());
+ assertEquals(epoch1, secondBatch.producerEpoch());
+ assertEquals(baseSequence1, secondBatch.baseSequence());
+ assertEquals(baseSequence1 + 2, secondBatch.lastSequence());
+ assertFalse(secondBatch.isTransactional());
+
+ MutableRecordBatch thirdBatch = batches.get(2);
+ assertEquals(2, thirdBatch.countOrNull().intValue());
+ assertEquals(3L, thirdBatch.baseOffset());
+ assertEquals(5L, thirdBatch.lastOffset());
+ assertEquals(pid2, thirdBatch.producerId());
+ assertEquals(epoch2, thirdBatch.producerEpoch());
+ assertEquals(baseSequence2, thirdBatch.baseSequence());
+ assertEquals(baseSequence2 + 2, thirdBatch.lastSequence());
+ assertTrue(thirdBatch.isTransactional());
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
index c948fd1..6443e4d 100644
--- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
@@ -94,6 +94,9 @@ public class RequestResponseTest {
checkRequest(createListOffsetRequest(1));
checkErrorResponse(createListOffsetRequest(1), new UnknownServerException());
checkResponse(createListOffsetResponse(1), 1);
+ checkRequest(createListOffsetRequest(2));
+ checkErrorResponse(createListOffsetRequest(2), new UnknownServerException());
+ checkResponse(createListOffsetResponse(2), 2);
checkRequest(MetadataRequest.Builder.allTopics().build((short) 2));
checkRequest(createMetadataRequest(1, asList("topic1")));
checkErrorResponse(createMetadataRequest(1, asList("topic1")), new UnknownServerException());
@@ -621,11 +624,24 @@ public class RequestResponseTest {
Map<TopicPartition, ListOffsetRequest.PartitionData> offsetData = Collections.singletonMap(
new TopicPartition("test", 0),
new ListOffsetRequest.PartitionData(1000000L, 10));
- return ListOffsetRequest.Builder.forConsumer(false).setOffsetData(offsetData).build((short) version);
+ return ListOffsetRequest.Builder
+ .forConsumer(false, IsolationLevel.READ_UNCOMMITTED)
+ .setOffsetData(offsetData)
+ .build((short) version);
} else if (version == 1) {
Map<TopicPartition, Long> offsetData = Collections.singletonMap(
new TopicPartition("test", 0), 1000000L);
- return ListOffsetRequest.Builder.forConsumer(true).setTargetTimes(offsetData).build((short) version);
+ return ListOffsetRequest.Builder
+ .forConsumer(true, IsolationLevel.READ_UNCOMMITTED)
+ .setTargetTimes(offsetData)
+ .build((short) version);
+ } else if (version == 2) {
+ Map<TopicPartition, Long> offsetData = Collections.singletonMap(
+ new TopicPartition("test", 0), 1000000L);
+ return ListOffsetRequest.Builder
+ .forConsumer(true, IsolationLevel.READ_COMMITTED)
+ .setTargetTimes(offsetData)
+ .build((short) version);
} else {
throw new IllegalArgumentException("Illegal ListOffsetRequest version " + version);
}
@@ -638,7 +654,7 @@ public class RequestResponseTest {
responseData.put(new TopicPartition("test", 0),
new ListOffsetResponse.PartitionData(Errors.NONE, asList(100L)));
return new ListOffsetResponse(responseData);
- } else if (version == 1) {
+ } else if (version == 1 || version == 2) {
Map<TopicPartition, ListOffsetResponse.PartitionData> responseData = new HashMap<>();
responseData.put(new TopicPartition("test", 0),
new ListOffsetResponse.PartitionData(Errors.NONE, 10000L, 100L));
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/cluster/Partition.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala
index 1eea8dc..1d13689 100755
--- a/core/src/main/scala/kafka/cluster/Partition.scala
+++ b/core/src/main/scala/kafka/cluster/Partition.scala
@@ -456,7 +456,7 @@ class Partition(val topic: String,
laggingReplicas
}
- def appendRecordsToLeader(records: MemoryRecords, requiredAcks: Int = 0) = {
+ def appendRecordsToLeader(records: MemoryRecords, isFromClient: Boolean, requiredAcks: Int = 0) = {
val (info, leaderHWIncremented) = inReadLock(leaderIsrUpdateLock) {
leaderReplicaIfLocal match {
case Some(leaderReplica) =>
@@ -470,7 +470,7 @@ class Partition(val topic: String,
.format(topicPartition, inSyncSize, minIsr))
}
- val info = log.appendAsLeader(records, leaderEpoch = this.leaderEpoch)
+ val info = log.appendAsLeader(records, leaderEpoch = this.leaderEpoch, isFromClient)
// probably unblock some follower fetch requests since log end offset has been updated
replicaManager.tryCompleteDelayedFetch(TopicPartitionOperationKey(this.topic, this.partitionId))
// we may need to increment high watermark since ISR could be down to 1
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/cluster/Replica.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/cluster/Replica.scala b/core/src/main/scala/kafka/cluster/Replica.scala
index a604b87..e3b1f2d 100644
--- a/core/src/main/scala/kafka/cluster/Replica.scala
+++ b/core/src/main/scala/kafka/cluster/Replica.scala
@@ -135,6 +135,7 @@ class Replica(val brokerId: Int,
def highWatermark_=(newHighWatermark: LogOffsetMetadata) {
if (isLocal) {
highWatermarkMetadata = newHighWatermark
+ log.foreach(_.onHighWatermarkIncremented(newHighWatermark.messageOffset))
trace(s"Setting high watermark for replica $brokerId partition $topicPartition to [$newHighWatermark]")
} else {
throw new KafkaException(s"Should not set high watermark on partition $topicPartition's non-local replica $brokerId")
@@ -143,6 +144,23 @@ class Replica(val brokerId: Int,
def highWatermark = highWatermarkMetadata
+ /**
+ * The last stable offset (LSO) is defined as the first offset such that all lower offsets have been "decided."
+ * Non-transactional messages are considered decided immediately, but transactional messages are only decided when
+ * the corresponding COMMIT or ABORT marker is written. This implies that the last stable offset will be equal
+ * to the high watermark if there are no transactional messages in the log. Note also that the LSO cannot advance
+ * beyond the high watermark.
+ */
+ def lastStableOffset: LogOffsetMetadata = {
+ log.map { log =>
+ log.firstUnstableOffset match {
+ case Some(offsetMetadata) if offsetMetadata.messageOffset < highWatermark.messageOffset => offsetMetadata
+ case _ => highWatermark
+ }
+ }.getOrElse(throw new KafkaException(s"Cannot fetch last stable offset on partition $topicPartition's " +
+ s"non-local replica $brokerId"))
+ }
+
def convertHWToLocalOffsetMetadata() = {
if (isLocal) {
highWatermarkMetadata = log.get.convertToOffsetMetadata(highWatermarkMetadata.messageOffset)
@@ -165,7 +183,10 @@ class Replica(val brokerId: Int,
replicaString.append("; Partition: " + partition.partitionId)
replicaString.append("; isLocal: " + isLocal)
replicaString.append("; lastCaughtUpTimeMs: " + lastCaughtUpTimeMs)
- if (isLocal) replicaString.append("; Highwatermark: " + highWatermark)
+ if (isLocal) {
+ replicaString.append("; Highwatermark: " + highWatermark)
+ replicaString.append("; LastStableOffset: " + lastStableOffset)
+ }
replicaString.toString
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala
index e711392..3eafdb7 100644
--- a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala
+++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala
@@ -38,7 +38,7 @@ import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.protocol.types.Type._
import org.apache.kafka.common.protocol.types.{ArrayOf, Field, Schema, Struct}
import org.apache.kafka.common.record._
-import org.apache.kafka.common.requests.OffsetFetchResponse
+import org.apache.kafka.common.requests.{IsolationLevel, OffsetFetchResponse}
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.common.utils.{Time, Utils}
@@ -225,7 +225,8 @@ class GroupMetadataManager(brokerId: Int,
replicaManager.appendRecords(
config.offsetCommitTimeoutMs.toLong,
config.offsetCommitRequiredAcks,
- true, // allow appending to internal offset topic
+ internalTopicsAllowed = true,
+ isFromClient = false,
delayedStore.partitionRecords,
delayedStore.callback)
}
@@ -429,7 +430,8 @@ class GroupMetadataManager(brokerId: Int,
case Some(log) =>
var currOffset = log.logStartOffset
- val buffer = ByteBuffer.allocate(config.loadBufferSize)
+ lazy val buffer = ByteBuffer.allocate(config.loadBufferSize)
+
// loop breaks if leader changes at any time during the load, since getHighWatermark is -1
val loadedOffsets = mutable.Map[GroupTopicPartition, OffsetAndMetadata]()
val removedOffsets = mutable.Set[GroupTopicPartition]()
@@ -437,12 +439,18 @@ class GroupMetadataManager(brokerId: Int,
val removedGroups = mutable.Set[String]()
while (currOffset < highWaterMark && !shuttingDown.get()) {
- buffer.clear()
- val fileRecords = log.read(currOffset, config.loadBufferSize, maxOffset = None, minOneMessage = true)
- .records.asInstanceOf[FileRecords]
- val bufferRead = fileRecords.readInto(buffer, 0)
+ val fetchDataInfo = log.read(currOffset, config.loadBufferSize, maxOffset = None, minOneMessage = true,
+ isolationLevel = IsolationLevel.READ_UNCOMMITTED)
+
+ val memRecords = fetchDataInfo.records match {
+ case records: MemoryRecords => records
+ case fileRecords: FileRecords =>
+ buffer.clear()
+ val bufferRead = fileRecords.readInto(buffer, 0)
+ MemoryRecords.readableRecords(bufferRead)
+ }
- MemoryRecords.readableRecords(bufferRead).batches.asScala.foreach { batch =>
+ memRecords.batches.asScala.foreach { batch =>
for (record <- batch.asScala) {
require(record.hasKey, "Group metadata/offset entry key should not be null")
GroupMetadataManager.readMessageKey(record.key) match {
@@ -630,7 +638,8 @@ class GroupMetadataManager(brokerId: Int,
// do not need to require acks since even if the tombstone is lost,
// it will be appended again in the next purge cycle
val records = MemoryRecords.withRecords(magicValue, 0L, compressionType, timestampType, tombstones: _*)
- partition.appendRecordsToLeader(records)
+ partition.appendRecordsToLeader(records, isFromClient = false, requiredAcks = 0)
+
offsetsRemoved += removedOffsets.size
trace(s"Successfully appended ${tombstones.size} tombstones to $appendPartition for expired/deleted " +
s"offsets and/or metadata for group $groupId")
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala
index f07ca91..7930cd0 100644
--- a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala
+++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala
@@ -31,6 +31,7 @@ import kafka.utils.{Logging, Pool, Scheduler, ZkUtils}
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record.{FileRecords, MemoryRecords, SimpleRecord}
+import org.apache.kafka.common.requests.IsolationLevel
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.common.utils.{Time, Utils}
@@ -158,8 +159,7 @@ class TransactionStateManager(brokerId: Int,
warn(s"Attempted to load offsets and group metadata from $topicPartition, but found no log")
case Some(log) =>
- val buffer = ByteBuffer.allocate(config.transactionLogLoadBufferSize)
-
+ lazy val buffer = ByteBuffer.allocate(config.transactionLogLoadBufferSize)
val loadedTransactions = mutable.Map.empty[String, TransactionMetadata]
val removedTransactionalIds = mutable.Set.empty[String]
@@ -169,11 +169,17 @@ class TransactionStateManager(brokerId: Int,
&& loadingPartitions.contains(topicPartition.partition())
&& !shuttingDown.get()) {
buffer.clear()
- val fileRecords = log.read(currOffset, config.transactionLogLoadBufferSize, maxOffset = None, minOneMessage = true)
- .records.asInstanceOf[FileRecords]
- val bufferRead = fileRecords.readInto(buffer, 0)
+ val fetchDataInfo = log.read(currOffset, config.transactionLogLoadBufferSize, maxOffset = None,
+ minOneMessage = true, isolationLevel = IsolationLevel.READ_UNCOMMITTED)
+ val memRecords = fetchDataInfo.records match {
+ case records: MemoryRecords => records
+ case fileRecords: FileRecords =>
+ buffer.clear()
+ val bufferRead = fileRecords.readInto(buffer, 0)
+ MemoryRecords.readableRecords(bufferRead)
+ }
- MemoryRecords.readableRecords(bufferRead).batches.asScala.foreach { batch =>
+ memRecords.batches.asScala.foreach { batch =>
for (record <- batch.asScala) {
require(record.hasKey, "Transaction state log's key should not be null")
TransactionLog.readMessageKey(record.key) match {
@@ -414,6 +420,7 @@ class TransactionStateManager(brokerId: Int,
txnMetadata.txnTimeoutMs.toLong,
TransactionLog.EnforcedRequiredAcks,
internalTopicsAllowed = true,
+ isFromClient = false,
recordsPerPartition,
updateCacheCallback)
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/log/AbstractIndex.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/AbstractIndex.scala b/core/src/main/scala/kafka/log/AbstractIndex.scala
index f7478ad..a125676 100644
--- a/core/src/main/scala/kafka/log/AbstractIndex.scala
+++ b/core/src/main/scala/kafka/log/AbstractIndex.scala
@@ -246,14 +246,26 @@ abstract class AbstractIndex[K, V](@volatile var file: File, val baseOffset: Lon
* @param target The index key to look for
* @return The slot found or -1 if the least entry in the index is larger than the target key or the index is empty
*/
- protected def indexSlotFor(idx: ByteBuffer, target: Long, searchEntity: IndexSearchEntity): Int = {
+ protected def largestLowerBoundSlotFor(idx: ByteBuffer, target: Long, searchEntity: IndexSearchEntity): Int =
+ indexSlotRangeFor(idx, target, searchEntity)._1
+
+ /**
+ * Find the smallest entry greater than or equal the target key or value. If none can be found, -1 is returned.
+ */
+ protected def smallestUpperBoundSlotFor(idx: ByteBuffer, target: Long, searchEntity: IndexSearchEntity): Int =
+ indexSlotRangeFor(idx, target, searchEntity)._2
+
+ /**
+ * Lookup lower and upper bounds for the given target.
+ */
+ private def indexSlotRangeFor(idx: ByteBuffer, target: Long, searchEntity: IndexSearchEntity): (Int, Int) = {
// check if the index is empty
if(_entries == 0)
- return -1
+ return (-1, -1)
// check if the target offset is smaller than the least offset
if(compareIndexEntry(parseEntry(idx, 0), target, searchEntity) > 0)
- return -1
+ return (-1, 0)
// binary search for the entry
var lo = 0
@@ -267,9 +279,10 @@ abstract class AbstractIndex[K, V](@volatile var file: File, val baseOffset: Lon
else if(compareResult < 0)
lo = mid
else
- return mid
+ return (mid, mid)
}
- lo
+
+ (lo, if (lo == _entries - 1) -1 else lo + 1)
}
private def compareIndexEntry(indexEntry: IndexEntry, target: Long, searchEntity: IndexSearchEntity): Int = {
[3/6] kafka git commit: KAFKA-5121;
Implement transaction index for KIP-98
Posted by jg...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/log/ProducerIdMapping.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/ProducerIdMapping.scala b/core/src/main/scala/kafka/log/ProducerIdMapping.scala
deleted file mode 100644
index bcadce5..0000000
--- a/core/src/main/scala/kafka/log/ProducerIdMapping.scala
+++ /dev/null
@@ -1,384 +0,0 @@
-/*
- * 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.
- */
-package kafka.log
-
-import java.io._
-import java.nio.ByteBuffer
-import java.nio.file.Files
-
-import kafka.common.KafkaException
-import kafka.utils.{Logging, nonthreadsafe}
-import org.apache.kafka.common.TopicPartition
-import org.apache.kafka.common.errors.{DuplicateSequenceNumberException, OutOfOrderSequenceException, ProducerFencedException}
-import org.apache.kafka.common.protocol.types._
-import org.apache.kafka.common.record.RecordBatch
-import org.apache.kafka.common.utils.{ByteUtils, Crc32C}
-
-import scala.collection.{immutable, mutable}
-
-private[log] object ProducerIdEntry {
- val Empty = ProducerIdEntry(RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
- -1, 0, RecordBatch.NO_TIMESTAMP)
-}
-
-private[log] case class ProducerIdEntry(epoch: Short, lastSeq: Int, lastOffset: Long, offsetDelta: Int, timestamp: Long) {
- def firstSeq: Int = lastSeq - offsetDelta
- def firstOffset: Long = lastOffset - offsetDelta
-
- def isDuplicate(batch: RecordBatch): Boolean = {
- batch.producerEpoch == epoch &&
- batch.baseSequence == firstSeq &&
- batch.lastSequence == lastSeq
- }
-}
-
-private[log] class ProducerAppendInfo(val pid: Long, initialEntry: ProducerIdEntry) {
- // the initialEntry here is the last successful appended batch. we validate incoming entries transitively, starting
- // with the last appended entry.
- private var epoch = initialEntry.epoch
- private var firstSeq = initialEntry.firstSeq
- private var lastSeq = initialEntry.lastSeq
- private var lastOffset = initialEntry.lastOffset
- private var maxTimestamp = initialEntry.timestamp
-
- private def validateAppend(epoch: Short, firstSeq: Int, lastSeq: Int) = {
- if (this.epoch > epoch) {
- throw new ProducerFencedException(s"Producer's epoch is no longer valid. There is probably another producer with a newer epoch. $epoch (request epoch), ${this.epoch} (server epoch)")
- } else if (this.epoch == RecordBatch.NO_PRODUCER_EPOCH || this.epoch < epoch) {
- if (firstSeq != 0)
- throw new OutOfOrderSequenceException(s"Invalid sequence number for new epoch: $epoch " +
- s"(request epoch), $firstSeq (seq. number)")
- } else if (firstSeq == this.firstSeq && lastSeq == this.lastSeq) {
- throw new DuplicateSequenceNumberException(s"Duplicate sequence number: $pid (pid), $firstSeq " +
- s"(seq. number), ${this.firstSeq} (expected seq. number)")
- } else if (firstSeq != this.lastSeq + 1L) {
- throw new OutOfOrderSequenceException(s"Invalid sequence number: $pid (pid), $firstSeq " +
- s"(seq. number), ${this.lastSeq} (expected seq. number)")
- }
- }
-
- def assignLastOffsetAndTimestamp(lastOffset: Long, lastTimestamp: Long): Unit = {
- this.lastOffset = lastOffset
- this.maxTimestamp = lastTimestamp
- }
-
- private def append(epoch: Short, firstSeq: Int, lastSeq: Int, lastTimestamp: Long, lastOffset: Long) {
- validateAppend(epoch, firstSeq, lastSeq)
- this.epoch = epoch
- this.firstSeq = firstSeq
- this.lastSeq = lastSeq
- this.maxTimestamp = lastTimestamp
- this.lastOffset = lastOffset
- }
-
- def append(batch: RecordBatch): Unit =
- append(batch.producerEpoch, batch.baseSequence, batch.lastSequence, batch.maxTimestamp, batch.lastOffset)
-
- def append(entry: ProducerIdEntry): Unit =
- append(entry.epoch, entry.firstSeq, entry.lastSeq, entry.timestamp, entry.lastOffset)
-
- def lastEntry: ProducerIdEntry =
- ProducerIdEntry(epoch, lastSeq, lastOffset, lastSeq - firstSeq, maxTimestamp)
-}
-
-class CorruptSnapshotException(msg: String) extends KafkaException(msg)
-
-object ProducerIdMapping {
- private val PidSnapshotVersion: Short = 1
- private val VersionField = "version"
- private val CrcField = "crc"
- private val PidField = "pid"
- private val LastSequenceField = "last_sequence"
- private val EpochField = "epoch"
- private val LastOffsetField = "last_offset"
- private val OffsetDeltaField = "offset_delta"
- private val TimestampField = "timestamp"
- private val PidEntriesField = "pid_entries"
-
- private val VersionOffset = 0
- private val CrcOffset = VersionOffset + 2
- private val PidEntriesOffset = CrcOffset + 4
-
- private val maxPidSnapshotsToRetain = 2
-
- val PidSnapshotEntrySchema = new Schema(
- new Field(PidField, Type.INT64, "The producer ID"),
- new Field(EpochField, Type.INT16, "Current epoch of the producer"),
- new Field(LastSequenceField, Type.INT32, "Last written sequence of the producer"),
- new Field(LastOffsetField, Type.INT64, "Last written offset of the producer"),
- new Field(OffsetDeltaField, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"),
- new Field(TimestampField, Type.INT64, "Max timestamp from the last written entry"))
- val PidSnapshotMapSchema = new Schema(
- new Field(VersionField, Type.INT16, "Version of the snapshot file"),
- new Field(CrcField, Type.UNSIGNED_INT32, "CRC of the snapshot data"),
- new Field(PidEntriesField, new ArrayOf(PidSnapshotEntrySchema), "The entries in the PID table"))
-
- def readSnapshot(file: File): Iterable[(Long, ProducerIdEntry)] = {
- val buffer = Files.readAllBytes(file.toPath)
- val struct = PidSnapshotMapSchema.read(ByteBuffer.wrap(buffer))
-
- val version = struct.getShort(VersionField)
- if (version != PidSnapshotVersion)
- throw new IllegalArgumentException(s"Unhandled snapshot file version $version")
-
- val crc = struct.getUnsignedInt(CrcField)
- val computedCrc = Crc32C.compute(buffer, PidEntriesOffset, buffer.length - PidEntriesOffset)
- if (crc != computedCrc)
- throw new CorruptSnapshotException(s"Snapshot file '$file' is corrupted (CRC is no longer valid). Stored crc: $crc. Computed crc: $computedCrc")
-
- struct.getArray(PidEntriesField).map { pidEntryObj =>
- val pidEntryStruct = pidEntryObj.asInstanceOf[Struct]
- val pid: Long = pidEntryStruct.getLong(PidField)
- val epoch = pidEntryStruct.getShort(EpochField)
- val seq = pidEntryStruct.getInt(LastSequenceField)
- val offset = pidEntryStruct.getLong(LastOffsetField)
- val timestamp = pidEntryStruct.getLong(TimestampField)
- val offsetDelta = pidEntryStruct.getInt(OffsetDeltaField)
- val newEntry = ProducerIdEntry(epoch, seq, offset, offsetDelta, timestamp)
- pid -> newEntry
- }
- }
-
- private def writeSnapshot(file: File, entries: mutable.Map[Long, ProducerIdEntry]) {
- val struct = new Struct(PidSnapshotMapSchema)
- struct.set(VersionField, PidSnapshotVersion)
- struct.set(CrcField, 0L) // we'll fill this after writing the entries
- val entriesArray = entries.map {
- case (pid, entry) =>
- val pidEntryStruct = struct.instance(PidEntriesField)
- pidEntryStruct.set(PidField, pid)
- .set(EpochField, entry.epoch)
- .set(LastSequenceField, entry.lastSeq)
- .set(LastOffsetField, entry.lastOffset)
- .set(OffsetDeltaField, entry.offsetDelta)
- .set(TimestampField, entry.timestamp)
- pidEntryStruct
- }.toArray
- struct.set(PidEntriesField, entriesArray)
-
- val buffer = ByteBuffer.allocate(struct.sizeOf)
- struct.writeTo(buffer)
- buffer.flip()
-
- // now fill in the CRC
- val crc = Crc32C.compute(buffer, PidEntriesOffset, buffer.limit - PidEntriesOffset)
- ByteUtils.writeUnsignedInt(buffer, CrcOffset, crc)
-
- val fos = new FileOutputStream(file)
- try {
- fos.write(buffer.array, buffer.arrayOffset, buffer.limit)
- } finally {
- fos.close()
- }
- }
-
- private def isSnapshotFile(name: String): Boolean = name.endsWith(Log.PidSnapshotFileSuffix)
-
-}
-
-/**
- * Maintains a mapping from ProducerIds (PIDs) to metadata about the last appended entries (e.g.
- * epoch, sequence number, last offset, etc.)
- *
- * The sequence number is the last number successfully appended to the partition for the given identifier.
- * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
- * appended to the partition.
- *
- * As long as a PID is contained in the map, the corresponding producer can continue to write data.
- * However, PIDs can be expired due to lack of recent use or if the last written entry has been deleted from
- * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
- * that the most recent entry from a given PID is retained in the log provided it hasn't expired due to
- * age. This ensures that PIDs will not be expired until either the max expiration time has been reached,
- * or if the topic also is configured for deletion, the segment containing the last written offset has
- * been deleted.
- */
-@nonthreadsafe
-class ProducerIdMapping(val config: LogConfig,
- val topicPartition: TopicPartition,
- val logDir: File,
- val maxPidExpirationMs: Int) extends Logging {
- import ProducerIdMapping._
-
- private val pidMap = mutable.Map[Long, ProducerIdEntry]()
- private var lastMapOffset = 0L
- private var lastSnapOffset = 0L
-
- /**
- * Returns the last offset of this map
- */
- def mapEndOffset = lastMapOffset
-
- /**
- * Get a copy of the active producers
- */
- def activePids: immutable.Map[Long, ProducerIdEntry] = pidMap.toMap
-
- private def loadFromSnapshot(logStartOffset: Long, currentTime: Long) {
- pidMap.clear()
-
- while (true) {
- latestSnapshotFile match {
- case Some(file) =>
- try {
- info(s"Loading PID mapping from snapshot file ${file.getName} for partition $topicPartition")
- readSnapshot(file).foreach { case (pid, entry) =>
- if (!isExpired(currentTime, entry))
- pidMap.put(pid, entry)
- }
-
- lastSnapOffset = Log.offsetFromFilename(file.getName)
- lastMapOffset = lastSnapOffset
- return
- } catch {
- case e: CorruptSnapshotException =>
- error(s"Snapshot file at ${file.getPath} is corrupt: ${e.getMessage}")
- Files.deleteIfExists(file.toPath)
- }
- case None =>
- lastSnapOffset = logStartOffset
- lastMapOffset = logStartOffset
- return
- }
- }
- }
-
- private def isExpired(currentTimeMs: Long, producerIdEntry: ProducerIdEntry) : Boolean =
- currentTimeMs - producerIdEntry.timestamp >= maxPidExpirationMs
-
-
- def removeExpiredPids(currentTimeMs: Long) {
- pidMap.retain { case (pid, lastEntry) =>
- !isExpired(currentTimeMs, lastEntry)
- }
- }
-
- /**
- * Truncate the PID mapping to the given offset range and reload the entries from the most recent
- * snapshot in range (if there is one).
- */
- def truncateAndReload(logStartOffset: Long, logEndOffset: Long, currentTimeMs: Long) {
- if (logEndOffset != mapEndOffset) {
- deleteSnapshotFiles { file =>
- val offset = Log.offsetFromFilename(file.getName)
- offset > logEndOffset || offset <= logStartOffset
- }
- loadFromSnapshot(logStartOffset, currentTimeMs)
- } else {
- expirePids(logStartOffset)
- }
- }
-
- /**
- * Update the mapping with the given append information
- */
- def update(appendInfo: ProducerAppendInfo): Unit = {
- if (appendInfo.pid == RecordBatch.NO_PRODUCER_ID)
- throw new IllegalArgumentException("Invalid PID passed to update")
- val entry = appendInfo.lastEntry
- pidMap.put(appendInfo.pid, entry)
- }
-
- def updateMapEndOffset(lastOffset: Long): Unit = {
- lastMapOffset = lastOffset
- }
-
- /**
- * Load a previously stored PID entry into the cache. Ignore the entry if the timestamp is older
- * than the current time minus the PID expiration time (i.e. if the PID has expired).
- */
- def load(pid: Long, entry: ProducerIdEntry, currentTimeMs: Long) {
- if (pid != RecordBatch.NO_PRODUCER_ID && !isExpired(currentTimeMs, entry))
- pidMap.put(pid, entry)
- }
-
- /**
- * Get the last written entry for the given PID.
- */
- def lastEntry(pid: Long): Option[ProducerIdEntry] = pidMap.get(pid)
-
- /**
- * Write a new snapshot if there have been updates since the last one.
- */
- def maybeTakeSnapshot() {
- // If not a new offset, then it is not worth taking another snapshot
- if (lastMapOffset > lastSnapOffset) {
- val snapshotFile = Log.pidSnapshotFilename(logDir, lastMapOffset)
- debug(s"Writing producer snapshot for partition $topicPartition at offset $lastMapOffset")
- writeSnapshot(snapshotFile, pidMap)
-
- // Update the last snap offset according to the serialized map
- lastSnapOffset = lastMapOffset
-
- maybeRemoveOldestSnapshot()
- }
- }
-
- /**
- * Get the last offset (exclusive) of the latest snapshot file.
- */
- def latestSnapshotOffset: Option[Long] = latestSnapshotFile.map(file => Log.offsetFromFilename(file.getName))
-
- /**
- * When we remove the head of the log due to retention, we need to clean up the id map. This method takes
- * the new start offset and expires all pids which have a smaller last written offset.
- */
- def expirePids(logStartOffset: Long) {
- pidMap.retain((pid, entry) => entry.lastOffset >= logStartOffset)
- deleteSnapshotFiles(file => Log.offsetFromFilename(file.getName) <= logStartOffset)
- if (lastMapOffset < logStartOffset)
- lastMapOffset = logStartOffset
- lastSnapOffset = latestSnapshotOffset.getOrElse(logStartOffset)
- }
-
- /**
- * Truncate the PID mapping and remove all snapshots. This resets the state of the mapping.
- */
- def truncate() {
- pidMap.clear()
- deleteSnapshotFiles()
- lastSnapOffset = 0L
- lastMapOffset = 0L
- }
-
- private def maybeRemoveOldestSnapshot() {
- val list = listSnapshotFiles
- if (list.size > maxPidSnapshotsToRetain) {
- val toDelete = list.minBy(file => Log.offsetFromFilename(file.getName))
- Files.deleteIfExists(toDelete.toPath)
- }
- }
-
- private def listSnapshotFiles: List[File] = {
- if (logDir.exists && logDir.isDirectory)
- logDir.listFiles.filter(f => f.isFile && isSnapshotFile(f.getName)).toList
- else
- List.empty[File]
- }
-
- private def latestSnapshotFile: Option[File] = {
- val files = listSnapshotFiles
- if (files.nonEmpty)
- Some(files.maxBy(file => Log.offsetFromFilename(file.getName)))
- else
- None
- }
-
- private def deleteSnapshotFiles(predicate: File => Boolean = _ => true) {
- listSnapshotFiles.filter(predicate).foreach(file => Files.deleteIfExists(file.toPath))
- }
-
-}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/log/ProducerStateManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/ProducerStateManager.scala b/core/src/main/scala/kafka/log/ProducerStateManager.scala
new file mode 100644
index 0000000..b1a43d2
--- /dev/null
+++ b/core/src/main/scala/kafka/log/ProducerStateManager.scala
@@ -0,0 +1,590 @@
+/*
+ * 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.
+ */
+package kafka.log
+
+import java.io._
+import java.nio.ByteBuffer
+import java.nio.file.Files
+
+import kafka.common.KafkaException
+import kafka.log.Log.offsetFromFilename
+import kafka.server.LogOffsetMetadata
+import kafka.utils.{Logging, nonthreadsafe, threadsafe}
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.errors._
+import org.apache.kafka.common.protocol.types._
+import org.apache.kafka.common.record.{ControlRecordType, EndTransactionMarker, RecordBatch}
+import org.apache.kafka.common.utils.{ByteUtils, Crc32C}
+
+import scala.collection.mutable.ListBuffer
+import scala.collection.{immutable, mutable}
+
+class CorruptSnapshotException(msg: String) extends KafkaException(msg)
+
+private[log] case class TxnMetadata(producerId: Long, var firstOffset: LogOffsetMetadata, var lastOffset: Option[Long] = None) {
+ def this(producerId: Long, firstOffset: Long) = this(producerId, LogOffsetMetadata(firstOffset))
+}
+
+private[log] object ProducerIdEntry {
+ val Empty = ProducerIdEntry(RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
+ -1, 0, RecordBatch.NO_TIMESTAMP, -1, None)
+}
+
+private[log] case class ProducerIdEntry(producerId: Long, producerEpoch: Short, lastSeq: Int, lastOffset: Long,
+ offsetDelta: Int, timestamp: Long, coordinatorEpoch: Int,
+ currentTxnFirstOffset: Option[Long]) {
+ def firstSeq: Int = lastSeq - offsetDelta
+ def firstOffset: Long = lastOffset - offsetDelta
+
+ def isDuplicate(batch: RecordBatch): Boolean = {
+ batch.producerEpoch == producerEpoch &&
+ batch.baseSequence == firstSeq &&
+ batch.lastSequence == lastSeq
+ }
+}
+
+/**
+ * This class is used to validate the records appended by a given producer before they are written to the log.
+ * It is initialized with the producer's state after the last successful append, and transitively validates the
+ * sequence numbers and epochs of each new record. Additionally, this class accumulates transaction metadata
+ * as the incoming records are validated.
+ */
+private[log] class ProducerAppendInfo(val producerId: Long, initialEntry: ProducerIdEntry, loadingFromLog: Boolean = false) {
+ private var producerEpoch = initialEntry.producerEpoch
+ private var firstSeq = initialEntry.firstSeq
+ private var lastSeq = initialEntry.lastSeq
+ private var lastOffset = initialEntry.lastOffset
+ private var maxTimestamp = initialEntry.timestamp
+ private var currentTxnFirstOffset = initialEntry.currentTxnFirstOffset
+ private var coordinatorEpoch = initialEntry.coordinatorEpoch
+ private val transactions = ListBuffer.empty[TxnMetadata]
+
+ def this(pid: Long, initialEntry: Option[ProducerIdEntry], loadingFromLog: Boolean) =
+ this(pid, initialEntry.getOrElse(ProducerIdEntry.Empty), loadingFromLog)
+
+ private def validateAppend(epoch: Short, firstSeq: Int, lastSeq: Int) = {
+ if (this.producerEpoch > epoch) {
+ throw new ProducerFencedException(s"Producer's epoch is no longer valid. There is probably another producer " +
+ s"with a newer epoch. $epoch (request epoch), ${this.producerEpoch} (server epoch)")
+ } else if (this.producerEpoch == RecordBatch.NO_PRODUCER_EPOCH || this.producerEpoch < epoch) {
+ if (firstSeq != 0)
+ throw new OutOfOrderSequenceException(s"Invalid sequence number for new epoch: $epoch " +
+ s"(request epoch), $firstSeq (seq. number)")
+ } else if (this.firstSeq == RecordBatch.NO_SEQUENCE && firstSeq != 0) {
+ // the epoch was bumped by a control record, so we expect the sequence number to be reset
+ throw new OutOfOrderSequenceException(s"Out of order sequence number: $producerId (pid), found $firstSeq " +
+ s"(incoming seq. number), but expected 0")
+ } else if (firstSeq == this.firstSeq && lastSeq == this.lastSeq) {
+ throw new DuplicateSequenceNumberException(s"Duplicate sequence number: pid: $producerId, (incomingBatch.firstSeq, " +
+ s"incomingBatch.lastSeq): ($firstSeq, $lastSeq), (lastEntry.firstSeq, lastEntry.lastSeq): " +
+ s"(${this.firstSeq}, ${this.lastSeq}).")
+ } else if (firstSeq != this.lastSeq + 1L) {
+ throw new OutOfOrderSequenceException(s"Out of order sequence number: $producerId (pid), $firstSeq " +
+ s"(incoming seq. number), ${this.lastSeq} (current end sequence number)")
+ }
+ }
+
+ def append(batch: RecordBatch): Option[CompletedTxn] = {
+ if (batch.isControlBatch) {
+ val record = batch.iterator.next()
+ val endTxnMarker = EndTransactionMarker.deserialize(record)
+ val completedTxn = appendEndTxnMarker(endTxnMarker, batch.producerEpoch, batch.baseOffset, record.timestamp)
+ Some(completedTxn)
+ } else {
+ append(batch.producerEpoch, batch.baseSequence, batch.lastSequence, batch.maxTimestamp, batch.lastOffset,
+ batch.isTransactional)
+ None
+ }
+ }
+
+ def append(epoch: Short,
+ firstSeq: Int,
+ lastSeq: Int,
+ lastTimestamp: Long,
+ lastOffset: Long,
+ isTransactional: Boolean): Unit = {
+ if (epoch != RecordBatch.NO_PRODUCER_EPOCH && !loadingFromLog)
+ // skip validation if this is the first entry when loading from the log. Log retention
+ // will generally have removed the beginning entries from each PID
+ validateAppend(epoch, firstSeq, lastSeq)
+
+ this.producerEpoch = epoch
+ this.firstSeq = firstSeq
+ this.lastSeq = lastSeq
+ this.maxTimestamp = lastTimestamp
+ this.lastOffset = lastOffset
+
+ if (currentTxnFirstOffset.isDefined && !isTransactional)
+ throw new InvalidTxnStateException(s"Expected transactional write from producer $producerId")
+
+ if (isTransactional && currentTxnFirstOffset.isEmpty) {
+ val firstOffset = lastOffset - (lastSeq - firstSeq)
+ currentTxnFirstOffset = Some(firstOffset)
+ transactions += new TxnMetadata(producerId, firstOffset)
+ }
+ }
+
+ def appendEndTxnMarker(endTxnMarker: EndTransactionMarker,
+ producerEpoch: Short,
+ offset: Long,
+ timestamp: Long): CompletedTxn = {
+ if (this.producerEpoch > producerEpoch)
+ throw new ProducerFencedException(s"Invalid producer epoch: $producerEpoch (zombie): ${this.producerEpoch} (current)")
+
+ if (this.coordinatorEpoch > endTxnMarker.coordinatorEpoch)
+ throw new TransactionCoordinatorFencedException(s"Invalid coordinator epoch: ${endTxnMarker.coordinatorEpoch} " +
+ s"(zombie), $coordinatorEpoch (current)")
+
+ if (producerEpoch > this.producerEpoch) {
+ // it is possible that this control record is the first record seen from a new epoch (the producer
+ // may fail before sending to the partition or the request itself could fail for some reason). In this
+ // case, we bump the epoch and reset the sequence numbers
+ this.producerEpoch = producerEpoch
+ this.firstSeq = RecordBatch.NO_SEQUENCE
+ this.lastSeq = RecordBatch.NO_SEQUENCE
+ } else {
+ // the control record is the last append to the log, so the last offset will be updated to point to it.
+ // However, the sequence numbers still point to the previous batch, so the duplicate check would no longer
+ // be correct: it would return the wrong offset. To fix this, we treat the control record as a batch
+ // of size 1 which uses the last appended sequence number.
+ this.firstSeq = this.lastSeq
+ }
+
+ val firstOffset = currentTxnFirstOffset match {
+ case Some(firstOffset) => firstOffset
+ case None =>
+ transactions += new TxnMetadata(producerId, offset)
+ offset
+ }
+
+ this.lastOffset = offset
+ this.currentTxnFirstOffset = None
+ this.maxTimestamp = timestamp
+ this.coordinatorEpoch = endTxnMarker.coordinatorEpoch
+ CompletedTxn(producerId, firstOffset, offset, endTxnMarker.controlType == ControlRecordType.ABORT)
+ }
+
+ def lastEntry: ProducerIdEntry =
+ ProducerIdEntry(producerId, producerEpoch, lastSeq, lastOffset, lastSeq - firstSeq, maxTimestamp,
+ coordinatorEpoch, currentTxnFirstOffset)
+
+ def startedTransactions: List[TxnMetadata] = transactions.toList
+
+ def maybeCacheTxnFirstOffsetMetadata(logOffsetMetadata: LogOffsetMetadata): Unit = {
+ // we will cache the log offset metadata if it corresponds to the starting offset of
+ // the last transaction that was started. This is optimized for leader appends where it
+ // is only possible to have one transaction started for each log append, and the log
+ // offset metadata will always match in that case since no data from other producers
+ // is mixed into the append
+ transactions.headOption.foreach { txn =>
+ if (txn.firstOffset.messageOffset == logOffsetMetadata.messageOffset)
+ txn.firstOffset = logOffsetMetadata
+ }
+ }
+
+}
+
+object ProducerStateManager {
+ private val PidSnapshotVersion: Short = 1
+ private val VersionField = "version"
+ private val CrcField = "crc"
+ private val PidField = "pid"
+ private val LastSequenceField = "last_sequence"
+ private val ProducerEpochField = "epoch"
+ private val LastOffsetField = "last_offset"
+ private val OffsetDeltaField = "offset_delta"
+ private val TimestampField = "timestamp"
+ private val PidEntriesField = "pid_entries"
+ private val CoordinatorEpochField = "coordinator_epoch"
+ private val CurrentTxnFirstOffsetField = "current_txn_first_offset"
+
+ private val VersionOffset = 0
+ private val CrcOffset = VersionOffset + 2
+ private val PidEntriesOffset = CrcOffset + 4
+
+ val PidSnapshotEntrySchema = new Schema(
+ new Field(PidField, Type.INT64, "The producer ID"),
+ new Field(ProducerEpochField, Type.INT16, "Current epoch of the producer"),
+ new Field(LastSequenceField, Type.INT32, "Last written sequence of the producer"),
+ new Field(LastOffsetField, Type.INT64, "Last written offset of the producer"),
+ new Field(OffsetDeltaField, Type.INT32, "The difference of the last sequence and first sequence in the last written batch"),
+ new Field(TimestampField, Type.INT64, "Max timestamp from the last written entry"),
+ new Field(CoordinatorEpochField, Type.INT32, "The epoch of the last transaction coordinator to send an end transaction marker"),
+ new Field(CurrentTxnFirstOffsetField, Type.INT64, "The first offset of the on-going transaction (-1 if there is none)"))
+ val PidSnapshotMapSchema = new Schema(
+ new Field(VersionField, Type.INT16, "Version of the snapshot file"),
+ new Field(CrcField, Type.UNSIGNED_INT32, "CRC of the snapshot data"),
+ new Field(PidEntriesField, new ArrayOf(PidSnapshotEntrySchema), "The entries in the PID table"))
+
+ def readSnapshot(file: File): Iterable[ProducerIdEntry] = {
+ val buffer = Files.readAllBytes(file.toPath)
+ val struct = PidSnapshotMapSchema.read(ByteBuffer.wrap(buffer))
+
+ val version = struct.getShort(VersionField)
+ if (version != PidSnapshotVersion)
+ throw new IllegalArgumentException(s"Unhandled snapshot file version $version")
+
+ val crc = struct.getUnsignedInt(CrcField)
+ val computedCrc = Crc32C.compute(buffer, PidEntriesOffset, buffer.length - PidEntriesOffset)
+ if (crc != computedCrc)
+ throw new CorruptSnapshotException(s"Snapshot file '$file' is corrupted (CRC is no longer valid). " +
+ s"Stored crc: $crc. Computed crc: $computedCrc")
+
+ struct.getArray(PidEntriesField).map { pidEntryObj =>
+ val pidEntryStruct = pidEntryObj.asInstanceOf[Struct]
+ val pid: Long = pidEntryStruct.getLong(PidField)
+ val epoch = pidEntryStruct.getShort(ProducerEpochField)
+ val seq = pidEntryStruct.getInt(LastSequenceField)
+ val offset = pidEntryStruct.getLong(LastOffsetField)
+ val timestamp = pidEntryStruct.getLong(TimestampField)
+ val offsetDelta = pidEntryStruct.getInt(OffsetDeltaField)
+ val coordinatorEpoch = pidEntryStruct.getInt(CoordinatorEpochField)
+ val currentTxnFirstOffset = pidEntryStruct.getLong(CurrentTxnFirstOffsetField)
+ val newEntry = ProducerIdEntry(pid, epoch, seq, offset, offsetDelta, timestamp,
+ coordinatorEpoch, if (currentTxnFirstOffset >= 0) Some(currentTxnFirstOffset) else None)
+ newEntry
+ }
+ }
+
+ private def writeSnapshot(file: File, entries: mutable.Map[Long, ProducerIdEntry]) {
+ val struct = new Struct(PidSnapshotMapSchema)
+ struct.set(VersionField, PidSnapshotVersion)
+ struct.set(CrcField, 0L) // we'll fill this after writing the entries
+ val entriesArray = entries.map {
+ case (pid, entry) =>
+ val pidEntryStruct = struct.instance(PidEntriesField)
+ pidEntryStruct.set(PidField, pid)
+ .set(ProducerEpochField, entry.producerEpoch)
+ .set(LastSequenceField, entry.lastSeq)
+ .set(LastOffsetField, entry.lastOffset)
+ .set(OffsetDeltaField, entry.offsetDelta)
+ .set(TimestampField, entry.timestamp)
+ .set(CoordinatorEpochField, entry.coordinatorEpoch)
+ .set(CurrentTxnFirstOffsetField, entry.currentTxnFirstOffset.getOrElse(-1L))
+ pidEntryStruct
+ }.toArray
+ struct.set(PidEntriesField, entriesArray)
+
+ val buffer = ByteBuffer.allocate(struct.sizeOf)
+ struct.writeTo(buffer)
+ buffer.flip()
+
+ // now fill in the CRC
+ val crc = Crc32C.compute(buffer, PidEntriesOffset, buffer.limit - PidEntriesOffset)
+ ByteUtils.writeUnsignedInt(buffer, CrcOffset, crc)
+
+ val fos = new FileOutputStream(file)
+ try {
+ fos.write(buffer.array, buffer.arrayOffset, buffer.limit)
+ } finally {
+ fos.close()
+ }
+ }
+
+ private def isSnapshotFile(name: String): Boolean = name.endsWith(Log.PidSnapshotFileSuffix)
+
+}
+
+/**
+ * Maintains a mapping from ProducerIds (PIDs) to metadata about the last appended entries (e.g.
+ * epoch, sequence number, last offset, etc.)
+ *
+ * The sequence number is the last number successfully appended to the partition for the given identifier.
+ * The epoch is used for fencing against zombie writers. The offset is the one of the last successful message
+ * appended to the partition.
+ *
+ * As long as a PID is contained in the map, the corresponding producer can continue to write data.
+ * However, PIDs can be expired due to lack of recent use or if the last written entry has been deleted from
+ * the log (e.g. if the retention policy is "delete"). For compacted topics, the log cleaner will ensure
+ * that the most recent entry from a given PID is retained in the log provided it hasn't expired due to
+ * age. This ensures that PIDs will not be expired until either the max expiration time has been reached,
+ * or if the topic also is configured for deletion, the segment containing the last written offset has
+ * been deleted.
+ */
+@nonthreadsafe
+class ProducerStateManager(val topicPartition: TopicPartition,
+ val logDir: File,
+ val maxPidExpirationMs: Int = 60 * 60 * 1000) extends Logging {
+ import ProducerStateManager._
+ import java.util
+
+ private val producers = mutable.Map.empty[Long, ProducerIdEntry]
+ private var lastMapOffset = 0L
+ private var lastSnapOffset = 0L
+
+ // ongoing transactions sorted by the first offset of the transaction
+ private val ongoingTxns = new util.TreeMap[Long, TxnMetadata]
+
+ // completed transactions whose markers are at offsets above the high watermark
+ private val unreplicatedTxns = new util.TreeMap[Long, TxnMetadata]
+
+ /**
+ * An unstable offset is one which is either undecided (i.e. its ultimate outcome is not yet known),
+ * or one that is decided, but may not have been replicated (i.e. any transaction which has a COMMIT/ABORT
+ * marker written at a higher offset than the current high watermark).
+ */
+ def firstUnstableOffset: Option[LogOffsetMetadata] = {
+ val unreplicatedFirstOffset = Option(unreplicatedTxns.firstEntry).map(_.getValue.firstOffset)
+ val undecidedFirstOffset = Option(ongoingTxns.firstEntry).map(_.getValue.firstOffset)
+ if (unreplicatedFirstOffset.isEmpty)
+ undecidedFirstOffset
+ else if (undecidedFirstOffset.isEmpty)
+ unreplicatedFirstOffset
+ else if (undecidedFirstOffset.get.messageOffset < unreplicatedFirstOffset.get.messageOffset)
+ undecidedFirstOffset
+ else
+ unreplicatedFirstOffset
+ }
+
+ /**
+ * Acknowledge all transactions which have been completed before a given offset. This allows the LSO
+ * to advance to the next unstable offset.
+ */
+ def onHighWatermarkUpdated(highWatermark: Long): Unit = {
+ removeUnreplicatedTransactions(highWatermark)
+ }
+
+ /**
+ * The first undecided offset is the earliest transactional message which has not yet been committed
+ * or aborted.
+ */
+ def firstUndecidedOffset: Option[Long] = Option(ongoingTxns.firstEntry).map(_.getValue.firstOffset.messageOffset)
+
+ /**
+ * Returns the last offset of this map
+ */
+ def mapEndOffset = lastMapOffset
+
+ /**
+ * Get a copy of the active producers
+ */
+ def activeProducers: immutable.Map[Long, ProducerIdEntry] = producers.toMap
+
+ def isEmpty: Boolean = producers.isEmpty && unreplicatedTxns.isEmpty
+
+ private def loadFromSnapshot(logStartOffset: Long, currentTime: Long) {
+ while (true) {
+ latestSnapshotFile match {
+ case Some(file) =>
+ try {
+ info(s"Loading producer state from snapshot file ${file.getName} for partition $topicPartition")
+ readSnapshot(file).filter(!isExpired(currentTime, _)).foreach(loadProducerEntry)
+ lastSnapOffset = offsetFromFilename(file.getName)
+ lastMapOffset = lastSnapOffset
+ return
+ } catch {
+ case e: CorruptSnapshotException =>
+ error(s"Snapshot file at ${file.getPath} is corrupt: ${e.getMessage}")
+ Files.deleteIfExists(file.toPath)
+ }
+ case None =>
+ lastSnapOffset = logStartOffset
+ lastMapOffset = logStartOffset
+ return
+ }
+ }
+ }
+
+ // visible for testing
+ private[log] def loadProducerEntry(entry: ProducerIdEntry): Unit = {
+ val pid = entry.producerId
+ producers.put(pid, entry)
+ entry.currentTxnFirstOffset.foreach { offset =>
+ ongoingTxns.put(offset, new TxnMetadata(pid, offset))
+ }
+ }
+
+ private def isExpired(currentTimeMs: Long, producerIdEntry: ProducerIdEntry): Boolean =
+ producerIdEntry.currentTxnFirstOffset.isEmpty && currentTimeMs - producerIdEntry.timestamp >= maxPidExpirationMs
+
+ /**
+ * Expire any PIDs which have been idle longer than the configured maximum expiration timeout.
+ */
+ def removeExpiredProducers(currentTimeMs: Long) {
+ producers.retain { case (pid, lastEntry) =>
+ !isExpired(currentTimeMs, lastEntry)
+ }
+ }
+
+ /**
+ * Truncate the PID mapping to the given offset range and reload the entries from the most recent
+ * snapshot in range (if there is one). Note that the log end offset is assumed to be less than
+ * or equal to the high watermark.
+ */
+ def truncateAndReload(logStartOffset: Long, logEndOffset: Long, currentTimeMs: Long) {
+ if (logEndOffset != mapEndOffset) {
+ producers.clear()
+ ongoingTxns.clear()
+
+ // since we assume that the offset is less than or equal to the high watermark, it is
+ // safe to clear the unreplicated transactions
+ unreplicatedTxns.clear()
+ deleteSnapshotFiles { file =>
+ val offset = offsetFromFilename(file.getName)
+ offset > logEndOffset || offset <= logStartOffset
+ }
+ loadFromSnapshot(logStartOffset, currentTimeMs)
+ } else {
+ evictUnretainedProducers(logStartOffset)
+ }
+ }
+
+ /**
+ * Update the mapping with the given append information
+ */
+ def update(appendInfo: ProducerAppendInfo): Unit = {
+ if (appendInfo.producerId == RecordBatch.NO_PRODUCER_ID)
+ throw new IllegalArgumentException("Invalid PID passed to update")
+
+ val entry = appendInfo.lastEntry
+ producers.put(appendInfo.producerId, entry)
+ appendInfo.startedTransactions.foreach { txn =>
+ ongoingTxns.put(txn.firstOffset.messageOffset, txn)
+ }
+ }
+
+ def updateMapEndOffset(lastOffset: Long): Unit = {
+ lastMapOffset = lastOffset
+ }
+
+ /**
+ * Get the last written entry for the given PID.
+ */
+ def lastEntry(producerId: Long): Option[ProducerIdEntry] = producers.get(producerId)
+
+ /**
+ * Take a snapshot at the current end offset if one does not already exist.
+ */
+ def takeSnapshot(): Unit = {
+ // If not a new offset, then it is not worth taking another snapshot
+ if (lastMapOffset > lastSnapOffset) {
+ val snapshotFile = Log.producerSnapshotFile(logDir, lastMapOffset)
+ debug(s"Writing producer snapshot for partition $topicPartition at offset $lastMapOffset")
+ writeSnapshot(snapshotFile, producers)
+
+ // Update the last snap offset according to the serialized map
+ lastSnapOffset = lastMapOffset
+ }
+ }
+
+ /**
+ * Get the last offset (exclusive) of the latest snapshot file.
+ */
+ def latestSnapshotOffset: Option[Long] = latestSnapshotFile.map(file => offsetFromFilename(file.getName))
+
+ /**
+ * Get the last offset (exclusive) of the oldest snapshot file.
+ */
+ def oldestSnapshotOffset: Option[Long] = oldestSnapshotFile.map(file => offsetFromFilename(file.getName))
+
+ /**
+ * When we remove the head of the log due to retention, we need to clean up the id map. This method takes
+ * the new start offset and removes all pids which have a smaller last written offset.
+ */
+ def evictUnretainedProducers(logStartOffset: Long) {
+ val evictedProducerEntries = producers.filter(_._2.lastOffset < logStartOffset)
+ val evictedProducerIds = evictedProducerEntries.keySet
+
+ producers --= evictedProducerIds
+ removeEvictedOngoingTransactions(evictedProducerIds)
+ removeUnreplicatedTransactions(logStartOffset)
+
+ deleteSnapshotFiles(file => offsetFromFilename(file.getName) <= logStartOffset)
+ if (lastMapOffset < logStartOffset)
+ lastMapOffset = logStartOffset
+ lastSnapOffset = latestSnapshotOffset.getOrElse(logStartOffset)
+ }
+
+ private def removeEvictedOngoingTransactions(expiredProducerIds: collection.Set[Long]): Unit = {
+ val iterator = ongoingTxns.entrySet.iterator
+ while (iterator.hasNext) {
+ val txnEntry = iterator.next()
+ if (expiredProducerIds.contains(txnEntry.getValue.producerId))
+ iterator.remove()
+ }
+ }
+
+ private def removeUnreplicatedTransactions(offset: Long): Unit = {
+ val iterator = unreplicatedTxns.entrySet.iterator
+ while (iterator.hasNext) {
+ val txnEntry = iterator.next()
+ val lastOffset = txnEntry.getValue.lastOffset
+ if (lastOffset.exists(_ < offset))
+ iterator.remove()
+ }
+ }
+
+ /**
+ * Truncate the PID mapping and remove all snapshots. This resets the state of the mapping.
+ */
+ def truncate() {
+ producers.clear()
+ ongoingTxns.clear()
+ unreplicatedTxns.clear()
+ deleteSnapshotFiles()
+ lastSnapOffset = 0L
+ lastMapOffset = 0L
+ }
+
+ /**
+ * Complete the transaction and return the last stable offset.
+ */
+ def completeTxn(completedTxn: CompletedTxn): Long = {
+ val txnMetdata = ongoingTxns.remove(completedTxn.firstOffset)
+ if (txnMetdata == null)
+ throw new IllegalArgumentException("Attempted to complete a transaction which was not started")
+
+ txnMetdata.lastOffset = Some(completedTxn.lastOffset)
+ unreplicatedTxns.put(completedTxn.firstOffset, txnMetdata)
+
+ val lastStableOffset = firstUndecidedOffset.getOrElse(completedTxn.lastOffset + 1)
+ lastStableOffset
+ }
+
+ @threadsafe
+ def deleteSnapshotsBefore(offset: Long): Unit = {
+ deleteSnapshotFiles(file => offsetFromFilename(file.getName) < offset)
+ }
+
+ private def listSnapshotFiles: List[File] = {
+ if (logDir.exists && logDir.isDirectory)
+ logDir.listFiles.filter(f => f.isFile && isSnapshotFile(f.getName)).toList
+ else
+ List.empty[File]
+ }
+
+ private def oldestSnapshotFile: Option[File] = {
+ val files = listSnapshotFiles
+ if (files.nonEmpty)
+ Some(files.minBy(file => offsetFromFilename(file.getName)))
+ else
+ None
+ }
+
+ private def latestSnapshotFile: Option[File] = {
+ val files = listSnapshotFiles
+ if (files.nonEmpty)
+ Some(files.maxBy(file => offsetFromFilename(file.getName)))
+ else
+ None
+ }
+
+ private def deleteSnapshotFiles(predicate: File => Boolean = _ => true) {
+ listSnapshotFiles.filter(predicate).foreach(file => Files.deleteIfExists(file.toPath))
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/log/TimeIndex.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/TimeIndex.scala b/core/src/main/scala/kafka/log/TimeIndex.scala
index 731b173..19ab71a 100644
--- a/core/src/main/scala/kafka/log/TimeIndex.scala
+++ b/core/src/main/scala/kafka/log/TimeIndex.scala
@@ -144,7 +144,7 @@ class TimeIndex(file: File,
def lookup(targetTimestamp: Long): TimestampOffset = {
maybeLock(lock) {
val idx = mmap.duplicate
- val slot = indexSlotFor(idx, targetTimestamp, IndexSearchType.KEY)
+ val slot = largestLowerBoundSlotFor(idx, targetTimestamp, IndexSearchType.KEY)
if (slot == -1)
TimestampOffset(RecordBatch.NO_TIMESTAMP, baseOffset)
else {
@@ -163,7 +163,7 @@ class TimeIndex(file: File,
override def truncateTo(offset: Long) {
inLock(lock) {
val idx = mmap.duplicate
- val slot = indexSlotFor(idx, offset, IndexSearchType.VALUE)
+ val slot = largestLowerBoundSlotFor(idx, offset, IndexSearchType.VALUE)
/* There are 3 cases for choosing the new size
* 1) if there is no entry in the index <= the offset, delete everything
@@ -206,4 +206,5 @@ class TimeIndex(file: File,
"Time index file " + file.getAbsolutePath + " is corrupt, found " + len +
" bytes which is not positive or not a multiple of 12.")
}
-}
\ No newline at end of file
+
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/log/TransactionIndex.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/TransactionIndex.scala b/core/src/main/scala/kafka/log/TransactionIndex.scala
new file mode 100644
index 0000000..bf6a6d4
--- /dev/null
+++ b/core/src/main/scala/kafka/log/TransactionIndex.scala
@@ -0,0 +1,243 @@
+/*
+ * 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.
+ */
+package kafka.log
+
+import java.io.{File, IOException}
+import java.nio.ByteBuffer
+import java.nio.channels.FileChannel
+import java.nio.file.StandardOpenOption
+
+import kafka.utils.{Logging, nonthreadsafe}
+import org.apache.kafka.common.KafkaException
+import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction
+import org.apache.kafka.common.utils.Utils
+
+import scala.collection.mutable.ListBuffer
+
+private[log] case class TxnIndexSearchResult(abortedTransactions: List[AbortedTransaction], isComplete: Boolean)
+
+/**
+ * The transaction index maintains metadata about the aborted transactions for each segment. This includes
+ * the start and end offsets for the aborted transactions and the last stable offset (LSO) at the time of
+ * the abort. This index is used to find the aborted transactions in the range of a given fetch request at
+ * the READ_COMMITTED isolation level.
+ *
+ * There is at most one transaction index for each log segment. The entries correspond to the transactions
+ * whose commit markers were written in the corresponding log segment. Note, however, that individual transactions
+ * may span multiple segments. Recovering the index therefore requires scanning the earlier segments in
+ * order to find the start of the transactions.
+ */
+@nonthreadsafe
+class TransactionIndex(val startOffset: Long, @volatile var file: File) extends Logging {
+ // note that the file is not created until we need it
+ @volatile private var maybeChannel: Option[FileChannel] = None
+ private var lastOffset: Option[Long] = None
+
+ if (file.exists)
+ openChannel()
+
+ def append(abortedTxn: AbortedTxn): Unit = {
+ lastOffset.foreach { offset =>
+ if (offset >= abortedTxn.lastOffset)
+ throw new IllegalArgumentException("The last offset of appended transactions must increase sequentially")
+ }
+ lastOffset = Some(abortedTxn.lastOffset)
+ Utils.writeFully(channel, abortedTxn.buffer.duplicate())
+ }
+
+ def flush(): Unit = maybeChannel.foreach(_.force(true))
+
+ def delete(): Boolean = {
+ maybeChannel.forall { channel =>
+ channel.force(true)
+ close()
+ file.delete()
+ }
+ }
+
+ private def channel: FileChannel = {
+ maybeChannel match {
+ case Some(channel) => channel
+ case None => openChannel()
+ }
+ }
+
+ private def openChannel(): FileChannel = {
+ val channel = FileChannel.open(file.toPath, StandardOpenOption.READ, StandardOpenOption.WRITE,
+ StandardOpenOption.CREATE)
+ maybeChannel = Some(channel)
+ channel.position(channel.size)
+ channel
+ }
+
+ def truncate() = {
+ maybeChannel.foreach(_.truncate(0))
+ lastOffset = None
+ }
+
+ def close(): Unit = {
+ maybeChannel.foreach(_.close())
+ maybeChannel = None
+ }
+
+ def renameTo(f: File): Unit = {
+ try {
+ if (file.exists)
+ Utils.atomicMoveWithFallback(file.toPath, f.toPath)
+ } finally file = f
+ }
+
+ def truncateTo(offset: Long): Unit = {
+ val buffer = ByteBuffer.allocate(AbortedTxn.TotalSize)
+ var newLastOffset: Option[Long] = None
+ for ((abortedTxn, position) <- iterator(() => buffer)) {
+ if (abortedTxn.lastOffset >= offset) {
+ channel.truncate(position)
+ lastOffset = newLastOffset
+ return
+ }
+ newLastOffset = Some(abortedTxn.lastOffset)
+ }
+ }
+
+ private def iterator(allocate: () => ByteBuffer): Iterator[(AbortedTxn, Int)] = {
+ maybeChannel match {
+ case None => Iterator.empty
+ case Some(channel) =>
+ var position = 0
+
+ new Iterator[(AbortedTxn, Int)] {
+ override def hasNext: Boolean = channel.position - position >= AbortedTxn.TotalSize
+
+ override def next(): (AbortedTxn, Int) = {
+ try {
+ val buffer = allocate()
+ Utils.readFully(channel, buffer, position)
+ buffer.flip()
+
+ val abortedTxn = new AbortedTxn(buffer)
+ if (abortedTxn.version > AbortedTxn.CurrentVersion)
+ throw new KafkaException(s"Unexpected aborted transaction version ${abortedTxn.version}, " +
+ s"current version is ${AbortedTxn.CurrentVersion}")
+ val nextEntry = (abortedTxn, position)
+ position += AbortedTxn.TotalSize
+ nextEntry
+ } catch {
+ case e: IOException =>
+ // We received an unexpected error reading from the index file. We propagate this as an
+ // UNKNOWN error to the consumer, which will cause it to retry the fetch.
+ throw new KafkaException(s"Failed to read from the transaction index $file", e)
+ }
+ }
+ }
+ }
+ }
+
+ def allAbortedTxns: List[AbortedTxn] = {
+ iterator(() => ByteBuffer.allocate(AbortedTxn.TotalSize)).map(_._1).toList
+ }
+
+ /**
+ * Collect all aborted transactions which overlap with a given fetch range.
+ *
+ * @param fetchOffset Inclusive first offset of the fetch range
+ * @param upperBoundOffset Exclusive last offset in the fetch range
+ * @return An object containing the aborted transactions and whether the search needs to continue
+ * into the next log segment.
+ */
+ def collectAbortedTxns(fetchOffset: Long, upperBoundOffset: Long): TxnIndexSearchResult = {
+ val abortedTransactions = ListBuffer.empty[AbortedTransaction]
+ val buffer = ByteBuffer.allocate(AbortedTxn.TotalSize)
+ for ((abortedTxn, _) <- iterator(() => buffer)) {
+ if (abortedTxn.lastOffset >= fetchOffset && abortedTxn.firstOffset < upperBoundOffset)
+ abortedTransactions += abortedTxn.asAbortedTransaction
+
+ if (abortedTxn.lastStableOffset >= upperBoundOffset)
+ return TxnIndexSearchResult(abortedTransactions.toList, isComplete = true)
+ }
+ TxnIndexSearchResult(abortedTransactions.toList, isComplete = false)
+ }
+
+ def sanityCheck(): Unit = {
+ val buffer = ByteBuffer.allocate(AbortedTxn.TotalSize)
+ for ((abortedTxn, _) <- iterator(() => buffer)) {
+ require(abortedTxn.lastOffset >= startOffset)
+ }
+ }
+
+}
+
+private[log] object AbortedTxn {
+ val VersionOffset = 0
+ val VersionSize = 2
+ val ProducerIdOffset = VersionOffset + VersionSize
+ val ProducerIdSize = 8
+ val FirstOffsetOffset = ProducerIdOffset + ProducerIdSize
+ val FirstOffsetSize = 8
+ val LastOffsetOffset = FirstOffsetOffset + FirstOffsetSize
+ val LastOffsetSize = 8
+ val LastStableOffsetOffset = LastOffsetOffset + LastOffsetSize
+ val LastStableOffsetSize = 8
+ val TotalSize = LastStableOffsetOffset + LastStableOffsetSize
+
+ val CurrentVersion: Short = 0
+}
+
+private[log] class AbortedTxn(val buffer: ByteBuffer) {
+ import AbortedTxn._
+
+ def this(producerId: Long,
+ firstOffset: Long,
+ lastOffset: Long,
+ lastStableOffset: Long) = {
+ this(ByteBuffer.allocate(AbortedTxn.TotalSize))
+ buffer.putShort(CurrentVersion)
+ buffer.putLong(producerId)
+ buffer.putLong(firstOffset)
+ buffer.putLong(lastOffset)
+ buffer.putLong(lastStableOffset)
+ buffer.flip()
+ }
+
+ def this(completedTxn: CompletedTxn, lastStableOffset: Long) =
+ this(completedTxn.producerId, completedTxn.firstOffset, completedTxn.lastOffset, lastStableOffset)
+
+ def version: Short = buffer.get(VersionOffset)
+
+ def producerId: Long = buffer.getLong(ProducerIdOffset)
+
+ def firstOffset: Long = buffer.getLong(FirstOffsetOffset)
+
+ def lastOffset: Long = buffer.getLong(LastOffsetOffset)
+
+ def lastStableOffset: Long = buffer.getLong(LastStableOffsetOffset)
+
+ def asAbortedTransaction: AbortedTransaction = new AbortedTransaction(producerId, firstOffset)
+
+ override def toString: String =
+ s"AbortedTxn(version=$version, producerId=$producerId, firstOffset=$firstOffset, " +
+ s"lastOffset=$lastOffset, lastStableOffset=$lastStableOffset)"
+
+ override def equals(any: Any): Boolean = {
+ any match {
+ case that: AbortedTxn => this.buffer.equals(that.buffer)
+ case _ => false
+ }
+ }
+
+ override def hashCode(): Int = buffer.hashCode
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/server/DelayedFetch.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala
index cbee78a..8c4731a 100644
--- a/core/src/main/scala/kafka/server/DelayedFetch.scala
+++ b/core/src/main/scala/kafka/server/DelayedFetch.scala
@@ -23,6 +23,7 @@ import kafka.metrics.KafkaMetricsGroup
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.errors.{NotLeaderForPartitionException, UnknownTopicOrPartitionException}
import org.apache.kafka.common.requests.FetchRequest.PartitionData
+import org.apache.kafka.common.requests.IsolationLevel
import scala.collection._
@@ -45,9 +46,11 @@ case class FetchMetadata(fetchMinBytes: Int,
fetchPartitionStatus: Seq[(TopicPartition, FetchPartitionStatus)]) {
override def toString = "[minBytes: " + fetchMinBytes + ", " +
- "onlyLeader:" + fetchOnlyLeader + ", "
- "onlyCommitted: " + fetchOnlyCommitted + ", "
- "partitionStatus: " + fetchPartitionStatus + "]"
+ "maxBytes:" + fetchMaxBytes + ", " +
+ "onlyLeader:" + fetchOnlyLeader + ", " +
+ "onlyCommitted: " + fetchOnlyCommitted + ", " +
+ "replicaId: " + replicaId + ", " +
+ "partitionStatus: " + fetchPartitionStatus + "]"
}
/**
* A delayed fetch operation that can be created by the replica manager and watched
@@ -57,6 +60,7 @@ class DelayedFetch(delayMs: Long,
fetchMetadata: FetchMetadata,
replicaManager: ReplicaManager,
quota: ReplicaQuota,
+ isolationLevel: IsolationLevel,
responseCallback: Seq[(TopicPartition, FetchPartitionData)] => Unit)
extends DelayedOperation(delayMs) {
@@ -80,7 +84,9 @@ class DelayedFetch(delayMs: Long,
if (fetchOffset != LogOffsetMetadata.UnknownOffsetMetadata) {
val replica = replicaManager.getLeaderReplicaIfLocal(topicPartition)
val endOffset =
- if (fetchMetadata.fetchOnlyCommitted)
+ if (isolationLevel == IsolationLevel.READ_COMMITTED)
+ replica.lastStableOffset
+ else if (fetchMetadata.fetchOnlyCommitted)
replica.highWatermark
else
replica.logEndOffset
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/server/FetchDataInfo.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/FetchDataInfo.scala b/core/src/main/scala/kafka/server/FetchDataInfo.scala
index acfb5b0..cbd54c0 100644
--- a/core/src/main/scala/kafka/server/FetchDataInfo.scala
+++ b/core/src/main/scala/kafka/server/FetchDataInfo.scala
@@ -18,7 +18,9 @@
package kafka.server
import org.apache.kafka.common.record.Records
+import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction
case class FetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata,
records: Records,
- firstEntryIncomplete: Boolean = false)
+ firstEntryIncomplete: Boolean = false,
+ abortedTransactions: Option[List[AbortedTransaction]] = None)
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/server/KafkaApis.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala
index 3d821f7..fbd74ac 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -21,6 +21,8 @@ import java.nio.ByteBuffer
import java.lang.{Long => JLong}
import java.util.{Collections, Properties}
import java.util
+import java.util.concurrent.ConcurrentHashMap
+import java.util.concurrent.atomic.AtomicInteger
import kafka.admin.{AdminUtils, RackAwareMode}
import kafka.api.{ControlledShutdownRequest, ControlledShutdownResponse}
@@ -41,7 +43,7 @@ import org.apache.kafka.common.internals.FatalExitError
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.{ApiKeys, Errors, Protocol}
-import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, TimestampType}
+import org.apache.kafka.common.record.{ControlRecordType, EndTransactionMarker, MemoryRecords, RecordBatch}
import org.apache.kafka.common.requests._
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.common.utils.{Time, Utils}
@@ -436,6 +438,7 @@ class KafkaApis(val requestChannel: RequestChannel,
produceRequest.timeout.toLong,
produceRequest.acks,
internalTopicsAllowed,
+ isFromClient = true,
authorizedRequestInfo,
sendResponseCallback)
@@ -495,8 +498,9 @@ class KafkaApis(val requestChannel: RequestChannel,
case _ => data
}
+ val abortedTransactions = convertedData.abortedTransactions.map(_.asJava).orNull
tp -> new FetchResponse.PartitionData(convertedData.error, convertedData.hw, FetchResponse.INVALID_LAST_STABLE_OFFSET,
- convertedData.logStartOffset, null, convertedData.records)
+ convertedData.logStartOffset, abortedTransactions, convertedData.records)
}
}
@@ -560,7 +564,8 @@ class KafkaApis(val requestChannel: RequestChannel,
versionId <= 2,
authorizedRequestInfo,
replicationQuota(fetchRequest),
- sendResponseCallback)
+ sendResponseCallback,
+ fetchRequest.isolationLevel)
}
}
@@ -589,7 +594,7 @@ class KafkaApis(val requestChannel: RequestChannel,
if (version == 0)
handleListOffsetRequestV0(request)
else
- handleListOffsetRequestV1(request)
+ handleListOffsetRequestV1AndAbove(request)
def createResponse(throttleTimeMs: Int): AbstractResponse = new ListOffsetResponse(throttleTimeMs, mergedResponseMap.asJava)
sendResponseMaybeThrottle(request, createResponse)
@@ -646,7 +651,7 @@ class KafkaApis(val requestChannel: RequestChannel,
responseMap ++ unauthorizedResponseStatus
}
- private def handleListOffsetRequestV1(request : RequestChannel.Request): Map[TopicPartition, ListOffsetResponse.PartitionData] = {
+ private def handleListOffsetRequestV1AndAbove(request : RequestChannel.Request): Map[TopicPartition, ListOffsetResponse.PartitionData] = {
val correlationId = request.header.correlationId
val clientId = request.header.clientId
val offsetRequest = request.body[ListOffsetRequest]
@@ -679,9 +684,13 @@ class KafkaApis(val requestChannel: RequestChannel,
replicaManager.getReplicaOrException(topicPartition)
val found = {
- if (fromConsumer && timestamp == ListOffsetRequest.LATEST_TIMESTAMP)
- TimestampOffset(RecordBatch.NO_TIMESTAMP, localReplica.highWatermark.messageOffset)
- else {
+ if (fromConsumer && timestamp == ListOffsetRequest.LATEST_TIMESTAMP) {
+ val lastFetchableOffset = offsetRequest.isolationLevel match {
+ case IsolationLevel.READ_COMMITTED => localReplica.lastStableOffset.messageOffset
+ case IsolationLevel.READ_UNCOMMITTED => localReplica.highWatermark.messageOffset
+ }
+ TimestampOffset(RecordBatch.NO_TIMESTAMP, lastFetchableOffset)
+ } else {
def allowed(timestampOffset: TimestampOffset): Boolean =
!fromConsumer || timestampOffset.offset <= localReplica.highWatermark.messageOffset
@@ -1415,9 +1424,45 @@ class KafkaApis(val requestChannel: RequestChannel,
def handleWriteTxnMarkersRequest(request: RequestChannel.Request): Unit = {
authorizeClusterAction(request)
- val emptyResponse = new java.util.HashMap[java.lang.Long, java.util.Map[TopicPartition, Errors]]()
- val responseBody = new WriteTxnMarkersResponse(emptyResponse)
- sendResponseExemptThrottle(request, new RequestChannel.Response(request, responseBody))
+ val writeTxnMarkersRequest = request.body[WriteTxnMarkersRequest]
+ val errors = new ConcurrentHashMap[java.lang.Long, java.util.Map[TopicPartition, Errors]]()
+ val markers = writeTxnMarkersRequest.markers
+ val numAppends = new AtomicInteger(markers.size)
+
+ if (numAppends.get == 0) {
+ sendResponseExemptThrottle(request, new RequestChannel.Response(request, new WriteTxnMarkersResponse(errors)))
+ return
+ }
+
+ def sendResponseCallback(pid: Long)(responseStatus: Map[TopicPartition, PartitionResponse]): Unit = {
+ errors.put(pid, responseStatus.mapValues(_.error).asJava)
+ if (numAppends.decrementAndGet() == 0)
+ sendResponseExemptThrottle(request, new RequestChannel.Response(request, new WriteTxnMarkersResponse(errors)))
+ }
+
+ // TODO: The current append API makes doing separate writes per producerId a little easier, but it would
+ // be nice to have only one append to the log. This requires pushing the building of the control records
+ // into Log so that we only append those having a valid producer epoch, and exposing a new appendControlRecord
+ // API in ReplicaManager. For now, we've done the simpler approach
+ for (marker <- markers.asScala) {
+ val producerId = marker.producerId
+ val controlRecords = marker.partitions.asScala.map { partition =>
+ val controlRecordType = marker.transactionResult match {
+ case TransactionResult.COMMIT => ControlRecordType.COMMIT
+ case TransactionResult.ABORT => ControlRecordType.ABORT
+ }
+ val endTxnMarker = new EndTransactionMarker(controlRecordType, marker.coordinatorEpoch)
+ partition -> MemoryRecords.withEndTransactionMarker(producerId, marker.producerEpoch, endTxnMarker)
+ }.toMap
+
+ replicaManager.appendRecords(
+ timeout = config.requestTimeoutMs.toLong,
+ requiredAcks = -1,
+ internalTopicsAllowed = true,
+ isFromClient = false,
+ entriesPerPartition = controlRecords,
+ sendResponseCallback(producerId))
+ }
}
def handleAddPartitionToTxnRequest(request: RequestChannel.Request): Unit = {
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/server/LogOffsetMetadata.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/LogOffsetMetadata.scala b/core/src/main/scala/kafka/server/LogOffsetMetadata.scala
index 05e9842..edc010e 100644
--- a/core/src/main/scala/kafka/server/LogOffsetMetadata.scala
+++ b/core/src/main/scala/kafka/server/LogOffsetMetadata.scala
@@ -44,7 +44,7 @@ case class LogOffsetMetadata(messageOffset: Long,
// check if this offset is already on an older segment compared with the given offset
def onOlderSegment(that: LogOffsetMetadata): Boolean = {
- if (messageOffsetOnly())
+ if (messageOffsetOnly)
throw new KafkaException(s"$this cannot compare its segment info with $that since it only has message offset info")
this.segmentBaseOffset < that.segmentBaseOffset
@@ -52,7 +52,7 @@ case class LogOffsetMetadata(messageOffset: Long,
// check if this offset is on the same segment with the given offset
def onSameSegment(that: LogOffsetMetadata): Boolean = {
- if (messageOffsetOnly())
+ if (messageOffsetOnly)
throw new KafkaException(s"$this cannot compare its segment info with $that since it only has message offset info")
this.segmentBaseOffset == that.segmentBaseOffset
@@ -68,14 +68,14 @@ case class LogOffsetMetadata(messageOffset: Long,
def positionDiff(that: LogOffsetMetadata): Int = {
if(!onSameSegment(that))
throw new KafkaException(s"$this cannot compare its segment position with $that since they are not on the same segment")
- if(messageOffsetOnly())
+ if(messageOffsetOnly)
throw new KafkaException(s"$this cannot compare its segment position with $that since it only has message offset info")
this.relativePositionInSegment - that.relativePositionInSegment
}
// decide if the offset metadata only contains message offset info
- def messageOffsetOnly(): Boolean = {
+ def messageOffsetOnly: Boolean = {
segmentBaseOffset == LogOffsetMetadata.UnknownSegBaseOffset && relativePositionInSegment == LogOffsetMetadata.UnknownFilePosition
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/server/ReplicaManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala
index de670e8..663ab1e 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -40,6 +40,7 @@ import org.apache.kafka.common.requests._
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.common.utils.Time
import org.apache.kafka.common.requests.FetchRequest.PartitionData
+import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction
import scala.collection._
import scala.collection.JavaConverters._
@@ -95,7 +96,8 @@ case class LogReadResult(info: FetchDataInfo,
}
-case class FetchPartitionData(error: Errors = Errors.NONE, hw: Long = -1L, logStartOffset: Long, records: Records)
+case class FetchPartitionData(error: Errors = Errors.NONE, hw: Long = -1L, logStartOffset: Long, records: Records,
+ abortedTransactions: Option[List[AbortedTransaction]] = None)
object LogReadResult {
val UnknownLogReadResult = LogReadResult(info = FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY),
@@ -334,12 +336,14 @@ class ReplicaManager(val config: KafkaConfig,
def appendRecords(timeout: Long,
requiredAcks: Short,
internalTopicsAllowed: Boolean,
+ isFromClient: Boolean,
entriesPerPartition: Map[TopicPartition, MemoryRecords],
responseCallback: Map[TopicPartition, PartitionResponse] => Unit) {
if (isValidRequiredAcks(requiredAcks)) {
val sTime = time.milliseconds
- val localProduceResults = appendToLocalLog(internalTopicsAllowed, entriesPerPartition, requiredAcks)
+ val localProduceResults = appendToLocalLog(internalTopicsAllowed = internalTopicsAllowed,
+ isFromClient = isFromClient, entriesPerPartition, requiredAcks)
debug("Produce to local log in %d ms".format(time.milliseconds - sTime))
val produceStatus = localProduceResults.map { case (topicPartition, result) =>
@@ -493,6 +497,7 @@ class ReplicaManager(val config: KafkaConfig,
* Append the messages to the local replica logs
*/
private def appendToLocalLog(internalTopicsAllowed: Boolean,
+ isFromClient: Boolean,
entriesPerPartition: Map[TopicPartition, MemoryRecords],
requiredAcks: Short): Map[TopicPartition, LogAppendResult] = {
trace("Append [%s] to local log ".format(entriesPerPartition))
@@ -510,7 +515,7 @@ class ReplicaManager(val config: KafkaConfig,
val partitionOpt = getPartition(topicPartition)
val info = partitionOpt match {
case Some(partition) =>
- partition.appendRecordsToLeader(records, requiredAcks)
+ partition.appendRecordsToLeader(records, isFromClient, requiredAcks)
case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d"
.format(topicPartition, localBrokerId))
@@ -566,7 +571,8 @@ class ReplicaManager(val config: KafkaConfig,
hardMaxBytesLimit: Boolean,
fetchInfos: Seq[(TopicPartition, PartitionData)],
quota: ReplicaQuota = UnboundedQuota,
- responseCallback: Seq[(TopicPartition, FetchPartitionData)] => Unit) {
+ responseCallback: Seq[(TopicPartition, FetchPartitionData)] => Unit,
+ isolationLevel: IsolationLevel) {
val isFromFollower = replicaId >= 0
val fetchOnlyFromLeader: Boolean = replicaId != Request.DebuggingConsumerId
val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(replicaId)
@@ -579,7 +585,8 @@ class ReplicaManager(val config: KafkaConfig,
fetchMaxBytes = fetchMaxBytes,
hardMaxBytesLimit = hardMaxBytesLimit,
readPartitionInfo = fetchInfos,
- quota = quota)
+ quota = quota,
+ isolationLevel = isolationLevel)
// if the fetch comes from the follower,
// update its corresponding log end offset
@@ -598,7 +605,8 @@ class ReplicaManager(val config: KafkaConfig,
// 4) some error happens while reading data
if (timeout <= 0 || fetchInfos.isEmpty || bytesReadable >= fetchMinBytes || errorReadingData) {
val fetchPartitionData = logReadResults.map { case (tp, result) =>
- tp -> FetchPartitionData(result.error, result.hw, result.leaderLogStartOffset, result.info.records)
+ tp -> FetchPartitionData(result.error, result.hw, result.leaderLogStartOffset, result.info.records,
+ result.info.abortedTransactions)
}
responseCallback(fetchPartitionData)
} else {
@@ -611,7 +619,7 @@ class ReplicaManager(val config: KafkaConfig,
}
val fetchMetadata = FetchMetadata(fetchMinBytes, fetchMaxBytes, hardMaxBytesLimit, fetchOnlyFromLeader,
fetchOnlyCommitted, isFromFollower, replicaId, fetchPartitionStatus)
- val delayedFetch = new DelayedFetch(timeout, fetchMetadata, this, quota, responseCallback)
+ val delayedFetch = new DelayedFetch(timeout, fetchMetadata, this, quota, isolationLevel, responseCallback)
// create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => new TopicPartitionOperationKey(tp) }
@@ -632,7 +640,8 @@ class ReplicaManager(val config: KafkaConfig,
fetchMaxBytes: Int,
hardMaxBytesLimit: Boolean,
readPartitionInfo: Seq[(TopicPartition, PartitionData)],
- quota: ReplicaQuota): Seq[(TopicPartition, LogReadResult)] = {
+ quota: ReplicaQuota,
+ isolationLevel: IsolationLevel = IsolationLevel.READ_UNCOMMITTED): Seq[(TopicPartition, LogReadResult)] = {
def read(tp: TopicPartition, fetchInfo: PartitionData, limitBytes: Int, minOneMessage: Boolean): LogReadResult = {
val offset = fetchInfo.fetchOffset
@@ -654,7 +663,9 @@ class ReplicaManager(val config: KafkaConfig,
getReplicaOrException(tp)
// decide whether to only fetch committed data (i.e. messages below high watermark)
- val maxOffsetOpt = if (readOnlyCommitted)
+ val maxOffsetOpt = if (isolationLevel == IsolationLevel.READ_COMMITTED)
+ Some(localReplica.lastStableOffset.messageOffset)
+ else if (readOnlyCommitted)
Some(localReplica.highWatermark.messageOffset)
else
None
@@ -674,7 +685,7 @@ class ReplicaManager(val config: KafkaConfig,
val adjustedFetchSize = math.min(partitionFetchSize, limitBytes)
// Try the read first, this tells us whether we need all of adjustedFetchSize for this partition
- val fetch = log.read(offset, adjustedFetchSize, maxOffsetOpt, minOneMessage)
+ val fetch = log.read(offset, adjustedFetchSize, maxOffsetOpt, minOneMessage, isolationLevel)
// If the partition is being throttled, simply return an empty set.
if (shouldLeaderThrottle(quota, tp, replicaId))
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/tools/DumpLogSegments.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala
index 7a5f671..0b0ad7b 100755
--- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala
+++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala
@@ -105,6 +105,8 @@ object DumpLogSegments {
dumpTimeIndex(file, indexSanityOnly, verifyOnly, timeIndexDumpErrors, maxMessageSize)
case Log.PidSnapshotFileSuffix =>
dumpPidSnapshot(file)
+ case Log.TxnIndexFileSuffix =>
+ dumpTxnIndex(file)
case _ =>
System.err.println(s"Ignoring unknown file $file")
}
@@ -131,11 +133,20 @@ object DumpLogSegments {
}
}
+ private def dumpTxnIndex(file: File): Unit = {
+ val index = new TransactionIndex(Log.offsetFromFilename(file.getName), file)
+ for (abortedTxn <- index.allAbortedTxns) {
+ println(s"version: ${abortedTxn.version} pid: ${abortedTxn.producerId} firstOffset: ${abortedTxn.firstOffset} " +
+ s"lastOffset: ${abortedTxn.lastOffset} lastStableOffset: ${abortedTxn.lastStableOffset}")
+ }
+ }
+
private def dumpPidSnapshot(file: File): Unit = {
try {
- ProducerIdMapping.readSnapshot(file).foreach { case (pid, entry) =>
- println(s"pid: $pid epoch: ${entry.epoch} lastSequence: ${entry.lastSeq} lastOffset: ${entry.lastOffset} " +
- s"offsetDelta: ${entry.offsetDelta} lastTimestamp: ${entry.timestamp}")
+ ProducerStateManager.readSnapshot(file).foreach { entry=>
+ println(s"producerId: ${entry.producerId} producerEpoch: ${entry.producerEpoch} lastSequence: ${entry.lastSeq} " +
+ s"lastOffset: ${entry.lastOffset} offsetDelta: ${entry.offsetDelta} lastTimestamp: ${entry.timestamp} " +
+ s"coordinatorEpoch: ${entry.coordinatorEpoch} currentTxnFirstOffset: ${entry.currentTxnFirstOffset}")
}
} catch {
case e: CorruptSnapshotException =>
@@ -349,9 +360,15 @@ object DumpLogSegments {
" headerKeys: " + record.headers.map(_.key).mkString("[", ",", "]"))
}
- if (record.isControlRecord) {
- val controlType = ControlRecordType.parse(record.key)
- print(s" controlType: $controlType")
+ if (batch.isControlBatch) {
+ val controlTypeId = ControlRecordType.parseTypeId(record.key)
+ ControlRecordType.fromTypeId(controlTypeId) match {
+ case ControlRecordType.ABORT | ControlRecordType.COMMIT =>
+ val endTxnMarker = EndTransactionMarker.deserialize(record)
+ print(s" endTxnMarker: ${endTxnMarker.controlType} coordinatorEpoch: ${endTxnMarker.coordinatorEpoch}")
+ case controlType =>
+ print(s" controlType: $controlType($controlTypeId)")
+ }
} else if (printContents) {
val (key, payload) = parser.parse(record)
key.foreach(key => print(s" key: $key"))
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
index 757e216..4277d26 100644
--- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
@@ -199,7 +199,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
}
private def createListOffsetsRequest = {
- requests.ListOffsetRequest.Builder.forConsumer(false).setTargetTimes(
+ requests.ListOffsetRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED).setTargetTimes(
Map(tp -> (0L: java.lang.Long)).asJava).
build()
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorResponseTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorResponseTest.scala
index b4aa56f..2dfbf48 100644
--- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorResponseTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorResponseTest.scala
@@ -1124,7 +1124,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
EasyMock.anyShort(),
- EasyMock.anyBoolean(),
+ internalTopicsAllowed = EasyMock.eq(true),
+ isFromClient = EasyMock.eq(false),
EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
override def answer = capturedArgument.getValue.apply(
@@ -1205,7 +1206,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
EasyMock.anyShort(),
- EasyMock.anyBoolean(),
+ internalTopicsAllowed = EasyMock.eq(true),
+ isFromClient = EasyMock.eq(false),
EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
override def answer = capturedArgument.getValue.apply(
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
index 387d4b3..9053e0a 100644
--- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
@@ -28,7 +28,7 @@ import kafka.utils.{KafkaScheduler, MockTime, TestUtils, ZkUtils}
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record._
-import org.apache.kafka.common.requests.OffsetFetchResponse
+import org.apache.kafka.common.requests.{IsolationLevel, OffsetFetchResponse}
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.easymock.{Capture, EasyMock, IAnswer}
import org.junit.Assert.{assertEquals, assertFalse, assertTrue}
@@ -509,7 +509,8 @@ class GroupMetadataManagerTest {
time.sleep(2)
EasyMock.reset(partition)
- EasyMock.expect(partition.appendRecordsToLeader(EasyMock.anyObject(classOf[MemoryRecords]), EasyMock.anyInt()))
+ EasyMock.expect(partition.appendRecordsToLeader(EasyMock.anyObject(classOf[MemoryRecords]),
+ isFromClient = EasyMock.eq(false), requiredAcks = EasyMock.anyInt()))
.andReturn(LogAppendInfo.UnknownLogAppendInfo)
EasyMock.replay(partition)
@@ -541,7 +542,8 @@ class GroupMetadataManagerTest {
EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andStubReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE))
EasyMock.expect(replicaManager.getPartition(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId))).andStubReturn(Some(partition))
- EasyMock.expect(partition.appendRecordsToLeader(EasyMock.capture(recordsCapture), EasyMock.anyInt()))
+ EasyMock.expect(partition.appendRecordsToLeader(EasyMock.capture(recordsCapture),
+ isFromClient = EasyMock.eq(false), requiredAcks = EasyMock.anyInt()))
.andReturn(LogAppendInfo.UnknownLogAppendInfo)
EasyMock.replay(replicaManager, partition)
@@ -588,7 +590,8 @@ class GroupMetadataManagerTest {
EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andStubReturn(Some(RecordBatch.CURRENT_MAGIC_VALUE))
EasyMock.expect(replicaManager.getPartition(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId))).andStubReturn(Some(partition))
- EasyMock.expect(partition.appendRecordsToLeader(EasyMock.capture(recordsCapture), EasyMock.anyInt()))
+ EasyMock.expect(partition.appendRecordsToLeader(EasyMock.capture(recordsCapture),
+ isFromClient = EasyMock.eq(false), requiredAcks = EasyMock.anyInt()))
.andReturn(LogAppendInfo.UnknownLogAppendInfo)
EasyMock.replay(replicaManager, partition)
@@ -664,7 +667,8 @@ class GroupMetadataManagerTest {
EasyMock.reset(partition)
val recordsCapture: Capture[MemoryRecords] = EasyMock.newCapture()
- EasyMock.expect(partition.appendRecordsToLeader(EasyMock.capture(recordsCapture), EasyMock.anyInt()))
+ EasyMock.expect(partition.appendRecordsToLeader(EasyMock.capture(recordsCapture),
+ isFromClient = EasyMock.eq(false), requiredAcks = EasyMock.anyInt()))
.andReturn(LogAppendInfo.UnknownLogAppendInfo)
EasyMock.replay(partition)
@@ -738,7 +742,8 @@ class GroupMetadataManagerTest {
// expect the offset tombstone
EasyMock.reset(partition)
- EasyMock.expect(partition.appendRecordsToLeader(EasyMock.anyObject(classOf[MemoryRecords]), EasyMock.anyInt()))
+ EasyMock.expect(partition.appendRecordsToLeader(EasyMock.anyObject(classOf[MemoryRecords]),
+ isFromClient = EasyMock.eq(false), requiredAcks = EasyMock.anyInt()))
.andReturn(LogAppendInfo.UnknownLogAppendInfo)
EasyMock.replay(partition)
@@ -758,7 +763,8 @@ class GroupMetadataManagerTest {
val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
EasyMock.anyShort(),
- EasyMock.anyBoolean(),
+ internalTopicsAllowed = EasyMock.eq(true),
+ isFromClient = EasyMock.eq(false),
EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
override def answer = capturedArgument.getValue.apply(
@@ -794,7 +800,7 @@ class GroupMetadataManagerTest {
EasyMock.expect(replicaManager.getLog(groupMetadataTopicPartition)).andStubReturn(Some(logMock))
EasyMock.expect(logMock.logStartOffset).andStubReturn(startOffset)
EasyMock.expect(replicaManager.getLogEndOffset(groupMetadataTopicPartition)).andStubReturn(Some(endOffset))
- EasyMock.expect(logMock.read(EasyMock.eq(startOffset), EasyMock.anyInt(), EasyMock.eq(None), EasyMock.eq(true)))
+ EasyMock.expect(logMock.read(EasyMock.eq(startOffset), EasyMock.anyInt(), EasyMock.eq(None), EasyMock.eq(true), EasyMock.eq(IsolationLevel.READ_UNCOMMITTED)))
.andReturn(FetchDataInfo(LogOffsetMetadata(startOffset), fileRecordsMock))
EasyMock.expect(fileRecordsMock.readInto(EasyMock.anyObject(classOf[ByteBuffer]), EasyMock.anyInt()))
.andReturn(records.buffer)
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala
index 94dc12b..09a89dd 100644
--- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala
@@ -27,6 +27,7 @@ import kafka.utils.TestUtils.fail
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record._
+import org.apache.kafka.common.requests.IsolationLevel
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.common.utils.MockTime
import org.junit.Assert.{assertEquals, assertFalse, assertTrue}
@@ -349,7 +350,8 @@ class TransactionStateManagerTest {
EasyMock.expect(replicaManager.getLogEndOffset(topicPartition)).andStubReturn(Some(endOffset))
EasyMock.expect(logMock.logStartOffset).andStubReturn(startOffset)
- EasyMock.expect(logMock.read(EasyMock.eq(startOffset), EasyMock.anyInt(), EasyMock.eq(None), EasyMock.eq(true)))
+ EasyMock.expect(logMock.read(EasyMock.eq(startOffset), EasyMock.anyInt(), EasyMock.eq(None), EasyMock.eq(true),
+ EasyMock.eq(IsolationLevel.READ_UNCOMMITTED)))
.andReturn(FetchDataInfo(LogOffsetMetadata(startOffset), fileRecordsMock))
EasyMock.expect(fileRecordsMock.readInto(EasyMock.anyObject(classOf[ByteBuffer]), EasyMock.anyInt()))
.andReturn(records.buffer)
@@ -363,7 +365,8 @@ class TransactionStateManagerTest {
val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
EasyMock.anyShort(),
- EasyMock.anyBoolean(),
+ internalTopicsAllowed = EasyMock.eq(true),
+ isFromClient = EasyMock.eq(false),
EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
EasyMock.capture(capturedArgument)))
.andAnswer(new IAnswer[Unit] {
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala
index cfd66de..a42ae22 100755
--- a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala
+++ b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala
@@ -57,7 +57,7 @@ class BrokerCompressionTest(messageCompression: String, brokerCompression: Strin
/* append two messages */
log.appendAsLeader(MemoryRecords.withRecords(CompressionType.forId(messageCompressionCode.codec), 0,
- new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes)), leaderEpoch = 0)
+ new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes)), leaderEpoch = 0)
def readBatch(offset: Int) = log.read(offset, 4096).records.batches.iterator.next()
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
index 19a97bc..8a119c2 100755
--- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
@@ -259,7 +259,7 @@ class LogCleanerIntegrationTest(compressionCodec: String) extends AbstractLogCle
for(_ <- 0 until numDups; key <- startKey until (startKey + numKeys)) yield {
val value = counter.toString
val appendInfo = log.appendAsLeader(TestUtils.singletonRecords(value = value.toString.getBytes, codec = codec,
- key = key.toString.getBytes, magicValue = magicValue), leaderEpoch = 0)
+ key = key.toString.getBytes, magicValue = magicValue), leaderEpoch = 0)
counter += 1
(key, value, appendInfo.firstOffset)
}
[4/6] kafka git commit: KAFKA-5121;
Implement transaction index for KIP-98
Posted by jg...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/log/Log.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala
index b7f340f..5722a43 100644
--- a/core/src/main/scala/kafka/log/Log.scala
+++ b/core/src/main/scala/kafka/log/Log.scala
@@ -30,9 +30,10 @@ import kafka.server.{BrokerTopicStats, FetchDataInfo, LogOffsetMetadata}
import kafka.utils._
import org.apache.kafka.common.errors.{CorruptRecordException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException, UnsupportedForMessageFormatException}
import org.apache.kafka.common.record._
-import org.apache.kafka.common.requests.ListOffsetRequest
+import org.apache.kafka.common.requests.{IsolationLevel, ListOffsetRequest}
import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
import scala.collection.{Seq, mutable}
import com.yammer.metrics.core.Gauge
import org.apache.kafka.common.utils.{Time, Utils}
@@ -40,10 +41,13 @@ import kafka.message.{BrokerCompressionCodec, CompressionCodec, NoCompressionCod
import kafka.server.checkpoints.{LeaderEpochCheckpointFile, LeaderEpochFile}
import kafka.server.epoch.{LeaderEpochCache, LeaderEpochFileCache}
import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction
+import java.util.Map.{Entry => JEntry}
+import java.lang.{Long => JLong}
object LogAppendInfo {
val UnknownLogAppendInfo = LogAppendInfo(-1, -1, RecordBatch.NO_TIMESTAMP, -1L, RecordBatch.NO_TIMESTAMP,
- NoCompressionCodec, NoCompressionCodec, -1, -1, offsetsMonotonic = false, Map.empty[Long, ProducerAppendInfo], false)
+ NoCompressionCodec, NoCompressionCodec, -1, -1, offsetsMonotonic = false)
}
/**
@@ -59,9 +63,6 @@ object LogAppendInfo {
* @param shallowCount The number of shallow messages
* @param validBytes The number of valid bytes
* @param offsetsMonotonic Are the offsets in this message set monotonically increasing
- * @param producerAppendInfos A map from a Pid to a ProducerAppendInfo, which is used to validate each Record in a
- * RecordBatch and keep track of metadata across Records in a RecordBatch.
- * @param isDuplicate Indicates whether the message set is a duplicate of a message at the tail of the log.
*/
case class LogAppendInfo(var firstOffset: Long,
var lastOffset: Long,
@@ -72,9 +73,19 @@ case class LogAppendInfo(var firstOffset: Long,
targetCodec: CompressionCodec,
shallowCount: Int,
validBytes: Int,
- offsetsMonotonic: Boolean,
- producerAppendInfos: Map[Long, ProducerAppendInfo],
- isDuplicate: Boolean = false)
+ offsetsMonotonic: Boolean)
+
+/**
+ * A class used to hold useful metadata about a completed transaction. This is used to build
+ * the transaction index after appending to the log.
+ *
+ * @param producerId The ID of the producer
+ * @param firstOffset The first offset (inclusive) of the transaction
+ * @param lastOffset The last offset (inclusive) of the transaction. This is always the offset of the
+ * COMMIT/ABORT control record which indicates the transaction's completion.
+ * @param isAborted Whether or not the transaction was aborted
+ */
+case class CompletedTxn(producerId: Long, firstOffset: Long, lastOffset: Long, isAborted: Boolean)
/**
* An append-only log for storing messages.
@@ -111,8 +122,7 @@ class Log(@volatile var dir: File,
scheduler: Scheduler,
time: Time = Time.SYSTEM,
val maxPidExpirationMs: Int = 60 * 60 * 1000,
- val pidExpirationCheckIntervalMs: Int = 10 * 60 * 1000,
- val pidSnapshotIntervalMs: Int = 60 * 1000) extends Logging with KafkaMetricsGroup {
+ val pidExpirationCheckIntervalMs: Int = 10 * 60 * 1000) extends Logging with KafkaMetricsGroup {
import kafka.log.Log._
@@ -133,8 +143,10 @@ class Log(@volatile var dir: File,
@volatile private var nextOffsetMetadata: LogOffsetMetadata = _
- /* Construct and load PID map */
- private val pidMap = new ProducerIdMapping(config, topicPartition, dir, maxPidExpirationMs)
+ /* The earliest offset which is part of an incomplete transaction. This is used to compute the LSO. */
+ @volatile var firstUnstableOffset: Option[LogOffsetMetadata] = None
+
+ private val producerStateManager = new ProducerStateManager(topicPartition, dir, maxPidExpirationMs)
/* the actual segments of the log */
private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
@@ -147,7 +159,7 @@ class Log(@volatile var dir: File,
loadSegments()
/* Calculate the offset of the next message */
- nextOffsetMetadata = new LogOffsetMetadata(activeSegment.nextOffset(), activeSegment.baseOffset,
+ nextOffsetMetadata = new LogOffsetMetadata(activeSegment.nextOffset, activeSegment.baseOffset,
activeSegment.size.toInt)
leaderEpochCache.clearLatest(nextOffsetMetadata.messageOffset)
@@ -157,7 +169,7 @@ class Log(@volatile var dir: File,
// The earliest leader epoch may not be flushed during a hard failure. Recover it here.
leaderEpochCache.clearEarliest(logStartOffset)
- buildAndRecoverPidMap(logEndOffset)
+ loadProducerState(logEndOffset)
info("Completed load of log %s with %d log segments, log start offset %d and log end offset %d in %d ms"
.format(name, segments.size(), logStartOffset, logEndOffset, time.milliseconds - startMs))
@@ -189,19 +201,12 @@ class Log(@volatile var dir: File,
},
tags)
- scheduler.schedule(name = "PeriodicPidExpirationCheck", fun = () => {
+ scheduler.schedule(name = "PeriodicProducerExpirationCheck", fun = () => {
lock synchronized {
- pidMap.removeExpiredPids(time.milliseconds)
+ producerStateManager.removeExpiredProducers(time.milliseconds)
}
}, period = pidExpirationCheckIntervalMs, unit = TimeUnit.MILLISECONDS)
- scheduler.schedule(name = "PeriodicPidSnapshotTask", fun = () => {
- lock synchronized {
- pidMap.maybeTakeSnapshot()
- }
- }, period = pidSnapshotIntervalMs, unit = TimeUnit.MILLISECONDS)
-
-
/** The name of this log */
def name = dir.getName()
@@ -212,13 +217,10 @@ class Log(@volatile var dir: File,
new LeaderEpochCheckpointFile(LeaderEpochFile.newFile(dir)))
}
- /* Load the log segments from the log files on disk */
- private def loadSegments() {
+ private def removeTempFilesAndCollectSwapFiles(): Set[File] = {
var swapFiles = Set[File]()
- // first do a pass through the files in the log directory and remove any temporary files
- // and find any interrupted swap operations
- for(file <- dir.listFiles if file.isFile) {
+ for (file <- dir.listFiles if file.isFile) {
if(!file.canRead)
throw new IOException("Could not read file " + file)
val filename = file.getName
@@ -229,48 +231,51 @@ class Log(@volatile var dir: File,
// we crashed in the middle of a swap operation, to recover:
// if a log, delete the .index file, complete the swap operation later
// if an index just delete it, it will be rebuilt
- val baseName = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
- if(baseName.getPath.endsWith(IndexFileSuffix)) {
+ val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, ""))
+ if (isIndexFile(baseFile)) {
Files.deleteIfExists(file.toPath)
- } else if(baseName.getPath.endsWith(LogFileSuffix)){
- // delete the index
- val index = new File(CoreUtils.replaceSuffix(baseName.getPath, LogFileSuffix, IndexFileSuffix))
- Files.deleteIfExists(index.toPath())
+ } else if (isLogFile(baseFile)) {
+ // delete the index files
+ val offset = offsetFromFilename(baseFile.getName)
+ Files.deleteIfExists(Log.offsetIndexFile(dir, offset).toPath)
+ Files.deleteIfExists(Log.timeIndexFile(dir, offset).toPath)
+ Files.deleteIfExists(Log.transactionIndexFile(dir, offset).toPath)
swapFiles += file
}
}
}
+ swapFiles
+ }
- // now do a second pass and load all the .log and all index files
- for(file <- dir.listFiles if file.isFile) {
+ private def loadSegmentFiles(): Unit = {
+ // load segments in ascending order because transactional data from one segment may depend on the
+ // segments that come before it
+ for (file <- dir.listFiles.sortBy(_.getName) if file.isFile) {
val filename = file.getName
- if(filename.endsWith(IndexFileSuffix) || filename.endsWith(TimeIndexFileSuffix)) {
+ if (isIndexFile(file)) {
// if it is an index file, make sure it has a corresponding .log file
- val logFile =
- if (filename.endsWith(TimeIndexFileSuffix))
- new File(file.getAbsolutePath.replace(TimeIndexFileSuffix, LogFileSuffix))
- else
- new File(file.getAbsolutePath.replace(IndexFileSuffix, LogFileSuffix))
-
- if(!logFile.exists) {
+ val offset = offsetFromFilename(filename)
+ val logFile = logFilename(dir, offset)
+ if (!logFile.exists) {
warn("Found an orphaned index file, %s, with no corresponding log file.".format(file.getAbsolutePath))
Files.deleteIfExists(file.toPath)
}
- } else if(filename.endsWith(LogFileSuffix)) {
- // if its a log file, load the corresponding log segment
+ } else if (isLogFile(file)) {
+ // if it's a log file, load the corresponding log segment
val startOffset = offsetFromFilename(filename)
- val indexFile = Log.indexFilename(dir, startOffset)
- val timeIndexFile = Log.timeIndexFilename(dir, startOffset)
+ val indexFile = Log.offsetIndexFile(dir, startOffset)
+ val timeIndexFile = Log.timeIndexFile(dir, startOffset)
+ val txnIndexFile = Log.transactionIndexFile(dir, startOffset)
val indexFileExists = indexFile.exists()
val timeIndexFileExists = timeIndexFile.exists()
val segment = new LogSegment(dir = dir,
- startOffset = startOffset,
- indexIntervalBytes = config.indexInterval,
- maxIndexSize = config.maxIndexSize,
- rollJitterMs = config.randomSegmentJitter,
- time = time,
- fileAlreadyExists = true)
+ startOffset = startOffset,
+ indexIntervalBytes = config.indexInterval,
+ maxIndexSize = config.maxIndexSize,
+ rollJitterMs = config.randomSegmentJitter,
+ time = time,
+ fileAlreadyExists = true)
if (indexFileExists) {
try {
@@ -279,25 +284,43 @@ class Log(@volatile var dir: File,
if (!timeIndexFileExists)
segment.timeIndex.resize(0)
segment.timeIndex.sanityCheck()
+ segment.txnIndex.sanityCheck()
} catch {
case e: java.lang.IllegalArgumentException =>
warn(s"Found a corrupted index file due to ${e.getMessage}}. deleting ${timeIndexFile.getAbsolutePath}, " +
- s"${indexFile.getAbsolutePath} and rebuilding index...")
+ s"${indexFile.getAbsolutePath}, and ${txnIndexFile.getAbsolutePath} and rebuilding index...")
Files.deleteIfExists(timeIndexFile.toPath)
Files.delete(indexFile.toPath)
- segment.recover(config.maxMessageSize)
+ segment.txnIndex.delete()
+ recoverSegment(segment)
}
} else {
- error("Could not find index file corresponding to log file %s, rebuilding index...".format(segment.log.file.getAbsolutePath))
- segment.recover(config.maxMessageSize)
+ error("Could not find offset index file corresponding to log file %s, rebuilding index...".format(segment.log.file.getAbsolutePath))
+ recoverSegment(segment)
}
segments.put(startOffset, segment)
}
}
+ }
+
+ private def recoverSegment(segment: LogSegment, leaderEpochCache: Option[LeaderEpochCache] = None): Int = lock synchronized {
+ val stateManager = new ProducerStateManager(topicPartition, dir, maxPidExpirationMs)
+ stateManager.truncateAndReload(logStartOffset, segment.baseOffset, time.milliseconds)
+ logSegments(stateManager.mapEndOffset, segment.baseOffset).foreach { segment =>
+ val startOffset = math.max(segment.baseOffset, stateManager.mapEndOffset)
+ val fetchDataInfo = segment.read(startOffset, None, Int.MaxValue)
+ if (fetchDataInfo != null)
+ loadProducersFromLog(stateManager, fetchDataInfo.records)
+ }
+ val bytesTruncated = segment.recover(config.maxMessageSize, stateManager, leaderEpochCache)
- // Finally, complete any interrupted swap operations. To be crash-safe,
- // log files that are replaced by the swap segment should be renamed to .deleted
- // before the swap file is restored as the new segment file.
+ // once we have recovered the segment's data, take a snapshot to ensure that we won't
+ // need to reload the same segment again while recovering another segment.
+ stateManager.takeSnapshot()
+ bytesTruncated
+ }
+
+ private def completeSwapOperations(swapFiles: Set[File]): Unit = {
for (swapFile <- swapFiles) {
val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, SwapFileSuffix, ""))
val filename = logFile.getName
@@ -306,18 +329,36 @@ class Log(@volatile var dir: File,
val index = new OffsetIndex(indexFile, baseOffset = startOffset, maxIndexSize = config.maxIndexSize)
val timeIndexFile = new File(CoreUtils.replaceSuffix(logFile.getPath, LogFileSuffix, TimeIndexFileSuffix) + SwapFileSuffix)
val timeIndex = new TimeIndex(timeIndexFile, baseOffset = startOffset, maxIndexSize = config.maxIndexSize)
+ val txnIndexFile = new File(CoreUtils.replaceSuffix(logFile.getPath, LogFileSuffix, TxnIndexFileSuffix) + SwapFileSuffix)
+ val txnIndex = new TransactionIndex(startOffset, txnIndexFile)
val swapSegment = new LogSegment(FileRecords.open(swapFile),
- index = index,
- timeIndex = timeIndex,
- baseOffset = startOffset,
- indexIntervalBytes = config.indexInterval,
- rollJitterMs = config.randomSegmentJitter,
- time = time)
+ index = index,
+ timeIndex = timeIndex,
+ txnIndex = txnIndex,
+ baseOffset = startOffset,
+ indexIntervalBytes = config.indexInterval,
+ rollJitterMs = config.randomSegmentJitter,
+ time = time)
info("Found log file %s from interrupted swap operation, repairing.".format(swapFile.getPath))
- swapSegment.recover(config.maxMessageSize)
- val oldSegments = logSegments(swapSegment.baseOffset, swapSegment.nextOffset)
+ recoverSegment(swapSegment)
+ val oldSegments = logSegments(swapSegment.baseOffset, swapSegment.nextOffset())
replaceSegments(swapSegment, oldSegments.toSeq, isRecoveredSwapFile = true)
}
+ }
+
+ /* Load the log segments from the log files on disk */
+ private def loadSegments() {
+ // first do a pass through the files in the log directory and remove any temporary files
+ // and find any interrupted swap operations
+ val swapFiles = removeTempFilesAndCollectSwapFiles()
+
+ // now do a second pass and load all the log and index files
+ loadSegmentFiles()
+
+ // Finally, complete any interrupted swap operations. To be crash-safe,
+ // log files that are replaced by the swap segment should be renamed to .deleted
+ // before the swap file is restored as the new segment file.
+ completeSwapOperations(swapFiles)
if(logSegments.isEmpty) {
// no existing segments, create a new mutable segment beginning at offset 0
@@ -330,13 +371,11 @@ class Log(@volatile var dir: File,
fileAlreadyExists = false,
initFileSize = this.initFileSize(),
preallocate = config.preallocate))
- } else {
- if (!dir.getAbsolutePath.endsWith(Log.DeleteDirSuffix)) {
- recoverLog()
- // reset the index size of the currently active log segment to allow more entries
- activeSegment.index.resize(config.maxIndexSize)
- activeSegment.timeIndex.resize(config.maxIndexSize)
- }
+ } else if (!dir.getAbsolutePath.endsWith(Log.DeleteDirSuffix)) {
+ recoverLog()
+ // reset the index size of the currently active log segment to allow more entries
+ activeSegment.index.resize(config.maxIndexSize)
+ activeSegment.timeIndex.resize(config.maxIndexSize)
}
}
@@ -347,66 +386,72 @@ class Log(@volatile var dir: File,
private def recoverLog() {
// if we have the clean shutdown marker, skip recovery
if(hasCleanShutdownFile) {
- this.recoveryPoint = activeSegment.nextOffset
+ this.recoveryPoint = activeSegment.nextOffset()
return
}
// okay we need to actually recovery this log
val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator
while(unflushed.hasNext) {
- val curr = unflushed.next
- info("Recovering unflushed segment %d in log %s.".format(curr.baseOffset, name))
+ val segment = unflushed.next
+ info("Recovering unflushed segment %d in log %s.".format(segment.baseOffset, name))
val truncatedBytes =
try {
- curr.recover(config.maxMessageSize, Some(leaderEpochCache))
+ recoverSegment(segment, Some(leaderEpochCache))
} catch {
case _: InvalidOffsetException =>
- val startOffset = curr.baseOffset
+ val startOffset = segment.baseOffset
warn("Found invalid offset during recovery for log " + dir.getName +". Deleting the corrupt segment and " +
"creating an empty one with starting offset " + startOffset)
- curr.truncateTo(startOffset)
+ segment.truncateTo(startOffset)
}
if(truncatedBytes > 0) {
// we had an invalid message, delete all remaining log
- warn("Corruption found in segment %d of log %s, truncating to offset %d.".format(curr.baseOffset, name, curr.nextOffset))
+ warn("Corruption found in segment %d of log %s, truncating to offset %d.".format(segment.baseOffset, name,
+ segment.nextOffset()))
unflushed.foreach(deleteSegment)
}
}
}
- /**
- * Creates an instance of id map for this log and updates the mapping
- * in the case it is missing some messages. Note that the id mapping
- * starts from a snapshot that is taken strictly before the log end
- * offset. Consequently, we need to process the tail of the log to update
- * the mapping.
- */
- private def buildAndRecoverPidMap(lastOffset: Long) {
- lock synchronized {
- info(s"Recovering PID mapping from offset $lastOffset for partition $topicPartition")
- val currentTimeMs = time.milliseconds
- pidMap.truncateAndReload(logStartOffset, lastOffset, currentTimeMs)
- logSegments(pidMap.mapEndOffset, lastOffset).foreach { segment =>
- val startOffset = math.max(segment.baseOffset, pidMap.mapEndOffset)
+ private def loadProducerState(lastOffset: Long): Unit = lock synchronized {
+ info(s"Loading producer state from offset $lastOffset for partition $topicPartition")
+ val currentTimeMs = time.milliseconds
+ producerStateManager.truncateAndReload(logStartOffset, lastOffset, currentTimeMs)
+
+ // only do the potentially expensive reloading of the last snapshot offset is lower than the
+ // log end offset (which would be the case on first startup) and there are active producers.
+ // if there are no active producers, then truncating shouldn't change that fact (although it
+ // could cause a producerId to expire earlier than expected), so we can skip the loading.
+ // This is an optimization for users which are not yet using idempotent/transactional features yet.
+ if (lastOffset > producerStateManager.mapEndOffset || !producerStateManager.isEmpty) {
+ logSegments(producerStateManager.mapEndOffset, lastOffset).foreach { segment =>
+ val startOffset = math.max(segment.baseOffset, producerStateManager.mapEndOffset)
val fetchDataInfo = segment.read(startOffset, Some(lastOffset), Int.MaxValue)
- if (fetchDataInfo != null) {
- fetchDataInfo.records.batches.asScala.foreach { batch =>
- if (batch.hasProducerId) {
- val pidEntry = ProducerIdEntry(batch.producerEpoch, batch.lastSequence, batch.lastOffset,
- batch.lastSequence - batch.baseSequence, batch.maxTimestamp)
- pidMap.load(batch.producerId, pidEntry, currentTimeMs)
- }
- }
- }
+ if (fetchDataInfo != null)
+ loadProducersFromLog(producerStateManager, fetchDataInfo.records)
}
- pidMap.updateMapEndOffset(lastOffset)
}
+
+ producerStateManager.updateMapEndOffset(lastOffset)
+ updateFirstUnstableOffset()
}
- private[log] def activePids: Map[Long, ProducerIdEntry] = {
- lock synchronized {
- pidMap.activePids
+ private def loadProducersFromLog(producerStateManager: ProducerStateManager, records: Records): Unit = {
+ val loadedProducers = mutable.Map.empty[Long, ProducerAppendInfo]
+ val completedTxns = ListBuffer.empty[CompletedTxn]
+ records.batches.asScala.foreach { batch =>
+ if (batch.hasProducerId) {
+ val lastEntry = producerStateManager.lastEntry(batch.producerId)
+ updateProducers(batch, loadedProducers, completedTxns, lastEntry, loadingFromLog = true)
+ }
}
+ loadedProducers.values.foreach(producerStateManager.update)
+ completedTxns.foreach(producerStateManager.completeTxn)
+ }
+
+ private[log] def activePids: Map[Long, ProducerIdEntry] = lock synchronized {
+ producerStateManager.activeProducers
}
/**
@@ -426,47 +471,50 @@ class Log(@volatile var dir: File,
def close() {
debug(s"Closing log $name")
lock synchronized {
+ producerStateManager.takeSnapshot()
logSegments.foreach(_.close())
}
}
/**
- * Append this message set to the active segment of the log, assigning offsets and Partition Leader Epochs
- * @param records The records to append
- * @throws KafkaStorageException If the append fails due to an I/O error.
- * @return Information about the appended messages including the first and last offset.
- */
- def appendAsLeader(records: MemoryRecords, leaderEpoch: Int): LogAppendInfo = {
- append(records, assignOffsets = true, leaderEpoch)
+ * Append this message set to the active segment of the log, assigning offsets and Partition Leader Epochs
+ * @param records The records to append
+ * @param isFromClient Whether or not this append is from a producer
+ * @throws KafkaStorageException If the append fails due to an I/O error.
+ * @return Information about the appended messages including the first and last offset.
+ */
+ def appendAsLeader(records: MemoryRecords, leaderEpoch: Int, isFromClient: Boolean = true): LogAppendInfo = {
+ append(records, isFromClient, assignOffsets = true, leaderEpoch)
}
/**
- * Append this message set to the active segment of the log without assigning offsets or Partition Leader Epochs
- * @param records The records to append
- * @throws KafkaStorageException If the append fails due to an I/O error.
- * @return Information about the appended messages including the first and last offset.
- */
+ * Append this message set to the active segment of the log without assigning offsets or Partition Leader Epochs
+ * @param records The records to append
+ * @throws KafkaStorageException If the append fails due to an I/O error.
+ * @return Information about the appended messages including the first and last offset.
+ */
def appendAsFollower(records: MemoryRecords): LogAppendInfo = {
- append(records, assignOffsets = false, leaderEpoch = -1)
+ append(records, isFromClient = false, assignOffsets = false, leaderEpoch = -1)
}
/**
- * Append this message set to the active segment of the log, rolling over to a fresh segment if necessary.
- *
- * This method will generally be responsible for assigning offsets to the messages,
- * however if the assignOffsets=false flag is passed we will only check that the existing offsets are valid.
- *
- * @param records The log records to append
- * @param assignOffsets Should the log assign offsets to this message set or blindly apply what it is given
- * @param leaderEpoch The partition's leader epoch which will be applied to messages when offsets are assigned on the leader
- * @throws KafkaStorageException If the append fails due to an I/O error.
- * @return Information about the appended messages including the first and last offset.
- */
- private def append(records: MemoryRecords, assignOffsets: Boolean = true, leaderEpoch: Int): LogAppendInfo = {
- val appendInfo = analyzeAndValidateRecords(records, isFromClient = assignOffsets)
+ * Append this message set to the active segment of the log, rolling over to a fresh segment if necessary.
+ *
+ * This method will generally be responsible for assigning offsets to the messages,
+ * however if the assignOffsets=false flag is passed we will only check that the existing offsets are valid.
+ *
+ * @param records The log records to append
+ * @param isFromClient Whether or not this append is from a producer
+ * @param assignOffsets Should the log assign offsets to this message set or blindly apply what it is given
+ * @param leaderEpoch The partition's leader epoch which will be applied to messages when offsets are assigned on the leader
+ * @throws KafkaStorageException If the append fails due to an I/O error.
+ * @return Information about the appended messages including the first and last offset.
+ */
+ private def append(records: MemoryRecords, isFromClient: Boolean, assignOffsets: Boolean = true, leaderEpoch: Int): LogAppendInfo = {
+ val appendInfo = analyzeAndValidateRecords(records, isFromClient = isFromClient)
// return if we have no valid messages or if this is a duplicate of the last appended entry
- if (appendInfo.shallowCount == 0 || appendInfo.isDuplicate)
+ if (appendInfo.shallowCount == 0)
return appendInfo
// trim any invalid bytes or partial messages before appending it to the on-disk log
@@ -483,15 +531,16 @@ class Log(@volatile var dir: File,
val now = time.milliseconds
val validateAndOffsetAssignResult = try {
LogValidator.validateMessagesAndAssignOffsets(validRecords,
- offset,
- now,
- appendInfo.sourceCodec,
- appendInfo.targetCodec,
- config.compact,
- config.messageFormatVersion.messageFormatVersion,
- config.messageTimestampType,
- config.messageTimestampDifferenceMaxMs,
- leaderEpoch)
+ offset,
+ now,
+ appendInfo.sourceCodec,
+ appendInfo.targetCodec,
+ config.compact,
+ config.messageFormatVersion.messageFormatVersion,
+ config.messageTimestampType,
+ config.messageTimestampDifferenceMaxMs,
+ leaderEpoch,
+ isFromClient)
} catch {
case e: IOException => throw new KafkaException("Error in validating messages while appending to log '%s'".format(name), e)
}
@@ -534,34 +583,56 @@ class Log(@volatile var dir: File,
.format(validRecords.sizeInBytes, config.segmentSize))
}
+ // now that we have valid records, offsets assigned, and timestamps updated, we need to
+ // validate the idempotent/transactional state of the producers and collect some metadata
+ val (updatedProducers, completedTxns, maybeDuplicate) = analyzeAndValidateProducerState(validRecords, isFromClient)
+ maybeDuplicate.foreach { duplicate =>
+ appendInfo.firstOffset = duplicate.firstOffset
+ appendInfo.lastOffset = duplicate.lastOffset
+ appendInfo.logAppendTime = duplicate.timestamp
+ return appendInfo
+ }
+
// maybe roll the log if this segment is full
val segment = maybeRoll(messagesSize = validRecords.sizeInBytes,
maxTimestampInMessages = appendInfo.maxTimestamp,
maxOffsetInMessages = appendInfo.lastOffset)
- // now append to the log
+ val logOffsetMetadata = LogOffsetMetadata(
+ messageOffset = appendInfo.firstOffset,
+ segmentBaseOffset = segment.baseOffset,
+ relativePositionInSegment = segment.size)
+
segment.append(firstOffset = appendInfo.firstOffset,
largestOffset = appendInfo.lastOffset,
largestTimestamp = appendInfo.maxTimestamp,
shallowOffsetOfMaxTimestamp = appendInfo.offsetOfMaxTimestamp,
records = validRecords)
- // update the PID sequence mapping
- for ((pid, producerAppendInfo) <- appendInfo.producerAppendInfos) {
- trace(s"Updating pid with sequence: $pid -> ${producerAppendInfo.lastEntry}")
+ // update the producer state
+ for ((producerId, producerAppendInfo) <- updatedProducers) {
+ trace(s"Updating producer $producerId state: ${producerAppendInfo.lastEntry}")
+ producerAppendInfo.maybeCacheTxnFirstOffsetMetadata(logOffsetMetadata)
+ producerStateManager.update(producerAppendInfo)
+ }
- if (assignOffsets)
- producerAppendInfo.assignLastOffsetAndTimestamp(appendInfo.lastOffset, appendInfo.maxTimestamp)
- pidMap.update(producerAppendInfo)
+ // update the transaction index with the true last stable offset. The last offset visible
+ // to consumers using READ_COMMITTED will be limited by this value and the high watermark.
+ for (completedTxn <- completedTxns) {
+ val lastStableOffset = producerStateManager.completeTxn(completedTxn)
+ segment.updateTxnIndex(completedTxn, lastStableOffset)
}
// always update the last pid map offset so that the snapshot reflects the current offset
// even if there isn't any idempotent data being written
- pidMap.updateMapEndOffset(appendInfo.lastOffset + 1)
+ producerStateManager.updateMapEndOffset(appendInfo.lastOffset + 1)
// increment the log end offset
updateLogEndOffset(appendInfo.lastOffset + 1)
+ // update the first unstable offset (which is used to compute LSO)
+ updateFirstUnstableOffset()
+
trace("Appended message set to log %s with first offset: %d, next offset: %d, and messages: %s"
.format(this.name, appendInfo.firstOffset, nextOffsetMetadata.messageOffset, validRecords))
@@ -575,6 +646,24 @@ class Log(@volatile var dir: File,
}
}
+ def onHighWatermarkIncremented(highWatermark: Long): Unit = {
+ lock synchronized {
+ producerStateManager.onHighWatermarkUpdated(highWatermark)
+ updateFirstUnstableOffset()
+ }
+ }
+
+ private def updateFirstUnstableOffset(): Unit = lock synchronized {
+ this.firstUnstableOffset = producerStateManager.firstUnstableOffset match {
+ case Some(logOffsetMetadata) if logOffsetMetadata.messageOffsetOnly =>
+ val offset = logOffsetMetadata.messageOffset
+ val segment = segments.floorEntry(offset).getValue
+ val position = segment.translateOffset(offset)
+ Some(LogOffsetMetadata(offset, segment.baseOffset, position.position))
+ case other => other
+ }
+ }
+
/**
* Increment the log start offset if the provided offset is larger.
*/
@@ -589,6 +678,23 @@ class Log(@volatile var dir: File,
}
}
+ private def analyzeAndValidateProducerState(records: MemoryRecords, isFromClient: Boolean):
+ (mutable.Map[Long, ProducerAppendInfo], List[CompletedTxn], Option[ProducerIdEntry]) = {
+ val updatedProducers = mutable.Map.empty[Long, ProducerAppendInfo]
+ val completedTxns = ListBuffer.empty[CompletedTxn]
+ for (batch <- records.batches.asScala if batch.hasProducerId) {
+ val maybeLastEntry = producerStateManager.lastEntry(batch.producerId)
+
+ // if this is a client produce request, there will be only one batch. If that batch matches
+ // the last appended entry for that producer, then this request is a duplicate and we return
+ // the last appended entry to the client.
+ if (isFromClient && maybeLastEntry.exists(_.isDuplicate(batch)))
+ return (updatedProducers, completedTxns.toList, maybeLastEntry)
+ updateProducers(batch, updatedProducers, completedTxns, maybeLastEntry, loadingFromLog = false)
+ }
+ (updatedProducers, completedTxns.toList, None)
+ }
+
/**
* Validate the following:
* <ol>
@@ -616,8 +722,6 @@ class Log(@volatile var dir: File,
var monotonic = true
var maxTimestamp = RecordBatch.NO_TIMESTAMP
var offsetOfMaxTimestamp = -1L
- var isDuplicate = false
- val producerAppendInfos = mutable.Map[Long, ProducerAppendInfo]()
for (batch <- records.batches.asScala) {
// we only validate V2 and higher to avoid potential compatibility issues with older clients
@@ -660,37 +764,23 @@ class Log(@volatile var dir: File,
val messageCodec = CompressionCodec.getCompressionCodec(batch.compressionType.id)
if (messageCodec != NoCompressionCodec)
sourceCodec = messageCodec
-
- val pid = batch.producerId
- if (pid != RecordBatch.NO_PRODUCER_ID) {
- producerAppendInfos.get(pid) match {
- case Some(appendInfo) => appendInfo.append(batch)
- case None =>
- val lastEntry = pidMap.lastEntry(pid).getOrElse(ProducerIdEntry.Empty)
- if (isFromClient && lastEntry.isDuplicate(batch)) {
- // This request is a duplicate so return the information about the existing entry. Note that for requests
- // coming from the client, there will only be one RecordBatch per request, so there will be only one iteration
- // of the loop and the values below will not be updated more than once.
- isDuplicate = true
- firstOffset = lastEntry.firstOffset
- lastOffset = lastEntry.lastOffset
- maxTimestamp = lastEntry.timestamp
- debug(s"Detected a duplicate for partition $topicPartition at (firstOffset, lastOffset): ($firstOffset, $lastOffset). " +
- "Ignoring the incoming record.")
- } else {
- val producerAppendInfo = new ProducerAppendInfo(pid, lastEntry)
- producerAppendInfos.put(pid, producerAppendInfo)
- producerAppendInfo.append(batch)
- }
- }
- }
}
// Apply broker-side compression if any
val targetCodec = BrokerCompressionCodec.getTargetCompressionCodec(config.compressionType, sourceCodec)
-
LogAppendInfo(firstOffset, lastOffset, maxTimestamp, offsetOfMaxTimestamp, RecordBatch.NO_TIMESTAMP, sourceCodec,
- targetCodec, shallowMessageCount, validBytesCount, monotonic, producerAppendInfos.toMap, isDuplicate)
+ targetCodec, shallowMessageCount, validBytesCount, monotonic)
+ }
+
+ private def updateProducers(batch: RecordBatch,
+ producers: mutable.Map[Long, ProducerAppendInfo],
+ completedTxns: ListBuffer[CompletedTxn],
+ lastEntry: Option[ProducerIdEntry],
+ loadingFromLog: Boolean): Unit = {
+ val pid = batch.producerId
+ val appendInfo = producers.getOrElseUpdate(pid, new ProducerAppendInfo(pid, lastEntry, loadingFromLog))
+ val maybeCompletedTxn = appendInfo.append(batch)
+ maybeCompletedTxn.foreach(completedTxns += _)
}
/**
@@ -721,11 +811,19 @@ class Log(@volatile var dir: File,
* @param maxLength The maximum number of bytes to read
* @param maxOffset The offset to read up to, exclusive. (i.e. this offset NOT included in the resulting message set)
* @param minOneMessage If this is true, the first message will be returned even if it exceeds `maxLength` (if one exists)
+ * @param isolationLevel The isolation level of the fetcher. The READ_UNCOMMITTED isolation level has the traditional
+ * read semantics (e.g. consumers are limited to fetching up to the high watermark). In
+ * READ_COMMITTED, consumers are limited to fetching up to the last stable offset. Additionally,
+ * in READ_COMMITTED, the transaction index is consulted after fetching to collect the list
+ * of aborted transactions in the fetch range which the consumer uses to filter the fetched
+ * records before they are returned to the user. Note that fetches from followers always use
+ * READ_UNCOMMITTED.
*
* @throws OffsetOutOfRangeException If startOffset is beyond the log end offset or before the log start offset
* @return The fetch data information including fetch starting offset metadata and messages read.
*/
- def read(startOffset: Long, maxLength: Int, maxOffset: Option[Long] = None, minOneMessage: Boolean = false): FetchDataInfo = {
+ def read(startOffset: Long, maxLength: Int, maxOffset: Option[Long] = None, minOneMessage: Boolean = false,
+ isolationLevel: IsolationLevel = IsolationLevel.READ_UNCOMMITTED): FetchDataInfo = {
trace("Reading %d bytes from offset %d in log %s of length %d bytes".format(maxLength, startOffset, name, size))
// Because we don't use lock for reading, the synchronization is a little bit tricky.
@@ -735,38 +833,43 @@ class Log(@volatile var dir: File,
if(startOffset == next)
return FetchDataInfo(currentNextOffsetMetadata, MemoryRecords.EMPTY)
- var entry = segments.floorEntry(startOffset)
+ var segmentEntry = segments.floorEntry(startOffset)
// return error on attempt to read beyond the log end offset or read below log start offset
- if(startOffset > next || entry == null || startOffset < logStartOffset)
+ if(startOffset > next || segmentEntry == null || startOffset < logStartOffset)
throw new OffsetOutOfRangeException("Request for offset %d but we only have log segments in the range %d to %d.".format(startOffset, logStartOffset, next))
// Do the read on the segment with a base offset less than the target offset
// but if that segment doesn't contain any messages with an offset greater than that
// continue to read from successive segments until we get some messages or we reach the end of the log
- while(entry != null) {
+ while(segmentEntry != null) {
+ val segment = segmentEntry.getValue
+
// If the fetch occurs on the active segment, there might be a race condition where two fetch requests occur after
// the message is appended but before the nextOffsetMetadata is updated. In that case the second fetch may
// cause OffsetOutOfRangeException. To solve that, we cap the reading up to exposed position instead of the log
// end of the active segment.
val maxPosition = {
- if (entry == segments.lastEntry) {
+ if (segmentEntry == segments.lastEntry) {
val exposedPos = nextOffsetMetadata.relativePositionInSegment.toLong
// Check the segment again in case a new segment has just rolled out.
- if (entry != segments.lastEntry)
+ if (segmentEntry != segments.lastEntry)
// New log segment has rolled out, we can read up to the file end.
- entry.getValue.size
+ segment.size
else
exposedPos
} else {
- entry.getValue.size
+ segment.size
}
}
- val fetchInfo = entry.getValue.read(startOffset, maxOffset, maxLength, maxPosition, minOneMessage)
- if(fetchInfo == null) {
- entry = segments.higherEntry(entry.getKey)
+ val fetchInfo = segment.read(startOffset, maxOffset, maxLength, maxPosition, minOneMessage)
+ if (fetchInfo == null) {
+ segmentEntry = segments.higherEntry(segmentEntry.getKey)
} else {
- return fetchInfo
+ return isolationLevel match {
+ case IsolationLevel.READ_UNCOMMITTED => fetchInfo
+ case IsolationLevel.READ_COMMITTED => addAbortedTransactions(startOffset, segmentEntry, fetchInfo)
+ }
}
}
@@ -776,6 +879,41 @@ class Log(@volatile var dir: File,
FetchDataInfo(nextOffsetMetadata, MemoryRecords.EMPTY)
}
+ private def addAbortedTransactions(startOffset: Long, segmentEntry: JEntry[JLong, LogSegment],
+ fetchInfo: FetchDataInfo): FetchDataInfo = {
+ val fetchSize = fetchInfo.records.sizeInBytes
+ val startOffsetPosition = OffsetPosition(fetchInfo.fetchOffsetMetadata.messageOffset,
+ fetchInfo.fetchOffsetMetadata.relativePositionInSegment)
+ val upperBoundOffset = segmentEntry.getValue.fetchUpperBoundOffset(startOffsetPosition, fetchSize).getOrElse {
+ val nextSegmentEntry = segments.higherEntry(segmentEntry.getKey)
+ if (nextSegmentEntry != null)
+ nextSegmentEntry.getValue.baseOffset
+ else
+ logEndOffset
+ }
+ val abortedTransactions = collectAbortedTransactions(startOffset, upperBoundOffset, segmentEntry)
+ FetchDataInfo(fetchOffsetMetadata = fetchInfo.fetchOffsetMetadata,
+ records = fetchInfo.records,
+ firstEntryIncomplete = fetchInfo.firstEntryIncomplete,
+ abortedTransactions = Some(abortedTransactions))
+ }
+
+ private def collectAbortedTransactions(startOffset: Long, upperBoundOffset: Long,
+ startingSegmentEntry: JEntry[JLong, LogSegment]): List[AbortedTransaction] = {
+ var segmentEntry = startingSegmentEntry
+ val abortedTransactions = ListBuffer.empty[AbortedTransaction]
+
+ while (segmentEntry != null) {
+ val searchResult = segmentEntry.getValue.collectAbortedTxns(startOffset, upperBoundOffset)
+ abortedTransactions ++= searchResult.abortedTransactions
+ if (searchResult.isComplete)
+ return abortedTransactions.toList
+
+ segmentEntry = segments.higherEntry(segmentEntry.getKey)
+ }
+ abortedTransactions.toList
+ }
+
/**
* Get an offset based on the given timestamp
* The offset returned is the offset of the first message whose timestamp is greater than or equals to the
@@ -860,7 +998,8 @@ class Log(@volatile var dir: File,
deletable.foreach(deleteSegment)
logStartOffset = math.max(logStartOffset, segments.firstEntry().getValue.baseOffset)
leaderEpochCache.clearEarliest(logStartOffset)
- pidMap.expirePids(logStartOffset)
+ producerStateManager.evictUnretainedProducers(logStartOffset)
+ updateFirstUnstableOffset()
}
}
numToDelete
@@ -934,7 +1073,7 @@ class Log(@volatile var dir: File,
def logEndOffsetMetadata: LogOffsetMetadata = nextOffsetMetadata
/**
- * The offset of the next message that will be appended to the log
+ * The offset of the next message that will be appended to the log
*/
def logEndOffset: Long = nextOffsetMetadata.messageOffset
@@ -990,9 +1129,10 @@ class Log(@volatile var dir: File,
lock synchronized {
val newOffset = math.max(expectedNextOffset, logEndOffset)
val logFile = logFilename(dir, newOffset)
- val indexFile = indexFilename(dir, newOffset)
- val timeIndexFile = timeIndexFilename(dir, newOffset)
- for(file <- List(logFile, indexFile, timeIndexFile); if file.exists) {
+ val offsetIdxFile = offsetIndexFile(dir, newOffset)
+ val timeIdxFile = timeIndexFile(dir, newOffset)
+ val txnIdxFile = transactionIndexFile(dir, newOffset)
+ for(file <- List(logFile, offsetIdxFile, timeIdxFile, txnIdxFile) if file.exists) {
warn("Newly rolled segment file " + file.getName + " already exists; deleting it first")
file.delete()
}
@@ -1007,6 +1147,15 @@ class Log(@volatile var dir: File,
seg.log.trim()
}
}
+
+ // take a snapshot of the producer state to facilitate recovery. It is useful to have the snapshot
+ // offset align with the new segment offset since this ensures we can recover the segment by beginning
+ // with the corresponding snapshot file and scanning the segment data. Because the segment base offset
+ // may actually be ahead of the current producer state end offset (which corresponds to the log end offset),
+ // we manually override the state offset here prior to taking the snapshot.
+ producerStateManager.updateMapEndOffset(newOffset)
+ producerStateManager.takeSnapshot()
+
val segment = new LogSegment(dir,
startOffset = newOffset,
indexIntervalBytes = config.indexInterval,
@@ -1053,6 +1202,12 @@ class Log(@volatile var dir: File,
time.milliseconds + " unflushed = " + unflushedMessages)
for(segment <- logSegments(this.recoveryPoint, offset))
segment.flush()
+
+ // now that we have flushed, we can cleanup old producer snapshots. However, it is useful to retain
+ // the snapshots from the recent segments in case we need to truncate and rebuild the producer state.
+ // Otherwise, we would always need to rebuild from the earliest segment.
+ producerStateManager.deleteSnapshotsBefore(minSnapshotOffsetToRetain(offset))
+
lock synchronized {
if(offset > this.recoveryPoint) {
this.recoveryPoint = offset
@@ -1061,6 +1216,17 @@ class Log(@volatile var dir: File,
}
}
+ def minSnapshotOffsetToRetain(flushedOffset: Long) = {
+ // always retain the producer snapshot from the last two segments. This solves the common case
+ // of truncating to an offset within the active segment, and the rarer case of truncating to the
+ // previous segment just after rolling the new segment.
+ var minSnapshotOffset = activeSegment.baseOffset
+ val previousSegment = segments.lowerEntry(activeSegment.baseOffset)
+ if (previousSegment != null)
+ minSnapshotOffset = previousSegment.getValue.baseOffset
+ math.min(flushedOffset, minSnapshotOffset)
+ }
+
/**
* Completely delete this log directory and all contents from the file system with no delay
*/
@@ -1073,11 +1239,25 @@ class Log(@volatile var dir: File,
}
}
- private[log] def maybeTakePidSnapshot(): Unit = pidMap.maybeTakeSnapshot()
+ // visible for testing
+ private[log] def takeProducerSnapshot(): Unit = lock synchronized {
+ producerStateManager.takeSnapshot()
+ }
- private[log] def latestPidSnapshotOffset: Option[Long] = pidMap.latestSnapshotOffset
+ // visible for testing
+ private[log] def latestProducerSnapshotOffset: Option[Long] = lock synchronized {
+ producerStateManager.latestSnapshotOffset
+ }
- private[log] def latestPidMapOffset: Long = pidMap.mapEndOffset
+ // visible for testing
+ private[log] def oldestProducerSnapshotOffset: Option[Long] = lock synchronized {
+ producerStateManager.oldestSnapshotOffset
+ }
+
+ // visible for testing
+ private[log] def latestProducerStateEndOffset: Long = lock synchronized {
+ producerStateManager.mapEndOffset
+ }
/**
* Truncate this log so that it ends with the greatest offset < targetOffset.
@@ -1103,7 +1283,7 @@ class Log(@volatile var dir: File,
this.recoveryPoint = math.min(targetOffset, this.recoveryPoint)
this.logStartOffset = math.min(targetOffset, this.logStartOffset)
leaderEpochCache.clearLatest(targetOffset)
- buildAndRecoverPidMap(targetOffset)
+ loadProducerState(targetOffset)
}
}
}
@@ -1130,8 +1310,9 @@ class Log(@volatile var dir: File,
updateLogEndOffset(newOffset)
leaderEpochCache.clear()
- pidMap.truncate()
- pidMap.updateMapEndOffset(newOffset)
+ producerStateManager.truncate()
+ producerStateManager.updateMapEndOffset(newOffset)
+ updateFirstUnstableOffset()
this.recoveryPoint = math.min(newOffset, this.recoveryPoint)
this.logStartOffset = newOffset
@@ -1282,6 +1463,9 @@ object Log {
val PidSnapshotFileSuffix = ".snapshot"
+ /** an (aborted) txn index */
+ val TxnIndexFileSuffix = ".txnindex"
+
/** a file that is scheduled to be deleted */
val DeletedFileSuffix = ".deleted"
@@ -1331,7 +1515,7 @@ object Log {
* @param dir The directory in which the log will reside
* @param offset The base offset of the log file
*/
- def indexFilename(dir: File, offset: Long) =
+ def offsetIndexFile(dir: File, offset: Long) =
new File(dir, filenamePrefixFromOffset(offset) + IndexFileSuffix)
/**
@@ -1340,7 +1524,7 @@ object Log {
* @param dir The directory in which the log will reside
* @param offset The base offset of the log file
*/
- def timeIndexFilename(dir: File, offset: Long) =
+ def timeIndexFile(dir: File, offset: Long) =
new File(dir, filenamePrefixFromOffset(offset) + TimeIndexFileSuffix)
/**
@@ -1349,9 +1533,12 @@ object Log {
* @param dir The directory in which the log will reside
* @param offset The last offset (exclusive) included in the snapshot
*/
- def pidSnapshotFilename(dir: File, offset: Long) =
+ def producerSnapshotFile(dir: File, offset: Long) =
new File(dir, filenamePrefixFromOffset(offset) + PidSnapshotFileSuffix)
+ def transactionIndexFile(dir: File, offset: Long) =
+ new File(dir, filenamePrefixFromOffset(offset) + TxnIndexFileSuffix)
+
def offsetFromFilename(filename: String): Long =
filename.substring(0, filename.indexOf('.')).toLong
@@ -1387,4 +1574,12 @@ object Log {
new TopicPartition(topic, partition.toInt)
}
+ private def isIndexFile(file: File): Boolean = {
+ val filename = file.getName
+ filename.endsWith(IndexFileSuffix) || filename.endsWith(TimeIndexFileSuffix) || filename.endsWith(TxnIndexFileSuffix)
+ }
+
+ private def isLogFile(file: File): Boolean =
+ file.getPath.endsWith(LogFileSuffix)
+
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/log/LogCleaner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala
index d0e8ec4..282e049 100644
--- a/core/src/main/scala/kafka/log/LogCleaner.scala
+++ b/core/src/main/scala/kafka/log/LogCleaner.scala
@@ -388,12 +388,18 @@ private[log] class Cleaner(val id: Int,
logFile.delete()
val indexFile = new File(segments.head.index.file.getPath + Log.CleanedFileSuffix)
val timeIndexFile = new File(segments.head.timeIndex.file.getPath + Log.CleanedFileSuffix)
+ val txnIndexFile = new File(segments.head.txnIndex.file.getPath + Log.CleanedFileSuffix)
indexFile.delete()
timeIndexFile.delete()
+ txnIndexFile.delete()
+
+ val startOffset = segments.head.baseOffset
val records = FileRecords.open(logFile, false, log.initFileSize(), log.config.preallocate)
- val index = new OffsetIndex(indexFile, segments.head.baseOffset, segments.head.index.maxIndexSize)
- val timeIndex = new TimeIndex(timeIndexFile, segments.head.baseOffset, segments.head.timeIndex.maxIndexSize)
- val cleaned = new LogSegment(records, index, timeIndex, segments.head.baseOffset, segments.head.indexIntervalBytes, log.config.randomSegmentJitter, time)
+ val index = new OffsetIndex(indexFile, startOffset, segments.head.index.maxIndexSize)
+ val timeIndex = new TimeIndex(timeIndexFile, startOffset, segments.head.timeIndex.maxIndexSize)
+ val txnIndex = new TransactionIndex(startOffset, txnIndexFile)
+ val cleaned = new LogSegment(records, index, timeIndex, txnIndex, startOffset,
+ segments.head.indexIntervalBytes, log.config.randomSegmentJitter, time)
try {
// clean segments into the new destination segment
@@ -451,7 +457,8 @@ private[log] class Cleaner(val id: Int,
activePids: Map[Long, ProducerIdEntry],
stats: CleanerStats) {
val logCleanerFilter = new RecordFilter {
- def shouldRetain(recordBatch: RecordBatch, record: Record): Boolean = shouldRetainMessage(source, map, retainDeletes, record, stats, activePids, recordBatch.producerId)
+ def shouldRetain(recordBatch: RecordBatch, record: Record): Boolean =
+ shouldRetainMessage(source, map, retainDeletes, recordBatch, record, stats, activePids)
}
var position = 0
@@ -492,17 +499,20 @@ private[log] class Cleaner(val id: Int,
private def shouldRetainMessage(source: kafka.log.LogSegment,
map: kafka.log.OffsetMap,
retainDeletes: Boolean,
+ batch: RecordBatch,
record: Record,
stats: CleanerStats,
- activePids: Map[Long, ProducerIdEntry],
- pid: Long): Boolean = {
- if (record.isControlRecord)
+ activeProducers: Map[Long, ProducerIdEntry]): Boolean = {
+ if (batch.isControlBatch)
return true
// retain the record if it is the last one produced by an active idempotent producer to ensure that
- // the PID is not removed from the log before it has been expired
- if (RecordBatch.NO_PRODUCER_ID < pid && activePids.get(pid).exists(_.lastOffset == record.offset))
- return true
+ // the producerId is not removed from the log before it has been expired
+ if (batch.hasProducerId) {
+ val producerId = batch.producerId
+ if (RecordBatch.NO_PRODUCER_ID < producerId && activeProducers.get(producerId).exists(_.lastOffset == record.offset))
+ return true
+ }
val pastLatestOffset = record.offset > map.latestOffset
if (pastLatestOffset)
@@ -638,8 +648,8 @@ private[log] class Cleaner(val id: Int,
throttler.maybeThrottle(records.sizeInBytes)
val startPosition = position
- for (record <- records.records.asScala) {
- if (!record.isControlRecord && record.hasKey && record.offset >= start) {
+ for (batch <- records.batches.asScala; record <- batch.asScala) {
+ if (!batch.isControlBatch && record.hasKey && record.offset >= start) {
if (map.size < maxDesiredMapSize)
map.put(record.key, record.offset)
else
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/log/LogManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala
index b89fc40..c621680 100755
--- a/core/src/main/scala/kafka/log/LogManager.scala
+++ b/core/src/main/scala/kafka/log/LogManager.scala
@@ -189,7 +189,7 @@ class LogManager(val logDirs: Array[File],
}
}
- jobs(cleanShutdownFile) = jobsForDir.map(pool.submit).toSeq
+ jobs(cleanShutdownFile) = jobsForDir.map(pool.submit)
}
@@ -282,7 +282,6 @@ class LogManager(val logDirs: Array[File],
jobs(dir) = jobsForDir.map(pool.submit).toSeq
}
-
try {
for ((dir, dirJobs) <- jobs) {
dirJobs.foreach(_.get)
@@ -312,7 +311,6 @@ class LogManager(val logDirs: Array[File],
info("Shutdown complete.")
}
-
/**
* Truncate the partition logs to the specified offsets and checkpoint the recovery point to this offset
*
@@ -454,7 +452,7 @@ class LogManager(val logDirs: Array[File],
case e: Throwable =>
error(s"Exception in kafka-delete-logs thread.", e)
}
-}
+ }
/**
* Rename the directory of the given topic-partition "logdir" as "logdir.uuid.delete" and
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/log/LogSegment.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala
index df3c372..d76b47a 100755
--- a/core/src/main/scala/kafka/log/LogSegment.scala
+++ b/core/src/main/scala/kafka/log/LogSegment.scala
@@ -27,14 +27,14 @@ import kafka.server.epoch.LeaderEpochCache
import kafka.server.{FetchDataInfo, LogOffsetMetadata}
import kafka.utils._
import org.apache.kafka.common.errors.CorruptRecordException
-import org.apache.kafka.common.record.FileRecords.LogEntryPosition
+import org.apache.kafka.common.record.FileRecords.LogOffsetPosition
import org.apache.kafka.common.record._
import org.apache.kafka.common.utils.Time
import scala.collection.JavaConverters._
import scala.math._
- /**
+/**
* A segment of the log. Each segment has two components: a log and an index. The log is a FileMessageSet containing
* the actual messages. The index is an OffsetIndex that maps from logical offsets to physical file positions. Each
* segment has a base offset which is an offset <= the least offset of any message in this segment and > any offset in
@@ -53,6 +53,7 @@ import scala.math._
class LogSegment(val log: FileRecords,
val index: OffsetIndex,
val timeIndex: TimeIndex,
+ val txnIndex: TransactionIndex,
val baseOffset: Long,
val indexIntervalBytes: Int,
val rollJitterMs: Long,
@@ -67,45 +68,49 @@ class LogSegment(val log: FileRecords,
private var rollingBasedTimestamp: Option[Long] = None
/* The maximum timestamp we see so far */
- @volatile private var maxTimestampSoFar = timeIndex.lastEntry.timestamp
- @volatile private var offsetOfMaxTimestamp = timeIndex.lastEntry.offset
+ @volatile private var maxTimestampSoFar: Long = timeIndex.lastEntry.timestamp
+ @volatile private var offsetOfMaxTimestamp: Long = timeIndex.lastEntry.offset
- def this(dir: File, startOffset: Long, indexIntervalBytes: Int, maxIndexSize: Int, rollJitterMs: Long, time: Time, fileAlreadyExists: Boolean = false, initFileSize: Int = 0, preallocate: Boolean = false) =
+ def this(dir: File, startOffset: Long, indexIntervalBytes: Int, maxIndexSize: Int, rollJitterMs: Long, time: Time,
+ fileAlreadyExists: Boolean = false, initFileSize: Int = 0, preallocate: Boolean = false) =
this(FileRecords.open(Log.logFilename(dir, startOffset), fileAlreadyExists, initFileSize, preallocate),
- new OffsetIndex(Log.indexFilename(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize),
- new TimeIndex(Log.timeIndexFilename(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize),
+ new OffsetIndex(Log.offsetIndexFile(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize),
+ new TimeIndex(Log.timeIndexFile(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize),
+ new TransactionIndex(startOffset, Log.transactionIndexFile(dir, startOffset)),
startOffset,
indexIntervalBytes,
rollJitterMs,
time)
/* Return the size in bytes of this log segment */
- def size: Long = log.sizeInBytes()
+ def size: Int = log.sizeInBytes()
- /**
- * checks that the argument offset can be represented as an integer offset relative to the baseOffset.
- */
- def canConvertToRelativeOffset(offset: Long): Boolean = {
- (offset - baseOffset) <= Integer.MAX_VALUE
- }
+ /**
+ * checks that the argument offset can be represented as an integer offset relative to the baseOffset.
+ */
+ def canConvertToRelativeOffset(offset: Long): Boolean = {
+ (offset - baseOffset) <= Integer.MAX_VALUE
+ }
- /**
+ /**
* Append the given messages starting with the given offset. Add
* an entry to the index if needed.
*
* It is assumed this method is being called from within a lock.
*
* @param firstOffset The first offset in the message set.
+ * @param largestOffset The last offset in the message set
* @param largestTimestamp The largest timestamp in the message set.
* @param shallowOffsetOfMaxTimestamp The offset of the message that has the largest timestamp in the messages to append.
* @param records The log entries to append.
+ * @return the physical position in the file of the appended records
*/
@nonthreadsafe
def append(firstOffset: Long,
largestOffset: Long,
largestTimestamp: Long,
shallowOffsetOfMaxTimestamp: Long,
- records: MemoryRecords) {
+ records: MemoryRecords): Unit = {
if (records.sizeInBytes > 0) {
trace("Inserting %d bytes at offset %d at position %d with largest timestamp %d at shallow offset %d"
.format(records.sizeInBytes, firstOffset, log.sizeInBytes(), largestTimestamp, shallowOffsetOfMaxTimestamp))
@@ -131,6 +136,28 @@ class LogSegment(val log: FileRecords,
}
}
+ @nonthreadsafe
+ def updateTxnIndex(completedTxn: CompletedTxn, lastStableOffset: Long) {
+ if (completedTxn.isAborted) {
+ trace(s"Writing aborted transaction $completedTxn to transaction index, last stable offset is $lastStableOffset")
+ txnIndex.append(new AbortedTxn(completedTxn, lastStableOffset))
+ }
+ }
+
+ private def updateProducerState(producerStateManager: ProducerStateManager, batch: RecordBatch): Unit = {
+ if (batch.hasProducerId) {
+ val producerId = batch.producerId
+ val lastEntry = producerStateManager.lastEntry(producerId)
+ val appendInfo = new ProducerAppendInfo(batch.producerId, lastEntry, loadingFromLog = true)
+ val maybeCompletedTxn = appendInfo.append(batch)
+ producerStateManager.update(appendInfo)
+ maybeCompletedTxn.foreach { completedTxn =>
+ val lastStableOffset = producerStateManager.completeTxn(completedTxn)
+ updateTxnIndex(completedTxn, lastStableOffset)
+ }
+ }
+ }
+
/**
* Find the physical file position for the first message with offset >= the requested offset.
*
@@ -144,7 +171,7 @@ class LogSegment(val log: FileRecords,
* message or null if no message meets this criteria.
*/
@threadsafe
- private[log] def translateOffset(offset: Long, startingFilePosition: Int = 0): LogEntryPosition = {
+ private[log] def translateOffset(offset: Long, startingFilePosition: Int = 0): LogOffsetPosition = {
val mapping = index.lookup(offset)
log.searchForOffsetWithSize(offset, max(mapping.position, startingFilePosition))
}
@@ -175,7 +202,7 @@ class LogSegment(val log: FileRecords,
if (startOffsetAndSize == null)
return null
- val startPosition = startOffsetAndSize.position.toInt
+ val startPosition = startOffsetAndSize.position
val offsetMetadata = new LogOffsetMetadata(startOffset, this.baseOffset, startPosition)
val adjustedMaxSize =
@@ -187,7 +214,7 @@ class LogSegment(val log: FileRecords,
return FetchDataInfo(offsetMetadata, MemoryRecords.EMPTY)
// calculate the length of the message set to read based on whether or not they gave us a maxOffset
- val length = maxOffset match {
+ val fetchSize: Int = maxOffset match {
case None =>
// no max offset, just read until the max position
min((maxPosition - startPosition).toInt, adjustedMaxSize)
@@ -207,24 +234,32 @@ class LogSegment(val log: FileRecords,
min(min(maxPosition, endPosition) - startPosition, adjustedMaxSize).toInt
}
- FetchDataInfo(offsetMetadata, log.read(startPosition, length),
+ FetchDataInfo(offsetMetadata, log.read(startPosition, fetchSize),
firstEntryIncomplete = adjustedMaxSize < startOffsetAndSize.size)
}
+ def fetchUpperBoundOffset(startOffsetPosition: OffsetPosition, fetchSize: Int): Option[Long] =
+ index.fetchUpperBoundOffset(startOffsetPosition, fetchSize).map(_.offset)
+
/**
* Run recovery on the given segment. This will rebuild the index from the log file and lop off any invalid bytes from the end of the log and index.
*
* @param maxMessageSize A bound the memory allocation in the case of a corrupt message size--we will assume any message larger than this
* is corrupt.
- * @param leaderEpochCache Optionally a cache for updating the leader epoch during recovery.
+ * @param producerStateManager Producer state corresponding to the segment's base offset. This is needed to recover
+ * the transaction index.
+ * @param leaderEpochCache Optionally a cache for updating the leader epoch during recovery.
* @return The number of bytes truncated from the log
*/
@nonthreadsafe
- def recover(maxMessageSize: Int, leaderEpochCache: Option[LeaderEpochCache] = None): Int = {
+ def recover(maxMessageSize: Int,
+ producerStateManager: ProducerStateManager,
+ leaderEpochCache: Option[LeaderEpochCache] = None): Int = {
index.truncate()
index.resize(index.maxIndexSize)
timeIndex.truncate()
timeIndex.resize(timeIndex.maxIndexSize)
+ txnIndex.truncate()
var validBytes = 0
var lastIndexEntry = 0
maxTimestampSoFar = RecordBatch.NO_TIMESTAMP
@@ -250,8 +285,9 @@ class LogSegment(val log: FileRecords,
if (batch.magic >= RecordBatch.MAGIC_VALUE_V2) {
leaderEpochCache.foreach { cache =>
if (batch.partitionLeaderEpoch > cache.latestEpoch()) // this is to avoid unnecessary warning in cache.assign()
- cache.assign(batch.partitionLeaderEpoch, batch.baseOffset())
+ cache.assign(batch.partitionLeaderEpoch, batch.baseOffset)
}
+ updateProducerState(producerStateManager, batch)
}
}
} catch {
@@ -268,22 +304,23 @@ class LogSegment(val log: FileRecords,
truncated
}
- def loadLargestTimestamp(readToLogEnd: Boolean = false) {
+ private def loadLargestTimestamp() {
// Get the last time index entry. If the time index is empty, it will return (-1, baseOffset)
val lastTimeIndexEntry = timeIndex.lastEntry
maxTimestampSoFar = lastTimeIndexEntry.timestamp
offsetOfMaxTimestamp = lastTimeIndexEntry.offset
- if (readToLogEnd) {
- val offsetPosition = index.lookup(lastTimeIndexEntry.offset)
- // Scan the rest of the messages to see if there is a larger timestamp after the last time index entry.
- val maxTimestampOffsetAfterLastEntry = log.largestTimestampAfter(offsetPosition.position)
- if (maxTimestampOffsetAfterLastEntry.timestamp > lastTimeIndexEntry.timestamp) {
- maxTimestampSoFar = maxTimestampOffsetAfterLastEntry.timestamp
- offsetOfMaxTimestamp = maxTimestampOffsetAfterLastEntry.offset
- }
+
+ val offsetPosition = index.lookup(lastTimeIndexEntry.offset)
+ // Scan the rest of the messages to see if there is a larger timestamp after the last time index entry.
+ val maxTimestampOffsetAfterLastEntry = log.largestTimestampAfter(offsetPosition.position)
+ if (maxTimestampOffsetAfterLastEntry.timestamp > lastTimeIndexEntry.timestamp) {
+ maxTimestampSoFar = maxTimestampOffsetAfterLastEntry.timestamp
+ offsetOfMaxTimestamp = maxTimestampOffsetAfterLastEntry.offset
}
}
+ def collectAbortedTxns(fetchOffset: Long, upperBoundOffset: Long): TxnIndexSearchResult =
+ txnIndex.collectAbortedTxns(fetchOffset, upperBoundOffset)
override def toString = "LogSegment(baseOffset=" + baseOffset + ", size=" + size + ")"
@@ -301,6 +338,7 @@ class LogSegment(val log: FileRecords,
return 0
index.truncateTo(offset)
timeIndex.truncateTo(offset)
+ txnIndex.truncateTo(offset)
// after truncation, reset and allocate more space for the (new currently active) index
index.resize(index.maxIndexSize)
timeIndex.resize(timeIndex.maxIndexSize)
@@ -310,9 +348,8 @@ class LogSegment(val log: FileRecords,
rollingBasedTimestamp = None
}
bytesSinceLastIndexEntry = 0
- // We may need to reload the max timestamp after truncation.
if (maxTimestampSoFar >= 0)
- loadLargestTimestamp(readToLogEnd = true)
+ loadLargestTimestamp()
bytesTruncated
}
@@ -323,14 +360,12 @@ class LogSegment(val log: FileRecords,
@threadsafe
def nextOffset(): Long = {
val ms = read(index.lastOffset, None, log.sizeInBytes)
- if (ms == null) {
+ if (ms == null)
baseOffset
- } else {
- ms.records.batches.asScala.lastOption match {
- case None => baseOffset
- case Some(last) => last.nextOffset
- }
- }
+ else
+ ms.records.batches.asScala.lastOption
+ .map(_.nextOffset)
+ .getOrElse(baseOffset)
}
/**
@@ -342,6 +377,7 @@ class LogSegment(val log: FileRecords,
log.flush()
index.flush()
timeIndex.flush()
+ txnIndex.flush()
}
}
@@ -365,6 +401,10 @@ class LogSegment(val log: FileRecords,
catch {
case e: IOException => throw kafkaStorageException("timeindex", e)
}
+ try txnIndex.renameTo(new File(CoreUtils.replaceSuffix(txnIndex.file.getPath, oldSuffix, newSuffix)))
+ catch {
+ case e: IOException => throw kafkaStorageException("txnindex", e)
+ }
}
/**
@@ -437,6 +477,7 @@ class LogSegment(val log: FileRecords,
CoreUtils.swallow(index.close())
CoreUtils.swallow(timeIndex.close())
CoreUtils.swallow(log.close())
+ CoreUtils.swallow(txnIndex.close())
}
/**
@@ -448,12 +489,15 @@ class LogSegment(val log: FileRecords,
val deletedLog = log.delete()
val deletedIndex = index.delete()
val deletedTimeIndex = timeIndex.delete()
- if(!deletedLog && log.file.exists)
+ val deletedTxnIndex = txnIndex.delete()
+ if (!deletedLog && log.file.exists)
throw new KafkaStorageException("Delete of log " + log.file.getName + " failed.")
- if(!deletedIndex && index.file.exists)
+ if (!deletedIndex && index.file.exists)
throw new KafkaStorageException("Delete of index " + index.file.getName + " failed.")
- if(!deletedTimeIndex && timeIndex.file.exists)
+ if (!deletedTimeIndex && timeIndex.file.exists)
throw new KafkaStorageException("Delete of time index " + timeIndex.file.getName + " failed.")
+ if (!deletedTxnIndex && txnIndex.file.exists)
+ throw new KafkaStorageException("Delete of transaction index " + txnIndex.file.getName + " failed.")
}
/**
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/log/LogValidator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/LogValidator.scala b/core/src/main/scala/kafka/log/LogValidator.scala
index c1777d5..33257fd 100644
--- a/core/src/main/scala/kafka/log/LogValidator.scala
+++ b/core/src/main/scala/kafka/log/LogValidator.scala
@@ -52,24 +52,32 @@ private[kafka] object LogValidator extends Logging {
magic: Byte,
timestampType: TimestampType,
timestampDiffMaxMs: Long,
- partitionLeaderEpoch: Int): ValidationAndOffsetAssignResult = {
+ partitionLeaderEpoch: Int,
+ isFromClient: Boolean): ValidationAndOffsetAssignResult = {
if (sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) {
// check the magic value
if (!records.hasMatchingMagic(magic))
convertAndAssignOffsetsNonCompressed(records, offsetCounter, compactedTopic, now, timestampType,
- timestampDiffMaxMs, magic, partitionLeaderEpoch)
+ timestampDiffMaxMs, magic, partitionLeaderEpoch, isFromClient)
else
// Do in-place validation, offset assignment and maybe set timestamp
assignOffsetsNonCompressed(records, offsetCounter, now, compactedTopic, timestampType, timestampDiffMaxMs,
- partitionLeaderEpoch)
+ partitionLeaderEpoch, isFromClient)
} else {
validateMessagesAndAssignOffsetsCompressed(records, offsetCounter, now, sourceCodec, targetCodec, compactedTopic,
- magic, timestampType, timestampDiffMaxMs, partitionLeaderEpoch)
+ magic, timestampType, timestampDiffMaxMs, partitionLeaderEpoch, isFromClient)
}
}
- private def validateBatch(batch: RecordBatch): Unit = {
- ensureNonTransactional(batch)
+ private def validateBatch(batch: RecordBatch, isFromClient: Boolean): Unit = {
+ if (isFromClient) {
+ if (batch.hasProducerId && batch.baseSequence < 0)
+ throw new InvalidRecordException(s"Invalid sequence number ${batch.baseSequence} in record batch " +
+ s"with producerId ${batch.producerId}")
+
+ if (batch.isControlBatch)
+ throw new InvalidRecordException("Clients are not allowed to write control records")
+ }
}
private def validateRecord(batch: RecordBatch, record: Record, now: Long, timestampType: TimestampType,
@@ -84,7 +92,6 @@ private[kafka] object LogValidator extends Logging {
if (batch.magic <= RecordBatch.MAGIC_VALUE_V1 && batch.isCompressed)
record.ensureValid()
- ensureNotControlRecord(record)
validateKey(record, compactedTopic)
validateTimestamp(batch, record, now, timestampType, timestampDiffMaxMs)
}
@@ -96,21 +103,22 @@ private[kafka] object LogValidator extends Logging {
timestampType: TimestampType,
timestampDiffMaxMs: Long,
toMagicValue: Byte,
- partitionLeaderEpoch: Int): ValidationAndOffsetAssignResult = {
+ partitionLeaderEpoch: Int,
+ isFromClient: Boolean): ValidationAndOffsetAssignResult = {
val sizeInBytesAfterConversion = AbstractRecords.estimateSizeInBytes(toMagicValue, offsetCounter.value,
CompressionType.NONE, records.records)
- val (pid, epoch, sequence) = {
+ val (producerId, producerEpoch, sequence, isTransactional) = {
val first = records.batches.asScala.head
- (first.producerId, first.producerEpoch, first.baseSequence)
+ (first.producerId, first.producerEpoch, first.baseSequence, first.isTransactional)
}
val newBuffer = ByteBuffer.allocate(sizeInBytesAfterConversion)
val builder = MemoryRecords.builder(newBuffer, toMagicValue, CompressionType.NONE, timestampType,
- offsetCounter.value, now, pid, epoch, sequence, false, partitionLeaderEpoch)
+ offsetCounter.value, now, producerId, producerEpoch, sequence, isTransactional, partitionLeaderEpoch)
for (batch <- records.batches.asScala) {
- validateBatch(batch)
+ validateBatch(batch, isFromClient)
for (record <- batch.asScala) {
validateRecord(batch, record, now, timestampType, timestampDiffMaxMs, compactedTopic)
@@ -133,21 +141,21 @@ private[kafka] object LogValidator extends Logging {
compactedTopic: Boolean,
timestampType: TimestampType,
timestampDiffMaxMs: Long,
- partitionLeaderEpoch: Int): ValidationAndOffsetAssignResult = {
+ partitionLeaderEpoch: Int,
+ isFromClient: Boolean): ValidationAndOffsetAssignResult = {
var maxTimestamp = RecordBatch.NO_TIMESTAMP
var offsetOfMaxTimestamp = -1L
val initialOffset = offsetCounter.value
var isMagicV2 = false
for (batch <- records.batches.asScala) {
- validateBatch(batch)
+ validateBatch(batch, isFromClient)
var maxBatchTimestamp = RecordBatch.NO_TIMESTAMP
var offsetOfMaxBatchTimestamp = -1L
for (record <- batch.asScala) {
validateRecord(batch, record, now, timestampType, timestampDiffMaxMs, compactedTopic)
-
val offset = offsetCounter.getAndIncrement()
if (batch.magic > RecordBatch.MAGIC_VALUE_V0 && record.timestamp > maxBatchTimestamp) {
maxBatchTimestamp = record.timestamp
@@ -206,7 +214,8 @@ private[kafka] object LogValidator extends Logging {
magic: Byte,
timestampType: TimestampType,
timestampDiffMaxMs: Long,
- partitionLeaderEpoch: Int): ValidationAndOffsetAssignResult = {
+ partitionLeaderEpoch: Int,
+ isFromClient: Boolean): ValidationAndOffsetAssignResult = {
// No in place assignment situation 1 and 2
var inPlaceAssignment = sourceCodec == targetCodec && magic > RecordBatch.MAGIC_VALUE_V0
@@ -216,14 +225,17 @@ private[kafka] object LogValidator extends Logging {
val validatedRecords = new mutable.ArrayBuffer[Record]
for (batch <- records.batches.asScala) {
- validateBatch(batch)
+ validateBatch(batch, isFromClient)
+
+ // Do not compress control records unless they are written compressed
+ if (sourceCodec == NoCompressionCodec && batch.isControlBatch)
+ inPlaceAssignment = true
for (record <- batch.asScala) {
validateRecord(batch, record, now, timestampType, timestampDiffMaxMs, compactedTopic)
if (sourceCodec != NoCompressionCodec && record.isCompressed)
throw new InvalidRecordException("Compressed outer record should not have an inner record with a " +
s"compression attribute set: $record")
-
if (batch.magic > RecordBatch.MAGIC_VALUE_V0 && magic > RecordBatch.MAGIC_VALUE_V0) {
// Check if we need to overwrite offset
// No in place assignment situation 3
@@ -242,15 +254,15 @@ private[kafka] object LogValidator extends Logging {
}
if (!inPlaceAssignment) {
- val (pid, epoch, sequence) = {
+ val (producerId, producerEpoch, sequence, isTransactional) = {
// note that we only reassign offsets for requests coming straight from a producer. For records with magic V2,
// there should be exactly one RecordBatch per request, so the following is all we need to do. For Records
// with older magic versions, there will never be a producer id, etc.
val first = records.batches.asScala.head
- (first.producerId, first.producerEpoch, first.baseSequence)
+ (first.producerId, first.producerEpoch, first.baseSequence, first.isTransactional)
}
buildRecordsAndAssignOffsets(magic, offsetCounter, timestampType, CompressionType.forId(targetCodec.codec), now,
- validatedRecords, pid, epoch, sequence, partitionLeaderEpoch)
+ validatedRecords, producerId, producerEpoch, sequence, isTransactional, partitionLeaderEpoch)
} else {
// we can update the batch only and write the compressed payload as is
val batch = records.batches.iterator.next()
@@ -274,14 +286,22 @@ private[kafka] object LogValidator extends Logging {
}
}
- private def buildRecordsAndAssignOffsets(magic: Byte, offsetCounter: LongRef, timestampType: TimestampType,
- compressionType: CompressionType, logAppendTime: Long,
+ private def buildRecordsAndAssignOffsets(magic: Byte,
+ offsetCounter: LongRef,
+ timestampType: TimestampType,
+ compressionType: CompressionType,
+ logAppendTime: Long,
validatedRecords: Seq[Record],
- producerId: Long, epoch: Short, baseSequence: Int, partitionLeaderEpoch: Int): ValidationAndOffsetAssignResult = {
- val estimatedSize = AbstractRecords.estimateSizeInBytes(magic, offsetCounter.value, compressionType, validatedRecords.asJava)
+ producerId: Long,
+ producerEpoch: Short,
+ baseSequence: Int,
+ isTransactional: Boolean,
+ partitionLeaderEpoch: Int): ValidationAndOffsetAssignResult = {
+ val estimatedSize = AbstractRecords.estimateSizeInBytes(magic, offsetCounter.value, compressionType,
+ validatedRecords.asJava)
val buffer = ByteBuffer.allocate(estimatedSize)
val builder = MemoryRecords.builder(buffer, magic, compressionType, timestampType, offsetCounter.value,
- logAppendTime, producerId, epoch, baseSequence, false, partitionLeaderEpoch)
+ logAppendTime, producerId, producerEpoch, baseSequence, isTransactional, partitionLeaderEpoch)
validatedRecords.foreach { record =>
builder.appendWithOffset(offsetCounter.getAndIncrement(), record)
@@ -297,17 +317,6 @@ private[kafka] object LogValidator extends Logging {
messageSizeMaybeChanged = true)
}
- private def ensureNonTransactional(batch: RecordBatch) {
- if (batch.isTransactional)
- throw new InvalidRecordException("Transactional messages are not currently supported")
- }
-
- private def ensureNotControlRecord(record: Record) {
- // Until we have implemented transaction support, we do not permit control records to be written
- if (record.isControlRecord)
- throw new InvalidRecordException("Control messages are not currently supported")
- }
-
private def validateKey(record: Record, compactedTopic: Boolean) {
if (compactedTopic && !record.hasKey)
throw new InvalidRecordException("Compacted topic cannot accept message without key.")
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/main/scala/kafka/log/OffsetIndex.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/OffsetIndex.scala b/core/src/main/scala/kafka/log/OffsetIndex.scala
index a54579f..e4939e8 100755
--- a/core/src/main/scala/kafka/log/OffsetIndex.scala
+++ b/core/src/main/scala/kafka/log/OffsetIndex.scala
@@ -85,7 +85,7 @@ class OffsetIndex(file: File, baseOffset: Long, maxIndexSize: Int = -1, writable
def lookup(targetOffset: Long): OffsetPosition = {
maybeLock(lock) {
val idx = mmap.duplicate
- val slot = indexSlotFor(idx, targetOffset, IndexSearchType.KEY)
+ val slot = largestLowerBoundSlotFor(idx, targetOffset, IndexSearchType.KEY)
if(slot == -1)
OffsetPosition(baseOffset, 0)
else
@@ -93,6 +93,22 @@ class OffsetIndex(file: File, baseOffset: Long, maxIndexSize: Int = -1, writable
}
}
+ /**
+ * Find an upper bound offset for the given fetch starting position and size. This is an offset which
+ * is guaranteed to be outside the fetched range, but note that it will not generally be the smallest
+ * such offset.
+ */
+ def fetchUpperBoundOffset(fetchOffset: OffsetPosition, fetchSize: Int): Option[OffsetPosition] = {
+ maybeLock(lock) {
+ val idx = mmap.duplicate
+ val slot = smallestUpperBoundSlotFor(idx, fetchOffset.position + fetchSize, IndexSearchType.VALUE)
+ if (slot == -1)
+ None
+ else
+ Some(parseEntry(idx, slot).asInstanceOf[OffsetPosition])
+ }
+ }
+
private def relativeOffset(buffer: ByteBuffer, n: Int): Int = buffer.getInt(n * entrySize)
private def physical(buffer: ByteBuffer, n: Int): Int = buffer.getInt(n * entrySize + 4)
@@ -140,7 +156,7 @@ class OffsetIndex(file: File, baseOffset: Long, maxIndexSize: Int = -1, writable
override def truncateTo(offset: Long) {
inLock(lock) {
val idx = mmap.duplicate
- val slot = indexSlotFor(idx, offset, IndexSearchType.KEY)
+ val slot = largestLowerBoundSlotFor(idx, offset, IndexSearchType.KEY)
/* There are 3 cases for choosing the new size
* 1) if there is no entry in the index <= the offset, delete everything
[6/6] kafka git commit: KAFKA-5121;
Implement transaction index for KIP-98
Posted by jg...@apache.org.
KAFKA-5121; Implement transaction index for KIP-98
Author: Jason Gustafson <ja...@confluent.io>
Reviewers: Ismael Juma <is...@juma.me.uk>, Jun Rao <ju...@gmail.com>
Closes #2910 from hachikuji/eos-txn-index
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/e71dce89
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/e71dce89
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/e71dce89
Branch: refs/heads/trunk
Commit: e71dce89c0da50f3eccc47d0fc050c92d5a99b88
Parents: 29994dd
Author: Jason Gustafson <ja...@confluent.io>
Authored: Sat May 6 11:49:35 2017 -0700
Committer: Jason Gustafson <ja...@confluent.io>
Committed: Sat May 6 11:49:35 2017 -0700
----------------------------------------------------------------------
.../clients/consumer/internals/Fetcher.java | 79 +--
.../TransactionCoordinatorFencedException.java | 30 +
.../apache/kafka/common/protocol/Errors.java | 17 +-
.../apache/kafka/common/protocol/Protocol.java | 17 +-
.../record/AbstractLegacyRecordBatch.java | 2 +-
.../kafka/common/record/AbstractRecords.java | 9 +-
.../kafka/common/record/ControlRecordType.java | 24 +-
.../kafka/common/record/DefaultRecord.java | 50 +-
.../kafka/common/record/DefaultRecordBatch.java | 22 +-
.../common/record/EndTransactionMarker.java | 124 ++++
.../kafka/common/record/FileLogInputStream.java | 6 +
.../apache/kafka/common/record/FileRecords.java | 13 +-
.../kafka/common/record/MemoryRecords.java | 109 +++-
.../common/record/MemoryRecordsBuilder.java | 66 +-
.../org/apache/kafka/common/record/Record.java | 9 -
.../apache/kafka/common/record/RecordBatch.java | 8 +
.../common/requests/ListOffsetRequest.java | 34 +-
.../org/apache/kafka/common/utils/Utils.java | 5 +
.../clients/consumer/internals/FetcherTest.java | 97 +--
.../common/record/DefaultRecordBatchTest.java | 31 +-
.../kafka/common/record/DefaultRecordTest.java | 45 +-
.../common/record/EndTransactionMarkerTest.java | 70 +++
.../kafka/common/record/FileRecordsTest.java | 11 +-
.../common/record/MemoryRecordsBuilderTest.java | 93 ++-
.../kafka/common/record/MemoryRecordsTest.java | 80 ++-
.../common/requests/RequestResponseTest.java | 22 +-
.../main/scala/kafka/cluster/Partition.scala | 4 +-
core/src/main/scala/kafka/cluster/Replica.scala | 23 +-
.../group/GroupMetadataManager.scala | 27 +-
.../transaction/TransactionStateManager.scala | 19 +-
.../main/scala/kafka/log/AbstractIndex.scala | 23 +-
core/src/main/scala/kafka/log/Log.scala | 611 ++++++++++++-------
core/src/main/scala/kafka/log/LogCleaner.scala | 34 +-
core/src/main/scala/kafka/log/LogManager.scala | 6 +-
core/src/main/scala/kafka/log/LogSegment.scala | 132 ++--
.../src/main/scala/kafka/log/LogValidator.scala | 83 +--
core/src/main/scala/kafka/log/OffsetIndex.scala | 20 +-
.../scala/kafka/log/ProducerIdMapping.scala | 384 ------------
.../scala/kafka/log/ProducerStateManager.scala | 590 ++++++++++++++++++
core/src/main/scala/kafka/log/TimeIndex.scala | 7 +-
.../main/scala/kafka/log/TransactionIndex.scala | 243 ++++++++
.../main/scala/kafka/server/DelayedFetch.scala | 14 +-
.../main/scala/kafka/server/FetchDataInfo.scala | 4 +-
.../src/main/scala/kafka/server/KafkaApis.scala | 67 +-
.../scala/kafka/server/LogOffsetMetadata.scala | 8 +-
.../scala/kafka/server/ReplicaManager.scala | 31 +-
.../scala/kafka/tools/DumpLogSegments.scala | 29 +-
.../kafka/api/AuthorizerIntegrationTest.scala | 2 +-
.../group/GroupCoordinatorResponseTest.scala | 6 +-
.../group/GroupMetadataManagerTest.scala | 22 +-
.../TransactionStateManagerTest.scala | 7 +-
.../unit/kafka/log/BrokerCompressionTest.scala | 2 +-
.../kafka/log/LogCleanerIntegrationTest.scala | 2 +-
.../log/LogCleanerLagIntegrationTest.scala | 3 +-
.../scala/unit/kafka/log/LogCleanerTest.scala | 2 +-
.../scala/unit/kafka/log/LogManagerTest.scala | 5 +-
.../scala/unit/kafka/log/LogSegmentTest.scala | 104 +++-
.../src/test/scala/unit/kafka/log/LogTest.scala | 555 +++++++++++++++--
.../scala/unit/kafka/log/LogValidatorTest.scala | 209 +++++--
.../scala/unit/kafka/log/OffsetIndexTest.scala | 24 +-
.../unit/kafka/log/ProducerIdMappingTest.scala | 291 ---------
.../kafka/log/ProducerStateManagerTest.scala | 562 +++++++++++++++++
.../unit/kafka/log/TransactionIndexTest.scala | 173 ++++++
.../scala/unit/kafka/server/LogOffsetTest.scala | 12 +-
.../kafka/server/ReplicaManagerQuotasTest.scala | 7 +-
.../unit/kafka/server/ReplicaManagerTest.scala | 151 ++++-
.../unit/kafka/server/RequestQuotaTest.scala | 7 +-
.../unit/kafka/server/SimpleFetchTest.scala | 7 +-
.../test/scala/unit/kafka/utils/TestUtils.scala | 2 +-
69 files changed, 4099 insertions(+), 1488 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
index 0c5c385..dc6c338 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
@@ -29,7 +29,6 @@ import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.errors.CorruptRecordException;
import org.apache.kafka.common.errors.InvalidMetadataException;
import org.apache.kafka.common.errors.InvalidTopicException;
import org.apache.kafka.common.errors.RecordTooLargeException;
@@ -669,7 +668,8 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
private RequestFuture<Map<TopicPartition, OffsetData>> sendListOffsetRequest(final Node node,
final Map<TopicPartition, Long> timestampsToSearch,
boolean requireTimestamp) {
- ListOffsetRequest.Builder builder = ListOffsetRequest.Builder.forConsumer(requireTimestamp)
+ ListOffsetRequest.Builder builder = ListOffsetRequest.Builder
+ .forConsumer(requireTimestamp, isolationLevel)
.setTargetTimes(timestampsToSearch);
log.trace("Sending ListOffsetRequest {} to broker {}", builder, node);
@@ -1003,12 +1003,18 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
return null;
}
currentBatch = batches.next();
-
maybeEnsureValid(currentBatch);
- if (isolationLevel == IsolationLevel.READ_COMMITTED && isBatchAborted(currentBatch)) {
- nextFetchOffset = currentBatch.lastOffset() + 1;
- continue;
+ if (isolationLevel == IsolationLevel.READ_COMMITTED && currentBatch.hasProducerId()) {
+ long producerId = currentBatch.producerId();
+ if (containsAbortMarker(currentBatch)) {
+ abortedProducerIds.remove(producerId);
+ } else if (isBatchAborted(currentBatch)) {
+ log.trace("Skipping aborted record batch with producerId {} and base offset {}, partition: {}",
+ producerId, currentBatch.baseOffset(), partition);
+ nextFetchOffset = currentBatch.lastOffset() + 1;
+ continue;
+ }
}
records = currentBatch.streamingIterator();
@@ -1022,7 +1028,7 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
nextFetchOffset = record.offset() + 1;
// control records are not returned to the user
- if (!record.isControlRecord())
+ if (!currentBatch.isControlBatch())
return record;
}
}
@@ -1046,7 +1052,7 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
}
private boolean isBatchAborted(RecordBatch batch) {
- /* When in READ_COMMITTED mode, we need to do the following for each incoming entry:
+ /* When in READ_COMMITTED mode, we need to do the following for each incoming entry:
* 0. Check whether the pid is in the 'abortedProducerIds' set && the entry does not include an abort marker.
* If so, skip the entry.
* 1. If the pid is in aborted pids and the entry contains an abort marker, remove the pid from
@@ -1056,47 +1062,48 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
* this means that the entry has been aborted. Add the pid to the aborted pids set, and remove
* the entry from the abort index.
*/
- FetchResponse.AbortedTransaction nextAbortedTransaction = abortedTransactions.peek();
- if (abortedProducerIds.contains(batch.producerId())
- || (nextAbortedTransaction != null && nextAbortedTransaction.producerId == batch.producerId() && nextAbortedTransaction.firstOffset <= batch.baseOffset())) {
- if (abortedProducerIds.contains(batch.producerId()) && containsAbortMarker(batch)) {
- abortedProducerIds.remove(batch.producerId());
- } else if (nextAbortedTransaction != null && nextAbortedTransaction.producerId == batch.producerId() && nextAbortedTransaction.firstOffset <= batch.baseOffset()) {
- abortedProducerIds.add(batch.producerId());
+ long producerId = batch.producerId();
+ if (abortedProducerIds.contains(producerId)) {
+ return true;
+ } else if (abortedTransactions != null && !abortedTransactions.isEmpty()) {
+ FetchResponse.AbortedTransaction nextAbortedTransaction = abortedTransactions.peek();
+ if (nextAbortedTransaction.producerId == producerId && nextAbortedTransaction.firstOffset <= batch.baseOffset()) {
+ abortedProducerIds.add(producerId);
abortedTransactions.poll();
+ return true;
}
- log.trace("Skipping aborted record batch with producerId {} and base offset {}, partition: {}", batch.producerId(), batch.baseOffset(), partition);
- return true;
}
return false;
}
private PriorityQueue<FetchResponse.AbortedTransaction> abortedTransactions(FetchResponse.PartitionData partition) {
- PriorityQueue<FetchResponse.AbortedTransaction> abortedTransactions = null;
- if (partition.abortedTransactions != null && !partition.abortedTransactions.isEmpty()) {
- abortedTransactions = new PriorityQueue<>(
- partition.abortedTransactions.size(),
- new Comparator<FetchResponse.AbortedTransaction>() {
- @Override
- public int compare(FetchResponse.AbortedTransaction o1, FetchResponse.AbortedTransaction o2) {
- return Long.compare(o1.firstOffset, o2.firstOffset);
- }
+ if (partition.abortedTransactions == null || partition.abortedTransactions.isEmpty())
+ return null;
+
+ PriorityQueue<FetchResponse.AbortedTransaction> abortedTransactions = new PriorityQueue<>(
+ partition.abortedTransactions.size(),
+ new Comparator<FetchResponse.AbortedTransaction>() {
+ @Override
+ public int compare(FetchResponse.AbortedTransaction o1, FetchResponse.AbortedTransaction o2) {
+ return Long.compare(o1.firstOffset, o2.firstOffset);
}
- );
- abortedTransactions.addAll(partition.abortedTransactions);
- } else {
- abortedTransactions = new PriorityQueue<>();
- }
+ }
+ );
+ abortedTransactions.addAll(partition.abortedTransactions);
return abortedTransactions;
}
private boolean containsAbortMarker(RecordBatch batch) {
+ if (!batch.isControlBatch())
+ return false;
+
Iterator<Record> batchIterator = batch.iterator();
- Record firstRecord = batchIterator.hasNext() ? batchIterator.next() : null;
- boolean containsAbortMarker = firstRecord != null && firstRecord.isControlRecord() && ControlRecordType.ABORT == ControlRecordType.parse(firstRecord.key());
- if (containsAbortMarker && batchIterator.hasNext())
- throw new CorruptRecordException("A record batch containing a control message contained more than one record. partition: " + partition + ", offset: " + batch.baseOffset());
- return containsAbortMarker;
+ if (!batchIterator.hasNext())
+ throw new InvalidRecordException("Invalid batch for partition " + partition + " at offset " +
+ batch.baseOffset() + " with control sequence set, but no records");
+
+ Record firstRecord = batchIterator.next();
+ return ControlRecordType.ABORT == ControlRecordType.parse(firstRecord.key());
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/main/java/org/apache/kafka/common/errors/TransactionCoordinatorFencedException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/TransactionCoordinatorFencedException.java b/clients/src/main/java/org/apache/kafka/common/errors/TransactionCoordinatorFencedException.java
new file mode 100644
index 0000000..583ce04
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/errors/TransactionCoordinatorFencedException.java
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+package org.apache.kafka.common.errors;
+
+public class TransactionCoordinatorFencedException extends ApiException {
+
+ private static final long serialVersionUID = 1L;
+
+ public TransactionCoordinatorFencedException(String message) {
+ super(message);
+ }
+
+ public TransactionCoordinatorFencedException(String message, Throwable cause) {
+ super(message, cause);
+ }
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
index 65bec4a..960fdda 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
@@ -21,11 +21,11 @@ import org.apache.kafka.common.errors.BrokerNotAvailableException;
import org.apache.kafka.common.errors.ClusterAuthorizationException;
import org.apache.kafka.common.errors.ConcurrentTransactionsException;
import org.apache.kafka.common.errors.ControllerMovedException;
+import org.apache.kafka.common.errors.CoordinatorLoadInProgressException;
+import org.apache.kafka.common.errors.CoordinatorNotAvailableException;
import org.apache.kafka.common.errors.CorruptRecordException;
import org.apache.kafka.common.errors.DuplicateSequenceNumberException;
import org.apache.kafka.common.errors.GroupAuthorizationException;
-import org.apache.kafka.common.errors.CoordinatorNotAvailableException;
-import org.apache.kafka.common.errors.CoordinatorLoadInProgressException;
import org.apache.kafka.common.errors.IllegalGenerationException;
import org.apache.kafka.common.errors.IllegalSaslStateException;
import org.apache.kafka.common.errors.InconsistentGroupProtocolException;
@@ -39,12 +39,11 @@ import org.apache.kafka.common.errors.InvalidReplicaAssignmentException;
import org.apache.kafka.common.errors.InvalidReplicationFactorException;
import org.apache.kafka.common.errors.InvalidRequestException;
import org.apache.kafka.common.errors.InvalidRequiredAcksException;
-import org.apache.kafka.common.errors.InvalidTxnTimeoutException;
-import org.apache.kafka.common.errors.OutOfOrderSequenceException;
import org.apache.kafka.common.errors.InvalidSessionTimeoutException;
import org.apache.kafka.common.errors.InvalidTimestampException;
import org.apache.kafka.common.errors.InvalidTopicException;
import org.apache.kafka.common.errors.InvalidTxnStateException;
+import org.apache.kafka.common.errors.InvalidTxnTimeoutException;
import org.apache.kafka.common.errors.LeaderNotAvailableException;
import org.apache.kafka.common.errors.NetworkException;
import org.apache.kafka.common.errors.NotControllerException;
@@ -54,6 +53,7 @@ import org.apache.kafka.common.errors.NotEnoughReplicasException;
import org.apache.kafka.common.errors.NotLeaderForPartitionException;
import org.apache.kafka.common.errors.OffsetMetadataTooLarge;
import org.apache.kafka.common.errors.OffsetOutOfRangeException;
+import org.apache.kafka.common.errors.OutOfOrderSequenceException;
import org.apache.kafka.common.errors.PolicyViolationException;
import org.apache.kafka.common.errors.ProducerFencedException;
import org.apache.kafka.common.errors.RebalanceInProgressException;
@@ -64,6 +64,7 @@ import org.apache.kafka.common.errors.RetriableException;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.errors.TopicAuthorizationException;
import org.apache.kafka.common.errors.TopicExistsException;
+import org.apache.kafka.common.errors.TransactionCoordinatorFencedException;
import org.apache.kafka.common.errors.UnknownMemberIdException;
import org.apache.kafka.common.errors.UnknownServerException;
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
@@ -460,6 +461,14 @@ public enum Errors {
public ApiException build(String message) {
return new ConcurrentTransactionsException(message);
}
+ }),
+ TRANSACTION_COORDINATOR_FENCED(52, "Indicates that the transaction coordinator sending a WriteTxnMarker " +
+ "is no longer the current coordinator for a given producer",
+ new ApiExceptionBuilder() {
+ @Override
+ public ApiException build(String message) {
+ return new TransactionCoordinatorFencedException(message);
+ }
});
private interface ApiExceptionBuilder {
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
index 16ec9ea..fb3c8c9 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
@@ -467,8 +467,21 @@ public class Protocol {
new ArrayOf(LIST_OFFSET_REQUEST_TOPIC_V1),
"Topics to list offsets."));
- /* v2 request is the same as v1. Throttle time has been added to response */
- public static final Schema LIST_OFFSET_REQUEST_V2 = LIST_OFFSET_REQUEST_V1;
+ public static final Schema LIST_OFFSET_REQUEST_V2 = new Schema(
+ new Field("replica_id",
+ INT32,
+ "Broker id of the follower. For normal consumers, use -1."),
+ new Field("isolation_level",
+ INT8,
+ "This setting controls the visibility of transactional records. Using READ_UNCOMMITTED " +
+ "(isolation_level = 0) makes all records visible. With READ_COMMITTED (isolation_level = 1), " +
+ "non-transactional and COMMITTED transactional records are visible. To be more concrete, " +
+ "READ_COMMITTED returns all data from offsets smaller than the current LSO (last stable offset), " +
+ "and enables the inclusion of the list of aborted transactions in the result, which allows " +
+ "consumers to discard ABORTED transactional records"),
+ new Field("topics",
+ new ArrayOf(LIST_OFFSET_REQUEST_TOPIC_V1),
+ "Topics to list offsets."));;
public static final Schema LIST_OFFSET_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
INT32,
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java
index 85fcb2a..7be4bdd 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java
@@ -206,7 +206,7 @@ public abstract class AbstractLegacyRecordBatch extends AbstractRecordBatch impl
}
@Override
- public boolean isControlRecord() {
+ public boolean isControlBatch() {
return false;
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java b/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java
index 87df7e4..cfda8a4 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java
@@ -56,6 +56,9 @@ public abstract class AbstractRecords implements Records {
int totalSizeEstimate = 0;
for (RecordBatch batch : batches) {
+ if (toMagic < RecordBatch.MAGIC_VALUE_V2 && batch.isControlBatch())
+ continue;
+
if (batch.magic() <= toMagic) {
totalSizeEstimate += batch.sizeInBytes();
recordBatchAndRecordsList.add(new RecordBatchAndRecords(batch, null, null));
@@ -94,12 +97,8 @@ public abstract class AbstractRecords implements Records {
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, batch.compressionType(),
timestampType, recordBatchAndRecords.baseOffset, logAppendTime);
- for (Record record : recordBatchAndRecords.records) {
- // control messages are only supported in v2 and above, so skip when down-converting
- if (magic < RecordBatch.MAGIC_VALUE_V2 && record.isControlRecord())
- continue;
+ for (Record record : recordBatchAndRecords.records)
builder.append(record);
- }
builder.close();
return builder.buffer();
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/main/java/org/apache/kafka/common/record/ControlRecordType.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/ControlRecordType.java b/clients/src/main/java/org/apache/kafka/common/record/ControlRecordType.java
index 790b2ee..d5ead14 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/ControlRecordType.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/ControlRecordType.java
@@ -49,6 +49,7 @@ public enum ControlRecordType {
private static final Logger log = LoggerFactory.getLogger(ControlRecordType.class);
static final short CURRENT_CONTROL_RECORD_KEY_VERSION = 0;
+ static final int CURRENT_CONTROL_RECORD_KEY_SIZE = 4;
private static final Schema CONTROL_RECORD_KEY_SCHEMA_VERSION_V0 = new Schema(
new Field("version", Type.INT16),
new Field("type", Type.INT16));
@@ -69,13 +70,24 @@ public enum ControlRecordType {
return struct;
}
- public static ControlRecordType parse(ByteBuffer key) {
+ public static short parseTypeId(ByteBuffer key) {
+ if (key.remaining() < CURRENT_CONTROL_RECORD_KEY_SIZE)
+ throw new InvalidRecordException("Invalid value size found for end control record key. Must have " +
+ "at least " + CURRENT_CONTROL_RECORD_KEY_SIZE + " bytes, but found only " + key.remaining());
+
short version = key.getShort(0);
+ if (version < 0)
+ throw new InvalidRecordException("Invalid version found for control record: " + version +
+ ". May indicate data corruption");
+
if (version != CURRENT_CONTROL_RECORD_KEY_VERSION)
- log.debug("Received unknown control record key version {}. Parsing as version {}", version,
+ log.debug("Received unknown control record key version {}. Parsing as version {}", version,
CURRENT_CONTROL_RECORD_KEY_VERSION);
- short type = key.getShort(2);
- switch (type) {
+ return key.getShort(2);
+ }
+
+ public static ControlRecordType fromTypeId(short typeId) {
+ switch (typeId) {
case 0:
return ABORT;
case 1:
@@ -84,4 +96,8 @@ public enum ControlRecordType {
return UNKNOWN;
}
}
+
+ public static ControlRecordType parse(ByteBuffer key) {
+ return fromTypeId(parseTypeId(key));
+ }
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java
index e0794d8..669c75d 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java
@@ -55,11 +55,9 @@ import static org.apache.kafka.common.utils.Utils.wrapNullable;
*
* The current record attributes are depicted below:
*
- * -----------------------------------
- * | Unused (1-7) | Control Flag (0) |
- * -----------------------------------
- *
- * The control flag is used to implement control records (see {@link ControlRecordType}).
+ * ----------------
+ * | Unused (0-7) |
+ * ----------------
*
* The offset and timestamp deltas compute the difference relative to the base offset and
* base timestamp of the log entry that this record is contained in.
@@ -69,7 +67,6 @@ public class DefaultRecord implements Record {
// excluding key, value and headers: 5 bytes length + 10 bytes timestamp + 5 bytes offset + 1 byte attributes
public static final int MAX_RECORD_OVERHEAD = 21;
- private static final int CONTROL_FLAG_MASK = 0x01;
private static final int NULL_VARINT_SIZE_BYTES = ByteUtils.sizeOfVarint(-1);
private final int sizeInBytes;
@@ -180,7 +177,6 @@ public class DefaultRecord implements Record {
* Write the record to `out` and return its crc.
*/
public static long writeTo(DataOutputStream out,
- boolean isControlRecord,
int offsetDelta,
long timestampDelta,
ByteBuffer key,
@@ -189,7 +185,7 @@ public class DefaultRecord implements Record {
int sizeInBytes = sizeOfBodyInBytes(offsetDelta, timestampDelta, key, value, headers);
ByteUtils.writeVarint(sizeInBytes, out);
- byte attributes = computeAttributes(isControlRecord);
+ byte attributes = 0; // there are no used record attributes at the moment
out.write(attributes);
ByteUtils.writeVarlong(timestampDelta, out);
@@ -241,15 +237,14 @@ public class DefaultRecord implements Record {
* Write the record to `out` and return its crc.
*/
public static long writeTo(ByteBuffer out,
- boolean isControlRecord,
int offsetDelta,
long timestampDelta,
ByteBuffer key,
ByteBuffer value,
Header[] headers) {
try {
- return writeTo(new DataOutputStream(new ByteBufferOutputStream(out)), isControlRecord, offsetDelta,
- timestampDelta, key, value, headers);
+ return writeTo(new DataOutputStream(new ByteBufferOutputStream(out)), offsetDelta, timestampDelta,
+ key, value, headers);
} catch (IOException e) {
// cannot actually be raised by ByteBufferOutputStream
throw new IllegalStateException("Unexpected exception raised from ByteBufferOutputStream", e);
@@ -290,11 +285,6 @@ public class DefaultRecord implements Record {
}
@Override
- public boolean isControlRecord() {
- return (attributes & CONTROL_FLAG_MASK) != 0;
- }
-
- @Override
public String toString() {
return String.format("DefaultRecord(offset=%d, timestamp=%d, key=%d bytes, value=%d bytes)",
offset,
@@ -421,10 +411,6 @@ public class DefaultRecord implements Record {
return new DefaultRecord(sizeInBytes, attributes, offset, timestamp, sequence, key, value, headers);
}
- private static byte computeAttributes(boolean isControlRecord) {
- return isControlRecord ? CONTROL_FLAG_MASK : (byte) 0;
- }
-
public static int sizeInBytes(int offsetDelta,
long timestampDelta,
byte[] key,
@@ -441,19 +427,35 @@ public class DefaultRecord implements Record {
return bodySize + ByteUtils.sizeOfVarint(bodySize);
}
+ public static int sizeInBytes(int offsetDelta,
+ long timestampDelta,
+ int keySize,
+ int valueSize,
+ Header[] headers) {
+ int bodySize = sizeOfBodyInBytes(offsetDelta, timestampDelta, keySize, valueSize, headers);
+ return bodySize + ByteUtils.sizeOfVarint(bodySize);
+ }
+
private static int sizeOfBodyInBytes(int offsetDelta,
long timestampDelta,
ByteBuffer key,
ByteBuffer value,
Header[] headers) {
- int size = 1; // always one byte for attributes
- size += ByteUtils.sizeOfVarint(offsetDelta);
- size += ByteUtils.sizeOfVarlong(timestampDelta);
int keySize = key == null ? -1 : key.remaining();
int valueSize = value == null ? -1 : value.remaining();
- size += sizeOf(keySize, valueSize, headers);
+ return sizeOfBodyInBytes(offsetDelta, timestampDelta, keySize, valueSize, headers);
+ }
+ private static int sizeOfBodyInBytes(int offsetDelta,
+ long timestampDelta,
+ int keySize,
+ int valueSize,
+ Header[] headers) {
+ int size = 1; // always one byte for attributes
+ size += ByteUtils.sizeOfVarint(offsetDelta);
+ size += ByteUtils.sizeOfVarlong(timestampDelta);
+ size += sizeOf(keySize, valueSize, headers);
return size;
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java
index 93cd2eb..f321c3b 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java
@@ -62,9 +62,9 @@ import static org.apache.kafka.common.record.Records.LOG_OVERHEAD;
*
* The current attributes are given below:
*
- * -----------------------------------------------------------------------------------
- * | Unused (5-15) | Transactional (4) | Timestamp Type (3) | Compression Type (0-2) |
- * -----------------------------------------------------------------------------------
+ * -------------------------------------------------------------------------------------------------
+ * | Unused (6-15) | Control (5) | Transactional (4) | Timestamp Type (3) | Compression Type (0-2) |
+ * -------------------------------------------------------------------------------------------------
*/
public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRecordBatch {
static final int BASE_OFFSET_OFFSET = 0;
@@ -98,6 +98,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe
private static final byte COMPRESSION_CODEC_MASK = 0x07;
private static final byte TRANSACTIONAL_FLAG_MASK = 0x10;
+ private static final int CONTROL_FLAG_MASK = 0x20;
private static final byte TIMESTAMP_TYPE_MASK = 0x08;
private final ByteBuffer buffer;
@@ -203,6 +204,11 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe
}
@Override
+ public boolean isControlBatch() {
+ return (attributes() & CONTROL_FLAG_MASK) > 0;
+ }
+
+ @Override
public int partitionLeaderEpoch() {
return buffer.getInt(PARTITION_LEADER_EPOCH_OFFSET);
}
@@ -284,7 +290,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe
if (timestampType() == timestampType && currentMaxTimestamp == maxTimestamp)
return;
- byte attributes = computeAttributes(compressionType(), timestampType, isTransactional());
+ byte attributes = computeAttributes(compressionType(), timestampType, isTransactional(), isControlBatch());
buffer.putShort(ATTRIBUTES_OFFSET, attributes);
buffer.putLong(MAX_TIMESTAMP_OFFSET, maxTimestamp);
long crc = computeChecksum();
@@ -330,12 +336,15 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe
return buffer != null ? buffer.hashCode() : 0;
}
- private static byte computeAttributes(CompressionType type, TimestampType timestampType, boolean isTransactional) {
+ private static byte computeAttributes(CompressionType type, TimestampType timestampType,
+ boolean isTransactional, boolean isControl) {
if (timestampType == TimestampType.NO_TIMESTAMP_TYPE)
throw new IllegalArgumentException("Timestamp type must be provided to compute attributes for message " +
"format v2 and above");
byte attributes = isTransactional ? TRANSACTIONAL_FLAG_MASK : 0;
+ if (isControl)
+ attributes |= CONTROL_FLAG_MASK;
if (type.id > 0)
attributes |= COMPRESSION_CODEC_MASK & type.id;
if (timestampType == TimestampType.LOG_APPEND_TIME)
@@ -356,6 +365,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe
short epoch,
int sequence,
boolean isTransactional,
+ boolean isControlBatch,
int partitionLeaderEpoch,
int numRecords) {
if (magic < RecordBatch.CURRENT_MAGIC_VALUE)
@@ -363,7 +373,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe
if (baseTimestamp < 0 && baseTimestamp != NO_TIMESTAMP)
throw new IllegalArgumentException("Invalid message timestamp " + baseTimestamp);
- short attributes = computeAttributes(compressionType, timestampType, isTransactional);
+ short attributes = computeAttributes(compressionType, timestampType, isTransactional, isControlBatch);
int position = buffer.position();
buffer.putLong(position + BASE_OFFSET_OFFSET, baseOffset);
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/main/java/org/apache/kafka/common/record/EndTransactionMarker.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/EndTransactionMarker.java b/clients/src/main/java/org/apache/kafka/common/record/EndTransactionMarker.java
new file mode 100644
index 0000000..726b52a
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/record/EndTransactionMarker.java
@@ -0,0 +1,124 @@
+/*
+ * 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.
+ */
+package org.apache.kafka.common.record;
+
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This class represents the control record which is written to the log to indicate the completion
+ * of a transaction. The record key specifies the {@link ControlRecordType control type} and the
+ * value embeds information useful for write validation (for now, just the coordinator epoch).
+ */
+public class EndTransactionMarker {
+ private static final Logger log = LoggerFactory.getLogger(EndTransactionMarker.class);
+
+ private static final short CURRENT_END_TXN_MARKER_VERSION = 0;
+ private static final Schema END_TXN_MARKER_SCHEMA_VERSION_V0 = new Schema(
+ new Field("version", Type.INT16),
+ new Field("coordinator_epoch", Type.INT32));
+ static final int CURRENT_END_TXN_MARKER_VALUE_SIZE = 6;
+ static final int CURRENT_END_TXN_SCHEMA_RECORD_SIZE = DefaultRecord.sizeInBytes(0, 0L,
+ ControlRecordType.CURRENT_CONTROL_RECORD_KEY_SIZE,
+ EndTransactionMarker.CURRENT_END_TXN_MARKER_VALUE_SIZE,
+ Record.EMPTY_HEADERS);
+
+ private final ControlRecordType type;
+ private final int coordinatorEpoch;
+
+ public EndTransactionMarker(ControlRecordType type, int coordinatorEpoch) {
+ ensureTransactionMarkerControlType(type);
+ this.type = type;
+ this.coordinatorEpoch = coordinatorEpoch;
+ }
+
+ public int coordinatorEpoch() {
+ return coordinatorEpoch;
+ }
+
+ public ControlRecordType controlType() {
+ return type;
+ }
+
+ private Struct buildRecordValue() {
+ Struct struct = new Struct(END_TXN_MARKER_SCHEMA_VERSION_V0);
+ struct.set("version", CURRENT_END_TXN_MARKER_VERSION);
+ struct.set("coordinator_epoch", coordinatorEpoch);
+ return struct;
+ }
+
+ public ByteBuffer serializeValue() {
+ Struct valueStruct = buildRecordValue();
+ ByteBuffer value = ByteBuffer.allocate(valueStruct.sizeOf());
+ valueStruct.writeTo(value);
+ value.flip();
+ return value;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+
+ EndTransactionMarker that = (EndTransactionMarker) o;
+ return coordinatorEpoch == that.coordinatorEpoch && type == that.type;
+ }
+
+ @Override
+ public int hashCode() {
+ int result = type != null ? type.hashCode() : 0;
+ result = 31 * result + coordinatorEpoch;
+ return result;
+ }
+
+ private static void ensureTransactionMarkerControlType(ControlRecordType type) {
+ if (type != ControlRecordType.COMMIT && type != ControlRecordType.ABORT)
+ throw new IllegalArgumentException("Invalid control record type for end transaction marker" + type);
+ }
+
+ public static EndTransactionMarker deserialize(Record record) {
+ ControlRecordType type = ControlRecordType.parse(record.key());
+ return deserializeValue(type, record.value());
+ }
+
+ static EndTransactionMarker deserializeValue(ControlRecordType type, ByteBuffer value) {
+ ensureTransactionMarkerControlType(type);
+
+ if (value.remaining() < CURRENT_END_TXN_MARKER_VALUE_SIZE)
+ throw new InvalidRecordException("Invalid value size found for end transaction marker. Must have " +
+ "at least " + CURRENT_END_TXN_MARKER_VALUE_SIZE + " bytes, but found only " + value.remaining());
+
+ short version = value.getShort(0);
+ if (version < 0)
+ throw new InvalidRecordException("Invalid version found for end transaction marker: " + version +
+ ". May indicate data corruption");
+
+ if (version > CURRENT_END_TXN_MARKER_VERSION)
+ log.debug("Received end transaction marker value version {}. Parsing as version {}", version,
+ CURRENT_END_TXN_MARKER_VERSION);
+
+ int coordinatorEpoch = value.getInt(2);
+ return new EndTransactionMarker(type, coordinatorEpoch);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java
index d5f10dc..1af5527 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java
@@ -279,6 +279,12 @@ public class FileLogInputStream implements LogInputStream<FileLogInputStream.Fil
}
@Override
+ public boolean isControlBatch() {
+ loadUnderlyingRecordBatch();
+ return underlying.isControlBatch();
+ }
+
+ @Override
public int partitionLeaderEpoch() {
loadUnderlyingRecordBatch();
return underlying.partitionLeaderEpoch();
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java
index dcd7845..16d3777 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java
@@ -224,7 +224,6 @@ public class FileRecords extends AbstractRecords implements Closeable {
" size of this log segment is " + originalSize + " bytes.");
if (targetSize < (int) channel.size()) {
channel.truncate(targetSize);
- channel.position(targetSize);
size.set(targetSize);
}
return originalSize - targetSize;
@@ -276,11 +275,11 @@ public class FileRecords extends AbstractRecords implements Closeable {
* @param targetOffset The offset to search for.
* @param startingPosition The starting position in the file to begin searching from.
*/
- public LogEntryPosition searchForOffsetWithSize(long targetOffset, int startingPosition) {
+ public LogOffsetPosition searchForOffsetWithSize(long targetOffset, int startingPosition) {
for (FileChannelRecordBatch batch : batchesFrom(startingPosition)) {
long offset = batch.lastOffset();
if (offset >= targetOffset)
- return new LogEntryPosition(offset, batch.position(), batch.sizeInBytes());
+ return new LogOffsetPosition(offset, batch.position(), batch.sizeInBytes());
}
return null;
}
@@ -429,12 +428,12 @@ public class FileRecords extends AbstractRecords implements Closeable {
}
}
- public static class LogEntryPosition {
+ public static class LogOffsetPosition {
public final long offset;
public final int position;
public final int size;
- public LogEntryPosition(long offset, int position, int size) {
+ public LogOffsetPosition(long offset, int position, int size) {
this.offset = offset;
this.position = position;
this.size = size;
@@ -447,7 +446,7 @@ public class FileRecords extends AbstractRecords implements Closeable {
if (o == null || getClass() != o.getClass())
return false;
- LogEntryPosition that = (LogEntryPosition) o;
+ LogOffsetPosition that = (LogOffsetPosition) o;
return offset == that.offset &&
position == that.position &&
@@ -465,7 +464,7 @@ public class FileRecords extends AbstractRecords implements Closeable {
@Override
public String toString() {
- return "LogEntryPosition(" +
+ return "LogOffsetPosition(" +
"offset=" + offset +
", position=" + position +
", size=" + size +
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
index 548cd45..c8754c7 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
@@ -292,6 +292,16 @@ public class MemoryRecords extends AbstractRecords {
return builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, timestampType, baseOffset);
}
+ public static MemoryRecordsBuilder idempotentBuilder(ByteBuffer buffer,
+ CompressionType compressionType,
+ long baseOffset,
+ long producerId,
+ short producerEpoch,
+ int baseSequence) {
+ return builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, TimestampType.CREATE_TIME,
+ baseOffset, System.currentTimeMillis(), producerId, producerEpoch, baseSequence);
+ }
+
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
byte magic,
CompressionType compressionType,
@@ -307,7 +317,8 @@ public class MemoryRecords extends AbstractRecords {
long baseOffset,
long logAppendTime) {
return builder(buffer, magic, compressionType, timestampType, baseOffset, logAppendTime,
- RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false, RecordBatch.NO_PARTITION_LEADER_EPOCH);
+ RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false,
+ RecordBatch.NO_PARTITION_LEADER_EPOCH);
}
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
@@ -320,7 +331,8 @@ public class MemoryRecords extends AbstractRecords {
if (timestampType == TimestampType.LOG_APPEND_TIME)
logAppendTime = System.currentTimeMillis();
return builder(buffer, magic, compressionType, timestampType, baseOffset, logAppendTime,
- RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, isTransactional, RecordBatch.NO_PARTITION_LEADER_EPOCH);
+ RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, isTransactional,
+ RecordBatch.NO_PARTITION_LEADER_EPOCH);
}
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
@@ -335,6 +347,18 @@ public class MemoryRecords extends AbstractRecords {
}
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
+ CompressionType compressionType,
+ long baseOffset,
+ long producerId,
+ short producerEpoch,
+ int baseSequence,
+ boolean isTransactional) {
+ return builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, TimestampType.CREATE_TIME, baseOffset,
+ RecordBatch.NO_TIMESTAMP, producerId, producerEpoch, baseSequence, isTransactional,
+ RecordBatch.NO_PARTITION_LEADER_EPOCH);
+ }
+
+ public static MemoryRecordsBuilder builder(ByteBuffer buffer,
byte magic,
CompressionType compressionType,
TimestampType timestampType,
@@ -359,18 +383,18 @@ public class MemoryRecords extends AbstractRecords {
boolean isTransactional,
int partitionLeaderEpoch) {
return new MemoryRecordsBuilder(buffer, magic, compressionType, timestampType, baseOffset,
- logAppendTime, producerId, producerEpoch, baseSequence, isTransactional, partitionLeaderEpoch,
+ logAppendTime, producerId, producerEpoch, baseSequence, isTransactional, false, partitionLeaderEpoch,
buffer.remaining());
}
-
public static MemoryRecords withRecords(CompressionType compressionType, SimpleRecord... records) {
return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, compressionType, records);
}
public static MemoryRecords withRecords(CompressionType compressionType, int partitionLeaderEpoch, SimpleRecord... records) {
- return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, 0L, compressionType, TimestampType.CREATE_TIME, RecordBatch.NO_PRODUCER_ID,
- RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, partitionLeaderEpoch, records);
+ return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, 0L, compressionType, TimestampType.CREATE_TIME,
+ RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
+ partitionLeaderEpoch, false, records);
}
public static MemoryRecords withRecords(byte magic, CompressionType compressionType, SimpleRecord... records) {
@@ -378,30 +402,52 @@ public class MemoryRecords extends AbstractRecords {
}
public static MemoryRecords withRecords(long initialOffset, CompressionType compressionType, SimpleRecord... records) {
- return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, initialOffset, compressionType, TimestampType.CREATE_TIME, records);
+ return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, initialOffset, compressionType, TimestampType.CREATE_TIME,
+ records);
}
public static MemoryRecords withRecords(long initialOffset, CompressionType compressionType, Integer partitionLeaderEpoch, SimpleRecord... records) {
return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, initialOffset, compressionType, TimestampType.CREATE_TIME, RecordBatch.NO_PRODUCER_ID,
- RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, partitionLeaderEpoch, records);
+ RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, partitionLeaderEpoch, false, records);
}
- public static MemoryRecords withRecords(byte magic, long initialOffset, CompressionType compressionType,
- long producerId, short producerEpoch, int baseSequence,
- int partitionLeaderEpoch, SimpleRecord... records) {
+ public static MemoryRecords withIdempotentRecords(byte magic, long initialOffset, CompressionType compressionType,
+ long producerId, short producerEpoch, int baseSequence,
+ int partitionLeaderEpoch, SimpleRecord... records) {
return withRecords(magic, initialOffset, compressionType, TimestampType.CREATE_TIME, producerId, producerEpoch,
- baseSequence, partitionLeaderEpoch, records);
+ baseSequence, partitionLeaderEpoch, false, records);
+ }
+
+ public static MemoryRecords withIdempotentRecords(long initialOffset, CompressionType compressionType, long producerId,
+ short producerEpoch, int baseSequence, int partitionLeaderEpoch,
+ SimpleRecord... records) {
+ return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, initialOffset, compressionType, TimestampType.CREATE_TIME,
+ producerId, producerEpoch, baseSequence, partitionLeaderEpoch, false, records);
+ }
+
+ public static MemoryRecords withTransactionalRecords(CompressionType compressionType, long producerId,
+ short producerEpoch, int baseSequence, SimpleRecord... records) {
+ return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, 0L, compressionType, TimestampType.CREATE_TIME,
+ producerId, producerEpoch, baseSequence, RecordBatch.NO_PARTITION_LEADER_EPOCH, true, records);
+ }
+
+ public static MemoryRecords withTransactionalRecords(long initialOffset, CompressionType compressionType, long producerId,
+ short producerEpoch, int baseSequence, SimpleRecord... records) {
+ return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, initialOffset, compressionType, TimestampType.CREATE_TIME,
+ producerId, producerEpoch, baseSequence, RecordBatch.NO_PARTITION_LEADER_EPOCH, true, records);
}
public static MemoryRecords withRecords(byte magic, long initialOffset, CompressionType compressionType,
TimestampType timestampType, SimpleRecord... records) {
return withRecords(magic, initialOffset, compressionType, timestampType, RecordBatch.NO_PRODUCER_ID,
- RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, RecordBatch.NO_PARTITION_LEADER_EPOCH, records);
+ RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ false, records);
}
- private static MemoryRecords withRecords(byte magic, long initialOffset, CompressionType compressionType,
- TimestampType timestampType, long producerId, short producerEpoch,
- int baseSequence, int partitionLeaderEpoch, SimpleRecord ... records) {
+ public static MemoryRecords withRecords(byte magic, long initialOffset, CompressionType compressionType,
+ TimestampType timestampType, long producerId, short producerEpoch,
+ int baseSequence, int partitionLeaderEpoch, boolean isTransactional,
+ SimpleRecord ... records) {
if (records.length == 0)
return MemoryRecords.EMPTY;
int sizeEstimate = AbstractRecords.estimateSizeInBytes(magic, compressionType, Arrays.asList(records));
@@ -409,11 +455,38 @@ public class MemoryRecords extends AbstractRecords {
long logAppendTime = RecordBatch.NO_TIMESTAMP;
if (timestampType == TimestampType.LOG_APPEND_TIME)
logAppendTime = System.currentTimeMillis();
- MemoryRecordsBuilder builder = builder(buffer, magic, compressionType, timestampType, initialOffset,
- logAppendTime, producerId, producerEpoch, baseSequence, false, partitionLeaderEpoch);
+ MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, magic, compressionType, timestampType,
+ initialOffset, logAppendTime, producerId, producerEpoch, baseSequence, isTransactional, false,
+ partitionLeaderEpoch, buffer.capacity());
for (SimpleRecord record : records)
builder.append(record);
return builder.build();
}
+ public static MemoryRecords withEndTransactionMarker(long producerId, short producerEpoch, EndTransactionMarker marker) {
+ return withEndTransactionMarker(0L, producerId, producerEpoch, marker);
+ }
+
+ public static MemoryRecords withEndTransactionMarker(long initialOffset, long producerId, short producerEpoch,
+ EndTransactionMarker marker) {
+ int endTxnMarkerBatchSize = DefaultRecordBatch.RECORD_BATCH_OVERHEAD +
+ EndTransactionMarker.CURRENT_END_TXN_SCHEMA_RECORD_SIZE;
+ ByteBuffer buffer = ByteBuffer.allocate(endTxnMarkerBatchSize);
+ writeEndTransactionalMarker(buffer, initialOffset, producerId, producerEpoch, marker);
+ buffer.flip();
+ return MemoryRecords.readableRecords(buffer);
+ }
+
+ public static void writeEndTransactionalMarker(ByteBuffer buffer, long initialOffset, long producerId,
+ short producerEpoch, EndTransactionMarker marker) {
+ boolean isTransactional = true;
+ boolean isControlBatch = true;
+ MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE,
+ TimestampType.CREATE_TIME, initialOffset, RecordBatch.NO_TIMESTAMP, producerId, producerEpoch,
+ RecordBatch.NO_SEQUENCE, isTransactional, isControlBatch, RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ buffer.capacity());
+ builder.appendEndTxnMarker(System.currentTimeMillis(), marker);
+ builder.close();
+ }
+
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java
index b9d65a5..f7451cf 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java
@@ -66,6 +66,7 @@ public class MemoryRecordsBuilder {
private final long baseOffset;
private final long logAppendTime;
private final boolean isTransactional;
+ private final boolean isControlBatch;
private final int partitionLeaderEpoch;
private final int writeLimit;
private final int initialCapacity;
@@ -112,17 +113,18 @@ public class MemoryRecordsBuilder {
short producerEpoch,
int baseSequence,
boolean isTransactional,
+ boolean isControlBatch,
int partitionLeaderEpoch,
int writeLimit) {
if (magic > RecordBatch.MAGIC_VALUE_V0 && timestampType == TimestampType.NO_TIMESTAMP_TYPE)
throw new IllegalArgumentException("TimestampType must be set for magic >= 0");
-
- if (isTransactional) {
- if (magic < RecordBatch.MAGIC_VALUE_V2)
- throw new IllegalArgumentException("Transactional messages are not supported for magic " + magic);
+ if (magic < RecordBatch.MAGIC_VALUE_V2) {
+ if (isTransactional)
+ throw new IllegalArgumentException("Transactional records are not supported for magic " + magic);
+ if (isControlBatch)
+ throw new IllegalArgumentException("Control records are not supported for magic " + magic);
}
-
this.magic = magic;
this.timestampType = timestampType;
this.compressionType = compressionType;
@@ -137,6 +139,7 @@ public class MemoryRecordsBuilder {
this.producerEpoch = producerEpoch;
this.baseSequence = baseSequence;
this.isTransactional = isTransactional;
+ this.isControlBatch = isControlBatch;
this.partitionLeaderEpoch = partitionLeaderEpoch;
this.writeLimit = writeLimit;
this.initialCapacity = buffer.capacity();
@@ -254,7 +257,7 @@ public class MemoryRecordsBuilder {
if (producerEpoch == RecordBatch.NO_PRODUCER_EPOCH)
throw new IllegalArgumentException("Invalid negative producer epoch");
- if (baseSequence == RecordBatch.NO_SEQUENCE)
+ if (baseSequence < 0 && !isControlBatch)
throw new IllegalArgumentException("Invalid negative sequence number used");
if (magic < RecordBatch.MAGIC_VALUE_V2)
@@ -298,7 +301,7 @@ public class MemoryRecordsBuilder {
}
DefaultRecordBatch.writeHeader(buffer, baseOffset, offsetDelta, size, magic, compressionType, timestampType,
- baseTimestamp, maxTimestamp, producerId, producerEpoch, baseSequence, isTransactional,
+ baseTimestamp, maxTimestamp, producerId, producerEpoch, baseSequence, isTransactional, isControlBatch,
partitionLeaderEpoch, numRecords);
buffer.position(pos);
@@ -326,26 +329,26 @@ public class MemoryRecordsBuilder {
}
private long appendWithOffset(long offset, boolean isControlRecord, long timestamp, ByteBuffer key,
- ByteBuffer value, Header[] headers) {
+ ByteBuffer value, Header[] headers) {
try {
+ if (isControlRecord != isControlBatch)
+ throw new IllegalArgumentException("Control records can only be appended to control batches");
+
if (lastOffset != null && offset <= lastOffset)
- throw new IllegalArgumentException(String.format("Illegal offset %s following previous offset %s (Offsets must increase monotonically).", offset, lastOffset));
+ throw new IllegalArgumentException(String.format("Illegal offset %s following previous offset %s " +
+ "(Offsets must increase monotonically).", offset, lastOffset));
if (timestamp < 0 && timestamp != RecordBatch.NO_TIMESTAMP)
throw new IllegalArgumentException("Invalid negative timestamp " + timestamp);
- if (magic < RecordBatch.MAGIC_VALUE_V2) {
- if (isControlRecord)
- throw new IllegalArgumentException("Magic v" + magic + " does not support control records");
- if (headers != null && headers.length > 0)
- throw new IllegalArgumentException("Magic v" + magic + " does not support record headers");
- }
+ if (magic < RecordBatch.MAGIC_VALUE_V2 && headers != null && headers.length > 0)
+ throw new IllegalArgumentException("Magic v" + magic + " does not support record headers");
if (baseTimestamp == null)
baseTimestamp = timestamp;
if (magic > RecordBatch.MAGIC_VALUE_V1)
- return appendDefaultRecord(offset, isControlRecord, timestamp, key, value, headers);
+ return appendDefaultRecord(offset, timestamp, key, value, headers);
else
return appendLegacyRecord(offset, timestamp, key, value);
} catch (IOException e) {
@@ -388,7 +391,7 @@ public class MemoryRecordsBuilder {
* @return crc of the record
*/
public long appendWithOffset(long offset, long timestamp, byte[] key, byte[] value) {
- return appendWithOffset(offset, false, timestamp, wrapNullable(key), wrapNullable(value), Record.EMPTY_HEADERS);
+ return appendWithOffset(offset, timestamp, wrapNullable(key), wrapNullable(value), Record.EMPTY_HEADERS);
}
/**
@@ -400,7 +403,7 @@ public class MemoryRecordsBuilder {
* @return crc of the record
*/
public long appendWithOffset(long offset, long timestamp, ByteBuffer key, ByteBuffer value) {
- return appendWithOffset(offset, false, timestamp, key, value, Record.EMPTY_HEADERS);
+ return appendWithOffset(offset, timestamp, key, value, Record.EMPTY_HEADERS);
}
/**
@@ -410,7 +413,7 @@ public class MemoryRecordsBuilder {
* @return crc of the record
*/
public long appendWithOffset(long offset, SimpleRecord record) {
- return appendWithOffset(offset, false, record.timestamp(), record.key(), record.value(), record.headers());
+ return appendWithOffset(offset, record.timestamp(), record.key(), record.value(), record.headers());
}
@@ -434,7 +437,7 @@ public class MemoryRecordsBuilder {
* @return crc of the record
*/
public long append(long timestamp, ByteBuffer key, ByteBuffer value, Header[] headers) {
- return appendWithOffset(nextSequentialOffset(), false, timestamp, key, value, headers);
+ return appendWithOffset(nextSequentialOffset(), timestamp, key, value, headers);
}
/**
@@ -476,7 +479,7 @@ public class MemoryRecordsBuilder {
* @param value The control record value
* @return crc of the record
*/
- public long appendControlRecord(long timestamp, ControlRecordType type, ByteBuffer value) {
+ private long appendControlRecord(long timestamp, ControlRecordType type, ByteBuffer value) {
Struct keyStruct = type.recordKey();
ByteBuffer key = ByteBuffer.allocate(keyStruct.sizeOf());
keyStruct.writeTo(key);
@@ -484,6 +487,15 @@ public class MemoryRecordsBuilder {
return appendWithOffset(nextSequentialOffset(), true, timestamp, key, value, Record.EMPTY_HEADERS);
}
+ public long appendEndTxnMarker(long timestamp, EndTransactionMarker marker) {
+ if (producerId == RecordBatch.NO_PRODUCER_ID)
+ throw new IllegalArgumentException("End transaction marker requires a valid producerId");
+ if (!isTransactional)
+ throw new IllegalArgumentException("End transaction marker depends on batch transactional flag being enabled");
+ ByteBuffer value = marker.serializeValue();
+ return appendControlRecord(timestamp, marker.controlType(), value);
+ }
+
/**
* Add a legacy record without doing offset/magic validation (this should only be used in testing).
* @param offset The offset of the record
@@ -509,8 +521,7 @@ public class MemoryRecordsBuilder {
* @param record the record to add
*/
public void append(Record record) {
- appendWithOffset(record.offset(), record.isControlRecord(), record.timestamp(), record.key(), record.value(),
- record.headers());
+ appendWithOffset(record.offset(), record.timestamp(), record.key(), record.value(), record.headers());
}
/**
@@ -519,8 +530,7 @@ public class MemoryRecordsBuilder {
* @param record The record to add
*/
public void appendWithOffset(long offset, Record record) {
- appendWithOffset(offset, record.isControlRecord(), record.timestamp(), record.key(), record.value(),
- record.headers());
+ appendWithOffset(offset, record.timestamp(), record.key(), record.value(), record.headers());
}
/**
@@ -542,12 +552,12 @@ public class MemoryRecordsBuilder {
appendWithOffset(nextSequentialOffset(), record);
}
- private long appendDefaultRecord(long offset, boolean isControlRecord, long timestamp,
- ByteBuffer key, ByteBuffer value, Header[] headers) throws IOException {
+ private long appendDefaultRecord(long offset, long timestamp, ByteBuffer key, ByteBuffer value,
+ Header[] headers) throws IOException {
ensureOpenForRecordAppend();
int offsetDelta = (int) (offset - baseOffset);
long timestampDelta = timestamp - baseTimestamp;
- long crc = DefaultRecord.writeTo(appendStream, isControlRecord, offsetDelta, timestampDelta, key, value, headers);
+ long crc = DefaultRecord.writeTo(appendStream, offsetDelta, timestampDelta, key, value, headers);
// TODO: The crc is useless for the new message format. Maybe we should let writeTo return the written size?
recordWritten(offset, timestamp, DefaultRecord.sizeInBytes(offsetDelta, timestampDelta, key, value, headers));
return crc;
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/main/java/org/apache/kafka/common/record/Record.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/Record.java b/clients/src/main/java/org/apache/kafka/common/record/Record.java
index fdf41b3..cba6fc5 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/Record.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/Record.java
@@ -133,15 +133,6 @@ public interface Record {
boolean hasTimestampType(TimestampType timestampType);
/**
- * Check whether this is a control record (i.e. whether the control bit is set in the record attributes).
- * For magic versions prior to 2, this is always false.
- *
- * @return Whether this is a control record
- */
- boolean isControlRecord();
-
-
- /**
* Get the headers. For magic versions 1 and below, this always returns an empty array.
*
* @return the array of headers
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java
index 4fd03e1..c984deb 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/RecordBatch.java
@@ -216,4 +216,12 @@ public interface RecordBatch extends Iterable<Record> {
* @return The closeable iterator
*/
CloseableIterator<Record> streamingIterator();
+
+ /**
+ * Check whether this is a control batch (i.e. whether the control bit is set in the batch attributes).
+ * For magic versions prior to 2, this is always false.
+ *
+ * @return Whether this is a batch containing control records
+ */
+ boolean isControlBatch();
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java
index 7dbffd1..03f6ee5 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java
@@ -40,6 +40,7 @@ public class ListOffsetRequest extends AbstractRequest {
public static final int DEBUGGING_REPLICA_ID = -2;
private static final String REPLICA_ID_KEY_NAME = "replica_id";
+ private static final String ISOLATION_LEVEL_KEY_NAME = "isolation_level";
private static final String TOPICS_KEY_NAME = "topics";
// topic level field names
@@ -52,6 +53,7 @@ public class ListOffsetRequest extends AbstractRequest {
private static final String MAX_NUM_OFFSETS_KEY_NAME = "max_num_offsets";
private final int replicaId;
+ private final IsolationLevel isolationLevel;
private final Map<TopicPartition, PartitionData> offsetData;
private final Map<TopicPartition, Long> partitionTimestamps;
private final Set<TopicPartition> duplicatePartitions;
@@ -59,23 +61,29 @@ public class ListOffsetRequest extends AbstractRequest {
public static class Builder extends AbstractRequest.Builder<ListOffsetRequest> {
private final int replicaId;
private final short minVersion;
+ private final IsolationLevel isolationLevel;
private Map<TopicPartition, PartitionData> offsetData = null;
private Map<TopicPartition, Long> partitionTimestamps = null;
public static Builder forReplica(short desiredVersion, int replicaId) {
- return new Builder((short) 0, desiredVersion, replicaId);
+ return new Builder((short) 0, desiredVersion, replicaId, IsolationLevel.READ_UNCOMMITTED);
}
- public static Builder forConsumer(boolean requireTimestamp) {
+ public static Builder forConsumer(boolean requireTimestamp, IsolationLevel isolationLevel) {
// If we need a timestamp in the response, the minimum RPC version we can send is v1. Otherwise, v0 is OK.
- short minVersion = requireTimestamp ? (short) 1 : (short) 0;
- return new Builder(minVersion, null, CONSUMER_REPLICA_ID);
+ short minVersion = 0;
+ if (isolationLevel == IsolationLevel.READ_COMMITTED)
+ minVersion = 2;
+ else if (requireTimestamp)
+ minVersion = 1;
+ return new Builder(minVersion, null, CONSUMER_REPLICA_ID, isolationLevel);
}
- private Builder(short minVersion, Short desiredVersion, int replicaId) {
+ private Builder(short minVersion, Short desiredVersion, int replicaId, IsolationLevel isolationLevel) {
super(ApiKeys.LIST_OFFSETS, desiredVersion);
this.minVersion = minVersion;
this.replicaId = replicaId;
+ this.isolationLevel = isolationLevel;
}
public Builder setOffsetData(Map<TopicPartition, PartitionData> offsetData) {
@@ -118,7 +126,7 @@ public class ListOffsetRequest extends AbstractRequest {
}
}
Map<TopicPartition, ?> m = (version == 0) ? offsetData : partitionTimestamps;
- return new ListOffsetRequest(replicaId, m, version);
+ return new ListOffsetRequest(replicaId, m, isolationLevel, version);
}
@Override
@@ -165,9 +173,10 @@ public class ListOffsetRequest extends AbstractRequest {
* Private constructor with a specified version.
*/
@SuppressWarnings("unchecked")
- private ListOffsetRequest(int replicaId, Map<TopicPartition, ?> targetTimes, short version) {
+ private ListOffsetRequest(int replicaId, Map<TopicPartition, ?> targetTimes, IsolationLevel isolationLevel, short version) {
super(version);
this.replicaId = replicaId;
+ this.isolationLevel = isolationLevel;
this.offsetData = version == 0 ? (Map<TopicPartition, PartitionData>) targetTimes : null;
this.partitionTimestamps = version >= 1 ? (Map<TopicPartition, Long>) targetTimes : null;
this.duplicatePartitions = Collections.emptySet();
@@ -177,6 +186,9 @@ public class ListOffsetRequest extends AbstractRequest {
super(version);
Set<TopicPartition> duplicatePartitions = new HashSet<>();
replicaId = struct.getInt(REPLICA_ID_KEY_NAME);
+ isolationLevel = struct.hasField(ISOLATION_LEVEL_KEY_NAME) ?
+ IsolationLevel.forId(struct.getByte(ISOLATION_LEVEL_KEY_NAME)) :
+ IsolationLevel.READ_UNCOMMITTED;
offsetData = new HashMap<>();
partitionTimestamps = new HashMap<>();
for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
@@ -223,7 +235,6 @@ public class ListOffsetRequest extends AbstractRequest {
switch (versionId) {
case 0:
case 1:
- return new ListOffsetResponse(responseData);
case 2:
return new ListOffsetResponse(throttleTimeMs, responseData);
default:
@@ -236,6 +247,10 @@ public class ListOffsetRequest extends AbstractRequest {
return replicaId;
}
+ public IsolationLevel isolationLevel() {
+ return isolationLevel;
+ }
+
@Deprecated
public Map<TopicPartition, PartitionData> offsetData() {
return offsetData;
@@ -262,6 +277,9 @@ public class ListOffsetRequest extends AbstractRequest {
Map<String, Map<Integer, Object>> topicsData = CollectionUtils.groupDataByTopic(targetTimes);
struct.set(REPLICA_ID_KEY_NAME, replicaId);
+
+ if (struct.hasField(ISOLATION_LEVEL_KEY_NAME))
+ struct.set(ISOLATION_LEVEL_KEY_NAME, isolationLevel.id());
List<Struct> topicArray = new ArrayList<>();
for (Map.Entry<String, Map<Integer, Object>> topicEntry: topicsData.entrySet()) {
Struct topicData = struct.instance(TOPICS_KEY_NAME);
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
index cb0ff89..24ee788 100755
--- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
@@ -762,6 +762,11 @@ public class Utils {
} while (bytesRead != -1 && destinationBuffer.hasRemaining());
}
+ public static void writeFully(FileChannel channel, ByteBuffer sourceBuffer) throws IOException {
+ while (sourceBuffer.hasRemaining())
+ channel.write(sourceBuffer);
+ }
+
/**
* Write the contents of a buffer to an output stream. The bytes are copied from the current position
* in the buffer.
[2/6] kafka git commit: KAFKA-5121;
Implement transaction index for KIP-98
Posted by jg...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala
index eb3f50c..bf634d7 100644
--- a/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala
@@ -109,7 +109,8 @@ class LogCleanerLagIntegrationTest(compressionCodecName: String) extends Abstrac
private def writeDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionType, timestamp: Long): Seq[(Int, Int)] = {
for (_ <- 0 until numDups; key <- 0 until numKeys) yield {
val count = counter
- log.appendAsLeader(TestUtils.singletonRecords(value = counter.toString.getBytes, codec = codec, key = key.toString.getBytes, timestamp = timestamp), leaderEpoch = 0)
+ log.appendAsLeader(TestUtils.singletonRecords(value = counter.toString.getBytes, codec = codec,
+ key = key.toString.getBytes, timestamp = timestamp), leaderEpoch = 0)
counter += 1
(key, count)
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
index 44d47c9..fe07fdd 100755
--- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
@@ -821,7 +821,7 @@ class LogCleanerTest extends JUnitSuite {
def record(key: Int, value: Int, pid: Long = RecordBatch.NO_PRODUCER_ID, epoch: Short = RecordBatch.NO_PRODUCER_EPOCH,
sequence: Int = RecordBatch.NO_SEQUENCE,
partitionLeaderEpoch: Int = RecordBatch.NO_PARTITION_LEADER_EPOCH): MemoryRecords = {
- MemoryRecords.withRecords(RecordBatch.CURRENT_MAGIC_VALUE, 0L, CompressionType.NONE, pid, epoch, sequence,
+ MemoryRecords.withIdempotentRecords(RecordBatch.CURRENT_MAGIC_VALUE, 0L, CompressionType.NONE, pid, epoch, sequence,
partitionLeaderEpoch, new SimpleRecord(key.toString.getBytes, value.toString.getBytes))
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
index 2f9396f..a6fe2e4 100755
--- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
@@ -151,8 +151,9 @@ class LogManagerTest {
assertEquals("Now there should be exactly 6 segments", 6, log.numberOfSegments)
time.sleep(log.config.fileDeleteDelayMs + 1)
- //There should be a log file, two indexes, the leader epoch checkpoint and the pid snapshot dir
- assertEquals("Files should have been deleted", log.numberOfSegments * 3 + 2, log.dir.list.length)
+ // there should be a log file, two indexes (the txn index is created lazily),
+ // the leader epoch checkpoint and two pid mapping files (one for the active and previous segments)
+ assertEquals("Files should have been deleted", log.numberOfSegments * 3 + 3, log.dir.list.length)
assertEquals("Should get empty fetch off new log.", 0, log.read(offset + 1, 1024).records.sizeInBytes)
try {
log.read(0, 1024)
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
index 3f531d9..4709b77 100644
--- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
@@ -16,31 +16,40 @@
*/
package kafka.log
+import java.io.File
+
import kafka.utils.TestUtils
import kafka.utils.TestUtils.checkEquals
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.record.MemoryRecords.withEndTransactionMarker
import org.apache.kafka.common.record.{RecordBatch, _}
-import org.apache.kafka.common.utils.Time
+import org.apache.kafka.common.utils.{Time, Utils}
import org.junit.Assert._
-import org.junit.{After, Test}
+import org.junit.{After, Before, Test}
import scala.collection.JavaConverters._
import scala.collection._
class LogSegmentTest {
+ val topicPartition = new TopicPartition("topic", 0)
val segments = mutable.ArrayBuffer[LogSegment]()
-
+ var logDir: File = _
+
/* create a segment with the given base offset */
def createSegment(offset: Long, indexIntervalBytes: Int = 10): LogSegment = {
val msFile = TestUtils.tempFile()
val ms = FileRecords.open(msFile)
val idxFile = TestUtils.tempFile()
val timeIdxFile = TestUtils.tempFile()
+ val txnIdxFile = TestUtils.tempFile()
idxFile.delete()
timeIdxFile.delete()
+ txnIdxFile.delete()
val idx = new OffsetIndex(idxFile, offset, 1000)
val timeIdx = new TimeIndex(timeIdxFile, offset, 1500)
- val seg = new LogSegment(ms, idx, timeIdx, offset, indexIntervalBytes, 0, Time.SYSTEM)
+ val txnIndex = new TransactionIndex(offset, txnIdxFile)
+ val seg = new LogSegment(ms, idx, timeIdx, txnIndex, offset, indexIntervalBytes, 0, Time.SYSTEM)
segments += seg
seg
}
@@ -51,12 +60,20 @@ class LogSegmentTest {
records.map { s => new SimpleRecord(offset * 10, s.getBytes) }: _*)
}
+ @Before
+ def setup(): Unit = {
+ logDir = TestUtils.tempDir()
+ }
+
@After
def teardown() {
for(seg <- segments) {
seg.index.delete()
+ seg.timeIndex.delete()
+ seg.txnIndex.delete()
seg.log.delete()
}
+ Utils.delete(logDir)
}
/**
@@ -153,7 +170,7 @@ class LogSegmentTest {
}
@Test
- def testReloadLargestTimestampAfterTruncation() {
+ def testReloadLargestTimestampAndNextOffsetAfterTruncation() {
val numMessages = 30
val seg = createSegment(40, 2 * records(0, "hello").sizeInBytes - 1)
var offset = 40
@@ -161,13 +178,15 @@ class LogSegmentTest {
seg.append(offset, offset, offset, offset, records(offset, "hello"))
offset += 1
}
+ assertEquals(offset, seg.nextOffset)
+
val expectedNumEntries = numMessages / 2 - 1
assertEquals(s"Should have $expectedNumEntries time indexes", expectedNumEntries, seg.timeIndex.entries)
seg.truncateTo(41)
assertEquals(s"Should have 0 time indexes", 0, seg.timeIndex.entries)
assertEquals(s"Largest timestamp should be 400", 400L, seg.largestTimestamp)
-
+ assertEquals(41, seg.nextOffset)
}
/**
@@ -217,7 +236,7 @@ class LogSegmentTest {
val seg = createSegment(40)
assertEquals(40, seg.nextOffset)
seg.append(50, 52, RecordBatch.NO_TIMESTAMP, -1L, records(50, "hello", "there", "you"))
- assertEquals(53, seg.nextOffset())
+ assertEquals(53, seg.nextOffset)
}
/**
@@ -246,11 +265,76 @@ class LogSegmentTest {
seg.append(i, i, RecordBatch.NO_TIMESTAMP, -1L, records(i, i.toString))
val indexFile = seg.index.file
TestUtils.writeNonsenseToFile(indexFile, 5, indexFile.length.toInt)
- seg.recover(64*1024)
+ seg.recover(64*1024, new ProducerStateManager(topicPartition, logDir))
for(i <- 0 until 100)
assertEquals(i, seg.read(i, Some(i + 1), 1024).records.records.iterator.next().offset)
}
+ @Test
+ def testRecoverTransactionIndex(): Unit = {
+ val segment = createSegment(100)
+ val epoch = 0.toShort
+ val sequence = 0
+
+ val pid1 = 5L
+ val pid2 = 10L
+
+ // append transactional records from pid1
+ segment.append(firstOffset = 100L, largestOffset = 101L, largestTimestamp = RecordBatch.NO_TIMESTAMP,
+ shallowOffsetOfMaxTimestamp = 100L, MemoryRecords.withTransactionalRecords(100L, CompressionType.NONE,
+ pid1, epoch, sequence, new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
+
+ // append transactional records from pid2
+ segment.append(firstOffset = 102L, largestOffset = 103L, largestTimestamp = RecordBatch.NO_TIMESTAMP,
+ shallowOffsetOfMaxTimestamp = 102L, MemoryRecords.withTransactionalRecords(102L, CompressionType.NONE,
+ pid2, epoch, sequence, new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
+
+ // append non-transactional records
+ segment.append(firstOffset = 104L, largestOffset = 105L, largestTimestamp = RecordBatch.NO_TIMESTAMP,
+ shallowOffsetOfMaxTimestamp = 104L, MemoryRecords.withRecords(104L, CompressionType.NONE,
+ new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
+
+ // abort the transaction from pid2 (note LSO should be 100L since the txn from pid1 has not completed)
+ segment.append(firstOffset = 106L, largestOffset = 106L, largestTimestamp = RecordBatch.NO_TIMESTAMP,
+ shallowOffsetOfMaxTimestamp = 106L, endTxnRecords(ControlRecordType.ABORT, pid2, epoch, offset = 106L))
+
+ // commit the transaction from pid1
+ segment.append(firstOffset = 107L, largestOffset = 107L, largestTimestamp = RecordBatch.NO_TIMESTAMP,
+ shallowOffsetOfMaxTimestamp = 107L, endTxnRecords(ControlRecordType.COMMIT, pid1, epoch, offset = 107L))
+
+ segment.recover(64 * 1024, new ProducerStateManager(topicPartition, logDir))
+
+ var abortedTxns = segment.txnIndex.allAbortedTxns
+ assertEquals(1, abortedTxns.size)
+ var abortedTxn = abortedTxns.head
+ assertEquals(pid2, abortedTxn.producerId)
+ assertEquals(102L, abortedTxn.firstOffset)
+ assertEquals(106L, abortedTxn.lastOffset)
+ assertEquals(100L, abortedTxn.lastStableOffset)
+
+ // recover again, but this time assuming the transaction from pid2 began on a previous segment
+ val stateManager = new ProducerStateManager(topicPartition, logDir)
+ stateManager.loadProducerEntry(ProducerIdEntry(pid2, epoch, 10, 90L, 5, RecordBatch.NO_TIMESTAMP, 0, Some(75L)))
+ segment.recover(64 * 1024, stateManager)
+
+ abortedTxns = segment.txnIndex.allAbortedTxns
+ assertEquals(1, abortedTxns.size)
+ abortedTxn = abortedTxns.head
+ assertEquals(pid2, abortedTxn.producerId)
+ assertEquals(75L, abortedTxn.firstOffset)
+ assertEquals(106L, abortedTxn.lastOffset)
+ assertEquals(100L, abortedTxn.lastStableOffset)
+ }
+
+ private def endTxnRecords(controlRecordType: ControlRecordType,
+ producerId: Long,
+ epoch: Short,
+ offset: Long = 0L,
+ coordinatorEpoch: Int = 0): MemoryRecords = {
+ val marker = new EndTransactionMarker(controlRecordType, coordinatorEpoch)
+ withEndTransactionMarker(offset, producerId, epoch, marker)
+ }
+
/**
* Create a segment with some data and an index. Then corrupt the index,
* and recover the segment, the entries should all be readable.
@@ -262,7 +346,7 @@ class LogSegmentTest {
seg.append(i, i, i * 10, i, records(i, i.toString))
val timeIndexFile = seg.timeIndex.file
TestUtils.writeNonsenseToFile(timeIndexFile, 5, timeIndexFile.length.toInt)
- seg.recover(64*1024)
+ seg.recover(64*1024, new ProducerStateManager(topicPartition, logDir))
for(i <- 0 until 100) {
assertEquals(i, seg.findOffsetByTimestamp(i * 10).get.offset)
if (i < 99)
@@ -286,7 +370,7 @@ class LogSegmentTest {
val recordPosition = seg.log.searchForOffsetWithSize(offsetToBeginCorruption, 0)
val position = recordPosition.position + TestUtils.random.nextInt(15)
TestUtils.writeNonsenseToFile(seg.log.file, position, (seg.log.file.length - position).toInt)
- seg.recover(64*1024)
+ seg.recover(64*1024, new ProducerStateManager(topicPartition, logDir))
assertEquals("Should have truncated off bad messages.", (0 until offsetToBeginCorruption).toList,
seg.log.batches.asScala.map(_.lastOffset).toList)
seg.delete()
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/log/LogTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala
index 0f82cd3..b11c94b 100755
--- a/core/src/test/scala/unit/kafka/log/LogTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogTest.scala
@@ -31,6 +31,8 @@ import kafka.server.KafkaConfig
import kafka.server.epoch.{EpochEntry, LeaderEpochFileCache}
import org.apache.kafka.common.record.MemoryRecords.RecordFilter
import org.apache.kafka.common.record._
+import org.apache.kafka.common.requests.FetchResponse.AbortedTransaction
+import org.apache.kafka.common.requests.IsolationLevel
import org.apache.kafka.common.utils.Utils
import scala.collection.JavaConverters._
@@ -58,7 +60,7 @@ class LogTest {
def createEmptyLogs(dir: File, offsets: Int*) {
for(offset <- offsets) {
Log.logFilename(dir, offset).createNewFile()
- Log.indexFilename(dir, offset).createNewFile()
+ Log.offsetIndexFile(dir, offset).createNewFile()
}
}
@@ -69,13 +71,13 @@ class LogTest {
val logFile = Log.logFilename(tmpDir, offset)
assertEquals(offset, Log.offsetFromFilename(logFile.getName))
- val offsetIndexFile = Log.indexFilename(tmpDir, offset)
+ val offsetIndexFile = Log.offsetIndexFile(tmpDir, offset)
assertEquals(offset, Log.offsetFromFilename(offsetIndexFile.getName))
- val timeIndexFile = Log.timeIndexFilename(tmpDir, offset)
+ val timeIndexFile = Log.timeIndexFile(tmpDir, offset)
assertEquals(offset, Log.offsetFromFilename(timeIndexFile.getName))
- val snapshotFile = Log.pidSnapshotFilename(tmpDir, offset)
+ val snapshotFile = Log.producerSnapshotFile(tmpDir, offset)
assertEquals(offset, Log.offsetFromFilename(snapshotFile.getName))
}
@@ -166,8 +168,8 @@ class LogTest {
val log = createLog(2048)
val records = TestUtils.records(List(new SimpleRecord(time.milliseconds, "key".getBytes, "value".getBytes)))
log.appendAsLeader(records, leaderEpoch = 0)
- log.maybeTakePidSnapshot()
- assertEquals(Some(1), log.latestPidSnapshotOffset)
+ log.takeProducerSnapshot()
+ assertEquals(Some(1), log.latestProducerSnapshotOffset)
}
@Test
@@ -253,18 +255,18 @@ class LogTest {
val log = createLog(2048)
log.appendAsLeader(TestUtils.records(List(new SimpleRecord("a".getBytes))), leaderEpoch = 0)
log.appendAsLeader(TestUtils.records(List(new SimpleRecord("b".getBytes))), leaderEpoch = 0)
- log.maybeTakePidSnapshot()
+ log.takeProducerSnapshot()
log.appendAsLeader(TestUtils.records(List(new SimpleRecord("c".getBytes))), leaderEpoch = 0)
- log.maybeTakePidSnapshot()
+ log.takeProducerSnapshot()
log.truncateTo(2)
- assertEquals(Some(2), log.latestPidSnapshotOffset)
- assertEquals(2, log.latestPidMapOffset)
+ assertEquals(Some(2), log.latestProducerSnapshotOffset)
+ assertEquals(2, log.latestProducerStateEndOffset)
log.truncateTo(1)
- assertEquals(None, log.latestPidSnapshotOffset)
- assertEquals(1, log.latestPidMapOffset)
+ assertEquals(None, log.latestProducerSnapshotOffset)
+ assertEquals(1, log.latestProducerStateEndOffset)
}
@Test
@@ -272,20 +274,20 @@ class LogTest {
val records = TestUtils.singletonRecords("foo".getBytes)
val log = createLog(records.sizeInBytes, messagesPerSegment = 1, retentionBytes = records.sizeInBytes * 2)
log.appendAsLeader(records, leaderEpoch = 0)
- log.maybeTakePidSnapshot()
+ log.takeProducerSnapshot()
log.appendAsLeader(TestUtils.singletonRecords("bar".getBytes), leaderEpoch = 0)
log.appendAsLeader(TestUtils.singletonRecords("baz".getBytes), leaderEpoch = 0)
- log.maybeTakePidSnapshot()
+ log.takeProducerSnapshot()
assertEquals(3, log.logSegments.size)
- assertEquals(3, log.latestPidMapOffset)
- assertEquals(Some(3), log.latestPidSnapshotOffset)
+ assertEquals(3, log.latestProducerStateEndOffset)
+ assertEquals(Some(3), log.latestProducerSnapshotOffset)
log.truncateFullyAndStartAt(29)
assertEquals(1, log.logSegments.size)
- assertEquals(None, log.latestPidSnapshotOffset)
- assertEquals(29, log.latestPidMapOffset)
+ assertEquals(None, log.latestProducerSnapshotOffset)
+ assertEquals(29, log.latestProducerStateEndOffset)
}
@Test
@@ -294,14 +296,14 @@ class LogTest {
val records = TestUtils.records(Seq(new SimpleRecord("foo".getBytes)), pid = pid1, epoch = 0, sequence = 0)
val log = createLog(records.sizeInBytes, messagesPerSegment = 1, retentionBytes = records.sizeInBytes * 2)
log.appendAsLeader(records, leaderEpoch = 0)
- log.maybeTakePidSnapshot()
+ log.takeProducerSnapshot()
val pid2 = 2L
log.appendAsLeader(TestUtils.records(Seq(new SimpleRecord("bar".getBytes)), pid = pid2, epoch = 0, sequence = 0),
leaderEpoch = 0)
log.appendAsLeader(TestUtils.records(Seq(new SimpleRecord("baz".getBytes)), pid = pid2, epoch = 0, sequence = 1),
leaderEpoch = 0)
- log.maybeTakePidSnapshot()
+ log.takeProducerSnapshot()
assertEquals(3, log.logSegments.size)
assertEquals(Set(pid1, pid2), log.activePids.keySet)
@@ -313,16 +315,69 @@ class LogTest {
}
@Test
- def testPeriodicPidSnapshot() {
- val snapshotInterval = 100
- val log = createLog(2048, pidSnapshotIntervalMs = snapshotInterval)
+ def testTakeSnapshotOnRollAndDeleteSnapshotOnFlush() {
+ val log = createLog(2048)
+ log.appendAsLeader(TestUtils.singletonRecords("a".getBytes), leaderEpoch = 0)
+ log.roll(1L)
+ assertEquals(Some(1L), log.latestProducerSnapshotOffset)
+ assertEquals(Some(1L), log.oldestProducerSnapshotOffset)
+
+ log.appendAsLeader(TestUtils.singletonRecords("b".getBytes), leaderEpoch = 0)
+ log.roll(2L)
+ assertEquals(Some(2L), log.latestProducerSnapshotOffset)
+ assertEquals(Some(1L), log.oldestProducerSnapshotOffset)
+
+ log.appendAsLeader(TestUtils.singletonRecords("c".getBytes), leaderEpoch = 0)
+ log.roll(3L)
+ assertEquals(Some(3L), log.latestProducerSnapshotOffset)
+
+ // roll triggers a flush at the starting offset of the new segment. we should
+ // retain the snapshots from the active segment and the previous segment, but
+ // the oldest one should be gone
+ assertEquals(Some(2L), log.oldestProducerSnapshotOffset)
+
+ // even if we flush within the active segment, the snapshot should remain
+ log.appendAsLeader(TestUtils.singletonRecords("baz".getBytes), leaderEpoch = 0)
+ log.flush(4L)
+ assertEquals(Some(3L), log.latestProducerSnapshotOffset)
+ assertEquals(Some(2L), log.oldestProducerSnapshotOffset)
+ }
- log.appendAsLeader(TestUtils.singletonRecords("foo".getBytes), leaderEpoch = 0)
- log.appendAsLeader(TestUtils.singletonRecords("bar".getBytes), leaderEpoch = 0)
- assertEquals(None, log.latestPidSnapshotOffset)
+ @Test
+ def testRebuildTransactionalState(): Unit = {
+ val log = createLog(1024 * 1024)
+
+ val pid = 137L
+ val epoch = 5.toShort
+ val seq = 0
+
+ // add some transactional records
+ val records = MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid, epoch, seq,
+ new SimpleRecord("foo".getBytes),
+ new SimpleRecord("bar".getBytes),
+ new SimpleRecord("baz".getBytes))
+ log.appendAsLeader(records, leaderEpoch = 0)
+ val commitAppendInfo = log.appendAsLeader(endTxnRecords(ControlRecordType.ABORT, pid, epoch),
+ isFromClient = false, leaderEpoch = 0)
+ log.onHighWatermarkIncremented(commitAppendInfo.lastOffset + 1)
+
+ // now there should be no first unstable offset
+ assertEquals(None, log.firstUnstableOffset)
+
+ log.close()
- time.sleep(snapshotInterval)
- assertEquals(Some(2), log.latestPidSnapshotOffset)
+ val reopenedLog = createLog(1024 * 1024)
+ reopenedLog.onHighWatermarkIncremented(commitAppendInfo.lastOffset + 1)
+ assertEquals(None, reopenedLog.firstUnstableOffset)
+ }
+
+ private def endTxnRecords(controlRecordType: ControlRecordType,
+ producerId: Long,
+ epoch: Short,
+ offset: Long = 0L,
+ coordinatorEpoch: Int = 0): MemoryRecords = {
+ val marker = new EndTransactionMarker(controlRecordType, coordinatorEpoch)
+ MemoryRecords.withEndTransactionMarker(offset, producerId, epoch, marker)
}
@Test
@@ -432,25 +487,25 @@ class LogTest {
time = time)
val epoch: Short = 0
-
val buffer = ByteBuffer.allocate(512)
- var builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, TimestampType.LOG_APPEND_TIME, 0L, time.milliseconds(), 1L, epoch, 0, false, 0)
+ var builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE,
+ TimestampType.LOG_APPEND_TIME, 0L, time.milliseconds(), 1L, epoch, 0, false, 0)
builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
builder.close()
- // Append a record with other pids.
- builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, TimestampType.LOG_APPEND_TIME, 1L, time.milliseconds(), 2L, epoch, 0, false, 0)
+ builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE,
+ TimestampType.LOG_APPEND_TIME, 1L, time.milliseconds(), 2L, epoch, 0, false, 0)
builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
builder.close()
- // Append a record with other pids.
- builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, TimestampType.LOG_APPEND_TIME, 2L, time.milliseconds(), 3L, epoch, 0, false, 0)
+ builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE,
+ TimestampType.LOG_APPEND_TIME, 2L, time.milliseconds(), 3L, epoch, 0, false, 0)
builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
builder.close()
- // Append a record with other pids.
- builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, TimestampType.LOG_APPEND_TIME, 3L, time.milliseconds(), 4L, epoch, 0, false, 0)
+ builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE,
+ TimestampType.LOG_APPEND_TIME, 3L, time.milliseconds(), 4L, epoch, 0, false, 0)
builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
builder.close()
@@ -473,46 +528,66 @@ class LogTest {
}
@Test(expected = classOf[DuplicateSequenceNumberException])
- def testMultiplePidsWithDuplicates() : Unit = {
- val logProps = new Properties()
+ def testDuplicateAppendToFollower() : Unit = {
+ val log = createLog(1024*1024)
+ val epoch: Short = 0
+ val pid = 1L
+ val baseSequence = 0
+ val partitionLeaderEpoch = 0
+ // this is a bit contrived. to trigger the duplicate case for a follower append, we have to append
+ // a batch with matching sequence numbers, but valid increasing offsets
+ log.appendAsFollower(MemoryRecords.withIdempotentRecords(0L, CompressionType.NONE, pid, epoch, baseSequence,
+ partitionLeaderEpoch, new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
+ log.appendAsFollower(MemoryRecords.withIdempotentRecords(2L, CompressionType.NONE, pid, epoch, baseSequence,
+ partitionLeaderEpoch, new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
+ }
- // create a log
- val log = new Log(logDir,
- LogConfig(logProps),
- recoveryPoint = 0L,
- scheduler = time.scheduler,
- time = time)
+ @Test(expected = classOf[DuplicateSequenceNumberException])
+ def testMultipleProducersWithDuplicatesInSingleAppend() : Unit = {
+ val log = createLog(1024*1024)
+ val pid1 = 1L
+ val pid2 = 2L
val epoch: Short = 0
val buffer = ByteBuffer.allocate(512)
- var builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, TimestampType.LOG_APPEND_TIME, 0L, time.milliseconds(), 1L, epoch, 0)
+ // pid1 seq = 0
+ var builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE,
+ TimestampType.LOG_APPEND_TIME, 0L, time.milliseconds(), pid1, epoch, 0)
builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
builder.close()
- // Append a record with other pids.
- builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, TimestampType.LOG_APPEND_TIME, 1L, time.milliseconds(), 2L, epoch, 0)
+ // pid2 seq = 0
+ builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE,
+ TimestampType.LOG_APPEND_TIME, 1L, time.milliseconds(), pid2, epoch, 0)
builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
builder.close()
- // Append a record with other pids.
- builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, TimestampType.LOG_APPEND_TIME, 2L, time.milliseconds(), 1L, epoch, 1)
+ // pid1 seq = 1
+ builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE,
+ TimestampType.LOG_APPEND_TIME, 2L, time.milliseconds(), pid1, epoch, 1)
builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
builder.close()
- builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, TimestampType.LOG_APPEND_TIME, 3L, time.milliseconds(), 2L, epoch, 1)
+ // pid2 seq = 1
+ builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE,
+ TimestampType.LOG_APPEND_TIME, 3L, time.milliseconds(), pid2, epoch, 1)
builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
builder.close()
- builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, TimestampType.LOG_APPEND_TIME, 4L, time.milliseconds(), 1L, epoch, 1)
+ // // pid1 seq = 1 (duplicate)
+ builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE,
+ TimestampType.LOG_APPEND_TIME, 4L, time.milliseconds(), pid1, epoch, 1)
builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
builder.close()
buffer.flip()
- log.appendAsFollower(MemoryRecords.readableRecords(buffer))
- // Should throw a duplicate seqeuence exception here.
+ val records = MemoryRecords.readableRecords(buffer)
+ records.batches.asScala.foreach(_.setPartitionLeaderEpoch(0))
+ log.appendAsFollower(records)
+ // Should throw a duplicate sequence exception here.
fail("should have thrown a DuplicateSequenceNumberException.")
}
@@ -1245,10 +1320,10 @@ class LogTest {
*/
@Test
def testBogusIndexSegmentsAreRemoved() {
- val bogusIndex1 = Log.indexFilename(logDir, 0)
- val bogusTimeIndex1 = Log.timeIndexFilename(logDir, 0)
- val bogusIndex2 = Log.indexFilename(logDir, 5)
- val bogusTimeIndex2 = Log.timeIndexFilename(logDir, 5)
+ val bogusIndex1 = Log.offsetIndexFile(logDir, 0)
+ val bogusTimeIndex1 = Log.timeIndexFile(logDir, 0)
+ val bogusIndex2 = Log.offsetIndexFile(logDir, 5)
+ val bogusTimeIndex2 = Log.timeIndexFile(logDir, 5)
def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds)
val logProps = new Properties()
@@ -1501,9 +1576,11 @@ class LogTest {
//This write will roll the segment, yielding a new segment with base offset = max(2, 1) = 2
log.appendAsFollower(set2)
assertEquals(2L, log.activeSegment.baseOffset)
+ assertTrue(Log.producerSnapshotFile(logDir, 2L).exists)
//This will also roll the segment, yielding a new segment with base offset = max(3, Integer.MAX_VALUE+3) = Integer.MAX_VALUE+3
log.appendAsFollower(set3)
assertEquals(Integer.MAX_VALUE.toLong + 3, log.activeSegment.baseOffset)
+ assertTrue(Log.producerSnapshotFile(logDir, Integer.MAX_VALUE.toLong + 3).exists)
//This will go into the existing log
log.appendAsFollower(set4)
assertEquals(Integer.MAX_VALUE.toLong + 3, log.activeSegment.baseOffset)
@@ -1990,11 +2067,301 @@ class LogTest {
builder.build()
}
+ def testFirstUnstableOffsetNoTransactionalData() {
+ val log = createLog(1024 * 1024)
+
+ val records = MemoryRecords.withRecords(CompressionType.NONE,
+ new SimpleRecord("foo".getBytes),
+ new SimpleRecord("bar".getBytes),
+ new SimpleRecord("baz".getBytes))
+
+ log.appendAsLeader(records, leaderEpoch = 0)
+ assertEquals(None, log.firstUnstableOffset)
+ }
+
+ @Test
+ def testFirstUnstableOffsetWithTransactionalData() {
+ val log = createLog(1024 * 1024)
+
+ val pid = 137L
+ val epoch = 5.toShort
+ var seq = 0
+
+ // add some transactional records
+ val records = MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid, epoch, seq,
+ new SimpleRecord("foo".getBytes),
+ new SimpleRecord("bar".getBytes),
+ new SimpleRecord("baz".getBytes))
+
+ val firstAppendInfo = log.appendAsLeader(records, leaderEpoch = 0)
+ assertEquals(Some(firstAppendInfo.firstOffset), log.firstUnstableOffset.map(_.messageOffset))
+
+ // add more transactional records
+ seq += 3
+ log.appendAsLeader(MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid, epoch, seq,
+ new SimpleRecord("blah".getBytes)), leaderEpoch = 0)
+
+ // LSO should not have changed
+ assertEquals(Some(firstAppendInfo.firstOffset), log.firstUnstableOffset.map(_.messageOffset))
+
+ // now transaction is committed
+ val commitAppendInfo = log.appendAsLeader(endTxnRecords(ControlRecordType.COMMIT, pid, epoch),
+ isFromClient = false, leaderEpoch = 0)
+
+ // first unstable offset is not updated until the high watermark is advanced
+ assertEquals(Some(firstAppendInfo.firstOffset), log.firstUnstableOffset.map(_.messageOffset))
+ log.onHighWatermarkIncremented(commitAppendInfo.lastOffset + 1)
+
+ // now there should be no first unstable offset
+ assertEquals(None, log.firstUnstableOffset)
+ }
+
+ @Test
+ def testTransactionIndexUpdated(): Unit = {
+ val log = createLog(1024 * 1024)
+ val epoch = 0.toShort
+
+ val pid1 = 1L
+ val pid2 = 2L
+ val pid3 = 3L
+ val pid4 = 4L
+
+ val appendPid1 = appendTransactionalAsLeader(log, pid1, epoch)
+ val appendPid2 = appendTransactionalAsLeader(log, pid2, epoch)
+ val appendPid3 = appendTransactionalAsLeader(log, pid3, epoch)
+ val appendPid4 = appendTransactionalAsLeader(log, pid4, epoch)
+
+ // mix transactional and non-transactional data
+ appendPid1(5) // nextOffset: 5
+ appendNonTransactionalAsLeader(log, 3) // 8
+ appendPid2(2) // 10
+ appendPid1(4) // 14
+ appendPid3(3) // 17
+ appendNonTransactionalAsLeader(log, 2) // 19
+ appendPid1(10) // 29
+ appendEndTxnMarkerAsLeader(log, pid1, epoch, ControlRecordType.ABORT) // 30
+ appendPid2(6) // 36
+ appendPid4(3) // 39
+ appendNonTransactionalAsLeader(log, 10) // 49
+ appendPid3(9) // 58
+ appendEndTxnMarkerAsLeader(log, pid3, epoch, ControlRecordType.COMMIT) // 59
+ appendPid4(8) // 67
+ appendPid2(7) // 74
+ appendEndTxnMarkerAsLeader(log, pid2, epoch, ControlRecordType.ABORT) // 75
+ appendNonTransactionalAsLeader(log, 10) // 85
+ appendPid4(4) // 89
+ appendEndTxnMarkerAsLeader(log, pid4, epoch, ControlRecordType.COMMIT) // 90
+
+ val abortedTransactions = allAbortedTransactions(log)
+ assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions)
+ }
+
+ @Test
+ def testRecoverTransactionIndex(): Unit = {
+ val log = createLog(128)
+ val epoch = 0.toShort
+
+ val pid1 = 1L
+ val pid2 = 2L
+ val pid3 = 3L
+ val pid4 = 4L
+
+ val appendPid1 = appendTransactionalAsLeader(log, pid1, epoch)
+ val appendPid2 = appendTransactionalAsLeader(log, pid2, epoch)
+ val appendPid3 = appendTransactionalAsLeader(log, pid3, epoch)
+ val appendPid4 = appendTransactionalAsLeader(log, pid4, epoch)
+
+ // mix transactional and non-transactional data
+ appendPid1(5) // nextOffset: 5
+ appendNonTransactionalAsLeader(log, 3) // 8
+ appendPid2(2) // 10
+ appendPid1(4) // 14
+ appendPid3(3) // 17
+ appendNonTransactionalAsLeader(log, 2) // 19
+ appendPid1(10) // 29
+ appendEndTxnMarkerAsLeader(log, pid1, epoch, ControlRecordType.ABORT) // 30
+ appendPid2(6) // 36
+ appendPid4(3) // 39
+ appendNonTransactionalAsLeader(log, 10) // 49
+ appendPid3(9) // 58
+ appendEndTxnMarkerAsLeader(log, pid3, epoch, ControlRecordType.COMMIT) // 59
+ appendPid4(8) // 67
+ appendPid2(7) // 74
+ appendEndTxnMarkerAsLeader(log, pid2, epoch, ControlRecordType.ABORT) // 75
+ appendNonTransactionalAsLeader(log, 10) // 85
+ appendPid4(4) // 89
+ appendEndTxnMarkerAsLeader(log, pid4, epoch, ControlRecordType.COMMIT) // 90
+
+ // delete all the offset and transaction index files to force recovery
+ log.logSegments.foreach { segment =>
+ segment.index.delete()
+ segment.txnIndex.delete()
+ }
+
+ log.close()
+
+ val reloadedLog = createLog(1024)
+ val abortedTransactions = allAbortedTransactions(reloadedLog)
+ assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions)
+ }
+
+ @Test
+ def testTransactionIndexUpdatedThroughReplication(): Unit = {
+ val epoch = 0.toShort
+ val log = createLog(1024 * 1024)
+ val buffer = ByteBuffer.allocate(2048)
+
+ val pid1 = 1L
+ val pid2 = 2L
+ val pid3 = 3L
+ val pid4 = 4L
+
+ val appendPid1 = appendTransactionalToBuffer(buffer, pid1, epoch)
+ val appendPid2 = appendTransactionalToBuffer(buffer, pid2, epoch)
+ val appendPid3 = appendTransactionalToBuffer(buffer, pid3, epoch)
+ val appendPid4 = appendTransactionalToBuffer(buffer, pid4, epoch)
+
+ appendPid1(0L, 5)
+ appendNonTransactionalToBuffer(buffer, 5L, 3)
+ appendPid2(8L, 2)
+ appendPid1(10L, 4)
+ appendPid3(14L, 3)
+ appendNonTransactionalToBuffer(buffer, 17L, 2)
+ appendPid1(19L, 10)
+ appendEndTxnMarkerToBuffer(buffer, pid1, epoch, 29L, ControlRecordType.ABORT)
+ appendPid2(30L, 6)
+ appendPid4(36L, 3)
+ appendNonTransactionalToBuffer(buffer, 39L, 10)
+ appendPid3(49L, 9)
+ appendEndTxnMarkerToBuffer(buffer, pid3, epoch, 58L, ControlRecordType.COMMIT)
+ appendPid4(59L, 8)
+ appendPid2(67L, 7)
+ appendEndTxnMarkerToBuffer(buffer, pid2, epoch, 74L, ControlRecordType.ABORT)
+ appendNonTransactionalToBuffer(buffer, 75L, 10)
+ appendPid4(85L, 4)
+ appendEndTxnMarkerToBuffer(buffer, pid4, epoch, 89L, ControlRecordType.COMMIT)
+
+ buffer.flip()
+
+ appendAsFollower(log, MemoryRecords.readableRecords(buffer))
+
+ val abortedTransactions = allAbortedTransactions(log)
+ assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions)
+ }
+
+ @Test(expected = classOf[TransactionCoordinatorFencedException])
+ def testZombieCoordinatorFenced(): Unit = {
+ val pid = 1L
+ val epoch = 0.toShort
+ val log = createLog(1024 * 1024)
+
+ val append = appendTransactionalAsLeader(log, pid, epoch)
+
+ append(10)
+ appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, coordinatorEpoch = 1)
+
+ append(5)
+ appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.COMMIT, coordinatorEpoch = 2)
- def createLog(messageSizeInBytes: Int, retentionMs: Int = -1, retentionBytes: Int = -1,
- cleanupPolicy: String = "delete", messagesPerSegment: Int = 5,
- maxPidExpirationMs: Int = 300000, pidExpirationCheckIntervalMs: Int = 30000,
- pidSnapshotIntervalMs: Int = 60000): Log = {
+ appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, coordinatorEpoch = 1)
+ }
+
+ @Test
+ def testLastStableOffsetWithMixedProducerData() {
+ val log = createLog(1024 * 1024)
+
+ // for convenience, both producers share the same epoch
+ val epoch = 5.toShort
+
+ val pid1 = 137L
+ val seq1 = 0
+ val pid2 = 983L
+ val seq2 = 0
+
+ // add some transactional records
+ val firstAppendInfo = log.appendAsLeader(MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid1, epoch, seq1,
+ new SimpleRecord("a".getBytes),
+ new SimpleRecord("b".getBytes),
+ new SimpleRecord("c".getBytes)), leaderEpoch = 0)
+ assertEquals(Some(firstAppendInfo.firstOffset), log.firstUnstableOffset.map(_.messageOffset))
+
+ // mix in some non-transactional data
+ log.appendAsLeader(MemoryRecords.withRecords(CompressionType.NONE,
+ new SimpleRecord("g".getBytes),
+ new SimpleRecord("h".getBytes),
+ new SimpleRecord("i".getBytes)), leaderEpoch = 0)
+
+ // append data from a second transactional producer
+ val secondAppendInfo = log.appendAsLeader(MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid2, epoch, seq2,
+ new SimpleRecord("d".getBytes),
+ new SimpleRecord("e".getBytes),
+ new SimpleRecord("f".getBytes)), leaderEpoch = 0)
+
+ // LSO should not have changed
+ assertEquals(Some(firstAppendInfo.firstOffset), log.firstUnstableOffset.map(_.messageOffset))
+
+ // now first producer's transaction is aborted
+ val abortAppendInfo = log.appendAsLeader(endTxnRecords(ControlRecordType.ABORT, pid1, epoch),
+ isFromClient = false, leaderEpoch = 0)
+ log.onHighWatermarkIncremented(abortAppendInfo.lastOffset + 1)
+
+ // LSO should now point to one less than the first offset of the second transaction
+ assertEquals(Some(secondAppendInfo.firstOffset), log.firstUnstableOffset.map(_.messageOffset))
+
+ // commit the second transaction
+ val commitAppendInfo = log.appendAsLeader(endTxnRecords(ControlRecordType.COMMIT, pid2, epoch),
+ isFromClient = false, leaderEpoch = 0)
+ log.onHighWatermarkIncremented(commitAppendInfo.lastOffset + 1)
+
+ // now there should be no first unstable offset
+ assertEquals(None, log.firstUnstableOffset)
+ }
+
+ @Test
+ def testAbortedTransactionSpanningMultipleSegments() {
+ val pid = 137L
+ val epoch = 5.toShort
+ var seq = 0
+
+ val records = MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid, epoch, seq,
+ new SimpleRecord("a".getBytes),
+ new SimpleRecord("b".getBytes),
+ new SimpleRecord("c".getBytes))
+
+ val log = createLog(messageSizeInBytes = records.sizeInBytes, messagesPerSegment = 1)
+
+ val firstAppendInfo = log.appendAsLeader(records, leaderEpoch = 0)
+ assertEquals(Some(firstAppendInfo.firstOffset), log.firstUnstableOffset.map(_.messageOffset))
+ assertEquals(Some(0L), log.firstUnstableOffset.map(_.segmentBaseOffset))
+
+ // this write should spill to the second segment
+ seq = 3
+ log.appendAsLeader(MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid, epoch, seq,
+ new SimpleRecord("d".getBytes),
+ new SimpleRecord("e".getBytes),
+ new SimpleRecord("f".getBytes)), leaderEpoch = 0)
+ assertEquals(Some(firstAppendInfo.firstOffset), log.firstUnstableOffset.map(_.messageOffset))
+ assertEquals(Some(0L), log.firstUnstableOffset.map(_.segmentBaseOffset))
+ assertEquals(3L, log.logEndOffsetMetadata.segmentBaseOffset)
+
+ // now abort the transaction
+ val appendInfo = log.appendAsLeader(endTxnRecords(ControlRecordType.ABORT, pid, epoch),
+ isFromClient = false, leaderEpoch = 0)
+ log.onHighWatermarkIncremented(appendInfo.lastOffset + 1)
+ assertEquals(None, log.firstUnstableOffset.map(_.messageOffset))
+
+ // now check that a fetch includes the aborted transaction
+ val fetchDataInfo = log.read(0L, 2048, isolationLevel = IsolationLevel.READ_COMMITTED)
+ assertEquals(1, fetchDataInfo.abortedTransactions.size)
+
+ assertTrue(fetchDataInfo.abortedTransactions.isDefined)
+ assertEquals(new AbortedTransaction(pid, 0), fetchDataInfo.abortedTransactions.get.head)
+ }
+
+ private def createLog(messageSizeInBytes: Int, retentionMs: Int = -1, retentionBytes: Int = -1,
+ cleanupPolicy: String = "delete", messagesPerSegment: Int = 5,
+ maxPidExpirationMs: Int = 300000, pidExpirationCheckIntervalMs: Int = 30000,
+ pidSnapshotIntervalMs: Int = 60000): Log = {
val logProps = new Properties()
logProps.put(LogConfig.SegmentBytesProp, messageSizeInBytes * messagesPerSegment: Integer)
logProps.put(LogConfig.RetentionMsProp, retentionMs: Integer)
@@ -2009,8 +2376,70 @@ class LogTest {
scheduler = time.scheduler,
time = time,
maxPidExpirationMs = maxPidExpirationMs,
- pidExpirationCheckIntervalMs = pidExpirationCheckIntervalMs,
- pidSnapshotIntervalMs = pidSnapshotIntervalMs)
+ pidExpirationCheckIntervalMs = pidExpirationCheckIntervalMs)
log
}
+
+ private def allAbortedTransactions(log: Log) = log.logSegments.flatMap(_.txnIndex.allAbortedTxns)
+
+ private def appendTransactionalAsLeader(log: Log, pid: Long, producerEpoch: Short): Int => Unit = {
+ var sequence = 0
+ numRecords: Int => {
+ val simpleRecords = (sequence until sequence + numRecords).map { seq =>
+ new SimpleRecord(s"$seq".getBytes)
+ }
+ val records = MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid,
+ producerEpoch, sequence, simpleRecords: _*)
+ log.appendAsLeader(records, leaderEpoch = 0)
+ sequence += numRecords
+ }
+ }
+
+ private def appendEndTxnMarkerAsLeader(log: Log, pid: Long, producerEpoch: Short,
+ controlType: ControlRecordType, coordinatorEpoch: Int = 0): Unit = {
+ val records = endTxnRecords(controlType, pid, producerEpoch, coordinatorEpoch = coordinatorEpoch)
+ log.appendAsLeader(records, isFromClient = false, leaderEpoch = 0)
+ }
+
+ private def appendNonTransactionalAsLeader(log: Log, numRecords: Int): Unit = {
+ val simpleRecords = (0 until numRecords).map { seq =>
+ new SimpleRecord(s"$seq".getBytes)
+ }
+ val records = MemoryRecords.withRecords(CompressionType.NONE, simpleRecords: _*)
+ log.appendAsLeader(records, leaderEpoch = 0)
+ }
+
+ private def appendTransactionalToBuffer(buffer: ByteBuffer, pid: Long, epoch: Short): (Long, Int) => Unit = {
+ var sequence = 0
+ (offset: Long, numRecords: Int) => {
+ val builder = MemoryRecords.builder(buffer, CompressionType.NONE, offset, pid, epoch, sequence, true)
+ for (seq <- sequence until sequence + numRecords) {
+ val record = new SimpleRecord(s"$seq".getBytes)
+ builder.append(record)
+ }
+
+ sequence += numRecords
+ builder.close()
+ }
+ }
+
+ private def appendEndTxnMarkerToBuffer(buffer: ByteBuffer, producerId: Long, producerEpoch: Short, offset: Long,
+ controlType: ControlRecordType, coordinatorEpoch: Int = 0): Unit = {
+ val marker = new EndTransactionMarker(controlType, coordinatorEpoch)
+ MemoryRecords.writeEndTransactionalMarker(buffer, offset, producerId, producerEpoch, marker)
+ }
+
+ private def appendNonTransactionalToBuffer(buffer: ByteBuffer, offset: Long, numRecords: Int): Unit = {
+ val builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.CREATE_TIME, offset)
+ (0 until numRecords).foreach { seq =>
+ builder.append(new SimpleRecord(s"$seq".getBytes))
+ }
+ builder.close()
+ }
+
+ private def appendAsFollower(log: Log, records: MemoryRecords, leaderEpoch: Int = 0): Unit = {
+ records.batches.asScala.foreach(_.setPartitionLeaderEpoch(leaderEpoch))
+ log.appendAsFollower(records)
+ }
+
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala
index 5b2c660..61fae80 100644
--- a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala
@@ -22,6 +22,7 @@ import kafka.common.LongRef
import kafka.message.{DefaultCompressionCodec, GZIPCompressionCodec, NoCompressionCodec, SnappyCompressionCodec}
import org.apache.kafka.common.errors.InvalidTimestampException
import org.apache.kafka.common.record._
+import org.apache.kafka.test.TestUtils
import org.junit.Assert._
import org.junit.Test
@@ -47,7 +48,8 @@ class LogValidatorTest {
magic = magic,
timestampType = TimestampType.LOG_APPEND_TIME,
timestampDiffMaxMs = 1000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true)
val validatedRecords = validatedResults.validatedRecords
assertEquals("message set size should not change", records.records.asScala.size, validatedRecords.records.asScala.size)
validatedRecords.batches.asScala.foreach(batch => validateLogAppendTime(now, batch))
@@ -79,7 +81,8 @@ class LogValidatorTest {
magic = targetMagic,
timestampType = TimestampType.LOG_APPEND_TIME,
timestampDiffMaxMs = 1000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true)
val validatedRecords = validatedResults.validatedRecords
assertEquals("message set size should not change", records.records.asScala.size, validatedRecords.records.asScala.size)
@@ -115,7 +118,8 @@ class LogValidatorTest {
magic = magic,
timestampType = TimestampType.LOG_APPEND_TIME,
timestampDiffMaxMs = 1000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true)
val validatedRecords = validatedResults.validatedRecords
assertEquals("message set size should not change", records.records.asScala.size,
@@ -141,14 +145,19 @@ class LogValidatorTest {
private def checkNonCompressed(magic: Byte) {
val now = System.currentTimeMillis()
val timestampSeq = Seq(now - 1, now + 1, now)
- val producerId = if (magic >= RecordBatch.MAGIC_VALUE_V2) 1324L else RecordBatch.NO_PRODUCER_ID
- val producerEpoch = if (magic >= RecordBatch.MAGIC_VALUE_V2) 10: Short else RecordBatch.NO_PRODUCER_EPOCH
- val baseSequence = if (magic >= RecordBatch.MAGIC_VALUE_V2) 20 else RecordBatch.NO_SEQUENCE
- val partitionLeaderEpoch = if (magic >= RecordBatch.MAGIC_VALUE_V2) 40 else RecordBatch.NO_PARTITION_LEADER_EPOCH
- val records = MemoryRecords.withRecords(magic, 0L, CompressionType.NONE, producerId, producerEpoch, baseSequence,
- partitionLeaderEpoch, new SimpleRecord(timestampSeq(0), "hello".getBytes),
- new SimpleRecord(timestampSeq(1), "there".getBytes), new SimpleRecord(timestampSeq(2), "beautiful".getBytes))
+ val (producerId, producerEpoch, baseSequence, isTransactional, partitionLeaderEpoch) =
+ if (magic >= RecordBatch.MAGIC_VALUE_V2)
+ (1324L, 10.toShort, 984, true, 40)
+ else
+ (RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false,
+ RecordBatch.NO_PARTITION_LEADER_EPOCH)
+
+ val records = MemoryRecords.withRecords(magic, 0L, CompressionType.GZIP, TimestampType.CREATE_TIME, producerId,
+ producerEpoch, baseSequence, partitionLeaderEpoch, isTransactional,
+ new SimpleRecord(timestampSeq(0), "hello".getBytes),
+ new SimpleRecord(timestampSeq(1), "there".getBytes),
+ new SimpleRecord(timestampSeq(2), "beautiful".getBytes))
val validatingResults = LogValidator.validateMessagesAndAssignOffsets(records,
offsetCounter = new LongRef(0),
@@ -159,7 +168,8 @@ class LogValidatorTest {
magic = magic,
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 1000L,
- partitionLeaderEpoch = partitionLeaderEpoch)
+ partitionLeaderEpoch = partitionLeaderEpoch,
+ isFromClient = true)
val validatedRecords = validatingResults.validatedRecords
var i = 0
@@ -170,6 +180,7 @@ class LogValidatorTest {
assertEquals(producerEpoch, batch.producerEpoch)
assertEquals(producerId, batch.producerId)
assertEquals(baseSequence, batch.baseSequence)
+ assertEquals(isTransactional, batch.isTransactional)
assertEquals(partitionLeaderEpoch, batch.partitionLeaderEpoch)
for (record <- batch.asScala) {
assertTrue(record.isValid)
@@ -195,14 +206,19 @@ class LogValidatorTest {
private def checkRecompression(magic: Byte): Unit = {
val now = System.currentTimeMillis()
val timestampSeq = Seq(now - 1, now + 1, now)
- val producerId = if (magic >= RecordBatch.MAGIC_VALUE_V2) 1324L else RecordBatch.NO_PRODUCER_ID
- val producerEpoch = if (magic >= RecordBatch.MAGIC_VALUE_V2) 10: Short else RecordBatch.NO_PRODUCER_EPOCH
- val baseSequence = if (magic >= RecordBatch.MAGIC_VALUE_V2) 20 else RecordBatch.NO_SEQUENCE
- val partitionLeaderEpoch = if (magic >= RecordBatch.MAGIC_VALUE_V2) 40 else RecordBatch.NO_PARTITION_LEADER_EPOCH
- val records = MemoryRecords.withRecords(magic, 0L, CompressionType.NONE, producerId, producerEpoch, baseSequence,
- partitionLeaderEpoch, new SimpleRecord(timestampSeq(0), "hello".getBytes),
- new SimpleRecord(timestampSeq(1), "there".getBytes), new SimpleRecord(timestampSeq(2), "beautiful".getBytes))
+ val (producerId, producerEpoch, baseSequence, isTransactional, partitionLeaderEpoch) =
+ if (magic >= RecordBatch.MAGIC_VALUE_V2)
+ (1324L, 10.toShort, 984, true, 40)
+ else
+ (RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false,
+ RecordBatch.NO_PARTITION_LEADER_EPOCH)
+
+ val records = MemoryRecords.withRecords(magic, 0L, CompressionType.GZIP, TimestampType.CREATE_TIME, producerId,
+ producerEpoch, baseSequence, partitionLeaderEpoch, isTransactional,
+ new SimpleRecord(timestampSeq(0), "hello".getBytes),
+ new SimpleRecord(timestampSeq(1), "there".getBytes),
+ new SimpleRecord(timestampSeq(2), "beautiful".getBytes))
val validatingResults = LogValidator.validateMessagesAndAssignOffsets(records,
offsetCounter = new LongRef(0),
@@ -213,7 +229,8 @@ class LogValidatorTest {
magic = magic,
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 1000L,
- partitionLeaderEpoch = partitionLeaderEpoch)
+ partitionLeaderEpoch = partitionLeaderEpoch,
+ isFromClient = true)
val validatedRecords = validatingResults.validatedRecords
var i = 0
@@ -257,7 +274,8 @@ class LogValidatorTest {
compactedTopic = false,
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 1000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true)
val validatedRecords = validatedResults.validatedRecords
for (batch <- validatedRecords.batches.asScala) {
@@ -292,7 +310,8 @@ class LogValidatorTest {
compactedTopic = false,
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 1000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true)
val validatedRecords = validatedResults.validatedRecords
for (batch <- validatedRecords.batches.asScala) {
@@ -317,24 +336,31 @@ class LogValidatorTest {
private def checkCompressed(magic: Byte) {
val now = System.currentTimeMillis()
val timestampSeq = Seq(now - 1, now + 1, now)
- val producerId = if (magic >= RecordBatch.MAGIC_VALUE_V2) 1324L else RecordBatch.NO_PRODUCER_ID
- val producerEpoch = if (magic >= RecordBatch.MAGIC_VALUE_V2) 10: Short else RecordBatch.NO_PRODUCER_EPOCH
- val baseSequence = if (magic >= RecordBatch.MAGIC_VALUE_V2) 20 else RecordBatch.NO_SEQUENCE
- val partitionLeaderEpoch = if (magic >= RecordBatch.MAGIC_VALUE_V2) 40 else RecordBatch.NO_PARTITION_LEADER_EPOCH
- val records = MemoryRecords.withRecords(magic, 0L, CompressionType.GZIP, producerId, producerEpoch, baseSequence,
- partitionLeaderEpoch, new SimpleRecord(timestampSeq(0), "hello".getBytes),
- new SimpleRecord(timestampSeq(1), "there".getBytes), new SimpleRecord(timestampSeq(2), "beautiful".getBytes))
+
+ val (producerId, producerEpoch, baseSequence, isTransactional, partitionLeaderEpoch) =
+ if (magic >= RecordBatch.MAGIC_VALUE_V2)
+ (1324L, 10.toShort, 984, true, 40)
+ else
+ (RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false,
+ RecordBatch.NO_PARTITION_LEADER_EPOCH)
+
+ val records = MemoryRecords.withRecords(magic, 0L, CompressionType.GZIP, TimestampType.CREATE_TIME, producerId,
+ producerEpoch, baseSequence, partitionLeaderEpoch, isTransactional,
+ new SimpleRecord(timestampSeq(0), "hello".getBytes),
+ new SimpleRecord(timestampSeq(1), "there".getBytes),
+ new SimpleRecord(timestampSeq(2), "beautiful".getBytes))
val validatedResults = LogValidator.validateMessagesAndAssignOffsets(records,
- offsetCounter = new LongRef(0),
- now = System.currentTimeMillis(),
- sourceCodec = DefaultCompressionCodec,
- targetCodec = DefaultCompressionCodec,
- magic = magic,
- compactedTopic = false,
- timestampType = TimestampType.CREATE_TIME,
- timestampDiffMaxMs = 1000L,
- partitionLeaderEpoch = partitionLeaderEpoch)
+ offsetCounter = new LongRef(0),
+ now = System.currentTimeMillis(),
+ sourceCodec = DefaultCompressionCodec,
+ targetCodec = DefaultCompressionCodec,
+ magic = magic,
+ compactedTopic = false,
+ timestampType = TimestampType.CREATE_TIME,
+ timestampDiffMaxMs = 1000L,
+ partitionLeaderEpoch = partitionLeaderEpoch,
+ isFromClient = true)
val validatedRecords = validatedResults.validatedRecords
var i = 0
@@ -378,7 +404,8 @@ class LogValidatorTest {
magic = RecordBatch.MAGIC_VALUE_V1,
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 1000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true)
}
@Test(expected = classOf[InvalidTimestampException])
@@ -396,7 +423,8 @@ class LogValidatorTest {
magic = RecordBatch.MAGIC_VALUE_V2,
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 1000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true)
}
@Test(expected = classOf[InvalidTimestampException])
@@ -414,7 +442,8 @@ class LogValidatorTest {
compactedTopic = false,
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 1000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true)
}
@Test(expected = classOf[InvalidTimestampException])
@@ -432,7 +461,8 @@ class LogValidatorTest {
compactedTopic = false,
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 1000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true)
}
@Test
@@ -449,7 +479,8 @@ class LogValidatorTest {
compactedTopic = false,
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 1000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH).validatedRecords, offset)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true).validatedRecords, offset)
}
@Test
@@ -466,7 +497,8 @@ class LogValidatorTest {
magic = RecordBatch.MAGIC_VALUE_V0,
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 1000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH).validatedRecords, offset)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true).validatedRecords, offset)
}
@Test
@@ -484,7 +516,8 @@ class LogValidatorTest {
magic = RecordBatch.MAGIC_VALUE_V1,
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH).validatedRecords
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true).validatedRecords
checkOffsets(messageWithOffset, offset)
}
@@ -503,7 +536,8 @@ class LogValidatorTest {
magic = RecordBatch.MAGIC_VALUE_V2,
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH).validatedRecords
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true).validatedRecords
checkOffsets(messageWithOffset, offset)
}
@@ -523,7 +557,8 @@ class LogValidatorTest {
magic = RecordBatch.MAGIC_VALUE_V1,
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH).validatedRecords
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true).validatedRecords
checkOffsets(compressedMessagesWithOffset, offset)
}
@@ -543,7 +578,8 @@ class LogValidatorTest {
magic = RecordBatch.MAGIC_VALUE_V2,
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH).validatedRecords
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true).validatedRecords
checkOffsets(compressedMessagesWithOffset, offset)
}
@@ -561,7 +597,8 @@ class LogValidatorTest {
magic = RecordBatch.MAGIC_VALUE_V1,
timestampType = TimestampType.LOG_APPEND_TIME,
timestampDiffMaxMs = 1000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH).validatedRecords, offset)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true).validatedRecords, offset)
}
@Test
@@ -578,7 +615,8 @@ class LogValidatorTest {
magic = RecordBatch.MAGIC_VALUE_V2,
timestampType = TimestampType.LOG_APPEND_TIME,
timestampDiffMaxMs = 1000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH).validatedRecords, offset)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true).validatedRecords, offset)
}
@Test
@@ -595,7 +633,8 @@ class LogValidatorTest {
magic = RecordBatch.MAGIC_VALUE_V1,
timestampType = TimestampType.LOG_APPEND_TIME,
timestampDiffMaxMs = 1000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH).validatedRecords, offset)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true).validatedRecords, offset)
}
@Test
@@ -612,7 +651,48 @@ class LogValidatorTest {
magic = RecordBatch.MAGIC_VALUE_V2,
timestampType = TimestampType.LOG_APPEND_TIME,
timestampDiffMaxMs = 1000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH).validatedRecords, offset)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true).validatedRecords, offset)
+ }
+
+ @Test(expected = classOf[InvalidRecordException])
+ def testControlRecordsNotAllowedFromClients() {
+ val offset = 1234567
+ val endTxnMarker = new EndTransactionMarker(ControlRecordType.COMMIT, 0)
+ val records = MemoryRecords.withEndTransactionMarker(23423L, 5, endTxnMarker)
+ LogValidator.validateMessagesAndAssignOffsets(records,
+ offsetCounter = new LongRef(offset),
+ now = System.currentTimeMillis(),
+ sourceCodec = NoCompressionCodec,
+ targetCodec = NoCompressionCodec,
+ compactedTopic = false,
+ magic = RecordBatch.CURRENT_MAGIC_VALUE,
+ timestampType = TimestampType.CREATE_TIME,
+ timestampDiffMaxMs = 5000L,
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true)
+ }
+
+ @Test
+ def testControlRecordsNotCompressed() {
+ val offset = 1234567
+ val endTxnMarker = new EndTransactionMarker(ControlRecordType.COMMIT, 0)
+ val records = MemoryRecords.withEndTransactionMarker(23423L, 5, endTxnMarker)
+ val result = LogValidator.validateMessagesAndAssignOffsets(records,
+ offsetCounter = new LongRef(offset),
+ now = System.currentTimeMillis(),
+ sourceCodec = NoCompressionCodec,
+ targetCodec = SnappyCompressionCodec,
+ compactedTopic = false,
+ magic = RecordBatch.CURRENT_MAGIC_VALUE,
+ timestampType = TimestampType.CREATE_TIME,
+ timestampDiffMaxMs = 5000L,
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = false)
+ val batches = TestUtils.toList(result.validatedRecords.batches)
+ assertEquals(1, batches.size)
+ val batch = batches.get(0)
+ assertFalse(batch.isCompressed)
}
@Test
@@ -630,7 +710,8 @@ class LogValidatorTest {
magic = RecordBatch.MAGIC_VALUE_V0,
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH).validatedRecords, offset)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true).validatedRecords, offset)
}
@Test
@@ -648,7 +729,8 @@ class LogValidatorTest {
magic = RecordBatch.MAGIC_VALUE_V0,
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH).validatedRecords, offset)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true).validatedRecords, offset)
}
@Test
@@ -665,7 +747,8 @@ class LogValidatorTest {
magic = RecordBatch.MAGIC_VALUE_V2,
timestampType = TimestampType.LOG_APPEND_TIME,
timestampDiffMaxMs = 1000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH).validatedRecords, offset)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true).validatedRecords, offset)
}
@Test
@@ -682,7 +765,8 @@ class LogValidatorTest {
magic = RecordBatch.MAGIC_VALUE_V2,
timestampType = TimestampType.LOG_APPEND_TIME,
timestampDiffMaxMs = 1000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH).validatedRecords, offset)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true).validatedRecords, offset)
}
@Test
@@ -700,7 +784,8 @@ class LogValidatorTest {
magic = RecordBatch.MAGIC_VALUE_V1,
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH).validatedRecords, offset)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true).validatedRecords, offset)
}
@Test
@@ -718,7 +803,8 @@ class LogValidatorTest {
magic = RecordBatch.MAGIC_VALUE_V1,
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH).validatedRecords, offset)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true).validatedRecords, offset)
}
@Test
@@ -736,7 +822,8 @@ class LogValidatorTest {
magic = RecordBatch.MAGIC_VALUE_V0,
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH).validatedRecords, offset)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true).validatedRecords, offset)
}
@Test
@@ -754,7 +841,8 @@ class LogValidatorTest {
magic = RecordBatch.MAGIC_VALUE_V0,
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH).validatedRecords, offset)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true).validatedRecords, offset)
}
@Test(expected = classOf[InvalidRecordException])
@@ -770,7 +858,8 @@ class LogValidatorTest {
magic = RecordBatch.MAGIC_VALUE_V1,
timestampType = TimestampType.CREATE_TIME,
timestampDiffMaxMs = 5000L,
- partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH)
+ partitionLeaderEpoch = RecordBatch.NO_PARTITION_LEADER_EPOCH,
+ isFromClient = true)
}
private def createRecords(magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE,
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala b/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala
index 7618cf7..506d99c 100644
--- a/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala
+++ b/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala
@@ -95,7 +95,29 @@ class OffsetIndexTest extends JUnitSuite {
idx.append(51, 0)
idx.append(50, 1)
}
-
+
+ @Test
+ def testFetchUpperBoundOffset() {
+ val first = OffsetPosition(0, 0)
+ val second = OffsetPosition(1, 10)
+ val third = OffsetPosition(2, 23)
+ val fourth = OffsetPosition(3, 37)
+
+ assertEquals(None, idx.fetchUpperBoundOffset(first, 5))
+
+ for (offsetPosition <- Seq(first, second, third, fourth))
+ idx.append(offsetPosition.offset, offsetPosition.position)
+
+ assertEquals(Some(second), idx.fetchUpperBoundOffset(first, 5))
+ assertEquals(Some(second), idx.fetchUpperBoundOffset(first, 10))
+ assertEquals(Some(third), idx.fetchUpperBoundOffset(first, 23))
+ assertEquals(Some(third), idx.fetchUpperBoundOffset(first, 22))
+ assertEquals(Some(fourth), idx.fetchUpperBoundOffset(second, 24))
+ assertEquals(None, idx.fetchUpperBoundOffset(fourth, 1))
+ assertEquals(None, idx.fetchUpperBoundOffset(first, 200))
+ assertEquals(None, idx.fetchUpperBoundOffset(second, 200))
+ }
+
@Test
def testReopen() {
val first = OffsetPosition(51, 0)
http://git-wip-us.apache.org/repos/asf/kafka/blob/e71dce89/core/src/test/scala/unit/kafka/log/ProducerIdMappingTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/ProducerIdMappingTest.scala b/core/src/test/scala/unit/kafka/log/ProducerIdMappingTest.scala
deleted file mode 100644
index 1bf983c..0000000
--- a/core/src/test/scala/unit/kafka/log/ProducerIdMappingTest.scala
+++ /dev/null
@@ -1,291 +0,0 @@
-/**
- * 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.
- */
-
-package kafka.log
-
-import java.io.File
-import java.util.Properties
-
-import kafka.utils.TestUtils
-import org.apache.kafka.common.TopicPartition
-import org.apache.kafka.common.errors.{DuplicateSequenceNumberException, OutOfOrderSequenceException, ProducerFencedException}
-import org.apache.kafka.common.utils.{MockTime, Utils}
-import org.junit.Assert._
-import org.junit.{After, Before, Test}
-import org.scalatest.junit.JUnitSuite
-
-class ProducerIdMappingTest extends JUnitSuite {
- var idMappingDir: File = null
- var config: LogConfig = null
- var idMapping: ProducerIdMapping = null
- val partition = new TopicPartition("test", 0)
- val pid = 1L
- val maxPidExpirationMs = 60 * 1000
- val time = new MockTime
-
- @Before
- def setUp(): Unit = {
- config = LogConfig(new Properties)
- idMappingDir = TestUtils.tempDir()
- idMapping = new ProducerIdMapping(config, partition, idMappingDir, maxPidExpirationMs)
- }
-
- @After
- def tearDown(): Unit = {
- Utils.delete(idMappingDir)
- }
-
- @Test
- def testBasicIdMapping(): Unit = {
- val epoch = 0.toShort
-
- // First entry for id 0 added
- checkAndUpdate(idMapping, pid, 0, epoch, 0L, 0L)
-
- // Second entry for id 0 added
- checkAndUpdate(idMapping, pid, 1, epoch, 0L, 1L)
-
- // Duplicate sequence number (matches previous sequence number)
- assertThrows[DuplicateSequenceNumberException] {
- checkAndUpdate(idMapping, pid, 1, epoch, 0L, 1L)
- }
-
- // Invalid sequence number (greater than next expected sequence number)
- assertThrows[OutOfOrderSequenceException] {
- checkAndUpdate(idMapping, pid, 5, epoch, 0L, 2L)
- }
-
- // Change epoch
- checkAndUpdate(idMapping, pid, 0, (epoch + 1).toShort, 0L, 3L)
-
- // Incorrect epoch
- assertThrows[ProducerFencedException] {
- checkAndUpdate(idMapping, pid, 0, epoch, 0L, 4L)
- }
- }
-
- @Test
- def testTakeSnapshot(): Unit = {
- val epoch = 0.toShort
- checkAndUpdate(idMapping, pid, 0, epoch, 0L, 0L)
- checkAndUpdate(idMapping, pid, 1, epoch, 1L, 1L)
-
- // Take snapshot
- idMapping.maybeTakeSnapshot()
-
- // Check that file exists and it is not empty
- assertEquals("Directory doesn't contain a single file as expected", 1, idMappingDir.list().length)
- assertTrue("Snapshot file is empty", idMappingDir.list().head.length > 0)
- }
-
- @Test
- def testRecoverFromSnapshot(): Unit = {
- val epoch = 0.toShort
- checkAndUpdate(idMapping, pid, 0, epoch, 0L, time.milliseconds)
- checkAndUpdate(idMapping, pid, 1, epoch, 1L, time.milliseconds)
- idMapping.maybeTakeSnapshot()
- val recoveredMapping = new ProducerIdMapping(config, partition, idMappingDir, maxPidExpirationMs)
- recoveredMapping.truncateAndReload(0L, 3L, time.milliseconds)
-
- // entry added after recovery
- checkAndUpdate(recoveredMapping, pid, 2, epoch, 2L, time.milliseconds)
- }
-
- @Test(expected = classOf[OutOfOrderSequenceException])
- def testRemoveExpiredPidsOnReload(): Unit = {
- val epoch = 0.toShort
- checkAndUpdate(idMapping, pid, 0, epoch, 0L, 0)
- checkAndUpdate(idMapping, pid, 1, epoch, 1L, 1)
-
- idMapping.maybeTakeSnapshot()
- val recoveredMapping = new ProducerIdMapping(config, partition, idMappingDir, maxPidExpirationMs)
- recoveredMapping.truncateAndReload(0L, 1L, 70000)
-
- // entry added after recovery. The pid should be expired now, and would not exist in the pid mapping. Hence
- // we should get an out of order sequence exception.
- checkAndUpdate(recoveredMapping, pid, 2, epoch, 2L, 70001)
- }
-
- @Test
- def testRemoveOldSnapshot(): Unit = {
- val epoch = 0.toShort
-
- checkAndUpdate(idMapping, pid, 0, epoch, 0L)
- checkAndUpdate(idMapping, pid, 1, epoch, 1L)
- idMapping.maybeTakeSnapshot()
- assertEquals(1, idMappingDir.listFiles().length)
- assertEquals(Set(2), currentSnapshotOffsets)
-
- checkAndUpdate(idMapping, pid, 2, epoch, 2L)
- idMapping.maybeTakeSnapshot()
- assertEquals(2, idMappingDir.listFiles().length)
- assertEquals(Set(2, 3), currentSnapshotOffsets)
-
- // we only retain two snapshot files, so the next snapshot should cause the oldest to be deleted
- checkAndUpdate(idMapping, pid, 3, epoch, 3L)
- idMapping.maybeTakeSnapshot()
- assertEquals(2, idMappingDir.listFiles().length)
- assertEquals(Set(3, 4), currentSnapshotOffsets)
- }
-
- @Test
- def testTruncate(): Unit = {
- val epoch = 0.toShort
-
- checkAndUpdate(idMapping, pid, 0, epoch, 0L)
- checkAndUpdate(idMapping, pid, 1, epoch, 1L)
- idMapping.maybeTakeSnapshot()
- assertEquals(1, idMappingDir.listFiles().length)
- assertEquals(Set(2), currentSnapshotOffsets)
-
- checkAndUpdate(idMapping, pid, 2, epoch, 2L)
- idMapping.maybeTakeSnapshot()
- assertEquals(2, idMappingDir.listFiles().length)
- assertEquals(Set(2, 3), currentSnapshotOffsets)
-
- idMapping.truncate()
-
- assertEquals(0, idMappingDir.listFiles().length)
- assertEquals(Set(), currentSnapshotOffsets)
-
- checkAndUpdate(idMapping, pid, 0, epoch, 0L)
- idMapping.maybeTakeSnapshot()
- assertEquals(1, idMappingDir.listFiles().length)
- assertEquals(Set(1), currentSnapshotOffsets)
- }
-
- @Test
- def testExpirePids(): Unit = {
- val epoch = 0.toShort
-
- checkAndUpdate(idMapping, pid, 0, epoch, 0L)
- checkAndUpdate(idMapping, pid, 1, epoch, 1L)
- idMapping.maybeTakeSnapshot()
-
- val anotherPid = 2L
- checkAndUpdate(idMapping, anotherPid, 0, epoch, 2L)
- checkAndUpdate(idMapping, anotherPid, 1, epoch, 3L)
- idMapping.maybeTakeSnapshot()
- assertEquals(Set(2, 4), currentSnapshotOffsets)
-
- idMapping.expirePids(2)
- assertEquals(Set(4), currentSnapshotOffsets)
- assertEquals(Set(anotherPid), idMapping.activePids.keySet)
- assertEquals(None, idMapping.lastEntry(pid))
-
- val maybeEntry = idMapping.lastEntry(anotherPid)
- assertTrue(maybeEntry.isDefined)
- assertEquals(3L, maybeEntry.get.lastOffset)
-
- idMapping.expirePids(3)
- assertEquals(Set(anotherPid), idMapping.activePids.keySet)
- assertEquals(Set(4), currentSnapshotOffsets)
- assertEquals(4, idMapping.mapEndOffset)
-
- idMapping.expirePids(5)
- assertEquals(Set(), idMapping.activePids.keySet)
- assertEquals(Set(), currentSnapshotOffsets)
- assertEquals(5, idMapping.mapEndOffset)
-
- idMapping.maybeTakeSnapshot()
- // shouldn't be any new snapshot because the log is empty
- assertEquals(Set(), currentSnapshotOffsets)
- }
-
- @Test
- def testSkipSnapshotIfOffsetUnchanged(): Unit = {
- val epoch = 0.toShort
- checkAndUpdate(idMapping, pid, 0, epoch, 0L, 0L)
-
- idMapping.maybeTakeSnapshot()
- assertEquals(1, idMappingDir.listFiles().length)
- assertEquals(Set(1), currentSnapshotOffsets)
-
- // nothing changed so there should be no new snapshot
- idMapping.maybeTakeSnapshot()
- assertEquals(1, idMappingDir.listFiles().length)
- assertEquals(Set(1), currentSnapshotOffsets)
- }
-
- @Test
- def testStartOffset(): Unit = {
- val epoch = 0.toShort
- val pid2 = 2L
- checkAndUpdate(idMapping, pid2, 0, epoch, 0L, 1L)
- checkAndUpdate(idMapping, pid, 0, epoch, 1L, 2L)
- checkAndUpdate(idMapping, pid, 1, epoch, 2L, 3L)
- checkAndUpdate(idMapping, pid, 2, epoch, 3L, 4L)
- idMapping.maybeTakeSnapshot()
-
- intercept[OutOfOrderSequenceException] {
- val recoveredMapping = new ProducerIdMapping(config, partition, idMappingDir, maxPidExpirationMs)
- recoveredMapping.truncateAndReload(0L, 1L, time.milliseconds)
- checkAndUpdate(recoveredMapping, pid2, 1, epoch, 4L, 5L)
- }
- }
-
- @Test(expected = classOf[OutOfOrderSequenceException])
- def testPidExpirationTimeout() {
- val epoch = 5.toShort
- val sequence = 37
- checkAndUpdate(idMapping, pid, sequence, epoch, 1L)
- time.sleep(maxPidExpirationMs + 1)
- idMapping.removeExpiredPids(time.milliseconds)
- checkAndUpdate(idMapping, pid, sequence + 1, epoch, 1L)
- }
-
- @Test
- def testLoadPid() {
- val epoch = 5.toShort
- val sequence = 37
- val createTimeMs = time.milliseconds
- idMapping.load(pid, ProducerIdEntry(epoch, sequence, 0L, 1, createTimeMs), time.milliseconds)
- checkAndUpdate(idMapping, pid, sequence + 1, epoch, 2L)
- }
-
- @Test(expected = classOf[OutOfOrderSequenceException])
- def testLoadIgnoresExpiredPids() {
- val epoch = 5.toShort
- val sequence = 37
-
- val createTimeMs = time.milliseconds
- time.sleep(maxPidExpirationMs + 1)
- val loadTimeMs = time.milliseconds
- idMapping.load(pid, ProducerIdEntry(epoch, sequence, 0L, 1, createTimeMs), loadTimeMs)
-
- // entry wasn't loaded, so this should fail
- checkAndUpdate(idMapping, pid, sequence + 1, epoch, 2L)
- }
-
- private def checkAndUpdate(mapping: ProducerIdMapping,
- pid: Long,
- seq: Int,
- epoch: Short,
- lastOffset: Long,
- timestamp: Long = time.milliseconds()): Unit = {
- val offsetDelta = 0
- val incomingPidEntry = ProducerIdEntry(epoch, seq, lastOffset, offsetDelta, timestamp)
- val producerAppendInfo = new ProducerAppendInfo(pid, mapping.lastEntry(pid).getOrElse(ProducerIdEntry.Empty))
- producerAppendInfo.append(incomingPidEntry)
- mapping.update(producerAppendInfo)
- mapping.updateMapEndOffset(lastOffset + 1)
- }
-
- private def currentSnapshotOffsets =
- idMappingDir.listFiles().map(file => Log.offsetFromFilename(file.getName)).toSet
-
-}