You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by ju...@apache.org on 2016/02/19 16:56:48 UTC

[1/5] kafka git commit: KAFKA-3025; Added timetamp to Message and use relative offset.

Repository: kafka
Updated Branches:
  refs/heads/trunk eee95228f -> 45c8195fa


http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 511060e..77f5d65 100644
--- a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala
+++ b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala
@@ -19,9 +19,12 @@ package kafka.message
 
 import java.nio._
 import java.util.concurrent.atomic.AtomicLong
+
+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
-import kafka.utils.TestUtils
 
 class ByteBufferMessageSetTest extends BaseMessageSetTestCases {
 
@@ -135,27 +138,240 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases {
       verifyShallowIterator(mixedMessageSet)
     }
   }
-  
+
+  @Test
+  def testMessageWithProvidedOffsetSeq() {
+    val offsets = Seq(0L, 2L)
+    val messages = new ByteBufferMessageSet(
+      compressionCodec = NoCompressionCodec,
+      offsetSeq = offsets,
+      new Message("hello".getBytes),
+      new Message("goodbye".getBytes))
+    val iter = messages.iterator
+    assertEquals("first offset should be 0", 0L, iter.next().offset)
+    assertEquals("second offset should be 2", 2L, iter.next().offset)
+  }
+
+  @Test
+  def testLogAppendTime() {
+    val startTime = 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 = -1L, codec = DefaultCompressionCodec)
+
+    val validatedMessages = messages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(0),
+                                                                      sourceCodec = NoCompressionCodec,
+                                                                      targetCodec = NoCompressionCodec,
+                                                                      messageFormatVersion = 1,
+                                                                      messageTimestampType = TimestampType.LOG_APPEND_TIME,
+                                                                      messageTimestampDiffMaxMs = 1000L)
+
+    val validatedCompressedMessages =
+      compressedMessagesWithRecompresion.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(0),
+                                                                          sourceCodec = DefaultCompressionCodec,
+                                                                          targetCodec = DefaultCompressionCodec,
+                                                                          messageFormatVersion = 1,
+                                                                          messageTimestampType = TimestampType.LOG_APPEND_TIME,
+                                                                          messageTimestampDiffMaxMs = 1000L)
+
+    val validatedCompressedMessagesWithoutRecompression =
+      compressedMessagesWithoutRecompression.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(0),
+                                                                              sourceCodec = DefaultCompressionCodec,
+                                                                              targetCodec = DefaultCompressionCodec,
+                                                                              messageFormatVersion = 1,
+                                                                              messageTimestampType = TimestampType.LOG_APPEND_TIME,
+                                                                              messageTimestampDiffMaxMs = 1000L)
+
+    val now = System.currentTimeMillis()
+    assertEquals("message set size should not change", messages.size, validatedMessages.size)
+    validatedMessages.foreach({case messageAndOffset => validateLogAppendTime(messageAndOffset.message)})
+
+    assertEquals("message set size should not change", compressedMessagesWithRecompresion.size, validatedCompressedMessages.size)
+    validatedCompressedMessages.foreach({case messageAndOffset => validateLogAppendTime(messageAndOffset.message)})
+    assertTrue("MessageSet should still valid", validatedCompressedMessages.shallowIterator.next().message.isValid)
+
+    assertEquals("message set size should not change", compressedMessagesWithoutRecompression.size,
+      validatedCompressedMessagesWithoutRecompression.size)
+    validatedCompressedMessagesWithoutRecompression.foreach({case messageAndOffset =>
+      validateLogAppendTime(messageAndOffset.message)})
+    assertTrue("MessageSet should still valid", validatedCompressedMessagesWithoutRecompression.shallowIterator.next().message.isValid)
+
+    def validateLogAppendTime(message: Message) {
+      message.ensureValid()
+      assertTrue(s"Timestamp of message $message should be between $startTime and $now",
+        message.timestamp >= startTime && message.timestamp <= now)
+      assertEquals(TimestampType.LOG_APPEND_TIME, message.timestampType)
+    }
+  }
+
+  @Test
+  def testCreateTime() {
+    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)
+
+    val validatedMessages = messages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(0),
+                                                                      sourceCodec = NoCompressionCodec,
+                                                                      targetCodec = NoCompressionCodec,
+                                                                      messageFormatVersion = 1,
+                                                                      messageTimestampType = TimestampType.CREATE_TIME,
+                                                                      messageTimestampDiffMaxMs = 1000L)
+
+    val validatedCompressedMessages =
+      compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(0),
+                                                          sourceCodec = DefaultCompressionCodec,
+                                                          targetCodec = DefaultCompressionCodec,
+                                                          messageFormatVersion = 1,
+                                                          messageTimestampType = TimestampType.CREATE_TIME,
+                                                          messageTimestampDiffMaxMs = 1000L)
+
+    for (messageAndOffset <- validatedMessages) {
+      messageAndOffset.message.ensureValid()
+      assertEquals(messageAndOffset.message.timestamp, now)
+      assertEquals(messageAndOffset.message.timestampType, TimestampType.CREATE_TIME)
+    }
+    for (messageAndOffset <- validatedCompressedMessages) {
+      messageAndOffset.message.ensureValid()
+      assertEquals(messageAndOffset.message.timestamp, now)
+      assertEquals(messageAndOffset.message.timestampType, TimestampType.CREATE_TIME)
+    }
+  }
+
   @Test
-  def testOffsetAssignment() {
-    val messages = new ByteBufferMessageSet(NoCompressionCodec,
-                                            new Message("hello".getBytes), 
-                                            new Message("there".getBytes), 
-                                            new Message("beautiful".getBytes))
-    val compressedMessages = new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec,
-                                                      messages = messages.map(_.message).toBuffer:_*)
-    // check uncompressed offsets 
+  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 AtomicLong(0),
+                                                sourceCodec = NoCompressionCodec,
+                                                targetCodec = NoCompressionCodec,
+                                                messageFormatVersion = 1,
+                                                messageTimestampType = TimestampType.CREATE_TIME,
+                                                messageTimestampDiffMaxMs = 1000L)
+      fail("Should throw InvalidMessageException.")
+    } catch {
+      case e: InvalidTimestampException =>
+    }
+
+    try {
+      compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(0),
+                                                          sourceCodec = DefaultCompressionCodec,
+                                                          targetCodec = DefaultCompressionCodec,
+                                                          messageFormatVersion = 1,
+                                                          messageTimestampType = TimestampType.CREATE_TIME,
+                                                          messageTimestampDiffMaxMs = 1000L)
+      fail("Should throw InvalidMessageException.")
+    } catch {
+      case e: 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)
-    var offset = 1234567
-    checkOffsets(messages.validateMessagesAndAssignOffsets(new AtomicLong(offset), NoCompressionCodec, NoCompressionCodec), offset)
+    val offset = 1234567
+    checkOffsets(messages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(offset),
+                                                           sourceCodec = NoCompressionCodec,
+                                                           targetCodec = NoCompressionCodec,
+                                                           messageFormatVersion = 0,
+                                                           messageTimestampType = TimestampType.CREATE_TIME,
+                                                           messageTimestampDiffMaxMs = 1000L), offset)
 
     // check compressed messages
     checkOffsets(compressedMessages, 0)
-    checkOffsets(compressedMessages.validateMessagesAndAssignOffsets(new AtomicLong(offset), DefaultCompressionCodec, DefaultCompressionCodec), offset)
+    checkOffsets(compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(offset),
+                                                                     sourceCodec = DefaultCompressionCodec,
+                                                                     targetCodec = DefaultCompressionCodec,
+                                                                     messageFormatVersion = 0,
+                                                                     messageTimestampType = TimestampType.CREATE_TIME,
+                                                                     messageTimestampDiffMaxMs = 1000L), 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 AtomicLong(offset),
+                                                                      sourceCodec = NoCompressionCodec,
+                                                                      targetCodec = NoCompressionCodec,
+                                                                      messageTimestampType = TimestampType.CREATE_TIME,
+                                                                      messageTimestampDiffMaxMs = 5000L)
+    checkOffsets(messageWithOffset, offset)
+
+    // check compressed messages
+    checkOffsets(compressedMessages, 0)
+    val compressedMessagesWithOffset = compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(offset),
+                                                                                           sourceCodec = DefaultCompressionCodec,
+                                                                                           targetCodec = DefaultCompressionCodec,
+                                                                                           messageTimestampType = TimestampType.CREATE_TIME,
+                                                                                           messageTimestampDiffMaxMs = 5000L)
+    checkOffsets(compressedMessagesWithOffset, offset)
+  }
+
+  @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 AtomicLong(offset),
+                                                             sourceCodec = NoCompressionCodec,
+                                                             targetCodec = NoCompressionCodec,
+                                                             messageFormatVersion = 1,
+                                                             messageTimestampType = TimestampType.LOG_APPEND_TIME,
+                                                             messageTimestampDiffMaxMs = 1000L), offset)
+
+    // check compressed messages
+    checkOffsets(compressedMessagesV0, 0)
+    checkOffsets(compressedMessagesV0.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(offset),
+                                                                       sourceCodec = DefaultCompressionCodec,
+                                                                       targetCodec = DefaultCompressionCodec,
+                                                                       messageFormatVersion = 1,
+                                                                       messageTimestampType = TimestampType.LOG_APPEND_TIME,
+                                                                       messageTimestampDiffMaxMs = 1000L), 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 AtomicLong(offset),
+                                                             sourceCodec = NoCompressionCodec,
+                                                             targetCodec = NoCompressionCodec,
+                                                             messageFormatVersion = 0,
+                                                             messageTimestampType = TimestampType.CREATE_TIME,
+                                                             messageTimestampDiffMaxMs = 5000L), offset)
+
+    // check compressed messages
+    checkOffsets(compressedMessagesV1, 0)
+    checkOffsets(compressedMessagesV1.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(offset),
+                                                                       sourceCodec = DefaultCompressionCodec,
+                                                                       targetCodec = DefaultCompressionCodec,
+                                                                       messageFormatVersion = 0,
+                                                                       messageTimestampType = TimestampType.CREATE_TIME,
+                                                                       messageTimestampDiffMaxMs = 5000L), offset)
   }
   
   /* 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.size > 0)
     var offset = baseOffset
     for(entry <- messages) {
       assertEquals("Unexpected offset in message set iterator", offset, entry.offset)
@@ -169,4 +385,22 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases {
     val deepOffsets = messageSet.iterator.map(msgAndOff => msgAndOff.offset).toSet
     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))
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 d8613f7..53b85ef 100644
--- a/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala
+++ b/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala
@@ -42,15 +42,17 @@ class MessageCompressionTest extends JUnitSuite {
     val bytes1k: Array[Byte] = (0 until 1000).map(_.toByte).toArray
     val bytes2k: Array[Byte] = (1000 until 2000).map(_.toByte).toArray
     val bytes3k: Array[Byte] = (3000 until 4000).map(_.toByte).toArray
-    val messages: List[Message] = List(new Message(bytes1k), new Message(bytes2k), new Message(bytes3k))
+    val messages: List[Message] = List(new Message(bytes1k, Message.NoTimestamp, Message.MagicValue_V1),
+                                       new Message(bytes2k, Message.NoTimestamp, Message.MagicValue_V1),
+                                       new Message(bytes3k, Message.NoTimestamp, Message.MagicValue_V1))
 
-    testCompressSize(GZIPCompressionCodec, messages, 388)
+    testCompressSize(GZIPCompressionCodec, messages, 396)
 
     if(isSnappyAvailable)
-      testCompressSize(SnappyCompressionCodec, messages, 491)
+      testCompressSize(SnappyCompressionCodec, messages, 502)
 
     if(isLZ4Available)
-      testCompressSize(LZ4CompressionCodec, messages, 380)
+      testCompressSize(LZ4CompressionCodec, messages, 387)
   }
 
   def testSimpleCompressDecompress(compressionCodec: CompressionCodec) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 1755633..3c8a41f 100755
--- a/core/src/test/scala/unit/kafka/message/MessageTest.scala
+++ b/core/src/test/scala/unit/kafka/message/MessageTest.scala
@@ -26,12 +26,13 @@ import org.junit.Assert._
 import org.scalatest.junit.JUnitSuite
 import org.junit.{Before, Test}
 import kafka.utils.TestUtils
-import kafka.utils.CoreUtils
 import org.apache.kafka.common.utils.Utils
 
-case class MessageTestVal(val key: Array[Byte],
+case class MessageTestVal(val key: Array[Byte], 
                           val payload: Array[Byte],
                           val codec: CompressionCodec,
+                          val timestamp: Long,
+                          val magicValue: Byte,
                           val message: Message)
 
 class MessageTest extends JUnitSuite {
@@ -43,24 +44,41 @@ class MessageTest extends JUnitSuite {
     val keys = Array(null, "key".getBytes, "".getBytes)
     val vals = Array("value".getBytes, "".getBytes, null)
     val codecs = Array(NoCompressionCodec, GZIPCompressionCodec, SnappyCompressionCodec, LZ4CompressionCodec)
-    for(k <- keys; v <- vals; codec <- codecs)
-      messages += new MessageTestVal(k, v, codec, new Message(v, k, codec))
+    val timestamps = Array(Message.NoTimestamp, 0L, 1L)
+    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))
+    }
+
+    def ensureValid(magicValue: Byte, timestamp: Long): Long =
+      if (magicValue > Message.MagicValue_V0) timestamp else Message.NoTimestamp
   }
 
   @Test
   def testFieldValues {
     for(v <- messages) {
+      // check payload
       if(v.payload == null) {
         assertTrue(v.message.isNull)
         assertEquals("Payload should be null", null, v.message.payload)
       } else {
         TestUtils.checkEquals(ByteBuffer.wrap(v.payload), v.message.payload)
       }
-      assertEquals(Message.CurrentMagicValue, v.message.magic)
+      // check timestamp
+      if (v.magicValue > Message.MagicValue_V0)
+        assertEquals("Timestamp should be the same", v.timestamp, v.message.timestamp)
+       else
+        assertEquals("Timestamp should be the NoTimestamp", Message.NoTimestamp, v.message.timestamp)
+
+      // check magic value
+      assertEquals(v.magicValue, v.message.magic)
+      // check key
       if(v.message.hasKey)
         TestUtils.checkEquals(ByteBuffer.wrap(v.key), v.message.key)
       else
         assertEquals(null, v.message.key)
+      // check compression codec
       assertEquals(v.codec, v.message.compressionCodec)
     }
   }
@@ -82,12 +100,61 @@ class MessageTest extends JUnitSuite {
       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))
-      val copy = new Message(bytes = v.payload, key = v.key, codec = v.codec)
+      val copy = new Message(bytes = v.payload, key = v.key, v.timestamp, codec = v.codec, v.magicValue)
       assertTrue("Should equal another message with the same content.", v.message.equals(copy))
     }
   }
 
   @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)
+  }
+
+  @Test(expected = classOf[IllegalArgumentException])
+  def testInvalidTimestamp() {
+    new Message("hello".getBytes, -3L, Message.MagicValue_V1)
+  }
+
+  @Test(expected = classOf[IllegalArgumentException])
+  def testInvalidMagicByte() {
+    new Message("hello".getBytes, 0L, 2.toByte)
+  }
+
+  @Test
   def testIsHashable() {
     // this is silly, but why not
     val m = new HashMap[Message, Message]()

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala b/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala
index 3993fdb..6f0ee1d 100644
--- a/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala
+++ b/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala
@@ -20,6 +20,7 @@ package kafka.message
 import java.io.{InputStream, ByteArrayInputStream, ByteArrayOutputStream}
 import java.nio.ByteBuffer
 import java.util.Random
+import org.apache.kafka.common.record.TimestampType
 import org.junit.Assert._
 import org.junit.Test
 import org.scalatest.junit.JUnitSuite
@@ -34,7 +35,7 @@ class MessageWriterTest extends JUnitSuite {
 
   private def mkMessageWithWriter(key: Array[Byte] = null, bytes: Array[Byte], codec: CompressionCodec): Message = {
     val writer = new MessageWriter(100)
-    writer.write(key = key, codec = codec) { output =>
+    writer.write(key = key, codec = codec, timestamp = Message.NoTimestamp, timestampType = TimestampType.CREATE_TIME, magicValue = Message.MagicValue_V1) { output =>
       val out = if (codec == NoCompressionCodec) output else CompressionFactory(codec, output)
       try {
         val p = rnd.nextInt(bytes.length)
@@ -101,7 +102,7 @@ class MessageWriterTest extends JUnitSuite {
   def testWithNoCompressionAttribute(): Unit = {
     val bytes = mkRandomArray(4096)
     val actual = mkMessageWithWriter(bytes = bytes, codec = NoCompressionCodec)
-    val expected = new Message(bytes, NoCompressionCodec)
+    val expected = new Message(bytes, Message.NoTimestamp, NoCompressionCodec, Message.MagicValue_V1)
     assertEquals(expected.buffer, actual.buffer)
   }
 
@@ -109,7 +110,7 @@ class MessageWriterTest extends JUnitSuite {
   def testWithCompressionAttribute(): Unit = {
     val bytes = mkRandomArray(4096)
     val actual = mkMessageWithWriter(bytes = bytes, codec = SnappyCompressionCodec)
-    val expected = new Message(compress(bytes, SnappyCompressionCodec), SnappyCompressionCodec)
+    val expected = new Message(compress(bytes, SnappyCompressionCodec), Message.NoTimestamp, SnappyCompressionCodec, Message.MagicValue_V1)
 
     assertEquals(
       decompress(toArray(expected.payload), SnappyCompressionCodec).toSeq,
@@ -122,7 +123,7 @@ class MessageWriterTest extends JUnitSuite {
     val key = mkRandomArray(123)
     val bytes = mkRandomArray(4096)
     val actual = mkMessageWithWriter(bytes = bytes, key = key, codec = NoCompressionCodec)
-    val expected = new Message(bytes = bytes, key = key, codec = NoCompressionCodec)
+    val expected = new Message(bytes = bytes, key = key, timestamp = Message.NoTimestamp, codec = NoCompressionCodec, magicValue = Message.MagicValue_V1)
 
     assertEquals(expected.buffer, actual.buffer)
   }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala
index 60d2588..f711ca4 100755
--- a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala
+++ b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala
@@ -396,13 +396,18 @@ class AsyncProducerTest {
     EasyMock.expect(producerPool.getProducer(0)).andReturn(mockSyncProducer).times(4)
     EasyMock.expect(producerPool.close())
     EasyMock.replay(producerPool)
-
+    val time = new Time {
+      override def nanoseconds: Long = 0L
+      override def milliseconds: Long = 0L
+      override def sleep(ms: Long): Unit = {}
+    }
     val handler = new DefaultEventHandler[Int,String](config,
                                                       partitioner = new FixedValuePartitioner(),
                                                       encoder = new StringEncoder(),
                                                       keyEncoder = new NullEncoder[Int](),
                                                       producerPool = producerPool,
-                                                      topicPartitionInfos = topicPartitionInfos)
+                                                      topicPartitionInfos = topicPartitionInfos,
+                                                      time = time)
     val data = msgs.map(m => new KeyedMessage[Int,String](topic1, 0, m)) ++ msgs.map(m => new KeyedMessage[Int,String](topic1, 1, m))
     handler.handle(data)
     handler.close()
@@ -466,11 +471,13 @@ class AsyncProducerTest {
   }
 
   def messagesToSet(messages: Seq[String]): ByteBufferMessageSet = {
-    new ByteBufferMessageSet(NoCompressionCodec, messages.map(m => new Message(m.getBytes)): _*)
+    new ByteBufferMessageSet(NoCompressionCodec, messages.map(m => new Message(m.getBytes, 0L, Message.MagicValue_V1)): _*)
   }
 
   def messagesToSet(key: Array[Byte], messages: Seq[Array[Byte]]): ByteBufferMessageSet = {
-    new ByteBufferMessageSet(NoCompressionCodec, messages.map(m => new Message(key = key, bytes = m)): _*)
+    new ByteBufferMessageSet(
+      NoCompressionCodec,
+      messages.map(m => new Message(key = key, bytes = m, timestamp = 0L, magicValue = Message.MagicValue_V1)): _*)
   }
 }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/test/scala/unit/kafka/producer/ProducerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala
index f356a69..12b3583 100755
--- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala
+++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala
@@ -17,6 +17,7 @@
 
 package kafka.producer
 
+import java.nio.ByteBuffer
 import java.util
 import java.util.Properties
 
@@ -30,6 +31,7 @@ import kafka.server.{KafkaConfig, KafkaRequestHandler, KafkaServer}
 import kafka.utils._
 import kafka.zk.ZooKeeperTestHarness
 import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.record.TimestampType
 import org.apache.log4j.{Level, Logger}
 import org.junit.Assert._
 import org.junit.{After, Before, Test}
@@ -164,10 +166,11 @@ class ProducerTest extends ZooKeeperTestHarness with Logging{
       keyEncoder = classOf[StringEncoder].getName,
       partitioner = classOf[StaticPartitioner].getName,
       producerProps = props1)
-
+    val startTime = System.currentTimeMillis()
     // Available partition ids should be 0.
     producer1.send(new KeyedMessage[String, String](topic, "test", "test1"))
     producer1.send(new KeyedMessage[String, String](topic, "test", "test2"))
+    val endTime = System.currentTimeMillis()
     // get the leader
     val leaderOpt = zkUtils.getLeaderForPartition(topic, 0)
     assertTrue("Leader for topic new-topic partition 0 should exist", leaderOpt.isDefined)
@@ -181,8 +184,19 @@ class ProducerTest extends ZooKeeperTestHarness with Logging{
       response2.messageSet("new-topic", 0).iterator.toBuffer
     }
     assertEquals("Should have fetched 2 messages", 2, messageSet.size)
-    assertEquals(new Message(bytes = "test1".getBytes, key = "test".getBytes), messageSet(0).message)
-    assertEquals(new Message(bytes = "test2".getBytes, key = "test".getBytes), messageSet(1).message)
+    // Message 1
+    assertTrue(ByteBuffer.wrap("test1".getBytes).equals(messageSet(0).message.payload))
+    assertTrue(ByteBuffer.wrap("test".getBytes).equals(messageSet(0).message.key))
+    assertTrue(messageSet(0).message.timestamp >= startTime && messageSet(0).message.timestamp < endTime)
+    assertEquals(TimestampType.CREATE_TIME, messageSet(0).message.timestampType)
+    assertEquals(Message.MagicValue_V1, messageSet(0).message.magic)
+
+    // Message 2
+    assertTrue(ByteBuffer.wrap("test2".getBytes).equals(messageSet(1).message.payload))
+    assertTrue(ByteBuffer.wrap("test".getBytes).equals(messageSet(1).message.key))
+    assertTrue(messageSet(1).message.timestamp >= startTime && messageSet(1).message.timestamp < endTime)
+    assertEquals(TimestampType.CREATE_TIME, messageSet(1).message.timestampType)
+    assertEquals(Message.MagicValue_V1, messageSet(1).message.magic)
     producer1.close()
 
     val props2 = new util.Properties()
@@ -227,7 +241,7 @@ class ProducerTest extends ZooKeeperTestHarness with Logging{
       keyEncoder = classOf[StringEncoder].getName,
       partitioner = classOf[StaticPartitioner].getName,
       producerProps = props)
-
+    val startTime = System.currentTimeMillis()
     try {
       // Available partition ids should be 0, 1, 2 and 3, all lead and hosted only
       // on broker 0
@@ -235,7 +249,7 @@ class ProducerTest extends ZooKeeperTestHarness with Logging{
     } catch {
       case e: Throwable => fail("Unexpected exception: " + e)
     }
-
+    val endTime = System.currentTimeMillis()
     // kill the broker
     server1.shutdown
     server1.awaitShutdown()
@@ -260,7 +274,12 @@ class ProducerTest extends ZooKeeperTestHarness with Logging{
       val response1 = getConsumer1().fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build())
       val messageSet1 = response1.messageSet(topic, 0).iterator
       assertTrue("Message set should have 1 message", messageSet1.hasNext)
-      assertEquals(new Message(bytes = "test1".getBytes, key = "test".getBytes), messageSet1.next.message)
+      val message = messageSet1.next.message
+      assertTrue(ByteBuffer.wrap("test1".getBytes).equals(message.payload))
+      assertTrue(ByteBuffer.wrap("test".getBytes).equals(message.key))
+      assertTrue(message.timestamp >= startTime && message.timestamp < endTime)
+      assertEquals(TimestampType.CREATE_TIME, message.timestampType)
+      assertEquals(Message.MagicValue_V1, message.magic)
       assertFalse("Message set should have another message", messageSet1.hasNext)
     } catch {
       case e: Exception => fail("Not expected", e)

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala
index 6e7b964..c1034fe 100644
--- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala
+++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala
@@ -20,7 +20,6 @@ package kafka.producer
 import java.net.SocketTimeoutException
 import java.util.Properties
 
-import org.junit.Assert
 import kafka.admin.AdminUtils
 import kafka.api.ProducerResponseStatus
 import kafka.common.TopicAndPartition
@@ -30,6 +29,7 @@ import kafka.server.KafkaConfig
 import kafka.utils._
 import org.apache.kafka.common.protocol.{Errors, SecurityProtocol}
 import org.junit.Test
+import org.junit.Assert._
 
 class SyncProducerTest extends KafkaServerTestHarness {
   private val messageBytes =  new Array[Byte](2)
@@ -48,28 +48,28 @@ class SyncProducerTest extends KafkaServerTestHarness {
     try {
       val response = producer.send(TestUtils.produceRequest("test", 0,
         new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = new Message(messageBytes)), acks = 1))
-      Assert.assertNotNull(response)
+      assertNotNull(response)
     } catch {
-      case e: Exception => Assert.fail("Unexpected failure sending message to broker. " + e.getMessage)
+      case e: Exception => fail("Unexpected failure sending message to broker. " + e.getMessage)
     }
     val firstEnd = SystemTime.milliseconds
-    Assert.assertTrue((firstEnd-firstStart) < 500)
+    assertTrue((firstEnd-firstStart) < 500)
     val secondStart = SystemTime.milliseconds
     try {
       val response = producer.send(TestUtils.produceRequest("test", 0,
         new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = new Message(messageBytes)), acks = 1))
-      Assert.assertNotNull(response)
+      assertNotNull(response)
     } catch {
-      case e: Exception => Assert.fail("Unexpected failure sending message to broker. " + e.getMessage)
+      case e: Exception => fail("Unexpected failure sending message to broker. " + e.getMessage)
     }
     val secondEnd = SystemTime.milliseconds
-    Assert.assertTrue((secondEnd-secondStart) < 500)
+    assertTrue((secondEnd-secondStart) < 500)
     try {
       val response = producer.send(TestUtils.produceRequest("test", 0,
         new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = new Message(messageBytes)), acks = 1))
-      Assert.assertNotNull(response)
+      assertNotNull(response)
     } catch {
-      case e: Exception => Assert.fail("Unexpected failure sending message to broker. " + e.getMessage)
+      case e: Exception => fail("Unexpected failure sending message to broker. " + e.getMessage)
     }
   }
 
@@ -87,8 +87,8 @@ class SyncProducerTest extends KafkaServerTestHarness {
 
     val producer = new SyncProducer(new SyncProducerConfig(props))
     val response = producer.send(emptyRequest)
-    Assert.assertTrue(response != null)
-    Assert.assertTrue(!response.hasError && response.status.size == 0)
+    assertTrue(response != null)
+    assertTrue(!response.hasError && response.status.size == 0)
   }
 
   @Test
@@ -103,18 +103,18 @@ class SyncProducerTest extends KafkaServerTestHarness {
     val messageSet1 = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = message1)
     val response1 = producer.send(TestUtils.produceRequest("test", 0, messageSet1, acks = 1))
 
-    Assert.assertEquals(1, response1.status.count(_._2.error != Errors.NONE.code))
-    Assert.assertEquals(Errors.MESSAGE_TOO_LARGE.code, response1.status(TopicAndPartition("test", 0)).error)
-    Assert.assertEquals(-1L, response1.status(TopicAndPartition("test", 0)).offset)
+    assertEquals(1, response1.status.count(_._2.error != Errors.NONE.code))
+    assertEquals(Errors.MESSAGE_TOO_LARGE.code, response1.status(TopicAndPartition("test", 0)).error)
+    assertEquals(-1L, response1.status(TopicAndPartition("test", 0)).offset)
 
-    val safeSize = configs(0).messageMaxBytes - Message.MessageOverhead - MessageSet.LogOverhead - 1
+    val safeSize = configs(0).messageMaxBytes - Message.MinMessageOverhead - Message.TimestampLength - MessageSet.LogOverhead - 1
     val message2 = new Message(new Array[Byte](safeSize))
     val messageSet2 = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = message2)
     val response2 = producer.send(TestUtils.produceRequest("test", 0, messageSet2, acks = 1))
 
-    Assert.assertEquals(1, response1.status.count(_._2.error != Errors.NONE.code))
-    Assert.assertEquals(Errors.NONE.code, response2.status(TopicAndPartition("test", 0)).error)
-    Assert.assertEquals(0, response2.status(TopicAndPartition("test", 0)).offset)
+    assertEquals(1, response1.status.count(_._2.error != Errors.NONE.code))
+    assertEquals(Errors.NONE.code, response2.status(TopicAndPartition("test", 0)).error)
+    assertEquals(0, response2.status(TopicAndPartition("test", 0)).offset)
   }
 
 
@@ -157,13 +157,14 @@ class SyncProducerTest extends KafkaServerTestHarness {
     val request = TestUtils.produceRequestWithAcks(Array("topic1", "topic2", "topic3"), Array(0), messages, 1)
     val response = producer.send(request)
 
-    Assert.assertNotNull(response)
-    Assert.assertEquals(request.correlationId, response.correlationId)
-    Assert.assertEquals(3, response.status.size)
+    assertNotNull(response)
+    assertEquals(request.correlationId, response.correlationId)
+    assertEquals(3, response.status.size)
     response.status.values.foreach {
-      case ProducerResponseStatus(error, nextOffset) =>
-        Assert.assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION.code, error)
-        Assert.assertEquals(-1L, nextOffset)
+      case ProducerResponseStatus(error, nextOffset, timestamp) =>
+        assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION.code, error)
+        assertEquals(-1L, nextOffset)
+        assertEquals(Message.NoTimestamp, timestamp)
     }
 
     // #2 - test that we get correct offsets when partition is owned by broker
@@ -173,20 +174,20 @@ class SyncProducerTest extends KafkaServerTestHarness {
     TestUtils.waitUntilLeaderIsElectedOrChanged(zkUtils, "topic3", 0)
 
     val response2 = producer.send(request)
-    Assert.assertNotNull(response2)
-    Assert.assertEquals(request.correlationId, response2.correlationId)
-    Assert.assertEquals(3, response2.status.size)
+    assertNotNull(response2)
+    assertEquals(request.correlationId, response2.correlationId)
+    assertEquals(3, response2.status.size)
 
     // the first and last message should have been accepted by broker
-    Assert.assertEquals(Errors.NONE.code, response2.status(TopicAndPartition("topic1", 0)).error)
-    Assert.assertEquals(Errors.NONE.code, response2.status(TopicAndPartition("topic3", 0)).error)
-    Assert.assertEquals(0, response2.status(TopicAndPartition("topic1", 0)).offset)
-    Assert.assertEquals(0, response2.status(TopicAndPartition("topic3", 0)).offset)
+    assertEquals(Errors.NONE.code, response2.status(TopicAndPartition("topic1", 0)).error)
+    assertEquals(Errors.NONE.code, response2.status(TopicAndPartition("topic3", 0)).error)
+    assertEquals(0, response2.status(TopicAndPartition("topic1", 0)).offset)
+    assertEquals(0, response2.status(TopicAndPartition("topic3", 0)).offset)
 
     // the middle message should have been rejected because broker doesn't lead partition
-    Assert.assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION.code,
+    assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION.code,
                         response2.status(TopicAndPartition("topic2", 0)).error)
-    Assert.assertEquals(-1, response2.status(TopicAndPartition("topic2", 0)).offset)
+    assertEquals(-1, response2.status(TopicAndPartition("topic2", 0)).offset)
   }
 
   @Test
@@ -207,14 +208,14 @@ class SyncProducerTest extends KafkaServerTestHarness {
     val t1 = SystemTime.milliseconds
     try {
       producer.send(request)
-      Assert.fail("Should have received timeout exception since request handling is stopped.")
+      fail("Should have received timeout exception since request handling is stopped.")
     } catch {
       case e: SocketTimeoutException => /* success */
-      case e: Throwable => Assert.fail("Unexpected exception when expecting timeout: " + e)
+      case e: Throwable => fail("Unexpected exception when expecting timeout: " + e)
     }
     val t2 = SystemTime.milliseconds
     // make sure we don't wait fewer than timeoutMs for a response
-    Assert.assertTrue((t2-t1) >= timeoutMs)
+    assertTrue((t2-t1) >= timeoutMs)
   }
 
   @Test
@@ -230,7 +231,7 @@ class SyncProducerTest extends KafkaServerTestHarness {
     val emptyRequest = new kafka.api.ProducerRequest(correlationId, clientId, ack, ackTimeoutMs, collection.mutable.Map[TopicAndPartition, ByteBufferMessageSet]())
     val producer = new SyncProducer(new SyncProducerConfig(props))
     val response = producer.send(emptyRequest)
-    Assert.assertTrue(response == null)
+    assertTrue(response == null)
   }
 
   @Test
@@ -250,6 +251,6 @@ class SyncProducerTest extends KafkaServerTestHarness {
     val response = producer.send(TestUtils.produceRequest(topicName, 0,
       new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = new Message(messageBytes)),-1))
 
-    Assert.assertEquals(Errors.NOT_ENOUGH_REPLICAS.code, response.status(TopicAndPartition(topicName, 0)).error)
+    assertEquals(Errors.NOT_ENOUGH_REPLICAS.code, response.status(TopicAndPartition(topicName, 0)).error)
   }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala
index 155eea0..2ccb7b8 100755
--- a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala
@@ -114,7 +114,7 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness {
     TestUtils.createTopic(zkUtils, topic, numPartitions = 1, replicationFactor = 1, servers = servers)
 
     val serializedBytes = {
-      val headerBytes = requestHeaderBytes(ApiKeys.PRODUCE.id, 1, null, correlationId)
+      val headerBytes = requestHeaderBytes(ApiKeys.PRODUCE.id, 2, null, correlationId)
       val messageBytes = "message".getBytes
       val request = new ProduceRequest(1, 10000, Map(topicPartition -> ByteBuffer.wrap(messageBytes)).asJava)
       val byteBuffer = ByteBuffer.allocate(headerBytes.length + request.sizeOf)

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
index 2479b37..7fe9ffc 100755
--- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
@@ -20,7 +20,7 @@ package kafka.server
 import java.util.Properties
 
 import junit.framework.Assert._
-import kafka.api.{ApiVersion, KAFKA_082}
+import kafka.api.{ApiVersion, KAFKA_0_8_2}
 import kafka.message._
 import kafka.utils.{CoreUtils, TestUtils}
 import org.apache.kafka.common.config.ConfigException
@@ -281,14 +281,18 @@ class KafkaConfigTest {
     val conf = KafkaConfig.fromProps(props)
     assertEquals(ApiVersion.latestVersion, conf.interBrokerProtocolVersion)
 
-    props.put(KafkaConfig.InterBrokerProtocolVersionProp,"0.8.2.0")
+    props.put(KafkaConfig.InterBrokerProtocolVersionProp, "0.8.2.0")
+    // We need to set the message format version to make the configuration valid.
+    props.put(KafkaConfig.MessageFormatVersionProp, "0.8.2.0")
     val conf2 = KafkaConfig.fromProps(props)
-    assertEquals(KAFKA_082, conf2.interBrokerProtocolVersion)
+    assertEquals(KAFKA_0_8_2, conf2.interBrokerProtocolVersion)
 
     // check that 0.8.2.0 is the same as 0.8.2.1
-    props.put(KafkaConfig.InterBrokerProtocolVersionProp,"0.8.2.1")
+    props.put(KafkaConfig.InterBrokerProtocolVersionProp, "0.8.2.1")
+    // We need to set the message format version to make the configuration valid
+    props.put(KafkaConfig.MessageFormatVersionProp, "0.8.2.1")
     val conf3 = KafkaConfig.fromProps(props)
-    assertEquals(KAFKA_082, conf3.interBrokerProtocolVersion)
+    assertEquals(KAFKA_0_8_2, conf3.interBrokerProtocolVersion)
 
     //check that latest is newer than 0.8.2
     assert(ApiVersion.latestVersion.onOrAfter(conf3.interBrokerProtocolVersion))
@@ -460,7 +464,7 @@ class KafkaConfigTest {
         case KafkaConfig.NumPartitionsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
         case KafkaConfig.LogDirsProp => // ignore string
         case KafkaConfig.LogDirProp => // ignore string
-        case KafkaConfig.LogSegmentBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", Message.MinHeaderSize - 1)
+        case KafkaConfig.LogSegmentBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", Message.MinMessageOverhead - 1)
 
         case KafkaConfig.LogRollTimeMillisProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
         case KafkaConfig.LogRollTimeHoursProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 b6bc4fc..5c2092c 100755
--- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
+++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
@@ -27,7 +27,7 @@ import kafka.consumer.SimpleConsumer
 import kafka.message.{NoCompressionCodec, ByteBufferMessageSet, Message}
 import kafka.zk.ZooKeeperTestHarness
 import kafka.admin.AdminUtils
-import kafka.api.{PartitionOffsetRequestInfo, FetchRequestBuilder, OffsetRequest}
+import kafka.api.{ApiVersion, PartitionOffsetRequestInfo, FetchRequestBuilder, OffsetRequest}
 import kafka.utils.TestUtils._
 import kafka.common.TopicAndPartition
 import org.junit.After
@@ -91,17 +91,17 @@ class LogOffsetTest extends ZooKeeperTestHarness {
       log.append(new ByteBufferMessageSet(NoCompressionCodec, message))
     log.flush()
 
-    val offsets = server.apis.fetchOffsets(logManager, new TopicPartition(topic, part), OffsetRequest.LatestTime, 10)
-    assertEquals(Seq(20L, 18L, 15L, 12L, 9L, 6L, 3L, 0), offsets)
+    val offsets = server.apis.fetchOffsets(logManager, new TopicPartition(topic, part), OffsetRequest.LatestTime, 15)
+    assertEquals(Seq(20L, 18L, 16L, 14L, 12L, 10L, 8L, 6L, 4L, 2L, 0L), offsets)
 
     waitUntilTrue(() => isLeaderLocalOnBroker(topic, part, server), "Leader should be elected")
     val topicAndPartition = TopicAndPartition(topic, part)
     val offsetRequest = OffsetRequest(
-      Map(topicAndPartition -> PartitionOffsetRequestInfo(OffsetRequest.LatestTime, 10)),
+      Map(topicAndPartition -> PartitionOffsetRequestInfo(OffsetRequest.LatestTime, 15)),
       replicaId = 0)
     val consumerOffsets =
       simpleConsumer.getOffsetsBefore(offsetRequest).partitionErrorAndOffsets(topicAndPartition).offsets
-    assertEquals(Seq(20L, 18L, 15L, 12L, 9L, 6L, 3L, 0), consumerOffsets)
+    assertEquals(Seq(20L, 18L, 16L, 14L, 12L, 10L, 8L, 6L, 4L, 2L, 0L), consumerOffsets)
 
     // try to fetch using latest offset
     val fetchResponse = simpleConsumer.fetch(
@@ -154,15 +154,15 @@ class LogOffsetTest extends ZooKeeperTestHarness {
 
     val now = time.milliseconds + 30000 // pretend it is the future to avoid race conditions with the fs
 
-    val offsets = server.apis.fetchOffsets(logManager, new TopicPartition(topic, part), now, 10)
-    assertEquals(Seq(20L, 18L, 15L, 12L, 9L, 6L, 3L, 0L), offsets)
+    val offsets = server.apis.fetchOffsets(logManager, new TopicPartition(topic, part), now, 15)
+    assertEquals(Seq(20L, 18L, 16L, 14L, 12L, 10L, 8L, 6L, 4L, 2L, 0L), offsets)
 
     waitUntilTrue(() => isLeaderLocalOnBroker(topic, part, server), "Leader should be elected")
     val topicAndPartition = TopicAndPartition(topic, part)
-    val offsetRequest = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(now, 10)), replicaId = 0)
+    val offsetRequest = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(now, 15)), replicaId = 0)
     val consumerOffsets =
       simpleConsumer.getOffsetsBefore(offsetRequest).partitionErrorAndOffsets(topicAndPartition).offsets
-    assertEquals(Seq(20L, 18L, 15L, 12L, 9L, 6L, 3L, 0L), consumerOffsets)
+    assertEquals(Seq(20L, 18L, 16L, 14L, 12L, 10L, 8L, 6L, 4L, 2L, 0L), consumerOffsets)
   }
 
   @Test
@@ -206,6 +206,7 @@ class LogOffsetTest extends ZooKeeperTestHarness {
     props.put("log.retention.check.interval.ms", (5*1000*60).toString)
     props.put("log.segment.bytes", logSize.toString)
     props.put("zookeeper.connect", zkConnect.toString)
+    props.put("message.format.version", "0.10.0")
     props
   }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala b/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala
index 66052fc..de92a24 100644
--- a/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala
+++ b/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala
@@ -35,11 +35,11 @@ class ConsoleConsumerTest extends JUnitSuite {
     val formatter = EasyMock.createNiceMock(classOf[MessageFormatter])
 
     //Stubs
-    val record = new BaseConsumerRecord("foo", 1, 1, Array[Byte](), Array[Byte]())
+    val record = new BaseConsumerRecord(topic = "foo", partition = 1, offset = 1, key = Array[Byte](), value = Array[Byte]())
 
     //Expectations
     val messageLimit: Int = 10
-    EasyMock.expect(formatter.writeTo(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject())).times(messageLimit)
+    EasyMock.expect(formatter.writeTo(EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject())).times(messageLimit)
     EasyMock.expect(consumer.receive()).andReturn(record).times(messageLimit)
 
     EasyMock.replay(consumer)

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 29e1082..2523083 100755
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -282,8 +282,11 @@ object TestUtils extends Logging {
    * Wrap the message in a message set
    * @param payload The bytes of the message
    */
-  def singleMessageSet(payload: Array[Byte], codec: CompressionCodec = NoCompressionCodec, key: Array[Byte] = null) =
-    new ByteBufferMessageSet(compressionCodec = codec, messages = new Message(payload, key))
+  def singleMessageSet(payload: Array[Byte],
+                       codec: CompressionCodec = NoCompressionCodec,
+                       key: Array[Byte] = null,
+                       magicValue: Byte = Message.CurrentMagicValue) =
+    new ByteBufferMessageSet(compressionCodec = codec, messages = new Message(payload, key, Message.NoTimestamp, magicValue))
 
   /**
    * Generate an array of random bytes

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/docs/upgrade.html
----------------------------------------------------------------------
diff --git a/docs/upgrade.html b/docs/upgrade.html
index ba59cc0..f6d67eb 100644
--- a/docs/upgrade.html
+++ b/docs/upgrade.html
@@ -17,6 +17,58 @@
 
 <h3><a id="upgrade" href="#upgrade">1.5 Upgrading From Previous Versions</a></h3>
 
+<h4><a id="upgrade_10" href="#upgrade_10">Upgrading from 0.8.x or 0.9.x to 0.10.0.0</a></h4>
+0.10.0.0 has <a href="#upgrade_10_performance_impact">potential performance impact during upgrade</a> and
+<a href="#upgrade_10_breaking">potential breaking changes</a> (please review before upgrading). Because new protocols
+are introduced, it is important to upgrade your Kafka clusters before upgrading your clients.
+
+<p><b>For a rolling upgrade:</b></p>
+
+<ol>
+    <li> Update server.properties file on all brokers and add the following property: inter.broker.protocol.version=CURRENT_KAFKA_VERSION(e.g. 0.8.2, 0.9.0.0).
+         We recommend the users to set message.format.version=CURRENT_KAFKA_VERSION as well to avoid performance regression
+         during upgrade. See <a href="#upgrade_10_performance_impact">potential performance impact during upgrade</a> for the details.
+    </li>
+    <li> Upgrade the brokers. This can be done a broker at a time by simply bringing it down, updating the code, and restarting it. </li>
+    <li> Once the entire cluster is upgraded, bump the protocol version by editing inter.broker.protocol.version and setting it to 0.10.0.0. </li>
+    <li> Restart the brokers one by one for the new protocol version to take effect. </li>
+</ol>
+
+<p><b>Note:</b> If you are willing to accept downtime, you can simply take all the brokers down, update the code and start all of them. They will start with the new protocol by default.
+
+<p><b>Note:</b> Bumping the protocol version and restarting can be done any time after the brokers were upgraded. It does not have to be immediately after.
+
+<h5><a id="upgrade_10_performance_impact" href="#upgrade_10_performance_impact">potential performance impact in 0.10.0.0 during upgrade</a></h5>
+<p>
+    Message format in 0.10.0 now includes a new timestamp field and uses relative offsets for compressed messages.
+    The on disk message format can be configured through message.format.version in server.properties file.
+    The default on-disk message format is 0.10.0. If a consumer client is on a version before 0.10.0.0, it only understands
+    message format before 0.10.0. In this case, the broker is able to convert messages of the format in 0.10.0 to earlier format
+    before sending a response to the consumer on an older version. However, the broker can't use zero-copy transfer in this case.
+
+    To avoid such message conversion before consumers are upgraded to 0.10.0.0, one can set the message format to 0.9.0 when
+    upgrading the broker to 0.10.0.0. This way, the broker can still use zero-copy transfer to send the data to the old
+    consumers. Once most consumers are upgraded, one can change the message format to 0.10.0 on the broker.
+</p>
+<p>
+    For clients that are upgraded to 0.10.0.0, there is no performance impact.
+</p>
+<p>
+    <b>Note:</b> By setting the message format version, one certifies all the existing messages are on or below that
+    message format version. Otherwise consumers before 0.10.0.0 might break. In particular, after the message format
+    is set to 0.10.0, one should not change it back to earlier format since it may break the consumer on versions before 0.10.0.0.
+</p>
+
+<h5><a id="upgrade_10_breaking" href="#upgrade_10_breaking">potential breaking changes in 0.10.0.0</a></h5>
+<ul>
+    <li> Starting from Kafka 0.10.0.0, message format version in Kafka is represented as the Kafka version. For example, message format 0.9.0 refers to the highest message version supported by Kafka 0.9.0. </li>
+    <li> Message format 0.10.0 is added and used by default to include a timestamp field in the messages and use relative offsets for compressed messages. </li>
+    <li> ProduceRequest/Response v2 is added and used by default to support message format 0.10.0 </li>
+    <li> FetchRequest/Response v2 is added and used by default to support message format 0.10.0 </li>
+    <li> MessageFormatter interface changed from <code>void writeTo(byte[] key, byte[] value, PrintStream output)</code> to
+         <code>void writeTo(byte[] key, byte[] value, long timestamp, TimestampType timestampType, PrintStream output)</code> </li>
+</ul>
+
 <h4><a id="upgrade_9" href="#upgrade_9">Upgrading from 0.8.0, 0.8.1.X or 0.8.2.X to 0.9.0.0</a></h4>
 
 0.9.0.0 has <a href="#upgrade_9_breaking">potential breaking changes</a> (please review before upgrading) and an inter-broker protocol change from previous versions. This means that upgraded brokers and clients may not be compatible with older versions. It is important that you upgrade your Kafka cluster before upgrading your clients. If you are using MirrorMaker downstream clusters should be upgraded first as well.

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java
----------------------------------------------------------------------
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java
index 66f78d2..0787204 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java
@@ -19,6 +19,7 @@ package org.apache.kafka.streams.processor.internals;
 
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.streams.processor.TimestampExtractor;
 
 import java.util.ArrayDeque;
@@ -76,7 +77,8 @@ public class RecordQueue {
             Object key = source.deserializeKey(rawRecord.topic(), rawRecord.key());
             Object value = source.deserializeValue(rawRecord.topic(), rawRecord.value());
 
-            ConsumerRecord<Object, Object> record = new ConsumerRecord<>(rawRecord.topic(), rawRecord.partition(), rawRecord.offset(), key, value);
+            ConsumerRecord<Object, Object> record = new ConsumerRecord<>(rawRecord.topic(), rawRecord.partition(),
+                rawRecord.offset(), rawRecord.timestamp(), TimestampType.CREATE_TIME, key, value);
             long timestamp = timestampExtractor.extract(record);
 
             StampedRecord stampedRecord = new StampedRecord(record, timestamp);

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java
index b91acdc..9d0c0e2 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
 
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.kafka.common.serialization.IntegerDeserializer;
 import org.apache.kafka.common.serialization.IntegerSerializer;
@@ -59,17 +60,17 @@ public class PartitionGroupTest {
 
         // add three 3 records with timestamp 1, 3, 5 to partition-1
         List<ConsumerRecord<byte[], byte[]>> list1 = Arrays.asList(
-            new ConsumerRecord<>("topic", 1, 1, recordKey, recordValue),
-            new ConsumerRecord<>("topic", 1, 3, recordKey, recordValue),
-            new ConsumerRecord<>("topic", 1, 5, recordKey, recordValue));
+            new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, recordKey, recordValue),
+            new ConsumerRecord<>("topic", 1, 3, 0L, TimestampType.CREATE_TIME, recordKey, recordValue),
+            new ConsumerRecord<>("topic", 1, 5, 0L, TimestampType.CREATE_TIME, recordKey, recordValue));
 
         group.addRawRecords(partition1, list1);
 
         // add three 3 records with timestamp 2, 4, 6 to partition-2
         List<ConsumerRecord<byte[], byte[]>> list2 = Arrays.asList(
-            new ConsumerRecord<>("topic", 1, 2, recordKey, recordValue),
-            new ConsumerRecord<>("topic", 1, 4, recordKey, recordValue),
-            new ConsumerRecord<>("topic", 1, 6, recordKey, recordValue));
+            new ConsumerRecord<>("topic", 1, 2, 0L, TimestampType.CREATE_TIME, recordKey, recordValue),
+            new ConsumerRecord<>("topic", 1, 4, 0L, TimestampType.CREATE_TIME, recordKey, recordValue),
+            new ConsumerRecord<>("topic", 1, 6, 0L, TimestampType.CREATE_TIME, recordKey, recordValue));
 
         group.addRawRecords(partition2, list2);
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java
index bc6f71b..916079d 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java
@@ -25,6 +25,7 @@ import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.Node;
 import org.apache.kafka.common.PartitionInfo;
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.serialization.IntegerSerializer;
 import org.apache.kafka.common.serialization.Serializer;
 import org.apache.kafka.common.utils.Utils;
@@ -83,7 +84,8 @@ public class ProcessorStateManagerTest {
         // buffer a record (we cannot use addRecord because we need to add records before assigning a partition)
         public void bufferRecord(ConsumerRecord<Integer, Integer> record) {
             recordBuffer.add(
-                new ConsumerRecord<>(record.topic(), record.partition(), record.offset(),
+                new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), 0L,
+                    TimestampType.CREATE_TIME,
                     serializer.serialize(record.topic(), record.key()),
                     serializer.serialize(record.topic(), record.value())));
             endOffset = record.offset();
@@ -267,7 +269,7 @@ public class ProcessorStateManagerTest {
                     int key = i * 10;
                     expectedKeys.add(key);
                     restoreConsumer.bufferRecord(
-                            new ConsumerRecord<>(persistentStoreTopicName, 2, offset, key, 0)
+                            new ConsumerRecord<>(persistentStoreTopicName, 2, 0L, offset, TimestampType.CREATE_TIME, key, 0)
                     );
                 }
 
@@ -320,7 +322,7 @@ public class ProcessorStateManagerTest {
                     int key = i;
                     expectedKeys.add(i);
                     restoreConsumer.bufferRecord(
-                            new ConsumerRecord<>(nonPersistentStoreTopicName, 2, offset, key, 0)
+                            new ConsumerRecord<>(nonPersistentStoreTopicName, 2, 0L, offset, TimestampType.CREATE_TIME, key, 0)
                     );
                 }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java
index 6e86410..614e2c7 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue;
 
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.kafka.common.serialization.IntegerDeserializer;
 import org.apache.kafka.common.serialization.IntegerSerializer;
@@ -50,9 +51,9 @@ public class RecordQueueTest {
 
         // add three 3 out-of-order records with timestamp 2, 1, 3
         List<ConsumerRecord<byte[], byte[]>> list1 = Arrays.asList(
-            new ConsumerRecord<>("topic", 1, 2, recordKey, recordValue),
-            new ConsumerRecord<>("topic", 1, 1, recordKey, recordValue),
-            new ConsumerRecord<>("topic", 1, 3, recordKey, recordValue));
+            new ConsumerRecord<>("topic", 1, 2, 0L, TimestampType.CREATE_TIME, recordKey, recordValue),
+            new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, recordKey, recordValue),
+            new ConsumerRecord<>("topic", 1, 3, 0L, TimestampType.CREATE_TIME, recordKey, recordValue));
 
         queue.addRawRecords(list1, timestampExtractor);
 
@@ -72,9 +73,9 @@ public class RecordQueueTest {
         // add three 3 out-of-order records with timestamp 4, 1, 2
         // now with 3, 4, 1, 2
         List<ConsumerRecord<byte[], byte[]>> list2 = Arrays.asList(
-            new ConsumerRecord<>("topic", 1, 4, recordKey, recordValue),
-            new ConsumerRecord<>("topic", 1, 1, recordKey, recordValue),
-            new ConsumerRecord<>("topic", 1, 2, recordKey, recordValue));
+            new ConsumerRecord<>("topic", 1, 4, 0L, TimestampType.CREATE_TIME, recordKey, recordValue),
+            new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, recordKey, recordValue),
+            new ConsumerRecord<>("topic", 1, 2, 0L, TimestampType.CREATE_TIME, recordKey, recordValue));
 
         queue.addRawRecords(list2, timestampExtractor);
 
@@ -99,9 +100,9 @@ public class RecordQueueTest {
 
         // add three more records with 4, 5, 6
         List<ConsumerRecord<byte[], byte[]>> list3 = Arrays.asList(
-            new ConsumerRecord<>("topic", 1, 4, recordKey, recordValue),
-            new ConsumerRecord<>("topic", 1, 5, recordKey, recordValue),
-            new ConsumerRecord<>("topic", 1, 6, recordKey, recordValue));
+            new ConsumerRecord<>("topic", 1, 4, 0L, TimestampType.CREATE_TIME, recordKey, recordValue),
+            new ConsumerRecord<>("topic", 1, 5, 0L, TimestampType.CREATE_TIME, recordKey, recordValue),
+            new ConsumerRecord<>("topic", 1, 6, 0L, TimestampType.CREATE_TIME, recordKey, recordValue));
 
         queue.addRawRecords(list3, timestampExtractor);
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java
index fd6f49f..e0be587 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java
@@ -24,6 +24,7 @@ import org.apache.kafka.clients.consumer.OffsetResetStrategy;
 import org.apache.kafka.common.Node;
 import org.apache.kafka.common.PartitionInfo;
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.serialization.IntegerSerializer;
 import org.apache.kafka.common.serialization.Serializer;
 import org.apache.kafka.common.utils.Utils;
@@ -152,7 +153,7 @@ public class StandbyTaskTest {
             restoreStateConsumer.assign(new ArrayList<>(task.changeLogPartitions()));
 
             task.update(partition1,
-                    records(new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, recordKey, recordValue))
+                    records(new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, 0L, TimestampType.CREATE_TIME, recordKey, recordValue))
             );
 
         } finally {
@@ -171,9 +172,9 @@ public class StandbyTaskTest {
             restoreStateConsumer.assign(new ArrayList<>(task.changeLogPartitions()));
 
             for (ConsumerRecord<Integer, Integer> record : Arrays.asList(
-                    new ConsumerRecord<>(partition2.topic(), partition2.partition(), 10, 1, 100),
-                    new ConsumerRecord<>(partition2.topic(), partition2.partition(), 20, 2, 100),
-                    new ConsumerRecord<>(partition2.topic(), partition2.partition(), 30, 3, 100))) {
+                    new ConsumerRecord<>(partition2.topic(), partition2.partition(), 10, 0L, TimestampType.CREATE_TIME, 1, 100),
+                    new ConsumerRecord<>(partition2.topic(), partition2.partition(), 20, 0L, TimestampType.CREATE_TIME, 2, 100),
+                    new ConsumerRecord<>(partition2.topic(), partition2.partition(), 30, 0L, TimestampType.CREATE_TIME, 3, 100))) {
                 restoreStateConsumer.bufferRecord(record);
             }
 
@@ -234,11 +235,11 @@ public class StandbyTaskTest {
             restoreStateConsumer.assign(new ArrayList<>(task.changeLogPartitions()));
 
             for (ConsumerRecord<Integer, Integer> record : Arrays.asList(
-                    new ConsumerRecord<>(ktable.topic(), ktable.partition(), 10, 1, 100),
-                    new ConsumerRecord<>(ktable.topic(), ktable.partition(), 20, 2, 100),
-                    new ConsumerRecord<>(ktable.topic(), ktable.partition(), 30, 3, 100),
-                    new ConsumerRecord<>(ktable.topic(), ktable.partition(), 40, 4, 100),
-                    new ConsumerRecord<>(ktable.topic(), ktable.partition(), 50, 5, 100))) {
+                    new ConsumerRecord<>(ktable.topic(), ktable.partition(), 10, 0L, TimestampType.CREATE_TIME, 1, 100),
+                    new ConsumerRecord<>(ktable.topic(), ktable.partition(), 20, 0L, TimestampType.CREATE_TIME, 2, 100),
+                    new ConsumerRecord<>(ktable.topic(), ktable.partition(), 30, 0L, TimestampType.CREATE_TIME, 3, 100),
+                    new ConsumerRecord<>(ktable.topic(), ktable.partition(), 40, 0L, TimestampType.CREATE_TIME, 4, 100),
+                    new ConsumerRecord<>(ktable.topic(), ktable.partition(), 50, 0L, TimestampType.CREATE_TIME, 5, 100))) {
                 restoreStateConsumer.bufferRecord(record);
             }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
index bf3b3b1..94f0ce3 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
@@ -22,6 +22,7 @@ import org.apache.kafka.clients.consumer.MockConsumer;
 import org.apache.kafka.clients.consumer.OffsetResetStrategy;
 import org.apache.kafka.clients.producer.MockProducer;
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.serialization.ByteArraySerializer;
 import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.kafka.common.serialization.IntegerDeserializer;
@@ -107,15 +108,15 @@ public class StreamTaskTest {
             StreamTask task = new StreamTask(new TaskId(0, 0), "jobId", partitions, topology, consumer, producer, restoreStateConsumer, config, null);
 
             task.addRecords(partition1, records(
-                    new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, recordKey, recordValue),
-                    new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, recordKey, recordValue),
-                    new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, recordKey, recordValue)
+                    new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, 0L, TimestampType.CREATE_TIME, recordKey, recordValue),
+                    new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, TimestampType.CREATE_TIME, recordKey, recordValue),
+                    new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, 0L, TimestampType.CREATE_TIME, recordKey, recordValue)
             ));
 
             task.addRecords(partition2, records(
-                    new ConsumerRecord<>(partition2.topic(), partition2.partition(), 25, recordKey, recordValue),
-                    new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, recordKey, recordValue),
-                    new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, recordKey, recordValue)
+                    new ConsumerRecord<>(partition2.topic(), partition2.partition(), 25, 0L, TimestampType.CREATE_TIME, recordKey, recordValue),
+                    new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, TimestampType.CREATE_TIME, recordKey, recordValue),
+                    new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, TimestampType.CREATE_TIME, recordKey, recordValue)
             ));
 
             assertEquals(5, task.process());
@@ -158,15 +159,15 @@ public class StreamTaskTest {
             StreamTask task = new StreamTask(new TaskId(1, 1), "jobId", partitions, topology, consumer, producer, restoreStateConsumer, config, null);
 
             task.addRecords(partition1, records(
-                    new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, recordKey, recordValue),
-                    new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, recordKey, recordValue)
+                    new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, 0L, TimestampType.CREATE_TIME, recordKey, recordValue),
+                    new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, TimestampType.CREATE_TIME, recordKey, recordValue)
             ));
 
             task.addRecords(partition2, records(
-                    new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, recordKey, recordValue),
-                    new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, recordKey, recordValue),
-                    new ConsumerRecord<>(partition2.topic(), partition2.partition(), 55, recordKey, recordValue),
-                    new ConsumerRecord<>(partition2.topic(), partition2.partition(), 65, recordKey, recordValue)
+                    new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, TimestampType.CREATE_TIME, recordKey, recordValue),
+                    new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, TimestampType.CREATE_TIME, recordKey, recordValue),
+                    new ConsumerRecord<>(partition2.topic(), partition2.partition(), 55, 0L, TimestampType.CREATE_TIME, recordKey, recordValue),
+                    new ConsumerRecord<>(partition2.topic(), partition2.partition(), 65, 0L, TimestampType.CREATE_TIME, recordKey, recordValue)
             ));
 
             assertEquals(5, task.process());
@@ -177,9 +178,9 @@ public class StreamTaskTest {
             assertTrue(consumer.paused().contains(partition2));
 
             task.addRecords(partition1, records(
-                    new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, recordKey, recordValue),
-                    new ConsumerRecord<>(partition1.topic(), partition1.partition(), 40, recordKey, recordValue),
-                    new ConsumerRecord<>(partition1.topic(), partition1.partition(), 50, recordKey, recordValue)
+                    new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, 0L, TimestampType.CREATE_TIME, recordKey, recordValue),
+                    new ConsumerRecord<>(partition1.topic(), partition1.partition(), 40, 0L, TimestampType.CREATE_TIME, recordKey, recordValue),
+                    new ConsumerRecord<>(partition1.topic(), partition1.partition(), 50, 0L, TimestampType.CREATE_TIME, recordKey, recordValue)
             ));
 
             assertEquals(2, consumer.paused().size());

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java
index 5edff28..e414d80 100644
--- a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java
+++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java
@@ -24,6 +24,7 @@ import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.common.PartitionInfo;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.serialization.ByteArraySerializer;
 import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.kafka.common.serialization.Serializer;
@@ -200,7 +201,7 @@ public class ProcessorTopologyTestDriver {
         }
         // Add the record ...
         long offset = offsetsByTopicPartition.get(tp).incrementAndGet();
-        task.addRecords(tp, records(new ConsumerRecord<byte[], byte[]>(tp.topic(), tp.partition(), offset, key, value)));
+        task.addRecords(tp, records(new ConsumerRecord<byte[], byte[]>(tp.topic(), tp.partition(), offset, 0L, TimestampType.CREATE_TIME, key, value)));
         producer.clear();
         // Process the record ...
         task.process();


[5/5] kafka git commit: KAFKA-3025; Added timetamp to Message and use relative offset.

Posted by ju...@apache.org.
KAFKA-3025; Added timetamp to Message and use relative offset.

See KIP-31 and KIP-32 for details.

A few notes on the patch:
1. This patch implements KIP-31 and KIP-32. The patch includes features in both KAFKA-3025,  KAFKA-3026 and KAFKA-3036
2. All unit tests passed.
3. The unit tests were run with new and old message format.
4. When message format conversion occurs during consumption, the consumer will not be able to detect the message size too large situation. I did not try to fix this because the situation seems rare and only happen during migration phase.

Author: Jiangjie Qin <be...@gmail.com>
Author: Ismael Juma <is...@juma.me.uk>
Author: Jiangjie (Becket) Qin <be...@gmail.com>

Reviewers: Jason Gustafson <ja...@confluent.io>, Anna Povzner <an...@confluent.io>, Ismael Juma <is...@juma.me.uk>, Guozhang Wang <wa...@gmail.com>, Jun Rao <ju...@gmail.com>

Closes #764 from becketqin/KAFKA-3025


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

Branch: refs/heads/trunk
Commit: 45c8195fa14c766b200c720f316836dbb84e9d8b
Parents: eee9522
Author: Jiangjie Qin <be...@gmail.com>
Authored: Fri Feb 19 07:56:40 2016 -0800
Committer: Jun Rao <ju...@gmail.com>
Committed: Fri Feb 19 07:56:40 2016 -0800

----------------------------------------------------------------------
 checkstyle/import-control.xml                   |   1 +
 .../kafka/clients/consumer/ConsumerRecord.java  |  32 +-
 .../clients/consumer/internals/Fetcher.java     |   5 +-
 .../kafka/clients/producer/KafkaProducer.java   |  11 +-
 .../kafka/clients/producer/MockProducer.java    |   5 +-
 .../kafka/clients/producer/ProducerRecord.java  |  61 ++-
 .../kafka/clients/producer/RecordMetadata.java  |  20 +-
 .../internals/FutureRecordMetadata.java         |  10 +-
 .../producer/internals/RecordAccumulator.java   |  16 +-
 .../clients/producer/internals/RecordBatch.java |  18 +-
 .../clients/producer/internals/Sender.java      |  15 +-
 .../errors/InvalidTimestampException.java       |  34 ++
 .../apache/kafka/common/protocol/Errors.java    |   5 +-
 .../apache/kafka/common/protocol/Protocol.java  |  52 ++-
 .../apache/kafka/common/record/Compressor.java  |  22 +-
 .../kafka/common/record/MemoryRecords.java      | 115 +++--
 .../org/apache/kafka/common/record/Record.java  | 146 ++++--
 .../kafka/common/record/TimestampType.java      |  62 +++
 .../kafka/common/requests/ProduceRequest.java   |   6 +-
 .../kafka/common/requests/ProduceResponse.java  |  40 +-
 .../org/apache/kafka/common/utils/Crc32.java    |  11 +
 .../clients/consumer/MockConsumerTest.java      |   5 +-
 .../internals/ConsumerInterceptorsTest.java     |   8 +-
 .../clients/consumer/internals/FetcherTest.java |  17 +-
 .../clients/producer/ProducerRecordTest.java    |   2 +-
 .../kafka/clients/producer/RecordSendTest.java  |   7 +-
 .../internals/ProducerInterceptorsTest.java     |   2 +-
 .../internals/RecordAccumulatorTest.java        |  30 +-
 .../clients/producer/internals/SenderTest.java  |  11 +-
 .../kafka/common/record/MemoryRecordsTest.java  |   8 +-
 .../apache/kafka/common/record/RecordTest.java  |  16 +-
 .../common/requests/RequestResponseTest.java    |  14 +-
 .../kafka/test/MockConsumerInterceptor.java     |   4 +-
 .../connect/runtime/WorkerSinkTaskTest.java     |   3 +-
 .../runtime/WorkerSinkTaskThreadedTest.java     |   5 +-
 .../connect/runtime/WorkerSourceTaskTest.java   |   2 +-
 .../connect/storage/KafkaConfigStorageTest.java |  25 +-
 .../storage/KafkaOffsetBackingStoreTest.java    |  17 +-
 .../kafka/connect/util/KafkaBasedLogTest.java   |  17 +-
 .../main/scala/kafka/admin/ConfigCommand.scala  |  18 +-
 .../main/scala/kafka/admin/TopicCommand.scala   |  27 +-
 core/src/main/scala/kafka/api/ApiVersion.scala  |  42 +-
 .../src/main/scala/kafka/api/FetchRequest.scala |   3 +-
 .../main/scala/kafka/api/ProducerRequest.scala  |   4 +-
 .../main/scala/kafka/api/ProducerResponse.scala |  14 +-
 .../main/scala/kafka/common/ErrorMapping.scala  |   1 +
 .../scala/kafka/consumer/BaseConsumer.scala     |  26 +-
 .../scala/kafka/consumer/ConsumerIterator.scala |   9 +-
 .../controller/ControllerChannelManager.scala   |   4 +-
 .../kafka/coordinator/GroupCoordinator.scala    |  36 +-
 .../coordinator/GroupMetadataManager.scala      |  44 +-
 .../main/scala/kafka/log/FileMessageSet.scala   |  63 ++-
 core/src/main/scala/kafka/log/Log.scala         |  33 +-
 core/src/main/scala/kafka/log/LogCleaner.scala  |  53 ++-
 core/src/main/scala/kafka/log/LogConfig.scala   |  39 +-
 core/src/main/scala/kafka/log/LogSegment.scala  |   2 +-
 .../kafka/message/ByteBufferMessageSet.scala    | 447 ++++++++++++++++---
 core/src/main/scala/kafka/message/Message.scala | 259 +++++++++--
 .../kafka/message/MessageAndMetadata.scala      |  11 +-
 .../main/scala/kafka/message/MessageSet.scala   |  25 +-
 .../scala/kafka/message/MessageWriter.scala     |  27 +-
 .../producer/async/DefaultEventHandler.scala    |  23 +-
 .../kafka/server/AbstractFetcherThread.scala    |   2 +-
 .../main/scala/kafka/server/ConfigHandler.scala |  14 +-
 .../src/main/scala/kafka/server/KafkaApis.scala |  46 +-
 .../main/scala/kafka/server/KafkaConfig.scala   |  25 +-
 .../main/scala/kafka/server/KafkaServer.scala   |  11 +-
 .../kafka/server/ReplicaFetcherThread.scala     |   7 +-
 .../scala/kafka/server/ReplicaManager.scala     |  21 +-
 .../scala/kafka/tools/ConsoleConsumer.scala     |  33 +-
 .../scala/kafka/tools/DumpLogSegments.scala     |   2 +-
 .../main/scala/kafka/tools/MirrorMaker.scala    |  16 +-
 .../scala/kafka/tools/ReplayLogProducer.scala   |   4 +-
 .../scala/kafka/tools/SimpleConsumerShell.scala |   3 +-
 .../kafka/api/BaseConsumerTest.scala            |  42 +-
 .../kafka/api/BaseProducerSendTest.scala        | 180 ++++++--
 .../kafka/api/PlaintextConsumerTest.scala       | 113 +++--
 .../kafka/api/ProducerCompressionTest.scala     |   5 +-
 .../api/RequestResponseSerializationTest.scala  |   1 +
 .../GroupCoordinatorResponseTest.scala          |  13 +-
 .../test/scala/unit/kafka/log/CleanerTest.scala |  13 +-
 .../unit/kafka/log/FileMessageSetTest.scala     |  68 +++
 .../kafka/log/LogCleanerIntegrationTest.scala   |   2 +-
 .../scala/unit/kafka/log/LogConfigTest.scala    |   3 +-
 .../scala/unit/kafka/log/LogManagerTest.scala   |   1 +
 .../src/test/scala/unit/kafka/log/LogTest.scala |  18 +-
 .../message/ByteBufferMessageSetTest.scala      | 260 ++++++++++-
 .../kafka/message/MessageCompressionTest.scala  |  10 +-
 .../scala/unit/kafka/message/MessageTest.scala  |  79 +++-
 .../unit/kafka/message/MessageWriterTest.scala  |   9 +-
 .../unit/kafka/producer/AsyncProducerTest.scala |  15 +-
 .../unit/kafka/producer/ProducerTest.scala      |  31 +-
 .../unit/kafka/producer/SyncProducerTest.scala  |  77 ++--
 .../unit/kafka/server/EdgeCaseRequestTest.scala |   2 +-
 .../unit/kafka/server/KafkaConfigTest.scala     |  16 +-
 .../scala/unit/kafka/server/LogOffsetTest.scala |  19 +-
 .../unit/kafka/tools/ConsoleConsumerTest.scala  |   4 +-
 .../test/scala/unit/kafka/utils/TestUtils.scala |   7 +-
 docs/upgrade.html                               |  52 +++
 .../processor/internals/RecordQueue.java        |   4 +-
 .../processor/internals/PartitionGroupTest.java |  13 +-
 .../internals/ProcessorStateManagerTest.java    |   8 +-
 .../processor/internals/RecordQueueTest.java    |  19 +-
 .../processor/internals/StandbyTaskTest.java    |  19 +-
 .../processor/internals/StreamTaskTest.java     |  31 +-
 .../kafka/test/ProcessorTopologyTestDriver.java |   3 +-
 106 files changed, 2688 insertions(+), 723 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/checkstyle/import-control.xml
----------------------------------------------------------------------
diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index a663cf7..b183b3d 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -84,6 +84,7 @@
     <subpackage name="requests">
       <allow pkg="org.apache.kafka.common.protocol" />
       <allow pkg="org.apache.kafka.common.network" />
+      <allow pkg="org.apache.kafka.common.record" />
       <!-- for testing -->
       <allow pkg="org.apache.kafka.common.errors" />
     </subpackage>

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java
index d4668c2..42e0a90 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java
@@ -12,6 +12,8 @@
  */
 package org.apache.kafka.clients.consumer;
 
+import org.apache.kafka.common.record.TimestampType;
+
 /**
  * A key/value pair to be received from Kafka. This consists of a topic name and a partition number, from which the
  * record is being received and an offset that points to the record in a Kafka partition.
@@ -20,6 +22,8 @@ public final class ConsumerRecord<K, V> {
     private final String topic;
     private final int partition;
     private final long offset;
+    private final long timestamp;
+    private final TimestampType timestampType;
     private final K key;
     private final V value;
 
@@ -29,15 +33,25 @@ public final class ConsumerRecord<K, V> {
      * @param topic The topic this record is received from
      * @param partition The partition of the topic this record is received from
      * @param offset The offset of this record in the corresponding Kafka partition
+     * @param timestamp The timestamp of the record.
+     * @param timestampType The timestamp type
      * @param key The key of the record, if one exists (null is allowed)
      * @param value The record contents
      */
-    public ConsumerRecord(String topic, int partition, long offset, K key, V value) {
+    public ConsumerRecord(String topic,
+                          int partition,
+                          long offset,
+                          long timestamp,
+                          TimestampType timestampType,
+                          K key,
+                          V value) {
         if (topic == null)
             throw new IllegalArgumentException("Topic cannot be null");
         this.topic = topic;
         this.partition = partition;
         this.offset = offset;
+        this.timestamp = timestamp;
+        this.timestampType = timestampType;
         this.key = key;
         this.value = value;
     }
@@ -77,9 +91,23 @@ public final class ConsumerRecord<K, V> {
         return offset;
     }
 
+    /**
+     * The timestamp of this record
+     */
+    public long timestamp() {
+        return timestamp;
+    }
+
+    /**
+     * The timestamp type of this record
+     */
+    public TimestampType timestampType() {
+        return timestampType;
+    }
+
     @Override
     public String toString() {
         return "ConsumerRecord(topic = " + topic() + ", partition = " + partition() + ", offset = " + offset()
-                + ", key = " + key + ", value = " + value + ")";
+                + ", " + timestampType + " = " + timestamp + ", key = " + key + ", value = " + value + ")";
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 e8f1f55..427664a 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
@@ -40,6 +40,7 @@ import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
 import org.apache.kafka.common.record.LogEntry;
 import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.requests.FetchRequest;
 import org.apache.kafka.common.requests.FetchResponse;
 import org.apache.kafka.common.requests.ListOffsetRequest;
@@ -614,12 +615,14 @@ public class Fetcher<K, V> {
             if (this.checkCrcs)
                 logEntry.record().ensureValid();
             long offset = logEntry.offset();
+            long timestamp = logEntry.record().timestamp();
+            TimestampType timestampType = logEntry.record().timestampType();
             ByteBuffer keyBytes = logEntry.record().key();
             K key = keyBytes == null ? null : this.keyDeserializer.deserialize(partition.topic(), Utils.toArray(keyBytes));
             ByteBuffer valueBytes = logEntry.record().value();
             V value = valueBytes == null ? null : this.valueDeserializer.deserialize(partition.topic(), Utils.toArray(valueBytes));
 
-            return new ConsumerRecord<>(partition.topic(), partition.partition(), offset, key, value);
+            return new ConsumerRecord<>(partition.topic(), partition.partition(), offset, timestamp, timestampType, key, value);
         } catch (KafkaException e) {
             throw e;
         } catch (RuntimeException e) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
index a76dc1a..a066512 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
@@ -359,8 +359,12 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
      * records waiting to be sent. This allows sending many records in parallel without blocking to wait for the
      * response after each one.
      * <p>
-     * The result of the send is a {@link RecordMetadata} specifying the partition the record was sent to and the offset
-     * it was assigned.
+     * The result of the send is a {@link RecordMetadata} specifying the partition the record was sent to, the offset
+     * it was assigned and the timestamp of the record. If
+     * {@link org.apache.kafka.common.record.TimestampType#CREATE_TIME CreateTime} is used by the topic, the timestamp
+     * will be the user provided timestamp or the record send time if the user did not specify a timestamp for the
+     * record. If {@link org.apache.kafka.common.record.TimestampType#LOG_APPEND_TIME LogAppendTime} is used for the
+     * topic, the timestamp will be the Kafka broker local time when the message is appended.
      * <p>
      * Since the send call is asynchronous it returns a {@link java.util.concurrent.Future Future} for the
      * {@link RecordMetadata} that will be assigned to this record. Invoking {@link java.util.concurrent.Future#get()
@@ -456,8 +460,9 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
             int serializedSize = Records.LOG_OVERHEAD + Record.recordSize(serializedKey, serializedValue);
             ensureValidRecordSize(serializedSize);
             TopicPartition tp = new TopicPartition(record.topic(), partition);
+            long timestamp = record.timestamp() == null ? time.milliseconds() : record.timestamp();
             log.trace("Sending record {} with callback {} to topic {} partition {}", record, callback, record.topic(), partition);
-            RecordAccumulator.RecordAppendResult result = accumulator.append(tp, serializedKey, serializedValue, callback, remainingWaitMs);
+            RecordAccumulator.RecordAppendResult result = accumulator.append(tp, timestamp, serializedKey, serializedValue, callback, remainingWaitMs);
             if (result.batchIsFull || result.newBatchCreated) {
                 log.trace("Waking up the sender since topic {} partition {} is either full or getting a new batch", record.topic(), partition);
                 this.sender.wakeup();

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
index 8388ab8..5f97bae 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
@@ -34,6 +34,7 @@ import org.apache.kafka.common.Metric;
 import org.apache.kafka.common.MetricName;
 import org.apache.kafka.common.PartitionInfo;
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.record.Record;
 import org.apache.kafka.common.serialization.Serializer;
 
 
@@ -116,10 +117,10 @@ public class MockProducer<K, V> implements Producer<K, V> {
         if (this.cluster.partitionsForTopic(record.topic()) != null)
             partition = partition(record, this.cluster);
         ProduceRequestResult result = new ProduceRequestResult();
-        FutureRecordMetadata future = new FutureRecordMetadata(result, 0);
+        FutureRecordMetadata future = new FutureRecordMetadata(result, 0, Record.NO_TIMESTAMP);
         TopicPartition topicPartition = new TopicPartition(record.topic(), partition);
         long offset = nextOffset(topicPartition);
-        Completion completion = new Completion(topicPartition, offset, new RecordMetadata(topicPartition, 0, offset), result, callback);
+        Completion completion = new Completion(topicPartition, offset, new RecordMetadata(topicPartition, 0, offset, Record.NO_TIMESTAMP), result, callback);
         this.sent.add(record);
         if (autoComplete)
             completion.complete(null);

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java
index 75cd51e..85b4d8d 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java
@@ -19,6 +19,22 @@ package org.apache.kafka.clients.producer;
  * If a valid partition number is specified that partition will be used when sending the record. If no partition is
  * specified but a key is present a partition will be chosen using a hash of the key. If neither key nor partition is
  * present a partition will be assigned in a round-robin fashion.
+ * <p>
+ * The record also has an associated timestamp. If the user did not provide a timestamp, the producer will stamp the
+ * record with its current time. The timestamp eventually used by Kafka depends on the timestamp type configured for
+ * the topic.
+ * <li>
+ * If the topic is configured to use {@link org.apache.kafka.common.record.TimestampType#CREATE_TIME CreateTime},
+ * the timestamp in the producer record will be used by the broker.
+ * </li>
+ * <li>
+ * If the topic is configured to use {@link org.apache.kafka.common.record.TimestampType#LOG_APPEND_TIME LogAppendTime},
+ * the timestamp in the producer record will be overwritten by the broker with the broker local time when it appends the
+ * message to its log.
+ * </li>
+ * <p>
+ * In either of the cases above, the timestamp that has actually been used will be returned to user in
+ * {@link RecordMetadata}
  */
 public final class ProducerRecord<K, V> {
 
@@ -26,22 +42,39 @@ public final class ProducerRecord<K, V> {
     private final Integer partition;
     private final K key;
     private final V value;
+    private final Long timestamp;
 
     /**
-     * Creates a record to be sent to a specified topic and partition
+     * Creates a record with a specified timestamp to be sent to a specified topic and partition
      * 
      * @param topic The topic the record will be appended to
      * @param partition The partition to which the record should be sent
+     * @param timestamp The timestamp of the record
      * @param key The key that will be included in the record
      * @param value The record contents
      */
-    public ProducerRecord(String topic, Integer partition, K key, V value) {
+    public ProducerRecord(String topic, Integer partition, Long timestamp, K key, V value) {
         if (topic == null)
             throw new IllegalArgumentException("Topic cannot be null");
+        if (timestamp != null && timestamp < 0)
+            throw new IllegalArgumentException("Invalid timestamp " + timestamp);
         this.topic = topic;
         this.partition = partition;
         this.key = key;
         this.value = value;
+        this.timestamp = timestamp;
+    }
+
+    /**
+     * Creates a record to be sent to a specified topic and partition
+     *
+     * @param topic The topic the record will be appended to
+     * @param partition The partition to which the record should be sent
+     * @param key The key that will be included in the record
+     * @param value The record contents
+     */
+    public ProducerRecord(String topic, Integer partition, K key, V value) {
+        this(topic, partition, null, key, value);
     }
 
     /**
@@ -52,7 +85,7 @@ public final class ProducerRecord<K, V> {
      * @param value The record contents
      */
     public ProducerRecord(String topic, K key, V value) {
-        this(topic, null, key, value);
+        this(topic, null, null, key, value);
     }
 
     /**
@@ -62,18 +95,18 @@ public final class ProducerRecord<K, V> {
      * @param value The record contents
      */
     public ProducerRecord(String topic, V value) {
-        this(topic, null, value);
+        this(topic, null, null, null, value);
     }
 
     /**
-     * The topic this record is being sent to
+     * @return The topic this record is being sent to
      */
     public String topic() {
         return topic;
     }
 
     /**
-     * The key (or null if no key is specified)
+     * @return The key (or null if no key is specified)
      */
     public K key() {
         return key;
@@ -87,7 +120,14 @@ public final class ProducerRecord<K, V> {
     }
 
     /**
-     * The partition to which the record will be sent (or null if no partition was specified)
+     * @return The timestamp
+     */
+    public Long timestamp() {
+        return timestamp;
+    }
+
+    /**
+     * @return The partition to which the record will be sent (or null if no partition was specified)
      */
     public Integer partition() {
         return partition;
@@ -97,7 +137,9 @@ public final class ProducerRecord<K, V> {
     public String toString() {
         String key = this.key == null ? "null" : this.key.toString();
         String value = this.value == null ? "null" : this.value.toString();
-        return "ProducerRecord(topic=" + topic + ", partition=" + partition + ", key=" + key + ", value=" + value;
+        String timestamp = this.timestamp == null ? "null" : this.timestamp.toString();
+        return "ProducerRecord(topic=" + topic + ", partition=" + partition + ", key=" + key + ", value=" + value +
+            ", timestamp=" + timestamp + ")";
     }
 
     @Override
@@ -117,6 +159,8 @@ public final class ProducerRecord<K, V> {
             return false;
         else if (value != null ? !value.equals(that.value) : that.value != null) 
             return false;
+        else if (timestamp != null ? !timestamp.equals(that.timestamp) : that.timestamp != null)
+            return false;
 
         return true;
     }
@@ -127,6 +171,7 @@ public final class ProducerRecord<K, V> {
         result = 31 * result + (partition != null ? partition.hashCode() : 0);
         result = 31 * result + (key != null ? key.hashCode() : 0);
         result = 31 * result + (value != null ? value.hashCode() : 0);
+        result = 31 * result + (timestamp != null ? timestamp.hashCode() : 0);
         return result;
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java
index a80f6b9..d9ea239 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java
@@ -24,18 +24,25 @@ import org.apache.kafka.common.TopicPartition;
 public final class RecordMetadata {
 
     private final long offset;
+    // The timestamp of the message.
+    // If LogAppendTime is used for the topic, the timestamp will be the timestamp returned by the broker.
+    // If CreateTime is used for the topic, the timestamp is the timestamp in the corresponding ProducerRecord if the
+    // user provided one. Otherwise, it will be the producer local time when the producer record was handed to the
+    // producer.
+    private final long timestamp;
     private final TopicPartition topicPartition;
 
-    private RecordMetadata(TopicPartition topicPartition, long offset) {
+    private RecordMetadata(TopicPartition topicPartition, long offset, long timestamp) {
         super();
         this.offset = offset;
+        this.timestamp = timestamp;
         this.topicPartition = topicPartition;
     }
 
-    public RecordMetadata(TopicPartition topicPartition, long baseOffset, long relativeOffset) {
+    public RecordMetadata(TopicPartition topicPartition, long baseOffset, long relativeOffset, long timestamp) {
         // ignore the relativeOffset if the base offset is -1,
         // since this indicates the offset is unknown
-        this(topicPartition, baseOffset == -1 ? baseOffset : baseOffset + relativeOffset);
+        this(topicPartition, baseOffset == -1 ? baseOffset : baseOffset + relativeOffset, timestamp);
     }
 
     /**
@@ -46,6 +53,13 @@ public final class RecordMetadata {
     }
 
     /**
+     * The timestamp of the record in the topic/partition.
+     */
+    public long timestamp() {
+        return timestamp;
+    }
+
+    /**
      * The topic the record was appended to
      */
     public String topic() {

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java
index e2d9ca8..a140371 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java
@@ -26,10 +26,12 @@ public final class FutureRecordMetadata implements Future<RecordMetadata> {
 
     private final ProduceRequestResult result;
     private final long relativeOffset;
+    private final long timestamp;
 
-    public FutureRecordMetadata(ProduceRequestResult result, long relativeOffset) {
+    public FutureRecordMetadata(ProduceRequestResult result, long relativeOffset, long timestamp) {
         this.result = result;
         this.relativeOffset = relativeOffset;
+        this.timestamp = timestamp;
     }
 
     @Override
@@ -59,13 +61,17 @@ public final class FutureRecordMetadata implements Future<RecordMetadata> {
     }
     
     RecordMetadata value() {
-        return new RecordMetadata(result.topicPartition(), this.result.baseOffset(), this.relativeOffset);
+        return new RecordMetadata(result.topicPartition(), this.result.baseOffset(), this.relativeOffset, this.timestamp);
     }
     
     public long relativeOffset() {
         return this.relativeOffset;
     }
 
+    public long timestamp() {
+        return this.timestamp;
+    }
+
     @Override
     public boolean isCancelled() {
         return false;

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 3c710c8..f1414f0 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
@@ -146,12 +146,18 @@ public final class RecordAccumulator {
      * <p>
      *
      * @param tp The topic/partition to which this record is being sent
+     * @param timestamp The timestamp of the record
      * @param key The key for the record
      * @param value The value for the record
      * @param callback The user-supplied callback to execute when the request is complete
      * @param maxTimeToBlock The maximum time in milliseconds to block for buffer memory to be available
      */
-    public RecordAppendResult append(TopicPartition tp, byte[] key, byte[] value, Callback callback, long maxTimeToBlock) throws InterruptedException {
+    public RecordAppendResult append(TopicPartition tp,
+                                     long timestamp,
+                                     byte[] key,
+                                     byte[] value,
+                                     Callback callback,
+                                     long maxTimeToBlock) throws InterruptedException {
         // We keep track of the number of appending thread to make sure we do not miss batches in
         // abortIncompleteBatches().
         appendsInProgress.incrementAndGet();
@@ -163,7 +169,7 @@ public final class RecordAccumulator {
                     throw new IllegalStateException("Cannot send after the producer is closed.");
                 RecordBatch last = dq.peekLast();
                 if (last != null) {
-                    FutureRecordMetadata future = last.tryAppend(key, value, callback, time.milliseconds());
+                    FutureRecordMetadata future = last.tryAppend(timestamp, key, value, callback, time.milliseconds());
                     if (future != null)
                         return new RecordAppendResult(future, dq.size() > 1 || last.records.isFull(), false);
                 }
@@ -179,7 +185,7 @@ public final class RecordAccumulator {
                     throw new IllegalStateException("Cannot send after the producer is closed.");
                 RecordBatch last = dq.peekLast();
                 if (last != null) {
-                    FutureRecordMetadata future = last.tryAppend(key, value, callback, time.milliseconds());
+                    FutureRecordMetadata future = last.tryAppend(timestamp, key, value, callback, time.milliseconds());
                     if (future != null) {
                         // Somebody else found us a batch, return the one we waited for! Hopefully this doesn't happen often...
                         free.deallocate(buffer);
@@ -188,7 +194,7 @@ public final class RecordAccumulator {
                 }
                 MemoryRecords records = MemoryRecords.emptyRecords(buffer, compression, this.batchSize);
                 RecordBatch batch = new RecordBatch(tp, records, time.milliseconds());
-                FutureRecordMetadata future = Utils.notNull(batch.tryAppend(key, value, callback, time.milliseconds()));
+                FutureRecordMetadata future = Utils.notNull(batch.tryAppend(timestamp, key, value, callback, time.milliseconds()));
 
                 dq.addLast(batch);
                 incomplete.add(batch);
@@ -454,7 +460,7 @@ public final class RecordAccumulator {
                 batch.records.close();
                 dq.remove(batch);
             }
-            batch.done(-1L, new IllegalStateException("Producer is closed forcefully."));
+            batch.done(-1L, Record.NO_TIMESTAMP, new IllegalStateException("Producer is closed forcefully."));
             deallocate(batch);
         }
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 3f18582..af9095d 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
@@ -44,6 +44,7 @@ public final class RecordBatch {
     public final ProduceRequestResult produceFuture;
     public long lastAppendTime;
     private final List<Thunk> thunks;
+    private long offsetCounter = 0L;
     private boolean retry;
 
     public RecordBatch(TopicPartition tp, MemoryRecords records, long now) {
@@ -62,14 +63,14 @@ public final class RecordBatch {
      * 
      * @return The RecordSend corresponding to this record or null if there isn't sufficient room.
      */
-    public FutureRecordMetadata tryAppend(byte[] key, byte[] value, Callback callback, long now) {
+    public FutureRecordMetadata tryAppend(long timestamp, byte[] key, byte[] value, Callback callback, long now) {
         if (!this.records.hasRoomFor(key, value)) {
             return null;
         } else {
-            this.records.append(0L, key, value);
+            this.records.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);
+            FutureRecordMetadata future = new FutureRecordMetadata(this.produceFuture, this.recordCount, timestamp);
             if (callback != null)
                 thunks.add(new Thunk(callback, future));
             this.recordCount++;
@@ -81,9 +82,10 @@ public final class RecordBatch {
      * Complete the request
      * 
      * @param baseOffset The base offset of the messages assigned by the server
+     * @param timestamp The timestamp returned by the broker.
      * @param exception The exception that occurred (or null if the request was successful)
      */
-    public void done(long baseOffset, RuntimeException exception) {
+    public void done(long baseOffset, long timestamp, RuntimeException exception) {
         log.trace("Produced messages to topic-partition {} with base offset offset {} and error: {}.",
                   topicPartition,
                   baseOffset,
@@ -93,7 +95,9 @@ public final class RecordBatch {
             try {
                 Thunk thunk = this.thunks.get(i);
                 if (exception == null) {
-                    RecordMetadata metadata = new RecordMetadata(this.topicPartition,  baseOffset, thunk.future.relativeOffset());
+                    // 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(),
+                        timestamp == Record.NO_TIMESTAMP ? thunk.future.timestamp() : timestamp);
                     thunk.callback.onCompletion(metadata, null);
                 } else {
                     thunk.callback.onCompletion(null, exception);
@@ -133,7 +137,7 @@ public final class RecordBatch {
         if ((this.records.isFull() && requestTimeout < (now - this.lastAppendTime)) || requestTimeout < (now - (this.lastAttemptMs + lingerMs))) {
             expire = true;
             this.records.close();
-            this.done(-1L, new TimeoutException("Batch Expired"));
+            this.done(-1L, Record.NO_TIMESTAMP, new TimeoutException("Batch Expired"));
         }
 
         return expire;
@@ -152,4 +156,4 @@ public final class RecordBatch {
     public void setRetry() {
         this.retry = true;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 aa30716..8e93973 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
@@ -41,6 +41,7 @@ import org.apache.kafka.common.metrics.stats.Max;
 import org.apache.kafka.common.metrics.stats.Rate;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.record.Record;
 import org.apache.kafka.common.requests.ProduceRequest;
 import org.apache.kafka.common.requests.ProduceResponse;
 import org.apache.kafka.common.requests.RequestSend;
@@ -243,7 +244,7 @@ public class Sender implements Runnable {
                                                                                                   .request()
                                                                                                   .destination());
             for (RecordBatch batch : batches.values())
-                completeBatch(batch, Errors.NETWORK_EXCEPTION, -1L, correlationId, now);
+                completeBatch(batch, Errors.NETWORK_EXCEPTION, -1L, Record.NO_TIMESTAMP, correlationId, now);
         } else {
             log.trace("Received produce response from node {} with correlation id {}",
                       response.request().request().destination(),
@@ -251,13 +252,12 @@ public class Sender implements Runnable {
             // if we have a response, parse it
             if (response.hasResponse()) {
                 ProduceResponse produceResponse = new ProduceResponse(response.responseBody());
-                for (Map.Entry<TopicPartition, ProduceResponse.PartitionResponse> entry : produceResponse.responses()
-                                                                                                         .entrySet()) {
+                for (Map.Entry<TopicPartition, ProduceResponse.PartitionResponse> entry : produceResponse.responses().entrySet()) {
                     TopicPartition tp = entry.getKey();
                     ProduceResponse.PartitionResponse partResp = entry.getValue();
                     Errors error = Errors.forCode(partResp.errorCode);
                     RecordBatch batch = batches.get(tp);
-                    completeBatch(batch, error, partResp.baseOffset, correlationId, now);
+                    completeBatch(batch, error, partResp.baseOffset, partResp.timestamp, correlationId, now);
                 }
                 this.sensors.recordLatency(response.request().request().destination(), response.requestLatencyMs());
                 this.sensors.recordThrottleTime(response.request().request().destination(),
@@ -265,7 +265,7 @@ public class Sender implements Runnable {
             } else {
                 // this is the acks = 0 case, just complete all requests
                 for (RecordBatch batch : batches.values())
-                    completeBatch(batch, Errors.NONE, -1L, correlationId, now);
+                    completeBatch(batch, Errors.NONE, -1L, Record.NO_TIMESTAMP, correlationId, now);
             }
         }
     }
@@ -276,10 +276,11 @@ public class Sender implements Runnable {
      * @param batch The record batch
      * @param error The error (or null if none)
      * @param baseOffset The base offset assigned to the records if successful
+     * @param timestamp The timestamp returned by the broker for this batch
      * @param correlationId The correlation id for the request
      * @param now The current POSIX time stamp in milliseconds
      */
-    private void completeBatch(RecordBatch batch, Errors error, long baseOffset, long correlationId, long now) {
+    private void completeBatch(RecordBatch batch, Errors error, long baseOffset, long timestamp, long correlationId, long now) {
         if (error != Errors.NONE && canRetry(batch, error)) {
             // retry
             log.warn("Got error produce response with correlation id {} on topic-partition {}, retrying ({} attempts left). Error: {}",
@@ -296,7 +297,7 @@ public class Sender implements Runnable {
             else
                 exception = error.exception();
             // tell the user the result of their request
-            batch.done(baseOffset, exception);
+            batch.done(baseOffset, timestamp, exception);
             this.accumulator.deallocate(batch);
             if (error != Errors.NONE)
                 this.sensors.recordErrors(batch.topicPartition.topic(), batch.recordCount);

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/clients/src/main/java/org/apache/kafka/common/errors/InvalidTimestampException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidTimestampException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidTimestampException.java
new file mode 100644
index 0000000..d2d285b
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidTimestampException.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.common.errors;
+
+/**
+ * Indicate the timestamp of a record is invalid.
+ */
+public class InvalidTimestampException extends ApiException {
+
+    private static final long serialVersionUID = 1L;
+
+    public InvalidTimestampException(String message) {
+        super(message);
+    }
+
+    public InvalidTimestampException(String message, Throwable cause) {
+        super(message, cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
index 4a20869..e7098fc 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
@@ -34,6 +34,7 @@ import org.apache.kafka.common.errors.InvalidFetchSizeException;
 import org.apache.kafka.common.errors.InvalidGroupIdException;
 import org.apache.kafka.common.errors.InvalidRequiredAcksException;
 import org.apache.kafka.common.errors.InvalidSessionTimeoutException;
+import org.apache.kafka.common.errors.InvalidTimestampException;
 import org.apache.kafka.common.errors.InvalidTopicException;
 import org.apache.kafka.common.errors.LeaderNotAvailableException;
 import org.apache.kafka.common.errors.NetworkException;
@@ -125,7 +126,9 @@ public enum Errors {
     GROUP_AUTHORIZATION_FAILED(30,
             new GroupAuthorizationException("Group authorization failed.")),
     CLUSTER_AUTHORIZATION_FAILED(31,
-            new ClusterAuthorizationException("Cluster authorization failed."));
+            new ClusterAuthorizationException("Cluster authorization failed.")),
+    INVALID_TIMESTAMP(32,
+            new InvalidTimestampException("The timestamp of the message is out of acceptable range."));
 
     private static final Logger log = LoggerFactory.getLogger(Errors.class);
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
index 48c64c2..3787d2c 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
@@ -110,7 +110,17 @@ public class Protocol {
                                                                                                                                       INT16),
                                                                                                                             new Field("base_offset",
                                                                                                                                       INT64))))))));
+    /**
+     * The body of PRODUCE_REQUEST_V1 is the same as PRODUCE_REQUEST_V0.
+     * The version number is bumped up to indicate that the client supports quota throttle time field in the response.
+     */
     public static final Schema PRODUCE_REQUEST_V1 = PRODUCE_REQUEST_V0;
+    /**
+     * The body of PRODUCE_REQUEST_V2 is the same as PRODUCE_REQUEST_V1.
+     * The version number is bumped up to indicate that message format V1 is used which has relative offset and
+     * timestamp.
+     */
+    public static final Schema PRODUCE_REQUEST_V2 = PRODUCE_REQUEST_V1;
 
     public static final Schema PRODUCE_RESPONSE_V1 = new Schema(new Field("responses",
                                                                           new ArrayOf(new Schema(new Field("topic", STRING),
@@ -126,9 +136,33 @@ public class Protocol {
                                                                           "Duration in milliseconds for which the request was throttled" +
                                                                               " due to quota violation. (Zero if the request did not violate any quota.)",
                                                                           0));
-
-    public static final Schema[] PRODUCE_REQUEST = new Schema[] {PRODUCE_REQUEST_V0, PRODUCE_REQUEST_V1};
-    public static final Schema[] PRODUCE_RESPONSE = new Schema[] {PRODUCE_RESPONSE_V0, PRODUCE_RESPONSE_V1};
+    /**
+     * PRODUCE_RESPONSE_V2 added a timestamp field in the per partition response status.
+     * The timestamp is log append time if the topic is configured to use log append time. Or it is NoTimestamp when create
+     * time is used for the topic.
+     */
+    public static final Schema PRODUCE_RESPONSE_V2 = new Schema(new Field("responses",
+                                                                new ArrayOf(new Schema(new Field("topic", STRING),
+                                                                                       new Field("partition_responses",
+                                                                                       new ArrayOf(new Schema(new Field("partition",
+                                                                                                                        INT32),
+                                                                                                              new Field("error_code",
+                                                                                                                        INT16),
+                                                                                                              new Field("base_offset",
+                                                                                                                        INT64),
+                                                                                                              new Field("timestamp",
+                                                                                                                        INT64,
+                                                                                                                        "The timestamp returned by broker after appending the messages. " +
+                                                                                                                            "If CreateTime is used for the topic, the timestamp will be -1. " +
+                                                                                                                            "If LogAppendTime is used for the topic, the timestamp will be " +
+                                                                                                                            "the broker local time when the messages are appended."))))))),
+                                                                new Field("throttle_time_ms",
+                                                                          INT32,
+                                                                          "Duration in milliseconds for which the request was throttled" +
+                                                                              " due to quota violation. (Zero if the request did not violate any quota.)",
+                                                                          0));
+    public static final Schema[] PRODUCE_REQUEST = new Schema[] {PRODUCE_REQUEST_V0, PRODUCE_REQUEST_V1, PRODUCE_REQUEST_V2};
+    public static final Schema[] PRODUCE_RESPONSE = new Schema[] {PRODUCE_RESPONSE_V0, PRODUCE_RESPONSE_V1, PRODUCE_RESPONSE_V2};
 
     /* Offset commit api */
     public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
@@ -364,6 +398,10 @@ public class Protocol {
     // The V1 Fetch Request body is the same as V0.
     // Only the version number is incremented to indicate a newer client
     public static final Schema FETCH_REQUEST_V1 = FETCH_REQUEST_V0;
+    // The V2 Fetch Request body is the same as V1.
+    // Only the version number is incremented to indicate the client support message format V1 which uses
+    // relative offset and has timestamp.
+    public static final Schema FETCH_REQUEST_V2 = FETCH_REQUEST_V1;
     public static final Schema FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
                                                                                   INT32,
                                                                                   "Topic partition id."),
@@ -386,9 +424,13 @@ public class Protocol {
                                                                         0),
                                                               new Field("responses",
                                                                       new ArrayOf(FETCH_RESPONSE_TOPIC_V0)));
+    // Even though fetch response v2 has the same protocol as v1, the record set in the response is different. In v1,
+    // record set only includes messages of v0 (magic byte 0). In v2, record set can include messages of v0 and v1
+    // (magic byte 0 and 1). For details, see ByteBufferMessageSet.
+    public static final Schema FETCH_RESPONSE_V2 = FETCH_RESPONSE_V1;
 
-    public static final Schema[] FETCH_REQUEST = new Schema[] {FETCH_REQUEST_V0, FETCH_REQUEST_V1};
-    public static final Schema[] FETCH_RESPONSE = new Schema[] {FETCH_RESPONSE_V0, FETCH_RESPONSE_V1};
+    public static final Schema[] FETCH_REQUEST = new Schema[] {FETCH_REQUEST_V0, FETCH_REQUEST_V1, FETCH_REQUEST_V2};
+    public static final Schema[] FETCH_RESPONSE = new Schema[] {FETCH_RESPONSE_V0, FETCH_RESPONSE_V1, FETCH_RESPONSE_V2};
 
     /* List groups api */
     public static final Schema LIST_GROUPS_REQUEST_V0 = new Schema();

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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
index c7ff2e6..cde2178 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/Compressor.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/Compressor.java
@@ -89,6 +89,7 @@ public class Compressor {
     public long writtenUncompressed;
     public long numRecords;
     public float compressionRate;
+    public long maxTimestamp;
 
     public Compressor(ByteBuffer buffer, CompressionType type, int blockSize) {
         this.type = type;
@@ -97,6 +98,7 @@ public class Compressor {
         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
@@ -136,10 +138,10 @@ public class Compressor {
             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, null, null, type, 0, -1);
+            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, valueSize);
+            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,
@@ -199,19 +201,21 @@ public class Compressor {
         }
     }
 
-    public void putRecord(byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
+    public void 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(key, value, type, valueOffset, valueSize);
+        long crc = Record.computeChecksum(timestamp, key, value, type, valueOffset, valueSize);
         byte attributes = Record.computeAttributes(type);
-        putRecord(crc, attributes, key, value, valueOffset, valueSize);
+        putRecord(crc, attributes, timestamp, key, value, valueOffset, valueSize);
+
     }
 
-    public void putRecord(byte[] key, byte[] value) {
-        putRecord(key, value, CompressionType.NONE, 0, -1);
+    public void putRecord(long timestamp, byte[] key, byte[] value) {
+        putRecord(timestamp, key, value, CompressionType.NONE, 0, -1);
     }
 
-    private void putRecord(final long crc, final byte attributes, final byte[] key, final byte[] value, final int valueOffset, final int valueSize) {
-        Record.write(this, crc, attributes, key, value, valueOffset, valueSize);
+    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) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 971f0a2..01da1e2 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
@@ -16,6 +16,7 @@ import java.io.DataInputStream;
 import java.io.EOFException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
 import java.util.Iterator;
 
 import org.apache.kafka.common.KafkaException;
@@ -88,14 +89,14 @@ public class MemoryRecords implements Records {
     /**
      * Append a new record and offset to the buffer
      */
-    public void append(long offset, byte[] key, byte[] value) {
+    public void 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);
-        compressor.putRecord(key, value);
+        compressor.putRecord(timestamp, key, value);
         compressor.recordWritten(size + Records.LOG_OVERHEAD);
     }
 
@@ -214,11 +215,50 @@ public class MemoryRecords implements Records {
         private final boolean shallow;
         private RecordsIterator innerIter;
 
+        // The variables for inner iterator
+        private final ArrayDeque<LogEntry> logEntries;
+        private final long absoluteBaseOffset;
+
         public RecordsIterator(ByteBuffer buffer, CompressionType type, boolean shallow) {
             this.type = type;
             this.buffer = buffer;
             this.shallow = shallow;
             this.stream = Compressor.wrapForInput(new ByteBufferInputStream(this.buffer), type);
+            this.logEntries = null;
+            this.absoluteBaseOffset = -1;
+        }
+
+        // Private constructor for inner iterator.
+        private RecordsIterator(LogEntry entry) {
+            this.type = entry.record().compressionType();
+            this.buffer = entry.record().value();
+            this.shallow = true;
+            this.stream = Compressor.wrapForInput(new ByteBufferInputStream(this.buffer), type);
+            long wrapperRecordOffset = entry.offset();
+            // If relative offset is used, we need to decompress the entire message first to compute
+            // the absolute offset.
+            if (entry.record().magic() > Record.MAGIC_VALUE_V0) {
+                this.logEntries = new ArrayDeque<>();
+                long wrapperRecordTimestamp = entry.record().timestamp();
+                while (true) {
+                    try {
+                        LogEntry logEntry = getNextEntryFromStream();
+                        Record recordWithTimestamp = new Record(logEntry.record().buffer(),
+                                                                wrapperRecordTimestamp,
+                                                                entry.record().timestampType());
+                        logEntries.add(new LogEntry(logEntry.offset(), recordWithTimestamp));
+                    } catch (EOFException e) {
+                        break;
+                    } catch (IOException e) {
+                        throw new KafkaException(e);
+                    }
+                }
+                this.absoluteBaseOffset = wrapperRecordOffset - logEntries.getLast().offset();
+            } else {
+                this.logEntries = null;
+                this.absoluteBaseOffset = -1;
+            }
+
         }
 
         /*
@@ -232,28 +272,16 @@ public class MemoryRecords implements Records {
         protected LogEntry makeNext() {
             if (innerDone()) {
                 try {
-                    // read the offset
-                    long offset = stream.readLong();
-                    // read record size
-                    int size = stream.readInt();
-                    if (size < 0)
-                        throw new IllegalStateException("Record with size " + size);
-                    // read the record, if compression is used we cannot depend on size
-                    // and hence has to do extra copy
-                    ByteBuffer rec;
-                    if (type == CompressionType.NONE) {
-                        rec = buffer.slice();
-                        int newPos = buffer.position() + size;
-                        if (newPos > buffer.limit())
-                            return allDone();
-                        buffer.position(newPos);
-                        rec.limit(size);
-                    } else {
-                        byte[] recordBuffer = new byte[size];
-                        stream.readFully(recordBuffer, 0, size);
-                        rec = ByteBuffer.wrap(recordBuffer);
+                    LogEntry entry = getNextEntry();
+                    // No more record to return.
+                    if (entry == null)
+                        return allDone();
+
+                    // Convert offset to absolute offset if needed.
+                    if (absoluteBaseOffset >= 0) {
+                        long absoluteOffset = absoluteBaseOffset + entry.offset();
+                        entry = new LogEntry(absoluteOffset, entry.record());
                     }
-                    LogEntry entry = new LogEntry(offset, new Record(rec));
 
                     // decide whether to go shallow or deep iteration if it is compressed
                     CompressionType compression = entry.record().compressionType();
@@ -264,8 +292,9 @@ public class MemoryRecords implements Records {
                         // 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
-                        ByteBuffer value = entry.record().value();
-                        innerIter = new RecordsIterator(value, compression, true);
+                        // There will be at least one element in the inner iterator, so we don't
+                        // need to call hasNext() here.
+                        innerIter = new RecordsIterator(entry);
                         return innerIter.next();
                     }
                 } catch (EOFException e) {
@@ -278,6 +307,42 @@ public class MemoryRecords implements Records {
             }
         }
 
+        private LogEntry getNextEntry() throws IOException {
+            if (logEntries != null)
+                return getNextEntryFromEntryList();
+            else
+                return getNextEntryFromStream();
+        }
+
+        private LogEntry getNextEntryFromEntryList() {
+            return logEntries.isEmpty() ? null : logEntries.remove();
+        }
+
+        private LogEntry getNextEntryFromStream() throws IOException {
+            // read the offset
+            long offset = stream.readLong();
+            // read record size
+            int size = stream.readInt();
+            if (size < 0)
+                throw new IllegalStateException("Record with size " + size);
+            // read the record, if compression is used we cannot depend on size
+            // and hence has to do extra copy
+            ByteBuffer rec;
+            if (type == CompressionType.NONE) {
+                rec = buffer.slice();
+                int newPos = buffer.position() + size;
+                if (newPos > buffer.limit())
+                    return null;
+                buffer.position(newPos);
+                rec.limit(size);
+            } else {
+                byte[] recordBuffer = new byte[size];
+                stream.readFully(recordBuffer, 0, size);
+                rec = ByteBuffer.wrap(recordBuffer);
+            }
+            return new LogEntry(offset, new Record(rec));
+        }
+
         private boolean innerDone() {
             return innerIter == null || !innerIter.hasNext();
         }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 50fac24..8390dc7 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
@@ -36,9 +36,13 @@ public final class Record {
     public static final int MAGIC_LENGTH = 1;
     public static final int ATTRIBUTES_OFFSET = MAGIC_OFFSET + MAGIC_LENGTH;
     public static final int ATTRIBUTE_LENGTH = 1;
-    public static final int KEY_SIZE_OFFSET = ATTRIBUTES_OFFSET + ATTRIBUTE_LENGTH;
+    public static final int TIMESTAMP_OFFSET = ATTRIBUTES_OFFSET + ATTRIBUTE_LENGTH;
+    public static final int TIMESTAMP_LENGTH = 8;
+    public static final int KEY_SIZE_OFFSET_V0 = ATTRIBUTES_OFFSET + ATTRIBUTE_LENGTH;
+    public static final int KEY_SIZE_OFFSET_V1 = TIMESTAMP_OFFSET + TIMESTAMP_LENGTH;
     public static final int KEY_SIZE_LENGTH = 4;
-    public static final int KEY_OFFSET = KEY_SIZE_OFFSET + KEY_SIZE_LENGTH;
+    public static final int KEY_OFFSET_V0 = KEY_SIZE_OFFSET_V0 + KEY_SIZE_LENGTH;
+    public static final int KEY_OFFSET_V1 = KEY_SIZE_OFFSET_V1 + KEY_SIZE_LENGTH;
     public static final int VALUE_SIZE_LENGTH = 4;
 
     /**
@@ -49,12 +53,18 @@ public final class Record {
     /**
      * The amount of overhead bytes in a record
      */
-    public static final int RECORD_OVERHEAD = HEADER_SIZE + KEY_SIZE_LENGTH + VALUE_SIZE_LENGTH;
+    public static final int RECORD_OVERHEAD = HEADER_SIZE + TIMESTAMP_LENGTH + KEY_SIZE_LENGTH + VALUE_SIZE_LENGTH;
+
+    /**
+     * The "magic" values
+     */
+    public static final byte MAGIC_VALUE_V0 = 0;
+    public static final byte MAGIC_VALUE_V1 = 1;
 
     /**
      * The current "magic" value
      */
-    public static final byte CURRENT_MAGIC_VALUE = 0;
+    public static final byte CURRENT_MAGIC_VALUE = MAGIC_VALUE_V1;
 
     /**
      * Specifies the mask for the compression code. 3 bits to hold the compression codec. 0 is reserved to indicate no
@@ -63,66 +73,92 @@ public final class Record {
     public static final int COMPRESSION_CODEC_MASK = 0x07;
 
     /**
+     * Specify the mask of timestamp type.
+     * 0 for CreateTime, 1 for LogAppendTime.
+     */
+    public static final byte TIMESTAMP_TYPE_MASK = 0x08;
+    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;
+
     private final ByteBuffer buffer;
+    private final Long wrapperRecordTimestamp;
+    private final TimestampType wrapperRecordTimestampType;
 
     public Record(ByteBuffer buffer) {
         this.buffer = buffer;
+        this.wrapperRecordTimestamp = null;
+        this.wrapperRecordTimestampType = null;
+    }
+
+    // Package private constructor for inner iteration.
+    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(byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
+    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, key, value, type, valueOffset, valueSize);
+        write(this.buffer, timestamp, key, value, type, valueOffset, valueSize);
         this.buffer.rewind();
     }
 
-    public Record(byte[] key, byte[] value, CompressionType type) {
-        this(key, value, type, 0, -1);
+    public Record(long timestamp, byte[] key, byte[] value, CompressionType type) {
+        this(timestamp, key, value, type, 0, -1);
     }
 
-    public Record(byte[] value, CompressionType type) {
-        this(null, value, type);
+    public Record(long timestamp, byte[] value, CompressionType type) {
+        this(timestamp, null, value, type);
     }
 
-    public Record(byte[] key, byte[] value) {
-        this(key, value, CompressionType.NONE);
+    public Record(long timestamp, byte[] key, byte[] value) {
+        this(timestamp, key, value, CompressionType.NONE);
     }
 
-    public Record(byte[] value) {
-        this(null, 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, byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
+    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, buffer.capacity());
-        compressor.putRecord(key, value, type, valueOffset, valueSize);
+        compressor.putRecord(timestamp, key, value, type, valueOffset, valueSize);
     }
 
-    public static void write(Compressor compressor, long crc, byte attributes, byte[] key, byte[] value, int valueOffset, int valueSize) {
+    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);
@@ -145,7 +181,7 @@ public final class Record {
     }
 
     public static int recordSize(int keySize, int valueSize) {
-        return CRC_LENGTH + MAGIC_LENGTH + ATTRIBUTE_LENGTH + KEY_SIZE_LENGTH + keySize + VALUE_SIZE_LENGTH + valueSize;
+        return CRC_LENGTH + MAGIC_LENGTH + ATTRIBUTE_LENGTH + TIMESTAMP_LENGTH + KEY_SIZE_LENGTH + keySize + VALUE_SIZE_LENGTH + valueSize;
     }
 
     public ByteBuffer buffer() {
@@ -171,13 +207,14 @@ public final class Record {
     /**
      * Compute the checksum of the record from the attributes, key and value payloads
      */
-    public static long computeChecksum(byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
+    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);
@@ -240,7 +277,10 @@ public final class Record {
      * The length of the key in bytes
      */
     public int keySize() {
-        return buffer.getInt(KEY_SIZE_OFFSET);
+        if (magic() == MAGIC_VALUE_V0)
+            return buffer.getInt(KEY_SIZE_OFFSET_V0);
+        else
+            return buffer.getInt(KEY_SIZE_OFFSET_V1);
     }
 
     /**
@@ -254,7 +294,10 @@ public final class Record {
      * The position where the value size is stored
      */
     private int valueSizeOffset() {
-        return KEY_OFFSET + Math.max(0, keySize());
+        if (magic() == MAGIC_VALUE_V0)
+            return KEY_OFFSET_V0 + Math.max(0, keySize());
+        else
+            return KEY_OFFSET_V1 + Math.max(0, keySize());
     }
 
     /**
@@ -279,6 +322,35 @@ public final class Record {
     }
 
     /**
+     * When magic value is greater than 0, the timestamp of a record is determined in the following way:
+     * 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
+     */
+    public long timestamp() {
+        if (magic() == MAGIC_VALUE_V0)
+            return NO_TIMESTAMP;
+        else {
+            // case 2
+            if (wrapperRecordTimestampType == TimestampType.LOG_APPEND_TIME && wrapperRecordTimestamp != null)
+                return wrapperRecordTimestamp;
+            // Case 1, 3
+            else
+                return buffer.getLong(TIMESTAMP_OFFSET);
+        }
+    }
+
+    /**
+     * The timestamp of the message.
+     */
+    public TimestampType timestampType() {
+        if (magic() == 0)
+            return TimestampType.NO_TIMESTAMP_TYPE;
+        else
+            return wrapperRecordTimestampType == null ? TimestampType.getTimestampType(attributes()) : wrapperRecordTimestampType;
+    }
+
+    /**
      * The compression type used with this record
      */
     public CompressionType compressionType() {
@@ -296,7 +368,10 @@ public final class Record {
      * A ByteBuffer containing the message key
      */
     public ByteBuffer key() {
-        return sliceDelimited(KEY_SIZE_OFFSET);
+        if (magic() == MAGIC_VALUE_V0)
+            return sliceDelimited(KEY_SIZE_OFFSET_V0);
+        else
+            return sliceDelimited(KEY_SIZE_OFFSET_V1);
     }
 
     /**
@@ -317,13 +392,24 @@ public final class Record {
     }
 
     public String toString() {
-        return String.format("Record(magic = %d, attributes = %d, compression = %s, crc = %d, key = %d bytes, value = %d bytes)",
-                             magic(),
-                             attributes(),
-                             compressionType(),
-                             checksum(),
-                             key() == null ? 0 : key().limit(),
-                             value() == null ? 0 : value().limit());
+        if (magic() > 0)
+            return String.format("Record(magic = %d, attributes = %d, compression = %s, crc = %d, %s = %d, key = %d bytes, value = %d bytes)",
+                                 magic(),
+                                 attributes(),
+                                 compressionType(),
+                                 checksum(),
+                                 timestampType(),
+                                 timestamp(),
+                                 key() == null ? 0 : key().limit(),
+                                 value() == null ? 0 : value().limit());
+        else
+            return String.format("Record(magic = %d, attributes = %d, compression = %s, crc = %d, key = %d bytes, value = %d bytes)",
+                                 magic(),
+                                 attributes(),
+                                 compressionType(),
+                                 checksum(),
+                                 key() == null ? 0 : key().limit(),
+                                 value() == null ? 0 : value().limit());
     }
 
     public boolean equals(Object other) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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
new file mode 100644
index 0000000..ab12a35
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java
@@ -0,0 +1,62 @@
+/**
+ * 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.util.NoSuchElementException;
+
+/**
+ * The timestamp type of the records.
+ */
+public enum TimestampType {
+    NO_TIMESTAMP_TYPE(-1, "NoTimestampType"), CREATE_TIME(0, "CreateTime"), LOG_APPEND_TIME(1, "LogAppendTime");
+
+    public final int value;
+    public final String name;
+    TimestampType(int value, String name) {
+        this.value = value;
+        this.name = name;
+    }
+
+    public static TimestampType getTimestampType(byte attributes) {
+        int timestampType = (attributes & Record.TIMESTAMP_TYPE_MASK) >> Record.TIMESTAMP_TYPE_ATTRIBUTE_OFFSET;
+        return timestampType == 0 ? CREATE_TIME : LOG_APPEND_TIME;
+    }
+
+    public static byte setTimestampType(byte attributes, TimestampType timestampType) {
+        return timestampType == CREATE_TIME ?
+                (byte) (attributes & ~Record.TIMESTAMP_TYPE_MASK) : (byte) (attributes | Record.TIMESTAMP_TYPE_MASK);
+    }
+
+    public static TimestampType forName(String name) {
+        switch (name) {
+            case "NoTimestampType":
+                return NO_TIMESTAMP_TYPE;
+            case "CreateTime":
+                return CREATE_TIME;
+            case "LogAppendTime":
+                return LOG_APPEND_TIME;
+            default:
+                throw new NoSuchElementException("Invalid timestamp type " + name);
+        }
+    }
+
+    @Override
+    public String toString() {
+        return name;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
index a915247..c7d41e6 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
@@ -19,6 +19,7 @@ import org.apache.kafka.common.protocol.Errors;
 import org.apache.kafka.common.protocol.ProtoUtils;
 import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.record.Record;
 import org.apache.kafka.common.utils.CollectionUtils;
 
 import java.nio.ByteBuffer;
@@ -98,14 +99,15 @@ public class ProduceRequest extends AbstractRequest {
         Map<TopicPartition, ProduceResponse.PartitionResponse> responseMap = new HashMap<TopicPartition, ProduceResponse.PartitionResponse>();
 
         for (Map.Entry<TopicPartition, ByteBuffer> entry : partitionRecords.entrySet()) {
-            responseMap.put(entry.getKey(), new ProduceResponse.PartitionResponse(Errors.forException(e).code(), ProduceResponse.INVALID_OFFSET));
+            responseMap.put(entry.getKey(), new ProduceResponse.PartitionResponse(Errors.forException(e).code(), ProduceResponse.INVALID_OFFSET, Record.NO_TIMESTAMP));
         }
 
         switch (versionId) {
             case 0:
                 return new ProduceResponse(responseMap);
             case 1:
-                return new ProduceResponse(responseMap, ProduceResponse.DEFAULT_THROTTLE_TIME);
+            case 2:
+                return new ProduceResponse(responseMap, ProduceResponse.DEFAULT_THROTTLE_TIME, versionId);
             default:
                 throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
                         versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.PRODUCE.id)));

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
index c213332..58175e1 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
@@ -52,6 +52,7 @@ public class ProduceResponse extends AbstractRequestResponse {
      */
 
     private static final String BASE_OFFSET_KEY_NAME = "base_offset";
+    private static final String TIMESTAMP_KEY_NAME = "timestamp";
 
     private final Map<TopicPartition, PartitionResponse> responses;
     private final int throttleTime;
@@ -68,18 +69,33 @@ public class ProduceResponse extends AbstractRequestResponse {
     }
 
     /**
-     * Constructor for Version 1
+     * Constructor for the latest version
      * @param responses Produced data grouped by topic-partition
      * @param throttleTime Time in milliseconds the response was throttled
      */
     public ProduceResponse(Map<TopicPartition, PartitionResponse> responses, int throttleTime) {
-        super(new Struct(CURRENT_SCHEMA));
+        this(responses, throttleTime, ProtoUtils.latestVersion(ApiKeys.PRODUCE.id));
+    }
+
+    /**
+     * Constructor for a specific version
+     * @param responses Produced data grouped by topic-partition
+     * @param throttleTime Time in milliseconds the response was throttled
+     * @param version the version of schema to use.
+     */
+    public ProduceResponse(Map<TopicPartition, PartitionResponse> responses, int throttleTime, int version) {
+        super(new Struct(ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, version)));
         initCommonFields(responses);
-        struct.set(THROTTLE_TIME_KEY_NAME, throttleTime);
+        if (struct.hasField(THROTTLE_TIME_KEY_NAME))
+            struct.set(THROTTLE_TIME_KEY_NAME, throttleTime);
         this.responses = responses;
         this.throttleTime = throttleTime;
     }
 
+    /**
+     * Constructor from a {@link Struct}. It is the caller's responsibility to pass in a struct with the latest schema.
+     * @param struct
+     */
     public ProduceResponse(Struct struct) {
         super(struct);
         responses = new HashMap<TopicPartition, PartitionResponse>();
@@ -91,8 +107,9 @@ public class ProduceResponse extends AbstractRequestResponse {
                 int partition = partRespStruct.getInt(PARTITION_KEY_NAME);
                 short errorCode = partRespStruct.getShort(ERROR_CODE_KEY_NAME);
                 long offset = partRespStruct.getLong(BASE_OFFSET_KEY_NAME);
+                long timestamp = partRespStruct.getLong(TIMESTAMP_KEY_NAME);
                 TopicPartition tp = new TopicPartition(topic, partition);
-                responses.put(tp, new PartitionResponse(errorCode, offset));
+                responses.put(tp, new PartitionResponse(errorCode, offset, timestamp));
             }
         }
         this.throttleTime = struct.getInt(THROTTLE_TIME_KEY_NAME);
@@ -107,9 +124,12 @@ public class ProduceResponse extends AbstractRequestResponse {
             List<Struct> partitionArray = new ArrayList<Struct>();
             for (Map.Entry<Integer, PartitionResponse> partitionEntry : entry.getValue().entrySet()) {
                 PartitionResponse part = partitionEntry.getValue();
-                Struct partStruct = topicData.instance(PARTITION_RESPONSES_KEY_NAME).set(PARTITION_KEY_NAME,
-                                                                                         partitionEntry.getKey()).set(
-                    ERROR_CODE_KEY_NAME, part.errorCode).set(BASE_OFFSET_KEY_NAME, part.baseOffset);
+                Struct partStruct = topicData.instance(PARTITION_RESPONSES_KEY_NAME)
+                        .set(PARTITION_KEY_NAME, partitionEntry.getKey())
+                        .set(ERROR_CODE_KEY_NAME, part.errorCode)
+                        .set(BASE_OFFSET_KEY_NAME, part.baseOffset);
+                if (partStruct.hasField(TIMESTAMP_KEY_NAME))
+                        partStruct.set(TIMESTAMP_KEY_NAME, part.timestamp);
                 partitionArray.add(partStruct);
             }
             topicData.set(PARTITION_RESPONSES_KEY_NAME, partitionArray.toArray());
@@ -129,10 +149,12 @@ public class ProduceResponse extends AbstractRequestResponse {
     public static final class PartitionResponse {
         public short errorCode;
         public long baseOffset;
+        public long timestamp;
 
-        public PartitionResponse(short errorCode, long baseOffset) {
+        public PartitionResponse(short errorCode, long baseOffset, long timestamp) {
             this.errorCode = errorCode;
             this.baseOffset = baseOffset;
+            this.timestamp = timestamp;
         }
 
         @Override
@@ -143,6 +165,8 @@ public class ProduceResponse extends AbstractRequestResponse {
             b.append(errorCode);
             b.append(",offset: ");
             b.append(baseOffset);
+            b.append(",timestamp: ");
+            b.append(timestamp);
             b.append('}');
             return b.toString();
         }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java b/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java
index 5b86700..caa0058 100644
--- a/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java
@@ -131,6 +131,17 @@ public class Crc32 implements Checksum {
         update((byte) input /* >> 0 */);
     }
 
+    final public void updateLong(long input) {
+        update((byte) (input >> 56));
+        update((byte) (input >> 48));
+        update((byte) (input >> 40));
+        update((byte) (input >> 32));
+        update((byte) (input >> 24));
+        update((byte) (input >> 16));
+        update((byte) (input >> 8));
+        update((byte) input /* >> 0 */);
+    }
+
     /*
      * CRC-32 lookup tables generated by the polynomial 0xEDB88320. See also TestPureJavaCrc32.Table.
      */


[4/5] kafka git commit: KAFKA-3025; Added timetamp to Message and use relative offset.

Posted by ju...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java
index fa06be9..3ef5c8b 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java
@@ -18,6 +18,7 @@ package org.apache.kafka.clients.consumer;
 
 import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener;
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.record.TimestampType;
 import org.junit.Test;
 
 import java.util.Arrays;
@@ -42,8 +43,8 @@ public class MockConsumerTest {
         beginningOffsets.put(new TopicPartition("test", 1), 0L);
         consumer.updateBeginningOffsets(beginningOffsets);
         consumer.seek(new TopicPartition("test", 0), 0);
-        ConsumerRecord<String, String> rec1 = new ConsumerRecord<String, String>("test", 0, 0, "key1", "value1");
-        ConsumerRecord<String, String> rec2 = new ConsumerRecord<String, String>("test", 0, 1, "key2", "value2");
+        ConsumerRecord<String, String> rec1 = new ConsumerRecord<String, String>("test", 0, 0, 0L, TimestampType.CREATE_TIME, "key1", "value1");
+        ConsumerRecord<String, String> rec2 = new ConsumerRecord<String, String>("test", 0, 1, 0L, TimestampType.CREATE_TIME, "key2", "value2");
         consumer.addRecord(rec1);
         consumer.addRecord(rec2);
         ConsumerRecords<String, String> recs = consumer.poll(1);

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerInterceptorsTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerInterceptorsTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerInterceptorsTest.java
index 45210a8..25843c7 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerInterceptorsTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerInterceptorsTest.java
@@ -23,6 +23,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecords;
 import org.apache.kafka.clients.consumer.OffsetAndMetadata;
 import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.record.TimestampType;
 import org.junit.Test;
 
 import java.util.ArrayList;
@@ -42,7 +43,8 @@ public class ConsumerInterceptorsTest {
     private final TopicPartition tp = new TopicPartition(topic, partition);
     private final TopicPartition filterTopicPart1 = new TopicPartition("test5", filterPartition1);
     private final TopicPartition filterTopicPart2 = new TopicPartition("test6", filterPartition2);
-    private final ConsumerRecord<Integer, Integer> consumerRecord = new ConsumerRecord<>(topic, partition, 0, 1, 1);
+    private final ConsumerRecord<Integer, Integer> consumerRecord =
+        new ConsumerRecord<>(topic, partition, 0, 0L, TimestampType.CREATE_TIME, 1, 1);
     private int onCommitCount = 0;
     private int onConsumeCount = 0;
 
@@ -115,9 +117,9 @@ public class ConsumerInterceptorsTest {
         List<ConsumerRecord<Integer, Integer>> list1 = new ArrayList<>();
         list1.add(consumerRecord);
         List<ConsumerRecord<Integer, Integer>> list2 = new ArrayList<>();
-        list2.add(new ConsumerRecord<>(filterTopicPart1.topic(), filterTopicPart1.partition(), 0, 1, 1));
+        list2.add(new ConsumerRecord<>(filterTopicPart1.topic(), filterTopicPart1.partition(), 0, 0L, TimestampType.CREATE_TIME, 1, 1));
         List<ConsumerRecord<Integer, Integer>> list3 = new ArrayList<>();
-        list3.add(new ConsumerRecord<>(filterTopicPart2.topic(), filterTopicPart2.partition(), 0, 1, 1));
+        list3.add(new ConsumerRecord<>(filterTopicPart2.topic(), filterTopicPart2.partition(), 0, 0L, TimestampType.CREATE_TIME, 1, 1));
         records.put(tp, list1);
         records.put(filterTopicPart1, list2);
         records.put(filterTopicPart2, list3);

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 5e750fd..97c3d85 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
@@ -39,6 +39,7 @@ import org.apache.kafka.common.protocol.Errors;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.record.CompressionType;
 import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.record.Record;
 import org.apache.kafka.common.requests.FetchResponse;
 import org.apache.kafka.common.requests.ListOffsetRequest;
 import org.apache.kafka.common.requests.ListOffsetResponse;
@@ -95,9 +96,9 @@ public class FetcherTest {
         metadata.update(cluster, time.milliseconds());
         client.setNode(node);
 
-        records.append(1L, "key".getBytes(), "value-1".getBytes());
-        records.append(2L, "key".getBytes(), "value-2".getBytes());
-        records.append(3L, "key".getBytes(), "value-3".getBytes());
+        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();
     }
 
@@ -133,9 +134,9 @@ public class FetcherTest {
         // 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, "key".getBytes(), "value-1".getBytes());
-        records.append(20L, "key".getBytes(), "value-2".getBytes());
-        records.append(30L, "key".getBytes(), "value-3".getBytes());
+        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();
 
         List<ConsumerRecord<byte[], byte[]>> consumerRecords;
@@ -164,7 +165,7 @@ public class FetcherTest {
         MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE);
         byte[] bytes = new byte[this.fetchSize];
         new Random().nextBytes(bytes);
-        records.append(1L, null, bytes);
+        records.append(1L, 0L, null, bytes);
         records.close();
 
         // resize the limit of the buffer to pretend it is only fetch-size large
@@ -469,7 +470,7 @@ public class FetcherTest {
             if (i > 1) {
                 this.records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE);
                 for (int v = 0; v < 3; v++) {
-                    this.records.append((long) i * 3 + v, "key".getBytes(), String.format("value-%d", v).getBytes());
+                    this.records.append((long) i * 3 + v, Record.NO_TIMESTAMP, "key".getBytes(), String.format("value-%d", v).getBytes());
                 }
                 this.records.close();
             }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java
index 7bb181e..f3db098 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java
@@ -45,7 +45,7 @@ public class ProducerRecordTest {
         ProducerRecord<String, Integer> valueMisMatch = new ProducerRecord<String, Integer>("test", 1 , "key", 2);
         assertFalse(producerRecord.equals(valueMisMatch));
 
-        ProducerRecord<String, Integer> nullFieldsRecord = new ProducerRecord<String, Integer>("topic", null, null, null);
+        ProducerRecord<String, Integer> nullFieldsRecord = new ProducerRecord<String, Integer>("topic", null, null, null, null);
         assertEquals(nullFieldsRecord, nullFieldsRecord);
         assertEquals(nullFieldsRecord.hashCode(), nullFieldsRecord.hashCode());
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java
index 1e5d1c2..5591129 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java
@@ -30,6 +30,7 @@ import org.apache.kafka.clients.producer.internals.FutureRecordMetadata;
 import org.apache.kafka.clients.producer.internals.ProduceRequestResult;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.record.Record;
 import org.junit.Test;
 
 public class RecordSendTest {
@@ -44,7 +45,7 @@ public class RecordSendTest {
     @Test
     public void testTimeout() throws Exception {
         ProduceRequestResult request = new ProduceRequestResult();
-        FutureRecordMetadata future = new FutureRecordMetadata(request, relOffset);
+        FutureRecordMetadata future = new FutureRecordMetadata(request, relOffset, Record.NO_TIMESTAMP);
         assertFalse("Request is not completed", future.isDone());
         try {
             future.get(5, TimeUnit.MILLISECONDS);
@@ -62,7 +63,7 @@ public class RecordSendTest {
      */
     @Test(expected = ExecutionException.class)
     public void testError() throws Exception {
-        FutureRecordMetadata future = new FutureRecordMetadata(asyncRequest(baseOffset, new CorruptRecordException(), 50L), relOffset);
+        FutureRecordMetadata future = new FutureRecordMetadata(asyncRequest(baseOffset, new CorruptRecordException(), 50L), relOffset, Record.NO_TIMESTAMP);
         future.get();
     }
 
@@ -71,7 +72,7 @@ public class RecordSendTest {
      */
     @Test
     public void testBlocking() throws Exception {
-        FutureRecordMetadata future = new FutureRecordMetadata(asyncRequest(baseOffset, null, 50L), relOffset);
+        FutureRecordMetadata future = new FutureRecordMetadata(asyncRequest(baseOffset, null, 50L), relOffset, Record.NO_TIMESTAMP);
         assertEquals(baseOffset + relOffset, future.get().offset());
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java
index 18a455f..26d15d0 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java
@@ -128,7 +128,7 @@ public class ProducerInterceptorsTest {
         ProducerInterceptors<Integer, String> interceptors = new ProducerInterceptors<>(interceptorList);
 
         // verify onAck is called on all interceptors
-        RecordMetadata meta = new RecordMetadata(tp, 0, 0);
+        RecordMetadata meta = new RecordMetadata(tp, 0, 0, 0);
         interceptors.onAcknowledgement(meta, null);
         assertEquals(2, onAckCount);
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 723e450..0f95ee5 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
@@ -77,10 +77,10 @@ public class RecordAccumulatorTest {
         RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 10L, 100L, metrics, time);
         int appends = 1024 / msgSize;
         for (int i = 0; i < appends; i++) {
-            accum.append(tp1, key, value, null, maxBlockTimeMs);
+            accum.append(tp1, 0L, key, value, null, maxBlockTimeMs);
             assertEquals("No partitions should be ready.", 0, accum.ready(cluster, now).readyNodes.size());
         }
-        accum.append(tp1, key, value, null, maxBlockTimeMs);
+        accum.append(tp1, 0L, key, value, null, maxBlockTimeMs);
         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());
@@ -99,7 +99,7 @@ public class RecordAccumulatorTest {
     public void testAppendLarge() throws Exception {
         int batchSize = 512;
         RecordAccumulator accum = new RecordAccumulator(batchSize, 10 * 1024, CompressionType.NONE, 0L, 100L, metrics, time);
-        accum.append(tp1, key, new byte[2 * batchSize], null, maxBlockTimeMs);
+        accum.append(tp1, 0L, key, new byte[2 * batchSize], null, maxBlockTimeMs);
         assertEquals("Our partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes);
     }
 
@@ -107,7 +107,7 @@ public class RecordAccumulatorTest {
     public void testLinger() throws Exception {
         long lingerMs = 10L;
         RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, lingerMs, 100L, metrics, time);
-        accum.append(tp1, key, value, null, maxBlockTimeMs);
+        accum.append(tp1, 0L, key, value, null, maxBlockTimeMs);
         assertEquals("No partitions should be ready", 0, accum.ready(cluster, time.milliseconds()).readyNodes.size());
         time.sleep(10);
         assertEquals("Our partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes);
@@ -129,7 +129,7 @@ public class RecordAccumulatorTest {
         List<TopicPartition> partitions = asList(tp1, tp2);
         for (TopicPartition tp : partitions) {
             for (int i = 0; i < appends; i++)
-                accum.append(tp, key, value, null, maxBlockTimeMs);
+                accum.append(tp, 0L, key, value, null, maxBlockTimeMs);
         }
         assertEquals("Partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes);
 
@@ -150,7 +150,7 @@ public class RecordAccumulatorTest {
                 public void run() {
                     for (int i = 0; i < msgs; i++) {
                         try {
-                            accum.append(new TopicPartition(topic, i % numParts), key, value, null, maxBlockTimeMs);
+                            accum.append(new TopicPartition(topic, i % numParts), 0L, key, value, null, maxBlockTimeMs);
                         } catch (Exception e) {
                             e.printStackTrace();
                         }
@@ -189,7 +189,7 @@ public class RecordAccumulatorTest {
 
         // Partition on node1 only
         for (int i = 0; i < appends; i++)
-            accum.append(tp1, key, value, null, maxBlockTimeMs);
+            accum.append(tp1, 0L, key, value, null, maxBlockTimeMs);
         RecordAccumulator.ReadyCheckResult result = accum.ready(cluster, time.milliseconds());
         assertEquals("No nodes should be ready.", 0, result.readyNodes.size());
         assertEquals("Next check time should be the linger time", lingerMs, result.nextReadyCheckDelayMs);
@@ -198,14 +198,14 @@ public class RecordAccumulatorTest {
 
         // Add partition on node2 only
         for (int i = 0; i < appends; i++)
-            accum.append(tp3, key, value, null, maxBlockTimeMs);
+            accum.append(tp3, 0L, key, value, null, maxBlockTimeMs);
         result = accum.ready(cluster, time.milliseconds());
         assertEquals("No nodes should be ready.", 0, result.readyNodes.size());
         assertEquals("Next check time should be defined by node1, half remaining linger time", lingerMs / 2, result.nextReadyCheckDelayMs);
 
         // Add data for another partition on node1, enough to make data sendable immediately
         for (int i = 0; i < appends + 1; i++)
-            accum.append(tp2, key, value, null, maxBlockTimeMs);
+            accum.append(tp2, 0L, key, value, null, maxBlockTimeMs);
         result = accum.ready(cluster, time.milliseconds());
         assertEquals("Node1 should be ready", Collections.singleton(node1), result.readyNodes);
         // Note this can actually be < linger time because it may use delays from partitions that aren't sendable
@@ -220,7 +220,7 @@ public class RecordAccumulatorTest {
         final RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, lingerMs, retryBackoffMs, metrics, time);
 
         long now = time.milliseconds();
-        accum.append(tp1, key, value, null, maxBlockTimeMs);
+        accum.append(tp1, 0L, key, value, null, maxBlockTimeMs);
         RecordAccumulator.ReadyCheckResult result = accum.ready(cluster, now + lingerMs + 1);
         assertEquals("Node1 should be ready", Collections.singleton(node1), result.readyNodes);
         Map<Integer, List<RecordBatch>> batches = accum.drain(cluster, result.readyNodes, Integer.MAX_VALUE, now + lingerMs + 1);
@@ -232,7 +232,7 @@ public class RecordAccumulatorTest {
         accum.reenqueue(batches.get(0).get(0), now);
 
         // Put message for partition 1 into accumulator
-        accum.append(tp2, key, value, null, maxBlockTimeMs);
+        accum.append(tp2, 0L, key, value, null, maxBlockTimeMs);
         result = accum.ready(cluster, now + lingerMs + 1);
         assertEquals("Node1 should be ready", Collections.singleton(node1), result.readyNodes);
 
@@ -256,7 +256,7 @@ public class RecordAccumulatorTest {
         long lingerMs = Long.MAX_VALUE;
         final RecordAccumulator accum = new RecordAccumulator(4 * 1024, 64 * 1024, CompressionType.NONE, lingerMs, 100L, metrics, time);
         for (int i = 0; i < 100; i++)
-            accum.append(new TopicPartition(topic, i % 3), key, value, null, maxBlockTimeMs);
+            accum.append(new TopicPartition(topic, i % 3), 0L, key, value, null, maxBlockTimeMs);
         RecordAccumulator.ReadyCheckResult result = accum.ready(cluster, time.milliseconds());
         assertEquals("No nodes should be ready.", 0, result.readyNodes.size());
         
@@ -287,7 +287,7 @@ public class RecordAccumulatorTest {
             }
         }
         for (int i = 0; i < 100; i++)
-            accum.append(new TopicPartition(topic, i % 3), key, value, new TestCallback(), maxBlockTimeMs);
+            accum.append(new TopicPartition(topic, i % 3), 0L, key, value, new TestCallback(), maxBlockTimeMs);
         RecordAccumulator.ReadyCheckResult result = accum.ready(cluster, time.milliseconds());
         assertEquals("No nodes should be ready.", 0, result.readyNodes.size());
 
@@ -304,12 +304,12 @@ public class RecordAccumulatorTest {
         RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 10, 100L, metrics, time);
         int appends = 1024 / msgSize;
         for (int i = 0; i < appends; i++) {
-            accum.append(tp1, key, value, null, maxBlockTimeMs);
+            accum.append(tp1, 0L, key, value, null, maxBlockTimeMs);
             assertEquals("No partitions should be ready.", 0, accum.ready(cluster, now).readyNodes.size());
         }
         time.sleep(2000);
         accum.ready(cluster, now);
-        accum.append(tp1, key, value, null, 0);
+        accum.append(tp1, 0L, key, value, null, 0);
         Set<Node> readyNodes = accum.ready(cluster, time.milliseconds()).readyNodes;
         assertEquals("Our partition's leader should be ready", Collections.singleton(node1), readyNodes);
         Cluster cluster = new Cluster(new ArrayList<Node>(), new ArrayList<PartitionInfo>(), Collections.<String>emptySet());

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java
index 14a839b..b983de5 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java
@@ -36,6 +36,7 @@ import org.apache.kafka.common.metrics.Metrics;
 import org.apache.kafka.common.protocol.Errors;
 import org.apache.kafka.common.protocol.types.Struct;
 import org.apache.kafka.common.record.CompressionType;
+import org.apache.kafka.common.record.Record;
 import org.apache.kafka.common.requests.ProduceResponse;
 import org.apache.kafka.common.utils.MockTime;
 import org.apache.kafka.test.TestUtils;
@@ -93,7 +94,7 @@ public class SenderTest {
     @Test
     public void testSimple() throws Exception {
         long offset = 0;
-        Future<RecordMetadata> future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), null, MAX_BLOCK_TIMEOUT).future;
+        Future<RecordMetadata> future = accumulator.append(tp, 0L, "key".getBytes(), "value".getBytes(), null, MAX_BLOCK_TIMEOUT).future;
         sender.run(time.milliseconds()); // connect
         sender.run(time.milliseconds()); // send produce request
         assertEquals("We should have a single produce request in flight.", 1, client.inFlightRequestCount());
@@ -112,7 +113,7 @@ public class SenderTest {
     public void testQuotaMetrics() throws Exception {
         final long offset = 0;
         for (int i = 1; i <= 3; i++) {
-            Future<RecordMetadata> future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), null, MAX_BLOCK_TIMEOUT).future;
+            Future<RecordMetadata> future = accumulator.append(tp, 0L, "key".getBytes(), "value".getBytes(), null, MAX_BLOCK_TIMEOUT).future;
             sender.run(time.milliseconds()); // send produce request
             client.respond(produceResponse(tp, offset, Errors.NONE.code(), 100 * i));
             sender.run(time.milliseconds());
@@ -141,7 +142,7 @@ public class SenderTest {
                                        "clientId",
                                        REQUEST_TIMEOUT);
             // do a successful retry
-            Future<RecordMetadata> future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), null, MAX_BLOCK_TIMEOUT).future;
+            Future<RecordMetadata> future = accumulator.append(tp, 0L, "key".getBytes(), "value".getBytes(), null, MAX_BLOCK_TIMEOUT).future;
             sender.run(time.milliseconds()); // connect
             sender.run(time.milliseconds()); // send produce request
             String id = client.requests().peek().request().destination();
@@ -162,7 +163,7 @@ public class SenderTest {
             assertEquals(offset, future.get().offset());
 
             // do an unsuccessful retry
-            future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), null, MAX_BLOCK_TIMEOUT).future;
+            future = accumulator.append(tp, 0L, "key".getBytes(), "value".getBytes(), null, MAX_BLOCK_TIMEOUT).future;
             sender.run(time.milliseconds()); // send produce request
             for (int i = 0; i < maxRetries + 1; i++) {
                 client.disconnect(client.requests().peek().request().destination());
@@ -188,7 +189,7 @@ public class SenderTest {
     }
 
     private Struct produceResponse(TopicPartition tp, long offset, int error, int throttleTimeMs) {
-        ProduceResponse.PartitionResponse resp = new ProduceResponse.PartitionResponse((short) error, offset);
+        ProduceResponse.PartitionResponse resp = new ProduceResponse.PartitionResponse((short) error, offset, Record.NO_TIMESTAMP);
         Map<TopicPartition, ProduceResponse.PartitionResponse> partResp = Collections.singletonMap(tp, resp);
         ProduceResponse response = new ProduceResponse(partResp, throttleTimeMs);
         return response.toStruct();

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 6e3a9ac..ed64f63 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
@@ -45,13 +45,13 @@ public class MemoryRecordsTest {
     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("a".getBytes(), "1".getBytes()),
-                                          new Record("b".getBytes(), "2".getBytes()),
-                                          new Record("c".getBytes(), "3".getBytes()));
+        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()));
         for (int i = 0; i < list.size(); i++) {
             Record r = list.get(i);
             recs1.append(i, r);
-            recs2.append(i, toArray(r.key()), toArray(r.value()));
+            recs2.append(i, 0L, toArray(r.key()), toArray(r.value()));
         }
         recs1.close();
         recs2.close();

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 957fc8f..6482529 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,16 +35,18 @@ import org.junit.runners.Parameterized.Parameters;
 @RunWith(value = Parameterized.class)
 public class RecordTest {
 
+    private long timestamp;
     private ByteBuffer key;
     private ByteBuffer value;
     private CompressionType compression;
     private Record record;
 
-    public RecordTest(byte[] key, byte[] value, CompressionType compression) {
+    public RecordTest(long timestamp, byte[] key, byte[] value, CompressionType compression) {
+        this.timestamp = timestamp;
         this.key = key == null ? null : ByteBuffer.wrap(key);
         this.value = value == null ? null : ByteBuffer.wrap(value);
         this.compression = compression;
-        this.record = new Record(key, value, compression);
+        this.record = new Record(timestamp, key, value, compression);
     }
 
     @Test
@@ -64,6 +66,7 @@ public class RecordTest {
     public void testChecksum() {
         assertEquals(record.checksum(), record.computeChecksum());
         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));
@@ -99,10 +102,11 @@ public class RecordTest {
         byte[] payload = new byte[1000];
         Arrays.fill(payload, (byte) 1);
         List<Object[]> values = new ArrayList<Object[]>();
-        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[] {key, value, compression});
+        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});
         return values;
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
index 5fc5551..edeaf63 100644
--- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
@@ -23,6 +23,7 @@ import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
 import org.apache.kafka.common.protocol.ProtoUtils;
 import org.apache.kafka.common.protocol.SecurityProtocol;
+import org.apache.kafka.common.record.Record;
 import org.junit.Test;
 
 import java.lang.reflect.Method;
@@ -82,7 +83,7 @@ public class RequestResponseTest {
                 createOffsetFetchRequest().getErrorResponse(0, new UnknownServerException()),
                 createOffsetFetchResponse(),
                 createProduceRequest(),
-                createProduceRequest().getErrorResponse(1, new UnknownServerException()),
+                createProduceRequest().getErrorResponse(2, new UnknownServerException()),
                 createProduceResponse(),
                 createStopReplicaRequest(),
                 createStopReplicaRequest().getErrorResponse(0, new UnknownServerException()),
@@ -122,16 +123,19 @@ public class RequestResponseTest {
     @Test
     public void produceResponseVersionTest() {
         Map<TopicPartition, ProduceResponse.PartitionResponse> responseData = new HashMap<TopicPartition, ProduceResponse.PartitionResponse>();
-        responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE.code(), 10000));
-
+        responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE.code(), 10000, Record.NO_TIMESTAMP));
         ProduceResponse v0Response = new ProduceResponse(responseData);
-        ProduceResponse v1Response = new ProduceResponse(responseData, 10);
+        ProduceResponse v1Response = new ProduceResponse(responseData, 10, 1);
+        ProduceResponse v2Response = new ProduceResponse(responseData, 10, 2);
         assertEquals("Throttle time must be zero", 0, v0Response.getThrottleTime());
         assertEquals("Throttle time must be 10", 10, v1Response.getThrottleTime());
+        assertEquals("Throttle time must be 10", 10, v2Response.getThrottleTime());
         assertEquals("Should use schema version 0", ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 0), v0Response.toStruct().schema());
         assertEquals("Should use schema version 1", ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 1), v1Response.toStruct().schema());
+        assertEquals("Should use schema version 2", ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, 2), v2Response.toStruct().schema());
         assertEquals("Response data does not match", responseData, v0Response.responses());
         assertEquals("Response data does not match", responseData, v1Response.responses());
+        assertEquals("Response data does not match", responseData, v2Response.responses());
     }
 
     @Test
@@ -316,7 +320,7 @@ public class RequestResponseTest {
 
     private AbstractRequestResponse createProduceResponse() {
         Map<TopicPartition, ProduceResponse.PartitionResponse> responseData = new HashMap<TopicPartition, ProduceResponse.PartitionResponse>();
-        responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE.code(), 10000));
+        responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE.code(), 10000, Record.NO_TIMESTAMP));
         return new ProduceResponse(responseData, 0);
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java b/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java
index 8295b54..3246578 100644
--- a/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java
+++ b/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java
@@ -25,6 +25,7 @@ import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.record.TimestampType;
 
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -55,7 +56,8 @@ public class MockConsumerInterceptor implements ConsumerInterceptor<String, Stri
         for (TopicPartition tp : records.partitions()) {
             List<ConsumerRecord<String, String>> lst = new ArrayList<>();
             for (ConsumerRecord<String, String> record: records.records(tp)) {
-                lst.add(new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), record.key(), record.value().toUpperCase()));
+                lst.add(new ConsumerRecord<>(record.topic(), record.partition(), record.offset(),
+                    0L, TimestampType.CREATE_TIME, record.key(), record.value().toUpperCase()));
             }
             recordMap.put(tp, lst);
         }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java
----------------------------------------------------------------------
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java
index 04b08b3..978e3a1 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java
@@ -23,6 +23,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecords;
 import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.kafka.clients.consumer.OffsetAndMetadata;
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.utils.Time;
 import org.apache.kafka.connect.data.Schema;
 import org.apache.kafka.connect.data.SchemaAndValue;
@@ -290,7 +291,7 @@ public class WorkerSinkTaskTest {
                     public ConsumerRecords<byte[], byte[]> answer() throws Throwable {
                         List<ConsumerRecord<byte[], byte[]>> records = new ArrayList<>();
                         for (int i = 0; i < numMessages; i++)
-                            records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned + i, RAW_KEY, RAW_VALUE));
+                            records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned + i, 0L, TimestampType.CREATE_TIME, RAW_KEY, RAW_VALUE));
                         recordsReturned += numMessages;
                         return new ConsumerRecords<>(
                                 numMessages > 0 ?

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java
----------------------------------------------------------------------
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java
index 3bf653e..e202209 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java
@@ -24,6 +24,7 @@ import org.apache.kafka.clients.consumer.KafkaConsumer;
 import org.apache.kafka.clients.consumer.OffsetAndMetadata;
 import org.apache.kafka.clients.consumer.OffsetCommitCallback;
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.utils.Time;
 import org.apache.kafka.connect.data.Schema;
 import org.apache.kafka.connect.data.SchemaAndValue;
@@ -519,7 +520,7 @@ public class WorkerSinkTaskThreadedTest extends ThreadedTest {
                                 Collections.singletonMap(
                                         new TopicPartition(TOPIC, PARTITION),
                                         Arrays.asList(
-                                                new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, RAW_KEY, RAW_VALUE)
+                                                new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, 0L, TimestampType.CREATE_TIME, RAW_KEY, RAW_VALUE)
                                         )));
                         recordsReturned++;
                         return records;
@@ -547,7 +548,7 @@ public class WorkerSinkTaskThreadedTest extends ThreadedTest {
                                 Collections.singletonMap(
                                         new TopicPartition(TOPIC, PARTITION),
                                         Arrays.asList(
-                                                new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, RAW_KEY, RAW_VALUE)
+                                                new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, 0L, TimestampType.CREATE_TIME, RAW_KEY, RAW_VALUE)
                                         )));
                         recordsReturned++;
                         return records;

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java
----------------------------------------------------------------------
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java
index 59e9b86..c6eb0c5 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java
@@ -356,7 +356,7 @@ public class WorkerSourceTaskTest extends ThreadedTest {
             public Future<RecordMetadata> answer() throws Throwable {
                 synchronized (producerCallbacks) {
                     for (org.apache.kafka.clients.producer.Callback cb : producerCallbacks.getValues()) {
-                        cb.onCompletion(new RecordMetadata(new TopicPartition("foo", 0), 0, 0), null);
+                        cb.onCompletion(new RecordMetadata(new TopicPartition("foo", 0), 0, 0, 0L), null);
                     }
                     producerCallbacks.reset();
                 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigStorageTest.java
----------------------------------------------------------------------
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigStorageTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigStorageTest.java
index e007f02..e878e12 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigStorageTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigStorageTest.java
@@ -22,6 +22,7 @@ import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.connect.data.Field;
 import org.apache.kafka.connect.data.Schema;
 import org.apache.kafka.connect.data.SchemaAndValue;
@@ -288,14 +289,14 @@ public class KafkaConfigStorageTest {
         expectConfigure();
         // Overwrite each type at least once to ensure we see the latest data after loading
         List<ConsumerRecord<String, byte[]>> existingRecords = Arrays.asList(
-                new ConsumerRecord<>(TOPIC, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)),
-                new ConsumerRecord<>(TOPIC, 0, 1, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(1)),
-                new ConsumerRecord<>(TOPIC, 0, 2, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)),
-                new ConsumerRecord<>(TOPIC, 0, 3, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(3)),
-                new ConsumerRecord<>(TOPIC, 0, 4, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(4)),
+                new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)),
+                new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(1)),
+                new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)),
+                new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(3)),
+                new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CREATE_TIME, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(4)),
                 // Connector after root update should make it through, task update shouldn't
-                new ConsumerRecord<>(TOPIC, 0, 5, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5)),
-                new ConsumerRecord<>(TOPIC, 0, 6, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(6)));
+                new ConsumerRecord<>(TOPIC, 0, 5, 0L, TimestampType.CREATE_TIME, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5)),
+                new ConsumerRecord<>(TOPIC, 0, 6, 0L, TimestampType.CREATE_TIME, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(6)));
         LinkedHashMap<byte[], Struct> deserialized = new LinkedHashMap();
         deserialized.put(CONFIGS_SERIALIZED.get(0), CONNECTOR_CONFIG_STRUCTS.get(0));
         deserialized.put(CONFIGS_SERIALIZED.get(1), TASK_CONFIG_STRUCTS.get(0));
@@ -342,12 +343,12 @@ public class KafkaConfigStorageTest {
 
         expectConfigure();
         List<ConsumerRecord<String, byte[]>> existingRecords = Arrays.asList(
-                new ConsumerRecord<>(TOPIC, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)),
+                new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)),
                 // This is the record that has been compacted:
                 //new ConsumerRecord<>(TOPIC, 0, 1, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(1)),
-                new ConsumerRecord<>(TOPIC, 0, 2, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)),
-                new ConsumerRecord<>(TOPIC, 0, 4, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(4)),
-                new ConsumerRecord<>(TOPIC, 0, 5, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5)));
+                new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)),
+                new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CREATE_TIME, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(4)),
+                new ConsumerRecord<>(TOPIC, 0, 5, 0L, TimestampType.CREATE_TIME, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5)));
         LinkedHashMap<byte[], Struct> deserialized = new LinkedHashMap();
         deserialized.put(CONFIGS_SERIALIZED.get(0), CONNECTOR_CONFIG_STRUCTS.get(0));
         deserialized.put(CONFIGS_SERIALIZED.get(2), TASK_CONFIG_STRUCTS.get(0));
@@ -483,7 +484,7 @@ public class KafkaConfigStorageTest {
                     public Future<Void> answer() throws Throwable {
                         TestFuture<Void> future = new TestFuture<Void>();
                         for (Map.Entry<String, byte[]> entry : serializedConfigs.entrySet())
-                            capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, logOffset++, entry.getKey(), entry.getValue()));
+                            capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, logOffset++, 0L, TimestampType.CREATE_TIME, entry.getKey(), entry.getValue()));
                         future.resolveOnGet((Void) null);
                         return future;
                     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java
----------------------------------------------------------------------
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java
index 4e54bf1..61763a8 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java
@@ -22,6 +22,7 @@ import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.connect.errors.ConnectException;
 import org.apache.kafka.connect.util.Callback;
 import org.apache.kafka.connect.util.KafkaBasedLog;
@@ -125,10 +126,10 @@ public class KafkaOffsetBackingStoreTest {
     public void testReloadOnStart() throws Exception {
         expectConfigure();
         expectStart(Arrays.asList(
-                new ConsumerRecord<>(TOPIC, 0, 0, TP0_KEY.array(), TP0_VALUE.array()),
-                new ConsumerRecord<>(TOPIC, 1, 0, TP1_KEY.array(), TP1_VALUE.array()),
-                new ConsumerRecord<>(TOPIC, 0, 1, TP0_KEY.array(), TP0_VALUE_NEW.array()),
-                new ConsumerRecord<>(TOPIC, 1, 1, TP1_KEY.array(), TP1_VALUE_NEW.array())
+                new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, TP0_KEY.array(), TP0_VALUE.array()),
+                new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, TP1_KEY.array(), TP1_VALUE.array()),
+                new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, TP0_KEY.array(), TP0_VALUE_NEW.array()),
+                new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CREATE_TIME, TP1_KEY.array(), TP1_VALUE_NEW.array())
         ));
         expectStop();
 
@@ -176,8 +177,8 @@ public class KafkaOffsetBackingStoreTest {
         PowerMock.expectLastCall().andAnswer(new IAnswer<Object>() {
             @Override
             public Object answer() throws Throwable {
-                capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 0, TP0_KEY.array(), TP0_VALUE.array()));
-                capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 0, TP1_KEY.array(), TP1_VALUE.array()));
+                capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, TP0_KEY.array(), TP0_VALUE.array()));
+                capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, TP1_KEY.array(), TP1_VALUE.array()));
                 secondGetReadToEndCallback.getValue().onCompletion(null, null);
                 return null;
             }
@@ -189,8 +190,8 @@ public class KafkaOffsetBackingStoreTest {
         PowerMock.expectLastCall().andAnswer(new IAnswer<Object>() {
             @Override
             public Object answer() throws Throwable {
-                capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 1, TP0_KEY.array(), TP0_VALUE_NEW.array()));
-                capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 1, TP1_KEY.array(), TP1_VALUE_NEW.array()));
+                capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, TP0_KEY.array(), TP0_VALUE_NEW.array()));
+                capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CREATE_TIME, TP1_KEY.array(), TP1_VALUE_NEW.array()));
                 thirdGetReadToEndCallback.getValue().onCompletion(null, null);
                 return null;
             }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java
----------------------------------------------------------------------
diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java
index ab370e3..b2246f5 100644
--- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java
+++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java
@@ -32,6 +32,7 @@ import org.apache.kafka.common.PartitionInfo;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.errors.LeaderNotAvailableException;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.record.TimestampType;
 import org.apache.kafka.common.utils.Time;
 import org.easymock.Capture;
 import org.easymock.EasyMock;
@@ -182,7 +183,7 @@ public class KafkaBasedLogTest {
                 consumer.schedulePollTask(new Runnable() {
                     @Override
                     public void run() {
-                        consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, TP0_KEY, TP0_VALUE));
+                        consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, TP0_KEY, TP0_VALUE));
                     }
                 });
                 consumer.scheduleNopPollTask();
@@ -190,7 +191,7 @@ public class KafkaBasedLogTest {
                 consumer.schedulePollTask(new Runnable() {
                     @Override
                     public void run() {
-                        consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, TP1_KEY, TP1_VALUE));
+                        consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, TP1_KEY, TP1_VALUE));
                     }
                 });
                 consumer.schedulePollTask(new Runnable() {
@@ -297,16 +298,16 @@ public class KafkaBasedLogTest {
                 consumer.schedulePollTask(new Runnable() {
                     @Override
                     public void run() {
-                        consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, TP0_KEY, TP0_VALUE));
-                        consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 1, TP0_KEY, TP0_VALUE_NEW));
-                        consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, TP1_KEY, TP1_VALUE));
+                        consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, TP0_KEY, TP0_VALUE));
+                        consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, TP0_KEY, TP0_VALUE_NEW));
+                        consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, TP1_KEY, TP1_VALUE));
                     }
                 });
 
                 consumer.schedulePollTask(new Runnable() {
                     @Override
                     public void run() {
-                        consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 1, TP1_KEY, TP1_VALUE_NEW));
+                        consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CREATE_TIME, TP1_KEY, TP1_VALUE_NEW));
                     }
                 });
 
@@ -362,8 +363,8 @@ public class KafkaBasedLogTest {
                 consumer.schedulePollTask(new Runnable() {
                     @Override
                     public void run() {
-                        consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, TP0_KEY, TP0_VALUE_NEW));
-                        consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, TP0_KEY, TP0_VALUE_NEW));
+                        consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, TP0_KEY, TP0_VALUE_NEW));
+                        consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, TP0_KEY, TP0_VALUE_NEW));
                     }
                 });
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/main/scala/kafka/admin/ConfigCommand.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala
index 82a6612..276689a 100644
--- a/core/src/main/scala/kafka/admin/ConfigCommand.scala
+++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala
@@ -17,18 +17,18 @@
 
 package kafka.admin
 
-import joptsimple._
 import java.util.Properties
+
+import joptsimple._
 import kafka.admin.TopicCommand._
-import kafka.consumer.ConsumerConfig
 import kafka.log.{Defaults, LogConfig}
 import kafka.server.{ClientConfigOverride, ConfigType}
-import kafka.utils.{ZkUtils, CommandLineUtils}
-import org.I0Itec.zkclient.ZkClient
-import scala.collection._
-import scala.collection.JavaConversions._
-import org.apache.kafka.common.utils.Utils
+import kafka.utils.{CommandLineUtils, ZkUtils}
 import org.apache.kafka.common.security.JaasUtils
+import org.apache.kafka.common.utils.Utils
+
+import scala.collection.JavaConversions._
+import scala.collection._
 
 
 /**
@@ -117,6 +117,10 @@ object ConfigCommand {
             "Invalid entity config: all configs to be added must be in the format \"key=val\".")
     val props = new Properties
     configsToBeAdded.foreach(pair => props.setProperty(pair(0).trim, pair(1).trim))
+    if (props.containsKey(LogConfig.MessageFormatVersionProp)) {
+      println(s"WARNING: The configuration ${LogConfig.MessageFormatVersionProp}=${props.getProperty(LogConfig.MessageFormatVersionProp)} is specified. " +
+        s"This configuration will be ignored if the value is on a version newer than the specified inter.broker.protocol.version in the broker.")
+    }
     props
   }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/main/scala/kafka/admin/TopicCommand.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala
index c17b5bc..d4212c5 100755
--- a/core/src/main/scala/kafka/admin/TopicCommand.scala
+++ b/core/src/main/scala/kafka/admin/TopicCommand.scala
@@ -17,21 +17,22 @@
 
 package kafka.admin
 
-import joptsimple._
 import java.util.Properties
-import kafka.common.{TopicExistsException, Topic, AdminCommandFailedException}
-import kafka.utils.CommandLineUtils
-import kafka.utils._
-import kafka.utils.ZkUtils._
-import org.I0Itec.zkclient.exception.ZkNodeExistsException
-import scala.collection._
-import scala.collection.JavaConversions._
-import kafka.log.{Defaults, LogConfig}
+
+import joptsimple._
+import kafka.common.{AdminCommandFailedException, Topic, TopicExistsException}
 import kafka.consumer.{ConsumerConfig, Whitelist}
+import kafka.coordinator.GroupCoordinator
+import kafka.log.{Defaults, LogConfig}
 import kafka.server.ConfigType
-import org.apache.kafka.common.utils.Utils
+import kafka.utils.ZkUtils._
+import kafka.utils._
+import org.I0Itec.zkclient.exception.ZkNodeExistsException
 import org.apache.kafka.common.security.JaasUtils
-import kafka.coordinator.GroupCoordinator
+import org.apache.kafka.common.utils.Utils
+
+import scala.collection.JavaConversions._
+import scala.collection._
 
 
 object TopicCommand extends Logging {
@@ -235,6 +236,10 @@ object TopicCommand extends Logging {
     val props = new Properties
     configsToBeAdded.foreach(pair => props.setProperty(pair(0).trim, pair(1).trim))
     LogConfig.validate(props)
+    if (props.containsKey(LogConfig.MessageFormatVersionProp)) {
+      println(s"WARNING: The configuration ${LogConfig.MessageFormatVersionProp}=${props.getProperty(LogConfig.MessageFormatVersionProp)} is specified. " +
+      s"This configuration will be ignored if the value is on a version newer than the specified inter.broker.protocol.version in the broker.")
+    }
     props
   }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 c9c1976..6b5fb7c 100644
--- a/core/src/main/scala/kafka/api/ApiVersion.scala
+++ b/core/src/main/scala/kafka/api/ApiVersion.scala
@@ -17,6 +17,8 @@
 
 package kafka.api
 
+import kafka.message.Message
+
 /**
  * This class contains the different Kafka versions.
  * Right now, we use them for upgrades - users can configure the version of the API brokers will use to communicate between themselves.
@@ -24,16 +26,29 @@ package kafka.api
  *
  * Note that the ID we initialize for each version is important.
  * We consider a version newer than another, if it has a higher ID (to avoid depending on lexicographic order)
+ * 
+ * Since the api protocol may change more than once within the same release and to facilitate people deploying code from
+ * trunk, we have the concept of internal versions (first introduced during the 0.10.0 development cycle). For example,
+ * the first time we introduce a version change in a release, say 0.10.0, we will add a config value "0.10.0-IV0" and a
+ * corresponding case object KAFKA_0_10_0-IV0. We will also add a config value "0.10.0" that will be mapped to the
+ * latest internal version object, which is KAFKA_0_10_0-IV0. When we change the protocol a second time while developing
+ * 0.10.0, we will add a new config value "0.10.0-IV1" and a corresponding case object KAFKA_0_10_0-IV1. We will change
+ * the config value "0.10.0" to map to the latest internal version object KAFKA_0_10_0-IV1. The config value of
+ * "0.10.0-IV0" is still mapped to KAFKA_0_10_0-IV0. This way, if people are deploying from trunk, they can use
+ * "0.10.0-IV0" and "0.10.0-IV1" to upgrade one internal version at a time. For most people who just want to use
+ * released version, they can use "0.10.0" when upgrading to the 0.10.0 release.
  */
 object ApiVersion {
   // This implicit is necessary due to: https://issues.scala-lang.org/browse/SI-8541
   implicit def orderingByVersion[A <: ApiVersion]: Ordering[A] = Ordering.by(_.id)
 
   private val versionNameMap = Map(
-    "0.8.0" -> KAFKA_080,
-    "0.8.1" -> KAFKA_081,
-    "0.8.2" -> KAFKA_082,
-    "0.9.0" -> KAFKA_090
+    "0.8.0" -> KAFKA_0_8_0,
+    "0.8.1" -> KAFKA_0_8_1,
+    "0.8.2" -> KAFKA_0_8_2,
+    "0.9.0" -> KAFKA_0_9_0,
+    "0.10.0-IV0" -> KAFKA_0_10_0_IV0,
+    "0.10.0" -> KAFKA_0_10_0_IV0
   )
 
   def apply(version: String): ApiVersion  = versionNameMap(version.split("\\.").slice(0,3).mkString("."))
@@ -43,6 +58,7 @@ object ApiVersion {
 
 sealed trait ApiVersion extends Ordered[ApiVersion] {
   val version: String
+  val messageFormatVersion: Byte
   val id: Int
 
   override def compare(that: ApiVersion): Int = {
@@ -57,22 +73,32 @@ sealed trait ApiVersion extends Ordered[ApiVersion] {
 }
 
 // Keep the IDs in order of versions
-case object KAFKA_080 extends ApiVersion {
+case object KAFKA_0_8_0 extends ApiVersion {
   val version: String = "0.8.0.X"
+  val messageFormatVersion: Byte = Message.MagicValue_V0
   val id: Int = 0
 }
 
-case object KAFKA_081 extends ApiVersion {
+case object KAFKA_0_8_1 extends ApiVersion {
   val version: String = "0.8.1.X"
+  val messageFormatVersion: Byte = Message.MagicValue_V0
   val id: Int = 1
 }
 
-case object KAFKA_082 extends ApiVersion {
+case object KAFKA_0_8_2 extends ApiVersion {
   val version: String = "0.8.2.X"
+  val messageFormatVersion: Byte = Message.MagicValue_V0
   val id: Int = 2
 }
 
-case object KAFKA_090 extends ApiVersion {
+case object KAFKA_0_9_0 extends ApiVersion {
   val version: String = "0.9.0.X"
+  val messageFormatVersion: Byte = Message.MagicValue_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 id: Int = 4
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/main/scala/kafka/api/FetchRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/FetchRequest.scala b/core/src/main/scala/kafka/api/FetchRequest.scala
index b43b8f4..f47942c 100644
--- a/core/src/main/scala/kafka/api/FetchRequest.scala
+++ b/core/src/main/scala/kafka/api/FetchRequest.scala
@@ -33,7 +33,7 @@ import scala.collection.immutable.Map
 case class PartitionFetchInfo(offset: Long, fetchSize: Int)
 
 object FetchRequest {
-  val CurrentVersion = 1.shortValue
+  val CurrentVersion = 2.shortValue
   val DefaultMaxWait = 0
   val DefaultMinBytes = 0
   val DefaultCorrelationId = 0
@@ -151,6 +151,7 @@ case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion,
         (topicAndPartition, FetchResponsePartitionData(Errors.forException(e).code, -1, MessageSet.Empty))
     }
     val errorResponse = FetchResponse(correlationId, fetchResponsePartitionData)
+    // Magic value does not matter here because the message set is empty
     requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, errorResponse)))
   }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/main/scala/kafka/api/ProducerRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/ProducerRequest.scala b/core/src/main/scala/kafka/api/ProducerRequest.scala
index 11f5009..30af841 100644
--- a/core/src/main/scala/kafka/api/ProducerRequest.scala
+++ b/core/src/main/scala/kafka/api/ProducerRequest.scala
@@ -27,7 +27,7 @@ import kafka.network.RequestChannel.Response
 import org.apache.kafka.common.protocol.{ApiKeys, Errors}
 
 object ProducerRequest {
-  val CurrentVersion = 1.shortValue
+  val CurrentVersion = 2.shortValue
 
   def readFrom(buffer: ByteBuffer): ProducerRequest = {
     val versionId: Short = buffer.getShort
@@ -135,7 +135,7 @@ case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion,
     else {
       val producerResponseStatus = data.map {
         case (topicAndPartition, data) =>
-          (topicAndPartition, ProducerResponseStatus(Errors.forException(e).code, -1l))
+          (topicAndPartition, ProducerResponseStatus(Errors.forException(e).code, -1l, Message.NoTimestamp))
       }
       val errorResponse = ProducerResponse(correlationId, producerResponseStatus)
       requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse)))

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/main/scala/kafka/api/ProducerResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/ProducerResponse.scala b/core/src/main/scala/kafka/api/ProducerResponse.scala
index 7e745cf..89d7538 100644
--- a/core/src/main/scala/kafka/api/ProducerResponse.scala
+++ b/core/src/main/scala/kafka/api/ProducerResponse.scala
@@ -18,6 +18,7 @@
 package kafka.api
 
 import java.nio.ByteBuffer
+import kafka.message.Message
 import org.apache.kafka.common.protocol.Errors
 
 import scala.collection.Map
@@ -25,7 +26,7 @@ import kafka.common.TopicAndPartition
 import kafka.api.ApiUtils._
 
 object ProducerResponse {
-  // readFrom assumes that the response is written using V1 format
+  // readFrom assumes that the response is written using V2 format
   def readFrom(buffer: ByteBuffer): ProducerResponse = {
     val correlationId = buffer.getInt
     val topicCount = buffer.getInt
@@ -36,7 +37,8 @@ object ProducerResponse {
         val partition = buffer.getInt
         val error = buffer.getShort
         val offset = buffer.getLong
-        (TopicAndPartition(topic, partition), ProducerResponseStatus(error, offset))
+        val timestamp = buffer.getLong
+        (TopicAndPartition(topic, partition), ProducerResponseStatus(error, offset, timestamp))
       })
     })
 
@@ -45,7 +47,7 @@ object ProducerResponse {
   }
 }
 
-case class ProducerResponseStatus(var error: Short, offset: Long)
+case class ProducerResponseStatus(var error: Short, offset: Long, timestamp: Long = Message.NoTimestamp)
 
 case class ProducerResponse(correlationId: Int,
                             status: Map[TopicAndPartition, ProducerResponseStatus],
@@ -72,7 +74,8 @@ case class ProducerResponse(correlationId: Int,
       currTopic._2.size * {
         4 + /* partition id */
         2 + /* error code */
-        8 /* offset */
+        8 + /* offset */
+        8 /* timestamp */
       }
     }) +
     throttleTimeSize
@@ -88,10 +91,11 @@ case class ProducerResponse(correlationId: Int,
       writeShortString(buffer, topic)
       buffer.putInt(errorsAndOffsets.size) // partition count
       errorsAndOffsets.foreach {
-        case((TopicAndPartition(_, partition), ProducerResponseStatus(error, nextOffset))) =>
+        case((TopicAndPartition(_, partition), ProducerResponseStatus(error, nextOffset, timestamp))) =>
           buffer.putInt(partition)
           buffer.putShort(error)
           buffer.putLong(nextOffset)
+          buffer.putLong(timestamp)
       }
     })
     // Throttle time is only supported on V1 style requests

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/main/scala/kafka/common/ErrorMapping.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/common/ErrorMapping.scala b/core/src/main/scala/kafka/common/ErrorMapping.scala
index e20b88c..9708c4e 100644
--- a/core/src/main/scala/kafka/common/ErrorMapping.scala
+++ b/core/src/main/scala/kafka/common/ErrorMapping.scala
@@ -62,6 +62,7 @@ object ErrorMapping {
   val TopicAuthorizationCode: Short = 29
   val GroupAuthorizationCode: Short = 30
   val ClusterAuthorizationCode: Short = 31
+  // 32: INVALID_TIMESTAMP
 
   private val exceptionToCode =
     Map[Class[Throwable], Short](

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/main/scala/kafka/consumer/BaseConsumer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/consumer/BaseConsumer.scala b/core/src/main/scala/kafka/consumer/BaseConsumer.scala
index 7942f57..3774e73 100644
--- a/core/src/main/scala/kafka/consumer/BaseConsumer.scala
+++ b/core/src/main/scala/kafka/consumer/BaseConsumer.scala
@@ -21,7 +21,9 @@ import java.util.Properties
 import java.util.regex.Pattern
 
 import kafka.common.StreamEndException
+import kafka.message.Message
 import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener
+import org.apache.kafka.common.record.TimestampType
 
 /**
  * A base consumer used to abstract both old and new consumer
@@ -35,7 +37,13 @@ trait BaseConsumer {
   def commit()
 }
 
-case class BaseConsumerRecord(topic: String, partition: Int, offset: Long, key: Array[Byte], value: Array[Byte])
+case class BaseConsumerRecord(topic: String,
+                              partition: Int,
+                              offset: Long,
+                              timestamp: Long = Message.NoTimestamp,
+                              timestampType: TimestampType = TimestampType.NO_TIMESTAMP_TYPE,
+                              key: Array[Byte],
+                              value: Array[Byte])
 
 class NewShinyConsumer(topic: Option[String], whitelist: Option[String], consumerProps: Properties, val timeoutMs: Long = Long.MaxValue) extends BaseConsumer {
   import org.apache.kafka.clients.consumer.KafkaConsumer
@@ -60,7 +68,13 @@ class NewShinyConsumer(topic: Option[String], whitelist: Option[String], consume
     }
 
     val record = recordIter.next
-    BaseConsumerRecord(record.topic, record.partition, record.offset, record.key, record.value)
+    BaseConsumerRecord(record.topic,
+                       record.partition,
+                       record.offset,
+                       record.timestamp,
+                       record.timestampType,
+                       record.key,
+                       record.value)
   }
 
   override def stop() {
@@ -89,7 +103,13 @@ class OldConsumer(topicFilter: TopicFilter, consumerProps: Properties) extends B
       throw new StreamEndException
 
     val messageAndMetadata = iter.next
-    BaseConsumerRecord(messageAndMetadata.topic, messageAndMetadata.partition, messageAndMetadata.offset, messageAndMetadata.key, messageAndMetadata.message)
+    BaseConsumerRecord(messageAndMetadata.topic,
+                       messageAndMetadata.partition,
+                       messageAndMetadata.offset,
+                       messageAndMetadata.timestamp,
+                       messageAndMetadata.timestampType,
+                       messageAndMetadata.key,
+                       messageAndMetadata.message)
   }
 
   override def stop() {

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/main/scala/kafka/consumer/ConsumerIterator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala
index 0c6c810..0735651 100755
--- a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala
+++ b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala
@@ -100,7 +100,14 @@ class ConsumerIterator[K, V](private val channel: BlockingQueue[FetchedDataChunk
 
     item.message.ensureValid() // validate checksum of message to ensure it is valid
 
-    new MessageAndMetadata(currentTopicInfo.topic, currentTopicInfo.partitionId, item.message, item.offset, keyDecoder, valueDecoder)
+    new MessageAndMetadata(currentTopicInfo.topic,
+                           currentTopicInfo.partitionId,
+                           item.message,
+                           item.offset,
+                           item.message.timestamp,
+                           item.message.timestampType,
+                           keyDecoder,
+                           valueDecoder)
   }
 
   def clearCurrentChunk() {

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
index 02ba814..513f383 100755
--- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
+++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
@@ -16,7 +16,7 @@
 */
 package kafka.controller
 
-import kafka.api.{LeaderAndIsr, KAFKA_090, PartitionStateInfo}
+import kafka.api.{LeaderAndIsr, KAFKA_0_9_0, PartitionStateInfo}
 import kafka.utils._
 import org.apache.kafka.clients.{ClientResponse, ClientRequest, ManualMetadataUpdater, NetworkClient}
 import org.apache.kafka.common.{BrokerEndPoint, TopicPartition, Node}
@@ -380,7 +380,7 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends  Logging
           topicPartition -> partitionState
         }
 
-        val version = if (controller.config.interBrokerProtocolVersion.onOrAfter(KAFKA_090)) (1: Short) else (0: Short)
+        val version = if (controller.config.interBrokerProtocolVersion.onOrAfter(KAFKA_0_9_0)) (1: Short) else (0: Short)
 
         val updateMetadataRequest =
           if (version == 0) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 2f1b842..c86e87b 100644
--- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala
+++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala
@@ -21,9 +21,10 @@ import java.util.concurrent.atomic.AtomicBoolean
 
 import kafka.common.{OffsetAndMetadata, OffsetMetadataAndError, TopicAndPartition}
 import kafka.log.LogConfig
-import kafka.message.{Message, UncompressedCodec}
+import kafka.message.UncompressedCodec
 import kafka.server._
 import kafka.utils._
+import org.apache.kafka.common.utils.Time
 import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.requests.JoinGroupRequest
 
@@ -48,7 +49,8 @@ case class JoinGroupResult(members: Map[String, Array[Byte]],
 class GroupCoordinator(val brokerId: Int,
                        val groupConfig: GroupConfig,
                        val offsetConfig: OffsetConfig,
-                       val groupManager: GroupMetadataManager) extends Logging {
+                       val groupManager: GroupMetadataManager,
+                       time: Time) extends Logging {
   type JoinCallback = JoinGroupResult => Unit
   type SyncCallback = (Array[Byte], Short) => Unit
 
@@ -59,13 +61,6 @@ class GroupCoordinator(val brokerId: Int,
   private var heartbeatPurgatory: DelayedOperationPurgatory[DelayedHeartbeat] = null
   private var joinPurgatory: DelayedOperationPurgatory[DelayedJoin] = null
 
-  def this(brokerId: Int,
-           groupConfig: GroupConfig,
-           offsetConfig: OffsetConfig,
-           replicaManager: ReplicaManager,
-           zkUtils: ZkUtils) = this(brokerId, groupConfig, offsetConfig,
-    new GroupMetadataManager(brokerId, offsetConfig, replicaManager, zkUtils))
-
   def offsetsTopicConfigs: Properties = {
     val props = new Properties
     props.put(LogConfig.CleanupPolicyProp, LogConfig.Compact)
@@ -563,7 +558,7 @@ class GroupCoordinator(val brokerId: Int,
    */
   private def completeAndScheduleNextHeartbeatExpiration(group: GroupMetadata, member: MemberMetadata) {
     // complete current heartbeat expectation
-    member.latestHeartbeat = SystemTime.milliseconds
+    member.latestHeartbeat = time.milliseconds()
     val memberKey = MemberKey(member.groupId, member.memberId)
     heartbeatPurgatory.checkAndComplete(memberKey)
 
@@ -731,7 +726,8 @@ object GroupCoordinator {
 
   def create(config: KafkaConfig,
              zkUtils: ZkUtils,
-             replicaManager: ReplicaManager): GroupCoordinator = {
+             replicaManager: ReplicaManager,
+             time: Time): GroupCoordinator = {
     val offsetConfig = OffsetConfig(maxMetadataSize = config.offsetMetadataMaxSize,
       loadBufferSize = config.offsetsLoadBufferSize,
       offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L,
@@ -743,22 +739,8 @@ object GroupCoordinator {
     val groupConfig = GroupConfig(groupMinSessionTimeoutMs = config.groupMinSessionTimeoutMs,
       groupMaxSessionTimeoutMs = config.groupMaxSessionTimeoutMs)
 
-    new GroupCoordinator(config.brokerId, groupConfig, offsetConfig, replicaManager, zkUtils)
+    val groupManager = new GroupMetadataManager(config.brokerId, offsetConfig, replicaManager, zkUtils, time)
+    new GroupCoordinator(config.brokerId, groupConfig, offsetConfig, groupManager, time)
   }
 
-  def create(config: KafkaConfig,
-             groupManager: GroupMetadataManager): GroupCoordinator = {
-    val offsetConfig = OffsetConfig(maxMetadataSize = config.offsetMetadataMaxSize,
-      loadBufferSize = config.offsetsLoadBufferSize,
-      offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L,
-      offsetsRetentionCheckIntervalMs = config.offsetsRetentionCheckIntervalMs,
-      offsetsTopicNumPartitions = config.offsetsTopicPartitions,
-      offsetsTopicReplicationFactor = config.offsetsTopicReplicationFactor,
-      offsetCommitTimeoutMs = config.offsetCommitTimeoutMs,
-      offsetCommitRequiredAcks = config.offsetCommitRequiredAcks)
-    val groupConfig = GroupConfig(groupMinSessionTimeoutMs = config.groupMinSessionTimeoutMs,
-      groupMaxSessionTimeoutMs = config.groupMaxSessionTimeoutMs)
-
-    new GroupCoordinator(config.brokerId, groupConfig, offsetConfig, groupManager)
-  }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 48818c3..b3e1bc1 100644
--- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala
+++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala
@@ -27,8 +27,10 @@ import org.apache.kafka.common.protocol.types.Type.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.record.TimestampType
 import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
 import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.common.utils.Time
 
 import kafka.utils._
 import kafka.common._
@@ -47,14 +49,14 @@ import java.util.concurrent.TimeUnit
 
 import com.yammer.metrics.core.Gauge
 
-
 case class DelayedStore(messageSet: Map[TopicPartition, MessageSet],
                         callback: Map[TopicPartition, PartitionResponse] => Unit)
 
 class GroupMetadataManager(val brokerId: Int,
                            val config: OffsetConfig,
                            replicaManager: ReplicaManager,
-                           zkUtils: ZkUtils) extends Logging with KafkaMetricsGroup {
+                           zkUtils: ZkUtils,
+                           time: Time) extends Logging with KafkaMetricsGroup {
 
   /* offsets cache */
   private val offsetsCache = new Pool[GroupTopicPartition, OffsetAndMetadata]
@@ -141,7 +143,8 @@ class GroupMetadataManager(val brokerId: Int,
       // if we crash or leaders move) since the new leaders will still expire the consumers with heartbeat and
       // retry removing this group.
       val groupPartition = partitionFor(group.groupId)
-      val tombstone = new Message(bytes = null, key = GroupMetadataManager.groupMetadataKey(group.groupId))
+      val tombstone = new Message(bytes = null, key = GroupMetadataManager.groupMetadataKey(group.groupId),
+        timestamp = time.milliseconds(), magicValue = getMessageFormatVersion(groupPartition))
 
       val partitionOpt = replicaManager.getPartition(GroupCoordinator.GroupMetadataTopicName, groupPartition)
       partitionOpt.foreach { partition =>
@@ -166,10 +169,11 @@ class GroupMetadataManager(val brokerId: Int,
   def prepareStoreGroup(group: GroupMetadata,
                         groupAssignment: Map[String, Array[Byte]],
                         responseCallback: Short => Unit): DelayedStore = {
-    // construct the message to append
     val message = new Message(
       key = GroupMetadataManager.groupMetadataKey(group.groupId),
-      bytes = GroupMetadataManager.groupMetadataValue(group, groupAssignment)
+      bytes = GroupMetadataManager.groupMetadataValue(group, groupAssignment),
+      timestamp = time.milliseconds(),
+      magicValue = getMessageFormatVersion(partitionFor(group.groupId))
     )
 
     val groupMetadataPartition = new TopicPartition(GroupCoordinator.GroupMetadataTopicName, partitionFor(group.groupId))
@@ -251,7 +255,9 @@ class GroupMetadataManager(val brokerId: Int,
     val messages = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) =>
       new Message(
         key = GroupMetadataManager.offsetCommitKey(groupId, topicAndPartition.topic, topicAndPartition.partition),
-        bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata)
+        bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata),
+        timestamp = time.milliseconds(),
+        magicValue = getMessageFormatVersion(partitionFor(groupId))
       )
     }.toSeq
 
@@ -358,7 +364,7 @@ class GroupMetadataManager(val brokerId: Int,
         }
       }
 
-      val startMs = SystemTime.milliseconds
+      val startMs = time.milliseconds()
       try {
         replicaManager.logManager.getLog(topicPartition) match {
           case Some(log) =>
@@ -437,7 +443,7 @@ class GroupMetadataManager(val brokerId: Int,
 
             if (!shuttingDown.get())
               info("Finished loading offsets from %s in %d milliseconds."
-                .format(topicPartition, SystemTime.milliseconds - startMs))
+                .format(topicPartition, time.milliseconds() - startMs))
           case None =>
             warn("No log found for " + topicPartition)
         }
@@ -532,7 +538,7 @@ class GroupMetadataManager(val brokerId: Int,
 
   private def deleteExpiredOffsets() {
     debug("Collecting expired offsets.")
-    val startMs = SystemTime.milliseconds
+    val startMs = time.milliseconds()
 
     val numExpiredOffsetsRemoved = inWriteLock(offsetExpireLock) {
       val expiredOffsets = offsetsCache.filter { case (groupTopicPartition, offsetAndMetadata) =>
@@ -551,7 +557,8 @@ class GroupMetadataManager(val brokerId: Int,
         val commitKey = GroupMetadataManager.offsetCommitKey(groupTopicAndPartition.group,
           groupTopicAndPartition.topicPartition.topic, groupTopicAndPartition.topicPartition.partition)
 
-        (offsetsPartition, new Message(bytes = null, key = commitKey))
+        (offsetsPartition, new Message(bytes = null, key = commitKey, timestamp = time.milliseconds(),
+          magicValue = getMessageFormatVersion(offsetsPartition)))
       }.groupBy { case (partition, tombstone) => partition }
 
       // Append the tombstone messages to the offset partitions. It is okay if the replicas don't receive these (say,
@@ -580,7 +587,7 @@ class GroupMetadataManager(val brokerId: Int,
       }.sum
     }
 
-    info("Removed %d expired offsets in %d milliseconds.".format(numExpiredOffsetsRemoved, SystemTime.milliseconds - startMs))
+    info("Removed %d expired offsets in %d milliseconds.".format(numExpiredOffsetsRemoved, time.milliseconds() - startMs))
   }
 
   private def getHighWatermark(partitionId: Int): Long = {
@@ -620,6 +627,13 @@ class GroupMetadataManager(val brokerId: Int,
       config.offsetsTopicNumPartitions
   }
 
+  private def getMessageFormatVersion(partition: Int): Byte = {
+    val groupMetadataTopicAndPartition = new TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, partition)
+    replicaManager.getMessageFormatVersion(groupMetadataTopicAndPartition).getOrElse {
+      throw new IllegalArgumentException(s"Message format version for partition $groupMetadataTopicPartitionCount not found")
+    }
+  }
+
   /**
    * Add the partition into the owned list
    *
@@ -954,9 +968,9 @@ object GroupMetadataManager {
   // Formatter for use with tools such as console consumer: Consumer should also set exclude.internal.topics to false.
   // (specify --formatter "kafka.coordinator.GroupMetadataManager\$OffsetsMessageFormatter" when consuming __consumer_offsets)
   class OffsetsMessageFormatter extends MessageFormatter {
-    def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream) {
+    def writeTo(key: Array[Byte], value: Array[Byte], timestamp: Long, timestampType: TimestampType, output: PrintStream) {
       val formattedKey = if (key == null) "NULL" else GroupMetadataManager.readMessageKey(ByteBuffer.wrap(key))
-
+      // We ignore the timestamp of the message because GroupMetadataMessage has its own timestamp.
       // only print if the message is an offset record
       if (formattedKey.isInstanceOf[OffsetKey]) {
         val groupTopicPartition = formattedKey.asInstanceOf[OffsetKey].toString
@@ -971,9 +985,9 @@ object GroupMetadataManager {
 
   // Formatter for use with tools to read group metadata history
   class GroupMetadataMessageFormatter extends MessageFormatter {
-    def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream) {
+    def writeTo(key: Array[Byte], value: Array[Byte], timestamp: Long, timestampType: TimestampType, output: PrintStream) {
       val formattedKey = if (key == null) "NULL" else GroupMetadataManager.readMessageKey(ByteBuffer.wrap(key))
-
+      // We ignore the timestamp of the message because GroupMetadataMessage has its own timestamp.
       // only print if the message is a group metadata record
       if (formattedKey.isInstanceOf[GroupMetadataKey]) {
         val groupId = formattedKey.asInstanceOf[GroupMetadataKey].key


[3/5] kafka git commit: KAFKA-3025; Added timetamp to Message and use relative offset.

Posted by ju...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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
index d4ce498..fe31ad4 100755
--- a/core/src/main/scala/kafka/log/FileMessageSet.scala
+++ b/core/src/main/scala/kafka/log/FileMessageSet.scala
@@ -31,6 +31,8 @@ import org.apache.kafka.common.errors.CorruptRecordException
 import org.apache.kafka.common.network.TransportLayer
 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.
@@ -139,7 +141,7 @@ class FileMessageSet private[kafka](@volatile var file: File,
       if(offset >= targetOffset)
         return OffsetPosition(offset, position)
       val messageSize = buffer.getInt()
-      if(messageSize < Message.MessageOverhead)
+      if(messageSize < Message.MinMessageOverhead)
         throw new IllegalStateException("Invalid message size: " + messageSize)
       position += MessageSet.LogOverhead + messageSize
     }
@@ -172,6 +174,63 @@ class FileMessageSet private[kafka](@volatile var file: File,
   }
 
   /**
+    * This method is called before we write messages to socket use zero-copy transfer. We need to
+    * make sure all the messages in the message set has expected magic value
+    * @param expectedMagicValue the magic value expected
+    * @return true if all messages has expected magic value, false otherwise
+    */
+  override def magicValueInAllWrapperMessages(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 specified message format.
+   */
+  def toMessageFormat(toMagicValue: Byte): ByteBufferMessageSet = {
+    val offsets = new ArrayBuffer[Long]
+    val newMessages = new ArrayBuffer[Message]
+    this.iterator().foreach(messageAndOffset => {
+      val message = messageAndOffset.message
+      if (message.compressionCodec == NoCompressionCodec) {
+        newMessages += messageAndOffset.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
+        }
+      }
+    })
+
+    // 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.toSeq,
+      newMessages: _*)
+  }
+
+  /**
    * Get a shallow iterator over the messages in the set.
    */
   override def iterator() = iterator(Int.MaxValue)
@@ -200,7 +259,7 @@ class FileMessageSet private[kafka](@volatile var file: File,
         sizeOffsetBuffer.rewind()
         val offset = sizeOffsetBuffer.getLong()
         val size = sizeOffsetBuffer.getInt()
-        if(size < Message.MinHeaderSize)
+        if(size < Message.MinMessageOverhead)
           return allDone()
         if(size > maxMessageSize)
           throw new CorruptRecordException("Message size exceeds the largest allowable message size (%d).".format(maxMessageSize))

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 32c194d..f8c0b77 100644
--- a/core/src/main/scala/kafka/log/Log.scala
+++ b/core/src/main/scala/kafka/log/Log.scala
@@ -28,26 +28,35 @@ import java.util.concurrent.{ConcurrentNavigableMap, ConcurrentSkipListMap}
 import java.util.concurrent.atomic._
 import java.text.NumberFormat
 import org.apache.kafka.common.errors.{OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException, CorruptRecordException}
+import org.apache.kafka.common.record.TimestampType
 
 import scala.collection.JavaConversions
 
 import com.yammer.metrics.core.Gauge
 
 object LogAppendInfo {
-  val UnknownLogAppendInfo = LogAppendInfo(-1, -1, NoCompressionCodec, NoCompressionCodec, -1, -1, false)
+  val UnknownLogAppendInfo = LogAppendInfo(-1, -1, Message.NoTimestamp, NoCompressionCodec, NoCompressionCodec, -1, -1, false)
 }
 
 /**
  * Struct to hold various quantities we compute about each message set before appending to the log
  * @param firstOffset The first offset in the message set
  * @param lastOffset The last offset in the message set
- * @param shallowCount The number of shallow messages
- * @param validBytes The number of valid bytes
+ * @param timestamp The log append time (if used) of the message set, otherwise Message.NoTimestamp
  * @param sourceCodec The source codec used in the message set (send by the producer)
  * @param targetCodec The target codec of the message set(after applying the broker compression configuration if any)
+ * @param shallowCount The number of shallow messages
+ * @param validBytes The number of valid bytes
  * @param offsetsMonotonic Are the offsets in this message set monotonically increasing
  */
-case class LogAppendInfo(var firstOffset: Long, var lastOffset: Long, sourceCodec: CompressionCodec, targetCodec: CompressionCodec, shallowCount: Int, validBytes: Int, offsetsMonotonic: Boolean)
+case class LogAppendInfo(var firstOffset: Long,
+                         var lastOffset: Long,
+                         var timestamp: Long,
+                         sourceCodec: CompressionCodec,
+                         targetCodec: CompressionCodec,
+                         shallowCount: Int,
+                         validBytes: Int,
+                         offsetsMonotonic: Boolean)
 
 
 /**
@@ -325,13 +334,23 @@ class Log(val dir: File,
         if (assignOffsets) {
           // assign offsets to the message set
           val offset = new AtomicLong(nextOffsetMetadata.messageOffset)
+          val now = SystemTime.milliseconds
           try {
-            validMessages = validMessages.validateMessagesAndAssignOffsets(offset, appendInfo.sourceCodec, appendInfo.targetCodec, config
-              .compact)
+            validMessages = validMessages.validateMessagesAndAssignOffsets(offset,
+                                                                           now,
+                                                                           appendInfo.sourceCodec,
+                                                                           appendInfo.targetCodec,
+                                                                           config.compact,
+                                                                           config.messageFormatVersion,
+                                                                           config.messageTimestampType,
+                                                                           config.messageTimestampDifferenceMaxMs)
           } catch {
             case e: IOException => throw new KafkaException("Error in validating messages while appending to log '%s'".format(name), e)
           }
           appendInfo.lastOffset = offset.get - 1
+          // If log append time is used, we put the timestamp assigned to the messages in the append info.
+          if (config.messageTimestampType == TimestampType.LOG_APPEND_TIME)
+            appendInfo.timestamp = now
         } else {
           // we are taking the offsets we are given
           if (!appendInfo.offsetsMonotonic || appendInfo.firstOffset < nextOffsetMetadata.messageOffset)
@@ -436,7 +455,7 @@ class Log(val dir: File,
     // Apply broker-side compression if any
     val targetCodec = BrokerCompressionCodec.getTargetCompressionCodec(config.compressionType, sourceCodec)
 
-    LogAppendInfo(firstOffset, lastOffset, sourceCodec, targetCodec, shallowMessageCount, validBytesCount, monotonic)
+    LogAppendInfo(firstOffset, lastOffset, Message.NoTimestamp, sourceCodec, targetCodec, shallowMessageCount, validBytesCount, monotonic)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 d5c247c..a3aff15 100644
--- a/core/src/main/scala/kafka/log/LogCleaner.scala
+++ b/core/src/main/scala/kafka/log/LogCleaner.scala
@@ -370,7 +370,7 @@ private[log] class Cleaner(val id: Int,
         val retainDeletes = old.lastModified > deleteHorizonMs
         info("Cleaning segment %s in log %s (last modified %s) into %s, %s deletes."
             .format(old.baseOffset, log.name, new Date(old.lastModified), cleaned.baseOffset, if(retainDeletes) "retaining" else "discarding"))
-        cleanInto(log.topicAndPartition, old, cleaned, map, retainDeletes)
+        cleanInto(log.topicAndPartition, old, cleaned, map, retainDeletes, log.config.messageFormatVersion)
       }
 
       // trim excess index
@@ -401,10 +401,14 @@ private[log] class Cleaner(val id: Int,
    * @param dest The cleaned log segment
    * @param map The key=>offset mapping
    * @param retainDeletes Should delete tombstones be retained while cleaning this segment
-   *
+   * @param messageFormatVersion the message format version to use after compaction
    */
-  private[log] def cleanInto(topicAndPartition: TopicAndPartition, source: LogSegment,
-                             dest: LogSegment, map: OffsetMap, retainDeletes: Boolean) {
+  private[log] def cleanInto(topicAndPartition: TopicAndPartition,
+                             source: LogSegment,
+                             dest: LogSegment,
+                             map: OffsetMap,
+                             retainDeletes: Boolean,
+                             messageFormatVersion: Byte) {
     var position = 0
     while (position < source.log.sizeInBytes) {
       checkDone(topicAndPartition)
@@ -420,19 +424,34 @@ private[log] class Cleaner(val id: Int,
         stats.readMessage(size)
         if (entry.message.compressionCodec == NoCompressionCodec) {
           if (shouldRetainMessage(source, map, retainDeletes, entry)) {
-            ByteBufferMessageSet.writeMessage(writeBuffer, entry.message, entry.offset)
+            val convertedMessage = entry.message.toFormatVersion(messageFormatVersion)
+            ByteBufferMessageSet.writeMessage(writeBuffer, convertedMessage, entry.offset)
             stats.recopyMessage(size)
           }
           messagesRead += 1
         } else {
-          val messages = ByteBufferMessageSet.deepIterator(entry.message)
+          // We use absolute offset to decide whether retain the message or not. This is handled by
+          // deep iterator.
+          val messages = ByteBufferMessageSet.deepIterator(entry)
+          var numberOfInnerMessages = 0
+          var formatConversionNeeded = false
           val retainedMessages = messages.filter(messageAndOffset => {
             messagesRead += 1
+            numberOfInnerMessages += 1
+            if (messageAndOffset.message.magic != messageFormatVersion)
+              formatConversionNeeded = true
             shouldRetainMessage(source, map, retainDeletes, messageAndOffset)
           }).toSeq
 
-          if (retainedMessages.nonEmpty)
-            compressMessages(writeBuffer, entry.message.compressionCodec, retainedMessages)
+          // There is no messages compacted out and no message format conversion, write the original message set back
+          if (retainedMessages.size == numberOfInnerMessages && !formatConversionNeeded)
+            ByteBufferMessageSet.writeMessage(writeBuffer, entry.message, entry.offset)
+          else if (retainedMessages.nonEmpty) {
+            val convertedRetainedMessages = retainedMessages.map(messageAndOffset => {
+              new MessageAndOffset(messageAndOffset.message.toFormatVersion(messageFormatVersion), messageAndOffset.offset)
+            })
+            compressMessages(writeBuffer, entry.message.compressionCodec, messageFormatVersion, convertedRetainedMessages)
+          }
         }
       }
 
@@ -452,7 +471,10 @@ private[log] class Cleaner(val id: Int,
     restoreBuffers()
   }
 
-  private def compressMessages(buffer: ByteBuffer, compressionCodec: CompressionCodec, messages: Seq[MessageAndOffset]) {
+  private def compressMessages(buffer: ByteBuffer,
+                               compressionCodec: CompressionCodec,
+                               messageFormatVersion: Byte,
+                               messages: Seq[MessageAndOffset]) {
     val messagesIterable = messages.toIterable.map(_.message)
     if (messages.isEmpty) {
       MessageSet.Empty.sizeInBytes
@@ -461,15 +483,24 @@ private[log] class Cleaner(val id: Int,
         ByteBufferMessageSet.writeMessage(buffer, messageOffset.message, messageOffset.offset)
       MessageSet.messageSetSize(messagesIterable)
     } else {
+      val magicAndTimestamp = MessageSet.magicAndLargestTimestamp(messages.map(_.message))
+      val firstAbsoluteOffset = messages.head.offset
       var offset = -1L
+      val timestampType = messages.head.message.timestampType
       val messageWriter = new MessageWriter(math.min(math.max(MessageSet.messageSetSize(messagesIterable) / 2, 1024), 1 << 16))
-      messageWriter.write(codec = compressionCodec) { outputStream =>
+      messageWriter.write(codec = compressionCodec, timestamp = magicAndTimestamp.timestamp, timestampType = timestampType, magicValue = messageFormatVersion) { outputStream =>
         val output = new DataOutputStream(CompressionFactory(compressionCodec, outputStream))
         try {
           for (messageOffset <- messages) {
             val message = messageOffset.message
             offset = messageOffset.offset
-            output.writeLong(offset)
+            // Use inner offset when magic value is greater than 0
+            if (messageFormatVersion > Message.MagicValue_V0) {
+              // The offset of the messages are absolute offset, compute the inner offset.
+              val innerOffset = messageOffset.offset - firstAbsoluteOffset
+              output.writeLong(innerOffset)
+            } else
+              output.writeLong(offset)
             output.writeInt(message.size)
             output.write(message.buffer.array, message.buffer.arrayOffset, message.buffer.limit)
           }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/main/scala/kafka/log/LogConfig.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala
index 7fc7e33..a8fffbd 100755
--- a/core/src/main/scala/kafka/log/LogConfig.scala
+++ b/core/src/main/scala/kafka/log/LogConfig.scala
@@ -18,12 +18,13 @@
 package kafka.log
 
 import java.util.Properties
+
+import kafka.api.ApiVersion
+import kafka.message.{BrokerCompressionCodec, Message}
 import kafka.server.KafkaConfig
-import org.apache.kafka.common.utils.Utils
-import scala.collection._
 import org.apache.kafka.common.config.{AbstractConfig, ConfigDef}
-import kafka.message.BrokerCompressionCodec
-import kafka.message.Message
+import org.apache.kafka.common.record.TimestampType
+import org.apache.kafka.common.utils.Utils
 
 object Defaults {
   val SegmentSize = kafka.server.Defaults.LogSegmentBytes
@@ -44,6 +45,9 @@ object Defaults {
   val MinInSyncReplicas = kafka.server.Defaults.MinInSyncReplicas
   val CompressionType = kafka.server.Defaults.CompressionType
   val PreAllocateEnable = kafka.server.Defaults.LogPreAllocateEnable
+  val MessageFormatVersion = kafka.server.Defaults.MessageFormatVersion
+  val MessageTimestampType = kafka.server.Defaults.MessageTimestampType
+  val MessageTimestampDifferenceMaxMs = kafka.server.Defaults.MessageTimestampDifferenceMaxMs
 }
 
 case class LogConfig(props: java.util.Map[_, _]) extends AbstractConfig(LogConfig.configDef, props, false) {
@@ -69,6 +73,9 @@ case class LogConfig(props: java.util.Map[_, _]) extends AbstractConfig(LogConfi
   val minInSyncReplicas = getInt(LogConfig.MinInSyncReplicasProp)
   val compressionType = getString(LogConfig.CompressionTypeProp).toLowerCase
   val preallocate = getBoolean(LogConfig.PreAllocateEnableProp)
+  val messageFormatVersion = ApiVersion(getString(LogConfig.MessageFormatVersionProp)).messageFormatVersion
+  val messageTimestampType = TimestampType.forName(getString(LogConfig.MessageTimestampTypeProp))
+  val messageTimestampDifferenceMaxMs = getLong(LogConfig.MessageTimestampDifferenceMaxMsProp)
 
   def randomSegmentJitter: Long =
     if (segmentJitterMs == 0) 0 else Utils.abs(scala.util.Random.nextInt()) % math.min(segmentJitterMs, segmentMs)
@@ -101,6 +108,9 @@ object LogConfig {
   val MinInSyncReplicasProp = "min.insync.replicas"
   val CompressionTypeProp = "compression.type"
   val PreAllocateEnableProp = "preallocate"
+  val MessageFormatVersionProp = KafkaConfig.MessageFormatVersionProp
+  val MessageTimestampTypeProp = KafkaConfig.MessageTimestampTypeProp
+  val MessageTimestampDifferenceMaxMsProp = KafkaConfig.MessageTimestampDifferenceMaxMsProp
 
   val SegmentSizeDoc = "The hard maximum for the size of a segment file in the log"
   val SegmentMsDoc = "The soft maximum on the amount of time before a new log segment is rolled"
@@ -125,16 +135,18 @@ object LogConfig {
     "standard compression codecs ('gzip', 'snappy', lz4). It additionally accepts 'uncompressed' which is equivalent to " +
     "no compression; and 'producer' which means retain the original compression codec set by the producer."
   val PreAllocateEnableDoc ="Should pre allocate file when create new segment?"
+  val MessageFormatVersionDoc = KafkaConfig.MessageFormatVersionDoc
+  val MessageTimestampTypeDoc = KafkaConfig.MessageTimestampTypeDoc
+  val MessageTimestampDifferenceMaxMsDoc = KafkaConfig.MessageTimestampDifferenceMaxMsDoc
 
   private val configDef = {
-    import ConfigDef.Range._
-    import ConfigDef.ValidString._
-    import ConfigDef.Type._
-    import ConfigDef.Importance._
-    import java.util.Arrays.asList
+    import org.apache.kafka.common.config.ConfigDef.Importance._
+    import org.apache.kafka.common.config.ConfigDef.Range._
+    import org.apache.kafka.common.config.ConfigDef.Type._
+    import org.apache.kafka.common.config.ConfigDef.ValidString._
 
     new ConfigDef()
-      .define(SegmentBytesProp, INT, Defaults.SegmentSize, atLeast(Message.MinHeaderSize), MEDIUM, SegmentSizeDoc)
+      .define(SegmentBytesProp, INT, Defaults.SegmentSize, atLeast(Message.MinMessageOverhead), MEDIUM, SegmentSizeDoc)
       .define(SegmentMsProp, LONG, Defaults.SegmentMs, atLeast(0), MEDIUM, SegmentMsDoc)
       .define(SegmentJitterMsProp, LONG, Defaults.SegmentJitterMs, atLeast(0), MEDIUM, SegmentJitterMsDoc)
       .define(SegmentIndexBytesProp, INT, Defaults.MaxIndexSize, atLeast(0), MEDIUM, MaxIndexSizeDoc)
@@ -158,12 +170,15 @@ object LogConfig {
       .define(CompressionTypeProp, STRING, Defaults.CompressionType, in(BrokerCompressionCodec.brokerCompressionOptions:_*), MEDIUM, CompressionTypeDoc)
       .define(PreAllocateEnableProp, BOOLEAN, Defaults.PreAllocateEnable,
         MEDIUM, PreAllocateEnableDoc)
+      .define(MessageFormatVersionProp, STRING, Defaults.MessageFormatVersion, MEDIUM, MessageFormatVersionDoc)
+      .define(MessageTimestampTypeProp, STRING, Defaults.MessageTimestampType, MEDIUM, MessageTimestampTypeDoc)
+      .define(MessageTimestampDifferenceMaxMsProp, LONG, Defaults.MessageTimestampDifferenceMaxMs, atLeast(0), MEDIUM, MessageTimestampDifferenceMaxMsDoc)
   }
 
   def apply(): LogConfig = LogConfig(new Properties())
 
   def configNames() = {
-    import JavaConversions._
+    import scala.collection.JavaConversions._
     configDef.names().toList.sorted
   }
 
@@ -182,7 +197,7 @@ object LogConfig {
    * Check that property names are valid
    */
   def validateNames(props: Properties) {
-    import JavaConversions._
+    import scala.collection.JavaConversions._
     val names = configDef.names()
     for(name <- props.keys)
       require(names.contains(name), "Unknown configuration \"%s\".".format(name))

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 aa37d52..9fc68a4 100755
--- a/core/src/main/scala/kafka/log/LogSegment.scala
+++ b/core/src/main/scala/kafka/log/LogSegment.scala
@@ -184,7 +184,7 @@ class LogSegment(val log: FileMessageSet,
               case NoCompressionCodec =>
                 entry.offset
               case _ =>
-                ByteBufferMessageSet.deepIterator(entry.message).next().offset
+                ByteBufferMessageSet.deepIterator(entry).next().offset
           }
           index.append(startOffset, validBytes)
           lastIndexEntry = validBytes

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 5a32de8..2867c78 100644
--- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala
+++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala
@@ -23,28 +23,49 @@ import kafka.common.KafkaException
 import java.nio.ByteBuffer
 import java.nio.channels._
 import java.io._
-import java.util.concurrent.atomic.AtomicLong
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong}
+
+import org.apache.kafka.common.errors.InvalidTimestampException
+import org.apache.kafka.common.record.TimestampType
+import org.apache.kafka.common.utils.Utils
+
+import scala.collection.mutable
+import scala.collection.mutable.ListBuffer
 
 object ByteBufferMessageSet {
 
-  private def create(offsetCounter: AtomicLong, compressionCodec: CompressionCodec, messages: Message*): ByteBuffer = {
+  private def create(offsetAssignor: OffsetAssigner,
+                     compressionCodec: CompressionCodec,
+                     wrapperMessageTimestamp: Option[Long],
+                     timestampType: TimestampType,
+                     messages: Message*): ByteBuffer = {
     if(messages.size == 0) {
       MessageSet.Empty.buffer
     } else if(compressionCodec == NoCompressionCodec) {
       val buffer = ByteBuffer.allocate(MessageSet.messageSetSize(messages))
       for(message <- messages)
-        writeMessage(buffer, message, offsetCounter.getAndIncrement)
+        writeMessage(buffer, message, offsetAssignor.nextAbsoluteOffset)
       buffer.rewind()
       buffer
     } else {
+      val magicAndTimestamp = wrapperMessageTimestamp match {
+        case Some(ts) => MagicAndTimestamp(messages.head.magic, ts)
+        case None => MessageSet.magicAndLargestTimestamp(messages)
+      }
       var offset = -1L
       val messageWriter = new MessageWriter(math.min(math.max(MessageSet.messageSetSize(messages) / 2, 1024), 1 << 16))
-      messageWriter.write(codec = compressionCodec) { outputStream =>
+      messageWriter.write(codec = compressionCodec, timestamp = magicAndTimestamp.timestamp, timestampType = timestampType, magicValue = magicAndTimestamp.magic) { outputStream =>
         val output = new DataOutputStream(CompressionFactory(compressionCodec, outputStream))
         try {
           for (message <- messages) {
-            offset = offsetCounter.getAndIncrement
-            output.writeLong(offset)
+            offset = offsetAssignor.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
+            if (magicAndTimestamp.magic > Message.MagicValue_V0)
+              output.writeLong(offsetAssignor.toInnerOffset(offset))
+            else
+              output.writeLong(offset)
             output.writeInt(message.size)
             output.write(message.buffer.array, message.buffer.arrayOffset, message.buffer.limit)
           }
@@ -59,40 +80,93 @@ object ByteBufferMessageSet {
     }
   }
 
-  /** Deep iterator that decompresses the message sets in-place. */
-  def deepIterator(wrapperMessage: Message): Iterator[MessageAndOffset] = {
+  /** Deep iterator that decompresses the message sets and adjusts timestamp and offset if needed. */
+  def deepIterator(wrapperMessageAndOffset: MessageAndOffset): Iterator[MessageAndOffset] = {
+
+    import Message._
+
     new IteratorTemplate[MessageAndOffset] {
 
+      val wrapperMessageOffset = wrapperMessageAndOffset.offset
+      val wrapperMessage = wrapperMessageAndOffset.message
+      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
+      if (wrapperMessage.payload == null)
+        throw new KafkaException("Message payload is null: " + wrapperMessage)
       val inputStream: InputStream = new ByteBufferBackedInputStream(wrapperMessage.payload)
       val compressed: DataInputStream = new DataInputStream(CompressionFactory(wrapperMessage.compressionCodec, inputStream))
+      var lastInnerOffset = -1L
 
-      override def makeNext(): MessageAndOffset = {
+      val messageAndOffsets = if (wrapperMessageAndOffset.message.magic > MagicValue_V0) {
+        var innerMessageAndOffsets = new mutable.Queue[MessageAndOffset]()
         try {
-          // read the offset
-          val offset = compressed.readLong()
-          // read record size
-          val size = compressed.readInt()
-
-          if (size < Message.MinHeaderSize)
-            throw new InvalidMessageException("Message found with corrupt size (" + size + ") in deep iterator")
-
-          // read the record into an intermediate record buffer
-          // and hence has to do extra copy
-          val bufferArray = new Array[Byte](size)
-          compressed.readFully(bufferArray, 0, size)
-          val buffer = ByteBuffer.wrap(bufferArray)
-
-          val newMessage = new Message(buffer)
-
-          // the decompressed message should not be a wrapper message since we do not allow nested compression
-          new MessageAndOffset(newMessage, offset)
+          while (true) {
+            innerMessageAndOffsets += readMessageFromStream()
+          }
         } catch {
           case eofe: EOFException =>
             compressed.close()
-            allDone()
           case ioe: IOException =>
             throw new KafkaException(ioe)
         }
+        Some(innerMessageAndOffsets)
+      } else {
+        None
+      }
+
+      private def readMessageFromStream() = {
+        // read the offset
+        val innerOffset = compressed.readLong()
+        // read record size
+        val size = compressed.readInt()
+
+        if (size < MinMessageOverhead)
+          throw new InvalidMessageException("Message found with corrupt size (" + size + ") in deep iterator")
+
+        // read the record into an intermediate record buffer
+        // and hence has to do extra copy
+        val bufferArray = new Array[Byte](size)
+        compressed.readFully(bufferArray, 0, size)
+        val buffer = ByteBuffer.wrap(bufferArray)
+
+        // Override the timestamp if necessary
+        val newMessage = new Message(buffer, wrapperMessageTimestampOpt, wrapperMessageTimestampTypeOpt)
+
+        // Inner message and wrapper message must have same magic value
+        if (newMessage.magic != wrapperMessage.magic)
+          throw new IllegalStateException(s"Compressed message has magic value ${wrapperMessage.magic} " +
+            s"but inner message has magic value ${newMessage.magic}")
+        lastInnerOffset = innerOffset
+        new MessageAndOffset(newMessage, innerOffset)
+      }
+
+      override def makeNext(): MessageAndOffset = {
+        messageAndOffsets match {
+          // Using inner offset and timestamps
+          case Some(innerMessageAndOffsets) =>
+            if (innerMessageAndOffsets.isEmpty)
+              allDone()
+            else {
+              val messageAndOffset = innerMessageAndOffsets.dequeue()
+              val message = messageAndOffset.message
+              val relativeOffset = messageAndOffset.offset - lastInnerOffset
+              val absoluteOffset = wrapperMessageOffset + relativeOffset
+              new MessageAndOffset(message, absoluteOffset)
+            }
+          // Not using inner offset and timestamps
+          case None =>
+            try {
+              readMessageFromStream()
+            } catch {
+              case eofe: EOFException =>
+                compressed.close()
+                allDone()
+              case ioe: IOException =>
+                throw new KafkaException(ioe)
+            }
+        }
       }
     }
   }
@@ -111,6 +185,20 @@ object ByteBufferMessageSet {
   }
 }
 
+private class OffsetAssigner(offsets: Seq[Long]) {
+  val index = new AtomicInteger(0)
+
+  def this(offsetCounter: AtomicLong, size: Int) {
+    this((offsetCounter.get() to offsetCounter.get + size).toSeq)
+    offsetCounter.addAndGet(size)
+  }
+
+  def nextAbsoluteOffset = offsets(index.getAndIncrement)
+
+  def toInnerOffset(offset: Long) = offset - offsets(0)
+
+}
+
 /**
  * A sequence of messages stored in a byte buffer
  *
@@ -120,22 +208,87 @@ object ByteBufferMessageSet {
  *
  * Option 2: Give it a list of messages along with instructions relating to serialization format. Producers will use this method.
  *
+ *
+ * When message format v1 is used, there will be following message format changes.
+ * - For non-compressed messages, with message v1 we are adding timestamp and timestamp type attribute. The offsets of
+ *   the messages remain absolute offsets.
+ * - For Compressed messages, with message v1 we are adding timestamp, timestamp type attribute bit and using
+ *   inner offsets (IO) for inner messages of compressed messages (see offset calculation details below). Timestamp type
+ *   attribute is only set in wrapper messages. Inner messages always have CreateTime as timestamp type in attributes.
+ *
+ * The way timestamp set is following:
+ * For non-compressed messages: timestamp and timestamp type attribute in the messages are set and used.
+ * For compressed messages:
+ * 1. Wrapper messages' timestamp type attribute is set to proper value
+ * 2. Wrapper messages' timestamp is set to:
+ *    - the max timestamp of inner messages if CreateTime is used
+ *    - the current server time if wrapper message's timestamp = LogAppendTime.
+ *      In this case the wrapper message timestamp is used and all the timestamps of inner messages are ignored.
+ * 3. Inner messages' timestamp will be:
+ *    - used when wrapper message's timestamp type is CreateTime
+ *    - ignored when wrapper message's timestamp type is LogAppendTime
+ * 4. Inner messages' timestamp type will always be ignored. However, producer must set the inner message timestamp
+ *    type to CreateTime, otherwise the messages will be rejected by broker.
+ *
+ *
+ * The way absolute offset calculated is the following:
+ * Ideally the conversion from relative offset(RO) to absolute offset(AO) should be:
+ *
+ * AO = AO_Of_Last_Inner_Message + RO
+ *
+ * However, note that the message sets sent by producers are compressed in a streaming way.
+ * And the relative offset of an inner message compared with the last inner message is not known until
+ * the last inner message is written.
+ * Unfortunately we are not able to change the previously written messages after the last message is written to
+ * the message set when stream compressing is used.
+ *
+ * To solve this issue, we use the following solution:
+ *
+ * 1. When the producer creates a message set, it simply writes all the messages into a compressed message set with
+ *    offset 0, 1, ... (inner offset).
+ * 2. The broker will set the offset of the wrapper message to the absolute offset of the last message in the
+ *    message set.
+ * 3. When a consumer sees the message set, it first decompresses the entire message set to find out the inner
+ *    offset (IO) of the last inner message. Then it computes RO and AO of previous messages:
+ *
+ *    RO = IO_of_a_message - IO_of_the_last_message
+ *    AO = AO_Of_Last_Inner_Message + RO
+ *
+ * 4. This solution works for compacted message set as well
+ *
  */
 class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Logging {
   private var shallowValidByteCount = -1
 
   def this(compressionCodec: CompressionCodec, messages: Message*) {
-    this(ByteBufferMessageSet.create(new AtomicLong(0), compressionCodec, messages:_*))
+    this(ByteBufferMessageSet.create(new OffsetAssigner(new AtomicLong(0), messages.size), compressionCodec,
+      None, TimestampType.CREATE_TIME, messages:_*))
   }
 
   def this(compressionCodec: CompressionCodec, offsetCounter: AtomicLong, messages: Message*) {
-    this(ByteBufferMessageSet.create(offsetCounter, compressionCodec, messages:_*))
+    this(ByteBufferMessageSet.create(new OffsetAssigner(offsetCounter, messages.size), compressionCodec,
+      None, TimestampType.CREATE_TIME, messages:_*))
+  }
+
+  def this(compressionCodec: CompressionCodec, offsetSeq: Seq[Long], messages: Message*) {
+    this(ByteBufferMessageSet.create(new OffsetAssigner(offsetSeq), compressionCodec,
+      None, TimestampType.CREATE_TIME, messages:_*))
   }
 
   def this(messages: Message*) {
     this(NoCompressionCodec, new AtomicLong(0), messages: _*)
   }
 
+  // This constructor is only used internally
+  private[kafka] def this(compressionCodec: CompressionCodec,
+                          offsetCounter: AtomicLong,
+                          wrapperMessageTimestamp: Option[Long],
+                          timestampType: TimestampType,
+                          messages: Message*) {
+    this(ByteBufferMessageSet.create(new OffsetAssigner(offsetCounter, messages.size), compressionCodec,
+      wrapperMessageTimestamp, timestampType, messages:_*))
+  }
+
   def getBuffer = buffer
 
   private def shallowValidBytes: Int = {
@@ -162,6 +315,14 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi
     written
   }
 
+  override def magicValueInAllWrapperMessages(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()
 
@@ -182,7 +343,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi
           return allDone()
         val offset = topIter.getLong()
         val size = topIter.getInt()
-        if(size < Message.MinHeaderSize)
+        if(size < Message.MinMessageOverhead)
           throw new InvalidMessageException("Message found with corrupt size (" + size + ") in shallow iterator")
 
         // we have an incomplete message
@@ -194,7 +355,6 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi
         message.limit(size)
         topIter.position(topIter.position + size)
         val newMessage = new Message(message)
-
         if(isShallow) {
           new MessageAndOffset(newMessage, offset)
         } else {
@@ -203,7 +363,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi
               innerIter = null
               new MessageAndOffset(newMessage, offset)
             case _ =>
-              innerIter = ByteBufferMessageSet.deepIterator(newMessage)
+              innerIter = ByteBufferMessageSet.deepIterator(new MessageAndOffset(newMessage, offset))
               if(!innerIter.hasNext)
                 innerIter = null
               makeNext()
@@ -226,48 +386,205 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi
   }
 
   /**
-   * Update the offsets for this message set and do further validation on messages. This method attempts to do an
-   * in-place conversion if there is no compression, but otherwise recopies the messages
+   * 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.
    */
   private[kafka] def validateMessagesAndAssignOffsets(offsetCounter: AtomicLong,
+                                                      now: Long = System.currentTimeMillis(),
                                                       sourceCodec: CompressionCodec,
                                                       targetCodec: CompressionCodec,
-                                                      compactedTopic: Boolean = false): ByteBufferMessageSet = {
-    if(sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) {
-      // do in-place validation and offset assignment
-      var messagePosition = 0
-      buffer.mark()
-      while(messagePosition < sizeInBytes - MessageSet.LogOverhead) {
-        buffer.position(messagePosition)
-        buffer.putLong(offsetCounter.getAndIncrement())
-        val messageSize = buffer.getInt()
-        val positionAfterKeySize = buffer.position + Message.KeySizeOffset + Message.KeySizeLength
-        if (compactedTopic && positionAfterKeySize < sizeInBytes) {
-          buffer.position(buffer.position() + Message.KeySizeOffset)
-          val keySize = buffer.getInt()
-          if (keySize <= 0) {
-            buffer.reset()
-            throw new InvalidMessageException("Compacted topic cannot accept message without key.")
-          }
-        }
-        messagePosition += MessageSet.LogOverhead + messageSize
+                                                      compactedTopic: Boolean = false,
+                                                      messageFormatVersion: Byte = Message.CurrentMagicValue,
+                                                      messageTimestampType: TimestampType,
+                                                      messageTimestampDiffMaxMs: Long): ByteBufferMessageSet = {
+    if (sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) {
+      // check the magic value
+      if (!magicValueInAllWrapperMessages(messageFormatVersion)) {
+        // Message format conversion
+        convertNonCompressedMessages(offsetCounter, compactedTopic, now, messageTimestampType, messageTimestampDiffMaxMs,
+          messageFormatVersion)
+      } else {
+        // Do in-place validation, offset assignment and maybe set timestamp
+        validateNonCompressedMessagesAndAssignOffsetInPlace(offsetCounter, now, compactedTopic, messageTimestampType,
+          messageTimestampDiffMaxMs)
       }
-      buffer.reset()
-      this
+
     } else {
-      // We need to deep-iterate over the message-set if any of these are true:
-      // (i) messages are compressed
-      // (ii) the topic is configured with a target compression codec so we need to recompress regardless of original codec
-      val messages = this.internalIterator(isShallow = false).map(messageAndOffset => {
-        if (compactedTopic && !messageAndOffset.message.hasKey)
-          throw new InvalidMessageException("Compacted topic cannot accept message without key.")
-
-        messageAndOffset.message
+      // 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
+      val expectedInnerOffset = new AtomicLong(0)
+      val validatedMessages = new ListBuffer[Message]
+      this.internalIterator(isShallow = false).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
+          maxTimestamp = math.max(maxTimestamp, message.timestamp)
+        }
+
+        // No in place assignment situation 4
+        if (message.magic != messageFormatVersion)
+          inPlaceAssignment = false
+
+        validatedMessages += message.toFormatVersion(messageFormatVersion)
       })
-      new ByteBufferMessageSet(compressionCodec = targetCodec, offsetCounter = offsetCounter, messages = messages.toBuffer:_*)
+      if (!inPlaceAssignment) {
+        // Cannot do in place assignment.
+        val wrapperMessageTimestamp = {
+          if (messageFormatVersion == Message.MagicValue_V0)
+            Some(Message.NoTimestamp)
+          else if (messageFormatVersion > Message.MagicValue_V0 && messageTimestampType == TimestampType.CREATE_TIME)
+            Some(maxTimestamp)
+          else // Log append time
+            Some(now)
+        }
+
+        new ByteBufferMessageSet(compressionCodec = targetCodec,
+                                 offsetCounter = offsetCounter,
+                                 wrapperMessageTimestamp = wrapperMessageTimestamp,
+                                 timestampType = messageTimestampType,
+                                 messages = validatedMessages.toBuffer: _*)
+      } 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)
+        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, TimestampType.setTimestampType(attributes, TimestampType.LOG_APPEND_TIME))
+        }
+
+        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()
+        this
+      }
+    }
+  }
+
+  // We create this method to save memory copy operation. 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: AtomicLong,
+                                           compactedTopic: Boolean,
+                                           now: Long,
+                                           timestampType: TimestampType,
+                                           messageTimestampDiffMaxMs: Long,
+                                           toMagicValue: Byte): ByteBufferMessageSet = {
+    val sizeInBytesAfterConversion = shallowValidBytes + this.internalIterator(isShallow = true).foldLeft(0)(
+      (sizeDiff, messageAndOffset) => sizeDiff + Message.headerSizeDiff(messageAndOffset.message.magic, toMagicValue))
+    val newBuffer = ByteBuffer.allocate(sizeInBytesAfterConversion)
+    var newMessagePosition = 0
+    this.internalIterator(isShallow = true).foreach {messageAndOffset =>
+      val message = messageAndOffset.message
+      validateMessageKey(message, compactedTopic)
+      validateTimestamp(message, now, timestampType, messageTimestampDiffMaxMs)
+      newBuffer.position(newMessagePosition)
+      // write offset.
+      newBuffer.putLong(offsetCounter.getAndIncrement)
+      // Write new message size
+      val newMessageSize = message.size + Message.headerSizeDiff(message.magic, toMagicValue)
+      newBuffer.putInt(newMessageSize)
+      // Create new message buffer
+      val newMessageBuffer = newBuffer.slice()
+      newMessageBuffer.limit(newMessageSize)
+      // Convert message
+      message.convertToBuffer(toMagicValue, newMessageBuffer, now, timestampType)
+
+      newMessagePosition += MessageSet.LogOverhead + newMessageSize
     }
+    newBuffer.rewind()
+    new ByteBufferMessageSet(newBuffer)
   }
 
+  private def validateNonCompressedMessagesAndAssignOffsetInPlace(offsetCounter: AtomicLong,
+                                                                  now: Long,
+                                                                  compactedTopic: Boolean,
+                                                                  timestampType: TimestampType,
+                                                                  timestampDiffMaxMs: Long): ByteBufferMessageSet = {
+    // do in-place validation and offset assignment
+    var messagePosition = 0
+    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.setTimestampType(message.attributes, TimestampType.LOG_APPEND_TIME))
+          Utils.writeUnsignedInt(message.buffer, Message.CrcOffset, message.computeChecksum())
+        }
+      }
+      messagePosition += MessageSet.LogOverhead + messageSize
+    }
+    buffer.reset()
+    this
+  }
+
+  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.")
+  }
 
   /**
    * The total number of bytes in this message set, including any partial trailing messages

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 999b115..51aa11a 100755
--- a/core/src/main/scala/kafka/message/Message.scala
+++ b/core/src/main/scala/kafka/message/Message.scala
@@ -18,6 +18,9 @@
 package kafka.message
 
 import java.nio._
+
+import org.apache.kafka.common.record.TimestampType
+
 import scala.math._
 import kafka.utils._
 import org.apache.kafka.common.utils.Utils
@@ -26,7 +29,7 @@ import org.apache.kafka.common.utils.Utils
  * Constants related to messages
  */
 object Message {
-  
+
   /**
    * The current offset and size for all the fixed-length fields
    */
@@ -36,83 +39,144 @@ object Message {
   val MagicLength = 1
   val AttributesOffset = MagicOffset + MagicLength
   val AttributesLength = 1
-  val KeySizeOffset = AttributesOffset + AttributesLength
+  // Only message format version 1 has the timestamp field.
+  val TimestampOffset = AttributesOffset + AttributesLength
+  val TimestampLength = 8
+  val KeySizeOffset_V0 = AttributesOffset + AttributesLength
+  val KeySizeOffset_V1 = TimestampOffset + TimestampLength
   val KeySizeLength = 4
-  val KeyOffset = KeySizeOffset + KeySizeLength
+  val KeyOffset_V0 = KeySizeOffset_V0 + KeySizeLength
+  val KeyOffset_V1 = KeySizeOffset_V1 + KeySizeLength
   val ValueSizeLength = 4
-  
-  /** The amount of overhead bytes in a message */
-  val MessageOverhead = KeyOffset + ValueSizeLength
-  
+
+  private val MessageHeaderSizeMap = Map (
+    0.asInstanceOf[Byte] -> (CrcLength + MagicLength + AttributesLength + KeySizeLength + ValueSizeLength),
+    1.asInstanceOf[Byte] -> (CrcLength + MagicLength + AttributesLength + TimestampLength + KeySizeLength + ValueSizeLength))
+
   /**
-   * The minimum valid size for the message header
+   * The amount of overhead bytes in a message
+   * This value is only used to check if the message size is valid or not. So the minimum possible message bytes is
+   * used here, which comes from a message in message format V0 with empty key and value.
    */
-  val MinHeaderSize = CrcLength + MagicLength + AttributesLength + KeySizeLength + ValueSizeLength
+  val MinMessageOverhead = KeyOffset_V0 + ValueSizeLength
   
   /**
-   * The current "magic" value
+   * The "magic" value
+   * When magic value is 0, the message uses absolute offset and does not have a timestamp field.
+   * When magic value is 1, the message uses relative offset and has a timestamp field.
    */
-  val CurrentMagicValue: Byte = 0
+  val MagicValue_V0: Byte = 0
+  val MagicValue_V1: Byte = 1
+  val CurrentMagicValue: Byte = 1
 
   /**
    * Specifies the mask for the compression code. 3 bits to hold the compression codec.
    * 0 is reserved to indicate no compression
    */
   val CompressionCodeMask: Int = 0x07
+  /**
+   * Specifies the mask for timestamp type. 1 bit at the 4th least significant bit.
+   * 0 for CreateTime, 1 for LogAppendTime
+   */
+  val TimestampTypeMask: Byte = 0x08
+  val TimestampTypeAttributeBitOffset: Int = 3
 
   /**
    * Compression code for uncompressed messages
    */
   val NoCompression: Int = 0
 
+  /**
+   * To indicate timestamp is not defined so "magic" value 0 will be used.
+   */
+  val NoTimestamp: Long = -1
+
+  /**
+   * Give the header size difference between different message versions.
+   */
+  def headerSizeDiff(fromMagicValue: Byte, toMagicValue: Byte) : Int =
+    MessageHeaderSizeMap(toMagicValue) - MessageHeaderSizeMap(fromMagicValue)
+
+
 }
 
 /**
  * A message. The format of an N byte message is the following:
  *
  * 1. 4 byte CRC32 of the message
- * 2. 1 byte "magic" identifier to allow format changes, value is 0 currently
- * 3. 1 byte "attributes" identifier to allow annotations on the message independent of the version (e.g. compression enabled, type of codec used)
- * 4. 4 byte key length, containing length K
- * 5. K byte key
- * 6. 4 byte payload length, containing length V
- * 7. V byte payload
+ * 2. 1 byte "magic" identifier to allow format changes, value is 0 or 1
+ * 3. 1 byte "attributes" identifier to allow annotations on the message independent of the version
+ *    bit 0 ~ 2 : Compression codec.
+ *      0 : no compression
+ *      1 : gzip
+ *      2 : snappy
+ *      3 : lz4
+ *    bit 3 : Timestamp type
+ *      0 : create time
+ *      1 : log append time
+ *    bit 4 ~ 7 : reserved
+ * 4. (Optional) 8 byte timestamp only if "magic" identifier is greater than 0
+ * 5. 4 byte key length, containing length K
+ * 6. K byte key
+ * 7. 4 byte payload length, containing length V
+ * 8. V byte payload
  *
  * Default constructor wraps an existing ByteBuffer with the Message object with no change to the contents.
+ * @param buffer the byte buffer of this message.
+ * @param wrapperMessageTimestamp the wrapper message timestamp, only not None when the message is an inner message
+ *                                of a compressed message.
+ * @param wrapperMessageTimestampType the wrapper message timestamp type, only not None when the message is an inner
+ *                                    message of a compressed message.
  */
-class Message(val buffer: ByteBuffer) {
+class Message(val buffer: ByteBuffer,
+              private val wrapperMessageTimestamp: Option[Long] = None,
+              private val wrapperMessageTimestampType: Option[TimestampType] = None) {
   
   import kafka.message.Message._
-  
+
   /**
    * A constructor to create a Message
    * @param bytes The payload of the message
-   * @param codec The compression codec used on the contents of the message (if any)
    * @param key The key of the message (null, if none)
+   * @param timestamp The timestamp of the message.
+   * @param timestampType The timestamp type of the message.
+   * @param codec The compression codec used on the contents of the message (if any)
    * @param payloadOffset The offset into the payload array used to extract payload
    * @param payloadSize The size of the payload to use
+   * @param magicValue the magic value to use
    */
   def this(bytes: Array[Byte], 
-           key: Array[Byte],            
+           key: Array[Byte],
+           timestamp: Long,
+           timestampType: TimestampType,
            codec: CompressionCodec, 
            payloadOffset: Int, 
-           payloadSize: Int) = {
-    this(ByteBuffer.allocate(Message.CrcLength + 
-                             Message.MagicLength + 
-                             Message.AttributesLength + 
+           payloadSize: Int,
+           magicValue: Byte) = {
+    this(ByteBuffer.allocate(Message.CrcLength +
+                             Message.MagicLength +
+                             Message.AttributesLength +
+                             (if (magicValue == Message.MagicValue_V0) 0
+                              else Message.TimestampLength) +
                              Message.KeySizeLength + 
                              (if(key == null) 0 else key.length) + 
                              Message.ValueSizeLength + 
                              (if(bytes == null) 0 
                               else if(payloadSize >= 0) payloadSize 
                               else bytes.length - payloadOffset)))
+    validateTimestampAndMagicValue(timestamp, magicValue)
     // skip crc, we will fill that in at the end
     buffer.position(MagicOffset)
-    buffer.put(CurrentMagicValue)
+    buffer.put(magicValue)
     var attributes: Byte = 0
-    if (codec.codec > 0)
-      attributes =  (attributes | (CompressionCodeMask & codec.codec)).toByte
+    if (codec.codec > 0) {
+      attributes = (attributes | (CompressionCodeMask & codec.codec)).toByte
+      attributes = TimestampType.setTimestampType(attributes, timestampType)
+    }
     buffer.put(attributes)
+    // Only put timestamp when "magic" value is greater than 0
+    if (magic > MagicValue_V0)
+      buffer.putLong(timestamp)
     if(key == null) {
       buffer.putInt(-1)
     } else {
@@ -126,22 +190,25 @@ class Message(val buffer: ByteBuffer) {
     if(bytes != null)
       buffer.put(bytes, payloadOffset, size)
     buffer.rewind()
-    
+
     // now compute the checksum and fill it in
     Utils.writeUnsignedInt(buffer, CrcOffset, computeChecksum)
   }
   
-  def this(bytes: Array[Byte], key: Array[Byte], codec: CompressionCodec) = 
-    this(bytes = bytes, key = key, codec = codec, payloadOffset = 0, payloadSize = -1)
+  def this(bytes: Array[Byte], key: Array[Byte], timestamp: Long, codec: CompressionCodec, magicValue: Byte) =
+    this(bytes = bytes, key = key, timestamp = timestamp, timestampType = TimestampType.CREATE_TIME, codec = codec, payloadOffset = 0, payloadSize = -1, magicValue = magicValue)
   
-  def this(bytes: Array[Byte], codec: CompressionCodec) = 
-    this(bytes = bytes, key = null, codec = codec)
+  def this(bytes: Array[Byte], timestamp: Long, codec: CompressionCodec, magicValue: Byte) =
+    this(bytes = bytes, key = null, timestamp = timestamp, codec = codec, magicValue = magicValue)
   
-  def this(bytes: Array[Byte], key: Array[Byte]) = 
-    this(bytes = bytes, key = key, codec = NoCompressionCodec)
+  def this(bytes: Array[Byte], key: Array[Byte], timestamp: Long, magicValue: Byte) =
+    this(bytes = bytes, key = key, timestamp = timestamp, codec = NoCompressionCodec, magicValue = magicValue)
     
-  def this(bytes: Array[Byte]) = 
-    this(bytes = bytes, key = null, codec = NoCompressionCodec)
+  def this(bytes: Array[Byte], timestamp: Long, magicValue: Byte) =
+    this(bytes = bytes, key = null, timestamp = timestamp, codec = NoCompressionCodec, magicValue = magicValue)
+
+  def this(bytes: Array[Byte]) =
+    this(bytes = bytes, key = null, timestamp = Message.NoTimestamp, codec = NoCompressionCodec, magicValue = Message.CurrentMagicValue)
     
   /**
    * Compute the checksum of the message from the message contents
@@ -171,11 +238,19 @@ class Message(val buffer: ByteBuffer) {
    * The complete serialized size of this message in bytes (including crc, header attributes, etc)
    */
   def size: Int = buffer.limit
-  
+
+  /**
+   * The position where the key size is stored.
+   */
+  def keySizeOffset = {
+    if (magic == MagicValue_V0) KeySizeOffset_V0
+    else KeySizeOffset_V1
+  }
+
   /**
    * The length of the key in bytes
    */
-  def keySize: Int = buffer.getInt(Message.KeySizeOffset)
+  def keySize: Int = buffer.getInt(keySizeOffset)
   
   /**
    * Does the message have a key?
@@ -185,7 +260,10 @@ class Message(val buffer: ByteBuffer) {
   /**
    * The position where the payload size is stored
    */
-  private def payloadSizeOffset = Message.KeyOffset + max(0, keySize)
+  def payloadSizeOffset = {
+    if (magic == MagicValue_V0) KeyOffset_V0 + max(0, keySize)
+    else KeyOffset_V1 + max(0, keySize)
+  }
   
   /**
    * The length of the message value in bytes
@@ -206,6 +284,33 @@ class Message(val buffer: ByteBuffer) {
    * The attributes stored with this message
    */
   def attributes: Byte = buffer.get(AttributesOffset)
+
+  /**
+   * The timestamp of the message, only available when the "magic" value is greater than 0
+   * When magic > 0, The timestamp of a message is determined in the following way:
+   * 1. wrapperMessageTimestampType = None and wrapperMessageTimestamp is None - Uncompressed message, timestamp and timestamp type are in the message.
+   * 2. wrapperMessageTimestampType = LogAppendTime and wrapperMessageTimestamp is defined - Compressed message using LogAppendTime
+   * 3. wrapperMessageTimestampType = CreateTime and wrapperMessageTimestamp is defined - Compressed message using CreateTime
+   */
+  def timestamp: Long = {
+    if (magic == MagicValue_V0)
+      Message.NoTimestamp
+    // Case 2
+    else if (wrapperMessageTimestampType.exists(_ == TimestampType.LOG_APPEND_TIME) && wrapperMessageTimestamp.isDefined)
+      wrapperMessageTimestamp.get
+    else // case 1, 3
+      buffer.getLong(Message.TimestampOffset)
+  }
+
+  /**
+   * The timestamp type of the message
+   */
+  def timestampType = {
+    if (magic == MagicValue_V0)
+      TimestampType.NO_TIMESTAMP_TYPE
+    else
+      wrapperMessageTimestampType.getOrElse(TimestampType.getTimestampType(attributes))
+  }
   
   /**
    * The compression codec used with this message
@@ -221,8 +326,54 @@ class Message(val buffer: ByteBuffer) {
   /**
    * A ByteBuffer containing the message key
    */
-  def key: ByteBuffer = sliceDelimited(KeySizeOffset)
-  
+  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)
+      new Message(byteBuffer)
+    }
+  }
+
+  def convertToBuffer(toMagicValue: Byte,
+                      byteBuffer: ByteBuffer,
+                      now: Long = NoTimestamp,
+                      timestampType: TimestampType = wrapperMessageTimestampType.getOrElse(TimestampType.getTimestampType(attributes))) {
+    if (byteBuffer.remaining() < size + headerSizeDiff(magic, toMagicValue))
+      throw new IndexOutOfBoundsException("The byte buffer does not have enough capacity to hold new message format " +
+        "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.setTimestampType(attributes, timestampType))
+      // 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.setTimestampType(attributes, TimestampType.CREATE_TIME))
+      // 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
    */
@@ -240,9 +391,25 @@ class Message(val buffer: ByteBuffer) {
     }
   }
 
-  override def toString(): String = 
-    "Message(magic = %d, attributes = %d, crc = %d, key = %s, payload = %s)".format(magic, attributes, checksum, key, payload)
-  
+  /**
+   * Validate the timestamp and "magic" value
+   */
+  private def validateTimestampAndMagicValue(timestamp: Long, magic: Byte) {
+    if (magic != MagicValue_V0 && magic != MagicValue_V1)
+      throw new IllegalArgumentException("Invalid magic value " + magic)
+    if (timestamp < 0 && timestamp != NoTimestamp)
+      throw new IllegalArgumentException("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}")
+  }
+
+  override def toString(): String = {
+    if (magic == MagicValue_V0)
+      s"Message(magic = $magic, attributes = $attributes, crc = $checksum, key = $key, payload = $payload)"
+    else
+      s"Message(magic = $magic, attributes = $attributes, $timestampType = $timestamp, crc = $checksum, key = $key, payload = $payload)"
+  }
+
   override def equals(any: Any): Boolean = {
     any match {
       case that: Message => this.buffer.equals(that.buffer)
@@ -251,5 +418,5 @@ class Message(val buffer: ByteBuffer) {
   }
   
   override def hashCode(): Int = buffer.hashCode
-  
+
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/main/scala/kafka/message/MessageAndMetadata.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/message/MessageAndMetadata.scala b/core/src/main/scala/kafka/message/MessageAndMetadata.scala
index 26b75c8..ac9ef77 100755
--- a/core/src/main/scala/kafka/message/MessageAndMetadata.scala
+++ b/core/src/main/scala/kafka/message/MessageAndMetadata.scala
@@ -18,12 +18,17 @@
 package kafka.message
 
 import kafka.serializer.Decoder
+import org.apache.kafka.common.record.TimestampType
 import org.apache.kafka.common.utils.Utils
 
-case class MessageAndMetadata[K, V](topic: String, partition: Int,
-                                    private val rawMessage: Message, offset: Long,
+case class MessageAndMetadata[K, V](topic: String,
+                                    partition: Int,
+                                    private val rawMessage: Message,
+                                    offset: Long,
+                                    timestamp: Long = Message.NoTimestamp,
+                                    timestampType: TimestampType = TimestampType.CREATE_TIME,
                                     keyDecoder: Decoder[K], valueDecoder: Decoder[V]) {
-
+  
   /**
    * Return the decoded message key and payload
    */

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 28b56e6..014788a 100644
--- a/core/src/main/scala/kafka/message/MessageSet.scala
+++ b/core/src/main/scala/kafka/message/MessageSet.scala
@@ -54,8 +54,26 @@ object MessageSet {
    */
   def entrySize(message: Message): Int = LogOverhead + message.size
 
+  /**
+   * Validate the "magic" values of messages are the same in a compressed message set and return the magic value of
+   * and the max timestamp of the inner messages.
+   */
+  def magicAndLargestTimestamp(messages: Seq[Message]): MagicAndTimestamp = {
+    val firstMagicValue = messages.head.magic
+    var largestTimestamp: Long = Message.NoTimestamp
+    for (message <- messages) {
+      if (message.magic != firstMagicValue)
+        throw new IllegalStateException("Messages in the same message set must have same magic value")
+      if (firstMagicValue > Message.MagicValue_V0)
+        largestTimestamp = math.max(largestTimestamp, message.timestamp)
+    }
+    MagicAndTimestamp(firstMagicValue, largestTimestamp)
+  }
+
 }
 
+case class MagicAndTimestamp(magic: Byte, timestamp: Long)
+
 /**
  * A set of messages with offsets. A message set has a fixed serialized form, though the container
  * for the bytes could be either in-memory or on disk. The format of each message is
@@ -70,7 +88,12 @@ abstract class MessageSet extends Iterable[MessageAndOffset] {
     * Less than the complete amount may be written, but no more than maxSize can be. The number
     * of bytes written is returned */
   def writeTo(channel: GatheringByteChannel, offset: Long, maxSize: Int): Int
-  
+
+  /**
+   * Check if all the wrapper messages in the message set have the expected magic value
+   */
+  def magicValueInAllWrapperMessages(expectedMagicValue: Byte): Boolean
+
   /**
    * Provides an iterator over the message/offset pairs in this set
    */

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/main/scala/kafka/message/MessageWriter.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/message/MessageWriter.scala b/core/src/main/scala/kafka/message/MessageWriter.scala
index 0c6040e..660772c 100755
--- a/core/src/main/scala/kafka/message/MessageWriter.scala
+++ b/core/src/main/scala/kafka/message/MessageWriter.scala
@@ -20,19 +20,31 @@ package kafka.message
 import java.io.{InputStream, OutputStream}
 import java.nio.ByteBuffer
 
+import org.apache.kafka.common.record.TimestampType
 import org.apache.kafka.common.utils.Crc32
 
 class MessageWriter(segmentSize: Int) extends BufferingOutputStream(segmentSize) {
 
   import Message._
 
-  def write(key: Array[Byte] = null, codec: CompressionCodec)(writePayload: OutputStream => Unit): Unit = {
+  def write(key: Array[Byte] = null,
+            codec: CompressionCodec,
+            timestamp: Long,
+            timestampType: TimestampType,
+            magicValue: Byte)(writePayload: OutputStream => Unit): Unit = {
     withCrc32Prefix {
-      write(CurrentMagicValue)
+      // write magic value
+      write(magicValue)
+      // write attributes
       var attributes: Byte = 0
       if (codec.codec > 0)
         attributes = (attributes | (CompressionCodeMask & codec.codec)).toByte
+      if (magicValue > MagicValue_V0)
+        attributes = TimestampType.setTimestampType(attributes, timestampType)
       write(attributes)
+      // Write timestamp
+      if (magicValue > MagicValue_V0)
+        writeLong(timestamp)
       // write the key
       if (key == null) {
         writeInt(-1)
@@ -61,6 +73,17 @@ class MessageWriter(segmentSize: Int) extends BufferingOutputStream(segmentSize)
     out.write(value)
   }
 
+  private def writeLong(value: Long): Unit = {
+    write((value >>> 56).toInt)
+    write((value >>> 48).toInt)
+    write((value >>> 40).toInt)
+    write((value >>> 32).toInt)
+    write((value >>> 24).toInt)
+    write((value >>> 16).toInt)
+    write((value >>> 8).toInt)
+    write(value.toInt)
+  }
+
   private def withCrc32Prefix(writeData: => Unit): Unit = {
     // get a writer for CRC value
     val crcWriter = reserve(CrcLength)

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala
index 4e67ba4..7abe48a 100755
--- a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala
+++ b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala
@@ -21,7 +21,7 @@ import kafka.common._
 import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet}
 import kafka.producer._
 import kafka.serializer.Encoder
-import kafka.utils.{CoreUtils, Logging, SystemTime}
+import kafka.utils._
 import org.apache.kafka.common.errors.{LeaderNotAvailableException, UnknownTopicOrPartitionException}
 import org.apache.kafka.common.protocol.Errors
 import scala.util.Random
@@ -36,8 +36,10 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
                                private val encoder: Encoder[V],
                                private val keyEncoder: Encoder[K],
                                private val producerPool: ProducerPool,
-                               private val topicPartitionInfos: HashMap[String, TopicMetadata] = new HashMap[String, TopicMetadata])
+                               private val topicPartitionInfos: HashMap[String, TopicMetadata] = new HashMap[String, TopicMetadata],
+                               private val time: Time = SystemTime)
   extends EventHandler[K,V] with Logging {
+
   val isSync = ("sync" == config.producerType)
 
   val correlationId = new AtomicInteger(0)
@@ -129,9 +131,22 @@ class DefaultEventHandler[K,V](config: ProducerConfig,
     events.foreach{e =>
       try {
         if(e.hasKey)
-          serializedMessages += new KeyedMessage[K,Message](topic = e.topic, key = e.key, partKey = e.partKey, message = new Message(key = keyEncoder.toBytes(e.key), bytes = encoder.toBytes(e.message)))
+          serializedMessages += new KeyedMessage[K,Message](
+            topic = e.topic,
+            key = e.key,
+            partKey = e.partKey,
+            message = new Message(key = keyEncoder.toBytes(e.key),
+                                  bytes = encoder.toBytes(e.message),
+                                  timestamp = time.milliseconds,
+                                  magicValue = Message.MagicValue_V1))
         else
-          serializedMessages += new KeyedMessage[K,Message](topic = e.topic, key = e.key, partKey = e.partKey, message = new Message(bytes = encoder.toBytes(e.message)))
+          serializedMessages += new KeyedMessage[K,Message](
+            topic = e.topic,
+            key = e.key,
+            partKey = e.partKey,
+            message = new Message(bytes = encoder.toBytes(e.message),
+                                  timestamp = time.milliseconds,
+                                  magicValue = Message.MagicValue_V1))
       } catch {
         case t: Throwable =>
           producerStats.serializationErrorRate.mark()

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 b3873a6..8b688b9 100755
--- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
+++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
@@ -104,7 +104,7 @@ abstract class AbstractFetcherThread(name: String,
     } catch {
       case t: Throwable =>
         if (isRunning.get) {
-          warn("Error in fetch %s. Possible cause: %s".format(fetchRequest, t.toString))
+          warn(s"Error in fetch $fetchRequest", t)
           inLock(partitionMapLock) {
             partitionsWithError ++= partitionMap.keys
             // there is an error occurred while fetching partitions, sleep a while

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/main/scala/kafka/server/ConfigHandler.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala
index bc599a0..9343fde 100644
--- a/core/src/main/scala/kafka/server/ConfigHandler.scala
+++ b/core/src/main/scala/kafka/server/ConfigHandler.scala
@@ -19,8 +19,10 @@ package kafka.server
 
 import java.util.Properties
 
+import kafka.api.ApiVersion
 import kafka.common.TopicAndPartition
 import kafka.log.{Log, LogConfig, LogManager}
+import kafka.utils.Logging
 import org.apache.kafka.common.metrics.Quota
 import org.apache.kafka.common.protocol.ApiKeys
 
@@ -38,12 +40,22 @@ trait ConfigHandler {
  * The TopicConfigHandler will process topic config changes in ZK.
  * The callback provides the topic name and the full properties set read from ZK
  */
-class TopicConfigHandler(private val logManager: LogManager) extends ConfigHandler {
+class TopicConfigHandler(private val logManager: LogManager, kafkaConfig: KafkaConfig) extends ConfigHandler with Logging {
 
   def processConfigChanges(topic : String, topicConfig : Properties) {
     val logs: mutable.Buffer[(TopicAndPartition, Log)] = logManager.logsByTopicPartition.toBuffer
     val logsByTopic: Map[String, mutable.Buffer[Log]] = logs.groupBy{ case (topicAndPartition, log) => topicAndPartition.topic }
             .mapValues{ case v: mutable.Buffer[(TopicAndPartition, Log)] => v.map(_._2) }
+    // Validate the compatibility of message format version.
+    Option(topicConfig.getProperty(LogConfig.MessageFormatVersionProp)) match {
+      case Some(versionString) =>
+        if (!kafkaConfig.interBrokerProtocolVersion.onOrAfter(ApiVersion(versionString))) {
+          topicConfig.remove(LogConfig.MessageFormatVersionProp)
+          warn(s"Log configuration ${LogConfig.MessageFormatVersionProp} is ignored for $topic because $versionString " +
+            s"is not compatible with Kafka inter broker protocol version ${kafkaConfig.interBrokerProtocolVersion}")
+        }
+      case _ =>
+    }
 
     if (logsByTopic.contains(topic)) {
       /* combine the default properties with the overrides in zk to create the new LogConfig */

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 f2e9533..bd02630 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -27,7 +27,7 @@ import kafka.common._
 import kafka.controller.KafkaController
 import kafka.coordinator.{GroupCoordinator, JoinGroupResult}
 import kafka.log._
-import kafka.message.{ByteBufferMessageSet, MessageSet}
+import kafka.message.{ByteBufferMessageSet, Message, MessageSet}
 import kafka.network._
 import kafka.network.RequestChannel.{Session, Response}
 import kafka.security.auth.{Authorizer, ClusterAction, Group, Create, Describe, Operation, Read, Resource, Topic, Write}
@@ -35,7 +35,7 @@ import kafka.utils.{Logging, SystemTime, ZKGroupTopicDirs, ZkUtils}
 import org.apache.kafka.common.errors.{InvalidTopicException, NotLeaderForPartitionException, UnknownTopicOrPartitionException,
 ClusterAuthorizationException}
 import org.apache.kafka.common.metrics.Metrics
-import org.apache.kafka.common.protocol.{ProtoUtils, ApiKeys, Errors, SecurityProtocol}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors, SecurityProtocol}
 import org.apache.kafka.common.requests.{ListOffsetRequest, ListOffsetResponse, GroupCoordinatorRequest, GroupCoordinatorResponse, ListGroupsResponse,
 DescribeGroupsRequest, DescribeGroupsResponse, HeartbeatRequest, HeartbeatResponse, JoinGroupRequest, JoinGroupResponse,
 LeaveGroupRequest, LeaveGroupResponse, ResponseHeader, ResponseSend, SyncGroupRequest, SyncGroupResponse, LeaderAndIsrRequest, LeaderAndIsrResponse,
@@ -330,7 +330,8 @@ class KafkaApis(val requestChannel: RequestChannel,
     // the callback for sending a produce response
     def sendResponseCallback(responseStatus: Map[TopicPartition, PartitionResponse]) {
 
-      val mergedResponseStatus = responseStatus ++ unauthorizedRequestInfo.mapValues(_ => new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED.code, -1))
+      val mergedResponseStatus = responseStatus ++ unauthorizedRequestInfo.mapValues(_ =>
+        new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED.code, -1, Message.NoTimestamp))
 
       var errorInResponse = false
 
@@ -367,7 +368,8 @@ class KafkaApis(val requestChannel: RequestChannel,
           val respHeader = new ResponseHeader(request.header.correlationId)
           val respBody = request.header.apiVersion match {
             case 0 => new ProduceResponse(mergedResponseStatus.asJava)
-            case 1 => new ProduceResponse(mergedResponseStatus.asJava, delayTimeMs)
+            case 1 => new ProduceResponse(mergedResponseStatus.asJava, delayTimeMs, 1)
+            case 2 => new ProduceResponse(mergedResponseStatus.asJava, delayTimeMs, 2)
             // This case shouldn't happen unless a new version of ProducerRequest is added without
             // updating this part of the code to handle it properly.
             case _ => throw new IllegalArgumentException("Version %d of ProducerRequest is not handled. Code must be updated."
@@ -426,7 +428,32 @@ class KafkaApis(val requestChannel: RequestChannel,
 
     // the callback for sending a fetch response
     def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData]) {
-      val mergedResponseStatus = responsePartitionData ++ unauthorizedResponseStatus
+
+      val convertedResponseStatus =
+        // Need to down-convert message when consumer only takes magic value 0.
+        if (fetchRequest.versionId <= 1) {
+          responsePartitionData.map({ case (tp, data) =>
+            tp -> {
+              // We only do down-conversion when:
+              // 1. The message format version configured for the topic is using magic value > 0, and
+              // 2. The message set contains message whose magic > 0
+              // This is to reduce the message format conversion as much as possible. The conversion will only occur
+              // when new message format is used for the topic and we see an old request.
+              // Please notice 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.
+              if (replicaManager.getMessageFormatVersion(tp).exists(_ > Message.MagicValue_V0) &&
+                  !data.messages.magicValueInAllWrapperMessages(Message.MagicValue_V0)) {
+                trace("Down converting message to V0 for fetch request from " + fetchRequest.clientId)
+                new FetchResponsePartitionData(data.error, data.hw, data.messages.asInstanceOf[FileMessageSet].toMessageFormat(Message.MagicValue_V0))
+              } else
+                data
+            }
+          })
+        } else
+          responsePartitionData
+
+      val mergedResponseStatus = convertedResponseStatus ++ unauthorizedResponseStatus
 
       mergedResponseStatus.foreach { case (topicAndPartition, data) =>
         if (data.error != Errors.NONE.code) {
@@ -440,6 +467,8 @@ class KafkaApis(val requestChannel: RequestChannel,
       }
 
       def fetchResponseCallback(delayTimeMs: Int) {
+          trace(s"Sending fetch response to ${fetchRequest.clientId} with ${convertedResponseStatus.values.map(_.messages.sizeInBytes).sum}" +
+            s" bytes")
         val response = FetchResponse(fetchRequest.correlationId, mergedResponseStatus, fetchRequest.versionId, delayTimeMs)
         requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, response)))
       }
@@ -453,10 +482,9 @@ class KafkaApis(val requestChannel: RequestChannel,
         fetchResponseCallback(0)
       } else {
         quotaManagers(ApiKeys.FETCH.id).recordAndMaybeThrottle(fetchRequest.clientId,
-                                                                   FetchResponse.responseSize(responsePartitionData
-                                                                                                      .groupBy(_._1.topic),
-                                                                                              fetchRequest.versionId),
-                                                                   fetchResponseCallback)
+                                                               FetchResponse.responseSize(responsePartitionData.groupBy(_._1.topic),
+                                                                                          fetchRequest.versionId),
+                                                               fetchResponseCallback)
       }
     }
 


[2/5] kafka git commit: KAFKA-3025; Added timetamp to Message and use relative offset.

Posted by ju...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/main/scala/kafka/server/KafkaConfig.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala
index 2c6311c..ae810eb 100755
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -94,9 +94,12 @@ object Defaults {
   val LogFlushSchedulerIntervalMs = Long.MaxValue
   val LogFlushOffsetCheckpointIntervalMs = 60000
   val LogPreAllocateEnable = false
+  val MessageFormatVersion = ApiVersion.latestVersion.toString()
   val NumRecoveryThreadsPerDataDir = 1
   val AutoCreateTopicsEnable = true
   val MinInSyncReplicas = 1
+  val MessageTimestampType = "CreateTime"
+  val MessageTimestampDifferenceMaxMs = Long.MaxValue
 
   /** ********* Replication configuration ***********/
   val ControllerSocketTimeoutMs = RequestTimeoutMs
@@ -251,9 +254,12 @@ object KafkaConfig {
   val LogFlushIntervalMsProp = "log.flush.interval.ms"
   val LogFlushOffsetCheckpointIntervalMsProp = "log.flush.offset.checkpoint.interval.ms"
   val LogPreAllocateProp = "log.preallocate"
+  val MessageFormatVersionProp = "message.format.version"
   val NumRecoveryThreadsPerDataDirProp = "num.recovery.threads.per.data.dir"
   val AutoCreateTopicsEnableProp = "auto.create.topics.enable"
   val MinInSyncReplicasProp = "min.insync.replicas"
+  val MessageTimestampTypeProp = "message.timestamp.type"
+  val MessageTimestampDifferenceMaxMsProp = "message.timestamp.difference.max.ms"
   /** ********* Replication configuration ***********/
   val ControllerSocketTimeoutMsProp = "controller.socket.timeout.ms"
   val DefaultReplicationFactorProp = "default.replication.factor"
@@ -417,6 +423,14 @@ object KafkaConfig {
   val NumRecoveryThreadsPerDataDirDoc = "The number of threads per data directory to be used for log recovery at startup and flushing at shutdown"
   val AutoCreateTopicsEnableDoc = "Enable auto creation of topic on the server"
   val MinInSyncReplicasDoc = "define the minimum number of replicas in ISR needed to satisfy a produce request with acks=all (or -1)"
+  val MessageFormatVersionDoc = "Specify the message format version the broker will use to append messages to the logs. The value should be a valid ApiVersion." +
+  "Some Examples are: 0.8.2, 0.9.0.0, 0.10.0. Check ApiVersion for detail. When setting the message format version, " +
+  "user certifies that all the existing messages on disk is at or below that version. Otherwise consumers before 0.10.0.0 will break."
+  val MessageTimestampTypeDoc = "Define whether the timestamp in the message is message create time or log append time. The value should be either" +
+  " \"CreateTime\" or \"LogAppendTime\""
+  val MessageTimestampDifferenceMaxMsDoc = "The maximum difference allowed between the timestamp when a broker receives " +
+  "a message and the timestamp specified in the message. If message.timestamp.type=CreateTime, a message will be rejected " +
+  "if the difference in timestamp exceeds this threshold. This configuration is ignored if message.timestamp.type=LogAppendTime."
   /** ********* Replication configuration ***********/
   val ControllerSocketTimeoutMsDoc = "The socket timeout for controller-to-broker channels"
   val ControllerMessageQueueSizeDoc = "The buffer size for controller-to-broker-channels"
@@ -556,7 +570,7 @@ object KafkaConfig {
       .define(NumPartitionsProp, INT, Defaults.NumPartitions, atLeast(1), MEDIUM, NumPartitionsDoc)
       .define(LogDirProp, STRING, Defaults.LogDir, HIGH, LogDirDoc)
       .define(LogDirsProp, STRING, null, HIGH, LogDirsDoc)
-      .define(LogSegmentBytesProp, INT, Defaults.LogSegmentBytes, atLeast(Message.MinHeaderSize), HIGH, LogSegmentBytesDoc)
+      .define(LogSegmentBytesProp, INT, Defaults.LogSegmentBytes, atLeast(Message.MinMessageOverhead), HIGH, LogSegmentBytesDoc)
 
       .define(LogRollTimeMillisProp, LONG, null, HIGH, LogRollTimeMillisDoc)
       .define(LogRollTimeHoursProp, INT, Defaults.LogRollHours, atLeast(1), HIGH, LogRollTimeHoursDoc)
@@ -591,6 +605,9 @@ object KafkaConfig {
       .define(NumRecoveryThreadsPerDataDirProp, INT, Defaults.NumRecoveryThreadsPerDataDir, atLeast(1), HIGH, NumRecoveryThreadsPerDataDirDoc)
       .define(AutoCreateTopicsEnableProp, BOOLEAN, Defaults.AutoCreateTopicsEnable, HIGH, AutoCreateTopicsEnableDoc)
       .define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), HIGH, MinInSyncReplicasDoc)
+      .define(MessageFormatVersionProp, STRING, Defaults.MessageFormatVersion, MEDIUM, MessageFormatVersionDoc)
+      .define(MessageTimestampTypeProp, STRING, Defaults.MessageTimestampType, in("CreateTime", "LogAppendTime"), MEDIUM, MessageTimestampTypeDoc)
+      .define(MessageTimestampDifferenceMaxMsProp, LONG, Defaults.MessageTimestampDifferenceMaxMs, atLeast(0), MEDIUM, MessageTimestampDifferenceMaxMsDoc)
 
       /** ********* Replication configuration ***********/
       .define(ControllerSocketTimeoutMsProp, INT, Defaults.ControllerSocketTimeoutMs, MEDIUM, ControllerSocketTimeoutMsDoc)
@@ -781,6 +798,9 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean) extends Abstra
   val logRetentionTimeMillis = getLogRetentionTimeMillis
   val minInSyncReplicas = getInt(KafkaConfig.MinInSyncReplicasProp)
   val logPreAllocateEnable: java.lang.Boolean = getBoolean(KafkaConfig.LogPreAllocateProp)
+  val messageFormatVersion = getString(KafkaConfig.MessageFormatVersionProp)
+  val messageTimestampType = getString(KafkaConfig.MessageTimestampTypeProp)
+  val messageTimestampDifferenceMaxMs = getLong(KafkaConfig.MessageTimestampDifferenceMaxMsProp)
 
   /** ********* Replication configuration ***********/
   val controllerSocketTimeoutMs: Int = getInt(KafkaConfig.ControllerSocketTimeoutMsProp)
@@ -958,6 +978,7 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean) extends Abstra
       s"${KafkaConfig.AdvertisedListenersProp} protocols must be equal to or a subset of ${KafkaConfig.ListenersProp} protocols. " +
       s"Found ${advertisedListeners.keySet}. The valid options based on currently configured protocols are ${listeners.keySet}"
     )
+    require(interBrokerProtocolVersion.onOrAfter(ApiVersion(messageFormatVersion)),
+      s"message.format.version $messageFormatVersion cannot be used when inter.broker.protocol.version is set to $interBrokerProtocolVersion")
   }
-
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/main/scala/kafka/server/KafkaServer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala
index 41719e2..e3e185f 100755
--- a/core/src/main/scala/kafka/server/KafkaServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaServer.scala
@@ -21,7 +21,7 @@ import java.net.{SocketTimeoutException}
 import java.util
 
 import kafka.admin._
-import kafka.api.KAFKA_090
+import kafka.api.KAFKA_0_9_0
 import kafka.log.LogConfig
 import kafka.log.CleanerConfig
 import kafka.log.LogManager
@@ -75,6 +75,9 @@ object KafkaServer {
     logProps.put(LogConfig.CompressionTypeProp, kafkaConfig.compressionType)
     logProps.put(LogConfig.UncleanLeaderElectionEnableProp, kafkaConfig.uncleanLeaderElectionEnable)
     logProps.put(LogConfig.PreAllocateEnableProp, kafkaConfig.logPreAllocateEnable)
+    logProps.put(LogConfig.MessageFormatVersionProp, kafkaConfig.messageFormatVersion)
+    logProps.put(LogConfig.MessageTimestampTypeProp, kafkaConfig.messageTimestampType)
+    logProps.put(LogConfig.MessageTimestampDifferenceMaxMsProp, kafkaConfig.messageTimestampDifferenceMaxMs)
     logProps
   }
 }
@@ -197,7 +200,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr
         kafkaController.startup()
 
         /* start kafka coordinator */
-        consumerCoordinator = GroupCoordinator.create(config, zkUtils, replicaManager)
+        consumerCoordinator = GroupCoordinator.create(config, zkUtils, replicaManager, kafkaMetricsTime)
         consumerCoordinator.startup()
 
         /* Get the authorizer and initialize it if one is specified.*/
@@ -216,7 +219,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr
         Mx4jLoader.maybeLoad()
 
         /* start dynamic config manager */
-        dynamicConfigHandlers = Map[String, ConfigHandler](ConfigType.Topic -> new TopicConfigHandler(logManager),
+        dynamicConfigHandlers = Map[String, ConfigHandler](ConfigType.Topic -> new TopicConfigHandler(logManager, config),
                                                            ConfigType.Client -> new ClientIdConfigHandler(apis.quotaManagers))
 
         // Apply all existing client configs to the ClientIdConfigHandler to bootstrap the overrides
@@ -512,7 +515,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr
       val shutdownSucceeded =
         // Before 0.9.0.0, `ControlledShutdownRequest` did not contain `client_id` and it's a mandatory field in
         // `RequestHeader`, which is used by `NetworkClient`
-        if (config.interBrokerProtocolVersion.onOrAfter(KAFKA_090))
+        if (config.interBrokerProtocolVersion.onOrAfter(KAFKA_0_9_0))
           networkClientControlledShutdown(config.controlledShutdownMaxRetries.intValue)
         else blockingChannelControlledShutdown(config.controlledShutdownMaxRetries.intValue)
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 c5f3360..2fdb46c 100644
--- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
+++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
@@ -23,7 +23,7 @@ import kafka.admin.AdminUtils
 import kafka.cluster.BrokerEndPoint
 import kafka.log.LogConfig
 import kafka.message.ByteBufferMessageSet
-import kafka.api.KAFKA_090
+import kafka.api.{KAFKA_0_10_0_IV0, KAFKA_0_9_0}
 import kafka.common.{KafkaStorageException, TopicAndPartition}
 import ReplicaFetcherThread._
 import org.apache.kafka.clients.{ManualMetadataUpdater, NetworkClient, ClientRequest, ClientResponse}
@@ -55,7 +55,10 @@ class ReplicaFetcherThread(name: String,
   type REQ = FetchRequest
   type PD = PartitionData
 
-  private val fetchRequestVersion: Short = if (brokerConfig.interBrokerProtocolVersion.onOrAfter(KAFKA_090)) 1 else 0
+  private val fetchRequestVersion: Short =
+    if (brokerConfig.interBrokerProtocolVersion.onOrAfter(KAFKA_0_10_0_IV0)) 2
+    else if (brokerConfig.interBrokerProtocolVersion.onOrAfter(KAFKA_0_9_0)) 1
+    else 0
   private val socketTimeout: Int = brokerConfig.replicaSocketTimeoutMs
   private val replicaId = brokerConfig.brokerId
   private val maxWait = brokerConfig.replicaFetchWaitMaxMs

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 61b6887..16b8c3a 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -26,11 +26,12 @@ import kafka.cluster.{Partition, Replica}
 import kafka.common._
 import kafka.controller.KafkaController
 import kafka.log.{LogAppendInfo, LogManager}
-import kafka.message.{ByteBufferMessageSet, MessageSet}
+import kafka.message.{ByteBufferMessageSet, InvalidMessageException, Message, MessageSet}
 import kafka.metrics.KafkaMetricsGroup
 import kafka.utils._
 import org.apache.kafka.common.errors.{OffsetOutOfRangeException, RecordBatchTooLargeException, ReplicaNotAvailableException, RecordTooLargeException,
-InvalidTopicException, ControllerMovedException, NotLeaderForPartitionException, CorruptRecordException, UnknownTopicOrPartitionException}
+InvalidTopicException, ControllerMovedException, NotLeaderForPartitionException, CorruptRecordException, UnknownTopicOrPartitionException,
+InvalidTimestampException}
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.protocol.Errors
@@ -332,7 +333,7 @@ class ReplicaManager(val config: KafkaConfig,
         topicPartition ->
                 ProducePartitionStatus(
                   result.info.lastOffset + 1, // required offset
-                  new PartitionResponse(result.errorCode, result.info.firstOffset)) // response status
+                  new PartitionResponse(result.errorCode, result.info.firstOffset, result.info.timestamp)) // response status
       }
 
       if (delayedRequestRequired(requiredAcks, messagesPerPartition, localProduceResults)) {
@@ -358,9 +359,9 @@ class ReplicaManager(val config: KafkaConfig,
       // Just return an error and don't handle the request at all
       val responseStatus = messagesPerPartition.map {
         case (topicAndPartition, messageSet) =>
-          (topicAndPartition ->
-                  new PartitionResponse(Errors.INVALID_REQUIRED_ACKS.code,
-                    LogAppendInfo.UnknownLogAppendInfo.firstOffset))
+          (topicAndPartition -> new PartitionResponse(Errors.INVALID_REQUIRED_ACKS.code,
+                                                      LogAppendInfo.UnknownLogAppendInfo.firstOffset,
+                                                      Message.NoTimestamp))
       }
       responseCallback(responseStatus)
     }
@@ -440,6 +441,10 @@ class ReplicaManager(val config: KafkaConfig,
             (topicPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(mstle)))
           case imse: CorruptRecordException =>
             (topicPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(imse)))
+          case ime : InvalidMessageException =>
+            (topicPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(ime)))
+          case itse : InvalidTimestampException =>
+            (topicPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(itse)))
           case t: Throwable =>
             BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).failedProduceRequestRate.mark()
             BrokerTopicStats.getBrokerAllTopicsStats.failedProduceRequestRate.mark()
@@ -568,6 +573,10 @@ class ReplicaManager(val config: KafkaConfig,
     }
   }
 
+  def getMessageFormatVersion(topicAndPartition: TopicAndPartition): Option[Byte] = {
+    getReplica(topicAndPartition.topic, topicAndPartition.partition).flatMap(_.log.map(_.config.messageFormatVersion))
+  }
+
   def maybeUpdateMetadataCache(correlationId: Int, updateMetadataRequest: UpdateMetadataRequest, metadataCache: MetadataCache) {
     replicaStateChangeLock synchronized {
       if(updateMetadataRequest.controllerEpoch < controllerEpoch) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/main/scala/kafka/tools/ConsoleConsumer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala
index 73743aa..fe2ce9f 100755
--- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala
+++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala
@@ -28,6 +28,7 @@ import kafka.metrics.KafkaMetricsReporter
 import kafka.utils._
 import org.apache.kafka.clients.consumer.ConsumerConfig
 import org.apache.kafka.common.errors.WakeupException
+import org.apache.kafka.common.record.TimestampType
 import org.apache.kafka.common.utils.Utils
 import org.apache.log4j.Logger
 
@@ -124,7 +125,7 @@ object ConsoleConsumer extends Logging {
       }
       messageCount += 1
       try {
-        formatter.writeTo(msg.key, msg.value, System.out)
+        formatter.writeTo(msg.key, msg.value, msg.timestamp, msg.timestampType, System.out)
       } catch {
         case e: Throwable =>
           if (skipMessageOnError) {
@@ -335,7 +336,7 @@ object ConsoleConsumer extends Logging {
 }
 
 trait MessageFormatter{
-  def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream)
+  def writeTo(key: Array[Byte], value: Array[Byte], timestamp: Long, timestampType: TimestampType, output: PrintStream)
 
   def init(props: Properties) {}
 
@@ -356,12 +357,16 @@ class DefaultMessageFormatter extends MessageFormatter {
       lineSeparator = props.getProperty("line.separator").getBytes
   }
 
-  def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream) {
+  def writeTo(key: Array[Byte], value: Array[Byte], timestamp: Long, timestampType: TimestampType, output: PrintStream) {
+    if (timestampType != TimestampType.NO_TIMESTAMP_TYPE) {
+      output.write(s"$timestampType:$timestamp".getBytes)
+      output.write(keySeparator)
+    }
     if (printKey) {
-      output.write(if (key == null) "null".getBytes() else key)
+      output.write(if (key == null) "null".getBytes else key)
       output.write(keySeparator)
     }
-    output.write(if (value == null) "null".getBytes() else value)
+    output.write(if (value == null) "null".getBytes else value)
     output.write(lineSeparator)
   }
 }
@@ -370,17 +375,19 @@ class LoggingMessageFormatter extends MessageFormatter   {
   private val defaultWriter: DefaultMessageFormatter = new DefaultMessageFormatter
   val logger = Logger.getLogger(getClass().getName)
 
-  def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream): Unit = {
-    defaultWriter.writeTo(key, value, output)
+  def writeTo(key: Array[Byte], value: Array[Byte], timestamp: Long, timestampType: TimestampType, output: PrintStream): Unit = {
+    defaultWriter.writeTo(key, value, timestamp, timestampType, output)
     if(logger.isInfoEnabled)
-      logger.info(s"key:${if (key == null) "null" else new String(key)}, value:${if (value == null) "null" else new String(value)}")
+      logger.info({if (timestampType != TimestampType.NO_TIMESTAMP_TYPE) s"$timestampType:$timestamp, " else ""} +
+                  s"key:${if (key == null) "null" else new String(key)}, " +
+                  s"value:${if (value == null) "null" else new String(value)}")
   }
 }
 
 class NoOpMessageFormatter extends MessageFormatter {
   override def init(props: Properties) {}
 
-  def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream) {}
+  def writeTo(key: Array[Byte], value: Array[Byte], timestamp: Long, timestampType: TimestampType, output: PrintStream){}
 }
 
 class ChecksumMessageFormatter extends MessageFormatter {
@@ -394,8 +401,12 @@ class ChecksumMessageFormatter extends MessageFormatter {
       topicStr = ""
   }
 
-  def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream) {
-    val chksum = new Message(value, key).checksum
+  def writeTo(key: Array[Byte], value: Array[Byte], timestamp: Long, timestampType: TimestampType, output: PrintStream) {
+    val chksum =
+      if (timestampType != TimestampType.NO_TIMESTAMP_TYPE)
+        new Message(value, key, timestamp, timestampType, NoCompressionCodec, 0, -1, Message.MagicValue_V1).checksum
+      else
+        new Message(value, key, Message.NoTimestamp, Message.MagicValue_V0).checksum
     output.println(topicStr + "checksum:" + chksum)
   }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 fd15014..3c41c7c 100755
--- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala
+++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala
@@ -274,7 +274,7 @@ object DumpLogSegments {
         case NoCompressionCodec =>
           getSingleMessageIterator(messageAndOffset)
         case _ =>
-          ByteBufferMessageSet.deepIterator(message)
+          ByteBufferMessageSet.deepIterator(messageAndOffset)
       }
     } else
       getSingleMessageIterator(messageAndOffset)

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/main/scala/kafka/tools/MirrorMaker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala
index a964f69..95b0aad 100755
--- a/core/src/main/scala/kafka/tools/MirrorMaker.scala
+++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala
@@ -485,7 +485,13 @@ object MirrorMaker extends Logging with KafkaMetricsGroup {
 
     override def receive() : BaseConsumerRecord = {
       val messageAndMetadata = iter.next()
-      BaseConsumerRecord(messageAndMetadata.topic, messageAndMetadata.partition, messageAndMetadata.offset, messageAndMetadata.key, messageAndMetadata.message)
+      BaseConsumerRecord(messageAndMetadata.topic,
+                         messageAndMetadata.partition,
+                         messageAndMetadata.offset,
+                         messageAndMetadata.timestamp,
+                         messageAndMetadata.timestampType,
+                         messageAndMetadata.key,
+                         messageAndMetadata.message)
     }
 
     override def stop() {
@@ -541,7 +547,13 @@ object MirrorMaker extends Logging with KafkaMetricsGroup {
 
       offsets.put(tp, record.offset + 1)
 
-      BaseConsumerRecord(record.topic, record.partition, record.offset, record.key, record.value)
+      BaseConsumerRecord(record.topic,
+                         record.partition,
+                         record.offset,
+                         record.timestamp,
+                         record.timestampType,
+                         record.key,
+                         record.value)
     }
 
     override def stop() {

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/main/scala/kafka/tools/ReplayLogProducer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/ReplayLogProducer.scala b/core/src/main/scala/kafka/tools/ReplayLogProducer.scala
index 2b8537b..d88ec41 100644
--- a/core/src/main/scala/kafka/tools/ReplayLogProducer.scala
+++ b/core/src/main/scala/kafka/tools/ReplayLogProducer.scala
@@ -139,8 +139,8 @@ object ReplayLogProducer extends Logging {
             stream
         for (messageAndMetadata <- iter) {
           try {
-            val response = producer.send(new ProducerRecord[Array[Byte],Array[Byte]](config.outputTopic,
-                                            messageAndMetadata.key(), messageAndMetadata.message()))
+            val response = producer.send(new ProducerRecord[Array[Byte],Array[Byte]](config.outputTopic, null,
+                                            messageAndMetadata.timestamp, messageAndMetadata.key(), messageAndMetadata.message()))
             if(config.isSync) {
               response.get()
             }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala
index 1c2023c..e20b061 100755
--- a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala
+++ b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala
@@ -220,7 +220,8 @@ object SimpleConsumerShell extends Logging {
                   System.out.println("next offset = " + offset)
                 val message = messageAndOffset.message
                 val key = if(message.hasKey) Utils.readBytes(message.key) else null
-                formatter.writeTo(key, if(message.isNull) null else Utils.readBytes(message.payload), System.out)
+                val value = if (message.isNull()) null else Utils.readBytes(message.payload)
+                formatter.writeTo(key, value, message.timestamp, message.timestampType, System.out)
                 numMessagesConsumed += 1
               } catch {
                 case e: Throwable =>

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
index bc3a6ce..f15c005 100644
--- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
+++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
@@ -14,8 +14,10 @@ package kafka.api
 
 import java.util
 
+import kafka.coordinator.GroupCoordinator
 import org.apache.kafka.clients.consumer._
-import org.apache.kafka.clients.producer.{Producer, ProducerConfig, ProducerRecord}
+import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord}
+import org.apache.kafka.common.record.TimestampType
 import org.apache.kafka.common.serialization.ByteArrayDeserializer
 import org.apache.kafka.common.{PartitionInfo, TopicPartition}
 
@@ -24,11 +26,10 @@ import kafka.server.KafkaConfig
 
 import java.util.ArrayList
 import org.junit.Assert._
-import org.junit.{Test, Before}
+import org.junit.{Before, Test}
 
-import scala.collection.mutable.Buffer
 import scala.collection.JavaConverters._
-import kafka.coordinator.GroupCoordinator
+import scala.collection.mutable.Buffer
 
 /**
  * Integration tests for the new consumer that cover basic usage as well as server failures
@@ -75,7 +76,7 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging {
     assertEquals(1, this.consumers(0).assignment.size)
 
     this.consumers(0).seek(tp, 0)
-    consumeAndVerifyRecords(this.consumers(0), numRecords = numRecords, startingOffset = 0)
+    consumeAndVerifyRecords(consumer = this.consumers(0), numRecords = numRecords, startingOffset = 0)
 
     // check async commit callbacks
     val commitCallback = new CountConsumerCommitCallback()
@@ -245,7 +246,7 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging {
 
     sendRecords(5)
     consumer0.subscribe(List(topic).asJava)
-    consumeAndVerifyRecords(consumer0, 5, 0)
+    consumeAndVerifyRecords(consumer = consumer0, numRecords = 5, startingOffset = 0)
     consumer0.pause(tp)
 
     // subscribe to a new topic to trigger a rebalance
@@ -253,7 +254,7 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging {
 
     // after rebalance, our position should be reset and our pause state lost,
     // so we should be able to consume from the beginning
-    consumeAndVerifyRecords(consumer0, 0, 5)
+    consumeAndVerifyRecords(consumer = consumer0, numRecords = 0, startingOffset = 5)
   }
 
   protected class TestConsumerReassignmentListener extends ConsumerRebalanceListener {
@@ -276,26 +277,33 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging {
   }
 
   protected def sendRecords(numRecords: Int, tp: TopicPartition) {
-    sendRecords(this.producers(0), numRecords, tp)
-  }
-
-  protected def sendRecords(producer: Producer[Array[Byte], Array[Byte]],
-                            numRecords: Int,
-                            tp: TopicPartition) {
     (0 until numRecords).foreach { i =>
-      producer.send(new ProducerRecord(tp.topic(), tp.partition(), s"key $i".getBytes, s"value $i".getBytes))
+      this.producers(0).send(new ProducerRecord(tp.topic(), tp.partition(), i.toLong, s"key $i".getBytes, s"value $i".getBytes))
     }
-    producer.flush()
+    this.producers(0).flush()
   }
 
-  protected def consumeAndVerifyRecords(consumer: Consumer[Array[Byte], Array[Byte]], numRecords: Int, startingOffset: Int,
-                                      startingKeyAndValueIndex: Int = 0, tp: TopicPartition = tp) {
+  protected def consumeAndVerifyRecords(consumer: Consumer[Array[Byte], Array[Byte]],
+                                        numRecords: Int,
+                                        startingOffset: Int,
+                                        startingKeyAndValueIndex: Int = 0,
+                                        startingTimestamp: Long = 0L,
+                                        timestampType: TimestampType = TimestampType.CREATE_TIME,
+                                        tp: TopicPartition = tp) {
     val records = consumeRecords(consumer, numRecords)
+    val now = System.currentTimeMillis()
     for (i <- 0 until numRecords) {
       val record = records.get(i)
       val offset = startingOffset + i
       assertEquals(tp.topic(), record.topic())
       assertEquals(tp.partition(), record.partition())
+      if (timestampType == TimestampType.CREATE_TIME) {
+        assertEquals(timestampType, record.timestampType())
+        val timestamp = startingTimestamp + i
+        assertEquals(timestamp.toLong, record.timestamp())
+      } else
+        assertTrue(s"Got unexpected timestamp ${record.timestamp()}. Timestamp should be between [$startingTimestamp, $now}]",
+          record.timestamp() >= startingTimestamp && record.timestamp() <= now)
       assertEquals(offset.toLong, record.offset())
       val keyAndValueIndex = startingKeyAndValueIndex + i
       assertEquals(s"key $keyAndValueIndex", new String(record.key()))

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala
index 42928a3..807b8bb 100644
--- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala
+++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala
@@ -17,19 +17,21 @@
 
 package kafka.api
 
-import java.io.File
 import java.util.Properties
-import java.util.concurrent.TimeUnit
+import java.util.concurrent.{ExecutionException, TimeUnit}
 
 import kafka.consumer.SimpleConsumer
 import kafka.integration.KafkaServerTestHarness
+import kafka.log.LogConfig
 import kafka.message.Message
 import kafka.server.KafkaConfig
 import kafka.utils.TestUtils
 import org.apache.kafka.clients.producer._
-import org.apache.kafka.common.errors.SerializationException
+import org.apache.kafka.common.errors.{InvalidTimestampException, SerializationException}
+import org.apache.kafka.common.record.TimestampType
 import org.junit.Assert._
 import org.junit.{After, Before, Test}
+
 import scala.collection.mutable.Buffer
 
 abstract class BaseProducerSendTest extends KafkaServerTestHarness {
@@ -54,8 +56,8 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
     super.setUp()
 
     // TODO: we need to migrate to new consumers when 0.9 is final
-    consumer1 = new SimpleConsumer("localhost", servers(0).boundPort(), 100, 1024*1024, "")
-    consumer2 = new SimpleConsumer("localhost", servers(1).boundPort(), 100, 1024*1024, "")
+    consumer1 = new SimpleConsumer("localhost", servers(0).boundPort(), 100, 1024 * 1024, "")
+    consumer2 = new SimpleConsumer("localhost", servers(1).boundPort(), 100, 1024 * 1024, "")
   }
 
   @After
@@ -88,6 +90,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
 
     object callback extends Callback {
       var offset = 0L
+
       def onCompletion(metadata: RecordMetadata, exception: Exception) {
         if (exception == null) {
           assertEquals(offset, metadata.offset())
@@ -105,24 +108,24 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
       TestUtils.createTopic(zkUtils, topic, 1, 2, servers)
 
       // send a normal record
-      val record0 = new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, "key".getBytes, "value".getBytes)
+      val record0 = new ProducerRecord[Array[Byte], Array[Byte]](topic, partition, "key".getBytes, "value".getBytes)
       assertEquals("Should have offset 0", 0L, producer.send(record0, callback).get.offset)
 
       // send a record with null value should be ok
-      val record1 = new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, "key".getBytes, null)
+      val record1 = new ProducerRecord[Array[Byte], Array[Byte]](topic, partition, "key".getBytes, null)
       assertEquals("Should have offset 1", 1L, producer.send(record1, callback).get.offset)
 
       // send a record with null key should be ok
-      val record2 = new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, null, "value".getBytes)
+      val record2 = new ProducerRecord[Array[Byte], Array[Byte]](topic, partition, null, "value".getBytes)
       assertEquals("Should have offset 2", 2L, producer.send(record2, callback).get.offset)
 
       // send a record with null part id should be ok
-      val record3 = new ProducerRecord[Array[Byte],Array[Byte]](topic, null, "key".getBytes, "value".getBytes)
+      val record3 = new ProducerRecord[Array[Byte], Array[Byte]](topic, null, "key".getBytes, "value".getBytes)
       assertEquals("Should have offset 3", 3L, producer.send(record3, callback).get.offset)
 
       // send a record with null topic should fail
       try {
-        val record4 = new ProducerRecord[Array[Byte],Array[Byte]](null, partition, "key".getBytes, "value".getBytes)
+        val record4 = new ProducerRecord[Array[Byte], Array[Byte]](null, partition, "key".getBytes, "value".getBytes)
         producer.send(record4, callback)
         fail("Should not allow sending a record without topic")
       } catch {
@@ -143,6 +146,81 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
   }
 
   @Test
+  def testSendCompressedMessageWithCreateTime() {
+    val producerProps = new Properties()
+    producerProps.setProperty(ProducerConfig.COMPRESSION_TYPE_CONFIG, "gzip")
+    val producer = createProducer(brokerList = brokerList, lingerMs = Long.MaxValue, props = Some(producerProps))
+    sendAndVerifyTimestamp(producer, TimestampType.CREATE_TIME)
+  }
+
+  @Test
+  def testSendNonCompressedMessageWithCreateTime() {
+    val producer = createProducer(brokerList = brokerList, lingerMs = Long.MaxValue)
+    sendAndVerifyTimestamp(producer, TimestampType.CREATE_TIME)
+  }
+
+  @Test
+  def testSendCompressedMessageWithLogAppendTime() {
+    val producerProps = new Properties()
+    producerProps.setProperty(ProducerConfig.COMPRESSION_TYPE_CONFIG, "gzip")
+    val producer = createProducer(brokerList = brokerList, lingerMs = Long.MaxValue, props = Some(producerProps))
+    sendAndVerifyTimestamp(producer, TimestampType.LOG_APPEND_TIME)
+  }
+
+  @Test
+  def testSendNonCompressedMessageWithLogApendTime() {
+    val producer = createProducer(brokerList = brokerList, lingerMs = Long.MaxValue)
+    sendAndVerifyTimestamp(producer, TimestampType.LOG_APPEND_TIME)
+  }
+
+  private def sendAndVerifyTimestamp(producer: KafkaProducer[Array[Byte], Array[Byte]], timestampType: TimestampType) {
+    val partition = new Integer(0)
+
+    val baseTimestamp = 123456L
+    val startTime = System.currentTimeMillis()
+
+    object callback extends Callback {
+      var offset = 0L
+      var timestampDiff = 1L
+
+      def onCompletion(metadata: RecordMetadata, exception: Exception) {
+        if (exception == null) {
+          assertEquals(offset, metadata.offset())
+          assertEquals(topic, metadata.topic())
+          if (timestampType == TimestampType.CREATE_TIME)
+            assertEquals(baseTimestamp + timestampDiff, metadata.timestamp())
+          else
+            assertTrue(metadata.timestamp() >= startTime && metadata.timestamp() <= System.currentTimeMillis())
+          assertEquals(partition, metadata.partition())
+          offset += 1
+          timestampDiff += 1
+        } else {
+          fail("Send callback returns the following exception", exception)
+        }
+      }
+    }
+
+    try {
+      // create topic
+      val topicProps = new Properties()
+      if (timestampType == TimestampType.LOG_APPEND_TIME)
+        topicProps.setProperty(LogConfig.MessageTimestampTypeProp, "LogAppendTime")
+      else
+        topicProps.setProperty(LogConfig.MessageTimestampTypeProp, "CreateTime")
+      TestUtils.createTopic(zkUtils, topic, 1, 2, servers, topicProps)
+
+      for (i <- 1 to numRecords) {
+        val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, partition, baseTimestamp + i, "key".getBytes, "value".getBytes)
+        producer.send(record, callback)
+      }
+      producer.close(5000L, TimeUnit.MILLISECONDS)
+      assertEquals(s"Should have offset $numRecords but only successfully sent ${callback.offset}", numRecords, callback.offset)
+    } finally {
+      producer.close()
+    }
+  }
+
+  @Test
   def testWrongSerializer() {
     // send a record with a wrong type should receive a serialization exception
     try {
@@ -155,7 +233,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
     }
   }
 
-  private def createProducerWithWrongSerializer(brokerList: String) : KafkaProducer[Array[Byte],Array[Byte]] = {
+  private def createProducerWithWrongSerializer(brokerList: String): KafkaProducer[Array[Byte], Array[Byte]] = {
     val producerProps = new Properties()
     producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer")
     producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer")
@@ -176,7 +254,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
       TestUtils.createTopic(zkUtils, topic, 1, 2, servers)
 
       // non-blocking send a list of records
-      val record0 = new ProducerRecord[Array[Byte],Array[Byte]](topic, null, "key".getBytes, "value".getBytes)
+      val record0 = new ProducerRecord[Array[Byte], Array[Byte]](topic, null, "key".getBytes, "value".getBytes)
       for (i <- 1 to numRecords)
         producer.send(record0)
       val response0 = producer.send(record0)
@@ -212,9 +290,10 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
       val leader1 = leaders(partition)
       assertTrue("Leader for topic \"topic\" partition 1 should exist", leader1.isDefined)
 
+      val now = System.currentTimeMillis()
       val responses =
         for (i <- 1 to numRecords)
-          yield producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, null, ("value" + i).getBytes))
+        yield producer.send(new ProducerRecord[Array[Byte], Array[Byte]](topic, partition, now, null, ("value" + i).getBytes))
       val futures = responses.toList
       futures.foreach(_.get)
       for (future <- futures)
@@ -228,7 +307,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
       }
 
       // make sure the fetched messages also respect the partitioning and ordering
-      val fetchResponse1 = if(leader1.get == configs(0).brokerId) {
+      val fetchResponse1 = if (leader1.get == configs(0).brokerId) {
         consumer1.fetch(new FetchRequestBuilder().addFetch(topic, partition, 0, Int.MaxValue).build())
       } else {
         consumer2.fetch(new FetchRequestBuilder().addFetch(topic, partition, 0, Int.MaxValue).build())
@@ -238,7 +317,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
 
       // TODO: also check topic and partition after they are added in the return messageSet
       for (i <- 0 to numRecords - 1) {
-        assertEquals(new Message(bytes = ("value" + (i + 1)).getBytes), messageSet1(i).message)
+        assertEquals(new Message(bytes = ("value" + (i + 1)).getBytes, now, Message.MagicValue_V1), messageSet1(i).message)
         assertEquals(i.toLong, messageSet1(i).offset)
       }
     } finally {
@@ -257,7 +336,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
 
     try {
       // Send a message to auto-create the topic
-      val record = new ProducerRecord[Array[Byte],Array[Byte]](topic, null, "key".getBytes, "value".getBytes)
+      val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, null, "key".getBytes, "value".getBytes)
       assertEquals("Should have offset 0", 0L, producer.send(record).get.offset)
 
       // double check that the topic is created with leader elected
@@ -277,7 +356,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
     try {
       TestUtils.createTopic(zkUtils, topic, 2, 2, servers)
       val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, "value".getBytes)
-      for(i <- 0 until 50) {
+      for (i <- 0 until 50) {
         val responses = (0 until numRecords) map (i => producer.send(record))
         assertTrue("No request is complete.", responses.forall(!_.isDone()))
         producer.flush()
@@ -302,7 +381,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
     val record0 = new ProducerRecord[Array[Byte], Array[Byte]](topic, 0, null, "value".getBytes)
 
     // Test closing from caller thread.
-    for(i <- 0 until 50) {
+    for (i <- 0 until 50) {
       val producer = createProducer(brokerList, lingerMs = Long.MaxValue)
       val responses = (0 until numRecords) map (i => producer.send(record0))
       assertTrue("No request is complete.", responses.forall(!_.isDone()))
@@ -349,23 +428,58 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
         producer.close(Long.MaxValue, TimeUnit.MICROSECONDS)
       }
     }
-    for(i <- 0 until 50) {
+    for (i <- 0 until 50) {
       val producer = createProducer(brokerList, lingerMs = Long.MaxValue)
-      // send message to partition 0
-      val responses = ((0 until numRecords) map (i => producer.send(record, new CloseCallback(producer))))
-      assertTrue("No request is complete.", responses.forall(!_.isDone()))
-      // flush the messages.
-      producer.flush()
-      assertTrue("All request are complete.", responses.forall(_.isDone()))
-      // Check the messages received by broker.
-      val fetchResponse = if (leader.get == configs(0).brokerId) {
-        consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, Int.MaxValue).build())
-      } else {
-        consumer2.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, Int.MaxValue).build())
+      try {
+        // send message to partition 0
+        val responses = ((0 until numRecords) map (i => producer.send(record, new CloseCallback(producer))))
+        assertTrue("No request is complete.", responses.forall(!_.isDone()))
+        // flush the messages.
+        producer.flush()
+        assertTrue("All request are complete.", responses.forall(_.isDone()))
+        // Check the messages received by broker.
+        val fetchResponse = if (leader.get == configs(0).brokerId) {
+          consumer1.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, Int.MaxValue).build())
+        } else {
+          consumer2.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, Int.MaxValue).build())
+        }
+        val expectedNumRecords = (i + 1) * numRecords
+        assertEquals("Fetch response to partition 0 should have %d messages.".format(expectedNumRecords),
+          expectedNumRecords, fetchResponse.messageSet(topic, 0).size)
+      } finally {
+        producer.close()
       }
-      val expectedNumRecords = (i + 1) * numRecords
-      assertEquals("Fetch response to partition 0 should have %d messages.".format(expectedNumRecords),
-        expectedNumRecords, fetchResponse.messageSet(topic, 0).size)
     }
   }
+
+  @Test
+  def testSendWithInvalidCreateTime() {
+    val topicProps = new Properties()
+    topicProps.setProperty(LogConfig.MessageTimestampDifferenceMaxMsProp, "1000");
+    TestUtils.createTopic(zkUtils, topic, 1, 2, servers, topicProps)
+
+    val producer = createProducer(brokerList = brokerList)
+    try {
+      producer.send(new ProducerRecord(topic, 0, System.currentTimeMillis() - 1001, "key".getBytes, "value".getBytes)).get()
+      fail("Should throw CorruptedRecordException")
+    } catch {
+      case e: ExecutionException => assertTrue(e.getCause.isInstanceOf[InvalidTimestampException])
+    } finally {
+      producer.close()
+    }
+
+    // Test compressed messages.
+    val producerProps = new Properties()
+    producerProps.setProperty(ProducerConfig.COMPRESSION_TYPE_CONFIG, "gzip")
+    val compressedProducer = createProducer(brokerList = brokerList, props = Some(producerProps))
+    try {
+      compressedProducer.send(new ProducerRecord(topic, 0, System.currentTimeMillis() - 1001, "key".getBytes, "value".getBytes)).get()
+      fail("Should throw CorruptedRecordException")
+    } catch {
+      case e: ExecutionException => assertTrue(e.getCause.isInstanceOf[InvalidTimestampException])
+    } finally {
+      compressedProducer.close()
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala
index b2f96e5..3d7cad3 100644
--- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala
+++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala
@@ -18,20 +18,23 @@ import java.util.Properties
 
 import java.util.regex.Pattern
 
+import kafka.log.LogConfig
 import kafka.server.KafkaConfig
 import kafka.utils.TestUtils
 import org.apache.kafka.clients.consumer._
-import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
+import org.apache.kafka.clients.producer.KafkaProducer
+import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer, ByteArraySerializer}
+import org.apache.kafka.test.{MockProducerInterceptor, MockConsumerInterceptor}
+import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord}
 import org.apache.kafka.common.TopicPartition
-import org.apache.kafka.common.record.CompressionType
-import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer, ByteArrayDeserializer, ByteArraySerializer}
 import org.apache.kafka.common.errors.{InvalidTopicException, RecordTooLargeException}
-import org.apache.kafka.test.{MockProducerInterceptor, MockConsumerInterceptor}
+import org.apache.kafka.common.record.{CompressionType, TimestampType}
+import org.apache.kafka.common.serialization.ByteArrayDeserializer
 import org.junit.Assert._
 import org.junit.Test
+
+import scala.collection.JavaConverters._
 import scala.collection.mutable.Buffer
-import scala.collection.JavaConverters
-import JavaConverters._
 
 /* We have some tests in this class instead of `BaseConsumerTest` in order to keep the build time under control. */
 class PlaintextConsumerTest extends BaseConsumerTest {
@@ -96,14 +99,14 @@ class PlaintextConsumerTest extends BaseConsumerTest {
   def testAutoOffsetReset() {
     sendRecords(1)
     this.consumers(0).assign(List(tp).asJava)
-    consumeAndVerifyRecords(this.consumers(0), numRecords = 1, startingOffset = 0)
+    consumeAndVerifyRecords(consumer = this.consumers(0), numRecords = 1, startingOffset = 0)
   }
 
   @Test
   def testGroupConsumption() {
     sendRecords(10)
     this.consumers(0).subscribe(List(topic).asJava)
-    consumeAndVerifyRecords(this.consumers(0), numRecords = 1, startingOffset = 0)
+    consumeAndVerifyRecords(consumer = this.consumers(0), numRecords = 1, startingOffset = 0)
   }
 
   @Test
@@ -263,7 +266,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
     assertFalse(partitions.isEmpty)
   }
 
-  @Test(expected=classOf[InvalidTopicException])
+  @Test(expected = classOf[InvalidTopicException])
   def testPartitionsForInvalidTopic() {
     this.consumers(0).partitionsFor(";3# ads,{234")
   }
@@ -288,7 +291,9 @@ class PlaintextConsumerTest extends BaseConsumerTest {
 
     consumer.seek(tp, mid)
     assertEquals(mid, consumer.position(tp))
-    consumeAndVerifyRecords(consumer, numRecords = 1, startingOffset = mid.toInt, startingKeyAndValueIndex = mid.toInt)
+
+    consumeAndVerifyRecords(consumer, numRecords = 1, startingOffset = mid.toInt, startingKeyAndValueIndex = mid.toInt,
+      startingTimestamp = mid.toLong)
 
     // Test seek compressed message
     sendCompressedMessages(totalRecords.toInt, tp2)
@@ -305,7 +310,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
     consumer.seek(tp2, mid)
     assertEquals(mid, consumer.position(tp2))
     consumeAndVerifyRecords(consumer, numRecords = 1, startingOffset = mid.toInt, startingKeyAndValueIndex = mid.toInt,
-      tp = tp2)
+      startingTimestamp = mid.toLong, tp = tp2)
   }
 
   private def sendCompressedMessages(numRecords: Int, tp: TopicPartition) {
@@ -314,17 +319,17 @@ class PlaintextConsumerTest extends BaseConsumerTest {
     producerProps.setProperty(ProducerConfig.LINGER_MS_CONFIG, Long.MaxValue.toString)
     val producer = TestUtils.createNewProducer(brokerList, securityProtocol = securityProtocol, trustStoreFile = trustStoreFile,
         retries = 0, lingerMs = Long.MaxValue, props = Some(producerProps))
-    sendRecords(producer, numRecords, tp)
+    (0 until numRecords).foreach { i =>
+      producer.send(new ProducerRecord(tp.topic(), tp.partition(), i.toLong, s"key $i".getBytes, s"value $i".getBytes))
+    }
     producer.close()
   }
 
+  @Test
   def testPositionAndCommit() {
     sendRecords(5)
 
-    // committed() on a partition with no committed offset throws an exception
-    intercept[NoOffsetForPartitionException] {
-      this.consumers(0).committed(new TopicPartition(topic, 15))
-    }
+    assertNull(this.consumers(0).committed(new TopicPartition(topic, 15)))
 
     // position() on a partition that we aren't subscribed to throws an exception
     intercept[IllegalArgumentException] {
@@ -337,7 +342,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
     this.consumers(0).commitSync()
     assertEquals(0L, this.consumers(0).committed(tp).offset)
 
-    consumeAndVerifyRecords(this.consumers(0), 5, 0)
+    consumeAndVerifyRecords(consumer = this.consumers(0), numRecords = 5, startingOffset = 0)
     assertEquals("After consuming 5 records, position should be 5", 5L, this.consumers(0).position(tp))
     this.consumers(0).commitSync()
     assertEquals("Committed offset should be returned", 5L, this.consumers(0).committed(tp).offset)
@@ -346,19 +351,19 @@ class PlaintextConsumerTest extends BaseConsumerTest {
 
     // another consumer in the same group should get the same position
     this.consumers(1).assign(List(tp).asJava)
-    consumeAndVerifyRecords(this.consumers(1), 1, 5)
+    consumeAndVerifyRecords(consumer = this.consumers(1), numRecords = 1, startingOffset = 5)
   }
 
   @Test
   def testPartitionPauseAndResume() {
     sendRecords(5)
     this.consumers(0).assign(List(tp).asJava)
-    consumeAndVerifyRecords(this.consumers(0), 5, 0)
+    consumeAndVerifyRecords(consumer = this.consumers(0), numRecords = 5, startingOffset = 0)
     this.consumers(0).pause(tp)
     sendRecords(5)
     assertTrue(this.consumers(0).poll(0).isEmpty)
     this.consumers(0).resume(tp)
-    consumeAndVerifyRecords(this.consumers(0), 5, 5)
+    consumeAndVerifyRecords(consumer = this.consumers(0), numRecords = 5, startingOffset = 5)
   }
 
   @Test
@@ -397,7 +402,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
     val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer())
 
     // produce a record that is larger than the configured fetch size
-    val record = new ProducerRecord[Array[Byte],Array[Byte]](tp.topic(), tp.partition(), "key".getBytes, new Array[Byte](maxFetchBytes + 1))
+    val record = new ProducerRecord[Array[Byte], Array[Byte]](tp.topic(), tp.partition(), "key".getBytes, new Array[Byte](maxFetchBytes + 1))
     this.producers(0).send(record)
 
     // consuming a too-large record should fail
@@ -534,7 +539,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
     producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
     producerProps.put(ProducerConfig.INTERCEPTOR_CLASSES_CONFIG, "org.apache.kafka.test.MockProducerInterceptor")
     producerProps.put("mock.interceptor.append", appendStr)
-    val testProducer = new KafkaProducer[String,String](producerProps, new StringSerializer, new StringSerializer)
+    val testProducer = new KafkaProducer[String, String](producerProps, new StringSerializer, new StringSerializer)
 
     // produce records
     val numRecords = 10
@@ -567,16 +572,16 @@ class PlaintextConsumerTest extends BaseConsumerTest {
 
     // commit sync and verify onCommit is called
     val commitCountBefore = MockConsumerInterceptor.ON_COMMIT_COUNT.intValue()
-    testConsumer.commitSync(Map[TopicPartition,OffsetAndMetadata]((tp, new OffsetAndMetadata(2L))).asJava)
+    testConsumer.commitSync(Map[TopicPartition, OffsetAndMetadata]((tp, new OffsetAndMetadata(2L))).asJava)
     assertEquals(2, testConsumer.committed(tp).offset)
-    assertEquals(commitCountBefore+1, MockConsumerInterceptor.ON_COMMIT_COUNT.intValue())
+    assertEquals(commitCountBefore + 1, MockConsumerInterceptor.ON_COMMIT_COUNT.intValue())
 
     // commit async and verify onCommit is called
     val commitCallback = new CountConsumerCommitCallback()
-    testConsumer.commitAsync(Map[TopicPartition,OffsetAndMetadata]((tp, new OffsetAndMetadata(5L))).asJava, commitCallback)
+    testConsumer.commitAsync(Map[TopicPartition, OffsetAndMetadata]((tp, new OffsetAndMetadata(5L))).asJava, commitCallback)
     awaitCommitCallback(testConsumer, commitCallback)
     assertEquals(5, testConsumer.committed(tp).offset)
-    assertEquals(commitCountBefore+2, MockConsumerInterceptor.ON_COMMIT_COUNT.intValue())
+    assertEquals(commitCountBefore + 2, MockConsumerInterceptor.ON_COMMIT_COUNT.intValue())
 
     testConsumer.close()
     testProducer.close()
@@ -593,7 +598,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
 
     // produce records
     val numRecords = 100
-    val testProducer = new KafkaProducer[String,String](this.producerConfig, new StringSerializer, new StringSerializer)
+    val testProducer = new KafkaProducer[String, String](this.producerConfig, new StringSerializer, new StringSerializer)
     (0 until numRecords).map { i =>
       testProducer.send(new ProducerRecord(tp.topic(), tp.partition(), s"key $i", s"value $i"))
     }.foreach(_.get)
@@ -617,8 +622,8 @@ class PlaintextConsumerTest extends BaseConsumerTest {
     // change subscription to trigger rebalance
     val commitCountBeforeRebalance = MockConsumerInterceptor.ON_COMMIT_COUNT.intValue()
     changeConsumerSubscriptionAndValidateAssignment(testConsumer,
-                                                    List(topic, topic2), Set(tp, tp2, new TopicPartition(topic2, 0),
-                                                    new TopicPartition(topic2, 1)),
+                                                    List(topic, topic2),
+                                                    Set(tp, tp2, new TopicPartition(topic2, 0), new TopicPartition(topic2, 1)),
                                                     rebalanceListener)
 
     // after rebalancing, we should have reset to the committed positions
@@ -644,14 +649,14 @@ class PlaintextConsumerTest extends BaseConsumerTest {
     producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
     producerProps.put(ProducerConfig.INTERCEPTOR_CLASSES_CONFIG, "org.apache.kafka.test.MockProducerInterceptor")
     producerProps.put("mock.interceptor.append", appendStr)
-    val testProducer = new KafkaProducer[Array[Byte],Array[Byte]](producerProps, new ByteArraySerializer(), new ByteArraySerializer())
+    val testProducer = new KafkaProducer[Array[Byte], Array[Byte]](producerProps, new ByteArraySerializer(), new ByteArraySerializer())
 
     // producing records should succeed
     testProducer.send(new ProducerRecord(tp.topic(), tp.partition(), s"key".getBytes, s"value will not be modified".getBytes))
 
     // create consumer with interceptor that has different key and value types from the consumer
     this.consumerConfig.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, "org.apache.kafka.test.MockConsumerInterceptor")
-    val testConsumer = new KafkaConsumer[Array[Byte],Array[Byte]](this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer())
+    val testConsumer = new KafkaConsumer[Array[Byte], Array[Byte]](this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer())
     testConsumer.assign(List(tp).asJava)
     testConsumer.seek(tp, 0)
 
@@ -664,6 +669,46 @@ class PlaintextConsumerTest extends BaseConsumerTest {
     testProducer.close()
   }
 
+  def testConsumeMessagesWithCreateTime() {
+    val numRecords = 50
+    // Test non-compressed messages
+    sendRecords(numRecords, tp)
+    this.consumers(0).assign(List(tp).asJava)
+    consumeAndVerifyRecords(consumer = this.consumers(0), numRecords = numRecords, startingOffset = 0, startingKeyAndValueIndex = 0,
+      startingTimestamp = 0)
+
+    // Test compressed messages
+    sendCompressedMessages(numRecords, tp2)
+    this.consumers(0).assign(List(tp2).asJava)
+    consumeAndVerifyRecords(consumer = this.consumers(0), numRecords = numRecords, tp = tp2, startingOffset = 0, startingKeyAndValueIndex = 0,
+      startingTimestamp = 0)
+  }
+
+  @Test
+  def testConsumeMessagesWithLogAppendTime() {
+    val topicName = "testConsumeMessagesWithLogAppendTime"
+    val topicProps = new Properties()
+    topicProps.setProperty(LogConfig.MessageTimestampTypeProp, "LogAppendTime")
+    TestUtils.createTopic(zkUtils, topicName, 2, 2, servers, topicProps)
+
+    val startTime = System.currentTimeMillis()
+    val numRecords = 50
+
+    // Test non-compressed messages
+    val tp1 = new TopicPartition(topicName, 0)
+    sendRecords(numRecords, tp1)
+    this.consumers(0).assign(List(tp1).asJava)
+    consumeAndVerifyRecords(consumer = this.consumers(0), numRecords = numRecords, tp = tp1, startingOffset = 0, startingKeyAndValueIndex = 0,
+      startingTimestamp = startTime, timestampType = TimestampType.LOG_APPEND_TIME)
+
+    // Test compressed messages
+    val tp2 = new TopicPartition(topicName, 1)
+    sendCompressedMessages(numRecords, tp2)
+    this.consumers(0).assign(List(tp2).asJava)
+    consumeAndVerifyRecords(consumer = this.consumers(0), numRecords = numRecords, tp = tp2, startingOffset = 0, startingKeyAndValueIndex = 0,
+      startingTimestamp = startTime, timestampType = TimestampType.LOG_APPEND_TIME)
+  }
+
   def runMultiConsumerSessionTimeoutTest(closeConsumer: Boolean): Unit = {
     // use consumers defined in this class plus one additional consumer
     // Use topic defined in this class + one additional topic
@@ -693,7 +738,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
 
     val maxSessionTimeout = this.serverConfig.getProperty(KafkaConfig.GroupMaxSessionTimeoutMsProp).toLong
     validateGroupAssignment(consumerPollers, subscriptions,
-      s"Did not get valid assignment for partitions ${subscriptions.asJava} after one consumer left", 3*maxSessionTimeout)
+      s"Did not get valid assignment for partitions ${subscriptions.asJava} after one consumer left", 3 * maxSessionTimeout)
 
     // done with pollers and consumers
     for (poller <- consumerPollers)
@@ -810,7 +855,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
 
     // wait until topics get re-assigned and validate assignment
     validateGroupAssignment(consumerPollers, subscriptions,
-        s"Did not get valid assignment for partitions ${subscriptions.asJava} after we added ${numOfConsumersToAdd} consumer(s)")
+      s"Did not get valid assignment for partitions ${subscriptions.asJava} after we added ${numOfConsumersToAdd} consumer(s)")
   }
 
   /**
@@ -844,7 +889,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
     }, s"Failed to call subscribe on all consumers in the group for subscription ${subscriptions}", 1000L)
 
     validateGroupAssignment(consumerPollers, subscriptions,
-        s"Did not get valid assignment for partitions ${subscriptions.asJava} after we changed subscription")
+      s"Did not get valid assignment for partitions ${subscriptions.asJava} after we changed subscription")
   }
 
   def changeConsumerSubscriptionAndValidateAssignment[K, V](consumer: Consumer[K, V],

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala
index 0d401f7..c4a2bd7 100755
--- a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala
@@ -87,8 +87,9 @@ class ProducerCompressionTest(compression: String) extends ZooKeeperTestHarness
         yield ("value" + i).getBytes
 
       // make sure the returned messages are correct
+      val now = System.currentTimeMillis()
       val responses = for (message <- messages)
-        yield producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic, null, null, message))
+        yield producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic, null, now, null, message))
       val futures = responses.toList
       for ((future, offset) <- futures zip (0 until numRecords)) {
         assertEquals(offset.toLong, future.get.offset)
@@ -101,7 +102,7 @@ class ProducerCompressionTest(compression: String) extends ZooKeeperTestHarness
 
       var index = 0
       for (message <- messages) {
-        assertEquals(new Message(bytes = message), messageSet(index).message)
+        assertEquals(new Message(bytes = message, now, Message.MagicValue_V1), messageSet(index).message)
         assertEquals(index.toLong, messageSet(index).offset)
         index += 1
       }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
index e4b8854..fafc4b0 100644
--- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
+++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
@@ -287,4 +287,5 @@ class RequestResponseSerializationTest extends JUnitSuite {
     // new response should have 4 bytes more than the old response since delayTime is an INT32
     assertEquals(oldClientResponse.sizeInBytes + 4, newClientResponse.sizeInBytes)
   }
+
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 7e6e765..587abd5 100644
--- a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala
@@ -17,12 +17,14 @@
 
 package kafka.coordinator
 
+import org.apache.kafka.common.record.Record
 import org.junit.Assert._
 
 import kafka.common.{OffsetAndMetadata, TopicAndPartition}
-import kafka.message.MessageSet
+import kafka.message.{Message, MessageSet}
 import kafka.server.{ReplicaManager, KafkaConfig}
 import kafka.utils._
+import org.apache.kafka.common.utils.SystemTime
 import org.apache.kafka.common.TopicPartition
 import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.requests.{OffsetCommitRequest, JoinGroupRequest}
@@ -87,7 +89,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
     EasyMock.expect(zkUtils.getPartitionAssignmentForTopics(Seq(GroupCoordinator.GroupMetadataTopicName))).andReturn(ret)
     EasyMock.replay(zkUtils)
 
-    groupCoordinator = GroupCoordinator.create(KafkaConfig.fromProps(props), zkUtils, replicaManager)
+    groupCoordinator = GroupCoordinator.create(KafkaConfig.fromProps(props), zkUtils, replicaManager, new SystemTime)
     groupCoordinator.startup()
 
     // add the partition into the owned partition list
@@ -833,9 +835,10 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
       EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
       override def answer = capturedArgument.getValue.apply(
         Map(new TopicPartition(GroupCoordinator.GroupMetadataTopicName, groupPartitionId) ->
-          new PartitionResponse(Errors.NONE.code, 0L)
+          new PartitionResponse(Errors.NONE.code, 0L, Record.NO_TIMESTAMP)
         )
       )})
+    EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(Some(Message.MagicValue_V1)).anyTimes()
     EasyMock.replay(replicaManager)
 
     groupCoordinator.handleSyncGroup(groupId, generation, leaderId, assignment, responseCallback)
@@ -909,9 +912,10 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
       EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
       override def answer = capturedArgument.getValue.apply(
         Map(new TopicPartition(GroupCoordinator.GroupMetadataTopicName, groupPartitionId) ->
-          new PartitionResponse(Errors.NONE.code, 0L)
+          new PartitionResponse(Errors.NONE.code, 0L, Record.NO_TIMESTAMP)
         )
       )})
+    EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(Some(Message.MagicValue_V1)).anyTimes()
     EasyMock.replay(replicaManager)
 
     groupCoordinator.handleCommitOffsets(groupId, consumerId, generationId, offsets, responseCallback)
@@ -922,6 +926,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
     val (responseFuture, responseCallback) = setupHeartbeatCallback
 
     EasyMock.expect(replicaManager.getPartition(GroupCoordinator.GroupMetadataTopicName, groupPartitionId)).andReturn(None)
+    EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(Some(Message.MagicValue_V1)).anyTimes()
     EasyMock.replay(replicaManager)
 
     groupCoordinator.handleLeaveGroup(groupId, consumerId, responseCallback)

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/test/scala/unit/kafka/log/CleanerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/CleanerTest.scala b/core/src/test/scala/unit/kafka/log/CleanerTest.scala
index a8092de..69218ba 100755
--- a/core/src/test/scala/unit/kafka/log/CleanerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/CleanerTest.scala
@@ -261,7 +261,8 @@ class CleanerTest extends JUnitSuite {
       log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes))
     
     // forward offset and append message to next segment at offset Int.MaxValue
-    val messageSet = new ByteBufferMessageSet(NoCompressionCodec, new AtomicLong(Int.MaxValue-1), new Message("hello".getBytes, "hello".getBytes))
+    val messageSet = new ByteBufferMessageSet(NoCompressionCodec, new AtomicLong(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))
     assertEquals(Int.MaxValue, log.activeSegment.index.lastOffset)
@@ -448,13 +449,19 @@ class CleanerTest extends JUnitSuite {
   def key(id: Int) = ByteBuffer.wrap(id.toString.getBytes)
   
   def message(key: Int, value: Int) = 
-    new ByteBufferMessageSet(new Message(key=key.toString.getBytes, bytes=value.toString.getBytes))
+    new ByteBufferMessageSet(new Message(key = key.toString.getBytes,
+                                         bytes = value.toString.getBytes,
+                                         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))
+    new ByteBufferMessageSet(new Message(key=key.toString.getBytes,
+                                         bytes=null,
+                                         timestamp = Message.NoTimestamp,
+                                         magicValue = Message.MagicValue_V1))
   
 }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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
index 95085f4..0179166 100644
--- a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala
+++ b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala
@@ -200,4 +200,72 @@ class FileMessageSetTest extends BaseMessageSetTestCases {
     assertEquals(oldposition, tempReopen.length)
   }
 
+  @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
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 de3d7a3..6b91611 100755
--- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
@@ -99,7 +99,7 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
       if (entry.message.compressionCodec == NoCompressionCodec)
         Stream.cons(entry, Stream.empty).iterator
       else
-        ByteBufferMessageSet.deepIterator(entry.message)
+        ByteBufferMessageSet.deepIterator(entry)
     }) yield {
       val key = TestUtils.readString(messageAndOffset.message.key).toInt
       val value = TestUtils.readString(messageAndOffset.message.payload).toInt

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/core/src/test/scala/unit/kafka/log/LogConfigTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala
index 51cd62c..1be9e65 100644
--- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala
@@ -61,6 +61,7 @@ class LogConfigTest {
         case LogConfig.CleanupPolicyProp => assertPropertyInvalid(name, "true", "foobar");
         case LogConfig.MinCleanableDirtyRatioProp => assertPropertyInvalid(name, "not_a_number", "-0.1", "1.2")
         case LogConfig.MinInSyncReplicasProp => assertPropertyInvalid(name, "not_a_number", "0", "-1")
+        case LogConfig.MessageFormatVersionProp => assertPropertyInvalid(name, "")
         case positiveIntProperty => assertPropertyInvalid(name, "not_a_number", "-1")
       }
     })
@@ -70,7 +71,7 @@ class LogConfigTest {
     values.foreach((value) => {
       val props = new Properties
       props.setProperty(name, value.toString)
-      intercept[ConfigException] {
+      intercept[Exception] {
         LogConfig(props)
       }
     })

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 46bfbed..91a4449 100755
--- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
@@ -20,6 +20,7 @@ package kafka.log
 import java.io._
 import java.util.Properties
 
+import kafka.api.ApiVersion
 import kafka.common._
 import kafka.server.OffsetCheckpoint
 import kafka.utils._

http://git-wip-us.apache.org/repos/asf/kafka/blob/45c8195f/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 47908e7..426b5e8 100755
--- a/core/src/test/scala/unit/kafka/log/LogTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogTest.scala
@@ -21,6 +21,7 @@ import java.io._
 import java.util.Properties
 import java.util.concurrent.atomic._
 import org.apache.kafka.common.errors.{OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException, CorruptRecordException}
+import kafka.api.ApiVersion
 import org.junit.Assert._
 import org.scalatest.junit.JUnitSuite
 import org.junit.{After, Before, Test}
@@ -132,6 +133,8 @@ class LogTest extends JUnitSuite {
 
     val logProps = new Properties()
     logProps.put(LogConfig.SegmentBytesProp, segmentSize: java.lang.Integer)
+    // We use need to use magic value 1 here because the test is message size sensitive.
+    logProps.put(LogConfig.MessageFormatVersionProp, ApiVersion.latestVersion.toString())
     // create a log
     val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
     assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
@@ -160,6 +163,8 @@ class LogTest extends JUnitSuite {
   def testAppendAndReadWithSequentialOffsets() {
     val logProps = new Properties()
     logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer)
+    // 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
 
@@ -264,7 +269,8 @@ class LogTest extends JUnitSuite {
     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)
+      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 lastRead = log.read(startOffset = numMessages, maxLength = 1024*1024, maxOffset = Some(numMessages + 1)).messageSet
@@ -290,7 +296,7 @@ class LogTest extends JUnitSuite {
     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)))
 
-    def read(offset: Int) = ByteBufferMessageSet.deepIterator(log.read(offset, 4096).messageSet.head.message)
+    def read(offset: Int) = ByteBufferMessageSet.deepIterator(log.read(offset, 4096).messageSet.head)
 
     /* 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)
@@ -343,6 +349,8 @@ class LogTest extends JUnitSuite {
     val configSegmentSize = messageSet.sizeInBytes - 1
     val logProps = new Properties()
     logProps.put(LogConfig.SegmentBytesProp, configSegmentSize: java.lang.Integer)
+    // 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)
 
     try {
@@ -355,8 +363,8 @@ class LogTest extends JUnitSuite {
 
   @Test
   def testCompactedTopicConstraints() {
-    val keyedMessage = new Message(bytes = "this message has a key".getBytes, key = "and here it is".getBytes)
-    val anotherKeyedMessage = new Message(bytes = "this message also has a key".getBytes, key ="another key".getBytes)
+    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 messageSetWithUnkeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, unkeyedMessage, keyedMessage)
@@ -404,7 +412,7 @@ 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".getBytes))
+    val second = new ByteBufferMessageSet(NoCompressionCodec, new Message("change (I need more bytes)".getBytes))
 
     // append messages to log
     val maxMessageSize = second.sizeInBytes - 1