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 2016/12/13 18:41:28 UTC

[1/9] kafka git commit: KAFKA-4390; Replace MessageSet usage with client-side alternatives

Repository: kafka
Updated Branches:
  refs/heads/trunk 6626b058c -> 67f1e5b91


http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala
index 3d1b485..4467394 100644
--- a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala
@@ -24,12 +24,12 @@ import kafka.utils.TestUtils
 import kafka.utils.TestUtils._
 import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
 import org.apache.kafka.common.TopicPartition
-import org.apache.kafka.common.protocol.{ApiKeys, Errors, ProtoUtils}
-import org.apache.kafka.common.record.{LogEntry, MemoryRecords}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.record.LogEntry
 import org.apache.kafka.common.requests.{FetchRequest, FetchResponse}
 import org.apache.kafka.common.serialization.StringSerializer
-import org.junit.Test
 import org.junit.Assert._
+import org.junit.Test
 
 import scala.collection.JavaConverters._
 import scala.util.Random
@@ -120,13 +120,13 @@ class FetchRequestTest extends BaseRequestTest {
     val fetchResponse3 = sendFetchRequest(leaderId, fetchRequest3)
     assertEquals(shuffledTopicPartitions3, fetchResponse3.responseData.keySet.asScala.toSeq)
     val responseSize3 = fetchResponse3.responseData.asScala.values.map { partitionData =>
-      logEntries(partitionData).map(_.size).sum
+      logEntries(partitionData).map(_.sizeInBytes).sum
     }.sum
     assertTrue(responseSize3 <= maxResponseBytes)
     val partitionData3 = fetchResponse3.responseData.get(partitionWithLargeMessage1)
     assertEquals(Errors.NONE.code, partitionData3.errorCode)
     assertTrue(partitionData3.highWatermark > 0)
-    val size3 = logEntries(partitionData3).map(_.size).sum
+    val size3 = logEntries(partitionData3).map(_.sizeInBytes).sum
     assertTrue(s"Expected $size3 to be smaller than $maxResponseBytes", size3 <= maxResponseBytes)
     assertTrue(s"Expected $size3 to be larger than $maxPartitionBytes", size3 > maxPartitionBytes)
     assertTrue(maxPartitionBytes < partitionData3.records.sizeInBytes)
@@ -138,13 +138,13 @@ class FetchRequestTest extends BaseRequestTest {
     val fetchResponse4 = sendFetchRequest(leaderId, fetchRequest4)
     assertEquals(shuffledTopicPartitions4, fetchResponse4.responseData.keySet.asScala.toSeq)
     val nonEmptyPartitions4 = fetchResponse4.responseData.asScala.toSeq.collect {
-      case (tp, partitionData) if logEntries(partitionData).map(_.size).sum > 0 => tp
+      case (tp, partitionData) if logEntries(partitionData).map(_.sizeInBytes).sum > 0 => tp
     }
     assertEquals(Seq(partitionWithLargeMessage2), nonEmptyPartitions4)
     val partitionData4 = fetchResponse4.responseData.get(partitionWithLargeMessage2)
     assertEquals(Errors.NONE.code, partitionData4.errorCode)
     assertTrue(partitionData4.highWatermark > 0)
-    val size4 = logEntries(partitionData4).map(_.size).sum
+    val size4 = logEntries(partitionData4).map(_.sizeInBytes).sum
     assertTrue(s"Expected $size4 to be larger than $maxResponseBytes", size4 > maxResponseBytes)
     assertTrue(maxResponseBytes < partitionData4.records.sizeInBytes)
   }
@@ -161,12 +161,11 @@ class FetchRequestTest extends BaseRequestTest {
     assertEquals(Errors.NONE.code, partitionData.errorCode)
     assertTrue(partitionData.highWatermark > 0)
     assertEquals(maxPartitionBytes, partitionData.records.sizeInBytes)
-    assertEquals(0, logEntries(partitionData).map(_.size).sum)
+    assertEquals(0, logEntries(partitionData).map(_.sizeInBytes).sum)
   }
 
   private def logEntries(partitionData: FetchResponse.PartitionData): Seq[LogEntry] = {
-    val memoryRecords = partitionData.records
-    memoryRecords.iterator.asScala.toIndexedSeq
+    partitionData.records.deepIterator.asScala.toIndexedSeq
   }
 
   private def checkFetchResponse(expectedPartitions: Seq[TopicPartition], fetchResponse: FetchResponse,
@@ -181,25 +180,25 @@ class FetchRequestTest extends BaseRequestTest {
       assertEquals(Errors.NONE.code, partitionData.errorCode)
       assertTrue(partitionData.highWatermark > 0)
 
-      val memoryRecords = partitionData.records
-      responseBufferSize += memoryRecords.sizeInBytes
+      val records = partitionData.records
+      responseBufferSize += records.sizeInBytes
 
-      val messages = memoryRecords.iterator.asScala.toIndexedSeq
-      assertTrue(messages.size < numMessagesPerPartition)
-      val messagesSize = messages.map(_.size).sum
-      responseSize += messagesSize
-      if (messagesSize == 0 && !emptyResponseSeen) {
-        assertEquals(0, memoryRecords.sizeInBytes)
+      val entries = records.shallowIterator.asScala.toIndexedSeq
+      assertTrue(entries.size < numMessagesPerPartition)
+      val entriesSize = entries.map(_.sizeInBytes).sum
+      responseSize += entriesSize
+      if (entriesSize == 0 && !emptyResponseSeen) {
+        assertEquals(0, records.sizeInBytes)
         emptyResponseSeen = true
       }
-      else if (messagesSize != 0 && !emptyResponseSeen) {
-        assertTrue(messagesSize <= maxPartitionBytes)
-        assertEquals(maxPartitionBytes, memoryRecords.sizeInBytes)
+      else if (entriesSize != 0 && !emptyResponseSeen) {
+        assertTrue(entriesSize <= maxPartitionBytes)
+        assertEquals(maxPartitionBytes, records.sizeInBytes)
       }
-      else if (messagesSize != 0 && emptyResponseSeen)
-        fail(s"Expected partition with size 0, but found $tp with size $messagesSize")
-      else if (memoryRecords.sizeInBytes != 0 && emptyResponseSeen)
-        fail(s"Expected partition buffer with size 0, but found $tp with size ${memoryRecords.sizeInBytes}")
+      else if (entriesSize != 0 && emptyResponseSeen)
+        fail(s"Expected partition with size 0, but found $tp with size $entriesSize")
+      else if (records.sizeInBytes != 0 && emptyResponseSeen)
+        fail(s"Expected partition buffer with size 0, but found $tp with size ${records.sizeInBytes}")
 
     }
 
@@ -208,7 +207,7 @@ class FetchRequestTest extends BaseRequestTest {
   }
 
   private def createTopics(numTopics: Int, numPartitions: Int): Map[TopicPartition, Int] = {
-    val topics = (0 until numPartitions).map(t => s"topic${t}")
+    val topics = (0 until numPartitions).map(t => s"topic$t")
     val topicConfig = new Properties
     topicConfig.setProperty(LogConfig.MinInSyncReplicasProp, 2.toString)
     topics.flatMap { topic =>
@@ -223,7 +222,7 @@ class FetchRequestTest extends BaseRequestTest {
       tp <- topicPartitions.toSeq
       messageIndex <- 0 until numMessagesPerPartition
     } yield {
-      val suffix = s"${tp}-${messageIndex}"
+      val suffix = s"$tp-$messageIndex"
       new ProducerRecord(tp.topic, tp.partition, s"key $suffix", s"value $suffix")
     }
     records.map(producer.send).foreach(_.get)

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala
index 2d51be9..aad37d1 100644
--- a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala
@@ -17,21 +17,20 @@
 package kafka.server
 
 import java.util.Properties
+import java.util.concurrent.atomic.AtomicBoolean
 
-import org.apache.kafka.common.metrics.Metrics
-import org.junit.{After, Before, Test}
-
-import collection.mutable.HashMap
-import collection.mutable.Map
 import kafka.cluster.{Partition, Replica}
-import org.easymock.EasyMock
 import kafka.log.Log
-import org.junit.Assert._
 import kafka.utils._
-import java.util.concurrent.atomic.AtomicBoolean
-
-import kafka.message.MessageSet
+import org.apache.kafka.common.metrics.Metrics
+import org.apache.kafka.common.record.MemoryRecords
 import org.apache.kafka.common.utils.Time
+import org.easymock.EasyMock
+import org.junit.Assert._
+import org.junit.{After, Before, Test}
+
+import scala.collection.mutable.{HashMap, Map}
+
 
 class IsrExpirationTest {
 
@@ -76,7 +75,7 @@ class IsrExpirationTest {
 
     // let the follower catch up to the Leader logEndOffset (15)
     (partition0.assignedReplicas() - leaderReplica).foreach(
-      r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(15L), MessageSet.Empty),
+      r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(15L), MemoryRecords.EMPTY),
                                                    -1L,
                                                    -1,
                                                    true)))
@@ -127,7 +126,7 @@ class IsrExpirationTest {
 
     // Make the remote replica not read to the end of log. It should be not be out of sync for at least 100 ms
     for(replica <- partition0.assignedReplicas() - leaderReplica)
-      replica.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(10L), MessageSet.Empty), -1L, -1, false))
+      replica.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(10L), MemoryRecords.EMPTY), -1L, -1, false))
 
     // Simulate 2 fetch requests spanning more than 100 ms which do not read to the end of the log.
     // The replicas will no longer be in ISR. We do 2 fetches because we want to simulate the case where the replica is lagging but is not stuck
@@ -137,7 +136,7 @@ class IsrExpirationTest {
     time.sleep(75)
 
     (partition0.assignedReplicas() - leaderReplica).foreach(
-      r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(11L), MessageSet.Empty), -1L, -1, false)))
+      r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(11L), MemoryRecords.EMPTY), -1L, -1, false)))
     partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs)
     assertEquals("No replica should be out of sync", Set.empty[Int], partition0OSR.map(_.brokerId))
 
@@ -149,7 +148,7 @@ class IsrExpirationTest {
 
     // Now actually make a fetch to the end of the log. The replicas should be back in ISR
     (partition0.assignedReplicas() - leaderReplica).foreach(
-      r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(15L), MessageSet.Empty), -1L, -1, true)))
+      r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(15L), MemoryRecords.EMPTY), -1L, -1, true)))
     partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs)
     assertEquals("No replica should be out of sync", Set.empty[Int], partition0OSR.map(_.brokerId))
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 70445d7..b577e7d 100755
--- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
+++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
@@ -26,12 +26,12 @@ import kafka.api.{FetchRequestBuilder, OffsetRequest, PartitionOffsetRequestInfo
 import kafka.common.TopicAndPartition
 import kafka.consumer.SimpleConsumer
 import kafka.log.{Log, LogSegment}
-import kafka.message.{ByteBufferMessageSet, Message, NoCompressionCodec}
 import kafka.utils.TestUtils._
 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.utils.{Time, Utils}
 import org.easymock.{EasyMock, IAnswer}
 import org.junit.Assert._
@@ -89,9 +89,9 @@ class LogOffsetTest extends ZooKeeperTestHarness {
                   "Log for partition [topic,0] should be created")
     val log = logManager.getLog(TopicAndPartition(topic, part)).get
 
-    val message = new Message(Integer.toString(42).getBytes())
+    val record = Record.create(Integer.toString(42).getBytes())
     for (_ <- 0 until 20)
-      log.append(new ByteBufferMessageSet(NoCompressionCodec, message))
+      log.append(MemoryRecords.withRecords(record))
     log.flush()
 
     val offsets = server.apis.fetchOffsets(logManager, new TopicPartition(topic, part), OffsetRequest.LatestTime, 15)
@@ -150,9 +150,9 @@ class LogOffsetTest extends ZooKeeperTestHarness {
 
     val logManager = server.getLogManager
     val log = logManager.createLog(TopicAndPartition(topic, part), logManager.defaultConfig)
-    val message = new Message(Integer.toString(42).getBytes())
+    val record = Record.create(Integer.toString(42).getBytes())
     for (_ <- 0 until 20)
-      log.append(new ByteBufferMessageSet(NoCompressionCodec, message))
+      log.append(MemoryRecords.withRecords(record))
     log.flush()
 
     val now = time.milliseconds + 30000 // pretend it is the future to avoid race conditions with the fs
@@ -179,9 +179,9 @@ class LogOffsetTest extends ZooKeeperTestHarness {
 
     val logManager = server.getLogManager
     val log = logManager.createLog(TopicAndPartition(topic, part), logManager.defaultConfig)
-    val message = new Message(Integer.toString(42).getBytes())
+    val record = Record.create(Integer.toString(42).getBytes())
     for (_ <- 0 until 20)
-      log.append(new ByteBufferMessageSet(NoCompressionCodec, message))
+      log.append(MemoryRecords.withRecords(record))
     log.flush()
 
     val offsets = server.apis.fetchOffsets(logManager, new TopicPartition(topic, part), OffsetRequest.EarliestTime, 10)

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala
index bd74dee..51be54c 100644
--- a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala
@@ -54,11 +54,11 @@ class ProduceRequestTest extends BaseRequestTest {
     }
 
     sendAndCheck(JTestUtils.partitionRecordsBuffer(0, CompressionType.NONE,
-      new Record(System.currentTimeMillis(), "key".getBytes, "value".getBytes)), 0)
+      Record.create(System.currentTimeMillis(), "key".getBytes, "value".getBytes)), 0)
 
     sendAndCheck(JTestUtils.partitionRecordsBuffer(0, CompressionType.GZIP,
-      new Record(System.currentTimeMillis(), "key1".getBytes, "value1".getBytes),
-      new Record(System.currentTimeMillis(), "key2".getBytes, "value2".getBytes)), 1)
+      Record.create(System.currentTimeMillis(), "key1".getBytes, "value1".getBytes),
+      Record.create(System.currentTimeMillis(), "key2".getBytes, "value2".getBytes)), 1)
   }
 
   /* returns a pair of partition id and leader id */
@@ -74,7 +74,7 @@ class ProduceRequestTest extends BaseRequestTest {
     val (partition, leader) = createTopicAndFindPartitionWithLeader("topic")
     val timestamp = 1000000
     val recordBuffer = JTestUtils.partitionRecordsBuffer(0, CompressionType.LZ4,
-      new Record(timestamp, "key".getBytes, "value".getBytes))
+      Record.create(timestamp, "key".getBytes, "value".getBytes))
     // Change the lz4 checksum value so that it doesn't match the contents
     recordBuffer.array.update(40, 0)
     val topicPartition = new TopicPartition("topic", partition)

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 378d382..a643f63 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
@@ -16,28 +16,29 @@
   */
 package kafka.server
 
-
 import java.util.Properties
 import java.util.concurrent.atomic.AtomicBoolean
 
 import kafka.cluster.Replica
 import kafka.common.TopicAndPartition
 import kafka.log.Log
-import kafka.message.{ByteBufferMessageSet, Message}
 import kafka.utils._
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.metrics.Metrics
+import org.apache.kafka.common.record.{MemoryRecords, Record}
 import org.apache.kafka.common.requests.FetchRequest.PartitionData
 import org.easymock.EasyMock
 import EasyMock._
 import org.junit.Assert._
 import org.junit.{After, Test}
 
+import scala.collection.JavaConverters._
+
 class ReplicaManagerQuotasTest {
   val configs = TestUtils.createBrokerConfigs(2, TestUtils.MockZkConnect).map(KafkaConfig.fromProps(_, new Properties()))
   val time = new MockTime
   val metrics = new Metrics
-  val message = new Message("some-data-in-a-message".getBytes())
+  val record = Record.create("some-data-in-a-message".getBytes())
   val topicAndPartition1 = TopicAndPartition("test-topic", 1)
   val topicAndPartition2 = TopicAndPartition("test-topic", 2)
   val fetchInfo = Seq(new TopicPartition(topicAndPartition1.topic, topicAndPartition1.partition) -> new PartitionData(0, 100),
@@ -63,10 +64,10 @@ class ReplicaManagerQuotasTest {
       readPartitionInfo = fetchInfo,
       quota = quota)
     assertEquals("Given two partitions, with only one throttled, we should get the first", 1,
-      fetch.find(_._1 == topicAndPartition1).get._2.info.messageSet.size)
+      fetch.find(_._1 == topicAndPartition1).get._2.info.records.shallowIterator.asScala.size)
 
     assertEquals("But we shouldn't get the second", 0,
-      fetch.find(_._1 == topicAndPartition2).get._2.info.messageSet.size)
+      fetch.find(_._1 == topicAndPartition2).get._2.info.records.shallowIterator.asScala.size)
   }
 
   @Test
@@ -88,9 +89,9 @@ class ReplicaManagerQuotasTest {
       readPartitionInfo = fetchInfo,
       quota = quota)
     assertEquals("Given two partitions, with both throttled, we should get no messages", 0,
-      fetch.find(_._1 == topicAndPartition1).get._2.info.messageSet.size)
+      fetch.find(_._1 == topicAndPartition1).get._2.info.records.shallowIterator.asScala.size)
     assertEquals("Given two partitions, with both throttled, we should get no messages", 0,
-      fetch.find(_._1 == topicAndPartition2).get._2.info.messageSet.size)
+      fetch.find(_._1 == topicAndPartition2).get._2.info.records.shallowIterator.asScala.size)
   }
 
   @Test
@@ -112,9 +113,9 @@ class ReplicaManagerQuotasTest {
       readPartitionInfo = fetchInfo,
       quota = quota)
     assertEquals("Given two partitions, with both non-throttled, we should get both messages", 1,
-      fetch.find(_._1 == topicAndPartition1).get._2.info.messageSet.size)
+      fetch.find(_._1 == topicAndPartition1).get._2.info.records.shallowIterator.asScala.size)
     assertEquals("Given two partitions, with both non-throttled, we should get both messages", 1,
-      fetch.find(_._1 == topicAndPartition2).get._2.info.messageSet.size)
+      fetch.find(_._1 == topicAndPartition2).get._2.info.records.shallowIterator.asScala.size)
   }
 
   @Test
@@ -136,13 +137,13 @@ class ReplicaManagerQuotasTest {
       readPartitionInfo = fetchInfo,
       quota = quota)
     assertEquals("Given two partitions, with only one throttled, we should get the first", 1,
-      fetch.find(_._1 == topicAndPartition1).get._2.info.messageSet.size)
+      fetch.find(_._1 == topicAndPartition1).get._2.info.records.shallowIterator.asScala.size)
 
     assertEquals("But we should get the second too since it's throttled but in sync", 1,
-      fetch.find(_._1 == topicAndPartition2).get._2.info.messageSet.size)
+      fetch.find(_._1 == topicAndPartition2).get._2.info.records.shallowIterator.asScala.size)
   }
 
-  def setUpMocks(fetchInfo: Seq[(TopicPartition, PartitionData)], message: Message = this.message, bothReplicasInSync: Boolean = false) {
+  def setUpMocks(fetchInfo: Seq[(TopicPartition, PartitionData)], record: Record = this.record, bothReplicasInSync: Boolean = false) {
     val zkUtils = createNiceMock(classOf[ZkUtils])
     val scheduler = createNiceMock(classOf[KafkaScheduler])
 
@@ -153,16 +154,16 @@ class ReplicaManagerQuotasTest {
 
     //if we ask for len 1 return a message
     expect(log.read(anyObject(), geq(1), anyObject(), anyObject())).andReturn(
-      new FetchDataInfo(
+      FetchDataInfo(
         new LogOffsetMetadata(0L, 0L, 0),
-        new ByteBufferMessageSet(message)
+        MemoryRecords.withRecords(record)
       )).anyTimes()
 
     //if we ask for len = 0, return 0 messages
     expect(log.read(anyObject(), EasyMock.eq(0), anyObject(), anyObject())).andReturn(
-      new FetchDataInfo(
+      FetchDataInfo(
         new LogOffsetMetadata(0L, 0L, 0),
-        new ByteBufferMessageSet()
+        MemoryRecords.EMPTY
       )).anyTimes()
     replay(log)
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 c6d66ba..421de32 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -17,18 +17,16 @@
 
 package kafka.server
 
-
 import java.io.File
 import java.util.concurrent.atomic.AtomicBoolean
 
-import kafka.api.FetchResponsePartitionData
 import kafka.cluster.Broker
 import kafka.common.TopicAndPartition
-import kafka.message.{ByteBufferMessageSet, Message, MessageSet}
 import kafka.utils.{MockScheduler, MockTime, TestUtils, ZkUtils}
 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.{MemoryRecords, Record, Records}
 import org.apache.kafka.common.requests.{LeaderAndIsrRequest, PartitionState}
 import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
 import org.apache.kafka.common.requests.FetchRequest.PartitionData
@@ -105,11 +103,11 @@ class ReplicaManagerTest {
       def callback(responseStatus: Map[TopicPartition, PartitionResponse]) = {
         assert(responseStatus.values.head.errorCode == Errors.INVALID_REQUIRED_ACKS.code)
       }
-      rm.appendMessages(
+      rm.appendRecords(
         timeout = 0,
         requiredAcks = 3,
         internalTopicsAllowed = false,
-        messagesPerPartition = Map(new TopicPartition("test1", 0) -> new ByteBufferMessageSet(new Message("first message".getBytes))),
+        entriesPerPartition = Map(new TopicPartition("test1", 0) -> MemoryRecords.withRecords(Record.create("first message".getBytes()))),
         responseCallback = callback)
     } finally {
       rm.shutdown(checkpointHW = false)
@@ -135,7 +133,7 @@ class ReplicaManagerTest {
       }
 
       var fetchCallbackFired = false
-      def fetchCallback(responseStatus: Seq[(TopicAndPartition, FetchResponsePartitionData)]) = {
+      def fetchCallback(responseStatus: Seq[(TopicAndPartition, FetchPartitionData)]) = {
         assertEquals("Should give NotLeaderForPartitionException", Errors.NOT_LEADER_FOR_PARTITION.code, responseStatus.map(_._2).head.error)
         fetchCallbackFired = true
       }
@@ -158,11 +156,11 @@ class ReplicaManagerTest {
       rm.getLeaderReplicaIfLocal(topic, 0)
 
       // Append a message.
-      rm.appendMessages(
+      rm.appendRecords(
         timeout = 1000,
         requiredAcks = -1,
         internalTopicsAllowed = false,
-        messagesPerPartition = Map(new TopicPartition(topic, 0) -> new ByteBufferMessageSet(new Message("first message".getBytes))),
+        entriesPerPartition = Map(new TopicPartition(topic, 0) -> MemoryRecords.withRecords(Record.create("first message".getBytes()))),
         responseCallback = produceCallback)
 
       // Fetch some messages
@@ -220,19 +218,19 @@ class ReplicaManagerTest {
       
       // Append a couple of messages.
       for(i <- 1 to 2)
-        rm.appendMessages(
+        rm.appendRecords(
           timeout = 1000,
           requiredAcks = -1,
           internalTopicsAllowed = false,
-          messagesPerPartition = Map(new TopicPartition(topic, 0) -> new ByteBufferMessageSet(new Message("message %d".format(i).getBytes))),
+          entriesPerPartition = Map(new TopicPartition(topic, 0) -> MemoryRecords.withRecords(Record.create("message %d".format(i).getBytes))),
           responseCallback = produceCallback)
       
       var fetchCallbackFired = false
       var fetchError = 0
-      var fetchedMessages: MessageSet = null
-      def fetchCallback(responseStatus: Seq[(TopicAndPartition, FetchResponsePartitionData)]) = {
+      var fetchedRecords: Records = null
+      def fetchCallback(responseStatus: Seq[(TopicAndPartition, FetchPartitionData)]) = {
         fetchError = responseStatus.map(_._2).head.error
-        fetchedMessages = responseStatus.map(_._2).head.messages
+        fetchedRecords = responseStatus.map(_._2).head.records
         fetchCallbackFired = true
       }
       
@@ -249,7 +247,7 @@ class ReplicaManagerTest {
       
       assertTrue(fetchCallbackFired)
       assertEquals("Should not give an exception", Errors.NONE.code, fetchError)
-      assertTrue("Should return some data", fetchedMessages.iterator.hasNext)
+      assertTrue("Should return some data", fetchedRecords.shallowIterator.hasNext)
       fetchCallbackFired = false
       
       // Fetch a message above the high watermark as a consumer
@@ -264,7 +262,7 @@ class ReplicaManagerTest {
           
         assertTrue(fetchCallbackFired)
         assertEquals("Should not give an exception", Errors.NONE.code, fetchError)
-        assertEquals("Should return empty response", MessageSet.Empty, fetchedMessages)
+        assertEquals("Should return empty response", MemoryRecords.EMPTY, fetchedRecords)
     } finally {
       rm.shutdown(checkpointHW = false)
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 b1ebeee..2f73a94 100644
--- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
+++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
@@ -21,7 +21,6 @@ import kafka.utils._
 import kafka.cluster.Replica
 import kafka.common.TopicAndPartition
 import kafka.log.Log
-import kafka.message.{ByteBufferMessageSet, Message, MessageSet}
 import kafka.server.QuotaFactory.UnboundedQuota
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.requests.FetchRequest.PartitionData
@@ -30,8 +29,10 @@ import java.util.Properties
 import java.util.concurrent.atomic.AtomicBoolean
 
 import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.record.{MemoryRecords, Record}
 import org.easymock.EasyMock
 import org.junit.Assert._
+import scala.collection.JavaConverters._
 
 class SimpleFetchTest {
 
@@ -53,8 +54,8 @@ class SimpleFetchTest {
   val partitionHW = 5
 
   val fetchSize = 100
-  val messagesToHW = new Message("messageToHW".getBytes())
-  val messagesToLEO = new Message("messageToLEO".getBytes())
+  val messagesToHW = Record.create("messageToHW".getBytes())
+  val messagesToLEO = Record.create("messageToLEO".getBytes())
 
   val topic = "test-topic"
   val partitionId = 0
@@ -79,14 +80,14 @@ class SimpleFetchTest {
     EasyMock.expect(log.logEndOffset).andReturn(leaderLEO).anyTimes()
     EasyMock.expect(log.logEndOffsetMetadata).andReturn(new LogOffsetMetadata(leaderLEO)).anyTimes()
     EasyMock.expect(log.read(0, fetchSize, Some(partitionHW), true)).andReturn(
-      new FetchDataInfo(
+      FetchDataInfo(
         new LogOffsetMetadata(0L, 0L, 0),
-        new ByteBufferMessageSet(messagesToHW)
+        MemoryRecords.withRecords(messagesToHW)
       )).anyTimes()
     EasyMock.expect(log.read(0, fetchSize, None, true)).andReturn(
-      new FetchDataInfo(
+      FetchDataInfo(
         new LogOffsetMetadata(0L, 0L, 0),
-        new ByteBufferMessageSet(messagesToLEO)
+        MemoryRecords.withRecords(messagesToLEO)
       )).anyTimes()
     EasyMock.replay(log)
 
@@ -110,7 +111,7 @@ class SimpleFetchTest {
     // create the follower replica with defined log end offset
     val followerReplica= new Replica(configs(1).brokerId, partition, time)
     val leo = new LogOffsetMetadata(followerLEO, 0L, followerLEO.toInt)
-    followerReplica.updateLogReadResult(new LogReadResult(FetchDataInfo(leo, MessageSet.Empty), -1L, -1, true))
+    followerReplica.updateLogReadResult(new LogReadResult(FetchDataInfo(leo, MemoryRecords.EMPTY), -1L, -1, true))
 
     // add both of them to ISR
     val allReplicas = List(leaderReplica, followerReplica)
@@ -153,7 +154,7 @@ class SimpleFetchTest {
         fetchMaxBytes = Int.MaxValue,
         hardMaxBytesLimit = false,
         readPartitionInfo = fetchInfo,
-        quota = UnboundedQuota).find(_._1 == topicAndPartition).get._2.info.messageSet.head.message)
+        quota = UnboundedQuota).find(_._1 == topicAndPartition).get._2.info.records.shallowIterator.next().record)
     assertEquals("Reading any data can return messages up to the end of the log", messagesToLEO,
       replicaManager.readFromLocalLog(
         replicaId = Request.OrdinaryConsumerId,
@@ -162,7 +163,7 @@ class SimpleFetchTest {
         fetchMaxBytes = Int.MaxValue,
         hardMaxBytesLimit = false,
         readPartitionInfo = fetchInfo,
-        quota = UnboundedQuota).find(_._1 == topicAndPartition).get._2.info.messageSet.head.message)
+        quota = UnboundedQuota).find(_._1 == topicAndPartition).get._2.info.records.shallowIterator().next().record)
 
     assertEquals("Counts should increment after fetch", initialTopicCount+2, BrokerTopicStats.getBrokerTopicStats(topic).totalFetchRequestRate.count())
     assertEquals("Counts should increment after fetch", initialAllTopicsCount+2, BrokerTopicStats.getBrokerAllTopicsStats().totalFetchRequestRate.count())

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 33ab58c..ede145a 100755
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -21,7 +21,7 @@ import java.io._
 import java.nio._
 import java.nio.channels._
 import java.util.concurrent.{Callable, Executors, TimeUnit}
-import java.util.{Properties, Random}
+import java.util.Properties
 import java.security.cert.X509Certificate
 import javax.net.ssl.X509TrustManager
 import charset.Charset
@@ -48,6 +48,7 @@ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, Produce
 import org.apache.kafka.clients.consumer.{KafkaConsumer, RangeAssignor}
 import org.apache.kafka.clients.CommonClientConfigs
 import org.apache.kafka.common.network.Mode
+import org.apache.kafka.common.record._
 import org.apache.kafka.common.serialization.{ByteArraySerializer, Serializer}
 import org.apache.kafka.common.utils.Time
 import org.apache.kafka.test.{TestUtils => JTestUtils}
@@ -269,16 +270,16 @@ object TestUtils extends Logging {
   }
 
   /**
-   * Wrap the message in a message set
-   *
-   * @param payload The bytes of the message
+   * Wrap a single record log buffer.
    */
-  def singleMessageSet(payload: Array[Byte],
-                       codec: CompressionCodec = NoCompressionCodec,
+  def singletonRecords(value: Array[Byte],
                        key: Array[Byte] = null,
-                       timestamp: Long = Message.NoTimestamp,
-                       magicValue: Byte = Message.CurrentMagicValue) =
-    new ByteBufferMessageSet(compressionCodec = codec, messages = new Message(payload, key, timestamp, magicValue))
+                       codec: CompressionType = CompressionType.NONE,
+                       timestamp: Long = Record.NO_TIMESTAMP,
+                       magicValue: Byte = Record.CURRENT_MAGIC_VALUE) = {
+    val record = Record.create(magicValue, timestamp, key, value)
+    MemoryRecords.withRecords(codec, record)
+  }
 
   /**
    * Generate an array of random bytes


[4/9] kafka git commit: KAFKA-4390; Replace MessageSet usage with client-side alternatives

Posted by jg...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 af64ffe..9e4c149 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -26,7 +26,7 @@ import kafka.cluster.{Partition, Replica}
 import kafka.common._
 import kafka.controller.KafkaController
 import kafka.log.{LogAppendInfo, LogManager}
-import kafka.message.{ByteBufferMessageSet, InvalidMessageException, Message, MessageSet}
+import kafka.message.InvalidMessageException
 import kafka.metrics.KafkaMetricsGroup
 import kafka.server.QuotaFactory.UnboundedQuota
 import kafka.utils._
@@ -34,6 +34,7 @@ import org.apache.kafka.common.errors.{ControllerMovedException, CorruptRecordEx
 import org.apache.kafka.common.TopicPartition
 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, StopReplicaRequest, UpdateMetadataRequest}
 import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
 import org.apache.kafka.common.utils.Time
@@ -78,9 +79,11 @@ case class LogReadResult(info: FetchDataInfo,
   }
 }
 
+case class FetchPartitionData(error: Short = Errors.NONE.code, hw: Long = -1L, records: Records)
+
 object LogReadResult {
   val UnknownLogReadResult = LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata,
-                                                         MessageSet.Empty),
+                                                         MemoryRecords.EMPTY),
                                            -1L,
                                            -1,
                                            false)
@@ -276,11 +279,7 @@ class ReplicaManager(val config: KafkaConfig,
   }
 
   def getPartition(topic: String, partitionId: Int): Option[Partition] = {
-    val partition = allPartitions.get((topic, partitionId))
-    if (partition == null)
-      None
-    else
-      Some(partition)
+    Option(allPartitions.get((topic, partitionId)))
   }
 
   def getReplicaOrException(topic: String, partition: Int): Replica = {
@@ -318,15 +317,15 @@ class ReplicaManager(val config: KafkaConfig,
    * Append messages to leader replicas of the partition, and wait for them to be replicated to other replicas;
    * the callback function will be triggered either when timeout or the required acks are satisfied
    */
-  def appendMessages(timeout: Long,
-                     requiredAcks: Short,
-                     internalTopicsAllowed: Boolean,
-                     messagesPerPartition: Map[TopicPartition, MessageSet],
-                     responseCallback: Map[TopicPartition, PartitionResponse] => Unit) {
+  def appendRecords(timeout: Long,
+                    requiredAcks: Short,
+                    internalTopicsAllowed: Boolean,
+                    entriesPerPartition: Map[TopicPartition, MemoryRecords],
+                    responseCallback: Map[TopicPartition, PartitionResponse] => Unit) {
 
     if (isValidRequiredAcks(requiredAcks)) {
       val sTime = time.milliseconds
-      val localProduceResults = appendToLocalLog(internalTopicsAllowed, messagesPerPartition, requiredAcks)
+      val localProduceResults = appendToLocalLog(internalTopicsAllowed, entriesPerPartition, requiredAcks)
       debug("Produce to local log in %d ms".format(time.milliseconds - sTime))
 
       val produceStatus = localProduceResults.map { case (topicPartition, result) =>
@@ -336,13 +335,13 @@ class ReplicaManager(val config: KafkaConfig,
                   new PartitionResponse(result.errorCode, result.info.firstOffset, result.info.logAppendTime)) // response status
       }
 
-      if (delayedRequestRequired(requiredAcks, messagesPerPartition, localProduceResults)) {
+      if (delayedRequestRequired(requiredAcks, entriesPerPartition, localProduceResults)) {
         // create delayed produce operation
         val produceMetadata = ProduceMetadata(requiredAcks, produceStatus)
         val delayedProduce = new DelayedProduce(timeout, produceMetadata, this, responseCallback)
 
         // create a list of (topic, partition) pairs to use as keys for this delayed produce operation
-        val producerRequestKeys = messagesPerPartition.keys.map(new TopicPartitionOperationKey(_)).toSeq
+        val producerRequestKeys = entriesPerPartition.keys.map(new TopicPartitionOperationKey(_)).toSeq
 
         // try to complete the request immediately, otherwise put it into the purgatory
         // this is because while the delayed produce operation is being created, new
@@ -357,9 +356,9 @@ class ReplicaManager(val config: KafkaConfig,
     } else {
       // If required.acks is outside accepted range, something is wrong with the client
       // Just return an error and don't handle the request at all
-      val responseStatus = messagesPerPartition.map { case (topicAndPartition, _) =>
+      val responseStatus = entriesPerPartition.map { case (topicAndPartition, _) =>
         topicAndPartition -> new PartitionResponse(Errors.INVALID_REQUIRED_ACKS.code,
-          LogAppendInfo.UnknownLogAppendInfo.firstOffset, Message.NoTimestamp)
+          LogAppendInfo.UnknownLogAppendInfo.firstOffset, Record.NO_TIMESTAMP)
       }
       responseCallback(responseStatus)
     }
@@ -370,11 +369,12 @@ class ReplicaManager(val config: KafkaConfig,
   // 1. required acks = -1
   // 2. there is data to append
   // 3. at least one partition append was successful (fewer errors than partitions)
-  private def delayedRequestRequired(requiredAcks: Short, messagesPerPartition: Map[TopicPartition, MessageSet],
-                                       localProduceResults: Map[TopicPartition, LogAppendResult]): Boolean = {
+  private def delayedRequestRequired(requiredAcks: Short,
+                                     entriesPerPartition: Map[TopicPartition, MemoryRecords],
+                                     localProduceResults: Map[TopicPartition, LogAppendResult]): Boolean = {
     requiredAcks == -1 &&
-    messagesPerPartition.nonEmpty &&
-    localProduceResults.values.count(_.error.isDefined) < messagesPerPartition.size
+    entriesPerPartition.nonEmpty &&
+    localProduceResults.values.count(_.error.isDefined) < entriesPerPartition.size
   }
 
   private def isValidRequiredAcks(requiredAcks: Short): Boolean = {
@@ -385,10 +385,10 @@ class ReplicaManager(val config: KafkaConfig,
    * Append the messages to the local replica logs
    */
   private def appendToLocalLog(internalTopicsAllowed: Boolean,
-                               messagesPerPartition: Map[TopicPartition, MessageSet],
+                               entriesPerPartition: Map[TopicPartition, MemoryRecords],
                                requiredAcks: Short): Map[TopicPartition, LogAppendResult] = {
-    trace("Append [%s] to local log ".format(messagesPerPartition))
-    messagesPerPartition.map { case (topicPartition, messages) =>
+    trace("Append [%s] to local log ".format(entriesPerPartition))
+    entriesPerPartition.map { case (topicPartition, records) =>
       BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).totalProduceRequestRate.mark()
       BrokerTopicStats.getBrokerAllTopicsStats().totalProduceRequestRate.mark()
 
@@ -402,7 +402,7 @@ class ReplicaManager(val config: KafkaConfig,
           val partitionOpt = getPartition(topicPartition.topic, topicPartition.partition)
           val info = partitionOpt match {
             case Some(partition) =>
-              partition.appendMessagesToLeader(messages.asInstanceOf[ByteBufferMessageSet], requiredAcks)
+              partition.appendRecordsToLeader(records, requiredAcks)
             case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d"
               .format(topicPartition, localBrokerId))
           }
@@ -414,13 +414,13 @@ class ReplicaManager(val config: KafkaConfig,
               info.lastOffset - info.firstOffset + 1
 
           // update stats for successfully appended bytes and messages as bytesInRate and messageInRate
-          BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).bytesInRate.mark(messages.sizeInBytes)
-          BrokerTopicStats.getBrokerAllTopicsStats.bytesInRate.mark(messages.sizeInBytes)
+          BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).bytesInRate.mark(records.sizeInBytes)
+          BrokerTopicStats.getBrokerAllTopicsStats.bytesInRate.mark(records.sizeInBytes)
           BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).messagesInRate.mark(numAppendedMessages)
           BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(numAppendedMessages)
 
           trace("%d bytes written to log %s-%d beginning at offset %d and ending at offset %d"
-            .format(messages.sizeInBytes, topicPartition.topic, topicPartition.partition, info.firstOffset, info.lastOffset))
+            .format(records.sizeInBytes, topicPartition.topic, topicPartition.partition, info.firstOffset, info.lastOffset))
           (topicPartition, LogAppendResult(info))
         } catch {
           // NOTE: Failed produce requests metric is not incremented for known exceptions
@@ -434,6 +434,7 @@ class ReplicaManager(val config: KafkaConfig,
                    _: RecordTooLargeException |
                    _: RecordBatchTooLargeException |
                    _: CorruptRecordException |
+                   _: InvalidRecordException |
                    _: InvalidMessageException |
                    _: InvalidTimestampException) =>
             (topicPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(e)))
@@ -458,7 +459,7 @@ class ReplicaManager(val config: KafkaConfig,
                     hardMaxBytesLimit: Boolean,
                     fetchInfos: Seq[(TopicPartition, PartitionData)],
                     quota: ReplicaQuota = UnboundedQuota,
-                    responseCallback: Seq[(TopicAndPartition, FetchResponsePartitionData)] => Unit) {
+                    responseCallback: Seq[(TopicAndPartition, FetchPartitionData)] => Unit) {
     val isFromFollower = replicaId >= 0
     val fetchOnlyFromLeader: Boolean = replicaId != Request.DebuggingConsumerId
     val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(replicaId)
@@ -480,7 +481,7 @@ class ReplicaManager(val config: KafkaConfig,
 
     // check if this fetch request can be satisfied right away
     val logReadResultValues = logReadResults.map { case (_, v) => v }
-    val bytesReadable = logReadResultValues.map(_.info.messageSet.sizeInBytes).sum
+    val bytesReadable = logReadResultValues.map(_.info.records.sizeInBytes).sum
     val errorReadingData = logReadResultValues.foldLeft(false) ((errorIncurred, readResult) =>
       errorIncurred || (readResult.errorCode != Errors.NONE.code))
 
@@ -490,7 +491,7 @@ 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 -> FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)
+        tp -> FetchPartitionData(result.errorCode, result.hw, result.info.records)
       }
       responseCallback(fetchPartitionData)
     } else {
@@ -568,16 +569,16 @@ class ReplicaManager(val config: KafkaConfig,
 
             // If the partition is being throttled, simply return an empty set.
             if (shouldLeaderThrottle(quota, TopicAndPartition(tp.topic, tp.partition), replicaId))
-              FetchDataInfo(fetch.fetchOffsetMetadata, MessageSet.Empty)
+              FetchDataInfo(fetch.fetchOffsetMetadata, MemoryRecords.EMPTY)
             // For FetchRequest version 3, we replace incomplete message sets with an empty one as consumers can make
             // progress in such cases and don't need to report a `RecordTooLargeException`
-            else if (!hardMaxBytesLimit && fetch.firstMessageSetIncomplete)
-              FetchDataInfo(fetch.fetchOffsetMetadata, MessageSet.Empty)
+            else if (!hardMaxBytesLimit && fetch.firstEntryIncomplete)
+              FetchDataInfo(fetch.fetchOffsetMetadata, MemoryRecords.EMPTY)
             else fetch
 
           case None =>
             error(s"Leader for partition $tp does not have a local log")
-            FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty)
+            FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY)
         }
 
         val readToEndOfLog = initialLogEndOffset.messageOffset - logReadInfo.fetchOffsetMetadata.messageOffset <= 0
@@ -590,12 +591,14 @@ class ReplicaManager(val config: KafkaConfig,
                  _: NotLeaderForPartitionException |
                  _: ReplicaNotAvailableException |
                  _: OffsetOutOfRangeException) =>
-          LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, partitionFetchSize, false, Some(e))
+          LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY), -1L,
+            partitionFetchSize, false, Some(e))
         case e: Throwable =>
           BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark()
           BrokerTopicStats.getBrokerAllTopicsStats().failedFetchRequestRate.mark()
-          error(s"Error processing fetch operation on partition ${tp}, offset $offset", e)
-          LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, partitionFetchSize, false, Some(e))
+          error(s"Error processing fetch operation on partition $tp, offset $offset", e)
+          LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY), -1L,
+            partitionFetchSize, false, Some(e))
       }
     }
 
@@ -604,7 +607,7 @@ class ReplicaManager(val config: KafkaConfig,
     var minOneMessage = !hardMaxBytesLimit
     readPartitionInfo.foreach { case (tp, fetchInfo) =>
       val readResult = read(tp, fetchInfo, limitBytes, minOneMessage)
-      val messageSetSize = readResult.info.messageSet.sizeInBytes
+      val messageSetSize = readResult.info.records.sizeInBytes
       // Once we read from a non-empty partition, we stop ignoring request and partition level size limits
       if (messageSetSize > 0)
         minOneMessage = false
@@ -625,9 +628,9 @@ class ReplicaManager(val config: KafkaConfig,
     quota.isThrottled(topicPartition) && quota.isQuotaExceeded && !isReplicaInSync
   }
 
-  def getMessageFormatVersion(topicAndPartition: TopicAndPartition): Option[Byte] =
+  def getMagicAndTimestampType(topicAndPartition: TopicAndPartition): Option[(Byte, TimestampType)] =
     getReplica(topicAndPartition.topic, topicAndPartition.partition).flatMap { replica =>
-      replica.log.map(_.config.messageFormatVersion.messageFormatVersion)
+      replica.log.map(log => (log.config.messageFormatVersion.messageFormatVersion, log.config.messageTimestampType))
     }
 
   def maybeUpdateMetadataCache(correlationId: Int, updateMetadataRequest: UpdateMetadataRequest, metadataCache: MetadataCache) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 221ef6c..ceff78c 100755
--- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala
+++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala
@@ -23,15 +23,16 @@ import java.nio.ByteBuffer
 import joptsimple.OptionParser
 import kafka.coordinator.{GroupMetadataKey, GroupMetadataManager, OffsetKey}
 import kafka.log._
-import kafka.message._
 import kafka.serializer.Decoder
 import kafka.utils._
 import org.apache.kafka.clients.consumer.internals.ConsumerProtocol
 import org.apache.kafka.common.KafkaException
+import org.apache.kafka.common.record.{CompressionType, FileRecords, LogEntry, Record}
 import org.apache.kafka.common.utils.Utils
 
 import scala.collection.mutable
 import scala.collection.mutable.ArrayBuffer
+import scala.collection.JavaConverters._
 
 object DumpLogSegments {
 
@@ -69,7 +70,7 @@ object DumpLogSegments {
 
     CommandLineUtils.checkRequiredArgs(parser, options, filesOpt)
 
-    val printDataLog = (options.has(printOpt) || options.has(offsetsOpt) || options.has(valueDecoderOpt) || options.has(keyDecoderOpt))
+    val printDataLog = options.has(printOpt) || options.has(offsetsOpt) || options.has(valueDecoderOpt) || options.has(keyDecoderOpt)
     val verifyOnly = options.has(verifyOpt)
     val indexSanityOnly = options.has(indexSanityOpt)
 
@@ -132,7 +133,7 @@ object DumpLogSegments {
                         maxMessageSize: Int) {
     val startOffset = file.getName().split("\\.")(0).toLong
     val logFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.LogFileSuffix)
-    val messageSet = new FileMessageSet(logFile, false)
+    val fileRecords = FileRecords.open(logFile, false)
     val index = new OffsetIndex(file, baseOffset = startOffset)
 
     //Check that index passes sanityCheck, this is the check that determines if indexes will be rebuilt on startup or not.
@@ -144,11 +145,11 @@ object DumpLogSegments {
 
     for(i <- 0 until index.entries) {
       val entry = index.entry(i)
-      val partialFileMessageSet: FileMessageSet = messageSet.read(entry.position, maxMessageSize)
-      val messageAndOffset = getIterator(partialFileMessageSet.head, isDeepIteration = true).next()
-      if(messageAndOffset.offset != entry.offset + index.baseOffset) {
+      val slice = fileRecords.read(entry.position, maxMessageSize)
+      val logEntry = getIterator(slice.shallowIterator.next, isDeepIteration = true).next()
+      if (logEntry.offset != entry.offset + index.baseOffset) {
         var misMatchesSeq = misMatchesForIndexFilesMap.getOrElse(file.getAbsolutePath, List[(Long, Long)]())
-        misMatchesSeq ::=(entry.offset + index.baseOffset, messageAndOffset.offset)
+        misMatchesSeq ::=(entry.offset + index.baseOffset, logEntry.offset)
         misMatchesForIndexFilesMap.put(file.getAbsolutePath, misMatchesSeq)
       }
       // since it is a sparse file, in the event of a crash there may be many zero entries, stop if we see one
@@ -164,9 +165,9 @@ object DumpLogSegments {
                             verifyOnly: Boolean,
                             timeIndexDumpErrors: TimeIndexDumpErrors,
                             maxMessageSize: Int) {
-    val startOffset = file.getName().split("\\.")(0).toLong
+    val startOffset = file.getName.split("\\.")(0).toLong
     val logFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.LogFileSuffix)
-    val messageSet = new FileMessageSet(logFile, false)
+    val fileRecords = FileRecords.open(logFile, false)
     val indexFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.IndexFileSuffix)
     val index = new OffsetIndex(indexFile, baseOffset = startOffset)
     val timeIndex = new TimeIndex(file, baseOffset = startOffset)
@@ -178,26 +179,26 @@ object DumpLogSegments {
       return
     }
 
-    var prevTimestamp = Message.NoTimestamp
+    var prevTimestamp = Record.NO_TIMESTAMP
     for(i <- 0 until timeIndex.entries) {
       val entry = timeIndex.entry(i)
       val position = index.lookup(entry.offset + timeIndex.baseOffset).position
-      val partialFileMessageSet: FileMessageSet = messageSet.read(position, Int.MaxValue)
-      val shallowIter = partialFileMessageSet.iterator
-      var maxTimestamp = Message.NoTimestamp
+      val partialFileRecords = fileRecords.read(position, Int.MaxValue)
+      val shallowEntries = partialFileRecords.shallowIterator.asScala
+      var maxTimestamp = Record.NO_TIMESTAMP
       // We first find the message by offset then check if the timestamp is correct.
-      val wrapperMessageOpt = shallowIter.find(_.offset >= entry.offset + timeIndex.baseOffset)
-      wrapperMessageOpt match {
+      val maybeLogEntry = shallowEntries.find(_.offset >= entry.offset + timeIndex.baseOffset)
+      maybeLogEntry match {
         case None =>
           timeIndexDumpErrors.recordShallowOffsetNotFound(file, entry.offset + timeIndex.baseOffset,
             -1.toLong)
-        case Some(wrapperMessage) if wrapperMessage.offset != entry.offset + timeIndex.baseOffset =>
+        case Some(logEntry) if logEntry.offset != entry.offset + timeIndex.baseOffset =>
           timeIndexDumpErrors.recordShallowOffsetNotFound(file, entry.offset + timeIndex.baseOffset,
-            wrapperMessage.offset)
-        case Some(wrapperMessage) =>
-          val deepIter = getIterator(wrapperMessage, isDeepIteration = true)
-          for (messageAndOffset <- deepIter)
-            maxTimestamp = math.max(maxTimestamp, messageAndOffset.message.timestamp)
+            logEntry.offset)
+        case Some(shallowLogEntry) =>
+          val deepIter = getIterator(shallowLogEntry, isDeepIteration = true)
+          for (deepLogEntry <- deepIter)
+            maxTimestamp = math.max(maxTimestamp, deepLogEntry.record.timestamp)
 
           if (maxTimestamp != entry.timestamp)
             timeIndexDumpErrors.recordMismatchTimeIndex(file, entry.timestamp, maxTimestamp)
@@ -216,20 +217,20 @@ object DumpLogSegments {
   }
 
   private trait MessageParser[K, V] {
-    def parse(message: Message): (Option[K], Option[V])
+    def parse(record: Record): (Option[K], Option[V])
   }
 
   private class DecoderMessageParser[K, V](keyDecoder: Decoder[K], valueDecoder: Decoder[V]) extends MessageParser[K, V] {
-    override def parse(message: Message): (Option[K], Option[V]) = {
-      if (message.isNull) {
+    override def parse(record: Record): (Option[K], Option[V]) = {
+      if (record.hasNullValue) {
         (None, None)
       } else {
-        val key = if (message.hasKey)
-          Some(keyDecoder.fromBytes(Utils.readBytes(message.key)))
+        val key = if (record.hasKey)
+          Some(keyDecoder.fromBytes(Utils.readBytes(record.key)))
         else
           None
 
-        val payload = Some(valueDecoder.fromBytes(Utils.readBytes(message.payload)))
+        val payload = Some(valueDecoder.fromBytes(Utils.readBytes(record.value)))
 
         (key, payload)
       }
@@ -249,7 +250,7 @@ object DumpLogSegments {
       val topicPartition = offsetKey.key.topicPartition
       val offset = GroupMetadataManager.readOffsetMessageValue(payload)
 
-      val keyString = s"offset::${group}:${topicPartition.topic}:${topicPartition.partition}"
+      val keyString = s"offset::$group:${topicPartition.topic}:${topicPartition.partition}"
       val valueString = if (offset.metadata.isEmpty)
         String.valueOf(offset.offset)
       else
@@ -271,27 +272,27 @@ object DumpLogSegments {
           if (userData.isEmpty)
             s"${member.memberId}=${partitionAssignment.partitions()}"
           else
-            s"${member.memberId}=${partitionAssignment.partitions()}:${userData}"
+            s"${member.memberId}=${partitionAssignment.partitions()}:$userData"
         } else {
           s"${member.memberId}=${hex(member.assignment)}"
         }
       }.mkString("{", ",", "}")
 
-      val keyString = s"metadata::${groupId}"
-      val valueString = s"${protocolType}:${group.protocol}:${group.generationId}:${assignment}"
+      val keyString = s"metadata::$groupId"
+      val valueString = s"$protocolType:${group.protocol}:${group.generationId}:$assignment"
 
       (Some(keyString), Some(valueString))
     }
 
-    override def parse(message: Message): (Option[String], Option[String]) = {
-      if (message.isNull)
+    override def parse(record: Record): (Option[String], Option[String]) = {
+      if (record.hasNullValue)
         (None, None)
-      else if (!message.hasKey) {
+      else if (!record.hasKey) {
         throw new KafkaException("Failed to decode message using offset topic decoder (message had a missing key)")
       } else {
-        GroupMetadataManager.readMessageKey(message.key) match {
-          case offsetKey: OffsetKey => parseOffsets(offsetKey, message.payload)
-          case groupMetadataKey: GroupMetadataKey => parseGroupMetadata(groupMetadataKey, message.payload)
+        GroupMetadataManager.readMessageKey(record.key) match {
+          case offsetKey: OffsetKey => parseOffsets(offsetKey, record.value)
+          case groupMetadataKey: GroupMetadataKey => parseGroupMetadata(groupMetadataKey, record.value)
           case _ => throw new KafkaException("Failed to decode message using offset topic decoder (message had an invalid key)")
         }
       }
@@ -307,70 +308,51 @@ object DumpLogSegments {
                       parser: MessageParser[_, _]) {
     val startOffset = file.getName().split("\\.")(0).toLong
     println("Starting offset: " + startOffset)
-    val messageSet = new FileMessageSet(file, false)
+    val messageSet = FileRecords.open(file, false)
     var validBytes = 0L
     var lastOffset = -1l
-    val shallowIterator = messageSet.iterator(maxMessageSize)
-    for(shallowMessageAndOffset <- shallowIterator) { // this only does shallow iteration
-      val itr = getIterator(shallowMessageAndOffset, isDeepIteration)
-      for (messageAndOffset <- itr) {
-        val msg = messageAndOffset.message
+    val shallowIterator = messageSet.shallowIterator(maxMessageSize).asScala
+    for (shallowLogEntry <- shallowIterator) { // this only does shallow iteration
+      val itr = getIterator(shallowLogEntry, isDeepIteration)
+      for (deepLogEntry <- itr) {
+        val record = deepLogEntry.record()
 
         if(lastOffset == -1)
-          lastOffset = messageAndOffset.offset
+          lastOffset = deepLogEntry.offset
         // If we are iterating uncompressed messages, offsets must be consecutive
-        else if (msg.compressionCodec == NoCompressionCodec && messageAndOffset.offset != lastOffset +1) {
+        else if (record.compressionType == CompressionType.NONE && deepLogEntry.offset != lastOffset +1) {
           var nonConsecutivePairsSeq = nonConsecutivePairsForLogFilesMap.getOrElse(file.getAbsolutePath, List[(Long, Long)]())
-          nonConsecutivePairsSeq ::=(lastOffset, messageAndOffset.offset)
+          nonConsecutivePairsSeq ::=(lastOffset, deepLogEntry.offset)
           nonConsecutivePairsForLogFilesMap.put(file.getAbsolutePath, nonConsecutivePairsSeq)
         }
-        lastOffset = messageAndOffset.offset
-
-        print("offset: " + messageAndOffset.offset + " position: " + validBytes +
-              " " + msg.timestampType + ": " + msg.timestamp + " isvalid: " + msg.isValid +
-              " payloadsize: " + msg.payloadSize + " magic: " + msg.magic +
-              " compresscodec: " + msg.compressionCodec + " crc: " + msg.checksum)
-        if(msg.hasKey)
-          print(" keysize: " + msg.keySize)
-        if(printContents) {
-          val (key, payload) = parser.parse(msg)
-          key.map(key => print(s" key: ${key}"))
-          payload.map(payload => print(s" payload: ${payload}"))
+        lastOffset = deepLogEntry.offset
+
+        print("offset: " + deepLogEntry.offset + " position: " + validBytes +
+              " " + record.timestampType + ": " + record.timestamp + " isvalid: " + record.isValid +
+              " payloadsize: " + record.valueSize + " magic: " + record.magic +
+              " compresscodec: " + record.compressionType + " crc: " + record.checksum)
+        if (record.hasKey)
+          print(" keysize: " + record.keySize)
+        if (printContents) {
+          val (key, payload) = parser.parse(record)
+          key.foreach(key => print(s" key: $key"))
+          payload.foreach(payload => print(s" payload: $payload"))
         }
         println()
       }
-      validBytes += MessageSet.entrySize(shallowMessageAndOffset.message)
+
+      validBytes += shallowLogEntry.sizeInBytes
     }
     val trailingBytes = messageSet.sizeInBytes - validBytes
     if(trailingBytes > 0)
       println("Found %d invalid bytes at the end of %s".format(trailingBytes, file.getName))
   }
 
-  private def getIterator(messageAndOffset: MessageAndOffset, isDeepIteration: Boolean) = {
-    if (isDeepIteration) {
-      val message = messageAndOffset.message
-      message.compressionCodec match {
-        case NoCompressionCodec =>
-          getSingleMessageIterator(messageAndOffset)
-        case _ =>
-          ByteBufferMessageSet.deepIterator(messageAndOffset)
-      }
-    } else
-      getSingleMessageIterator(messageAndOffset)
-  }
-
-  private def getSingleMessageIterator(messageAndOffset: MessageAndOffset) = {
-    new IteratorTemplate[MessageAndOffset] {
-      var messageIterated = false
-
-      override def makeNext(): MessageAndOffset = {
-        if (!messageIterated) {
-          messageIterated = true
-          messageAndOffset
-        } else
-          allDone()
-      }
-    }
+  private def getIterator(logEntry: LogEntry, isDeepIteration: Boolean): Iterator[LogEntry] = {
+    if (isDeepIteration)
+      logEntry.iterator.asScala
+    else
+      Iterator(logEntry)
   }
 
   class TimeIndexDumpErrors {

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
index 479b43c..c483021 100644
--- a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
+++ b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
@@ -17,25 +17,26 @@
 
 package kafka.tools
 
-import joptsimple.OptionParser
-import kafka.cluster.BrokerEndPoint
-import kafka.message.{ByteBufferMessageSet, MessageAndOffset, MessageSet}
+import java.text.SimpleDateFormat
+import java.util.Date
 import java.util.concurrent.CountDownLatch
 import java.util.concurrent.atomic.AtomicReference
-
-import kafka.client.ClientUtils
 import java.util.regex.{Pattern, PatternSyntaxException}
 
+import joptsimple.OptionParser
 import kafka.api._
-import java.text.SimpleDateFormat
-import java.util.Date
-
+import kafka.client.ClientUtils
+import kafka.cluster.BrokerEndPoint
 import kafka.common.TopicAndPartition
-import kafka.utils._
 import kafka.consumer.{ConsumerConfig, SimpleConsumer, Whitelist}
+import kafka.message.{ByteBufferMessageSet, MessageSet}
+import kafka.utils._
 import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.utils.Time
 
+import scala.collection.JavaConverters._
+
+
 /**
  *  For verifying the consistency among replicas.
  *
@@ -149,15 +150,15 @@ object ReplicaVerificationTool extends Logging {
     debug("Selected topic partitions: " + topicPartitionReplicaList)
     val topicAndPartitionsPerBroker: Map[Int, Seq[TopicAndPartition]] = topicPartitionReplicaList.groupBy(_.replicaId)
       .map { case (brokerId, partitions) =>
-               brokerId -> partitions.map { case partition => new TopicAndPartition(partition.topic, partition.partitionId) } }
+               brokerId -> partitions.map { partition => TopicAndPartition(partition.topic, partition.partitionId) } }
     debug("Topic partitions per broker: " + topicAndPartitionsPerBroker)
     val expectedReplicasPerTopicAndPartition: Map[TopicAndPartition, Int] =
-          topicPartitionReplicaList.groupBy(replica => new TopicAndPartition(replica.topic, replica.partitionId))
+          topicPartitionReplicaList.groupBy(replica => TopicAndPartition(replica.topic, replica.partitionId))
           .map { case (topicAndPartition, replicaSet) => topicAndPartition -> replicaSet.size }
     debug("Expected replicas per topic partition: " + expectedReplicasPerTopicAndPartition)
     val leadersPerBroker: Map[Int, Seq[TopicAndPartition]] = filteredTopicMetadata.flatMap { topicMetadataResponse =>
       topicMetadataResponse.partitionsMetadata.map { partitionMetadata =>
-        (new TopicAndPartition(topicMetadataResponse.topic, partitionMetadata.partitionId), partitionMetadata.leader.get.id)
+        (TopicAndPartition(topicMetadataResponse.topic, partitionMetadata.partitionId), partitionMetadata.leader.get.id)
       }
     }.groupBy(_._2).mapValues(topicAndPartitionAndLeaderIds => topicAndPartitionAndLeaderIds.map { case (topicAndPartition, _) =>
        topicAndPartition
@@ -200,8 +201,6 @@ object ReplicaVerificationTool extends Logging {
 
 private case class TopicPartitionReplica(topic: String,  partitionId: Int,  replicaId: Int)
 
-private case class ReplicaAndMessageIterator(replicaId: Int, iterator: Iterator[MessageAndOffset])
-
 private case class MessageInfo(replicaId: Int, offset: Long, nextOffset: Long, checksum: Long)
 
 private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPartition, Int],
@@ -276,41 +275,42 @@ private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPa
       assert(fetchResponsePerReplica.size == expectedReplicasPerTopicAndPartition(topicAndPartition),
             "fetched " + fetchResponsePerReplica.size + " replicas for " + topicAndPartition + ", but expected "
             + expectedReplicasPerTopicAndPartition(topicAndPartition) + " replicas")
-      val messageIteratorMap = fetchResponsePerReplica.map {
+      val logEntryIteratorMap = fetchResponsePerReplica.map {
         case(replicaId, fetchResponse) =>
-          replicaId -> fetchResponse.messages.asInstanceOf[ByteBufferMessageSet].shallowIterator}
+          replicaId -> fetchResponse.messages.asInstanceOf[ByteBufferMessageSet].asRecords.shallowIterator.asScala
+      }
       val maxHw = fetchResponsePerReplica.values.map(_.hw).max
 
       // Iterate one message at a time from every replica, until high watermark is reached.
       var isMessageInAllReplicas = true
       while (isMessageInAllReplicas) {
         var messageInfoFromFirstReplicaOpt: Option[MessageInfo] = None
-        for ( (replicaId, messageIterator) <- messageIteratorMap) {
+        for ( (replicaId, logEntries) <- logEntryIteratorMap) {
           try {
-            if (messageIterator.hasNext) {
-              val messageAndOffset = messageIterator.next()
+            if (logEntries.hasNext) {
+              val logEntry = logEntries.next()
 
               // only verify up to the high watermark
-              if (messageAndOffset.offset >= fetchResponsePerReplica.get(replicaId).hw)
+              if (logEntry.offset >= fetchResponsePerReplica.get(replicaId).hw)
                 isMessageInAllReplicas = false
               else {
                 messageInfoFromFirstReplicaOpt match {
                   case None =>
                     messageInfoFromFirstReplicaOpt = Some(
-                      MessageInfo(replicaId, messageAndOffset.offset,messageAndOffset.nextOffset, messageAndOffset.message.checksum))
+                      MessageInfo(replicaId, logEntry.offset,logEntry.nextOffset, logEntry.record.checksum))
                   case Some(messageInfoFromFirstReplica) =>
-                    if (messageInfoFromFirstReplica.offset != messageAndOffset.offset) {
+                    if (messageInfoFromFirstReplica.offset != logEntry.offset) {
                       println(ReplicaVerificationTool.getCurrentTimeString + ": partition " + topicAndPartition
                         + ": replica " + messageInfoFromFirstReplica.replicaId + "'s offset "
                         + messageInfoFromFirstReplica.offset + " doesn't match replica "
-                        + replicaId + "'s offset " + messageAndOffset.offset)
+                        + replicaId + "'s offset " + logEntry.offset)
                       System.exit(1)
                     }
-                    if (messageInfoFromFirstReplica.checksum != messageAndOffset.message.checksum)
+                    if (messageInfoFromFirstReplica.checksum != logEntry.record.checksum)
                       println(ReplicaVerificationTool.getCurrentTimeString + ": partition "
-                        + topicAndPartition + " has unmatched checksum at offset " + messageAndOffset.offset + "; replica "
+                        + topicAndPartition + " has unmatched checksum at offset " + logEntry.offset + "; replica "
                         + messageInfoFromFirstReplica.replicaId + "'s checksum " + messageInfoFromFirstReplica.checksum
-                        + "; replica " + replicaId + "'s checksum " + messageAndOffset.message.checksum)
+                        + "; replica " + replicaId + "'s checksum " + logEntry.record.checksum)
                 }
               }
             } else

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala
index f36e146..51e987a 100644
--- a/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala
@@ -50,11 +50,13 @@ class GroupCoordinatorIntegrationTest extends KafkaServerTestHarness {
     def getGroupMetadataLogOpt: Option[Log] =
       logManager.getLog(TopicAndPartition(Topic.GroupMetadataTopicName, 0))
 
-    TestUtils.waitUntilTrue(() => getGroupMetadataLogOpt.exists(_.logSegments.exists(_.log.nonEmpty)),
+    TestUtils.waitUntilTrue(() => getGroupMetadataLogOpt.exists(_.logSegments.exists(_.log.shallowIterator.asScala.nonEmpty)),
                             "Commit message not appended in time")
 
     val logSegments = getGroupMetadataLogOpt.get.logSegments
-    val incorrectCompressionCodecs = logSegments.flatMap(_.log.map(_.message.compressionCodec)).filter(_ != offsetsTopicCompressionCodec)
+    val incorrectCompressionCodecs = logSegments
+      .flatMap(_.log.shallowIterator.asScala.map(_.record.compressionType.id))
+      .filter(_ != offsetsTopicCompressionCodec.codec)
     assertEquals("Incorrect compression codecs should be empty", Seq.empty, incorrectCompressionCodecs)
 
     consumer.close()

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/kafka/tools/TestLogCleaning.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/kafka/tools/TestLogCleaning.scala b/core/src/test/scala/kafka/tools/TestLogCleaning.scala
index 51f02d1..ecf7408 100755
--- a/core/src/test/scala/kafka/tools/TestLogCleaning.scala
+++ b/core/src/test/scala/kafka/tools/TestLogCleaning.scala
@@ -21,12 +21,15 @@ import joptsimple.OptionParser
 import java.util.Properties
 import java.util.Random
 import java.io._
+
 import kafka.consumer._
 import kafka.serializer._
 import kafka.utils._
-import kafka.log.FileMessageSet
 import kafka.log.Log
-import org.apache.kafka.clients.producer.{ProducerRecord, KafkaProducer, ProducerConfig}
+import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
+import org.apache.kafka.common.record.FileRecords
+
+import scala.collection.JavaConverters._
 
 /**
  * This is a torture test that runs against an existing broker. Here is how it works:
@@ -135,15 +138,15 @@ object TestLogCleaning {
   
   def dumpLog(dir: File) {
     require(dir.exists, "Non-existent directory: " + dir.getAbsolutePath)
-    for(file <- dir.list.sorted; if file.endsWith(Log.LogFileSuffix)) {
-      val ms = new FileMessageSet(new File(dir, file))
-      for(entry <- ms) {
-        val key = TestUtils.readString(entry.message.key)
+    for (file <- dir.list.sorted; if file.endsWith(Log.LogFileSuffix)) {
+      val fileRecords = FileRecords.open(new File(dir, file))
+      for (entry <- fileRecords.shallowIterator.asScala) {
+        val key = TestUtils.readString(entry.record.key)
         val content = 
-          if(entry.message.isNull)
+          if(entry.record.hasNullValue)
             null
           else
-            TestUtils.readString(entry.message.payload)
+            TestUtils.readString(entry.record.value)
         println("offset = %s, key = %s, content = %s".format(entry.offset, key, content))
       }
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/other/kafka/StressTestLog.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/other/kafka/StressTestLog.scala b/core/src/test/scala/other/kafka/StressTestLog.scala
index f5cee0c..3381fb7 100755
--- a/core/src/test/scala/other/kafka/StressTestLog.scala
+++ b/core/src/test/scala/other/kafka/StressTestLog.scala
@@ -20,10 +20,10 @@ package kafka
 import java.util.Properties
 import java.util.concurrent.atomic._
 
-import kafka.message._
 import kafka.log._
 import kafka.utils._
 import org.apache.kafka.clients.consumer.OffsetOutOfRangeException
+import org.apache.kafka.common.record.FileRecords
 import org.apache.kafka.common.utils.Utils
 
 /**
@@ -36,13 +36,13 @@ object StressTestLog {
   def main(args: Array[String]) {
     val dir = TestUtils.randomPartitionLogDir(TestUtils.tempDir())
     val time = new MockTime
-    val logProprties = new Properties()
-    logProprties.put(LogConfig.SegmentBytesProp, 64*1024*1024: java.lang.Integer)
-    logProprties.put(LogConfig.MaxMessageBytesProp, Int.MaxValue: java.lang.Integer)
-    logProprties.put(LogConfig.SegmentIndexBytesProp, 1024*1024: java.lang.Integer)
+    val logProperties = new Properties()
+    logProperties.put(LogConfig.SegmentBytesProp, 64*1024*1024: java.lang.Integer)
+    logProperties.put(LogConfig.MaxMessageBytesProp, Int.MaxValue: java.lang.Integer)
+    logProperties.put(LogConfig.SegmentIndexBytesProp, 1024*1024: java.lang.Integer)
 
     val log = new Log(dir = dir,
-                      config = LogConfig(logProprties),
+                      config = LogConfig(logProperties),
                       recoveryPoint = 0L,
                       scheduler = time.scheduler,
                       time = time)
@@ -84,7 +84,7 @@ object StressTestLog {
   class WriterThread(val log: Log) extends WorkerThread {
     @volatile var offset = 0
     override def work() {
-      val logAppendInfo = log.append(TestUtils.singleMessageSet(offset.toString.getBytes))
+      val logAppendInfo = log.append(TestUtils.singletonRecords(offset.toString.getBytes))
       require(logAppendInfo.firstOffset == offset && logAppendInfo.lastOffset == offset)
       offset += 1
       if(offset % 1000 == 0)
@@ -96,11 +96,11 @@ object StressTestLog {
     @volatile var offset = 0
     override def work() {
       try {
-        log.read(offset, 1024, Some(offset+1)).messageSet match {
-          case read: FileMessageSet if read.sizeInBytes > 0 => {
-            val first = read.head
+        log.read(offset, 1024, Some(offset+1)).records match {
+          case read: FileRecords if read.sizeInBytes > 0 => {
+            val first = read.shallowIterator.next()
             require(first.offset == offset, "We should either read nothing or the message we asked for.")
-            require(MessageSet.entrySize(first.message) == read.sizeInBytes, "Expected %d but got %d.".format(MessageSet.entrySize(first.message), read.sizeInBytes))
+            require(first.sizeInBytes == read.sizeInBytes, "Expected %d but got %d.".format(first.sizeInBytes, read.sizeInBytes))
             offset += 1
           }
           case _ =>

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala
index 6fef2b3..f0883ad 100755
--- a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala
+++ b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala
@@ -22,13 +22,14 @@ import java.nio._
 import java.nio.channels._
 import java.util.{Properties, Random}
 
+import joptsimple._
 import kafka.log._
-import kafka.utils._
 import kafka.message._
+import kafka.utils._
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record}
+import org.apache.kafka.common.utils.{Time, Utils}
 
 import scala.math._
-import joptsimple._
-import org.apache.kafka.common.utils.{Time, Utils}
 
 /**
  * This test does linear writes using either a kafka log or a file and measures throughput and latency.
@@ -64,7 +65,7 @@ object TestLinearWriteSpeed {
                            .withRequiredArg
                            .describedAs("ms")
                            .ofType(classOf[java.lang.Long])
-                           .defaultsTo(1000)
+                           .defaultsTo(1000L)
    val maxThroughputOpt = parser.accepts("max-throughput-mb", "The maximum throughput.")
                            .withRequiredArg
                            .describedAs("mb")
@@ -81,7 +82,7 @@ object TestLinearWriteSpeed {
                             .ofType(classOf[java.lang.String])
                             .defaultsTo(NoCompressionCodec.name)
    val mmapOpt = parser.accepts("mmap", "Do writes to memory-mapped files.")
-   val channelOpt = parser.accepts("channel", "Do writes to file channesl.")
+   val channelOpt = parser.accepts("channel", "Do writes to file channels.")
    val logOpt = parser.accepts("log", "Do writes to kafka logs.")
                           
     val options = parser.parse(args : _*)
@@ -101,9 +102,9 @@ object TestLinearWriteSpeed {
     val rand = new Random
     rand.nextBytes(buffer.array)
     val numMessages = bufferSize / (messageSize + MessageSet.LogOverhead)
-    val messageSet = new ByteBufferMessageSet(compressionCodec = compressionCodec,
-      messages = (0 until numMessages).map(_ => new Message(new Array[Byte](messageSize))): _*)
-    
+    val messageSet = MemoryRecords.withRecords(CompressionType.forId(compressionCodec.codec),
+      (0 until numMessages).map(_ => Record.create(new Array[Byte](messageSize))): _*)
+
     val writables = new Array[Writable](numFiles)
     val scheduler = new KafkaScheduler(1)
     scheduler.startup()
@@ -199,7 +200,7 @@ object TestLinearWriteSpeed {
     }
   }
   
-  class LogWritable(val dir: File, config: LogConfig, scheduler: Scheduler, val messages: ByteBufferMessageSet) extends Writable {
+  class LogWritable(val dir: File, config: LogConfig, scheduler: Scheduler, val messages: MemoryRecords) extends Writable {
     Utils.delete(dir)
     val log = new Log(dir, config, 0L, scheduler, Time.SYSTEM)
     def write(): Int = {

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
index d1fcbc0..b98822d 100644
--- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
+++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
@@ -301,7 +301,7 @@ class DeleteTopicTest extends ZooKeeperTestHarness {
     var counter = 0
     for (_ <- 0 until numDups; key <- 0 until numKeys) yield {
       val count = counter
-      log.append(TestUtils.singleMessageSet(payload = counter.toString.getBytes, key = key.toString.getBytes), assignOffsets = true)
+      log.append(TestUtils.singletonRecords(value = counter.toString.getBytes, key = key.toString.getBytes), assignOffsets = true)
       counter += 1
       (key, count)
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala
index a981e68..1c5a526 100644
--- a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala
@@ -18,20 +18,20 @@
 package kafka.coordinator
 
 import kafka.utils.timer.MockTimer
-import org.apache.kafka.common.record.Record
+import org.apache.kafka.common.record.{MemoryRecords, Record, TimestampType}
 import org.junit.Assert._
 import kafka.common.{OffsetAndMetadata, Topic}
-import kafka.message.{Message, MessageSet}
-import kafka.server.{DelayedOperationPurgatory, ReplicaManager, KafkaConfig}
+import kafka.server.{DelayedOperationPurgatory, KafkaConfig, ReplicaManager}
 import kafka.utils._
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.protocol.Errors
-import org.apache.kafka.common.requests.{OffsetCommitRequest, JoinGroupRequest}
+import org.apache.kafka.common.requests.{JoinGroupRequest, OffsetCommitRequest}
 import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
-import org.easymock.{Capture, IAnswer, EasyMock}
+import org.easymock.{Capture, EasyMock, IAnswer}
 import org.junit.{After, Before, Test}
 import org.scalatest.junit.JUnitSuite
 import java.util.concurrent.TimeUnit
+
 import scala.collection._
 import scala.concurrent.duration.Duration
 import scala.concurrent.{Await, Future, Promise}
@@ -305,7 +305,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
 
     EasyMock.reset(replicaManager)
     EasyMock.expect(replicaManager.getPartition(Topic.GroupMetadataTopicName, groupPartitionId)).andReturn(None)
-    EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(Some(Message.MagicValue_V1)).anyTimes()
+    EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
+      .andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes()
     EasyMock.replay(replicaManager)
 
     timer.advanceClock(DefaultSessionTimeout + 100)
@@ -988,17 +989,18 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
 
     val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
 
-    EasyMock.expect(replicaManager.appendMessages(EasyMock.anyLong(),
+    EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
       EasyMock.anyShort(),
       EasyMock.anyBoolean(),
-      EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MessageSet]],
+      EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
       EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
       override def answer = capturedArgument.getValue.apply(
         Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) ->
           new PartitionResponse(Errors.NONE.code, 0L, Record.NO_TIMESTAMP)
         )
       )})
-    EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(Some(Message.MagicValue_V1)).anyTimes()
+    EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
+      .andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes()
     EasyMock.replay(replicaManager)
 
     groupCoordinator.handleSyncGroup(groupId, generation, leaderId, assignment, responseCallback)
@@ -1069,17 +1071,18 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
 
     val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
 
-    EasyMock.expect(replicaManager.appendMessages(EasyMock.anyLong(),
+    EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
       EasyMock.anyShort(),
       EasyMock.anyBoolean(),
-      EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MessageSet]],
+      EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
       EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
       override def answer = capturedArgument.getValue.apply(
         Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) ->
           new PartitionResponse(Errors.NONE.code, 0L, Record.NO_TIMESTAMP)
         )
       )})
-    EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(Some(Message.MagicValue_V1)).anyTimes()
+    EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
+      .andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes()
     EasyMock.replay(replicaManager)
 
     groupCoordinator.handleCommitOffsets(groupId, consumerId, generationId, offsets, responseCallback)
@@ -1090,7 +1093,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
     val (responseFuture, responseCallback) = setupHeartbeatCallback
 
     EasyMock.expect(replicaManager.getPartition(Topic.GroupMetadataTopicName, groupPartitionId)).andReturn(None)
-    EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(Some(Message.MagicValue_V1)).anyTimes()
+    EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
+      .andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes()
     EasyMock.replay(replicaManager)
 
     groupCoordinator.handleLeaveGroup(groupId, consumerId, responseCallback)

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala
index 6c03476..62b7f42 100644
--- a/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala
@@ -21,12 +21,11 @@ import kafka.api.ApiVersion
 import kafka.cluster.Partition
 import kafka.common.{OffsetAndMetadata, Topic}
 import kafka.log.LogAppendInfo
-import kafka.message.{ByteBufferMessageSet, Message, MessageSet}
 import kafka.server.{KafkaConfig, ReplicaManager}
 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.Record
+import org.apache.kafka.common.record.{MemoryRecords, Record, TimestampType}
 import org.apache.kafka.common.requests.OffsetFetchResponse
 import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
 import org.easymock.{Capture, EasyMock, IAnswer}
@@ -34,6 +33,7 @@ import org.junit.{After, Before, Test}
 import org.junit.Assert._
 
 import scala.collection._
+import JavaConverters._
 
 class GroupMetadataManagerTest {
 
@@ -50,7 +50,6 @@ class GroupMetadataManagerTest {
   val rebalanceTimeout = 60000
   val sessionTimeout = 10000
 
-
   @Before
   def setUp() {
     val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(nodeId = 0, zkConnect = ""))
@@ -176,7 +175,7 @@ class GroupMetadataManagerTest {
 
   @Test
   def testStoreNonEmptyGroupWhenCoordinatorHasMoved() {
-    EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(None)
+    EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())).andReturn(None)
     val memberId = "memberId"
     val clientId = "clientId"
     val clientHost = "localhost"
@@ -245,7 +244,7 @@ class GroupMetadataManagerTest {
 
   @Test
   def testCommitOffsetWhenCoordinatorHasMoved() {
-    EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(None)
+    EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())).andReturn(None)
     val memberId = ""
     val generationId = -1
     val topicPartition = new TopicPartition("foo", 0)
@@ -363,7 +362,7 @@ class GroupMetadataManagerTest {
     time.sleep(2)
 
     EasyMock.reset(partition)
-    EasyMock.expect(partition.appendMessagesToLeader(EasyMock.anyObject(classOf[ByteBufferMessageSet]), EasyMock.anyInt()))
+    EasyMock.expect(partition.appendRecordsToLeader(EasyMock.anyObject(classOf[MemoryRecords]), EasyMock.anyInt()))
       .andReturn(LogAppendInfo.UnknownLogAppendInfo)
     EasyMock.replay(partition)
 
@@ -391,24 +390,74 @@ class GroupMetadataManagerTest {
 
     // expect the group metadata tombstone
     EasyMock.reset(partition)
-    val messageSetCapture: Capture[ByteBufferMessageSet] = EasyMock.newCapture()
+    val recordsCapture: Capture[MemoryRecords] = EasyMock.newCapture()
+
+    EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
+      .andStubReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME))
+    EasyMock.expect(replicaManager.getPartition(Topic.GroupMetadataTopicName, groupPartitionId)).andStubReturn(Some(partition))
+    EasyMock.expect(partition.appendRecordsToLeader(EasyMock.capture(recordsCapture), EasyMock.anyInt()))
+      .andReturn(LogAppendInfo.UnknownLogAppendInfo)
+    EasyMock.replay(replicaManager, partition)
+
+    groupMetadataManager.cleanupGroupMetadata()
+
+    assertTrue(recordsCapture.hasCaptured)
+
+    val records = recordsCapture.getValue.records.asScala.toList
+    assertEquals(1, records.size)
+
+    val metadataTombstone = records.head
+    assertTrue(metadataTombstone.hasKey)
+    assertTrue(metadataTombstone.hasNullValue)
+    assertEquals(Record.MAGIC_VALUE_V1, metadataTombstone.magic)
+    assertEquals(TimestampType.CREATE_TIME, metadataTombstone.timestampType)
+    assertTrue(metadataTombstone.timestamp > 0)
+
+    val groupKey = GroupMetadataManager.readMessageKey(metadataTombstone.key).asInstanceOf[GroupMetadataKey]
+    assertEquals(groupId, groupKey.key)
+
+    // the full group should be gone since all offsets were removed
+    assertEquals(None, groupMetadataManager.getGroup(groupId))
+    val cachedOffsets = groupMetadataManager.getOffsets(groupId, Seq(topicPartition1, topicPartition2))
+    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), cachedOffsets.get(topicPartition1).map(_.offset))
+    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), cachedOffsets.get(topicPartition2).map(_.offset))
+  }
+
+  @Test
+  def testGroupMetadataRemovalWithLogAppendTime() {
+    val topicPartition1 = new TopicPartition("foo", 0)
+    val topicPartition2 = new TopicPartition("foo", 1)
+
+    groupMetadataManager.addPartitionOwnership(groupPartitionId)
+
+    val group = new GroupMetadata(groupId)
+    groupMetadataManager.addGroup(group)
+    group.generationId = 5
+
+    // expect the group metadata tombstone
+    EasyMock.reset(partition)
+    val recordsCapture: Capture[MemoryRecords] = EasyMock.newCapture()
 
-    EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andStubReturn(Some(Message.MagicValue_V1))
+    EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
+      .andStubReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.LOG_APPEND_TIME))
     EasyMock.expect(replicaManager.getPartition(Topic.GroupMetadataTopicName, groupPartitionId)).andStubReturn(Some(partition))
-    EasyMock.expect(partition.appendMessagesToLeader(EasyMock.capture(messageSetCapture), EasyMock.anyInt()))
+    EasyMock.expect(partition.appendRecordsToLeader(EasyMock.capture(recordsCapture), EasyMock.anyInt()))
       .andReturn(LogAppendInfo.UnknownLogAppendInfo)
     EasyMock.replay(replicaManager, partition)
 
     groupMetadataManager.cleanupGroupMetadata()
 
-    assertTrue(messageSetCapture.hasCaptured)
+    assertTrue(recordsCapture.hasCaptured)
 
-    val messageSet = messageSetCapture.getValue
-    assertEquals(1, messageSet.size)
+    val records = recordsCapture.getValue.records.asScala.toList
+    assertEquals(1, records.size)
 
-    val metadataTombstone = messageSet.head.message
+    val metadataTombstone = records.head
     assertTrue(metadataTombstone.hasKey)
-    assertTrue(metadataTombstone.isNull)
+    assertTrue(metadataTombstone.hasNullValue)
+    assertEquals(Record.MAGIC_VALUE_V1, metadataTombstone.magic)
+    assertEquals(TimestampType.LOG_APPEND_TIME, metadataTombstone.timestampType)
+    assertTrue(metadataTombstone.timestamp > 0)
 
     val groupKey = GroupMetadataManager.readMessageKey(metadataTombstone.key).asInstanceOf[GroupMetadataKey]
     assertEquals(groupId, groupKey.key)
@@ -463,22 +512,22 @@ class GroupMetadataManagerTest {
 
     // expect the offset tombstone
     EasyMock.reset(partition)
-    val messageSetCapture: Capture[ByteBufferMessageSet] = EasyMock.newCapture()
+    val recordsCapture: Capture[MemoryRecords] = EasyMock.newCapture()
 
-    EasyMock.expect(partition.appendMessagesToLeader(EasyMock.capture(messageSetCapture), EasyMock.anyInt()))
+    EasyMock.expect(partition.appendRecordsToLeader(EasyMock.capture(recordsCapture), EasyMock.anyInt()))
       .andReturn(LogAppendInfo.UnknownLogAppendInfo)
     EasyMock.replay(partition)
 
     groupMetadataManager.cleanupGroupMetadata()
 
-    assertTrue(messageSetCapture.hasCaptured)
+    assertTrue(recordsCapture.hasCaptured)
 
     // verify the tombstones are correct and only for the expired offsets
-    val messageSet = messageSetCapture.getValue
-    assertEquals(2, messageSet.size)
-    messageSet.map(_.message).foreach { message =>
+    val records = recordsCapture.getValue.records.asScala.toList
+    assertEquals(2, records.size)
+    records.foreach { message =>
       assertTrue(message.hasKey)
-      assertTrue(message.isNull)
+      assertTrue(message.hasNullValue)
       val offsetKey = GroupMetadataManager.readMessageKey(message.key).asInstanceOf[OffsetKey]
       assertEquals(groupId, offsetKey.key.group)
       assertEquals("foo", offsetKey.key.topicPartition.topic)
@@ -539,7 +588,7 @@ class GroupMetadataManagerTest {
 
     // expect the offset tombstone
     EasyMock.reset(partition)
-    EasyMock.expect(partition.appendMessagesToLeader(EasyMock.anyObject(classOf[ByteBufferMessageSet]), EasyMock.anyInt()))
+    EasyMock.expect(partition.appendRecordsToLeader(EasyMock.anyObject(classOf[MemoryRecords]), EasyMock.anyInt()))
       .andReturn(LogAppendInfo.UnknownLogAppendInfo)
     EasyMock.replay(partition)
 
@@ -557,17 +606,18 @@ class GroupMetadataManagerTest {
 
   private def expectAppendMessage(error: Errors) {
     val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
-    EasyMock.expect(replicaManager.appendMessages(EasyMock.anyLong(),
+    EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
       EasyMock.anyShort(),
       EasyMock.anyBoolean(),
-      EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MessageSet]],
+      EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
       EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
       override def answer = capturedArgument.getValue.apply(
         Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) ->
           new PartitionResponse(error.code, 0L, Record.NO_TIMESTAMP)
         )
       )})
-    EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andStubReturn(Some(Message.MagicValue_V1))
+    EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
+      .andStubReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME))
   }
 
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 791bdb0..296dc15 100755
--- a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala
+++ b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala
@@ -25,9 +25,10 @@ import org.junit.Assert._
 import org.junit.runner.RunWith
 import org.junit.runners.Parameterized
 import org.junit.runners.Parameterized.Parameters
-import org.apache.kafka.common.record.CompressionType
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record}
 import org.apache.kafka.common.utils.Utils
 import java.util.{Collection, Properties}
+
 import scala.collection.JavaConverters._
 
 @RunWith(value = classOf[Parameterized])
@@ -50,22 +51,22 @@ class BrokerCompressionTest(messageCompression: String, brokerCompression: Strin
   def testBrokerSideCompression() {
     val messageCompressionCode = CompressionCodec.getCompressionCodec(messageCompression)
     val logProps = new Properties()
-    logProps.put(LogConfig.CompressionTypeProp,brokerCompression)
+    logProps.put(LogConfig.CompressionTypeProp, brokerCompression)
     /*configure broker-side compression  */
     val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
 
     /* append two messages */
-    log.append(new ByteBufferMessageSet(messageCompressionCode, new Message("hello".getBytes), new Message("there".getBytes)))
+    log.append(MemoryRecords.withRecords(CompressionType.forId(messageCompressionCode.codec),
+      Record.create("hello".getBytes), Record.create("there".getBytes)))
 
-    def readMessage(offset: Int) = log.read(offset, 4096).messageSet.head.message
+    def readMessage(offset: Int) = log.read(offset, 4096).records.shallowIterator.next().record
 
     if (!brokerCompression.equals("producer")) {
       val brokerCompressionCode = BrokerCompressionCodec.getCompressionCodec(brokerCompression)
-      assertEquals("Compression at offset 0 should produce " + brokerCompressionCode.name, brokerCompressionCode, readMessage(0).compressionCodec)
+      assertEquals("Compression at offset 0 should produce " + brokerCompressionCode.name, brokerCompressionCode.codec, readMessage(0).compressionType.id)
     }
     else
-      assertEquals("Compression at offset 0 should produce " + messageCompressionCode.name, messageCompressionCode, readMessage(0).compressionCodec)
-
+      assertEquals("Compression at offset 0 should produce " + messageCompressionCode.name, messageCompressionCode.codec, readMessage(0).compressionType.id)
   }
 
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala
deleted file mode 100644
index a7f0446..0000000
--- a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala
+++ /dev/null
@@ -1,354 +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._
-import java.nio.channels._
-
-import kafka.common.LongRef
-import org.junit.Assert._
-import kafka.utils.TestUtils._
-import kafka.message._
-import kafka.common.KafkaException
-import org.easymock.EasyMock
-import org.junit.Test
-
-class FileMessageSetTest extends BaseMessageSetTestCases {
-  
-  val messageSet = createMessageSet(messages)
-  
-  def createMessageSet(messages: Seq[Message]): FileMessageSet = {
-    val set = new FileMessageSet(tempFile())
-    set.append(new ByteBufferMessageSet(NoCompressionCodec, messages: _*))
-    set.flush()
-    set
-  }
-
-  /**
-   * Test that the cached size variable matches the actual file size as we append messages
-   */
-  @Test
-  def testFileSize() {
-    assertEquals(messageSet.channel.size, messageSet.sizeInBytes)
-    for (_ <- 0 until 20) {
-      messageSet.append(singleMessageSet("abcd".getBytes))
-      assertEquals(messageSet.channel.size, messageSet.sizeInBytes)
-    } 
-  }
-  
-  /**
-   * Test that adding invalid bytes to the end of the log doesn't break iteration
-   */
-  @Test
-  def testIterationOverPartialAndTruncation() {
-    testPartialWrite(0, messageSet)
-    testPartialWrite(2, messageSet)
-    testPartialWrite(4, messageSet)
-    testPartialWrite(5, messageSet)
-    testPartialWrite(6, messageSet)
-  }
-  
-  def testPartialWrite(size: Int, messageSet: FileMessageSet) {
-    val buffer = ByteBuffer.allocate(size)
-    for (_ <- 0 until size)
-      buffer.put(0: Byte)
-    buffer.rewind()
-    messageSet.channel.write(buffer)
-    // appending those bytes should not change the contents
-    checkEquals(messages.iterator, messageSet.map(m => m.message).iterator)
-  }
-  
-  /**
-   * Iterating over the file does file reads but shouldn't change the position of the underlying FileChannel.
-   */
-  @Test
-  def testIterationDoesntChangePosition() {
-    val position = messageSet.channel.position
-    checkEquals(messages.iterator, messageSet.map(m => m.message).iterator)
-    assertEquals(position, messageSet.channel.position)
-  }
-  
-  /**
-   * Test a simple append and read.
-   */
-  @Test
-  def testRead() {
-    var read = messageSet.read(0, messageSet.sizeInBytes)
-    checkEquals(messageSet.iterator, read.iterator)
-    val items = read.iterator.toList
-    val sec = items.tail.head
-    read = messageSet.read(position = MessageSet.entrySize(sec.message), size = messageSet.sizeInBytes)
-    assertEquals("Try a read starting from the second message", items.tail, read.toList)
-    read = messageSet.read(MessageSet.entrySize(sec.message), MessageSet.entrySize(sec.message))
-    assertEquals("Try a read of a single message starting from the second message", List(items.tail.head), read.toList)
-  }
-  
-  /**
-   * Test the MessageSet.searchFor API.
-   */
-  @Test
-  def testSearch() {
-    // append a new message with a high offset
-    val lastMessage = new Message("test".getBytes)
-    messageSet.append(new ByteBufferMessageSet(NoCompressionCodec, new LongRef(50), lastMessage))
-    val messages = messageSet.toSeq
-    var position = 0
-    val message1Size = MessageSet.entrySize(messages.head.message)
-    assertEquals("Should be able to find the first message by its offset",
-                 (OffsetPosition(0L, position), message1Size),
-                 messageSet.searchForOffsetWithSize(0, 0))
-    position += message1Size
-    val message2Size = MessageSet.entrySize(messages(1).message)
-    assertEquals("Should be able to find second message when starting from 0", 
-                 (OffsetPosition(1L, position), message2Size),
-                 messageSet.searchForOffsetWithSize(1, 0))
-    assertEquals("Should be able to find second message starting from its offset", 
-                 (OffsetPosition(1L, position), message2Size),
-                 messageSet.searchForOffsetWithSize(1, position))
-    position += message2Size + MessageSet.entrySize(messages(2).message)
-    val message4Size = MessageSet.entrySize(messages(3).message)
-    assertEquals("Should be able to find fourth message from a non-existant offset", 
-                 (OffsetPosition(50L, position), message4Size),
-                 messageSet.searchForOffsetWithSize(3, position))
-    assertEquals("Should be able to find fourth message by correct offset",
-                 (OffsetPosition(50L, position), message4Size),
-                 messageSet.searchForOffsetWithSize(50,  position))
-  }
-  
-  /**
-   * Test that the message set iterator obeys start and end slicing
-   */
-  @Test
-  def testIteratorWithLimits() {
-    val message = messageSet.toList(1)
-    val start = messageSet.searchForOffsetWithSize(1, 0)._1.position
-    val size = message.message.size + 12
-    val slice = messageSet.read(start, size)
-    assertEquals(List(message), slice.toList)
-    val slice2 = messageSet.read(start, size - 1)
-    assertEquals(List(), slice2.toList)
-  }
-
-  /**
-   * Test the truncateTo method lops off messages and appropriately updates the size
-   */
-  @Test
-  def testTruncate() {
-    val message = messageSet.toList.head
-    val end = messageSet.searchForOffsetWithSize(1, 0)._1.position
-    messageSet.truncateTo(end)
-    assertEquals(List(message), messageSet.toList)
-    assertEquals(MessageSet.entrySize(message.message), messageSet.sizeInBytes)
-  }
-
-  /**
-    * Test that truncateTo only calls truncate on the FileChannel if the size of the
-    * FileChannel is bigger than the target size. This is important because some JVMs
-    * change the mtime of the file, even if truncate should do nothing.
-    */
-  @Test
-  def testTruncateNotCalledIfSizeIsSameAsTargetSize() {
-    val channelMock = EasyMock.createMock(classOf[FileChannel])
-
-    EasyMock.expect(channelMock.size).andReturn(42L).atLeastOnce()
-    EasyMock.expect(channelMock.position(42L)).andReturn(null)
-    EasyMock.replay(channelMock)
-
-    val msgSet = new FileMessageSet(tempFile(), channelMock)
-    msgSet.truncateTo(42)
-
-    EasyMock.verify(channelMock)
-  }
-
-  /**
-    * Expect a KafkaException if targetSize is bigger than the size of
-    * the FileMessageSet.
-    */
-  @Test
-  def testTruncateNotCalledIfSizeIsBiggerThanTargetSize() {
-    val channelMock = EasyMock.createMock(classOf[FileChannel])
-
-    EasyMock.expect(channelMock.size).andReturn(42L).atLeastOnce()
-    EasyMock.expect(channelMock.position(42L)).andReturn(null)
-    EasyMock.replay(channelMock)
-
-    val msgSet = new FileMessageSet(tempFile(), channelMock)
-
-    try {
-      msgSet.truncateTo(43)
-      fail("Should throw KafkaException")
-    } catch {
-      case _: KafkaException => // expected
-    }
-
-    EasyMock.verify(channelMock)
-  }
-
-  /**
-    * see #testTruncateNotCalledIfSizeIsSameAsTargetSize
-    */
-  @Test
-  def testTruncateIfSizeIsDifferentToTargetSize() {
-    val channelMock = EasyMock.createMock(classOf[FileChannel])
-
-    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)
-
-    val msgSet = new FileMessageSet(tempFile(), channelMock)
-    msgSet.truncateTo(23)
-
-    EasyMock.verify(channelMock)
-  }
-
-
-  /**
-   * Test the new FileMessageSet with pre allocate as true
-   */
-  @Test
-  def testPreallocateTrue() {
-    val temp = tempFile()
-    val set = new FileMessageSet(temp, false, 512 *1024 *1024, true)
-    val position = set.channel.position
-    val size = set.sizeInBytes()
-    assertEquals(0, position)
-    assertEquals(0, size)
-    assertEquals(512 *1024 *1024, temp.length)
-  }
-
-  /**
-   * Test the new FileMessageSet with pre allocate as false
-   */
-  @Test
-  def testPreallocateFalse() {
-    val temp = tempFile()
-    val set = new FileMessageSet(temp, false, 512 *1024 *1024, false)
-    val position = set.channel.position
-    val size = set.sizeInBytes()
-    assertEquals(0, position)
-    assertEquals(0, size)
-    assertEquals(0, temp.length)
-  }
-
-  /**
-   * Test the new FileMessageSet with pre allocate as true and file has been clearly shut down, the file will be truncate to end of valid data.
-   */
-  @Test
-  def testPreallocateClearShutdown() {
-    val temp = tempFile()
-    val set = new FileMessageSet(temp, false, 512 *1024 *1024, true)
-    set.append(new ByteBufferMessageSet(NoCompressionCodec, messages: _*))
-    val oldposition = set.channel.position
-    val oldsize = set.sizeInBytes()
-    assertEquals(messageSet.sizeInBytes, oldposition)
-    assertEquals(messageSet.sizeInBytes, oldsize)
-    set.close()
-
-    val tempReopen = new File(temp.getAbsolutePath())
-    val setReopen = new FileMessageSet(tempReopen, true, 512 *1024 *1024, true)
-    val position = setReopen.channel.position
-    val size = setReopen.sizeInBytes()
-
-    assertEquals(oldposition, position)
-    assertEquals(oldposition, size)
-    assertEquals(oldposition, tempReopen.length)
-  }
-
-  @Test
-  def testFormatConversionWithPartialMessage() {
-    val message = messageSet.toList(1)
-    val start = messageSet.searchForOffsetWithSize(1, 0)._1.position
-    val size = message.message.size + 12
-    val slice = messageSet.read(start, size - 1)
-    val messageV0 = slice.toMessageFormat(Message.MagicValue_V0)
-    assertEquals("No message should be there", 0, messageV0.size)
-    assertEquals(s"There should be ${size - 1} bytes", size - 1, messageV0.sizeInBytes)
-  }
-
-  @Test
-  def testMessageFormatConversion() {
-
-    // Prepare messages.
-    val offsets = Seq(0L, 2L)
-    val messagesV0 = Seq(new Message("hello".getBytes, "k1".getBytes, Message.NoTimestamp, Message.MagicValue_V0),
-      new Message("goodbye".getBytes, "k2".getBytes, Message.NoTimestamp, Message.MagicValue_V0))
-    val messageSetV0 = new ByteBufferMessageSet(
-      compressionCodec = NoCompressionCodec,
-      offsetSeq = offsets,
-      messages = messagesV0:_*)
-    val compressedMessageSetV0 = new ByteBufferMessageSet(
-      compressionCodec = DefaultCompressionCodec,
-      offsetSeq = offsets,
-      messages = messagesV0:_*)
-
-    val messagesV1 = Seq(new Message("hello".getBytes, "k1".getBytes, 1L, Message.MagicValue_V1),
-                         new Message("goodbye".getBytes, "k2".getBytes, 2L, Message.MagicValue_V1))
-    val messageSetV1 = new ByteBufferMessageSet(
-      compressionCodec = NoCompressionCodec,
-      offsetSeq = offsets,
-      messages = messagesV1:_*)
-    val compressedMessageSetV1 = new ByteBufferMessageSet(
-      compressionCodec = DefaultCompressionCodec,
-      offsetSeq = offsets,
-      messages = messagesV1:_*)
-
-    // Down conversion
-    // down conversion for non-compressed messages
-    var fileMessageSet = new FileMessageSet(tempFile())
-    fileMessageSet.append(messageSetV1)
-    fileMessageSet.flush()
-    var convertedMessageSet = fileMessageSet.toMessageFormat(Message.MagicValue_V0)
-    verifyConvertedMessageSet(convertedMessageSet, Message.MagicValue_V0)
-
-    // down conversion for compressed messages
-    fileMessageSet = new FileMessageSet(tempFile())
-    fileMessageSet.append(compressedMessageSetV1)
-    fileMessageSet.flush()
-    convertedMessageSet = fileMessageSet.toMessageFormat(Message.MagicValue_V0)
-    verifyConvertedMessageSet(convertedMessageSet, Message.MagicValue_V0)
-
-    // Up conversion. In reality we only do down conversion, but up conversion should work as well.
-    // up conversion for non-compressed messages
-    fileMessageSet = new FileMessageSet(tempFile())
-    fileMessageSet.append(messageSetV0)
-    fileMessageSet.flush()
-    convertedMessageSet = fileMessageSet.toMessageFormat(Message.MagicValue_V1)
-    verifyConvertedMessageSet(convertedMessageSet, Message.MagicValue_V1)
-
-    // up conversion for compressed messages
-    fileMessageSet = new FileMessageSet(tempFile())
-    fileMessageSet.append(compressedMessageSetV0)
-    fileMessageSet.flush()
-    convertedMessageSet = fileMessageSet.toMessageFormat(Message.MagicValue_V1)
-    verifyConvertedMessageSet(convertedMessageSet, Message.MagicValue_V1)
-
-    def verifyConvertedMessageSet(convertedMessageSet: MessageSet, magicByte: Byte) {
-      var i = 0
-      for (messageAndOffset <- convertedMessageSet) {
-        assertEquals("magic byte should be 1", magicByte, messageAndOffset.message.magic)
-        assertEquals("offset should not change", offsets(i), messageAndOffset.offset)
-        assertEquals("key should not change", messagesV0(i).key, messageAndOffset.message.key)
-        assertEquals("payload should not change", messagesV0(i).payload, messageAndOffset.message.payload)
-        i += 1
-      }
-    }
-  }
-}


[2/9] kafka git commit: KAFKA-4390; Replace MessageSet usage with client-side alternatives

Posted by jg...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 d18719a..fcf9c89 100755
--- a/core/src/test/scala/unit/kafka/log/LogTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogTest.scala
@@ -22,15 +22,16 @@ import java.util.Properties
 
 import org.apache.kafka.common.errors.{CorruptRecordException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException}
 import kafka.api.ApiVersion
-import kafka.common.LongRef
 import org.junit.Assert._
 import org.scalatest.junit.JUnitSuite
 import org.junit.{After, Before, Test}
-import kafka.message._
 import kafka.utils._
 import kafka.server.KafkaConfig
+import org.apache.kafka.common.record._
 import org.apache.kafka.common.utils.Utils
 
+import scala.collection.JavaConverters._
+
 class LogTest extends JUnitSuite {
 
   val tmpDir = TestUtils.tempDir()
@@ -63,7 +64,7 @@ class LogTest extends JUnitSuite {
    */
   @Test
   def testTimeBasedLogRoll() {
-    val set = TestUtils.singleMessageSet("test".getBytes)
+    val set = TestUtils.singletonRecords("test".getBytes)
 
     val logProps = new Properties()
     logProps.put(LogConfig.SegmentMsProp, (1 * 60 * 60L): java.lang.Long)
@@ -91,7 +92,7 @@ class LogTest extends JUnitSuite {
 
     // Append a message with timestamp to a segment whose first messgae do not have a timestamp.
     val setWithTimestamp =
-      TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds + log.config.segmentMs + 1)
+      TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds + log.config.segmentMs + 1)
     log.append(setWithTimestamp)
     assertEquals("Segment should not have been rolled out because the log rolling should be based on wall clock.", 4, log.numberOfSegments)
 
@@ -105,14 +106,14 @@ class LogTest extends JUnitSuite {
     log.append(setWithTimestamp)
     assertEquals("Log should not roll because the roll should depend on timestamp of the first message.", 5, log.numberOfSegments)
 
-    val setWithExpiredTimestamp = TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds)
+    val setWithExpiredTimestamp = TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds)
     log.append(setWithExpiredTimestamp)
     assertEquals("Log should roll because the timestamp in the message should make the log segment expire.", 6, log.numberOfSegments)
 
     val numSegments = log.numberOfSegments
     time.sleep(log.config.segmentMs + 1)
-    log.append(new ByteBufferMessageSet())
-    assertEquals("Appending an empty message set should not roll log even if succient time has passed.", numSegments, log.numberOfSegments)
+    log.append(MemoryRecords.withLogEntries())
+    assertEquals("Appending an empty message set should not roll log even if sufficient time has passed.", numSegments, log.numberOfSegments)
   }
 
   /**
@@ -121,7 +122,7 @@ class LogTest extends JUnitSuite {
    */
   @Test
   def testTimeBasedLogRollJitter() {
-    val set = TestUtils.singleMessageSet("test".getBytes)
+    val set = TestUtils.singletonRecords("test".getBytes)
     val maxJitter = 20 * 60L
 
     val logProps = new Properties()
@@ -149,7 +150,7 @@ class LogTest extends JUnitSuite {
    */
   @Test
   def testSizeBasedLogRoll() {
-    val set = TestUtils.singleMessageSet("test".getBytes)
+    val set = TestUtils.singletonRecords("test".getBytes)
     val setSize = set.sizeInBytes
     val msgPerSeg = 10
     val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages
@@ -176,7 +177,7 @@ class LogTest extends JUnitSuite {
   def testLoadEmptyLog() {
     createEmptyLogs(logDir, 0)
     val log = new Log(logDir, logConfig, recoveryPoint = 0L, time.scheduler, time = time)
-    log.append(TestUtils.singleMessageSet("test".getBytes))
+    log.append(TestUtils.singletonRecords("test".getBytes))
   }
 
   /**
@@ -189,16 +190,17 @@ class LogTest extends JUnitSuite {
     // We use need to use magic value 1 here because the test is message size sensitive.
     logProps.put(LogConfig.MessageFormatVersionProp, ApiVersion.latestVersion.toString)
     val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
-    val messages = (0 until 100 by 2).map(id => new Message(id.toString.getBytes)).toArray
+    val records = (0 until 100 by 2).map(id => Record.create(id.toString.getBytes)).toArray
+
+    for(i <- records.indices)
+      log.append(MemoryRecords.withRecords(records(i)))
 
-    for(i <- 0 until messages.length)
-      log.append(new ByteBufferMessageSet(NoCompressionCodec, messages = messages(i)))
-    for(i <- 0 until messages.length) {
-      val read = log.read(i, 100, Some(i+1)).messageSet.head
+    for(i <- records.indices) {
+      val read = log.read(i, 100, Some(i+1)).records.shallowIterator.next()
       assertEquals("Offset read should match order appended.", i, read.offset)
-      assertEquals("Message should match appended.", messages(i), read.message)
+      assertEquals("Message should match appended.", records(i), read.record)
     }
-    assertEquals("Reading beyond the last message returns nothing.", 0, log.read(messages.length, 100, None).messageSet.size)
+    assertEquals("Reading beyond the last message returns nothing.", 0, log.read(records.length, 100, None).records.shallowIterator.asScala.size)
   }
 
   /**
@@ -211,16 +213,16 @@ class LogTest extends JUnitSuite {
     logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer)
     val log = new Log(logDir,  LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
     val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray
-    val messages = messageIds.map(id => new Message(id.toString.getBytes))
+    val records = messageIds.map(id => Record.create(id.toString.getBytes))
 
     // now test the case that we give the offsets and use non-sequential offsets
-    for(i <- 0 until messages.length)
-      log.append(new ByteBufferMessageSet(NoCompressionCodec, new LongRef(messageIds(i)), messages = messages(i)), assignOffsets = false)
+    for(i <- records.indices)
+      log.append(MemoryRecords.withLogEntries(LogEntry.create(messageIds(i), records(i))), assignOffsets = false)
     for(i <- 50 until messageIds.max) {
       val idx = messageIds.indexWhere(_ >= i)
-      val read = log.read(i, 100, None).messageSet.head
+      val read = log.read(i, 100, None).records.shallowIterator.next()
       assertEquals("Offset read should match message id.", messageIds(idx), read.offset)
-      assertEquals("Message should match appended.", messages(idx), read.message)
+      assertEquals("Message should match appended.", records(idx), read.record)
     }
   }
 
@@ -238,12 +240,12 @@ class LogTest extends JUnitSuite {
 
     // keep appending until we have two segments with only a single message in the second segment
     while(log.numberOfSegments == 1)
-      log.append(new ByteBufferMessageSet(NoCompressionCodec, messages = new Message("42".getBytes)))
+      log.append(MemoryRecords.withRecords(Record.create("42".getBytes)))
 
     // now manually truncate off all but one message from the first segment to create a gap in the messages
     log.logSegments.head.truncateTo(1)
 
-    assertEquals("A read should now return the last message in the log", log.logEndOffset - 1, log.read(1, 200, None).messageSet.head.offset)
+    assertEquals("A read should now return the last message in the log", log.logEndOffset - 1, log.read(1, 200, None).records.shallowIterator.next().offset)
   }
 
   @Test
@@ -252,12 +254,11 @@ class LogTest extends JUnitSuite {
     logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer)
     val log = new Log(logDir,  LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
     val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray
-    val messages = messageIds.map(id => new Message(id.toString.getBytes))
+    val records = messageIds.map(id => Record.create(id.toString.getBytes))
 
     // now test the case that we give the offsets and use non-sequential offsets
-    for (i <- 0 until messages.length)
-      log.append(new ByteBufferMessageSet(NoCompressionCodec, new LongRef(messageIds(i)), messages = messages(i)),
-        assignOffsets = false)
+    for (i <- records.indices)
+      log.append(MemoryRecords.withLogEntries(LogEntry.create(messageIds(i), records(i))), assignOffsets = false)
 
     for (i <- 50 until messageIds.max) {
       val idx = messageIds.indexWhere(_ >= i)
@@ -265,13 +266,13 @@ class LogTest extends JUnitSuite {
         log.read(i, 1, minOneMessage = true),
         log.read(i, 100, minOneMessage = true),
         log.read(i, 100, Some(10000), minOneMessage = true)
-      ).map(_.messageSet.head)
+      ).map(_.records.shallowIterator.next())
       reads.foreach { read =>
         assertEquals("Offset read should match message id.", messageIds(idx), read.offset)
-        assertEquals("Message should match appended.", messages(idx), read.message)
+        assertEquals("Message should match appended.", records(idx), read.record)
       }
 
-      assertEquals(Seq.empty, log.read(i, 1, Some(1), minOneMessage = true).messageSet.toIndexedSeq)
+      assertEquals(Seq.empty, log.read(i, 1, Some(1), minOneMessage = true).records.shallowIterator.asScala.toIndexedSeq)
     }
 
   }
@@ -282,15 +283,14 @@ class LogTest extends JUnitSuite {
     logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer)
     val log = new Log(logDir,  LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
     val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray
-    val messages = messageIds.map(id => new Message(id.toString.getBytes))
+    val records = messageIds.map(id => Record.create(id.toString.getBytes))
 
     // now test the case that we give the offsets and use non-sequential offsets
-    for (i <- 0 until messages.length)
-      log.append(new ByteBufferMessageSet(NoCompressionCodec, new LongRef(messageIds(i)), messages = messages(i)),
-        assignOffsets = false)
+    for (i <- records.indices)
+      log.append(MemoryRecords.withLogEntries(LogEntry.create(messageIds(i), records(i))), assignOffsets = false)
 
     for (i <- 50 until messageIds.max) {
-      assertEquals(MessageSet.Empty, log.read(i, 0).messageSet)
+      assertEquals(MemoryRecords.EMPTY, log.read(i, 0).records)
 
       // we return an incomplete message instead of an empty one for the case below
       // we use this mechanism to tell consumers of the fetch request version 2 and below that the message size is
@@ -298,9 +298,9 @@ class LogTest extends JUnitSuite {
       // in fetch request version 3, we no longer need this as we return oversized messages from the first non-empty
       // partition
       val fetchInfo = log.read(i, 1)
-      assertTrue(fetchInfo.firstMessageSetIncomplete)
-      assertTrue(fetchInfo.messageSet.isInstanceOf[FileMessageSet])
-      assertEquals(1, fetchInfo.messageSet.sizeInBytes)
+      assertTrue(fetchInfo.firstEntryIncomplete)
+      assertTrue(fetchInfo.records.isInstanceOf[FileRecords])
+      assertEquals(1, fetchInfo.records.sizeInBytes)
     }
   }
 
@@ -318,9 +318,9 @@ class LogTest extends JUnitSuite {
     // set up replica log starting with offset 1024 and with one message (at offset 1024)
     logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer)
     val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
-    log.append(new ByteBufferMessageSet(NoCompressionCodec, messages = new Message("42".getBytes)))
+    log.append(MemoryRecords.withRecords(Record.create("42".getBytes)))
 
-    assertEquals("Reading at the log end offset should produce 0 byte read.", 0, log.read(1025, 1000).messageSet.sizeInBytes)
+    assertEquals("Reading at the log end offset should produce 0 byte read.", 0, log.read(1025, 1000).records.sizeInBytes)
 
     try {
       log.read(0, 1000)
@@ -336,7 +336,7 @@ class LogTest extends JUnitSuite {
       case _: OffsetOutOfRangeException => // This is good.
     }
 
-    assertEquals("Reading from below the specified maxOffset should produce 0 byte read.", 0, log.read(1025, 1000, Some(1024)).messageSet.sizeInBytes)
+    assertEquals("Reading from below the specified maxOffset should produce 0 byte read.", 0, log.read(1025, 1000, Some(1024)).records.sizeInBytes)
   }
 
   /**
@@ -350,21 +350,22 @@ class LogTest extends JUnitSuite {
     logProps.put(LogConfig.SegmentBytesProp, 100: java.lang.Integer)
     val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
     val numMessages = 100
-    val messageSets = (0 until numMessages).map(i => TestUtils.singleMessageSet(i.toString.getBytes))
+    val messageSets = (0 until numMessages).map(i => TestUtils.singletonRecords(i.toString.getBytes))
     messageSets.foreach(log.append(_))
     log.flush
 
     /* do successive reads to ensure all our messages are there */
     var offset = 0L
     for(i <- 0 until numMessages) {
-      val messages = log.read(offset, 1024*1024).messageSet
-      assertEquals("Offsets not equal", offset, messages.head.offset)
-      assertEquals("Messages not equal at offset " + offset, messageSets(i).head.message,
-        messages.head.message.toFormatVersion(messageSets(i).head.message.magic))
-      offset = messages.head.offset + 1
+      val messages = log.read(offset, 1024*1024).records.shallowIterator
+      val head = messages.next()
+      assertEquals("Offsets not equal", offset, head.offset)
+      assertEquals("Messages not equal at offset " + offset, messageSets(i).shallowIterator.next().record,
+        head.record.convert(messageSets(i).shallowIterator.next().record.magic))
+      offset = head.offset + 1
     }
-    val lastRead = log.read(startOffset = numMessages, maxLength = 1024*1024, maxOffset = Some(numMessages + 1)).messageSet
-    assertEquals("Should be no more messages", 0, lastRead.size)
+    val lastRead = log.read(startOffset = numMessages, maxLength = 1024*1024, maxOffset = Some(numMessages + 1)).records
+    assertEquals("Should be no more messages", 0, lastRead.shallowIterator.asScala.size)
 
     // check that rolling the log forced a flushed the log--the flush is asyn so retry in case of failure
     TestUtils.retry(1000L){
@@ -383,10 +384,10 @@ class LogTest extends JUnitSuite {
     val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
 
     /* append 2 compressed message sets, each with two messages giving offsets 0, 1, 2, 3 */
-    log.append(new ByteBufferMessageSet(DefaultCompressionCodec, new Message("hello".getBytes), new Message("there".getBytes)))
-    log.append(new ByteBufferMessageSet(DefaultCompressionCodec, new Message("alpha".getBytes), new Message("beta".getBytes)))
+    log.append(MemoryRecords.withRecords(CompressionType.GZIP, Record.create("hello".getBytes), Record.create("there".getBytes)))
+    log.append(MemoryRecords.withRecords(CompressionType.GZIP, Record.create("alpha".getBytes), Record.create("beta".getBytes)))
 
-    def read(offset: Int) = ByteBufferMessageSet.deepIterator(log.read(offset, 4096).messageSet.head)
+    def read(offset: Int) = log.read(offset, 4096).records.deepIterator
 
     /* we should always get the first message in the compressed set when reading any offset in the set */
     assertEquals("Read at offset 0 should produce 0", 0, read(0).next().offset)
@@ -408,7 +409,7 @@ class LogTest extends JUnitSuite {
       logProps.put(LogConfig.RetentionMsProp, 0: java.lang.Integer)
       val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
       for(i <- 0 until messagesToAppend)
-        log.append(TestUtils.singleMessageSet(payload = i.toString.getBytes, timestamp = time.milliseconds - 10))
+        log.append(TestUtils.singletonRecords(value = i.toString.getBytes, timestamp = time.milliseconds - 10))
 
       val currOffset = log.logEndOffset
       assertEquals(currOffset, messagesToAppend)
@@ -422,7 +423,7 @@ class LogTest extends JUnitSuite {
       assertEquals("Still no change in the logEndOffset", currOffset, log.logEndOffset)
       assertEquals("Should still be able to append and should get the logEndOffset assigned to the new append",
                    currOffset,
-                   log.append(TestUtils.singleMessageSet("hello".getBytes)).firstOffset)
+                   log.append(TestUtils.singletonRecords("hello".getBytes)).firstOffset)
 
       // cleanup the log
       log.delete()
@@ -435,7 +436,7 @@ class LogTest extends JUnitSuite {
    */
   @Test
   def testMessageSetSizeCheck() {
-    val messageSet = new ByteBufferMessageSet(NoCompressionCodec, new Message ("You".getBytes), new Message("bethe".getBytes))
+    val messageSet = MemoryRecords.withRecords(Record.create("You".getBytes), Record.create("bethe".getBytes))
     // append messages to log
     val configSegmentSize = messageSet.sizeInBytes - 1
     val logProps = new Properties()
@@ -454,17 +455,17 @@ class LogTest extends JUnitSuite {
 
   @Test
   def testCompactedTopicConstraints() {
-    val keyedMessage = new Message(bytes = "this message has a key".getBytes, key = "and here it is".getBytes, Message.NoTimestamp, Message.CurrentMagicValue)
-    val anotherKeyedMessage = new Message(bytes = "this message also has a key".getBytes, key ="another key".getBytes, Message.NoTimestamp, Message.CurrentMagicValue)
-    val unkeyedMessage = new Message(bytes = "this message does not have a key".getBytes)
+    val keyedMessage = Record.create(Record.CURRENT_MAGIC_VALUE, Record.NO_TIMESTAMP, "and here it is".getBytes, "this message has a key".getBytes)
+    val anotherKeyedMessage = Record.create(Record.CURRENT_MAGIC_VALUE, Record.NO_TIMESTAMP, "another key".getBytes, "this message also has a key".getBytes)
+    val unkeyedMessage = Record.create("this message does not have a key".getBytes)
 
-    val messageSetWithUnkeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, unkeyedMessage, keyedMessage)
-    val messageSetWithOneUnkeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, unkeyedMessage)
-    val messageSetWithCompressedKeyedMessage = new ByteBufferMessageSet(GZIPCompressionCodec, keyedMessage)
-    val messageSetWithCompressedUnkeyedMessage = new ByteBufferMessageSet(GZIPCompressionCodec, keyedMessage, unkeyedMessage)
+    val messageSetWithUnkeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, unkeyedMessage, keyedMessage)
+    val messageSetWithOneUnkeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, unkeyedMessage)
+    val messageSetWithCompressedKeyedMessage = MemoryRecords.withRecords(CompressionType.GZIP, keyedMessage)
+    val messageSetWithCompressedUnkeyedMessage = MemoryRecords.withRecords(CompressionType.GZIP, keyedMessage, unkeyedMessage)
 
-    val messageSetWithKeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, keyedMessage)
-    val messageSetWithKeyedMessages = new ByteBufferMessageSet(NoCompressionCodec, keyedMessage, anotherKeyedMessage)
+    val messageSetWithKeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, keyedMessage)
+    val messageSetWithKeyedMessages = MemoryRecords.withRecords(CompressionType.NONE, keyedMessage, anotherKeyedMessage)
 
     val logProps = new Properties()
     logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact)
@@ -502,8 +503,8 @@ class LogTest extends JUnitSuite {
    */
   @Test
   def testMessageSizeCheck() {
-    val first = new ByteBufferMessageSet(NoCompressionCodec, new Message ("You".getBytes), new Message("bethe".getBytes))
-    val second = new ByteBufferMessageSet(NoCompressionCodec, new Message("change (I need more bytes)".getBytes))
+    val first = MemoryRecords.withRecords(CompressionType.NONE, Record.create("You".getBytes), Record.create("bethe".getBytes))
+    val second = MemoryRecords.withRecords(CompressionType.NONE, Record.create("change (I need more bytes)".getBytes))
 
     // append messages to log
     val maxMessageSize = second.sizeInBytes - 1
@@ -537,7 +538,7 @@ class LogTest extends JUnitSuite {
     val config = LogConfig(logProps)
     var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
     for(i <- 0 until numMessages)
-      log.append(TestUtils.singleMessageSet(payload = TestUtils.randomBytes(messageSize),
+      log.append(TestUtils.singletonRecords(value = TestUtils.randomBytes(messageSize),
         timestamp = time.milliseconds + i * 10))
     assertEquals("After appending %d messages to an empty log, the log end offset should be %d".format(numMessages, numMessages), numMessages, log.logEndOffset)
     val lastIndexOffset = log.activeSegment.index.lastOffset
@@ -585,7 +586,7 @@ class LogTest extends JUnitSuite {
     val log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
 
     val messages = (0 until numMessages).map { i =>
-      new ByteBufferMessageSet(NoCompressionCodec, new LongRef(100 + i), new Message(i.toString.getBytes(), time.milliseconds + i, Message.MagicValue_V1))
+      MemoryRecords.withLogEntries(LogEntry.create(100 + i, Record.create(Record.MAGIC_VALUE_V1, time.milliseconds + i, i.toString.getBytes())))
     }
     messages.foreach(log.append(_, assignOffsets = false))
     val timeIndexEntries = log.logSegments.foldLeft(0) { (entries, segment) => entries + segment.timeIndex.entries }
@@ -608,7 +609,7 @@ class LogTest extends JUnitSuite {
     val config = LogConfig(logProps)
     var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
     for(i <- 0 until numMessages)
-      log.append(TestUtils.singleMessageSet(payload = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10))
+      log.append(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10))
     val indexFiles = log.logSegments.map(_.index.file)
     val timeIndexFiles = log.logSegments.map(_.timeIndex.file)
     log.close()
@@ -623,7 +624,7 @@ class LogTest extends JUnitSuite {
     assertTrue("The index should have been rebuilt", log.logSegments.head.index.entries > 0)
     assertTrue("The time index should have been rebuilt", log.logSegments.head.timeIndex.entries > 0)
     for(i <- 0 until numMessages) {
-      assertEquals(i, log.read(i, 100, None).messageSet.head.offset)
+      assertEquals(i, log.read(i, 100, None).records.shallowIterator.next().offset)
       if (i == 0)
         assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetsByTimestamp(time.milliseconds + i * 10).get.offset)
       else
@@ -647,7 +648,7 @@ class LogTest extends JUnitSuite {
     val config = LogConfig(logProps)
     var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
     for(i <- 0 until numMessages)
-      log.append(TestUtils.singleMessageSet(payload = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10))
+      log.append(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10))
     val timeIndexFiles = log.logSegments.map(_.timeIndex.file)
     log.close()
 
@@ -676,7 +677,7 @@ class LogTest extends JUnitSuite {
     val config = LogConfig(logProps)
     var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
     for(i <- 0 until numMessages)
-      log.append(TestUtils.singleMessageSet(payload = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10))
+      log.append(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10))
     val indexFiles = log.logSegments.map(_.index.file)
     val timeIndexFiles = log.logSegments.map(_.timeIndex.file)
     log.close()
@@ -699,7 +700,7 @@ class LogTest extends JUnitSuite {
     log = new Log(logDir, config, recoveryPoint = 200L, time.scheduler, time)
     assertEquals("Should have %d messages when log is reopened".format(numMessages), numMessages, log.logEndOffset)
     for(i <- 0 until numMessages) {
-      assertEquals(i, log.read(i, 100, None).messageSet.head.offset)
+      assertEquals(i, log.read(i, 100, None).records.shallowIterator.next().offset)
       if (i == 0)
         assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetsByTimestamp(time.milliseconds + i * 10).get.offset)
       else
@@ -713,7 +714,7 @@ class LogTest extends JUnitSuite {
    */
   @Test
   def testTruncateTo() {
-    val set = TestUtils.singleMessageSet("test".getBytes)
+    val set = TestUtils.singletonRecords("test".getBytes)
     val setSize = set.sizeInBytes
     val msgPerSeg = 10
     val segmentSize = msgPerSeg * setSize  // each segment will be 10 messages
@@ -770,7 +771,7 @@ class LogTest extends JUnitSuite {
    */
   @Test
   def testIndexResizingAtTruncation() {
-    val setSize = TestUtils.singleMessageSet(payload = "test".getBytes).sizeInBytes
+    val setSize = TestUtils.singletonRecords(value = "test".getBytes).sizeInBytes
     val msgPerSeg = 10
     val segmentSize = msgPerSeg * setSize  // each segment will be 10 messages
     val logProps = new Properties()
@@ -781,12 +782,12 @@ class LogTest extends JUnitSuite {
     assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
 
     for (i<- 1 to msgPerSeg)
-      log.append(TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds + i))
+      log.append(TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds + i))
     assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
 
     time.sleep(msgPerSeg)
     for (i<- 1 to msgPerSeg)
-      log.append(TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds + i))
+      log.append(TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds + i))
     assertEquals("There should be exactly 2 segment.", 2, log.numberOfSegments)
     val expectedEntries = msgPerSeg - 1
 
@@ -800,7 +801,7 @@ class LogTest extends JUnitSuite {
 
     time.sleep(msgPerSeg)
     for (i<- 1 to msgPerSeg)
-      log.append(TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds + i))
+      log.append(TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds + i))
     assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
   }
 
@@ -814,7 +815,7 @@ class LogTest extends JUnitSuite {
     val bogusIndex2 = Log.indexFilename(logDir, 5)
     val bogusTimeIndex2 = Log.timeIndexFilename(logDir, 5)
 
-    val set = TestUtils.singleMessageSet("test".getBytes)
+    val set = TestUtils.singletonRecords("test".getBytes)
     val logProps = new Properties()
     logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
     logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
@@ -842,7 +843,7 @@ class LogTest extends JUnitSuite {
    */
   @Test
   def testReopenThenTruncate() {
-    val set = TestUtils.singleMessageSet("test".getBytes)
+    val set = TestUtils.singletonRecords("test".getBytes)
     val logProps = new Properties()
     logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
     logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
@@ -875,7 +876,7 @@ class LogTest extends JUnitSuite {
    */
   @Test
   def testAsyncDelete() {
-    val set = TestUtils.singleMessageSet("test".getBytes)
+    val set = TestUtils.singletonRecords("test".getBytes)
     val asyncDeleteMs = 1000
     val logProps = new Properties()
     logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
@@ -921,7 +922,7 @@ class LogTest extends JUnitSuite {
    */
   @Test
   def testOpenDeletesObsoleteFiles() {
-    val set = TestUtils.singleMessageSet("test".getBytes)
+    val set = TestUtils.singletonRecords("test".getBytes)
     val logProps = new Properties()
     logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
     logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
@@ -957,10 +958,10 @@ class LogTest extends JUnitSuite {
                       recoveryPoint = 0L,
                       time.scheduler,
                       time)
-    log.append(new ByteBufferMessageSet(new Message(bytes = null)))
-    val messageSet = log.read(0, 4096, None).messageSet
-    assertEquals(0, messageSet.head.offset)
-    assertTrue("Message payload should be null.", messageSet.head.message.isNull)
+    log.append(MemoryRecords.withRecords(Record.create(null)))
+    val head = log.read(0, 4096, None).records.shallowIterator().next()
+    assertEquals(0, head.offset)
+    assertTrue("Message payload should be null.", head.record.hasNullValue)
   }
 
   @Test(expected = classOf[IllegalArgumentException])
@@ -970,9 +971,9 @@ class LogTest extends JUnitSuite {
       recoveryPoint = 0L,
       time.scheduler,
       time)
-    val messages = (0 until 2).map(id => new Message(id.toString.getBytes)).toArray
-    messages.foreach(message => log.append(new ByteBufferMessageSet(message)))
-    val invalidMessage = new ByteBufferMessageSet(new Message(1.toString.getBytes))
+    val messages = (0 until 2).map(id => Record.create(id.toString.getBytes)).toArray
+    messages.foreach(record => log.append(MemoryRecords.withRecords(record)))
+    val invalidMessage = MemoryRecords.withRecords(Record.create(1.toString.getBytes))
     log.append(invalidMessage, assignOffsets = false)
   }
 
@@ -984,7 +985,7 @@ class LogTest extends JUnitSuite {
     logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
     logProps.put(LogConfig.MaxMessageBytesProp, 64*1024: java.lang.Integer)
     val config = LogConfig(logProps)
-    val set = TestUtils.singleMessageSet("test".getBytes)
+    val set = TestUtils.singletonRecords("test".getBytes)
     val recoveryPoint = 50L
     for (_ <- 0 until 50) {
       // create a log and write some messages to it
@@ -997,7 +998,7 @@ class LogTest extends JUnitSuite {
       val numMessages = 50 + TestUtils.random.nextInt(50)
       for (_ <- 0 until numMessages)
         log.append(set)
-      val messages = log.logSegments.flatMap(_.log.iterator.toList)
+      val messages = log.logSegments.flatMap(_.log.deepIterator.asScala.toList)
       log.close()
 
       // corrupt index and log by appending random bytes
@@ -1007,7 +1008,8 @@ class LogTest extends JUnitSuite {
       // attempt recovery
       log = new Log(logDir, config, recoveryPoint, time.scheduler, time)
       assertEquals(numMessages, log.logEndOffset)
-      assertEquals("Messages in the log after recovery should be the same.", messages, log.logSegments.flatMap(_.log.iterator.toList))
+      assertEquals("Messages in the log after recovery should be the same.", messages,
+        log.logSegments.flatMap(_.log.deepIterator.asScala.toList))
       Utils.delete(logDir)
     }
   }
@@ -1020,7 +1022,7 @@ class LogTest extends JUnitSuite {
     logProps.put(LogConfig.MaxMessageBytesProp, 64*1024: java.lang.Integer)
     logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
     val config = LogConfig(logProps)
-    val set = TestUtils.singleMessageSet("test".getBytes)
+    val set = TestUtils.singletonRecords("test".getBytes)
     val parentLogDir = logDir.getParentFile
     assertTrue("Data directory %s must exist", parentLogDir.isDirectory)
     val cleanShutdownFile = new File(parentLogDir, Log.CleanShutdownFile)
@@ -1121,7 +1123,7 @@ class LogTest extends JUnitSuite {
 
   @Test
   def testDeleteOldSegmentsMethod() {
-    val set = TestUtils.singleMessageSet("test".getBytes)
+    val set = TestUtils.singletonRecords("test".getBytes)
     val logProps = new Properties()
     logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
     logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
@@ -1154,7 +1156,7 @@ class LogTest extends JUnitSuite {
 
   @Test
   def shouldDeleteSizeBasedSegments() {
-    val set = TestUtils.singleMessageSet("test".getBytes)
+    val set = TestUtils.singletonRecords("test".getBytes)
     val log = createLog(set.sizeInBytes, retentionBytes = set.sizeInBytes * 10)
 
     // append some messages to create some segments
@@ -1167,7 +1169,7 @@ class LogTest extends JUnitSuite {
 
   @Test
   def shouldNotDeleteSizeBasedSegmentsWhenUnderRetentionSize() {
-    val set = TestUtils.singleMessageSet("test".getBytes)
+    val set = TestUtils.singletonRecords("test".getBytes)
     val log = createLog(set.sizeInBytes, retentionBytes = set.sizeInBytes * 15)
 
     // append some messages to create some segments
@@ -1180,7 +1182,7 @@ class LogTest extends JUnitSuite {
 
   @Test
   def shouldDeleteTimeBasedSegmentsReadyToBeDeleted() {
-    val set = TestUtils.singleMessageSet("test".getBytes, timestamp = 10)
+    val set = TestUtils.singletonRecords("test".getBytes, timestamp = 10)
     val log = createLog(set.sizeInBytes, retentionMs = 10000)
 
     // append some messages to create some segments
@@ -1193,7 +1195,7 @@ class LogTest extends JUnitSuite {
 
   @Test
   def shouldNotDeleteTimeBasedSegmentsWhenNoneReadyToBeDeleted() {
-    val set = TestUtils.singleMessageSet("test".getBytes, timestamp = time.milliseconds)
+    val set = TestUtils.singletonRecords("test".getBytes, timestamp = time.milliseconds)
     val log = createLog(set.sizeInBytes, retentionMs = 10000000)
 
     // append some messages to create some segments
@@ -1206,7 +1208,7 @@ class LogTest extends JUnitSuite {
 
   @Test
   def shouldNotDeleteSegmentsWhenPolicyDoesNotIncludeDelete() {
-    val set = TestUtils.singleMessageSet("test".getBytes, key = "test".getBytes(), timestamp = 10L)
+    val set = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes(), timestamp = 10L)
     val log = createLog(set.sizeInBytes,
       retentionMs = 10000,
       cleanupPolicy = "compact")
@@ -1225,7 +1227,7 @@ class LogTest extends JUnitSuite {
 
   @Test
   def shouldDeleteSegmentsReadyToBeDeletedWhenCleanupPolicyIsCompactAndDelete() {
-    val set = TestUtils.singleMessageSet("test".getBytes, key = "test".getBytes,timestamp = 10L)
+    val set = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes,timestamp = 10L)
     val log = createLog(set.sizeInBytes,
       retentionMs = 10000,
       cleanupPolicy = "compact,delete")

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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
new file mode 100644
index 0000000..72c5b16
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala
@@ -0,0 +1,395 @@
+/**
+ * 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.nio.ByteBuffer
+
+import kafka.common.LongRef
+import kafka.message._
+import org.apache.kafka.common.errors.InvalidTimestampException
+import org.apache.kafka.common.record._
+import org.junit.Assert._
+import org.junit.Test
+import org.scalatest.junit.JUnitSuite
+
+import scala.collection.JavaConverters._
+
+class LogValidatorTest extends JUnitSuite {
+
+  @Test
+  def testLogAppendTimeNonCompressed() {
+    val now = System.currentTimeMillis()
+    // The timestamps should be overwritten
+    val records = createRecords(magicValue = Record.MAGIC_VALUE_V1, timestamp = 0L, codec = CompressionType.NONE)
+    val validatedResults = LogValidator.validateMessagesAndAssignOffsets(records,
+      offsetCounter = new LongRef(0),
+      now = now,
+      sourceCodec = NoCompressionCodec,
+      targetCodec = NoCompressionCodec,
+      messageFormatVersion = Record.MAGIC_VALUE_V1,
+      messageTimestampType = TimestampType.LOG_APPEND_TIME,
+      messageTimestampDiffMaxMs = 1000L)
+    val validatedRecords = validatedResults.validatedRecords
+    assertEquals("number of messages should not change", records.deepIterator.asScala.size, validatedRecords.deepIterator.asScala.size)
+    validatedRecords.deepIterator.asScala.foreach(logEntry => validateLogAppendTime(now, logEntry.record))
+    assertEquals(s"Max timestamp should be $now", now, validatedResults.maxTimestamp)
+    assertEquals(s"The offset of max timestamp should be 0", 0, validatedResults.shallowOffsetOfMaxTimestamp)
+    assertFalse("Message size should not have been changed", validatedResults.messageSizeMaybeChanged)
+  }
+
+  @Test
+  def testLogAppendTimeWithRecompression() {
+    val now = System.currentTimeMillis()
+    // The timestamps should be overwritten
+    val records = createRecords(magicValue = Record.MAGIC_VALUE_V0, codec = CompressionType.GZIP)
+    val validatedResults = LogValidator.validateMessagesAndAssignOffsets(
+      records,
+      offsetCounter = new LongRef(0),
+      now = now,
+      sourceCodec = DefaultCompressionCodec,
+      targetCodec = DefaultCompressionCodec,
+      messageFormatVersion = Record.MAGIC_VALUE_V1,
+      messageTimestampType = TimestampType.LOG_APPEND_TIME,
+      messageTimestampDiffMaxMs = 1000L)
+    val validatedRecords = validatedResults.validatedRecords
+
+    assertEquals("number of messages should not change", records.deepIterator.asScala.size, validatedRecords.deepIterator.asScala.size)
+    validatedRecords.deepIterator.asScala.foreach(logEntry => validateLogAppendTime(now, logEntry.record))
+    assertTrue("MessageSet should still valid", validatedRecords.shallowIterator.next().record.isValid)
+    assertEquals(s"Max timestamp should be $now", now, validatedResults.maxTimestamp)
+    assertEquals(s"The offset of max timestamp should be ${records.deepIterator.asScala.size - 1}",
+      records.deepIterator.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestamp)
+    assertTrue("Message size may have been changed", validatedResults.messageSizeMaybeChanged)
+  }
+
+  @Test
+  def testLogAppendTimeWithoutRecompression() {
+    val now = System.currentTimeMillis()
+    // The timestamps should be overwritten
+    val records = createRecords(magicValue = Record.MAGIC_VALUE_V1,
+      timestamp = 0L, codec = CompressionType.GZIP)
+    val validatedResults = LogValidator.validateMessagesAndAssignOffsets(
+      records,
+      offsetCounter = new LongRef(0),
+      now = now,
+      sourceCodec = DefaultCompressionCodec,
+      targetCodec = DefaultCompressionCodec,
+      messageFormatVersion = Record.MAGIC_VALUE_V1,
+      messageTimestampType = TimestampType.LOG_APPEND_TIME,
+      messageTimestampDiffMaxMs = 1000L)
+    val validatedRecords = validatedResults.validatedRecords
+
+    assertEquals("number of messages should not change", records.deepIterator.asScala.size,
+      validatedRecords.deepIterator.asScala.size)
+    validatedRecords.deepIterator.asScala.foreach(logEntry => validateLogAppendTime(now, logEntry.record))
+    assertTrue("MessageSet should still valid", validatedRecords.shallowIterator.next().record.isValid)
+    assertEquals(s"Max timestamp should be $now", now, validatedResults.maxTimestamp)
+    assertEquals(s"The offset of max timestamp should be ${records.deepIterator.asScala.size - 1}",
+      records.deepIterator.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestamp)
+    assertFalse("Message size should not have been changed", validatedResults.messageSizeMaybeChanged)
+  }
+
+  @Test
+  def testCreateTimeNonCompressed() {
+    val now = System.currentTimeMillis()
+    val timestampSeq = Seq(now - 1, now + 1, now)
+    val records =
+      MemoryRecords.withRecords(CompressionType.NONE,
+        Record.create(Record.MAGIC_VALUE_V1, timestampSeq(0), "hello".getBytes),
+        Record.create(Record.MAGIC_VALUE_V1, timestampSeq(1), "there".getBytes),
+        Record.create(Record.MAGIC_VALUE_V1, timestampSeq(2), "beautiful".getBytes))
+
+    val validatingResults = LogValidator.validateMessagesAndAssignOffsets(records,
+      offsetCounter = new LongRef(0),
+      now = System.currentTimeMillis(),
+      sourceCodec = NoCompressionCodec,
+      targetCodec = NoCompressionCodec,
+      messageFormatVersion = Record.MAGIC_VALUE_V1,
+      messageTimestampType = TimestampType.CREATE_TIME,
+      messageTimestampDiffMaxMs = 1000L)
+    val validatedRecords = validatingResults.validatedRecords
+
+    var i = 0
+    for (logEntry <- validatedRecords.deepIterator.asScala) {
+      logEntry.record.ensureValid()
+      assertEquals(logEntry.record.timestamp, timestampSeq(i))
+      assertEquals(logEntry.record.timestampType, TimestampType.CREATE_TIME)
+      i += 1
+    }
+    assertEquals(s"Max timestamp should be ${now + 1}", now + 1, validatingResults.maxTimestamp)
+    assertEquals(s"Offset of max timestamp should be 1", 1, validatingResults.shallowOffsetOfMaxTimestamp)
+    assertFalse("Message size should not have been changed", validatingResults.messageSizeMaybeChanged)
+  }
+
+  @Test
+  def testCreateTimeCompressed() {
+    val now = System.currentTimeMillis()
+    val timestampSeq = Seq(now - 1, now + 1, now)
+    val records =
+      MemoryRecords.withRecords(CompressionType.GZIP,
+        Record.create(Record.MAGIC_VALUE_V1, timestampSeq(0), "hello".getBytes),
+        Record.create(Record.MAGIC_VALUE_V1, timestampSeq(1), "there".getBytes),
+        Record.create(Record.MAGIC_VALUE_V1, timestampSeq(2), "beautiful".getBytes))
+
+    val validatedResults =
+      LogValidator.validateMessagesAndAssignOffsets(records,
+        offsetCounter = new LongRef(0),
+        now = System.currentTimeMillis(),
+        sourceCodec = DefaultCompressionCodec,
+        targetCodec = DefaultCompressionCodec,
+        messageFormatVersion = Record.MAGIC_VALUE_V1,
+        messageTimestampType = TimestampType.CREATE_TIME,
+        messageTimestampDiffMaxMs = 1000L)
+    val validatedRecords = validatedResults.validatedRecords
+
+    var i = 0
+    for (logEntry <- validatedRecords.deepIterator.asScala) {
+      logEntry.record.ensureValid()
+      assertEquals(logEntry.record.timestamp, timestampSeq(i))
+      assertEquals(logEntry.record.timestampType, TimestampType.CREATE_TIME)
+      i += 1
+    }
+    assertEquals(s"Max timestamp should be ${now + 1}", now + 1, validatedResults.maxTimestamp)
+    assertEquals(s"Offset of max timestamp should be ${validatedRecords.deepIterator.asScala.size - 1}",
+      validatedRecords.deepIterator.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestamp)
+    assertFalse("Message size should not have been changed", validatedResults.messageSizeMaybeChanged)
+  }
+
+  @Test(expected = classOf[InvalidTimestampException])
+  def testInvalidCreateTimeNonCompressed() {
+    val now = System.currentTimeMillis()
+    val records = createRecords(magicValue = Record.MAGIC_VALUE_V1, timestamp = now - 1001L,
+      codec = CompressionType.NONE)
+    LogValidator.validateMessagesAndAssignOffsets(
+      records,
+      offsetCounter = new LongRef(0),
+      now = System.currentTimeMillis(),
+      sourceCodec = NoCompressionCodec,
+      targetCodec = NoCompressionCodec,
+      messageFormatVersion = Record.MAGIC_VALUE_V1,
+      messageTimestampType = TimestampType.CREATE_TIME,
+      messageTimestampDiffMaxMs = 1000L)
+  }
+
+  @Test(expected = classOf[InvalidTimestampException])
+  def testInvalidCreateTimeCompressed() {
+    val now = System.currentTimeMillis()
+    val records = createRecords(magicValue = Record.MAGIC_VALUE_V1, timestamp = now - 1001L,
+      codec = CompressionType.GZIP)
+    LogValidator.validateMessagesAndAssignOffsets(
+      records,
+      offsetCounter = new LongRef(0),
+      now = System.currentTimeMillis(),
+      sourceCodec = DefaultCompressionCodec,
+      targetCodec = DefaultCompressionCodec,
+      messageFormatVersion = Record.MAGIC_VALUE_V1,
+      messageTimestampType = TimestampType.CREATE_TIME,
+      messageTimestampDiffMaxMs = 1000L)
+  }
+  @Test
+  def testAbsoluteOffsetAssignmentNonCompressed() {
+    val records = createRecords(magicValue = Record.MAGIC_VALUE_V0, codec = CompressionType.NONE)
+    val offset = 1234567
+    checkOffsets(records, 0)
+    checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records,
+      offsetCounter = new LongRef(offset),
+      now = System.currentTimeMillis(),
+      sourceCodec = NoCompressionCodec,
+      targetCodec = NoCompressionCodec,
+      messageFormatVersion = Record.MAGIC_VALUE_V0,
+      messageTimestampType = TimestampType.CREATE_TIME,
+      messageTimestampDiffMaxMs = 1000L).validatedRecords, offset)
+  }
+
+  @Test
+  def testAbsoluteOffsetAssignmentCompressed() {
+    val records = createRecords(magicValue = Record.MAGIC_VALUE_V0, codec = CompressionType.GZIP)
+    val offset = 1234567
+    checkOffsets(records, 0)
+    checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records,
+      offsetCounter = new LongRef(offset),
+      now = System.currentTimeMillis(),
+      sourceCodec = DefaultCompressionCodec,
+      targetCodec = DefaultCompressionCodec,
+      messageFormatVersion = Record.MAGIC_VALUE_V0,
+      messageTimestampType = TimestampType.CREATE_TIME,
+      messageTimestampDiffMaxMs = 1000L).validatedRecords, offset)
+  }
+
+  @Test
+  def testRelativeOffsetAssignmentNonCompressed() {
+    val now = System.currentTimeMillis()
+    val records = createRecords(magicValue = Record.MAGIC_VALUE_V1, timestamp = now, codec = CompressionType.NONE)
+    val offset = 1234567
+    checkOffsets(records, 0)
+    val messageWithOffset = LogValidator.validateMessagesAndAssignOffsets(records,
+      offsetCounter = new LongRef(offset),
+      now = System.currentTimeMillis(),
+      sourceCodec = NoCompressionCodec,
+      targetCodec = NoCompressionCodec,
+      messageTimestampType = TimestampType.CREATE_TIME,
+      messageTimestampDiffMaxMs = 5000L).validatedRecords
+    checkOffsets(messageWithOffset, offset)
+  }
+
+  @Test
+  def testRelativeOffsetAssignmentCompressed() {
+    val now = System.currentTimeMillis()
+    val records = createRecords(magicValue = Record.MAGIC_VALUE_V1, timestamp = now, codec = CompressionType.GZIP)
+    val offset = 1234567
+    checkOffsets(records, 0)
+    val compressedMessagesWithOffset = LogValidator.validateMessagesAndAssignOffsets(
+      records,
+      offsetCounter = new LongRef(offset),
+      now = System.currentTimeMillis(),
+      sourceCodec = DefaultCompressionCodec,
+      targetCodec = DefaultCompressionCodec,
+      messageTimestampType = TimestampType.CREATE_TIME,
+      messageTimestampDiffMaxMs = 5000L).validatedRecords
+    checkOffsets(compressedMessagesWithOffset, offset)
+  }
+
+  @Test
+  def testOffsetAssignmentAfterMessageFormatConversionV0NonCompressed() {
+    val records = createRecords(magicValue = Record.MAGIC_VALUE_V0, codec = CompressionType.NONE)
+    checkOffsets(records, 0)
+    val offset = 1234567
+    checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records,
+      offsetCounter = new LongRef(offset),
+      now = System.currentTimeMillis(),
+      sourceCodec = NoCompressionCodec,
+      targetCodec = NoCompressionCodec,
+      messageFormatVersion = Record.MAGIC_VALUE_V1,
+      messageTimestampType = TimestampType.LOG_APPEND_TIME,
+      messageTimestampDiffMaxMs = 1000L).validatedRecords, offset)
+  }
+
+  @Test
+  def testOffsetAssignmentAfterMessageFormatConversionV0Compressed() {
+    val records = createRecords(magicValue = Record.MAGIC_VALUE_V0, codec = CompressionType.GZIP)
+    val offset = 1234567
+    checkOffsets(records, 0)
+    checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records,
+      offsetCounter = new LongRef(offset),
+      now = System.currentTimeMillis(),
+      sourceCodec = DefaultCompressionCodec,
+      targetCodec = DefaultCompressionCodec,
+      messageFormatVersion = Record.MAGIC_VALUE_V1,
+      messageTimestampType = TimestampType.LOG_APPEND_TIME,
+      messageTimestampDiffMaxMs = 1000L).validatedRecords, offset)
+  }
+
+  @Test
+  def testOffsetAssignmentAfterMessageFormatConversionV1NonCompressed() {
+    val offset = 1234567
+    val now = System.currentTimeMillis()
+    val records = createRecords(Record.MAGIC_VALUE_V1, now, codec = CompressionType.NONE)
+    checkOffsets(records, 0)
+    checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records,
+      offsetCounter = new LongRef(offset),
+      now = System.currentTimeMillis(),
+      sourceCodec = NoCompressionCodec,
+      targetCodec = NoCompressionCodec,
+      messageFormatVersion = Record.MAGIC_VALUE_V0,
+      messageTimestampType = TimestampType.CREATE_TIME,
+      messageTimestampDiffMaxMs = 5000L).validatedRecords, offset)
+  }
+
+  @Test
+  def testOffsetAssignmentAfterMessageFormatConversionV1Compressed() {
+    val offset = 1234567
+    val now = System.currentTimeMillis()
+    val records = createRecords(Record.MAGIC_VALUE_V1, now, CompressionType.GZIP)
+    checkOffsets(records, 0)
+    checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records,
+      offsetCounter = new LongRef(offset),
+      now = System.currentTimeMillis(),
+      sourceCodec = DefaultCompressionCodec,
+      targetCodec = DefaultCompressionCodec,
+      messageFormatVersion = Record.MAGIC_VALUE_V0,
+      messageTimestampType = TimestampType.CREATE_TIME,
+      messageTimestampDiffMaxMs = 5000L).validatedRecords, offset)
+  }
+
+  @Test(expected = classOf[InvalidRecordException])
+  def testInvalidInnerMagicVersion(): Unit = {
+    val offset = 1234567
+    val records = recordsWithInvalidInnerMagic(offset)
+    LogValidator.validateMessagesAndAssignOffsets(records,
+      offsetCounter = new LongRef(offset),
+      now = System.currentTimeMillis(),
+      sourceCodec = SnappyCompressionCodec,
+      targetCodec = SnappyCompressionCodec,
+      messageTimestampType = TimestampType.CREATE_TIME,
+      messageTimestampDiffMaxMs = 5000L)
+  }
+
+  private def createRecords(magicValue: Byte = Message.CurrentMagicValue,
+                              timestamp: Long = Message.NoTimestamp,
+                              codec: CompressionType = CompressionType.NONE): MemoryRecords = {
+    if (magicValue == Record.MAGIC_VALUE_V0) {
+      MemoryRecords.withRecords(
+        codec,
+        Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, "hello".getBytes),
+        Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, "there".getBytes),
+        Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, "beautiful".getBytes))
+    } else {
+      MemoryRecords.withRecords(
+        codec,
+        Record.create(Record.MAGIC_VALUE_V1, timestamp, "hello".getBytes),
+        Record.create(Record.MAGIC_VALUE_V1, timestamp, "there".getBytes),
+        Record.create(Record.MAGIC_VALUE_V1, timestamp, "beautiful".getBytes))
+    }
+  }
+
+  /* check that offsets are assigned consecutively from the given base offset */
+  private def checkOffsets(records: MemoryRecords, baseOffset: Long) {
+    assertTrue("Message set should not be empty", records.deepIterator.asScala.nonEmpty)
+    var offset = baseOffset
+    for (entry <- records.deepIterator.asScala) {
+      assertEquals("Unexpected offset in message set iterator", offset, entry.offset)
+      offset += 1
+    }
+  }
+
+  private def recordsWithInvalidInnerMagic(initialOffset: Long): MemoryRecords = {
+    val records = (0 until 20).map(id =>
+      Record.create(Record.MAGIC_VALUE_V0,
+        Record.NO_TIMESTAMP,
+        id.toString.getBytes,
+        id.toString.getBytes))
+
+    val buffer = ByteBuffer.allocate(math.min(math.max(records.map(_.sizeInBytes()).sum / 2, 1024), 1 << 16))
+    val builder = MemoryRecords.builder(buffer, Record.MAGIC_VALUE_V1, CompressionType.GZIP,
+      TimestampType.CREATE_TIME)
+
+    var offset = initialOffset
+    records.foreach { record =>
+      builder.appendUnchecked(offset, record)
+      offset += 1
+    }
+
+    builder.build()
+  }
+
+  def validateLogAppendTime(now: Long, record: Record) {
+    record.ensureValid()
+    assertEquals(s"Timestamp of message $record should be $now", now, record.timestamp)
+    assertEquals(TimestampType.LOG_APPEND_TIME, record.timestampType)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala b/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala
index 476a577..bd3ed68 100644
--- a/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala
+++ b/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala
@@ -18,13 +18,11 @@
 package kafka.message
 
 import java.nio.ByteBuffer
-import java.nio.channels.{FileChannel, GatheringByteChannel}
-import java.nio.file.StandardOpenOption
+import java.nio.channels.GatheringByteChannel
 
 import org.junit.Assert._
 import kafka.utils.TestUtils._
-import kafka.log.FileMessageSet
-import kafka.utils.TestUtils
+import org.apache.kafka.common.record.FileRecords
 import org.scalatest.junit.JUnitSuite
 import org.junit.Test
 
@@ -94,7 +92,7 @@ trait BaseMessageSetTestCases extends JUnitSuite {
   @Test
   def testWriteToChannelThatConsumesPartially() {
     val bytesToConsumePerBuffer = 50
-    val messages = (0 until 10).map(_ => new Message(TestUtils.randomString(100).getBytes))
+    val messages = (0 until 10).map(_ => new Message(randomString(100).getBytes))
     val messageSet = createMessageSet(messages)
     val messageSetSize = messageSet.sizeInBytes
 
@@ -119,15 +117,15 @@ trait BaseMessageSetTestCases extends JUnitSuite {
     // do the write twice to ensure the message set is restored to its original state
     for (_ <- 0 to 1) {
       val file = tempFile()
-      val channel = FileChannel.open(file.toPath, StandardOpenOption.READ, StandardOpenOption.WRITE)
+      val fileRecords = FileRecords.open(file, true)
       try {
-        val written = write(channel)
+        val written = write(fileRecords.channel)
+        fileRecords.resize() // resize since we wrote to the channel directly
+
         assertEquals("Expect to write the number of bytes in the set.", set.sizeInBytes, written)
-        val newSet = new FileMessageSet(file, channel)
-        checkEquals(set.iterator, newSet.iterator)
-      } finally channel.close()
+        checkEquals(set.asRecords.deepIterator, fileRecords.deepIterator())
+      } finally fileRecords.close()
     }
   }
   
 }
-

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala
index e2cfb87..5e22433 100644
--- a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala
+++ b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala
@@ -17,13 +17,9 @@
 
 package kafka.message
 
-import java.io.DataOutputStream
 import java.nio._
 
-import kafka.common.LongRef
 import kafka.utils.TestUtils
-import org.apache.kafka.common.errors.InvalidTimestampException
-import org.apache.kafka.common.record.TimestampType
 import org.junit.Assert._
 import org.junit.Test
 
@@ -151,295 +147,6 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases {
     assertEquals("second offset should be 2", 2L, iter.next().offset)
   }
 
-  @Test
-  def testLogAppendTime() {
-    val now = System.currentTimeMillis()
-    // The timestamps should be overwritten
-    val messages = getMessages(magicValue = Message.MagicValue_V1, timestamp = 0L, codec = NoCompressionCodec)
-    val compressedMessagesWithRecompresion = getMessages(magicValue = Message.MagicValue_V0, codec = DefaultCompressionCodec)
-    val compressedMessagesWithoutRecompression =
-      getMessages(magicValue = Message.MagicValue_V1, timestamp = 0L, codec = DefaultCompressionCodec)
-
-    val validatingResults = messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
-                                                                      now = now,
-                                                                      sourceCodec = NoCompressionCodec,
-                                                                      targetCodec = NoCompressionCodec,
-                                                                      messageFormatVersion = 1,
-                                                                      messageTimestampType = TimestampType.LOG_APPEND_TIME,
-                                                                      messageTimestampDiffMaxMs = 1000L)
-    val validatedMessages = validatingResults.validatedMessages
-
-    val validatingCompressedMessagesResults =
-      compressedMessagesWithRecompresion.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
-                                                                          now = now,
-                                                                          sourceCodec = DefaultCompressionCodec,
-                                                                          targetCodec = DefaultCompressionCodec,
-                                                                          messageFormatVersion = 1,
-                                                                          messageTimestampType = TimestampType.LOG_APPEND_TIME,
-                                                                          messageTimestampDiffMaxMs = 1000L)
-    val validatedCompressedMessages = validatingCompressedMessagesResults.validatedMessages
-
-    val validatingCompressedMessagesWithoutRecompressionResults =
-      compressedMessagesWithoutRecompression.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
-                                                                              now = now,
-                                                                              sourceCodec = DefaultCompressionCodec,
-                                                                              targetCodec = DefaultCompressionCodec,
-                                                                              messageFormatVersion = 1,
-                                                                              messageTimestampType = TimestampType.LOG_APPEND_TIME,
-                                                                              messageTimestampDiffMaxMs = 1000L)
-
-    val validatedCompressedMessagesWithoutRecompression = validatingCompressedMessagesWithoutRecompressionResults.validatedMessages
-
-    assertEquals("message set size should not change", messages.size, validatedMessages.size)
-    validatedMessages.foreach(messageAndOffset => validateLogAppendTime(messageAndOffset.message))
-    assertEquals(s"Max timestamp should be $now", now, validatingResults.maxTimestamp)
-    assertEquals(s"The offset of max timestamp should be 0", 0, validatingResults.offsetOfMaxTimestamp)
-    assertFalse("Message size should not have been changed", validatingResults.messageSizeMaybeChanged)
-
-    assertEquals("message set size should not change", compressedMessagesWithRecompresion.size, validatedCompressedMessages.size)
-    validatedCompressedMessages.foreach(messageAndOffset => validateLogAppendTime(messageAndOffset.message))
-    assertTrue("MessageSet should still valid", validatedCompressedMessages.shallowIterator.next().message.isValid)
-    assertEquals(s"Max timestamp should be $now", now, validatingCompressedMessagesResults.maxTimestamp)
-    assertEquals(s"The offset of max timestamp should be ${compressedMessagesWithRecompresion.size - 1}",
-      compressedMessagesWithRecompresion.size - 1, validatingCompressedMessagesResults.offsetOfMaxTimestamp)
-    assertTrue("Message size may have been changed", validatingCompressedMessagesResults.messageSizeMaybeChanged)
-
-    assertEquals("message set size should not change", compressedMessagesWithoutRecompression.size,
-      validatedCompressedMessagesWithoutRecompression.size)
-    validatedCompressedMessagesWithoutRecompression.foreach(messageAndOffset => validateLogAppendTime(messageAndOffset.message))
-    assertTrue("MessageSet should still valid", validatedCompressedMessagesWithoutRecompression.shallowIterator.next().message.isValid)
-    assertEquals(s"Max timestamp should be $now", now, validatingCompressedMessagesWithoutRecompressionResults.maxTimestamp)
-    assertEquals(s"The offset of max timestamp should be ${compressedMessagesWithoutRecompression.size - 1}",
-      compressedMessagesWithoutRecompression.size - 1, validatingCompressedMessagesWithoutRecompressionResults.offsetOfMaxTimestamp)
-    assertFalse("Message size should not have been changed", validatingCompressedMessagesWithoutRecompressionResults.messageSizeMaybeChanged)
-
-    def validateLogAppendTime(message: Message) {
-      message.ensureValid()
-      assertEquals(s"Timestamp of message $message should be $now", now, message.timestamp)
-      assertEquals(TimestampType.LOG_APPEND_TIME, message.timestampType)
-    }
-  }
-
-  @Test
-  def testCreateTime() {
-    val now = System.currentTimeMillis()
-    val timestampSeq = Seq(now - 1, now + 1, now)
-    val messages =
-      new ByteBufferMessageSet(NoCompressionCodec,
-                               new Message("hello".getBytes, timestamp = timestampSeq(0), magicValue = Message.MagicValue_V1),
-                               new Message("there".getBytes, timestamp = timestampSeq(1), magicValue = Message.MagicValue_V1),
-                               new Message("beautiful".getBytes, timestamp = timestampSeq(2), magicValue = Message.MagicValue_V1))
-    val compressedMessages =
-      new ByteBufferMessageSet(DefaultCompressionCodec,
-                               new Message("hello".getBytes, timestamp = timestampSeq(0), magicValue = Message.MagicValue_V1),
-                               new Message("there".getBytes, timestamp = timestampSeq(1), magicValue = Message.MagicValue_V1),
-                               new Message("beautiful".getBytes, timestamp = timestampSeq(2), magicValue = Message.MagicValue_V1))
-
-    val validatingResults = messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
-                                                                      now = System.currentTimeMillis(),
-                                                                      sourceCodec = NoCompressionCodec,
-                                                                      targetCodec = NoCompressionCodec,
-                                                                      messageFormatVersion = 1,
-                                                                      messageTimestampType = TimestampType.CREATE_TIME,
-                                                                      messageTimestampDiffMaxMs = 1000L)
-    val validatedMessages = validatingResults.validatedMessages
-
-    val validatingCompressedMessagesResults =
-      compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
-                                                          now = System.currentTimeMillis(),
-                                                          sourceCodec = DefaultCompressionCodec,
-                                                          targetCodec = DefaultCompressionCodec,
-                                                          messageFormatVersion = 1,
-                                                          messageTimestampType = TimestampType.CREATE_TIME,
-                                                          messageTimestampDiffMaxMs = 1000L)
-    val validatedCompressedMessages = validatingCompressedMessagesResults.validatedMessages
-
-    var i = 0
-    for (messageAndOffset <- validatedMessages) {
-      messageAndOffset.message.ensureValid()
-      assertEquals(messageAndOffset.message.timestamp, timestampSeq(i))
-      assertEquals(messageAndOffset.message.timestampType, TimestampType.CREATE_TIME)
-      i += 1
-    }
-    assertEquals(s"Max timestamp should be ${now + 1}", now + 1, validatingResults.maxTimestamp)
-    assertEquals(s"Offset of max timestamp should be 1", 1, validatingResults.offsetOfMaxTimestamp)
-    assertFalse("Message size should not have been changed", validatingResults.messageSizeMaybeChanged)
-    i = 0
-    for (messageAndOffset <- validatedCompressedMessages) {
-      messageAndOffset.message.ensureValid()
-      assertEquals(messageAndOffset.message.timestamp, timestampSeq(i))
-      assertEquals(messageAndOffset.message.timestampType, TimestampType.CREATE_TIME)
-      i += 1
-    }
-    assertEquals(s"Max timestamp should be ${now + 1}", now + 1, validatingResults.maxTimestamp)
-    assertEquals(s"Offset of max timestamp should be ${validatedCompressedMessages.size - 1}",
-      validatedCompressedMessages.size - 1, validatingCompressedMessagesResults.offsetOfMaxTimestamp)
-    assertFalse("Message size should not have been changed", validatingCompressedMessagesResults.messageSizeMaybeChanged)
-  }
-
-  @Test
-  def testInvalidCreateTime() {
-    val now = System.currentTimeMillis()
-    val messages = getMessages(magicValue = Message.MagicValue_V1, timestamp = now - 1001L, codec = NoCompressionCodec)
-    val compressedMessages = getMessages(magicValue = Message.MagicValue_V1, timestamp = now - 1001L, codec = DefaultCompressionCodec)
-
-    try {
-      messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
-                                                now = System.currentTimeMillis(),
-                                                sourceCodec = NoCompressionCodec,
-                                                targetCodec = NoCompressionCodec,
-                                                messageFormatVersion = 1,
-                                                messageTimestampType = TimestampType.CREATE_TIME,
-                                                messageTimestampDiffMaxMs = 1000L)
-      fail("Should throw InvalidMessageException.")
-    } catch {
-      case _: InvalidTimestampException =>
-    }
-
-    try {
-      compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
-                                                          now = System.currentTimeMillis(),
-                                                          sourceCodec = DefaultCompressionCodec,
-                                                          targetCodec = DefaultCompressionCodec,
-                                                          messageFormatVersion = 1,
-                                                          messageTimestampType = TimestampType.CREATE_TIME,
-                                                          messageTimestampDiffMaxMs = 1000L)
-      fail("Should throw InvalidMessageException.")
-    } catch {
-      case _: InvalidTimestampException =>
-    }
-  }
-
-  @Test
-  def testAbsoluteOffsetAssignment() {
-    val messages = getMessages(magicValue = Message.MagicValue_V0, codec = NoCompressionCodec)
-    val compressedMessages = getMessages(magicValue = Message.MagicValue_V0, codec = DefaultCompressionCodec)
-    // check uncompressed offsets
-    checkOffsets(messages, 0)
-    val offset = 1234567
-    checkOffsets(messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
-                                                           now = System.currentTimeMillis(),
-                                                           sourceCodec = NoCompressionCodec,
-                                                           targetCodec = NoCompressionCodec,
-                                                           messageFormatVersion = 0,
-                                                           messageTimestampType = TimestampType.CREATE_TIME,
-                                                           messageTimestampDiffMaxMs = 1000L).validatedMessages, offset)
-
-    // check compressed messages
-    checkOffsets(compressedMessages, 0)
-    checkOffsets(compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
-                                                                     now = System.currentTimeMillis(),
-                                                                     sourceCodec = DefaultCompressionCodec,
-                                                                     targetCodec = DefaultCompressionCodec,
-                                                                     messageFormatVersion = 0,
-                                                                     messageTimestampType = TimestampType.CREATE_TIME,
-                                                                     messageTimestampDiffMaxMs = 1000L).validatedMessages, offset)
-
-  }
-
-  @Test
-  def testRelativeOffsetAssignment() {
-    val now = System.currentTimeMillis()
-    val messages = getMessages(magicValue = Message.MagicValue_V1, timestamp = now, codec = NoCompressionCodec)
-    val compressedMessages = getMessages(magicValue = Message.MagicValue_V1, timestamp = now, codec = DefaultCompressionCodec)
-
-    // check uncompressed offsets
-    checkOffsets(messages, 0)
-    val offset = 1234567
-    val messageWithOffset = messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
-                                                                      now = System.currentTimeMillis(),
-                                                                      sourceCodec = NoCompressionCodec,
-                                                                      targetCodec = NoCompressionCodec,
-                                                                      messageTimestampType = TimestampType.CREATE_TIME,
-                                                                      messageTimestampDiffMaxMs = 5000L).validatedMessages
-    checkOffsets(messageWithOffset, offset)
-
-    // check compressed messages
-    checkOffsets(compressedMessages, 0)
-    val compressedMessagesWithOffset = compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
-                                                                                           now = System.currentTimeMillis(),
-                                                                                           sourceCodec = DefaultCompressionCodec,
-                                                                                           targetCodec = DefaultCompressionCodec,
-                                                                                           messageTimestampType = TimestampType.CREATE_TIME,
-                                                                                           messageTimestampDiffMaxMs = 5000L).validatedMessages
-    checkOffsets(compressedMessagesWithOffset, offset)
-  }
-
-  @Test(expected = classOf[InvalidMessageException])
-  def testInvalidInnerMagicVersion(): Unit = {
-    val offset = 1234567
-    val messages = messageSetWithInvalidInnerMagic(SnappyCompressionCodec, offset)
-    messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
-      now = System.currentTimeMillis(),
-      sourceCodec = SnappyCompressionCodec,
-      targetCodec = SnappyCompressionCodec,
-      messageTimestampType = TimestampType.CREATE_TIME,
-      messageTimestampDiffMaxMs = 5000L).validatedMessages
-  }
-
-
-  @Test
-  def testOffsetAssignmentAfterMessageFormatConversion() {
-    // Check up conversion
-    val messagesV0 = getMessages(magicValue = Message.MagicValue_V0, codec = NoCompressionCodec)
-    val compressedMessagesV0 = getMessages(magicValue = Message.MagicValue_V0, codec = DefaultCompressionCodec)
-    // check uncompressed offsets
-    checkOffsets(messagesV0, 0)
-    val offset = 1234567
-    checkOffsets(messagesV0.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
-                                                             now = System.currentTimeMillis(),
-                                                             sourceCodec = NoCompressionCodec,
-                                                             targetCodec = NoCompressionCodec,
-                                                             messageFormatVersion = 1,
-                                                             messageTimestampType = TimestampType.LOG_APPEND_TIME,
-                                                             messageTimestampDiffMaxMs = 1000L).validatedMessages, offset)
-
-    // check compressed messages
-    checkOffsets(compressedMessagesV0, 0)
-    checkOffsets(compressedMessagesV0.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
-                                                                       now = System.currentTimeMillis(),
-                                                                       sourceCodec = DefaultCompressionCodec,
-                                                                       targetCodec = DefaultCompressionCodec,
-                                                                       messageFormatVersion = 1,
-                                                                       messageTimestampType = TimestampType.LOG_APPEND_TIME,
-                                                                       messageTimestampDiffMaxMs = 1000L).validatedMessages, offset)
-
-    // Check down conversion
-    val now = System.currentTimeMillis()
-    val messagesV1 = getMessages(Message.MagicValue_V1, now, NoCompressionCodec)
-    val compressedMessagesV1 = getMessages(Message.MagicValue_V1, now, DefaultCompressionCodec)
-
-    // check uncompressed offsets
-    checkOffsets(messagesV1, 0)
-    checkOffsets(messagesV1.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
-                                                             now = System.currentTimeMillis(),
-                                                             sourceCodec = NoCompressionCodec,
-                                                             targetCodec = NoCompressionCodec,
-                                                             messageFormatVersion = 0,
-                                                             messageTimestampType = TimestampType.CREATE_TIME,
-                                                             messageTimestampDiffMaxMs = 5000L).validatedMessages, offset)
-
-    // check compressed messages
-    checkOffsets(compressedMessagesV1, 0)
-    checkOffsets(compressedMessagesV1.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
-                                                                       now = System.currentTimeMillis(),
-                                                                       sourceCodec = DefaultCompressionCodec,
-                                                                       targetCodec = DefaultCompressionCodec,
-                                                                       messageFormatVersion = 0,
-                                                                       messageTimestampType = TimestampType.CREATE_TIME,
-                                                                       messageTimestampDiffMaxMs = 5000L).validatedMessages, offset)
-  }
-
-  @Test
-  def testWriteFullyTo() {
-    checkWriteFullyToWithMessageSet(createMessageSet(Array[Message]()))
-    checkWriteFullyToWithMessageSet(createMessageSet(messages))
-  }
-
-  def checkWriteFullyToWithMessageSet(messageSet: ByteBufferMessageSet) {
-    checkWriteWithMessageSet(messageSet, messageSet.writeFullyTo)
-  }
-  
   /* check that offsets are assigned based on byte offset from the given base offset */
   def checkOffsets(messages: ByteBufferMessageSet, baseOffset: Long) {
     assertTrue("Message set should not be empty", messages.nonEmpty)
@@ -457,59 +164,4 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases {
     assertTrue(shallowOffsets.subsetOf(deepOffsets))
   }
 
-  private def getMessages(magicValue: Byte = Message.CurrentMagicValue,
-                          timestamp: Long = Message.NoTimestamp,
-                          codec: CompressionCodec = NoCompressionCodec): ByteBufferMessageSet = {
-    if (magicValue == Message.MagicValue_V0) {
-      new ByteBufferMessageSet(
-        codec,
-        new Message("hello".getBytes, Message.NoTimestamp, Message.MagicValue_V0),
-        new Message("there".getBytes, Message.NoTimestamp, Message.MagicValue_V0),
-        new Message("beautiful".getBytes, Message.NoTimestamp, Message.MagicValue_V0))
-    } else {
-      new ByteBufferMessageSet(
-        codec,
-        new Message("hello".getBytes, timestamp = timestamp, magicValue = Message.MagicValue_V1),
-        new Message("there".getBytes, timestamp = timestamp, magicValue = Message.MagicValue_V1),
-        new Message("beautiful".getBytes, timestamp = timestamp, magicValue = Message.MagicValue_V1))
-    }
-  }
-
-  private def messageSetWithInvalidInnerMagic(codec: CompressionCodec,
-                                              initialOffset: Long): ByteBufferMessageSet = {
-    val messages = (0 until 20).map(id =>
-      new Message(key = id.toString.getBytes,
-        bytes = id.toString.getBytes,
-        timestamp = Message.NoTimestamp,
-        magicValue = Message.MagicValue_V0))
-
-    val messageWriter = new MessageWriter(math.min(math.max(MessageSet.messageSetSize(messages) / 2, 1024), 1 << 16))
-    var lastOffset = initialOffset
-
-    messageWriter.write(
-      codec = codec,
-      timestamp = System.currentTimeMillis(),
-      timestampType = TimestampType.CREATE_TIME,
-      magicValue = Message.MagicValue_V1) { outputStream =>
-
-      val output = new DataOutputStream(CompressionFactory(codec, Message.MagicValue_V1, outputStream))
-      try {
-        for (message <- messages) {
-          val innerOffset = lastOffset - initialOffset
-          output.writeLong(innerOffset)
-          output.writeInt(message.size)
-          output.write(message.buffer.array, message.buffer.arrayOffset, message.buffer.limit)
-          lastOffset += 1
-        }
-      } finally {
-        output.close()
-      }
-    }
-    val buffer = ByteBuffer.allocate(messageWriter.size + MessageSet.LogOverhead)
-    ByteBufferMessageSet.writeMessage(buffer, messageWriter, lastOffset - 1)
-    buffer.rewind()
-
-    new ByteBufferMessageSet(buffer)
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala b/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala
index e8abfe1..5d2c8fb 100644
--- a/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala
+++ b/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala
@@ -70,7 +70,7 @@ class MessageCompressionTest extends JUnitSuite {
     testCompressSize(GZIPCompressionCodec, messages, 396)
 
     if(isSnappyAvailable)
-      testCompressSize(SnappyCompressionCodec, messages, 502)
+      testCompressSize(SnappyCompressionCodec, messages, 1063)
 
     if(isLZ4Available)
       testCompressSize(LZ4CompressionCodec, messages, 387)

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/message/MessageTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/message/MessageTest.scala b/core/src/test/scala/unit/kafka/message/MessageTest.scala
index 5c02125..46c25af 100755
--- a/core/src/test/scala/unit/kafka/message/MessageTest.scala
+++ b/core/src/test/scala/unit/kafka/message/MessageTest.scala
@@ -48,7 +48,7 @@ class MessageTest extends JUnitSuite {
     val magicValues = Array(Message.MagicValue_V0, Message.MagicValue_V1)
     for(k <- keys; v <- vals; codec <- codecs; t <- timestamps; mv <- magicValues) {
       val timestamp = ensureValid(mv, t)
-      messages += new MessageTestVal(k, v, codec, timestamp, mv, new Message(v, k, timestamp, codec, mv))
+      messages += MessageTestVal(k, v, codec, timestamp, mv, new Message(v, k, timestamp, codec, mv))
     }
 
     def ensureValid(magicValue: Byte, timestamp: Long): Long =
@@ -96,7 +96,7 @@ class MessageTest extends JUnitSuite {
 
   @Test
   def testEquality() {
-    for(v <- messages) {
+    for (v <- messages) {
       assertFalse("Should not equal null", v.message.equals(null))
       assertFalse("Should not equal a random string", v.message.equals("asdf"))
       assertTrue("Should equal itself", v.message.equals(v.message))
@@ -105,40 +105,6 @@ class MessageTest extends JUnitSuite {
     }
   }
 
-  @Test
-  def testMessageFormatConversion() {
-
-    def convertAndVerify(v: MessageTestVal, fromMessageFormat: Byte, toMessageFormat: Byte) {
-      assertEquals("Message should be the same when convert to the same version.",
-        v.message.toFormatVersion(fromMessageFormat), v.message)
-      val convertedMessage = v.message.toFormatVersion(toMessageFormat)
-      assertEquals("Size difference is not expected value", convertedMessage.size - v.message.size,
-        Message.headerSizeDiff(fromMessageFormat, toMessageFormat))
-      assertTrue("Message should still be valid", convertedMessage.isValid)
-      assertEquals("Timestamp should be NoTimestamp", convertedMessage.timestamp, Message.NoTimestamp)
-      assertEquals(s"Magic value should be $toMessageFormat now", convertedMessage.magic, toMessageFormat)
-      if (convertedMessage.hasKey)
-        assertEquals("Message key should not change", convertedMessage.key, ByteBuffer.wrap(v.key))
-      else
-        assertNull(convertedMessage.key)
-      if(v.payload == null) {
-        assertTrue(convertedMessage.isNull)
-        assertEquals("Payload should be null", null, convertedMessage.payload)
-      } else {
-        assertEquals("Message payload should not change", convertedMessage.payload, ByteBuffer.wrap(v.payload))
-      }
-      assertEquals("Compression codec should not change", convertedMessage.compressionCodec, v.codec)
-    }
-
-    for (v <- messages) {
-      if (v.magicValue == Message.MagicValue_V0) {
-        convertAndVerify(v, Message.MagicValue_V0, Message.MagicValue_V1)
-      } else if (v.magicValue == Message.MagicValue_V1) {
-        convertAndVerify(v, Message.MagicValue_V1, Message.MagicValue_V0)
-      }
-    }
-  }
-
   @Test(expected = classOf[IllegalArgumentException])
   def testInvalidTimestampAndMagicValueCombination() {
       new Message("hello".getBytes, 0L, Message.MagicValue_V0)

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
index 7d6ad91..5c9f035 100644
--- a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
+++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
@@ -19,17 +19,17 @@ package kafka.server
 
 import com.yammer.metrics.Metrics
 import kafka.cluster.BrokerEndPoint
-import kafka.message.{ByteBufferMessageSet, Message, NoCompressionCodec}
 import kafka.server.AbstractFetcherThread.{FetchRequest, PartitionData}
 import kafka.utils.TestUtils
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.common.record.{MemoryRecords, Record}
 import org.junit.Assert.{assertFalse, assertTrue}
 import org.junit.{Before, Test}
 
 import scala.collection.JavaConverters._
-import scala.collection.{mutable, Map}
+import scala.collection.{Map, mutable}
 
 class AbstractFetcherThreadTest {
 
@@ -91,10 +91,10 @@ class AbstractFetcherThreadTest {
     override def offset(topicAndPartition: TopicPartition): Long = offsets(topicAndPartition)
   }
 
-  class TestPartitionData(byteBufferMessageSet: ByteBufferMessageSet) extends PartitionData {
+  class TestPartitionData(records: MemoryRecords = MemoryRecords.EMPTY) extends PartitionData {
     override def errorCode: Short = Errors.NONE.code
 
-    override def toByteBufferMessageSet: ByteBufferMessageSet = byteBufferMessageSet
+    override def toRecords: MemoryRecords = records
 
     override def highWatermark: Long = 0L
 
@@ -119,7 +119,7 @@ class AbstractFetcherThreadTest {
     override def handlePartitionsWithErrors(partitions: Iterable[TopicPartition]): Unit = {}
 
     override protected def fetch(fetchRequest: DummyFetchRequest): Seq[(TopicPartition, TestPartitionData)] =
-      fetchRequest.offsets.mapValues(_ => new TestPartitionData(new ByteBufferMessageSet())).toSeq
+      fetchRequest.offsets.mapValues(_ => new TestPartitionData()).toSeq
 
     override protected def buildFetchRequest(partitionMap: collection.Seq[(TopicPartition, PartitionFetchState)]): DummyFetchRequest =
       new DummyFetchRequest(partitionMap.map { case (k, v) => (k, v.offset) }.toMap)
@@ -156,8 +156,8 @@ class AbstractFetcherThreadTest {
     @volatile var fetchCount = 0
 
     private val normalPartitionDataSet = List(
-      new TestPartitionData(new ByteBufferMessageSet(NoCompressionCodec, Seq(0L), new Message("hello".getBytes))),
-      new TestPartitionData(new ByteBufferMessageSet(NoCompressionCodec, Seq(1L), new Message("hello".getBytes)))
+      new TestPartitionData(MemoryRecords.withRecords(0L, Record.create("hello".getBytes()))),
+      new TestPartitionData(MemoryRecords.withRecords(1L, Record.create("hello".getBytes())))
     )
 
     override def processPartitionData(topicAndPartition: TopicPartition,
@@ -170,10 +170,10 @@ class AbstractFetcherThreadTest {
             .format(topicAndPartition, fetchOffset, logEndOffset))
 
       // Now check message's crc
-      val messages = partitionData.toByteBufferMessageSet
-      for (messageAndOffset <- messages.shallowIterator) {
-        messageAndOffset.message.ensureValid()
-        logEndOffset = messageAndOffset.nextOffset
+      val records = partitionData.toRecords
+      for (entry <- records.shallowIterator.asScala) {
+        entry.record.ensureValid()
+        logEndOffset = entry.nextOffset
       }
     }
 
@@ -181,12 +181,12 @@ class AbstractFetcherThreadTest {
       fetchCount += 1
       // Set the first fetch to get a corrupted message
       if (fetchCount == 1) {
-        val corruptedMessage = new Message("hello".getBytes)
-        val badChecksum = (corruptedMessage.checksum + 1 % Int.MaxValue).toInt
+        val corruptedRecord = Record.create("hello".getBytes())
+        val badChecksum = (corruptedRecord.checksum + 1 % Int.MaxValue).toInt
         // Garble checksum
-        Utils.writeUnsignedInt(corruptedMessage.buffer, Message.CrcOffset, badChecksum)
-        val byteBufferMessageSet = new ByteBufferMessageSet(NoCompressionCodec, corruptedMessage)
-        fetchRequest.offsets.mapValues(_ => new TestPartitionData(byteBufferMessageSet)).toSeq
+        Utils.writeUnsignedInt(corruptedRecord.buffer, Record.CRC_OFFSET, badChecksum)
+        val records = MemoryRecords.withRecords(corruptedRecord)
+        fetchRequest.offsets.mapValues(_ => new TestPartitionData(records)).toSeq
       } else
       // Then, the following fetches get the normal data
         fetchRequest.offsets.mapValues(v => normalPartitionDataSet(v.toInt)).toSeq


[3/9] kafka git commit: KAFKA-4390; Replace MessageSet usage with client-side alternatives

Posted by jg...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 250c8b8..65c2d05 100755
--- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
@@ -22,10 +22,9 @@ import java.util.Properties
 
 import kafka.api.{KAFKA_0_10_0_IV1, KAFKA_0_9_0}
 import kafka.common.TopicAndPartition
-import kafka.message._
 import kafka.server.OffsetCheckpoint
 import kafka.utils._
-import org.apache.kafka.common.record.CompressionType
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record}
 import org.apache.kafka.common.utils.Utils
 import org.junit.Assert._
 import org.junit._
@@ -43,7 +42,7 @@ import scala.util.Random
 @RunWith(value = classOf[Parameterized])
 class LogCleanerIntegrationTest(compressionCodec: String) {
 
-  val codec = CompressionCodec.getCompressionCodec(compressionCodec)
+  val codec = CompressionType.forName(compressionCodec)
   val time = new MockTime()
   val segmentSize = 256
   val deleteDelay = 1000
@@ -56,7 +55,7 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
   @Test
   def cleanerTest() {
     val largeMessageKey = 20
-    val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, Message.MagicValue_V1)
+    val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, Record.MAGIC_VALUE_V1)
     val maxMessageSize = largeMessageSet.sizeInBytes
 
     cleaner = makeCleaner(parts = 3, maxMessageSize = maxMessageSize)
@@ -133,13 +132,13 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
   }
 
   // returns (value, ByteBufferMessageSet)
-  private def createLargeSingleMessageSet(key: Int, messageFormatVersion: Byte): (String, ByteBufferMessageSet) = {
+  private def createLargeSingleMessageSet(key: Int, messageFormatVersion: Byte): (String, MemoryRecords) = {
     def messageValue(length: Int): String = {
       val random = new Random(0)
       new String(random.alphanumeric.take(length).toArray)
     }
     val value = messageValue(128)
-    val messageSet = TestUtils.singleMessageSet(payload = value.getBytes, codec = codec, key = key.toString.getBytes,
+    val messageSet = TestUtils.singletonRecords(value = value.getBytes, codec = codec, key = key.toString.getBytes,
       magicValue = messageFormatVersion)
     (value, messageSet)
   }
@@ -147,9 +146,9 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
   @Test
   def testCleanerWithMessageFormatV0(): Unit = {
     val largeMessageKey = 20
-    val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, Message.MagicValue_V0)
+    val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, Record.MAGIC_VALUE_V0)
     val maxMessageSize = codec match {
-      case NoCompressionCodec => largeMessageSet.sizeInBytes
+      case CompressionType.NONE => largeMessageSet.sizeInBytes
       case _ =>
         // the broker assigns absolute offsets for message format 0 which potentially causes the compressed size to
         // increase because the broker offsets are larger than the ones assigned by the client
@@ -165,7 +164,7 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
     props.put(LogConfig.MessageFormatVersionProp, KAFKA_0_9_0.version)
     log.config = new LogConfig(props)
 
-    val appends = writeDups(numKeys = 100, numDups = 3, log = log, codec = codec, magicValue = Message.MagicValue_V0)
+    val appends = writeDups(numKeys = 100, numDups = 3, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V0)
     val startSize = log.size
     cleaner.startup()
 
@@ -177,14 +176,14 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
     checkLogAfterAppendingDups(log, startSize, appends)
 
     val appends2: Seq[(Int, String, Long)] = {
-      val dupsV0 = writeDups(numKeys = 40, numDups = 3, log = log, codec = codec, magicValue = Message.MagicValue_V0)
+      val dupsV0 = writeDups(numKeys = 40, numDups = 3, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V0)
       val appendInfo = log.append(largeMessageSet, assignOffsets = true)
       val largeMessageOffset = appendInfo.firstOffset
 
       // also add some messages with version 1 to check that we handle mixed format versions correctly
       props.put(LogConfig.MessageFormatVersionProp, KAFKA_0_10_0_IV1.version)
       log.config = new LogConfig(props)
-      val dupsV1 = writeDups(startKey = 30, numKeys = 40, numDups = 3, log = log, codec = codec, magicValue = Message.MagicValue_V1)
+      val dupsV1 = writeDups(startKey = 30, numKeys = 40, numDups = 3, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V1)
       appends ++ dupsV0 ++ Seq((largeMessageKey, largeMessageValue, largeMessageOffset)) ++ dupsV1
     }
     val firstDirty2 = log.activeSegment.baseOffset
@@ -205,15 +204,15 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
 
     // with compression enabled, these messages will be written as a single message containing
     // all of the individual messages
-    var appendsV0 = writeDupsSingleMessageSet(numKeys = 2, numDups = 3, log = log, codec = codec, magicValue = Message.MagicValue_V0)
-    appendsV0 ++= writeDupsSingleMessageSet(numKeys = 2, startKey = 3, numDups = 2, log = log, codec = codec, magicValue = Message.MagicValue_V0)
+    var appendsV0 = writeDupsSingleMessageSet(numKeys = 2, numDups = 3, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V0)
+    appendsV0 ++= writeDupsSingleMessageSet(numKeys = 2, startKey = 3, numDups = 2, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V0)
 
     props.put(LogConfig.MessageFormatVersionProp, KAFKA_0_10_0_IV1.version)
     log.config = new LogConfig(props)
 
-    var appendsV1 = writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = Message.MagicValue_V1)
-    appendsV1 ++= writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = Message.MagicValue_V1)
-    appendsV1 ++= writeDupsSingleMessageSet(startKey = 6, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = Message.MagicValue_V1)
+    var appendsV1 = writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V1)
+    appendsV1 ++= writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V1)
+    appendsV1 ++= writeDupsSingleMessageSet(startKey = 6, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V1)
 
     val appends = appendsV0 ++ appendsV1
 
@@ -250,32 +249,27 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
   }
 
   private def readFromLog(log: Log): Iterable[(Int, String, Long)] = {
-
-    def messageIterator(entry: MessageAndOffset): Iterator[MessageAndOffset] =
-      // create single message iterator or deep iterator depending on compression codec
-      if (entry.message.compressionCodec == NoCompressionCodec) Iterator(entry)
-      else ByteBufferMessageSet.deepIterator(entry)
-
-    for (segment <- log.logSegments; entry <- segment.log; messageAndOffset <- messageIterator(entry)) yield {
-      val key = TestUtils.readString(messageAndOffset.message.key).toInt
-      val value = TestUtils.readString(messageAndOffset.message.payload)
-      (key, value, messageAndOffset.offset)
+    import JavaConverters._
+    for (segment <- log.logSegments; deepLogEntry <- segment.log.deepIterator.asScala) yield {
+      val key = TestUtils.readString(deepLogEntry.record.key).toInt
+      val value = TestUtils.readString(deepLogEntry.record.value)
+      (key, value, deepLogEntry.offset)
     }
   }
 
-  private def writeDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionCodec,
-                        startKey: Int = 0, magicValue: Byte = Message.CurrentMagicValue): Seq[(Int, String, Long)] = {
+  private def writeDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionType,
+                        startKey: Int = 0, magicValue: Byte = Record.CURRENT_MAGIC_VALUE): Seq[(Int, String, Long)] = {
     for(_ <- 0 until numDups; key <- startKey until (startKey + numKeys)) yield {
-      val payload = counter.toString
-      val appendInfo = log.append(TestUtils.singleMessageSet(payload = payload.toString.getBytes, codec = codec,
+      val value = counter.toString
+      val appendInfo = log.append(TestUtils.singletonRecords(value = value.toString.getBytes, codec = codec,
         key = key.toString.getBytes, magicValue = magicValue), assignOffsets = true)
       counter += 1
-      (key, payload, appendInfo.firstOffset)
+      (key, value, appendInfo.firstOffset)
     }
   }
 
-  private def writeDupsSingleMessageSet(numKeys: Int, numDups: Int, log: Log, codec: CompressionCodec,
-                                        startKey: Int = 0, magicValue: Byte = Message.CurrentMagicValue): Seq[(Int, String, Long)] = {
+  private def writeDupsSingleMessageSet(numKeys: Int, numDups: Int, log: Log, codec: CompressionType,
+                                        startKey: Int = 0, magicValue: Byte = Record.CURRENT_MAGIC_VALUE): Seq[(Int, String, Long)] = {
     val kvs = for (_ <- 0 until numDups; key <- startKey until (startKey + numKeys)) yield {
       val payload = counter.toString
       counter += 1
@@ -283,11 +277,11 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
     }
 
     val messages = kvs.map { case (key, payload) =>
-      new Message(payload.toString.getBytes, key.toString.getBytes, Message.NoTimestamp, magicValue)
+      Record.create(magicValue, key.toString.getBytes, payload.toString.getBytes)
     }
 
-    val messageSet = new ByteBufferMessageSet(compressionCodec = codec, messages: _*)
-    val appendInfo = log.append(messageSet, assignOffsets = true)
+    val records = MemoryRecords.withRecords(codec, messages: _*)
+    val appendInfo = log.append(records, assignOffsets = true)
     val offsets = appendInfo.firstOffset to appendInfo.lastOffset
 
     kvs.zip(offsets).map { case (kv, offset) => (kv._1, kv._2, offset) }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 5e029fc..abab3bf 100644
--- a/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala
@@ -21,7 +21,6 @@ import java.io.File
 import java.util.Properties
 
 import kafka.common.TopicAndPartition
-import kafka.message._
 import kafka.utils._
 import org.apache.kafka.common.record.CompressionType
 import org.apache.kafka.common.utils.Utils
@@ -33,7 +32,6 @@ import org.junit.runners.Parameterized.Parameters
 
 import scala.collection._
 
-
 /**
   * This is an integration test that tests the fully integrated log cleaner
   */
@@ -52,7 +50,7 @@ class LogCleanerLagIntegrationTest(compressionCodecName: String) extends Logging
   val logDir = TestUtils.tempDir()
   var counter = 0
   val topics = Array(TopicAndPartition("log", 0), TopicAndPartition("log", 1), TopicAndPartition("log", 2))
-  val compressionCodec = CompressionCodec.getCompressionCodec(compressionCodecName)
+  val compressionCodec = CompressionType.forName(compressionCodecName)
 
   @Test
   def cleanerTest(): Unit = {
@@ -96,7 +94,7 @@ class LogCleanerLagIntegrationTest(compressionCodecName: String) extends Logging
 
     val compactedSize = log.logSegments(0L, activeSegAtT0.baseOffset).map(_.size).sum
     debug(s"after cleaning the compacted size up to active segment at T0: $compactedSize")
-    val lastCleaned = cleaner.cleanerManager.allCleanerCheckpoints.get(TopicAndPartition("log", 0)).get
+    val lastCleaned = cleaner.cleanerManager.allCleanerCheckpoints(TopicAndPartition("log", 0))
     assertTrue(s"log cleaner should have processed up to offset $firstBlock1SegmentBaseOffset, but lastCleaned=$lastCleaned", lastCleaned >= firstBlock1SegmentBaseOffset)
     assertTrue(s"log should have been compacted: size up to offset of active segment at T0=$sizeUpToActiveSegmentAtT0 compacted size=$compactedSize",
       sizeUpToActiveSegmentAtT0 > compactedSize)
@@ -106,23 +104,19 @@ class LogCleanerLagIntegrationTest(compressionCodecName: String) extends Logging
   }
 
   private def readFromLog(log: Log): Iterable[(Int, Int)] = {
-    for (segment <- log.logSegments; entry <- segment.log; messageAndOffset <- {
-      // create single message iterator or deep iterator depending on compression codec
-      if (entry.message.compressionCodec == NoCompressionCodec)
-        Stream.cons(entry, Stream.empty).iterator
-      else
-        ByteBufferMessageSet.deepIterator(entry)
-    }) yield {
-      val key = TestUtils.readString(messageAndOffset.message.key).toInt
-      val value = TestUtils.readString(messageAndOffset.message.payload).toInt
+    import JavaConverters._
+
+    for (segment <- log.logSegments; logEntry <- segment.log.deepIterator.asScala) yield {
+      val key = TestUtils.readString(logEntry.record.key).toInt
+      val value = TestUtils.readString(logEntry.record.value).toInt
       key -> value
     }
   }
 
-  private def writeDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionCodec, timestamp: Long): Seq[(Int, Int)] = {
+  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.append(TestUtils.singleMessageSet(payload = counter.toString.getBytes, codec = codec, key = key.toString.getBytes, timestamp = timestamp), assignOffsets = true)
+      log.append(TestUtils.singletonRecords(value = counter.toString.getBytes, codec = codec, key = key.toString.getBytes, timestamp = timestamp), assignOffsets = true)
       counter += 1
       (key, count)
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
index 0cd52d6..5dfa268 100644
--- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
@@ -21,8 +21,8 @@ import java.io.File
 import java.util.Properties
 
 import kafka.common._
-import kafka.message._
 import kafka.utils._
+import org.apache.kafka.common.record.{MemoryRecords, Record}
 import org.apache.kafka.common.utils.Utils
 import org.junit.Assert._
 import org.junit.{After, Test}
@@ -54,8 +54,8 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
     */
   @Test
   def testLogsWithSegmentsToDeleteShouldNotConsiderCleanupPolicyDeleteLogs(): Unit = {
-    val messageSet = TestUtils.singleMessageSet("test".getBytes)
-    val log: Log = createLog(messageSet.sizeInBytes * 5, LogConfig.Delete)
+    val records = TestUtils.singletonRecords("test".getBytes)
+    val log: Log = createLog(records.sizeInBytes * 5, LogConfig.Delete)
     val cleanerManager: LogCleanerManager = createCleanerManager(log)
 
     val readyToDelete = cleanerManager.deletableLogs().size
@@ -67,8 +67,8 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
     */
   @Test
   def testLogsWithSegmentsToDeleteShouldConsiderCleanupPolicyCompactDeleteLogs(): Unit = {
-    val messageSet = TestUtils.singleMessageSet("test".getBytes, key="test".getBytes)
-    val log: Log = createLog(messageSet.sizeInBytes * 5, LogConfig.Compact + "," + LogConfig.Delete)
+    val records = TestUtils.singletonRecords("test".getBytes, key="test".getBytes)
+    val log: Log = createLog(records.sizeInBytes * 5, LogConfig.Compact + "," + LogConfig.Delete)
     val cleanerManager: LogCleanerManager = createCleanerManager(log)
 
     val readyToDelete = cleanerManager.deletableLogs().size
@@ -81,8 +81,8 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
     */
   @Test
   def testLogsWithSegmentsToDeleteShouldNotConsiderCleanupPolicyCompactLogs(): Unit = {
-    val messageSet = TestUtils.singleMessageSet("test".getBytes, key="test".getBytes)
-    val log: Log = createLog(messageSet.sizeInBytes * 5, LogConfig.Compact)
+    val records = TestUtils.singletonRecords("test".getBytes, key="test".getBytes)
+    val log: Log = createLog(records.sizeInBytes * 5, LogConfig.Compact)
     val cleanerManager: LogCleanerManager = createCleanerManager(log)
 
     val readyToDelete = cleanerManager.deletableLogs().size
@@ -100,7 +100,7 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
     val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
 
     while(log.numberOfSegments < 8)
-      log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = time.milliseconds))
+      log.append(logEntries(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = time.milliseconds))
 
     val topicAndPartition = TopicAndPartition("log", 0)
     val lastClean = Map(topicAndPartition-> 0L)
@@ -123,7 +123,7 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
 
     val t0 = time.milliseconds
     while(log.numberOfSegments < 4)
-      log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = t0))
+      log.append(logEntries(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = t0))
 
     val activeSegAtT0 = log.activeSegment
 
@@ -131,7 +131,7 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
     val t1 = time.milliseconds
 
     while (log.numberOfSegments < 8)
-      log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = t1))
+      log.append(logEntries(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = t1))
 
     val topicAndPartition = TopicAndPartition("log", 0)
     val lastClean = Map(topicAndPartition-> 0L)
@@ -155,7 +155,7 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
 
     val t0 = time.milliseconds
     while (log.numberOfSegments < 8)
-      log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = t0))
+      log.append(logEntries(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = t0))
 
     time.sleep(compactionLag + 1)
 
@@ -192,10 +192,7 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
   private def makeLog(dir: File = logDir, config: LogConfig = logConfig) =
     new Log(dir = dir, config = config, recoveryPoint = 0L, scheduler = time.scheduler, time = time)
 
-  private def message(key: Int, value: Int, timestamp: Long) =
-    new ByteBufferMessageSet(new Message(key = key.toString.getBytes,
-      bytes = value.toString.getBytes,
-      timestamp = timestamp,
-      magicValue = Message.MagicValue_V1))
+  private def logEntries(key: Int, value: Int, timestamp: Long) =
+    MemoryRecords.withRecords(Record.create(timestamp, key.toString.getBytes, value.toString.getBytes))
 
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 d80fba1..a99d4b9 100755
--- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
@@ -17,21 +17,21 @@
 
 package kafka.log
 
-import java.io.{DataOutputStream, File}
+import java.io.File
 import java.nio._
 import java.nio.file.Paths
 import java.util.Properties
 
 import kafka.common._
-import kafka.message._
 import kafka.utils._
-import org.apache.kafka.common.record.{MemoryRecords, TimestampType}
+import org.apache.kafka.common.record._
 import org.apache.kafka.common.utils.Utils
 import org.junit.Assert._
 import org.junit.{After, Test}
 import org.scalatest.junit.JUnitSuite
 
 import scala.collection._
+import JavaConverters._
 
 /**
  * Unit tests for the log cleaning logic
@@ -66,7 +66,7 @@ class LogCleanerTest extends JUnitSuite {
 
     // append messages to the log until we have four segments
     while(log.numberOfSegments < 4)
-      log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
+      log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
     val keysFound = keysInLog(log)
     assertEquals(0L until log.logEndOffset, keysFound)
 
@@ -100,7 +100,7 @@ class LogCleanerTest extends JUnitSuite {
     val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
 
     while(log.numberOfSegments < 2)
-      log.append(message(log.logEndOffset.toInt, Array.fill(largeMessageSize)(0: Byte)))
+      log.append(record(log.logEndOffset.toInt, Array.fill(largeMessageSize)(0: Byte)))
     val keysFound = keysInLog(log)
     assertEquals(0L until log.logEndOffset, keysFound)
 
@@ -123,23 +123,23 @@ class LogCleanerTest extends JUnitSuite {
     logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer)
 
     val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
-    
+
     // append messages with the keys 0 through N
     while(log.numberOfSegments < 2)
-      log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
-      
+      log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
+
     // delete all even keys between 0 and N
     val leo = log.logEndOffset
     for(key <- 0 until leo.toInt by 2)
-      log.append(deleteMessage(key))
-      
+      log.append(tombstoneRecord(key))
+
     // append some new unique keys to pad out to a new active segment
     while(log.numberOfSegments < 4)
-      log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
-      
+      log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
+
     cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0, log.activeSegment.baseOffset))
     val keys = keysInLog(log).toSet
-    assertTrue("None of the keys we deleted should still exist.", 
+    assertTrue("None of the keys we deleted should still exist.",
                (0 until leo.toInt by 2).forall(!keys.contains(_)))
   }
 
@@ -151,11 +151,11 @@ class LogCleanerTest extends JUnitSuite {
 
     val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
 
-    log.append(message(0,0)) // offset 0
-    log.append(message(1,1)) // offset 1
-    log.append(message(0,0)) // offset 2
-    log.append(message(1,1)) // offset 3
-    log.append(message(0,0)) // offset 4
+    log.append(record(0,0)) // offset 0
+    log.append(record(1,1)) // offset 1
+    log.append(record(0,0)) // offset 2
+    log.append(record(1,1)) // offset 3
+    log.append(record(0,0)) // offset 4
     // roll the segment, so we can clean the messages already appended
     log.roll()
 
@@ -180,11 +180,11 @@ class LogCleanerTest extends JUnitSuite {
 
     val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
 
-    log.append(message(0,0)) // offset 0
-    log.append(message(1,1)) // offset 1
-    log.append(message(0,0)) // offset 2
-    log.append(message(1,1)) // offset 3
-    log.append(message(0,0)) // offset 4
+    log.append(record(0,0)) // offset 0
+    log.append(record(1,1)) // offset 1
+    log.append(record(0,0)) // offset 2
+    log.append(record(1,1)) // offset 3
+    log.append(record(0,0)) // offset 4
     // roll the segment, so we can clean the messages already appended
     log.roll()
 
@@ -218,18 +218,18 @@ class LogCleanerTest extends JUnitSuite {
 
     // append messages with the keys 0 through N-1, values equal offset
     while(log.numberOfSegments <= numCleanableSegments)
-      log.append(message(log.logEndOffset.toInt % N, log.logEndOffset.toInt))
+      log.append(record(log.logEndOffset.toInt % N, log.logEndOffset.toInt))
 
     // at this point one message past the cleanable segments has been added
     // the entire segment containing the first uncleanable offset should not be cleaned.
     val firstUncleanableOffset = log.logEndOffset + 1  // +1  so it is past the baseOffset
 
     while(log.numberOfSegments < numTotalSegments - 1)
-      log.append(message(log.logEndOffset.toInt % N, log.logEndOffset.toInt))
+      log.append(record(log.logEndOffset.toInt % N, log.logEndOffset.toInt))
 
     // the last (active) segment has just one message
 
-    def distinctValuesBySegment = log.logSegments.map(s => s.log.map(m => TestUtils.readString(m.message.payload)).toSet.size).toSeq
+    def distinctValuesBySegment = log.logSegments.map(s => s.log.shallowIterator.asScala.map(m => TestUtils.readString(m.record.value)).toSet.size).toSeq
 
     val disctinctValuesBySegmentBeforeClean = distinctValuesBySegment
     assertTrue("Test is not effective unless each segment contains duplicates. Increase segment size or decrease number of keys.",
@@ -253,7 +253,7 @@ class LogCleanerTest extends JUnitSuite {
     val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
 
     // create 6 segments with only one message in each segment
-    val messageSet = TestUtils.singleMessageSet(payload = Array.fill[Byte](50)(0), key = 1.toString.getBytes)
+    val messageSet = TestUtils.singletonRecords(value = Array.fill[Byte](50)(0), key = 1.toString.getBytes)
     for (_ <- 0 until 6)
       log.append(messageSet, assignOffsets = true)
 
@@ -271,7 +271,7 @@ class LogCleanerTest extends JUnitSuite {
     val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
 
     // create 6 segments with only one message in each segment
-    val messageSet = TestUtils.singleMessageSet(payload = Array.fill[Byte](50)(0), key = 1.toString.getBytes)
+    val messageSet = TestUtils.singletonRecords(value = Array.fill[Byte](50)(0), key = 1.toString.getBytes)
     for (_ <- 0 until 6)
       log.append(messageSet, assignOffsets = true)
 
@@ -305,14 +305,14 @@ class LogCleanerTest extends JUnitSuite {
 
     // append unkeyed messages
     while(log.numberOfSegments < 2)
-      log.append(unkeyedMessage(log.logEndOffset.toInt))
+      log.append(unkeyedRecord(log.logEndOffset.toInt))
     val numInvalidMessages = unkeyedMessageCountInLog(log)
 
     val sizeWithUnkeyedMessages = log.size
 
     // append keyed messages
     while(log.numberOfSegments < 3)
-      log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
+      log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
 
     val expectedSizeAfterCleaning = log.size - sizeWithUnkeyedMessages
     val (_, stats) = cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0, log.activeSegment.baseOffset))
@@ -321,17 +321,17 @@ class LogCleanerTest extends JUnitSuite {
     assertEquals("Log should only contain keyed messages after cleaning.", expectedSizeAfterCleaning, log.size)
     assertEquals("Cleaner should have seen %d invalid messages.", numInvalidMessages, stats.invalidMessagesRead)
   }
-  
+
   /* extract all the keys from a log */
   def keysInLog(log: Log): Iterable[Int] =
-    log.logSegments.flatMap(s => s.log.filter(!_.message.isNull).filter(_.message.hasKey).map(m => TestUtils.readString(m.message.key).toInt))
+    log.logSegments.flatMap(s => s.log.shallowIterator.asScala.filter(!_.record.hasNullValue).filter(_.record.hasKey).map(m => TestUtils.readString(m.record.key).toInt))
 
   /* extract all the offsets from a log */
   def offsetsInLog(log: Log): Iterable[Long] =
-    log.logSegments.flatMap(s => s.log.filter(!_.message.isNull).filter(_.message.hasKey).map(m => m.offset))
+    log.logSegments.flatMap(s => s.log.shallowIterator.asScala.filter(!_.record.hasNullValue).filter(_.record.hasKey).map(m => m.offset))
 
   def unkeyedMessageCountInLog(log: Log) =
-    log.logSegments.map(s => s.log.filter(!_.message.isNull).count(m => !m.message.hasKey)).sum
+    log.logSegments.map(s => s.log.shallowIterator.asScala.filter(!_.record.hasNullValue).count(m => !m.record.hasKey)).sum
 
   def abortCheckDone(topicAndPartition: TopicAndPartition): Unit = {
     throw new LogCleaningAbortedException()
@@ -350,7 +350,7 @@ class LogCleanerTest extends JUnitSuite {
 
     // append messages to the log until we have four segments
     while(log.numberOfSegments < 4)
-      log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
+      log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
 
     val keys = keysInLog(log)
     val map = new FakeOffsetMap(Int.MaxValue)
@@ -371,20 +371,20 @@ class LogCleanerTest extends JUnitSuite {
     logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
 
     val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
-    
+
     // append some messages to the log
     var i = 0
     while(log.numberOfSegments < 10) {
-      log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes))
+      log.append(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes))
       i += 1
     }
-    
+
     // grouping by very large values should result in a single group with all the segments in it
     var groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue)
     assertEquals(1, groups.size)
     assertEquals(log.numberOfSegments, groups.head.size)
     checkSegmentOrder(groups)
-    
+
     // grouping by very small values should result in all groups having one entry
     groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = 1, maxIndexSize = Int.MaxValue)
     assertEquals(log.numberOfSegments, groups.size)
@@ -396,20 +396,20 @@ class LogCleanerTest extends JUnitSuite {
     checkSegmentOrder(groups)
 
     val groupSize = 3
-    
+
     // check grouping by log size
     val logSize = log.logSegments.take(groupSize).map(_.size).sum.toInt + 1
     groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = logSize, maxIndexSize = Int.MaxValue)
     checkSegmentOrder(groups)
     assertTrue("All but the last group should be the target size.", groups.dropRight(1).forall(_.size == groupSize))
-    
+
     // check grouping by index size
     val indexSize = log.logSegments.take(groupSize).map(_.index.sizeInBytes).sum + 1
     groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = indexSize)
     checkSegmentOrder(groups)
     assertTrue("All but the last group should be the target size.", groups.dropRight(1).forall(_.size == groupSize))
   }
-  
+
   /**
    * Validate the logic for grouping log segments together for cleaning when only a small number of
    * messages are retained, but the range of offsets is greater than Int.MaxValue. A group should not
@@ -425,47 +425,45 @@ class LogCleanerTest extends JUnitSuite {
     logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
 
     val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
-       
+
     // fill up first segment
     while (log.numberOfSegments == 1)
-      log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes))
-    
+      log.append(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes))
+
     // forward offset and append message to next segment at offset Int.MaxValue
-    val messageSet = new ByteBufferMessageSet(NoCompressionCodec, new LongRef(Int.MaxValue - 1),
-      new Message("hello".getBytes, "hello".getBytes, Message.NoTimestamp, Message.MagicValue_V1))
-    log.append(messageSet, assignOffsets = false)
-    log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes))
+    val records = MemoryRecords.withLogEntries(LogEntry.create(Int.MaxValue - 1, Record.create("hello".getBytes, "hello".getBytes)))
+    log.append(records, assignOffsets = false)
+    log.append(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes))
     assertEquals(Int.MaxValue, log.activeSegment.index.lastOffset)
-    
+
     // grouping should result in a single group with maximum relative offset of Int.MaxValue
     var groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue)
     assertEquals(1, groups.size)
-    
+
     // append another message, making last offset of second segment > Int.MaxValue
-    log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes))
-    
+    log.append(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes))
+
     // grouping should not group the two segments to ensure that maximum relative offset in each group <= Int.MaxValue
     groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue)
     assertEquals(2, groups.size)
     checkSegmentOrder(groups)
-    
+
     // append more messages, creating new segments, further grouping should still occur
     while (log.numberOfSegments < 4)
-      log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes))
+      log.append(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes))
 
     groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue)
     assertEquals(log.numberOfSegments - 1, groups.size)
     for (group <- groups)
       assertTrue("Relative offset greater than Int.MaxValue", group.last.index.lastOffset - group.head.index.baseOffset <= Int.MaxValue)
     checkSegmentOrder(groups)
-    
   }
-  
+
   private def checkSegmentOrder(groups: Seq[Seq[LogSegment]]): Unit = {
     val offsets = groups.flatMap(_.map(_.baseOffset))
     assertEquals("Offsets should be in increasing order.", offsets.sorted, offsets)
   }
-  
+
   /**
    * Test building an offset map off the log
    */
@@ -496,8 +494,7 @@ class LogCleanerTest extends JUnitSuite {
     checkRange(map, segments(1).baseOffset.toInt, segments(3).baseOffset.toInt)
     checkRange(map, segments(3).baseOffset.toInt, log.logEndOffset.toInt)
   }
-  
-  
+
   /**
    * Tests recovery if broker crashes at the following stages during the cleaning sequence
    * <ol>
@@ -516,8 +513,8 @@ class LogCleanerTest extends JUnitSuite {
     logProps.put(LogConfig.FileDeleteDelayMsProp, 10: java.lang.Integer)
 
     val config = LogConfig.fromProps(logConfig.originals, logProps)
-      
-    def recoverAndCheck(config: LogConfig, expectedKeys : Iterable[Int]) : Log = {   
+
+    def recoverAndCheck(config: LogConfig, expectedKeys : Iterable[Int]) : Log = {
       // Recover log file and check that after recovery, keys are as expected
       // and all temporary files have been deleted
       val recoveredLog = makeLog(config = config)
@@ -530,25 +527,25 @@ class LogCleanerTest extends JUnitSuite {
       assertEquals(expectedKeys, keysInLog(recoveredLog))
       recoveredLog
     }
-    
+
     // create a log and append some messages
     var log = makeLog(config = config)
     var messageCount = 0
     while(log.numberOfSegments < 10) {
-      log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
+      log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
       messageCount += 1
     }
     val allKeys = keysInLog(log)
-    
+
     // pretend we have odd-numbered keys
     val offsetMap = new FakeOffsetMap(Int.MaxValue)
     for (k <- 1 until messageCount by 2)
       offsetMap.put(key(k), Long.MaxValue)
-     
+
     // clean the log
     cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats())
     var cleanedKeys = keysInLog(log)
-    
+
     // 1) Simulate recovery just after .cleaned file is created, before rename to .swap
     //    On recovery, clean operation is aborted. All messages should be present in the log
     log.logSegments.head.changeFileSuffixes("", Log.CleanedFileSuffix)
@@ -556,44 +553,44 @@ class LogCleanerTest extends JUnitSuite {
       Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, "")))
     }
     log = recoverAndCheck(config, allKeys)
-    
+
     // clean again
     cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats())
     cleanedKeys = keysInLog(log)
-    
+
     // 2) Simulate recovery just after swap file is created, before old segment files are
-    //    renamed to .deleted. Clean operation is resumed during recovery. 
+    //    renamed to .deleted. Clean operation is resumed during recovery.
     log.logSegments.head.changeFileSuffixes("", Log.SwapFileSuffix)
     for (file <- dir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) {
       Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, "")))
-    }   
+    }
     log = recoverAndCheck(config, cleanedKeys)
 
     // add some more messages and clean the log again
     while(log.numberOfSegments < 10) {
-      log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
+      log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
       messageCount += 1
     }
     for (k <- 1 until messageCount by 2)
-      offsetMap.put(key(k), Long.MaxValue)    
+      offsetMap.put(key(k), Long.MaxValue)
     cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats())
     cleanedKeys = keysInLog(log)
-    
+
     // 3) Simulate recovery after swap file is created and old segments files are renamed
     //    to .deleted. Clean operation is resumed during recovery.
     log.logSegments.head.changeFileSuffixes("", Log.SwapFileSuffix)
     log = recoverAndCheck(config, cleanedKeys)
-    
+
     // add some more messages and clean the log again
     while(log.numberOfSegments < 10) {
-      log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
+      log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
       messageCount += 1
     }
     for (k <- 1 until messageCount by 2)
-      offsetMap.put(key(k), Long.MaxValue)    
+      offsetMap.put(key(k), Long.MaxValue)
     cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats())
     cleanedKeys = keysInLog(log)
-    
+
     // 4) Simulate recovery after swap is complete, but async deletion
     //    is not yet complete. Clean operation is resumed during recovery.
     recoverAndCheck(config, cleanedKeys)
@@ -631,11 +628,11 @@ class LogCleanerTest extends JUnitSuite {
     val log = makeLog()
     val cleaner = makeCleaner(2)
 
-    log.append(message(0,0))
-    log.append(message(1,1))
-    log.append(message(2,2))
-    log.append(message(3,3))
-    log.append(message(4,4))
+    log.append(record(0,0))
+    log.append(record(1,1))
+    log.append(record(2,2))
+    log.append(record(3,3))
+    log.append(record(4,4))
     log.roll()
 
     val stats = new CleanerStats()
@@ -653,7 +650,7 @@ class LogCleanerTest extends JUnitSuite {
    */
   @Test
   def testCleanCorruptMessageSet() {
-    val codec = SnappyCompressionCodec
+    val codec = CompressionType.GZIP
 
     val logProps = new Properties()
     logProps.put(LogConfig.CompressionTypeProp, codec.name)
@@ -682,10 +679,10 @@ class LogCleanerTest extends JUnitSuite {
 
     cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0, log.activeSegment.baseOffset))
 
-    for (segment <- log.logSegments; shallowMessage <- segment.log.iterator; deepMessage <- ByteBufferMessageSet.deepIterator(shallowMessage)) {
-      assertEquals(shallowMessage.message.magic, deepMessage.message.magic)
-      val value = TestUtils.readString(deepMessage.message.payload).toLong
-      assertEquals(deepMessage.offset, value)
+    for (segment <- log.logSegments; shallowLogEntry <- segment.log.shallowIterator.asScala; deepLogEntry <- shallowLogEntry.asScala) {
+      assertEquals(shallowLogEntry.record.magic, deepLogEntry.record.magic)
+      val value = TestUtils.readString(deepLogEntry.record.value).toLong
+      assertEquals(deepLogEntry.offset, value)
     }
   }
 
@@ -704,7 +701,7 @@ class LogCleanerTest extends JUnitSuite {
     val corruptedMessage = invalidCleanedMessage(offset, set)
     val records = MemoryRecords.readableRecords(corruptedMessage.buffer)
 
-    for (logEntry <- records.iterator.asScala) {
+    for (logEntry <- records.deepIterator.asScala) {
       val offset = logEntry.offset
       val value = TestUtils.readString(logEntry.record.value).toLong
       assertEquals(offset, value)
@@ -718,94 +715,64 @@ class LogCleanerTest extends JUnitSuite {
 
   private def invalidCleanedMessage(initialOffset: Long,
                                     keysAndValues: Iterable[(Int, Int)],
-                                    codec: CompressionCodec = SnappyCompressionCodec): ByteBufferMessageSet = {
+                                    codec: CompressionType = CompressionType.GZIP): MemoryRecords = {
     // this function replicates the old versions of the cleaner which under some circumstances
     // would write invalid compressed message sets with the outer magic set to 1 and the inner
     // magic set to 0
-
-    val messages = keysAndValues.map(kv =>
-      new Message(key = kv._1.toString.getBytes,
-        bytes = kv._2.toString.getBytes,
-        timestamp = Message.NoTimestamp,
-        magicValue = Message.MagicValue_V0))
-
-    val messageWriter = new MessageWriter(math.min(math.max(MessageSet.messageSetSize(messages) / 2, 1024), 1 << 16))
-    var lastOffset = initialOffset
-
-    messageWriter.write(
-      codec = codec,
-      timestamp = Message.NoTimestamp,
-      timestampType = TimestampType.CREATE_TIME,
-      magicValue = Message.MagicValue_V1) { outputStream =>
-
-      val output = new DataOutputStream(CompressionFactory(codec, Message.MagicValue_V1, outputStream))
-      try {
-        for (message <- messages) {
-          val innerOffset = lastOffset - initialOffset
-          output.writeLong(innerOffset)
-          output.writeInt(message.size)
-          output.write(message.buffer.array, message.buffer.arrayOffset, message.buffer.limit)
-          lastOffset += 1
-        }
-      } finally {
-        output.close()
-      }
+    val records = keysAndValues.map(kv =>
+      Record.create(Record.MAGIC_VALUE_V0,
+        Record.NO_TIMESTAMP,
+        kv._1.toString.getBytes,
+        kv._2.toString.getBytes))
+
+    val buffer = ByteBuffer.allocate(math.min(math.max(records.map(_.sizeInBytes()).sum / 2, 1024), 1 << 16))
+    val builder = MemoryRecords.builder(buffer, Record.MAGIC_VALUE_V1, codec, TimestampType.CREATE_TIME)
+
+    var offset = initialOffset
+    records.foreach { record =>
+      builder.appendUnchecked(offset, record)
+      offset += 1
     }
-    val buffer = ByteBuffer.allocate(messageWriter.size + MessageSet.LogOverhead)
-    ByteBufferMessageSet.writeMessage(buffer, messageWriter, lastOffset - 1)
-    buffer.rewind()
 
-    new ByteBufferMessageSet(buffer)
+    builder.build()
   }
 
   private def messageWithOffset(key: Int, value: Int, offset: Long) =
-    new ByteBufferMessageSet(NoCompressionCodec, Seq(offset),
-                             new Message(key = key.toString.getBytes,
-                                         bytes = value.toString.getBytes,
-                                         timestamp = Message.NoTimestamp,
-                                         magicValue = Message.MagicValue_V1))
-  
-  
+    MemoryRecords.withLogEntries(LogEntry.create(offset, Record.create(key.toString.getBytes, value.toString.getBytes)))
+
   def makeLog(dir: File = dir, config: LogConfig = logConfig) =
     new Log(dir = dir, config = config, recoveryPoint = 0L, scheduler = time.scheduler, time = time)
 
   def noOpCheckDone(topicAndPartition: TopicAndPartition) { /* do nothing */  }
 
   def makeCleaner(capacity: Int, checkDone: (TopicAndPartition) => Unit = noOpCheckDone, maxMessageSize: Int = 64*1024) =
-    new Cleaner(id = 0, 
-                offsetMap = new FakeOffsetMap(capacity), 
+    new Cleaner(id = 0,
+                offsetMap = new FakeOffsetMap(capacity),
                 ioBufferSize = maxMessageSize,
                 maxIoBufferSize = maxMessageSize,
                 dupBufferLoadFactor = 0.75,
-                throttler = throttler, 
+                throttler = throttler,
                 time = time,
                 checkDone = checkDone )
-  
+
   def writeToLog(log: Log, seq: Iterable[(Int, Int)]): Iterable[Long] = {
     for((key, value) <- seq)
-      yield log.append(message(key, value)).firstOffset
+      yield log.append(record(key, value)).firstOffset
   }
-  
+
   def key(id: Int) = ByteBuffer.wrap(id.toString.getBytes)
-  
-  def message(key: Int, value: Int): ByteBufferMessageSet =
-    message(key, value.toString.getBytes)
-
-  def message(key: Int, value: Array[Byte]) =
-    new ByteBufferMessageSet(new Message(key = key.toString.getBytes,
-                                         bytes = value,
-                                         timestamp = Message.NoTimestamp,
-                                         magicValue = Message.MagicValue_V1))
-
-  def unkeyedMessage(value: Int) =
-    new ByteBufferMessageSet(new Message(bytes = value.toString.getBytes))
-
-  def deleteMessage(key: Int) =
-    new ByteBufferMessageSet(new Message(key = key.toString.getBytes,
-                                         bytes = null,
-                                         timestamp = Message.NoTimestamp,
-                                         magicValue = Message.MagicValue_V1))
-  
+
+  def record(key: Int, value: Int): MemoryRecords =
+    record(key, value.toString.getBytes)
+
+  def record(key: Int, value: Array[Byte]) =
+    MemoryRecords.withRecords(Record.create(key.toString.getBytes, value))
+
+  def unkeyedRecord(value: Int) =
+    MemoryRecords.withRecords(Record.create(value.toString.getBytes))
+
+  def tombstoneRecord(key: Int) = record(key, null)
+
 }
 
 class FakeOffsetMap(val slots: Int) extends OffsetMap {

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 5421da9..40e6228 100755
--- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
@@ -67,7 +67,7 @@ class LogManagerTest {
     val log = logManager.createLog(TopicAndPartition(name, 0), logConfig)
     val logFile = new File(logDir, name + "-0")
     assertTrue(logFile.exists)
-    log.append(TestUtils.singleMessageSet("test".getBytes()))
+    log.append(TestUtils.singletonRecords("test".getBytes()))
   }
 
   /**
@@ -89,7 +89,7 @@ class LogManagerTest {
     val log = logManager.createLog(TopicAndPartition(name, 0), logConfig)
     var offset = 0L
     for(_ <- 0 until 200) {
-      val set = TestUtils.singleMessageSet("test".getBytes())
+      val set = TestUtils.singletonRecords("test".getBytes())
       val info = log.append(set)
       offset = info.lastOffset
     }
@@ -101,7 +101,7 @@ class LogManagerTest {
     assertEquals("Now there should only be only one segment in the index.", 1, log.numberOfSegments)
     time.sleep(log.config.fileDeleteDelayMs + 1)
     assertEquals("Files should have been deleted", log.numberOfSegments * 3, log.dir.list.length)
-    assertEquals("Should get empty fetch off new log.", 0, log.read(offset+1, 1024).messageSet.sizeInBytes)
+    assertEquals("Should get empty fetch off new log.", 0, log.read(offset+1, 1024).records.sizeInBytes)
 
     try {
       log.read(0, 1024)
@@ -110,7 +110,7 @@ class LogManagerTest {
       case _: OffsetOutOfRangeException => // This is good.
     }
     // log should still be appendable
-    log.append(TestUtils.singleMessageSet("test".getBytes()))
+    log.append(TestUtils.singletonRecords("test".getBytes()))
   }
 
   /**
@@ -118,7 +118,7 @@ class LogManagerTest {
    */
   @Test
   def testCleanupSegmentsToMaintainSize() {
-    val setSize = TestUtils.singleMessageSet("test".getBytes()).sizeInBytes
+    val setSize = TestUtils.singletonRecords("test".getBytes()).sizeInBytes
     logManager.shutdown()
     val logProps = new Properties()
     logProps.put(LogConfig.SegmentBytesProp, 10 * setSize: java.lang.Integer)
@@ -135,7 +135,7 @@ class LogManagerTest {
     // add a bunch of messages that should be larger than the retentionSize
     val numMessages = 200
     for (_ <- 0 until numMessages) {
-      val set = TestUtils.singleMessageSet("test".getBytes())
+      val set = TestUtils.singletonRecords("test".getBytes())
       val info = log.append(set)
       offset = info.firstOffset
     }
@@ -147,7 +147,7 @@ class LogManagerTest {
     assertEquals("Now there should be exactly 6 segments", 6, log.numberOfSegments)
     time.sleep(log.config.fileDeleteDelayMs + 1)
     assertEquals("Files should have been deleted", log.numberOfSegments * 3, log.dir.list.length)
-    assertEquals("Should get empty fetch off new log.", 0, log.read(offset + 1, 1024).messageSet.sizeInBytes)
+    assertEquals("Should get empty fetch off new log.", 0, log.read(offset + 1, 1024).records.sizeInBytes)
     try {
       log.read(0, 1024)
       fail("Should get exception from fetching earlier.")
@@ -155,7 +155,7 @@ class LogManagerTest {
       case _: OffsetOutOfRangeException => // This is good.
     }
     // log should still be appendable
-    log.append(TestUtils.singleMessageSet("test".getBytes()))
+    log.append(TestUtils.singletonRecords("test".getBytes()))
   }
 
   /**
@@ -169,7 +169,7 @@ class LogManagerTest {
     val log = logManager.createLog(TopicAndPartition(name, 0), LogConfig.fromProps(logConfig.originals, logProps))
     var offset = 0L
     for (_ <- 0 until 200) {
-      val set = TestUtils.singleMessageSet("test".getBytes(), key="test".getBytes())
+      val set = TestUtils.singletonRecords("test".getBytes(), key="test".getBytes())
       val info = log.append(set)
       offset = info.lastOffset
     }
@@ -198,7 +198,7 @@ class LogManagerTest {
     val log = logManager.createLog(TopicAndPartition(name, 0), config)
     val lastFlush = log.lastFlushTime
     for (_ <- 0 until 200) {
-      val set = TestUtils.singleMessageSet("test".getBytes())
+      val set = TestUtils.singletonRecords("test".getBytes())
       log.append(set)
     }
     time.sleep(logManager.InitialTaskDelayMs)
@@ -280,7 +280,7 @@ class LogManagerTest {
     val logs = topicAndPartitions.map(this.logManager.createLog(_, logConfig))
     logs.foreach(log => {
       for (_ <- 0 until 50)
-        log.append(TestUtils.singleMessageSet("test".getBytes()))
+        log.append(TestUtils.singletonRecords("test".getBytes()))
 
       log.flush()
     })

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 f02c5cb..d99981a 100644
--- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
@@ -16,15 +16,13 @@
  */
  package kafka.log
 
-import org.junit.Assert._
-import java.util.concurrent.atomic._
-
-import kafka.common.LongRef
-import org.junit.{After, Test}
 import kafka.utils.TestUtils
-import kafka.message._
+import org.apache.kafka.common.record.{FileRecords, MemoryRecords, Record}
 import org.apache.kafka.common.utils.Time
+import org.junit.Assert._
+import org.junit.{After, Test}
 
+import scala.collection.JavaConverters._
 import scala.collection._
 
 class LogSegmentTest {
@@ -34,7 +32,7 @@ class LogSegmentTest {
   /* create a segment with the given base offset */
   def createSegment(offset: Long, indexIntervalBytes: Int = 10): LogSegment = {
     val msFile = TestUtils.tempFile()
-    val ms = new FileMessageSet(msFile)
+    val ms = FileRecords.open(msFile)
     val idxFile = TestUtils.tempFile()
     val timeIdxFile = TestUtils.tempFile()
     idxFile.delete()
@@ -47,12 +45,10 @@ class LogSegmentTest {
   }
   
   /* create a ByteBufferMessageSet for the given messages starting from the given offset */
-  def messages(offset: Long, messages: String*): ByteBufferMessageSet = {
-    new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, 
-                             offsetCounter = new LongRef(offset),
-                             messages = messages.map(s => new Message(s.getBytes, offset * 10, Message.MagicValue_V1)):_*)
+  def records(offset: Long, records: String*): MemoryRecords = {
+    MemoryRecords.withRecords(offset, records.map(s => Record.create(Record.MAGIC_VALUE_V1, offset * 10, s.getBytes)):_*)
   }
-  
+
   @After
   def teardown() {
     for(seg <- segments) {
@@ -60,7 +56,7 @@ class LogSegmentTest {
       seg.log.delete()
     }
   }
-  
+
   /**
    * A read on an empty log segment should return null
    */
@@ -70,7 +66,7 @@ class LogSegmentTest {
     val read = seg.read(startOffset = 40, maxSize = 300, maxOffset = None)
     assertNull("Read beyond the last offset in the segment should be null", read)
   }
-  
+
   /**
    * Reading from before the first offset in the segment should return messages
    * beginning with the first message in the segment
@@ -78,12 +74,12 @@ class LogSegmentTest {
   @Test
   def testReadBeforeFirstOffset() {
     val seg = createSegment(40)
-    val ms = messages(50, "hello", "there", "little", "bee")
-    seg.append(50, Message.NoTimestamp, -1L, ms)
-    val read = seg.read(startOffset = 41, maxSize = 300, maxOffset = None).messageSet
-    assertEquals(ms.toList, read.toList)
+    val ms = records(50, "hello", "there", "little", "bee")
+    seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
+    val read = seg.read(startOffset = 41, maxSize = 300, maxOffset = None).records
+    assertEquals(ms.deepIterator.asScala.toList, read.deepIterator.asScala.toList)
   }
-  
+
   /**
    * If we set the startOffset and maxOffset for the read to be the same value
    * we should get only the first message in the log
@@ -92,28 +88,28 @@ class LogSegmentTest {
   def testMaxOffset() {
     val baseOffset = 50
     val seg = createSegment(baseOffset)
-    val ms = messages(baseOffset, "hello", "there", "beautiful")
-    seg.append(baseOffset, Message.NoTimestamp, -1L, ms)
-    def validate(offset: Long) = 
-      assertEquals(ms.filter(_.offset == offset).toList, 
-                   seg.read(startOffset = offset, maxSize = 1024, maxOffset = Some(offset+1)).messageSet.toList)
+    val ms = records(baseOffset, "hello", "there", "beautiful")
+    seg.append(baseOffset, Record.NO_TIMESTAMP, -1L, ms)
+    def validate(offset: Long) =
+      assertEquals(ms.deepIterator.asScala.filter(_.offset == offset).toList,
+                   seg.read(startOffset = offset, maxSize = 1024, maxOffset = Some(offset+1)).records.deepIterator.asScala.toList)
     validate(50)
     validate(51)
     validate(52)
   }
-  
+
   /**
    * If we read from an offset beyond the last offset in the segment we should get null
    */
   @Test
   def testReadAfterLast() {
     val seg = createSegment(40)
-    val ms = messages(50, "hello", "there")
-    seg.append(50, Message.NoTimestamp, -1L, ms)
+    val ms = records(50, "hello", "there")
+    seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
     val read = seg.read(startOffset = 52, maxSize = 200, maxOffset = None)
     assertNull("Read beyond the last offset in the segment should give null", read)
   }
-  
+
   /**
    * If we read from an offset which doesn't exist we should get a message set beginning
    * with the least offset greater than the given startOffset.
@@ -121,14 +117,14 @@ class LogSegmentTest {
   @Test
   def testReadFromGap() {
     val seg = createSegment(40)
-    val ms = messages(50, "hello", "there")
-    seg.append(50, Message.NoTimestamp, -1L, ms)
-    val ms2 = messages(60, "alpha", "beta")
-    seg.append(60, Message.NoTimestamp, -1L, ms2)
+    val ms = records(50, "hello", "there")
+    seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
+    val ms2 = records(60, "alpha", "beta")
+    seg.append(60, Record.NO_TIMESTAMP, -1L, ms2)
     val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None)
-    assertEquals(ms2.toList, read.messageSet.toList)
+    assertEquals(ms2.deepIterator.asScala.toList, read.records.deepIterator.asScala.toList)
   }
-  
+
   /**
    * In a loop append two messages then truncate off the second of those messages and check that we can read
    * the first but not the second message.
@@ -138,18 +134,18 @@ class LogSegmentTest {
     val seg = createSegment(40)
     var offset = 40
     for (_ <- 0 until 30) {
-      val ms1 = messages(offset, "hello")
-      seg.append(offset, Message.NoTimestamp, -1L, ms1)
-      val ms2 = messages(offset + 1, "hello")
-      seg.append(offset + 1, Message.NoTimestamp, -1L, ms2)
+      val ms1 = records(offset, "hello")
+      seg.append(offset, Record.NO_TIMESTAMP, -1L, ms1)
+      val ms2 = records(offset + 1, "hello")
+      seg.append(offset + 1, Record.NO_TIMESTAMP, -1L, ms2)
       // check that we can read back both messages
       val read = seg.read(offset, None, 10000)
-      assertEquals(List(ms1.head, ms2.head), read.messageSet.toList)
+      assertEquals(List(ms1.deepIterator.next(), ms2.deepIterator.next()), read.records.deepIterator.asScala.toList)
       // now truncate off the last message
       seg.truncateTo(offset + 1)
       val read2 = seg.read(offset, None, 10000)
-      assertEquals(1, read2.messageSet.size)
-      assertEquals(ms1.head, read2.messageSet.head)
+      assertEquals(1, read2.records.deepIterator.asScala.size)
+      assertEquals(ms1.deepIterator.next(), read2.records.deepIterator.next())
       offset += 1
     }
   }
@@ -157,10 +153,10 @@ class LogSegmentTest {
   @Test
   def testReloadLargestTimestampAfterTruncation() {
     val numMessages = 30
-    val seg = createSegment(40, 2 * messages(0, "hello").sizeInBytes - 1)
+    val seg = createSegment(40, 2 * records(0, "hello").sizeInBytes - 1)
     var offset = 40
     for (_ <- 0 until numMessages) {
-      seg.append(offset, offset, offset, messages(offset, "hello"))
+      seg.append(offset, offset, offset, records(offset, "hello"))
       offset += 1
     }
     val expectedNumEntries = numMessages / 2 - 1
@@ -179,10 +175,10 @@ class LogSegmentTest {
   def testTruncateFull() {
     // test the case where we fully truncate the log
     val seg = createSegment(40)
-    seg.append(40, Message.NoTimestamp, -1L, messages(40, "hello", "there"))
+    seg.append(40, Record.NO_TIMESTAMP, -1L, records(40, "hello", "there"))
     seg.truncateTo(0)
     assertNull("Segment should be empty.", seg.read(0, None, 1024))
-    seg.append(40, Message.NoTimestamp, -1L, messages(40, "hello", "there"))
+    seg.append(40, Record.NO_TIMESTAMP, -1L, records(40, "hello", "there"))
   }
 
   /**
@@ -190,11 +186,11 @@ class LogSegmentTest {
    */
   @Test
   def testFindOffsetByTimestamp() {
-    val messageSize = messages(0, s"msg00").sizeInBytes
+    val messageSize = records(0, s"msg00").sizeInBytes
     val seg = createSegment(40, messageSize * 2 - 1)
     // Produce some messages
     for (i <- 40 until 50)
-      seg.append(i, i * 10, i, messages(i, s"msg$i"))
+      seg.append(i, i * 10, i, records(i, s"msg$i"))
 
     assertEquals(490, seg.largestTimestamp)
     // Search for an indexed timestamp
@@ -218,10 +214,10 @@ class LogSegmentTest {
   def testNextOffsetCalculation() {
     val seg = createSegment(40)
     assertEquals(40, seg.nextOffset)
-    seg.append(50, Message.NoTimestamp, -1L, messages(50, "hello", "there", "you"))
+    seg.append(50, Record.NO_TIMESTAMP, -1L, records(50, "hello", "there", "you"))
     assertEquals(53, seg.nextOffset())
   }
-  
+
   /**
    * Test that we can change the file suffixes for the log and index files
    */
@@ -236,7 +232,7 @@ class LogSegmentTest {
     assertTrue(seg.log.file.exists)
     assertTrue(seg.index.file.exists)
   }
-  
+
   /**
    * Create a segment with some data and an index. Then corrupt the index,
    * and recover the segment, the entries should all be readable.
@@ -245,12 +241,12 @@ class LogSegmentTest {
   def testRecoveryFixesCorruptIndex() {
     val seg = createSegment(0)
     for(i <- 0 until 100)
-      seg.append(i, Message.NoTimestamp, -1L, messages(i, i.toString))
+      seg.append(i, Record.NO_TIMESTAMP, -1L, records(i, i.toString))
     val indexFile = seg.index.file
     TestUtils.writeNonsenseToFile(indexFile, 5, indexFile.length.toInt)
     seg.recover(64*1024)
     for(i <- 0 until 100)
-      assertEquals(i, seg.read(i, Some(i + 1), 1024).messageSet.head.offset)
+      assertEquals(i, seg.read(i, Some(i + 1), 1024).records.deepIterator.next().offset)
   }
 
   /**
@@ -261,7 +257,7 @@ class LogSegmentTest {
   def testRecoveryFixesCorruptTimeIndex() {
     val seg = createSegment(0)
     for(i <- 0 until 100)
-      seg.append(i, i * 10, i, messages(i, i.toString))
+      seg.append(i, i * 10, i, records(i, i.toString))
     val timeIndexFile = seg.timeIndex.file
     TestUtils.writeNonsenseToFile(timeIndexFile, 5, timeIndexFile.length.toInt)
     seg.recover(64*1024)
@@ -271,7 +267,7 @@ class LogSegmentTest {
         assertEquals(i + 1, seg.findOffsetByTimestamp(i * 10 + 1).get.offset)
     }
   }
-  
+
   /**
    * Randomly corrupt a log a number of times and attempt recovery.
    */
@@ -281,13 +277,15 @@ class LogSegmentTest {
     for (_ <- 0 until 10) {
       val seg = createSegment(0)
       for(i <- 0 until messagesAppended)
-        seg.append(i, Message.NoTimestamp, -1L, messages(i, i.toString))
+        seg.append(i, Record.NO_TIMESTAMP, -1L, records(i, i.toString))
       val offsetToBeginCorruption = TestUtils.random.nextInt(messagesAppended)
       // start corrupting somewhere in the middle of the chosen record all the way to the end
-      val position = seg.log.searchForOffsetWithSize(offsetToBeginCorruption, 0)._1.position + TestUtils.random.nextInt(15)
-      TestUtils.writeNonsenseToFile(seg.log.file, position, seg.log.file.length.toInt - position)
+
+      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)
-      assertEquals("Should have truncated off bad messages.", (0 until offsetToBeginCorruption).toList, seg.log.map(_.offset).toList)
+      assertEquals("Should have truncated off bad messages.", (0 until offsetToBeginCorruption).toList, seg.log.shallowIterator.asScala.map(_.offset).toList)
       seg.delete()
     }
   }
@@ -304,12 +302,12 @@ class LogSegmentTest {
   @Test
   def testCreateWithInitFileSizeAppendMessage() {
     val seg = createSegment(40, false, 512*1024*1024, true)
-    val ms = messages(50, "hello", "there")
-    seg.append(50, Message.NoTimestamp, -1L, ms)
-    val ms2 = messages(60, "alpha", "beta")
-    seg.append(60, Message.NoTimestamp, -1L, ms2)
+    val ms = records(50, "hello", "there")
+    seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
+    val ms2 = records(60, "alpha", "beta")
+    seg.append(60, Record.NO_TIMESTAMP, -1L, ms2)
     val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None)
-    assertEquals(ms2.toList, read.messageSet.toList)
+    assertEquals(ms2.deepIterator.asScala.toList, read.records.deepIterator.asScala.toList)
   }
 
   /* create a segment with   pre allocate and clearly shut down*/
@@ -318,12 +316,12 @@ class LogSegmentTest {
     val tempDir = TestUtils.tempDir()
     val seg = new LogSegment(tempDir, 40, 10, 1000, 0, Time.SYSTEM, false, 512*1024*1024, true)
 
-    val ms = messages(50, "hello", "there")
-    seg.append(50, Message.NoTimestamp, -1L, ms)
-    val ms2 = messages(60, "alpha", "beta")
-    seg.append(60, Message.NoTimestamp, -1L, ms2)
+    val ms = records(50, "hello", "there")
+    seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
+    val ms2 = records(60, "alpha", "beta")
+    seg.append(60, Record.NO_TIMESTAMP, -1L, ms2)
     val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None)
-    assertEquals(ms2.toList, read.messageSet.toList)
+    assertEquals(ms2.deepIterator.asScala.toList, read.records.deepIterator.asScala.toList)
     val oldSize = seg.log.sizeInBytes()
     val oldPosition = seg.log.channel.position
     val oldFileSize = seg.log.file.length
@@ -336,7 +334,7 @@ class LogSegmentTest {
     segments += segReopen
 
     val readAgain = segReopen.read(startOffset = 55, maxSize = 200, maxOffset = None)
-    assertEquals(ms2.toList, readAgain.messageSet.toList)
+    assertEquals(ms2.deepIterator.asScala.toList, readAgain.records.deepIterator.asScala.toList)
     val size = segReopen.log.sizeInBytes()
     val position = segReopen.log.channel.position
     val fileSize = segReopen.log.file.length


[8/9] kafka git commit: KAFKA-4390; Replace MessageSet usage with client-side alternatives

Posted by jg...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/LogInputStream.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/LogInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/LogInputStream.java
index 4a4d569..a9af651 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/LogInputStream.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/LogInputStream.java
@@ -20,10 +20,14 @@ import java.io.IOException;
 
 /**
  * An abstraction between an underlying input stream and record iterators, a LogInputStream
- * returns only the shallow log entries, depending on {@link org.apache.kafka.common.record.RecordsIterator.DeepRecordsIterator}
- * for the deep iteration.
+ * returns only the shallow log entries, depending on {@link RecordsIterator.DeepRecordsIterator}
+ * for the deep iteration. The generic typing allows for implementations which present only
+ * a view of the log entries, which enables more efficient iteration when the record data is
+ * not actually needed. See for example {@link org.apache.kafka.common.record.FileLogInputStream.FileChannelLogEntry}
+ * in which the record is not brought into memory until needed.
+ * @param <T> Type parameter of the log entry
  */
-interface LogInputStream {
+interface LogInputStream<T extends LogEntry> {
 
     /**
      * Get the next log entry from the underlying input stream.
@@ -31,5 +35,5 @@ interface LogInputStream {
      * @return The next log entry or null if there is none
      * @throws IOException for any IO errors
      */
-    LogEntry nextEntry() throws IOException;
+    T nextEntry() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 65ccf98..b945062 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
@@ -12,197 +12,185 @@
  */
 package org.apache.kafka.common.record;
 
-import java.io.DataInputStream;
+import org.apache.kafka.common.record.ByteBufferLogInputStream.ByteBufferLogEntry;
+
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.channels.GatheringByteChannel;
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Iterator;
+import java.util.List;
 
 /**
- * A {@link Records} implementation backed by a ByteBuffer.
+ * A {@link Records} implementation backed by a ByteBuffer. This is used only for reading or
+ * modifying in-place an existing buffer of log entries. To create a new buffer see {@link MemoryRecordsBuilder},
+ * or one of the {@link #builder(ByteBuffer, byte, CompressionType, TimestampType) builder} variants.
  */
-public class MemoryRecords implements Records {
+public class MemoryRecords extends AbstractRecords {
 
     public final static MemoryRecords EMPTY = MemoryRecords.readableRecords(ByteBuffer.allocate(0));
 
-    private final static int WRITE_LIMIT_FOR_READABLE_ONLY = -1;
-
-    // the compressor used for appends-only
-    private final Compressor compressor;
-
-    // the write limit for writable buffer, which may be smaller than the buffer capacity
-    private final int writeLimit;
-
-    // the capacity of the initial buffer, which is only used for de-allocation of writable records
-    private final int initialCapacity;
-
     // the underlying buffer used for read; while the records are still writable it is null
     private ByteBuffer buffer;
-
-    // indicate if the memory records is writable or not (i.e. used for appends or read-only)
-    private boolean writable;
+    private int validBytes = -1;
 
     // Construct a writable memory records
-    private MemoryRecords(ByteBuffer buffer, CompressionType type, boolean writable, int writeLimit) {
-        this.writable = writable;
-        this.writeLimit = writeLimit;
-        this.initialCapacity = buffer.capacity();
-        if (this.writable) {
-            this.buffer = null;
-            this.compressor = new Compressor(buffer, type);
-        } else {
-            this.buffer = buffer;
-            this.compressor = null;
-        }
-    }
-
-    public static MemoryRecords emptyRecords(ByteBuffer buffer, CompressionType type, int writeLimit) {
-        return new MemoryRecords(buffer, type, true, writeLimit);
-    }
-
-    public static MemoryRecords emptyRecords(ByteBuffer buffer, CompressionType type) {
-        // use the buffer capacity as the default write limit
-        return emptyRecords(buffer, type, buffer.capacity());
+    private MemoryRecords(ByteBuffer buffer) {
+        this.buffer = buffer;
     }
 
-    public static MemoryRecords readableRecords(ByteBuffer buffer) {
-        return new MemoryRecords(buffer, CompressionType.NONE, false, WRITE_LIMIT_FOR_READABLE_ONLY);
+    @Override
+    public int sizeInBytes() {
+        return buffer.limit();
     }
 
-    /**
-     * Append the given record and offset to the buffer
-     */
-    public void append(long offset, Record record) {
-        if (!writable)
-            throw new IllegalStateException("Memory records is not writable");
-
-        int size = record.size();
-        compressor.putLong(offset);
-        compressor.putInt(size);
-        compressor.put(record.buffer());
-        compressor.recordWritten(size + Records.LOG_OVERHEAD);
-        record.buffer().rewind();
+    @Override
+    public long writeTo(GatheringByteChannel channel, long position, int length) throws IOException {
+        ByteBuffer dup = buffer.duplicate();
+        int pos = (int) position;
+        dup.position(pos);
+        dup.limit(pos + length);
+        return channel.write(dup);
     }
 
     /**
-     * Append a new record and offset to the buffer
-     * @return crc of the record
+     * Write all records to the given channel (including partial records).
+     * @param channel The channel to write to
+     * @return The number of bytes written
+     * @throws IOException For any IO errors writing to the channel
      */
-    public long append(long offset, long timestamp, byte[] key, byte[] value) {
-        if (!writable)
-            throw new IllegalStateException("Memory records is not writable");
-
-        int size = Record.recordSize(key, value);
-        compressor.putLong(offset);
-        compressor.putInt(size);
-        long crc = compressor.putRecord(timestamp, key, value);
-        compressor.recordWritten(size + Records.LOG_OVERHEAD);
-        return crc;
+    public int writeFullyTo(GatheringByteChannel channel) throws IOException {
+        buffer.mark();
+        int written = 0;
+        while (written < sizeInBytes())
+            written += channel.write(buffer);
+        buffer.reset();
+        return written;
     }
 
     /**
-     * Check if we have room for a new record containing the given key/value pair
-     *
-     * Note that the return value is based on the estimate of the bytes written to the compressor, which may not be
-     * accurate if compression is really used. When this happens, the following append may cause dynamic buffer
-     * re-allocation in the underlying byte buffer stream.
-     *
-     * There is an exceptional case when appending a single message whose size is larger than the batch size, the
-     * capacity will be the message size which is larger than the write limit, i.e. the batch size. In this case
-     * the checking should be based on the capacity of the initialized buffer rather than the write limit in order
-     * to accept this single record.
+     * The total number of bytes in this message set not including any partial, trailing messages. This
+     * may be smaller than what is returned by {@link #sizeInBytes()}.
+     * @return The number of valid bytes
      */
-    public boolean hasRoomFor(byte[] key, byte[] value) {
-        if (!this.writable)
-            return false;
+    public int validBytes() {
+        if (validBytes >= 0)
+            return validBytes;
 
-        return this.compressor.numRecordsWritten() == 0 ?
-            this.initialCapacity >= Records.LOG_OVERHEAD + Record.recordSize(key, value) :
-            this.writeLimit >= this.compressor.estimatedBytesWritten() + Records.LOG_OVERHEAD + Record.recordSize(key, value);
-    }
+        int bytes = 0;
+        Iterator<ByteBufferLogEntry> iterator = shallowIterator();
+        while (iterator.hasNext())
+            bytes += iterator.next().sizeInBytes();
 
-    public boolean isFull() {
-        return !this.writable || this.writeLimit <= this.compressor.estimatedBytesWritten();
+        this.validBytes = bytes;
+        return bytes;
     }
 
     /**
-     * Close this batch for no more appends
+     * Filter the records into the provided ByteBuffer.
+     * @param filter The filter function
+     * @param buffer The byte buffer to write the filtered records to
+     * @return A FilterResult with a summary of the output (for metrics)
      */
-    public void close() {
-        if (writable) {
-            // close the compressor to fill-in wrapper message metadata if necessary
-            compressor.close();
-
-            // flip the underlying buffer to be ready for reads
-            buffer = compressor.buffer();
-            buffer.flip();
-
-            // reset the writable flag
-            writable = false;
+    public FilterResult filterTo(LogEntryFilter filter, ByteBuffer buffer) {
+        long maxTimestamp = Record.NO_TIMESTAMP;
+        long shallowOffsetOfMaxTimestamp = -1L;
+        int messagesRead = 0;
+        int bytesRead = 0;
+        int messagesRetained = 0;
+        int bytesRetained = 0;
+
+        Iterator<ByteBufferLogEntry> shallowIterator = shallowIterator();
+        while (shallowIterator.hasNext()) {
+            ByteBufferLogEntry shallowEntry = shallowIterator.next();
+            bytesRead += shallowEntry.sizeInBytes();
+
+            // We use the absolute offset to decide whether to retain the message or not (this is handled by the
+            // deep iterator). Because of KAFKA-4298, we have to allow for the possibility that a previous version
+            // corrupted the log by writing a compressed message set with a wrapper magic value not matching the magic
+            // of the inner messages. This will be fixed as we recopy the messages to the destination buffer.
+
+            Record shallowRecord = shallowEntry.record();
+            byte shallowMagic = shallowRecord.magic();
+            boolean writeOriginalEntry = true;
+            List<LogEntry> retainedEntries = new ArrayList<>();
+
+            for (LogEntry deepEntry : shallowEntry) {
+                Record deepRecord = deepEntry.record();
+                messagesRead += 1;
+
+                if (filter.shouldRetain(deepEntry)) {
+                    // Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite
+                    // the corrupted entry with correct data.
+                    if (shallowMagic != deepRecord.magic())
+                        writeOriginalEntry = false;
+
+                    retainedEntries.add(deepEntry);
+                } else {
+                    writeOriginalEntry = false;
+                }
+            }
+
+            if (writeOriginalEntry) {
+                // There are no messages compacted out and no message format conversion, write the original message set back
+                shallowEntry.writeTo(buffer);
+                messagesRetained += retainedEntries.size();
+                bytesRetained += shallowEntry.sizeInBytes();
+
+                if (shallowRecord.timestamp() > maxTimestamp) {
+                    maxTimestamp = shallowRecord.timestamp();
+                    shallowOffsetOfMaxTimestamp = shallowEntry.offset();
+                }
+            } else if (!retainedEntries.isEmpty()) {
+                ByteBuffer slice = buffer.slice();
+                MemoryRecordsBuilder builder = builderWithEntries(slice, shallowRecord.timestampType(), shallowRecord.compressionType(),
+                        shallowRecord.timestamp(), retainedEntries);
+                MemoryRecords records = builder.build();
+                buffer.position(buffer.position() + slice.position());
+                messagesRetained += retainedEntries.size();
+                bytesRetained += records.sizeInBytes();
+
+                MemoryRecordsBuilder.RecordsInfo info = builder.info();
+                if (info.maxTimestamp > maxTimestamp) {
+                    maxTimestamp = info.maxTimestamp;
+                    shallowOffsetOfMaxTimestamp = info.shallowOffsetOfMaxTimestamp;
+                }
+            }
         }
+
+        return new FilterResult(messagesRead, bytesRead, messagesRetained, bytesRetained, maxTimestamp, shallowOffsetOfMaxTimestamp);
     }
 
     /**
-     * The size of this record set
+     * Get the byte buffer that backs this instance for reading.
      */
-    @Override
-    public int sizeInBytes() {
-        if (writable) {
-            return compressor.buffer().position();
-        } else {
-            return buffer.limit();
-        }
+    public ByteBuffer buffer() {
+        return buffer.duplicate();
     }
 
     @Override
-    public long writeTo(GatheringByteChannel channel, long offset, int length) throws IOException {
-        ByteBuffer dup = buffer.duplicate();
-        int position = (int) offset;
-        dup.position(position);
-        dup.limit(position + length);
-        return channel.write(dup);
+    public Iterator<ByteBufferLogEntry> shallowIterator() {
+        return RecordsIterator.shallowIterator(new ByteBufferLogInputStream(buffer.duplicate(), Integer.MAX_VALUE));
     }
 
-    /**
-     * The compression rate of this record set
-     */
-    public double compressionRate() {
-        if (compressor == null)
-            return 1.0;
-        else
-            return compressor.compressionRate();
+    @Override
+    public Iterator<LogEntry> deepIterator() {
+        return deepIterator(false);
     }
 
-    /**
-     * Return the capacity of the initial buffer, for writable records
-     * it may be different from the current buffer's capacity
-     */
-    public int initialCapacity() {
-        return this.initialCapacity;
+    public Iterator<LogEntry> deepIterator(boolean ensureMatchingMagic) {
+        return deepIterator(ensureMatchingMagic, Integer.MAX_VALUE);
     }
 
-    /**
-     * Get the byte buffer that backs this records instance for reading
-     */
-    public ByteBuffer buffer() {
-        if (writable)
-            throw new IllegalStateException("The memory records must not be writable any more before getting its underlying buffer");
-
-        return buffer.duplicate();
+    public Iterator<LogEntry> deepIterator(boolean ensureMatchingMagic, int maxMessageSize) {
+        return new RecordsIterator(new ByteBufferLogInputStream(buffer.duplicate(), maxMessageSize), false,
+                ensureMatchingMagic, maxMessageSize);
     }
 
     @Override
-    public Iterator<LogEntry> iterator() {
-        ByteBuffer input = this.buffer.duplicate();
-        if (writable)
-            // flip on a duplicate buffer for reading
-            input.flip();
-        return new RecordsIterator(new ByteBufferLogInputStream(input), false);
-    }
-    
-    @Override
     public String toString() {
-        Iterator<LogEntry> iter = iterator();
+        Iterator<LogEntry> iter = deepIterator();
         StringBuilder builder = new StringBuilder();
         builder.append('[');
         while (iter.hasNext()) {
@@ -214,16 +202,13 @@ public class MemoryRecords implements Records {
             builder.append("record=");
             builder.append(entry.record());
             builder.append(")");
+            if (iter.hasNext())
+                builder.append(", ");
         }
         builder.append(']');
         return builder.toString();
     }
 
-    /** Visible for testing */
-    public boolean isWritable() {
-        return writable;
-    }
-
     @Override
     public boolean equals(Object o) {
         if (this == o) return true;
@@ -232,7 +217,6 @@ public class MemoryRecords implements Records {
         MemoryRecords that = (MemoryRecords) o;
 
         return buffer.equals(that.buffer);
-
     }
 
     @Override
@@ -240,28 +224,153 @@ public class MemoryRecords implements Records {
         return buffer.hashCode();
     }
 
-    private static class ByteBufferLogInputStream implements LogInputStream {
-        private final DataInputStream stream;
-        private final ByteBuffer buffer;
+    public interface LogEntryFilter {
+        boolean shouldRetain(LogEntry entry);
+    }
 
-        private ByteBufferLogInputStream(ByteBuffer buffer) {
-            this.stream = new DataInputStream(new ByteBufferInputStream(buffer));
-            this.buffer = buffer;
+    public static class FilterResult {
+        public final int messagesRead;
+        public final int bytesRead;
+        public final int messagesRetained;
+        public final int bytesRetained;
+        public final long maxTimestamp;
+        public final long shallowOffsetOfMaxTimestamp;
+
+        public FilterResult(int messagesRead,
+                            int bytesRead,
+                            int messagesRetained,
+                            int bytesRetained,
+                            long maxTimestamp,
+                            long shallowOffsetOfMaxTimestamp) {
+            this.messagesRead = messagesRead;
+            this.bytesRead = bytesRead;
+            this.messagesRetained = messagesRetained;
+            this.bytesRetained = bytesRetained;
+            this.maxTimestamp = maxTimestamp;
+            this.shallowOffsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp;
         }
+    }
 
-        public LogEntry nextEntry() throws IOException {
-            long offset = stream.readLong();
-            int size = stream.readInt();
-            if (size < 0)
-                throw new IllegalStateException("Record with size " + size);
-
-            ByteBuffer slice = buffer.slice();
-            int newPos = buffer.position() + size;
-            if (newPos > buffer.limit())
-                return null;
-            buffer.position(newPos);
-            slice.limit(size);
-            return new LogEntry(offset, new Record(slice));
-        }
+    public static MemoryRecordsBuilder builder(ByteBuffer buffer,
+                                               CompressionType compressionType,
+                                               TimestampType timestampType,
+                                               int writeLimit) {
+        return new MemoryRecordsBuilder(buffer, Record.CURRENT_MAGIC_VALUE, compressionType, timestampType, 0L, System.currentTimeMillis(), writeLimit);
+    }
+
+    public static MemoryRecordsBuilder builder(ByteBuffer buffer,
+                                               byte magic,
+                                               CompressionType compressionType,
+                                               TimestampType timestampType,
+                                               long baseOffset,
+                                               long logAppendTime) {
+        return new MemoryRecordsBuilder(buffer, magic, compressionType, timestampType, baseOffset, logAppendTime, buffer.capacity());
+    }
+
+    public static MemoryRecordsBuilder builder(ByteBuffer buffer,
+                                               CompressionType compressionType,
+                                               TimestampType timestampType) {
+        // use the buffer capacity as the default write limit
+        return builder(buffer, compressionType, timestampType, buffer.capacity());
+    }
+
+    public static MemoryRecordsBuilder builder(ByteBuffer buffer,
+                                               byte magic,
+                                               CompressionType compressionType,
+                                               TimestampType timestampType) {
+        return builder(buffer, magic, compressionType, timestampType, 0L);
+    }
+
+    public static MemoryRecordsBuilder builder(ByteBuffer buffer,
+                                               byte magic,
+                                               CompressionType compressionType,
+                                               TimestampType timestampType,
+                                               long baseOffset) {
+        return builder(buffer, magic, compressionType, timestampType, baseOffset, System.currentTimeMillis());
+    }
+
+    public static MemoryRecords readableRecords(ByteBuffer buffer) {
+        return new MemoryRecords(buffer);
+    }
+
+    public static MemoryRecords withLogEntries(CompressionType compressionType, List<LogEntry> entries) {
+        return withLogEntries(TimestampType.CREATE_TIME, compressionType, System.currentTimeMillis(), entries);
+    }
+
+    public static MemoryRecords withLogEntries(LogEntry ... entries) {
+        return withLogEntries(CompressionType.NONE, Arrays.asList(entries));
+    }
+
+    public static MemoryRecords withRecords(CompressionType compressionType, long initialOffset, List<Record> records) {
+        return withRecords(initialOffset, TimestampType.CREATE_TIME, compressionType, System.currentTimeMillis(), records);
     }
+
+    public static MemoryRecords withRecords(Record ... records) {
+        return withRecords(CompressionType.NONE, 0L, Arrays.asList(records));
+    }
+
+    public static MemoryRecords withRecords(long initialOffset, Record ... records) {
+        return withRecords(CompressionType.NONE, initialOffset, Arrays.asList(records));
+    }
+
+    public static MemoryRecords withRecords(CompressionType compressionType, Record ... records) {
+        return withRecords(compressionType, 0L, Arrays.asList(records));
+    }
+
+    public static MemoryRecords withRecords(TimestampType timestampType, CompressionType compressionType, Record ... records) {
+        return withRecords(0L, timestampType, compressionType, System.currentTimeMillis(), Arrays.asList(records));
+    }
+
+    public static MemoryRecords withRecords(long initialOffset,
+                                            TimestampType timestampType,
+                                            CompressionType compressionType,
+                                            long logAppendTime,
+                                            List<Record> records) {
+        return withLogEntries(timestampType, compressionType, logAppendTime, buildLogEntries(initialOffset, records));
+    }
+
+    private static MemoryRecords withLogEntries(TimestampType timestampType,
+                                                CompressionType compressionType,
+                                                long logAppendTime,
+                                                List<LogEntry> entries) {
+        if (entries.isEmpty())
+            return MemoryRecords.EMPTY;
+        return builderWithEntries(timestampType, compressionType, logAppendTime, entries).build();
+    }
+
+    private static List<LogEntry> buildLogEntries(long initialOffset, List<Record> records) {
+        List<LogEntry> entries = new ArrayList<>();
+        for (Record record : records)
+            entries.add(LogEntry.create(initialOffset++, record));
+        return entries;
+    }
+
+    public static MemoryRecordsBuilder builderWithEntries(TimestampType timestampType,
+                                                          CompressionType compressionType,
+                                                          long logAppendTime,
+                                                          List<LogEntry> entries) {
+        ByteBuffer buffer = ByteBuffer.allocate(estimatedSize(compressionType, entries));
+        return builderWithEntries(buffer, timestampType, compressionType, logAppendTime, entries);
+    }
+
+    private static MemoryRecordsBuilder builderWithEntries(ByteBuffer buffer,
+                                                           TimestampType timestampType,
+                                                           CompressionType compressionType,
+                                                           long logAppendTime,
+                                                           List<LogEntry> entries) {
+        if (entries.isEmpty())
+            throw new IllegalArgumentException();
+
+        LogEntry firstEntry = entries.iterator().next();
+        long firstOffset = firstEntry.offset();
+        byte magic = firstEntry.record().magic();
+
+        MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compressionType, timestampType,
+                firstOffset, logAppendTime);
+        for (LogEntry entry : entries)
+            builder.append(entry);
+
+        return builder;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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
new file mode 100644
index 0000000..b90a9e6
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java
@@ -0,0 +1,461 @@
+/**
+ * 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.KafkaException;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.lang.reflect.Constructor;
+import java.nio.ByteBuffer;
+import java.util.zip.GZIPInputStream;
+import java.util.zip.GZIPOutputStream;
+
+/**
+ * This class is used to write new log data in memory, i.e. this is the write path for {@link MemoryRecords}.
+ * It transparently handles compression and exposes methods for appending new entries, possibly with message
+ * format conversion.
+ */
+public class MemoryRecordsBuilder {
+
+    static private final float COMPRESSION_RATE_DAMPING_FACTOR = 0.9f;
+    static private final float COMPRESSION_RATE_ESTIMATION_FACTOR = 1.05f;
+    static private final int COMPRESSION_DEFAULT_BUFFER_SIZE = 1024;
+
+    private static final float[] TYPE_TO_RATE;
+
+    static {
+        int maxTypeId = -1;
+        for (CompressionType type : CompressionType.values())
+            maxTypeId = Math.max(maxTypeId, type.id);
+        TYPE_TO_RATE = new float[maxTypeId + 1];
+        for (CompressionType type : CompressionType.values()) {
+            TYPE_TO_RATE[type.id] = type.rate;
+        }
+    }
+
+    // dynamically load the snappy and lz4 classes to avoid runtime dependency if we are not using compression
+    // caching constructors to avoid invoking of Class.forName method for each batch
+    private static MemoizingConstructorSupplier snappyOutputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
+        @Override
+        public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
+            return Class.forName("org.xerial.snappy.SnappyOutputStream")
+                .getConstructor(OutputStream.class, Integer.TYPE);
+        }
+    });
+
+    private static MemoizingConstructorSupplier lz4OutputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
+        @Override
+        public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
+            return Class.forName("org.apache.kafka.common.record.KafkaLZ4BlockOutputStream")
+                .getConstructor(OutputStream.class, Boolean.TYPE);
+        }
+    });
+
+    private static MemoizingConstructorSupplier snappyInputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
+        @Override
+        public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
+            return Class.forName("org.xerial.snappy.SnappyInputStream")
+                .getConstructor(InputStream.class);
+        }
+    });
+
+    private static MemoizingConstructorSupplier lz4InputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
+        @Override
+        public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
+            return Class.forName("org.apache.kafka.common.record.KafkaLZ4BlockInputStream")
+                .getConstructor(InputStream.class, Boolean.TYPE);
+        }
+    });
+
+    private final TimestampType timestampType;
+    private final CompressionType compressionType;
+    private final DataOutputStream appendStream;
+    private final ByteBufferOutputStream bufferStream;
+    private final byte magic;
+    private final int initPos;
+    private final long baseOffset;
+    private final long logAppendTime;
+    private final int writeLimit;
+    private final int initialCapacity;
+
+    private MemoryRecords builtRecords;
+    private long writtenUncompressed;
+    private long numRecords;
+    private float compressionRate;
+    private long maxTimestamp;
+    private long offsetOfMaxTimestamp;
+    private long lastOffset = -1;
+
+    public MemoryRecordsBuilder(ByteBuffer buffer,
+                                byte magic,
+                                CompressionType compressionType,
+                                TimestampType timestampType,
+                                long baseOffset,
+                                long logAppendTime,
+                                int writeLimit) {
+        this.magic = magic;
+        this.timestampType = timestampType;
+        this.compressionType = compressionType;
+        this.baseOffset = baseOffset;
+        this.logAppendTime = logAppendTime;
+        this.initPos = buffer.position();
+        this.numRecords = 0;
+        this.writtenUncompressed = 0;
+        this.compressionRate = 1;
+        this.maxTimestamp = Record.NO_TIMESTAMP;
+        this.writeLimit = writeLimit;
+        this.initialCapacity = buffer.capacity();
+
+        if (compressionType != CompressionType.NONE) {
+            // for compressed records, leave space for the header and the shallow message metadata
+            // and move the starting position to the value payload offset
+            buffer.position(initPos + Records.LOG_OVERHEAD + Record.recordOverhead(magic));
+        }
+
+        // create the stream
+        bufferStream = new ByteBufferOutputStream(buffer);
+        appendStream = wrapForOutput(bufferStream, compressionType, magic, COMPRESSION_DEFAULT_BUFFER_SIZE);
+    }
+
+    public ByteBuffer buffer() {
+        return bufferStream.buffer();
+    }
+
+    public int initialCapacity() {
+        return initialCapacity;
+    }
+
+    public double compressionRate() {
+        return compressionRate;
+    }
+
+    /**
+     * Close this builder and return the resulting buffer.
+     * @return The built log buffer
+     */
+    public MemoryRecords build() {
+        close();
+        return builtRecords;
+    }
+
+    /**
+     * Get the max timestamp and its offset. If the log append time is used, then the offset will
+     * be either the first offset in the set if no compression is used or the last offset otherwise.
+     * @return The max timestamp and its offset
+     */
+    public RecordsInfo info() {
+        if (timestampType == TimestampType.LOG_APPEND_TIME)
+            return new RecordsInfo(logAppendTime,  lastOffset);
+        else
+            return new RecordsInfo(maxTimestamp, compressionType == CompressionType.NONE ? offsetOfMaxTimestamp : lastOffset);
+    }
+
+    public void close() {
+        if (builtRecords != null)
+            return;
+
+        try {
+            appendStream.close();
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+
+        if (compressionType != CompressionType.NONE)
+            writerCompressedWrapperHeader();
+
+        ByteBuffer buffer = buffer().duplicate();
+        buffer.flip();
+        buffer.position(initPos);
+        builtRecords = MemoryRecords.readableRecords(buffer.slice());
+    }
+
+    private void writerCompressedWrapperHeader() {
+        ByteBuffer buffer = bufferStream.buffer();
+        int pos = buffer.position();
+        buffer.position(initPos);
+
+        int wrapperSize = pos - initPos - Records.LOG_OVERHEAD;
+        int writtenCompressed = wrapperSize - Record.recordOverhead(magic);
+        LogEntry.writeHeader(buffer, lastOffset, wrapperSize);
+
+        long timestamp = timestampType == TimestampType.LOG_APPEND_TIME ? logAppendTime : maxTimestamp;
+        Record.writeCompressedRecordHeader(buffer, magic, wrapperSize, timestamp, compressionType, timestampType);
+
+        buffer.position(pos);
+
+        // update the compression ratio
+        this.compressionRate = (float) writtenCompressed / this.writtenUncompressed;
+        TYPE_TO_RATE[compressionType.id] = TYPE_TO_RATE[compressionType.id] * COMPRESSION_RATE_DAMPING_FACTOR +
+            compressionRate * (1 - COMPRESSION_RATE_DAMPING_FACTOR);
+    }
+
+    /**
+     * Append a new record and offset to the buffer
+     * @param offset The absolute offset of the record in the log buffer
+     * @param timestamp The record timestamp
+     * @param key The record key
+     * @param value The record value
+     * @return crc of the record
+     */
+    public long append(long offset, long timestamp, byte[] key, byte[] value) {
+        try {
+            if (lastOffset > 0 && offset <= lastOffset)
+                throw new IllegalArgumentException(String.format("Illegal offset %s following previous offset %s (Offsets must increase monotonically).", offset, lastOffset));
+
+            int size = Record.recordSize(magic, key, value);
+            LogEntry.writeHeader(appendStream, toInnerOffset(offset), size);
+
+            if (timestampType == TimestampType.LOG_APPEND_TIME)
+                timestamp = logAppendTime;
+            long crc = Record.write(appendStream, magic, timestamp, key, value, CompressionType.NONE, timestampType);
+            recordWritten(offset, timestamp, size + Records.LOG_OVERHEAD);
+            return crc;
+        } catch (IOException e) {
+            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
+        }
+    }
+
+    /**
+     * Add the record, converting to the desired magic value if necessary.
+     * @param offset The offset of the record
+     * @param record The record to add
+     */
+    public void convertAndAppend(long offset, Record record) {
+        if (magic == record.magic()) {
+            append(offset, record);
+            return;
+        }
+
+        if (lastOffset > 0 && offset <= lastOffset)
+            throw new IllegalArgumentException(String.format("Illegal offset %s following previous offset %s (Offsets must increase monotonically).", offset, lastOffset));
+
+        try {
+            int size = record.convertedSize(magic);
+            LogEntry.writeHeader(appendStream, toInnerOffset(offset), size);
+            long timestamp = timestampType == TimestampType.LOG_APPEND_TIME ? logAppendTime : record.timestamp();
+            record.convertTo(appendStream, magic, timestamp, timestampType);
+            recordWritten(offset, timestamp, size + Records.LOG_OVERHEAD);
+        } catch (IOException e) {
+            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
+        }
+    }
+
+    /**
+     * Add a record without doing offset/magic validation (this should only be used in testing).
+     * @param offset The offset of the record
+     * @param record The record to add
+     */
+    public void appendUnchecked(long offset, Record record) {
+        try {
+            int size = record.sizeInBytes();
+            LogEntry.writeHeader(appendStream, toInnerOffset(offset), size);
+
+            ByteBuffer buffer = record.buffer().duplicate();
+            appendStream.write(buffer.array(), buffer.arrayOffset(), buffer.limit());
+
+            recordWritten(offset, record.timestamp(), size + Records.LOG_OVERHEAD);
+        } catch (IOException e) {
+            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
+        }
+    }
+
+    /**
+     * Append the given log entry. The entry's record must have a magic which matches the magic use to
+     * construct this builder and the offset must be greater than the last appended entry.
+     * @param entry The entry to append
+     */
+    public void append(LogEntry entry) {
+        append(entry.offset(), entry.record());
+    }
+
+    /**
+     * Add a record with a given offset. The record must have a magic which matches the magic use to
+     * construct this builder and the offset must be greater than the last appended entry.
+     * @param offset The offset of the record
+     * @param record The record to add
+     */
+    public void append(long offset, Record record) {
+        if (record.magic() != magic)
+            throw new IllegalArgumentException("Inner log entries must have matching magic values as the wrapper");
+        if (lastOffset > 0 && offset <= lastOffset)
+            throw new IllegalArgumentException(String.format("Illegal offset %s following previous offset %s (Offsets must increase monotonically).", offset, lastOffset));
+        appendUnchecked(offset, record);
+    }
+
+    private long toInnerOffset(long offset) {
+        // use relative offsets for compressed messages with magic v1
+        if (magic > 0 && compressionType != CompressionType.NONE)
+            return offset - baseOffset;
+        return offset;
+    }
+
+    private void recordWritten(long offset, long timestamp, int size) {
+        numRecords += 1;
+        writtenUncompressed += size;
+        lastOffset = offset;
+
+        if (timestamp > maxTimestamp) {
+            maxTimestamp = timestamp;
+            offsetOfMaxTimestamp = offset;
+        }
+    }
+
+    /**
+     * Get an estimate of the number of bytes written (based on the estimation factor hard-coded in {@link CompressionType}.
+     * @return The estimated number of bytes written
+     */
+    private int estimatedBytesWritten() {
+        if (compressionType == CompressionType.NONE) {
+            return buffer().position();
+        } else {
+            // estimate the written bytes to the underlying byte buffer based on uncompressed written bytes
+            return (int) (writtenUncompressed * TYPE_TO_RATE[compressionType.id] * COMPRESSION_RATE_ESTIMATION_FACTOR);
+        }
+    }
+
+    /**
+     * Check if we have room for a new record containing the given key/value pair
+     *
+     * Note that the return value is based on the estimate of the bytes written to the compressor, which may not be
+     * accurate if compression is really used. When this happens, the following append may cause dynamic buffer
+     * re-allocation in the underlying byte buffer stream.
+     *
+     * There is an exceptional case when appending a single message whose size is larger than the batch size, the
+     * capacity will be the message size which is larger than the write limit, i.e. the batch size. In this case
+     * the checking should be based on the capacity of the initialized buffer rather than the write limit in order
+     * to accept this single record.
+     */
+    public boolean hasRoomFor(byte[] key, byte[] value) {
+        return !isFull() && (numRecords == 0 ?
+                this.initialCapacity >= Records.LOG_OVERHEAD + Record.recordSize(magic, key, value) :
+                this.writeLimit >= estimatedBytesWritten() + Records.LOG_OVERHEAD + Record.recordSize(magic, key, value));
+    }
+
+    public boolean isClosed() {
+        return builtRecords != null;
+    }
+
+    public boolean isFull() {
+        return isClosed() || this.writeLimit <= estimatedBytesWritten();
+    }
+
+    public int sizeInBytes() {
+        return builtRecords != null ? builtRecords.sizeInBytes() : estimatedBytesWritten();
+    }
+
+    private static DataOutputStream wrapForOutput(ByteBufferOutputStream buffer, CompressionType type, byte messageVersion, int bufferSize) {
+        try {
+            switch (type) {
+                case NONE:
+                    return buffer;
+                case GZIP:
+                    return new DataOutputStream(new GZIPOutputStream(buffer, bufferSize));
+                case SNAPPY:
+                    try {
+                        OutputStream stream = (OutputStream) snappyOutputStreamSupplier.get().newInstance(buffer, bufferSize);
+                        return new DataOutputStream(stream);
+                    } catch (Exception e) {
+                        throw new KafkaException(e);
+                    }
+                case LZ4:
+                    try {
+                        OutputStream stream = (OutputStream) lz4OutputStreamSupplier.get().newInstance(buffer,
+                                messageVersion == Record.MAGIC_VALUE_V0);
+                        return new DataOutputStream(stream);
+                    } catch (Exception e) {
+                        throw new KafkaException(e);
+                    }
+                default:
+                    throw new IllegalArgumentException("Unknown compression type: " + type);
+            }
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    public static DataInputStream wrapForInput(ByteBufferInputStream buffer, CompressionType type, byte messageVersion) {
+        try {
+            switch (type) {
+                case NONE:
+                    return buffer;
+                case GZIP:
+                    return new DataInputStream(new GZIPInputStream(buffer));
+                case SNAPPY:
+                    try {
+                        InputStream stream = (InputStream) snappyInputStreamSupplier.get().newInstance(buffer);
+                        return new DataInputStream(stream);
+                    } catch (Exception e) {
+                        throw new KafkaException(e);
+                    }
+                case LZ4:
+                    try {
+                        InputStream stream = (InputStream) lz4InputStreamSupplier.get().newInstance(buffer,
+                                messageVersion == Record.MAGIC_VALUE_V0);
+                        return new DataInputStream(stream);
+                    } catch (Exception e) {
+                        throw new KafkaException(e);
+                    }
+                default:
+                    throw new IllegalArgumentException("Unknown compression type: " + type);
+            }
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    private interface ConstructorSupplier {
+        Constructor get() throws ClassNotFoundException, NoSuchMethodException;
+    }
+
+    // this code is based on Guava's @see{com.google.common.base.Suppliers.MemoizingSupplier}
+    private static class MemoizingConstructorSupplier {
+        final ConstructorSupplier delegate;
+        transient volatile boolean initialized;
+        transient Constructor value;
+
+        public MemoizingConstructorSupplier(ConstructorSupplier delegate) {
+            this.delegate = delegate;
+        }
+
+        public Constructor get() throws NoSuchMethodException, ClassNotFoundException {
+            if (!initialized) {
+                synchronized (this) {
+                    if (!initialized) {
+                        value = delegate.get();
+                        initialized = true;
+                    }
+                }
+            }
+            return value;
+        }
+    }
+
+    public static class RecordsInfo {
+        public final long maxTimestamp;
+        public final long shallowOffsetOfMaxTimestamp;
+
+        public RecordsInfo(long maxTimestamp,
+                           long shallowOffsetOfMaxTimestamp) {
+            this.maxTimestamp = maxTimestamp;
+            this.shallowOffsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 09cb80d..0c0fa3c 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
@@ -16,11 +16,15 @@
  */
 package org.apache.kafka.common.record;
 
-import java.nio.ByteBuffer;
-
+import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.utils.Crc32;
 import org.apache.kafka.common.utils.Utils;
 
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import static org.apache.kafka.common.utils.Utils.wrapNullable;
 
 /**
  * A record: a serialized key and value along with the associated CRC and other fields
@@ -53,7 +57,12 @@ public final class Record {
     /**
      * The amount of overhead bytes in a record
      */
-    public static final int RECORD_OVERHEAD = HEADER_SIZE + TIMESTAMP_LENGTH + KEY_SIZE_LENGTH + VALUE_SIZE_LENGTH;
+    public static final int RECORD_OVERHEAD_V0 = HEADER_SIZE + KEY_SIZE_LENGTH + VALUE_SIZE_LENGTH;
+
+    /**
+     * The amount of overhead bytes in a record
+     */
+    public static final int RECORD_OVERHEAD_V1 = HEADER_SIZE + TIMESTAMP_LENGTH + KEY_SIZE_LENGTH + VALUE_SIZE_LENGTH;
 
     /**
      * The "magic" values
@@ -80,11 +89,6 @@ public final class Record {
     public static final int TIMESTAMP_TYPE_ATTRIBUTE_OFFSET = 3;
 
     /**
-     * Compression code for uncompressed records
-     */
-    public static final int NO_COMPRESSION = 0;
-
-    /**
      * Timestamp value for records without a timestamp
      */
     public static final long NO_TIMESTAMP = -1L;
@@ -94,155 +98,20 @@ public final class Record {
     private final TimestampType wrapperRecordTimestampType;
 
     public Record(ByteBuffer buffer) {
-        this.buffer = buffer;
-        this.wrapperRecordTimestamp = null;
-        this.wrapperRecordTimestampType = null;
+        this(buffer, null, null);
     }
 
-    // Package private constructor for inner iteration.
-    Record(ByteBuffer buffer, Long wrapperRecordTimestamp, TimestampType wrapperRecordTimestampType) {
+    public Record(ByteBuffer buffer, Long wrapperRecordTimestamp, TimestampType wrapperRecordTimestampType) {
         this.buffer = buffer;
         this.wrapperRecordTimestamp = wrapperRecordTimestamp;
         this.wrapperRecordTimestampType = wrapperRecordTimestampType;
     }
 
     /**
-     * A constructor to create a LogRecord. If the record's compression type is not none, then
-     * its value payload should be already compressed with the specified type; the constructor
-     * would always write the value payload as is and will not do the compression itself.
-     *
-     * @param timestamp The timestamp of the record
-     * @param key The key of the record (null, if none)
-     * @param value The record value
-     * @param type The compression type used on the contents of the record (if any)
-     * @param valueOffset The offset into the payload array used to extract payload
-     * @param valueSize The size of the payload to use
-     */
-    public Record(long timestamp, byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
-        this(ByteBuffer.allocate(recordSize(key == null ? 0 : key.length,
-            value == null ? 0 : valueSize >= 0 ? valueSize : value.length - valueOffset)));
-        write(this.buffer, timestamp, key, value, type, valueOffset, valueSize);
-        this.buffer.rewind();
-    }
-
-    public Record(long timestamp, byte[] key, byte[] value, CompressionType type) {
-        this(timestamp, key, value, type, 0, -1);
-    }
-
-    public Record(long timestamp, byte[] value, CompressionType type) {
-        this(timestamp, null, value, type);
-    }
-
-    public Record(long timestamp, byte[] key, byte[] value) {
-        this(timestamp, key, value, CompressionType.NONE);
-    }
-
-    public Record(long timestamp, byte[] value) {
-        this(timestamp, null, value, CompressionType.NONE);
-    }
-
-    // Write a record to the buffer, if the record's compression type is none, then
-    // its value payload should be already compressed with the specified type
-    public static void write(ByteBuffer buffer, long timestamp, byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
-        // construct the compressor with compression type none since this function will not do any
-        //compression according to the input type, it will just write the record's payload as is
-        Compressor compressor = new Compressor(buffer, CompressionType.NONE);
-        try {
-            compressor.putRecord(timestamp, key, value, type, valueOffset, valueSize);
-        } finally {
-            compressor.close();
-        }
-    }
-
-    public static void write(Compressor compressor, long crc, byte attributes, long timestamp, byte[] key, byte[] value, int valueOffset, int valueSize) {
-        // write crc
-        compressor.putInt((int) (crc & 0xffffffffL));
-        // write magic value
-        compressor.putByte(CURRENT_MAGIC_VALUE);
-        // write attributes
-        compressor.putByte(attributes);
-        // write timestamp
-        compressor.putLong(timestamp);
-        // write the key
-        if (key == null) {
-            compressor.putInt(-1);
-        } else {
-            compressor.putInt(key.length);
-            compressor.put(key, 0, key.length);
-        }
-        // write the value
-        if (value == null) {
-            compressor.putInt(-1);
-        } else {
-            int size = valueSize >= 0 ? valueSize : (value.length - valueOffset);
-            compressor.putInt(size);
-            compressor.put(value, valueOffset, size);
-        }
-    }
-
-    public static int recordSize(byte[] key, byte[] value) {
-        return recordSize(key == null ? 0 : key.length, value == null ? 0 : value.length);
-    }
-
-    public static int recordSize(int keySize, int valueSize) {
-        return CRC_LENGTH + MAGIC_LENGTH + ATTRIBUTE_LENGTH + TIMESTAMP_LENGTH + KEY_SIZE_LENGTH + keySize + VALUE_SIZE_LENGTH + valueSize;
-    }
-
-    public ByteBuffer buffer() {
-        return this.buffer;
-    }
-
-    public static byte computeAttributes(CompressionType type) {
-        byte attributes = 0;
-        if (type.id > 0)
-            attributes = (byte) (attributes | (COMPRESSION_CODEC_MASK & type.id));
-        return attributes;
-    }
-
-    /**
-     * Compute the checksum of the record from the record contents
-     */
-    public static long computeChecksum(ByteBuffer buffer, int position, int size) {
-        Crc32 crc = new Crc32();
-        crc.update(buffer.array(), buffer.arrayOffset() + position, size);
-        return crc.getValue();
-    }
-
-    /**
-     * Compute the checksum of the record from the attributes, key and value payloads
-     */
-    public static long computeChecksum(long timestamp, byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
-        Crc32 crc = new Crc32();
-        crc.update(CURRENT_MAGIC_VALUE);
-        byte attributes = 0;
-        if (type.id > 0)
-            attributes = (byte) (attributes | (COMPRESSION_CODEC_MASK & type.id));
-        crc.update(attributes);
-        crc.updateLong(timestamp);
-        // update for the key
-        if (key == null) {
-            crc.updateInt(-1);
-        } else {
-            crc.updateInt(key.length);
-            crc.update(key, 0, key.length);
-        }
-        // update for the value
-        if (value == null) {
-            crc.updateInt(-1);
-        } else {
-            int size = valueSize >= 0 ? valueSize : (value.length - valueOffset);
-            crc.updateInt(size);
-            crc.update(value, valueOffset, size);
-        }
-        return crc.getValue();
-    }
-
-
-    /**
      * Compute the checksum of the record from the record contents
      */
     public long computeChecksum() {
-        return computeChecksum(buffer, MAGIC_OFFSET, buffer.limit() - MAGIC_OFFSET);
+        return Utils.computeChecksum(buffer, MAGIC_OFFSET, buffer.limit() - MAGIC_OFFSET);
     }
 
     /**
@@ -256,7 +125,15 @@ public final class Record {
      * Returns true if the crc stored with the record matches the crc computed off the record contents
      */
     public boolean isValid() {
-        return size() >= CRC_LENGTH && checksum() == computeChecksum();
+        return sizeInBytes() >= CRC_LENGTH && checksum() == computeChecksum();
+    }
+
+    public Long wrapperRecordTimestamp() {
+        return wrapperRecordTimestamp;
+    }
+
+    public TimestampType wrapperRecordTimestampType() {
+        return wrapperRecordTimestampType;
     }
 
     /**
@@ -264,9 +141,9 @@ public final class Record {
      */
     public void ensureValid() {
         if (!isValid()) {
-            if (size() < CRC_LENGTH)
+            if (sizeInBytes() < CRC_LENGTH)
                 throw new InvalidRecordException("Record is corrupt (crc could not be retrieved as the record is too "
-                        + "small, size = " + size() + ")");
+                        + "small, size = " + sizeInBytes() + ")");
             else
                 throw new InvalidRecordException("Record is corrupt (stored crc = " + checksum()
                         + ", computed crc = " + computeChecksum() + ")");
@@ -274,14 +151,17 @@ public final class Record {
     }
 
     /**
-     * The complete serialized size of this record in bytes (including crc, header attributes, etc)
+     * The complete serialized size of this record in bytes (including crc, header attributes, etc), but
+     * excluding the log overhead (offset and record size).
+     * @return the size in bytes
      */
-    public int size() {
+    public int sizeInBytes() {
         return buffer.limit();
     }
 
     /**
      * The length of the key in bytes
+     * @return the size in bytes of the key (0 if the key is null)
      */
     public int keySize() {
         if (magic() == MAGIC_VALUE_V0)
@@ -292,6 +172,7 @@ public final class Record {
 
     /**
      * Does the record have a key?
+     * @return true if so, false otherwise
      */
     public boolean hasKey() {
         return keySize() >= 0;
@@ -309,13 +190,23 @@ public final class Record {
 
     /**
      * The length of the value in bytes
+     * @return the size in bytes of the value (0 if the value is null)
      */
     public int valueSize() {
         return buffer.getInt(valueSizeOffset());
     }
 
     /**
-     * The magic version of this record
+     * Check whether the value field of this record is null.
+     * @return true if the value is null, false otherwise
+     */
+    public boolean hasNullValue() {
+        return valueSize() < 0;
+    }
+
+    /**
+     * The magic value (i.e. message format version) of this record
+     * @return the magic value
      */
     public byte magic() {
         return buffer.get(MAGIC_OFFSET);
@@ -323,6 +214,7 @@ public final class Record {
 
     /**
      * The attributes stored with this record
+     * @return the attributes
      */
     public byte attributes() {
         return buffer.get(ATTRIBUTES_OFFSET);
@@ -333,6 +225,8 @@ public final class Record {
      * 1. wrapperRecordTimestampType = null and wrapperRecordTimestamp is null - Uncompressed message, timestamp is in the message.
      * 2. wrapperRecordTimestampType = LOG_APPEND_TIME and WrapperRecordTimestamp is not null - Compressed message using LOG_APPEND_TIME
      * 3. wrapperRecordTimestampType = CREATE_TIME and wrapperRecordTimestamp is not null - Compressed message using CREATE_TIME
+     *
+     * @return the timestamp as determined above
      */
     public long timestamp() {
         if (magic() == MAGIC_VALUE_V0)
@@ -349,6 +243,8 @@ public final class Record {
 
     /**
      * The timestamp of the message.
+     * @return the timstamp type or {@link TimestampType#NO_TIMESTAMP_TYPE} if the magic is 0 or the message has
+     *   been up-converted.
      */
     public TimestampType timestampType() {
         if (magic() == 0)
@@ -366,36 +262,30 @@ public final class Record {
 
     /**
      * A ByteBuffer containing the value of this record
+     * @return the value or null if the value for this record is null
      */
     public ByteBuffer value() {
-        return sliceDelimited(valueSizeOffset());
+        return Utils.sizeDelimited(buffer, valueSizeOffset());
     }
 
     /**
      * A ByteBuffer containing the message key
+     * @return the buffer or null if the key for this record is null
      */
     public ByteBuffer key() {
         if (magic() == MAGIC_VALUE_V0)
-            return sliceDelimited(KEY_SIZE_OFFSET_V0);
+            return Utils.sizeDelimited(buffer, KEY_SIZE_OFFSET_V0);
         else
-            return sliceDelimited(KEY_SIZE_OFFSET_V1);
+            return Utils.sizeDelimited(buffer, KEY_SIZE_OFFSET_V1);
     }
 
     /**
-     * Read a size-delimited byte buffer starting at the given offset
+     * Get the underlying buffer backing this record instance.
+     *
+     * @return the buffer
      */
-    private ByteBuffer sliceDelimited(int start) {
-        int size = buffer.getInt(start);
-        if (size < 0) {
-            return null;
-        } else {
-            ByteBuffer b = buffer.duplicate();
-            b.position(start + 4);
-            b = b.slice();
-            b.limit(size);
-            b.rewind();
-            return b;
-        }
+    public ByteBuffer buffer() {
+        return this.buffer;
     }
 
     public String toString() {
@@ -434,4 +324,316 @@ public final class Record {
         return buffer.hashCode();
     }
 
+    /**
+     * Get the size of this record if converted to the given format.
+     *
+     * @param toMagic The target magic version to convert to
+     * @return The size in bytes after conversion
+     */
+    public int convertedSize(byte toMagic) {
+        return recordSize(toMagic, Math.max(0, keySize()), Math.max(0, valueSize()));
+    }
+
+    /**
+     * Convert this record to another message format.
+     *
+     * @param toMagic The target magic version to convert to
+     * @return A new record instance with a freshly allocated ByteBuffer.
+     */
+    public Record convert(byte toMagic) {
+        if (toMagic == magic())
+            return this;
+
+        ByteBuffer buffer = ByteBuffer.allocate(convertedSize(toMagic));
+        TimestampType timestampType = wrapperRecordTimestampType != null ?
+                wrapperRecordTimestampType : TimestampType.forAttributes(attributes());
+        convertTo(buffer, toMagic, timestamp(), timestampType);
+        buffer.rewind();
+        return new Record(buffer);
+    }
+
+    private void convertTo(ByteBuffer buffer, byte toMagic, long timestamp, TimestampType timestampType) {
+        if (compressionType() != CompressionType.NONE)
+            throw new IllegalArgumentException("Cannot use convertTo for deep conversion");
+
+        write(buffer, toMagic, timestamp, key(), value(), CompressionType.NONE, timestampType);
+    }
+
+    /**
+     * Convert this record to another message format and write the converted data to the provided outputs stream.
+     *
+     * @param out The output stream to write the converted data to
+     * @param toMagic The target magic version for conversion
+     * @param timestamp The timestamp to use in the converted record (for up-conversion)
+     * @param timestampType The timestamp type to use in the converted record (for up-conversion)
+     * @throws IOException for any IO errors writing the converted record.
+     */
+    public void convertTo(DataOutputStream out, byte toMagic, long timestamp, TimestampType timestampType) throws IOException {
+        if (compressionType() != CompressionType.NONE)
+            throw new IllegalArgumentException("Cannot use convertTo for deep conversion");
+
+        write(out, toMagic, timestamp, key(), value(), CompressionType.NONE, timestampType);
+    }
+
+    /**
+     * Create a new record instance. If the record's compression type is not none, then
+     * its value payload should be already compressed with the specified type; the constructor
+     * would always write the value payload as is and will not do the compression itself.
+     *
+     * @param magic The magic value to use
+     * @param timestamp The timestamp of the record
+     * @param key The key of the record (null, if none)
+     * @param value The record value
+     * @param compressionType The compression type used on the contents of the record (if any)
+     * @param timestampType The timestamp type to be used for this record
+     */
+    public static Record create(byte magic,
+                                long timestamp,
+                                byte[] key,
+                                byte[] value,
+                                CompressionType compressionType,
+                                TimestampType timestampType) {
+        int keySize = key == null ? 0 : key.length;
+        int valueSize = value == null ? 0 : value.length;
+        ByteBuffer buffer = ByteBuffer.allocate(recordSize(magic, keySize, valueSize));
+        write(buffer, magic, timestamp, wrapNullable(key), wrapNullable(value), compressionType, timestampType);
+        buffer.rewind();
+        return new Record(buffer);
+    }
+
+    public static Record create(long timestamp, byte[] key, byte[] value) {
+        return create(CURRENT_MAGIC_VALUE, timestamp, key, value, CompressionType.NONE, TimestampType.CREATE_TIME);
+    }
+
+    public static Record create(byte magic, long timestamp, byte[] key, byte[] value) {
+        return create(magic, timestamp, key, value, CompressionType.NONE, TimestampType.CREATE_TIME);
+    }
+
+    public static Record create(byte magic, TimestampType timestampType, long timestamp, byte[] key, byte[] value) {
+        return create(magic, timestamp, key, value, CompressionType.NONE, timestampType);
+    }
+
+    public static Record create(byte magic, long timestamp, byte[] value) {
+        return create(magic, timestamp, null, value, CompressionType.NONE, TimestampType.CREATE_TIME);
+    }
+
+    public static Record create(byte magic, byte[] key, byte[] value) {
+        return create(magic, NO_TIMESTAMP, key, value);
+    }
+
+    public static Record create(byte[] key, byte[] value) {
+        return create(NO_TIMESTAMP, key, value);
+    }
+
+    public static Record create(byte[] value) {
+        return create(CURRENT_MAGIC_VALUE, NO_TIMESTAMP, null, value, CompressionType.NONE, TimestampType.CREATE_TIME);
+    }
+
+    /**
+     * Write the header for a compressed record set in-place (i.e. assuming the compressed record data has already
+     * been written at the value offset in a wrapped record). This lets you dynamically create a compressed message
+     * set, and then go back later and fill in its size and CRC, which saves the need for copying to another buffer.
+     *
+     * @param buffer The buffer containing the compressed record data positioned at the first offset of the
+     * @param magic The magic value of the record set
+     * @param recordSize The size of the record (including record overhead)
+     * @param timestamp The timestamp of the wrapper record
+     * @param compressionType The compression type used
+     * @param timestampType The timestamp type of the wrapper record
+     */
+    public static void writeCompressedRecordHeader(ByteBuffer buffer,
+                                                   byte magic,
+                                                   int recordSize,
+                                                   long timestamp,
+                                                   CompressionType compressionType,
+                                                   TimestampType timestampType) {
+        int recordPosition = buffer.position();
+        int valueSize = recordSize - recordOverhead(magic);
+
+        // write the record header with a null value (the key is always null for the wrapper)
+        write(buffer, magic, timestamp, null, null, compressionType, timestampType);
+
+        // now fill in the value size
+        buffer.putInt(recordPosition + keyOffset(magic), valueSize);
+
+        // compute and fill the crc from the beginning of the message
+        long crc = Utils.computeChecksum(buffer, recordPosition + MAGIC_OFFSET, recordSize - MAGIC_OFFSET);
+        Utils.writeUnsignedInt(buffer, recordPosition + CRC_OFFSET, crc);
+    }
+
+    private static void write(ByteBuffer buffer,
+                              byte magic,
+                              long timestamp,
+                              ByteBuffer key,
+                              ByteBuffer value,
+                              CompressionType compressionType,
+                              TimestampType timestampType) {
+        try {
+            ByteBufferOutputStream out = new ByteBufferOutputStream(buffer);
+            write(out, magic, timestamp, key, value, compressionType, timestampType);
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    /**
+     * Write the record data with the given compression type and return the computed crc.
+     *
+     * @param out The output stream to write to
+     * @param magic The magic value to be used
+     * @param timestamp The timestamp of the record
+     * @param key The record key
+     * @param value The record value
+     * @param compressionType The compression type
+     * @param timestampType The timestamp type
+     * @return the computed CRC for this record.
+     * @throws IOException for any IO errors writing to the output stream.
+     */
+    public static long write(DataOutputStream out,
+                             byte magic,
+                             long timestamp,
+                             byte[] key,
+                             byte[] value,
+                             CompressionType compressionType,
+                             TimestampType timestampType) throws IOException {
+        return write(out, magic, timestamp, wrapNullable(key), wrapNullable(value), compressionType, timestampType);
+    }
+
+    private static long write(DataOutputStream out,
+                              byte magic,
+                              long timestamp,
+                              ByteBuffer key,
+                              ByteBuffer value,
+                              CompressionType compressionType,
+                              TimestampType timestampType) throws IOException {
+        byte attributes = computeAttributes(magic, compressionType, timestampType);
+        long crc = computeChecksum(magic, attributes, timestamp, key, value);
+        write(out, magic, crc, attributes, timestamp, key, value);
+        return crc;
+    }
+
+
+    /**
+     * Write a record using raw fields (without validation). This should only be used in testing.
+     */
+    public static void write(DataOutputStream out,
+                             byte magic,
+                             long crc,
+                             byte attributes,
+                             long timestamp,
+                             byte[] key,
+                             byte[] value) throws IOException {
+        write(out, magic, crc, attributes, timestamp, wrapNullable(key), wrapNullable(value));
+    }
+
+    // Write a record to the buffer, if the record's compression type is none, then
+    // its value payload should be already compressed with the specified type
+    private static void write(DataOutputStream out,
+                              byte magic,
+                              long crc,
+                              byte attributes,
+                              long timestamp,
+                              ByteBuffer key,
+                              ByteBuffer value) throws IOException {
+        if (magic != MAGIC_VALUE_V0 && magic != MAGIC_VALUE_V1)
+            throw new IllegalArgumentException("Invalid magic value " + magic);
+        if (timestamp < 0 && timestamp != NO_TIMESTAMP)
+            throw new IllegalArgumentException("Invalid message timestamp " + timestamp);
+
+        // write crc
+        out.writeInt((int) (crc & 0xffffffffL));
+        // write magic value
+        out.writeByte(magic);
+        // write attributes
+        out.writeByte(attributes);
+
+        // maybe write timestamp
+        if (magic > 0)
+            out.writeLong(timestamp);
+
+        // write the key
+        if (key == null) {
+            out.writeInt(-1);
+        } else {
+            int size = key.remaining();
+            out.writeInt(size);
+            out.write(key.array(), key.arrayOffset(), size);
+        }
+        // write the value
+        if (value == null) {
+            out.writeInt(-1);
+        } else {
+            int size = value.remaining();
+            out.writeInt(size);
+            out.write(value.array(), value.arrayOffset(), size);
+        }
+    }
+
+    public static int recordSize(byte[] key, byte[] value) {
+        return recordSize(CURRENT_MAGIC_VALUE, key, value);
+    }
+
+    public static int recordSize(byte magic, byte[] key, byte[] value) {
+        return recordSize(magic, key == null ? 0 : key.length, value == null ? 0 : value.length);
+    }
+
+    private static int recordSize(byte magic, int keySize, int valueSize) {
+        return recordOverhead(magic) + keySize + valueSize;
+    }
+
+    // visible only for testing
+    public static byte computeAttributes(byte magic, CompressionType type, TimestampType timestampType) {
+        byte attributes = 0;
+        if (type.id > 0)
+            attributes = (byte) (attributes | (COMPRESSION_CODEC_MASK & type.id));
+        if (magic > 0)
+            return timestampType.updateAttributes(attributes);
+        return attributes;
+    }
+
+    // visible only for testing
+    public static long computeChecksum(byte magic, byte attributes, long timestamp, byte[] key, byte[] value) {
+        return computeChecksum(magic, attributes, timestamp, wrapNullable(key), wrapNullable(value));
+    }
+
+    /**
+     * Compute the checksum of the record from the attributes, key and value payloads
+     */
+    private static long computeChecksum(byte magic, byte attributes, long timestamp, ByteBuffer key, ByteBuffer value) {
+        Crc32 crc = new Crc32();
+        crc.update(magic);
+        crc.update(attributes);
+        if (magic > 0)
+            crc.updateLong(timestamp);
+        // update for the key
+        if (key == null) {
+            crc.updateInt(-1);
+        } else {
+            int size = key.remaining();
+            crc.updateInt(size);
+            crc.update(key.array(), key.arrayOffset(), size);
+        }
+        // update for the value
+        if (value == null) {
+            crc.updateInt(-1);
+        } else {
+            int size = value.remaining();
+            crc.updateInt(size);
+            crc.update(value.array(), value.arrayOffset(), size);
+        }
+        return crc.getValue();
+    }
+
+    public static int recordOverhead(byte magic) {
+        if (magic == 0)
+            return RECORD_OVERHEAD_V0;
+        return RECORD_OVERHEAD_V1;
+    }
+
+    private static int keyOffset(byte magic) {
+        if (magic == 0)
+            return KEY_OFFSET_V0;
+        return KEY_OFFSET_V1;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/Records.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/Records.java b/clients/src/main/java/org/apache/kafka/common/record/Records.java
index 3bc043f..823d2b7 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/Records.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/Records.java
@@ -18,32 +18,74 @@ package org.apache.kafka.common.record;
 
 import java.io.IOException;
 import java.nio.channels.GatheringByteChannel;
+import java.util.Iterator;
 
 /**
- * A binary format which consists of a 4 byte size, an 8 byte offset, and the record bytes. See {@link MemoryRecords}
- * for the in-memory representation.
+ * Interface for accessing the records contained in a log. The log itself is represented as a sequence of log entries.
+ * Each log entry consists of an 8 byte offset, a 4 byte record size, and a "shallow" {@link Record record}.
+ * If the entry is not compressed, then each entry will have only the shallow record contained inside it. If it is
+ * compressed, the entry contains "deep" records, which are packed into the value field of the shallow record. To iterate
+ * over the shallow records, use {@link #shallowIterator()}; for the deep records, use {@link #deepIterator()}. Note
+ * that the deep iterator handles both compressed and non-compressed entries: if the entry is not compressed, the
+ * shallow record is returned; otherwise, the shallow record is decompressed and the deep entries are returned.
+ * See {@link MemoryRecords} for the in-memory representation and {@link FileRecords} for the on-disk representation.
  */
-public interface Records extends Iterable<LogEntry> {
+public interface Records {
 
-    int SIZE_LENGTH = 4;
+    int OFFSET_OFFSET = 0;
     int OFFSET_LENGTH = 8;
-    int LOG_OVERHEAD = SIZE_LENGTH + OFFSET_LENGTH;
+    int SIZE_OFFSET = OFFSET_OFFSET + OFFSET_LENGTH;
+    int SIZE_LENGTH = 4;
+    int LOG_OVERHEAD = SIZE_OFFSET + SIZE_LENGTH;
 
     /**
-     * The size of these records in bytes
-     * @return The size in bytes
+     * The size of these records in bytes.
+     * @return The size in bytes of the records
      */
     int sizeInBytes();
 
     /**
-     * Write the messages in this set to the given channel starting at the given offset byte.
+     * Write the contents of this buffer to a channel.
      * @param channel The channel to write to
-     * @param position The position within this record set to begin writing from
+     * @param position The position in the buffer to write from
      * @param length The number of bytes to write
-     * @return The number of bytes written to the channel (which may be fewer than requested)
-     * @throws IOException For any IO errors copying the
+     * @return The number of bytes written
+     * @throws IOException For any IO errors
      */
     long writeTo(GatheringByteChannel channel, long position, int length) throws IOException;
 
+    /**
+     * Get the shallow log entries in this log buffer. Note that the signature allows subclasses
+     * to return a more specific log entry type. This enables optimizations such as in-place offset
+     * assignment (see {@link ByteBufferLogInputStream.ByteBufferLogEntry}), and partial reading of
+     * record data (see {@link FileLogInputStream.FileChannelLogEntry#magic()}.
+     * @return An iterator over the shallow entries of the log
+     */
+    Iterator<? extends LogEntry> shallowIterator();
+
+    /**
+     * Get the deep log entries (i.e. descend into compressed message sets). For the deep records,
+     * there are fewer options for optimization since the data must be decompressed before it can be
+     * returned. Hence there is little advantage in allowing subclasses to return a more specific type
+     * as we do for {@link #shallowIterator()}.
+     * @return An iterator over the deep entries of the log
+     */
+    Iterator<LogEntry> deepIterator();
+
+    /**
+     * Check whether all shallow entries in this buffer have a certain magic value.
+     * @param magic The magic value to check
+     * @return true if all shallow entries have a matching magic value, false otherwise
+     */
+    boolean hasMatchingShallowMagic(byte magic);
+
+
+    /**
+     * Convert all entries in this buffer to the format passed as a parameter. Note that this requires
+     * deep iteration since all of the deep records must also be converted to the desired format.
+     * @param toMagic The magic value to convert to
+     * @return A Records (which may or may not be the same instance)
+     */
+    Records toMessageFormat(byte toMagic);
 
 }


[6/9] kafka git commit: KAFKA-4390; Replace MessageSet usage with client-side alternatives

Posted by jg...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java b/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java
index 6482529..551d820 100644
--- a/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java
@@ -35,18 +35,22 @@ import org.junit.runners.Parameterized.Parameters;
 @RunWith(value = Parameterized.class)
 public class RecordTest {
 
+    private byte magic;
     private long timestamp;
     private ByteBuffer key;
     private ByteBuffer value;
     private CompressionType compression;
+    private TimestampType timestampType;
     private Record record;
 
-    public RecordTest(long timestamp, byte[] key, byte[] value, CompressionType compression) {
+    public RecordTest(byte magic, long timestamp, byte[] key, byte[] value, CompressionType compression) {
+        this.magic = magic;
         this.timestamp = timestamp;
+        this.timestampType = TimestampType.CREATE_TIME;
         this.key = key == null ? null : ByteBuffer.wrap(key);
         this.value = value == null ? null : ByteBuffer.wrap(value);
         this.compression = compression;
-        this.record = new Record(timestamp, key, value, compression);
+        this.record = Record.create(magic, timestamp, key, value, compression, timestampType);
     }
 
     @Test
@@ -56,22 +60,33 @@ public class RecordTest {
         assertEquals(key, record.key());
         if (key != null)
             assertEquals(key.limit(), record.keySize());
-        assertEquals(Record.CURRENT_MAGIC_VALUE, record.magic());
+        assertEquals(magic, record.magic());
         assertEquals(value, record.value());
         if (value != null)
             assertEquals(value.limit(), record.valueSize());
+        if (magic > 0) {
+            assertEquals(timestamp, record.timestamp());
+            assertEquals(timestampType, record.timestampType());
+        } else {
+            assertEquals(Record.NO_TIMESTAMP, record.timestamp());
+            assertEquals(TimestampType.NO_TIMESTAMP_TYPE, record.timestampType());
+        }
     }
 
     @Test
     public void testChecksum() {
         assertEquals(record.checksum(), record.computeChecksum());
+
+        byte attributes = Record.computeAttributes(magic, this.compression, TimestampType.CREATE_TIME);
         assertEquals(record.checksum(), Record.computeChecksum(
-            this.timestamp,
-            this.key == null ? null : this.key.array(),
-            this.value == null ? null : this.value.array(),
-            this.compression, 0, -1));
+                magic,
+                attributes,
+                this.timestamp,
+                this.key == null ? null : this.key.array(),
+                this.value == null ? null : this.value.array()
+        ));
         assertTrue(record.isValid());
-        for (int i = Record.CRC_OFFSET + Record.CRC_LENGTH; i < record.size(); i++) {
+        for (int i = Record.CRC_OFFSET + Record.CRC_LENGTH; i < record.sizeInBytes(); i++) {
             Record copy = copyOf(record);
             copy.buffer().put(i, (byte) 69);
             assertFalse(copy.isValid());
@@ -85,7 +100,7 @@ public class RecordTest {
     }
 
     private Record copyOf(Record record) {
-        ByteBuffer buffer = ByteBuffer.allocate(record.size());
+        ByteBuffer buffer = ByteBuffer.allocate(record.sizeInBytes());
         record.buffer().put(buffer);
         buffer.rewind();
         record.buffer().rewind();
@@ -101,12 +116,13 @@ public class RecordTest {
     public static Collection<Object[]> data() {
         byte[] payload = new byte[1000];
         Arrays.fill(payload, (byte) 1);
-        List<Object[]> values = new ArrayList<Object[]>();
-        for (long timestamp : Arrays.asList(Record.NO_TIMESTAMP, 0L, 1L))
-            for (byte[] key : Arrays.asList(null, "".getBytes(), "key".getBytes(), payload))
-                for (byte[] value : Arrays.asList(null, "".getBytes(), "value".getBytes(), payload))
-                    for (CompressionType compression : CompressionType.values())
-                        values.add(new Object[] {timestamp, key, value, compression});
+        List<Object[]> values = new ArrayList<>();
+        for (byte magic : Arrays.asList(Record.MAGIC_VALUE_V0, Record.MAGIC_VALUE_V1))
+            for (long timestamp : Arrays.asList(Record.NO_TIMESTAMP, 0L, 1L))
+                for (byte[] key : Arrays.asList(null, "".getBytes(), "key".getBytes(), payload))
+                    for (byte[] value : Arrays.asList(null, "".getBytes(), "value".getBytes(), payload))
+                        for (CompressionType compression : CompressionType.values())
+                            values.add(new Object[] {magic, timestamp, key, value, compression});
         return values;
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/test/java/org/apache/kafka/common/record/SimpleRecordTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/record/SimpleRecordTest.java b/clients/src/test/java/org/apache/kafka/common/record/SimpleRecordTest.java
index aabadfe..427c743 100644
--- a/clients/src/test/java/org/apache/kafka/common/record/SimpleRecordTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/record/SimpleRecordTest.java
@@ -20,35 +20,29 @@ import org.junit.Test;
 
 import java.nio.ByteBuffer;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 public class SimpleRecordTest {
 
     /* This scenario can happen if the record size field is corrupt and we end up allocating a buffer that is too small */
-    @Test
+    @Test(expected = InvalidRecordException.class)
     public void testIsValidWithTooSmallBuffer() {
         ByteBuffer buffer = ByteBuffer.allocate(2);
         Record record = new Record(buffer);
         assertFalse(record.isValid());
-        try {
-            record.ensureValid();
-            fail("InvalidRecordException should have been thrown");
-        } catch (InvalidRecordException e) { }
+        record.ensureValid();
     }
 
-    @Test
+    @Test(expected = InvalidRecordException.class)
     public void testIsValidWithChecksumMismatch() {
         ByteBuffer buffer = ByteBuffer.allocate(4);
         // set checksum
         buffer.putInt(2);
         Record record = new Record(buffer);
         assertFalse(record.isValid());
-        try {
-            record.ensureValid();
-            fail("InvalidRecordException should have been thrown");
-        } catch (InvalidRecordException e) { }
+        record.ensureValid();
     }
 
     @Test
@@ -63,4 +57,40 @@ public class SimpleRecordTest {
         record.ensureValid();
     }
 
+    @Test
+    public void testConvertFromV0ToV1() {
+        byte[][] keys = new byte[][] {"a".getBytes(), "".getBytes(), null, "b".getBytes()};
+        byte[][] values = new byte[][] {"1".getBytes(), "".getBytes(), "2".getBytes(), null};
+
+        for (int i = 0; i < keys.length; i++) {
+            Record record = Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, keys[i], values[i]);
+            Record converted = record.convert(Record.MAGIC_VALUE_V1);
+
+            assertEquals(Record.MAGIC_VALUE_V1, converted.magic());
+            assertEquals(Record.NO_TIMESTAMP, converted.timestamp());
+            assertEquals(record.key(), converted.key());
+            assertEquals(record.value(), converted.value());
+            assertTrue(record.isValid());
+            assertEquals(record.convertedSize(Record.MAGIC_VALUE_V1), converted.sizeInBytes());
+        }
+    }
+
+    @Test
+    public void testConvertFromV1ToV0() {
+        byte[][] keys = new byte[][] {"a".getBytes(), "".getBytes(), null, "b".getBytes()};
+        byte[][] values = new byte[][] {"1".getBytes(), "".getBytes(), "2".getBytes(), null};
+
+        for (int i = 0; i < keys.length; i++) {
+            Record record = Record.create(Record.MAGIC_VALUE_V1, System.currentTimeMillis(), keys[i], values[i]);
+            Record converted = record.convert(Record.MAGIC_VALUE_V0);
+
+            assertEquals(Record.MAGIC_VALUE_V0, converted.magic());
+            assertEquals(Record.NO_TIMESTAMP, converted.timestamp());
+            assertEquals(record.key(), converted.key());
+            assertEquals(record.value(), converted.value());
+            assertTrue(record.isValid());
+            assertEquals(record.convertedSize(Record.MAGIC_VALUE_V0), converted.sizeInBytes());
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/test/java/org/apache/kafka/common/record/TimestampTypeTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/record/TimestampTypeTest.java b/clients/src/test/java/org/apache/kafka/common/record/TimestampTypeTest.java
new file mode 100644
index 0000000..4759715
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/record/TimestampTypeTest.java
@@ -0,0 +1,37 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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 static org.junit.Assert.assertEquals;
+
+public class TimestampTypeTest {
+
+    @Test
+    public void toAndFromAttributesCreateTime() {
+        byte attributes = TimestampType.CREATE_TIME.updateAttributes((byte) 0);
+        assertEquals(TimestampType.CREATE_TIME, TimestampType.forAttributes(attributes));
+    }
+
+    @Test
+    public void toAndFromAttributesLogAppendTime() {
+        byte attributes = TimestampType.LOG_APPEND_TIME.updateAttributes((byte) 0);
+        assertEquals(TimestampType.LOG_APPEND_TIME, TimestampType.forAttributes(attributes));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/test/java/org/apache/kafka/test/TestUtils.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/test/TestUtils.java b/clients/src/test/java/org/apache/kafka/test/TestUtils.java
index d3280e5..4e80b61 100644
--- a/clients/src/test/java/org/apache/kafka/test/TestUtils.java
+++ b/clients/src/test/java/org/apache/kafka/test/TestUtils.java
@@ -23,8 +23,10 @@ import org.apache.kafka.common.Node;
 import org.apache.kafka.common.PartitionInfo;
 import org.apache.kafka.common.record.CompressionType;
 import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.record.MemoryRecordsBuilder;
 import org.apache.kafka.common.record.Record;
 import org.apache.kafka.common.record.Records;
+import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.utils.Utils;
 
 import javax.xml.bind.DatatypeConverter;
@@ -35,6 +37,7 @@ import java.nio.file.Files;
 import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -46,6 +49,7 @@ import java.util.regex.Pattern;
 
 import static java.util.Arrays.asList;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -185,13 +189,13 @@ public class TestUtils {
     public static ByteBuffer partitionRecordsBuffer(final long offset, final CompressionType compressionType, final Record... records) {
         int bufferSize = 0;
         for (final Record record : records)
-            bufferSize += Records.LOG_OVERHEAD + record.size();
+            bufferSize += Records.LOG_OVERHEAD + record.sizeInBytes();
         final ByteBuffer buffer = ByteBuffer.allocate(bufferSize);
-        final MemoryRecords memoryRecords = MemoryRecords.emptyRecords(buffer, compressionType);
+        MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, compressionType, TimestampType.CREATE_TIME);
+        long nextOffset = offset;
         for (final Record record : records)
-            memoryRecords.append(offset, record);
-        memoryRecords.close();
-        return memoryRecords.buffer();
+            builder.append(nextOffset++, record);
+        return builder.build().buffer();
     }
 
     public static Properties producerConfig(final String bootstrapServers,
@@ -309,4 +313,22 @@ public class TestUtils {
             fail(clusterId + " cannot be converted back to UUID.");
         }
     }
+
+    /**
+     * Throw an exception if the two iterators are of differing lengths or contain
+     * different messages on their Nth element
+     */
+    public static <T> void checkEquals(Iterator<T> s1, Iterator<T> s2) {
+        while (s1.hasNext() && s2.hasNext())
+            assertEquals(s1.next(), s2.next());
+        assertFalse("Iterators have uneven length--first has more", s1.hasNext());
+        assertFalse("Iterators have uneven length--second has more", s2.hasNext());
+    }
+
+    public static <T> List<T> toList(Iterator<T> iterator) {
+        List<T> res = new ArrayList<>();
+        while (iterator.hasNext())
+            res.add(iterator.next());
+        return res;
+    }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/api/ApiVersion.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/ApiVersion.scala b/core/src/main/scala/kafka/api/ApiVersion.scala
index 895c1b1..4052639 100644
--- a/core/src/main/scala/kafka/api/ApiVersion.scala
+++ b/core/src/main/scala/kafka/api/ApiVersion.scala
@@ -17,7 +17,7 @@
 
 package kafka.api
 
-import kafka.message.Message
+import org.apache.kafka.common.record.Record
 
 /**
  * This class contains the different Kafka versions.
@@ -87,54 +87,54 @@ sealed trait ApiVersion extends Ordered[ApiVersion] {
 // Keep the IDs in order of versions
 case object KAFKA_0_8_0 extends ApiVersion {
   val version: String = "0.8.0.X"
-  val messageFormatVersion: Byte = Message.MagicValue_V0
+  val messageFormatVersion: Byte = Record.MAGIC_VALUE_V0
   val id: Int = 0
 }
 
 case object KAFKA_0_8_1 extends ApiVersion {
   val version: String = "0.8.1.X"
-  val messageFormatVersion: Byte = Message.MagicValue_V0
+  val messageFormatVersion: Byte = Record.MAGIC_VALUE_V0
   val id: Int = 1
 }
 
 case object KAFKA_0_8_2 extends ApiVersion {
   val version: String = "0.8.2.X"
-  val messageFormatVersion: Byte = Message.MagicValue_V0
+  val messageFormatVersion: Byte = Record.MAGIC_VALUE_V0
   val id: Int = 2
 }
 
 case object KAFKA_0_9_0 extends ApiVersion {
   val version: String = "0.9.0.X"
-  val messageFormatVersion: Byte = Message.MagicValue_V0
+  val messageFormatVersion: Byte = Record.MAGIC_VALUE_V0
   val id: Int = 3
 }
 
 case object KAFKA_0_10_0_IV0 extends ApiVersion {
   val version: String = "0.10.0-IV0"
-  val messageFormatVersion: Byte = Message.MagicValue_V1
+  val messageFormatVersion: Byte = Record.MAGIC_VALUE_V1
   val id: Int = 4
 }
 
 case object KAFKA_0_10_0_IV1 extends ApiVersion {
   val version: String = "0.10.0-IV1"
-  val messageFormatVersion: Byte = Message.MagicValue_V1
+  val messageFormatVersion: Byte = Record.MAGIC_VALUE_V1
   val id: Int = 5
 }
 
 case object KAFKA_0_10_1_IV0 extends ApiVersion {
   val version: String = "0.10.1-IV0"
-  val messageFormatVersion: Byte = Message.MagicValue_V1
+  val messageFormatVersion: Byte = Record.MAGIC_VALUE_V1
   val id: Int = 6
 }
 
 case object KAFKA_0_10_1_IV1 extends ApiVersion {
   val version: String = "0.10.1-IV1"
-  val messageFormatVersion: Byte = Message.MagicValue_V1
+  val messageFormatVersion: Byte = Record.MAGIC_VALUE_V1
   val id: Int = 7
 }
 
 case object KAFKA_0_10_1_IV2 extends ApiVersion {
   val version: String = "0.10.1-IV2"
-  val messageFormatVersion: Byte = Message.MagicValue_V1
+  val messageFormatVersion: Byte = Record.MAGIC_VALUE_V1
   val id: Int = 8
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 7e52a91..9eb92cd 100755
--- a/core/src/main/scala/kafka/cluster/Partition.scala
+++ b/core/src/main/scala/kafka/cluster/Partition.scala
@@ -25,7 +25,6 @@ import kafka.log.LogConfig
 import kafka.server._
 import kafka.metrics.KafkaMetricsGroup
 import kafka.controller.KafkaController
-import kafka.message.ByteBufferMessageSet
 import java.io.IOException
 import java.util.concurrent.locks.ReentrantReadWriteLock
 
@@ -34,6 +33,7 @@ import org.apache.kafka.common.protocol.Errors
 
 import scala.collection.JavaConverters._
 import com.yammer.metrics.core.Gauge
+import org.apache.kafka.common.record.MemoryRecords
 import org.apache.kafka.common.requests.PartitionState
 import org.apache.kafka.common.utils.Time
 
@@ -190,7 +190,7 @@ class Partition(val topic: String,
       allReplicas.foreach(replica => getOrCreateReplica(replica))
       val newInSyncReplicas = partitionStateInfo.isr.asScala.map(r => getOrCreateReplica(r)).toSet
       // remove assigned replicas that have been removed by the controller
-      (assignedReplicas().map(_.brokerId) -- allReplicas).foreach(removeReplica(_))
+      (assignedReplicas().map(_.brokerId) -- allReplicas).foreach(removeReplica)
       inSyncReplicas = newInSyncReplicas
       leaderEpoch = partitionStateInfo.leaderEpoch
       zkVersion = partitionStateInfo.zkVersion
@@ -440,7 +440,7 @@ class Partition(val topic: String,
     laggingReplicas
   }
 
-  def appendMessagesToLeader(messages: ByteBufferMessageSet, requiredAcks: Int = 0) = {
+  def appendRecordsToLeader(records: MemoryRecords, requiredAcks: Int = 0) = {
     val (info, leaderHWIncremented) = inReadLock(leaderIsrUpdateLock) {
       val leaderReplicaOpt = leaderReplicaIfLocal()
       leaderReplicaOpt match {
@@ -455,7 +455,7 @@ class Partition(val topic: String,
               .format(topic, partitionId, inSyncSize, minIsr))
           }
 
-          val info = log.append(messages, assignOffsets = true)
+          val info = log.append(records, assignOffsets = true)
           // 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
@@ -480,7 +480,7 @@ class Partition(val topic: String,
       newLeaderAndIsr, controllerEpoch, zkVersion)
 
     if(updateSucceeded) {
-      replicaManager.recordIsrChange(new TopicAndPartition(topic, partitionId))
+      replicaManager.recordIsrChange(TopicAndPartition(topic, partitionId))
       inSyncReplicas = newIsr
       zkVersion = newVersion
       trace("ISR updated to [%s] and zkVersion updated to [%d]".format(newIsr.mkString(","), zkVersion))

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala
index c47efb7..f702b9d 100644
--- a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala
+++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala
@@ -17,14 +17,16 @@
 
 package kafka.consumer
 
-import kafka.api.{OffsetRequest, Request, FetchRequestBuilder, FetchResponsePartitionData}
+import kafka.api.{FetchRequestBuilder, FetchResponsePartitionData, OffsetRequest, Request}
 import kafka.cluster.BrokerEndPoint
 import kafka.message.ByteBufferMessageSet
-import kafka.server.{PartitionFetchState, AbstractFetcherThread}
+import kafka.server.{AbstractFetcherThread, PartitionFetchState}
 import kafka.common.{ErrorMapping, TopicAndPartition}
+
 import scala.collection.Map
 import ConsumerFetcherThread._
 import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.record.MemoryRecords
 
 class ConsumerFetcherThread(name: String,
                             val config: ConsumerConfig,
@@ -81,7 +83,7 @@ class ConsumerFetcherThread(name: String,
       case OffsetRequest.LargestTimeString => OffsetRequest.LatestTime
       case _ => OffsetRequest.LatestTime
     }
-    val topicAndPartition = new TopicAndPartition(topicPartition.topic, topicPartition.partition)
+    val topicAndPartition = TopicAndPartition(topicPartition.topic, topicPartition.partition)
     val newOffset = simpleConsumer.earliestOrLatestOffset(topicAndPartition, startTimestamp, Request.OrdinaryConsumerId)
     val pti = partitionMap(topicPartition)
     pti.resetFetchOffset(newOffset)
@@ -123,7 +125,7 @@ object ConsumerFetcherThread {
 
   class PartitionData(val underlying: FetchResponsePartitionData) extends AbstractFetcherThread.PartitionData {
     def errorCode: Short = underlying.error
-    def toByteBufferMessageSet: ByteBufferMessageSet = underlying.messages.asInstanceOf[ByteBufferMessageSet]
+    def toRecords: MemoryRecords = underlying.messages.asInstanceOf[ByteBufferMessageSet].asRecords
     def highWatermark: Long = underlying.hw
     def exception: Option[Throwable] =
       if (errorCode == ErrorMapping.NoError) None else Some(ErrorMapping.exceptionFor(errorCode))

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala
index 0c53345..db40482 100644
--- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala
+++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala
@@ -458,14 +458,14 @@ class GroupCoordinator(val brokerId: Int,
   def handleFetchOffsets(groupId: String,
                          partitions: Seq[TopicPartition]): Map[TopicPartition, OffsetFetchResponse.PartitionData] = {
     if (!isActive.get) {
-      partitions.map { case topicPartition =>
+      partitions.map { topicPartition =>
         (topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code))}.toMap
     } else if (!isCoordinatorForGroup(groupId)) {
       debug("Could not fetch offsets for group %s (not group coordinator).".format(groupId))
-      partitions.map { case topicPartition =>
+      partitions.map { topicPartition =>
         (topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.NOT_COORDINATOR_FOR_GROUP.code))}.toMap
     } else if (isCoordinatorLoadingInProgress(groupId)) {
-      partitions.map { case topicPartition =>
+      partitions.map { topicPartition =>
         (topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.GROUP_LOAD_IN_PROGRESS.code))}.toMap
     } else {
       // return offsets blindly regardless the current group state since the group may be using

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala
index e55bcaa..a97b527 100644
--- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala
+++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala
@@ -17,38 +17,31 @@
 
 package kafka.coordinator
 
-import org.apache.kafka.common.TopicPartition
-import org.apache.kafka.common.protocol.Errors
-import org.apache.kafka.common.protocol.types.{ArrayOf, Field, Schema, Struct}
-import org.apache.kafka.common.protocol.types.Type.STRING
-import org.apache.kafka.common.protocol.types.Type.NULLABLE_STRING
-import org.apache.kafka.common.protocol.types.Type.INT32
-import org.apache.kafka.common.protocol.types.Type.INT64
-import org.apache.kafka.common.protocol.types.Type.BYTES
-import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
-import org.apache.kafka.common.requests.OffsetFetchResponse
-import org.apache.kafka.common.utils.Utils
-import org.apache.kafka.common.utils.Time
-import org.apache.kafka.clients.consumer.ConsumerRecord
-import kafka.utils._
-import kafka.common._
-import kafka.message._
-import kafka.log.FileMessageSet
-import kafka.metrics.KafkaMetricsGroup
-import kafka.common.TopicAndPartition
-import kafka.common.MessageFormatter
-import kafka.server.ReplicaManager
-
-import scala.collection._
 import java.io.PrintStream
 import java.nio.ByteBuffer
-import java.util.concurrent.atomic.AtomicBoolean
 import java.util.concurrent.TimeUnit
+import java.util.concurrent.atomic.AtomicBoolean
 import java.util.concurrent.locks.ReentrantLock
 
 import com.yammer.metrics.core.Gauge
 import kafka.api.{ApiVersion, KAFKA_0_10_1_IV0}
+import kafka.common.{MessageFormatter, TopicAndPartition, _}
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.ReplicaManager
 import kafka.utils.CoreUtils.inLock
+import kafka.utils._
+import org.apache.kafka.clients.consumer.ConsumerRecord
+import org.apache.kafka.common.TopicPartition
+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.ProduceResponse.PartitionResponse
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.collection.JavaConverters._
+import scala.collection._
 
 class GroupMetadataManager(val brokerId: Int,
                            val interBrokerProtocolVersion: ApiVersion,
@@ -57,6 +50,8 @@ class GroupMetadataManager(val brokerId: Int,
                            zkUtils: ZkUtils,
                            time: Time) extends Logging with KafkaMetricsGroup {
 
+  private val compressionType: CompressionType = CompressionType.forId(config.offsetsTopicCompressionCodec.codec)
+
   private val groupMetadataCache = new Pool[String, GroupMetadata]
 
   /* lock protecting access to loading and owned partition sets */
@@ -135,13 +130,11 @@ class GroupMetadataManager(val brokerId: Int,
     }
   }
 
-
   def prepareStoreGroup(group: GroupMetadata,
                         groupAssignment: Map[String, Array[Byte]],
                         responseCallback: Errors => Unit): Option[DelayedStore] = {
-    val magicValueAndTimestampOpt = getMessageFormatVersionAndTimestamp(partitionFor(group.groupId))
-    magicValueAndTimestampOpt match {
-      case Some((magicValue, timestamp)) =>
+    getMagicAndTimestamp(partitionFor(group.groupId)) match {
+      case Some((magicValue, timestampType, timestamp)) =>
         val groupMetadataValueVersion = {
           if (interBrokerProtocolVersion < KAFKA_0_10_1_IV0)
             0.toShort
@@ -149,17 +142,12 @@ class GroupMetadataManager(val brokerId: Int,
             GroupMetadataManager.CURRENT_GROUP_VALUE_SCHEMA_VERSION
         }
 
-        val message = new Message(
-          key = GroupMetadataManager.groupMetadataKey(group.groupId),
-          bytes = GroupMetadataManager.groupMetadataValue(group, groupAssignment, version = groupMetadataValueVersion),
-          timestamp = timestamp,
-          magicValue = magicValue)
+        val record = Record.create(magicValue, timestampType, timestamp,
+          GroupMetadataManager.groupMetadataKey(group.groupId),
+          GroupMetadataManager.groupMetadataValue(group, groupAssignment, version = groupMetadataValueVersion))
 
         val groupMetadataPartition = new TopicPartition(Topic.GroupMetadataTopicName, partitionFor(group.groupId))
-
-        val groupMetadataMessageSet = Map(groupMetadataPartition ->
-          new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, message))
-
+        val groupMetadataRecords = Map(groupMetadataPartition -> MemoryRecords.withRecords(timestampType, compressionType, record))
         val generationId = group.generationId
 
         // set the callback function to insert the created group into cache after log append completed
@@ -212,7 +200,7 @@ class GroupMetadataManager(val brokerId: Int,
 
           responseCallback(responseError)
         }
-        Some(DelayedStore(groupMetadataMessageSet, putCacheCallback))
+        Some(DelayedStore(groupMetadataRecords, putCacheCallback))
 
       case None =>
         responseCallback(Errors.NOT_COORDINATOR_FOR_GROUP)
@@ -222,11 +210,11 @@ class GroupMetadataManager(val brokerId: Int,
 
   def store(delayedStore: DelayedStore) {
     // call replica manager to append the group message
-    replicaManager.appendMessages(
+    replicaManager.appendRecords(
       config.offsetCommitTimeoutMs.toLong,
       config.offsetCommitRequiredAcks,
       true, // allow appending to internal offset topic
-      delayedStore.messageSet,
+      delayedStore.partitionRecords,
       delayedStore.callback)
   }
 
@@ -244,22 +232,17 @@ class GroupMetadataManager(val brokerId: Int,
     }
 
     // construct the message set to append
-    val magicValueAndTimestampOpt = getMessageFormatVersionAndTimestamp(partitionFor(group.groupId))
-    magicValueAndTimestampOpt match {
-      case Some((magicValue, timestamp)) =>
-        val messages = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) =>
-          new Message(
-            key = GroupMetadataManager.offsetCommitKey(group.groupId, topicAndPartition.topic, topicAndPartition.partition),
-            bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata),
-            timestamp = timestamp,
-            magicValue = magicValue
-          )
+    getMagicAndTimestamp(partitionFor(group.groupId)) match {
+      case Some((magicValue, timestampType, timestamp)) =>
+        val records = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) =>
+          Record.create(magicValue, timestampType, timestamp,
+            GroupMetadataManager.offsetCommitKey(group.groupId, topicAndPartition.topic, topicAndPartition.partition),
+            GroupMetadataManager.offsetCommitValue(offsetAndMetadata))
         }.toSeq
 
         val offsetTopicPartition = new TopicPartition(Topic.GroupMetadataTopicName, partitionFor(group.groupId))
 
-        val offsetsAndMetadataMessageSet = Map(offsetTopicPartition ->
-          new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*))
+        val entries = Map(offsetTopicPartition -> MemoryRecords.withRecords(timestampType, compressionType, records:_*))
 
         // set the callback function to insert offsets into cache after log append completed
         def putCacheCallback(responseStatus: Map[TopicPartition, PartitionResponse]) {
@@ -330,7 +313,7 @@ class GroupMetadataManager(val brokerId: Int,
           group.prepareOffsetCommit(offsetMetadata)
         }
 
-        Some(DelayedStore(offsetsAndMetadataMessageSet, putCacheCallback))
+        Some(DelayedStore(entries, putCacheCallback))
 
       case None =>
         val commitStatus = offsetMetadata.map { case (topicAndPartition, offsetAndMetadata) =>
@@ -412,28 +395,30 @@ class GroupMetadataManager(val brokerId: Int,
 
             while (currOffset < getHighWatermark(offsetsPartition) && !shuttingDown.get()) {
               buffer.clear()
-              val messages = log.read(currOffset, config.loadBufferSize, minOneMessage = true).messageSet.asInstanceOf[FileMessageSet]
-              messages.readInto(buffer, 0)
-              val messageSet = new ByteBufferMessageSet(buffer)
-              messageSet.foreach { msgAndOffset =>
-                require(msgAndOffset.message.key != null, "Offset entry key should not be null")
-                val baseKey = GroupMetadataManager.readMessageKey(msgAndOffset.message.key)
+              val fileRecords = log.read(currOffset, config.loadBufferSize, minOneMessage = true).records.asInstanceOf[FileRecords]
+              fileRecords.readInto(buffer, 0)
+
+              MemoryRecords.readableRecords(buffer).deepIterator.asScala.foreach { entry =>
+                val record = entry.record
+
+                require(record.hasKey, "Offset entry key should not be null")
+                val baseKey = GroupMetadataManager.readMessageKey(record.key)
 
                 if (baseKey.isInstanceOf[OffsetKey]) {
                   // load offset
                   val key = baseKey.key.asInstanceOf[GroupTopicPartition]
-                  if (msgAndOffset.message.payload == null) {
+                  if (record.hasNullValue) {
                     loadedOffsets.remove(key)
                     removedOffsets.add(key)
                   } else {
-                    val value = GroupMetadataManager.readOffsetMessageValue(msgAndOffset.message.payload)
+                    val value = GroupMetadataManager.readOffsetMessageValue(record.value)
                     loadedOffsets.put(key, value)
                     removedOffsets.remove(key)
                   }
                 } else {
                   // load group metadata
                   val groupId = baseKey.key.asInstanceOf[String]
-                  val groupMetadata = GroupMetadataManager.readGroupMessageValue(groupId, msgAndOffset.message.payload)
+                  val groupMetadata = GroupMetadataManager.readGroupMessageValue(groupId, record.value)
                   if (groupMetadata != null) {
                     trace(s"Loaded group metadata for group ${groupMetadata.groupId} with generation ${groupMetadata.generationId}")
                     removedGroups.remove(groupId)
@@ -444,7 +429,7 @@ class GroupMetadataManager(val brokerId: Int,
                   }
                 }
 
-                currOffset = msgAndOffset.nextOffset
+                currOffset = entry.nextOffset
               }
             }
 
@@ -467,8 +452,8 @@ class GroupMetadataManager(val brokerId: Int,
 
             removedGroups.foreach { groupId =>
               if (groupMetadataCache.contains(groupId))
-                throw new IllegalStateException(s"Unexpected unload of active group ${groupId} while " +
-                  s"loading partition ${topicPartition}")
+                throw new IllegalStateException(s"Unexpected unload of active group $groupId while " +
+                  s"loading partition $topicPartition")
             }
 
             if (!shuttingDown.get())
@@ -572,15 +557,15 @@ class GroupMetadataManager(val brokerId: Int,
       }
 
       val offsetsPartition = partitionFor(groupId)
-      getMessageFormatVersionAndTimestamp(offsetsPartition) match {
-        case Some((magicValue, timestamp)) =>
+      getMagicAndTimestamp(offsetsPartition) match {
+        case Some((magicValue, timestampType, timestamp)) =>
           val partitionOpt = replicaManager.getPartition(Topic.GroupMetadataTopicName, offsetsPartition)
           partitionOpt.foreach { partition =>
             val appendPartition = TopicAndPartition(Topic.GroupMetadataTopicName, offsetsPartition)
             val tombstones = expiredOffsets.map { case (topicPartition, offsetAndMetadata) =>
               trace(s"Removing expired offset and metadata for $groupId, $topicPartition: $offsetAndMetadata")
               val commitKey = GroupMetadataManager.offsetCommitKey(groupId, topicPartition.topic, topicPartition.partition)
-              new Message(bytes = null, key = commitKey, timestamp = timestamp, magicValue = magicValue)
+              Record.create(magicValue, timestampType, timestamp, commitKey, null)
             }.toBuffer
             trace(s"Marked ${expiredOffsets.size} offsets in $appendPartition for deletion.")
 
@@ -590,8 +575,7 @@ class GroupMetadataManager(val brokerId: Int,
               // Append the tombstone messages to the partition. It is okay if the replicas don't receive these (say,
               // if we crash or leaders move) since the new leaders will still expire the consumers with heartbeat and
               // retry removing this group.
-              tombstones += new Message(bytes = null, key = GroupMetadataManager.groupMetadataKey(group.groupId),
-                timestamp = timestamp, magicValue = magicValue)
+              tombstones += Record.create(magicValue, timestampType, timestamp, GroupMetadataManager.groupMetadataKey(group.groupId), null)
               trace(s"Group $groupId removed from the metadata cache and marked for deletion in $appendPartition.")
             }
 
@@ -599,7 +583,7 @@ class GroupMetadataManager(val brokerId: Int,
               try {
                 // do not need to require acks since even if the tombstone is lost,
                 // it will be appended again in the next purge cycle
-                partition.appendMessagesToLeader(new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, tombstones: _*))
+                partition.appendRecordsToLeader(MemoryRecords.withRecords(timestampType, compressionType, tombstones: _*))
                 offsetsRemoved += expiredOffsets.size
                 trace(s"Successfully appended ${tombstones.size} tombstones to $appendPartition for expired offsets and/or metadata for group $groupId")
               } catch {
@@ -663,16 +647,11 @@ class GroupMetadataManager(val brokerId: Int,
    * @param   partition  Partition of GroupMetadataTopic
    * @return  Option[(MessageFormatVersion, TimeStamp)] if replica is local, None otherwise
    */
-  private def getMessageFormatVersionAndTimestamp(partition: Int): Option[(Byte, Long)] = {
+  private def getMagicAndTimestamp(partition: Int): Option[(Byte, TimestampType, Long)] = {
     val groupMetadataTopicAndPartition = TopicAndPartition(Topic.GroupMetadataTopicName, partition)
-    replicaManager.getMessageFormatVersion(groupMetadataTopicAndPartition).map { messageFormatVersion =>
-      val timestamp = {
-        if (messageFormatVersion == Message.MagicValue_V0)
-          Message.NoTimestamp
-        else
-          time.milliseconds()
-      }
-      (messageFormatVersion, timestamp)
+    replicaManager.getMagicAndTimestampType(groupMetadataTopicAndPartition).map { case (messageFormatVersion, timestampType) =>
+      val timestamp = if (messageFormatVersion == Record.MAGIC_VALUE_V0) Record.NO_TIMESTAMP else time.milliseconds()
+      (messageFormatVersion, timestampType, timestamp)
     }
   }
 
@@ -964,7 +943,7 @@ object GroupMetadataManager {
    * @return an offset-metadata object from the message
    */
   def readOffsetMessageValue(buffer: ByteBuffer): OffsetAndMetadata = {
-    if(buffer == null) { // tombstone
+    if (buffer == null) { // tombstone
       null
     } else {
       val version = buffer.getShort
@@ -997,7 +976,7 @@ object GroupMetadataManager {
    * @return a group metadata object from the message
    */
   def readGroupMessageValue(groupId: String, buffer: ByteBuffer): GroupMetadata = {
-    if(buffer == null) { // tombstone
+    if (buffer == null) { // tombstone
       null
     } else {
       val version = buffer.getShort
@@ -1016,23 +995,22 @@ object GroupMetadataManager {
         group.leaderId = value.get(LEADER_KEY).asInstanceOf[String]
         group.protocol = value.get(PROTOCOL_KEY).asInstanceOf[String]
 
-        memberMetadataArray.foreach {
-          case memberMetadataObj =>
-            val memberMetadata = memberMetadataObj.asInstanceOf[Struct]
-            val memberId = memberMetadata.get(MEMBER_ID_KEY).asInstanceOf[String]
-            val clientId = memberMetadata.get(CLIENT_ID_KEY).asInstanceOf[String]
-            val clientHost = memberMetadata.get(CLIENT_HOST_KEY).asInstanceOf[String]
-            val sessionTimeout = memberMetadata.get(SESSION_TIMEOUT_KEY).asInstanceOf[Int]
-            val rebalanceTimeout = if (version == 0) sessionTimeout else memberMetadata.get(REBALANCE_TIMEOUT_KEY).asInstanceOf[Int]
+        memberMetadataArray.foreach { memberMetadataObj =>
+          val memberMetadata = memberMetadataObj.asInstanceOf[Struct]
+          val memberId = memberMetadata.get(MEMBER_ID_KEY).asInstanceOf[String]
+          val clientId = memberMetadata.get(CLIENT_ID_KEY).asInstanceOf[String]
+          val clientHost = memberMetadata.get(CLIENT_HOST_KEY).asInstanceOf[String]
+          val sessionTimeout = memberMetadata.get(SESSION_TIMEOUT_KEY).asInstanceOf[Int]
+          val rebalanceTimeout = if (version == 0) sessionTimeout else memberMetadata.get(REBALANCE_TIMEOUT_KEY).asInstanceOf[Int]
 
-            val subscription = Utils.toArray(memberMetadata.get(SUBSCRIPTION_KEY).asInstanceOf[ByteBuffer])
+          val subscription = Utils.toArray(memberMetadata.get(SUBSCRIPTION_KEY).asInstanceOf[ByteBuffer])
 
-            val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeout, sessionTimeout,
-              protocolType, List((group.protocol, subscription)))
+          val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeout, sessionTimeout,
+            protocolType, List((group.protocol, subscription)))
 
-            member.assignment = Utils.toArray(memberMetadata.get(ASSIGNMENT_KEY).asInstanceOf[ByteBuffer])
+          member.assignment = Utils.toArray(memberMetadata.get(ASSIGNMENT_KEY).asInstanceOf[ByteBuffer])
 
-            group.add(memberId, member)
+          group.add(memberId, member)
         }
 
         group
@@ -1087,7 +1065,7 @@ object GroupMetadataManager {
 
 }
 
-case class DelayedStore(messageSet: Map[TopicPartition, MessageSet],
+case class DelayedStore(partitionRecords: Map[TopicPartition, MemoryRecords],
                         callback: Map[TopicPartition, PartitionResponse] => Unit)
 
 case class GroupTopicPartition(group: String, topicPartition: TopicPartition) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/log/FileMessageSet.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/FileMessageSet.scala b/core/src/main/scala/kafka/log/FileMessageSet.scala
deleted file mode 100755
index 506f5b9..0000000
--- a/core/src/main/scala/kafka/log/FileMessageSet.scala
+++ /dev/null
@@ -1,445 +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._
-import java.nio.channels._
-import java.util.concurrent.atomic._
-import java.util.concurrent.TimeUnit
-
-import kafka.utils._
-import kafka.message._
-import kafka.common.KafkaException
-import kafka.metrics.{KafkaMetricsGroup, KafkaTimer}
-import org.apache.kafka.common.errors.CorruptRecordException
-import org.apache.kafka.common.record.FileRecords
-import org.apache.kafka.common.utils.Utils
-
-import scala.collection.mutable.ArrayBuffer
-
-/**
- * An on-disk message set. An optional start and end position can be applied to the message set
- * which will allow slicing a subset of the file.
- * @param file The file name for the underlying log data
- * @param channel the underlying file channel used
- * @param start A lower bound on the absolute position in the file from which the message set begins
- * @param end The upper bound on the absolute position in the file at which the message set ends
- * @param isSlice Should the start and end parameters be used for slicing?
- */
-@nonthreadsafe
-class FileMessageSet private[kafka](@volatile var file: File,
-                                    private[log] val channel: FileChannel,
-                                    private[log] val start: Int,
-                                    private[log] val end: Int,
-                                    isSlice: Boolean) extends MessageSet {
-  /* the size of the message set in bytes */
-  private val _size =
-    if(isSlice)
-      new AtomicInteger(end - start) // don't check the file size if this is just a slice view
-    else
-      new AtomicInteger(math.min(channel.size.toInt, end) - start)
-
-  /* if this is not a slice, update the file pointer to the end of the file */
-  if (!isSlice)
-    /* set the file position to the last byte in the file */
-    channel.position(math.min(channel.size.toInt, end))
-
-  /**
-   * Create a file message set with no slicing.
-   */
-  def this(file: File, channel: FileChannel) =
-    this(file, channel, start = 0, end = Int.MaxValue, isSlice = false)
-
-  /**
-   * Create a file message set with no slicing
-   */
-  def this(file: File) =
-    this(file, FileMessageSet.openChannel(file, mutable = true))
-
-  /**
-   * Create a file message set with no slicing, and with initFileSize and preallocate.
-   * For windows NTFS and some old LINUX file system, set preallocate to true and initFileSize
-   * with one value (for example 512 * 1024 *1024 ) can improve the kafka produce performance.
-   * If it's new file and preallocate is true, end will be set to 0.  Otherwise set to Int.MaxValue.
-   */
-  def this(file: File, fileAlreadyExists: Boolean, initFileSize: Int, preallocate: Boolean) =
-      this(file,
-        channel = FileMessageSet.openChannel(file, mutable = true, fileAlreadyExists, initFileSize, preallocate),
-        start = 0,
-        end = if (!fileAlreadyExists && preallocate) 0 else Int.MaxValue,
-        isSlice = false)
-
-  /**
-   * Create a file message set with mutable option
-   */
-  def this(file: File, mutable: Boolean) = this(file, FileMessageSet.openChannel(file, mutable))
-
-  /**
-   * Create a slice view of the file message set that begins and ends at the given byte offsets
-   */
-  def this(file: File, channel: FileChannel, start: Int, end: Int) =
-    this(file, channel, start, end, isSlice = true)
-
-  /**
-   * Return a message set which is a view into this set starting from the given position and with the given size limit.
-   *
-   * If the size is beyond the end of the file, the end will be based on the size of the file at the time of the read.
-   *
-   * If this message set is already sliced, the position will be taken relative to that slicing.
-   *
-   * @param position The start position to begin the read from
-   * @param size The number of bytes after the start position to include
-   *
-   * @return A sliced wrapper on this message set limited based on the given position and size
-   */
-  def read(position: Int, size: Int): FileMessageSet = {
-    if(position < 0)
-      throw new IllegalArgumentException("Invalid position: " + position)
-    if(size < 0)
-      throw new IllegalArgumentException("Invalid size: " + size)
-    new FileMessageSet(file,
-                       channel,
-                       start = this.start + position,
-                       end = {
-                         // Handle the integer overflow
-                         if (this.start + position + size < 0)
-                           sizeInBytes()
-                         else
-                           math.min(this.start + position + size, sizeInBytes())
-                       })
-  }
-
-  override def asRecords: FileRecords = new FileRecords(file, channel, start, end, isSlice)
-
-  /**
-   * Search forward for the file position of the last offset that is greater than or equal to the target offset
-   * and return its physical position and the size of the message (including log overhead) at the returned offset. If
-   * no such offsets are found, return null.
-   *
-   * @param targetOffset The offset to search for.
-   * @param startingPosition The starting position in the file to begin searching from.
-   */
-  def searchForOffsetWithSize(targetOffset: Long, startingPosition: Int): (OffsetPosition, Int) = {
-    var position = startingPosition
-    val buffer = ByteBuffer.allocate(MessageSet.LogOverhead)
-    val size = sizeInBytes()
-    while(position + MessageSet.LogOverhead < size) {
-      buffer.rewind()
-      channel.read(buffer, position)
-      if(buffer.hasRemaining)
-        throw new IllegalStateException("Failed to read complete buffer for targetOffset %d startPosition %d in %s"
-          .format(targetOffset, startingPosition, file.getAbsolutePath))
-      buffer.rewind()
-      val offset = buffer.getLong()
-      val messageSize = buffer.getInt()
-      if (messageSize < Message.MinMessageOverhead)
-        throw new IllegalStateException("Invalid message size: " + messageSize)
-      if (offset >= targetOffset)
-        return (OffsetPosition(offset, position), messageSize + MessageSet.LogOverhead)
-      position += MessageSet.LogOverhead + messageSize
-    }
-    null
-  }
-
-  /**
-   * Search forward for the message whose timestamp is greater than or equals to the target timestamp.
-   *
-   * @param targetTimestamp The timestamp to search for.
-   * @param startingPosition The starting position to search.
-   * @return The timestamp and offset of the message found. None, if no message is found.
-   */
-  def searchForTimestamp(targetTimestamp: Long, startingPosition: Int): Option[TimestampOffset] = {
-    val messagesToSearch = read(startingPosition, sizeInBytes)
-    for (messageAndOffset <- messagesToSearch) {
-      val message = messageAndOffset.message
-      if (message.timestamp >= targetTimestamp) {
-        // We found a message
-        message.compressionCodec match {
-          case NoCompressionCodec =>
-            return Some(TimestampOffset(messageAndOffset.message.timestamp, messageAndOffset.offset))
-          case _ =>
-            // Iterate over the inner messages to get the exact offset.
-            for (innerMessageAndOffset <- ByteBufferMessageSet.deepIterator(messageAndOffset)) {
-              val timestamp = innerMessageAndOffset.message.timestamp
-              if (timestamp >= targetTimestamp)
-                return Some(TimestampOffset(innerMessageAndOffset.message.timestamp, innerMessageAndOffset.offset))
-            }
-            throw new IllegalStateException(s"The message set (max timestamp = ${message.timestamp}, max offset = ${messageAndOffset.offset}" +
-                s" should contain target timestamp $targetTimestamp but it does not.")
-        }
-      }
-    }
-    None
-  }
-
-  /**
-   * Return the largest timestamp of the messages after a given position in this file message set.
-   * @param startingPosition The starting position.
-   * @return The largest timestamp of the messages after the given position.
-   */
-  def largestTimestampAfter(startingPosition: Int): TimestampOffset = {
-    var maxTimestamp = Message.NoTimestamp
-    var offsetOfMaxTimestamp = -1L
-    val messagesToSearch = read(startingPosition, Int.MaxValue)
-    for (messageAndOffset <- messagesToSearch) {
-      if (messageAndOffset.message.timestamp > maxTimestamp) {
-        maxTimestamp = messageAndOffset.message.timestamp
-        offsetOfMaxTimestamp = messageAndOffset.offset
-      }
-    }
-    TimestampOffset(maxTimestamp, offsetOfMaxTimestamp)
-  }
-
-  /**
-    * This method is called before we write messages to the socket using zero-copy transfer. We need to
-    * make sure all the messages in the message set have the expected magic value.
-    *
-    * @param expectedMagicValue the magic value expected
-    * @return true if all messages have expected magic value, false otherwise
-    */
-  override def isMagicValueInAllWrapperMessages(expectedMagicValue: Byte): Boolean = {
-    var location = start
-    val offsetAndSizeBuffer = ByteBuffer.allocate(MessageSet.LogOverhead)
-    val crcAndMagicByteBuffer = ByteBuffer.allocate(Message.CrcLength + Message.MagicLength)
-    while (location < end) {
-      offsetAndSizeBuffer.rewind()
-      channel.read(offsetAndSizeBuffer, location)
-      if (offsetAndSizeBuffer.hasRemaining)
-        return true
-      offsetAndSizeBuffer.rewind()
-      offsetAndSizeBuffer.getLong // skip offset field
-      val messageSize = offsetAndSizeBuffer.getInt
-      if (messageSize < Message.MinMessageOverhead)
-        throw new IllegalStateException("Invalid message size: " + messageSize)
-      crcAndMagicByteBuffer.rewind()
-      channel.read(crcAndMagicByteBuffer, location + MessageSet.LogOverhead)
-      if (crcAndMagicByteBuffer.get(Message.MagicOffset) != expectedMagicValue)
-        return false
-      location += (MessageSet.LogOverhead + messageSize)
-    }
-    true
-  }
-
-  /**
-   * Convert this message set to use the specified message format.
-   */
-  def toMessageFormat(toMagicValue: Byte): MessageSet = {
-    val offsets = new ArrayBuffer[Long]
-    val newMessages = new ArrayBuffer[Message]
-    this.foreach { messageAndOffset =>
-      val message = messageAndOffset.message
-      if (message.compressionCodec == NoCompressionCodec) {
-        newMessages += message.toFormatVersion(toMagicValue)
-        offsets += messageAndOffset.offset
-      } else {
-        // File message set only has shallow iterator. We need to do deep iteration here if needed.
-        val deepIter = ByteBufferMessageSet.deepIterator(messageAndOffset)
-        for (innerMessageAndOffset <- deepIter) {
-          newMessages += innerMessageAndOffset.message.toFormatVersion(toMagicValue)
-          offsets += innerMessageAndOffset.offset
-        }
-      }
-    }
-
-    if (sizeInBytes > 0 && newMessages.isEmpty) {
-      // This indicates that the message is too large. We just return all the bytes in the file message set.
-      this
-    } else {
-      // We use the offset seq to assign offsets so the offset of the messages does not change.
-      new ByteBufferMessageSet(
-        compressionCodec = this.headOption.map(_.message.compressionCodec).getOrElse(NoCompressionCodec),
-        offsetSeq = offsets,
-        newMessages: _*)
-    }
-  }
-
-  /**
-   * Get a shallow iterator over the messages in the set.
-   */
-  override def iterator: Iterator[MessageAndOffset] = iterator(Int.MaxValue)
-
-  /**
-   * Get an iterator over the messages in the set. We only do shallow iteration here.
-   * @param maxMessageSize A limit on allowable message size to avoid allocating unbounded memory.
-   * If we encounter a message larger than this we throw an InvalidMessageException.
-   * @return The iterator.
-   */
-  def iterator(maxMessageSize: Int): Iterator[MessageAndOffset] = {
-    new IteratorTemplate[MessageAndOffset] {
-      var location = start
-      val sizeOffsetLength = 12
-      val sizeOffsetBuffer = ByteBuffer.allocate(sizeOffsetLength)
-
-      override def makeNext(): MessageAndOffset = {
-        if(location + sizeOffsetLength >= end)
-          return allDone()
-
-        // read the size of the item
-        sizeOffsetBuffer.rewind()
-        channel.read(sizeOffsetBuffer, location)
-        if(sizeOffsetBuffer.hasRemaining)
-          return allDone()
-
-        sizeOffsetBuffer.rewind()
-        val offset = sizeOffsetBuffer.getLong()
-        val size = sizeOffsetBuffer.getInt()
-        if(size < Message.MinMessageOverhead || location + sizeOffsetLength + size > end)
-          return allDone()
-        if(size > maxMessageSize)
-          throw new CorruptRecordException("Message size exceeds the largest allowable message size (%d).".format(maxMessageSize))
-
-        // read the item itself
-        val buffer = ByteBuffer.allocate(size)
-        channel.read(buffer, location + sizeOffsetLength)
-        if(buffer.hasRemaining)
-          return allDone()
-        buffer.rewind()
-
-        // increment the location and return the item
-        location += size + sizeOffsetLength
-        MessageAndOffset(new Message(buffer), offset)
-      }
-    }
-  }
-
-  /**
-   * The number of bytes taken up by this file set
-   */
-  def sizeInBytes(): Int = _size.get()
-
-  /**
-   * Append these messages to the message set
-   */
-  def append(messages: ByteBufferMessageSet) {
-    val written = messages.writeFullyTo(channel)
-    _size.getAndAdd(written)
-  }
-
-  /**
-   * Commit all written data to the physical disk
-   */
-  def flush() = {
-    channel.force(true)
-  }
-
-  /**
-   * Close this message set
-   */
-  def close() {
-    flush()
-    trim()
-    channel.close()
-  }
-
-  /**
-   * Trim file when close or roll to next file
-   */
-  def trim() {
-    truncateTo(sizeInBytes())
-  }
-
-  /**
-   * Delete this message set from the filesystem
-   * @return True iff this message set was deleted.
-   */
-  def delete(): Boolean = {
-    CoreUtils.swallow(channel.close())
-    file.delete()
-  }
-
-  /**
-   * Truncate this file message set to the given size in bytes. Note that this API does no checking that the
-   * given size falls on a valid message boundary.
-   * In some versions of the JDK truncating to the same size as the file message set will cause an
-   * update of the files mtime, so truncate is only performed if the targetSize is smaller than the
-   * size of the underlying FileChannel.
-   * It is expected that no other threads will do writes to the log when this function is called.
-   * @param targetSize The size to truncate to. Must be between 0 and sizeInBytes.
-   * @return The number of bytes truncated off
-   */
-  def truncateTo(targetSize: Int): Int = {
-    val originalSize = sizeInBytes
-    if(targetSize > originalSize || targetSize < 0)
-      throw new KafkaException("Attempt to truncate log segment to " + targetSize + " bytes failed, " +
-                               " size of this log segment is " + originalSize + " bytes.")
-    if (targetSize < channel.size.toInt) {
-      channel.truncate(targetSize)
-      channel.position(targetSize)
-      _size.set(targetSize)
-    }
-    originalSize - targetSize
-  }
-
-  /**
-   * Read from the underlying file into the buffer starting at the given position
-   */
-  def readInto(buffer: ByteBuffer, relativePosition: Int): ByteBuffer = {
-    channel.read(buffer, relativePosition + this.start)
-    buffer.flip()
-    buffer
-  }
-
-  /**
-   * Rename the file that backs this message set
-   * @throws IOException if rename fails.
-   */
-  def renameTo(f: File) {
-    try Utils.atomicMoveWithFallback(file.toPath, f.toPath)
-    finally this.file = f
-  }
-
-}
-
-object FileMessageSet extends Logging
-{
-  //preserve the previous logger name after moving logger aspect from FileMessageSet to companion
-  override val loggerName = classOf[FileMessageSet].getName
-
-  /**
-   * Open a channel for the given file
-   * For windows NTFS and some old LINUX file system, set preallocate to true and initFileSize
-   * with one value (for example 512 * 1025 *1024 ) can improve the kafka produce performance.
-   * @param file File path
-   * @param mutable mutable
-   * @param fileAlreadyExists File already exists or not
-   * @param initFileSize The size used for pre allocate file, for example 512 * 1025 *1024
-   * @param preallocate Pre allocate file or not, gotten from configuration.
-   */
-  def openChannel(file: File, mutable: Boolean, fileAlreadyExists: Boolean = false, initFileSize: Int = 0, preallocate: Boolean = false): FileChannel = {
-    if (mutable) {
-      if (fileAlreadyExists)
-        new RandomAccessFile(file, "rw").getChannel()
-      else {
-        if (preallocate) {
-          val randomAccessFile = new RandomAccessFile(file, "rw")
-          randomAccessFile.setLength(initFileSize)
-          randomAccessFile.getChannel()
-        }
-        else
-          new RandomAccessFile(file, "rw").getChannel()
-      }
-    }
-    else
-      new FileInputStream(file).getChannel()
-  }
-}
-
-object LogFlushStats extends KafkaMetricsGroup {
-  val logFlushTimer = new KafkaTimer(newTimer("LogFlushRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS))
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 6acc8d2..d58a066 100644
--- a/core/src/main/scala/kafka/log/Log.scala
+++ b/core/src/main/scala/kafka/log/Log.scala
@@ -19,7 +19,6 @@ package kafka.log
 
 import kafka.api.KAFKA_0_10_0_IV0
 import kafka.utils._
-import kafka.message._
 import kafka.common._
 import kafka.metrics.KafkaMetricsGroup
 import kafka.server.{BrokerTopicStats, FetchDataInfo, LogOffsetMetadata}
@@ -29,16 +28,18 @@ import java.util.concurrent.atomic._
 import java.text.NumberFormat
 
 import org.apache.kafka.common.errors.{CorruptRecordException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException, UnsupportedForMessageFormatException}
-import org.apache.kafka.common.record.TimestampType
+import org.apache.kafka.common.record._
 import org.apache.kafka.common.requests.ListOffsetRequest
 
 import scala.collection.Seq
 import scala.collection.JavaConverters._
 import com.yammer.metrics.core.Gauge
 import org.apache.kafka.common.utils.{Time, Utils}
+import kafka.message.{BrokerCompressionCodec, CompressionCodec, NoCompressionCodec}
 
 object LogAppendInfo {
-  val UnknownLogAppendInfo = LogAppendInfo(-1, -1, Message.NoTimestamp, -1L, Message.NoTimestamp, NoCompressionCodec, NoCompressionCodec, -1, -1, offsetsMonotonic = false)
+  val UnknownLogAppendInfo = LogAppendInfo(-1, -1, Record.NO_TIMESTAMP, -1L, Record.NO_TIMESTAMP,
+    NoCompressionCodec, NoCompressionCodec, -1, -1, offsetsMonotonic = false)
 }
 
 /**
@@ -243,7 +244,7 @@ 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 swapSegment = new LogSegment(new FileMessageSet(file = swapFile),
+      val swapSegment = new LogSegment(FileRecords.open(swapFile),
                                        index = index,
                                        timeIndex = timeIndex,
                                        baseOffset = startOffset,
@@ -338,20 +339,20 @@ class Log(@volatile var dir: File,
    * 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 messages The message set to append
+   * @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
    * @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 append(messages: ByteBufferMessageSet, assignOffsets: Boolean = true): LogAppendInfo = {
-    val appendInfo = analyzeAndValidateMessageSet(messages)
+  def append(records: MemoryRecords, assignOffsets: Boolean = true): LogAppendInfo = {
+    val appendInfo = analyzeAndValidateRecords(records)
 
     // if we have any valid messages, append them to the log
     if (appendInfo.shallowCount == 0)
       return appendInfo
 
     // trim any invalid bytes or partial messages before appending it to the on-disk log
-    var validMessages = trimInvalidBytes(messages, appendInfo)
+    var validRecords = trimInvalidBytes(records, appendInfo)
 
     try {
       // they are valid, insert them in the log
@@ -363,20 +364,21 @@ class Log(@volatile var dir: File,
           appendInfo.firstOffset = offset.value
           val now = time.milliseconds
           val validateAndOffsetAssignResult = try {
-            validMessages.validateMessagesAndAssignOffsets(offset,
-                                                           now,
-                                                           appendInfo.sourceCodec,
-                                                           appendInfo.targetCodec,
-                                                           config.compact,
-                                                           config.messageFormatVersion.messageFormatVersion,
-                                                           config.messageTimestampType,
-                                                           config.messageTimestampDifferenceMaxMs)
+            LogValidator.validateMessagesAndAssignOffsets(validRecords,
+                                                          offset,
+                                                          now,
+                                                          appendInfo.sourceCodec,
+                                                          appendInfo.targetCodec,
+                                                          config.compact,
+                                                          config.messageFormatVersion.messageFormatVersion,
+                                                          config.messageTimestampType,
+                                                          config.messageTimestampDifferenceMaxMs)
           } catch {
             case e: IOException => throw new KafkaException("Error in validating messages while appending to log '%s'".format(name), e)
           }
-          validMessages = validateAndOffsetAssignResult.validatedMessages
+          validRecords = validateAndOffsetAssignResult.validatedRecords
           appendInfo.maxTimestamp = validateAndOffsetAssignResult.maxTimestamp
-          appendInfo.offsetOfMaxTimestamp = validateAndOffsetAssignResult.offsetOfMaxTimestamp
+          appendInfo.offsetOfMaxTimestamp = validateAndOffsetAssignResult.shallowOffsetOfMaxTimestamp
           appendInfo.lastOffset = offset.value - 1
           if (config.messageTimestampType == TimestampType.LOG_APPEND_TIME)
             appendInfo.logAppendTime = now
@@ -384,14 +386,14 @@ class Log(@volatile var dir: File,
           // re-validate message sizes if there's a possibility that they have changed (due to re-compression or message
           // format conversion)
           if (validateAndOffsetAssignResult.messageSizeMaybeChanged) {
-            for (messageAndOffset <- validMessages.shallowIterator) {
-              if (MessageSet.entrySize(messageAndOffset.message) > config.maxMessageSize) {
+            for (logEntry <- validRecords.shallowIterator.asScala) {
+              if (logEntry.sizeInBytes > config.maxMessageSize) {
                 // we record the original message set size instead of the trimmed size
                 // to be consistent with pre-compression bytesRejectedRate recording
-                BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(messages.sizeInBytes)
-                BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(messages.sizeInBytes)
+                BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(records.sizeInBytes)
+                BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(records.sizeInBytes)
                 throw new RecordTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d."
-                  .format(MessageSet.entrySize(messageAndOffset.message), config.maxMessageSize))
+                  .format(logEntry.sizeInBytes, config.maxMessageSize))
               }
             }
           }
@@ -399,28 +401,27 @@ class Log(@volatile var dir: File,
         } else {
           // we are taking the offsets we are given
           if (!appendInfo.offsetsMonotonic || appendInfo.firstOffset < nextOffsetMetadata.messageOffset)
-            throw new IllegalArgumentException("Out of order offsets found in " + messages)
+            throw new IllegalArgumentException("Out of order offsets found in " + records.deepIterator.asScala.map(_.offset))
         }
 
         // check messages set size may be exceed config.segmentSize
-        if (validMessages.sizeInBytes > config.segmentSize) {
+        if (validRecords.sizeInBytes > config.segmentSize) {
           throw new RecordBatchTooLargeException("Message set size is %d bytes which exceeds the maximum configured segment size of %d."
-            .format(validMessages.sizeInBytes, config.segmentSize))
+            .format(validRecords.sizeInBytes, config.segmentSize))
         }
 
         // maybe roll the log if this segment is full
-        val segment = maybeRoll(messagesSize = validMessages.sizeInBytes,
-                                maxTimestampInMessages = appendInfo.maxTimestamp)
+        val segment = maybeRoll(messagesSize = validRecords.sizeInBytes, maxTimestampInMessages = appendInfo.maxTimestamp)
 
         // now append to the log
         segment.append(firstOffset = appendInfo.firstOffset, largestTimestamp = appendInfo.maxTimestamp,
-          offsetOfLargestTimestamp = appendInfo.offsetOfMaxTimestamp, messages = validMessages)
+          shallowOffsetOfMaxTimestamp = appendInfo.offsetOfMaxTimestamp, records = validRecords)
 
         // increment the log end offset
         updateLogEndOffset(appendInfo.lastOffset + 1)
 
         trace("Appended message set to log %s with first offset: %d, next offset: %d, and messages: %s"
-          .format(this.name, appendInfo.firstOffset, nextOffsetMetadata.messageOffset, validMessages))
+          .format(this.name, appendInfo.firstOffset, nextOffsetMetadata.messageOffset, validRecords))
 
         if (unflushedMessages >= config.flushInterval)
           flush()
@@ -449,73 +450,74 @@ class Log(@volatile var dir: File,
    * <li> Whether any compression codec is used (if many are used, then the last one is given)
    * </ol>
    */
-  private def analyzeAndValidateMessageSet(messages: ByteBufferMessageSet): LogAppendInfo = {
+  private def analyzeAndValidateRecords(records: MemoryRecords): LogAppendInfo = {
     var shallowMessageCount = 0
     var validBytesCount = 0
     var firstOffset, lastOffset = -1L
     var sourceCodec: CompressionCodec = NoCompressionCodec
     var monotonic = true
-    var maxTimestamp = Message.NoTimestamp
+    var maxTimestamp = Record.NO_TIMESTAMP
     var offsetOfMaxTimestamp = -1L
-    for(messageAndOffset <- messages.shallowIterator) {
+    for (entry <- records.shallowIterator.asScala) {
       // update the first offset if on the first message
       if(firstOffset < 0)
-        firstOffset = messageAndOffset.offset
+        firstOffset = entry.offset
       // check that offsets are monotonically increasing
-      if(lastOffset >= messageAndOffset.offset)
+      if(lastOffset >= entry.offset)
         monotonic = false
       // update the last offset seen
-      lastOffset = messageAndOffset.offset
+      lastOffset = entry.offset
 
-      val m = messageAndOffset.message
+      val record = entry.record
 
       // Check if the message sizes are valid.
-      val messageSize = MessageSet.entrySize(m)
+      val messageSize = entry.sizeInBytes
       if(messageSize > config.maxMessageSize) {
-        BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(messages.sizeInBytes)
-        BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(messages.sizeInBytes)
+        BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(records.sizeInBytes)
+        BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(records.sizeInBytes)
         throw new RecordTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d."
           .format(messageSize, config.maxMessageSize))
       }
 
       // check the validity of the message by checking CRC
-      m.ensureValid()
-      if (m.timestamp > maxTimestamp) {
-        maxTimestamp = m.timestamp
+      record.ensureValid()
+      if (record.timestamp > maxTimestamp) {
+        maxTimestamp = record.timestamp
         offsetOfMaxTimestamp = lastOffset
       }
       shallowMessageCount += 1
       validBytesCount += messageSize
 
-      val messageCodec = m.compressionCodec
-      if(messageCodec != NoCompressionCodec)
+      val messageCodec = CompressionCodec.getCompressionCodec(record.compressionType.id)
+      if (messageCodec != NoCompressionCodec)
         sourceCodec = messageCodec
     }
 
     // Apply broker-side compression if any
     val targetCodec = BrokerCompressionCodec.getTargetCompressionCodec(config.compressionType, sourceCodec)
 
-    LogAppendInfo(firstOffset, lastOffset, maxTimestamp, offsetOfMaxTimestamp, Message.NoTimestamp, sourceCodec, targetCodec, shallowMessageCount, validBytesCount, monotonic)
+    LogAppendInfo(firstOffset, lastOffset, maxTimestamp, offsetOfMaxTimestamp, Record.NO_TIMESTAMP, sourceCodec,
+      targetCodec, shallowMessageCount, validBytesCount, monotonic)
   }
 
   /**
    * Trim any invalid bytes from the end of this message set (if there are any)
    *
-   * @param messages The message set to trim
+   * @param records The records to trim
    * @param info The general information of the message set
    * @return A trimmed message set. This may be the same as what was passed in or it may not.
    */
-  private def trimInvalidBytes(messages: ByteBufferMessageSet, info: LogAppendInfo): ByteBufferMessageSet = {
-    val messageSetValidBytes = info.validBytes
-    if(messageSetValidBytes < 0)
-      throw new CorruptRecordException("Illegal length of message set " + messageSetValidBytes + " Message set cannot be appended to log. Possible causes are corrupted produce requests")
-    if(messageSetValidBytes == messages.sizeInBytes) {
-      messages
+  private def trimInvalidBytes(records: MemoryRecords, info: LogAppendInfo): MemoryRecords = {
+    val validBytes = info.validBytes
+    if (validBytes < 0)
+      throw new CorruptRecordException("Illegal length of message set " + validBytes + " Message set cannot be appended to log. Possible causes are corrupted produce requests")
+    if (validBytes == records.sizeInBytes) {
+      records
     } else {
       // trim invalid bytes
-      val validByteBuffer = messages.buffer.duplicate()
-      validByteBuffer.limit(messageSetValidBytes)
-      new ByteBufferMessageSet(validByteBuffer)
+      val validByteBuffer = records.buffer.duplicate()
+      validByteBuffer.limit(validBytes)
+      MemoryRecords.readableRecords(validByteBuffer)
     }
   }
 
@@ -538,7 +540,7 @@ class Log(@volatile var dir: File,
     val currentNextOffsetMetadata = nextOffsetMetadata
     val next = currentNextOffsetMetadata.messageOffset
     if(startOffset == next)
-      return FetchDataInfo(currentNextOffsetMetadata, MessageSet.Empty)
+      return FetchDataInfo(currentNextOffsetMetadata, MemoryRecords.EMPTY)
 
     var entry = segments.floorEntry(startOffset)
 
@@ -578,7 +580,7 @@ class Log(@volatile var dir: File,
     // okay we are beyond the end of the last segment with no data fetched although the start offset is in range,
     // this can happen when all messages with offset larger than start offsets have been deleted.
     // In this case, we will return the empty set with log end offset metadata
-    FetchDataInfo(nextOffsetMetadata, MessageSet.Empty)
+    FetchDataInfo(nextOffsetMetadata, MemoryRecords.EMPTY)
   }
 
   /**
@@ -610,9 +612,9 @@ class Log(@volatile var dir: File,
     val segmentsCopy = logSegments.toBuffer
     // For the earliest and latest, we do not need to return the timestamp.
     if (targetTimestamp == ListOffsetRequest.EARLIEST_TIMESTAMP)
-        return Some(TimestampOffset(Message.NoTimestamp, segmentsCopy.head.baseOffset))
+        return Some(TimestampOffset(Record.NO_TIMESTAMP, segmentsCopy.head.baseOffset))
     else if (targetTimestamp == ListOffsetRequest.LATEST_TIMESTAMP)
-        return Some(TimestampOffset(Message.NoTimestamp, logEndOffset))
+        return Some(TimestampOffset(Record.NO_TIMESTAMP, logEndOffset))
 
     val targetSeg = {
       // Get all the segments whose largest timestamp is smaller than target timestamp
@@ -656,7 +658,7 @@ class Log(@volatile var dir: File,
         if (segments.size == numToDelete)
           roll()
         // remove the segments for lookups
-        deletable.foreach(deleteSegment(_))
+        deletable.foreach(deleteSegment)
       }
       numToDelete
     }
@@ -865,7 +867,7 @@ class Log(@volatile var dir: File,
         truncateFullyAndStartAt(targetOffset)
       } else {
         val deletable = logSegments.filter(segment => segment.baseOffset > targetOffset)
-        deletable.foreach(deleteSegment(_))
+        deletable.foreach(deleteSegment)
         activeSegment.truncateTo(targetOffset)
         updateLogEndOffset(targetOffset)
         this.recoveryPoint = math.min(targetOffset, this.recoveryPoint)
@@ -882,7 +884,7 @@ class Log(@volatile var dir: File,
     debug("Truncate and start log '" + name + "' to " + newOffset)
     lock synchronized {
       val segmentsToDelete = logSegments.toList
-      segmentsToDelete.foreach(deleteSegment(_))
+      segmentsToDelete.foreach(deleteSegment)
       addSegment(new LogSegment(dir,
                                 newOffset,
                                 indexIntervalBytes = config.indexInterval,

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 4a76b0c..c5a73d5 100644
--- a/core/src/main/scala/kafka/log/LogCleaner.scala
+++ b/core/src/main/scala/kafka/log/LogCleaner.scala
@@ -17,20 +17,21 @@
 
 package kafka.log
 
-import java.io.{DataOutputStream, File}
+import java.io.File
 import java.nio._
 import java.util.Date
 import java.util.concurrent.{CountDownLatch, TimeUnit}
 
 import com.yammer.metrics.core.Gauge
 import kafka.common._
-import kafka.message._
 import kafka.metrics.KafkaMetricsGroup
 import kafka.utils._
+import org.apache.kafka.common.record.{FileRecords, LogEntry, MemoryRecords}
 import org.apache.kafka.common.utils.Time
+import MemoryRecords.LogEntryFilter
 
-import scala.Iterable
 import scala.collection._
+import JavaConverters._
 
 /**
  * The cleaner is responsible for removing obsolete records from logs which have the dedupe retention strategy.
@@ -390,10 +391,10 @@ private[log] class Cleaner(val id: Int,
     val timeIndexFile = new File(segments.head.timeIndex.file.getPath + Log.CleanedFileSuffix)
     indexFile.delete()
     timeIndexFile.delete()
-    val messages = new FileMessageSet(logFile, fileAlreadyExists = false, initFileSize = log.initFileSize(), preallocate = log.config.preallocate)
+    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(messages, index, timeIndex, segments.head.baseOffset, segments.head.indexIntervalBytes, log.config.randomSegmentJitter, time)
+    val cleaned = new LogSegment(records, index, timeIndex, segments.head.baseOffset, segments.head.indexIntervalBytes, log.config.randomSegmentJitter, time)
 
     try {
       // clean segments into the new destination segment
@@ -449,8 +450,12 @@ private[log] class Cleaner(val id: Int,
                              retainDeletes: Boolean,
                              maxLogMessageSize: Int,
                              stats: CleanerStats) {
-    def shouldRetain(messageAndOffset: MessageAndOffset): Boolean =
-      shouldRetainMessage(source, map, retainDeletes, messageAndOffset, stats)
+    def shouldRetainEntry(logEntry: LogEntry): Boolean =
+      shouldRetainMessage(source, map, retainDeletes, logEntry, stats)
+
+    class LogCleanerFilter extends LogEntryFilter {
+      def shouldRetain(logEntry: LogEntry): Boolean = shouldRetainEntry(logEntry)
+    }
 
     var position = 0
     while (position < source.log.sizeInBytes) {
@@ -460,10 +465,9 @@ private[log] class Cleaner(val id: Int,
       writeBuffer.clear()
 
       source.log.readInto(readBuffer, position)
-      val messages = new ByteBufferMessageSet(readBuffer)
-      throttler.maybeThrottle(messages.sizeInBytes)
-      val result = messages.filterInto(writeBuffer, shouldRetain)
-
+      val records = MemoryRecords.readableRecords(readBuffer)
+      throttler.maybeThrottle(records.sizeInBytes)
+      val result = records.filterTo(new LogCleanerFilter, writeBuffer)
       stats.readMessages(result.messagesRead, result.bytesRead)
       stats.recopyMessages(result.messagesRetained, result.bytesRetained)
 
@@ -472,9 +476,10 @@ private[log] class Cleaner(val id: Int,
       // if any messages are to be retained, write them out
       if (writeBuffer.position > 0) {
         writeBuffer.flip()
-        val retained = new ByteBufferMessageSet(writeBuffer)
-        dest.append(firstOffset = retained.head.offset, largestTimestamp = result.maxTimestamp,
-          offsetOfLargestTimestamp = result.offsetOfMaxTimestamp, messages = retained)
+
+        val retained = MemoryRecords.readableRecords(writeBuffer)
+        dest.append(firstOffset = retained.deepIterator().next().offset, largestTimestamp = result.maxTimestamp,
+          shallowOffsetOfMaxTimestamp = result.shallowOffsetOfMaxTimestamp, records = retained)
         throttler.maybeThrottle(writeBuffer.limit)
       }
       
@@ -488,21 +493,22 @@ private[log] class Cleaner(val id: Int,
   private def shouldRetainMessage(source: kafka.log.LogSegment,
                                   map: kafka.log.OffsetMap,
                                   retainDeletes: Boolean,
-                                  entry: kafka.message.MessageAndOffset,
+                                  entry: LogEntry,
                                   stats: CleanerStats): Boolean = {
     val pastLatestOffset = entry.offset > map.latestOffset
     if (pastLatestOffset)
       return true
 
-    val key = entry.message.key
-    if (key != null) {
+
+    if (entry.record.hasKey) {
+      val key = entry.record.key
       val foundOffset = map.get(key)
       /* two cases in which we can get rid of a message:
        *   1) if there exists a message with the same key but higher offset
        *   2) if the message is a delete "tombstone" marker and enough time has passed
        */
       val redundant = foundOffset >= 0 && entry.offset < foundOffset
-      val obsoleteDelete = !retainDeletes && entry.message.isNull
+      val obsoleteDelete = !retainDeletes && entry.record.hasNullValue
       !redundant && !obsoleteDelete
     } else {
       stats.invalidMessage()
@@ -620,12 +626,12 @@ private[log] class Cleaner(val id: Int,
       checkDone(topicAndPartition)
       readBuffer.clear()
       segment.log.readInto(readBuffer, position)
-      val messages = new ByteBufferMessageSet(readBuffer)
-      throttler.maybeThrottle(messages.sizeInBytes)
+      val records = MemoryRecords.readableRecords(readBuffer)
+      throttler.maybeThrottle(records.sizeInBytes)
 
       val startPosition = position
-      for (entry <- messages) {
-        val message = entry.message
+      for (entry <- records.deepIterator.asScala) {
+        val message = entry.record
         if (message.hasKey && entry.offset >= start) {
           if (map.size < maxDesiredMapSize)
             map.put(message.key, entry.offset)
@@ -634,8 +640,9 @@ private[log] class Cleaner(val id: Int,
         }
         stats.indexMessagesRead(1)
       }
-      position += messages.validBytes
-      stats.indexBytesRead(messages.validBytes)
+      val bytesRead = records.validBytes
+      position += bytesRead
+      stats.indexBytesRead(bytesRead)
 
       // if we didn't read even one complete message, our read buffer may be too small
       if(position == startPosition)

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 ed79946..953fca4 100755
--- a/core/src/main/scala/kafka/log/LogManager.scala
+++ b/core/src/main/scala/kafka/log/LogManager.scala
@@ -440,7 +440,7 @@ class LogManager(val logDirs: Array[File],
         removedLog.dir = renamedDir
         // change the file pointers for log and index file
         for (logSegment <- removedLog.logSegments) {
-          logSegment.log.file = new File(renamedDir, logSegment.log.file.getName)
+          logSegment.log.setFile(new File(renamedDir, logSegment.log.file.getName))
           logSegment.index.file = new File(renamedDir, logSegment.index.file.getName)
         }
 


[5/9] kafka git commit: KAFKA-4390; Replace MessageSet usage with client-side alternatives

Posted by jg...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 c5418e3..bd4eb68 100755
--- a/core/src/main/scala/kafka/log/LogSegment.scala
+++ b/core/src/main/scala/kafka/log/LogSegment.scala
@@ -16,15 +16,20 @@
  */
 package kafka.log
 
-import kafka.message._
+import java.io.{File, IOException}
+import java.util.concurrent.TimeUnit
+
 import kafka.common._
-import kafka.utils._
+import kafka.metrics.{KafkaMetricsGroup, KafkaTimer}
 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._
 import org.apache.kafka.common.utils.Time
 
+import scala.collection.JavaConverters._
 import scala.math._
-import java.io.{File, IOException}
 
  /**
  * A segment of the log. Each segment has two components: a log and an index. The log is a FileMessageSet containing
@@ -42,7 +47,7 @@ import java.io.{File, IOException}
  * @param time The time instance
  */
 @nonthreadsafe
-class LogSegment(val log: FileMessageSet,
+class LogSegment(val log: FileRecords,
                  val index: OffsetIndex,
                  val timeIndex: TimeIndex,
                  val baseOffset: Long,
@@ -63,7 +68,7 @@ class LogSegment(val log: FileMessageSet,
   @volatile private var offsetOfMaxTimestamp = 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) =
-    this(new FileMessageSet(file = Log.logFilename(dir, startOffset), fileAlreadyExists = fileAlreadyExists, initFileSize = initFileSize, preallocate = preallocate),
+    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),
          startOffset,
@@ -82,23 +87,25 @@ class LogSegment(val log: FileMessageSet,
    *
    * @param firstOffset The first offset in the message set.
    * @param largestTimestamp The largest timestamp in the message set.
-   * @param offsetOfLargestTimestamp The offset of the message that has the largest timestamp in the messages to append.
-   * @param messages The messages to append.
+   * @param shallowOffsetOfMaxTimestamp The offset of the message that has the largest timestamp in the messages to append.
+   * @param records The log entries to append.
    */
   @nonthreadsafe
-  def append(firstOffset: Long, largestTimestamp: Long, offsetOfLargestTimestamp: Long, messages: ByteBufferMessageSet) {
-    if (messages.sizeInBytes > 0) {
-      trace("Inserting %d bytes at offset %d at position %d with largest timestamp %d at offset %d"
-          .format(messages.sizeInBytes, firstOffset, log.sizeInBytes(), largestTimestamp, offsetOfLargestTimestamp))
+  def append(firstOffset: Long, largestTimestamp: Long, shallowOffsetOfMaxTimestamp: Long, records: MemoryRecords) {
+    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))
       val physicalPosition = log.sizeInBytes()
       if (physicalPosition == 0)
         rollingBasedTimestamp = Some(largestTimestamp)
       // append the messages
-      log.append(messages)
+      val appendedBytes = log.append(records)
+      trace(s"Appended $appendedBytes to ${log.file()} at offset $firstOffset")
+
       // Update the in memory max timestamp and corresponding offset.
       if (largestTimestamp > maxTimestampSoFar) {
         maxTimestampSoFar = largestTimestamp
-        offsetOfMaxTimestamp = offsetOfLargestTimestamp
+        offsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp
       }
       // append an entry to the index (if needed)
       if(bytesSinceLastIndexEntry > indexIntervalBytes) {
@@ -106,7 +113,7 @@ class LogSegment(val log: FileMessageSet,
         timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp)
         bytesSinceLastIndexEntry = 0
       }
-      bytesSinceLastIndexEntry += messages.sizeInBytes
+      bytesSinceLastIndexEntry += records.sizeInBytes
     }
   }
 
@@ -123,7 +130,7 @@ class LogSegment(val log: FileMessageSet,
     *        message or null if no message meets this criteria.
    */
   @threadsafe
-  private[log] def translateOffset(offset: Long, startingFilePosition: Int = 0): (OffsetPosition, Int) = {
+  private[log] def translateOffset(offset: Long, startingFilePosition: Int = 0): LogEntryPosition = {
     val mapping = index.lookup(offset)
     log.searchForOffsetWithSize(offset, max(mapping.position, startingFilePosition))
   }
@@ -154,40 +161,40 @@ class LogSegment(val log: FileMessageSet,
     if (startOffsetAndSize == null)
       return null
 
-    val (startPosition, messageSetSize) = startOffsetAndSize
-    val offsetMetadata = new LogOffsetMetadata(startOffset, this.baseOffset, startPosition.position)
+    val startPosition = startOffsetAndSize.position.toInt
+    val offsetMetadata = new LogOffsetMetadata(startOffset, this.baseOffset, startPosition)
 
     val adjustedMaxSize =
-      if (minOneMessage) math.max(maxSize, messageSetSize)
+      if (minOneMessage) math.max(maxSize, startOffsetAndSize.size)
       else maxSize
 
     // return a log segment but with zero size in the case below
     if (adjustedMaxSize == 0)
-      return FetchDataInfo(offsetMetadata, MessageSet.Empty)
+      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 {
       case None =>
         // no max offset, just read until the max position
-        min((maxPosition - startPosition.position).toInt, adjustedMaxSize)
+        min((maxPosition - startPosition).toInt, adjustedMaxSize)
       case Some(offset) =>
         // there is a max offset, translate it to a file position and use that to calculate the max read size;
         // when the leader of a partition changes, it's possible for the new leader's high watermark to be less than the
         // true high watermark in the previous leader for a short window. In this window, if a consumer fetches on an
         // offset between new leader's high watermark and the log end offset, we want to return an empty response.
         if (offset < startOffset)
-          return FetchDataInfo(offsetMetadata, MessageSet.Empty, firstMessageSetIncomplete = false)
-        val mapping = translateOffset(offset, startPosition.position)
+          return FetchDataInfo(offsetMetadata, MemoryRecords.EMPTY, firstEntryIncomplete = false)
+        val mapping = translateOffset(offset, startPosition)
         val endPosition =
           if (mapping == null)
             logSize // the max offset is off the end of the log, use the end of the file
           else
-            mapping._1.position
-        min(min(maxPosition, endPosition) - startPosition.position, adjustedMaxSize).toInt
+            mapping.position
+        min(min(maxPosition, endPosition) - startPosition, adjustedMaxSize).toInt
     }
 
-    FetchDataInfo(offsetMetadata, log.read(startPosition.position, length),
-      firstMessageSetIncomplete = adjustedMaxSize < messageSetSize)
+    FetchDataInfo(offsetMetadata, log.read(startPosition, length),
+      firstEntryIncomplete = adjustedMaxSize < startOffsetAndSize.size)
   }
 
   /**
@@ -205,16 +212,16 @@ class LogSegment(val log: FileMessageSet,
     timeIndex.resize(timeIndex.maxIndexSize)
     var validBytes = 0
     var lastIndexEntry = 0
-    val iter = log.iterator(maxMessageSize)
-    maxTimestampSoFar = Message.NoTimestamp
+    val iter = log.shallowIterator(maxMessageSize)
+    maxTimestampSoFar = Record.NO_TIMESTAMP
     try {
-      while(iter.hasNext) {
-        val entry = iter.next
-        entry.message.ensureValid()
+      for (entry <- iter.asScala) {
+        val record = entry.record
+        record.ensureValid()
 
         // The max timestamp should have been put in the outer message, so we don't need to iterate over the inner messages.
-        if (entry.message.timestamp > maxTimestampSoFar) {
-          maxTimestampSoFar = entry.message.timestamp
+        if (record.timestamp > maxTimestampSoFar) {
+          maxTimestampSoFar = record.timestamp
           offsetOfMaxTimestamp = entry.offset
         }
 
@@ -225,11 +232,12 @@ class LogSegment(val log: FileMessageSet,
           timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp)
           lastIndexEntry = validBytes
         }
-        validBytes += MessageSet.entrySize(entry.message)
+        validBytes += entry.sizeInBytes()
       }
     } catch {
       case e: CorruptRecordException =>
-        logger.warn("Found invalid messages in log segment %s at byte offset %d: %s.".format(log.file.getAbsolutePath, validBytes, e.getMessage))
+        logger.warn("Found invalid messages in log segment %s at byte offset %d: %s."
+          .format(log.file.getAbsolutePath, validBytes, e.getMessage))
     }
     val truncated = log.sizeInBytes - validBytes
     log.truncateTo(validBytes)
@@ -276,8 +284,7 @@ class LogSegment(val log: FileMessageSet,
     // after truncation, reset and allocate more space for the (new currently  active) index
     index.resize(index.maxIndexSize)
     timeIndex.resize(timeIndex.maxIndexSize)
-    val (offsetPosition, _) = mapping
-    val bytesTruncated = log.truncateTo(offsetPosition.position)
+    val bytesTruncated = log.truncateTo(mapping.position.toInt)
     if(log.sizeInBytes == 0) {
       created = time.milliseconds
       rollingBasedTimestamp = None
@@ -296,10 +303,10 @@ class LogSegment(val log: FileMessageSet,
   @threadsafe
   def nextOffset(): Long = {
     val ms = read(index.lastOffset, None, log.sizeInBytes)
-    if(ms == null) {
+    if (ms == null) {
       baseOffset
     } else {
-      ms.messageSet.lastOption match {
+      ms.records.shallowIterator.asScala.toSeq.lastOption match {
         case None => baseOffset
         case Some(last) => last.nextOffset
       }
@@ -360,9 +367,9 @@ class LogSegment(val log: FileMessageSet,
   def timeWaitedForRoll(now: Long, messageTimestamp: Long) : Long = {
     // Load the timestamp of the first message into memory
     if (rollingBasedTimestamp.isEmpty) {
-      val iter = log.iterator
+      val iter = log.shallowIterator
       if (iter.hasNext)
-        rollingBasedTimestamp = Some(iter.next.message.timestamp)
+        rollingBasedTimestamp = Some(iter.next.record.timestamp)
     }
     rollingBasedTimestamp match {
       case Some(t) if t >= 0 => messageTimestamp - t
@@ -394,8 +401,11 @@ class LogSegment(val log: FileMessageSet,
     // Get the index entry with a timestamp less than or equal to the target timestamp
     val timestampOffset = timeIndex.lookup(timestamp)
     val position = index.lookup(timestampOffset.offset).position
+
     // Search the timestamp
-    log.searchForTimestamp(timestamp, position)
+    Option(log.searchForTimestamp(timestamp, position)).map { timestampAndOffset =>
+      TimestampOffset(timestampAndOffset.timestamp, timestampAndOffset.offset)
+    }
   }
 
   /**
@@ -444,3 +454,7 @@ class LogSegment(val log: FileMessageSet,
     timeIndex.file.setLastModified(ms)
   }
 }
+
+object LogFlushStats extends KafkaMetricsGroup {
+  val logFlushTimer = new KafkaTimer(newTimer("LogFlushRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS))
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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
new file mode 100644
index 0000000..d9f27e4
--- /dev/null
+++ b/core/src/main/scala/kafka/log/LogValidator.scala
@@ -0,0 +1,239 @@
+/**
+ * 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.nio.ByteBuffer
+
+import kafka.common.LongRef
+import kafka.message.{CompressionCodec, InvalidMessageException, NoCompressionCodec}
+import org.apache.kafka.common.errors.InvalidTimestampException
+import org.apache.kafka.common.record._
+
+import scala.collection.mutable
+import scala.collection.JavaConverters._
+
+private[kafka] object LogValidator {
+
+  /**
+   * Update the offsets for this message set and do further validation on messages including:
+   * 1. Messages for compacted topics must have keys
+   * 2. When magic value = 1, inner messages of a compressed message set must have monotonically increasing offsets
+   *    starting from 0.
+   * 3. When magic value = 1, validate and maybe overwrite timestamps of messages.
+   *
+   * This method will convert the messages in the following scenarios:
+   * A. Magic value of a message = 0 and messageFormatVersion is 1
+   * B. Magic value of a message = 1 and messageFormatVersion is 0
+   *
+   * If no format conversion or value overwriting is required for messages, this method will perform in-place
+   * operations and avoid re-compression.
+   *
+   * Returns a ValidationAndOffsetAssignResult containing the validated message set, maximum timestamp, the offset
+   * of the shallow message with the max timestamp and a boolean indicating whether the message sizes may have changed.
+   */
+  private[kafka] def validateMessagesAndAssignOffsets(records: MemoryRecords,
+                                                      offsetCounter: LongRef,
+                                                      now: Long,
+                                                      sourceCodec: CompressionCodec,
+                                                      targetCodec: CompressionCodec,
+                                                      compactedTopic: Boolean = false,
+                                                      messageFormatVersion: Byte = Record.CURRENT_MAGIC_VALUE,
+                                                      messageTimestampType: TimestampType,
+                                                      messageTimestampDiffMaxMs: Long): ValidationAndOffsetAssignResult = {
+    if (sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) {
+      // check the magic value
+      if (!records.hasMatchingShallowMagic(messageFormatVersion))
+        convertAndAssignOffsetsNonCompressed(records, offsetCounter, compactedTopic, now, messageTimestampType,
+          messageTimestampDiffMaxMs, messageFormatVersion)
+      else
+        // Do in-place validation, offset assignment and maybe set timestamp
+        assignOffsetsNonCompressed(records, offsetCounter, now, compactedTopic, messageTimestampType,
+          messageTimestampDiffMaxMs)
+    } else {
+      // Deal with compressed messages
+      // We cannot do in place assignment in one of the following situations:
+      // 1. Source and target compression codec are different
+      // 2. When magic value to use is 0 because offsets need to be overwritten
+      // 3. When magic value to use is above 0, but some fields of inner messages need to be overwritten.
+      // 4. Message format conversion is needed.
+
+      // No in place assignment situation 1 and 2
+      var inPlaceAssignment = sourceCodec == targetCodec && messageFormatVersion > Record.MAGIC_VALUE_V0
+
+      var maxTimestamp = Record.NO_TIMESTAMP
+      val expectedInnerOffset = new LongRef(0)
+      val validatedRecords = new mutable.ArrayBuffer[Record]
+
+      records.deepIterator(true).asScala.foreach { logEntry =>
+        val record = logEntry.record
+        validateKey(record, compactedTopic)
+
+        if (record.magic > Record.MAGIC_VALUE_V0 && messageFormatVersion > Record.MAGIC_VALUE_V0) {
+          // No in place assignment situation 3
+          // Validate the timestamp
+          validateTimestamp(record, now, messageTimestampType, messageTimestampDiffMaxMs)
+          // Check if we need to overwrite offset
+          if (logEntry.offset != expectedInnerOffset.getAndIncrement())
+            inPlaceAssignment = false
+          if (record.timestamp > maxTimestamp)
+            maxTimestamp = record.timestamp
+        }
+
+        if (sourceCodec != NoCompressionCodec && logEntry.isCompressed)
+          throw new InvalidMessageException("Compressed outer record should not have an inner record with a " +
+            s"compression attribute set: $record")
+
+        // No in place assignment situation 4
+        if (record.magic != messageFormatVersion)
+          inPlaceAssignment = false
+
+        validatedRecords += record.convert(messageFormatVersion)
+      }
+
+      if (!inPlaceAssignment) {
+        val entries = validatedRecords.map(record => LogEntry.create(offsetCounter.getAndIncrement(), record))
+        val builder = MemoryRecords.builderWithEntries(messageTimestampType, CompressionType.forId(targetCodec.codec),
+          now, entries.asJava)
+        builder.close()
+        val info = builder.info
+
+        ValidationAndOffsetAssignResult(
+          validatedRecords = builder.build(),
+          maxTimestamp = info.maxTimestamp,
+          shallowOffsetOfMaxTimestamp = info.shallowOffsetOfMaxTimestamp,
+          messageSizeMaybeChanged = true)
+      } else {
+        // ensure the inner messages are valid
+        validatedRecords.foreach(_.ensureValid)
+
+        // we can update the wrapper message only and write the compressed payload as is
+        val entry = records.shallowIterator.next()
+        val offset = offsetCounter.addAndGet(validatedRecords.size) - 1
+        entry.setOffset(offset)
+        if (messageTimestampType == TimestampType.CREATE_TIME)
+          entry.setCreateTime(maxTimestamp)
+        else if (messageTimestampType == TimestampType.LOG_APPEND_TIME)
+          entry.setLogAppendTime(now)
+
+        ValidationAndOffsetAssignResult(validatedRecords = records,
+          maxTimestamp = if (messageTimestampType == TimestampType.LOG_APPEND_TIME) now else maxTimestamp,
+          shallowOffsetOfMaxTimestamp = offset,
+          messageSizeMaybeChanged = false)
+      }
+    }
+  }
+
+  private def convertAndAssignOffsetsNonCompressed(records: MemoryRecords,
+                                                   offsetCounter: LongRef,
+                                                   compactedTopic: Boolean,
+                                                   now: Long,
+                                                   timestampType: TimestampType,
+                                                   messageTimestampDiffMaxMs: Long,
+                                                   toMagicValue: Byte): ValidationAndOffsetAssignResult = {
+    val sizeInBytesAfterConversion = records.shallowIterator.asScala.map { logEntry =>
+      logEntry.record.convertedSize(toMagicValue)
+    }.sum
+
+    val newBuffer = ByteBuffer.allocate(sizeInBytesAfterConversion)
+    val builder = MemoryRecords.builder(newBuffer, toMagicValue, CompressionType.NONE, timestampType,
+      offsetCounter.value, now)
+
+    records.shallowIterator.asScala.foreach { logEntry =>
+      val record = logEntry.record
+      validateKey(record, compactedTopic)
+      validateTimestamp(record, now, timestampType, messageTimestampDiffMaxMs)
+      builder.convertAndAppend(offsetCounter.getAndIncrement(), record)
+    }
+
+    builder.close()
+    val info = builder.info
+
+    ValidationAndOffsetAssignResult(
+      validatedRecords = builder.build(),
+      maxTimestamp = info.maxTimestamp,
+      shallowOffsetOfMaxTimestamp = info.shallowOffsetOfMaxTimestamp,
+      messageSizeMaybeChanged = true)
+  }
+
+  private def assignOffsetsNonCompressed(records: MemoryRecords,
+                                         offsetCounter: LongRef,
+                                         now: Long,
+                                         compactedTopic: Boolean,
+                                         timestampType: TimestampType,
+                                         timestampDiffMaxMs: Long): ValidationAndOffsetAssignResult = {
+    var maxTimestamp = Record.NO_TIMESTAMP
+    var offsetOfMaxTimestamp = -1L
+    val firstOffset = offsetCounter.value
+
+    for (entry <- records.shallowIterator.asScala) {
+      val record = entry.record
+      validateKey(record, compactedTopic)
+
+      val offset = offsetCounter.getAndIncrement()
+      entry.setOffset(offset)
+
+      if (record.magic > Record.MAGIC_VALUE_V0) {
+        validateTimestamp(record, now, timestampType, timestampDiffMaxMs)
+
+        if (timestampType == TimestampType.LOG_APPEND_TIME)
+          entry.setLogAppendTime(now)
+        else if (record.timestamp > maxTimestamp) {
+          maxTimestamp = record.timestamp
+          offsetOfMaxTimestamp = offset
+        }
+      }
+    }
+
+    if (timestampType == TimestampType.LOG_APPEND_TIME) {
+      maxTimestamp = now
+      offsetOfMaxTimestamp = firstOffset
+    }
+
+    ValidationAndOffsetAssignResult(
+      validatedRecords = records,
+      maxTimestamp = maxTimestamp,
+      shallowOffsetOfMaxTimestamp = offsetOfMaxTimestamp,
+      messageSizeMaybeChanged = false)
+  }
+
+  private def validateKey(record: Record, compactedTopic: Boolean) {
+    if (compactedTopic && !record.hasKey)
+      throw new InvalidMessageException("Compacted topic cannot accept message without key.")
+  }
+
+  /**
+   * This method validates the timestamps of a message.
+   * If the message is using create time, this method checks if it is within acceptable range.
+   */
+  private def validateTimestamp(record: Record,
+                                now: Long,
+                                timestampType: TimestampType,
+                                timestampDiffMaxMs: Long) {
+    if (timestampType == TimestampType.CREATE_TIME && math.abs(record.timestamp - now) > timestampDiffMaxMs)
+      throw new InvalidTimestampException(s"Timestamp ${record.timestamp} of message is out of range. " +
+        s"The timestamp should be within [${now - timestampDiffMaxMs}, ${now + timestampDiffMaxMs}")
+    if (record.timestampType == TimestampType.LOG_APPEND_TIME)
+      throw new InvalidTimestampException(s"Invalid timestamp type in message $record. Producer should not set " +
+        s"timestamp type to LogAppendTime.")
+  }
+
+  case class ValidationAndOffsetAssignResult(validatedRecords: MemoryRecords,
+                                             maxTimestamp: Long,
+                                             shallowOffsetOfMaxTimestamp: Long,
+                                             messageSizeMaybeChanged: Boolean)
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 7f24081..21c212f 100644
--- a/core/src/main/scala/kafka/log/TimeIndex.scala
+++ b/core/src/main/scala/kafka/log/TimeIndex.scala
@@ -21,9 +21,9 @@ import java.io.File
 import java.nio.ByteBuffer
 
 import kafka.common.InvalidOffsetException
-import kafka.message.Message
 import kafka.utils.CoreUtils._
 import kafka.utils.Logging
+import org.apache.kafka.common.record.Record
 
 /**
  * An index that maps from the timestamp to the logical offsets of the messages in a segment. This index might be
@@ -69,7 +69,7 @@ class TimeIndex(file: File,
   def lastEntry: TimestampOffset = {
     inLock(lock) {
       _entries match {
-        case 0 => TimestampOffset(Message.NoTimestamp, baseOffset)
+        case 0 => TimestampOffset(Record.NO_TIMESTAMP, baseOffset)
         case s => parseEntry(mmap, s - 1).asInstanceOf[TimestampOffset]
       }
     }
@@ -145,7 +145,7 @@ class TimeIndex(file: File,
       val idx = mmap.duplicate
       val slot = indexSlotFor(idx, targetTimestamp, IndexSearchType.KEY)
       if (slot == -1)
-        TimestampOffset(Message.NoTimestamp, baseOffset)
+        TimestampOffset(Record.NO_TIMESTAMP, baseOffset)
       else {
         val entry = parseEntry(idx, slot).asInstanceOf[TimestampOffset]
         TimestampOffset(entry.timestamp, entry.offset)

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala
index 096344d..2c8fef6 100644
--- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala
+++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala
@@ -17,20 +17,13 @@
 
 package kafka.message
 
-import kafka.utils.{CoreUtils, IteratorTemplate, Logging}
-import kafka.common.{KafkaException, LongRef}
 import java.nio.ByteBuffer
-import java.nio.channels._
-import java.io._
-import java.util.ArrayDeque
 
-import kafka.message.ByteBufferMessageSet.FilterResult
-import org.apache.kafka.common.errors.InvalidTimestampException
-import org.apache.kafka.common.record.{MemoryRecords, TimestampType}
-import org.apache.kafka.common.utils.Utils
+import kafka.common.LongRef
+import kafka.utils.Logging
+import org.apache.kafka.common.record._
 
-import scala.collection.mutable
-import scala.collection.mutable.ArrayBuffer
+import scala.collection.JavaConverters._
 
 object ByteBufferMessageSet {
 
@@ -41,204 +34,19 @@ object ByteBufferMessageSet {
                      messages: Message*): ByteBuffer = {
     if (messages.isEmpty)
       MessageSet.Empty.buffer
-    else if (compressionCodec == NoCompressionCodec) {
-      val buffer = ByteBuffer.allocate(MessageSet.messageSetSize(messages))
-      for (message <- messages) writeMessage(buffer, message, offsetAssigner.nextAbsoluteOffset())
-      buffer.rewind()
-      buffer
-    } else {
+    else {
       val magicAndTimestamp = wrapperMessageTimestamp match {
         case Some(ts) => MagicAndTimestamp(messages.head.magic, ts)
         case None => MessageSet.magicAndLargestTimestamp(messages)
       }
-      val (messageWriter, lastOffset) = writeCompressedMessages(compressionCodec, offsetAssigner, magicAndTimestamp,
-        timestampType, messages)
-      val buffer = ByteBuffer.allocate(messageWriter.size + MessageSet.LogOverhead)
-      writeMessage(buffer, messageWriter, lastOffset)
-      buffer.rewind()
-      buffer
-    }
-  }
-
-  /** Deep iterator that decompresses the message sets and adjusts timestamp and offset if needed. */
-  def deepIterator(wrapperMessageAndOffset: MessageAndOffset, ensureMatchingMagic: Boolean = false): Iterator[MessageAndOffset] = {
-
-    import Message._
-
-    new IteratorTemplate[MessageAndOffset] {
-
-      val MessageAndOffset(wrapperMessage, wrapperMessageOffset) = wrapperMessageAndOffset
-
-      if (wrapperMessage.payload == null)
-        throw new KafkaException(s"Message payload is null: $wrapperMessage")
-
-      val wrapperMessageTimestampOpt: Option[Long] =
-        if (wrapperMessage.magic > MagicValue_V0) Some(wrapperMessage.timestamp) else None
-      val wrapperMessageTimestampTypeOpt: Option[TimestampType] =
-        if (wrapperMessage.magic > MagicValue_V0) Some(wrapperMessage.timestampType) else None
-
-      var lastInnerOffset = -1L
-
-      val messageAndOffsets = {
-        val inputStream = new ByteBufferBackedInputStream(wrapperMessage.payload)
-        val compressed = try {
-          new DataInputStream(CompressionFactory(wrapperMessage.compressionCodec, wrapperMessage.magic, inputStream))
-        } catch {
-          case ioe: IOException =>
-            throw new InvalidMessageException(s"Failed to instantiate input stream compressed with ${wrapperMessage.compressionCodec}", ioe)
-        }
-
-        val innerMessageAndOffsets = new ArrayDeque[MessageAndOffset]()
-        try {
-          while (true)
-            innerMessageAndOffsets.add(readMessageFromStream(compressed))
-        } catch {
-          case _: EOFException =>
-            // we don't do anything at all here, because the finally
-            // will close the compressed input stream, and we simply
-            // want to return the innerMessageAndOffsets
-          case ioe: IOException =>
-            throw new InvalidMessageException(s"Error while reading message from stream compressed with ${wrapperMessage.compressionCodec}", ioe)
-        } finally {
-          CoreUtils.swallow(compressed.close())
-        }
-
-        innerMessageAndOffsets
-      }
 
-      private def readMessageFromStream(compressed: DataInputStream): MessageAndOffset = {
-        val innerOffset = compressed.readLong()
-        val recordSize = compressed.readInt()
-
-        if (recordSize < MinMessageOverhead)
-          throw new InvalidMessageException(s"Message found with corrupt size `$recordSize` in deep iterator")
-
-        // read the record into an intermediate record buffer (i.e. extra copy needed)
-        val bufferArray = new Array[Byte](recordSize)
-        compressed.readFully(bufferArray, 0, recordSize)
-        val buffer = ByteBuffer.wrap(bufferArray)
-
-        // Override the timestamp if necessary
-        val newMessage = new Message(buffer, wrapperMessageTimestampOpt, wrapperMessageTimestampTypeOpt)
-
-        // Due to KAFKA-4298, it is possible for the inner and outer magic values to differ. We ignore
-        // this and depend on the outer message in order to decide how to compute the respective offsets
-        // for the inner messages
-        if (ensureMatchingMagic && newMessage.magic != wrapperMessage.magic)
-          throw new InvalidMessageException(s"Compressed message has magic value ${wrapperMessage.magic} " +
-            s"but inner message has magic value ${newMessage.magic}")
-
-        lastInnerOffset = innerOffset
-        MessageAndOffset(newMessage, innerOffset)
-      }
-
-      override def makeNext(): MessageAndOffset = {
-        messageAndOffsets.pollFirst() match {
-          case null => allDone()
-          case nextMessage@ MessageAndOffset(message, offset) =>
-            if (wrapperMessage.magic > MagicValue_V0) {
-              val relativeOffset = offset - lastInnerOffset
-              val absoluteOffset = wrapperMessageOffset + relativeOffset
-              MessageAndOffset(message, absoluteOffset)
-            } else {
-              nextMessage
-            }
-        }
-      }
+      val entries = messages.map(message => LogEntry.create(offsetAssigner.nextAbsoluteOffset(), message.asRecord))
+      val builder = MemoryRecords.builderWithEntries(timestampType, CompressionType.forId(compressionCodec.codec),
+        magicAndTimestamp.timestamp, entries.asJava)
+      builder.build().buffer
     }
   }
 
-  private def writeCompressedMessages(codec: CompressionCodec,
-                                      offsetAssigner: OffsetAssigner,
-                                      magicAndTimestamp: MagicAndTimestamp,
-                                      timestampType: TimestampType,
-                                      messages: Seq[Message]): (MessageWriter, Long) = {
-    require(codec != NoCompressionCodec, s"compressionCodec must not be $NoCompressionCodec")
-    require(messages.nonEmpty, "cannot write empty compressed message set")
-
-    var offset = -1L
-    val magic = magicAndTimestamp.magic
-    val messageWriter = new MessageWriter(math.min(math.max(MessageSet.messageSetSize(messages) / 2, 1024), 1 << 16))
-    messageWriter.write(
-      codec = codec,
-      timestamp = magicAndTimestamp.timestamp,
-      timestampType = timestampType,
-      magicValue = magic) { outputStream =>
-
-      val output = new DataOutputStream(CompressionFactory(codec, magic, outputStream))
-      try {
-        for (message <- messages) {
-          offset = offsetAssigner.nextAbsoluteOffset()
-
-          if (message.magic != magicAndTimestamp.magic)
-            throw new IllegalArgumentException("Messages in the message set must have same magic value")
-
-          // Use inner offset if magic value is greater than 0
-          val innerOffset = if (magicAndTimestamp.magic > Message.MagicValue_V0)
-            offsetAssigner.toInnerOffset(offset)
-          else
-            offset
-
-          output.writeLong(innerOffset)
-          output.writeInt(message.size)
-          output.write(message.buffer.array, message.buffer.arrayOffset, message.buffer.limit)
-        }
-      } finally {
-        output.close()
-      }
-    }
-
-    (messageWriter, offset)
-  }
-
-  private[kafka] def writeCompressedMessages(buffer: ByteBuffer,
-                                             codec: CompressionCodec,
-                                             messageAndOffsets: Seq[MessageAndOffset]): Int = {
-    require(codec != NoCompressionCodec, s"compressionCodec must not be $NoCompressionCodec")
-
-    if (messageAndOffsets.isEmpty)
-      0
-    else {
-      val messages = messageAndOffsets.map(_.message)
-      val magicAndTimestamp = MessageSet.magicAndLargestTimestamp(messages)
-
-      // ensure that we use the magic from the first message in the set when writing the wrapper
-      // message in order to fix message sets corrupted by KAFKA-4298
-      val magic = magicAndTimestamp.magic
-
-      val firstMessageAndOffset = messageAndOffsets.head
-      val firstAbsoluteOffset = firstMessageAndOffset.offset
-      val offsetAssigner = OffsetAssigner(firstAbsoluteOffset, magic, messageAndOffsets)
-      val timestampType = firstMessageAndOffset.message.timestampType
-
-      val (messageWriter, lastOffset) = writeCompressedMessages(codec, offsetAssigner, magicAndTimestamp,
-        timestampType, messages)
-
-      writeMessage(buffer, messageWriter, lastOffset)
-      messageWriter.size + MessageSet.LogOverhead
-    }
-  }
-
-  private[kafka] def writeMessage(buffer: ByteBuffer, message: Message, offset: Long) {
-    buffer.putLong(offset)
-    buffer.putInt(message.size)
-    buffer.put(message.buffer)
-    message.buffer.rewind()
-  }
-
-  private[kafka] def writeMessage(buffer: ByteBuffer, messageWriter: MessageWriter, offset: Long) {
-    buffer.putLong(offset)
-    buffer.putInt(messageWriter.size)
-    messageWriter.writeTo(buffer)
-  }
-
-
-  case class FilterResult(messagesRead: Int,
-                          bytesRead: Int,
-                          messagesRetained: Int,
-                          bytesRetained: Int,
-                          maxTimestamp: Long,
-                          offsetOfMaxTimestamp: Long)
 }
 
 private object OffsetAssigner {
@@ -246,9 +54,6 @@ private object OffsetAssigner {
   def apply(offsetCounter: LongRef, size: Int): OffsetAssigner =
     new OffsetAssigner(offsetCounter.value to offsetCounter.addAndGet(size))
 
-  def apply(baseOffset: Long, magic: Byte, messageAndOffsets: Seq[MessageAndOffset]): OffsetAssigner =
-    new OffsetAssigner(messageAndOffsets.map(_.offset))
-
 }
 
 private class OffsetAssigner(offsets: Seq[Long]) {
@@ -322,7 +127,6 @@ private class OffsetAssigner(offsets: Seq[Long]) {
  *
  */
 class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Logging {
-  private var shallowValidByteCount = -1
 
   private[kafka] def this(compressionCodec: CompressionCodec,
                           offsetCounter: LongRef,
@@ -354,33 +158,6 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi
 
   override def asRecords: MemoryRecords = MemoryRecords.readableRecords(buffer.duplicate())
 
-  private def shallowValidBytes: Int = {
-    if (shallowValidByteCount < 0) {
-      this.shallowValidByteCount = this.internalIterator(isShallow = true).map { messageAndOffset =>
-        MessageSet.entrySize(messageAndOffset.message)
-      }.sum
-    }
-    shallowValidByteCount
-  }
-
-  /** Write the messages in this set to the given channel */
-  def writeFullyTo(channel: GatheringByteChannel): Int = {
-    buffer.mark()
-    var written = 0
-    while (written < sizeInBytes)
-      written += channel.write(buffer)
-    buffer.reset()
-    written
-  }
-
-  override def isMagicValueInAllWrapperMessages(expectedMagicValue: Byte): Boolean = {
-    for (messageAndOffset <- shallowIterator) {
-      if (messageAndOffset.message.magic != expectedMagicValue)
-        return false
-    }
-    true
-  }
-
   /** default iterator that iterates over decompressed messages */
   override def iterator: Iterator[MessageAndOffset] = internalIterator()
 
@@ -388,365 +165,12 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi
   def shallowIterator: Iterator[MessageAndOffset] = internalIterator(isShallow = true)
 
   /** When flag isShallow is set to be true, we do a shallow iteration: just traverse the first level of messages. **/
-  private def internalIterator(isShallow: Boolean = false, ensureMatchingMagic: Boolean = false): Iterator[MessageAndOffset] = {
-    new IteratorTemplate[MessageAndOffset] {
-      var topIter = buffer.slice()
-      var innerIter: Iterator[MessageAndOffset] = null
-
-      def innerDone(): Boolean = (innerIter == null || !innerIter.hasNext)
-
-      def makeNextOuter: MessageAndOffset = {
-        // if there isn't at least an offset and size, we are done
-        if (topIter.remaining < 12)
-          return allDone()
-        val offset = topIter.getLong()
-        val size = topIter.getInt()
-        if(size < Message.MinMessageOverhead)
-          throw new InvalidMessageException("Message found with corrupt size (" + size + ") in shallow iterator")
-
-        // we have an incomplete message
-        if(topIter.remaining < size)
-          return allDone()
-
-        // read the current message and check correctness
-        val message = topIter.slice()
-        message.limit(size)
-        topIter.position(topIter.position + size)
-        val newMessage = new Message(message)
-        if(isShallow) {
-          MessageAndOffset(newMessage, offset)
-        } else {
-          newMessage.compressionCodec match {
-            case NoCompressionCodec =>
-              innerIter = null
-              MessageAndOffset(newMessage, offset)
-            case _ =>
-              innerIter = ByteBufferMessageSet.deepIterator(MessageAndOffset(newMessage, offset), ensureMatchingMagic)
-              if(!innerIter.hasNext)
-                innerIter = null
-              makeNext()
-          }
-        }
-      }
-
-      override def makeNext(): MessageAndOffset = {
-        if(isShallow){
-          makeNextOuter
-        } else {
-          if(innerDone())
-            makeNextOuter
-          else
-            innerIter.next()
-        }
-      }
-
-    }
-  }
-
-  def filterInto(buffer: ByteBuffer,
-                 filter: MessageAndOffset => Boolean): FilterResult = {
-    var maxTimestamp = Message.NoTimestamp
-    var offsetOfMaxTimestamp = -1L
-    var messagesRead = 0
-    var bytesRead = 0
-    var messagesRetained = 0
-    var bytesRetained = 0
-
-    for (shallowMessageAndOffset <- shallowIterator) {
-      val shallowMessage = shallowMessageAndOffset.message
-      val shallowOffset = shallowMessageAndOffset.offset
-      val size = MessageSet.entrySize(shallowMessageAndOffset.message)
-
-      messagesRead += 1
-      bytesRead += size
-
-      if (shallowMessageAndOffset.message.compressionCodec == NoCompressionCodec) {
-        if (filter(shallowMessageAndOffset)) {
-          ByteBufferMessageSet.writeMessage(buffer, shallowMessage, shallowOffset)
-          messagesRetained += 1
-          bytesRetained += size
-
-          if (shallowMessage.timestamp > maxTimestamp) {
-            maxTimestamp = shallowMessage.timestamp
-            offsetOfMaxTimestamp = shallowOffset
-          }
-        }
-        messagesRead += 1
-      } else {
-        // We use the absolute offset to decide whether to retain the message or not (this is handled by the
-        // deep iterator). Because of KAFKA-4298, we have to allow for the possibility that a previous version
-        // corrupted the log by writing a compressed message set with a wrapper magic value not matching the magic
-        // of the inner messages. This will be fixed as we recopy the messages to the destination segment.
-
-        var writeOriginalMessageSet = true
-        val retainedMessages = ArrayBuffer[MessageAndOffset]()
-        val shallowMagic = shallowMessage.magic
-
-        for (deepMessageAndOffset <- ByteBufferMessageSet.deepIterator(shallowMessageAndOffset)) {
-          messagesRead += 1
-          if (filter(deepMessageAndOffset)) {
-            // Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite
-            // the corrupted entry with correct data.
-            if (shallowMagic != deepMessageAndOffset.message.magic)
-              writeOriginalMessageSet = false
-
-            retainedMessages += deepMessageAndOffset
-            // We need the max timestamp and last offset for time index
-            if (deepMessageAndOffset.message.timestamp > maxTimestamp)
-              maxTimestamp = deepMessageAndOffset.message.timestamp
-          }
-          else writeOriginalMessageSet = false
-        }
-        offsetOfMaxTimestamp = if (retainedMessages.nonEmpty) retainedMessages.last.offset else -1L
-        // There are no messages compacted out and no message format conversion, write the original message set back
-        if (writeOriginalMessageSet)
-          ByteBufferMessageSet.writeMessage(buffer, shallowMessage, shallowOffset)
-        else if (retainedMessages.nonEmpty) {
-          val compressedSize = ByteBufferMessageSet.writeCompressedMessages(buffer, shallowMessage.compressionCodec, retainedMessages)
-          messagesRetained += 1
-          bytesRetained += compressedSize
-        }
-      }
-    }
-
-    FilterResult(messagesRead, bytesRead, messagesRetained, bytesRetained, maxTimestamp, offsetOfMaxTimestamp)
-  }
-
-  /**
-   * Update the offsets for this message set and do further validation on messages including:
-   * 1. Messages for compacted topics must have keys
-   * 2. When magic value = 1, inner messages of a compressed message set must have monotonically increasing offsets
-   *    starting from 0.
-   * 3. When magic value = 1, validate and maybe overwrite timestamps of messages.
-   *
-   * This method will convert the messages in the following scenarios:
-   * A. Magic value of a message = 0 and messageFormatVersion is 1
-   * B. Magic value of a message = 1 and messageFormatVersion is 0
-   *
-   * If no format conversion or value overwriting is required for messages, this method will perform in-place
-   * operations and avoid re-compression.
-   *
-   * Returns a ValidationAndOffsetAssignResult containing the validated message set, maximum timestamp, the offset
-   * of the shallow message with the max timestamp and a boolean indicating whether the message sizes may have changed.
-   */
-  private[kafka] def validateMessagesAndAssignOffsets(offsetCounter: LongRef,
-                                                      now: Long,
-                                                      sourceCodec: CompressionCodec,
-                                                      targetCodec: CompressionCodec,
-                                                      compactedTopic: Boolean = false,
-                                                      messageFormatVersion: Byte = Message.CurrentMagicValue,
-                                                      messageTimestampType: TimestampType,
-                                                      messageTimestampDiffMaxMs: Long): ValidationAndOffsetAssignResult = {
-    if (sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) {
-      // check the magic value
-      if (!isMagicValueInAllWrapperMessages(messageFormatVersion))
-        convertNonCompressedMessages(offsetCounter, compactedTopic, now, messageTimestampType, messageTimestampDiffMaxMs,
-          messageFormatVersion)
-      else
-        // Do in-place validation, offset assignment and maybe set timestamp
-        validateNonCompressedMessagesAndAssignOffsetInPlace(offsetCounter, now, compactedTopic, messageTimestampType,
-          messageTimestampDiffMaxMs)
-    } else {
-      // Deal with compressed messages
-      // We cannot do in place assignment in one of the following situations:
-      // 1. Source and target compression codec are different
-      // 2. When magic value to use is 0 because offsets need to be overwritten
-      // 3. When magic value to use is above 0, but some fields of inner messages need to be overwritten.
-      // 4. Message format conversion is needed.
-
-      // No in place assignment situation 1 and 2
-      var inPlaceAssignment = sourceCodec == targetCodec && messageFormatVersion > Message.MagicValue_V0
-
-      var maxTimestamp = Message.NoTimestamp
-      var offsetOfMaxTimestamp = -1L
-      val expectedInnerOffset = new LongRef(0)
-      val validatedMessages = new mutable.ArrayBuffer[Message]
-
-      this.internalIterator(isShallow = false, ensureMatchingMagic = true).foreach { messageAndOffset =>
-        val message = messageAndOffset.message
-        validateMessageKey(message, compactedTopic)
-
-        if (message.magic > Message.MagicValue_V0 && messageFormatVersion > Message.MagicValue_V0) {
-          // No in place assignment situation 3
-          // Validate the timestamp
-          validateTimestamp(message, now, messageTimestampType, messageTimestampDiffMaxMs)
-          // Check if we need to overwrite offset
-          if (messageAndOffset.offset != expectedInnerOffset.getAndIncrement())
-            inPlaceAssignment = false
-          if (message.timestamp > maxTimestamp) {
-            maxTimestamp = message.timestamp
-            offsetOfMaxTimestamp = offsetCounter.value + expectedInnerOffset.value - 1
-          }
-        }
-
-        if (sourceCodec != NoCompressionCodec && message.compressionCodec != NoCompressionCodec)
-          throw new InvalidMessageException("Compressed outer message should not have an inner message with a " +
-            s"compression attribute set: $message")
-
-        // No in place assignment situation 4
-        if (message.magic != messageFormatVersion)
-          inPlaceAssignment = false
-
-        validatedMessages += message.toFormatVersion(messageFormatVersion)
-      }
-
-      if (!inPlaceAssignment) {
-        // Cannot do in place assignment.
-        val (largestTimestampOfMessageSet, offsetOfMaxTimestampInMessageSet) = {
-          if (messageFormatVersion == Message.MagicValue_V0)
-            (Some(Message.NoTimestamp), -1L)
-          else if (messageTimestampType == TimestampType.CREATE_TIME)
-            (Some(maxTimestamp), {if (targetCodec == NoCompressionCodec) offsetOfMaxTimestamp else offsetCounter.value + validatedMessages.length - 1})
-          else // Log append time
-            (Some(now), {if (targetCodec == NoCompressionCodec) offsetCounter.value else offsetCounter.value + validatedMessages.length - 1})
-        }
-
-        ValidationAndOffsetAssignResult(validatedMessages = new ByteBufferMessageSet(compressionCodec = targetCodec,
-                                                                                     offsetCounter = offsetCounter,
-                                                                                     wrapperMessageTimestamp = largestTimestampOfMessageSet,
-                                                                                     timestampType = messageTimestampType,
-                                                                                     messages = validatedMessages: _*),
-                                        maxTimestamp = largestTimestampOfMessageSet.get,
-                                        offsetOfMaxTimestamp = offsetOfMaxTimestampInMessageSet,
-                                        messageSizeMaybeChanged = true)
-      } else {
-        // Do not do re-compression but simply update the offset, timestamp and attributes field of the wrapper message.
-        buffer.putLong(0, offsetCounter.addAndGet(validatedMessages.size) - 1)
-        // validate the messages
-        validatedMessages.foreach(_.ensureValid())
-
-        var crcUpdateNeeded = true
-        val timestampOffset = MessageSet.LogOverhead + Message.TimestampOffset
-        val attributeOffset = MessageSet.LogOverhead + Message.AttributesOffset
-        val timestamp = buffer.getLong(timestampOffset)
-        val attributes = buffer.get(attributeOffset)
-        buffer.putLong(timestampOffset, maxTimestamp)
-        if (messageTimestampType == TimestampType.CREATE_TIME && timestamp == maxTimestamp)
-          // We don't need to recompute crc if the timestamp is not updated.
-          crcUpdateNeeded = false
-        else if (messageTimestampType == TimestampType.LOG_APPEND_TIME) {
-          // Set timestamp type and timestamp
-          buffer.putLong(timestampOffset, now)
-          buffer.put(attributeOffset, messageTimestampType.updateAttributes(attributes))
-        }
-
-        if (crcUpdateNeeded) {
-          // need to recompute the crc value
-          buffer.position(MessageSet.LogOverhead)
-          val wrapperMessage = new Message(buffer.slice())
-          Utils.writeUnsignedInt(buffer, MessageSet.LogOverhead + Message.CrcOffset, wrapperMessage.computeChecksum)
-        }
-        buffer.rewind()
-        // For compressed messages,
-        ValidationAndOffsetAssignResult(validatedMessages = this,
-                                        maxTimestamp = buffer.getLong(timestampOffset),
-                                        offsetOfMaxTimestamp = buffer.getLong(0),
-                                        messageSizeMaybeChanged = false)
-      }
-    }
-  }
-
-  // We create this method to avoid a memory copy. It reads from the original message set and directly
-  // writes the converted messages into new message set buffer. Hence we don't need to allocate memory for each
-  // individual message during message format conversion.
-  private def convertNonCompressedMessages(offsetCounter: LongRef,
-                                           compactedTopic: Boolean,
-                                           now: Long,
-                                           timestampType: TimestampType,
-                                           messageTimestampDiffMaxMs: Long,
-                                           toMagicValue: Byte): ValidationAndOffsetAssignResult = {
-    val sizeInBytesAfterConversion = shallowValidBytes + this.internalIterator(isShallow = true).map { messageAndOffset =>
-      Message.headerSizeDiff(messageAndOffset.message.magic, toMagicValue)
-    }.sum
-    val newBuffer = ByteBuffer.allocate(sizeInBytesAfterConversion)
-    var newMessagePosition = 0
-    var maxTimestamp = Message.NoTimestamp
-    var offsetOfMaxTimestamp = -1L
-    this.internalIterator(isShallow = true).foreach { case MessageAndOffset(message, _) =>
-      validateMessageKey(message, compactedTopic)
-      validateTimestamp(message, now, timestampType, messageTimestampDiffMaxMs)
-      newBuffer.position(newMessagePosition)
-      newBuffer.putLong(offsetCounter.getAndIncrement())
-      val newMessageSize = message.size + Message.headerSizeDiff(message.magic, toMagicValue)
-      newBuffer.putInt(newMessageSize)
-      val newMessageBuffer = newBuffer.slice()
-      newMessageBuffer.limit(newMessageSize)
-      message.convertToBuffer(toMagicValue, newMessageBuffer, now, timestampType)
-      if (toMagicValue > Message.MagicValue_V0) {
-        val timestamp = newMessageBuffer.getLong(Message.TimestampOffset)
-        if (maxTimestamp < timestamp) {
-          maxTimestamp = timestamp
-          offsetOfMaxTimestamp = offsetCounter.value - 1
-        }
-      }
-      newMessagePosition += MessageSet.LogOverhead + newMessageSize
-    }
-    newBuffer.rewind()
-    new ValidationAndOffsetAssignResult(validatedMessages = new ByteBufferMessageSet(newBuffer),
-                                        maxTimestamp = maxTimestamp,
-                                        offsetOfMaxTimestamp = offsetOfMaxTimestamp,
-                                        messageSizeMaybeChanged = true)
-  }
-
-  private def validateNonCompressedMessagesAndAssignOffsetInPlace(offsetCounter: LongRef,
-                                                                  now: Long,
-                                                                  compactedTopic: Boolean,
-                                                                  timestampType: TimestampType,
-                                                                  timestampDiffMaxMs: Long): ValidationAndOffsetAssignResult = {
-    // do in-place validation and offset assignment
-    var messagePosition = 0
-    var maxTimestamp = Message.NoTimestamp
-    var offsetOfMaxTimestamp = -1L
-    buffer.mark()
-    while (messagePosition < sizeInBytes - MessageSet.LogOverhead) {
-      buffer.position(messagePosition)
-      buffer.putLong(offsetCounter.getAndIncrement())
-      val messageSize = buffer.getInt()
-      val messageBuffer = buffer.slice()
-      messageBuffer.limit(messageSize)
-      val message = new Message(messageBuffer)
-      validateMessageKey(message, compactedTopic)
-      if (message.magic > Message.MagicValue_V0) {
-        validateTimestamp(message, now, timestampType, timestampDiffMaxMs)
-        if (timestampType == TimestampType.LOG_APPEND_TIME) {
-          message.buffer.putLong(Message.TimestampOffset, now)
-          message.buffer.put(Message.AttributesOffset, timestampType.updateAttributes(message.attributes))
-          Utils.writeUnsignedInt(message.buffer, Message.CrcOffset, message.computeChecksum)
-        }
-        if (message.timestamp > maxTimestamp) {
-          maxTimestamp = message.timestamp
-          offsetOfMaxTimestamp = offsetCounter.value - 1
-        }
-      }
-
-      messagePosition += MessageSet.LogOverhead + messageSize
-    }
-    buffer.reset()
-    ValidationAndOffsetAssignResult(validatedMessages = this,
-                                    maxTimestamp = maxTimestamp,
-                                    offsetOfMaxTimestamp = offsetOfMaxTimestamp,
-                                    messageSizeMaybeChanged = false)
-  }
-
-  private def validateMessageKey(message: Message, compactedTopic: Boolean) {
-    if (compactedTopic && !message.hasKey)
-      throw new InvalidMessageException("Compacted topic cannot accept message without key.")
-  }
-
-  /**
-   * This method validates the timestamps of a message.
-   * If the message is using create time, this method checks if it is within acceptable range.
-   */
-  private def validateTimestamp(message: Message,
-                                now: Long,
-                                timestampType: TimestampType,
-                                timestampDiffMaxMs: Long) {
-    if (timestampType == TimestampType.CREATE_TIME && math.abs(message.timestamp - now) > timestampDiffMaxMs)
-      throw new InvalidTimestampException(s"Timestamp ${message.timestamp} of message is out of range. " +
-        s"The timestamp should be within [${now - timestampDiffMaxMs}, ${now + timestampDiffMaxMs}")
-    if (message.timestampType == TimestampType.LOG_APPEND_TIME)
-      throw new InvalidTimestampException(s"Invalid timestamp type in message $message. Producer should not set " +
-        s"timestamp type to LogAppendTime.")
+  private def internalIterator(isShallow: Boolean = false): Iterator[MessageAndOffset] = {
+    val entries = if (isShallow)
+      asRecords.shallowIterator
+    else
+      asRecords.deepIterator
+    entries.asScala.map(MessageAndOffset.fromLogEntry)
   }
 
   /**
@@ -757,7 +181,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi
   /**
    * The total number of bytes in this message set not including any partial, trailing messages
    */
-  def validBytes: Int = shallowValidBytes
+  def validBytes: Int = asRecords.validBytes
 
   /**
    * Two message sets are equal if their respective byte buffers are equal
@@ -773,8 +197,3 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi
   override def hashCode: Int = buffer.hashCode
 
 }
-
-case class ValidationAndOffsetAssignResult(validatedMessages: ByteBufferMessageSet,
-                                           maxTimestamp: Long,
-                                           offsetOfMaxTimestamp: Long,
-                                           messageSizeMaybeChanged: Boolean)

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/message/Message.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala
index bb91078..175b7e9 100755
--- a/core/src/main/scala/kafka/message/Message.scala
+++ b/core/src/main/scala/kafka/message/Message.scala
@@ -19,7 +19,7 @@ package kafka.message
 
 import java.nio._
 
-import org.apache.kafka.common.record.TimestampType
+import org.apache.kafka.common.record.{Record, TimestampType}
 
 import scala.math._
 import kafka.utils._
@@ -98,6 +98,11 @@ object Message {
     MessageHeaderSizeMap(toMagicValue) - MessageHeaderSizeMap(fromMagicValue)
 
 
+  def fromRecord(record: Record): Message = {
+    val wrapperTimestamp: Option[Long] = if (record.wrapperRecordTimestamp() == null) None else Some(record.wrapperRecordTimestamp())
+    val wrapperTimestampType = Option(record.wrapperRecordTimestampType())
+    new Message(record.buffer, wrapperTimestamp, wrapperTimestampType)
+  }
 }
 
 /**
@@ -134,6 +139,15 @@ class Message(val buffer: ByteBuffer,
   
   import kafka.message.Message._
 
+  private[message] def asRecord: Record = {
+    wrapperMessageTimestamp match {
+      case None => new Record(buffer)
+      case Some(timestamp) =>
+        val timestampType = wrapperMessageTimestampType.orNull
+        new Record(buffer, timestamp, timestampType)
+    }
+  }
+
   /**
    * A constructor to create a Message
    * @param bytes The payload of the message
@@ -328,52 +342,6 @@ class Message(val buffer: ByteBuffer,
   def key: ByteBuffer = sliceDelimited(keySizeOffset)
 
   /**
-   * convert the message to specified format
-   */
-  def toFormatVersion(toMagicValue: Byte): Message = {
-    if (magic == toMagicValue)
-      this
-    else {
-      val byteBuffer = ByteBuffer.allocate(size + Message.headerSizeDiff(magic, toMagicValue))
-      // Copy bytes from old messages to new message
-      convertToBuffer(toMagicValue, byteBuffer, Message.NoTimestamp)
-      new Message(byteBuffer)
-    }
-  }
-
-  def convertToBuffer(toMagicValue: Byte,
-                      byteBuffer: ByteBuffer,
-                      now: Long,
-                      timestampType: TimestampType = wrapperMessageTimestampType.getOrElse(TimestampType.forAttributes(attributes))) {
-    if (byteBuffer.remaining() < size + headerSizeDiff(magic, toMagicValue))
-      throw new IndexOutOfBoundsException("The byte buffer does not have enough capacity to hold new message format " +
-        s"version $toMagicValue")
-    if (toMagicValue == Message.MagicValue_V1) {
-      // Up-conversion, reserve CRC and update magic byte
-      byteBuffer.position(Message.MagicOffset)
-      byteBuffer.put(Message.MagicValue_V1)
-      byteBuffer.put(timestampType.updateAttributes(attributes))
-      // Up-conversion, insert the timestamp field
-      if (timestampType == TimestampType.LOG_APPEND_TIME)
-        byteBuffer.putLong(now)
-      else
-        byteBuffer.putLong(Message.NoTimestamp)
-      byteBuffer.put(buffer.array(), buffer.arrayOffset() + Message.KeySizeOffset_V0, size - Message.KeySizeOffset_V0)
-    } else {
-      // Down-conversion, reserve CRC and update magic byte
-      byteBuffer.position(Message.MagicOffset)
-      byteBuffer.put(Message.MagicValue_V0)
-      byteBuffer.put(TimestampType.CREATE_TIME.updateAttributes(attributes))
-      // Down-conversion, skip the timestamp field
-      byteBuffer.put(buffer.array(), buffer.arrayOffset() + Message.KeySizeOffset_V1, size - Message.KeySizeOffset_V1)
-    }
-    // update crc value
-    val newMessage = new Message(byteBuffer)
-    Utils.writeUnsignedInt(byteBuffer, Message.CrcOffset, newMessage.computeChecksum)
-    byteBuffer.rewind()
-  }
-
-  /**
    * Read a size-delimited byte buffer starting at the given offset
    */
   private def sliceDelimited(start: Int): ByteBuffer = {
@@ -399,7 +367,7 @@ class Message(val buffer: ByteBuffer,
     if (timestamp < 0 && timestamp != NoTimestamp)
       throw new IllegalArgumentException(s"Invalid message timestamp $timestamp")
     if (magic == MagicValue_V0 && timestamp != NoTimestamp)
-      throw new IllegalArgumentException(s"Invalid timestamp $timestamp. Timestamp must be ${NoTimestamp} when magic = ${MagicValue_V0}")
+      throw new IllegalArgumentException(s"Invalid timestamp $timestamp. Timestamp must be $NoTimestamp when magic = $MagicValue_V0")
   }
 
   override def toString: String = {

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/message/MessageAndOffset.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/message/MessageAndOffset.scala b/core/src/main/scala/kafka/message/MessageAndOffset.scala
index fab6898..46630c6 100644
--- a/core/src/main/scala/kafka/message/MessageAndOffset.scala
+++ b/core/src/main/scala/kafka/message/MessageAndOffset.scala
@@ -17,6 +17,13 @@
 
 package kafka.message
 
+import org.apache.kafka.common.record.LogEntry
+
+object MessageAndOffset {
+  def fromLogEntry(logEntry : LogEntry): MessageAndOffset = {
+    MessageAndOffset(Message.fromRecord(logEntry.record), logEntry.offset)
+  }
+}
 
 case class MessageAndOffset(message: Message, offset: Long) {
   
@@ -28,9 +35,10 @@ case class MessageAndOffset(message: Message, offset: Long) {
   /**
    * We need to decompress the message, if required, to get the offset of the first uncompressed message.
    */
-  def firstOffset: Long = message.compressionCodec match {
-    case NoCompressionCodec => offset
-    case _ => ByteBufferMessageSet.deepIterator(this).next().offset
+  def firstOffset: Long = toLogEntry.firstOffset
+
+  def toLogEntry: LogEntry = {
+    LogEntry.create(offset, message.asRecord)
   }
 }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/message/MessageSet.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/message/MessageSet.scala b/core/src/main/scala/kafka/message/MessageSet.scala
index ffa27fa..a44a362 100644
--- a/core/src/main/scala/kafka/message/MessageSet.scala
+++ b/core/src/main/scala/kafka/message/MessageSet.scala
@@ -18,7 +18,6 @@
 package kafka.message
 
 import java.nio._
-import java.nio.channels._
 
 import org.apache.kafka.common.record.Records
 
@@ -73,11 +72,6 @@ case class MagicAndTimestamp(magic: Byte, timestamp: Long)
 abstract class MessageSet extends Iterable[MessageAndOffset] {
 
   /**
-   * Check if all the wrapper messages in the message set have the expected magic value
-   */
-  def isMagicValueInAllWrapperMessages(expectedMagicValue: Byte): Boolean
-
-  /**
    * Provides an iterator over the message/offset pairs in this set
    */
   def iterator: Iterator[MessageAndOffset]
@@ -99,7 +93,7 @@ abstract class MessageSet extends Iterable[MessageAndOffset] {
   override def toString: String = {
     val builder = new StringBuilder()
     builder.append(getClass.getSimpleName + "(")
-    val iter = this.iterator
+    val iter = this.asRecords.shallowIterator()
     var i = 0
     while(iter.hasNext && i < 100) {
       val message = iter.next

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
index b3a8751..f959ce2 100755
--- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
+++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
@@ -21,7 +21,6 @@ import java.util.concurrent.locks.ReentrantLock
 
 import kafka.cluster.BrokerEndPoint
 import kafka.consumer.PartitionTopicInfo
-import kafka.message.ByteBufferMessageSet
 import kafka.utils.{DelayedItem, Pool, ShutdownableThread}
 import kafka.common.{ClientIdAndBroker, KafkaException}
 import kafka.metrics.KafkaMetricsGroup
@@ -38,6 +37,7 @@ import java.util.concurrent.atomic.AtomicLong
 import com.yammer.metrics.core.Gauge
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.internals.PartitionStates
+import org.apache.kafka.common.record.MemoryRecords
 
 /**
  *  Abstract class for fetching data from multiple partitions from the same broker.
@@ -144,15 +144,15 @@ abstract class AbstractFetcherThread(name: String,
               Errors.forCode(partitionData.errorCode) match {
                 case Errors.NONE =>
                   try {
-                    val messages = partitionData.toByteBufferMessageSet
-                    val newOffset = messages.shallowIterator.toSeq.lastOption.map(_.nextOffset).getOrElse(
+                    val records = partitionData.toRecords
+                    val newOffset = records.shallowIterator.asScala.toSeq.lastOption.map(_.nextOffset).getOrElse(
                       currentPartitionFetchState.offset)
 
                     fetcherLagStats.getAndMaybePut(topic, partitionId).lag = Math.max(0L, partitionData.highWatermark - newOffset)
                     // Once we hand off the partition data to the subclass, we can't mess with it any more in this thread
                     processPartitionData(topicPartition, currentPartitionFetchState.offset, partitionData)
 
-                    val validBytes = messages.validBytes
+                    val validBytes = records.validBytes
                     if (validBytes > 0) {
                       // Update partitionStates only if there is no exception during processPartitionData
                       partitionStates.updateAndMoveToEnd(topicPartition, new PartitionFetchState(newOffset))
@@ -260,7 +260,7 @@ object AbstractFetcherThread {
   trait PartitionData {
     def errorCode: Short
     def exception: Option[Throwable]
-    def toByteBufferMessageSet: ByteBufferMessageSet
+    def toRecords: MemoryRecords
     def highWatermark: Long
   }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 001051f..743c994 100644
--- a/core/src/main/scala/kafka/server/DelayedFetch.scala
+++ b/core/src/main/scala/kafka/server/DelayedFetch.scala
@@ -19,7 +19,6 @@ package kafka.server
 
 import java.util.concurrent.TimeUnit
 
-import kafka.api.FetchResponsePartitionData
 import kafka.common.TopicAndPartition
 import kafka.metrics.KafkaMetricsGroup
 import org.apache.kafka.common.TopicPartition
@@ -59,7 +58,7 @@ class DelayedFetch(delayMs: Long,
                    fetchMetadata: FetchMetadata,
                    replicaManager: ReplicaManager,
                    quota: ReplicaQuota,
-                   responseCallback: Seq[(TopicAndPartition, FetchResponsePartitionData)] => Unit)
+                   responseCallback: Seq[(TopicAndPartition, FetchPartitionData)] => Unit)
   extends DelayedOperation(delayMs) {
 
   /**
@@ -152,7 +151,7 @@ class DelayedFetch(delayMs: Long,
     )
 
     val fetchPartitionData = logReadResults.map { case (tp, result) =>
-      tp -> FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)
+      tp -> FetchPartitionData(result.errorCode, result.hw, result.info.records)
     }
 
     responseCallback(fetchPartitionData)

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 9d6d437..acfb5b0 100644
--- a/core/src/main/scala/kafka/server/FetchDataInfo.scala
+++ b/core/src/main/scala/kafka/server/FetchDataInfo.scala
@@ -17,7 +17,8 @@
 
 package kafka.server
 
-import kafka.message.MessageSet
+import org.apache.kafka.common.record.Records
 
-case class FetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata, messageSet: MessageSet,
-                         firstMessageSetIncomplete: Boolean = false)
+case class FetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata,
+                         records: Records,
+                         firstEntryIncomplete: Boolean = false)

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 447e4e2..a4a1e2a 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -23,14 +23,13 @@ import java.util.{Collections, Properties}
 import java.util
 
 import kafka.admin.{AdminUtils, RackAwareMode}
-import kafka.api.{ControlledShutdownRequest, ControlledShutdownResponse, FetchResponsePartitionData}
+import kafka.api.{ControlledShutdownRequest, ControlledShutdownResponse}
 import kafka.cluster.Partition
 import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota}
 import kafka.common._
 import kafka.controller.KafkaController
 import kafka.coordinator.{GroupCoordinator, JoinGroupResult}
 import kafka.log._
-import kafka.message.{ByteBufferMessageSet, Message}
 import kafka.network._
 import kafka.network.RequestChannel.{Response, Session}
 import kafka.security.auth
@@ -39,7 +38,7 @@ import kafka.utils.{Logging, ZKGroupTopicDirs, ZkUtils}
 import org.apache.kafka.common.errors.{ClusterAuthorizationException, NotLeaderForPartitionException, TopicExistsException, UnknownTopicOrPartitionException, UnsupportedForMessageFormatException}
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.protocol.{ApiKeys, Errors, Protocol, SecurityProtocol}
-import org.apache.kafka.common.record.MemoryRecords
+import org.apache.kafka.common.record.{MemoryRecords, Record}
 import org.apache.kafka.common.requests._
 import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
 import org.apache.kafka.common.utils.{Time, Utils}
@@ -359,9 +358,9 @@ class KafkaApis(val requestChannel: RequestChannel,
 
       val mergedResponseStatus = responseStatus ++
         unauthorizedForWriteRequestInfo.mapValues(_ =>
-           new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED.code, -1, Message.NoTimestamp)) ++
+           new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED.code, -1, Record.NO_TIMESTAMP)) ++
         nonExistingOrUnauthorizedForDescribeTopics.mapValues(_ =>
-           new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION.code, -1, Message.NoTimestamp))
+           new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION.code, -1, Record.NO_TIMESTAMP))
 
       var errorInResponse = false
 
@@ -422,17 +421,12 @@ class KafkaApis(val requestChannel: RequestChannel,
     else {
       val internalTopicsAllowed = request.header.clientId == AdminUtils.AdminClientId
 
-      // Convert Records to ByteBufferMessageSet
-      val authorizedMessagesPerPartition = authorizedRequestInfo.map {
-        case (topicPartition, records) => (topicPartition, new ByteBufferMessageSet(records.buffer))
-      }
-
       // call the replica manager to append messages to the replicas
-      replicaManager.appendMessages(
+      replicaManager.appendRecords(
         produceRequest.timeout.toLong,
         produceRequest.acks,
         internalTopicsAllowed,
-        authorizedMessagesPerPartition,
+        authorizedRequestInfo,
         sendResponseCallback)
 
       // if the request is put into the purgatory, it will have a held reference
@@ -467,7 +461,7 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
 
     // the callback for sending a fetch response
-    def sendResponseCallback(responsePartitionData: Seq[(TopicAndPartition, FetchResponsePartitionData)]) {
+    def sendResponseCallback(responsePartitionData: Seq[(TopicAndPartition, FetchPartitionData)]) {
       val convertedPartitionData = {
         // Need to down-convert message when consumer only takes magic value 0.
         responsePartitionData.map { case (tp, data) =>
@@ -480,18 +474,18 @@ class KafkaApis(val requestChannel: RequestChannel,
           // Please note that if the message format is changed from a higher version back to lower version this
           // test might break because some messages in new message format can be delivered to consumers before 0.10.0.0
           // without format down conversion.
-          val convertedData = if (versionId <= 1 && replicaManager.getMessageFormatVersion(tp).exists(_ > Message.MagicValue_V0) &&
-            !data.messages.isMagicValueInAllWrapperMessages(Message.MagicValue_V0)) {
+          val convertedData = if (versionId <= 1 && replicaManager.getMagicAndTimestampType(tp).exists(_._1 > Record.MAGIC_VALUE_V0) &&
+            !data.records.hasMatchingShallowMagic(Record.MAGIC_VALUE_V0)) {
             trace(s"Down converting message to V0 for fetch request from $clientId")
-            new FetchResponsePartitionData(data.error, data.hw, data.messages.asInstanceOf[FileMessageSet].toMessageFormat(Message.MagicValue_V0))
+            FetchPartitionData(data.error, data.hw, data.records.toMessageFormat(Record.MAGIC_VALUE_V0))
           } else data
 
-          val records = convertedData.messages.asRecords
-          new TopicPartition(tp.topic, tp.partition) -> new FetchResponse.PartitionData(convertedData.error, convertedData.hw, records)
+          new TopicPartition(tp.topic, tp.partition) -> new FetchResponse.PartitionData(convertedData.error, convertedData.hw, convertedData.records)
         }
       }
 
       val mergedPartitionData = convertedPartitionData ++ unauthorizedForReadPartitionData ++ nonExistingOrUnauthorizedForDescribePartitionData
+
       val fetchedPartitionData = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData]()
 
       mergedPartitionData.foreach { case (topicPartition, data) =>
@@ -660,7 +654,7 @@ class KafkaApis(val requestChannel: RequestChannel,
 
           val found = {
             if (fromConsumer && timestamp == ListOffsetRequest.LATEST_TIMESTAMP)
-              TimestampOffset(Message.NoTimestamp, localReplica.highWatermark.messageOffset)
+              TimestampOffset(Record.NO_TIMESTAMP, localReplica.highWatermark.messageOffset)
             else {
               def allowed(timestampOffset: TimestampOffset): Boolean =
                 !fromConsumer || timestampOffset.offset <= localReplica.highWatermark.messageOffset

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
index 84c3feb..54a2e05 100644
--- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
+++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
@@ -23,7 +23,6 @@ import java.util
 import kafka.admin.AdminUtils
 import kafka.cluster.BrokerEndPoint
 import kafka.log.LogConfig
-import kafka.message.ByteBufferMessageSet
 import kafka.api.{KAFKA_0_10_0_IV0, KAFKA_0_10_1_IV1, KAFKA_0_10_1_IV2, KAFKA_0_9_0}
 import kafka.common.{KafkaStorageException, TopicAndPartition}
 import ReplicaFetcherThread._
@@ -119,19 +118,19 @@ class ReplicaFetcherThread(name: String,
       val topic = topicPartition.topic
       val partitionId = topicPartition.partition
       val replica = replicaMgr.getReplica(topic, partitionId).get
-      val messageSet = partitionData.toByteBufferMessageSet
+      val records = partitionData.toRecords
 
-      maybeWarnIfMessageOversized(messageSet, topicPartition)
+      maybeWarnIfOversizedRecords(records, topicPartition)
 
       if (fetchOffset != replica.logEndOffset.messageOffset)
         throw new RuntimeException("Offset mismatch for partition %s: fetched offset = %d, log end offset = %d.".format(topicPartition, fetchOffset, replica.logEndOffset.messageOffset))
       if (logger.isTraceEnabled)
         trace("Follower %d has replica log end offset %d for partition %s. Received %d messages and leader hw %d"
-          .format(replica.brokerId, replica.logEndOffset.messageOffset, topicPartition, messageSet.sizeInBytes, partitionData.highWatermark))
-      replica.log.get.append(messageSet, assignOffsets = false)
+          .format(replica.brokerId, replica.logEndOffset.messageOffset, topicPartition, records.sizeInBytes, partitionData.highWatermark))
+      replica.log.get.append(records, assignOffsets = false)
       if (logger.isTraceEnabled)
         trace("Follower %d has replica log end offset %d after appending %d bytes of messages for partition %s"
-          .format(replica.brokerId, replica.logEndOffset.messageOffset, messageSet.sizeInBytes, topicPartition))
+          .format(replica.brokerId, replica.logEndOffset.messageOffset, records.sizeInBytes, topicPartition))
       val followerHighWatermark = replica.logEndOffset.messageOffset.min(partitionData.highWatermark)
       // for the follower replica, we do not need to keep
       // its segment base offset the physical position,
@@ -141,7 +140,7 @@ class ReplicaFetcherThread(name: String,
         trace("Follower %d set replica high watermark for partition [%s,%d] to %s"
           .format(replica.brokerId, topic, partitionId, followerHighWatermark))
       if (quota.isThrottled(TopicAndPartition(topic, partitionId)))
-        quota.record(messageSet.sizeInBytes)
+        quota.record(records.sizeInBytes)
     } catch {
       case e: KafkaStorageException =>
         fatal(s"Disk error while replicating data for $topicPartition", e)
@@ -149,9 +148,9 @@ class ReplicaFetcherThread(name: String,
     }
   }
 
-  def maybeWarnIfMessageOversized(messageSet: ByteBufferMessageSet, topicPartition: TopicPartition): Unit = {
+  def maybeWarnIfOversizedRecords(records: MemoryRecords, topicPartition: TopicPartition): Unit = {
     // oversized messages don't cause replication to fail from fetch request version 3 (KIP-74)
-    if (fetchRequestVersion <= 2 && messageSet.sizeInBytes > 0 && messageSet.validBytes <= 0)
+    if (fetchRequestVersion <= 2 && records.sizeInBytes > 0 && records.validBytes <= 0)
       error(s"Replication is failing due to a message that is greater than replica.fetch.max.bytes for partition $topicPartition. " +
         "This generally occurs when the max.message.bytes has been overridden to exceed this value and a suitably large " +
         "message has also been sent. To fix this problem increase replica.fetch.max.bytes in your broker config to be " +
@@ -323,9 +322,8 @@ object ReplicaFetcherThread {
 
     def errorCode: Short = underlying.errorCode
 
-    def toByteBufferMessageSet: ByteBufferMessageSet = {
-      val buffer = underlying.records.asInstanceOf[MemoryRecords].buffer
-      new ByteBufferMessageSet(buffer)
+    def toRecords: MemoryRecords = {
+      underlying.records.asInstanceOf[MemoryRecords]
     }
 
     def highWatermark: Long = underlying.highWatermark


[9/9] kafka git commit: KAFKA-4390; Replace MessageSet usage with client-side alternatives

Posted by jg...@apache.org.
KAFKA-4390; Replace MessageSet usage with client-side alternatives

Author: Jason Gustafson <ja...@confluent.io>

Reviewers: Ismael Juma <is...@juma.me.uk>, Guozhang Wang <wa...@gmail.com>, Jun Rao <ju...@gmail.com>

Closes #2140 from hachikuji/KAFKA4390


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/67f1e5b9
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/67f1e5b9
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/67f1e5b9

Branch: refs/heads/trunk
Commit: 67f1e5b91bf073151ff57d5d656693e385726697
Parents: 6626b05
Author: Jason Gustafson <ja...@confluent.io>
Authored: Tue Dec 13 10:26:25 2016 -0800
Committer: Jason Gustafson <ja...@confluent.io>
Committed: Tue Dec 13 10:26:25 2016 -0800

----------------------------------------------------------------------
 checkstyle/import-control.xml                   |   1 +
 .../clients/consumer/internals/Fetcher.java     |   7 +-
 .../clients/producer/internals/BufferPool.java  |   2 +-
 .../producer/internals/RecordAccumulator.java   |  30 +-
 .../clients/producer/internals/RecordBatch.java |  58 +-
 .../clients/producer/internals/Sender.java      |  10 +-
 .../kafka/common/record/AbstractRecords.java    |  92 +++
 .../common/record/ByteBufferInputStream.java    |  37 +-
 .../common/record/ByteBufferLogInputStream.java | 119 ++++
 .../common/record/ByteBufferOutputStream.java   |  54 +-
 .../kafka/common/record/CompressionType.java    |   2 +-
 .../apache/kafka/common/record/Compressor.java  | 332 ----------
 .../kafka/common/record/FileLogInputStream.java | 166 +++++
 .../apache/kafka/common/record/FileRecords.java | 465 ++++++++++++--
 .../common/record/InvalidRecordException.java   |   4 +-
 .../apache/kafka/common/record/LogEntry.java    | 151 ++++-
 .../kafka/common/record/LogInputStream.java     |  12 +-
 .../kafka/common/record/MemoryRecords.java      | 443 +++++++++-----
 .../common/record/MemoryRecordsBuilder.java     | 461 ++++++++++++++
 .../org/apache/kafka/common/record/Record.java  | 538 +++++++++++-----
 .../org/apache/kafka/common/record/Records.java |  64 +-
 .../kafka/common/record/RecordsIterator.java    | 179 ++++--
 .../kafka/common/record/TimestampType.java      |   1 +
 .../org/apache/kafka/common/utils/Utils.java    |  51 ++
 .../clients/consumer/KafkaConsumerTest.java     |   7 +-
 .../clients/consumer/internals/FetcherTest.java | 100 +--
 .../internals/RecordAccumulatorTest.java        |  51 +-
 .../record/ByteBufferLogInputStreamTest.java    | 110 ++++
 .../kafka/common/record/FileRecordsTest.java    | 410 +++++++++++++
 .../common/record/MemoryRecordsBuilderTest.java | 253 ++++++++
 .../kafka/common/record/MemoryRecordsTest.java  | 194 +++++-
 .../apache/kafka/common/record/RecordTest.java  |  46 +-
 .../kafka/common/record/SimpleRecordTest.java   |  52 +-
 .../kafka/common/record/TimestampTypeTest.java  |  37 ++
 .../java/org/apache/kafka/test/TestUtils.java   |  32 +-
 core/src/main/scala/kafka/api/ApiVersion.scala  |  20 +-
 .../main/scala/kafka/cluster/Partition.scala    |  10 +-
 .../kafka/consumer/ConsumerFetcherThread.scala  |  10 +-
 .../kafka/coordinator/GroupCoordinator.scala    |   6 +-
 .../coordinator/GroupMetadataManager.scala      | 170 +++--
 .../main/scala/kafka/log/FileMessageSet.scala   | 445 --------------
 core/src/main/scala/kafka/log/Log.scala         | 128 ++--
 core/src/main/scala/kafka/log/LogCleaner.scala  |  55 +-
 core/src/main/scala/kafka/log/LogManager.scala  |   2 +-
 core/src/main/scala/kafka/log/LogSegment.scala  |  98 +--
 .../src/main/scala/kafka/log/LogValidator.scala | 239 ++++++++
 core/src/main/scala/kafka/log/TimeIndex.scala   |   6 +-
 .../kafka/message/ByteBufferMessageSet.scala    | 613 +------------------
 core/src/main/scala/kafka/message/Message.scala |  64 +-
 .../scala/kafka/message/MessageAndOffset.scala  |  14 +-
 .../main/scala/kafka/message/MessageSet.scala   |   8 +-
 .../kafka/server/AbstractFetcherThread.scala    |  10 +-
 .../main/scala/kafka/server/DelayedFetch.scala  |   5 +-
 .../main/scala/kafka/server/FetchDataInfo.scala |   7 +-
 .../src/main/scala/kafka/server/KafkaApis.scala |  32 +-
 .../kafka/server/ReplicaFetcherThread.scala     |  22 +-
 .../scala/kafka/server/ReplicaManager.scala     |  85 +--
 .../scala/kafka/tools/DumpLogSegments.scala     | 150 ++---
 .../kafka/tools/ReplicaVerificationTool.scala   |  52 +-
 .../api/GroupCoordinatorIntegrationTest.scala   |   6 +-
 .../scala/kafka/tools/TestLogCleaning.scala     |  19 +-
 .../test/scala/other/kafka/StressTestLog.scala  |  22 +-
 .../other/kafka/TestLinearWriteSpeed.scala      |  19 +-
 .../unit/kafka/admin/DeleteTopicTest.scala      |   2 +-
 .../GroupCoordinatorResponseTest.scala          |  30 +-
 .../coordinator/GroupMetadataManagerTest.scala  | 100 ++-
 .../unit/kafka/log/BrokerCompressionTest.scala  |  15 +-
 .../unit/kafka/log/FileMessageSetTest.scala     | 354 -----------
 .../kafka/log/LogCleanerIntegrationTest.scala   |  66 +-
 .../log/LogCleanerLagIntegrationTest.scala      |  24 +-
 .../unit/kafka/log/LogCleanerManagerTest.scala  |  29 +-
 .../scala/unit/kafka/log/LogCleanerTest.scala   | 275 ++++-----
 .../scala/unit/kafka/log/LogManagerTest.scala   |  22 +-
 .../scala/unit/kafka/log/LogSegmentTest.scala   | 136 ++--
 .../src/test/scala/unit/kafka/log/LogTest.scala | 204 +++---
 .../scala/unit/kafka/log/LogValidatorTest.scala | 395 ++++++++++++
 .../kafka/message/BaseMessageSetTestCases.scala |  20 +-
 .../message/ByteBufferMessageSetTest.scala      | 348 -----------
 .../kafka/message/MessageCompressionTest.scala  |   2 +-
 .../scala/unit/kafka/message/MessageTest.scala  |  38 +-
 .../server/AbstractFetcherThreadTest.scala      |  32 +-
 .../unit/kafka/server/FetchRequestTest.scala    |  53 +-
 .../unit/kafka/server/ISRExpirationTest.scala   |  27 +-
 .../scala/unit/kafka/server/LogOffsetTest.scala |  14 +-
 .../unit/kafka/server/ProduceRequestTest.scala  |   8 +-
 .../kafka/server/ReplicaManagerQuotasTest.scala |  33 +-
 .../unit/kafka/server/ReplicaManagerTest.scala  |  28 +-
 .../unit/kafka/server/SimpleFetchTest.scala     |  21 +-
 .../test/scala/unit/kafka/utils/TestUtils.scala |  19 +-
 89 files changed, 5269 insertions(+), 3914 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/checkstyle/import-control.xml
----------------------------------------------------------------------
diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index 8eebdb5..62cd77a 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -89,6 +89,7 @@
       <allow pkg="net.jpountz" />
       <allow pkg="org.apache.kafka.common.record" />
       <allow pkg="org.apache.kafka.common.network" />
+      <allow pkg="org.apache.kafka.common.errors" />
     </subpackage>
 
     <subpackage name="requests">

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 6fb4229..3b9d49c 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
@@ -61,6 +61,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Locale;
@@ -686,11 +687,13 @@ public class Fetcher<K, V> {
                 }
 
                 List<ConsumerRecord<K, V>> parsed = new ArrayList<>();
-                for (LogEntry logEntry : partition.records) {
+                Iterator<LogEntry> deepIterator = partition.records.deepIterator();
+                while (deepIterator.hasNext()) {
+                    LogEntry logEntry = deepIterator.next();
                     // Skip the messages earlier than current position.
                     if (logEntry.offset() >= position) {
                         parsed.add(parseRecord(tp, logEntry));
-                        bytes += logEntry.size();
+                        bytes += logEntry.sizeInBytes();
                     }
                 }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java
index b42b0ec..077215c 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java
@@ -198,7 +198,7 @@ public final class BufferPool {
      * memory as free.
      * 
      * @param buffer The buffer to return
-     * @param size The size of the buffer to mark as deallocated, note that this maybe smaller than buffer.capacity
+     * @param size The size of the buffer to mark as deallocated, note that this may be smaller than buffer.capacity
      *             since the buffer may re-allocate itself during in-place compression
      */
     public void deallocate(ByteBuffer buffer, int size) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
index fa1e513..06d39ec 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
@@ -27,8 +27,10 @@ import org.apache.kafka.common.metrics.Sensor;
 import org.apache.kafka.common.metrics.stats.Rate;
 import org.apache.kafka.common.record.CompressionType;
 import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.record.MemoryRecordsBuilder;
 import org.apache.kafka.common.record.Record;
 import org.apache.kafka.common.record.Records;
+import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.utils.CopyOnWriteMap;
 import org.apache.kafka.common.utils.Time;
 import org.apache.kafka.common.utils.Utils;
@@ -49,7 +51,7 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
 /**
- * This class acts as a queue that accumulates records into {@link org.apache.kafka.common.record.MemoryRecords}
+ * This class acts as a queue that accumulates records into {@link MemoryRecords}
  * instances to be sent to the server.
  * <p>
  * The accumulator uses a bounded amount of memory and append calls will block when that memory is exhausted, unless
@@ -77,7 +79,7 @@ public final class RecordAccumulator {
     /**
      * Create a new record accumulator
      * 
-     * @param batchSize The size to use when allocating {@link org.apache.kafka.common.record.MemoryRecords} instances
+     * @param batchSize The size to use when allocating {@link MemoryRecords} instances
      * @param totalSize The maximum memory the record accumulator can use.
      * @param compression The compression codec for the records
      * @param lingerMs An artificial delay time to add before declaring a records instance that isn't full ready for
@@ -190,13 +192,13 @@ public final class RecordAccumulator {
                     free.deallocate(buffer);
                     return appendResult;
                 }
-                MemoryRecords records = MemoryRecords.emptyRecords(buffer, compression, this.batchSize);
-                RecordBatch batch = new RecordBatch(tp, records, time.milliseconds());
+                MemoryRecordsBuilder recordsBuilder = MemoryRecords.builder(buffer, compression, TimestampType.CREATE_TIME, this.batchSize);
+                RecordBatch batch = new RecordBatch(tp, recordsBuilder, time.milliseconds());
                 FutureRecordMetadata future = Utils.notNull(batch.tryAppend(timestamp, key, value, callback, time.milliseconds()));
 
                 dq.addLast(batch);
                 incomplete.add(batch);
-                return new RecordAppendResult(future, dq.size() > 1 || batch.records.isFull(), true);
+                return new RecordAppendResult(future, dq.size() > 1 || batch.isFull(), true);
             }
         } finally {
             appendsInProgress.decrementAndGet();
@@ -212,9 +214,9 @@ public final class RecordAccumulator {
         if (last != null) {
             FutureRecordMetadata future = last.tryAppend(timestamp, key, value, callback, time.milliseconds());
             if (future == null)
-                last.records.close();
+                last.close();
             else
-                return new RecordAppendResult(future, deque.size() > 1 || last.records.isFull(), false);
+                return new RecordAppendResult(future, deque.size() > 1 || last.isFull(), false);
         }
         return null;
     }
@@ -240,7 +242,7 @@ public final class RecordAccumulator {
                     Iterator<RecordBatch> batchIterator = dq.iterator();
                     while (batchIterator.hasNext()) {
                         RecordBatch batch = batchIterator.next();
-                        boolean isFull = batch != lastBatch || batch.records.isFull();
+                        boolean isFull = batch != lastBatch || batch.isFull();
                         // check if the batch is expired
                         if (batch.maybeExpire(requestTimeout, retryBackoffMs, now, this.lingerMs, isFull)) {
                             expiredBatches.add(batch);
@@ -319,7 +321,7 @@ public final class RecordAccumulator {
                         long waitedTimeMs = nowMs - batch.lastAttemptMs;
                         long timeToWaitMs = backingOff ? retryBackoffMs : lingerMs;
                         long timeLeftMs = Math.max(timeToWaitMs - waitedTimeMs, 0);
-                        boolean full = deque.size() > 1 || batch.records.isFull();
+                        boolean full = deque.size() > 1 || batch.isFull();
                         boolean expired = waitedTimeMs >= timeToWaitMs;
                         boolean sendable = full || expired || exhausted || closed || flushInProgress();
                         if (sendable && !backingOff) {
@@ -389,15 +391,15 @@ public final class RecordAccumulator {
                                 boolean backoff = first.attempts > 0 && first.lastAttemptMs + retryBackoffMs > now;
                                 // Only drain the batch if it is not during backoff period.
                                 if (!backoff) {
-                                    if (size + first.records.sizeInBytes() > maxSize && !ready.isEmpty()) {
+                                    if (size + first.sizeInBytes() > maxSize && !ready.isEmpty()) {
                                         // there is a rare case that a single batch size is larger than the request size due
                                         // to compression; in this case we will still eventually send this batch in a single
                                         // request
                                         break;
                                     } else {
                                         RecordBatch batch = deque.pollFirst();
-                                        batch.records.close();
-                                        size += batch.records.sizeInBytes();
+                                        batch.close();
+                                        size += batch.sizeInBytes();
                                         ready.add(batch);
                                         batch.drainedMs = now;
                                     }
@@ -437,7 +439,7 @@ public final class RecordAccumulator {
      */
     public void deallocate(RecordBatch batch) {
         incomplete.remove(batch);
-        free.deallocate(batch.records.buffer(), batch.records.initialCapacity());
+        free.deallocate(batch.buffer(), batch.initialCapacity());
     }
     
     /**
@@ -507,7 +509,7 @@ public final class RecordAccumulator {
             Deque<RecordBatch> dq = getDeque(batch.topicPartition);
             // Close the batch before aborting
             synchronized (dq) {
-                batch.records.close();
+                batch.close();
                 dq.remove(batch);
             }
             batch.done(-1L, Record.NO_TIMESTAMP, new IllegalStateException("Producer is closed forcefully."));

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java
index 6706bfd..e9ef441 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java
@@ -12,18 +12,20 @@
  */
 package org.apache.kafka.clients.producer.internals;
 
-import java.util.ArrayList;
-import java.util.List;
-
 import org.apache.kafka.clients.producer.Callback;
 import org.apache.kafka.clients.producer.RecordMetadata;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.errors.TimeoutException;
 import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.record.MemoryRecordsBuilder;
 import org.apache.kafka.common.record.Record;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
 /**
  * A batch of records that is or will be sent.
  * 
@@ -39,21 +41,21 @@ public final class RecordBatch {
     public final long createdMs;
     public long drainedMs;
     public long lastAttemptMs;
-    public final MemoryRecords records;
     public final TopicPartition topicPartition;
     public final ProduceRequestResult produceFuture;
     public long lastAppendTime;
     private final List<Thunk> thunks;
     private long offsetCounter = 0L;
     private boolean retry;
+    private final MemoryRecordsBuilder recordsBuilder;
 
-    public RecordBatch(TopicPartition tp, MemoryRecords records, long now) {
+    public RecordBatch(TopicPartition tp, MemoryRecordsBuilder recordsBuilder, long now) {
         this.createdMs = now;
         this.lastAttemptMs = now;
-        this.records = records;
+        this.recordsBuilder = recordsBuilder;
         this.topicPartition = tp;
         this.produceFuture = new ProduceRequestResult();
-        this.thunks = new ArrayList<Thunk>();
+        this.thunks = new ArrayList<>();
         this.lastAppendTime = createdMs;
         this.retry = false;
     }
@@ -64,10 +66,10 @@ public final class RecordBatch {
      * @return The RecordSend corresponding to this record or null if there isn't sufficient room.
      */
     public FutureRecordMetadata tryAppend(long timestamp, byte[] key, byte[] value, Callback callback, long now) {
-        if (!this.records.hasRoomFor(key, value)) {
+        if (!recordsBuilder.hasRoomFor(key, value)) {
             return null;
         } else {
-            long checksum = this.records.append(offsetCounter++, timestamp, key, value);
+            long checksum = this.recordsBuilder.append(offsetCounter++, timestamp, key, value);
             this.maxRecordSize = Math.max(this.maxRecordSize, Record.recordSize(key, value));
             this.lastAppendTime = now;
             FutureRecordMetadata future = new FutureRecordMetadata(this.produceFuture, this.recordCount,
@@ -94,9 +96,8 @@ public final class RecordBatch {
                   baseOffset,
                   exception);
         // execute callbacks
-        for (int i = 0; i < this.thunks.size(); i++) {
+        for (Thunk thunk : thunks) {
             try {
-                Thunk thunk = this.thunks.get(i);
                 if (exception == null) {
                     // If the timestamp returned by server is NoTimestamp, that means CreateTime is used. Otherwise LogAppendTime is used.
                     RecordMetadata metadata = new RecordMetadata(this.topicPartition,  baseOffset, thunk.future.relativeOffset(),
@@ -156,7 +157,7 @@ public final class RecordBatch {
         }
 
         if (expire) {
-            this.records.close();
+            close();
             this.done(-1L, Record.NO_TIMESTAMP,
                       new TimeoutException("Expiring " + recordCount + " record(s) for " + topicPartition + " due to " + errorMessage));
         }
@@ -177,4 +178,37 @@ public final class RecordBatch {
     public void setRetry() {
         this.retry = true;
     }
+
+    public MemoryRecords records() {
+        return recordsBuilder.build();
+    }
+
+    public int sizeInBytes() {
+        return recordsBuilder.sizeInBytes();
+    }
+
+    public double compressionRate() {
+        return recordsBuilder.compressionRate();
+    }
+
+    public boolean isFull() {
+        return recordsBuilder.isFull();
+    }
+
+    public void close() {
+        recordsBuilder.close();
+    }
+
+    public ByteBuffer buffer() {
+        return recordsBuilder.buffer();
+    }
+
+    public int initialCapacity() {
+        return recordsBuilder.initialCapacity();
+    }
+
+    public boolean isWritable() {
+        return !recordsBuilder.isClosed();
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
index 7555b71..1f54c0b 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
@@ -345,7 +345,7 @@ public class Sender implements Runnable {
         final Map<TopicPartition, RecordBatch> recordsByPartition = new HashMap<>(batches.size());
         for (RecordBatch batch : batches) {
             TopicPartition tp = batch.topicPartition;
-            produceRecordsByPartition.put(tp, batch.records);
+            produceRecordsByPartition.put(tp, batch.records());
             recordsByPartition.put(tp, batch);
         }
 
@@ -505,17 +505,17 @@ public class Sender implements Runnable {
                     // per-topic bytes send rate
                     String topicByteRateName = "topic." + topic + ".bytes";
                     Sensor topicByteRate = Utils.notNull(this.metrics.getSensor(topicByteRateName));
-                    topicByteRate.record(batch.records.sizeInBytes());
+                    topicByteRate.record(batch.sizeInBytes());
 
                     // per-topic compression rate
                     String topicCompressionRateName = "topic." + topic + ".compression-rate";
                     Sensor topicCompressionRate = Utils.notNull(this.metrics.getSensor(topicCompressionRateName));
-                    topicCompressionRate.record(batch.records.compressionRate());
+                    topicCompressionRate.record(batch.compressionRate());
 
                     // global metrics
-                    this.batchSizeSensor.record(batch.records.sizeInBytes(), now);
+                    this.batchSizeSensor.record(batch.sizeInBytes(), now);
                     this.queueTimeSensor.record(batch.drainedMs - batch.createdMs, now);
-                    this.compressionRateSensor.record(batch.records.compressionRate());
+                    this.compressionRateSensor.record(batch.compressionRate());
                     this.maxRecordSizeSensor.record(batch.maxRecordSize, now);
                     records += batch.recordCount;
                 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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
new file mode 100644
index 0000000..3794dc6
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java
@@ -0,0 +1,92 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.utils.AbstractIterator;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+public abstract class AbstractRecords implements Records {
+
+    @Override
+    public boolean hasMatchingShallowMagic(byte magic) {
+        Iterator<? extends LogEntry> iterator = shallowIterator();
+        while (iterator.hasNext())
+            if (iterator.next().magic() != magic)
+                return false;
+        return true;
+    }
+
+    /**
+     * Convert this message set to use the specified message format.
+     */
+    @Override
+    public Records toMessageFormat(byte toMagic) {
+        List<LogEntry> converted = new ArrayList<>();
+        Iterator<LogEntry> deepIterator = deepIterator();
+        while (deepIterator.hasNext()) {
+            LogEntry entry = deepIterator.next();
+            converted.add(LogEntry.create(entry.offset(), entry.record().convert(toMagic)));
+        }
+
+        if (converted.isEmpty()) {
+            // This indicates that the message is too large, which indicates that the buffer is not large
+            // enough to hold a full log entry. We just return all the bytes in the file message set.
+            // Even though the message set does not have the right format version, we expect old clients
+            // to raise an error to the user after reading the message size and seeing that there
+            // are not enough available bytes in the response to read the full message.
+            return this;
+        } else {
+            // We use the first message to determine the compression type for the resulting message set.
+            // This could result in message sets which are either larger or smaller than the original size.
+            // For example, it could end up larger if most messages were previously compressed, but
+            // it just so happens that the first one is not. There is also some risk that this can
+            // cause some timestamp information to be lost (e.g. if the timestamp type was changed) since
+            // we are essentially merging multiple message sets. However, currently this method is only
+            // used for down-conversion, so we've ignored the problem.
+            CompressionType compressionType = shallowIterator().next().record().compressionType();
+            return MemoryRecords.withLogEntries(compressionType, converted);
+        }
+    }
+
+    public static int estimatedSize(CompressionType compressionType, Iterable<LogEntry> entries) {
+        int size = 0;
+        for (LogEntry entry : entries)
+            size += entry.sizeInBytes();
+        // NOTE: 1024 is the minimum block size for snappy encoding
+        return compressionType == CompressionType.NONE ? size : Math.min(Math.max(size / 2, 1024), 1 << 16);
+    }
+
+    /**
+     * Get an iterator over the deep records.
+     * @return An iterator over the records
+     */
+    public Iterator<Record> records() {
+        return new AbstractIterator<Record>() {
+            private final Iterator<? extends LogEntry> deepEntries = deepIterator();
+            @Override
+            protected Record makeNext() {
+                if (deepEntries.hasNext())
+                    return deepEntries.next().record();
+                return allDone();
+            }
+        };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java
index 84668a5..b25f949 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java
@@ -16,34 +16,41 @@
  */
 package org.apache.kafka.common.record;
 
+import java.io.DataInputStream;
 import java.io.InputStream;
 import java.nio.ByteBuffer;
 
 /**
  * A byte buffer backed input inputStream
  */
-public class ByteBufferInputStream extends InputStream {
-
-    private ByteBuffer buffer;
+public class ByteBufferInputStream extends DataInputStream {
 
     public ByteBufferInputStream(ByteBuffer buffer) {
-        this.buffer = buffer;
+        super(new UnderlyingInputStream(buffer));
     }
 
-    public int read() {
-        if (!buffer.hasRemaining()) {
-            return -1;
+    private static class UnderlyingInputStream extends InputStream {
+        private ByteBuffer buffer;
+
+        public UnderlyingInputStream(ByteBuffer buffer) {
+            this.buffer = buffer;
         }
-        return buffer.get() & 0xFF;
-    }
 
-    public int read(byte[] bytes, int off, int len) {
-        if (!buffer.hasRemaining()) {
-            return -1;
+        public int read() {
+            if (!buffer.hasRemaining()) {
+                return -1;
+            }
+            return buffer.get() & 0xFF;
         }
 
-        len = Math.min(len, buffer.remaining());
-        buffer.get(bytes, off, len);
-        return len;
+        public int read(byte[] bytes, int off, int len) {
+            if (!buffer.hasRemaining()) {
+                return -1;
+            }
+
+            len = Math.min(len, buffer.remaining());
+            buffer.get(bytes, off, len);
+            return len;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/ByteBufferLogInputStream.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferLogInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferLogInputStream.java
new file mode 100644
index 0000000..ae0c91b
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferLogInputStream.java
@@ -0,0 +1,119 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import static org.apache.kafka.common.record.Records.LOG_OVERHEAD;
+import static org.apache.kafka.common.record.Records.OFFSET_OFFSET;
+
+/**
+ * A byte buffer backed log input stream. This class avoids the need to copy records by returning
+ * slices from the underlying byte buffer.
+ */
+class ByteBufferLogInputStream implements LogInputStream<ByteBufferLogInputStream.ByteBufferLogEntry> {
+    private final ByteBuffer buffer;
+    private final int maxMessageSize;
+
+    ByteBufferLogInputStream(ByteBuffer buffer, int maxMessageSize) {
+        this.buffer = buffer;
+        this.maxMessageSize = maxMessageSize;
+    }
+
+    public ByteBufferLogEntry nextEntry() throws IOException {
+        int remaining = buffer.remaining();
+        if (remaining < LOG_OVERHEAD)
+            return null;
+
+        int recordSize = buffer.getInt(buffer.position() + Records.SIZE_OFFSET);
+        if (recordSize < Record.RECORD_OVERHEAD_V0)
+            throw new CorruptRecordException(String.format("Record size is less than the minimum record overhead (%d)", Record.RECORD_OVERHEAD_V0));
+        if (recordSize > maxMessageSize)
+            throw new CorruptRecordException(String.format("Record size exceeds the largest allowable message size (%d).", maxMessageSize));
+
+        int entrySize = recordSize + LOG_OVERHEAD;
+        if (remaining < entrySize)
+            return null;
+
+        ByteBuffer entrySlice = buffer.slice();
+        entrySlice.limit(entrySize);
+        buffer.position(buffer.position() + entrySize);
+        return new ByteBufferLogEntry(entrySlice);
+    }
+
+    public static class ByteBufferLogEntry extends LogEntry {
+        private final ByteBuffer buffer;
+        private final Record record;
+
+        private ByteBufferLogEntry(ByteBuffer buffer) {
+            this.buffer = buffer;
+            buffer.position(LOG_OVERHEAD);
+            this.record = new Record(buffer.slice());
+            buffer.position(OFFSET_OFFSET);
+        }
+
+        @Override
+        public long offset() {
+            return buffer.getLong(OFFSET_OFFSET);
+        }
+
+        @Override
+        public Record record() {
+            return record;
+        }
+
+        public void setOffset(long offset) {
+            buffer.putLong(OFFSET_OFFSET, offset);
+        }
+
+        public void setCreateTime(long timestamp) {
+            if (record.magic() == Record.MAGIC_VALUE_V0)
+                throw new IllegalArgumentException("Cannot set timestamp for a record with magic = 0");
+
+            long currentTimestamp = record.timestamp();
+            // We don't need to recompute crc if the timestamp is not updated.
+            if (record.timestampType() == TimestampType.CREATE_TIME && currentTimestamp == timestamp)
+                return;
+
+            byte attributes = record.attributes();
+            buffer.put(LOG_OVERHEAD + Record.ATTRIBUTES_OFFSET, TimestampType.CREATE_TIME.updateAttributes(attributes));
+            buffer.putLong(LOG_OVERHEAD + Record.TIMESTAMP_OFFSET, timestamp);
+            long crc = record.computeChecksum();
+            Utils.writeUnsignedInt(buffer, LOG_OVERHEAD + Record.CRC_OFFSET, crc);
+        }
+
+        public void setLogAppendTime(long timestamp) {
+            if (record.magic() == Record.MAGIC_VALUE_V0)
+                throw new IllegalArgumentException("Cannot set timestamp for a record with magic = 0");
+
+            byte attributes = record.attributes();
+            buffer.put(LOG_OVERHEAD + Record.ATTRIBUTES_OFFSET, TimestampType.LOG_APPEND_TIME.updateAttributes(attributes));
+            buffer.putLong(LOG_OVERHEAD + Record.TIMESTAMP_OFFSET, timestamp);
+            long crc = record.computeChecksum();
+            Utils.writeUnsignedInt(buffer, LOG_OVERHEAD + Record.CRC_OFFSET, crc);
+        }
+
+        public ByteBuffer buffer() {
+            return buffer;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java
index 1c9fbaa..3fb7f49 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java
@@ -16,42 +16,54 @@
  */
 package org.apache.kafka.common.record;
 
+import java.io.DataOutputStream;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
 
 /**
  * A byte buffer backed output outputStream
  */
-public class ByteBufferOutputStream extends OutputStream {
+public class ByteBufferOutputStream extends DataOutputStream {
 
     private static final float REALLOCATION_FACTOR = 1.1f;
 
-    private ByteBuffer buffer;
-
     public ByteBufferOutputStream(ByteBuffer buffer) {
-        this.buffer = buffer;
+        super(new UnderlyingOutputStream(buffer));
     }
 
-    public void write(int b) {
-        if (buffer.remaining() < 1)
-            expandBuffer(buffer.capacity() + 1);
-        buffer.put((byte) b);
+    public ByteBuffer buffer() {
+        return ((UnderlyingOutputStream) out).buffer;
     }
 
-    public void write(byte[] bytes, int off, int len) {
-        if (buffer.remaining() < len)
-            expandBuffer(buffer.capacity() + len);
-        buffer.put(bytes, off, len);
-    }
+    public static class UnderlyingOutputStream extends OutputStream {
+        private ByteBuffer buffer;
 
-    public ByteBuffer buffer() {
-        return buffer;
-    }
+        public UnderlyingOutputStream(ByteBuffer buffer) {
+            this.buffer = buffer;
+        }
 
-    private void expandBuffer(int size) {
-        int expandSize = Math.max((int) (buffer.capacity() * REALLOCATION_FACTOR), size);
-        ByteBuffer temp = ByteBuffer.allocate(expandSize);
-        temp.put(buffer.array(), buffer.arrayOffset(), buffer.position());
-        buffer = temp;
+        public void write(int b) {
+            if (buffer.remaining() < 1)
+                expandBuffer(buffer.capacity() + 1);
+            buffer.put((byte) b);
+        }
+
+        public void write(byte[] bytes, int off, int len) {
+            if (buffer.remaining() < len)
+                expandBuffer(buffer.capacity() + len);
+            buffer.put(bytes, off, len);
+        }
+
+        public ByteBuffer buffer() {
+            return buffer;
+        }
+
+        private void expandBuffer(int size) {
+            int expandSize = Math.max((int) (buffer.capacity() * REALLOCATION_FACTOR), size);
+            ByteBuffer temp = ByteBuffer.allocate(expandSize);
+            temp.put(buffer.array(), buffer.arrayOffset(), buffer.position());
+            buffer = temp;
+        }
     }
+
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java b/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java
index 65a7e43..e1d4754 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java
@@ -26,7 +26,7 @@ public enum CompressionType {
     public final String name;
     public final float rate;
 
-    private CompressionType(int id, String name, float rate) {
+    CompressionType(int id, String name, float rate) {
         this.id = id;
         this.name = name;
         this.rate = rate;

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/Compressor.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/Compressor.java b/clients/src/main/java/org/apache/kafka/common/record/Compressor.java
deleted file mode 100644
index a806975..0000000
--- a/clients/src/main/java/org/apache/kafka/common/record/Compressor.java
+++ /dev/null
@@ -1,332 +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 org.apache.kafka.common.record;
-
-import java.lang.reflect.Constructor;
-import org.apache.kafka.common.KafkaException;
-import org.apache.kafka.common.utils.Utils;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.zip.GZIPInputStream;
-import java.util.zip.GZIPOutputStream;
-
-public class Compressor {
-
-    static private final float COMPRESSION_RATE_DAMPING_FACTOR = 0.9f;
-    static private final float COMPRESSION_RATE_ESTIMATION_FACTOR = 1.05f;
-    static private final int COMPRESSION_DEFAULT_BUFFER_SIZE = 1024;
-
-    private static final float[] TYPE_TO_RATE;
-
-    static {
-        int maxTypeId = -1;
-        for (CompressionType type : CompressionType.values())
-            maxTypeId = Math.max(maxTypeId, type.id);
-        TYPE_TO_RATE = new float[maxTypeId + 1];
-        for (CompressionType type : CompressionType.values()) {
-            TYPE_TO_RATE[type.id] = type.rate;
-        }
-    }
-
-    // dynamically load the snappy and lz4 classes to avoid runtime dependency if we are not using compression
-    // caching constructors to avoid invoking of Class.forName method for each batch
-    private static MemoizingConstructorSupplier snappyOutputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
-        @Override
-        public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
-            return Class.forName("org.xerial.snappy.SnappyOutputStream")
-                .getConstructor(OutputStream.class, Integer.TYPE);
-        }
-    });
-
-    private static MemoizingConstructorSupplier lz4OutputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
-        @Override
-        public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
-            return Class.forName("org.apache.kafka.common.record.KafkaLZ4BlockOutputStream")
-                .getConstructor(OutputStream.class);
-        }
-    });
-
-    private static MemoizingConstructorSupplier snappyInputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
-        @Override
-        public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
-            return Class.forName("org.xerial.snappy.SnappyInputStream")
-                .getConstructor(InputStream.class);
-        }
-    });
-
-    private static MemoizingConstructorSupplier lz4InputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
-        @Override
-        public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
-            return Class.forName("org.apache.kafka.common.record.KafkaLZ4BlockInputStream")
-                .getConstructor(InputStream.class, Boolean.TYPE);
-        }
-    });
-
-    private final CompressionType type;
-    private final DataOutputStream appendStream;
-    private final ByteBufferOutputStream bufferStream;
-    private final int initPos;
-
-    public long writtenUncompressed;
-    public long numRecords;
-    public float compressionRate;
-    public long maxTimestamp;
-
-    public Compressor(ByteBuffer buffer, CompressionType type) {
-        this.type = type;
-        this.initPos = buffer.position();
-
-        this.numRecords = 0;
-        this.writtenUncompressed = 0;
-        this.compressionRate = 1;
-        this.maxTimestamp = Record.NO_TIMESTAMP;
-
-        if (type != CompressionType.NONE) {
-            // for compressed records, leave space for the header and the shallow message metadata
-            // and move the starting position to the value payload offset
-            buffer.position(initPos + Records.LOG_OVERHEAD + Record.RECORD_OVERHEAD);
-        }
-
-        // create the stream
-        bufferStream = new ByteBufferOutputStream(buffer);
-        appendStream = wrapForOutput(bufferStream, type, COMPRESSION_DEFAULT_BUFFER_SIZE);
-    }
-
-    public ByteBuffer buffer() {
-        return bufferStream.buffer();
-    }
-
-    public double compressionRate() {
-        return compressionRate;
-    }
-
-    public void close() {
-        try {
-            appendStream.close();
-        } catch (IOException e) {
-            throw new KafkaException(e);
-        }
-
-        if (type != CompressionType.NONE) {
-            ByteBuffer buffer = bufferStream.buffer();
-            int pos = buffer.position();
-            // write the header, for the end offset write as number of records - 1
-            buffer.position(initPos);
-            buffer.putLong(numRecords - 1);
-            buffer.putInt(pos - initPos - Records.LOG_OVERHEAD);
-            // write the shallow message (the crc and value size are not correct yet)
-            Record.write(buffer, maxTimestamp, null, null, type, 0, -1);
-            // compute the fill the value size
-            int valueSize = pos - initPos - Records.LOG_OVERHEAD - Record.RECORD_OVERHEAD;
-            buffer.putInt(initPos + Records.LOG_OVERHEAD + Record.KEY_OFFSET_V1, valueSize);
-            // compute and fill the crc at the beginning of the message
-            long crc = Record.computeChecksum(buffer,
-                initPos + Records.LOG_OVERHEAD + Record.MAGIC_OFFSET,
-                pos - initPos - Records.LOG_OVERHEAD - Record.MAGIC_OFFSET);
-            Utils.writeUnsignedInt(buffer, initPos + Records.LOG_OVERHEAD + Record.CRC_OFFSET, crc);
-            // reset the position
-            buffer.position(pos);
-
-            // update the compression ratio
-            this.compressionRate = (float) buffer.position() / this.writtenUncompressed;
-            TYPE_TO_RATE[type.id] = TYPE_TO_RATE[type.id] * COMPRESSION_RATE_DAMPING_FACTOR +
-                compressionRate * (1 - COMPRESSION_RATE_DAMPING_FACTOR);
-        }
-    }
-
-    // Note that for all the write operations below, IO exceptions should
-    // never be thrown since the underlying ByteBufferOutputStream does not throw IOException;
-    // therefore upon encountering this issue we just close the append stream.
-
-    public void putLong(final long value) {
-        try {
-            appendStream.writeLong(value);
-        } catch (IOException e) {
-            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
-        }
-    }
-
-    public void putInt(final int value) {
-        try {
-            appendStream.writeInt(value);
-        } catch (IOException e) {
-            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
-        }
-    }
-
-    public void put(final ByteBuffer buffer) {
-        try {
-            appendStream.write(buffer.array(), buffer.arrayOffset(), buffer.limit());
-        } catch (IOException e) {
-            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
-        }
-    }
-
-    public void putByte(final byte value) {
-        try {
-            appendStream.write(value);
-        } catch (IOException e) {
-            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
-        }
-    }
-
-    public void put(final byte[] bytes, final int offset, final int len) {
-        try {
-            appendStream.write(bytes, offset, len);
-        } catch (IOException e) {
-            throw new KafkaException("I/O exception when writing to the append stream, closing", e);
-        }
-    }
-
-    /**
-     * @return CRC of the record
-     */
-    public long putRecord(long timestamp, byte[] key, byte[] value, CompressionType type,
-                          int valueOffset, int valueSize) {
-        // put a record as un-compressed into the underlying stream
-        long crc = Record.computeChecksum(timestamp, key, value, type, valueOffset, valueSize);
-        byte attributes = Record.computeAttributes(type);
-        putRecord(crc, attributes, timestamp, key, value, valueOffset, valueSize);
-        return crc;
-    }
-
-    /**
-     * Put a record as uncompressed into the underlying stream
-     * @return CRC of the record
-     */
-    public long putRecord(long timestamp, byte[] key, byte[] value) {
-        return putRecord(timestamp, key, value, CompressionType.NONE, 0, -1);
-    }
-
-    private void putRecord(final long crc, final byte attributes, final long timestamp, final byte[] key, final byte[] value, final int valueOffset, final int valueSize) {
-        maxTimestamp = Math.max(maxTimestamp, timestamp);
-        Record.write(this, crc, attributes, timestamp, key, value, valueOffset, valueSize);
-    }
-
-    public void recordWritten(int size) {
-        numRecords += 1;
-        writtenUncompressed += size;
-    }
-
-    public long numRecordsWritten() {
-        return numRecords;
-    }
-
-    public long estimatedBytesWritten() {
-        if (type == CompressionType.NONE) {
-            return bufferStream.buffer().position();
-        } else {
-            // estimate the written bytes to the underlying byte buffer based on uncompressed written bytes
-            return (long) (writtenUncompressed * TYPE_TO_RATE[type.id] * COMPRESSION_RATE_ESTIMATION_FACTOR);
-        }
-    }
-
-    // the following two functions also need to be public since they are used in MemoryRecords.iteration
-
-    public static DataOutputStream wrapForOutput(ByteBufferOutputStream buffer, CompressionType type, int bufferSize) {
-        try {
-            switch (type) {
-                case NONE:
-                    return new DataOutputStream(buffer);
-                case GZIP:
-                    return new DataOutputStream(new GZIPOutputStream(buffer, bufferSize));
-                case SNAPPY:
-                    try {
-                        OutputStream stream = (OutputStream) snappyOutputStreamSupplier.get().newInstance(buffer, bufferSize);
-                        return new DataOutputStream(stream);
-                    } catch (Exception e) {
-                        throw new KafkaException(e);
-                    }
-                case LZ4:
-                    try {
-                        OutputStream stream = (OutputStream) lz4OutputStreamSupplier.get().newInstance(buffer);
-                        return new DataOutputStream(stream);
-                    } catch (Exception e) {
-                        throw new KafkaException(e);
-                    }
-                default:
-                    throw new IllegalArgumentException("Unknown compression type: " + type);
-            }
-        } catch (IOException e) {
-            throw new KafkaException(e);
-        }
-    }
-
-    public static DataInputStream wrapForInput(ByteBufferInputStream buffer, CompressionType type, byte messageVersion) {
-        try {
-            switch (type) {
-                case NONE:
-                    return new DataInputStream(buffer);
-                case GZIP:
-                    return new DataInputStream(new GZIPInputStream(buffer));
-                case SNAPPY:
-                    try {
-                        InputStream stream = (InputStream) snappyInputStreamSupplier.get().newInstance(buffer);
-                        return new DataInputStream(stream);
-                    } catch (Exception e) {
-                        throw new KafkaException(e);
-                    }
-                case LZ4:
-                    try {
-                        InputStream stream = (InputStream) lz4InputStreamSupplier.get().newInstance(buffer,
-                                messageVersion == Record.MAGIC_VALUE_V0);
-                        return new DataInputStream(stream);
-                    } catch (Exception e) {
-                        throw new KafkaException(e);
-                    }
-                default:
-                    throw new IllegalArgumentException("Unknown compression type: " + type);
-            }
-        } catch (IOException e) {
-            throw new KafkaException(e);
-        }
-    }
-
-    private interface ConstructorSupplier {
-        Constructor get() throws ClassNotFoundException, NoSuchMethodException;
-    }
-
-    // this code is based on Guava's @see{com.google.common.base.Suppliers.MemoizingSupplier}
-    private static class MemoizingConstructorSupplier {
-        final ConstructorSupplier delegate;
-        transient volatile boolean initialized;
-        transient Constructor value;
-
-        public MemoizingConstructorSupplier(ConstructorSupplier delegate) {
-            this.delegate = delegate;
-        }
-
-        public Constructor get() throws NoSuchMethodException, ClassNotFoundException {
-            if (!initialized) {
-                synchronized (this) {
-                    if (!initialized) {
-                        Constructor constructor = delegate.get();
-                        value = constructor;
-                        initialized = true;
-                        return constructor;
-                    }
-                }
-            }
-            return value;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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
new file mode 100644
index 0000000..ae393b0
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java
@@ -0,0 +1,166 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.KafkaException;
+import org.apache.kafka.common.errors.CorruptRecordException;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+
+/**
+ * A log input stream which is backed by a {@link FileChannel}.
+ */
+public class FileLogInputStream implements LogInputStream<FileLogInputStream.FileChannelLogEntry> {
+    private int position;
+    private final int end;
+    private final FileChannel channel;
+    private final int maxRecordSize;
+    private final ByteBuffer logHeaderBuffer = ByteBuffer.allocate(Records.LOG_OVERHEAD);
+
+    /**
+     * Create a new log input stream over the FileChannel
+     * @param channel Underlying FileChannel
+     * @param maxRecordSize Maximum size of records
+     * @param start Position in the file channel to start from
+     * @param end Position in the file channel not to read past
+     */
+    public FileLogInputStream(FileChannel channel,
+                              int maxRecordSize,
+                              int start,
+                              int end) {
+        this.channel = channel;
+        this.maxRecordSize = maxRecordSize;
+        this.position = start;
+        this.end = end;
+    }
+
+    @Override
+    public FileChannelLogEntry nextEntry() throws IOException {
+        if (position + Records.LOG_OVERHEAD >= end)
+            return null;
+
+        logHeaderBuffer.rewind();
+        channel.read(logHeaderBuffer, position);
+        if (logHeaderBuffer.hasRemaining())
+            return null;
+
+        logHeaderBuffer.rewind();
+        long offset = logHeaderBuffer.getLong();
+        int size = logHeaderBuffer.getInt();
+
+        if (size < Record.RECORD_OVERHEAD_V0)
+            throw new CorruptRecordException(String.format("Record size is smaller than minimum record overhead (%d).", Record.RECORD_OVERHEAD_V0));
+
+        if (size > maxRecordSize)
+            throw new CorruptRecordException(String.format("Record size exceeds the largest allowable message size (%d).", maxRecordSize));
+
+        if (position + Records.LOG_OVERHEAD + size > end)
+            return null;
+
+        FileChannelLogEntry logEntry = new FileChannelLogEntry(offset, channel, position, size);
+        position += logEntry.sizeInBytes();
+        return logEntry;
+    }
+
+    /**
+     * Log entry backed by an underlying FileChannel. This allows iteration over the shallow log
+     * entries without needing to read the record data into memory until it is needed. The downside
+     * is that entries will generally no longer be readable when the underlying channel is closed.
+     */
+    public static class FileChannelLogEntry extends LogEntry {
+        private final long offset;
+        private final FileChannel channel;
+        private final int position;
+        private final int recordSize;
+        private Record record = null;
+
+        private FileChannelLogEntry(long offset,
+                                   FileChannel channel,
+                                   int position,
+                                   int recordSize) {
+            this.offset = offset;
+            this.channel = channel;
+            this.position = position;
+            this.recordSize = recordSize;
+        }
+
+        @Override
+        public long offset() {
+            return offset;
+        }
+
+        public int position() {
+            return position;
+        }
+
+        @Override
+        public byte magic() {
+            if (record != null)
+                return record.magic();
+
+            try {
+                byte[] magic = new byte[1];
+                ByteBuffer buf = ByteBuffer.wrap(magic);
+                channel.read(buf, position + Records.LOG_OVERHEAD + Record.MAGIC_OFFSET);
+                if (buf.hasRemaining())
+                    throw new KafkaException("Failed to read magic byte from FileChannel " + channel);
+                return magic[0];
+            } catch (IOException e) {
+                throw new KafkaException(e);
+            }
+        }
+
+        /**
+         * Force load the record and its data (key and value) into memory.
+         * @return The resulting record
+         * @throws IOException for any IO errors reading from the underlying file
+         */
+        private Record loadRecord() throws IOException {
+            if (record != null)
+                return record;
+
+            ByteBuffer recordBuffer = ByteBuffer.allocate(recordSize);
+            channel.read(recordBuffer, position + Records.LOG_OVERHEAD);
+            if (recordBuffer.hasRemaining())
+                throw new IOException("Failed to read full record from channel " + channel);
+
+            recordBuffer.rewind();
+            record = new Record(recordBuffer);
+            return record;
+        }
+
+        @Override
+        public Record record() {
+            if (record != null)
+                return record;
+
+            try {
+                return loadRecord();
+            } catch (IOException e) {
+                throw new KafkaException(e);
+            }
+        }
+
+        @Override
+        public int sizeInBytes() {
+            return Records.LOG_OVERHEAD + recordSize;
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 bdae08d..faf61e9 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
@@ -18,22 +18,31 @@ package org.apache.kafka.common.record;
 
 import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.network.TransportLayer;
+import org.apache.kafka.common.record.FileLogInputStream.FileChannelLogEntry;
+import org.apache.kafka.common.utils.Utils;
 
+import java.io.Closeable;
 import java.io.File;
+import java.io.FileInputStream;
 import java.io.IOException;
+import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.channels.GatheringByteChannel;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicInteger;
 
 /**
- * File-backed record set.
+ * A {@link Records} implementation backed by a file. An optional start and end position can be applied to this
+ * instance to enable slicing a range of the log records.
  */
-public class FileRecords implements Records {
-    private final File file;
+public class FileRecords extends AbstractRecords implements Closeable {
+    private final boolean isSlice;
     private final FileChannel channel;
-    private final long start;
-    private final long end;
-    private final long size;
+    private final int start;
+    private final int end;
+    private volatile File file;
+    private final AtomicInteger size;
 
     public FileRecords(File file,
                        FileChannel channel,
@@ -44,83 +53,435 @@ public class FileRecords implements Records {
         this.channel = channel;
         this.start = start;
         this.end = end;
+        this.isSlice = isSlice;
+        this.size = new AtomicInteger();
 
-        if (isSlice)
-            this.size = end - start;
-        else
-            this.size = Math.min(channel.size(), end) - start;
+        // set the initial size of the buffer
+        resize();
+    }
+
+    public void resize() throws IOException {
+        if (isSlice) {
+            size.set(end - start);
+        } else {
+            int limit = Math.min((int) channel.size(), end);
+            size.set(limit - start);
+
+            // if this is not a slice, update the file pointer to the end of the file
+            // set the file position to the last byte in the file
+            channel.position(limit);
+        }
     }
 
     @Override
     public int sizeInBytes() {
-        return (int) size;
+        return size.get();
+    }
+
+    /**
+     * Get the underlying file.
+     * @return The file
+     */
+    public File file() {
+        return file;
+    }
+
+    /**
+     * Get the underlying file channel.
+     * @return The file channel
+     */
+    public FileChannel channel() {
+        return channel;
+    }
+
+    /**
+     * Read log entries into a given buffer.
+     * @param buffer The buffer to write the entries to
+     * @param position Position in the buffer to read from
+     * @return The same buffer
+     * @throws IOException
+     */
+    public ByteBuffer readInto(ByteBuffer buffer, int position) throws IOException {
+        channel.read(buffer, position + this.start);
+        buffer.flip();
+        return buffer;
+    }
+
+    /**
+     * Return a slice of records from this instance, which is a view into this set starting from the given position
+     * and with the given size limit.
+     *
+     * If the size is beyond the end of the file, the end will be based on the size of the file at the time of the read.
+     *
+     * If this message set is already sliced, the position will be taken relative to that slicing.
+     *
+     * @param position The start position to begin the read from
+     * @param size The number of bytes after the start position to include
+     * @return A sliced wrapper on this message set limited based on the given position and size
+     */
+    public FileRecords read(int position, int size) throws IOException {
+        if (position < 0)
+            throw new IllegalArgumentException("Invalid position: " + position);
+        if (size < 0)
+            throw new IllegalArgumentException("Invalid size: " + size);
+
+        final int end;
+        if (this.start + position + size < 0)
+            end = sizeInBytes();
+        else
+            end = Math.min(this.start + position + size, sizeInBytes());
+        return new FileRecords(file, channel, this.start + position, end, true);
+    }
+
+    /**
+     * Append log entries to the buffer
+     * @param records The records to append
+     * @return the number of bytes written to the underlying file
+     */
+    public int append(MemoryRecords records) throws IOException {
+        int written = records.writeFullyTo(channel);
+        size.getAndAdd(written);
+        return written;
+    }
+
+    /**
+     * Commit all written data to the physical disk
+     */
+    public void flush() throws IOException {
+        channel.force(true);
+    }
+
+    /**
+     * Close this record set
+     */
+    public void close() throws IOException {
+        flush();
+        trim();
+        channel.close();
+    }
+
+    /**
+     * Delete this message set from the filesystem
+     * @return True iff this message set was deleted.
+     */
+    public boolean delete() {
+        Utils.closeQuietly(channel, "FileChannel");
+        return file.delete();
+    }
+
+    /**
+     * Trim file when close or roll to next file
+     */
+    public void trim() throws IOException {
+        truncateTo(sizeInBytes());
+    }
+
+    /**
+     * Update the file reference (to be used with caution since this does not reopen the file channel)
+     * @param file The new file to use
+     */
+    public void setFile(File file) {
+        this.file = file;
+    }
+
+    /**
+     * Rename the file that backs this message set
+     * @throws IOException if rename fails.
+     */
+    public void renameTo(File f) throws IOException {
+        try {
+            Utils.atomicMoveWithFallback(file.toPath(), f.toPath());
+        }  finally {
+            this.file = f;
+        }
+    }
+
+    /**
+     * Truncate this file message set to the given size in bytes. Note that this API does no checking that the
+     * given size falls on a valid message boundary.
+     * In some versions of the JDK truncating to the same size as the file message set will cause an
+     * update of the files mtime, so truncate is only performed if the targetSize is smaller than the
+     * size of the underlying FileChannel.
+     * It is expected that no other threads will do writes to the log when this function is called.
+     * @param targetSize The size to truncate to. Must be between 0 and sizeInBytes.
+     * @return The number of bytes truncated off
+     */
+    public int truncateTo(int targetSize) throws IOException {
+        int originalSize = sizeInBytes();
+        if (targetSize > originalSize || targetSize < 0)
+            throw new KafkaException("Attempt to truncate log segment to " + targetSize + " bytes failed, " +
+                    " 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;
     }
 
     @Override
     public long writeTo(GatheringByteChannel destChannel, long offset, int length) throws IOException {
         long newSize = Math.min(channel.size(), end) - start;
-        if (newSize < size)
+        if (newSize < size.get())
             throw new KafkaException(String.format("Size of FileRecords %s has been truncated during write: old size %d, new size %d", file.getAbsolutePath(), size, newSize));
 
-        if (offset > size)
-            throw new KafkaException(String.format("The requested offset %d is out of range. The size of this FileRecords is %d.", offset, size));
-
         long position = start + offset;
-        long count = Math.min(length, this.size - offset);
+        long count = Math.min(length, size.get());
+        final long bytesTransferred;
         if (destChannel instanceof TransportLayer) {
             TransportLayer tl = (TransportLayer) destChannel;
-            return tl.transferFrom(this.channel, position, count);
+            bytesTransferred = tl.transferFrom(channel, position, count);
         } else {
-            return this.channel.transferTo(position, count, destChannel);
+            bytesTransferred = channel.transferTo(position, count, destChannel);
+        }
+        return bytesTransferred;
+    }
+
+    /**
+     * Search forward for the file position of the last offset that is greater than or equal to the target offset
+     * and return its physical position and the size of the message (including log overhead) at the returned offset. If
+     * no such offsets are found, return null.
+     *
+     * @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) {
+        Iterator<FileChannelLogEntry> iterator = shallowIteratorFrom(Integer.MAX_VALUE, startingPosition);
+        while (iterator.hasNext()) {
+            FileChannelLogEntry entry = iterator.next();
+            long offset = entry.offset();
+            if (offset >= targetOffset)
+                return new LogEntryPosition(offset, entry.position(), entry.sizeInBytes());
+        }
+        return null;
+    }
+
+    /**
+     * Search forward for the message whose timestamp is greater than or equals to the target timestamp.
+     *
+     * @param targetTimestamp The timestamp to search for.
+     * @param startingPosition The starting position to search.
+     * @return The timestamp and offset of the message found. None, if no message is found.
+     */
+    public TimestampAndOffset searchForTimestamp(long targetTimestamp, int startingPosition) {
+        Iterator<FileChannelLogEntry> shallowIterator = shallowIteratorFrom(startingPosition);
+        while (shallowIterator.hasNext()) {
+            LogEntry shallowEntry = shallowIterator.next();
+            Record shallowRecord = shallowEntry.record();
+            if (shallowRecord.timestamp() >= targetTimestamp) {
+                // We found a message
+                for (LogEntry deepLogEntry : shallowEntry) {
+                    long timestamp = deepLogEntry.record().timestamp();
+                    if (timestamp >= targetTimestamp)
+                        return new TimestampAndOffset(timestamp, deepLogEntry.offset());
+                }
+                throw new IllegalStateException(String.format("The message set (max timestamp = %s, max offset = %s" +
+                        " should contain target timestamp %s, but does not.", shallowRecord.timestamp(),
+                        shallowEntry.offset(), targetTimestamp));
+            }
+        }
+        return null;
+    }
+
+    /**
+     * Return the largest timestamp of the messages after a given position in this file message set.
+     * @param startingPosition The starting position.
+     * @return The largest timestamp of the messages after the given position.
+     */
+    public TimestampAndOffset largestTimestampAfter(int startingPosition) {
+        long maxTimestamp = Record.NO_TIMESTAMP;
+        long offsetOfMaxTimestamp = -1L;
+
+        Iterator<FileChannelLogEntry> shallowIterator = shallowIteratorFrom(startingPosition);
+        while (shallowIterator.hasNext()) {
+            LogEntry shallowEntry = shallowIterator.next();
+            long timestamp = shallowEntry.record().timestamp();
+            if (timestamp > maxTimestamp) {
+                maxTimestamp = timestamp;
+                offsetOfMaxTimestamp = shallowEntry.offset();
+            }
         }
+        return new TimestampAndOffset(maxTimestamp, offsetOfMaxTimestamp);
+    }
+
+    /**
+     * Get an iterator over the shallow entries in the file. Note that the entries are
+     * backed by the open file channel. When the channel is closed (i.e. when this instance
+     * is closed), the entries will generally no longer be readable.
+     * @return An iterator over the shallow entries
+     */
+    @Override
+    public Iterator<FileChannelLogEntry> shallowIterator() {
+        return shallowIteratorFrom(start);
+    }
+
+    /**
+     * Get an iterator over the shallow entries, enforcing a maximum record size
+     * @param maxRecordSize The maximum allowable size of individual records (including compressed record sets)
+     * @return An iterator over the shallow entries
+     */
+    public Iterator<FileChannelLogEntry> shallowIterator(int maxRecordSize) {
+        return shallowIteratorFrom(maxRecordSize, start);
+    }
+
+    private Iterator<FileChannelLogEntry> shallowIteratorFrom(int start) {
+        return shallowIteratorFrom(Integer.MAX_VALUE, start);
+    }
+
+    private Iterator<FileChannelLogEntry> shallowIteratorFrom(int maxRecordSize, int start) {
+        final int end;
+        if (isSlice)
+            end = this.end;
+        else
+            end = this.sizeInBytes();
+        FileLogInputStream inputStream = new FileLogInputStream(channel, maxRecordSize, start, end);
+        return RecordsIterator.shallowIterator(inputStream);
     }
 
     @Override
-    public RecordsIterator iterator() {
-        return new RecordsIterator(new FileLogInputStream(channel, start, end), false);
+    public Iterator<LogEntry> deepIterator() {
+        final int end;
+        if (isSlice)
+            end = this.end;
+        else
+            end = this.sizeInBytes();
+        FileLogInputStream inputStream = new FileLogInputStream(channel, Integer.MAX_VALUE, start, end);
+        return new RecordsIterator(inputStream, false, false, Integer.MAX_VALUE);
+    }
+
+    public static FileRecords open(File file,
+                                   boolean mutable,
+                                   boolean fileAlreadyExists,
+                                   int initFileSize,
+                                   boolean preallocate) throws IOException {
+        FileChannel channel = openChannel(file, mutable, fileAlreadyExists, initFileSize, preallocate);
+        int end = (!fileAlreadyExists && preallocate) ? 0 : Integer.MAX_VALUE;
+        return new FileRecords(file, channel, 0, end, false);
+    }
+
+    public static FileRecords open(File file,
+                                   boolean fileAlreadyExists,
+                                   int initFileSize,
+                                   boolean preallocate) throws IOException {
+        return open(file, true, fileAlreadyExists, initFileSize, preallocate);
+    }
+
+    public static FileRecords open(File file, boolean mutable) throws IOException {
+        return open(file, mutable, false, 0, false);
+    }
+
+    public static FileRecords open(File file) throws IOException {
+        return open(file, true);
+    }
+
+    /**
+     * Open a channel for the given file
+     * For windows NTFS and some old LINUX file system, set preallocate to true and initFileSize
+     * with one value (for example 512 * 1025 *1024 ) can improve the kafka produce performance.
+     * @param file File path
+     * @param mutable mutable
+     * @param fileAlreadyExists File already exists or not
+     * @param initFileSize The size used for pre allocate file, for example 512 * 1025 *1024
+     * @param preallocate Pre allocate file or not, gotten from configuration.
+     */
+    private static FileChannel openChannel(File file,
+                                           boolean mutable,
+                                           boolean fileAlreadyExists,
+                                           int initFileSize,
+                                           boolean preallocate) throws IOException {
+        if (mutable) {
+            if (fileAlreadyExists) {
+                return new RandomAccessFile(file, "rw").getChannel();
+            } else {
+                if (preallocate) {
+                    RandomAccessFile randomAccessFile = new RandomAccessFile(file, "rw");
+                    randomAccessFile.setLength(initFileSize);
+                    return randomAccessFile.getChannel();
+                } else {
+                    return new RandomAccessFile(file, "rw").getChannel();
+                }
+            }
+        } else {
+            return new FileInputStream(file).getChannel();
+        }
     }
 
-    private static class FileLogInputStream implements LogInputStream {
-        private long position;
-        protected final long end;
-        protected final FileChannel channel;
-        private final ByteBuffer logHeaderBuffer = ByteBuffer.allocate(Records.LOG_OVERHEAD);
+    public static class LogEntryPosition {
+        public final long offset;
+        public final int position;
+        public final int size;
 
-        public FileLogInputStream(FileChannel channel, long start, long end) {
-            this.channel = channel;
-            this.position = start;
-            this.end = end;
+        public LogEntryPosition(long offset, int position, int size) {
+            this.offset = offset;
+            this.position = position;
+            this.size = size;
         }
 
         @Override
-        public LogEntry nextEntry() throws IOException {
-            if (position + Records.LOG_OVERHEAD >= end)
-                return null;
+        public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
 
-            logHeaderBuffer.rewind();
-            channel.read(logHeaderBuffer, position);
-            if (logHeaderBuffer.hasRemaining())
-                return null;
+            LogEntryPosition that = (LogEntryPosition) o;
 
-            logHeaderBuffer.rewind();
-            long offset = logHeaderBuffer.getLong();
-            int size = logHeaderBuffer.getInt();
-            if (size < 0)
-                throw new IllegalStateException("Record with size " + size);
+            if (offset != that.offset) return false;
+            if (position != that.position) return false;
+            return size == that.size;
 
-            if (position + Records.LOG_OVERHEAD + size > end)
-                return null;
+        }
 
-            ByteBuffer recordBuffer = ByteBuffer.allocate(size);
-            channel.read(recordBuffer, position + Records.LOG_OVERHEAD);
-            if (recordBuffer.hasRemaining())
-                return null;
-            recordBuffer.rewind();
+        @Override
+        public int hashCode() {
+            int result = (int) (offset ^ (offset >>> 32));
+            result = 31 * result + position;
+            result = 31 * result + size;
+            return result;
+        }
 
-            Record record = new Record(recordBuffer);
-            LogEntry logEntry = new LogEntry(offset, record);
-            position += logEntry.size();
-            return logEntry;
+        @Override
+        public String toString() {
+            return "LogEntryPosition(" +
+                    "offset=" + offset +
+                    ", position=" + position +
+                    ", size=" + size +
+                    ')';
         }
     }
+
+    public static class TimestampAndOffset {
+        public final long timestamp;
+        public final long offset;
+
+        public TimestampAndOffset(long timestamp, long offset) {
+            this.timestamp = timestamp;
+            this.offset = offset;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            TimestampAndOffset that = (TimestampAndOffset) o;
+
+            if (timestamp != that.timestamp) return false;
+            return offset == that.offset;
+        }
+
+        @Override
+        public int hashCode() {
+            int result = (int) (timestamp ^ (timestamp >>> 32));
+            result = 31 * result + (int) (offset ^ (offset >>> 32));
+            return result;
+        }
+
+        @Override
+        public String toString() {
+            return "TimestampAndOffset(" +
+                    "timestamp=" + timestamp +
+                    ", offset=" + offset +
+                    ')';
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/InvalidRecordException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/InvalidRecordException.java b/clients/src/main/java/org/apache/kafka/common/record/InvalidRecordException.java
index a1009ca..ee60713 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/InvalidRecordException.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/InvalidRecordException.java
@@ -16,9 +16,9 @@
  */
 package org.apache.kafka.common.record;
 
-import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.errors.CorruptRecordException;
 
-public class InvalidRecordException extends KafkaException {
+public class InvalidRecordException extends CorruptRecordException {
 
     private static final long serialVersionUID = 1;
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java b/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java
index 2e54b56..d2db356 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java
@@ -16,33 +16,156 @@
  */
 package org.apache.kafka.common.record;
 
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.Iterator;
+
+import static org.apache.kafka.common.record.Records.LOG_OVERHEAD;
+
 /**
  * An offset and record pair
  */
-public final class LogEntry {
+public abstract class LogEntry implements Iterable<LogEntry> {
 
-    private final long offset;
-    private final Record record;
+    /**
+     * Get the offset of this entry. Note that if this entry contains a compressed
+     * message set, then this offset will be the last offset of the nested entries
+     * @return the last offset contained in this entry
+     */
+    public abstract long offset();
 
-    public LogEntry(long offset, Record record) {
-        this.offset = offset;
-        this.record = record;
+    /**
+     * Get the shallow record for this log entry.
+     * @return the shallow record
+     */
+    public abstract Record record();
+
+    /**
+     * Get the first offset of the records contained in this entry. Note that this
+     * generally requires deep iteration, which requires decompression, so this should
+     * be used with caution.
+     * @return The first offset contained in this entry
+     */
+    public long firstOffset() {
+        return iterator().next().offset();
     }
 
-    public long offset() {
-        return this.offset;
+    /**
+     * Get the offset following this entry (i.e. the last offset contained in this entry plus one).
+     * @return the next consecutive offset following this entry
+     */
+    public long nextOffset() {
+        return offset() + 1;
     }
 
-    public Record record() {
-        return this.record;
+    /**
+     * Get the message format version of this entry (i.e its magic value).
+     * @return the magic byte
+     */
+    public byte magic() {
+        return record().magic();
     }
 
     @Override
     public String toString() {
-        return "LogEntry(" + offset + ", " + record + ")";
+        return "LogEntry(" + offset() + ", " + record() + ")";
+    }
+
+    /**
+     * Get the size in bytes of this entry, including the size of the record and the log overhead.
+     * @return The size in bytes of this entry
+     */
+    public int sizeInBytes() {
+        return record().sizeInBytes() + LOG_OVERHEAD;
+    }
+
+    /**
+     * Check whether this entry contains a compressed message set.
+     * @return true if so, false otherwise
+     */
+    public boolean isCompressed() {
+        return record().compressionType() != CompressionType.NONE;
+    }
+
+    /**
+     * Write this entry into a buffer.
+     * @param buffer The buffer to write the entry to
+     */
+    public void writeTo(ByteBuffer buffer) {
+        writeHeader(buffer, offset(), record().sizeInBytes());
+        buffer.put(record().buffer().duplicate());
+    }
+
+    /**
+     * Get an iterator for the nested entries contained within this log entry. Note that
+     * if the entry is not compressed, then this method will return an iterator over the
+     * shallow entry only (i.e. this object).
+     * @return An iterator over the entries contained within this log entry
+     */
+    @Override
+    public Iterator<LogEntry> iterator() {
+        if (isCompressed())
+            return new RecordsIterator.DeepRecordsIterator(this, false, Integer.MAX_VALUE);
+        return Collections.singletonList(this).iterator();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || !(o instanceof LogEntry)) return false;
+
+        LogEntry that = (LogEntry) o;
+
+        if (offset() != that.offset()) return false;
+        Record thisRecord = record();
+        Record thatRecord = that.record();
+        return thisRecord != null ? thisRecord.equals(thatRecord) : thatRecord == null;
+    }
+
+    @Override
+    public int hashCode() {
+        long offset = offset();
+        Record record = record();
+        int result = (int) (offset ^ (offset >>> 32));
+        result = 31 * result + (record != null ? record.hashCode() : 0);
+        return result;
+    }
+
+    public static void writeHeader(ByteBuffer buffer, long offset, int size) {
+        buffer.putLong(offset);
+        buffer.putInt(size);
+    }
+
+    public static void writeHeader(DataOutputStream out, long offset, int size) throws IOException {
+        out.writeLong(offset);
+        out.writeInt(size);
     }
-    
-    public int size() {
-        return record.size() + Records.LOG_OVERHEAD;
+
+    private static class SimpleLogEntry extends LogEntry {
+        private final long offset;
+        private final Record record;
+
+        public SimpleLogEntry(long offset, Record record) {
+            this.offset = offset;
+            this.record = record;
+        }
+
+        @Override
+        public long offset() {
+            return offset;
+        }
+
+        @Override
+        public Record record() {
+            return record;
+        }
+
+    }
+
+    public static LogEntry create(long offset, Record record) {
+        return new SimpleLogEntry(offset, record);
     }
+
 }


[7/9] kafka git commit: KAFKA-4390; Replace MessageSet usage with client-side alternatives

Posted by jg...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java b/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java
index 1bc8a65..4a678d5 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java
@@ -17,6 +17,7 @@
 package org.apache.kafka.common.record;
 
 import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.errors.CorruptRecordException;
 import org.apache.kafka.common.utils.AbstractIterator;
 import org.apache.kafka.common.utils.Utils;
 
@@ -25,51 +26,58 @@ import java.io.EOFException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayDeque;
+import java.util.Iterator;
 
+/**
+ * An iterator which handles both the shallow and deep iteration of record sets.
+ */
 public class RecordsIterator extends AbstractIterator<LogEntry> {
-    private final LogInputStream logStream;
     private final boolean shallow;
+    private final boolean ensureMatchingMagic;
+    private final int masRecordSize;
+    private final ShallowRecordsIterator<?> shallowIter;
     private DeepRecordsIterator innerIter;
 
-    public RecordsIterator(LogInputStream logStream, boolean shallow) {
-        this.logStream = logStream;
+    public RecordsIterator(LogInputStream<?> logInputStream,
+                           boolean shallow,
+                           boolean ensureMatchingMagic,
+                           int masRecordSize) {
+        this.shallowIter = new ShallowRecordsIterator<>(logInputStream);
         this.shallow = shallow;
+        this.ensureMatchingMagic = ensureMatchingMagic;
+        this.masRecordSize = masRecordSize;
     }
 
-    /*
-     * Read the next record from the buffer.
-     *
-     * Note that in the compressed message set, each message value size is set as the size of the un-compressed
-     * version of the message value, so when we do de-compression allocating an array of the specified size for
-     * reading compressed value data is sufficient.
+    /**
+     * Get a shallow iterator over the given input stream.
+     * @param logInputStream The log input stream to read the entries from
+     * @param <T> The type of the log entry
+     * @return The shallow iterator.
      */
+    public static <T extends LogEntry> Iterator<T> shallowIterator(LogInputStream<T> logInputStream) {
+        return new ShallowRecordsIterator<>(logInputStream);
+    }
+
     @Override
     protected LogEntry makeNext() {
         if (innerDone()) {
-            try {
-                LogEntry entry = logStream.nextEntry();
-                // No more record to return.
-                if (entry == null)
-                    return allDone();
-
-                // decide whether to go shallow or deep iteration if it is compressed
-                CompressionType compressionType = entry.record().compressionType();
-                if (compressionType == CompressionType.NONE || shallow) {
-                    return entry;
-                } else {
-                    // init the inner iterator with the value payload of the message,
-                    // which will de-compress the payload to a set of messages;
-                    // since we assume nested compression is not allowed, the deep iterator
-                    // would not try to further decompress underlying messages
-                    // There will be at least one element in the inner iterator, so we don't
-                    // need to call hasNext() here.
-                    innerIter = new DeepRecordsIterator(entry);
-                    return innerIter.next();
-                }
-            } catch (EOFException e) {
+            if (!shallowIter.hasNext())
                 return allDone();
-            } catch (IOException e) {
-                throw new KafkaException(e);
+
+            LogEntry entry = shallowIter.next();
+
+            // decide whether to go shallow or deep iteration if it is compressed
+            if (shallow || !entry.isCompressed()) {
+                return entry;
+            } else {
+                // init the inner iterator with the value payload of the message,
+                // which will de-compress the payload to a set of messages;
+                // since we assume nested compression is not allowed, the deep iterator
+                // would not try to further decompress underlying messages
+                // There will be at least one element in the inner iterator, so we don't
+                // need to call hasNext() here.
+                innerIter = new DeepRecordsIterator(entry, ensureMatchingMagic, masRecordSize);
+                return innerIter.next();
             }
         } else {
             return innerIter.next();
@@ -80,38 +88,70 @@ public class RecordsIterator extends AbstractIterator<LogEntry> {
         return innerIter == null || !innerIter.hasNext();
     }
 
-    private static class DataLogInputStream implements LogInputStream {
+    private static class DataLogInputStream implements LogInputStream<LogEntry> {
         private final DataInputStream stream;
+        protected final int maxMessageSize;
 
-        private DataLogInputStream(DataInputStream stream) {
+        DataLogInputStream(DataInputStream stream, int maxMessageSize) {
             this.stream = stream;
+            this.maxMessageSize = maxMessageSize;
         }
 
         public LogEntry nextEntry() throws IOException {
-            long offset = stream.readLong();
-            int size = stream.readInt();
-            if (size < 0)
-                throw new IllegalStateException("Record with size " + size);
-
-            byte[] recordBuffer = new byte[size];
-            stream.readFully(recordBuffer, 0, size);
-            ByteBuffer buf = ByteBuffer.wrap(recordBuffer);
-            return new LogEntry(offset, new Record(buf));
+            try {
+                long offset = stream.readLong();
+                int size = stream.readInt();
+                if (size < Record.RECORD_OVERHEAD_V0)
+                    throw new CorruptRecordException(String.format("Record size is less than the minimum record overhead (%d)", Record.RECORD_OVERHEAD_V0));
+                if (size > maxMessageSize)
+                    throw new CorruptRecordException(String.format("Record size exceeds the largest allowable message size (%d).", maxMessageSize));
+
+                byte[] recordBuffer = new byte[size];
+                stream.readFully(recordBuffer, 0, size);
+                ByteBuffer buf = ByteBuffer.wrap(recordBuffer);
+                return LogEntry.create(offset, new Record(buf));
+            } catch (EOFException e) {
+                return null;
+            }
         }
     }
 
-    private static class DeepRecordsIterator extends AbstractIterator<LogEntry> {
+    private static class ShallowRecordsIterator<T extends LogEntry> extends AbstractIterator<T> {
+        private final LogInputStream<T> logStream;
+
+        public ShallowRecordsIterator(LogInputStream<T> logStream) {
+            this.logStream = logStream;
+        }
+
+        @Override
+        protected T makeNext() {
+            try {
+                T entry = logStream.nextEntry();
+                if (entry == null)
+                    return allDone();
+                return entry;
+            } catch (IOException e) {
+                throw new KafkaException(e);
+            }
+        }
+    }
+
+    public static class DeepRecordsIterator extends AbstractIterator<LogEntry> {
         private final ArrayDeque<LogEntry> logEntries;
         private final long absoluteBaseOffset;
+        private final byte wrapperMagic;
+
+        public DeepRecordsIterator(LogEntry wrapperEntry, boolean ensureMatchingMagic, int maxMessageSize) {
+            Record wrapperRecord = wrapperEntry.record();
+            this.wrapperMagic = wrapperRecord.magic();
 
-        private DeepRecordsIterator(LogEntry entry) {
-            CompressionType compressionType = entry.record().compressionType();
-            ByteBuffer buffer = entry.record().value();
-            DataInputStream stream = Compressor.wrapForInput(new ByteBufferInputStream(buffer), compressionType, entry.record().magic());
-            LogInputStream logStream = new DataLogInputStream(stream);
+            CompressionType compressionType = wrapperRecord.compressionType();
+            ByteBuffer buffer = wrapperRecord.value();
+            DataInputStream stream = MemoryRecordsBuilder.wrapForInput(new ByteBufferInputStream(buffer), compressionType, wrapperRecord.magic());
+            LogInputStream logStream = new DataLogInputStream(stream, maxMessageSize);
 
-            long wrapperRecordOffset = entry.offset();
-            long wrapperRecordTimestamp = entry.record().timestamp();
+            long wrapperRecordOffset = wrapperEntry.offset();
+            long wrapperRecordTimestamp = wrapperRecord.timestamp();
             this.logEntries = new ArrayDeque<>();
 
             // If relative offset is used, we need to decompress the entire message first to compute
@@ -119,22 +159,27 @@ public class RecordsIterator extends AbstractIterator<LogEntry> {
             // do the same for message format version 0
             try {
                 while (true) {
-                    try {
-                        LogEntry logEntry = logStream.nextEntry();
-                        if (entry.record().magic() > Record.MAGIC_VALUE_V0) {
-                            Record recordWithTimestamp = new Record(
-                                    logEntry.record().buffer(),
-                                    wrapperRecordTimestamp,
-                                    entry.record().timestampType()
-                            );
-                            logEntry = new LogEntry(logEntry.offset(), recordWithTimestamp);
-                        }
-                        logEntries.add(logEntry);
-                    } catch (EOFException e) {
+                    LogEntry logEntry = logStream.nextEntry();
+                    if (logEntry == null)
                         break;
+
+                    Record record = logEntry.record();
+                    byte magic = record.magic();
+
+                    if (ensureMatchingMagic && magic != wrapperMagic)
+                        throw new InvalidRecordException("Compressed message magic does not match wrapper magic");
+
+                    if (magic > Record.MAGIC_VALUE_V0) {
+                        Record recordWithTimestamp = new Record(
+                                record.buffer(),
+                                wrapperRecordTimestamp,
+                                wrapperRecord.timestampType()
+                        );
+                        logEntry = LogEntry.create(logEntry.offset(), recordWithTimestamp);
                     }
+                    logEntries.addLast(logEntry);
                 }
-                if (entry.record().magic() > Record.MAGIC_VALUE_V0)
+                if (wrapperMagic > Record.MAGIC_VALUE_V0)
                     this.absoluteBaseOffset = wrapperRecordOffset - logEntries.getLast().offset();
                 else
                     this.absoluteBaseOffset = -1;
@@ -155,12 +200,10 @@ public class RecordsIterator extends AbstractIterator<LogEntry> {
             // Convert offset to absolute offset if needed.
             if (absoluteBaseOffset >= 0) {
                 long absoluteOffset = absoluteBaseOffset + entry.offset();
-                entry = new LogEntry(absoluteOffset, entry.record());
+                entry = LogEntry.create(absoluteOffset, entry.record());
             }
 
-            // decide whether to go shallow or deep iteration if it is compressed
-            CompressionType compression = entry.record().compressionType();
-            if (compression != CompressionType.NONE)
+            if (entry.isCompressed())
                 throw new InvalidRecordException("Inner messages must not be compressed");
 
             return entry;

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java b/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java
index 62fd814..55c966a 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java
@@ -27,6 +27,7 @@ public enum TimestampType {
 
     public final int id;
     public final String name;
+
     TimestampType(int id, String name) {
         this.id = id;
         this.name = name;

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 c3c1045..c5e6716 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
@@ -266,6 +266,24 @@ public class Utils {
     }
 
     /**
+     * Convert a ByteBuffer to a nullable array.
+     * @param buffer The buffer to convert
+     * @return The resulting array or null if the buffer is null
+     */
+    public static byte[] toNullableArray(ByteBuffer buffer) {
+        return buffer == null ? null : toArray(buffer);
+    }
+
+    /**
+     * Wrap an array as a nullable ByteBuffer.
+     * @param array The nullable array to wrap
+     * @return The wrapping ByteBuffer or null if array is null
+     */
+    public static ByteBuffer wrapNullable(byte[] array) {
+        return array == null ? null : ByteBuffer.wrap(array);
+    }
+
+    /**
      * Read a byte array from the given offset and size in the buffer
      */
     public static byte[] toArray(ByteBuffer buffer, int offset, int size) {
@@ -733,4 +751,37 @@ public class Utils {
     public static int longHashcode(long value) {
         return (int) (value ^ (value >>> 32));
     }
+
+    /**
+     * Read a size-delimited byte buffer starting at the given offset.
+     * @param buffer Buffer containing the size and data
+     * @param start Offset in the buffer to read from
+     * @return A slice of the buffer containing only the delimited data (excluding the size)
+     */
+    public static ByteBuffer sizeDelimited(ByteBuffer buffer, int start) {
+        int size = buffer.getInt(start);
+        if (size < 0) {
+            return null;
+        } else {
+            ByteBuffer b = buffer.duplicate();
+            b.position(start + 4);
+            b = b.slice();
+            b.limit(size);
+            b.rewind();
+            return b;
+        }
+    }
+
+    /**
+     * Compute the checksum of a range of data
+     * @param buffer Buffer containing the data to checksum
+     * @param start Offset in the buffer to read from
+     * @param size The number of bytes to include
+     */
+    public static long computeChecksum(ByteBuffer buffer, int start, int size) {
+        Crc32 crc = new Crc32();
+        crc.update(buffer.array(), buffer.arrayOffset() + start, size);
+        return crc.getValue();
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
index ad6c127..a4386f8 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
@@ -39,6 +39,8 @@ import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
 import org.apache.kafka.common.record.CompressionType;
 import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.record.MemoryRecordsBuilder;
+import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.requests.AbstractRequest;
 import org.apache.kafka.common.requests.FetchResponse;
 import org.apache.kafka.common.requests.FetchResponse.PartitionData;
@@ -1323,11 +1325,10 @@ public class KafkaConsumerTest {
             TopicPartition partition = fetchEntry.getKey();
             long fetchOffset = fetchEntry.getValue().offset;
             int fetchCount = fetchEntry.getValue().count;
-            MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE);
+            MemoryRecordsBuilder records = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME);
             for (int i = 0; i < fetchCount; i++)
                 records.append(fetchOffset + i, 0L, ("key-" + i).getBytes(), ("value-" + i).getBytes());
-            records.close();
-            tpResponses.put(partition, new FetchResponse.PartitionData(Errors.NONE.code(), 0, records));
+            tpResponses.put(partition, new FetchResponse.PartitionData(Errors.NONE.code(), 0, records.build()));
         }
         return new FetchResponse(tpResponses, 0);
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 6d5896f..15075cb 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
@@ -37,10 +37,12 @@ import org.apache.kafka.common.errors.TopicAuthorizationException;
 import org.apache.kafka.common.metrics.KafkaMetric;
 import org.apache.kafka.common.metrics.Metrics;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.record.ByteBufferOutputStream;
 import org.apache.kafka.common.record.CompressionType;
-import org.apache.kafka.common.record.Compressor;
 import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.record.MemoryRecordsBuilder;
 import org.apache.kafka.common.record.Record;
+import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.requests.AbstractRequest;
 import org.apache.kafka.common.requests.FetchRequest;
 import org.apache.kafka.common.requests.FetchResponse;
@@ -93,8 +95,8 @@ public class FetcherTest {
     private static final double EPSILON = 0.0001;
     private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100, 1000);
 
-    private MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE);
-    private MemoryRecords nextRecords = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE);
+    private MemoryRecords records;
+    private MemoryRecords nextRecords;
     private Fetcher<byte[], byte[]> fetcher = createFetcher(subscriptions, metrics);
     private Metrics fetcherMetrics = new Metrics(time);
     private Fetcher<byte[], byte[]> fetcherNoAutoReset = createFetcher(subscriptionsNoAutoReset, fetcherMetrics);
@@ -104,14 +106,16 @@ public class FetcherTest {
         metadata.update(cluster, time.milliseconds());
         client.setNode(node);
 
-        records.append(1L, 0L, "key".getBytes(), "value-1".getBytes());
-        records.append(2L, 0L, "key".getBytes(), "value-2".getBytes());
-        records.append(3L, 0L, "key".getBytes(), "value-3".getBytes());
-        records.close();
+        MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME);
+        builder.append(1L, 0L, "key".getBytes(), "value-1".getBytes());
+        builder.append(2L, 0L, "key".getBytes(), "value-2".getBytes());
+        builder.append(3L, 0L, "key".getBytes(), "value-3".getBytes());
+        records = builder.build();
 
-        nextRecords.append(4L, 0L, "key".getBytes(), "value-4".getBytes());
-        nextRecords.append(5L, 0L, "key".getBytes(), "value-5".getBytes());
-        nextRecords.close();
+        builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME);
+        builder.append(4L, 0L, "key".getBytes(), "value-4".getBytes());
+        builder.append(5L, 0L, "key".getBytes(), "value-5".getBytes());
+        nextRecords = builder.build();
     }
 
     @After
@@ -129,7 +133,7 @@ public class FetcherTest {
         assertEquals(1, fetcher.sendFetches());
         assertFalse(fetcher.hasCompletedFetches());
 
-        client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0));
+        client.prepareResponse(fetchResponse(this.records, Errors.NONE.code(), 100L, 0));
         consumerClient.poll(0);
         assertTrue(fetcher.hasCompletedFetches());
 
@@ -154,7 +158,7 @@ public class FetcherTest {
         assertEquals(1, fetcher.sendFetches());
         assertFalse(fetcher.hasCompletedFetches());
 
-        client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NOT_LEADER_FOR_PARTITION.code(), 100L, 0));
+        client.prepareResponse(fetchResponse(this.records, Errors.NOT_LEADER_FOR_PARTITION.code(), 100L, 0));
         consumerClient.poll(0);
         assertTrue(fetcher.hasCompletedFetches());
 
@@ -192,7 +196,7 @@ public class FetcherTest {
         subscriptions.assignFromUser(singleton(tp));
         subscriptions.seek(tp, 1);
 
-        client.prepareResponse(matchesOffset(tp, 1), fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0));
+        client.prepareResponse(matchesOffset(tp, 1), fetchResponse(this.records, Errors.NONE.code(), 100L, 0));
 
         assertEquals(1, fetcher.sendFetches());
         consumerClient.poll(0);
@@ -206,29 +210,30 @@ public class FetcherTest {
     }
 
     @Test
-    public void testParseInvalidRecord() {
+    public void testParseInvalidRecord() throws Exception {
         ByteBuffer buffer = ByteBuffer.allocate(1024);
-        Compressor compressor = new Compressor(buffer, CompressionType.NONE);
+        ByteBufferOutputStream out = new ByteBufferOutputStream(buffer);
 
+        byte magic = Record.CURRENT_MAGIC_VALUE;
         byte[] key = "foo".getBytes();
         byte[] value = "baz".getBytes();
         long offset = 0;
         long timestamp = 500L;
 
         int size = Record.recordSize(key, value);
-        long crc = Record.computeChecksum(timestamp, key, value, CompressionType.NONE, 0, -1);
+        byte attributes = Record.computeAttributes(magic, CompressionType.NONE, TimestampType.CREATE_TIME);
+        long crc = Record.computeChecksum(magic, attributes, timestamp, key, value);
 
         // write one valid record
-        compressor.putLong(offset);
-        compressor.putInt(size);
-        Record.write(compressor, crc, Record.computeAttributes(CompressionType.NONE), timestamp, key, value, 0, -1);
+        out.writeLong(offset);
+        out.writeInt(size);
+        Record.write(out, magic, crc, Record.computeAttributes(magic, CompressionType.NONE, TimestampType.CREATE_TIME), timestamp, key, value);
 
         // and one invalid record (note the crc)
-        compressor.putLong(offset);
-        compressor.putInt(size);
-        Record.write(compressor, crc + 1, Record.computeAttributes(CompressionType.NONE), timestamp, key, value, 0, -1);
+        out.writeLong(offset);
+        out.writeInt(size);
+        Record.write(out, magic, crc + 1, Record.computeAttributes(magic, CompressionType.NONE, TimestampType.CREATE_TIME), timestamp, key, value);
 
-        compressor.close();
         buffer.flip();
 
         subscriptions.assignFromUser(singleton(tp));
@@ -236,7 +241,7 @@ public class FetcherTest {
 
         // normal fetch
         assertEquals(1, fetcher.sendFetches());
-        client.prepareResponse(fetchResponse(buffer, Errors.NONE.code(), 100L, 0));
+        client.prepareResponse(fetchResponse(MemoryRecords.readableRecords(buffer), Errors.NONE.code(), 100L, 0));
         consumerClient.poll(0);
         try {
             fetcher.fetchedRecords();
@@ -255,8 +260,8 @@ public class FetcherTest {
         subscriptions.assignFromUser(singleton(tp));
         subscriptions.seek(tp, 1);
 
-        client.prepareResponse(matchesOffset(tp, 1), fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0));
-        client.prepareResponse(matchesOffset(tp, 4), fetchResponse(this.nextRecords.buffer(), Errors.NONE.code(), 100L, 0));
+        client.prepareResponse(matchesOffset(tp, 1), fetchResponse(this.records, Errors.NONE.code(), 100L, 0));
+        client.prepareResponse(matchesOffset(tp, 4), fetchResponse(this.nextRecords, Errors.NONE.code(), 100L, 0));
 
         assertEquals(1, fetcher.sendFetches());
         consumerClient.poll(0);
@@ -287,11 +292,11 @@ public class FetcherTest {
         // if we are fetching from a compacted topic, there may be gaps in the returned records
         // this test verifies the fetcher updates the current fetched/consumed positions correctly for this case
 
-        MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE);
-        records.append(15L, 0L, "key".getBytes(), "value-1".getBytes());
-        records.append(20L, 0L, "key".getBytes(), "value-2".getBytes());
-        records.append(30L, 0L, "key".getBytes(), "value-3".getBytes());
-        records.close();
+        MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME);
+        builder.append(15L, 0L, "key".getBytes(), "value-1".getBytes());
+        builder.append(20L, 0L, "key".getBytes(), "value-2".getBytes());
+        builder.append(30L, 0L, "key".getBytes(), "value-3".getBytes());
+        MemoryRecords records = builder.build();
 
         List<ConsumerRecord<byte[], byte[]>> consumerRecords;
         subscriptions.assignFromUser(singleton(tp));
@@ -299,7 +304,7 @@ public class FetcherTest {
 
         // normal fetch
         assertEquals(1, fetcher.sendFetches());
-        client.prepareResponse(fetchResponse(records.buffer(), Errors.NONE.code(), 100L, 0));
+        client.prepareResponse(fetchResponse(records, Errors.NONE.code(), 100L, 0));
         consumerClient.poll(0);
         consumerRecords = fetcher.fetchedRecords().get(tp);
         assertEquals(3, consumerRecords.size());
@@ -317,7 +322,7 @@ public class FetcherTest {
 
         // resize the limit of the buffer to pretend it is only fetch-size large
         assertEquals(1, fetcher.sendFetches());
-        client.prepareResponse(fetchResponse(this.records.buffer(), Errors.TOPIC_AUTHORIZATION_FAILED.code(), 100L, 0));
+        client.prepareResponse(fetchResponse(this.records, Errors.TOPIC_AUTHORIZATION_FAILED.code(), 100L, 0));
         consumerClient.poll(0);
         try {
             fetcher.fetchedRecords();
@@ -337,7 +342,7 @@ public class FetcherTest {
 
         // Now the rebalance happens and fetch positions are cleared
         subscriptions.assignFromSubscribed(singleton(tp));
-        client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0));
+        client.prepareResponse(fetchResponse(this.records, Errors.NONE.code(), 100L, 0));
         consumerClient.poll(0);
 
         // The active fetch should be ignored since its position is no longer valid
@@ -352,7 +357,7 @@ public class FetcherTest {
         assertEquals(1, fetcher.sendFetches());
         subscriptions.pause(tp);
 
-        client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0));
+        client.prepareResponse(fetchResponse(this.records, Errors.NONE.code(), 100L, 0));
         consumerClient.poll(0);
         assertNull(fetcher.fetchedRecords().get(tp));
     }
@@ -373,7 +378,7 @@ public class FetcherTest {
         subscriptions.seek(tp, 0);
 
         assertEquals(1, fetcher.sendFetches());
-        client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NOT_LEADER_FOR_PARTITION.code(), 100L, 0));
+        client.prepareResponse(fetchResponse(this.records, Errors.NOT_LEADER_FOR_PARTITION.code(), 100L, 0));
         consumerClient.poll(0);
         assertEquals(0, fetcher.fetchedRecords().size());
         assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds()));
@@ -385,7 +390,7 @@ public class FetcherTest {
         subscriptions.seek(tp, 0);
 
         assertEquals(1, fetcher.sendFetches());
-        client.prepareResponse(fetchResponse(this.records.buffer(), Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), 100L, 0));
+        client.prepareResponse(fetchResponse(this.records, Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), 100L, 0));
         consumerClient.poll(0);
         assertEquals(0, fetcher.fetchedRecords().size());
         assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds()));
@@ -397,7 +402,7 @@ public class FetcherTest {
         subscriptions.seek(tp, 0);
 
         assertEquals(1, fetcher.sendFetches());
-        client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
+        client.prepareResponse(fetchResponse(this.records, Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
         consumerClient.poll(0);
         assertEquals(0, fetcher.fetchedRecords().size());
         assertTrue(subscriptions.isOffsetResetNeeded(tp));
@@ -412,7 +417,7 @@ public class FetcherTest {
         subscriptions.seek(tp, 0);
 
         assertEquals(1, fetcher.sendFetches());
-        client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
+        client.prepareResponse(fetchResponse(this.records, Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
         subscriptions.seek(tp, 1);
         consumerClient.poll(0);
         assertEquals(0, fetcher.fetchedRecords().size());
@@ -426,7 +431,7 @@ public class FetcherTest {
         subscriptionsNoAutoReset.seek(tp, 0);
 
         assertTrue(fetcherNoAutoReset.sendFetches() > 0);
-        client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
+        client.prepareResponse(fetchResponse(this.records, Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
         consumerClient.poll(0);
         assertFalse(subscriptionsNoAutoReset.isOffsetResetNeeded(tp));
         subscriptionsNoAutoReset.seek(tp, 2);
@@ -439,7 +444,7 @@ public class FetcherTest {
         subscriptionsNoAutoReset.seek(tp, 0);
 
         fetcherNoAutoReset.sendFetches();
-        client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
+        client.prepareResponse(fetchResponse(this.records, Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
         consumerClient.poll(0);
 
         assertFalse(subscriptionsNoAutoReset.isOffsetResetNeeded(tp));
@@ -459,7 +464,7 @@ public class FetcherTest {
         subscriptions.seek(tp, 0);
 
         assertEquals(1, fetcher.sendFetches());
-        client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0), true);
+        client.prepareResponse(fetchResponse(this.records, Errors.NONE.code(), 100L, 0), true);
         consumerClient.poll(0);
         assertEquals(0, fetcher.fetchedRecords().size());
 
@@ -611,14 +616,14 @@ public class FetcherTest {
             // We need to make sure the message offset grows. Otherwise they will be considered as already consumed
             // and filtered out by consumer.
             if (i > 1) {
-                this.records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE);
+                MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME);
                 for (int v = 0; v < 3; v++) {
-                    this.records.append((long) i * 3 + v, Record.NO_TIMESTAMP, "key".getBytes(), String.format("value-%d", v).getBytes());
+                    builder.append((long) i * 3 + v, Record.NO_TIMESTAMP, "key".getBytes(), String.format("value-%d", v).getBytes());
                 }
-                this.records.close();
+                this.records = builder.build();
             }
             assertEquals(1, fetcher.sendFetches());
-            client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 100 * i));
+            client.prepareResponse(fetchResponse(this.records, Errors.NONE.code(), 100L, 100 * i));
             consumerClient.poll(0);
             records = fetcher.fetchedRecords().get(tp);
             assertEquals(3, records.size());
@@ -722,8 +727,7 @@ public class FetcherTest {
         return new ListOffsetResponse(allPartitionData, 1);
     }
 
-    private FetchResponse fetchResponse(ByteBuffer buffer, short error, long hw, int throttleTime) {
-        MemoryRecords records = MemoryRecords.readableRecords(buffer);
+    private FetchResponse fetchResponse(MemoryRecords records, short error, long hw, int throttleTime) {
         return new FetchResponse(Collections.singletonMap(tp, new FetchResponse.PartitionData(error, hw, records)), throttleTime);
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java
index 28521e8..4f25bdf 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java
@@ -12,23 +12,6 @@
  */
 package org.apache.kafka.clients.producer.internals;
 
-import static java.util.Arrays.asList;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Deque;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-
 import org.apache.kafka.clients.producer.Callback;
 import org.apache.kafka.clients.producer.RecordMetadata;
 import org.apache.kafka.common.Cluster;
@@ -45,6 +28,23 @@ import org.apache.kafka.common.utils.Time;
 import org.junit.After;
 import org.junit.Test;
 
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static java.util.Arrays.asList;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 public class RecordAccumulatorTest {
 
     private String topic = "test";
@@ -84,7 +84,7 @@ public class RecordAccumulatorTest {
             accum.append(tp1, 0L, key, value, null, maxBlockTimeMs);
             Deque<RecordBatch> partitionBatches = accum.batches().get(tp1);
             assertEquals(1, partitionBatches.size());
-            assertTrue(partitionBatches.peekFirst().records.isWritable());
+            assertTrue(partitionBatches.peekFirst().isWritable());
             assertEquals("No partitions should be ready.", 0, accum.ready(cluster, now).readyNodes.size());
         }
 
@@ -93,15 +93,15 @@ public class RecordAccumulatorTest {
         Deque<RecordBatch> partitionBatches = accum.batches().get(tp1);
         assertEquals(2, partitionBatches.size());
         Iterator<RecordBatch> partitionBatchesIterator = partitionBatches.iterator();
-        assertFalse(partitionBatchesIterator.next().records.isWritable());
-        assertTrue(partitionBatchesIterator.next().records.isWritable());
+        assertFalse(partitionBatchesIterator.next().isWritable());
+        assertTrue(partitionBatchesIterator.next().isWritable());
         assertEquals("Our partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes);
 
         List<RecordBatch> batches = accum.drain(cluster, Collections.singleton(node1), Integer.MAX_VALUE, 0).get(node1.id());
         assertEquals(1, batches.size());
         RecordBatch batch = batches.get(0);
 
-        Iterator<LogEntry> iter = batch.records.iterator();
+        Iterator<LogEntry> iter = batch.records().deepIterator();
         for (int i = 0; i < appends; i++) {
             LogEntry entry = iter.next();
             assertEquals("Keys should match", ByteBuffer.wrap(key), entry.record().key());
@@ -130,7 +130,7 @@ public class RecordAccumulatorTest {
         assertEquals(1, batches.size());
         RecordBatch batch = batches.get(0);
 
-        Iterator<LogEntry> iter = batch.records.iterator();
+        Iterator<LogEntry> iter = batch.records().deepIterator();
         LogEntry entry = iter.next();
         assertEquals("Keys should match", ByteBuffer.wrap(key), entry.record().key());
         assertEquals("Values should match", ByteBuffer.wrap(value), entry.record().value());
@@ -159,7 +159,7 @@ public class RecordAccumulatorTest {
         final int msgs = 10000;
         final int numParts = 2;
         final RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 0L, 100L, metrics, time);
-        List<Thread> threads = new ArrayList<Thread>();
+        List<Thread> threads = new ArrayList<>();
         for (int i = 0; i < numThreads; i++) {
             threads.add(new Thread() {
                 public void run() {
@@ -182,8 +182,11 @@ public class RecordAccumulatorTest {
             List<RecordBatch> batches = accum.drain(cluster, nodes, 5 * 1024, 0).get(node1.id());
             if (batches != null) {
                 for (RecordBatch batch : batches) {
-                    for (LogEntry entry : batch.records)
+                    Iterator<LogEntry> deepEntries = batch.records().deepIterator();
+                    while (deepEntries.hasNext()) {
+                        deepEntries.next();
                         read++;
+                    }
                     accum.deallocate(batch);
                 }
             }

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/test/java/org/apache/kafka/common/record/ByteBufferLogInputStreamTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/record/ByteBufferLogInputStreamTest.java b/clients/src/test/java/org/apache/kafka/common/record/ByteBufferLogInputStreamTest.java
new file mode 100644
index 0000000..62e8a05
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/record/ByteBufferLogInputStreamTest.java
@@ -0,0 +1,110 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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 java.util.Iterator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class ByteBufferLogInputStreamTest {
+
+    @Test
+    public void iteratorIgnoresIncompleteEntries() {
+        ByteBuffer buffer = ByteBuffer.allocate(2048);
+        MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, Record.MAGIC_VALUE_V1, CompressionType.NONE, TimestampType.CREATE_TIME, 0L);
+        builder.append(0L, 15L, "a".getBytes(), "1".getBytes());
+        builder.append(1L, 20L, "b".getBytes(), "2".getBytes());
+
+        ByteBuffer recordsBuffer = builder.build().buffer();
+        recordsBuffer.limit(recordsBuffer.limit() - 5);
+
+        Iterator<ByteBufferLogInputStream.ByteBufferLogEntry> iterator = MemoryRecords.readableRecords(recordsBuffer).shallowIterator();
+        assertTrue(iterator.hasNext());
+        ByteBufferLogInputStream.ByteBufferLogEntry first = iterator.next();
+        assertEquals(0L, first.offset());
+
+        assertFalse(iterator.hasNext());
+    }
+
+    @Test
+    public void testSetCreateTimeV1() {
+        ByteBuffer buffer = ByteBuffer.allocate(2048);
+        MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, Record.MAGIC_VALUE_V1, CompressionType.NONE, TimestampType.CREATE_TIME, 0L);
+        builder.append(0L, 15L, "a".getBytes(), "1".getBytes());
+        Iterator<ByteBufferLogInputStream.ByteBufferLogEntry> iterator = builder.build().shallowIterator();
+
+        assertTrue(iterator.hasNext());
+        ByteBufferLogInputStream.ByteBufferLogEntry entry = iterator.next();
+
+        long createTimeMs = 20L;
+        entry.setCreateTime(createTimeMs);
+
+        assertEquals(TimestampType.CREATE_TIME, entry.record().timestampType());
+        assertEquals(createTimeMs, entry.record().timestamp());
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testSetCreateTimeNotAllowedV0() {
+        ByteBuffer buffer = ByteBuffer.allocate(2048);
+        MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, Record.MAGIC_VALUE_V0, CompressionType.NONE, TimestampType.CREATE_TIME, 0L);
+        builder.append(0L, 15L, "a".getBytes(), "1".getBytes());
+        Iterator<ByteBufferLogInputStream.ByteBufferLogEntry> iterator = builder.build().shallowIterator();
+
+        assertTrue(iterator.hasNext());
+        ByteBufferLogInputStream.ByteBufferLogEntry entry = iterator.next();
+
+        long createTimeMs = 20L;
+        entry.setCreateTime(createTimeMs);
+    }
+
+    @Test
+    public void testSetLogAppendTimeV1() {
+        ByteBuffer buffer = ByteBuffer.allocate(2048);
+        MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, Record.MAGIC_VALUE_V1, CompressionType.NONE, TimestampType.CREATE_TIME, 0L);
+        builder.append(0L, 15L, "a".getBytes(), "1".getBytes());
+        Iterator<ByteBufferLogInputStream.ByteBufferLogEntry> iterator = builder.build().shallowIterator();
+
+        assertTrue(iterator.hasNext());
+        ByteBufferLogInputStream.ByteBufferLogEntry entry = iterator.next();
+
+        long logAppendTime = 20L;
+        entry.setLogAppendTime(logAppendTime);
+
+        assertEquals(TimestampType.LOG_APPEND_TIME, entry.record().timestampType());
+        assertEquals(logAppendTime, entry.record().timestamp());
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testSetLogAppendTimeNotAllowedV0() {
+        ByteBuffer buffer = ByteBuffer.allocate(2048);
+        MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, Record.MAGIC_VALUE_V0, CompressionType.NONE, TimestampType.CREATE_TIME, 0L);
+        builder.append(0L, 15L, "a".getBytes(), "1".getBytes());
+        Iterator<ByteBufferLogInputStream.ByteBufferLogEntry> iterator = builder.build().shallowIterator();
+
+        assertTrue(iterator.hasNext());
+        ByteBufferLogInputStream.ByteBufferLogEntry entry = iterator.next();
+
+        long logAppendTime = 20L;
+        entry.setLogAppendTime(logAppendTime);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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
new file mode 100644
index 0000000..7e2c256
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java
@@ -0,0 +1,410 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.KafkaException;
+import org.apache.kafka.test.TestUtils;
+import org.easymock.EasyMock;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.apache.kafka.test.TestUtils.tempFile;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class FileRecordsTest {
+
+    private Record[] records = new Record[] {
+            Record.create("abcd".getBytes()),
+            Record.create("efgh".getBytes()),
+            Record.create("ijkl".getBytes())
+    };
+    private FileRecords fileRecords;
+
+    @Before
+    public void setup() throws IOException {
+        this.fileRecords = createFileRecords(records);
+    }
+
+    /**
+     * Test that the cached size variable matches the actual file size as we append messages
+     */
+    @Test
+    public void testFileSize() throws IOException {
+        assertEquals(fileRecords.channel().size(), fileRecords.sizeInBytes());
+        for (int i = 0; i < 20; i++) {
+            fileRecords.append(MemoryRecords.withRecords(Record.create("abcd".getBytes())));
+            assertEquals(fileRecords.channel().size(), fileRecords.sizeInBytes());
+        }
+    }
+
+    /**
+     * Test that adding invalid bytes to the end of the log doesn't break iteration
+     */
+    @Test
+    public void testIterationOverPartialAndTruncation() throws IOException {
+        testPartialWrite(0, fileRecords);
+        testPartialWrite(2, fileRecords);
+        testPartialWrite(4, fileRecords);
+        testPartialWrite(5, fileRecords);
+        testPartialWrite(6, fileRecords);
+    }
+
+    private void testPartialWrite(int size, FileRecords fileRecords) throws IOException {
+        ByteBuffer buffer = ByteBuffer.allocate(size);
+        for (int i = 0; i < size; i++)
+            buffer.put((byte) 0);
+
+        buffer.rewind();
+
+        fileRecords.channel().write(buffer);
+
+        // appending those bytes should not change the contents
+        TestUtils.checkEquals(Arrays.asList(records).iterator(), fileRecords.records());
+    }
+
+    /**
+     * Iterating over the file does file reads but shouldn't change the position of the underlying FileChannel.
+     */
+    @Test
+    public void testIterationDoesntChangePosition() throws IOException {
+        long position = fileRecords.channel().position();
+        TestUtils.checkEquals(Arrays.asList(records).iterator(), fileRecords.records());
+        assertEquals(position, fileRecords.channel().position());
+    }
+
+    /**
+     * Test a simple append and read.
+     */
+    @Test
+    public void testRead() throws IOException {
+        FileRecords read = fileRecords.read(0, fileRecords.sizeInBytes());
+        TestUtils.checkEquals(fileRecords.shallowIterator(), read.shallowIterator());
+
+        List<LogEntry> items = shallowEntries(read);
+        LogEntry second = items.get(1);
+
+        read = fileRecords.read(second.sizeInBytes(), fileRecords.sizeInBytes());
+        assertEquals("Try a read starting from the second message",
+                items.subList(1, 3), shallowEntries(read));
+
+        read = fileRecords.read(second.sizeInBytes(), second.sizeInBytes());
+        assertEquals("Try a read of a single message starting from the second message",
+                Collections.singletonList(second), shallowEntries(read));
+    }
+
+    /**
+     * Test the MessageSet.searchFor API.
+     */
+    @Test
+    public void testSearch() throws IOException {
+        // append a new message with a high offset
+        Record lastMessage = Record.create("test".getBytes());
+        fileRecords.append(MemoryRecords.withRecords(50L, lastMessage));
+
+        List<LogEntry> entries = shallowEntries(fileRecords);
+        int position = 0;
+
+        int message1Size = entries.get(0).sizeInBytes();
+        assertEquals("Should be able to find the first message by its offset",
+                new FileRecords.LogEntryPosition(0L, position, message1Size),
+                fileRecords.searchForOffsetWithSize(0, 0));
+        position += message1Size;
+
+        int message2Size = entries.get(1).sizeInBytes();
+        assertEquals("Should be able to find second message when starting from 0",
+                new FileRecords.LogEntryPosition(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),
+                fileRecords.searchForOffsetWithSize(1, position));
+        position += message2Size + entries.get(2).sizeInBytes();
+
+        int message4Size = entries.get(3).sizeInBytes();
+        assertEquals("Should be able to find fourth message from a non-existant offset",
+                new FileRecords.LogEntryPosition(50L, position, message4Size),
+                fileRecords.searchForOffsetWithSize(3, position));
+        assertEquals("Should be able to find fourth message by correct offset",
+                new FileRecords.LogEntryPosition(50L, position, message4Size),
+                fileRecords.searchForOffsetWithSize(50,  position));
+    }
+
+    /**
+     * Test that the message set iterator obeys start and end slicing
+     */
+    @Test
+    public void testIteratorWithLimits() throws IOException {
+        LogEntry entry = shallowEntries(fileRecords).get(1);
+        int start = fileRecords.searchForOffsetWithSize(1, 0).position;
+        int size = entry.sizeInBytes();
+        FileRecords slice = fileRecords.read(start, size);
+        assertEquals(Collections.singletonList(entry), shallowEntries(slice));
+        FileRecords slice2 = fileRecords.read(start, size - 1);
+        assertEquals(Collections.emptyList(), shallowEntries(slice2));
+    }
+
+    /**
+     * Test the truncateTo method lops off messages and appropriately updates the size
+     */
+    @Test
+    public void testTruncate() throws IOException {
+        LogEntry entry = shallowEntries(fileRecords).get(0);
+        int end = fileRecords.searchForOffsetWithSize(1, 0).position;
+        fileRecords.truncateTo(end);
+        assertEquals(Collections.singletonList(entry), shallowEntries(fileRecords));
+        assertEquals(entry.sizeInBytes(), fileRecords.sizeInBytes());
+    }
+
+    /**
+     * Test that truncateTo only calls truncate on the FileChannel if the size of the
+     * FileChannel is bigger than the target size. This is important because some JVMs
+     * change the mtime of the file, even if truncate should do nothing.
+     */
+    @Test
+    public void testTruncateNotCalledIfSizeIsSameAsTargetSize() throws IOException {
+        FileChannel channelMock = EasyMock.createMock(FileChannel.class);
+
+        EasyMock.expect(channelMock.size()).andReturn(42L).atLeastOnce();
+        EasyMock.expect(channelMock.position(42L)).andReturn(null);
+        EasyMock.replay(channelMock);
+
+        FileRecords fileRecords = new FileRecords(tempFile(), channelMock, 0, Integer.MAX_VALUE, false);
+        fileRecords.truncateTo(42);
+
+        EasyMock.verify(channelMock);
+    }
+
+    /**
+     * Expect a KafkaException if targetSize is bigger than the size of
+     * the FileRecords.
+     */
+    @Test
+    public void testTruncateNotCalledIfSizeIsBiggerThanTargetSize() throws IOException {
+        FileChannel channelMock = EasyMock.createMock(FileChannel.class);
+
+        EasyMock.expect(channelMock.size()).andReturn(42L).atLeastOnce();
+        EasyMock.expect(channelMock.position(42L)).andReturn(null);
+        EasyMock.replay(channelMock);
+
+        FileRecords fileRecords = new FileRecords(tempFile(), channelMock, 0, Integer.MAX_VALUE, false);
+
+        try {
+            fileRecords.truncateTo(43);
+            fail("Should throw KafkaException");
+        } catch (KafkaException e) {
+            // expected
+        }
+
+        EasyMock.verify(channelMock);
+    }
+
+    /**
+     * see #testTruncateNotCalledIfSizeIsSameAsTargetSize
+     */
+    @Test
+    public void testTruncateIfSizeIsDifferentToTargetSize() throws IOException {
+        FileChannel channelMock = EasyMock.createMock(FileChannel.class);
+
+        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);
+        fileRecords.truncateTo(23);
+
+        EasyMock.verify(channelMock);
+    }
+
+    /**
+     * Test the new FileRecords with pre allocate as true
+     */
+    @Test
+    public void testPreallocateTrue() throws IOException {
+        File temp = tempFile();
+        FileRecords fileRecords = FileRecords.open(temp, false, 512 * 1024 * 1024, true);
+        long position = fileRecords.channel().position();
+        int size = fileRecords.sizeInBytes();
+        assertEquals(0, position);
+        assertEquals(0, size);
+        assertEquals(512 * 1024 * 1024, temp.length());
+    }
+
+    /**
+     * Test the new FileRecords with pre allocate as false
+     */
+    @Test
+    public void testPreallocateFalse() throws IOException {
+        File temp = tempFile();
+        FileRecords set = FileRecords.open(temp, false, 512 * 1024 * 1024, false);
+        long position = set.channel().position();
+        int size = set.sizeInBytes();
+        assertEquals(0, position);
+        assertEquals(0, size);
+        assertEquals(0, temp.length());
+    }
+
+    /**
+     * Test the new FileRecords with pre allocate as true and file has been clearly shut down, the file will be truncate to end of valid data.
+     */
+    @Test
+    public void testPreallocateClearShutdown() throws IOException {
+        File temp = tempFile();
+        FileRecords set = FileRecords.open(temp, false, 512 * 1024 * 1024, true);
+        set.append(MemoryRecords.withRecords(records));
+
+        int oldPosition = (int) set.channel().position();
+        int oldSize = set.sizeInBytes();
+        assertEquals(fileRecords.sizeInBytes(), oldPosition);
+        assertEquals(fileRecords.sizeInBytes(), oldSize);
+        set.close();
+
+        File tempReopen = new File(temp.getAbsolutePath());
+        FileRecords setReopen = FileRecords.open(tempReopen, true, 512 * 1024 * 1024, true);
+        int position = (int) setReopen.channel().position();
+        int size = setReopen.sizeInBytes();
+
+        assertEquals(oldPosition, position);
+        assertEquals(oldPosition, size);
+        assertEquals(oldPosition, tempReopen.length());
+    }
+
+    @Test
+    public void testFormatConversionWithPartialMessage() throws IOException {
+        LogEntry entry = shallowEntries(fileRecords).get(1);
+        int start = fileRecords.searchForOffsetWithSize(1, 0).position;
+        int size = entry.sizeInBytes();
+        FileRecords slice = fileRecords.read(start, size - 1);
+        Records messageV0 = slice.toMessageFormat(Record.MAGIC_VALUE_V0);
+        assertTrue("No message should be there", shallowEntries(messageV0).isEmpty());
+        assertEquals("There should be " + (size - 1) + " bytes", size - 1, messageV0.sizeInBytes());
+    }
+
+    @Test
+    public void testConvertNonCompressedToMagic1() throws IOException {
+        List<LogEntry> entries = Arrays.asList(
+                LogEntry.create(0L, Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, "k1".getBytes(), "hello".getBytes())),
+                LogEntry.create(2L, Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, "k2".getBytes(), "goodbye".getBytes())));
+        MemoryRecords records = MemoryRecords.withLogEntries(CompressionType.NONE, entries);
+
+        // Up conversion. In reality we only do down conversion, but up conversion should work as well.
+        // up conversion for non-compressed messages
+        try (FileRecords fileRecords = FileRecords.open(tempFile())) {
+            fileRecords.append(records);
+            fileRecords.flush();
+            Records convertedRecords = fileRecords.toMessageFormat(Record.MAGIC_VALUE_V1);
+            verifyConvertedMessageSet(entries, convertedRecords, Record.MAGIC_VALUE_V1);
+        }
+    }
+
+    @Test
+    public void testConvertCompressedToMagic1() throws IOException {
+        List<LogEntry> entries = Arrays.asList(
+                LogEntry.create(0L, Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, "k1".getBytes(), "hello".getBytes())),
+                LogEntry.create(2L, Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, "k2".getBytes(), "goodbye".getBytes())));
+        MemoryRecords records = MemoryRecords.withLogEntries(CompressionType.GZIP, entries);
+
+        // up conversion for compressed messages
+        try (FileRecords fileRecords = FileRecords.open(tempFile())) {
+            fileRecords.append(records);
+            fileRecords.flush();
+            Records convertedRecords = fileRecords.toMessageFormat(Record.MAGIC_VALUE_V1);
+            verifyConvertedMessageSet(entries, convertedRecords, Record.MAGIC_VALUE_V1);
+        }
+    }
+
+    @Test
+    public void testConvertNonCompressedToMagic0() throws IOException {
+        List<LogEntry> entries = Arrays.asList(
+                LogEntry.create(0L, Record.create(Record.MAGIC_VALUE_V1, 1L, "k1".getBytes(), "hello".getBytes())),
+                LogEntry.create(2L, Record.create(Record.MAGIC_VALUE_V1, 2L, "k2".getBytes(), "goodbye".getBytes())));
+        MemoryRecords records = MemoryRecords.withLogEntries(CompressionType.NONE, entries);
+
+        // down conversion for non-compressed messages
+        try (FileRecords fileRecords = FileRecords.open(tempFile())) {
+            fileRecords.append(records);
+            fileRecords.flush();
+            Records convertedRecords = fileRecords.toMessageFormat(Record.MAGIC_VALUE_V0);
+            verifyConvertedMessageSet(entries, convertedRecords, Record.MAGIC_VALUE_V0);
+        }
+    }
+
+    @Test
+    public void testConvertCompressedToMagic0() throws IOException {
+        List<LogEntry> entries = Arrays.asList(
+                LogEntry.create(0L, Record.create(Record.MAGIC_VALUE_V1, 1L, "k1".getBytes(), "hello".getBytes())),
+                LogEntry.create(2L, Record.create(Record.MAGIC_VALUE_V1, 2L, "k2".getBytes(), "goodbye".getBytes())));
+        MemoryRecords records = MemoryRecords.withLogEntries(CompressionType.GZIP, entries);
+
+        // down conversion for compressed messages
+        try (FileRecords fileRecords = FileRecords.open(tempFile())) {
+            fileRecords.append(records);
+            fileRecords.flush();
+            Records convertedRecords = fileRecords.toMessageFormat(Record.MAGIC_VALUE_V0);
+            verifyConvertedMessageSet(entries, convertedRecords, Record.MAGIC_VALUE_V0);
+        }
+    }
+
+    private void verifyConvertedMessageSet(List<LogEntry> initialEntries, Records convertedRecords, byte magicByte) {
+        int i = 0;
+        for (LogEntry logEntry : deepEntries(convertedRecords)) {
+            assertEquals("magic byte should be " + magicByte, magicByte, logEntry.record().magic());
+            assertEquals("offset should not change", initialEntries.get(i).offset(), logEntry.offset());
+            assertEquals("key should not change", initialEntries.get(i).record().key(), logEntry.record().key());
+            assertEquals("payload should not change", initialEntries.get(i).record().value(), logEntry.record().value());
+            i += 1;
+        }
+    }
+
+    private static List<LogEntry> shallowEntries(Records buffer) {
+        List<LogEntry> entries = new ArrayList<>();
+        Iterator<? extends LogEntry> iterator = buffer.shallowIterator();
+        while (iterator.hasNext())
+            entries.add(iterator.next());
+        return entries;
+    }
+
+    private static List<LogEntry> deepEntries(Records buffer) {
+        List<LogEntry> entries = new ArrayList<>();
+        Iterator<? extends LogEntry> iterator = buffer.shallowIterator();
+        while (iterator.hasNext()) {
+            for (LogEntry deepEntry : iterator.next())
+                entries.add(deepEntry);
+        }
+        return entries;
+    }
+
+    private FileRecords createFileRecords(Record ... records) throws IOException {
+        FileRecords fileRecords = FileRecords.open(tempFile());
+        fileRecords.append(MemoryRecords.withRecords(records));
+        fileRecords.flush();
+        return fileRecords;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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
new file mode 100644
index 0000000..40fa212
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java
@@ -0,0 +1,253 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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 org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+@RunWith(value = Parameterized.class)
+public class MemoryRecordsBuilderTest {
+
+    private final CompressionType compressionType;
+    private final int bufferOffset;
+
+    public MemoryRecordsBuilderTest(int bufferOffset, CompressionType compressionType) {
+        this.bufferOffset = bufferOffset;
+        this.compressionType = compressionType;
+    }
+
+    @Test
+    public void testCompressionRateV0() {
+        ByteBuffer buffer = ByteBuffer.allocate(1024);
+        buffer.position(bufferOffset);
+
+        Record[] records = new Record[] {
+                Record.create(Record.MAGIC_VALUE_V0, 0L, "a".getBytes(), "1".getBytes()),
+                Record.create(Record.MAGIC_VALUE_V0, 1L, "b".getBytes(), "2".getBytes()),
+                Record.create(Record.MAGIC_VALUE_V0, 2L, "c".getBytes(), "3".getBytes()),
+        };
+
+        MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.MAGIC_VALUE_V0, compressionType,
+                TimestampType.CREATE_TIME, 0L, 0L, buffer.capacity());
+
+        int uncompressedSize = 0;
+        long offset = 0L;
+        for (Record record : records) {
+            uncompressedSize += record.sizeInBytes() + Records.LOG_OVERHEAD;
+            builder.append(offset++, record);
+        }
+
+        MemoryRecords built = builder.build();
+        if (compressionType == CompressionType.NONE) {
+            assertEquals(1.0, builder.compressionRate(), 0.00001);
+        } else {
+            int compressedSize = built.sizeInBytes() - Records.LOG_OVERHEAD - Record.RECORD_OVERHEAD_V0;
+            double computedCompressionRate = (double) compressedSize / uncompressedSize;
+            assertEquals(computedCompressionRate, builder.compressionRate(), 0.00001);
+        }
+    }
+
+    @Test
+    public void testCompressionRateV1() {
+        ByteBuffer buffer = ByteBuffer.allocate(1024);
+        buffer.position(bufferOffset);
+
+        Record[] records = new Record[] {
+                Record.create(Record.MAGIC_VALUE_V1, 0L, "a".getBytes(), "1".getBytes()),
+                Record.create(Record.MAGIC_VALUE_V1, 1L, "b".getBytes(), "2".getBytes()),
+                Record.create(Record.MAGIC_VALUE_V1, 2L, "c".getBytes(), "3".getBytes()),
+        };
+
+        MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.MAGIC_VALUE_V1, compressionType,
+                TimestampType.CREATE_TIME, 0L, 0L, buffer.capacity());
+
+        int uncompressedSize = 0;
+        long offset = 0L;
+        for (Record record : records) {
+            uncompressedSize += record.sizeInBytes() + Records.LOG_OVERHEAD;
+            builder.append(offset++, record);
+        }
+
+        MemoryRecords built = builder.build();
+        if (compressionType == CompressionType.NONE) {
+            assertEquals(1.0, builder.compressionRate(), 0.00001);
+        } else {
+            int compressedSize = built.sizeInBytes() - Records.LOG_OVERHEAD - Record.RECORD_OVERHEAD_V1;
+            double computedCompressionRate = (double) compressedSize / uncompressedSize;
+            assertEquals(computedCompressionRate, builder.compressionRate(), 0.00001);
+        }
+    }
+
+    @Test
+    public void buildUsingLogAppendTime() {
+        ByteBuffer buffer = ByteBuffer.allocate(1024);
+        buffer.position(bufferOffset);
+
+        long logAppendTime = System.currentTimeMillis();
+        MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.MAGIC_VALUE_V1, compressionType,
+                TimestampType.LOG_APPEND_TIME, 0L, logAppendTime, buffer.capacity());
+        builder.append(0L, 0L, "a".getBytes(), "1".getBytes());
+        builder.append(1L, 0L, "b".getBytes(), "2".getBytes());
+        builder.append(2L, 0L, "c".getBytes(), "3".getBytes());
+        MemoryRecords records = builder.build();
+
+        MemoryRecordsBuilder.RecordsInfo info = builder.info();
+        assertEquals(logAppendTime, info.maxTimestamp);
+
+        assertEquals(2L, info.shallowOffsetOfMaxTimestamp);
+
+        Iterator<Record> iterator = records.records();
+        while (iterator.hasNext()) {
+            Record record = iterator.next();
+            assertEquals(TimestampType.LOG_APPEND_TIME, record.timestampType());
+            assertEquals(logAppendTime, record.timestamp());
+        }
+    }
+
+    @Test
+    public void convertUsingLogAppendTime() {
+        ByteBuffer buffer = ByteBuffer.allocate(1024);
+        buffer.position(bufferOffset);
+
+        long logAppendTime = System.currentTimeMillis();
+        MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.MAGIC_VALUE_V1, compressionType,
+                TimestampType.LOG_APPEND_TIME, 0L, logAppendTime, buffer.capacity());
+
+        builder.convertAndAppend(0L, Record.create(Record.MAGIC_VALUE_V0, 0L, "a".getBytes(), "1".getBytes()));
+        builder.convertAndAppend(1L, Record.create(Record.MAGIC_VALUE_V0, 0L, "b".getBytes(), "2".getBytes()));
+        builder.convertAndAppend(2L, Record.create(Record.MAGIC_VALUE_V0, 0L, "c".getBytes(), "3".getBytes()));
+        MemoryRecords records = builder.build();
+
+        MemoryRecordsBuilder.RecordsInfo info = builder.info();
+        assertEquals(logAppendTime, info.maxTimestamp);
+
+        assertEquals(2L, info.shallowOffsetOfMaxTimestamp);
+
+        Iterator<Record> iterator = records.records();
+        while (iterator.hasNext()) {
+            Record record = iterator.next();
+            assertEquals(TimestampType.LOG_APPEND_TIME, record.timestampType());
+            assertEquals(logAppendTime, record.timestamp());
+        }
+    }
+
+    @Test
+    public void buildUsingCreateTime() {
+        ByteBuffer buffer = ByteBuffer.allocate(1024);
+        buffer.position(bufferOffset);
+
+        long logAppendTime = System.currentTimeMillis();
+        MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.MAGIC_VALUE_V1, compressionType,
+                TimestampType.CREATE_TIME, 0L, logAppendTime, buffer.capacity());
+        builder.append(0L, 0L, "a".getBytes(), "1".getBytes());
+        builder.append(1L, 2L, "b".getBytes(), "2".getBytes());
+        builder.append(2L, 1L, "c".getBytes(), "3".getBytes());
+        MemoryRecords records = builder.build();
+
+        MemoryRecordsBuilder.RecordsInfo info = builder.info();
+        assertEquals(2L, info.maxTimestamp);
+
+        if (compressionType == CompressionType.NONE)
+            assertEquals(1L, info.shallowOffsetOfMaxTimestamp);
+        else
+            assertEquals(2L, info.shallowOffsetOfMaxTimestamp);
+
+        Iterator<Record> iterator = records.records();
+        int i = 0;
+        long[] expectedTimestamps = new long[] {0L, 2L, 1L};
+        while (iterator.hasNext()) {
+            Record record = iterator.next();
+            assertEquals(TimestampType.CREATE_TIME, record.timestampType());
+            assertEquals(expectedTimestamps[i++], record.timestamp());
+        }
+    }
+
+    @Test
+    public void writePastLimit() {
+        ByteBuffer buffer = ByteBuffer.allocate(64);
+        buffer.position(bufferOffset);
+
+        long logAppendTime = System.currentTimeMillis();
+        MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.MAGIC_VALUE_V1, compressionType,
+                TimestampType.CREATE_TIME, 0L, logAppendTime, buffer.capacity());
+        builder.append(0L, 0L, "a".getBytes(), "1".getBytes());
+        builder.append(1L, 1L, "b".getBytes(), "2".getBytes());
+
+        assertFalse(builder.hasRoomFor("c".getBytes(), "3".getBytes()));
+        builder.append(2L, 2L, "c".getBytes(), "3".getBytes());
+        MemoryRecords records = builder.build();
+
+        MemoryRecordsBuilder.RecordsInfo info = builder.info();
+        assertEquals(2L, info.maxTimestamp);
+        assertEquals(2L, info.shallowOffsetOfMaxTimestamp);
+
+        Iterator<Record> iterator = records.records();
+        long i = 0L;
+        while (iterator.hasNext()) {
+            Record record = iterator.next();
+            assertEquals(TimestampType.CREATE_TIME, record.timestampType());
+            assertEquals(i++, record.timestamp());
+        }
+    }
+
+    @Test
+    public void convertUsingCreateTime() {
+        ByteBuffer buffer = ByteBuffer.allocate(1024);
+        buffer.position(bufferOffset);
+
+        long logAppendTime = System.currentTimeMillis();
+        MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.MAGIC_VALUE_V1, compressionType,
+                TimestampType.CREATE_TIME, 0L, logAppendTime, buffer.capacity());
+
+        builder.convertAndAppend(0L, Record.create(Record.MAGIC_VALUE_V0, 0L, "a".getBytes(), "1".getBytes()));
+        builder.convertAndAppend(0L, Record.create(Record.MAGIC_VALUE_V0, 0L, "b".getBytes(), "2".getBytes()));
+        builder.convertAndAppend(0L, Record.create(Record.MAGIC_VALUE_V0, 0L, "c".getBytes(), "3".getBytes()));
+        MemoryRecords records = builder.build();
+
+        MemoryRecordsBuilder.RecordsInfo info = builder.info();
+        assertEquals(Record.NO_TIMESTAMP, info.maxTimestamp);
+        assertEquals(0L, info.shallowOffsetOfMaxTimestamp);
+
+        Iterator<Record> iterator = records.records();
+        while (iterator.hasNext()) {
+            Record record = iterator.next();
+            assertEquals(TimestampType.CREATE_TIME, record.timestampType());
+            assertEquals(Record.NO_TIMESTAMP, record.timestamp());
+        }
+    }
+
+    @Parameterized.Parameters
+    public static Collection<Object[]> data() {
+        List<Object[]> values = new ArrayList<>();
+        for (int bufferOffset : Arrays.asList(0, 15))
+            for (CompressionType compressionType : CompressionType.values())
+                values.add(new Object[] {bufferOffset, compressionType});
+        return values;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/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 b1117f1..ef0fbeb 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
@@ -16,53 +16,64 @@
  */
 package org.apache.kafka.common.record;
 
-import static org.apache.kafka.common.utils.Utils.toArray;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import org.apache.kafka.test.TestUtils;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import static java.util.Arrays.asList;
+import static org.apache.kafka.common.utils.Utils.toNullableArray;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 @RunWith(value = Parameterized.class)
 public class MemoryRecordsTest {
 
     private CompressionType compression;
+    private byte magic;
+    private long firstOffset;
 
-    public MemoryRecordsTest(CompressionType compression) {
+    public MemoryRecordsTest(byte magic, long firstOffset, CompressionType compression) {
+        this.magic = magic;
         this.compression = compression;
+        this.firstOffset = firstOffset;
     }
 
     @Test
     public void testIterator() {
-        MemoryRecords recs1 = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), compression);
-        MemoryRecords recs2 = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), compression);
-        List<Record> list = Arrays.asList(new Record(0L, "a".getBytes(), "1".getBytes()),
-                                          new Record(0L, "b".getBytes(), "2".getBytes()),
-                                          new Record(0L, "c".getBytes(), "3".getBytes()));
+        MemoryRecordsBuilder builder1 = MemoryRecords.builder(ByteBuffer.allocate(1024), magic, compression, TimestampType.CREATE_TIME, firstOffset);
+        MemoryRecordsBuilder builder2 = MemoryRecords.builder(ByteBuffer.allocate(1024), magic, compression, TimestampType.CREATE_TIME, firstOffset);
+        List<Record> list = asList(
+                Record.create(magic, 1L, "a".getBytes(), "1".getBytes()),
+                Record.create(magic, 2L, "b".getBytes(), "2".getBytes()),
+                Record.create(magic, 3L, "c".getBytes(), "3".getBytes()),
+                Record.create(magic, 4L, null, "4".getBytes()),
+                Record.create(magic, 5L, "e".getBytes(), null),
+                Record.create(magic, 6L, null, null));
+
         for (int i = 0; i < list.size(); i++) {
             Record r = list.get(i);
-            recs1.append(i, r);
-            recs2.append(i, 0L, toArray(r.key()), toArray(r.value()));
+            builder1.append(firstOffset + i, r);
+            builder2.append(firstOffset + i, i + 1, toNullableArray(r.key()), toNullableArray(r.value()));
         }
-        recs1.close();
-        recs2.close();
+
+        MemoryRecords recs1 = builder1.build();
+        MemoryRecords recs2 = builder2.build();
 
         for (int iteration = 0; iteration < 2; iteration++) {
-            for (MemoryRecords recs : Arrays.asList(recs1, recs2)) {
-                Iterator<LogEntry> iter = recs.iterator();
+            for (MemoryRecords recs : asList(recs1, recs2)) {
+                Iterator<LogEntry> iter = recs.deepIterator();
                 for (int i = 0; i < list.size(); i++) {
                     assertTrue(iter.hasNext());
                     LogEntry entry = iter.next();
-                    assertEquals((long) i, entry.offset());
+                    assertEquals(firstOffset + i, entry.offset());
                     assertEquals(list.get(i), entry.record());
                     entry.record().ensureValid();
                 }
@@ -73,20 +84,145 @@ public class MemoryRecordsTest {
 
     @Test
     public void testHasRoomForMethod() {
-        MemoryRecords recs1 = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), compression);
-        recs1.append(0, new Record(0L, "a".getBytes(), "1".getBytes()));
+        MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), magic, compression, TimestampType.CREATE_TIME);
+        builder.append(0, Record.create(magic, 0L, "a".getBytes(), "1".getBytes()));
+
+        assertTrue(builder.hasRoomFor("b".getBytes(), "2".getBytes()));
+        builder.close();
+        assertFalse(builder.hasRoomFor("b".getBytes(), "2".getBytes()));
+    }
+
+    @Test
+    public void testFilterTo() {
+        ByteBuffer buffer = ByteBuffer.allocate(2048);
+        MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME);
+        builder.append(0L, 10L, null, "a".getBytes());
+        builder.close();
+
+        builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, 1L);
+        builder.append(1L, 11L, "1".getBytes(), "b".getBytes());
+        builder.append(2L, 12L, null, "c".getBytes());
+        builder.close();
 
-        assertTrue(recs1.hasRoomFor("b".getBytes(), "2".getBytes()));
-        recs1.close();
-        assertFalse(recs1.hasRoomFor("b".getBytes(), "2".getBytes()));
+        builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, 3L);
+        builder.append(3L, 13L, null, "d".getBytes());
+        builder.append(4L, 20L, "4".getBytes(), "e".getBytes());
+        builder.append(5L, 15L, "5".getBytes(), "f".getBytes());
+        builder.close();
+
+        builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, 6L);
+        builder.append(6L, 16L, "6".getBytes(), "g".getBytes());
+        builder.close();
+
+        buffer.flip();
+
+        ByteBuffer filtered = ByteBuffer.allocate(2048);
+        MemoryRecords.FilterResult result = MemoryRecords.readableRecords(buffer).filterTo(new RetainNonNullKeysFilter(), filtered);
+
+        filtered.flip();
+
+        assertEquals(7, result.messagesRead);
+        assertEquals(4, result.messagesRetained);
+        assertEquals(buffer.limit(), result.bytesRead);
+        assertEquals(filtered.limit(), result.bytesRetained);
+        if (magic > 0) {
+            assertEquals(20L, result.maxTimestamp);
+            if (compression == CompressionType.NONE)
+                assertEquals(4L, result.shallowOffsetOfMaxTimestamp);
+            else
+                assertEquals(5L, result.shallowOffsetOfMaxTimestamp);
+        }
 
+        MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered);
+
+        List<ByteBufferLogInputStream.ByteBufferLogEntry> shallowEntries = TestUtils.toList(filteredRecords.shallowIterator());
+        List<Long> expectedOffsets = compression == CompressionType.NONE ? asList(1L, 4L, 5L, 6L) : asList(1L, 5L, 6L);
+        assertEquals(expectedOffsets.size(), shallowEntries.size());
+
+        for (int i = 0; i < expectedOffsets.size(); i++) {
+            LogEntry shallowEntry = shallowEntries.get(i);
+            assertEquals(expectedOffsets.get(i).longValue(), shallowEntry.offset());
+            assertEquals(magic, shallowEntry.record().magic());
+            assertEquals(compression, shallowEntry.record().compressionType());
+            assertEquals(magic == Record.MAGIC_VALUE_V0 ? TimestampType.NO_TIMESTAMP_TYPE : TimestampType.CREATE_TIME,
+                    shallowEntry.record().timestampType());
+        }
+
+        List<LogEntry> deepEntries = TestUtils.toList(filteredRecords.deepIterator());
+        assertEquals(4, deepEntries.size());
+
+        LogEntry first = deepEntries.get(0);
+        assertEquals(1L, first.offset());
+        assertEquals(Record.create(magic, 11L, "1".getBytes(), "b".getBytes()), first.record());
+
+        LogEntry second = deepEntries.get(1);
+        assertEquals(4L, second.offset());
+        assertEquals(Record.create(magic, 20L, "4".getBytes(), "e".getBytes()), second.record());
+
+        LogEntry third = deepEntries.get(2);
+        assertEquals(5L, third.offset());
+        assertEquals(Record.create(magic, 15L, "5".getBytes(), "f".getBytes()), third.record());
+
+        LogEntry fourth = deepEntries.get(3);
+        assertEquals(6L, fourth.offset());
+        assertEquals(Record.create(magic, 16L, "6".getBytes(), "g".getBytes()), fourth.record());
+    }
+
+    @Test
+    public void testFilterToPreservesLogAppendTime() {
+        long logAppendTime = System.currentTimeMillis();
+
+        ByteBuffer buffer = ByteBuffer.allocate(2048);
+        MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compression,
+                TimestampType.LOG_APPEND_TIME, 0L, logAppendTime);
+        builder.append(0L, 10L, null, "a".getBytes());
+        builder.close();
+
+        builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.LOG_APPEND_TIME, 1L, logAppendTime);
+        builder.append(1L, 11L, "1".getBytes(), "b".getBytes());
+        builder.append(2L, 12L, null, "c".getBytes());
+        builder.close();
+
+        builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.LOG_APPEND_TIME, 3L, logAppendTime);
+        builder.append(3L, 13L, null, "d".getBytes());
+        builder.append(4L, 14L, "4".getBytes(), "e".getBytes());
+        builder.append(5L, 15L, "5".getBytes(), "f".getBytes());
+        builder.close();
+
+        buffer.flip();
+
+        ByteBuffer filtered = ByteBuffer.allocate(2048);
+        MemoryRecords.readableRecords(buffer).filterTo(new RetainNonNullKeysFilter(), filtered);
+
+        filtered.flip();
+        MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered);
+
+        List<ByteBufferLogInputStream.ByteBufferLogEntry> shallowEntries = TestUtils.toList(filteredRecords.shallowIterator());
+        assertEquals(compression == CompressionType.NONE ? 3 : 2, shallowEntries.size());
+
+        for (LogEntry shallowEntry : shallowEntries) {
+            assertEquals(compression, shallowEntry.record().compressionType());
+            if (magic > Record.MAGIC_VALUE_V0) {
+                assertEquals(TimestampType.LOG_APPEND_TIME, shallowEntry.record().timestampType());
+                assertEquals(logAppendTime, shallowEntry.record().timestamp());
+            }
+        }
     }
 
     @Parameterized.Parameters
     public static Collection<Object[]> data() {
-        List<Object[]> values = new ArrayList<Object[]>();
-        for (CompressionType type: CompressionType.values())
-            values.add(new Object[] {type});
+        List<Object[]> values = new ArrayList<>();
+        for (long firstOffset : asList(0L, 57L))
+            for (byte magic : asList(Record.MAGIC_VALUE_V0, Record.MAGIC_VALUE_V1))
+                for (CompressionType type: CompressionType.values())
+                    values.add(new Object[] {magic, firstOffset, type});
         return values;
     }
+
+    private static class RetainNonNullKeysFilter implements MemoryRecords.LogEntryFilter {
+        @Override
+        public boolean shouldRetain(LogEntry entry) {
+            return entry.record().hasKey();
+        }
+    }
 }