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
-
-}