From ce01646b378a9ffab5f340fef3c10644fe290d52 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Thu, 7 Jan 2016 06:30:39 -0800 Subject: [PATCH 01/33] Added timetamp to Message and use relative offset. See KIP-31 and KIP-32 for details. --- .../kafka/common/protocol/Protocol.java | 31 +- .../src/main/scala/kafka/api/ApiVersion.scala | 30 +- .../main/scala/kafka/api/FetchRequest.scala | 5 +- .../main/scala/kafka/api/FetchResponse.scala | 46 ++- .../scala/kafka/api/ProducerRequest.scala | 2 +- .../controller/ControllerChannelManager.scala | 4 +- .../coordinator/GroupMetadataManager.scala | 16 +- .../main/scala/kafka/log/FileMessageSet.scala | 31 +- core/src/main/scala/kafka/log/Log.scala | 10 +- .../src/main/scala/kafka/log/LogCleaner.scala | 17 +- core/src/main/scala/kafka/log/LogConfig.scala | 17 + .../src/main/scala/kafka/log/LogSegment.scala | 2 +- .../kafka/message/ByteBufferMessageSet.scala | 378 +++++++++++++++--- .../main/scala/kafka/message/Message.scala | 210 ++++++++-- .../main/scala/kafka/message/MessageSet.scala | 26 +- .../scala/kafka/message/MessageWriter.scala | 17 +- .../producer/async/DefaultEventHandler.scala | 17 +- .../kafka/server/AbstractFetcherThread.scala | 2 +- .../main/scala/kafka/server/KafkaApis.scala | 18 +- .../main/scala/kafka/server/KafkaConfig.scala | 17 + .../main/scala/kafka/server/KafkaServer.scala | 7 +- .../kafka/server/ReplicaFetcherThread.scala | 7 +- .../scala/kafka/server/ReplicaManager.scala | 4 +- .../scala/kafka/tools/ConsoleConsumer.scala | 2 +- .../scala/kafka/tools/DumpLogSegments.scala | 2 +- .../kafka/api/BaseProducerSendTest.scala | 3 +- .../kafka/api/PlaintextConsumerTest.scala | 6 + .../kafka/api/ProducerCompressionTest.scala | 2 +- .../RequestResponseSerializationTest.scala | 97 ++++- .../scala/unit/kafka/log/CleanerTest.scala | 13 +- .../kafka/log/LogCleanerIntegrationTest.scala | 2 +- .../scala/unit/kafka/log/LogConfigTest.scala | 3 +- .../scala/unit/kafka/log/LogManagerTest.scala | 3 + .../test/scala/unit/kafka/log/LogTest.scala | 18 +- .../message/ByteBufferMessageSetTest.scala | 248 +++++++++++- .../message/MessageCompressionTest.scala | 4 +- .../unit/kafka/message/MessageTest.scala | 102 ++++- .../kafka/message/MessageWriterTest.scala | 8 +- .../kafka/producer/AsyncProducerTest.scala | 6 +- .../unit/kafka/producer/ProducerTest.scala | 6 +- .../kafka/producer/SyncProducerTest.scala | 2 +- .../unit/kafka/server/KafkaConfigTest.scala | 6 +- .../unit/kafka/server/LogOffsetTest.scala | 19 +- .../scala/unit/kafka/utils/TestUtils.scala | 7 +- 44 files changed, 1272 insertions(+), 201 deletions(-) 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 48c64c206f075..f1d53212b9d3f 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 the client support 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 the 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,13 @@ 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 has same body as PRODUCE_RESPONSE_V1. + * The version is bumped up to accommodate PRODUCE_REQUEST_V2. + */ + public static final Schema PRODUCE_RESPONSE_V2 = PRODUCE_RESPONSE_V1; + 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 +378,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 +404,12 @@ public class Protocol { 0), new Field("responses", new ArrayOf(FETCH_RESPONSE_TOPIC_V0))); + // Fetch response V2 is the same as fetch response V1. The version number is bumped up to indicate that the Message + // format in the response is in message format V1, i.e. the messages use relative offsets and has timestamps. + 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(); diff --git a/core/src/main/scala/kafka/api/ApiVersion.scala b/core/src/main/scala/kafka/api/ApiVersion.scala index c9c19761ec288..faee50c6995c0 100644 --- a/core/src/main/scala/kafka/api/ApiVersion.scala +++ b/core/src/main/scala/kafka/api/ApiVersion.scala @@ -24,16 +24,24 @@ 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) + * + * If there is a draft protocol version between protocols of two official releases. The suffix "-DV#" will be added to + * help users who are running on trunk upgrade. For example: + * 1. Kafka 0.9.0 is released + * 2. After that some protocol change are made and will be released in 0.10.0. The version will be named 0.10.0-DV0 + * (DV stands for draft version) + * 3. When Kafka 0.10.0 is released. The official version 0.10.0 will be the same as the last draft version. */ 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-DV0" -> KAFKA_0_10_0_DV0 ) def apply(version: String): ApiVersion = versionNameMap(version.split("\\.").slice(0,3).mkString(".")) @@ -57,22 +65,28 @@ 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 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 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 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 id: Int = 3 } + +// This is a between-release protocol version +case object KAFKA_0_10_0_DV0 extends ApiVersion { + val version: String = "0.10.0-DV0" + val id: Int = 4 +} diff --git a/core/src/main/scala/kafka/api/FetchRequest.scala b/core/src/main/scala/kafka/api/FetchRequest.scala index b43b8f425bb40..8f26779a21785 100644 --- a/core/src/main/scala/kafka/api/FetchRequest.scala +++ b/core/src/main/scala/kafka/api/FetchRequest.scala @@ -22,7 +22,7 @@ import kafka.api.ApiUtils._ import kafka.common.TopicAndPartition import kafka.consumer.ConsumerConfig import kafka.network.RequestChannel -import kafka.message.MessageSet +import kafka.message.{Message, MessageSet} import java.util.concurrent.atomic.AtomicInteger import java.nio.ByteBuffer @@ -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))) } diff --git a/core/src/main/scala/kafka/api/FetchResponse.scala b/core/src/main/scala/kafka/api/FetchResponse.scala index 1066d7f9b4bb2..4f325d1600754 100644 --- a/core/src/main/scala/kafka/api/FetchResponse.scala +++ b/core/src/main/scala/kafka/api/FetchResponse.scala @@ -20,14 +20,15 @@ package kafka.api import java.nio.ByteBuffer import java.nio.channels.GatheringByteChannel -import kafka.common.TopicAndPartition -import kafka.message.{MessageSet, ByteBufferMessageSet} +import kafka.common.{TopicAndPartition, ErrorMapping} +import kafka.message.{NoCompressionCodec, Message, MessageSet, ByteBufferMessageSet} import kafka.api.ApiUtils._ import org.apache.kafka.common.KafkaException import org.apache.kafka.common.network.{Send, MultiSend} import org.apache.kafka.common.protocol.Errors import scala.collection._ +import scala.collection.mutable.ArrayBuffer object FetchResponsePartitionData { def readFrom(buffer: ByteBuffer): FetchResponsePartitionData = { @@ -48,6 +49,36 @@ object FetchResponsePartitionData { case class FetchResponsePartitionData(error: Short = Errors.NONE.code, hw: Long = -1L, messages: MessageSet) { val sizeInBytes = FetchResponsePartitionData.headerSize + messages.sizeInBytes + + def toMessageFormat(toMagicValue: Byte): FetchResponsePartitionData = { + if (messages.hasMagicValue(toMagicValue)) + this + else { + val offsets = new ArrayBuffer[Long] + val newMessages = new ArrayBuffer[Message] + messages.iterator.foreach(messageAndOffset => { + val message = messageAndOffset.message + // File message set only has shallow iterator. We need to do deep iteration here if needed. + if (message.compressionCodec == NoCompressionCodec) { + newMessages += messageAndOffset.message.toFormatVersion(toMagicValue) + offsets += messageAndOffset.offset + } else { + 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. + val newMessageSet = new ByteBufferMessageSet( + compressionCodec = messages.headOption.map(_.message.compressionCodec).getOrElse(NoCompressionCodec), + offsetSeq = offsets.toSeq, + newMessages: _*) + new FetchResponsePartitionData(error, hw, newMessageSet) + } + } } // SENDS @@ -200,13 +231,20 @@ object FetchResponse { case class FetchResponse(correlationId: Int, data: Map[TopicAndPartition, FetchResponsePartitionData], requestVersion: Int = 0, - throttleTimeMs: Int = 0) + throttleTimeMs: Int = 0, + magicValueToUse: Byte = Message.CurrentMagicValue) extends RequestOrResponse() { + + /** + * Convert the message format if necessary. + */ + lazy val dataAfterVersionConversion = data.map{case (topicAndPartition, partitionData) => + topicAndPartition -> partitionData.toMessageFormat(magicValueToUse)} /** * Partitions the data into a map of maps (one for each topic). */ - lazy val dataGroupedByTopic = data.groupBy{ case (topicAndPartition, fetchData) => topicAndPartition.topic } + lazy val dataGroupedByTopic = dataAfterVersionConversion.groupBy{ case (topicAndPartition, fetchData) => topicAndPartition.topic } val headerSizeInBytes = FetchResponse.headerSize(requestVersion) lazy val sizeInBytes = FetchResponse.responseSize(dataGroupedByTopic, requestVersion) diff --git a/core/src/main/scala/kafka/api/ProducerRequest.scala b/core/src/main/scala/kafka/api/ProducerRequest.scala index 11f50099b755a..00d09194ff799 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 diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index e52a9d30f8e59..ef3d6e57a87a4 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.{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) { diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index 48818c3edff8e..434d2fe90db08 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -141,7 +141,9 @@ 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)) + // Tombstone does not care about message magic value, simply use magic value 0. + val tombstone = new Message(bytes = null, key = GroupMetadataManager.groupMetadataKey(group.groupId), + timestamp = Message.NoTimestamp, magicValue = Message.MagicValue_V0) val partitionOpt = replicaManager.getPartition(GroupCoordinator.GroupMetadataTopicName, groupPartition) partitionOpt.foreach { partition => @@ -166,10 +168,12 @@ class GroupMetadataManager(val brokerId: Int, def prepareStoreGroup(group: GroupMetadata, groupAssignment: Map[String, Array[Byte]], responseCallback: Short => Unit): DelayedStore = { - // construct the message to append + // construct the message to append. GroupMetadata does not use message timestamp so we use magic value 0 val message = new Message( key = GroupMetadataManager.groupMetadataKey(group.groupId), - bytes = GroupMetadataManager.groupMetadataValue(group, groupAssignment) + bytes = GroupMetadataManager.groupMetadataValue(group, groupAssignment), + timestamp = Message.NoTimestamp, + magicValue = Message.MagicValue_V0 ) 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 = Message.NoTimestamp, + magicValue = Message.MagicValue_V0 ) }.toSeq @@ -551,7 +557,7 @@ 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 = Message.NoTimestamp, magicValue = Message.MagicValue_V0)) }.groupBy { case (partition, tombstone) => partition } // Append the tombstone messages to the offset partitions. It is okay if the replicas don't receive these (say, diff --git a/core/src/main/scala/kafka/log/FileMessageSet.scala b/core/src/main/scala/kafka/log/FileMessageSet.scala index d4ce4986dff3f..9246cf57f0a0c 100755 --- a/core/src/main/scala/kafka/log/FileMessageSet.scala +++ b/core/src/main/scala/kafka/log/FileMessageSet.scala @@ -139,7 +139,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 } @@ -171,6 +171,35 @@ class FileMessageSet private[kafka](@volatile var file: File, bytesTransferred } + /** + * 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 hasMagicValue(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 + } + /** * Get a shallow iterator over the messages in the set. */ diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 32c194d6ade93..709f6d04eaad3 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -17,6 +17,7 @@ package kafka.log +import kafka.message.Message.TimestampType import kafka.utils._ import kafka.message._ import kafka.common._ @@ -326,8 +327,13 @@ class Log(val dir: File, // assign offsets to the message set val offset = new AtomicLong(nextOffsetMetadata.messageOffset) try { - validMessages = validMessages.validateMessagesAndAssignOffsets(offset, appendInfo.sourceCodec, appendInfo.targetCodec, config - .compact) + validMessages = validMessages.validateMessagesAndAssignOffsets(offset, + 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) } diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index d5c247cab95c3..41910ab92761a 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -425,7 +425,9 @@ private[log] class Cleaner(val id: Int, } messagesRead += 1 } else { - val messages = ByteBufferMessageSet.deepIterator(entry.message) + // We use absolute offset to compare decide whether retain the message or not. This is handled by + // deep iterator. + val messages = ByteBufferMessageSet.deepIterator(entry) val retainedMessages = messages.filter(messageAndOffset => { messagesRead += 1 shouldRetainMessage(source, map, retainDeletes, messageAndOffset) @@ -461,15 +463,24 @@ private[log] class Cleaner(val id: Int, ByteBufferMessageSet.writeMessage(buffer, messageOffset.message, messageOffset.offset) MessageSet.messageSetSize(messagesIterable) } else { + val messageSetTimestamp = MessageSet.validateMagicValuesAndGetTimestamp(messages.map(_.message)) + val firstAbsoluteOffset = messages.head.offset var offset = -1L + val magicValue = messages.head.message.magic 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 = messageSetTimestamp, magicValue = magicValue) { outputStream => val output = new DataOutputStream(CompressionFactory(compressionCodec, outputStream)) try { for (messageOffset <- messages) { val message = messageOffset.message offset = messageOffset.offset - output.writeLong(offset) + // Use relative offset when magic value is greater than 0 + if (magicValue > Message.MagicValue_V0) { + // The offset of the messages are absolute offset, compute the relative offset. + val relativeOffset = messageOffset.offset - firstAbsoluteOffset + output.writeLong(relativeOffset) + } else + output.writeLong(offset) output.writeInt(message.size) output.write(message.buffer.array, message.buffer.arrayOffset, message.buffer.limit) } diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index 7fc7e33bc770d..8709c5f2e3692 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -18,6 +18,8 @@ package kafka.log import java.util.Properties +import kafka.api.ApiVersion +import kafka.message.Message.TimestampType import kafka.server.KafkaConfig import org.apache.kafka.common.utils.Utils import scala.collection._ @@ -44,6 +46,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 +74,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)) + val messageTimestampType = TimestampType.withName(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 +109,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,6 +136,9 @@ 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._ @@ -158,6 +172,9 @@ 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()) diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index aa37d52a0ef7a..9fc68a4a92350 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 diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 5a32de83b6b0b..3a16252587c7c 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -17,34 +17,53 @@ package kafka.message +import kafka.api.{KAFKA_0_10_0_DV0, ApiVersion} +import kafka.message.Message.TimestampType +import kafka.message.Message.TimestampType.TimestampType import kafka.utils.{IteratorTemplate, Logging} 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.utils.{Crc32, 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, + messageSetTimestampAssignor: (Seq[Message]) => Long, + 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 messageSetTimestamp = messageSetTimestampAssignor(messages) var offset = -1L + val magicValue = messages.head.magic 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 = messageSetTimestamp, magicValue = magicValue) { outputStream => val output = new DataOutputStream(CompressionFactory(compressionCodec, outputStream)) try { for (message <- messages) { - offset = offsetCounter.getAndIncrement - output.writeLong(offset) + offset = offsetAssignor.nextAbsoluteOffset + if (message.magic != magicValue) + throw new IllegalArgumentException("Messages in the same compressed message set must have same magic value") + // Use relative offset if magic value is greater than 0 + if (magicValue > Message.MagicValue_V0) + output.writeLong(offsetAssignor.toRelativeOffset(offset)) + else + output.writeLong(offset) output.writeInt(message.size) output.write(message.buffer.array, message.buffer.arrayOffset, message.buffer.limit) } @@ -59,21 +78,53 @@ 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 adjust 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 wrapperMessageTimestamp = if (wrapperMessage.magic > MagicValue_V0) wrapperMessage.timestamp else NoTimestamp + if (wrapperMessage.payload == null) + throw new RuntimeException("wrapper message = " + wrapperMessage) val inputStream: InputStream = new ByteBufferBackedInputStream(wrapperMessage.payload) val compressed: DataInputStream = new DataInputStream(CompressionFactory(wrapperMessage.compressionCodec, inputStream)) - - override def makeNext(): MessageAndOffset = { - try { + val messageAndOffsets = new mutable.Queue[MessageAndOffset] + var lastInnerOffset = -1L + // When magic value is greater than 0, relative offset will be used. + // 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 stream compressing 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 writtern to + // the message set when stream compressing is used. + // + // To solve this issue, we use the following solution: + // 1. When producer create a message set, it simply write 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 = Inner_Offset_of_a_message - Inner_Offset_of_the_last_message + // AO = AO_Of_Last_Inner_Message + RO + // + // 4. This solution works for compacted message set as well + try { + while (true) { // read the offset - val offset = compressed.readLong() + val innerOffset = compressed.readLong() // read record size val size = compressed.readInt() - if (size < Message.MinHeaderSize) + if (size < MinHeaderSize) throw new InvalidMessageException("Message found with corrupt size (" + size + ") in deep iterator") // read the record into an intermediate record buffer @@ -81,18 +132,43 @@ object ByteBufferMessageSet { 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) + // 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}") + if (newMessage.magic > MagicValue_V0 && newMessage.timestamp == InheritedTimestamp) { + // We need to use the wrapper message timestamp when inner message timestamp is Message.InheritedTimestamp + buffer.position(TimestampOffset) + buffer.putLong(wrapperMessageTimestamp) + buffer.rewind() + } + messageAndOffsets += new MessageAndOffset(newMessage, innerOffset) + lastInnerOffset = innerOffset + } + } catch { + case eofe: EOFException => + compressed.close() + case ioe: IOException => + throw new KafkaException(ioe) + } - // the decompressed message should not be a wrapper message since we do not allow nested compression - new MessageAndOffset(newMessage, offset) - } catch { - case eofe: EOFException => - compressed.close() - allDone() - case ioe: IOException => - throw new KafkaException(ioe) + override def makeNext(): MessageAndOffset = { + if (messageAndOffsets.size == 0) + allDone() + else { + val messageAndOffset = messageAndOffsets.dequeue() + val message = messageAndOffset.message + // If magic value is greater than 0, relative offset and timestamp will be used. + if (message.magic > MagicValue_V0) { + val relativeOffset = messageAndOffset.offset - lastInnerOffset + val absoluteOffset = wrapperMessageOffset + relativeOffset + new MessageAndOffset(message, absoluteOffset) + } else + messageAndOffset } + } } } @@ -111,6 +187,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 toRelativeOffset(offset: Long) = offset - offsets(0) + +} + /** * A sequence of messages stored in a byte buffer * @@ -125,11 +215,26 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi 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, + MessageSet.validateMagicValuesAndGetTimestamp, messages:_*)) } def this(compressionCodec: CompressionCodec, offsetCounter: AtomicLong, messages: Message*) { - this(ByteBufferMessageSet.create(offsetCounter, compressionCodec, messages:_*)) + this(ByteBufferMessageSet.create(new OffsetAssigner(offsetCounter, messages.size), compressionCodec, + MessageSet.validateMagicValuesAndGetTimestamp, messages:_*)) + } + + def this(compressionCodec: CompressionCodec, + offsetCounter: AtomicLong, + messageSetTimestampAssignor: (Message*) => Long, + messages: Message*) { + this(ByteBufferMessageSet.create(new OffsetAssigner(offsetCounter, messages.size), compressionCodec, + messageSetTimestampAssignor, messages:_*)) + } + + def this(compressionCodec: CompressionCodec, offsetSeq: Seq[Long], messages: Message*) { + this(ByteBufferMessageSet.create(new OffsetAssigner(offsetSeq), compressionCodec, + MessageSet.validateMagicValuesAndGetTimestamp, messages:_*)) } def this(messages: Message*) { @@ -162,6 +267,14 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi written } + override def hasMagicValue(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() @@ -194,7 +307,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 +315,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 +338,198 @@ 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 based on the following scenarios: + * A. Magic value of a message = 0 and messageFormatVersion is above or equals to 0.10.0-DV1 + * B. Magic value of a message = 1 and messageFormatVersion is lower than 0.10.0-DV1 + * + * If no format conversion or value overwriting is required for messages, this method will perform in-place + * operations and avoids re-compression. */ private[kafka] def validateMessagesAndAssignOffsets(offsetCounter: AtomicLong, 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: ApiVersion = ApiVersion.latestVersion, + messageTimestampType: TimestampType, + messageTimestampDiffMaxMs: Long): ByteBufferMessageSet = { + val magicValueToUse = if (messageFormatVersion.onOrAfter(KAFKA_0_10_0_DV0)) Message.MagicValue_V1 else Message.MagicValue_V0 + if (sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) { + // check the magic value + if (!hasMagicValue(magicValueToUse)) { + // up-conversion + convertNonCompressedMessages(offsetCounter, compactedTopic, magicValueToUse) + } else { + // Do in-place validation, offset assignment and maybe set timestamp + validateNonCompressedMessagesAndAssignOffsetInPlace(offsetCounter, 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 only need to do re-compression in one of the followings situation: + // 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. + + // Re-compression situation 1 and 2 + var requireReCompression = sourceCodec != targetCodec || magicValueToUse == Message.MagicValue_V0 + + var maxTimestamp = Message.NoTimestamp + val expectedRelativeOffset = 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 && magicValueToUse > Message.MagicValue_V0) { + // Re-compression situation 3 + // Check if we need to overwrite timestamp + if (validateTimestamps(message, messageTimestampType, messageTimestampDiffMaxMs, targetCodec)) + requireReCompression = true + // Check if we need to overwrite offset + if (messageAndOffset.offset != expectedRelativeOffset.getAndIncrement) + requireReCompression = true + maxTimestamp = math.max(maxTimestamp, message.timestamp) + } + + // Re-compression situation 4 + if (message.magic != magicValueToUse) + requireReCompression = true + + // The conversion have the following impact on timestamp + // V0 -> V1: All messages will have NoTimestamp(-2) + // V1 -> V0: Timestamp field will be removed + // V1 -> V1: All the timestamp validated above will be used + validatedMessages += message.toFormatVersion(magicValueToUse) }) - new ByteBufferMessageSet(compressionCodec = targetCodec, offsetCounter = offsetCounter, messages = messages.toBuffer:_*) + + if (requireReCompression) { + // Re-compression required. + val messageSetTimestampAssignor = (messages: Seq[Message]) => { + if (magicValueToUse == Message.MagicValue_V0) + Message.NoTimestamp + else if (magicValueToUse > Message.MagicValue_V0 && messageTimestampType == TimestampType.CreateTime) + maxTimestamp + else // Log append time + System.currentTimeMillis() + } + + new ByteBufferMessageSet(compressionCodec = targetCodec, + offsetCounter = offsetCounter, + messageSetTimestampAssignor = messageSetTimestampAssignor, + messages = validatedMessages.toBuffer: _*) + } else { + // Do not do re-compression but simply update the offset and timestamp field of the wrapper message. + buffer.putLong(0, offsetCounter.addAndGet(validatedMessages.size) - 1) + + if (magicValueToUse > Message.MagicValue_V0) { + if (messageTimestampType == TimestampType.CreateTime) + buffer.putLong(MessageSet.LogOverhead + Message.TimestampOffset, maxTimestamp) + else if (messageTimestampType == TimestampType.LogAppendTime) + buffer.putLong(MessageSet.LogOverhead + Message.TimestampOffset, System.currentTimeMillis()) + + // 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 + } } } + private def convertNonCompressedMessages(offsetCounter: AtomicLong, + compactedTopic: Boolean, + toMagicValue: Byte): ByteBufferMessageSet = { + // Get message count, shallow iterator is in-place + var sizeInBytesAfterConversion = shallowValidBytes + this.internalIterator(isShallow = true).foreach(messageAndOffset => + sizeInBytesAfterConversion += 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) + 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 if necessary + if (message.magic != toMagicValue) + message.convertToBuffer(toMagicValue, newMessageBuffer) + else + newMessageBuffer.put(message.buffer.array(), message.buffer.arrayOffset(), message.size) + + newMessagePosition += MessageSet.LogOverhead + newMessageSize + } + newBuffer.rewind() + new ByteBufferMessageSet(newBuffer) + } + + private def validateNonCompressedMessagesAndAssignOffsetInPlace(offsetCounter: AtomicLong, + 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) + validateTimestamps(message, timestampType, timestampDiffMaxMs, NoCompressionCodec) + // update CRC for the message if the timestamp is LogAppendTime + if (timestampType == TimestampType.LogAppendTime) + Utils.writeUnsignedInt(messageBuffer, 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.") + } + + private def validateTimestamps(message: Message, + timestampType: TimestampType, + timestampDiffMaxMs: Long, + compressionCodec: CompressionCodec): Boolean = { + val now = System.currentTimeMillis() + var overwritten = false + if (timestampType == TimestampType.CreateTime && math.abs(message.timestamp - now) > timestampDiffMaxMs) + throw new InvalidMessageException(s"Timestamp ${message.timestamp} of message is out of range. " + + s"The timestamp should be within [${now - timestampDiffMaxMs}, ${now + timestampDiffMaxMs}") + else if (timestampType == TimestampType.LogAppendTime) { + if (compressionCodec == NoCompressionCodec) + message.buffer.putLong(Message.TimestampOffset, now) + else { + if (message.timestamp != Message.InheritedTimestamp) { + message.buffer.putLong(Message.TimestampOffset, Message.InheritedTimestamp) + overwritten = true + } + } + } + overwritten + } /** * The total number of bytes in this message set, including any partial trailing messages diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala index 999b11500f9ff..8686876d5041a 100755 --- a/core/src/main/scala/kafka/message/Message.scala +++ b/core/src/main/scala/kafka/message/Message.scala @@ -26,7 +26,13 @@ import org.apache.kafka.common.utils.Utils * Constants related to messages */ object Message { - + + case object TimestampType extends Enumeration { + type TimestampType = Value + val CreateTime = Value(0, "CreateTime") + val LogAppendTime = Value(1, "LogAppendTime") + } + /** * The current offset and size for all the fixed-length fields */ @@ -36,23 +42,47 @@ 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 + // Message format version 0 does not have timestamp field. + val KeySizeOffset_V0 = AttributesOffset + AttributesLength + // Message format version 1 has timestamp field. + val KeySizeOffset_V1 = TimestampOffset + TimestampLength val KeySizeLength = 4 - val KeyOffset = KeySizeOffset + KeySizeLength + // Message format version 0 does not have timestamp field. + val KeyOffset_V0 = KeySizeOffset_V0 + KeySizeLength + // Message format version 1 has timestamp field. + 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 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 come from a message in message format V0 with empty key and value. + */ + val MinMessageOverhead = KeyOffset_V0 + ValueSizeLength /** * The minimum valid size for the message header + * The MinHeaderSize does not include the TimestampLength for backward compatibility. + * 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 come from message format V0. */ - val MinHeaderSize = CrcLength + MagicLength + AttributesLength + KeySizeLength + ValueSizeLength + val MinHeaderSize = MessageHeaderSizeMap.values.min /** - * 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. @@ -65,25 +95,42 @@ object Message { */ val NoCompression: Int = 0 + /** + * To indicate timestamp is not defined so "magic" value 0 will be used. + */ + val NoTimestamp: Long = -2 + + /** + * Derived timestamp is used for inner messages of compressed message when message.timestamp.type is set to + * LogAppendTime. In that case the timestamp of the message is defined by the wrapper message's timestamp. + */ + val InheritedTimestamp: 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 + * 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 (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 + * 4. (Optional) 8 byte timestamp only if "magic" identifier is 1 + * 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. */ class Message(val buffer: ByteBuffer) { import kafka.message.Message._ - + /** * A constructor to create a Message * @param bytes The payload of the message @@ -93,26 +140,34 @@ class Message(val buffer: ByteBuffer) { * @param payloadSize The size of the payload to use */ def this(bytes: Array[Byte], - key: Array[Byte], + key: Array[Byte], + timestamp: Long, 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 buffer.put(attributes) + // Only put timestamp when "magic" value is 1 + if (magic == MagicValue_V1) + buffer.putLong(timestamp) if(key == null) { buffer.putInt(-1) } else { @@ -126,22 +181,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, 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 +229,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 +251,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 +275,15 @@ 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 + */ + def timestamp: Long = { + if (magic == MagicValue_V0) + throw new IllegalStateException("The message with magic byte 0 does not have a timestamp") + buffer.getLong(TimestampOffset) + } /** * The compression codec used with this message @@ -221,8 +299,48 @@ 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) { + 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 (magic == Message.MagicValue_V0 && toMagicValue == Message.MagicValue_V1) { + // Up-conversion, reserve CRC and update magic byte + byteBuffer.position(Message.MagicOffset) + byteBuffer.put(Message.MagicValue_V1) + byteBuffer.put(attributes) + // Up-conversion, insert the timestamp field + 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(attributes) + // Down-conversion, skip the timestamp field + byteBuffer.put(buffer.array(), buffer.arrayOffset() + Message.KeySizeOffset_V1, size - Message.KeySizeOffset_V1) + } + // update crc value + val newMessage = new Message(byteBuffer) + Utils.writeUnsignedInt(byteBuffer, Message.CrcOffset, newMessage.computeChecksum()) + byteBuffer.rewind() + } + /** * Read a size-delimited byte buffer starting at the given offset */ @@ -240,9 +358,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 && timestamp != InheritedTimestamp) + 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, timestamp = $timestamp, crc = $checksum, key = $key, payload = $payload)" + } + override def equals(any: Any): Boolean = { any match { case that: Message => this.buffer.equals(that.buffer) diff --git a/core/src/main/scala/kafka/message/MessageSet.scala b/core/src/main/scala/kafka/message/MessageSet.scala index 28b56e68cfdbb..84107d6736883 100644 --- a/core/src/main/scala/kafka/message/MessageSet.scala +++ b/core/src/main/scala/kafka/message/MessageSet.scala @@ -54,6 +54,25 @@ 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 max timestamp + * of the inner messages. + */ + def validateMagicValuesAndGetTimestamp(messages: Seq[Message]): Long = { + val sampleMagicValue = messages.head.magic + val sampleTimestamp = if (sampleMagicValue > Message.MagicValue_V0) messages.head.timestamp else Message.NoTimestamp + var largestTimestamp: Long = Message.NoTimestamp + for (message <- messages) { + if (sampleTimestamp == Message.InheritedTimestamp && message.timestamp != sampleTimestamp) + throw new IllegalStateException("Messages in the same compressed message set should have same timestamp type") + if (message.magic != sampleMagicValue) + throw new IllegalStateException("Messages in the same compressed message set must have same magic value") + if (sampleMagicValue > Message.MagicValue_V0) + largestTimestamp = math.max(largestTimestamp, message.timestamp) + } + largestTimestamp + } + } /** @@ -70,7 +89,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 messages in the message set have the expected magic value + */ + def hasMagicValue(expectedMagicValue: Byte): Boolean + /** * Provides an iterator over the message/offset pairs in this set */ diff --git a/core/src/main/scala/kafka/message/MessageWriter.scala b/core/src/main/scala/kafka/message/MessageWriter.scala index 0c6040e9f5f27..cc8ee38b36fb3 100755 --- a/core/src/main/scala/kafka/message/MessageWriter.scala +++ b/core/src/main/scala/kafka/message/MessageWriter.scala @@ -26,13 +26,15 @@ 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, magicValue: Byte)(writePayload: OutputStream => Unit): Unit = { withCrc32Prefix { - write(CurrentMagicValue) + write(magicValue) var attributes: Byte = 0 if (codec.codec > 0) attributes = (attributes | (CompressionCodeMask & codec.codec)).toByte write(attributes) + if (magicValue > MagicValue_V0) + writeLong(timestamp) // write the key if (key == null) { writeInt(-1) @@ -61,6 +63,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) diff --git a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala index 4e67ba4ce5073..e30963dac9046 100755 --- a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala +++ b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala @@ -129,9 +129,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 = Message.NoTimestamp, + magicValue = Message.MagicValue_V0)) 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 = Message.NoTimestamp, + magicValue = Message.MagicValue_V0)) } catch { case t: Throwable => producerStats.serializationErrorRate.mark() diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index b3873a6135836..4286d0d338ec0 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("Error in fetch %s. + fetchRequest", t) inLock(partitionMapLock) { partitionsWithError ++= partitionMap.keys // there is an error occurred while fetching partitions, sleep a while diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index f2e95332e8f72..2b7200cf2bc2b 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} @@ -426,6 +426,18 @@ class KafkaApis(val requestChannel: RequestChannel, // the callback for sending a fetch response def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData]) { + + // determine the magic value to use + // Only send messages with magic value 1 when client supports it and server has this format on disk. + val magicValueToUse = + if (fetchRequest.versionId > 1 && config.messageFormatVersion.onOrAfter(KAFKA_0_10_0_DV0)) + // TODO: Change magic value to 1 after o.a.k.client side change is done. + // We cannot send message format 1 back yet, because replica fetcher thread uses o.a.k.clients schema + // that does not have magic value 1 yet. + Message.MagicValue_V0 + else + Message.MagicValue_V0 + val mergedResponseStatus = responsePartitionData ++ unauthorizedResponseStatus mergedResponseStatus.foreach { case (topicAndPartition, data) => @@ -440,7 +452,9 @@ class KafkaApis(val requestChannel: RequestChannel, } def fetchResponseCallback(delayTimeMs: Int) { - val response = FetchResponse(fetchRequest.correlationId, mergedResponseStatus, fetchRequest.versionId, delayTimeMs) + info(s"Sending fetch response to ${fetchRequest.clientId} with ${responsePartitionData.values.map(_.messages.size).sum}" + + s" messages using magic value $magicValueToUse") + val response = FetchResponse(fetchRequest.correlationId, mergedResponseStatus, fetchRequest.versionId, delayTimeMs, magicValueToUse) requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, response))) } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 2c6311cdf0a22..faedb0a5e75c1 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -23,6 +23,7 @@ import kafka.api.ApiVersion import kafka.cluster.EndPoint import kafka.consumer.ConsumerConfig import kafka.coordinator.OffsetConfig +import kafka.message.Message.TimestampType import kafka.message.{BrokerCompressionCodec, CompressionCodec, Message, MessageSet} import kafka.utils.CoreUtils import org.apache.kafka.clients.CommonClientConfigs @@ -94,9 +95,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 +255,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 = "messge.timestamp.type" + val MessageTimestampDifferenceMaxMsProp = "message.timestamp.difference.max.ms" /** ********* Replication configuration ***********/ val ControllerSocketTimeoutMsProp = "controller.socket.timeout.ms" val DefaultReplicationFactorProp = "default.replication.factor" @@ -417,6 +424,10 @@ 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." + val MessageTimestampTypeDoc = "Define the whether the timestamp in the message is message create time or log append time." + val MessageTimestampDifferenceMaxMsDoc = "Set maximum allowed time difference between broker local time and message's timestamp. " + + "This configuration only works when message.timestamp.type=CreateTime. A message will be rejected if its timestamp exceeds this threshold." /** ********* Replication configuration ***********/ val ControllerSocketTimeoutMsDoc = "The socket timeout for controller-to-broker channels" val ControllerMessageQueueSizeDoc = "The buffer size for controller-to-broker-channels" @@ -591,6 +602,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 +795,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 = ApiVersion(getString(KafkaConfig.MessageFormatVersionProp)) + val messageTimestampType = getString(KafkaConfig.MessageTimestampTypeProp) + val messageTimestampDifferenceMaxMs = getLong(KafkaConfig.MessageTimestampDifferenceMaxMsProp) /** ********* Replication configuration ***********/ val controllerSocketTimeoutMs: Int = getInt(KafkaConfig.ControllerSocketTimeoutMsProp) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 41719e2add7f3..9d6936ada8a75 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.toString()) + logProps.put(LogConfig.MessageTimestampTypeProp, kafkaConfig.messageTimestampType) + logProps.put(LogConfig.MessageTimestampDifferenceMaxMsProp, kafkaConfig.messageTimestampDifferenceMaxMs) logProps } } @@ -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) diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index c5f33609fd05d..8c454a373e30e 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_DV0, 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_DV0)) 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 diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 61b6887e47ea3..788875cc9a14b 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -26,7 +26,7 @@ import kafka.cluster.{Partition, Replica} import kafka.common._ import kafka.controller.KafkaController import kafka.log.{LogAppendInfo, LogManager} -import kafka.message.{ByteBufferMessageSet, MessageSet} +import kafka.message.{InvalidMessageException, ByteBufferMessageSet, MessageSet} import kafka.metrics.KafkaMetricsGroup import kafka.utils._ import org.apache.kafka.common.errors.{OffsetOutOfRangeException, RecordBatchTooLargeException, ReplicaNotAvailableException, RecordTooLargeException, @@ -440,6 +440,8 @@ class ReplicaManager(val config: KafkaConfig, (topicPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(mstle))) case imse: CorruptRecordException => (topicPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(imse))) + case ime : InvalidMessageException => + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(ime))) case t: Throwable => BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).failedProduceRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats.failedProduceRequestRate.mark() diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala index 73743aa46f3bb..132e4459ea1d6 100755 --- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala @@ -395,7 +395,7 @@ class ChecksumMessageFormatter extends MessageFormatter { } def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream) { - val chksum = new Message(value, key).checksum + val chksum = new Message(value, key, Message.NoTimestamp, Message.MagicValue_V0).checksum output.println(topicStr + "checksum:" + chksum) } } diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index fd15014e5eb6b..3c41c7cdff93c 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) diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala index 42928a355cc02..c970fb90933e1 100644 --- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala @@ -17,7 +17,6 @@ package kafka.api -import java.io.File import java.util.Properties import java.util.concurrent.TimeUnit @@ -238,7 +237,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, Message.NoTimestamp, Message.MagicValue_V0), messageSet1(i).message) assertEquals(i.toLong, messageSet1(i).offset) } } finally { diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index b2f96e56dd614..7af1781e3a9c1 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -36,6 +36,12 @@ 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 { + // TODO: Remove this after o.a.k.clients has Record version 1 (relative offset + timestamp) + // We need this config because testRecordTooLarge does not work when message format conversion occurs on server + // side. In that case we lose the residue part of the file message set from zero-copy transfer, which is exactly + // what we are depending on to determine if message size is too large. + this.serverConfig.setProperty(KafkaConfig.MessageFormatVersionProp, "0.9.0") + @Test def testAutoCommitOnClose() { this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true") diff --git a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala index 0d401f7ba77c8..9bbe005d3466a 100755 --- a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala @@ -101,7 +101,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, Message.NoTimestamp, Message.MagicValue_V0), messageSet(index).message) assertEquals(index.toLong, messageSet(index).offset) index += 1 } diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index e4b8854f6e5a2..c055ae4486e8e 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -20,11 +20,13 @@ package kafka.api import kafka.cluster.{EndPoint, Broker} import kafka.common.{OffsetAndMetadata, OffsetMetadataAndError} import kafka.common._ -import kafka.message.{Message, ByteBufferMessageSet} +import kafka.log.FileMessageSet +import kafka.message._ import kafka.utils.SystemTime import kafka.controller.LeaderIsrAndControllerEpoch import kafka.common.TopicAndPartition +import kafka.utils.TestUtils._ import java.nio.ByteBuffer @@ -287,4 +289,97 @@ 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) } + + @Test + def testFetchResponseFormatConversion() { + // Up conversion + 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:_*) + + // up conversion for non-compressed messages + var fileMessageSet = new FileMessageSet(tempFile()) + fileMessageSet.append(messageSetV0) + fileMessageSet.flush() + var convertedPartitionData = new FetchResponsePartitionData(messages = fileMessageSet).toMessageFormat(Message.MagicValue_V1) + var i = 0 + for (messageAndOffset <- convertedPartitionData.messages) { + val message = messageAndOffset.message + val offset = messageAndOffset.offset + assertEquals("magic byte should be 1", Message.MagicValue_V1, message.magic) + assertEquals("offset should not change", offsets(i), offset) + assertEquals("key should not change", messagesV0(i).key, message.key) + assertEquals("payload should not change", messagesV0(i).payload, message.payload) + i += 1 + } + + fileMessageSet = new FileMessageSet(tempFile()) + fileMessageSet.append(compressedMessageSetV0) + fileMessageSet.flush() + // up conversion for compressed messages + convertedPartitionData = new FetchResponsePartitionData(messages = fileMessageSet).toMessageFormat(Message.MagicValue_V1) + i = 0 + for (messageAndOffset <- convertedPartitionData.messages) { + val message = messageAndOffset.message + val offset = messageAndOffset.offset + assertEquals("magic byte should be 1", Message.MagicValue_V1, message.magic) + assertEquals("offset should not change", offsets(i), offset) + assertEquals("key should not change", messagesV0(i).key, message.key) + assertEquals("payload should not change", messagesV0(i).payload, message.payload) + i += 1 + } + + // Down conversion + 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 for non-compressed messages + fileMessageSet = new FileMessageSet(tempFile()) + fileMessageSet.append(messageSetV1) + fileMessageSet.flush() + convertedPartitionData = new FetchResponsePartitionData(messages = fileMessageSet).toMessageFormat(Message.MagicValue_V0) + i = 0 + for (messageAndOffset <- convertedPartitionData.messages) { + val message = messageAndOffset.message + val offset = messageAndOffset.offset + assertEquals("magic byte should be 0", Message.MagicValue_V0, message.magic) + assertEquals("offset should not change", offsets(i), offset) + assertEquals("key should not change", messagesV0(i).key, message.key) + assertEquals("payload should not change", messagesV0(i).payload, message.payload) + i += 1 + } + + fileMessageSet = new FileMessageSet(tempFile()) + fileMessageSet.append(compressedMessageSetV1) + fileMessageSet.flush() + // down conversion for compressed messages + convertedPartitionData = new FetchResponsePartitionData(messages = fileMessageSet).toMessageFormat(Message.MagicValue_V0) + i = 0 + for (messageAndOffset <- convertedPartitionData.messages) { + val message = messageAndOffset.message + val offset = messageAndOffset.offset + assertEquals("magic byte should be 0", Message.MagicValue_V0, message.magic) + assertEquals("offset should not change", offsets(i), offset) + assertEquals("key should not change", messagesV0(i).key, message.key) + assertEquals("payload should not change", messagesV0(i).payload, message.payload) + i += 1 + } + } + } diff --git a/core/src/test/scala/unit/kafka/log/CleanerTest.scala b/core/src/test/scala/unit/kafka/log/CleanerTest.scala index a8092de40b928..c5c3eb4fd2a7b 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_V0)) 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_V0)) 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_V0)) } diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index de3d7a33d0cbd..6b91611f1e9cb 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 diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 51cd62c30ccc4..1be9e6557aacc 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) } }) diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 46bfbed476e49..f272acb00fadb 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._ @@ -36,6 +37,8 @@ class LogManagerTest { logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) logProps.put(LogConfig.SegmentIndexBytesProp, 4096: java.lang.Integer) logProps.put(LogConfig.RetentionMsProp, maxLogAgeMs: java.lang.Integer) + // We need to use magic value 1 here because some tests requires no message format conversion. + logProps.put(LogConfig.MessageFormatVersionProp, ApiVersion.latestVersion.toString()) val logConfig = LogConfig(logProps) var logDir: File = null var logManager: LogManager = null diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 47908e718aa9b..426b5e825e84c 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 diff --git a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala index 511060ebbb3a7..267e383b20f10 100644 --- a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala @@ -19,6 +19,8 @@ package kafka.message import java.nio._ import java.util.concurrent.atomic.AtomicLong +import kafka.api.ApiVersion +import kafka.message.Message.TimestampType import org.junit.Assert._ import org.junit.Test import kafka.utils.TestUtils @@ -135,27 +137,233 @@ 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 compressedMessages = getMessages(magicValue = Message.MagicValue_V1, timestamp = 5L, 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 = ApiVersion.latestVersion, + messageTimestampType = TimestampType.LogAppendTime, + messageTimestampDiffMaxMs = 1000L) + + val validatedCompressedMessages = + compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(0), + sourceCodec = DefaultCompressionCodec, + targetCodec = DefaultCompressionCodec, + messageFormatVersion = ApiVersion.latestVersion, + messageTimestampType = TimestampType.LogAppendTime, + messageTimestampDiffMaxMs = 1000L) + + val validatedCompressedMessagesWithoutRecompression = + compressedMessagesWithoutRecompression.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(0), + sourceCodec = DefaultCompressionCodec, + targetCodec = DefaultCompressionCodec, + messageFormatVersion = ApiVersion.latestVersion, + messageTimestampType = TimestampType.LogAppendTime, + messageTimestampDiffMaxMs = 1000L) + + val now = System.currentTimeMillis() + assertEquals("message set size should not change", messages.size, validatedMessages.size) + for (messageAndOffset <- validatedMessages) + assertTrue(messageAndOffset.message.timestamp >= startTime && messageAndOffset.message.timestamp <= now) + + assertEquals("message set size should not change", compressedMessages.size, validatedCompressedMessages.size) + for (messageAndOffset <- validatedCompressedMessages) { + assertTrue(s"Timestamp of message ${messageAndOffset.message}} should be between $startTime and $now", + messageAndOffset.message.timestamp >= startTime && messageAndOffset.message.timestamp <= now) + } + assertTrue("MessageSet should still valid", validatedCompressedMessages.shallowIterator.next().message.isValid) + + assertEquals("message set size should not change", compressedMessagesWithoutRecompression.size, + validatedCompressedMessagesWithoutRecompression.size) + for (messageAndOffset <- validatedCompressedMessagesWithoutRecompression) { + assertTrue(s"Timestamp of message ${messageAndOffset.message}} should be between $startTime and $now", + messageAndOffset.message.timestamp >= startTime && messageAndOffset.message.timestamp <= now) + } + assertTrue("MessageSet should still valid", validatedCompressedMessagesWithoutRecompression.shallowIterator.next().message.isValid) + } + + @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 = ApiVersion.latestVersion, + messageTimestampType = TimestampType.CreateTime, + messageTimestampDiffMaxMs = 1000L) + + val validatedCompressedMessages = + compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(0), + sourceCodec = DefaultCompressionCodec, + targetCodec = DefaultCompressionCodec, + messageFormatVersion = ApiVersion.latestVersion, + messageTimestampType = TimestampType.CreateTime, + messageTimestampDiffMaxMs = 1000L) + + for (messageAndOffset <- validatedMessages) + assertEquals(messageAndOffset.message.timestamp, now) + for (messageAndOffset <- validatedCompressedMessages) + assertEquals(messageAndOffset.message.timestamp, now) + } + + @Test + def testInvalidCreateTime() { + val now = System.currentTimeMillis() + val messages = getMessages(magicValue = Message.MagicValue_V1, timestamp = now - 1001L, codec = NoCompressionCodec) + val compressedMessages = getMessages(magicValue = Message.MagicValue_V1, timestamp = now - 1001L, codec = DefaultCompressionCodec) + + try { + messages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(0), + sourceCodec = NoCompressionCodec, + targetCodec = NoCompressionCodec, + messageFormatVersion = ApiVersion.latestVersion, + messageTimestampType = TimestampType.CreateTime, + messageTimestampDiffMaxMs = 1000L) + fail("Should throw InvalidMessageException.") + } catch { + case e: InvalidMessageException => + } + + try { + compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(0), + sourceCodec = DefaultCompressionCodec, + targetCodec = DefaultCompressionCodec, + messageFormatVersion = ApiVersion.latestVersion, + messageTimestampType = TimestampType.CreateTime, + messageTimestampDiffMaxMs = 1000L) + fail("Should throw InvalidMessageException.") + } catch { + case e: InvalidMessageException => + } + } @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 testAbsoluteOffsetAssignment() { + val messages = getMessages(magicValue = Message.MagicValue_V0, codec = NoCompressionCodec) + val compressedMessages = getMessages(magicValue = Message.MagicValue_V0, codec = DefaultCompressionCodec) + // check uncompressed offsets + checkOffsets(messages, 0) + val offset = 1234567 + checkOffsets(messages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(offset), + sourceCodec = NoCompressionCodec, + targetCodec = NoCompressionCodec, + messageFormatVersion = ApiVersion("0.9.0"), + messageTimestampType = TimestampType.CreateTime, + messageTimestampDiffMaxMs = 1000L), offset) + + // check compressed messages + checkOffsets(compressedMessages, 0) + checkOffsets(compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(offset), + sourceCodec = DefaultCompressionCodec, + targetCodec = DefaultCompressionCodec, + messageFormatVersion = ApiVersion("0.9.0"), + messageTimestampType = TimestampType.CreateTime, + 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) - var offset = 1234567 - checkOffsets(messages.validateMessagesAndAssignOffsets(new AtomicLong(offset), NoCompressionCodec, NoCompressionCodec), offset) + val offset = 1234567 + val messageWithOffset = messages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(offset), + sourceCodec = NoCompressionCodec, + targetCodec = NoCompressionCodec, + messageTimestampType = TimestampType.CreateTime, + messageTimestampDiffMaxMs = 5000L) + checkOffsets(messageWithOffset, offset) // check compressed messages checkOffsets(compressedMessages, 0) - checkOffsets(compressedMessages.validateMessagesAndAssignOffsets(new AtomicLong(offset), DefaultCompressionCodec, DefaultCompressionCodec), offset) + val compressedMessagesWithOffset = compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(offset), + sourceCodec = DefaultCompressionCodec, + targetCodec = DefaultCompressionCodec, + messageTimestampType = TimestampType.CreateTime, + 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 = ApiVersion.latestVersion, + messageTimestampType = TimestampType.LogAppendTime, + messageTimestampDiffMaxMs = 1000L), offset) + + // check compressed messages + checkOffsets(compressedMessagesV0, 0) + checkOffsets(compressedMessagesV0.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(offset), + sourceCodec = DefaultCompressionCodec, + targetCodec = DefaultCompressionCodec, + messageFormatVersion = ApiVersion.latestVersion, + messageTimestampType = TimestampType.LogAppendTime, + 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 = ApiVersion("0.9.0"), + messageTimestampType = TimestampType.CreateTime, + messageTimestampDiffMaxMs = 5000L), offset) + + // check compressed messages + checkOffsets(compressedMessagesV1, 0) + checkOffsets(compressedMessagesV1.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(offset), + sourceCodec = DefaultCompressionCodec, + targetCodec = DefaultCompressionCodec, + messageFormatVersion = ApiVersion("0.9.0"), + messageTimestampType = TimestampType.CreateTime, + 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 +377,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), + new Message("there".getBytes), + new Message("beautiful".getBytes)) + } 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)) + } + } } diff --git a/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala b/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala index d8613f7a96113..cf1afeff7e080 100644 --- a/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala @@ -42,7 +42,9 @@ 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_V0), + new Message(bytes2k, Message.NoTimestamp, Message.MagicValue_V0), + new Message(bytes3k, Message.NoTimestamp, Message.MagicValue_V0)) testCompressSize(GZIPCompressionCodec, messages, 388) diff --git a/core/src/test/scala/unit/kafka/message/MessageTest.scala b/core/src/test/scala/unit/kafka/message/MessageTest.scala index 1755633bc3c0f..8bff121d61e8d 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,46 @@ 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 { + try { + v.message.timestamp + fail("message.timestamp should throw exception.") + } catch { + case e: IllegalStateException => + } + } + // 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,11 +105,78 @@ 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() { + for (v <- messages) { + if (v.magicValue == Message.MagicValue_V0) { + assertEquals("Message should be the same when convert to the same version.", + v.message.toFormatVersion(Message.MagicValue_V0), v.message) + val messageV1 = v.message.toFormatVersion(Message.MagicValue_V1) + assertEquals("Size difference is not expected value", messageV1.size - v.message.size, + Message.headerSizeDiff(Message.MagicValue_V0, Message.MagicValue_V1)) + assertTrue("Message should still be valid", messageV1.isValid) + assertEquals("Timestamp should be NoTimestamp", messageV1.timestamp, Message.NoTimestamp) + assertEquals("Magic value should be 1 now", messageV1.magic, Message.MagicValue_V1) + if (messageV1.hasKey) + assertEquals("Message key should not change", messageV1.key, ByteBuffer.wrap(v.key)) + else + assertNull(messageV1.key) + if(v.payload == null) { + assertTrue(messageV1.isNull) + assertEquals("Payload should be null", null, messageV1.payload) + } else { + assertEquals("Message payload should not change", messageV1.payload, ByteBuffer.wrap(v.payload)) + } + assertEquals("Compression codec should not change", messageV1.compressionCodec, v.codec) + } else if (v.magicValue == Message.MagicValue_V1) { + assertEquals("Message should be the same when convert to the same version.", + v.message.toFormatVersion(Message.MagicValue_V1), v.message) + val messageV0 = v.message.toFormatVersion(Message.MagicValue_V0) + assertEquals("Size difference is not expected value", messageV0.size - v.message.size, + Message.headerSizeDiff(Message.MagicValue_V1, Message.MagicValue_V0)) + assertTrue("Message should still be valid", messageV0.isValid) + try { + messageV0.timestamp + fail("message.timestamp should throw exception.") + } catch { + case e: IllegalStateException => + } + assertEquals("Magic value should be 1 now", messageV0.magic, Message.MagicValue_V0) + if (messageV0.hasKey) + assertEquals("Message key should not change", messageV0.key, ByteBuffer.wrap(v.key)) + else + assertNull(messageV0.key) + if(v.payload == null) { + assertTrue(messageV0.isNull) + assertEquals("Payload should be null", null, messageV0.payload) + } else { + assertEquals("Message payload should not change", messageV0.payload, ByteBuffer.wrap(v.payload)) + } + assertEquals("Compression codec should not change", messageV0.compressionCodec, v.codec) + } + } + } + + @Test(expected = classOf[IllegalArgumentException]) + def testInValidTimestampAndMagicValueCombination() { + new Message("hello".getBytes, Message.InheritedTimestamp, Message.MagicValue_V0) + } + + @Test(expected = classOf[IllegalArgumentException]) + def testInValidTimestamp() { + new Message("hello".getBytes, -3L, Message.MagicValue_V0) + } + + @Test(expected = classOf[IllegalArgumentException]) + def testInValidMagicByte() { + new Message("hello".getBytes, 0L, 2) + } + @Test def testIsHashable() { // this is silly, but why not diff --git a/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala b/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala index 3993fdb1cc68b..d38108755e0cd 100644 --- a/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala @@ -34,7 +34,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, magicValue = Message.MagicValue_V0) { output => val out = if (codec == NoCompressionCodec) output else CompressionFactory(codec, output) try { val p = rnd.nextInt(bytes.length) @@ -101,7 +101,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_V0) assertEquals(expected.buffer, actual.buffer) } @@ -109,7 +109,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_V0) assertEquals( decompress(toArray(expected.payload), SnappyCompressionCodec).toSeq, @@ -122,7 +122,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_V0) assertEquals(expected.buffer, actual.buffer) } diff --git a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala index 60d25881751f9..cea4ec590bf36 100755 --- a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala @@ -466,11 +466,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, Message.NoTimestamp, Message.MagicValue_V0)): _*) } 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 = Message.NoTimestamp, magicValue = Message.MagicValue_V0)): _*) } } diff --git a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala index f356a69c8e86e..ab735beea2e17 100755 --- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala @@ -181,8 +181,8 @@ 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) + assertEquals(new Message(bytes = "test1".getBytes, key = "test".getBytes, timestamp = Message.NoTimestamp, magicValue = Message.MagicValue_V0), messageSet(0).message) + assertEquals(new Message(bytes = "test2".getBytes, key = "test".getBytes, timestamp = Message.NoTimestamp, magicValue = Message.MagicValue_V0), messageSet(1).message) producer1.close() val props2 = new util.Properties() @@ -260,7 +260,7 @@ 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) + assertEquals(new Message(bytes = "test1".getBytes, key = "test".getBytes, timestamp = Message.NoTimestamp, magicValue = Message.MagicValue_V0), messageSet1.next.message) assertFalse("Message set should have another message", messageSet1.hasNext) } catch { case e: Exception => fail("Not expected", e) diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala index 6e7b964b03ef0..351c2db8f2cc7 100644 --- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala @@ -107,7 +107,7 @@ class SyncProducerTest extends KafkaServerTestHarness { Assert.assertEquals(Errors.MESSAGE_TOO_LARGE.code, response1.status(TopicAndPartition("test", 0)).error) Assert.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)) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 2479b37e48ade..c9ac97acf5c17 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 @@ -283,12 +283,12 @@ class KafkaConfigTest { props.put(KafkaConfig.InterBrokerProtocolVersionProp,"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") 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)) diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index b6bc4fc3d6119..f9ce10b1de0a5 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", ApiVersion.latestVersion.toString()) props } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 29e10827740ad..252308304d445 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 From d16091c8441f64b08f4a985bf53b0e043f3331b1 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Wed, 20 Jan 2016 03:46:22 -0800 Subject: [PATCH 02/33] Added timestamp and relative offset to clients package. --- checkstyle/import-control.xml | 1 + .../org/apache/kafka/clients/Metadata.java | 21 +++ .../clients/consumer/ConsumerRecord.java | 11 +- .../clients/consumer/internals/Fetcher.java | 3 +- .../kafka/clients/producer/KafkaProducer.java | 13 +- .../kafka/clients/producer/MockProducer.java | 5 +- .../clients/producer/ProducerRecord.java | 52 +++++- .../clients/producer/RecordMetadata.java | 15 +- .../internals/FutureRecordMetadata.java | 10 +- .../producer/internals/RecordAccumulator.java | 16 +- .../producer/internals/RecordBatch.java | 16 +- .../clients/producer/internals/Sender.java | 18 +- .../kafka/common/protocol/Protocol.java | 22 ++- .../kafka/common/record/Compressor.java | 22 ++- .../kafka/common/record/MemoryRecords.java | 112 +++++++++--- .../apache/kafka/common/record/Record.java | 113 +++++++++--- .../kafka/common/requests/ProduceRequest.java | 3 +- .../common/requests/ProduceResponse.java | 19 +- .../org/apache/kafka/common/utils/Crc32.java | 11 ++ .../clients/consumer/MockConsumerTest.java | 4 +- .../consumer/internals/FetcherTest.java | 14 +- .../clients/producer/ProducerRecordTest.java | 2 +- .../clients/producer/RecordSendTest.java | 7 +- .../internals/RecordAccumulatorTest.java | 30 ++-- .../producer/internals/SenderTest.java | 11 +- .../common/record/MemoryRecordsTest.java | 8 +- .../kafka/common/record/RecordTest.java | 16 +- .../common/requests/RequestResponseTest.java | 15 +- .../connect/runtime/WorkerSinkTaskTest.java | 2 +- .../runtime/WorkerSinkTaskThreadedTest.java | 4 +- .../connect/runtime/WorkerSourceTaskTest.java | 2 +- .../storage/KafkaConfigStorageTest.java | 24 +-- .../storage/KafkaOffsetBackingStoreTest.java | 16 +- .../kafka/connect/util/KafkaBasedLogTest.java | 16 +- .../scala/kafka/api/ProducerRequest.scala | 2 +- .../scala/kafka/api/ProducerResponse.scala | 12 +- core/src/main/scala/kafka/log/Log.scala | 19 +- .../kafka/message/ByteBufferMessageSet.scala | 167 ++++++++++-------- .../main/scala/kafka/message/Message.scala | 9 +- .../main/scala/kafka/server/KafkaApis.scala | 11 +- .../scala/kafka/server/ReplicaManager.scala | 10 +- .../kafka/api/BaseConsumerTest.scala | 14 +- .../kafka/api/BaseProducerSendTest.scala | 81 ++++++++- .../kafka/api/PlaintextConsumerTest.scala | 109 ++++++++---- .../kafka/api/ProducerCompressionTest.scala | 5 +- .../message/ByteBufferMessageSetTest.scala | 6 +- .../unit/kafka/message/MessageTest.scala | 2 +- .../unit/kafka/producer/ProducerTest.scala | 6 +- .../kafka/producer/SyncProducerTest.scala | 75 ++++---- .../processor/internals/RecordQueue.java | 2 +- .../internals/PartitionGroupTest.java | 12 +- .../internals/ProcessorStateManagerTest.java | 6 +- .../processor/internals/RecordQueueTest.java | 18 +- .../processor/internals/StandbyTaskTest.java | 18 +- .../processor/internals/StreamTaskTest.java | 30 ++-- .../test/ProcessorTopologyTestDriver.java | 2 +- 56 files changed, 870 insertions(+), 400 deletions(-) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index a663cf7c5aa3a..b183b3d6081dd 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -84,6 +84,7 @@ + diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java index 73a9f333cc7a4..7f97c806b9f50 100644 --- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.record.Record; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,6 +46,7 @@ public final class Metadata { private Cluster cluster; private boolean needUpdate; private final Set topics; + private final Set topicsUsingLogAppendTime; private final List listeners; private boolean needMetadataForAllTopics; @@ -70,6 +72,7 @@ public Metadata(long refreshBackoffMs, long metadataExpireMs) { this.cluster = Cluster.empty(); this.needUpdate = false; this.topics = new HashSet(); + this.topicsUsingLogAppendTime = new HashSet(); this.listeners = new ArrayList<>(); this.needMetadataForAllTopics = false; } @@ -88,6 +91,24 @@ public synchronized void add(String topic) { topics.add(topic); } + /** + * Record the timestamp type used by a topic. This information is needed to determine whether the timestamp + * should be overwritten or not. + */ + public synchronized void recordTopicTimestampType(String topic, Record.TimestampType timestampType) { + if (timestampType == Record.TimestampType.LogAppendTime) + topicsUsingLogAppendTime.add(topic); + else + topicsUsingLogAppendTime.remove(topic); + } + + /** + * Check if the topic is using log append time or not. + */ + public synchronized boolean isUsingLogAppendTime(String topic) { + return topicsUsingLogAppendTime.contains(topic); + } + /** * The next time to update the cluster info is the maximum of the time the current info will expire and the time the * current info can be updated (i.e. backoff time has elapsed); If an update has been request then the expiry time 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 d4668c2ddc0f1..10a166c86eff1 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 @@ -20,6 +20,7 @@ public final class ConsumerRecord { private final String topic; private final int partition; private final long offset; + private final long timestamp; private final K key; private final V value; @@ -29,15 +30,17 @@ public final class ConsumerRecord { * @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 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, 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.key = key; this.value = value; } @@ -77,9 +80,13 @@ public long offset() { return offset; } + public long timestamp() { + return timestamp; + } + @Override public String toString() { return "ConsumerRecord(topic = " + topic() + ", partition = " + partition() + ", offset = " + offset() - + ", key = " + key + ", value = " + value + ")"; + + ", timestamp = " + timestamp + ", key = " + key + ", value = " + value + ")"; } } 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 e8f1f55bb3458..d1b17dad6aae9 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 @@ -614,12 +614,13 @@ private ConsumerRecord parseRecord(TopicPartition partition, LogEntry logE if (this.checkCrcs) logEntry.record().ensureValid(); long offset = logEntry.offset(); + long timestamp = logEntry.record().timestamp(); 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, key, value); } catch (KafkaException e) { throw e; } catch (RuntimeException e) { 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 a76dc1ac87d7a..00c1865acf0eb 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 @@ -456,8 +456,9 @@ private Future doSend(ProducerRecord record, Callback call int serializedSize = Records.LOG_OVERHEAD + Record.recordSize(serializedKey, serializedValue); ensureValidRecordSize(serializedSize); TopicPartition tp = new TopicPartition(record.topic(), partition); + long timestamp = getTimestamp(record.topic(), 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(); @@ -527,6 +528,16 @@ private long waitOnMetadata(String topic, long maxWaitMs) throws InterruptedExce return time.milliseconds() - begin; } + private long getTimestamp(String topic, Long timestamp) { + // If log append times is used for the topic, we overwrite the timestamp to avoid server side re-compression. + if (metadata.isUsingLogAppendTime(topic)) + return Record.INHERITED_TIMESTAMP; + else if (timestamp == null) + return time.milliseconds(); + else + return timestamp; + } + /** * Validate that the record size isn't too large */ 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 8388ab89efbfa..5f97bae78e604 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.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 synchronized Future send(ProducerRecord record, Cal 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); 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 75cd51e6d9a8a..c9d880d00332d 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 @@ -12,6 +12,8 @@ */ package org.apache.kafka.clients.producer; +import org.apache.kafka.common.record.Record; + /** * A key/value pair to be sent to Kafka. This consists of a topic name to which the record is being sent, an optional * partition number, and an optional key and value. @@ -26,22 +28,39 @@ public final class ProducerRecord { 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 to be sent to a specified topic, partition and timestamp * * @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 < Record.NO_TIMESTAMP) + 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 +71,18 @@ public ProducerRecord(String topic, Integer partition, K key, V value) { * @param value The record contents */ public ProducerRecord(String topic, K key, V value) { - this(topic, null, key, value); + this(topic, null, null, key, value); + } + + /** + * Create a record with specific timestamp but no key + * + * @param topic The topic the record will be appended to + * @param value The record contents + * @param timestamp The timestamp of the record + */ + public ProducerRecord(String topic, Long timestamp, V value) { + this(topic, null, timestamp, null, value); } /** @@ -62,7 +92,7 @@ public ProducerRecord(String topic, K key, V value) { * @param value The record contents */ public ProducerRecord(String topic, V value) { - this(topic, null, value); + this(topic, null, null, null, value); } /** @@ -86,6 +116,13 @@ public V value() { return value; } + /** + * @return The timestamp + */ + public Long timestamp() { + return timestamp; + } + /** * The partition to which the record will be sent (or null if no partition was specified) */ @@ -97,7 +134,9 @@ public Integer partition() { 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 +156,8 @@ else if (topic != null ? !topic.equals(that.topic) : that.topic != null) 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 +168,7 @@ public int hashCode() { 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; } } 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 a80f6b959cb58..4e25c7db23795 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,20 @@ public final class RecordMetadata { private final long offset; + 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); } /** @@ -45,6 +47,13 @@ public long offset() { return this.offset; } + /** + * The timestamp of the record in the topic/partition. + */ + public long timestamp() { + return timestamp; + } + /** * The topic the record was appended to */ 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 e2d9ca87ad01b..a140371327aa8 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 { 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 @@ RecordMetadata valueOrError() throws ExecutionException { } 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; 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 3c710c8e7f3a6..7018d4484cb61 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 @@ -148,10 +148,16 @@ public double measure(MetricConfig config, long now) { * @param tp The topic/partition to which this record is being sent * @param key The key for the record * @param value The value for the record + * @param timestamp The timestamp of 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 RecordAppendResult append(TopicPartition tp, byte[] key, byte[] value, Ca 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 RecordAppendResult append(TopicPartition tp, byte[] key, byte[] value, Ca 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 RecordAppendResult append(TopicPartition tp, byte[] key, byte[] value, Ca } 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 @@ private void abortBatches() { 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); } } 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 3f185829487ec..82ef83ebf585c 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 @@ -14,6 +14,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.concurrent.atomic.AtomicLong; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.RecordMetadata; @@ -35,6 +36,7 @@ public final class RecordBatch { public int recordCount = 0; public int maxRecordSize = 0; + private final AtomicLong offsetCounter = new AtomicLong(0); public volatile int attempts = 0; public final long createdMs; public long drainedMs; @@ -62,14 +64,14 @@ public RecordBatch(TopicPartition tp, MemoryRecords records, long now) { * * @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.getAndIncrement(), 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++; @@ -83,7 +85,7 @@ public FutureRecordMetadata tryAppend(byte[] key, byte[] value, Callback callbac * @param baseOffset The base offset of the messages assigned by the server * @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 void done(long baseOffset, RuntimeException exception) { 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 create times 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 boolean maybeExpire(int requestTimeout, long now, long lingerMs) { 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; 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 aa30716a95daa..7d4589d0adbcb 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.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 @@ private void handleProduceResponse(ClientResponse response, Map entry : produceResponse.responses() - .entrySet()) { + for (Map.Entry 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); + if (error == Errors.NONE) + metadata.recordTopicTimestampType(tp.topic(), + partResp.timestamp >= 0 ? Record.TimestampType.LogAppendTime : Record.TimestampType.CreateTime); } this.sensors.recordLatency(response.request().request().destination(), response.requestLatencyMs()); this.sensors.recordThrottleTime(response.request().request().destination(), @@ -265,7 +268,7 @@ private void handleProduceResponse(ClientResponse response, Map { private final boolean shallow; private RecordsIterator innerIter; + // The variables for inner iterator + private final LinkedList logEntries; + private final long lastInnerOffset; + private final long wrapperRecordOffset; + 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.lastInnerOffset = -1L; + this.wrapperRecordOffset = -1L; + } + + private RecordsIterator(LogEntry entry, CompressionType type, boolean shallow) { + this.type = type; + this.buffer = entry.record().value(); + this.shallow = shallow; + this.stream = Compressor.wrapForInput(new ByteBufferInputStream(this.buffer), type); + this.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 LinkedList<>(); + long wrapperRecordTimestamp = entry.record().timestamp(); + while (true) { + try { + LogEntry logEntry = getNextEntryFromStream(); + Record recordWithTimestamp = new Record(logEntry.record().buffer(), wrapperRecordTimestamp); + logEntries.add(new LogEntry(logEntry.offset(), recordWithTimestamp)); + } catch (EOFException e) { + break; + } catch (IOException e) { + throw new KafkaException(e); + } + } + this.lastInnerOffset = logEntries.getLast().offset(); + } else { + this.logEntries = null; + this.lastInnerOffset = -1L; + } + } /* @@ -232,28 +271,16 @@ public RecordsIterator(ByteBuffer buffer, CompressionType type, boolean shallow) 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 (entry.record().magic() > Record.MAGIC_VALUE_V0 && lastInnerOffset >= 0) { + long absoluteOffset = entry.offset() - lastInnerOffset + wrapperRecordOffset; + 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 +291,7 @@ protected LogEntry makeNext() { // 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); + innerIter = new RecordsIterator(entry, compression, true); return innerIter.next(); } } catch (EOFException e) { @@ -278,6 +304,42 @@ protected LogEntry makeNext() { } } + private LogEntry getNextEntry() throws IOException { + if (logEntries != null) + return getNextEntryFromBuffer(); + else + return getNextEntryFromStream(); + } + + private LogEntry getNextEntryFromBuffer() { + 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(); } 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 50fac24597ca9..7323f1077ca62 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 @@ -27,6 +27,17 @@ */ public final class Record { + public enum TimestampType { + CreateTime(0, "CreateTime"), LogAppendTime(1, "LogAppendTime"); + + public int value; + public String name; + TimestampType(int value, String name) { + this.value = value; + this.name = name; + } + } + /** * The current offset and size for all the fixed-length fields */ @@ -36,9 +47,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 +64,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 = 1; /** * Specifies the mask for the compression code. 3 bits to hold the compression codec. 0 is reserved to indicate no @@ -67,10 +88,28 @@ public final class Record { */ public static final int NO_COMPRESSION = 0; + /** + * Timestamp value for compressed records whose timestamp inherit from wrapper record. + */ + public static final long INHERITED_TIMESTAMP = -1; + + /** + * Timestamp value for records without a timestamp + */ + public static final long NO_TIMESTAMP = -2; + private final ByteBuffer buffer; + private final long wrapperRecordTimestamp; public Record(ByteBuffer buffer) { this.buffer = buffer; + this.wrapperRecordTimestamp = NO_TIMESTAMP; + } + + // Package private constructor for inner iteration. + Record(ByteBuffer buffer, long wrapperRecordTimestamp) { + this.buffer = buffer; + this.wrapperRecordTimestamp = wrapperRecordTimestamp; } /** @@ -84,45 +123,47 @@ public Record(ByteBuffer buffer) { * @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 +186,7 @@ public static int recordSize(byte[] key, byte[] value) { } 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 +212,14 @@ public static long computeChecksum(ByteBuffer buffer, int position, int size) { /** * 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 +282,10 @@ public int size() { * 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 +299,10 @@ public boolean hasKey() { * 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()); } /** @@ -278,6 +326,21 @@ public byte attributes() { return buffer.get(ATTRIBUTES_OFFSET); } + /** + * The timestamp of this record + */ + public long timestamp() { + if (magic() == MAGIC_VALUE_V0) + return NO_TIMESTAMP; + else { + long timestamp = buffer.getLong(TIMESTAMP_OFFSET); + if (timestamp == INHERITED_TIMESTAMP) + return wrapperRecordTimestamp; + else + return timestamp; + } + } + /** * The compression type used with this record */ @@ -296,7 +359,10 @@ public ByteBuffer value() { * 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,11 +383,12 @@ private ByteBuffer sliceDelimited(int start) { } public String toString() { - return String.format("Record(magic = %d, attributes = %d, compression = %s, crc = %d, key = %d bytes, value = %d bytes)", + return String.format("Record(magic = %d, attributes = %d, compression = %s, crc = %d, timestamp = %d, key = %d bytes, value = %d bytes)", magic(), attributes(), compressionType(), checksum(), + timestamp(), key() == null ? 0 : key().limit(), value() == null ? 0 : value().limit()); } 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 a91524733ae77..a41639de123f4 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.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,7 +99,7 @@ public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { Map responseMap = new HashMap(); for (Map.Entry 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) { 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 c213332079df7..cf6739ca75037 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 responses; private final int throttleTime; @@ -91,8 +92,9 @@ public ProduceResponse(Struct struct) { 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 +109,12 @@ private void initCommonFields(Map responses) List partitionArray = new ArrayList(); for (Map.Entry 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 +134,12 @@ public int getThrottleTime() { 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 +150,8 @@ public String toString() { b.append(errorCode); b.append(",offset: "); b.append(baseOffset); + b.append(",timestamp: "); + b.append(timestamp); b.append('}'); return b.toString(); } 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 5b867001065df..caa005879c656 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 @@ final public void updateInt(int input) { 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. */ 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 fa06be9e2ba2a..1182782f9e381 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 @@ -42,8 +42,8 @@ public void testSimpleMock() { beginningOffsets.put(new TopicPartition("test", 1), 0L); consumer.updateBeginningOffsets(beginningOffsets); consumer.seek(new TopicPartition("test", 0), 0); - ConsumerRecord rec1 = new ConsumerRecord("test", 0, 0, "key1", "value1"); - ConsumerRecord rec2 = new ConsumerRecord("test", 0, 1, "key2", "value2"); + ConsumerRecord rec1 = new ConsumerRecord("test", 0, 0, 0L, "key1", "value1"); + ConsumerRecord rec2 = new ConsumerRecord("test", 0, 1, 0L, "key2", "value2"); consumer.addRecord(rec1); consumer.addRecord(rec2); ConsumerRecords recs = consumer.poll(1); 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 5e750fdfe0140..e8b39efb24cc0 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 @@ -95,9 +95,9 @@ public void setup() throws Exception { 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 +133,9 @@ public void testFetchNonContinuousRecords() { // 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> consumerRecords; @@ -164,7 +164,7 @@ public void testFetchRecordTooLarge() { 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 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 7bb181e46c474..f3db098d43c02 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 void testEqualsAndHashCode() { ProducerRecord valueMisMatch = new ProducerRecord("test", 1 , "key", 2); assertFalse(producerRecord.equals(valueMisMatch)); - ProducerRecord nullFieldsRecord = new ProducerRecord("topic", null, null, null); + ProducerRecord nullFieldsRecord = new ProducerRecord("topic", null, null, null, null); assertEquals(nullFieldsRecord, nullFieldsRecord); assertEquals(nullFieldsRecord.hashCode(), nullFieldsRecord.hashCode()); } 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 1e5d1c2d6159b..55911293ff764 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.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 void testTimeout() throws Exception { */ @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 void testError() throws Exception { */ @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()); } 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 723e450660729..0f95ee5d494db 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 void testFull() throws Exception { 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 batches = accum.drain(cluster, Collections.singleton(node1), Integer.MAX_VALUE, 0).get(node1.id()); assertEquals(1, batches.size()); @@ -99,7 +99,7 @@ public void testFull() throws Exception { 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 void testAppendLarge() throws Exception { 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 void testPartialDrain() throws Exception { List 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 void testStressfulSituation() throws Exception { 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 void testNextReadyCheckDelay() throws Exception { // 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 void testNextReadyCheckDelay() throws Exception { // 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 void testRetryBackoff() throws Exception { 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> batches = accum.drain(cluster, result.readyNodes, Integer.MAX_VALUE, now + lingerMs + 1); @@ -232,7 +232,7 @@ public void testRetryBackoff() throws Exception { 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 void testFlush() throws Exception { 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 void onCompletion(RecordMetadata metadata, Exception exception) { } } 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 void testExpiredBatches() throws InterruptedException { 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 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(), new ArrayList(), Collections.emptySet()); 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 14a839b8b65fc..b983de51f82fc 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.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 void tearDown() { @Test public void testSimple() throws Exception { long offset = 0; - Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), null, MAX_BLOCK_TIMEOUT).future; + Future 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 void testSimple() throws Exception { public void testQuotaMetrics() throws Exception { final long offset = 0; for (int i = 1; i <= 3; i++) { - Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), null, MAX_BLOCK_TIMEOUT).future; + Future 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 void testRetries() throws Exception { "clientId", REQUEST_TIMEOUT); // do a successful retry - Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), null, MAX_BLOCK_TIMEOUT).future; + Future 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 void testRetries() throws Exception { 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 @@ private void completedWithError(Future future, Errors error) thr } 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 partResp = Collections.singletonMap(tp, resp); ProduceResponse response = new ProduceResponse(partResp, throttleTimeMs); return response.toStruct(); 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 6e3a9ac5ad154..ed64f63f10d91 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 MemoryRecordsTest(CompressionType compression) { public void testIterator() { MemoryRecords recs1 = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), compression); MemoryRecords recs2 = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), compression); - List list = Arrays.asList(new Record("a".getBytes(), "1".getBytes()), - new Record("b".getBytes(), "2".getBytes()), - new Record("c".getBytes(), "3".getBytes())); + List 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(); 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 957fc8fa3999c..c5843c4b0a2e4 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 @@ @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 void testFields() { 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 static Collection data() { byte[] payload = new byte[1000]; Arrays.fill(payload, (byte) 1); List values = new ArrayList(); - 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, Record.INHERITED_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; } 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 db9c81a012ae5..a483909b1bc7b 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 @@ -22,6 +22,7 @@ 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; @@ -121,16 +122,16 @@ private void checkSerialization(AbstractRequestResponse req, Integer version) th @Test public void produceResponseVersionTest() { Map responseData = new HashMap(); - 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); + // No need to verify V1 here because + ProduceResponse v2Response = new ProduceResponse(responseData, 10); 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 1", 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 @@ -315,7 +316,7 @@ private AbstractRequest createProduceRequest() { private AbstractRequestResponse createProduceResponse() { Map responseData = new HashMap(); - 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); } 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 04b08b379d931..1c7c723b9a94f 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 @@ -290,7 +290,7 @@ private void expectConsumerPoll(final int numMessages) { public ConsumerRecords answer() throws Throwable { List> 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, RAW_KEY, RAW_VALUE)); recordsReturned += numMessages; return new ConsumerRecords<>( numMessages > 0 ? 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 3bf653e983136..84c9f89729d3f 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 @@ -519,7 +519,7 @@ public ConsumerRecords answer() throws Throwable { 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, RAW_KEY, RAW_VALUE) ))); recordsReturned++; return records; @@ -547,7 +547,7 @@ public ConsumerRecords answer() throws Throwable { 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, RAW_KEY, RAW_VALUE) ))); recordsReturned++; return records; 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 1f557e45e4075..3b0464ff57cad 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 @@ -354,7 +354,7 @@ private Capture> expectSendRecord(boolean anyTime public Future 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(); } 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 e007f020c91a2..d82ceed7ad15b 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 @@ -288,14 +288,14 @@ public void testRestore() throws Exception { expectConfigure(); // Overwrite each type at least once to ensure we see the latest data after loading List> 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, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)), + new ConsumerRecord<>(TOPIC, 0, 1, 0L, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(1)), + new ConsumerRecord<>(TOPIC, 0, 2, 0L, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)), + new ConsumerRecord<>(TOPIC, 0, 3, 0L, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(3)), + new ConsumerRecord<>(TOPIC, 0, 4, 0L, 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, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5)), + new ConsumerRecord<>(TOPIC, 0, 6, 0L, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(6))); LinkedHashMap 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 +342,12 @@ public void testPutTaskConfigsDoesNotResolveAllInconsistencies() throws Exceptio expectConfigure(); List> existingRecords = Arrays.asList( - new ConsumerRecord<>(TOPIC, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)), + new ConsumerRecord<>(TOPIC, 0, 0, 0L, 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, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)), + new ConsumerRecord<>(TOPIC, 0, 4, 0L, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(4)), + new ConsumerRecord<>(TOPIC, 0, 5, 0L, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5))); LinkedHashMap 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 +483,7 @@ private void expectReadToEnd(final LinkedHashMap serializedConfi public Future answer() throws Throwable { TestFuture future = new TestFuture(); for (Map.Entry 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, entry.getKey(), entry.getValue())); future.resolveOnGet((Void) null); return future; } 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 4e54bf18fc741..a3be7fcbe53a1 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 @@ -125,10 +125,10 @@ public void testStartStop() throws Exception { 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, TP0_KEY.array(), TP0_VALUE.array()), + new ConsumerRecord<>(TOPIC, 1, 0, 0L, TP1_KEY.array(), TP1_VALUE.array()), + new ConsumerRecord<>(TOPIC, 0, 1, 0L, TP0_KEY.array(), TP0_VALUE_NEW.array()), + new ConsumerRecord<>(TOPIC, 1, 1, 0L, TP1_KEY.array(), TP1_VALUE_NEW.array()) )); expectStop(); @@ -176,8 +176,8 @@ public Object answer() throws Throwable { PowerMock.expectLastCall().andAnswer(new IAnswer() { @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, TP0_KEY.array(), TP0_VALUE.array())); + capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 0, 0L, TP1_KEY.array(), TP1_VALUE.array())); secondGetReadToEndCallback.getValue().onCompletion(null, null); return null; } @@ -189,8 +189,8 @@ public Object answer() throws Throwable { PowerMock.expectLastCall().andAnswer(new IAnswer() { @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, TP0_KEY.array(), TP0_VALUE_NEW.array())); + capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 1, 0L, TP1_KEY.array(), TP1_VALUE_NEW.array())); thirdGetReadToEndCallback.getValue().onCompletion(null, null); return null; } 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 ab370e31ef29b..5efe2db315e08 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 @@ -182,7 +182,7 @@ public void run() { 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, TP0_KEY, TP0_VALUE)); } }); consumer.scheduleNopPollTask(); @@ -190,7 +190,7 @@ public void run() { 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, TP1_KEY, TP1_VALUE)); } }); consumer.schedulePollTask(new Runnable() { @@ -297,16 +297,16 @@ public void run() { 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, TP0_KEY, TP0_VALUE)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 1, 0L, TP0_KEY, TP0_VALUE_NEW)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, 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, TP1_KEY, TP1_VALUE_NEW)); } }); @@ -362,8 +362,8 @@ public void run() { 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, TP0_KEY, TP0_VALUE_NEW)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TP0_KEY, TP0_VALUE_NEW)); } }); diff --git a/core/src/main/scala/kafka/api/ProducerRequest.scala b/core/src/main/scala/kafka/api/ProducerRequest.scala index 00d09194ff799..30af8410b6afc 100644 --- a/core/src/main/scala/kafka/api/ProducerRequest.scala +++ b/core/src/main/scala/kafka/api/ProducerRequest.scala @@ -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))) diff --git a/core/src/main/scala/kafka/api/ProducerResponse.scala b/core/src/main/scala/kafka/api/ProducerResponse.scala index 7e745cf2abbe5..3e0203cc510cb 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 @@ -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 diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 709f6d04eaad3..277dfe81838df 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -35,20 +35,28 @@ 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 timestamp The log append time (if used) of the message set, otherwise Message.NoTimestamp * @param shallowCount The number of shallow messages * @param validBytes The number of valid bytes * @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 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) /** @@ -326,8 +334,10 @@ class Log(val dir: File, if (assignOffsets) { // assign offsets to the message set val offset = new AtomicLong(nextOffsetMetadata.messageOffset) + val now = System.currentTimeMillis() try { validMessages = validMessages.validateMessagesAndAssignOffsets(offset, + now, appendInfo.sourceCodec, appendInfo.targetCodec, config.compact, @@ -338,6 +348,9 @@ class Log(val dir: File, 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.LogAppendTime) + appendInfo.timestamp = now } else { // we are taking the offsets we are given if (!appendInfo.offsetsMonotonic || appendInfo.firstOffset < nextOffsetMetadata.messageOffset) @@ -442,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) } /** diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 3a16252587c7c..7fbffacc87997 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -92,7 +92,6 @@ object ByteBufferMessageSet { throw new RuntimeException("wrapper message = " + wrapperMessage) val inputStream: InputStream = new ByteBufferBackedInputStream(wrapperMessage.payload) val compressed: DataInputStream = new DataInputStream(CompressionFactory(wrapperMessage.compressionCodec, inputStream)) - val messageAndOffsets = new mutable.Queue[MessageAndOffset] var lastInnerOffset = -1L // When magic value is greater than 0, relative offset will be used. // Ideally the conversion from relative offset(RO) to absolute offset(AO) should be: @@ -117,58 +116,72 @@ object ByteBufferMessageSet { // AO = AO_Of_Last_Inner_Message + RO // // 4. This solution works for compacted message set as well - try { - while (true) { - // read the offset - val innerOffset = compressed.readLong() - // read record size - val size = compressed.readInt() - - if (size < 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) - // Override the timestamp if necessary - val newMessage = new Message(buffer) - // 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}") - if (newMessage.magic > MagicValue_V0 && newMessage.timestamp == InheritedTimestamp) { - // We need to use the wrapper message timestamp when inner message timestamp is Message.InheritedTimestamp - buffer.position(TimestampOffset) - buffer.putLong(wrapperMessageTimestamp) - buffer.rewind() + val messageAndOffsets = if (wrapperMessageAndOffset.message.magic > MagicValue_V0) { + var innerMessageAndOffsets = new mutable.Queue[MessageAndOffset]() + try { + while (true) { + innerMessageAndOffsets += readMessageFromStream() } - messageAndOffsets += new MessageAndOffset(newMessage, innerOffset) - lastInnerOffset = innerOffset + } catch { + case eofe: EOFException => + compressed.close() + case ioe: IOException => + throw new KafkaException(ioe) } - } catch { - case eofe: EOFException => - compressed.close() - 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 < 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) + // Override the timestamp if necessary + val newMessage = new Message(buffer = buffer, wrapperMessageTimestamp = wrapperMessageTimestamp) + // 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 = { - if (messageAndOffsets.size == 0) - allDone() - else { - val messageAndOffset = messageAndOffsets.dequeue() - val message = messageAndOffset.message - // If magic value is greater than 0, relative offset and timestamp will be used. - if (message.magic > MagicValue_V0) { - val relativeOffset = messageAndOffset.offset - lastInnerOffset - val absoluteOffset = wrapperMessageOffset + relativeOffset - new MessageAndOffset(message, absoluteOffset) - } else - messageAndOffset + messageAndOffsets match { + // Using relative offset and timestamps + case Some(innerMessageAndOffsets) => + if (innerMessageAndOffsets.isEmpty) + allDone() + else { + val messageAndOffset = messageAndOffsets.get.dequeue() + val message = messageAndOffset.message + val relativeOffset = messageAndOffset.offset - lastInnerOffset + val absoluteOffset = wrapperMessageOffset + relativeOffset + new MessageAndOffset(message, absoluteOffset) + } + // Not using relative offset and timestamps + case None => + try { + readMessageFromStream() + } catch { + case eofe: EOFException => + compressed.close() + allDone() + case ioe: IOException => + throw new KafkaException(ioe) + } } - } } } @@ -352,6 +365,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi * operations and avoids re-compression. */ private[kafka] def validateMessagesAndAssignOffsets(offsetCounter: AtomicLong, + now: Long = System.currentTimeMillis(), sourceCodec: CompressionCodec, targetCodec: CompressionCodec, compactedTopic: Boolean = false, @@ -366,7 +380,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi convertNonCompressedMessages(offsetCounter, compactedTopic, magicValueToUse) } else { // Do in-place validation, offset assignment and maybe set timestamp - validateNonCompressedMessagesAndAssignOffsetInPlace(offsetCounter, compactedTopic, messageTimestampType, + validateNonCompressedMessagesAndAssignOffsetInPlace(offsetCounter, now, compactedTopic, messageTimestampType, messageTimestampDiffMaxMs) } @@ -390,7 +404,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi if (message.magic > Message.MagicValue_V0 && magicValueToUse > Message.MagicValue_V0) { // Re-compression situation 3 // Check if we need to overwrite timestamp - if (validateTimestamps(message, messageTimestampType, messageTimestampDiffMaxMs, targetCodec)) + if (validateTimestamps(message, now, messageTimestampType, messageTimestampDiffMaxMs, targetCodec)) requireReCompression = true // Check if we need to overwrite offset if (messageAndOffset.offset != expectedRelativeOffset.getAndIncrement) @@ -417,7 +431,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi else if (magicValueToUse > Message.MagicValue_V0 && messageTimestampType == TimestampType.CreateTime) maxTimestamp else // Log append time - System.currentTimeMillis() + now } new ByteBufferMessageSet(compressionCodec = targetCodec, @@ -429,15 +443,22 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi buffer.putLong(0, offsetCounter.addAndGet(validatedMessages.size) - 1) if (magicValueToUse > Message.MagicValue_V0) { - if (messageTimestampType == TimestampType.CreateTime) - buffer.putLong(MessageSet.LogOverhead + Message.TimestampOffset, maxTimestamp) - else if (messageTimestampType == TimestampType.LogAppendTime) - buffer.putLong(MessageSet.LogOverhead + Message.TimestampOffset, System.currentTimeMillis()) - - // 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()) + var crcUpdateNeeded = true + if (messageTimestampType == TimestampType.CreateTime) { + if (buffer.getLong(MessageSet.LogOverhead + Message.TimestampOffset) == maxTimestamp) + // We don't need to recompute crc if the timestamp is not updated. + crcUpdateNeeded = false + else + buffer.putLong(MessageSet.LogOverhead + Message.TimestampOffset, maxTimestamp) + } else if (messageTimestampType == TimestampType.LogAppendTime) + buffer.putLong(MessageSet.LogOverhead + Message.TimestampOffset, now) + + 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 @@ -449,9 +470,8 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi compactedTopic: Boolean, toMagicValue: Byte): ByteBufferMessageSet = { // Get message count, shallow iterator is in-place - var sizeInBytesAfterConversion = shallowValidBytes - this.internalIterator(isShallow = true).foreach(messageAndOffset => - sizeInBytesAfterConversion += Message.headerSizeDiff(messageAndOffset.message.magic, toMagicValue)) + 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 => @@ -479,6 +499,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi } private def validateNonCompressedMessagesAndAssignOffsetInPlace(offsetCounter: AtomicLong, + now: Long, compactedTopic: Boolean, timestampType: TimestampType, timestampDiffMaxMs: Long): ByteBufferMessageSet = { @@ -493,11 +514,12 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi messageBuffer.limit(messageSize) val message = new Message(messageBuffer) validateMessageKey(message, compactedTopic) - if (message.magic > Message.MagicValue_V0) - validateTimestamps(message, timestampType, timestampDiffMaxMs, NoCompressionCodec) - // update CRC for the message if the timestamp is LogAppendTime - if (timestampType == TimestampType.LogAppendTime) - Utils.writeUnsignedInt(messageBuffer, Message.CrcOffset, message.computeChecksum()) + if (message.magic > Message.MagicValue_V0) { + validateTimestamps(message, now, timestampType, timestampDiffMaxMs, NoCompressionCodec) + // update CRC for the message if the timestamp is LogAppendTime + if (timestampType == TimestampType.LogAppendTime) + Utils.writeUnsignedInt(messageBuffer, Message.CrcOffset, message.computeChecksum()) + } messagePosition += MessageSet.LogOverhead + messageSize } buffer.reset() @@ -510,23 +532,28 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi } private def validateTimestamps(message: Message, + now: Long, timestampType: TimestampType, timestampDiffMaxMs: Long, compressionCodec: CompressionCodec): Boolean = { - val now = System.currentTimeMillis() var overwritten = false if (timestampType == TimestampType.CreateTime && math.abs(message.timestamp - now) > timestampDiffMaxMs) throw new InvalidMessageException(s"Timestamp ${message.timestamp} of message is out of range. " + s"The timestamp should be within [${now - timestampDiffMaxMs}, ${now + timestampDiffMaxMs}") else if (timestampType == TimestampType.LogAppendTime) { - if (compressionCodec == NoCompressionCodec) + if (compressionCodec == NoCompressionCodec) { message.buffer.putLong(Message.TimestampOffset, now) - else { + // We have to update crc after updating the timestamp. + Utils.writeUnsignedInt(message.buffer, Message.CrcOffset, message.computeChecksum()) + } else { if (message.timestamp != Message.InheritedTimestamp) { message.buffer.putLong(Message.TimestampOffset, Message.InheritedTimestamp) + // We have to update crc after updating the timestamp. + Utils.writeUnsignedInt(message.buffer, Message.CrcOffset, message.computeChecksum()) overwritten = true } } + } overwritten } diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala index 8686876d5041a..3754e37f3ef0e 100755 --- a/core/src/main/scala/kafka/message/Message.scala +++ b/core/src/main/scala/kafka/message/Message.scala @@ -127,7 +127,7 @@ object Message { * * Default constructor wraps an existing ByteBuffer with the Message object with no change to the contents. */ -class Message(val buffer: ByteBuffer) { +class Message(val buffer: ByteBuffer, val wrapperMessageTimestamp: Long = Message.NoTimestamp) { import kafka.message.Message._ @@ -282,7 +282,12 @@ class Message(val buffer: ByteBuffer) { def timestamp: Long = { if (magic == MagicValue_V0) throw new IllegalStateException("The message with magic byte 0 does not have a timestamp") - buffer.getLong(TimestampOffset) + + val timestamp = buffer.getLong(TimestampOffset) + if (timestamp == Message.InheritedTimestamp) + wrapperMessageTimestamp + else + timestamp } /** diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 2b7200cf2bc2b..4efef81915aa2 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -36,6 +36,7 @@ import org.apache.kafka.common.errors.{InvalidTopicException, NotLeaderForPartit ClusterAuthorizationException} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.{ProtoUtils, ApiKeys, Errors, SecurityProtocol} +import org.apache.kafka.common.protocol.types.Schema 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 +331,12 @@ class KafkaApis(val requestChannel: RequestChannel, // the callback for sending a produce response def sendResponseCallback(responseStatus: Map[TopicPartition, PartitionResponse]) { +<<<<<<< HEAD val mergedResponseStatus = responseStatus ++ unauthorizedRequestInfo.mapValues(_ => new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED.code, -1)) +======= + val mergedResponseStatus = responseStatus ++ unauthorizedRequestInfo.mapValues(_ => + ProducerResponseStatus(Errors.TOPIC_AUTHORIZATION_FAILED.code, -1, Message.NoTimestamp)) +>>>>>>> Added timestamp and relative offset to clients package. var errorInResponse = false @@ -431,10 +437,7 @@ class KafkaApis(val requestChannel: RequestChannel, // Only send messages with magic value 1 when client supports it and server has this format on disk. val magicValueToUse = if (fetchRequest.versionId > 1 && config.messageFormatVersion.onOrAfter(KAFKA_0_10_0_DV0)) - // TODO: Change magic value to 1 after o.a.k.client side change is done. - // We cannot send message format 1 back yet, because replica fetcher thread uses o.a.k.clients schema - // that does not have magic value 1 yet. - Message.MagicValue_V0 + Message.MagicValue_V1 else Message.MagicValue_V0 diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 788875cc9a14b..986eb890602db 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -26,7 +26,7 @@ import kafka.cluster.{Partition, Replica} import kafka.common._ import kafka.controller.KafkaController import kafka.log.{LogAppendInfo, LogManager} -import kafka.message.{InvalidMessageException, ByteBufferMessageSet, MessageSet} +import kafka.message.{Message, InvalidMessageException, ByteBufferMessageSet, MessageSet} import kafka.metrics.KafkaMetricsGroup import kafka.utils._ import org.apache.kafka.common.errors.{OffsetOutOfRangeException, RecordBatchTooLargeException, ReplicaNotAvailableException, RecordTooLargeException, @@ -332,7 +332,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 +358,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) } diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index bc3a6cef623c5..fa4437addb23a 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -16,6 +16,7 @@ import java.util import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.producer.{Producer, ProducerConfig, ProducerRecord} +import org.apache.kafka.common.record.Record.TimestampType import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.apache.kafka.common.{PartitionInfo, TopicPartition} @@ -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,6 +277,7 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { } protected def sendRecords(numRecords: Int, tp: TopicPartition) { +<<<<<<< HEAD sendRecords(this.producers(0), numRecords, tp) } @@ -296,6 +298,12 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { val offset = startingOffset + i assertEquals(tp.topic(), record.topic()) assertEquals(tp.partition(), record.partition()) + if (timestampType == TimestampType.CreateTime) { + 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())) diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala index c970fb90933e1..ebe4693306bd3 100644 --- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala @@ -22,11 +22,13 @@ import java.util.concurrent.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.record.Record.TimestampType import org.junit.Assert._ import org.junit.{After, Before, Test} import scala.collection.mutable.Buffer @@ -141,6 +143,80 @@ 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.CreateTime) + } + + @Test + def testSendNonCompressedMessageWithCreateTime() { + val producer = createProducer(brokerList = brokerList, lingerMs = Long.MaxValue) + sendAndVerifyTimestamp(producer, TimestampType.CreateTime) + } + + @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.LogAppendTime) + } + + @Test + def testSendNonCompressedMessageWithLogApendTime() { + val producer = createProducer(brokerList = brokerList, lingerMs = Long.MaxValue) + sendAndVerifyTimestamp(producer, TimestampType.LogAppendTime) + } + + 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.CreateTime) + 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.LogAppendTime) + 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.flush() + assertEquals("Should have offset " + numRecords, numRecords, callback.offset) + } finally { + producer.close() + } + } + @Test def testWrongSerializer() { // send a record with a wrong type should receive a serialization exception @@ -211,9 +287,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) @@ -237,7 +314,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, Message.NoTimestamp, Message.MagicValue_V0), 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 { diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 7af1781e3a9c1..7e152b644696a 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -18,13 +18,17 @@ 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.common.TopicPartition +import org.apache.kafka.clients.producer.KafkaProducer +import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer, ByteArraySerializer} +import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.common.record.CompressionType -import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer, ByteArrayDeserializer, ByteArraySerializer} +import org.apache.kafka.common.record.Record.TimestampType +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.apache.kafka.common.errors.{InvalidTopicException, RecordTooLargeException} import org.apache.kafka.test.{MockProducerInterceptor, MockConsumerInterceptor} import org.junit.Assert._ @@ -36,12 +40,6 @@ 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 { - // TODO: Remove this after o.a.k.clients has Record version 1 (relative offset + timestamp) - // We need this config because testRecordTooLarge does not work when message format conversion occurs on server - // side. In that case we lose the residue part of the file message set from zero-copy transfer, which is exactly - // what we are depending on to determine if message size is too large. - this.serverConfig.setProperty(KafkaConfig.MessageFormatVersionProp, "0.9.0") - @Test def testAutoCommitOnClose() { this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true") @@ -102,14 +100,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 @@ -269,7 +267,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") } @@ -294,6 +292,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { consumer.seek(tp, mid) assertEquals(mid, consumer.position(tp)) + consumeAndVerifyRecords(consumer, numRecords = 1, startingOffset = mid.toInt, startingKeyAndValueIndex = mid.toInt) // Test seek compressed message @@ -319,18 +318,16 @@ class PlaintextConsumerTest extends BaseConsumerTest { producerProps.setProperty(ProducerConfig.COMPRESSION_TYPE_CONFIG, CompressionType.GZIP.name) 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)) + retries = 0, lingerMs = Long.MaxValue, props = Some(producerProps)) sendRecords(producer, numRecords, tp) 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] { @@ -343,7 +340,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) @@ -352,19 +349,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 @@ -403,7 +400,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 @@ -540,7 +537,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 @@ -573,16 +570,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() @@ -599,7 +596,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) @@ -623,9 +620,9 @@ 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)), - rebalanceListener) + 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 assertEquals(10, testConsumer.committed(tp).offset) @@ -650,14 +647,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) @@ -670,6 +667,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 + sendCompressedRecords(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.LogAppendTime) + + // Test compressed messages + val tp2 = new TopicPartition(topicName, 1) + sendCompressedRecords(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.LogAppendTime) + } + def runMultiConsumerSessionTimeoutTest(closeConsumer: Boolean): Unit = { // use consumers defined in this class plus one additional consumer // Use topic defined in this class + one additional topic @@ -699,7 +736,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) @@ -816,7 +853,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)") } /** @@ -850,7 +887,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], diff --git a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala index 9bbe005d3466a..c4a2bd788956d 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, Message.NoTimestamp, Message.MagicValue_V0), messageSet(index).message) + assertEquals(new Message(bytes = message, now, Message.MagicValue_V1), messageSet(index).message) assertEquals(index.toLong, messageSet(index).offset) index += 1 } diff --git a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala index 267e383b20f10..990da7d4d38db 100644 --- a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala @@ -185,11 +185,14 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { val now = System.currentTimeMillis() assertEquals("message set size should not change", messages.size, validatedMessages.size) - for (messageAndOffset <- validatedMessages) + for (messageAndOffset <- validatedMessages) { + messageAndOffset.message.ensureValid() assertTrue(messageAndOffset.message.timestamp >= startTime && messageAndOffset.message.timestamp <= now) + } assertEquals("message set size should not change", compressedMessages.size, validatedCompressedMessages.size) for (messageAndOffset <- validatedCompressedMessages) { + messageAndOffset.message.ensureValid() assertTrue(s"Timestamp of message ${messageAndOffset.message}} should be between $startTime and $now", messageAndOffset.message.timestamp >= startTime && messageAndOffset.message.timestamp <= now) } @@ -198,6 +201,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { assertEquals("message set size should not change", compressedMessagesWithoutRecompression.size, validatedCompressedMessagesWithoutRecompression.size) for (messageAndOffset <- validatedCompressedMessagesWithoutRecompression) { + messageAndOffset.message.ensureValid() assertTrue(s"Timestamp of message ${messageAndOffset.message}} should be between $startTime and $now", messageAndOffset.message.timestamp >= startTime && messageAndOffset.message.timestamp <= now) } diff --git a/core/src/test/scala/unit/kafka/message/MessageTest.scala b/core/src/test/scala/unit/kafka/message/MessageTest.scala index 8bff121d61e8d..16b6f5471c08a 100755 --- a/core/src/test/scala/unit/kafka/message/MessageTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageTest.scala @@ -176,7 +176,7 @@ class MessageTest extends JUnitSuite { def testInValidMagicByte() { new Message("hello".getBytes, 0L, 2) } - + @Test def testIsHashable() { // this is silly, but why not diff --git a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala index ab735beea2e17..ce3b6224ce703 100755 --- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala @@ -181,8 +181,8 @@ 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, timestamp = Message.NoTimestamp, magicValue = Message.MagicValue_V0), messageSet(0).message) - assertEquals(new Message(bytes = "test2".getBytes, key = "test".getBytes, timestamp = Message.NoTimestamp, magicValue = Message.MagicValue_V0), messageSet(1).message) + assertEquals(new Message(bytes = "test1".getBytes, key = "test".getBytes, timestamp = Message.NoTimestamp, magicValue = Message.MagicValue_V1), messageSet(0).message) + assertEquals(new Message(bytes = "test2".getBytes, key = "test".getBytes, timestamp = Message.NoTimestamp, magicValue = Message.MagicValue_V1), messageSet(1).message) producer1.close() val props2 = new util.Properties() @@ -260,7 +260,7 @@ 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, timestamp = Message.NoTimestamp, magicValue = Message.MagicValue_V0), messageSet1.next.message) + assertEquals(new Message(bytes = "test1".getBytes, key = "test".getBytes, timestamp = Message.NoTimestamp, magicValue = Message.MagicValue_V1), messageSet1.next.message) assertFalse("Message set should have another message", messageSet1.hasNext) } catch { case e: Exception => fail("Not expected", e) diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala index 351c2db8f2cc7..c1034febde779 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.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) } } 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 66f78d2150994..a59a06969d12c 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 @@ -76,7 +76,7 @@ public int addRawRecords(Iterable> rawRecords, Ti Object key = source.deserializeKey(rawRecord.topic(), rawRecord.key()); Object value = source.deserializeValue(rawRecord.topic(), rawRecord.value()); - ConsumerRecord record = new ConsumerRecord<>(rawRecord.topic(), rawRecord.partition(), rawRecord.offset(), key, value); + ConsumerRecord record = new ConsumerRecord<>(rawRecord.topic(), rawRecord.partition(), rawRecord.offset(), rawRecord.timestamp(), key, value); long timestamp = timestampExtractor.extract(record); StampedRecord stampedRecord = new StampedRecord(record, timestamp); 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 b91acdce13bfc..fac9f2cbb66c1 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 @@ -59,17 +59,17 @@ public void testTimeTracking() { // add three 3 records with timestamp 1, 3, 5 to partition-1 List> 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, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 3, 0L, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 5, 0L, recordKey, recordValue)); group.addRawRecords(partition1, list1); // add three 3 records with timestamp 2, 4, 6 to partition-2 List> 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, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 4, 0L, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 6, 0L, recordKey, recordValue)); group.addRawRecords(partition2, list2); 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 bc6f71b15c23b..89b66a94224ed 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 @@ -83,7 +83,7 @@ public void reset() { // buffer a record (we cannot use addRecord because we need to add records before assigning a partition) public void bufferRecord(ConsumerRecord record) { recordBuffer.add( - new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), + new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), 0L, serializer.serialize(record.topic(), record.key()), serializer.serialize(record.topic(), record.value()))); endOffset = record.offset(); @@ -267,7 +267,7 @@ public void testRegisterPersistentStore() throws IOException { int key = i * 10; expectedKeys.add(key); restoreConsumer.bufferRecord( - new ConsumerRecord<>(persistentStoreTopicName, 2, offset, key, 0) + new ConsumerRecord<>(persistentStoreTopicName, 2, 0L, offset, key, 0) ); } @@ -320,7 +320,7 @@ public void testRegisterNonPersistentStore() throws IOException { int key = i; expectedKeys.add(i); restoreConsumer.bufferRecord( - new ConsumerRecord<>(nonPersistentStoreTopicName, 2, offset, key, 0) + new ConsumerRecord<>(nonPersistentStoreTopicName, 2, 0L, offset, key, 0) ); } 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 6e86410b19ed6..284ba5d754191 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 @@ -50,9 +50,9 @@ public void testTimeTracking() { // add three 3 out-of-order records with timestamp 2, 1, 3 List> 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, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 1, 0L, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 3, 0L, recordKey, recordValue)); queue.addRawRecords(list1, timestampExtractor); @@ -72,9 +72,9 @@ public void testTimeTracking() { // add three 3 out-of-order records with timestamp 4, 1, 2 // now with 3, 4, 1, 2 List> 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, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 1, 0L, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 2, 0L, recordKey, recordValue)); queue.addRawRecords(list2, timestampExtractor); @@ -99,9 +99,9 @@ public void testTimeTracking() { // add three more records with 4, 5, 6 List> 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, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 5, 0L, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 6, 0L, recordKey, recordValue)); queue.addRawRecords(list3, timestampExtractor); 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 fd6f49f6736a9..dbe7fd3dc3396 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 @@ -152,7 +152,7 @@ public void testUpdateNonPersistentStore() throws Exception { 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, recordKey, recordValue)) ); } finally { @@ -171,9 +171,9 @@ public void testUpdate() throws Exception { restoreStateConsumer.assign(new ArrayList<>(task.changeLogPartitions())); for (ConsumerRecord 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, 1, 100), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 20, 0L, 2, 100), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 30, 0L, 3, 100))) { restoreStateConsumer.bufferRecord(record); } @@ -234,11 +234,11 @@ public void testUpdateKTable() throws Exception { restoreStateConsumer.assign(new ArrayList<>(task.changeLogPartitions())); for (ConsumerRecord 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, 1, 100), + new ConsumerRecord<>(ktable.topic(), ktable.partition(), 20, 0L, 2, 100), + new ConsumerRecord<>(ktable.topic(), ktable.partition(), 30, 0L, 3, 100), + new ConsumerRecord<>(ktable.topic(), ktable.partition(), 40, 0L, 4, 100), + new ConsumerRecord<>(ktable.topic(), ktable.partition(), 50, 0L, 5, 100))) { restoreStateConsumer.bufferRecord(record); } 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 bf3b3b181fa65..b36304c052787 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 @@ -107,15 +107,15 @@ public void testProcessOrder() throws Exception { 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, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, 0L, 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, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, recordKey, recordValue) )); assertEquals(5, task.process()); @@ -158,15 +158,15 @@ public void testPauseResume() throws Exception { 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, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, 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, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 55, 0L, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 65, 0L, recordKey, recordValue) )); assertEquals(5, task.process()); @@ -177,9 +177,9 @@ public void testPauseResume() throws Exception { 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, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 40, 0L, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 50, 0L, recordKey, recordValue) )); assertEquals(2, consumer.paused().size()); 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 5edff28836feb..36b61a76940c7 100644 --- a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java @@ -200,7 +200,7 @@ public void process(String topicName, byte[] key, byte[] value) { } // Add the record ... long offset = offsetsByTopicPartition.get(tp).incrementAndGet(); - task.addRecords(tp, records(new ConsumerRecord(tp.topic(), tp.partition(), offset, key, value))); + task.addRecords(tp, records(new ConsumerRecord(tp.topic(), tp.partition(), offset, 0L, key, value))); producer.clear(); // Process the record ... task.process(); From 3b233831719a1f4ef01fe276d2a4d8413bb45ee6 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Wed, 20 Jan 2016 21:07:52 -0800 Subject: [PATCH 03/33] Rebased on trunk. Addressed Jason and Anna's comments. --- .../kafka/common/requests/ProduceRequest.java | 4 +++- .../kafka/common/requests/ProduceResponse.java | 15 +++++++++++++-- .../common/requests/RequestResponseTest.java | 11 +++++++---- core/src/main/scala/kafka/api/FetchRequest.scala | 2 +- .../src/main/scala/kafka/api/FetchResponse.scala | 2 +- .../kafka/message/ByteBufferMessageSet.scala | 16 +++++++++++++++- core/src/main/scala/kafka/message/Message.scala | 4 ++-- core/src/main/scala/kafka/server/KafkaApis.scala | 9 +++------ .../main/scala/kafka/server/KafkaConfig.scala | 2 +- .../main/scala/kafka/server/ReplicaManager.scala | 2 +- .../GroupCoordinatorResponseTest.scala | 5 +++-- 11 files changed, 50 insertions(+), 22 deletions(-) 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 a41639de123f4..146c8b39a7713 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 @@ -106,7 +106,9 @@ public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { case 0: return new ProduceResponse(responseMap); case 1: - return new ProduceResponse(responseMap, ProduceResponse.DEFAULT_THROTTLE_TIME); + return new ProduceResponse(responseMap, ProduceResponse.DEFAULT_THROTTLE_TIME, versionId); + 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))); 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 cf6739ca75037..b57ee7c982fd3 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 @@ -74,9 +74,20 @@ public ProduceResponse(Map responses) { * @param throttleTime Time in milliseconds the response was throttled */ public ProduceResponse(Map responses, int throttleTime) { - super(new Struct(CURRENT_SCHEMA)); + this(responses, throttleTime, (int) ProtoUtils.latestVersion(ApiKeys.PRODUCE.id)); + } + + /** + * Constructor for 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 responses, int throttleTime, int version) { + super(new Struct(ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, version))); initCommonFields(responses); - struct.set(THROTTLE_TIME_KEY_NAME, throttleTime); + if (version > 0) + struct.set(THROTTLE_TIME_KEY_NAME, throttleTime); this.responses = responses; this.throttleTime = throttleTime; } 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 a483909b1bc7b..242fb44c0bc45 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 @@ -82,7 +82,7 @@ public void testSerialization() throws Exception { createOffsetFetchRequest().getErrorResponse(0, new UnknownServerException()), createOffsetFetchResponse(), createProduceRequest(), - createProduceRequest().getErrorResponse(1, new UnknownServerException()), + createProduceRequest().getErrorResponse(2, new UnknownServerException()), createProduceResponse(), createStopReplicaRequest(), createStopReplicaRequest().getErrorResponse(0, new UnknownServerException()), @@ -125,12 +125,15 @@ public void produceResponseVersionTest() { responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE.code(), 10000, Record.NO_TIMESTAMP)); ProduceResponse v0Response = new ProduceResponse(responseData); // No need to verify V1 here because - ProduceResponse v2Response = 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, v2Response.getThrottleTime()); + assertEquals("Throttle time must be 10", 10, v1Response.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, 2), v2Response.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()); } diff --git a/core/src/main/scala/kafka/api/FetchRequest.scala b/core/src/main/scala/kafka/api/FetchRequest.scala index 8f26779a21785..f47942ce1f6f9 100644 --- a/core/src/main/scala/kafka/api/FetchRequest.scala +++ b/core/src/main/scala/kafka/api/FetchRequest.scala @@ -22,7 +22,7 @@ import kafka.api.ApiUtils._ import kafka.common.TopicAndPartition import kafka.consumer.ConsumerConfig import kafka.network.RequestChannel -import kafka.message.{Message, MessageSet} +import kafka.message.MessageSet import java.util.concurrent.atomic.AtomicInteger import java.nio.ByteBuffer diff --git a/core/src/main/scala/kafka/api/FetchResponse.scala b/core/src/main/scala/kafka/api/FetchResponse.scala index 4f325d1600754..b7b9f6a00603c 100644 --- a/core/src/main/scala/kafka/api/FetchResponse.scala +++ b/core/src/main/scala/kafka/api/FetchResponse.scala @@ -20,7 +20,7 @@ package kafka.api import java.nio.ByteBuffer import java.nio.channels.GatheringByteChannel -import kafka.common.{TopicAndPartition, ErrorMapping} +import kafka.common.TopicAndPartition import kafka.message.{NoCompressionCodec, Message, MessageSet, ByteBufferMessageSet} import kafka.api.ApiUtils._ import org.apache.kafka.common.KafkaException diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 7fbffacc87997..410b3a1909265 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -531,6 +531,19 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi throw new InvalidMessageException("Compacted topic cannot accept message without key.") } + /** + * This method validates the timestamps of a message. + * 1. If the message is using create time, this method checks if it is with acceptable range. + * 2. If the message is using log append time, this method will overwrite the timestamp of the message. + * + * The return value of this method indicates whether the timestamp of a message is overwritten. + * In case 1, the method either throw exception (the timestamp is out of range) or return false. + * In case 2, the method returns: + * a) True, if the timestamp of the message has been overwritten. This means re-compression is required if the + * the message is an inner message of a compressed message. + * b) False, if the timestamp is not overwritten. This only occurs when the timestamp of an inner message from a + * compressed message is already set to [[Message.InheritedTimestamp]] + */ private def validateTimestamps(message: Message, now: Long, timestampType: TimestampType, @@ -545,7 +558,9 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi message.buffer.putLong(Message.TimestampOffset, now) // We have to update crc after updating the timestamp. Utils.writeUnsignedInt(message.buffer, Message.CrcOffset, message.computeChecksum()) + overwritten = true } else { + // We only overwrite the timestamp if necessary if (message.timestamp != Message.InheritedTimestamp) { message.buffer.putLong(Message.TimestampOffset, Message.InheritedTimestamp) // We have to update crc after updating the timestamp. @@ -553,7 +568,6 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi overwritten = true } } - } overwritten } diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala index 3754e37f3ef0e..cb6894c9da97d 100755 --- a/core/src/main/scala/kafka/message/Message.scala +++ b/core/src/main/scala/kafka/message/Message.scala @@ -165,8 +165,8 @@ class Message(val buffer: ByteBuffer, val wrapperMessageTimestamp: Long = Messag if (codec.codec > 0) attributes = (attributes | (CompressionCodeMask & codec.codec)).toByte buffer.put(attributes) - // Only put timestamp when "magic" value is 1 - if (magic == MagicValue_V1) + // Only put timestamp when "magic" value is greater than 0 + if (magic > MagicValue_V0) buffer.putLong(timestamp) if(key == null) { buffer.putInt(-1) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 4efef81915aa2..245f19a9d8b3a 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -331,12 +331,8 @@ class KafkaApis(val requestChannel: RequestChannel, // the callback for sending a produce response def sendResponseCallback(responseStatus: Map[TopicPartition, PartitionResponse]) { -<<<<<<< HEAD - val mergedResponseStatus = responseStatus ++ unauthorizedRequestInfo.mapValues(_ => new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED.code, -1)) -======= val mergedResponseStatus = responseStatus ++ unauthorizedRequestInfo.mapValues(_ => - ProducerResponseStatus(Errors.TOPIC_AUTHORIZATION_FAILED.code, -1, Message.NoTimestamp)) ->>>>>>> Added timestamp and relative offset to clients package. + new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED.code, -1, Message.NoTimestamp)) var errorInResponse = false @@ -373,7 +369,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." diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index faedb0a5e75c1..78fb9722e7ae3 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -259,7 +259,7 @@ object KafkaConfig { val NumRecoveryThreadsPerDataDirProp = "num.recovery.threads.per.data.dir" val AutoCreateTopicsEnableProp = "auto.create.topics.enable" val MinInSyncReplicasProp = "min.insync.replicas" - val MessageTimestampTypeProp = "messge.timestamp.type" + val MessageTimestampTypeProp = "message.timestamp.type" val MessageTimestampDifferenceMaxMsProp = "message.timestamp.difference.max.ms" /** ********* Replication configuration ***********/ val ControllerSocketTimeoutMsProp = "controller.socket.timeout.ms" diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 986eb890602db..b3eef8f14f84a 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -441,7 +441,7 @@ class ReplicaManager(val config: KafkaConfig, case imse: CorruptRecordException => (topicPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(imse))) case ime : InvalidMessageException => - (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(ime))) + (topicPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(ime))) case t: Throwable => BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).failedProduceRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats.failedProduceRequestRate.mark() diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala index 7e6e7656be6c0..200f98d10898c 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala @@ -17,6 +17,7 @@ package kafka.coordinator +import org.apache.kafka.common.record.Record import org.junit.Assert._ import kafka.common.{OffsetAndMetadata, TopicAndPartition} @@ -833,7 +834,7 @@ 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.replay(replicaManager) @@ -909,7 +910,7 @@ 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.replay(replicaManager) From 813e0dd3a06ade5ee494caa4bb5754c4ab0626e8 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Fri, 22 Jan 2016 10:59:58 -0800 Subject: [PATCH 04/33] Updated java doc for KafkaProducer and RecordMetadata. Added comments for getTimestamp() in KafkaProducer. --- .../org/apache/kafka/clients/producer/KafkaProducer.java | 9 ++++++--- .../apache/kafka/clients/producer/ProducerRecord.java | 9 +++++++++ 2 files changed, 15 insertions(+), 3 deletions(-) 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 00c1865acf0eb..e39085a30eb36 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,8 @@ public Future send(ProducerRecord record) { * records waiting to be sent. This allows sending many records in parallel without blocking to wait for the * response after each one. *

- * 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. *

* 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() @@ -529,7 +529,10 @@ private long waitOnMetadata(String topic, long maxWaitMs) throws InterruptedExce } private long getTimestamp(String topic, Long timestamp) { - // If log append times is used for the topic, we overwrite the timestamp to avoid server side re-compression. + // If log append time is used for the topic, and we are sending records with timestamp other than INHERITED_TIMESTAMP, + // the broker will overwrite the timestamp and do the re-compression if compression codec is configured. + // To avoid broker side re-compression, we overwrite the timestamp to INHERITED_TIMESTAMP if the topic is using + // log append time. if (metadata.isUsingLogAppendTime(topic)) return Record.INHERITED_TIMESTAMP; else if (timestamp == null) 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 c9d880d00332d..9dad743e8b7e2 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 @@ -21,6 +21,15 @@ * 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. + *

+ * The record also has an associated timestamp. If user did not provide a timestamp, the producer will stamp the record + * with a timestamp depending on what is the timestamp type used by the topic. + *

  • + * If the topic is configured to use {@link org.apache.kafka.common.record.Record.TimestampType#CreateTime CreateTime} + * the timestamp will be the producer current time. + *
  • + * If the topic is configured to use {@link org.apache.kafka.common.record.Record.TimestampType#LogAppendTime LogAppendTime} + * the timestamp will be the time when Kafka broker accepts the record. */ public final class ProducerRecord { From b424c45af4c55e37e8699e529a62da2620828e70 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Tue, 26 Jan 2016 18:47:24 -0800 Subject: [PATCH 05/33] Use the timestamp attribute bit as Guozhang proposed. --- .../org/apache/kafka/clients/Metadata.java | 21 ---- .../clients/consumer/ConsumerRecord.java | 16 ++- .../clients/consumer/internals/Fetcher.java | 4 +- .../kafka/clients/producer/KafkaProducer.java | 15 +-- .../clients/producer/internals/Sender.java | 3 - .../kafka/common/record/MemoryRecords.java | 8 +- .../apache/kafka/common/record/Record.java | 44 +++++-- .../clients/consumer/MockConsumerTest.java | 5 +- .../kafka/common/record/RecordTest.java | 2 +- .../connect/runtime/WorkerSinkTaskTest.java | 3 +- .../runtime/WorkerSinkTaskThreadedTest.java | 5 +- .../storage/KafkaConfigStorageTest.java | 25 ++-- .../storage/KafkaOffsetBackingStoreTest.java | 17 +-- .../kafka/connect/util/KafkaBasedLogTest.java | 17 +-- .../main/scala/kafka/api/FetchResponse.scala | 24 +--- .../src/main/scala/kafka/log/LogCleaner.scala | 3 +- .../kafka/message/ByteBufferMessageSet.scala | 114 ++++++++---------- .../main/scala/kafka/message/Message.scala | 94 +++++++++++---- .../main/scala/kafka/message/MessageSet.scala | 35 +++++- .../scala/kafka/message/MessageWriter.scala | 12 +- .../kafka/api/BaseConsumerTest.scala | 1 + .../unit/kafka/message/MessageTest.scala | 4 +- .../kafka/message/MessageWriterTest.scala | 3 +- .../processor/internals/RecordQueue.java | 4 +- .../internals/PartitionGroupTest.java | 13 +- .../internals/ProcessorStateManagerTest.java | 6 +- .../processor/internals/RecordQueueTest.java | 19 +-- .../processor/internals/StandbyTaskTest.java | 19 +-- .../processor/internals/StreamTaskTest.java | 31 ++--- .../test/ProcessorTopologyTestDriver.java | 3 +- 30 files changed, 319 insertions(+), 251 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java index 7f97c806b9f50..73a9f333cc7a4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -22,7 +22,6 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.errors.TimeoutException; -import org.apache.kafka.common.record.Record; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,7 +45,6 @@ public final class Metadata { private Cluster cluster; private boolean needUpdate; private final Set topics; - private final Set topicsUsingLogAppendTime; private final List listeners; private boolean needMetadataForAllTopics; @@ -72,7 +70,6 @@ public Metadata(long refreshBackoffMs, long metadataExpireMs) { this.cluster = Cluster.empty(); this.needUpdate = false; this.topics = new HashSet(); - this.topicsUsingLogAppendTime = new HashSet(); this.listeners = new ArrayList<>(); this.needMetadataForAllTopics = false; } @@ -91,24 +88,6 @@ public synchronized void add(String topic) { topics.add(topic); } - /** - * Record the timestamp type used by a topic. This information is needed to determine whether the timestamp - * should be overwritten or not. - */ - public synchronized void recordTopicTimestampType(String topic, Record.TimestampType timestampType) { - if (timestampType == Record.TimestampType.LogAppendTime) - topicsUsingLogAppendTime.add(topic); - else - topicsUsingLogAppendTime.remove(topic); - } - - /** - * Check if the topic is using log append time or not. - */ - public synchronized boolean isUsingLogAppendTime(String topic) { - return topicsUsingLogAppendTime.contains(topic); - } - /** * The next time to update the cluster info is the maximum of the time the current info will expire and the time the * current info can be updated (i.e. backoff time has elapsed); If an update has been request then the expiry time 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 10a166c86eff1..160c7f46bd127 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.Record; + /** * 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. @@ -21,6 +23,7 @@ public final class ConsumerRecord { private final int partition; private final long offset; private final long timestamp; + private final Record.TimestampType timestampType; private final K key; private final V value; @@ -34,13 +37,20 @@ public final class ConsumerRecord { * @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, long timestamp, K key, V value) { + public ConsumerRecord(String topic, + int partition, + long offset, + long timestamp, + Record.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; } @@ -84,6 +94,10 @@ public long timestamp() { return timestamp; } + public Record.TimestampType timestampType() { + return timestampType; + } + @Override public String toString() { return "ConsumerRecord(topic = " + topic() + ", partition = " + partition() + ", offset = " + offset() 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 d1b17dad6aae9..8027f1b1fd83f 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.Errors; import org.apache.kafka.common.record.LogEntry; import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.Record; import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.requests.FetchResponse; import org.apache.kafka.common.requests.ListOffsetRequest; @@ -615,12 +616,13 @@ private ConsumerRecord parseRecord(TopicPartition partition, LogEntry logE logEntry.record().ensureValid(); long offset = logEntry.offset(); long timestamp = logEntry.record().timestamp(); + Record.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, timestamp, key, value); + return new ConsumerRecord<>(partition.topic(), partition.partition(), offset, timestamp, timestampType, key, value); } catch (KafkaException e) { throw e; } catch (RuntimeException e) { 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 e39085a30eb36..9b0410cd405d5 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 @@ -456,7 +456,7 @@ private Future doSend(ProducerRecord record, Callback call int serializedSize = Records.LOG_OVERHEAD + Record.recordSize(serializedKey, serializedValue); ensureValidRecordSize(serializedSize); TopicPartition tp = new TopicPartition(record.topic(), partition); - long timestamp = getTimestamp(record.topic(), record.timestamp()); + 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, timestamp, serializedKey, serializedValue, callback, remainingWaitMs); if (result.batchIsFull || result.newBatchCreated) { @@ -528,19 +528,6 @@ private long waitOnMetadata(String topic, long maxWaitMs) throws InterruptedExce return time.milliseconds() - begin; } - private long getTimestamp(String topic, Long timestamp) { - // If log append time is used for the topic, and we are sending records with timestamp other than INHERITED_TIMESTAMP, - // the broker will overwrite the timestamp and do the re-compression if compression codec is configured. - // To avoid broker side re-compression, we overwrite the timestamp to INHERITED_TIMESTAMP if the topic is using - // log append time. - if (metadata.isUsingLogAppendTime(topic)) - return Record.INHERITED_TIMESTAMP; - else if (timestamp == null) - return time.milliseconds(); - else - return timestamp; - } - /** * Validate that the record size isn't too large */ 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 7d4589d0adbcb..8e9397324883b 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 @@ -258,9 +258,6 @@ private void handleProduceResponse(ClientResponse response, Map= 0 ? Record.TimestampType.LogAppendTime : Record.TimestampType.CreateTime); } this.sensors.recordLatency(response.request().request().destination(), response.requestLatencyMs()); this.sensors.recordThrottleTime(response.request().request().destination(), 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 c55e9671c47d3..2b43db7b409d3 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 @@ -244,7 +244,9 @@ private RecordsIterator(LogEntry entry, CompressionType type, boolean shallow) { while (true) { try { LogEntry logEntry = getNextEntryFromStream(); - Record recordWithTimestamp = new Record(logEntry.record().buffer(), wrapperRecordTimestamp); + Record recordWithTimestamp = new Record(logEntry.record().buffer(), + wrapperRecordTimestamp, + entry.record().timestampType()); logEntries.add(new LogEntry(logEntry.offset(), recordWithTimestamp)); } catch (EOFException e) { break; @@ -306,12 +308,12 @@ protected LogEntry makeNext() { private LogEntry getNextEntry() throws IOException { if (logEntries != null) - return getNextEntryFromBuffer(); + return getNextEntryFromEntryList(); else return getNextEntryFromStream(); } - private LogEntry getNextEntryFromBuffer() { + private LogEntry getNextEntryFromEntryList() { return logEntries.isEmpty() ? null : logEntries.remove(); } 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 7323f1077ca62..66ec87d356e37 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,6 +36,16 @@ public enum TimestampType { this.value = value; this.name = name; } + + public static TimestampType getTimestampType(byte attributes) { + int timestampType = (attributes & TIMESTAMP_TYPE_MASK) >> TIMESTAMP_TYPE_ATTRIBUTE_OFFSET; + return timestampType == 0 ? CreateTime : LogAppendTime; + } + + public static byte setTimestampType(byte attributes, TimestampType timestampType) { + return timestampType == CreateTime ? + (byte) (attributes & ~TIMESTAMP_TYPE_MASK) : (byte) (attributes | TIMESTAMP_TYPE_MASK); + } } /** @@ -78,38 +88,43 @@ public enum TimestampType { public static final byte CURRENT_MAGIC_VALUE = 1; /** - * Specifies the mask for the compression code. 3 bits to hold the compression codec. 0 is reserved to indicate no + * Specifies the mask for the compression code. 4 bits to hold the compression codec. 0 is reserved to indicate no * compression */ - public static final int COMPRESSION_CODEC_MASK = 0x07; + public static final int COMPRESSION_CODEC_MASK = 0x0F; /** - * Compression code for uncompressed records + * Specify the mask of timestamp type. + * 0 for CreateTime, 1 for LogAppendTime. */ - public static final int NO_COMPRESSION = 0; + public static final byte TIMESTAMP_TYPE_MASK = 0x10; + public static final int TIMESTAMP_TYPE_ATTRIBUTE_OFFSET = 4; /** - * Timestamp value for compressed records whose timestamp inherit from wrapper record. + * Compression code for uncompressed records */ - public static final long INHERITED_TIMESTAMP = -1; + public static final int NO_COMPRESSION = 0; /** * Timestamp value for records without a timestamp */ - public static final long NO_TIMESTAMP = -2; + public static final long NO_TIMESTAMP = -1; private final ByteBuffer buffer; - private final long wrapperRecordTimestamp; + private final Long wrapperRecordTimestamp; + private final TimestampType timestampTypeToUse; public Record(ByteBuffer buffer) { this.buffer = buffer; - this.wrapperRecordTimestamp = NO_TIMESTAMP; + this.wrapperRecordTimestamp = null; + this.timestampTypeToUse = null; } // Package private constructor for inner iteration. - Record(ByteBuffer buffer, long wrapperRecordTimestamp) { + Record(ByteBuffer buffer, Long wrapperRecordTimestamp, TimestampType timestampTypeToUse) { this.buffer = buffer; this.wrapperRecordTimestamp = wrapperRecordTimestamp; + this.timestampTypeToUse = timestampTypeToUse; } /** @@ -333,14 +348,17 @@ public long timestamp() { if (magic() == MAGIC_VALUE_V0) return NO_TIMESTAMP; else { - long timestamp = buffer.getLong(TIMESTAMP_OFFSET); - if (timestamp == INHERITED_TIMESTAMP) + if (timestampTypeToUse == TimestampType.LogAppendTime && wrapperRecordTimestamp != null) return wrapperRecordTimestamp; else - return timestamp; + return buffer.getLong(TIMESTAMP_OFFSET); } } + public TimestampType timestampType() { + return timestampTypeToUse == null ? TimestampType.getTimestampType(attributes()) : timestampTypeToUse; + } + /** * The compression type used with this record */ 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 1182782f9e381..915762ff0b37b 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 @@ import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.record.Record; import org.junit.Test; import java.util.Arrays; @@ -42,8 +43,8 @@ public void testSimpleMock() { beginningOffsets.put(new TopicPartition("test", 1), 0L); consumer.updateBeginningOffsets(beginningOffsets); consumer.seek(new TopicPartition("test", 0), 0); - ConsumerRecord rec1 = new ConsumerRecord("test", 0, 0, 0L, "key1", "value1"); - ConsumerRecord rec2 = new ConsumerRecord("test", 0, 1, 0L, "key2", "value2"); + ConsumerRecord rec1 = new ConsumerRecord("test", 0, 0, 0L, Record.TimestampType.CreateTime, "key1", "value1"); + ConsumerRecord rec2 = new ConsumerRecord("test", 0, 1, 0L, Record.TimestampType.CreateTime, "key2", "value2"); consumer.addRecord(rec1); consumer.addRecord(rec2); ConsumerRecords recs = consumer.poll(1); 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 c5843c4b0a2e4..6482529c5c9b5 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 @@ -102,7 +102,7 @@ public static Collection data() { byte[] payload = new byte[1000]; Arrays.fill(payload, (byte) 1); List values = new ArrayList(); - for (long timestamp : Arrays.asList(Record.NO_TIMESTAMP, Record.INHERITED_TIMESTAMP, 0L, 1L)) + 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()) 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 1c7c723b9a94f..b499f651e725c 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.KafkaConsumer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.record.Record; 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 @@ private void expectConsumerPoll(final int numMessages) { public ConsumerRecords answer() throws Throwable { List> records = new ArrayList<>(); for (int i = 0; i < numMessages; i++) - records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned + i, 0L, RAW_KEY, RAW_VALUE)); + records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned + i, 0L, Record.TimestampType.CreateTime, RAW_KEY, RAW_VALUE)); recordsReturned += numMessages; return new ConsumerRecords<>( numMessages > 0 ? 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 84c9f89729d3f..cdf34cf40f719 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.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetCommitCallback; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.record.Record; 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 ConsumerRecords answer() throws Throwable { Collections.singletonMap( new TopicPartition(TOPIC, PARTITION), Arrays.asList( - new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, 0L, RAW_KEY, RAW_VALUE) + new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, 0L, Record.TimestampType.CreateTime, RAW_KEY, RAW_VALUE) ))); recordsReturned++; return records; @@ -547,7 +548,7 @@ public ConsumerRecords answer() throws Throwable { Collections.singletonMap( new TopicPartition(TOPIC, PARTITION), Arrays.asList( - new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, 0L, RAW_KEY, RAW_VALUE) + new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, 0L, Record.TimestampType.CreateTime, RAW_KEY, RAW_VALUE) ))); recordsReturned++; return records; 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 d82ceed7ad15b..5645b52b58b43 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.ConsumerRecord; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.record.Record; 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 void testRestore() throws Exception { expectConfigure(); // Overwrite each type at least once to ensure we see the latest data after loading List> existingRecords = Arrays.asList( - new ConsumerRecord<>(TOPIC, 0, 0, 0L, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)), - new ConsumerRecord<>(TOPIC, 0, 1, 0L, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(1)), - new ConsumerRecord<>(TOPIC, 0, 2, 0L, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)), - new ConsumerRecord<>(TOPIC, 0, 3, 0L, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(3)), - new ConsumerRecord<>(TOPIC, 0, 4, 0L, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(4)), + new ConsumerRecord<>(TOPIC, 0, 0, 0L, Record.TimestampType.CreateTime, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)), + new ConsumerRecord<>(TOPIC, 0, 1, 0L, Record.TimestampType.CreateTime, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(1)), + new ConsumerRecord<>(TOPIC, 0, 2, 0L, Record.TimestampType.CreateTime, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)), + new ConsumerRecord<>(TOPIC, 0, 3, 0L, Record.TimestampType.CreateTime, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(3)), + new ConsumerRecord<>(TOPIC, 0, 4, 0L, Record.TimestampType.CreateTime, 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, 0L, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5)), - new ConsumerRecord<>(TOPIC, 0, 6, 0L, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(6))); + new ConsumerRecord<>(TOPIC, 0, 5, 0L, Record.TimestampType.CreateTime, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5)), + new ConsumerRecord<>(TOPIC, 0, 6, 0L, Record.TimestampType.CreateTime, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(6))); LinkedHashMap 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 void testPutTaskConfigsDoesNotResolveAllInconsistencies() throws Exceptio expectConfigure(); List> existingRecords = Arrays.asList( - new ConsumerRecord<>(TOPIC, 0, 0, 0L, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)), + new ConsumerRecord<>(TOPIC, 0, 0, 0L, Record.TimestampType.CreateTime, 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, 0L, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)), - new ConsumerRecord<>(TOPIC, 0, 4, 0L, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(4)), - new ConsumerRecord<>(TOPIC, 0, 5, 0L, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5))); + new ConsumerRecord<>(TOPIC, 0, 2, 0L, Record.TimestampType.CreateTime, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)), + new ConsumerRecord<>(TOPIC, 0, 4, 0L, Record.TimestampType.CreateTime, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(4)), + new ConsumerRecord<>(TOPIC, 0, 5, 0L, Record.TimestampType.CreateTime, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5))); LinkedHashMap 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 @@ private void expectReadToEnd(final LinkedHashMap serializedConfi public Future answer() throws Throwable { TestFuture future = new TestFuture(); for (Map.Entry entry : serializedConfigs.entrySet()) - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, logOffset++, 0L, entry.getKey(), entry.getValue())); + capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, logOffset++, 0L, Record.TimestampType.CreateTime, entry.getKey(), entry.getValue())); future.resolveOnGet((Void) null); return future; } 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 a3be7fcbe53a1..bea99b8cef474 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.ConsumerRecord; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.record.Record; 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 void testStartStop() throws Exception { public void testReloadOnStart() throws Exception { expectConfigure(); expectStart(Arrays.asList( - new ConsumerRecord<>(TOPIC, 0, 0, 0L, TP0_KEY.array(), TP0_VALUE.array()), - new ConsumerRecord<>(TOPIC, 1, 0, 0L, TP1_KEY.array(), TP1_VALUE.array()), - new ConsumerRecord<>(TOPIC, 0, 1, 0L, TP0_KEY.array(), TP0_VALUE_NEW.array()), - new ConsumerRecord<>(TOPIC, 1, 1, 0L, TP1_KEY.array(), TP1_VALUE_NEW.array()) + new ConsumerRecord<>(TOPIC, 0, 0, 0L, Record.TimestampType.CreateTime, TP0_KEY.array(), TP0_VALUE.array()), + new ConsumerRecord<>(TOPIC, 1, 0, 0L, Record.TimestampType.CreateTime, TP1_KEY.array(), TP1_VALUE.array()), + new ConsumerRecord<>(TOPIC, 0, 1, 0L, Record.TimestampType.CreateTime, TP0_KEY.array(), TP0_VALUE_NEW.array()), + new ConsumerRecord<>(TOPIC, 1, 1, 0L, Record.TimestampType.CreateTime, TP1_KEY.array(), TP1_VALUE_NEW.array()) )); expectStop(); @@ -176,8 +177,8 @@ public Object answer() throws Throwable { PowerMock.expectLastCall().andAnswer(new IAnswer() { @Override public Object answer() throws Throwable { - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 0, 0L, TP0_KEY.array(), TP0_VALUE.array())); - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 0, 0L, TP1_KEY.array(), TP1_VALUE.array())); + capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 0, 0L, Record.TimestampType.CreateTime, TP0_KEY.array(), TP0_VALUE.array())); + capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 0, 0L, Record.TimestampType.CreateTime, TP1_KEY.array(), TP1_VALUE.array())); secondGetReadToEndCallback.getValue().onCompletion(null, null); return null; } @@ -189,8 +190,8 @@ public Object answer() throws Throwable { PowerMock.expectLastCall().andAnswer(new IAnswer() { @Override public Object answer() throws Throwable { - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 1, 0L, TP0_KEY.array(), TP0_VALUE_NEW.array())); - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 1, 0L, TP1_KEY.array(), TP1_VALUE_NEW.array())); + capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 1, 0L, Record.TimestampType.CreateTime, TP0_KEY.array(), TP0_VALUE_NEW.array())); + capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 1, 0L, Record.TimestampType.CreateTime, TP1_KEY.array(), TP1_VALUE_NEW.array())); thirdGetReadToEndCallback.getValue().onCompletion(null, null); return null; } 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 5efe2db315e08..26bc5a5b694aa 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.TopicPartition; import org.apache.kafka.common.errors.LeaderNotAvailableException; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.Record; import org.apache.kafka.common.utils.Time; import org.easymock.Capture; import org.easymock.EasyMock; @@ -182,7 +183,7 @@ public void run() { consumer.schedulePollTask(new Runnable() { @Override public void run() { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TP0_KEY, TP0_VALUE)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, Record.TimestampType.CreateTime, TP0_KEY, TP0_VALUE)); } }); consumer.scheduleNopPollTask(); @@ -190,7 +191,7 @@ public void run() { consumer.schedulePollTask(new Runnable() { @Override public void run() { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TP1_KEY, TP1_VALUE)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, Record.TimestampType.CreateTime, TP1_KEY, TP1_VALUE)); } }); consumer.schedulePollTask(new Runnable() { @@ -297,16 +298,16 @@ public void run() { consumer.schedulePollTask(new Runnable() { @Override public void run() { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TP0_KEY, TP0_VALUE)); - consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 1, 0L, TP0_KEY, TP0_VALUE_NEW)); - consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TP1_KEY, TP1_VALUE)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, Record.TimestampType.CreateTime, TP0_KEY, TP0_VALUE)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 1, 0L, Record.TimestampType.CreateTime, TP0_KEY, TP0_VALUE_NEW)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, Record.TimestampType.CreateTime, TP1_KEY, TP1_VALUE)); } }); consumer.schedulePollTask(new Runnable() { @Override public void run() { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 1, 0L, TP1_KEY, TP1_VALUE_NEW)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 1, 0L, Record.TimestampType.CreateTime, TP1_KEY, TP1_VALUE_NEW)); } }); @@ -362,8 +363,8 @@ public void run() { consumer.schedulePollTask(new Runnable() { @Override public void run() { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TP0_KEY, TP0_VALUE_NEW)); - consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TP0_KEY, TP0_VALUE_NEW)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, Record.TimestampType.CreateTime, TP0_KEY, TP0_VALUE_NEW)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, Record.TimestampType.CreateTime, TP0_KEY, TP0_VALUE_NEW)); } }); diff --git a/core/src/main/scala/kafka/api/FetchResponse.scala b/core/src/main/scala/kafka/api/FetchResponse.scala index b7b9f6a00603c..f3a06c236578c 100644 --- a/core/src/main/scala/kafka/api/FetchResponse.scala +++ b/core/src/main/scala/kafka/api/FetchResponse.scala @@ -54,29 +54,7 @@ case class FetchResponsePartitionData(error: Short = Errors.NONE.code, hw: Long if (messages.hasMagicValue(toMagicValue)) this else { - val offsets = new ArrayBuffer[Long] - val newMessages = new ArrayBuffer[Message] - messages.iterator.foreach(messageAndOffset => { - val message = messageAndOffset.message - // File message set only has shallow iterator. We need to do deep iteration here if needed. - if (message.compressionCodec == NoCompressionCodec) { - newMessages += messageAndOffset.message.toFormatVersion(toMagicValue) - offsets += messageAndOffset.offset - } else { - 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. - val newMessageSet = new ByteBufferMessageSet( - compressionCodec = messages.headOption.map(_.message.compressionCodec).getOrElse(NoCompressionCodec), - offsetSeq = offsets.toSeq, - newMessages: _*) - new FetchResponsePartitionData(error, hw, newMessageSet) + new FetchResponsePartitionData(error, hw, messages.toMessageFormat(toMagicValue)) } } } diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 41910ab92761a..87dc10b303618 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -467,8 +467,9 @@ private[log] class Cleaner(val id: Int, val firstAbsoluteOffset = messages.head.offset var offset = -1L val magicValue = messages.head.message.magic + 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, timestamp = messageSetTimestamp, magicValue = magicValue) { outputStream => + messageWriter.write(codec = compressionCodec, timestamp = messageSetTimestamp, timestampType = timestampType, magicValue = magicValue) { outputStream => val output = new DataOutputStream(CompressionFactory(compressionCodec, outputStream)) try { for (messageOffset <- messages) { diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 410b3a1909265..0ccdb1cc48e63 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -38,6 +38,7 @@ object ByteBufferMessageSet { private def create(offsetAssignor: OffsetAssigner, compressionCodec: CompressionCodec, messageSetTimestampAssignor: (Seq[Message]) => Long, + timestampType: TimestampType, messages: Message*): ByteBuffer = { if(messages.size == 0) { MessageSet.Empty.buffer @@ -52,7 +53,7 @@ object ByteBufferMessageSet { var offset = -1L val magicValue = messages.head.magic val messageWriter = new MessageWriter(math.min(math.max(MessageSet.messageSetSize(messages) / 2, 1024), 1 << 16)) - messageWriter.write(codec = compressionCodec, timestamp = messageSetTimestamp, magicValue = magicValue) { outputStream => + messageWriter.write(codec = compressionCodec, timestamp = messageSetTimestamp, timestampType = timestampType, magicValue = magicValue) { outputStream => val output = new DataOutputStream(CompressionFactory(compressionCodec, outputStream)) try { for (message <- messages) { @@ -87,7 +88,10 @@ object ByteBufferMessageSet { val wrapperMessageOffset = wrapperMessageAndOffset.offset val wrapperMessage = wrapperMessageAndOffset.message - val wrapperMessageTimestamp = if (wrapperMessage.magic > MagicValue_V0) wrapperMessage.timestamp else NoTimestamp + 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 RuntimeException("wrapper message = " + wrapperMessage) val inputStream: InputStream = new ByteBufferBackedInputStream(wrapperMessage.payload) @@ -147,8 +151,10 @@ object ByteBufferMessageSet { 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 = buffer, wrapperMessageTimestamp = wrapperMessageTimestamp) + 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} " + @@ -229,25 +235,26 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi def this(compressionCodec: CompressionCodec, messages: Message*) { this(ByteBufferMessageSet.create(new OffsetAssigner(new AtomicLong(0), messages.size), compressionCodec, - MessageSet.validateMagicValuesAndGetTimestamp, messages:_*)) + MessageSet.validateMagicValuesAndGetTimestamp, TimestampType.CreateTime, messages:_*)) } def this(compressionCodec: CompressionCodec, offsetCounter: AtomicLong, messages: Message*) { this(ByteBufferMessageSet.create(new OffsetAssigner(offsetCounter, messages.size), compressionCodec, - MessageSet.validateMagicValuesAndGetTimestamp, messages:_*)) + MessageSet.validateMagicValuesAndGetTimestamp, TimestampType.CreateTime, messages:_*)) } def this(compressionCodec: CompressionCodec, offsetCounter: AtomicLong, messageSetTimestampAssignor: (Message*) => Long, + timestampType: TimestampType, messages: Message*) { this(ByteBufferMessageSet.create(new OffsetAssigner(offsetCounter, messages.size), compressionCodec, - messageSetTimestampAssignor, messages:_*)) + messageSetTimestampAssignor, timestampType, messages:_*)) } def this(compressionCodec: CompressionCodec, offsetSeq: Seq[Long], messages: Message*) { this(ByteBufferMessageSet.create(new OffsetAssigner(offsetSeq), compressionCodec, - MessageSet.validateMagicValuesAndGetTimestamp, messages:_*)) + MessageSet.validateMagicValuesAndGetTimestamp, TimestampType.CreateTime, messages:_*)) } def this(messages: Message*) { @@ -376,8 +383,8 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi if (sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) { // check the magic value if (!hasMagicValue(magicValueToUse)) { - // up-conversion - convertNonCompressedMessages(offsetCounter, compactedTopic, magicValueToUse) + // Message format conversion + convertNonCompressedMessages(offsetCounter, compactedTopic, now, messageTimestampType, magicValueToUse) } else { // Do in-place validation, offset assignment and maybe set timestamp validateNonCompressedMessagesAndAssignOffsetInPlace(offsetCounter, now, compactedTopic, messageTimestampType, @@ -403,9 +410,8 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi validateMessageKey(message, compactedTopic) if (message.magic > Message.MagicValue_V0 && magicValueToUse > Message.MagicValue_V0) { // Re-compression situation 3 - // Check if we need to overwrite timestamp - if (validateTimestamps(message, now, messageTimestampType, messageTimestampDiffMaxMs, targetCodec)) - requireReCompression = true + // Validate the timestamp + validateTimestamp(message, now, messageTimestampType, messageTimestampDiffMaxMs) // Check if we need to overwrite offset if (messageAndOffset.offset != expectedRelativeOffset.getAndIncrement) requireReCompression = true @@ -416,10 +422,6 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi if (message.magic != magicValueToUse) requireReCompression = true - // The conversion have the following impact on timestamp - // V0 -> V1: All messages will have NoTimestamp(-2) - // V1 -> V0: Timestamp field will be removed - // V1 -> V1: All the timestamp validated above will be used validatedMessages += message.toFormatVersion(magicValueToUse) }) @@ -437,21 +439,32 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi new ByteBufferMessageSet(compressionCodec = targetCodec, offsetCounter = offsetCounter, messageSetTimestampAssignor = messageSetTimestampAssignor, + timestampType = messageTimestampType, messages = validatedMessages.toBuffer: _*) } else { - // Do not do re-compression but simply update the offset and timestamp field of the wrapper message. + // 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) if (magicValueToUse > Message.MagicValue_V0) { 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.CreateTime) { - if (buffer.getLong(MessageSet.LogOverhead + Message.TimestampOffset) == maxTimestamp) + if (timestamp == maxTimestamp && TimestampType.getTimestampType(attributes) == TimestampType.CreateTime) // We don't need to recompute crc if the timestamp is not updated. crcUpdateNeeded = false - else - buffer.putLong(MessageSet.LogOverhead + Message.TimestampOffset, maxTimestamp) - } else if (messageTimestampType == TimestampType.LogAppendTime) - buffer.putLong(MessageSet.LogOverhead + Message.TimestampOffset, now) + else { + // Set timestamp type and timestamp + buffer.putLong(timestampOffset, maxTimestamp) + buffer.put(attributeOffset, TimestampType.setTimestampType(attributes, TimestampType.CreateTime)) + } + } else if (messageTimestampType == TimestampType.LogAppendTime) { + // Set timestamp type and timestamp + buffer.putLong(timestampOffset, now) + buffer.put(attributeOffset, TimestampType.setTimestampType(attributes, TimestampType.LogAppendTime)) + } if (crcUpdateNeeded) { // need to recompute the crc value @@ -468,6 +481,8 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi private def convertNonCompressedMessages(offsetCounter: AtomicLong, compactedTopic: Boolean, + now: Long, + timestampType: TimestampType, toMagicValue: Byte): ByteBufferMessageSet = { // Get message count, shallow iterator is in-place val sizeInBytesAfterConversion = shallowValidBytes + this.internalIterator(isShallow = true).foldLeft(0)( @@ -486,11 +501,8 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi // Create new message buffer val newMessageBuffer = newBuffer.slice() newMessageBuffer.limit(newMessageSize) - // Convert message if necessary - if (message.magic != toMagicValue) - message.convertToBuffer(toMagicValue, newMessageBuffer) - else - newMessageBuffer.put(message.buffer.array(), message.buffer.arrayOffset(), message.size) + // Convert message + message.convertToBuffer(toMagicValue, newMessageBuffer, now, timestampType) newMessagePosition += MessageSet.LogOverhead + newMessageSize } @@ -515,10 +527,12 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi val message = new Message(messageBuffer) validateMessageKey(message, compactedTopic) if (message.magic > Message.MagicValue_V0) { - validateTimestamps(message, now, timestampType, timestampDiffMaxMs, NoCompressionCodec) - // update CRC for the message if the timestamp is LogAppendTime - if (timestampType == TimestampType.LogAppendTime) - Utils.writeUnsignedInt(messageBuffer, Message.CrcOffset, message.computeChecksum()) + validateTimestamp(message, now, timestampType, timestampDiffMaxMs) + if (timestampType == TimestampType.LogAppendTime) { + message.buffer.putLong(Message.TimestampOffset, now) + // We have to update crc after updating the timestamp. + Utils.writeUnsignedInt(message.buffer, Message.CrcOffset, message.computeChecksum()) + } } messagePosition += MessageSet.LogOverhead + messageSize } @@ -534,42 +548,16 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi /** * This method validates the timestamps of a message. * 1. If the message is using create time, this method checks if it is with acceptable range. - * 2. If the message is using log append time, this method will overwrite the timestamp of the message. - * - * The return value of this method indicates whether the timestamp of a message is overwritten. - * In case 1, the method either throw exception (the timestamp is out of range) or return false. - * In case 2, the method returns: - * a) True, if the timestamp of the message has been overwritten. This means re-compression is required if the - * the message is an inner message of a compressed message. - * b) False, if the timestamp is not overwritten. This only occurs when the timestamp of an inner message from a - * compressed message is already set to [[Message.InheritedTimestamp]] + * 2. If the message is using log append time and is an uncompressed message, this method will overwrite the + * timestamp of the message. */ - private def validateTimestamps(message: Message, - now: Long, - timestampType: TimestampType, - timestampDiffMaxMs: Long, - compressionCodec: CompressionCodec): Boolean = { - var overwritten = false + private def validateTimestamp(message: Message, + now: Long, + timestampType: TimestampType, + timestampDiffMaxMs: Long) { if (timestampType == TimestampType.CreateTime && math.abs(message.timestamp - now) > timestampDiffMaxMs) throw new InvalidMessageException(s"Timestamp ${message.timestamp} of message is out of range. " + s"The timestamp should be within [${now - timestampDiffMaxMs}, ${now + timestampDiffMaxMs}") - else if (timestampType == TimestampType.LogAppendTime) { - if (compressionCodec == NoCompressionCodec) { - message.buffer.putLong(Message.TimestampOffset, now) - // We have to update crc after updating the timestamp. - Utils.writeUnsignedInt(message.buffer, Message.CrcOffset, message.computeChecksum()) - overwritten = true - } else { - // We only overwrite the timestamp if necessary - if (message.timestamp != Message.InheritedTimestamp) { - message.buffer.putLong(Message.TimestampOffset, Message.InheritedTimestamp) - // We have to update crc after updating the timestamp. - Utils.writeUnsignedInt(message.buffer, Message.CrcOffset, message.computeChecksum()) - overwritten = true - } - } - } - overwritten } /** diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala index cb6894c9da97d..840b346136534 100755 --- a/core/src/main/scala/kafka/message/Message.scala +++ b/core/src/main/scala/kafka/message/Message.scala @@ -18,6 +18,8 @@ package kafka.message import java.nio._ +import kafka.message.Message.TimestampType.TimestampType + import scala.math._ import kafka.utils._ import org.apache.kafka.common.utils.Utils @@ -31,6 +33,21 @@ object Message { type TimestampType = Value val CreateTime = Value(0, "CreateTime") val LogAppendTime = Value(1, "LogAppendTime") + + def getTimestampType(attribute: Byte) = { + (attribute & TimestampTypeMask) >> TimestampTypeAttributeBitOffset match { + case 0 => CreateTime + case 1 => LogAppendTime + } + } + + def setTimestampType(attribute: Byte, timestampType: TimestampType): Byte = { + if (timestampType == CreateTime) + (attribute & ~TimestampTypeMask).toByte + else + (attribute | TimestampTypeMask).toByte + } + } /** @@ -85,10 +102,16 @@ object Message { val CurrentMagicValue: Byte = 1 /** - * Specifies the mask for the compression code. 3 bits to hold the compression codec. + * Specifies the mask for the compression code. 4 bits to hold the compression codec. * 0 is reserved to indicate no compression */ - val CompressionCodeMask: Int = 0x07 + val CompressionCodeMask: Int = 0x0F + /** + * Specifies the mask for timestmap type. 1 bit at the 5th least significant bit. + * 0 for CreateTime, 1 for LogAppendTime + */ + val TimestampTypeMask: Byte = 0x10 + val TimestampTypeAttributeBitOffset: Int = 4 /** * Compression code for uncompressed messages @@ -98,19 +121,15 @@ object Message { /** * To indicate timestamp is not defined so "magic" value 0 will be used. */ - val NoTimestamp: Long = -2 - - /** - * Derived timestamp is used for inner messages of compressed message when message.timestamp.type is set to - * LogAppendTime. In that case the timestamp of the message is defined by the wrapper message's timestamp. - */ - val InheritedTimestamp: Long = -1 + val NoTimestamp: Long = -1 /** * Give the header size difference between different message versions. */ def headerSizeDiff(fromMagicValue: Byte, toMagicValue: Byte) : Int = MessageHeaderSizeMap(toMagicValue) - MessageHeaderSizeMap(fromMagicValue) + + } /** @@ -127,7 +146,9 @@ object Message { * * Default constructor wraps an existing ByteBuffer with the Message object with no change to the contents. */ -class Message(val buffer: ByteBuffer, val wrapperMessageTimestamp: Long = Message.NoTimestamp) { +class Message(val buffer: ByteBuffer, + private val wrapperMessageTimestamp: Option[Long] = None, + private val timestampTypeToUse: Option[TimestampType] = None) { import kafka.message.Message._ @@ -279,15 +300,34 @@ class Message(val buffer: ByteBuffer, val wrapperMessageTimestamp: Long = Messag /** * The timestamp of the message, only available when the "magic" value is greater than 0 */ - def timestamp: Long = { +def timestamp: Long = { if (magic == MagicValue_V0) throw new IllegalStateException("The message with magic byte 0 does not have a timestamp") - val timestamp = buffer.getLong(TimestampOffset) - if (timestamp == Message.InheritedTimestamp) - wrapperMessageTimestamp - else - timestamp + /** + * The timestamp of a message is determined in the following way: + * 1. TimestampType = LogAppendTime and + * a) WrapperMessageTimestamp is not defined - Uncompressed message using LogAppendTime + * b) WrapperMessageTimestamp is defined - Compressed message using LogAppendTime + * 2. TimestampType = CreateTime and + * a) WrapperMessageTimestamp is not defined - Uncompressed message using CreateTime + * b) WrapperMessageTimestamp is defined - Compressed message using CreateTime + */ + // Case 1b + if (timestampTypeToUse.exists(_ == TimestampType.LogAppendTime) && wrapperMessageTimestamp.isDefined) + wrapperMessageTimestamp.get + else // case 1b, 2a, 2b + buffer.getLong(Message.TimestampOffset) + } + + /** + * The timestamp type of the message + */ + def timestampType = { + if (magic == MagicValue_V0) + throw new IllegalStateException("The message with magic byte 0 does not have a timestamp") + + timestampTypeToUse.getOrElse(TimestampType.getTimestampType(attributes)) } /** @@ -320,23 +360,29 @@ class Message(val buffer: ByteBuffer, val wrapperMessageTimestamp: Long = Messag } } - def convertToBuffer(toMagicValue: Byte, byteBuffer: ByteBuffer) { + def convertToBuffer(toMagicValue: Byte, + byteBuffer: ByteBuffer, + now: Long = NoTimestamp, + timestampType: TimestampType = timestampTypeToUse.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 (magic == Message.MagicValue_V0 && toMagicValue == Message.MagicValue_V1) { + if (toMagicValue == Message.MagicValue_V1) { // Up-conversion, reserve CRC and update magic byte byteBuffer.position(Message.MagicOffset) byteBuffer.put(Message.MagicValue_V1) - byteBuffer.put(attributes) + byteBuffer.put(TimestampType.setTimestampType(attributes, timestampType)) // Up-conversion, insert the timestamp field - byteBuffer.putLong(Message.NoTimestamp) + if (timestampType == TimestampType.LogAppendTime) + 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(attributes) + byteBuffer.put(TimestampType.setTimestampType(attributes, TimestampType.CreateTime)) // Down-conversion, skip the timestamp field byteBuffer.put(buffer.array(), buffer.arrayOffset() + Message.KeySizeOffset_V1, size - Message.KeySizeOffset_V1) } @@ -369,7 +415,7 @@ class Message(val buffer: ByteBuffer, val wrapperMessageTimestamp: Long = Messag 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 && timestamp != InheritedTimestamp) + 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}") @@ -379,7 +425,7 @@ class Message(val buffer: ByteBuffer, val wrapperMessageTimestamp: Long = Messag if (magic == MagicValue_V0) s"Message(magic = $magic, attributes = $attributes, crc = $checksum, key = $key, payload = $payload)" else - s"Message(magic = $magic, attributes = $attributes, timestamp = $timestamp, crc = $checksum, key = $key, payload = $payload)" + s"Message(magic = $magic, attributes = $attributes, $timestampType = $timestamp, crc = $checksum, key = $key, payload = $payload)" } override def equals(any: Any): Boolean = { @@ -390,5 +436,5 @@ class Message(val buffer: ByteBuffer, val wrapperMessageTimestamp: Long = Messag } override def hashCode(): Int = buffer.hashCode - + } diff --git a/core/src/main/scala/kafka/message/MessageSet.scala b/core/src/main/scala/kafka/message/MessageSet.scala index 84107d6736883..8d6e04626a78f 100644 --- a/core/src/main/scala/kafka/message/MessageSet.scala +++ b/core/src/main/scala/kafka/message/MessageSet.scala @@ -20,6 +20,10 @@ package kafka.message import java.nio._ import java.nio.channels._ +import kafka.log.FileMessageSet + +import scala.collection.mutable.ArrayBuffer + /** * Message set helper functions */ @@ -60,11 +64,8 @@ object MessageSet { */ def validateMagicValuesAndGetTimestamp(messages: Seq[Message]): Long = { val sampleMagicValue = messages.head.magic - val sampleTimestamp = if (sampleMagicValue > Message.MagicValue_V0) messages.head.timestamp else Message.NoTimestamp var largestTimestamp: Long = Message.NoTimestamp for (message <- messages) { - if (sampleTimestamp == Message.InheritedTimestamp && message.timestamp != sampleTimestamp) - throw new IllegalStateException("Messages in the same compressed message set should have same timestamp type") if (message.magic != sampleMagicValue) throw new IllegalStateException("Messages in the same compressed message set must have same magic value") if (sampleMagicValue > Message.MagicValue_V0) @@ -105,6 +106,34 @@ abstract class MessageSet extends Iterable[MessageAndOffset] { */ def sizeInBytes: Int + /** + * 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 + // File message set only has shallow iterator. We need to do deep iteration here if needed. + if (message.compressionCodec == NoCompressionCodec || !this.isInstanceOf[FileMessageSet]) { + newMessages += messageAndOffset.message.toFormatVersion(toMagicValue) + offsets += messageAndOffset.offset + } else { + 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: _*) + } + /** * Print this message set's contents. If the message set has more than 100 messages, just * print the first 100. diff --git a/core/src/main/scala/kafka/message/MessageWriter.scala b/core/src/main/scala/kafka/message/MessageWriter.scala index cc8ee38b36fb3..664a9f7f537d1 100755 --- a/core/src/main/scala/kafka/message/MessageWriter.scala +++ b/core/src/main/scala/kafka/message/MessageWriter.scala @@ -20,19 +20,29 @@ package kafka.message import java.io.{InputStream, OutputStream} import java.nio.ByteBuffer +import kafka.message.Message.TimestampType.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, timestamp: Long, magicValue: Byte)(writePayload: OutputStream => Unit): Unit = { + def write(key: Array[Byte] = null, + codec: CompressionCodec, + timestamp: Long, + timestampType: TimestampType, + magicValue: Byte)(writePayload: OutputStream => Unit): Unit = { withCrc32Prefix { + // 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 diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index fa4437addb23a..7c9077bb2d05d 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -299,6 +299,7 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { assertEquals(tp.topic(), record.topic()) assertEquals(tp.partition(), record.partition()) if (timestampType == TimestampType.CreateTime) { + assertEquals(timestampType, record.timestampType()) val timestamp = startingTimestamp + i assertEquals(timestamp.toLong, record.timestamp()) } else diff --git a/core/src/test/scala/unit/kafka/message/MessageTest.scala b/core/src/test/scala/unit/kafka/message/MessageTest.scala index 16b6f5471c08a..146b08ab5d669 100755 --- a/core/src/test/scala/unit/kafka/message/MessageTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageTest.scala @@ -164,7 +164,7 @@ class MessageTest extends JUnitSuite { @Test(expected = classOf[IllegalArgumentException]) def testInValidTimestampAndMagicValueCombination() { - new Message("hello".getBytes, Message.InheritedTimestamp, Message.MagicValue_V0) + new Message("hello".getBytes, 0L, Message.MagicValue_V0) } @Test(expected = classOf[IllegalArgumentException]) @@ -174,7 +174,7 @@ class MessageTest extends JUnitSuite { @Test(expected = classOf[IllegalArgumentException]) def testInValidMagicByte() { - new Message("hello".getBytes, 0L, 2) + new Message("hello".getBytes, 0L, 2.toByte) } @Test diff --git a/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala b/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala index d38108755e0cd..05f10544b1c01 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 kafka.message.Message.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, timestamp = Message.NoTimestamp, magicValue = Message.MagicValue_V0) { output => + writer.write(key = key, codec = codec, timestamp = Message.NoTimestamp, timestampType = TimestampType.CreateTime, magicValue = Message.MagicValue_V0) { output => val out = if (codec == NoCompressionCodec) output else CompressionFactory(codec, output) try { val p = rnd.nextInt(bytes.length) 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 a59a06969d12c..4189b07aad791 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 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.record.Record; import org.apache.kafka.streams.processor.TimestampExtractor; import java.util.ArrayDeque; @@ -76,7 +77,8 @@ public int addRawRecords(Iterable> rawRecords, Ti Object key = source.deserializeKey(rawRecord.topic(), rawRecord.key()); Object value = source.deserializeValue(rawRecord.topic(), rawRecord.value()); - ConsumerRecord record = new ConsumerRecord<>(rawRecord.topic(), rawRecord.partition(), rawRecord.offset(), rawRecord.timestamp(), key, value); + ConsumerRecord record = new ConsumerRecord<>(rawRecord.topic(), rawRecord.partition(), + rawRecord.offset(), rawRecord.timestamp(), Record.TimestampType.CreateTime, key, value); long timestamp = timestampExtractor.extract(record); StampedRecord stampedRecord = new StampedRecord(record, timestamp); 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 fac9f2cbb66c1..a7de588ba9a7b 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 org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.record.Record; 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 void testTimeTracking() { // add three 3 records with timestamp 1, 3, 5 to partition-1 List> list1 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 1, 0L, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 3, 0L, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 5, 0L, recordKey, recordValue)); + new ConsumerRecord<>("topic", 1, 1, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 3, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 5, 0L, Record.TimestampType.CreateTime, recordKey, recordValue)); group.addRawRecords(partition1, list1); // add three 3 records with timestamp 2, 4, 6 to partition-2 List> list2 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 2, 0L, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 4, 0L, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 6, 0L, recordKey, recordValue)); + new ConsumerRecord<>("topic", 1, 2, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 4, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 6, 0L, Record.TimestampType.CreateTime, recordKey, recordValue)); group.addRawRecords(partition2, list2); 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 89b66a94224ed..be6b8ce75d53b 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.Node; 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.IntegerSerializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.Utils; @@ -84,6 +85,7 @@ public void reset() { public void bufferRecord(ConsumerRecord record) { recordBuffer.add( new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), 0L, + Record.TimestampType.CreateTime, serializer.serialize(record.topic(), record.key()), serializer.serialize(record.topic(), record.value()))); endOffset = record.offset(); @@ -267,7 +269,7 @@ public void testRegisterPersistentStore() throws IOException { int key = i * 10; expectedKeys.add(key); restoreConsumer.bufferRecord( - new ConsumerRecord<>(persistentStoreTopicName, 2, 0L, offset, key, 0) + new ConsumerRecord<>(persistentStoreTopicName, 2, 0L, offset, Record.TimestampType.CreateTime, key, 0) ); } @@ -320,7 +322,7 @@ public void testRegisterNonPersistentStore() throws IOException { int key = i; expectedKeys.add(i); restoreConsumer.bufferRecord( - new ConsumerRecord<>(nonPersistentStoreTopicName, 2, 0L, offset, key, 0) + new ConsumerRecord<>(nonPersistentStoreTopicName, 2, 0L, offset, Record.TimestampType.CreateTime, key, 0) ); } 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 284ba5d754191..9d98cef8dedd9 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 org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.record.Record; 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 void testTimeTracking() { // add three 3 out-of-order records with timestamp 2, 1, 3 List> list1 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 2, 0L, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 1, 0L, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 3, 0L, recordKey, recordValue)); + new ConsumerRecord<>("topic", 1, 2, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 1, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 3, 0L, Record.TimestampType.CreateTime, recordKey, recordValue)); queue.addRawRecords(list1, timestampExtractor); @@ -72,9 +73,9 @@ public void testTimeTracking() { // add three 3 out-of-order records with timestamp 4, 1, 2 // now with 3, 4, 1, 2 List> list2 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 4, 0L, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 1, 0L, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 2, 0L, recordKey, recordValue)); + new ConsumerRecord<>("topic", 1, 4, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 1, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 2, 0L, Record.TimestampType.CreateTime, recordKey, recordValue)); queue.addRawRecords(list2, timestampExtractor); @@ -99,9 +100,9 @@ public void testTimeTracking() { // add three more records with 4, 5, 6 List> list3 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 4, 0L, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 5, 0L, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 6, 0L, recordKey, recordValue)); + new ConsumerRecord<>("topic", 1, 4, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 5, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 6, 0L, Record.TimestampType.CreateTime, recordKey, recordValue)); queue.addRawRecords(list3, timestampExtractor); 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 dbe7fd3dc3396..6ac88c684063c 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.common.Node; 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.IntegerSerializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.Utils; @@ -152,7 +153,7 @@ public void testUpdateNonPersistentStore() throws Exception { restoreStateConsumer.assign(new ArrayList<>(task.changeLogPartitions())); task.update(partition1, - records(new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, 0L, recordKey, recordValue)) + records(new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, 0L, Record.TimestampType.CreateTime, recordKey, recordValue)) ); } finally { @@ -171,9 +172,9 @@ public void testUpdate() throws Exception { restoreStateConsumer.assign(new ArrayList<>(task.changeLogPartitions())); for (ConsumerRecord record : Arrays.asList( - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 10, 0L, 1, 100), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 20, 0L, 2, 100), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 30, 0L, 3, 100))) { + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 10, 0L, Record.TimestampType.CreateTime, 1, 100), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 20, 0L, Record.TimestampType.CreateTime, 2, 100), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 30, 0L, Record.TimestampType.CreateTime, 3, 100))) { restoreStateConsumer.bufferRecord(record); } @@ -234,11 +235,11 @@ public void testUpdateKTable() throws Exception { restoreStateConsumer.assign(new ArrayList<>(task.changeLogPartitions())); for (ConsumerRecord record : Arrays.asList( - new ConsumerRecord<>(ktable.topic(), ktable.partition(), 10, 0L, 1, 100), - new ConsumerRecord<>(ktable.topic(), ktable.partition(), 20, 0L, 2, 100), - new ConsumerRecord<>(ktable.topic(), ktable.partition(), 30, 0L, 3, 100), - new ConsumerRecord<>(ktable.topic(), ktable.partition(), 40, 0L, 4, 100), - new ConsumerRecord<>(ktable.topic(), ktable.partition(), 50, 0L, 5, 100))) { + new ConsumerRecord<>(ktable.topic(), ktable.partition(), 10, 0L, Record.TimestampType.CreateTime, 1, 100), + new ConsumerRecord<>(ktable.topic(), ktable.partition(), 20, 0L, Record.TimestampType.CreateTime, 2, 100), + new ConsumerRecord<>(ktable.topic(), ktable.partition(), 30, 0L, Record.TimestampType.CreateTime, 3, 100), + new ConsumerRecord<>(ktable.topic(), ktable.partition(), 40, 0L, Record.TimestampType.CreateTime, 4, 100), + new ConsumerRecord<>(ktable.topic(), ktable.partition(), 50, 0L, Record.TimestampType.CreateTime, 5, 100))) { restoreStateConsumer.bufferRecord(record); } 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 b36304c052787..9703e1339b2dc 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.OffsetResetStrategy; import org.apache.kafka.clients.producer.MockProducer; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.record.Record; 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 void testProcessOrder() throws Exception { 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, 0L, recordKey, recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, recordKey, recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, 0L, recordKey, recordValue) + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, 0L, Record.TimestampType.CreateTime, recordKey, recordValue) )); task.addRecords(partition2, records( - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 25, 0L, recordKey, recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, recordKey, recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, recordKey, recordValue) + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 25, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, Record.TimestampType.CreateTime, recordKey, recordValue) )); assertEquals(5, task.process()); @@ -158,15 +159,15 @@ public void testPauseResume() throws Exception { 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, 0L, recordKey, recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, recordKey, recordValue) + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, Record.TimestampType.CreateTime, recordKey, recordValue) )); task.addRecords(partition2, records( - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, recordKey, recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, recordKey, recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 55, 0L, recordKey, recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 65, 0L, recordKey, recordValue) + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 55, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 65, 0L, Record.TimestampType.CreateTime, recordKey, recordValue) )); assertEquals(5, task.process()); @@ -177,9 +178,9 @@ public void testPauseResume() throws Exception { assertTrue(consumer.paused().contains(partition2)); task.addRecords(partition1, records( - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, 0L, recordKey, recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 40, 0L, recordKey, recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 50, 0L, recordKey, recordValue) + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 40, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 50, 0L, Record.TimestampType.CreateTime, recordKey, recordValue) )); assertEquals(2, consumer.paused().size()); 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 36b61a76940c7..2e765765e87a9 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.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.record.Record; 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 void process(String topicName, byte[] key, byte[] value) { } // Add the record ... long offset = offsetsByTopicPartition.get(tp).incrementAndGet(); - task.addRecords(tp, records(new ConsumerRecord(tp.topic(), tp.partition(), offset, 0L, key, value))); + task.addRecords(tp, records(new ConsumerRecord(tp.topic(), tp.partition(), offset, 0L, Record.TimestampType.CreateTime, key, value))); producer.clear(); // Process the record ... task.process(); From e76aa20a250674c08f91bf19089dd3c733b4bc29 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Wed, 27 Jan 2016 10:44:44 -0800 Subject: [PATCH 06/33] Fixed issue where timestmap type is not updated for non-compressed message without format conversion. --- .../scala/kafka/message/ByteBufferMessageSet.scala | 1 + .../kafka/message/ByteBufferMessageSetTest.scala | 13 +++++++++++-- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 0ccdb1cc48e63..bd74ad2163a4a 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -530,6 +530,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi validateTimestamp(message, now, timestampType, timestampDiffMaxMs) if (timestampType == TimestampType.LogAppendTime) { message.buffer.putLong(Message.TimestampOffset, now) + message.buffer.put(Message.AttributesOffset, TimestampType.setTimestampType(message.attributes, TimestampType.LogAppendTime)) // We have to update crc after updating the timestamp. Utils.writeUnsignedInt(message.buffer, Message.CrcOffset, message.computeChecksum()) } diff --git a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala index 990da7d4d38db..062695f88e826 100644 --- a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala @@ -188,6 +188,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { for (messageAndOffset <- validatedMessages) { messageAndOffset.message.ensureValid() assertTrue(messageAndOffset.message.timestamp >= startTime && messageAndOffset.message.timestamp <= now) + assertEquals(TimestampType.LogAppendTime, messageAndOffset.message.timestampType) } assertEquals("message set size should not change", compressedMessages.size, validatedCompressedMessages.size) @@ -195,6 +196,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { messageAndOffset.message.ensureValid() assertTrue(s"Timestamp of message ${messageAndOffset.message}} should be between $startTime and $now", messageAndOffset.message.timestamp >= startTime && messageAndOffset.message.timestamp <= now) + assertEquals(TimestampType.LogAppendTime, messageAndOffset.message.timestampType) } assertTrue("MessageSet should still valid", validatedCompressedMessages.shallowIterator.next().message.isValid) @@ -204,6 +206,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { messageAndOffset.message.ensureValid() assertTrue(s"Timestamp of message ${messageAndOffset.message}} should be between $startTime and $now", messageAndOffset.message.timestamp >= startTime && messageAndOffset.message.timestamp <= now) + assertEquals(TimestampType.LogAppendTime, messageAndOffset.message.timestampType) } assertTrue("MessageSet should still valid", validatedCompressedMessagesWithoutRecompression.shallowIterator.next().message.isValid) } @@ -229,10 +232,16 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { messageTimestampType = TimestampType.CreateTime, messageTimestampDiffMaxMs = 1000L) - for (messageAndOffset <- validatedMessages) + for (messageAndOffset <- validatedMessages) { + messageAndOffset.message.ensureValid() assertEquals(messageAndOffset.message.timestamp, now) - for (messageAndOffset <- validatedCompressedMessages) + assertEquals(messageAndOffset.message.timestampType, TimestampType.CreateTime) + } + for (messageAndOffset <- validatedCompressedMessages) { + messageAndOffset.message.ensureValid() assertEquals(messageAndOffset.message.timestamp, now) + assertEquals(messageAndOffset.message.timestampType, TimestampType.CreateTime) + } } @Test From 60108b08d01ea3917a5dcb788c1e9dad32fd373d Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Wed, 27 Jan 2016 18:55:06 -0800 Subject: [PATCH 07/33] Change the compression codec bits back to 3 --- .../main/java/org/apache/kafka/common/record/Record.java | 6 +++--- core/src/main/scala/kafka/message/Message.scala | 8 ++++---- 2 files changed, 7 insertions(+), 7 deletions(-) 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 66ec87d356e37..e04e7a5abebd6 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 @@ -91,14 +91,14 @@ public static byte setTimestampType(byte attributes, TimestampType timestampType * Specifies the mask for the compression code. 4 bits to hold the compression codec. 0 is reserved to indicate no * compression */ - public static final int COMPRESSION_CODEC_MASK = 0x0F; + 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 = 0x10; - public static final int TIMESTAMP_TYPE_ATTRIBUTE_OFFSET = 4; + public static final byte TIMESTAMP_TYPE_MASK = 0x08; + public static final int TIMESTAMP_TYPE_ATTRIBUTE_OFFSET = 3; /** * Compression code for uncompressed records diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala index 840b346136534..6e82513b201e6 100755 --- a/core/src/main/scala/kafka/message/Message.scala +++ b/core/src/main/scala/kafka/message/Message.scala @@ -105,13 +105,13 @@ object Message { * Specifies the mask for the compression code. 4 bits to hold the compression codec. * 0 is reserved to indicate no compression */ - val CompressionCodeMask: Int = 0x0F + val CompressionCodeMask: Int = 0x07 /** - * Specifies the mask for timestmap type. 1 bit at the 5th least significant bit. + * Specifies the mask for timestamp type. 1 bit at the 5th least significant bit. * 0 for CreateTime, 1 for LogAppendTime */ - val TimestampTypeMask: Byte = 0x10 - val TimestampTypeAttributeBitOffset: Int = 4 + val TimestampTypeMask: Byte = 0x08 + val TimestampTypeAttributeBitOffset: Int = 3 /** * Compression code for uncompressed messages From 8606c41b23a3275331fa6216aaf81b4d75641bcf Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Wed, 27 Jan 2016 22:16:46 -0800 Subject: [PATCH 08/33] Extracted TimestampType out of Record and Message. --- .../clients/consumer/ConsumerRecord.java | 8 ++-- .../clients/consumer/internals/Fetcher.java | 4 +- .../apache/kafka/common/record/Record.java | 21 ---------- .../kafka/common/record/TimestampType.java | 42 +++++++++++++++++++ .../clients/consumer/MockConsumerTest.java | 6 +-- core/src/main/scala/kafka/log/Log.scala | 1 - core/src/main/scala/kafka/log/LogConfig.scala | 4 +- .../kafka/message/ByteBufferMessageSet.scala | 3 +- .../main/scala/kafka/message/Message.scala | 23 +--------- .../scala/kafka/message/MessageWriter.scala | 2 +- .../scala/kafka/message/TimestampType.scala | 42 +++++++++++++++++++ .../main/scala/kafka/server/KafkaConfig.scala | 1 - .../kafka/api/BaseConsumerTest.scala | 4 +- .../kafka/api/BaseProducerSendTest.scala | 2 +- .../kafka/api/PlaintextConsumerTest.scala | 5 ++- .../message/ByteBufferMessageSetTest.scala | 1 - .../kafka/message/MessageWriterTest.scala | 1 - 17 files changed, 103 insertions(+), 67 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/record/TimestampType.java create mode 100644 core/src/main/scala/kafka/message/TimestampType.scala 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 160c7f46bd127..91f2ebe2093cd 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,7 +12,7 @@ */ package org.apache.kafka.clients.consumer; -import org.apache.kafka.common.record.Record; +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 @@ -23,7 +23,7 @@ public final class ConsumerRecord { private final int partition; private final long offset; private final long timestamp; - private final Record.TimestampType timestampType; + private final TimestampType timestampType; private final K key; private final V value; @@ -41,7 +41,7 @@ public ConsumerRecord(String topic, int partition, long offset, long timestamp, - Record.TimestampType timestampType, + TimestampType timestampType, K key, V value) { if (topic == null) @@ -94,7 +94,7 @@ public long timestamp() { return timestamp; } - public Record.TimestampType timestampType() { + public TimestampType timestampType() { return timestampType; } 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 8027f1b1fd83f..427664ac6e984 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,7 +40,7 @@ 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.Record; +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; @@ -616,7 +616,7 @@ private ConsumerRecord parseRecord(TopicPartition partition, LogEntry logE logEntry.record().ensureValid(); long offset = logEntry.offset(); long timestamp = logEntry.record().timestamp(); - Record.TimestampType timestampType = logEntry.record().timestampType(); + 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(); 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 e04e7a5abebd6..1253d5231b8f7 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 @@ -27,27 +27,6 @@ */ public final class Record { - public enum TimestampType { - CreateTime(0, "CreateTime"), LogAppendTime(1, "LogAppendTime"); - - public int value; - public String name; - TimestampType(int value, String name) { - this.value = value; - this.name = name; - } - - public static TimestampType getTimestampType(byte attributes) { - int timestampType = (attributes & TIMESTAMP_TYPE_MASK) >> TIMESTAMP_TYPE_ATTRIBUTE_OFFSET; - return timestampType == 0 ? CreateTime : LogAppendTime; - } - - public static byte setTimestampType(byte attributes, TimestampType timestampType) { - return timestampType == CreateTime ? - (byte) (attributes & ~TIMESTAMP_TYPE_MASK) : (byte) (attributes | TIMESTAMP_TYPE_MASK); - } - } - /** * The current offset and size for all the fixed-length fields */ 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 0000000000000..48fb58f6789ad --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java @@ -0,0 +1,42 @@ +/** + * 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; + +/** + * The timestamp type of the records. + */ +public enum TimestampType { + CreateTime(0, "CreateTime"), LogAppendTime(1, "LogAppendTime"); + + public int value; + public 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 ? CreateTime : LogAppendTime; + } + + public static byte setTimestampType(byte attributes, TimestampType timestampType) { + return timestampType == CreateTime ? + (byte) (attributes & ~Record.TIMESTAMP_TYPE_MASK) : (byte) (attributes | Record.TIMESTAMP_TYPE_MASK); + } +} 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 915762ff0b37b..af8aa50d37765 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,7 +18,7 @@ import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.record.Record; +import org.apache.kafka.common.record.TimestampType; import org.junit.Test; import java.util.Arrays; @@ -43,8 +43,8 @@ public void testSimpleMock() { beginningOffsets.put(new TopicPartition("test", 1), 0L); consumer.updateBeginningOffsets(beginningOffsets); consumer.seek(new TopicPartition("test", 0), 0); - ConsumerRecord rec1 = new ConsumerRecord("test", 0, 0, 0L, Record.TimestampType.CreateTime, "key1", "value1"); - ConsumerRecord rec2 = new ConsumerRecord("test", 0, 1, 0L, Record.TimestampType.CreateTime, "key2", "value2"); + ConsumerRecord rec1 = new ConsumerRecord("test", 0, 0, 0L, TimestampType.CreateTime, "key1", "value1"); + ConsumerRecord rec2 = new ConsumerRecord("test", 0, 1, 0L, TimestampType.CreateTime, "key2", "value2"); consumer.addRecord(rec1); consumer.addRecord(rec2); ConsumerRecords recs = consumer.poll(1); diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 277dfe81838df..a5e3acee4e72a 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -17,7 +17,6 @@ package kafka.log -import kafka.message.Message.TimestampType import kafka.utils._ import kafka.message._ import kafka.common._ diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index 8709c5f2e3692..651caad4e9db2 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -19,13 +19,11 @@ package kafka.log import java.util.Properties import kafka.api.ApiVersion -import kafka.message.Message.TimestampType 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 kafka.message.{TimestampType, BrokerCompressionCodec, Message} object Defaults { val SegmentSize = kafka.server.Defaults.LogSegmentBytes diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index bd74ad2163a4a..55a3123ef486a 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -18,8 +18,7 @@ package kafka.message import kafka.api.{KAFKA_0_10_0_DV0, ApiVersion} -import kafka.message.Message.TimestampType -import kafka.message.Message.TimestampType.TimestampType +import kafka.message.TimestampType.TimestampType import kafka.utils.{IteratorTemplate, Logging} import kafka.common.KafkaException diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala index 6e82513b201e6..78b500719087e 100755 --- a/core/src/main/scala/kafka/message/Message.scala +++ b/core/src/main/scala/kafka/message/Message.scala @@ -18,7 +18,7 @@ package kafka.message import java.nio._ -import kafka.message.Message.TimestampType.TimestampType +import kafka.message.TimestampType.TimestampType import scala.math._ import kafka.utils._ @@ -29,27 +29,6 @@ import org.apache.kafka.common.utils.Utils */ object Message { - case object TimestampType extends Enumeration { - type TimestampType = Value - val CreateTime = Value(0, "CreateTime") - val LogAppendTime = Value(1, "LogAppendTime") - - def getTimestampType(attribute: Byte) = { - (attribute & TimestampTypeMask) >> TimestampTypeAttributeBitOffset match { - case 0 => CreateTime - case 1 => LogAppendTime - } - } - - def setTimestampType(attribute: Byte, timestampType: TimestampType): Byte = { - if (timestampType == CreateTime) - (attribute & ~TimestampTypeMask).toByte - else - (attribute | TimestampTypeMask).toByte - } - - } - /** * The current offset and size for all the fixed-length fields */ diff --git a/core/src/main/scala/kafka/message/MessageWriter.scala b/core/src/main/scala/kafka/message/MessageWriter.scala index 664a9f7f537d1..01ec5c6186730 100755 --- a/core/src/main/scala/kafka/message/MessageWriter.scala +++ b/core/src/main/scala/kafka/message/MessageWriter.scala @@ -20,7 +20,7 @@ package kafka.message import java.io.{InputStream, OutputStream} import java.nio.ByteBuffer -import kafka.message.Message.TimestampType.TimestampType +import kafka.message.TimestampType.TimestampType import org.apache.kafka.common.utils.Crc32 class MessageWriter(segmentSize: Int) extends BufferingOutputStream(segmentSize) { diff --git a/core/src/main/scala/kafka/message/TimestampType.scala b/core/src/main/scala/kafka/message/TimestampType.scala new file mode 100644 index 0000000000000..a959a22e7c6c6 --- /dev/null +++ b/core/src/main/scala/kafka/message/TimestampType.scala @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.message + +/** + * The timestamp type of the messages. + */ +case object TimestampType extends Enumeration { + type TimestampType = Value + val CreateTime = Value(0, "CreateTime") + val LogAppendTime = Value(1, "LogAppendTime") + + def getTimestampType(attribute: Byte) = { + (attribute & Message.TimestampTypeMask) >> Message.TimestampTypeAttributeBitOffset match { + case 0 => CreateTime + case 1 => LogAppendTime + } + } + + def setTimestampType(attribute: Byte, timestampType: TimestampType): Byte = { + if (timestampType == CreateTime) + (attribute & ~Message.TimestampTypeMask).toByte + else + (attribute | Message.TimestampTypeMask).toByte + } + +} diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 78fb9722e7ae3..408d84bb90201 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -23,7 +23,6 @@ import kafka.api.ApiVersion import kafka.cluster.EndPoint import kafka.consumer.ConsumerConfig import kafka.coordinator.OffsetConfig -import kafka.message.Message.TimestampType import kafka.message.{BrokerCompressionCodec, CompressionCodec, Message, MessageSet} import kafka.utils.CoreUtils import org.apache.kafka.clients.CommonClientConfigs diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index 7c9077bb2d05d..45631eacbf812 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -15,8 +15,8 @@ package kafka.api import java.util import org.apache.kafka.clients.consumer._ -import org.apache.kafka.clients.producer.{Producer, ProducerConfig, ProducerRecord} -import org.apache.kafka.common.record.Record.TimestampType +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} diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala index ebe4693306bd3..a8be84f4530ee 100644 --- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala @@ -28,7 +28,7 @@ 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.record.Record.TimestampType +import org.apache.kafka.common.record.TimestampType import org.junit.Assert._ import org.junit.{After, Before, Test} import scala.collection.mutable.Buffer diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 7e152b644696a..48e85f677733c 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -24,9 +24,10 @@ import kafka.utils.TestUtils import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.producer.KafkaProducer import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer, ByteArraySerializer} -import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} +import org.apache.kafka.clients.producer.ProducerConfig import org.apache.kafka.common.record.CompressionType -import org.apache.kafka.common.record.Record.TimestampType +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.common.record.TimestampType import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.apache.kafka.common.errors.{InvalidTopicException, RecordTooLargeException} diff --git a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala index 062695f88e826..ba391559b1030 100644 --- a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala @@ -20,7 +20,6 @@ package kafka.message import java.nio._ import java.util.concurrent.atomic.AtomicLong import kafka.api.ApiVersion -import kafka.message.Message.TimestampType import org.junit.Assert._ import org.junit.Test import kafka.utils.TestUtils diff --git a/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala b/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala index 05f10544b1c01..8436b3fd4bdc6 100644 --- a/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala @@ -20,7 +20,6 @@ package kafka.message import java.io.{InputStream, ByteArrayInputStream, ByteArrayOutputStream} import java.nio.ByteBuffer import java.util.Random -import kafka.message.Message.TimestampType import org.junit.Assert._ import org.junit.Test import org.scalatest.junit.JUnitSuite From 78b093622b3926a4bf026231e488f53292a620c6 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Thu, 28 Jan 2016 09:41:26 -0800 Subject: [PATCH 09/33] Fixed connect and streams test --- .../connect/runtime/WorkerSinkTaskTest.java | 4 +-- .../runtime/WorkerSinkTaskThreadedTest.java | 6 ++-- .../storage/KafkaConfigStorageTest.java | 26 +++++++-------- .../storage/KafkaOffsetBackingStoreTest.java | 18 +++++------ .../kafka/connect/util/KafkaBasedLogTest.java | 18 +++++------ .../processor/internals/RecordQueue.java | 4 +-- .../internals/PartitionGroupTest.java | 14 ++++---- .../internals/ProcessorStateManagerTest.java | 8 ++--- .../processor/internals/RecordQueueTest.java | 20 ++++++------ .../processor/internals/StandbyTaskTest.java | 20 ++++++------ .../processor/internals/StreamTaskTest.java | 32 +++++++++---------- .../test/ProcessorTopologyTestDriver.java | 4 +-- 12 files changed, 87 insertions(+), 87 deletions(-) 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 b499f651e725c..6c8f59ca84e45 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,7 +23,7 @@ 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.Record; +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; @@ -291,7 +291,7 @@ private void expectConsumerPoll(final int numMessages) { public ConsumerRecords answer() throws Throwable { List> records = new ArrayList<>(); for (int i = 0; i < numMessages; i++) - records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned + i, 0L, Record.TimestampType.CreateTime, RAW_KEY, RAW_VALUE)); + records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned + i, 0L, TimestampType.CreateTime, RAW_KEY, RAW_VALUE)); recordsReturned += numMessages; return new ConsumerRecords<>( numMessages > 0 ? 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 cdf34cf40f719..3615c43462a2c 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,7 +24,7 @@ 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.Record; +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; @@ -520,7 +520,7 @@ public ConsumerRecords answer() throws Throwable { Collections.singletonMap( new TopicPartition(TOPIC, PARTITION), Arrays.asList( - new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, 0L, Record.TimestampType.CreateTime, RAW_KEY, RAW_VALUE) + new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, 0L, TimestampType.CreateTime, RAW_KEY, RAW_VALUE) ))); recordsReturned++; return records; @@ -548,7 +548,7 @@ public ConsumerRecords answer() throws Throwable { Collections.singletonMap( new TopicPartition(TOPIC, PARTITION), Arrays.asList( - new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, 0L, Record.TimestampType.CreateTime, RAW_KEY, RAW_VALUE) + new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, 0L, TimestampType.CreateTime, RAW_KEY, RAW_VALUE) ))); recordsReturned++; return records; 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 5645b52b58b43..3db3ff9d1f0a2 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,7 +22,7 @@ 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.Record; +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; @@ -289,14 +289,14 @@ public void testRestore() throws Exception { expectConfigure(); // Overwrite each type at least once to ensure we see the latest data after loading List> existingRecords = Arrays.asList( - new ConsumerRecord<>(TOPIC, 0, 0, 0L, Record.TimestampType.CreateTime, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)), - new ConsumerRecord<>(TOPIC, 0, 1, 0L, Record.TimestampType.CreateTime, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(1)), - new ConsumerRecord<>(TOPIC, 0, 2, 0L, Record.TimestampType.CreateTime, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)), - new ConsumerRecord<>(TOPIC, 0, 3, 0L, Record.TimestampType.CreateTime, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(3)), - new ConsumerRecord<>(TOPIC, 0, 4, 0L, Record.TimestampType.CreateTime, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(4)), + new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CreateTime, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)), + new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CreateTime, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(1)), + new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CreateTime, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)), + new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CreateTime, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(3)), + new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CreateTime, 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, 0L, Record.TimestampType.CreateTime, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5)), - new ConsumerRecord<>(TOPIC, 0, 6, 0L, Record.TimestampType.CreateTime, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(6))); + new ConsumerRecord<>(TOPIC, 0, 5, 0L, TimestampType.CreateTime, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5)), + new ConsumerRecord<>(TOPIC, 0, 6, 0L, TimestampType.CreateTime, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(6))); LinkedHashMap 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)); @@ -343,12 +343,12 @@ public void testPutTaskConfigsDoesNotResolveAllInconsistencies() throws Exceptio expectConfigure(); List> existingRecords = Arrays.asList( - new ConsumerRecord<>(TOPIC, 0, 0, 0L, Record.TimestampType.CreateTime, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)), + new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CreateTime, 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, 0L, Record.TimestampType.CreateTime, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)), - new ConsumerRecord<>(TOPIC, 0, 4, 0L, Record.TimestampType.CreateTime, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(4)), - new ConsumerRecord<>(TOPIC, 0, 5, 0L, Record.TimestampType.CreateTime, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5))); + new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CreateTime, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)), + new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CreateTime, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(4)), + new ConsumerRecord<>(TOPIC, 0, 5, 0L, TimestampType.CreateTime, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5))); LinkedHashMap 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)); @@ -484,7 +484,7 @@ private void expectReadToEnd(final LinkedHashMap serializedConfi public Future answer() throws Throwable { TestFuture future = new TestFuture(); for (Map.Entry entry : serializedConfigs.entrySet()) - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, logOffset++, 0L, Record.TimestampType.CreateTime, entry.getKey(), entry.getValue())); + capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, logOffset++, 0L, TimestampType.CreateTime, entry.getKey(), entry.getValue())); future.resolveOnGet((Void) null); return future; } 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 bea99b8cef474..a803eab2acaa2 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,7 +22,7 @@ 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.Record; +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; @@ -126,10 +126,10 @@ public void testStartStop() throws Exception { public void testReloadOnStart() throws Exception { expectConfigure(); expectStart(Arrays.asList( - new ConsumerRecord<>(TOPIC, 0, 0, 0L, Record.TimestampType.CreateTime, TP0_KEY.array(), TP0_VALUE.array()), - new ConsumerRecord<>(TOPIC, 1, 0, 0L, Record.TimestampType.CreateTime, TP1_KEY.array(), TP1_VALUE.array()), - new ConsumerRecord<>(TOPIC, 0, 1, 0L, Record.TimestampType.CreateTime, TP0_KEY.array(), TP0_VALUE_NEW.array()), - new ConsumerRecord<>(TOPIC, 1, 1, 0L, Record.TimestampType.CreateTime, TP1_KEY.array(), TP1_VALUE_NEW.array()) + new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CreateTime, TP0_KEY.array(), TP0_VALUE.array()), + new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CreateTime, TP1_KEY.array(), TP1_VALUE.array()), + new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CreateTime, TP0_KEY.array(), TP0_VALUE_NEW.array()), + new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CreateTime, TP1_KEY.array(), TP1_VALUE_NEW.array()) )); expectStop(); @@ -177,8 +177,8 @@ public Object answer() throws Throwable { PowerMock.expectLastCall().andAnswer(new IAnswer() { @Override public Object answer() throws Throwable { - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 0, 0L, Record.TimestampType.CreateTime, TP0_KEY.array(), TP0_VALUE.array())); - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 0, 0L, Record.TimestampType.CreateTime, TP1_KEY.array(), TP1_VALUE.array())); + capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CreateTime, TP0_KEY.array(), TP0_VALUE.array())); + capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CreateTime, TP1_KEY.array(), TP1_VALUE.array())); secondGetReadToEndCallback.getValue().onCompletion(null, null); return null; } @@ -190,8 +190,8 @@ public Object answer() throws Throwable { PowerMock.expectLastCall().andAnswer(new IAnswer() { @Override public Object answer() throws Throwable { - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 1, 0L, Record.TimestampType.CreateTime, TP0_KEY.array(), TP0_VALUE_NEW.array())); - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 1, 0L, Record.TimestampType.CreateTime, TP1_KEY.array(), TP1_VALUE_NEW.array())); + capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CreateTime, TP0_KEY.array(), TP0_VALUE_NEW.array())); + capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CreateTime, TP1_KEY.array(), TP1_VALUE_NEW.array())); thirdGetReadToEndCallback.getValue().onCompletion(null, null); return null; } 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 26bc5a5b694aa..fac6038b20ecd 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,7 +32,7 @@ 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.Record; +import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.Time; import org.easymock.Capture; import org.easymock.EasyMock; @@ -183,7 +183,7 @@ public void run() { consumer.schedulePollTask(new Runnable() { @Override public void run() { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, Record.TimestampType.CreateTime, TP0_KEY, TP0_VALUE)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CreateTime, TP0_KEY, TP0_VALUE)); } }); consumer.scheduleNopPollTask(); @@ -191,7 +191,7 @@ public void run() { consumer.schedulePollTask(new Runnable() { @Override public void run() { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, Record.TimestampType.CreateTime, TP1_KEY, TP1_VALUE)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CreateTime, TP1_KEY, TP1_VALUE)); } }); consumer.schedulePollTask(new Runnable() { @@ -298,16 +298,16 @@ public void run() { consumer.schedulePollTask(new Runnable() { @Override public void run() { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, Record.TimestampType.CreateTime, TP0_KEY, TP0_VALUE)); - consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 1, 0L, Record.TimestampType.CreateTime, TP0_KEY, TP0_VALUE_NEW)); - consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, Record.TimestampType.CreateTime, TP1_KEY, TP1_VALUE)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CreateTime, TP0_KEY, TP0_VALUE)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CreateTime, TP0_KEY, TP0_VALUE_NEW)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CreateTime, TP1_KEY, TP1_VALUE)); } }); consumer.schedulePollTask(new Runnable() { @Override public void run() { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 1, 0L, Record.TimestampType.CreateTime, TP1_KEY, TP1_VALUE_NEW)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CreateTime, TP1_KEY, TP1_VALUE_NEW)); } }); @@ -363,8 +363,8 @@ public void run() { consumer.schedulePollTask(new Runnable() { @Override public void run() { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, Record.TimestampType.CreateTime, TP0_KEY, TP0_VALUE_NEW)); - consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, Record.TimestampType.CreateTime, TP0_KEY, TP0_VALUE_NEW)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CreateTime, TP0_KEY, TP0_VALUE_NEW)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CreateTime, TP0_KEY, TP0_VALUE_NEW)); } }); 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 4189b07aad791..79ff5ad7eb568 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,7 +19,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.record.Record; +import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.streams.processor.TimestampExtractor; import java.util.ArrayDeque; @@ -78,7 +78,7 @@ public int addRawRecords(Iterable> rawRecords, Ti Object value = source.deserializeValue(rawRecord.topic(), rawRecord.value()); ConsumerRecord record = new ConsumerRecord<>(rawRecord.topic(), rawRecord.partition(), - rawRecord.offset(), rawRecord.timestamp(), Record.TimestampType.CreateTime, key, value); + rawRecord.offset(), rawRecord.timestamp(), TimestampType.CreateTime, key, value); long timestamp = timestampExtractor.extract(record); StampedRecord stampedRecord = new StampedRecord(record, timestamp); 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 a7de588ba9a7b..02e394fc3057e 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,7 +21,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.record.Record; +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; @@ -60,17 +60,17 @@ public void testTimeTracking() { // add three 3 records with timestamp 1, 3, 5 to partition-1 List> list1 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 1, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 3, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 5, 0L, Record.TimestampType.CreateTime, recordKey, recordValue)); + new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 3, 0L, TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 5, 0L, TimestampType.CreateTime, recordKey, recordValue)); group.addRawRecords(partition1, list1); // add three 3 records with timestamp 2, 4, 6 to partition-2 List> list2 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 2, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 4, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 6, 0L, Record.TimestampType.CreateTime, recordKey, recordValue)); + new ConsumerRecord<>("topic", 1, 2, 0L, TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 4, 0L, TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 6, 0L, TimestampType.CreateTime, recordKey, recordValue)); group.addRawRecords(partition2, list2); 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 be6b8ce75d53b..d131fbdb67067 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,7 +25,7 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.record.Record; +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; @@ -85,7 +85,7 @@ public void reset() { public void bufferRecord(ConsumerRecord record) { recordBuffer.add( new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), 0L, - Record.TimestampType.CreateTime, + TimestampType.CreateTime, serializer.serialize(record.topic(), record.key()), serializer.serialize(record.topic(), record.value()))); endOffset = record.offset(); @@ -269,7 +269,7 @@ public void testRegisterPersistentStore() throws IOException { int key = i * 10; expectedKeys.add(key); restoreConsumer.bufferRecord( - new ConsumerRecord<>(persistentStoreTopicName, 2, 0L, offset, Record.TimestampType.CreateTime, key, 0) + new ConsumerRecord<>(persistentStoreTopicName, 2, 0L, offset, TimestampType.CreateTime, key, 0) ); } @@ -322,7 +322,7 @@ public void testRegisterNonPersistentStore() throws IOException { int key = i; expectedKeys.add(i); restoreConsumer.bufferRecord( - new ConsumerRecord<>(nonPersistentStoreTopicName, 2, 0L, offset, Record.TimestampType.CreateTime, key, 0) + new ConsumerRecord<>(nonPersistentStoreTopicName, 2, 0L, offset, TimestampType.CreateTime, key, 0) ); } 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 9d98cef8dedd9..605c08872ddb0 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,7 +22,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.record.Record; +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; @@ -51,9 +51,9 @@ public void testTimeTracking() { // add three 3 out-of-order records with timestamp 2, 1, 3 List> list1 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 2, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 1, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 3, 0L, Record.TimestampType.CreateTime, recordKey, recordValue)); + new ConsumerRecord<>("topic", 1, 2, 0L, TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 3, 0L, TimestampType.CreateTime, recordKey, recordValue)); queue.addRawRecords(list1, timestampExtractor); @@ -73,9 +73,9 @@ public void testTimeTracking() { // add three 3 out-of-order records with timestamp 4, 1, 2 // now with 3, 4, 1, 2 List> list2 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 4, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 1, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 2, 0L, Record.TimestampType.CreateTime, recordKey, recordValue)); + new ConsumerRecord<>("topic", 1, 4, 0L, TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 2, 0L, TimestampType.CreateTime, recordKey, recordValue)); queue.addRawRecords(list2, timestampExtractor); @@ -100,9 +100,9 @@ public void testTimeTracking() { // add three more records with 4, 5, 6 List> list3 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 4, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 5, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 6, 0L, Record.TimestampType.CreateTime, recordKey, recordValue)); + new ConsumerRecord<>("topic", 1, 4, 0L, TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 5, 0L, TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 6, 0L, TimestampType.CreateTime, recordKey, recordValue)); queue.addRawRecords(list3, timestampExtractor); 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 6ac88c684063c..a2d69cc4cbb95 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,7 +24,7 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.record.Record; +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; @@ -153,7 +153,7 @@ public void testUpdateNonPersistentStore() throws Exception { restoreStateConsumer.assign(new ArrayList<>(task.changeLogPartitions())); task.update(partition1, - records(new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, 0L, Record.TimestampType.CreateTime, recordKey, recordValue)) + records(new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, 0L, TimestampType.CreateTime, recordKey, recordValue)) ); } finally { @@ -172,9 +172,9 @@ public void testUpdate() throws Exception { restoreStateConsumer.assign(new ArrayList<>(task.changeLogPartitions())); for (ConsumerRecord record : Arrays.asList( - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 10, 0L, Record.TimestampType.CreateTime, 1, 100), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 20, 0L, Record.TimestampType.CreateTime, 2, 100), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 30, 0L, Record.TimestampType.CreateTime, 3, 100))) { + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 10, 0L, TimestampType.CreateTime, 1, 100), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 20, 0L, TimestampType.CreateTime, 2, 100), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 30, 0L, TimestampType.CreateTime, 3, 100))) { restoreStateConsumer.bufferRecord(record); } @@ -235,11 +235,11 @@ public void testUpdateKTable() throws Exception { restoreStateConsumer.assign(new ArrayList<>(task.changeLogPartitions())); for (ConsumerRecord record : Arrays.asList( - new ConsumerRecord<>(ktable.topic(), ktable.partition(), 10, 0L, Record.TimestampType.CreateTime, 1, 100), - new ConsumerRecord<>(ktable.topic(), ktable.partition(), 20, 0L, Record.TimestampType.CreateTime, 2, 100), - new ConsumerRecord<>(ktable.topic(), ktable.partition(), 30, 0L, Record.TimestampType.CreateTime, 3, 100), - new ConsumerRecord<>(ktable.topic(), ktable.partition(), 40, 0L, Record.TimestampType.CreateTime, 4, 100), - new ConsumerRecord<>(ktable.topic(), ktable.partition(), 50, 0L, Record.TimestampType.CreateTime, 5, 100))) { + new ConsumerRecord<>(ktable.topic(), ktable.partition(), 10, 0L, TimestampType.CreateTime, 1, 100), + new ConsumerRecord<>(ktable.topic(), ktable.partition(), 20, 0L, TimestampType.CreateTime, 2, 100), + new ConsumerRecord<>(ktable.topic(), ktable.partition(), 30, 0L, TimestampType.CreateTime, 3, 100), + new ConsumerRecord<>(ktable.topic(), ktable.partition(), 40, 0L, TimestampType.CreateTime, 4, 100), + new ConsumerRecord<>(ktable.topic(), ktable.partition(), 50, 0L, TimestampType.CreateTime, 5, 100))) { restoreStateConsumer.bufferRecord(record); } 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 9703e1339b2dc..ad2f5b96fc3fb 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,7 +22,7 @@ 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.Record; +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; @@ -108,15 +108,15 @@ public void testProcessOrder() throws Exception { 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, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, 0L, Record.TimestampType.CreateTime, recordKey, recordValue) + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, 0L, TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, 0L, TimestampType.CreateTime, recordKey, recordValue) )); task.addRecords(partition2, records( - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 25, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, Record.TimestampType.CreateTime, recordKey, recordValue) + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 25, 0L, TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, TimestampType.CreateTime, recordKey, recordValue) )); assertEquals(5, task.process()); @@ -159,15 +159,15 @@ public void testPauseResume() throws Exception { 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, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, Record.TimestampType.CreateTime, recordKey, recordValue) + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, 0L, TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, TimestampType.CreateTime, recordKey, recordValue) )); task.addRecords(partition2, records( - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 55, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 65, 0L, Record.TimestampType.CreateTime, recordKey, recordValue) + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 55, 0L, TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 65, 0L, TimestampType.CreateTime, recordKey, recordValue) )); assertEquals(5, task.process()); @@ -178,9 +178,9 @@ public void testPauseResume() throws Exception { assertTrue(consumer.paused().contains(partition2)); task.addRecords(partition1, records( - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 40, 0L, Record.TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 50, 0L, Record.TimestampType.CreateTime, recordKey, recordValue) + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, 0L, TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 40, 0L, TimestampType.CreateTime, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 50, 0L, TimestampType.CreateTime, recordKey, recordValue) )); assertEquals(2, consumer.paused().size()); 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 2e765765e87a9..60cc5bc08df54 100644 --- a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java @@ -24,7 +24,7 @@ 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.Record; +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; @@ -201,7 +201,7 @@ public void process(String topicName, byte[] key, byte[] value) { } // Add the record ... long offset = offsetsByTopicPartition.get(tp).incrementAndGet(); - task.addRecords(tp, records(new ConsumerRecord(tp.topic(), tp.partition(), offset, 0L, Record.TimestampType.CreateTime, key, value))); + task.addRecords(tp, records(new ConsumerRecord(tp.topic(), tp.partition(), offset, 0L, TimestampType.CreateTime, key, value))); producer.clear(); // Process the record ... task.process(); From 0a1a0b81c3465e884397b67aac89fc7de37677c4 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Thu, 28 Jan 2016 15:34:56 -0800 Subject: [PATCH 10/33] Addressed Anna's comments. --- .../kafka/message/ByteBufferMessageSet.scala | 23 ++++++++++++------- 1 file changed, 15 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 55a3123ef486a..a669d412d670b 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -527,12 +527,21 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi validateMessageKey(message, compactedTopic) if (message.magic > Message.MagicValue_V0) { validateTimestamp(message, now, timestampType, timestampDiffMaxMs) - if (timestampType == TimestampType.LogAppendTime) { - message.buffer.putLong(Message.TimestampOffset, now) - message.buffer.put(Message.AttributesOffset, TimestampType.setTimestampType(message.attributes, TimestampType.LogAppendTime)) - // We have to update crc after updating the timestamp. - Utils.writeUnsignedInt(message.buffer, Message.CrcOffset, message.computeChecksum()) + val crcUpdateNeeded = { + if (timestampType == TimestampType.LogAppendTime) { + message.buffer.putLong(Message.TimestampOffset, now) + message.buffer.put(Message.AttributesOffset, TimestampType.setTimestampType(message.attributes, TimestampType.LogAppendTime)) + true + } else if (timestampType == TimestampType.CreateTime && + TimestampType.getTimestampType(message.attributes) != TimestampType.CreateTime) { + message.buffer.put(Message.AttributesOffset, TimestampType.setTimestampType(message.attributes, TimestampType.CreateTime)) + true + } else + false } + // We have to update crc after updating the timestamp or timestamp type. + if (crcUpdateNeeded) + Utils.writeUnsignedInt(message.buffer, Message.CrcOffset, message.computeChecksum()) } messagePosition += MessageSet.LogOverhead + messageSize } @@ -547,9 +556,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi /** * This method validates the timestamps of a message. - * 1. If the message is using create time, this method checks if it is with acceptable range. - * 2. If the message is using log append time and is an uncompressed message, this method will overwrite the - * timestamp of the message. + * If the message is using create time, this method checks if it is with acceptable range. */ private def validateTimestamp(message: Message, now: Long, From b0ba4a0884c405a93ce06ce22d84690fb4194c72 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Fri, 29 Jan 2016 03:24:28 -0800 Subject: [PATCH 11/33] Addressed Jun's comments --- .../kafka/clients/consumer/ConsumerRecord.java | 2 +- .../kafka/clients/producer/ProducerRecord.java | 12 +++++------- .../producer/internals/RecordBatch.java | 1 + .../apache/kafka/common/protocol/Protocol.java | 2 +- .../kafka/common/record/MemoryRecords.java | 18 +++++++++--------- .../org/apache/kafka/common/record/Record.java | 7 ++++--- .../kafka/common/record/TimestampType.java | 5 +++++ .../kafka/message/ByteBufferMessageSet.scala | 4 ++-- .../src/main/scala/kafka/message/Message.scala | 4 ++-- 9 files changed, 30 insertions(+), 25 deletions(-) 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 91f2ebe2093cd..ee40cd5b4eec4 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 @@ -101,6 +101,6 @@ public TimestampType timestampType() { @Override public String toString() { return "ConsumerRecord(topic = " + topic() + ", partition = " + partition() + ", offset = " + offset() - + ", timestamp = " + timestamp + ", key = " + key + ", value = " + value + ")"; + + ", " + timestampType + " = " + timestamp + ", key = " + key + ", value = " + value + ")"; } } 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 9dad743e8b7e2..f14e89134b88e 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 @@ -12,8 +12,6 @@ */ package org.apache.kafka.clients.producer; -import org.apache.kafka.common.record.Record; - /** * A key/value pair to be sent to Kafka. This consists of a topic name to which the record is being sent, an optional * partition number, and an optional key and value. @@ -25,10 +23,10 @@ * The record also has an associated timestamp. If user did not provide a timestamp, the producer will stamp the record * with a timestamp depending on what is the timestamp type used by the topic. *
  • - * If the topic is configured to use {@link org.apache.kafka.common.record.Record.TimestampType#CreateTime CreateTime} - * the timestamp will be the producer current time. + * If the topic is configured to use {@link org.apache.kafka.common.record.TimestampType#CreateTime CreateTime} + * the timestamp will be user specified timestamp or the producer's current time. *
  • - * If the topic is configured to use {@link org.apache.kafka.common.record.Record.TimestampType#LogAppendTime LogAppendTime} + * If the topic is configured to use {@link org.apache.kafka.common.record.TimestampType#LogAppendTime LogAppendTime} * the timestamp will be the time when Kafka broker accepts the record. */ public final class ProducerRecord { @@ -40,7 +38,7 @@ public final class ProducerRecord { private final Long timestamp; /** - * Creates a record to be sent to a specified topic, partition and 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 @@ -51,7 +49,7 @@ public final class ProducerRecord { 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 < Record.NO_TIMESTAMP) + if (timestamp != null && timestamp < 0) throw new IllegalArgumentException("Invalid timestamp " + timestamp); this.topic = topic; this.partition = partition; 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 82ef83ebf585c..7bbbff5f910b2 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 @@ -83,6 +83,7 @@ public FutureRecordMetadata tryAppend(long timestamp, byte[] key, byte[] value, * 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, long timestamp, RuntimeException exception) { 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 b543dada2527f..171e9e723e13c 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 @@ -421,7 +421,7 @@ public class Protocol { new Field("responses", new ArrayOf(FETCH_RESPONSE_TOPIC_V0))); // Fetch response V2 is the same as fetch response V1. The version number is bumped up to indicate that the Message - // format in the response is in message format V1, i.e. the messages use relative offsets and has timestamps. + // format in the response could be up to message format V1, i.e. the messages use relative offsets and has timestamps. public static final Schema FETCH_RESPONSE_V2 = FETCH_RESPONSE_V1; public static final Schema[] FETCH_REQUEST = new Schema[] {FETCH_REQUEST_V0, FETCH_REQUEST_V1, FETCH_REQUEST_V2}; 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 2b43db7b409d3..00f1e6a26bd24 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 @@ -217,7 +217,7 @@ public static class RecordsIterator extends AbstractIterator { // The variables for inner iterator private final LinkedList logEntries; - private final long lastInnerOffset; + private final long lastInnerRelativeOffset; private final long wrapperRecordOffset; public RecordsIterator(ByteBuffer buffer, CompressionType type, boolean shallow) { @@ -226,14 +226,14 @@ public RecordsIterator(ByteBuffer buffer, CompressionType type, boolean shallow) this.shallow = shallow; this.stream = Compressor.wrapForInput(new ByteBufferInputStream(this.buffer), type); this.logEntries = null; - this.lastInnerOffset = -1L; + this.lastInnerRelativeOffset = -1L; this.wrapperRecordOffset = -1L; } - private RecordsIterator(LogEntry entry, CompressionType type, boolean shallow) { + private RecordsIterator(LogEntry entry, CompressionType type) { this.type = type; this.buffer = entry.record().value(); - this.shallow = shallow; + this.shallow = true; this.stream = Compressor.wrapForInput(new ByteBufferInputStream(this.buffer), type); this.wrapperRecordOffset = entry.offset(); // If relative offset is used, we need to decompress the entire message first to compute @@ -254,10 +254,10 @@ private RecordsIterator(LogEntry entry, CompressionType type, boolean shallow) { throw new KafkaException(e); } } - this.lastInnerOffset = logEntries.getLast().offset(); + this.lastInnerRelativeOffset = logEntries.getLast().offset(); } else { this.logEntries = null; - this.lastInnerOffset = -1L; + this.lastInnerRelativeOffset = -1L; } } @@ -279,8 +279,8 @@ protected LogEntry makeNext() { return allDone(); // Convert offset to absolute offset if needed. - if (entry.record().magic() > Record.MAGIC_VALUE_V0 && lastInnerOffset >= 0) { - long absoluteOffset = entry.offset() - lastInnerOffset + wrapperRecordOffset; + if (lastInnerRelativeOffset >= 0) { + long absoluteOffset = entry.offset() - lastInnerRelativeOffset + wrapperRecordOffset; entry = new LogEntry(absoluteOffset, entry.record()); } @@ -293,7 +293,7 @@ protected LogEntry makeNext() { // 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 - innerIter = new RecordsIterator(entry, compression, true); + innerIter = new RecordsIterator(entry, compression); return innerIter.next(); } } catch (EOFException e) { 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 1253d5231b8f7..8a2978a1b238c 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 @@ -67,7 +67,7 @@ public final class Record { public static final byte CURRENT_MAGIC_VALUE = 1; /** - * Specifies the mask for the compression code. 4 bits to hold the compression codec. 0 is reserved to indicate no + * Specifies the mask for the compression code. 3 bits to hold the compression codec. 0 is reserved to indicate no * compression */ public static final int COMPRESSION_CODEC_MASK = 0x07; @@ -87,7 +87,7 @@ public final class Record { /** * Timestamp value for records without a timestamp */ - public static final long NO_TIMESTAMP = -1; + public static final long NO_TIMESTAMP = -1L; private final ByteBuffer buffer; private final Long wrapperRecordTimestamp; @@ -380,11 +380,12 @@ private ByteBuffer sliceDelimited(int start) { } public String toString() { - return String.format("Record(magic = %d, attributes = %d, compression = %s, crc = %d, timestamp = %d, key = %d bytes, value = %d bytes)", + 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()); diff --git a/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java b/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java index 48fb58f6789ad..eb91625ea0846 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java +++ b/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java @@ -39,4 +39,9 @@ public static byte setTimestampType(byte attributes, TimestampType timestampType return timestampType == CreateTime ? (byte) (attributes & ~Record.TIMESTAMP_TYPE_MASK) : (byte) (attributes | Record.TIMESTAMP_TYPE_MASK); } + + @Override + public String toString() { + return name; + } } diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index a669d412d670b..58e54c9eecd63 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -364,8 +364,8 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi * 3. When magic value = 1, validate and maybe overwrite timestamps of messages. * * This method will convert the messages based on the following scenarios: - * A. Magic value of a message = 0 and messageFormatVersion is above or equals to 0.10.0-DV1 - * B. Magic value of a message = 1 and messageFormatVersion is lower than 0.10.0-DV1 + * A. Magic value of a message = 0 and messageFormatVersion is above or equals to 0.10.0-DV0 + * B. Magic value of a message = 1 and messageFormatVersion is lower than 0.10.0-DV0 * * If no format conversion or value overwriting is required for messages, this method will perform in-place * operations and avoids re-compression. diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala index 78b500719087e..2d8f731cc1ce6 100755 --- a/core/src/main/scala/kafka/message/Message.scala +++ b/core/src/main/scala/kafka/message/Message.scala @@ -81,7 +81,7 @@ object Message { val CurrentMagicValue: Byte = 1 /** - * Specifies the mask for the compression code. 4 bits to hold the compression codec. + * 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 @@ -288,7 +288,7 @@ def timestamp: Long = { * 1. TimestampType = LogAppendTime and * a) WrapperMessageTimestamp is not defined - Uncompressed message using LogAppendTime * b) WrapperMessageTimestamp is defined - Compressed message using LogAppendTime - * 2. TimestampType = CreateTime and + * 2. TimestampType = CreateTime Or is not defined and * a) WrapperMessageTimestamp is not defined - Uncompressed message using CreateTime * b) WrapperMessageTimestamp is defined - Compressed message using CreateTime */ From 1824bfaf1ce4e8bc657817a627034cda745435d5 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Mon, 1 Feb 2016 16:41:29 -0800 Subject: [PATCH 12/33] Added test for invalid timestamp --- .../clients/producer/ProducerRecord.java | 2 +- .../kafka/api/BaseProducerSendTest.scala | 19 ++++++++++++++++++- 2 files changed, 19 insertions(+), 2 deletions(-) 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 f14e89134b88e..7be5df7774e3e 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 @@ -143,7 +143,7 @@ public String toString() { String value = this.value == null ? "null" : this.value.toString(); String timestamp = this.timestamp == null ? "null" : this.timestamp.toString(); return "ProducerRecord(topic=" + topic + ", partition=" + partition + ", key=" + key + ", value=" + value + - "timestamp=" + timestamp + ")"; + ", timestamp=" + timestamp + ")"; } @Override diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala index a8be84f4530ee..56da8aba0182c 100644 --- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala @@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit import kafka.consumer.SimpleConsumer import kafka.integration.KafkaServerTestHarness import kafka.log.LogConfig -import kafka.message.Message +import kafka.message.{InvalidMessageException, Message} import kafka.server.KafkaConfig import kafka.utils.TestUtils import org.apache.kafka.clients.producer._ @@ -171,6 +171,23 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { sendAndVerifyTimestamp(producer, TimestampType.LogAppendTime) } + @Test(expected = classOf[InvalidMessageException]) + def testSendNonCompressedMessageWithInvalidCreateTime() { + val topicProps = new Properties() + topicProps.setProperty(LogConfig.MessageTimestampDifferenceMaxMsProp, "1000"); + TestUtils.createTopic(zkUtils, topic, 1, 2, servers, topicProps) + + + val producer = createProducer(brokerList = brokerList) + producer.send(new ProducerRecord(topic, 0, System.currentTimeMillis() - 1001, "key".getBytes, "value".getBytes)).get() + + // Test compressed messages. + val producerProps = new Properties() + producerProps.setProperty(ProducerConfig.COMPRESSION_TYPE_CONFIG, "gzip") + val compressedProducer = createProducer(brokerList = brokerList, lingerMs = Long.MaxValue, props = Some(producerProps)) + compressedProducer.send(new ProducerRecord(topic, 0, System.currentTimeMillis() - 1001, "key".getBytes, "value".getBytes)).get() + } + private def sendAndVerifyTimestamp(producer: KafkaProducer[Array[Byte], Array[Byte]], timestampType: TimestampType) { val partition = new Integer(0) From c691fd0a7ee477f07cb6db242e72ec6c48842aba Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Tue, 2 Feb 2016 11:35:13 -0800 Subject: [PATCH 13/33] Addressed Jun's comments --- .../kafka/common/protocol/Protocol.java | 5 +- .../apache/kafka/common/record/Record.java | 19 ++-- .../common/requests/ProduceResponse.java | 8 +- .../src/main/scala/kafka/api/ApiVersion.scala | 20 ++-- .../main/scala/kafka/api/FetchResponse.scala | 19 +--- .../main/scala/kafka/log/FileMessageSet.scala | 30 ++++++ .../kafka/message/ByteBufferMessageSet.scala | 18 ++-- .../main/scala/kafka/message/Message.scala | 27 +++--- .../main/scala/kafka/message/MessageSet.scala | 28 ------ .../main/scala/kafka/server/KafkaApis.scala | 32 ++++--- .../kafka/server/ReplicaFetcherThread.scala | 4 +- .../kafka/api/BaseProducerSendTest.scala | 17 ---- .../RequestResponseSerializationTest.scala | 92 ------------------ .../unit/kafka/log/FileMessageSetTest.scala | 93 +++++++++++++++++++ 14 files changed, 200 insertions(+), 212 deletions(-) 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 171e9e723e13c..a2f05c2adf914 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 @@ -420,8 +420,9 @@ public class Protocol { 0), new Field("responses", new ArrayOf(FETCH_RESPONSE_TOPIC_V0))); - // Fetch response V2 is the same as fetch response V1. The version number is bumped up to indicate that the Message - // format in the response could be up to message format V1, i.e. the messages use relative offsets and has timestamps. + // 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, FETCH_REQUEST_V2}; 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 8a2978a1b238c..ed01a5f5743b9 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 @@ -91,19 +91,19 @@ public final class Record { private final ByteBuffer buffer; private final Long wrapperRecordTimestamp; - private final TimestampType timestampTypeToUse; + private final TimestampType wrapperRecordTimestampType; public Record(ByteBuffer buffer) { this.buffer = buffer; this.wrapperRecordTimestamp = null; - this.timestampTypeToUse = null; + this.wrapperRecordTimestampType = null; } // Package private constructor for inner iteration. - Record(ByteBuffer buffer, Long wrapperRecordTimestamp, TimestampType timestampTypeToUse) { + Record(ByteBuffer buffer, Long wrapperRecordTimestamp, TimestampType wrapperRecordTimestampType) { this.buffer = buffer; this.wrapperRecordTimestamp = wrapperRecordTimestamp; - this.timestampTypeToUse = timestampTypeToUse; + this.wrapperRecordTimestampType = wrapperRecordTimestampType; } /** @@ -321,21 +321,26 @@ public byte attributes() { } /** - * The timestamp of this 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 = LogAppendTime and WrapperRecordTimestamp is not null - Compressed message using LogAppendTime + * 3. wrapperRecordTimestampType = CreateTime and wrapperRecordTimestamp is not null - Compressed message using CreateTime */ public long timestamp() { if (magic() == MAGIC_VALUE_V0) return NO_TIMESTAMP; else { - if (timestampTypeToUse == TimestampType.LogAppendTime && wrapperRecordTimestamp != null) + // case 2 + if (wrapperRecordTimestampType == TimestampType.LogAppendTime && wrapperRecordTimestamp != null) return wrapperRecordTimestamp; + // Case 1, 3 else return buffer.getLong(TIMESTAMP_OFFSET); } } public TimestampType timestampType() { - return timestampTypeToUse == null ? TimestampType.getTimestampType(attributes()) : timestampTypeToUse; + return wrapperRecordTimestampType == null ? TimestampType.getTimestampType(attributes()) : wrapperRecordTimestampType; } /** 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 b57ee7c982fd3..b4a2d422e0374 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 @@ -69,7 +69,7 @@ public ProduceResponse(Map responses) { } /** - * Constructor for Version 1 + * Constructor for latest version * @param responses Produced data grouped by topic-partition * @param throttleTime Time in milliseconds the response was throttled */ @@ -86,12 +86,16 @@ public ProduceResponse(Map responses, int thr public ProduceResponse(Map responses, int throttleTime, int version) { super(new Struct(ProtoUtils.responseSchema(ApiKeys.PRODUCE.id, version))); initCommonFields(responses); - if (version > 0) + 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(); diff --git a/core/src/main/scala/kafka/api/ApiVersion.scala b/core/src/main/scala/kafka/api/ApiVersion.scala index faee50c6995c0..9017fc35191cc 100644 --- a/core/src/main/scala/kafka/api/ApiVersion.scala +++ b/core/src/main/scala/kafka/api/ApiVersion.scala @@ -25,12 +25,12 @@ 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) * - * If there is a draft protocol version between protocols of two official releases. The suffix "-DV#" will be added to + * If there is a draft protocol version between protocols of two official releases. The suffix "-IV#" will be added to * help users who are running on trunk upgrade. For example: * 1. Kafka 0.9.0 is released - * 2. After that some protocol change are made and will be released in 0.10.0. The version will be named 0.10.0-DV0 - * (DV stands for draft version) - * 3. When Kafka 0.10.0 is released. The official version 0.10.0 will be the same as the last draft version. + * 2. After that some protocol change are made and will be released in 0.10.0. The version will be named 0.10.0-IV0 + * (DV stands for internal version) + * 3. When Kafka 0.10.0 is released. The official version 0.10.0 will be the same as the last internal version. */ object ApiVersion { // This implicit is necessary due to: https://issues.scala-lang.org/browse/SI-8541 @@ -41,7 +41,8 @@ object ApiVersion { "0.8.1" -> KAFKA_0_8_1, "0.8.2" -> KAFKA_0_8_2, "0.9.0" -> KAFKA_0_9_0, - "0.10.0-DV0" -> KAFKA_0_10_0_DV0 + "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(".")) @@ -86,7 +87,12 @@ case object KAFKA_0_9_0 extends ApiVersion { } // This is a between-release protocol version -case object KAFKA_0_10_0_DV0 extends ApiVersion { - val version: String = "0.10.0-DV0" +case object KAFKA_0_10_0_IV0 extends ApiVersion { + val version: String = "0.10.0-IV0" val id: Int = 4 } + +case object KAFKA_0_10_0 extends ApiVersion { + val version: String = "0.10.0-IV0" + val id: Int = 5 +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/FetchResponse.scala b/core/src/main/scala/kafka/api/FetchResponse.scala index f3a06c236578c..42f93c192b16e 100644 --- a/core/src/main/scala/kafka/api/FetchResponse.scala +++ b/core/src/main/scala/kafka/api/FetchResponse.scala @@ -49,14 +49,6 @@ object FetchResponsePartitionData { case class FetchResponsePartitionData(error: Short = Errors.NONE.code, hw: Long = -1L, messages: MessageSet) { val sizeInBytes = FetchResponsePartitionData.headerSize + messages.sizeInBytes - - def toMessageFormat(toMagicValue: Byte): FetchResponsePartitionData = { - if (messages.hasMagicValue(toMagicValue)) - this - else { - new FetchResponsePartitionData(error, hw, messages.toMessageFormat(toMagicValue)) - } - } } // SENDS @@ -209,20 +201,13 @@ object FetchResponse { case class FetchResponse(correlationId: Int, data: Map[TopicAndPartition, FetchResponsePartitionData], requestVersion: Int = 0, - throttleTimeMs: Int = 0, - magicValueToUse: Byte = Message.CurrentMagicValue) + throttleTimeMs: Int = 0) extends RequestOrResponse() { - - /** - * Convert the message format if necessary. - */ - lazy val dataAfterVersionConversion = data.map{case (topicAndPartition, partitionData) => - topicAndPartition -> partitionData.toMessageFormat(magicValueToUse)} /** * Partitions the data into a map of maps (one for each topic). */ - lazy val dataGroupedByTopic = dataAfterVersionConversion.groupBy{ case (topicAndPartition, fetchData) => topicAndPartition.topic } + lazy val dataGroupedByTopic = data.groupBy{ case (topicAndPartition, fetchData) => topicAndPartition.topic } val headerSizeInBytes = FetchResponse.headerSize(requestVersion) lazy val sizeInBytes = FetchResponse.responseSize(dataGroupedByTopic, requestVersion) diff --git a/core/src/main/scala/kafka/log/FileMessageSet.scala b/core/src/main/scala/kafka/log/FileMessageSet.scala index 9246cf57f0a0c..d12274999c209 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. @@ -200,6 +202,34 @@ class FileMessageSet private[kafka](@volatile var file: File, 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. */ diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 58e54c9eecd63..26d587ef11a83 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -17,7 +17,7 @@ package kafka.message -import kafka.api.{KAFKA_0_10_0_DV0, ApiVersion} +import kafka.api.{KAFKA_0_10_0_IV0, ApiVersion} import kafka.message.TimestampType.TimestampType import kafka.utils.{IteratorTemplate, Logging} import kafka.common.KafkaException @@ -104,11 +104,11 @@ object ByteBufferMessageSet { // However, Note that the message sets sent by producers are compressed in a stream compressing 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 writtern to + // 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 producer create a message set, it simply write all the messages into a compressed message set with + // 1. When producer create 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. @@ -169,7 +169,7 @@ object ByteBufferMessageSet { if (innerMessageAndOffsets.isEmpty) allDone() else { - val messageAndOffset = messageAndOffsets.get.dequeue() + val messageAndOffset = innerMessageAndOffsets.dequeue() val message = messageAndOffset.message val relativeOffset = messageAndOffset.offset - lastInnerOffset val absoluteOffset = wrapperMessageOffset + relativeOffset @@ -364,8 +364,8 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi * 3. When magic value = 1, validate and maybe overwrite timestamps of messages. * * This method will convert the messages based on the following scenarios: - * A. Magic value of a message = 0 and messageFormatVersion is above or equals to 0.10.0-DV0 - * B. Magic value of a message = 1 and messageFormatVersion is lower than 0.10.0-DV0 + * A. Magic value of a message = 0 and messageFormatVersion is above or equals to 0.10.0-IV0 + * B. Magic value of a message = 1 and messageFormatVersion is lower than 0.10.0-IV0 * * If no format conversion or value overwriting is required for messages, this method will perform in-place * operations and avoids re-compression. @@ -378,7 +378,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi messageFormatVersion: ApiVersion = ApiVersion.latestVersion, messageTimestampType: TimestampType, messageTimestampDiffMaxMs: Long): ByteBufferMessageSet = { - val magicValueToUse = if (messageFormatVersion.onOrAfter(KAFKA_0_10_0_DV0)) Message.MagicValue_V1 else Message.MagicValue_V0 + val magicValueToUse = if (messageFormatVersion.onOrAfter(KAFKA_0_10_0_IV0)) Message.MagicValue_V1 else Message.MagicValue_V0 if (sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) { // check the magic value if (!hasMagicValue(magicValueToUse)) { @@ -443,7 +443,8 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi } 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()) if (magicValueToUse > Message.MagicValue_V0) { var crcUpdateNeeded = true val timestampOffset = MessageSet.LogOverhead + Message.TimestampOffset @@ -524,6 +525,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi val messageBuffer = buffer.slice() messageBuffer.limit(messageSize) val message = new Message(messageBuffer) + message.ensureValid() validateMessageKey(message, compactedTopic) if (message.magic > Message.MagicValue_V0) { validateTimestamp(message, now, timestampType, timestampDiffMaxMs) diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala index 2d8f731cc1ce6..88cf662cb33da 100755 --- a/core/src/main/scala/kafka/message/Message.scala +++ b/core/src/main/scala/kafka/message/Message.scala @@ -117,7 +117,7 @@ object Message { * 1. 4 byte CRC32 of the message * 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 (e.g. compression enabled, type of codec used) - * 4. (Optional) 8 byte timestamp only if "magic" identifier is 1 + * 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 @@ -127,7 +127,7 @@ object Message { */ class Message(val buffer: ByteBuffer, private val wrapperMessageTimestamp: Option[Long] = None, - private val timestampTypeToUse: Option[TimestampType] = None) { + private val wrapperMessageTimestampType: Option[TimestampType] = None) { import kafka.message.Message._ @@ -278,24 +278,19 @@ class Message(val buffer: ByteBuffer, /** * 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. TimestampType = None and wrapperMessageTimestamp is None - Uncompressed message, timestamp and timestamp type are in the message. + * 2. TimestampType = LogAppendTime and wrapperMessageTimestamp is defined - Compressed message using LogAppendTime + * 3. TimestampType = CreateTime and wrapperMessageTimestamp is defined - Compressed message using CreateTime */ def timestamp: Long = { if (magic == MagicValue_V0) throw new IllegalStateException("The message with magic byte 0 does not have a timestamp") - /** - * The timestamp of a message is determined in the following way: - * 1. TimestampType = LogAppendTime and - * a) WrapperMessageTimestamp is not defined - Uncompressed message using LogAppendTime - * b) WrapperMessageTimestamp is defined - Compressed message using LogAppendTime - * 2. TimestampType = CreateTime Or is not defined and - * a) WrapperMessageTimestamp is not defined - Uncompressed message using CreateTime - * b) WrapperMessageTimestamp is defined - Compressed message using CreateTime - */ - // Case 1b - if (timestampTypeToUse.exists(_ == TimestampType.LogAppendTime) && wrapperMessageTimestamp.isDefined) + // Case 2 + if (wrapperMessageTimestampType.exists(_ == TimestampType.LogAppendTime) && wrapperMessageTimestamp.isDefined) wrapperMessageTimestamp.get - else // case 1b, 2a, 2b + else // case 1, 3 buffer.getLong(Message.TimestampOffset) } @@ -306,7 +301,7 @@ def timestamp: Long = { if (magic == MagicValue_V0) throw new IllegalStateException("The message with magic byte 0 does not have a timestamp") - timestampTypeToUse.getOrElse(TimestampType.getTimestampType(attributes)) + wrapperMessageTimestampType.getOrElse(TimestampType.getTimestampType(attributes)) } /** @@ -342,7 +337,7 @@ def timestamp: Long = { def convertToBuffer(toMagicValue: Byte, byteBuffer: ByteBuffer, now: Long = NoTimestamp, - timestampType: TimestampType = timestampTypeToUse.getOrElse(TimestampType.getTimestampType(attributes))) { + 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) diff --git a/core/src/main/scala/kafka/message/MessageSet.scala b/core/src/main/scala/kafka/message/MessageSet.scala index 8d6e04626a78f..f48fbddf1beee 100644 --- a/core/src/main/scala/kafka/message/MessageSet.scala +++ b/core/src/main/scala/kafka/message/MessageSet.scala @@ -106,34 +106,6 @@ abstract class MessageSet extends Iterable[MessageAndOffset] { */ def sizeInBytes: Int - /** - * 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 - // File message set only has shallow iterator. We need to do deep iteration here if needed. - if (message.compressionCodec == NoCompressionCodec || !this.isInstanceOf[FileMessageSet]) { - newMessages += messageAndOffset.message.toFormatVersion(toMagicValue) - offsets += messageAndOffset.offset - } else { - 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: _*) - } - /** * Print this message set's contents. If the message set has more than 100 messages, just * print the first 100. diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 245f19a9d8b3a..b0158a81736d2 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -35,8 +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.types.Schema +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, @@ -430,15 +429,20 @@ class KafkaApis(val requestChannel: RequestChannel, // the callback for sending a fetch response def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData]) { - // determine the magic value to use - // Only send messages with magic value 1 when client supports it and server has this format on disk. - val magicValueToUse = - if (fetchRequest.versionId > 1 && config.messageFormatVersion.onOrAfter(KAFKA_0_10_0_DV0)) - Message.MagicValue_V1 - else - Message.MagicValue_V0 - - val mergedResponseStatus = responsePartitionData ++ unauthorizedResponseStatus + val convertedResponseStatus = + // Need to down-convert message when consumer only takes magic value 0. + if (fetchRequest.versionId <= 1) { + responsePartitionData.mapValues({ case data => + if (!data.messages.hasMagicValue(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) { @@ -452,9 +456,9 @@ class KafkaApis(val requestChannel: RequestChannel, } def fetchResponseCallback(delayTimeMs: Int) { - info(s"Sending fetch response to ${fetchRequest.clientId} with ${responsePartitionData.values.map(_.messages.size).sum}" + - s" messages using magic value $magicValueToUse") - val response = FetchResponse(fetchRequest.correlationId, mergedResponseStatus, fetchRequest.versionId, delayTimeMs, magicValueToUse) + trace(s"Sending fetch response to ${fetchRequest.clientId} with ${convertedResponseStatus.values.map(_.messages.size).sum}" + + s" messages") + val response = FetchResponse(fetchRequest.correlationId, mergedResponseStatus, fetchRequest.versionId, delayTimeMs) requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, response))) } diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 8c454a373e30e..2fdb46c156a05 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_0_10_0_DV0, KAFKA_0_9_0} +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} @@ -56,7 +56,7 @@ class ReplicaFetcherThread(name: String, type PD = PartitionData private val fetchRequestVersion: Short = - if (brokerConfig.interBrokerProtocolVersion.onOrAfter(KAFKA_0_10_0_DV0)) 2 + 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 diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala index 56da8aba0182c..5d97a76492b91 100644 --- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala @@ -171,23 +171,6 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { sendAndVerifyTimestamp(producer, TimestampType.LogAppendTime) } - @Test(expected = classOf[InvalidMessageException]) - def testSendNonCompressedMessageWithInvalidCreateTime() { - val topicProps = new Properties() - topicProps.setProperty(LogConfig.MessageTimestampDifferenceMaxMsProp, "1000"); - TestUtils.createTopic(zkUtils, topic, 1, 2, servers, topicProps) - - - val producer = createProducer(brokerList = brokerList) - producer.send(new ProducerRecord(topic, 0, System.currentTimeMillis() - 1001, "key".getBytes, "value".getBytes)).get() - - // Test compressed messages. - val producerProps = new Properties() - producerProps.setProperty(ProducerConfig.COMPRESSION_TYPE_CONFIG, "gzip") - val compressedProducer = createProducer(brokerList = brokerList, lingerMs = Long.MaxValue, props = Some(producerProps)) - compressedProducer.send(new ProducerRecord(topic, 0, System.currentTimeMillis() - 1001, "key".getBytes, "value".getBytes)).get() - } - private def sendAndVerifyTimestamp(producer: KafkaProducer[Array[Byte], Array[Byte]], timestampType: TimestampType) { val partition = new Integer(0) diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index c055ae4486e8e..d15d3d3e9b6e8 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -290,96 +290,4 @@ class RequestResponseSerializationTest extends JUnitSuite { assertEquals(oldClientResponse.sizeInBytes + 4, newClientResponse.sizeInBytes) } - @Test - def testFetchResponseFormatConversion() { - // Up conversion - 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:_*) - - // up conversion for non-compressed messages - var fileMessageSet = new FileMessageSet(tempFile()) - fileMessageSet.append(messageSetV0) - fileMessageSet.flush() - var convertedPartitionData = new FetchResponsePartitionData(messages = fileMessageSet).toMessageFormat(Message.MagicValue_V1) - var i = 0 - for (messageAndOffset <- convertedPartitionData.messages) { - val message = messageAndOffset.message - val offset = messageAndOffset.offset - assertEquals("magic byte should be 1", Message.MagicValue_V1, message.magic) - assertEquals("offset should not change", offsets(i), offset) - assertEquals("key should not change", messagesV0(i).key, message.key) - assertEquals("payload should not change", messagesV0(i).payload, message.payload) - i += 1 - } - - fileMessageSet = new FileMessageSet(tempFile()) - fileMessageSet.append(compressedMessageSetV0) - fileMessageSet.flush() - // up conversion for compressed messages - convertedPartitionData = new FetchResponsePartitionData(messages = fileMessageSet).toMessageFormat(Message.MagicValue_V1) - i = 0 - for (messageAndOffset <- convertedPartitionData.messages) { - val message = messageAndOffset.message - val offset = messageAndOffset.offset - assertEquals("magic byte should be 1", Message.MagicValue_V1, message.magic) - assertEquals("offset should not change", offsets(i), offset) - assertEquals("key should not change", messagesV0(i).key, message.key) - assertEquals("payload should not change", messagesV0(i).payload, message.payload) - i += 1 - } - - // Down conversion - 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 for non-compressed messages - fileMessageSet = new FileMessageSet(tempFile()) - fileMessageSet.append(messageSetV1) - fileMessageSet.flush() - convertedPartitionData = new FetchResponsePartitionData(messages = fileMessageSet).toMessageFormat(Message.MagicValue_V0) - i = 0 - for (messageAndOffset <- convertedPartitionData.messages) { - val message = messageAndOffset.message - val offset = messageAndOffset.offset - assertEquals("magic byte should be 0", Message.MagicValue_V0, message.magic) - assertEquals("offset should not change", offsets(i), offset) - assertEquals("key should not change", messagesV0(i).key, message.key) - assertEquals("payload should not change", messagesV0(i).payload, message.payload) - i += 1 - } - - fileMessageSet = new FileMessageSet(tempFile()) - fileMessageSet.append(compressedMessageSetV1) - fileMessageSet.flush() - // down conversion for compressed messages - convertedPartitionData = new FetchResponsePartitionData(messages = fileMessageSet).toMessageFormat(Message.MagicValue_V0) - i = 0 - for (messageAndOffset <- convertedPartitionData.messages) { - val message = messageAndOffset.message - val offset = messageAndOffset.offset - assertEquals("magic byte should be 0", Message.MagicValue_V0, message.magic) - assertEquals("offset should not change", offsets(i), offset) - assertEquals("key should not change", messagesV0(i).key, message.key) - assertEquals("payload should not change", messagesV0(i).payload, message.payload) - i += 1 - } - } - } diff --git a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala index 95085f49144ab..d3257f7ea1a20 100644 --- a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala @@ -200,4 +200,97 @@ 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 converion + // down conversion for non-compressed messages + var fileMessageSet = new FileMessageSet(tempFile()) + fileMessageSet.append(messageSetV1) + fileMessageSet.flush() + var convertedMessageSet = fileMessageSet.toMessageFormat(Message.MagicValue_V0) + var i = 0 + for (messageAndOffset <- convertedMessageSet) { + val message = messageAndOffset.message + val offset = messageAndOffset.offset + assertEquals("magic byte should be 0", Message.MagicValue_V0, message.magic) + assertEquals("offset should not change", offsets(i), offset) + assertEquals("key should not change", messagesV0(i).key, message.key) + assertEquals("payload should not change", messagesV0(i).payload, message.payload) + i += 1 + } + + fileMessageSet = new FileMessageSet(tempFile()) + fileMessageSet.append(compressedMessageSetV1) + fileMessageSet.flush() + // down conversion for compressed messages + convertedMessageSet = fileMessageSet.toMessageFormat(Message.MagicValue_V0) + i = 0 + for (messageAndOffset <- convertedMessageSet) { + val message = messageAndOffset.message + val offset = messageAndOffset.offset + assertEquals("magic byte should be 0", Message.MagicValue_V0, message.magic) + assertEquals("offset should not change", offsets(i), offset) + assertEquals("key should not change", messagesV0(i).key, message.key) + assertEquals("payload should not change", messagesV0(i).payload, message.payload) + i += 1 + } + + // 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) + i = 0 + for (messageAndOffset <- convertedMessageSet) { + val message = messageAndOffset.message + val offset = messageAndOffset.offset + assertEquals("magic byte should be 1", Message.MagicValue_V1, message.magic) + assertEquals("offset should not change", offsets(i), offset) + assertEquals("key should not change", messagesV0(i).key, message.key) + assertEquals("payload should not change", messagesV0(i).payload, message.payload) + i += 1 + } + + fileMessageSet = new FileMessageSet(tempFile()) + fileMessageSet.append(compressedMessageSetV0) + fileMessageSet.flush() + // up conversion for compressed messages + convertedMessageSet = fileMessageSet.toMessageFormat(Message.MagicValue_V1) + i = 0 + for (messageAndOffset <- convertedMessageSet) { + val message = messageAndOffset.message + val offset = messageAndOffset.offset + assertEquals("magic byte should be 1", Message.MagicValue_V1, message.magic) + assertEquals("offset should not change", offsets(i), offset) + assertEquals("key should not change", messagesV0(i).key, message.key) + assertEquals("payload should not change", messagesV0(i).payload, message.payload) + i += 1 + } + } } From b8e65cb21a491fbc17fe0c64b1aeb165568549e0 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Tue, 2 Feb 2016 16:22:36 -0800 Subject: [PATCH 14/33] Addressed Jun's comments --- .../main/scala/kafka/log/FileMessageSet.scala | 2 +- .../kafka/message/ByteBufferMessageSet.scala | 90 +++++++++++-------- .../main/scala/kafka/message/MessageSet.scala | 2 +- .../scala/kafka/message/MessageWriter.scala | 2 +- .../main/scala/kafka/server/KafkaApis.scala | 30 ++++--- .../main/scala/kafka/server/KafkaConfig.scala | 3 +- .../scala/kafka/server/ReplicaManager.scala | 4 + .../message/ByteBufferMessageSetTest.scala | 2 +- 8 files changed, 83 insertions(+), 52 deletions(-) diff --git a/core/src/main/scala/kafka/log/FileMessageSet.scala b/core/src/main/scala/kafka/log/FileMessageSet.scala index d12274999c209..91a5945d470b8 100755 --- a/core/src/main/scala/kafka/log/FileMessageSet.scala +++ b/core/src/main/scala/kafka/log/FileMessageSet.scala @@ -179,7 +179,7 @@ class FileMessageSet private[kafka](@volatile var file: File, * @param expectedMagicValue the magic value expected * @return true if all messages has expected magic value, false otherwise */ - override def hasMagicValue(expectedMagicValue: Byte): Boolean = { + override def magicValueInAllMessages(expectedMagicValue: Byte): Boolean = { var location = start val offsetAndSizeBuffer = ByteBuffer.allocate(MessageSet.LogOverhead) val crcAndMagicByteBuffer = ByteBuffer.allocate(Message.CrcLength + Message.MagicLength) diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 26d587ef11a83..93860daf6b894 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -96,29 +96,7 @@ object ByteBufferMessageSet { val inputStream: InputStream = new ByteBufferBackedInputStream(wrapperMessage.payload) val compressed: DataInputStream = new DataInputStream(CompressionFactory(wrapperMessage.compressionCodec, inputStream)) var lastInnerOffset = -1L - // When magic value is greater than 0, relative offset will be used. - // 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 stream compressing 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 producer create 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 = Inner_Offset_of_a_message - Inner_Offset_of_the_last_message - // AO = AO_Of_Last_Inner_Message + RO - // - // 4. This solution works for compacted message set as well + val messageAndOffsets = if (wrapperMessageAndOffset.message.magic > MagicValue_V0) { var innerMessageAndOffsets = new mutable.Queue[MessageAndOffset]() try { @@ -228,6 +206,52 @@ private class OffsetAssigner(offsets: Seq[Long]) { * * 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 relative + * offsets for inner messages of compressed messages. + * + * The way timestamp set is following: + * For non-compressed messages: timestamp and timestamp type attribute in the messages is 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 + * + * + * The way relative offset calculated is 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 stream compressing 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 producer create 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 = Inner_Offset_of_a_message - Inner_Offset_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 @@ -286,7 +310,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi written } - override def hasMagicValue(expectedMagicValue: Byte): Boolean = { + override def magicValueInAllMessages(expectedMagicValue: Byte): Boolean = { for (messageAndOffset <- shallowIterator) { if (messageAndOffset.message.magic != expectedMagicValue) return false @@ -381,7 +405,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi val magicValueToUse = if (messageFormatVersion.onOrAfter(KAFKA_0_10_0_IV0)) Message.MagicValue_V1 else Message.MagicValue_V0 if (sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) { // check the magic value - if (!hasMagicValue(magicValueToUse)) { + if (!magicValueInAllMessages(magicValueToUse)) { // Message format conversion convertNonCompressedMessages(offsetCounter, compactedTopic, now, messageTimestampType, magicValueToUse) } else { @@ -451,16 +475,10 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi val attributeOffset = MessageSet.LogOverhead + Message.AttributesOffset val timestamp = buffer.getLong(timestampOffset) val attributes = buffer.get(attributeOffset) - if (messageTimestampType == TimestampType.CreateTime) { - if (timestamp == maxTimestamp && TimestampType.getTimestampType(attributes) == TimestampType.CreateTime) + if (messageTimestampType == TimestampType.CreateTime && timestamp == maxTimestamp) // We don't need to recompute crc if the timestamp is not updated. crcUpdateNeeded = false - else { - // Set timestamp type and timestamp - buffer.putLong(timestampOffset, maxTimestamp) - buffer.put(attributeOffset, TimestampType.setTimestampType(attributes, TimestampType.CreateTime)) - } - } else if (messageTimestampType == TimestampType.LogAppendTime) { + else if (messageTimestampType == TimestampType.LogAppendTime) { // Set timestamp type and timestamp buffer.putLong(timestampOffset, now) buffer.put(attributeOffset, TimestampType.setTimestampType(attributes, TimestampType.LogAppendTime)) @@ -534,10 +552,6 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi message.buffer.putLong(Message.TimestampOffset, now) message.buffer.put(Message.AttributesOffset, TimestampType.setTimestampType(message.attributes, TimestampType.LogAppendTime)) true - } else if (timestampType == TimestampType.CreateTime && - TimestampType.getTimestampType(message.attributes) != TimestampType.CreateTime) { - message.buffer.put(Message.AttributesOffset, TimestampType.setTimestampType(message.attributes, TimestampType.CreateTime)) - true } else false } @@ -567,6 +581,8 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi if (timestampType == TimestampType.CreateTime && math.abs(message.timestamp - now) > timestampDiffMaxMs) throw new InvalidMessageException(s"Timestamp ${message.timestamp} of message is out of range. " + s"The timestamp should be within [${now - timestampDiffMaxMs}, ${now + timestampDiffMaxMs}") + if (message.timestampType == TimestampType.LogAppendTime) + throw new InvalidMessageException(s"Invalid message $message. Producer should not set timestamp type to LogAppendTime.") } /** diff --git a/core/src/main/scala/kafka/message/MessageSet.scala b/core/src/main/scala/kafka/message/MessageSet.scala index f48fbddf1beee..26cab50b08c43 100644 --- a/core/src/main/scala/kafka/message/MessageSet.scala +++ b/core/src/main/scala/kafka/message/MessageSet.scala @@ -94,7 +94,7 @@ abstract class MessageSet extends Iterable[MessageAndOffset] { /** * Check if all the messages in the message set have the expected magic value */ - def hasMagicValue(expectedMagicValue: Byte): Boolean + def magicValueInAllMessages(expectedMagicValue: Byte): Boolean /** * Provides an iterator over the message/offset pairs in this set diff --git a/core/src/main/scala/kafka/message/MessageWriter.scala b/core/src/main/scala/kafka/message/MessageWriter.scala index 01ec5c6186730..46c6ff34f510f 100755 --- a/core/src/main/scala/kafka/message/MessageWriter.scala +++ b/core/src/main/scala/kafka/message/MessageWriter.scala @@ -40,7 +40,7 @@ class MessageWriter(segmentSize: Int) extends BufferingOutputStream(segmentSize) if (codec.codec > 0) attributes = (attributes | (CompressionCodeMask & codec.codec)).toByte if (magicValue > MagicValue_V0) - attributes = TimestampType.setTimestampType(attributes, timestampType) + attributes = TimestampType.setTimestampType(attributes, timestampType) write(attributes) // Write timestamp if (magicValue > MagicValue_V0) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index b0158a81736d2..c0e75225149a7 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -432,12 +432,23 @@ class KafkaApis(val requestChannel: RequestChannel, val convertedResponseStatus = // Need to down-convert message when consumer only takes magic value 0. if (fetchRequest.versionId <= 1) { - responsePartitionData.mapValues({ case data => - if (!data.messages.hasMagicValue(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 + 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 consumer without + // format down conversion. + if (replicaManager.getMessageFormatVersion(tp).exists(_.onOrAfter(KAFKA_0_10_0_IV0)) && + !data.messages.magicValueInAllMessages(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 @@ -471,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) } } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 408d84bb90201..c94249e3bb59d 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -424,7 +424,8 @@ object KafkaConfig { 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." - val MessageTimestampTypeDoc = "Define the whether the timestamp in the message is message create time or log append time." + val MessageTimestampTypeDoc = "Define the whether the timestamp in the message is message create time or log append time. The value should be either" + + " \"CreateTime\" or \"LogAppendTime\"" val MessageTimestampDifferenceMaxMsDoc = "Set maximum allowed time difference between broker local time and message's timestamp. " + "This configuration only works when message.timestamp.type=CreateTime. A message will be rejected if its timestamp exceeds this threshold." /** ********* Replication configuration ***********/ diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index b3eef8f14f84a..be7fea69fd3f2 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -570,6 +570,10 @@ class ReplicaManager(val config: KafkaConfig, } } + def getMessageFormatVersion(topicAndPartition: TopicAndPartition) = { + getReplica(topicAndPartition.topic, topicAndPartition.partition).flatMap(_.log.map(_.config.messageFormatVersion)) + } + def maybeUpdateMetadataCache(correlationId: Int, updateMetadataRequest: UpdateMetadataRequest, metadataCache: MetadataCache) { replicaStateChangeLock synchronized { if(updateMetadataRequest.controllerEpoch < controllerEpoch) { diff --git a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala index ba391559b1030..dd6af4cba4218 100644 --- a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala @@ -273,7 +273,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { case e: InvalidMessageException => } } - + @Test def testAbsoluteOffsetAssignment() { val messages = getMessages(magicValue = Message.MagicValue_V0, codec = NoCompressionCodec) From fd1afde8862fe3db51a3a373d6f11352b798b20f Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Wed, 3 Feb 2016 12:27:26 -0800 Subject: [PATCH 15/33] Addressed Jun's comments --- .../kafka/clients/producer/RecordMetadata.java | 3 +++ .../scala/kafka/api/ProducerResponse.scala | 2 +- core/src/main/scala/kafka/log/LogCleaner.scala | 7 ++++++- .../kafka/message/ByteBufferMessageSet.scala | 18 +++++++++--------- 4 files changed, 19 insertions(+), 11 deletions(-) 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 4e25c7db23795..19d11b2de1794 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,6 +24,9 @@ public final class RecordMetadata { private final long offset; + // The timestamp of the message. + // If LogAppendTime is used for the topic, the timestamp will be a timestamp returned by broker. + // If CreateTime is used for the topic, the timestamp is the timestamp in the corresponding ProducerRecord. private final long timestamp; private final TopicPartition topicPartition; diff --git a/core/src/main/scala/kafka/api/ProducerResponse.scala b/core/src/main/scala/kafka/api/ProducerResponse.scala index 3e0203cc510cb..89d753833dd0a 100644 --- a/core/src/main/scala/kafka/api/ProducerResponse.scala +++ b/core/src/main/scala/kafka/api/ProducerResponse.scala @@ -26,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 diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 87dc10b303618..d4d9f67a7ae72 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -428,12 +428,17 @@ private[log] class Cleaner(val id: Int, // We use absolute offset to compare decide whether retain the message or not. This is handled by // deep iterator. val messages = ByteBufferMessageSet.deepIterator(entry) + var numberOfInnerMessages = 0 val retainedMessages = messages.filter(messageAndOffset => { messagesRead += 1 + numberOfInnerMessages += 1 shouldRetainMessage(source, map, retainDeletes, messageAndOffset) }).toSeq - if (retainedMessages.nonEmpty) + // There is no messages compacted out, write the original message set back + if (retainedMessages.size == numberOfInnerMessages) + ByteBufferMessageSet.writeMessage(writeBuffer, entry.message, entry.offset) + else if (retainedMessages.nonEmpty) compressMessages(writeBuffer, entry.message.compressionCodec, retainedMessages) } } diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 93860daf6b894..be26f0df4a530 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -416,14 +416,14 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi } else { // Deal with compressed messages - // We only need to do re-compression in one of the followings situation: + // We cannot do in place assignment in one of the followings situation: // 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. - // Re-compression situation 1 and 2 - var requireReCompression = sourceCodec != targetCodec || magicValueToUse == Message.MagicValue_V0 + // No in place assignment situation 1 and 2 + var inPlaceAssignment = sourceCodec == targetCodec && magicValueToUse > Message.MagicValue_V0 var maxTimestamp = Message.NoTimestamp val expectedRelativeOffset = new AtomicLong(0) @@ -432,24 +432,24 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi val message = messageAndOffset.message validateMessageKey(message, compactedTopic) if (message.magic > Message.MagicValue_V0 && magicValueToUse > Message.MagicValue_V0) { - // Re-compression situation 3 + // No in place assignment situation 3 // Validate the timestamp validateTimestamp(message, now, messageTimestampType, messageTimestampDiffMaxMs) // Check if we need to overwrite offset if (messageAndOffset.offset != expectedRelativeOffset.getAndIncrement) - requireReCompression = true + inPlaceAssignment = false maxTimestamp = math.max(maxTimestamp, message.timestamp) } - // Re-compression situation 4 + // No in place assignment situation 4 if (message.magic != magicValueToUse) - requireReCompression = true + inPlaceAssignment = false validatedMessages += message.toFormatVersion(magicValueToUse) }) - if (requireReCompression) { - // Re-compression required. + if (!inPlaceAssignment) { + // Cannot do in place assignment required. val messageSetTimestampAssignor = (messages: Seq[Message]) => { if (magicValueToUse == Message.MagicValue_V0) Message.NoTimestamp From 56fc79f58f24beaa60b498f3ba82f55ed0fab976 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Wed, 3 Feb 2016 21:27:27 -0800 Subject: [PATCH 16/33] Changed message format version to v0 and v1 Addressed Jun's comments Use message format v1 for GroupMetadataTopic Use message format v1 in old producer (always sending time) Add message format v1 in tools, including console consumer and mirror maker. Rebased on trunk --- .../kafka/common/record/MemoryRecords.java | 16 ++--- .../apache/kafka/common/record/Record.java | 35 +++++++--- .../kafka/common/record/TimestampType.java | 6 +- .../consumer/internals/FetcherTest.java | 3 +- .../scala/kafka/consumer/BaseConsumer.scala | 25 ++++++- .../kafka/consumer/ConsumerIterator.scala | 9 ++- .../coordinator/GroupMetadataManager.scala | 22 +++---- core/src/main/scala/kafka/log/Log.scala | 2 +- .../src/main/scala/kafka/log/LogCleaner.scala | 19 ++++-- core/src/main/scala/kafka/log/LogConfig.scala | 7 +- .../kafka/message/ByteBufferMessageSet.scala | 46 +++++++------ .../main/scala/kafka/message/Message.scala | 19 +++--- .../kafka/message/MessageAndMetadata.scala | 10 ++- .../scala/kafka/message/MessageWriter.scala | 1 - .../scala/kafka/message/TimestampType.scala | 34 ++++++++-- .../producer/async/DefaultEventHandler.scala | 14 ++-- .../kafka/server/AbstractFetcherThread.scala | 2 +- .../scala/kafka/server/ConfigHandler.scala | 14 +++- .../main/scala/kafka/server/KafkaApis.scala | 2 +- .../main/scala/kafka/server/KafkaConfig.scala | 15 +++-- .../main/scala/kafka/server/KafkaServer.scala | 4 +- .../scala/kafka/tools/ConsoleConsumer.scala | 32 +++++---- .../main/scala/kafka/tools/MirrorMaker.scala | 18 ++++- .../scala/kafka/tools/ReplayLogProducer.scala | 4 +- .../kafka/tools/SimpleConsumerShell.scala | 3 +- .../kafka/api/BaseConsumerTest.scala | 17 ++--- .../kafka/api/PlaintextConsumerTest.scala | 29 ++++---- .../scala/unit/kafka/log/LogManagerTest.scala | 2 +- .../test/scala/unit/kafka/log/LogTest.scala | 6 +- .../message/ByteBufferMessageSetTest.scala | 66 +++++++++---------- .../unit/kafka/message/MessageTest.scala | 20 ++---- .../kafka/message/MessageWriterTest.scala | 2 +- .../kafka/producer/AsyncProducerTest.scala | 13 ++-- .../unit/kafka/producer/ProducerTest.scala | 32 +++++++-- .../unit/kafka/server/KafkaConfigTest.scala | 2 + .../unit/kafka/server/LogOffsetTest.scala | 2 +- .../kafka/tools/ConsoleConsumerTest.scala | 4 +- 37 files changed, 340 insertions(+), 217 deletions(-) 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 00f1e6a26bd24..4ebe3dce2ad57 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 @@ -217,8 +217,7 @@ public static class RecordsIterator extends AbstractIterator { // The variables for inner iterator private final LinkedList logEntries; - private final long lastInnerRelativeOffset; - private final long wrapperRecordOffset; + private final long absoluteBaseOffset; public RecordsIterator(ByteBuffer buffer, CompressionType type, boolean shallow) { this.type = type; @@ -226,8 +225,7 @@ public RecordsIterator(ByteBuffer buffer, CompressionType type, boolean shallow) this.shallow = shallow; this.stream = Compressor.wrapForInput(new ByteBufferInputStream(this.buffer), type); this.logEntries = null; - this.lastInnerRelativeOffset = -1L; - this.wrapperRecordOffset = -1L; + this.absoluteBaseOffset = -1; } private RecordsIterator(LogEntry entry, CompressionType type) { @@ -235,7 +233,7 @@ private RecordsIterator(LogEntry entry, CompressionType type) { this.buffer = entry.record().value(); this.shallow = true; this.stream = Compressor.wrapForInput(new ByteBufferInputStream(this.buffer), type); - this.wrapperRecordOffset = entry.offset(); + 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) { @@ -254,10 +252,10 @@ private RecordsIterator(LogEntry entry, CompressionType type) { throw new KafkaException(e); } } - this.lastInnerRelativeOffset = logEntries.getLast().offset(); + this.absoluteBaseOffset = wrapperRecordOffset - logEntries.getLast().offset(); } else { this.logEntries = null; - this.lastInnerRelativeOffset = -1L; + this.absoluteBaseOffset = -1; } } @@ -279,8 +277,8 @@ protected LogEntry makeNext() { return allDone(); // Convert offset to absolute offset if needed. - if (lastInnerRelativeOffset >= 0) { - long absoluteOffset = entry.offset() - lastInnerRelativeOffset + wrapperRecordOffset; + if (absoluteBaseOffset >= 0) { + long absoluteOffset = absoluteBaseOffset + entry.offset(); entry = new LogEntry(absoluteOffset, entry.record()); } 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 ed01a5f5743b9..155b1e467e777 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 @@ -339,8 +339,14 @@ public long timestamp() { } } + /** + * The timestamp of the message. + */ public TimestampType timestampType() { - return wrapperRecordTimestampType == null ? TimestampType.getTimestampType(attributes()) : wrapperRecordTimestampType; + if (magic() == 0) + return TimestampType.NoTimestampType; + else + return wrapperRecordTimestampType == null ? TimestampType.getTimestampType(attributes()) : wrapperRecordTimestampType; } /** @@ -385,15 +391,24 @@ private ByteBuffer sliceDelimited(int start) { } public String toString() { - 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()); + 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) { diff --git a/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java b/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java index eb91625ea0846..34073c289d59a 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java +++ b/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java @@ -21,10 +21,10 @@ * The timestamp type of the records. */ public enum TimestampType { - CreateTime(0, "CreateTime"), LogAppendTime(1, "LogAppendTime"); + NoTimestampType(-1, "NoTimestampType"), CreateTime(0, "CreateTime"), LogAppendTime(1, "LogAppendTime"); - public int value; - public String name; + public final int value; + public final String name; TimestampType(int value, String name) { this.value = value; this.name = name; 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 e8b39efb24cc0..97c3d85a57185 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.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; @@ -469,7 +470,7 @@ public void testQuotaMetrics() throws Exception { 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(); } diff --git a/core/src/main/scala/kafka/consumer/BaseConsumer.scala b/core/src/main/scala/kafka/consumer/BaseConsumer.scala index 7942f5752e7ea..76b5d7af01ba0 100644 --- a/core/src/main/scala/kafka/consumer/BaseConsumer.scala +++ b/core/src/main/scala/kafka/consumer/BaseConsumer.scala @@ -21,6 +21,7 @@ import java.util.Properties import java.util.regex.Pattern import kafka.common.StreamEndException +import kafka.message.{NoTimestampType, TimestampType, Message} import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener /** @@ -35,7 +36,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 = NoTimestampType, + 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 +67,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(), + TimestampType.forName(record.timestampType().name), + record.key, + record.value) } override def stop() { @@ -89,7 +102,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() { diff --git a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala index 0c6c810bdc567..07356517fed2f 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() { diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index 434d2fe90db08..a59add3fa628a 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -141,9 +141,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) - // Tombstone does not care about message magic value, simply use magic value 0. val tombstone = new Message(bytes = null, key = GroupMetadataManager.groupMetadataKey(group.groupId), - timestamp = Message.NoTimestamp, magicValue = Message.MagicValue_V0) + timestamp = System.currentTimeMillis(), magicValue = Message.MagicValue_V1) val partitionOpt = replicaManager.getPartition(GroupCoordinator.GroupMetadataTopicName, groupPartition) partitionOpt.foreach { partition => @@ -168,12 +167,11 @@ class GroupMetadataManager(val brokerId: Int, def prepareStoreGroup(group: GroupMetadata, groupAssignment: Map[String, Array[Byte]], responseCallback: Short => Unit): DelayedStore = { - // construct the message to append. GroupMetadata does not use message timestamp so we use magic value 0 val message = new Message( key = GroupMetadataManager.groupMetadataKey(group.groupId), bytes = GroupMetadataManager.groupMetadataValue(group, groupAssignment), - timestamp = Message.NoTimestamp, - magicValue = Message.MagicValue_V0 + timestamp = System.currentTimeMillis(), + magicValue = Message.MagicValue_V1 ) val groupMetadataPartition = new TopicPartition(GroupCoordinator.GroupMetadataTopicName, partitionFor(group.groupId)) @@ -256,8 +254,8 @@ class GroupMetadataManager(val brokerId: Int, new Message( key = GroupMetadataManager.offsetCommitKey(groupId, topicAndPartition.topic, topicAndPartition.partition), bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata), - timestamp = Message.NoTimestamp, - magicValue = Message.MagicValue_V0 + timestamp = System.currentTimeMillis(), + magicValue = Message.MagicValue_V1 ) }.toSeq @@ -557,7 +555,7 @@ class GroupMetadataManager(val brokerId: Int, val commitKey = GroupMetadataManager.offsetCommitKey(groupTopicAndPartition.group, groupTopicAndPartition.topicPartition.topic, groupTopicAndPartition.topicPartition.partition) - (offsetsPartition, new Message(bytes = null, key = commitKey, timestamp = Message.NoTimestamp, magicValue = Message.MagicValue_V0)) + (offsetsPartition, new Message(bytes = null, key = commitKey, timestamp = System.currentTimeMillis(), magicValue = Message.MagicValue_V1)) }.groupBy { case (partition, tombstone) => partition } // Append the tombstone messages to the offset partitions. It is okay if the replicas don't receive these (say, @@ -960,9 +958,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 @@ -977,9 +975,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 diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index a5e3acee4e72a..c6aaf71195f1c 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -348,7 +348,7 @@ class Log(val dir: File, } 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.LogAppendTime) + if (config.messageTimestampType == LogAppendTime) appendInfo.timestamp = now } else { // we are taking the offsets we are given diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index d4d9f67a7ae72..a5511b2793849 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 @@ -403,8 +403,12 @@ private[log] class Cleaner(val id: Int, * @param retainDeletes Should delete tombstones be retained while cleaning this segment * */ - 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: Int) { var position = 0 while (position < source.log.sizeInBytes) { checkDone(topicAndPartition) @@ -439,7 +443,7 @@ private[log] class Cleaner(val id: Int, if (retainedMessages.size == numberOfInnerMessages) ByteBufferMessageSet.writeMessage(writeBuffer, entry.message, entry.offset) else if (retainedMessages.nonEmpty) - compressMessages(writeBuffer, entry.message.compressionCodec, retainedMessages) + compressMessages(writeBuffer, entry.message.compressionCodec, messageFormatVersion, retainedMessages) } } @@ -459,7 +463,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: Int, + messages: Seq[MessageAndOffset]) { val messagesIterable = messages.toIterable.map(_.message) if (messages.isEmpty) { MessageSet.Empty.sizeInBytes @@ -471,7 +478,7 @@ private[log] class Cleaner(val id: Int, val messageSetTimestamp = MessageSet.validateMagicValuesAndGetTimestamp(messages.map(_.message)) val firstAbsoluteOffset = messages.head.offset var offset = -1L - val magicValue = messages.head.message.magic + val magicValue = if (messageFormatVersion == 0) Message.MagicValue_V0 else Message.MagicValue_V1 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, timestamp = messageSetTimestamp, timestampType = timestampType, magicValue = magicValue) { outputStream => diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index 651caad4e9db2..a974d6c9e8097 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -18,7 +18,6 @@ package kafka.log import java.util.Properties -import kafka.api.ApiVersion import kafka.server.KafkaConfig import org.apache.kafka.common.utils.Utils import scala.collection._ @@ -72,8 +71,8 @@ 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)) - val messageTimestampType = TimestampType.withName(getString(LogConfig.MessageTimestampTypeProp)) + val messageFormatVersion = Integer.parseInt(getString(LogConfig.MessageFormatVersionProp).substring(1)) + val messageTimestampType = TimestampType.forName(getString(LogConfig.MessageTimestampTypeProp)) val messageTimestampDifferenceMaxMs = getLong(LogConfig.MessageTimestampDifferenceMaxMsProp) def randomSegmentJitter: Long = @@ -170,7 +169,7 @@ 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(MessageFormatVersionProp, STRING, Defaults.MessageFormatVersion, in("v0", "v1"), MEDIUM, MessageFormatVersionDoc) .define(MessageTimestampTypeProp, STRING, Defaults.MessageTimestampType, MEDIUM, MessageTimestampTypeDoc) .define(MessageTimestampDifferenceMaxMsProp, LONG, Defaults.MessageTimestampDifferenceMaxMs, atLeast(0), MEDIUM, MessageTimestampDifferenceMaxMsDoc) } diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index be26f0df4a530..09f9faefb99e8 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -17,8 +17,6 @@ package kafka.message -import kafka.api.{KAFKA_0_10_0_IV0, ApiVersion} -import kafka.message.TimestampType.TimestampType import kafka.utils.{IteratorTemplate, Logging} import kafka.common.KafkaException @@ -27,7 +25,7 @@ import java.nio.channels._ import java.io._ import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} -import org.apache.kafka.common.utils.{Crc32, Utils} +import org.apache.kafka.common.utils.Utils import scala.collection.mutable import scala.collection.mutable.ListBuffer @@ -36,7 +34,7 @@ object ByteBufferMessageSet { private def create(offsetAssignor: OffsetAssigner, compressionCodec: CompressionCodec, - messageSetTimestampAssignor: (Seq[Message]) => Long, + wrapperMessageTimestamp: Option[Long], timestampType: TimestampType, messages: Message*): ByteBuffer = { if(messages.size == 0) { @@ -48,7 +46,7 @@ object ByteBufferMessageSet { buffer.rewind() buffer } else { - val messageSetTimestamp = messageSetTimestampAssignor(messages) + val messageSetTimestamp = wrapperMessageTimestamp.getOrElse(MessageSet.validateMagicValuesAndGetTimestamp(messages)) var offset = -1L val magicValue = messages.head.magic val messageWriter = new MessageWriter(math.min(math.max(MessageSet.messageSetSize(messages) / 2, 1024), 1 << 16)) @@ -258,26 +256,26 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi def this(compressionCodec: CompressionCodec, messages: Message*) { this(ByteBufferMessageSet.create(new OffsetAssigner(new AtomicLong(0), messages.size), compressionCodec, - MessageSet.validateMagicValuesAndGetTimestamp, TimestampType.CreateTime, messages:_*)) + None, CreateTime, messages:_*)) } def this(compressionCodec: CompressionCodec, offsetCounter: AtomicLong, messages: Message*) { this(ByteBufferMessageSet.create(new OffsetAssigner(offsetCounter, messages.size), compressionCodec, - MessageSet.validateMagicValuesAndGetTimestamp, TimestampType.CreateTime, messages:_*)) + None, CreateTime, messages:_*)) } def this(compressionCodec: CompressionCodec, offsetCounter: AtomicLong, - messageSetTimestampAssignor: (Message*) => Long, + wrapperMessageTimestamp: Option[Long], timestampType: TimestampType, messages: Message*) { this(ByteBufferMessageSet.create(new OffsetAssigner(offsetCounter, messages.size), compressionCodec, - messageSetTimestampAssignor, timestampType, messages:_*)) + None, timestampType, messages:_*)) } def this(compressionCodec: CompressionCodec, offsetSeq: Seq[Long], messages: Message*) { this(ByteBufferMessageSet.create(new OffsetAssigner(offsetSeq), compressionCodec, - MessageSet.validateMagicValuesAndGetTimestamp, TimestampType.CreateTime, messages:_*)) + None, CreateTime, messages:_*)) } def this(messages: Message*) { @@ -388,8 +386,8 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi * 3. When magic value = 1, validate and maybe overwrite timestamps of messages. * * This method will convert the messages based on the following scenarios: - * A. Magic value of a message = 0 and messageFormatVersion is above or equals to 0.10.0-IV0 - * B. Magic value of a message = 1 and messageFormatVersion is lower than 0.10.0-IV0 + * A. Magic value of a message = 0 and messageFormatVersion is 0 + * B. Magic value of a message = 1 and messageFormatVersion is 1 * * If no format conversion or value overwriting is required for messages, this method will perform in-place * operations and avoids re-compression. @@ -399,10 +397,10 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi sourceCodec: CompressionCodec, targetCodec: CompressionCodec, compactedTopic: Boolean = false, - messageFormatVersion: ApiVersion = ApiVersion.latestVersion, + messageFormatVersion: Int = Message.CurrentMagicValue.toInt, messageTimestampType: TimestampType, messageTimestampDiffMaxMs: Long): ByteBufferMessageSet = { - val magicValueToUse = if (messageFormatVersion.onOrAfter(KAFKA_0_10_0_IV0)) Message.MagicValue_V1 else Message.MagicValue_V0 + val magicValueToUse = if (messageFormatVersion > 0) Message.MagicValue_V1 else Message.MagicValue_V0 if (sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) { // check the magic value if (!magicValueInAllMessages(magicValueToUse)) { @@ -450,10 +448,10 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi if (!inPlaceAssignment) { // Cannot do in place assignment required. - val messageSetTimestampAssignor = (messages: Seq[Message]) => { + val wrapperMessageTimestamp = { if (magicValueToUse == Message.MagicValue_V0) Message.NoTimestamp - else if (magicValueToUse > Message.MagicValue_V0 && messageTimestampType == TimestampType.CreateTime) + else if (magicValueToUse > Message.MagicValue_V0 && messageTimestampType == CreateTime) maxTimestamp else // Log append time now @@ -461,7 +459,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi new ByteBufferMessageSet(compressionCodec = targetCodec, offsetCounter = offsetCounter, - messageSetTimestampAssignor = messageSetTimestampAssignor, + wrapperMessageTimestamp = Some(wrapperMessageTimestamp), timestampType = messageTimestampType, messages = validatedMessages.toBuffer: _*) } else { @@ -475,13 +473,13 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi val attributeOffset = MessageSet.LogOverhead + Message.AttributesOffset val timestamp = buffer.getLong(timestampOffset) val attributes = buffer.get(attributeOffset) - if (messageTimestampType == TimestampType.CreateTime && timestamp == maxTimestamp) + if (messageTimestampType == CreateTime && timestamp == maxTimestamp) // We don't need to recompute crc if the timestamp is not updated. crcUpdateNeeded = false - else if (messageTimestampType == TimestampType.LogAppendTime) { + else if (messageTimestampType == LogAppendTime) { // Set timestamp type and timestamp buffer.putLong(timestampOffset, now) - buffer.put(attributeOffset, TimestampType.setTimestampType(attributes, TimestampType.LogAppendTime)) + buffer.put(attributeOffset, TimestampType.setTimestampType(attributes, LogAppendTime)) } if (crcUpdateNeeded) { @@ -548,9 +546,9 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi if (message.magic > Message.MagicValue_V0) { validateTimestamp(message, now, timestampType, timestampDiffMaxMs) val crcUpdateNeeded = { - if (timestampType == TimestampType.LogAppendTime) { + if (timestampType == LogAppendTime) { message.buffer.putLong(Message.TimestampOffset, now) - message.buffer.put(Message.AttributesOffset, TimestampType.setTimestampType(message.attributes, TimestampType.LogAppendTime)) + message.buffer.put(Message.AttributesOffset, TimestampType.setTimestampType(message.attributes, LogAppendTime)) true } else false @@ -578,10 +576,10 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi now: Long, timestampType: TimestampType, timestampDiffMaxMs: Long) { - if (timestampType == TimestampType.CreateTime && math.abs(message.timestamp - now) > timestampDiffMaxMs) + if (timestampType == CreateTime && math.abs(message.timestamp - now) > timestampDiffMaxMs) throw new InvalidMessageException(s"Timestamp ${message.timestamp} of message is out of range. " + s"The timestamp should be within [${now - timestampDiffMaxMs}, ${now + timestampDiffMaxMs}") - if (message.timestampType == TimestampType.LogAppendTime) + if (message.timestampType == LogAppendTime) throw new InvalidMessageException(s"Invalid message $message. Producer should not set timestamp type to LogAppendTime.") } diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala index 88cf662cb33da..acf6eecffddfa 100755 --- a/core/src/main/scala/kafka/message/Message.scala +++ b/core/src/main/scala/kafka/message/Message.scala @@ -18,7 +18,6 @@ package kafka.message import java.nio._ -import kafka.message.TimestampType.TimestampType import scala.math._ import kafka.utils._ @@ -135,6 +134,8 @@ class Message(val buffer: ByteBuffer, * 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 timestamp The timestamp of the message. + * @param timestampType The timestamp type of the message. Default to CreateTime * @param key The key of the message (null, if none) * @param payloadOffset The offset into the payload array used to extract payload * @param payloadSize The size of the payload to use @@ -142,6 +143,7 @@ class Message(val buffer: ByteBuffer, def this(bytes: Array[Byte], key: Array[Byte], timestamp: Long, + timestampType: TimestampType = CreateTime, codec: CompressionCodec, payloadOffset: Int, payloadSize: Int, @@ -285,10 +287,9 @@ class Message(val buffer: ByteBuffer, */ def timestamp: Long = { if (magic == MagicValue_V0) - throw new IllegalStateException("The message with magic byte 0 does not have a timestamp") - + Message.NoTimestamp // Case 2 - if (wrapperMessageTimestampType.exists(_ == TimestampType.LogAppendTime) && wrapperMessageTimestamp.isDefined) + else if (wrapperMessageTimestampType.exists(_ == LogAppendTime) && wrapperMessageTimestamp.isDefined) wrapperMessageTimestamp.get else // case 1, 3 buffer.getLong(Message.TimestampOffset) @@ -299,9 +300,9 @@ def timestamp: Long = { */ def timestampType = { if (magic == MagicValue_V0) - throw new IllegalStateException("The message with magic byte 0 does not have a timestamp") - - wrapperMessageTimestampType.getOrElse(TimestampType.getTimestampType(attributes)) + NoTimestampType + else + wrapperMessageTimestampType.getOrElse(TimestampType.getTimestampType(attributes)) } /** @@ -347,7 +348,7 @@ def timestamp: Long = { byteBuffer.put(Message.MagicValue_V1) byteBuffer.put(TimestampType.setTimestampType(attributes, timestampType)) // Up-conversion, insert the timestamp field - if (timestampType == TimestampType.LogAppendTime) + if (timestampType == LogAppendTime) byteBuffer.putLong(now) else byteBuffer.putLong(Message.NoTimestamp) @@ -356,7 +357,7 @@ def timestamp: Long = { // Down-conversion, reserve CRC and update magic byte byteBuffer.position(Message.MagicOffset) byteBuffer.put(Message.MagicValue_V0) - byteBuffer.put(TimestampType.setTimestampType(attributes, TimestampType.CreateTime)) + byteBuffer.put(TimestampType.setTimestampType(attributes, CreateTime)) // Down-conversion, skip the timestamp field byteBuffer.put(buffer.array(), buffer.arrayOffset() + Message.KeySizeOffset_V1, size - Message.KeySizeOffset_V1) } diff --git a/core/src/main/scala/kafka/message/MessageAndMetadata.scala b/core/src/main/scala/kafka/message/MessageAndMetadata.scala index 26b75c8432d7b..9c079fcc57372 100755 --- a/core/src/main/scala/kafka/message/MessageAndMetadata.scala +++ b/core/src/main/scala/kafka/message/MessageAndMetadata.scala @@ -20,10 +20,14 @@ package kafka.message import kafka.serializer.Decoder 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 = CreateTime, keyDecoder: Decoder[K], valueDecoder: Decoder[V]) { - + /** * Return the decoded message key and payload */ diff --git a/core/src/main/scala/kafka/message/MessageWriter.scala b/core/src/main/scala/kafka/message/MessageWriter.scala index 46c6ff34f510f..16043062729a4 100755 --- a/core/src/main/scala/kafka/message/MessageWriter.scala +++ b/core/src/main/scala/kafka/message/MessageWriter.scala @@ -20,7 +20,6 @@ package kafka.message import java.io.{InputStream, OutputStream} import java.nio.ByteBuffer -import kafka.message.TimestampType.TimestampType import org.apache.kafka.common.utils.Crc32 class MessageWriter(segmentSize: Int) extends BufferingOutputStream(segmentSize) { diff --git a/core/src/main/scala/kafka/message/TimestampType.scala b/core/src/main/scala/kafka/message/TimestampType.scala index a959a22e7c6c6..67399486fa0ed 100644 --- a/core/src/main/scala/kafka/message/TimestampType.scala +++ b/core/src/main/scala/kafka/message/TimestampType.scala @@ -20,10 +20,7 @@ package kafka.message /** * The timestamp type of the messages. */ -case object TimestampType extends Enumeration { - type TimestampType = Value - val CreateTime = Value(0, "CreateTime") - val LogAppendTime = Value(1, "LogAppendTime") +object TimestampType { def getTimestampType(attribute: Byte) = { (attribute & Message.TimestampTypeMask) >> Message.TimestampTypeAttributeBitOffset match { @@ -39,4 +36,33 @@ case object TimestampType extends Enumeration { (attribute | Message.TimestampTypeMask).toByte } + def forName(name: String): TimestampType = { + name match { + case CreateTime.name => CreateTime + case LogAppendTime.name => LogAppendTime + case NoTimestampType.name => NoTimestampType + } + } + +} + +sealed trait TimestampType { + def value: Int + def name: String + override def toString = name } + +case object NoTimestampType extends TimestampType { + val value = -1 + val name = "NoTimestampType" +} + +case object CreateTime extends TimestampType { + val value = 0 + val name = "CreateTime" +} + +case object LogAppendTime extends TimestampType { + val value = 1 + val name = "LogAppendTime" +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala index e30963dac9046..7abe48a36fc38 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) @@ -135,16 +137,16 @@ class DefaultEventHandler[K,V](config: ProducerConfig, partKey = e.partKey, message = new Message(key = keyEncoder.toBytes(e.key), bytes = encoder.toBytes(e.message), - timestamp = Message.NoTimestamp, - magicValue = Message.MagicValue_V0)) + 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), - timestamp = Message.NoTimestamp, - magicValue = Message.MagicValue_V0)) + timestamp = time.milliseconds, + magicValue = Message.MagicValue_V1)) } catch { case t: Throwable => producerStats.serializationErrorRate.mark() diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index 4286d0d338ec0..8b688b9885f3a 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. + fetchRequest", t) + warn(s"Error in fetch $fetchRequest", t) inLock(partitionMapLock) { partitionsWithError ++= partitionMap.keys // there is an error occurred while fetching partitions, sleep a while diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala index bc599a0f72184..05c7250f9b595 100644 --- a/core/src/main/scala/kafka/server/ConfigHandler.scala +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -21,6 +21,7 @@ import java.util.Properties 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 +39,23 @@ 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) => + val version = Integer.parseInt(versionString.substring(1)) + if (!kafkaConfig.validateMessageFormatVersion(version)) { + 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 */ diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index c0e75225149a7..015414ff48725 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -442,7 +442,7 @@ class KafkaApis(val requestChannel: RequestChannel, // 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 consumer without // format down conversion. - if (replicaManager.getMessageFormatVersion(tp).exists(_.onOrAfter(KAFKA_0_10_0_IV0)) && + if (replicaManager.getMessageFormatVersion(tp).exists(_ > 0) && !data.messages.magicValueInAllMessages(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)) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index c94249e3bb59d..8b45296b939ea 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -19,7 +19,7 @@ package kafka.server import java.util.Properties -import kafka.api.ApiVersion +import kafka.api.{KAFKA_0_10_0_IV0, ApiVersion} import kafka.cluster.EndPoint import kafka.consumer.ConsumerConfig import kafka.coordinator.OffsetConfig @@ -94,7 +94,7 @@ object Defaults { val LogFlushSchedulerIntervalMs = Long.MaxValue val LogFlushOffsetCheckpointIntervalMs = 60000 val LogPreAllocateEnable = false - val MessageFormatVersion = ApiVersion.latestVersion.toString() + val MessageFormatVersion = "v1" val NumRecoveryThreadsPerDataDir = 1 val AutoCreateTopicsEnable = true val MinInSyncReplicas = 1 @@ -423,7 +423,7 @@ 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." + val MessageFormatVersionDoc = "Specify the message format version the broker will use to append messages to the logs. Valid values are \"v0\" and \"v1\" (case sensitive)." val MessageTimestampTypeDoc = "Define the whether the timestamp in the message is message create time or log append time. The value should be either" + " \"CreateTime\" or \"LogAppendTime\"" val MessageTimestampDifferenceMaxMsDoc = "Set maximum allowed time difference between broker local time and message's timestamp. " + @@ -602,7 +602,7 @@ 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(MessageFormatVersionProp, STRING, Defaults.MessageFormatVersion, in("v0", "v1"), MEDIUM, MessageFormatVersionDoc) .define(MessageTimestampTypeProp, STRING, Defaults.MessageTimestampType, in("CreateTime", "LogAppendTime"), MEDIUM, MessageTimestampTypeDoc) .define(MessageTimestampDifferenceMaxMsProp, LONG, Defaults.MessageTimestampDifferenceMaxMs, atLeast(0), MEDIUM, MessageTimestampDifferenceMaxMsDoc) @@ -795,7 +795,7 @@ 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 = ApiVersion(getString(KafkaConfig.MessageFormatVersionProp)) + val messageFormatVersion = getString(KafkaConfig.MessageFormatVersionProp) val messageTimestampType = getString(KafkaConfig.MessageTimestampTypeProp) val messageTimestampDifferenceMaxMs = getLong(KafkaConfig.MessageTimestampDifferenceMaxMsProp) @@ -975,6 +975,11 @@ 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(validateMessageFormatVersion(Integer.parseInt(messageFormatVersion.substring(1))), s"message.format.version $messageFormatVersion cannot " + + s"be used when inter.broker.protocol.version is set to $interBrokerProtocolVersion") } + def validateMessageFormatVersion(messageFormatVersion: Int): Boolean = { + !(messageFormatVersion > 0 && !interBrokerProtocolVersion.onOrAfter(KAFKA_0_10_0_IV0)) + } } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 9d6936ada8a75..33217bd2294d6 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -75,7 +75,7 @@ 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.toString()) + logProps.put(LogConfig.MessageFormatVersionProp, kafkaConfig.messageFormatVersion) logProps.put(LogConfig.MessageTimestampTypeProp, kafkaConfig.messageTimestampType) logProps.put(LogConfig.MessageTimestampDifferenceMaxMsProp, kafkaConfig.messageTimestampDifferenceMaxMs) logProps @@ -219,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 diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala index 132e4459ea1d6..177d57bf52f95 100755 --- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala @@ -124,7 +124,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 +335,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 +356,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 != NoTimestampType) { + 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 +374,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(s"key:${if (key == null) "null" else new String(key)}, " + + s"value:${if (value == null) "null" else new String(value)}, " + + {if (timestampType != NoTimestampType) s"$timestampType:$timestamp" else ""}) } } 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 +400,12 @@ class ChecksumMessageFormatter extends MessageFormatter { topicStr = "" } - def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream) { - val chksum = new Message(value, key, Message.NoTimestamp, Message.MagicValue_V0).checksum + def writeTo(key: Array[Byte], value: Array[Byte], timestamp: Long, timestampType: TimestampType, output: PrintStream) { + val chksum = + if (timestampType != NoTimestampType) + 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) } } diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index a964f6940e5f1..61177de79f6aa 100755 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -28,7 +28,7 @@ import joptsimple.OptionParser import kafka.client.ClientUtils import kafka.consumer.{BaseConsumerRecord, ConsumerIterator, BaseConsumer, Blacklist, ConsumerConfig, ConsumerThreadId, ConsumerTimeoutException, TopicFilter, Whitelist, ZookeeperConsumerConnector} import kafka.javaapi.consumer.ConsumerRebalanceListener -import kafka.message.MessageAndMetadata +import kafka.message.{TimestampType, MessageAndMetadata} import kafka.metrics.KafkaMetricsGroup import kafka.serializer.DefaultDecoder import kafka.utils.{CommandLineUtils, CoreUtils, Logging} @@ -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(), + TimestampType.forName(record.timestampType().name), + record.key, + record.value) } override def stop() { diff --git a/core/src/main/scala/kafka/tools/ReplayLogProducer.scala b/core/src/main/scala/kafka/tools/ReplayLogProducer.scala index 2b8537b89fafa..d88ec4101e1f2 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() } diff --git a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala index 1c2023c0a7e01..e20b0618ce8c9 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 => diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index 45631eacbf812..58b483ed6b4cf 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -14,6 +14,7 @@ package kafka.api import java.util +import kafka.coordinator.GroupCoordinator import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.common.record.TimestampType @@ -25,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 @@ -277,17 +277,10 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { } protected def sendRecords(numRecords: Int, tp: TopicPartition) { -<<<<<<< HEAD - 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, diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 48e85f677733c..3151b746a1026 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -24,19 +24,17 @@ import kafka.utils.TestUtils import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.producer.KafkaProducer import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer, ByteArraySerializer} -import org.apache.kafka.clients.producer.ProducerConfig -import org.apache.kafka.common.record.CompressionType -import org.apache.kafka.clients.producer.ProducerRecord -import org.apache.kafka.common.record.TimestampType +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.serialization.ByteArrayDeserializer 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 { @@ -294,7 +292,8 @@ 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) @@ -311,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) { @@ -319,8 +318,10 @@ class PlaintextConsumerTest extends BaseConsumerTest { producerProps.setProperty(ProducerConfig.COMPRESSION_TYPE_CONFIG, CompressionType.GZIP.name) 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) + retries = 0, lingerMs = Long.MaxValue, props = Some(producerProps)) + (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() } @@ -677,7 +678,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { startingTimestamp = 0) // Test compressed messages - sendCompressedRecords(numRecords, tp2) + 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) @@ -702,7 +703,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { // Test compressed messages val tp2 = new TopicPartition(topicName, 1) - sendCompressedRecords(numRecords, tp2) + 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.LogAppendTime) diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index f272acb00fadb..199f8b90cc271 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -38,7 +38,7 @@ class LogManagerTest { logProps.put(LogConfig.SegmentIndexBytesProp, 4096: java.lang.Integer) logProps.put(LogConfig.RetentionMsProp, maxLogAgeMs: java.lang.Integer) // We need to use magic value 1 here because some tests requires no message format conversion. - logProps.put(LogConfig.MessageFormatVersionProp, ApiVersion.latestVersion.toString()) + logProps.put(LogConfig.MessageFormatVersionProp, "v1") val logConfig = LogConfig(logProps) var logDir: File = null var logManager: LogManager = null diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 426b5e825e84c..c73a06647f1fe 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -134,7 +134,7 @@ 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()) + logProps.put(LogConfig.MessageFormatVersionProp, "v1") // 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) @@ -164,7 +164,7 @@ class LogTest extends JUnitSuite { 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()) + logProps.put(LogConfig.MessageFormatVersionProp, "v1") 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 @@ -350,7 +350,7 @@ class LogTest extends JUnitSuite { 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()) + logProps.put(LogConfig.MessageFormatVersionProp, "v1") val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) try { diff --git a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala index dd6af4cba4218..02ff2fc4bc903 100644 --- a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala @@ -162,24 +162,24 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { val validatedMessages = messages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(0), sourceCodec = NoCompressionCodec, targetCodec = NoCompressionCodec, - messageFormatVersion = ApiVersion.latestVersion, - messageTimestampType = TimestampType.LogAppendTime, + messageFormatVersion = 1, + messageTimestampType = LogAppendTime, messageTimestampDiffMaxMs = 1000L) val validatedCompressedMessages = compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(0), sourceCodec = DefaultCompressionCodec, targetCodec = DefaultCompressionCodec, - messageFormatVersion = ApiVersion.latestVersion, - messageTimestampType = TimestampType.LogAppendTime, + messageFormatVersion = 1, + messageTimestampType = LogAppendTime, messageTimestampDiffMaxMs = 1000L) val validatedCompressedMessagesWithoutRecompression = compressedMessagesWithoutRecompression.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(0), sourceCodec = DefaultCompressionCodec, targetCodec = DefaultCompressionCodec, - messageFormatVersion = ApiVersion.latestVersion, - messageTimestampType = TimestampType.LogAppendTime, + messageFormatVersion = 1, + messageTimestampType = LogAppendTime, messageTimestampDiffMaxMs = 1000L) val now = System.currentTimeMillis() @@ -187,7 +187,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { for (messageAndOffset <- validatedMessages) { messageAndOffset.message.ensureValid() assertTrue(messageAndOffset.message.timestamp >= startTime && messageAndOffset.message.timestamp <= now) - assertEquals(TimestampType.LogAppendTime, messageAndOffset.message.timestampType) + assertEquals(LogAppendTime, messageAndOffset.message.timestampType) } assertEquals("message set size should not change", compressedMessages.size, validatedCompressedMessages.size) @@ -195,7 +195,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { messageAndOffset.message.ensureValid() assertTrue(s"Timestamp of message ${messageAndOffset.message}} should be between $startTime and $now", messageAndOffset.message.timestamp >= startTime && messageAndOffset.message.timestamp <= now) - assertEquals(TimestampType.LogAppendTime, messageAndOffset.message.timestampType) + assertEquals(LogAppendTime, messageAndOffset.message.timestampType) } assertTrue("MessageSet should still valid", validatedCompressedMessages.shallowIterator.next().message.isValid) @@ -205,7 +205,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { messageAndOffset.message.ensureValid() assertTrue(s"Timestamp of message ${messageAndOffset.message}} should be between $startTime and $now", messageAndOffset.message.timestamp >= startTime && messageAndOffset.message.timestamp <= now) - assertEquals(TimestampType.LogAppendTime, messageAndOffset.message.timestampType) + assertEquals(LogAppendTime, messageAndOffset.message.timestampType) } assertTrue("MessageSet should still valid", validatedCompressedMessagesWithoutRecompression.shallowIterator.next().message.isValid) } @@ -219,27 +219,27 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { val validatedMessages = messages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(0), sourceCodec = NoCompressionCodec, targetCodec = NoCompressionCodec, - messageFormatVersion = ApiVersion.latestVersion, - messageTimestampType = TimestampType.CreateTime, + messageFormatVersion = 1, + messageTimestampType = CreateTime, messageTimestampDiffMaxMs = 1000L) val validatedCompressedMessages = compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(0), sourceCodec = DefaultCompressionCodec, targetCodec = DefaultCompressionCodec, - messageFormatVersion = ApiVersion.latestVersion, - messageTimestampType = TimestampType.CreateTime, + messageFormatVersion = 1, + messageTimestampType = CreateTime, messageTimestampDiffMaxMs = 1000L) for (messageAndOffset <- validatedMessages) { messageAndOffset.message.ensureValid() assertEquals(messageAndOffset.message.timestamp, now) - assertEquals(messageAndOffset.message.timestampType, TimestampType.CreateTime) + assertEquals(messageAndOffset.message.timestampType, CreateTime) } for (messageAndOffset <- validatedCompressedMessages) { messageAndOffset.message.ensureValid() assertEquals(messageAndOffset.message.timestamp, now) - assertEquals(messageAndOffset.message.timestampType, TimestampType.CreateTime) + assertEquals(messageAndOffset.message.timestampType, CreateTime) } } @@ -253,8 +253,8 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { messages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(0), sourceCodec = NoCompressionCodec, targetCodec = NoCompressionCodec, - messageFormatVersion = ApiVersion.latestVersion, - messageTimestampType = TimestampType.CreateTime, + messageFormatVersion = 1, + messageTimestampType = CreateTime, messageTimestampDiffMaxMs = 1000L) fail("Should throw InvalidMessageException.") } catch { @@ -265,8 +265,8 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(0), sourceCodec = DefaultCompressionCodec, targetCodec = DefaultCompressionCodec, - messageFormatVersion = ApiVersion.latestVersion, - messageTimestampType = TimestampType.CreateTime, + messageFormatVersion = 1, + messageTimestampType = CreateTime, messageTimestampDiffMaxMs = 1000L) fail("Should throw InvalidMessageException.") } catch { @@ -284,8 +284,8 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { checkOffsets(messages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(offset), sourceCodec = NoCompressionCodec, targetCodec = NoCompressionCodec, - messageFormatVersion = ApiVersion("0.9.0"), - messageTimestampType = TimestampType.CreateTime, + messageFormatVersion = 0, + messageTimestampType = CreateTime, messageTimestampDiffMaxMs = 1000L), offset) // check compressed messages @@ -293,8 +293,8 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { checkOffsets(compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(offset), sourceCodec = DefaultCompressionCodec, targetCodec = DefaultCompressionCodec, - messageFormatVersion = ApiVersion("0.9.0"), - messageTimestampType = TimestampType.CreateTime, + messageFormatVersion = 0, + messageTimestampType = CreateTime, messageTimestampDiffMaxMs = 1000L), offset) } @@ -311,7 +311,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { val messageWithOffset = messages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(offset), sourceCodec = NoCompressionCodec, targetCodec = NoCompressionCodec, - messageTimestampType = TimestampType.CreateTime, + messageTimestampType = CreateTime, messageTimestampDiffMaxMs = 5000L) checkOffsets(messageWithOffset, offset) @@ -320,7 +320,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { val compressedMessagesWithOffset = compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(offset), sourceCodec = DefaultCompressionCodec, targetCodec = DefaultCompressionCodec, - messageTimestampType = TimestampType.CreateTime, + messageTimestampType = CreateTime, messageTimestampDiffMaxMs = 5000L) checkOffsets(compressedMessagesWithOffset, offset) } @@ -336,8 +336,8 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { checkOffsets(messagesV0.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(offset), sourceCodec = NoCompressionCodec, targetCodec = NoCompressionCodec, - messageFormatVersion = ApiVersion.latestVersion, - messageTimestampType = TimestampType.LogAppendTime, + messageFormatVersion = 1, + messageTimestampType = LogAppendTime, messageTimestampDiffMaxMs = 1000L), offset) // check compressed messages @@ -345,8 +345,8 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { checkOffsets(compressedMessagesV0.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(offset), sourceCodec = DefaultCompressionCodec, targetCodec = DefaultCompressionCodec, - messageFormatVersion = ApiVersion.latestVersion, - messageTimestampType = TimestampType.LogAppendTime, + messageFormatVersion = 1, + messageTimestampType = LogAppendTime, messageTimestampDiffMaxMs = 1000L), offset) // Check down conversion @@ -359,8 +359,8 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { checkOffsets(messagesV1.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(offset), sourceCodec = NoCompressionCodec, targetCodec = NoCompressionCodec, - messageFormatVersion = ApiVersion("0.9.0"), - messageTimestampType = TimestampType.CreateTime, + messageFormatVersion = 0, + messageTimestampType = CreateTime, messageTimestampDiffMaxMs = 5000L), offset) // check compressed messages @@ -368,8 +368,8 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { checkOffsets(compressedMessagesV1.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(offset), sourceCodec = DefaultCompressionCodec, targetCodec = DefaultCompressionCodec, - messageFormatVersion = ApiVersion("0.9.0"), - messageTimestampType = TimestampType.CreateTime, + messageFormatVersion = 0, + messageTimestampType = CreateTime, messageTimestampDiffMaxMs = 5000L), offset) } diff --git a/core/src/test/scala/unit/kafka/message/MessageTest.scala b/core/src/test/scala/unit/kafka/message/MessageTest.scala index 146b08ab5d669..3f4433ec3fc4d 100755 --- a/core/src/test/scala/unit/kafka/message/MessageTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageTest.scala @@ -66,16 +66,11 @@ class MessageTest extends JUnitSuite { TestUtils.checkEquals(ByteBuffer.wrap(v.payload), v.message.payload) } // check timestamp - if (v.magicValue > Message.MagicValue_V0) { + if (v.magicValue > Message.MagicValue_V0) assertEquals("Timestamp should be the same", v.timestamp, v.message.timestamp) - } else { - try { - v.message.timestamp - fail("message.timestamp should throw exception.") - } catch { - case e: IllegalStateException => - } - } + else + assertEquals("Timestamp should be the NoTimestamp", Message.NoTimestamp, v.message.timestamp) + // check magic value assertEquals(v.magicValue, v.message.magic) // check key @@ -140,12 +135,7 @@ class MessageTest extends JUnitSuite { assertEquals("Size difference is not expected value", messageV0.size - v.message.size, Message.headerSizeDiff(Message.MagicValue_V1, Message.MagicValue_V0)) assertTrue("Message should still be valid", messageV0.isValid) - try { - messageV0.timestamp - fail("message.timestamp should throw exception.") - } catch { - case e: IllegalStateException => - } + assertEquals("Message should have NoTimestamp", Message.NoTimestamp, messageV0.timestamp) assertEquals("Magic value should be 1 now", messageV0.magic, Message.MagicValue_V0) if (messageV0.hasKey) assertEquals("Message key should not change", messageV0.key, ByteBuffer.wrap(v.key)) diff --git a/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala b/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala index 8436b3fd4bdc6..35b4d3a3d05bc 100644 --- a/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala @@ -34,7 +34,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, timestamp = Message.NoTimestamp, timestampType = TimestampType.CreateTime, magicValue = Message.MagicValue_V0) { output => + writer.write(key = key, codec = codec, timestamp = Message.NoTimestamp, timestampType = CreateTime, magicValue = Message.MagicValue_V0) { output => val out = if (codec == NoCompressionCodec) output else CompressionFactory(codec, output) try { val p = rnd.nextInt(bytes.length) diff --git a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala index cea4ec590bf36..f711ca4e5bf7b 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,13 +471,13 @@ class AsyncProducerTest { } def messagesToSet(messages: Seq[String]): ByteBufferMessageSet = { - new ByteBufferMessageSet(NoCompressionCodec, messages.map(m => new Message(m.getBytes, Message.NoTimestamp, Message.MagicValue_V0)): _*) + 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, timestamp = Message.NoTimestamp, magicValue = Message.MagicValue_V0)): _*) + messages.map(m => new Message(key = key, bytes = m, timestamp = 0L, magicValue = Message.MagicValue_V1)): _*) } } diff --git a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala index ce3b6224ce703..0bc81991ad567 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 @@ -24,7 +25,7 @@ import kafka.admin.AdminUtils import kafka.api.FetchRequestBuilder import kafka.common.FailedToSendMessageException import kafka.consumer.SimpleConsumer -import kafka.message.Message +import kafka.message.{CreateTime, Message} import kafka.serializer.StringEncoder import kafka.server.{KafkaConfig, KafkaRequestHandler, KafkaServer} import kafka.utils._ @@ -164,10 +165,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 +183,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, timestamp = Message.NoTimestamp, magicValue = Message.MagicValue_V1), messageSet(0).message) - assertEquals(new Message(bytes = "test2".getBytes, key = "test".getBytes, timestamp = Message.NoTimestamp, magicValue = Message.MagicValue_V1), 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(CreateTime, 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(CreateTime, messageSet(1).message.timestampType) + assertEquals(Message.MagicValue_V1, messageSet(1).message.magic) producer1.close() val props2 = new util.Properties() @@ -227,7 +240,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 +248,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 +273,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, timestamp = Message.NoTimestamp, magicValue = Message.MagicValue_V1), 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(CreateTime, 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) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index c9ac97acf5c17..b1120bd511084 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -282,11 +282,13 @@ class KafkaConfigTest { assertEquals(ApiVersion.latestVersion, conf.interBrokerProtocolVersion) props.put(KafkaConfig.InterBrokerProtocolVersionProp,"0.8.2.0") + props.put(KafkaConfig.MessageFormatVersionProp, "v0") val conf2 = KafkaConfig.fromProps(props) 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.MessageFormatVersionProp, "v0") val conf3 = KafkaConfig.fromProps(props) assertEquals(KAFKA_0_8_2, conf3.interBrokerProtocolVersion) diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index f9ce10b1de0a5..94b9f52239477 100755 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -206,7 +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", ApiVersion.latestVersion.toString()) + props.put("message.format.version", "v1") props } diff --git a/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala b/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala index 66052fcd8d7dd..de92a24cc8ef4 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) From 67201a6cbbfae1ece48a47f23d9ead2761359e9e Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Mon, 8 Feb 2016 14:30:58 -0800 Subject: [PATCH 17/33] Updated upgrade doc --- docs/upgrade.html | 41 +++++++++++++++++++++++++++++++++++++++++ 1 file changed, 41 insertions(+) diff --git a/docs/upgrade.html b/docs/upgrade.html index ba59cc0d475ff..7e158f874a131 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -17,6 +17,47 @@

    1.5 Upgrading From Previous Versions

    +

    Upgrading from 0.9.x to 0.10.0.0

    +0.10.0.0 has potential performance impact and protocol changes (please review before upgrading). +Because new protocols are introduced, it is important to upgrade your Kafka clusters before upgrading your clients. + +

    For a rolling upgrade:

    + +
      +
    1. Update server.properties file on all brokers and add the following properties: inter.broker.protocol.version=0.9.0.0, message.format.version=v0
    2. +
    3. Upgrade the brokers. This can be done a broker at a time by simply bringing it down, updating the code, and restarting it.
    4. +
    5. Once the entire cluster is upgraded, bump the protocol version by editing inter.broker.protocol.version and setting it to 0.10.0.0.
    6. +
    7. Restart the brokers one by one for the new protocol version to take effect.
    8. +
    + +

    Note: 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. + +

    Note: Bumping the protocol version and restarting can be done any time after the brokers were upgraded. It does not have to be immediately after. + +

    potential performance impact in 0.10.0.0
    +

    + Message format v1 is added to include a new timestamp field and use relative offsets for compressed messages in 0.10.0.0. + The on disk message format can be configured through message.format.version in server.properties file. + A 0.10.0.0 broker is able to serve old clients with message format v0. Messages will be converted to v0 if on disk + message format version is v1. However, Brokers will lose zero-copy when message format conversion occurs. +

    +

    + 0.10.0.0 clients can handle both message format v0 and v1. Hence there is no performance impact for 0.10.0.0 clients + if message.format.version is set to v0 on broker. Broker will not convert the messages but just return message v0 + to new clients. +

    +

    + To avoid performance impact due to message format conversion, only set message.format.version=v1 after upgrading your clients. + message.format.version is a topic configuration, so you can also dynamically set it for a specific topic by changing its topic configuration. +

    + +
    protocol change for 0.10.0.0
    +
      +
    • Message format v1 is added to include a timestamp field in the messages and use relative offsets for compressed messages.
    • +
    • ProduceRequest/Response v2 is added to support message format v1
    • +
    • FetchRequest/Response v2 is added to support message format v1
    • +
    +

    Upgrading from 0.8.0, 0.8.1.X or 0.8.2.X to 0.9.0.0

    0.9.0.0 has potential breaking changes (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. From 92a39bab019894f81e1aafc6105205dde134c366 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Wed, 10 Feb 2016 16:40:57 -0800 Subject: [PATCH 18/33] Addressed Jun's comments Rebased on trunk --- .../clients/producer/ProducerRecord.java | 23 ++- .../producer/internals/RecordAccumulator.java | 2 +- .../kafka/common/protocol/Protocol.java | 6 +- .../apache/kafka/common/record/Record.java | 3 +- .../internals/ConsumerInterceptorsTest.java | 8 +- .../internals/ProducerInterceptorsTest.java | 2 +- .../kafka/test/MockConsumerInterceptor.java | 4 +- .../src/main/scala/kafka/api/ApiVersion.scala | 7 +- .../main/scala/kafka/api/FetchResponse.scala | 3 +- .../coordinator/GroupMetadataManager.scala | 8 +- .../main/scala/kafka/log/FileMessageSet.scala | 2 +- core/src/main/scala/kafka/log/Log.scala | 4 +- .../src/main/scala/kafka/log/LogCleaner.scala | 30 ++-- core/src/main/scala/kafka/log/LogConfig.scala | 21 ++- .../kafka/message/ByteBufferMessageSet.scala | 136 +++++++++--------- .../main/scala/kafka/message/Message.scala | 30 +++- .../main/scala/kafka/message/MessageSet.scala | 16 +-- .../main/scala/kafka/server/KafkaApis.scala | 15 +- core/src/test/resources/log4j.properties | 4 +- .../kafka/api/BaseConsumerTest.scala | 10 +- .../kafka/api/BaseProducerSendTest.scala | 97 +++++++++---- .../message/ByteBufferMessageSetTest.scala | 26 ++-- docs/upgrade.html | 19 ++- 23 files changed, 269 insertions(+), 207 deletions(-) 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 7be5df7774e3e..034fbf31ae042 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 @@ -21,13 +21,19 @@ * present a partition will be assigned in a round-robin fashion. *

    * The record also has an associated timestamp. If user did not provide a timestamp, the producer will stamp the record - * with a timestamp depending on what is the timestamp type used by the topic. + * with its current time. The timestamp eventually used by Kafka depends on the timestam *

  • * If the topic is configured to use {@link org.apache.kafka.common.record.TimestampType#CreateTime CreateTime} - * the timestamp will be user specified timestamp or the producer's current time. + * the timestamp in the producer record will be used by broker. + *
  • *
  • * If the topic is configured to use {@link org.apache.kafka.common.record.TimestampType#LogAppendTime LogAppendTime} - * the timestamp will be the time when Kafka broker accepts the record. + * the timestamp in the producer record will be overwritten by broker with broker local time when broker append the + * message to its log. + *
  • + *

    + * In either of the above case, the timestamp that has actually been used will be returned to user in + * {@link RecordMetadata} */ public final class ProducerRecord { @@ -81,17 +87,6 @@ public ProducerRecord(String topic, K key, V value) { this(topic, null, null, key, value); } - /** - * Create a record with specific timestamp but no key - * - * @param topic The topic the record will be appended to - * @param value The record contents - * @param timestamp The timestamp of the record - */ - public ProducerRecord(String topic, Long timestamp, V value) { - this(topic, null, timestamp, null, value); - } - /** * Create a record with no key * 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 7018d4484cb61..f1414f01472e2 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,9 +146,9 @@ public double measure(MetricConfig config, long now) { *

    * * @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 timestamp The timestamp of 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 */ 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 a2f05c2adf914..52c919e27d099 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 @@ -151,7 +151,11 @@ public class Protocol { new Field("base_offset", INT64), new Field("timestamp", - INT64))))))), + 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" + 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 155b1e467e777..388de93893dd4 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 @@ -110,7 +110,8 @@ public Record(ByteBuffer buffer) { * 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) 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 45210a880bb14..d0748c167a45b 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.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 consumerRecord = new ConsumerRecord<>(topic, partition, 0, 1, 1); + private final ConsumerRecord consumerRecord = + new ConsumerRecord<>(topic, partition, 0, 0L, TimestampType.CreateTime, 1, 1); private int onCommitCount = 0; private int onConsumeCount = 0; @@ -115,9 +117,9 @@ public void testOnConsumeChain() { List> list1 = new ArrayList<>(); list1.add(consumerRecord); List> 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.CreateTime, 1, 1)); List> 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.CreateTime, 1, 1)); records.put(tp, list1); records.put(filterTopicPart1, list2); records.put(filterTopicPart2, list3); 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 18a455f2ae09e..26d15d0f15ba6 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 void testOnAcknowledgementChain() { ProducerInterceptors 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); 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 8295b54c38e39..ea875cd2a762b 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.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 ConsumerRecords onConsume(ConsumerRecords for (TopicPartition tp : records.partitions()) { List> lst = new ArrayList<>(); for (ConsumerRecord 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.CreateTime, record.key(), record.value().toUpperCase())); } recordMap.put(tp, lst); } diff --git a/core/src/main/scala/kafka/api/ApiVersion.scala b/core/src/main/scala/kafka/api/ApiVersion.scala index 9017fc35191cc..2de7d7b4783dc 100644 --- a/core/src/main/scala/kafka/api/ApiVersion.scala +++ b/core/src/main/scala/kafka/api/ApiVersion.scala @@ -29,7 +29,7 @@ package kafka.api * help users who are running on trunk upgrade. For example: * 1. Kafka 0.9.0 is released * 2. After that some protocol change are made and will be released in 0.10.0. The version will be named 0.10.0-IV0 - * (DV stands for internal version) + * (IV stands for internal version) * 3. When Kafka 0.10.0 is released. The official version 0.10.0 will be the same as the last internal version. */ object ApiVersion { @@ -90,9 +90,4 @@ case object KAFKA_0_9_0 extends ApiVersion { case object KAFKA_0_10_0_IV0 extends ApiVersion { val version: String = "0.10.0-IV0" val id: Int = 4 -} - -case object KAFKA_0_10_0 extends ApiVersion { - val version: String = "0.10.0-IV0" - val id: Int = 5 } \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/FetchResponse.scala b/core/src/main/scala/kafka/api/FetchResponse.scala index 42f93c192b16e..1066d7f9b4bb2 100644 --- a/core/src/main/scala/kafka/api/FetchResponse.scala +++ b/core/src/main/scala/kafka/api/FetchResponse.scala @@ -21,14 +21,13 @@ import java.nio.ByteBuffer import java.nio.channels.GatheringByteChannel import kafka.common.TopicAndPartition -import kafka.message.{NoCompressionCodec, Message, MessageSet, ByteBufferMessageSet} +import kafka.message.{MessageSet, ByteBufferMessageSet} import kafka.api.ApiUtils._ import org.apache.kafka.common.KafkaException import org.apache.kafka.common.network.{Send, MultiSend} import org.apache.kafka.common.protocol.Errors import scala.collection._ -import scala.collection.mutable.ArrayBuffer object FetchResponsePartitionData { def readFrom(buffer: ByteBuffer): FetchResponsePartitionData = { diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index a59add3fa628a..b487e6eb8dacd 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -142,7 +142,7 @@ class GroupMetadataManager(val brokerId: Int, // retry removing this group. val groupPartition = partitionFor(group.groupId) val tombstone = new Message(bytes = null, key = GroupMetadataManager.groupMetadataKey(group.groupId), - timestamp = System.currentTimeMillis(), magicValue = Message.MagicValue_V1) + timestamp = SystemTime.milliseconds, magicValue = Message.MagicValue_V1) val partitionOpt = replicaManager.getPartition(GroupCoordinator.GroupMetadataTopicName, groupPartition) partitionOpt.foreach { partition => @@ -170,7 +170,7 @@ class GroupMetadataManager(val brokerId: Int, val message = new Message( key = GroupMetadataManager.groupMetadataKey(group.groupId), bytes = GroupMetadataManager.groupMetadataValue(group, groupAssignment), - timestamp = System.currentTimeMillis(), + timestamp = SystemTime.milliseconds, magicValue = Message.MagicValue_V1 ) @@ -254,7 +254,7 @@ class GroupMetadataManager(val brokerId: Int, new Message( key = GroupMetadataManager.offsetCommitKey(groupId, topicAndPartition.topic, topicAndPartition.partition), bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata), - timestamp = System.currentTimeMillis(), + timestamp = SystemTime.milliseconds, magicValue = Message.MagicValue_V1 ) }.toSeq @@ -555,7 +555,7 @@ class GroupMetadataManager(val brokerId: Int, val commitKey = GroupMetadataManager.offsetCommitKey(groupTopicAndPartition.group, groupTopicAndPartition.topicPartition.topic, groupTopicAndPartition.topicPartition.partition) - (offsetsPartition, new Message(bytes = null, key = commitKey, timestamp = System.currentTimeMillis(), magicValue = Message.MagicValue_V1)) + (offsetsPartition, new Message(bytes = null, key = commitKey, timestamp = SystemTime.milliseconds, magicValue = Message.MagicValue_V1)) }.groupBy { case (partition, tombstone) => partition } // Append the tombstone messages to the offset partitions. It is okay if the replicas don't receive these (say, diff --git a/core/src/main/scala/kafka/log/FileMessageSet.scala b/core/src/main/scala/kafka/log/FileMessageSet.scala index 91a5945d470b8..b47a643275b58 100755 --- a/core/src/main/scala/kafka/log/FileMessageSet.scala +++ b/core/src/main/scala/kafka/log/FileMessageSet.scala @@ -179,7 +179,7 @@ class FileMessageSet private[kafka](@volatile var file: File, * @param expectedMagicValue the magic value expected * @return true if all messages has expected magic value, false otherwise */ - override def magicValueInAllMessages(expectedMagicValue: Byte): Boolean = { + override def magicValueInAllWrapperMessages(expectedMagicValue: Byte): Boolean = { var location = start val offsetAndSizeBuffer = ByteBuffer.allocate(MessageSet.LogOverhead) val crcAndMagicByteBuffer = ByteBuffer.allocate(Message.CrcLength + Message.MagicLength) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index c6aaf71195f1c..c0119a6707b15 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -42,10 +42,10 @@ object LogAppendInfo { * @param firstOffset The first offset in the message set * @param lastOffset The last offset in the message set * @param timestamp The log append time (if used) of the message set, otherwise Message.NoTimestamp - * @param shallowCount The number of shallow messages - * @param validBytes The number of valid bytes * @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, diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index a5511b2793849..a1ad00889066d 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -401,14 +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, - messageFormatVersion: Int) { + messageFormatVersion: Byte) { var position = 0 while (position < source.log.sizeInBytes) { checkDone(topicAndPartition) @@ -433,17 +433,24 @@ private[log] class Cleaner(val id: Int, // 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 - // There is no messages compacted out, write the original message set back - if (retainedMessages.size == numberOfInnerMessages) + // 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) - compressMessages(writeBuffer, entry.message.compressionCodec, messageFormatVersion, retainedMessages) + else if (retainedMessages.nonEmpty) { + val convertedRetainedMessages = retainedMessages.map(messageAndOffset => { + new MessageAndOffset(messageAndOffset.message.toFormatVersion(messageFormatVersion), messageAndOffset.offset) + }) + compressMessages(writeBuffer, entry.message.compressionCodec, messageFormatVersion, convertedRetainedMessages) + } } } @@ -465,7 +472,7 @@ private[log] class Cleaner(val id: Int, private def compressMessages(buffer: ByteBuffer, compressionCodec: CompressionCodec, - messageFormatVersion: Int, + messageFormatVersion: Byte, messages: Seq[MessageAndOffset]) { val messagesIterable = messages.toIterable.map(_.message) if (messages.isEmpty) { @@ -478,20 +485,19 @@ private[log] class Cleaner(val id: Int, val messageSetTimestamp = MessageSet.validateMagicValuesAndGetTimestamp(messages.map(_.message)) val firstAbsoluteOffset = messages.head.offset var offset = -1L - val magicValue = if (messageFormatVersion == 0) Message.MagicValue_V0 else Message.MagicValue_V1 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, timestamp = messageSetTimestamp, timestampType = timestampType, magicValue = magicValue) { outputStream => + messageWriter.write(codec = compressionCodec, timestamp = messageSetTimestamp, timestampType = timestampType, magicValue = messageFormatVersion) { outputStream => val output = new DataOutputStream(CompressionFactory(compressionCodec, outputStream)) try { for (messageOffset <- messages) { val message = messageOffset.message offset = messageOffset.offset // Use relative offset when magic value is greater than 0 - if (magicValue > Message.MagicValue_V0) { + if (messageFormatVersion > Message.MagicValue_V0) { // The offset of the messages are absolute offset, compute the relative offset. - val relativeOffset = messageOffset.offset - firstAbsoluteOffset - output.writeLong(relativeOffset) + val innerOffset = messageOffset.offset - firstAbsoluteOffset + output.writeLong(innerOffset) } else output.writeLong(offset) output.writeInt(message.size) diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index a974d6c9e8097..c8bfe310fb2e7 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -18,11 +18,11 @@ package kafka.log import java.util.Properties + +import kafka.message.{BrokerCompressionCodec, Message, TimestampType} 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.{TimestampType, BrokerCompressionCodec, Message} +import org.apache.kafka.common.utils.Utils object Defaults { val SegmentSize = kafka.server.Defaults.LogSegmentBytes @@ -71,7 +71,7 @@ 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 = Integer.parseInt(getString(LogConfig.MessageFormatVersionProp).substring(1)) + val messageFormatVersion = Integer.parseInt(getString(LogConfig.MessageFormatVersionProp).substring(1)).toByte val messageTimestampType = TimestampType.forName(getString(LogConfig.MessageTimestampTypeProp)) val messageTimestampDifferenceMaxMs = getLong(LogConfig.MessageTimestampDifferenceMaxMsProp) @@ -138,11 +138,10 @@ object LogConfig { 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) @@ -177,7 +176,7 @@ object LogConfig { def apply(): LogConfig = LogConfig(new Properties()) def configNames() = { - import JavaConversions._ + import scala.collection.JavaConversions._ configDef.names().toList.sorted } @@ -196,7 +195,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)) diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 09f9faefb99e8..75778aa76e185 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -46,20 +46,21 @@ object ByteBufferMessageSet { buffer.rewind() buffer } else { + System.out.println("wrapper message timestamp " + wrapperMessageTimestamp) val messageSetTimestamp = wrapperMessageTimestamp.getOrElse(MessageSet.validateMagicValuesAndGetTimestamp(messages)) var offset = -1L - val magicValue = messages.head.magic + val firstMagicValue = messages.head.magic val messageWriter = new MessageWriter(math.min(math.max(MessageSet.messageSetSize(messages) / 2, 1024), 1 << 16)) - messageWriter.write(codec = compressionCodec, timestamp = messageSetTimestamp, timestampType = timestampType, magicValue = magicValue) { outputStream => + messageWriter.write(codec = compressionCodec, timestamp = messageSetTimestamp, timestampType = timestampType, magicValue = firstMagicValue) { outputStream => val output = new DataOutputStream(CompressionFactory(compressionCodec, outputStream)) try { for (message <- messages) { offset = offsetAssignor.nextAbsoluteOffset - if (message.magic != magicValue) - throw new IllegalArgumentException("Messages in the same compressed message set must have same magic value") + if (message.magic != firstMagicValue) + throw new IllegalArgumentException("Messages in the message set must have same magic value") // Use relative offset if magic value is greater than 0 - if (magicValue > Message.MagicValue_V0) - output.writeLong(offsetAssignor.toRelativeOffset(offset)) + if (firstMagicValue > Message.MagicValue_V0) + output.writeLong(offsetAssignor.toInnerOffset(offset)) else output.writeLong(offset) output.writeInt(message.size) @@ -76,7 +77,7 @@ object ByteBufferMessageSet { } } - /** Deep iterator that decompresses the message sets and adjust timestamp and offset if needed. */ + /** Deep iterator that decompresses the message sets and adjusts timestamp and offset if needed. */ def deepIterator(wrapperMessageAndOffset: MessageAndOffset): Iterator[MessageAndOffset] = { import Message._ @@ -90,7 +91,7 @@ object ByteBufferMessageSet { val wrapperMessageTimestampTypeOpt: Option[TimestampType] = if (wrapperMessage.magic > MagicValue_V0) Some(wrapperMessage.timestampType) else None if (wrapperMessage.payload == null) - throw new RuntimeException("wrapper message = " + wrapperMessage) + throw new RuntimeException("Message payload is null: " + wrapperMessage) val inputStream: InputStream = new ByteBufferBackedInputStream(wrapperMessage.payload) val compressed: DataInputStream = new DataInputStream(CompressionFactory(wrapperMessage.compressionCodec, inputStream)) var lastInnerOffset = -1L @@ -191,7 +192,7 @@ private class OffsetAssigner(offsets: Seq[Long]) { def nextAbsoluteOffset = offsets(index.getAndIncrement) - def toRelativeOffset(offset: Long) = offset - offsets(0) + def toInnerOffset(offset: Long) = offset - offsets(0) } @@ -209,7 +210,8 @@ private class OffsetAssigner(offsets: Seq[Long]) { * - 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 relative - * offsets for inner messages of compressed messages. + * offsets for inner messages of compressed messages. 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 is set and used. @@ -222,10 +224,11 @@ private class OffsetAssigner(offsets: Seq[Long]) { * 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 + * 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 relative offset calculated is following: + * The way absolute offset calculated is following: * Ideally the conversion from relative offset(RO) to absolute offset(AO) should be: * * AO = AO_Of_Last_Inner_Message + RO @@ -245,7 +248,7 @@ private class OffsetAssigner(offsets: Seq[Long]) { * 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 = Inner_Offset_of_a_message - Inner_Offset_of_the_last_message + * 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 @@ -264,15 +267,6 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi None, CreateTime, messages:_*)) } - def this(compressionCodec: CompressionCodec, - offsetCounter: AtomicLong, - wrapperMessageTimestamp: Option[Long], - timestampType: TimestampType, - messages: Message*) { - this(ByteBufferMessageSet.create(new OffsetAssigner(offsetCounter, messages.size), compressionCodec, - None, timestampType, messages:_*)) - } - def this(compressionCodec: CompressionCodec, offsetSeq: Seq[Long], messages: Message*) { this(ByteBufferMessageSet.create(new OffsetAssigner(offsetSeq), compressionCodec, None, CreateTime, messages:_*)) @@ -282,6 +276,16 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi 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 = { @@ -308,7 +312,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi written } - override def magicValueInAllMessages(expectedMagicValue: Byte): Boolean = { + override def magicValueInAllWrapperMessages(expectedMagicValue: Byte): Boolean = { for (messageAndOffset <- shallowIterator) { if (messageAndOffset.message.magic != expectedMagicValue) return false @@ -397,15 +401,14 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi sourceCodec: CompressionCodec, targetCodec: CompressionCodec, compactedTopic: Boolean = false, - messageFormatVersion: Int = Message.CurrentMagicValue.toInt, + messageFormatVersion: Byte = Message.CurrentMagicValue, messageTimestampType: TimestampType, messageTimestampDiffMaxMs: Long): ByteBufferMessageSet = { - val magicValueToUse = if (messageFormatVersion > 0) Message.MagicValue_V1 else Message.MagicValue_V0 if (sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) { // check the magic value - if (!magicValueInAllMessages(magicValueToUse)) { + if (!magicValueInAllWrapperMessages(messageFormatVersion)) { // Message format conversion - convertNonCompressedMessages(offsetCounter, compactedTopic, now, messageTimestampType, magicValueToUse) + convertNonCompressedMessages(offsetCounter, compactedTopic, now, messageTimestampType, messageFormatVersion) } else { // Do in-place validation, offset assignment and maybe set timestamp validateNonCompressedMessagesAndAssignOffsetInPlace(offsetCounter, now, compactedTopic, messageTimestampType, @@ -421,7 +424,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi // 4. Message format conversion is needed. // No in place assignment situation 1 and 2 - var inPlaceAssignment = sourceCodec == targetCodec && magicValueToUse > Message.MagicValue_V0 + var inPlaceAssignment = sourceCodec == targetCodec && messageFormatVersion > Message.MagicValue_V0 var maxTimestamp = Message.NoTimestamp val expectedRelativeOffset = new AtomicLong(0) @@ -429,7 +432,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi this.internalIterator(isShallow = false).foreach(messageAndOffset => { val message = messageAndOffset.message validateMessageKey(message, compactedTopic) - if (message.magic > Message.MagicValue_V0 && magicValueToUse > Message.MagicValue_V0) { + if (message.magic > Message.MagicValue_V0 && messageFormatVersion > Message.MagicValue_V0) { // No in place assignment situation 3 // Validate the timestamp validateTimestamp(message, now, messageTimestampType, messageTimestampDiffMaxMs) @@ -440,26 +443,25 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi } // No in place assignment situation 4 - if (message.magic != magicValueToUse) + if (message.magic != messageFormatVersion) inPlaceAssignment = false - validatedMessages += message.toFormatVersion(magicValueToUse) + validatedMessages += message.toFormatVersion(messageFormatVersion) }) - if (!inPlaceAssignment) { - // Cannot do in place assignment required. + // Cannot do in place assignment. val wrapperMessageTimestamp = { - if (magicValueToUse == Message.MagicValue_V0) - Message.NoTimestamp - else if (magicValueToUse > Message.MagicValue_V0 && messageTimestampType == CreateTime) - maxTimestamp + if (messageFormatVersion == Message.MagicValue_V0) + Some(Message.NoTimestamp) + else if (messageFormatVersion > Message.MagicValue_V0 && messageTimestampType == CreateTime) + Some(maxTimestamp) else // Log append time - now + Some(now) } new ByteBufferMessageSet(compressionCodec = targetCodec, offsetCounter = offsetCounter, - wrapperMessageTimestamp = Some(wrapperMessageTimestamp), + wrapperMessageTimestamp = wrapperMessageTimestamp, timestampType = messageTimestampType, messages = validatedMessages.toBuffer: _*) } else { @@ -467,27 +469,26 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi buffer.putLong(0, offsetCounter.addAndGet(validatedMessages.size) - 1) // validate the messages validatedMessages.foreach(_.ensureValid()) - if (magicValueToUse > Message.MagicValue_V0) { - 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 == CreateTime && timestamp == maxTimestamp) - // We don't need to recompute crc if the timestamp is not updated. - crcUpdateNeeded = false - else if (messageTimestampType == LogAppendTime) { - // Set timestamp type and timestamp - buffer.putLong(timestampOffset, now) - buffer.put(attributeOffset, TimestampType.setTimestampType(attributes, LogAppendTime)) - } - 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()) - } + 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 == CreateTime && timestamp == maxTimestamp) + // We don't need to recompute crc if the timestamp is not updated. + crcUpdateNeeded = false + else if (messageTimestampType == LogAppendTime) { + // Set timestamp type and timestamp + buffer.putLong(timestampOffset, now) + buffer.put(attributeOffset, TimestampType.setTimestampType(attributes, LogAppendTime)) + } + + 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 @@ -495,6 +496,9 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi } } + // We create this method to save memory copy operation. It reads from the original message set and directly + // write the converted messages into new message set buffer. Hence we don't need to allocate memory for each message + // during message conversion. private def convertNonCompressedMessages(offsetCounter: AtomicLong, compactedTopic: Boolean, now: Long, @@ -545,17 +549,11 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi validateMessageKey(message, compactedTopic) if (message.magic > Message.MagicValue_V0) { validateTimestamp(message, now, timestampType, timestampDiffMaxMs) - val crcUpdateNeeded = { - if (timestampType == LogAppendTime) { - message.buffer.putLong(Message.TimestampOffset, now) - message.buffer.put(Message.AttributesOffset, TimestampType.setTimestampType(message.attributes, LogAppendTime)) - true - } else - false - } - // We have to update crc after updating the timestamp or timestamp type. - if (crcUpdateNeeded) + if (timestampType == LogAppendTime) { + message.buffer.putLong(Message.TimestampOffset, now) + message.buffer.put(Message.AttributesOffset, TimestampType.setTimestampType(message.attributes, LogAppendTime)) Utils.writeUnsignedInt(message.buffer, Message.CrcOffset, message.computeChecksum()) + } } messagePosition += MessageSet.LogOverhead + messageSize } diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala index acf6eecffddfa..57591ddfaf3c4 100755 --- a/core/src/main/scala/kafka/message/Message.scala +++ b/core/src/main/scala/kafka/message/Message.scala @@ -58,7 +58,7 @@ object Message { /** * 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 come from a message in message format V0 with empty key and value. + * used here, which comes from a message in message format V0 with empty key and value. */ val MinMessageOverhead = KeyOffset_V0 + ValueSizeLength @@ -66,7 +66,7 @@ object Message { * The minimum valid size for the message header * The MinHeaderSize does not include the TimestampLength for backward compatibility. * 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 come from message format V0. + * used here, which comes from message format V0. */ val MinHeaderSize = MessageHeaderSizeMap.values.min @@ -85,7 +85,7 @@ object Message { */ val CompressionCodeMask: Int = 0x07 /** - * Specifies the mask for timestamp type. 1 bit at the 5th least significant bit. + * Specifies the mask for timestamp type. 1 bit at the 4th least significant bit. * 0 for CreateTime, 1 for LogAppendTime */ val TimestampTypeMask: Byte = 0x08 @@ -115,7 +115,16 @@ object Message { * * 1. 4 byte CRC32 of the message * 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 (e.g. compression enabled, type of codec used) + * 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 @@ -123,6 +132,11 @@ object Message { * 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, private val wrapperMessageTimestamp: Option[Long] = None, @@ -164,8 +178,10 @@ class Message(val buffer: ByteBuffer, buffer.position(MagicOffset) 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) @@ -285,7 +301,7 @@ class Message(val buffer: ByteBuffer, * 2. TimestampType = LogAppendTime and wrapperMessageTimestamp is defined - Compressed message using LogAppendTime * 3. TimestampType = CreateTime and wrapperMessageTimestamp is defined - Compressed message using CreateTime */ -def timestamp: Long = { + def timestamp: Long = { if (magic == MagicValue_V0) Message.NoTimestamp // Case 2 diff --git a/core/src/main/scala/kafka/message/MessageSet.scala b/core/src/main/scala/kafka/message/MessageSet.scala index 26cab50b08c43..2585b9d6874e1 100644 --- a/core/src/main/scala/kafka/message/MessageSet.scala +++ b/core/src/main/scala/kafka/message/MessageSet.scala @@ -20,10 +20,6 @@ package kafka.message import java.nio._ import java.nio.channels._ -import kafka.log.FileMessageSet - -import scala.collection.mutable.ArrayBuffer - /** * Message set helper functions */ @@ -63,12 +59,12 @@ object MessageSet { * of the inner messages. */ def validateMagicValuesAndGetTimestamp(messages: Seq[Message]): Long = { - val sampleMagicValue = messages.head.magic + val firstMagicValue = messages.head.magic var largestTimestamp: Long = Message.NoTimestamp for (message <- messages) { - if (message.magic != sampleMagicValue) - throw new IllegalStateException("Messages in the same compressed message set must have same magic value") - if (sampleMagicValue > Message.MagicValue_V0) + 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) } largestTimestamp @@ -92,9 +88,9 @@ abstract class MessageSet extends Iterable[MessageAndOffset] { def writeTo(channel: GatheringByteChannel, offset: Long, maxSize: Int): Int /** - * Check if all the messages in the message set have the expected magic value + * Check if all the wrapper messages in the message set have the expected magic value */ - def magicValueInAllMessages(expectedMagicValue: Byte): Boolean + def magicValueInAllWrapperMessages(expectedMagicValue: Byte): Boolean /** * Provides an iterator over the message/offset pairs in this set diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 015414ff48725..ee3b9c91599d6 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -19,6 +19,7 @@ package kafka.server import java.nio.ByteBuffer import java.lang.{Long => JLong, Short => JShort} +import java.util.concurrent.atomic.{AtomicLong, AtomicInteger} import kafka.admin.AdminUtils import kafka.api._ @@ -64,6 +65,8 @@ class KafkaApis(val requestChannel: RequestChannel, this.logIdent = "[KafkaApi-%d] ".format(brokerId) // Store all the quota managers for each type of request val quotaManagers: Map[Short, ClientQuotaManager] = instantiateQuotaManagers(config) + var numFetch = new AtomicInteger(0) + var totalTime = new AtomicLong(0L) /** * Top-level method that handles all requests and multiplexes to the right api @@ -440,10 +443,10 @@ class KafkaApis(val requestChannel: RequestChannel, // 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 consumer without - // format down conversion. + // 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(_ > 0) && - !data.messages.magicValueInAllMessages(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 @@ -467,7 +470,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def fetchResponseCallback(delayTimeMs: Int) { - trace(s"Sending fetch response to ${fetchRequest.clientId} with ${convertedResponseStatus.values.map(_.messages.size).sum}" + + trace(s"Sending fetch response to ${fetchRequest.clientId} with ${convertedResponseStatus.values.map(_.messages.sizeInBytes).sum}" + s" messages") val response = FetchResponse(fetchRequest.correlationId, mergedResponseStatus, fetchRequest.versionId, delayTimeMs) requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, response))) @@ -483,8 +486,8 @@ class KafkaApis(val requestChannel: RequestChannel, } else { quotaManagers(ApiKeys.FETCH.id).recordAndMaybeThrottle(fetchRequest.clientId, FetchResponse.responseSize(responsePartitionData.groupBy(_._1.topic), - fetchRequest.versionId), - fetchResponseCallback) + fetchRequest.versionId), + fetchResponseCallback) } } diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 1b7d5d8f7d5fa..e92c0c0f20de1 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -12,14 +12,14 @@ # 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. -log4j.rootLogger=OFF, stdout +log4j.rootLogger=INFO, stdout log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n log4j.logger.kafka=ERROR -log4j.logger.org.apache.kafka=ERROR +log4j.logger.org.apache.kafka=INFO # zkclient can be verbose, during debugging it is common to adjust is separately log4j.logger.org.I0Itec.zkclient.ZkClient=WARN diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index 58b483ed6b4cf..93873818fa099 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -283,9 +283,15 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { 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.CreateTime, + 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 diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala index 5d97a76492b91..11e4aa26e99c6 100644 --- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala @@ -18,12 +18,12 @@ package kafka.api 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.{InvalidMessageException, Message} +import kafka.message.Message import kafka.server.KafkaConfig import kafka.utils.TestUtils import org.apache.kafka.clients.producer._ @@ -55,8 +55,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 @@ -89,6 +89,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()) @@ -106,24 +107,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 { @@ -180,6 +181,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { object callback extends Callback { var offset = 0L var timestampDiff = 1L + def onCompletion(metadata: RecordMetadata, exception: Exception) { if (exception == null) { assertEquals(offset, metadata.offset()) @@ -230,7 +232,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") @@ -251,7 +253,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) @@ -290,7 +292,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { val now = System.currentTimeMillis() val responses = for (i <- 1 to numRecords) - yield producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, now, 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) @@ -304,7 +306,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()) @@ -333,7 +335,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 @@ -353,7 +355,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() @@ -378,7 +380,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())) @@ -425,23 +427,56 @@ 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 { + intercept[ExecutionException] { + producer.send(new ProducerRecord(topic, 0, System.currentTimeMillis() - 1001, "key".getBytes, "value".getBytes)).get() + } + } 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 { + intercept[ExecutionException] { + compressedProducer.send(new ProducerRecord(topic, 0, System.currentTimeMillis() - 1001, "key".getBytes, "value".getBytes)).get() + } + } finally { + compressedProducer.close() + } + } + } diff --git a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala index 02ff2fc4bc903..ecde7325a84af 100644 --- a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala @@ -19,10 +19,10 @@ package kafka.message import java.nio._ import java.util.concurrent.atomic.AtomicLong -import kafka.api.ApiVersion + +import kafka.utils.TestUtils import org.junit.Assert._ import org.junit.Test -import kafka.utils.TestUtils class ByteBufferMessageSetTest extends BaseMessageSetTestCases { @@ -155,7 +155,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { val startTime = System.currentTimeMillis() // The timestamps should be overwritten val messages = getMessages(magicValue = Message.MagicValue_V1, timestamp = 0L, codec = NoCompressionCodec) - val compressedMessages = getMessages(magicValue = Message.MagicValue_V1, timestamp = 5L, codec = DefaultCompressionCodec) + val compressedMessagesWithRecompresion = getMessages(magicValue = Message.MagicValue_V0, codec = DefaultCompressionCodec) val compressedMessagesWithoutRecompression = getMessages(magicValue = Message.MagicValue_V1, timestamp = -1L, codec = DefaultCompressionCodec) @@ -167,12 +167,12 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { messageTimestampDiffMaxMs = 1000L) val validatedCompressedMessages = - compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(0), - sourceCodec = DefaultCompressionCodec, - targetCodec = DefaultCompressionCodec, - messageFormatVersion = 1, - messageTimestampType = LogAppendTime, - messageTimestampDiffMaxMs = 1000L) + compressedMessagesWithRecompresion.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(0), + sourceCodec = DefaultCompressionCodec, + targetCodec = DefaultCompressionCodec, + messageFormatVersion = 1, + messageTimestampType = LogAppendTime, + messageTimestampDiffMaxMs = 1000L) val validatedCompressedMessagesWithoutRecompression = compressedMessagesWithoutRecompression.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(0), @@ -190,7 +190,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { assertEquals(LogAppendTime, messageAndOffset.message.timestampType) } - assertEquals("message set size should not change", compressedMessages.size, validatedCompressedMessages.size) + assertEquals("message set size should not change", compressedMessagesWithRecompresion.size, validatedCompressedMessages.size) for (messageAndOffset <- validatedCompressedMessages) { messageAndOffset.message.ensureValid() assertTrue(s"Timestamp of message ${messageAndOffset.message}} should be between $startTime and $now", @@ -396,9 +396,9 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { if (magicValue == Message.MagicValue_V0) { new ByteBufferMessageSet( codec, - new Message("hello".getBytes), - new Message("there".getBytes), - new Message("beautiful".getBytes)) + 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, diff --git a/docs/upgrade.html b/docs/upgrade.html index 7e158f874a131..527c2c1cab3ca 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -18,7 +18,7 @@

    1.5 Upgrading From Previous Versions

    Upgrading from 0.9.x to 0.10.0.0

    -0.10.0.0 has potential performance impact and protocol changes (please review before upgrading). +0.10.0.0 has potential performance impact during upgrade and protocol changes (please review before upgrading). Because new protocols are introduced, it is important to upgrade your Kafka clusters before upgrading your clients.

    For a rolling upgrade:

    @@ -34,21 +34,26 @@

    Upgrading from 0.9.x to 0.10.0.0Note: Bumping the protocol version and restarting can be done any time after the brokers were upgraded. It does not have to be immediately after. -

    potential performance impact in 0.10.0.0
    +
    potential performance impact in 0.10.0.0 during upgrade

    Message format v1 is added to include a new timestamp field and use relative offsets for compressed messages in 0.10.0.0. The on disk message format can be configured through message.format.version in server.properties file. - A 0.10.0.0 broker is able to serve old clients with message format v0. Messages will be converted to v0 if on disk - message format version is v1. However, Brokers will lose zero-copy when message format conversion occurs. + The default on-disk message format in 0.10.0 in v1. If a consumer client is on a version before 0.10.0.0, it only understands + message format v0. In this case, the broker is able to convert messages of format v1 to v0 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, one can set the message format to v0 after + upgrading the broker to 0.10.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 v1 on the broker.

    0.10.0.0 clients can handle both message format v0 and v1. Hence there is no performance impact for 0.10.0.0 clients if message.format.version is set to v0 on broker. Broker will not convert the messages but just return message v0 - to new clients. + to 0.10.0.0 consumer.

    - To avoid performance impact due to message format conversion, only set message.format.version=v1 after upgrading your clients. - message.format.version is a topic configuration, so you can also dynamically set it for a specific topic by changing its topic configuration. + Note: By setting the message format version, one certifies all the existing messages is on or below that + message format version. Otherwise consumers before 0.10.0.0 might break.

    protocol change for 0.10.0.0
    From acc7189707b0f5d5f4493bc3b2cb045a23788d15 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Fri, 12 Feb 2016 20:02:17 -0800 Subject: [PATCH 19/33] Updated doc for message format version --- core/src/main/scala/kafka/server/KafkaConfig.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 8b45296b939ea..a144984a1f09b 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -423,7 +423,8 @@ 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. Valid values are \"v0\" and \"v1\" (case sensitive)." + val MessageFormatVersionDoc = "Specify the message format version the broker will use to append messages to the logs. Valid values are \"v0\" and \"v1\" (case sensitive). " + + "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 the whether the timestamp in the message is message create time or log append time. The value should be either" + " \"CreateTime\" or \"LogAppendTime\"" val MessageTimestampDifferenceMaxMsDoc = "Set maximum allowed time difference between broker local time and message's timestamp. " + From b38877f3bb0be9a78caaa069ba9bc3dc64a5bf98 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Sat, 13 Feb 2016 18:18:51 -0800 Subject: [PATCH 20/33] Fix EdgeCaseRequestTest --- core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala index 155eea0f0b797..2ccb7b878d3f7 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) From def267302c14551d50c2fa82dcbc4487d394e20f Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Sat, 13 Feb 2016 21:55:50 -0800 Subject: [PATCH 21/33] remove default value for Message constructor. --- core/src/main/scala/kafka/message/Message.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala index 57591ddfaf3c4..76066ec99c4a8 100755 --- a/core/src/main/scala/kafka/message/Message.scala +++ b/core/src/main/scala/kafka/message/Message.scala @@ -157,7 +157,7 @@ class Message(val buffer: ByteBuffer, def this(bytes: Array[Byte], key: Array[Byte], timestamp: Long, - timestampType: TimestampType = CreateTime, + timestampType: TimestampType, codec: CompressionCodec, payloadOffset: Int, payloadSize: Int, @@ -205,7 +205,7 @@ class Message(val buffer: ByteBuffer, } def this(bytes: Array[Byte], key: Array[Byte], timestamp: Long, codec: CompressionCodec, magicValue: Byte) = - this(bytes = bytes, key = key, timestamp = timestamp, codec = codec, payloadOffset = 0, payloadSize = -1, magicValue = magicValue) + this(bytes = bytes, key = key, timestamp = timestamp, timestampType = CreateTime, codec = codec, payloadOffset = 0, payloadSize = -1, magicValue = magicValue) def this(bytes: Array[Byte], timestamp: Long, codec: CompressionCodec, magicValue: Byte) = this(bytes = bytes, key = null, timestamp = timestamp, codec = codec, magicValue = magicValue) From a0f76bb459f767e183943777cf4fa84841cca2ec Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Mon, 15 Feb 2016 16:28:31 -0800 Subject: [PATCH 22/33] Addressed Jun's comments. --- .../clients/consumer/ConsumerRecord.java | 1 + .../kafka/clients/producer/KafkaProducer.java | 5 +- .../clients/producer/ProducerRecord.java | 2 +- .../clients/producer/RecordMetadata.java | 4 +- .../producer/internals/RecordBatch.java | 2 +- .../errors/InvalidTimestampException.java | 34 ++++++++++ .../apache/kafka/common/protocol/Errors.java | 5 +- .../kafka/common/protocol/Protocol.java | 2 +- .../kafka/common/record/MemoryRecords.java | 7 +- .../common/requests/RequestResponseTest.java | 2 +- .../src/main/scala/kafka/api/ApiVersion.scala | 23 +++++-- .../scala/kafka/common/ErrorMapping.scala | 1 + .../coordinator/GroupMetadataManager.scala | 14 ++-- .../src/main/scala/kafka/log/LogCleaner.scala | 13 ++-- core/src/main/scala/kafka/log/LogConfig.scala | 5 +- .../kafka/message/ByteBufferMessageSet.scala | 41 +++++++----- .../main/scala/kafka/message/Message.scala | 6 +- .../main/scala/kafka/message/MessageSet.scala | 6 +- .../scala/kafka/server/ConfigHandler.scala | 4 +- .../main/scala/kafka/server/KafkaApis.scala | 5 +- .../main/scala/kafka/server/KafkaConfig.scala | 17 ++--- .../scala/kafka/server/ReplicaManager.scala | 11 ++-- core/src/test/resources/log4j.properties | 4 +- .../kafka/api/BaseProducerSendTest.scala | 20 +++--- .../kafka/api/PlaintextConsumerTest.scala | 6 +- .../RequestResponseSerializationTest.scala | 4 +- .../GroupCoordinatorResponseTest.scala | 5 +- .../scala/unit/kafka/log/CleanerTest.scala | 6 +- .../unit/kafka/log/FileMessageSetTest.scala | 2 +- .../scala/unit/kafka/log/LogManagerTest.scala | 2 - .../test/scala/unit/kafka/log/LogTest.scala | 6 +- .../message/ByteBufferMessageSetTest.scala | 33 ++++------ .../message/MessageCompressionTest.scala | 12 ++-- .../unit/kafka/message/MessageTest.scala | 65 ++++++++----------- .../kafka/message/MessageWriterTest.scala | 8 +-- .../unit/kafka/server/KafkaConfigTest.scala | 4 +- .../unit/kafka/server/LogOffsetTest.scala | 2 +- docs/upgrade.html | 23 ++++--- 38 files changed, 232 insertions(+), 180 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/InvalidTimestampException.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 ee40cd5b4eec4..8aa3c03530b37 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 @@ -34,6 +34,7 @@ public final class ConsumerRecord { * @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 */ 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 9b0410cd405d5..0bcc4ed5781ba 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 @@ -360,7 +360,10 @@ public Future send(ProducerRecord record) { * response after each one. *

    * 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. + * it was assigned and the timestamp of the record. If {@link org.apache.kafka.common.record.TimestampType#CreateTime} + * is used by the topic, the timestamp will be the user provided timestamp or the record send time if user did not + * specify a timestamp for the record. If {@link org.apache.kafka.common.record.TimestampType#LogAppendTime} is + * used for the topic, the timestamp will be the Kafka broker local time when the message is appended. *

    * 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() 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 034fbf31ae042..f4dcce7be146b 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 @@ -21,7 +21,7 @@ * present a partition will be assigned in a round-robin fashion. *

    * The record also has an associated timestamp. If 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 timestam + * with its current time. The timestamp eventually used by Kafka depends on the timestamp type configured for the topic. *

  • * If the topic is configured to use {@link org.apache.kafka.common.record.TimestampType#CreateTime CreateTime} * the timestamp in the producer record will be used by broker. 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 19d11b2de1794..917f99b1d6166 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 @@ -26,7 +26,9 @@ public final class RecordMetadata { private final long offset; // The timestamp of the message. // If LogAppendTime is used for the topic, the timestamp will be a timestamp returned by broker. - // If CreateTime is used for the topic, the timestamp is the timestamp in the corresponding ProducerRecord. + // If CreateTime is used for the topic, the timestamp is the timestamp in the corresponding ProducerRecord if + // 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; 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 7bbbff5f910b2..37e067b4071f0 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 @@ -96,7 +96,7 @@ public void done(long baseOffset, long timestamp, RuntimeException exception) { try { Thunk thunk = this.thunks.get(i); if (exception == null) { - // If the timestamp returned by server is NoTimestamp, that means create times is used. Otherwise LogAppendTime is used. + // 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); 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 0000000000000..d2d285bc97412 --- /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); + } +} 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 4a2086954e10b..e7098fc05fcdd 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.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); 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 52c919e27d099..e2b1cbf0337c6 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 @@ -117,7 +117,7 @@ public class Protocol { 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 the message format V1 is used which has relative offset and + * 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; 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 4ebe3dce2ad57..88d8e3028489a 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 @@ -228,8 +228,9 @@ public RecordsIterator(ByteBuffer buffer, CompressionType type, boolean shallow) this.absoluteBaseOffset = -1; } - private RecordsIterator(LogEntry entry, CompressionType type) { - this.type = type; + // 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); @@ -291,7 +292,7 @@ protected LogEntry makeNext() { // 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 - innerIter = new RecordsIterator(entry, compression); + innerIter = new RecordsIterator(entry); return innerIter.next(); } } catch (EOFException e) { 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 242fb44c0bc45..7e088330f349c 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 @@ -124,11 +124,11 @@ public void produceResponseVersionTest() { Map responseData = new HashMap(); responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE.code(), 10000, Record.NO_TIMESTAMP)); ProduceResponse v0Response = new ProduceResponse(responseData); - // No need to verify V1 here because 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()); diff --git a/core/src/main/scala/kafka/api/ApiVersion.scala b/core/src/main/scala/kafka/api/ApiVersion.scala index 2de7d7b4783dc..aa6dd56719f86 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. @@ -25,12 +27,15 @@ 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) * - * If there is a draft protocol version between protocols of two official releases. The suffix "-IV#" will be added to - * help users who are running on trunk upgrade. For example: - * 1. Kafka 0.9.0 is released - * 2. After that some protocol change are made and will be released in 0.10.0. The version will be named 0.10.0-IV0 - * (IV stands for internal version) - * 3. When Kafka 0.10.0 is released. The official version 0.10.0 will be the same as the last internal version. + * Since the api protocol may change more than once within the same release, to facilitate people deploying code from + * trunk, we introduce internal versions since 0.10.0. For example, the first time that we introduce a version change + * in 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. 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 0.10.0 release. */ object ApiVersion { // This implicit is necessary due to: https://issues.scala-lang.org/browse/SI-8541 @@ -52,6 +57,7 @@ object ApiVersion { sealed trait ApiVersion extends Ordered[ApiVersion] { val version: String + val messageFormatVersion: Byte val id: Int override def compare(that: ApiVersion): Int = { @@ -68,26 +74,31 @@ sealed trait ApiVersion extends Ordered[ApiVersion] { // Keep the IDs in order of versions case object KAFKA_0_8_0 extends ApiVersion { val version: String = "0.8.0.X" + val messageFormatVersion: Byte = Message.MagicValue_V0 val id: Int = 0 } case object KAFKA_0_8_1 extends ApiVersion { val version: String = "0.8.1.X" + val messageFormatVersion: Byte = Message.MagicValue_V0 val id: Int = 1 } case object KAFKA_0_8_2 extends ApiVersion { val version: String = "0.8.2.X" + val messageFormatVersion: Byte = Message.MagicValue_V0 val id: Int = 2 } case object KAFKA_0_9_0 extends ApiVersion { val version: String = "0.9.0.X" + val messageFormatVersion: Byte = Message.MagicValue_V0 val id: Int = 3 } // This is a between-release protocol version 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 } \ No newline at end of file diff --git a/core/src/main/scala/kafka/common/ErrorMapping.scala b/core/src/main/scala/kafka/common/ErrorMapping.scala index e20b88c3d995c..9708c4e5394ec 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]( diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index b487e6eb8dacd..975cb0efc3e23 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -142,7 +142,7 @@ class GroupMetadataManager(val brokerId: Int, // retry removing this group. val groupPartition = partitionFor(group.groupId) val tombstone = new Message(bytes = null, key = GroupMetadataManager.groupMetadataKey(group.groupId), - timestamp = SystemTime.milliseconds, magicValue = Message.MagicValue_V1) + timestamp = SystemTime.milliseconds, magicValue = getMessageFormatVersion(groupPartition)) val partitionOpt = replicaManager.getPartition(GroupCoordinator.GroupMetadataTopicName, groupPartition) partitionOpt.foreach { partition => @@ -171,7 +171,7 @@ class GroupMetadataManager(val brokerId: Int, key = GroupMetadataManager.groupMetadataKey(group.groupId), bytes = GroupMetadataManager.groupMetadataValue(group, groupAssignment), timestamp = SystemTime.milliseconds, - magicValue = Message.MagicValue_V1 + magicValue = getMessageFormatVersion(partitionFor(group.groupId)) ) val groupMetadataPartition = new TopicPartition(GroupCoordinator.GroupMetadataTopicName, partitionFor(group.groupId)) @@ -255,7 +255,7 @@ class GroupMetadataManager(val brokerId: Int, key = GroupMetadataManager.offsetCommitKey(groupId, topicAndPartition.topic, topicAndPartition.partition), bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata), timestamp = SystemTime.milliseconds, - magicValue = Message.MagicValue_V1 + magicValue = getMessageFormatVersion(partitionFor(groupId)) ) }.toSeq @@ -555,7 +555,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, timestamp = SystemTime.milliseconds, magicValue = Message.MagicValue_V1)) + (offsetsPartition, new Message(bytes = null, key = commitKey, timestamp = SystemTime.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, @@ -624,6 +625,11 @@ class GroupMetadataManager(val brokerId: Int, config.offsetsTopicNumPartitions } + private def getMessageFormatVersion(partition: Int) = { + val groupMetadataTopicAndPartition = new TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, partition) + replicaManager.getMessageFormatVersion(groupMetadataTopicAndPartition).get + } + /** * Add the partition into the owned list * diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index a1ad00889066d..06b598842e26d 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -424,12 +424,13 @@ 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 { - // We use absolute offset to compare decide whether retain the message or not. This is handled by + // 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 @@ -482,20 +483,20 @@ private[log] class Cleaner(val id: Int, ByteBufferMessageSet.writeMessage(buffer, messageOffset.message, messageOffset.offset) MessageSet.messageSetSize(messagesIterable) } else { - val messageSetTimestamp = MessageSet.validateMagicValuesAndGetTimestamp(messages.map(_.message)) + val magicAndTimestamp = MessageSet.validateMagicValuesAndGetTimestamp(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, timestamp = messageSetTimestamp, timestampType = timestampType, magicValue = messageFormatVersion) { 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 - // Use relative offset when magic value is greater than 0 + // 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 relative offset. + // The offset of the messages are absolute offset, compute the inner offset. val innerOffset = messageOffset.offset - firstAbsoluteOffset output.writeLong(innerOffset) } else diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index c8bfe310fb2e7..08d6e997388f8 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -19,6 +19,7 @@ package kafka.log import java.util.Properties +import kafka.api.ApiVersion import kafka.message.{BrokerCompressionCodec, Message, TimestampType} import kafka.server.KafkaConfig import org.apache.kafka.common.config.{AbstractConfig, ConfigDef} @@ -71,7 +72,7 @@ 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 = Integer.parseInt(getString(LogConfig.MessageFormatVersionProp).substring(1)).toByte + val messageFormatVersion = ApiVersion(getString(LogConfig.MessageFormatVersionProp)).messageFormatVersion val messageTimestampType = TimestampType.forName(getString(LogConfig.MessageTimestampTypeProp)) val messageTimestampDifferenceMaxMs = getLong(LogConfig.MessageTimestampDifferenceMaxMsProp) @@ -168,7 +169,7 @@ object LogConfig { .define(CompressionTypeProp, STRING, Defaults.CompressionType, in(BrokerCompressionCodec.brokerCompressionOptions:_*), MEDIUM, CompressionTypeDoc) .define(PreAllocateEnableProp, BOOLEAN, Defaults.PreAllocateEnable, MEDIUM, PreAllocateEnableDoc) - .define(MessageFormatVersionProp, STRING, Defaults.MessageFormatVersion, in("v0", "v1"), MEDIUM, MessageFormatVersionDoc) + .define(MessageFormatVersionProp, STRING, Defaults.MessageFormatVersion, MEDIUM, MessageFormatVersionDoc) .define(MessageTimestampTypeProp, STRING, Defaults.MessageTimestampType, MEDIUM, MessageTimestampTypeDoc) .define(MessageTimestampDifferenceMaxMsProp, LONG, Defaults.MessageTimestampDifferenceMaxMs, atLeast(0), MEDIUM, MessageTimestampDifferenceMaxMsDoc) } diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 75778aa76e185..31bc9aea058c3 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -25,6 +25,7 @@ import java.nio.channels._ import java.io._ import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} +import org.apache.kafka.common.errors.InvalidTimestampException import org.apache.kafka.common.utils.Utils import scala.collection.mutable @@ -46,20 +47,21 @@ object ByteBufferMessageSet { buffer.rewind() buffer } else { - System.out.println("wrapper message timestamp " + wrapperMessageTimestamp) - val messageSetTimestamp = wrapperMessageTimestamp.getOrElse(MessageSet.validateMagicValuesAndGetTimestamp(messages)) + val magicAndTimestamp = wrapperMessageTimestamp match { + case Some(ts) => MagicAndTimestamp(messages.head.magic, ts) + case None => MessageSet.validateMagicValuesAndGetTimestamp(messages) + } var offset = -1L - val firstMagicValue = messages.head.magic val messageWriter = new MessageWriter(math.min(math.max(MessageSet.messageSetSize(messages) / 2, 1024), 1 << 16)) - messageWriter.write(codec = compressionCodec, timestamp = messageSetTimestamp, timestampType = timestampType, magicValue = firstMagicValue) { 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 = offsetAssignor.nextAbsoluteOffset - if (message.magic != firstMagicValue) + if (message.magic != magicAndTimestamp.magic) throw new IllegalArgumentException("Messages in the message set must have same magic value") // Use relative offset if magic value is greater than 0 - if (firstMagicValue > Message.MagicValue_V0) + if (magicAndTimestamp.magic > Message.MagicValue_V0) output.writeLong(offsetAssignor.toInnerOffset(offset)) else output.writeLong(offset) @@ -91,7 +93,7 @@ object ByteBufferMessageSet { val wrapperMessageTimestampTypeOpt: Option[TimestampType] = if (wrapperMessage.magic > MagicValue_V0) Some(wrapperMessage.timestampType) else None if (wrapperMessage.payload == null) - throw new RuntimeException("Message payload is null: " + wrapperMessage) + 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 @@ -209,9 +211,9 @@ private class OffsetAssigner(offsets: Seq[Long]) { * 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 relative - * offsets for inner messages of compressed messages. Timestamp type attribute is only set in wrapper messages. - * Inner messages always have CreateTime as timestamp type in attributes. + * - 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 is set and used. @@ -241,7 +243,7 @@ private class OffsetAssigner(offsets: Seq[Long]) { * * To solve this issue, we use the following solution: * - * 1. When producer create a message set, it simply writes all the messages into a compressed message set with + * 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. @@ -408,7 +410,8 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi // check the magic value if (!magicValueInAllWrapperMessages(messageFormatVersion)) { // Message format conversion - convertNonCompressedMessages(offsetCounter, compactedTopic, now, messageTimestampType, messageFormatVersion) + convertNonCompressedMessages(offsetCounter, compactedTopic, now, messageTimestampType, messageTimestampDiffMaxMs, + messageFormatVersion) } else { // Do in-place validation, offset assignment and maybe set timestamp validateNonCompressedMessagesAndAssignOffsetInPlace(offsetCounter, now, compactedTopic, messageTimestampType, @@ -417,7 +420,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi } else { // Deal with compressed messages - // We cannot do in place assignment in one of the followings situation: + // 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. @@ -427,7 +430,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi var inPlaceAssignment = sourceCodec == targetCodec && messageFormatVersion > Message.MagicValue_V0 var maxTimestamp = Message.NoTimestamp - val expectedRelativeOffset = new AtomicLong(0) + val expectedInnerOffset = new AtomicLong(0) val validatedMessages = new ListBuffer[Message] this.internalIterator(isShallow = false).foreach(messageAndOffset => { val message = messageAndOffset.message @@ -437,7 +440,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi // Validate the timestamp validateTimestamp(message, now, messageTimestampType, messageTimestampDiffMaxMs) // Check if we need to overwrite offset - if (messageAndOffset.offset != expectedRelativeOffset.getAndIncrement) + if (messageAndOffset.offset != expectedInnerOffset.getAndIncrement) inPlaceAssignment = false maxTimestamp = math.max(maxTimestamp, message.timestamp) } @@ -503,6 +506,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi compactedTopic: Boolean, now: Long, timestampType: TimestampType, + messageTimestampDiffMaxMs: Long, toMagicValue: Byte): ByteBufferMessageSet = { // Get message count, shallow iterator is in-place val sizeInBytesAfterConversion = shallowValidBytes + this.internalIterator(isShallow = true).foldLeft(0)( @@ -512,6 +516,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi 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) @@ -545,7 +550,6 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi val messageBuffer = buffer.slice() messageBuffer.limit(messageSize) val message = new Message(messageBuffer) - message.ensureValid() validateMessageKey(message, compactedTopic) if (message.magic > Message.MagicValue_V0) { validateTimestamp(message, now, timestampType, timestampDiffMaxMs) @@ -575,10 +579,11 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi timestampType: TimestampType, timestampDiffMaxMs: Long) { if (timestampType == CreateTime && math.abs(message.timestamp - now) > timestampDiffMaxMs) - throw new InvalidMessageException(s"Timestamp ${message.timestamp} of message is out of range. " + + 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 == LogAppendTime) - throw new InvalidMessageException(s"Invalid message $message. Producer should not set timestamp type to LogAppendTime.") + throw new InvalidTimestampException(s"Invalid timestamp type in message $message. Producer should not set " + + s"timestamp type to LogAppendTime.") } /** diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala index 76066ec99c4a8..fc65d13126156 100755 --- a/core/src/main/scala/kafka/message/Message.scala +++ b/core/src/main/scala/kafka/message/Message.scala @@ -297,9 +297,9 @@ class Message(val buffer: ByteBuffer, /** * 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. TimestampType = None and wrapperMessageTimestamp is None - Uncompressed message, timestamp and timestamp type are in the message. - * 2. TimestampType = LogAppendTime and wrapperMessageTimestamp is defined - Compressed message using LogAppendTime - * 3. TimestampType = CreateTime and wrapperMessageTimestamp is defined - Compressed message using CreateTime + * 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) diff --git a/core/src/main/scala/kafka/message/MessageSet.scala b/core/src/main/scala/kafka/message/MessageSet.scala index 2585b9d6874e1..c3169cd790a40 100644 --- a/core/src/main/scala/kafka/message/MessageSet.scala +++ b/core/src/main/scala/kafka/message/MessageSet.scala @@ -58,7 +58,7 @@ object MessageSet { * Validate the "magic" values of messages are the same in a compressed message set and return the max timestamp * of the inner messages. */ - def validateMagicValuesAndGetTimestamp(messages: Seq[Message]): Long = { + def validateMagicValuesAndGetTimestamp(messages: Seq[Message]): MagicAndTimestamp = { val firstMagicValue = messages.head.magic var largestTimestamp: Long = Message.NoTimestamp for (message <- messages) { @@ -67,11 +67,13 @@ object MessageSet { if (firstMagicValue > Message.MagicValue_V0) largestTimestamp = math.max(largestTimestamp, message.timestamp) } - largestTimestamp + 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 diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala index 05c7250f9b595..f32675d1a08ab 100644 --- a/core/src/main/scala/kafka/server/ConfigHandler.scala +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -19,6 +19,7 @@ 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 @@ -48,8 +49,7 @@ class TopicConfigHandler(private val logManager: LogManager, kafkaConfig: KafkaC // Validate the compatibility of message format version. Option(topicConfig.getProperty(LogConfig.MessageFormatVersionProp)) match { case Some(versionString) => - val version = Integer.parseInt(versionString.substring(1)) - if (!kafkaConfig.validateMessageFormatVersion(version)) { + if (kafkaConfig.interBrokerProtocolVersion.messageFormatVersion < ApiVersion(versionString).messageFormatVersion) { 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}") diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index ee3b9c91599d6..7cf16a8d74c54 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -19,7 +19,6 @@ package kafka.server import java.nio.ByteBuffer import java.lang.{Long => JLong, Short => JShort} -import java.util.concurrent.atomic.{AtomicLong, AtomicInteger} import kafka.admin.AdminUtils import kafka.api._ @@ -65,8 +64,6 @@ class KafkaApis(val requestChannel: RequestChannel, this.logIdent = "[KafkaApi-%d] ".format(brokerId) // Store all the quota managers for each type of request val quotaManagers: Map[Short, ClientQuotaManager] = instantiateQuotaManagers(config) - var numFetch = new AtomicInteger(0) - var totalTime = new AtomicLong(0L) /** * Top-level method that handles all requests and multiplexes to the right api @@ -471,7 +468,7 @@ 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" messages") + s" bytes") val response = FetchResponse(fetchRequest.correlationId, mergedResponseStatus, fetchRequest.versionId, delayTimeMs) requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, response))) } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index a144984a1f09b..719e9f7fa3085 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -94,7 +94,7 @@ object Defaults { val LogFlushSchedulerIntervalMs = Long.MaxValue val LogFlushOffsetCheckpointIntervalMs = 60000 val LogPreAllocateEnable = false - val MessageFormatVersion = "v1" + val MessageFormatVersion = ApiVersion.latestVersion.toString() val NumRecoveryThreadsPerDataDir = 1 val AutoCreateTopicsEnable = true val MinInSyncReplicas = 1 @@ -423,9 +423,10 @@ 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. Valid values are \"v0\" and \"v1\" (case sensitive). " + + 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-IV0. 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 the whether the timestamp in the message is message create time or log append time. The value should be either" + + 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 = "Set maximum allowed time difference between broker local time and message's timestamp. " + "This configuration only works when message.timestamp.type=CreateTime. A message will be rejected if its timestamp exceeds this threshold." @@ -603,7 +604,7 @@ 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, in("v0", "v1"), MEDIUM, MessageFormatVersionDoc) + .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) @@ -976,11 +977,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(validateMessageFormatVersion(Integer.parseInt(messageFormatVersion.substring(1))), s"message.format.version $messageFormatVersion cannot " + - s"be used when inter.broker.protocol.version is set to $interBrokerProtocolVersion") - } - - def validateMessageFormatVersion(messageFormatVersion: Int): Boolean = { - !(messageFormatVersion > 0 && !interBrokerProtocolVersion.onOrAfter(KAFKA_0_10_0_IV0)) + require(interBrokerProtocolVersion.messageFormatVersion >= ApiVersion(messageFormatVersion).messageFormatVersion, + s"message.format.version $messageFormatVersion cannot be used when inter.broker.protocol.version is set to $interBrokerProtocolVersion") } } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index be7fea69fd3f2..8873638905154 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -26,20 +26,19 @@ import kafka.cluster.{Partition, Replica} import kafka.common._ import kafka.controller.KafkaController import kafka.log.{LogAppendInfo, LogManager} -import kafka.message.{Message, InvalidMessageException, 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} import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.errors._ import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.requests.{LeaderAndIsrRequest, StopReplicaRequest, UpdateMetadataRequest} import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse +import org.apache.kafka.common.requests.{LeaderAndIsrRequest, StopReplicaRequest, UpdateMetadataRequest} import org.apache.kafka.common.utils.{Time => JTime} -import scala.collection._ import scala.collection.JavaConverters._ +import scala.collection._ /* * Result metadata of a log append operation on the log @@ -442,6 +441,8 @@ class ReplicaManager(val config: KafkaConfig, (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() diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index e92c0c0f20de1..1b7d5d8f7d5fa 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -12,14 +12,14 @@ # 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. -log4j.rootLogger=INFO, stdout +log4j.rootLogger=OFF, stdout log4j.appender.stdout=org.apache.log4j.ConsoleAppender log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n log4j.logger.kafka=ERROR -log4j.logger.org.apache.kafka=INFO +log4j.logger.org.apache.kafka=ERROR # zkclient can be verbose, during debugging it is common to adjust is separately log4j.logger.org.I0Itec.zkclient.ZkClient=WARN diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala index 11e4aa26e99c6..88934960589a2 100644 --- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala @@ -27,7 +27,7 @@ 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, CorruptRecordException, SerializationException} import org.apache.kafka.common.record.TimestampType import org.junit.Assert._ import org.junit.{After, Before, Test} @@ -212,8 +212,8 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, partition, baseTimestamp + i, "key".getBytes, "value".getBytes) producer.send(record, callback) } - producer.flush() - assertEquals("Should have offset " + numRecords, numRecords, callback.offset) + producer.close(5000L, TimeUnit.MILLISECONDS) + assertEquals(s"Should have offset $numRecords but only successfully sent ${callback.offset}", numRecords, callback.offset) } finally { producer.close() } @@ -459,9 +459,10 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { val producer = createProducer(brokerList = brokerList) try { - intercept[ExecutionException] { - producer.send(new ProducerRecord(topic, 0, System.currentTimeMillis() - 1001, "key".getBytes, "value".getBytes)).get() - } + 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() } @@ -471,9 +472,10 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { producerProps.setProperty(ProducerConfig.COMPRESSION_TYPE_CONFIG, "gzip") val compressedProducer = createProducer(brokerList = brokerList, props = Some(producerProps)) try { - intercept[ExecutionException] { - compressedProducer.send(new ProducerRecord(topic, 0, System.currentTimeMillis() - 1001, "key".getBytes, "value".getBytes)).get() - } + 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() } diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 3151b746a1026..b969c26febab9 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -622,9 +622,9 @@ 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)), - rebalanceListener) + 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 assertEquals(10, testConsumer.committed(tp).offset) diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index d15d3d3e9b6e8..fafc4b0aafc26 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -20,13 +20,11 @@ package kafka.api import kafka.cluster.{EndPoint, Broker} import kafka.common.{OffsetAndMetadata, OffsetMetadataAndError} import kafka.common._ -import kafka.log.FileMessageSet -import kafka.message._ +import kafka.message.{Message, ByteBufferMessageSet} import kafka.utils.SystemTime import kafka.controller.LeaderIsrAndControllerEpoch import kafka.common.TopicAndPartition -import kafka.utils.TestUtils._ import java.nio.ByteBuffer diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala index 200f98d10898c..9282c4991a8ee 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala @@ -21,7 +21,7 @@ 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.TopicPartition @@ -837,6 +837,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { 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) @@ -913,6 +914,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { 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) @@ -923,6 +925,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) diff --git a/core/src/test/scala/unit/kafka/log/CleanerTest.scala b/core/src/test/scala/unit/kafka/log/CleanerTest.scala index c5c3eb4fd2a7b..69218ba2e2e86 100755 --- a/core/src/test/scala/unit/kafka/log/CleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/CleanerTest.scala @@ -262,7 +262,7 @@ class CleanerTest extends JUnitSuite { // 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, Message.NoTimestamp, Message.MagicValue_V0)) + 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) @@ -452,7 +452,7 @@ class CleanerTest extends JUnitSuite { new ByteBufferMessageSet(new Message(key = key.toString.getBytes, bytes = value.toString.getBytes, timestamp = Message.NoTimestamp, - magicValue = Message.MagicValue_V0)) + magicValue = Message.MagicValue_V1)) def unkeyedMessage(value: Int) = new ByteBufferMessageSet(new Message(bytes=value.toString.getBytes)) @@ -461,7 +461,7 @@ class CleanerTest extends JUnitSuite { new ByteBufferMessageSet(new Message(key=key.toString.getBytes, bytes=null, timestamp = Message.NoTimestamp, - magicValue = Message.MagicValue_V0)) + magicValue = Message.MagicValue_V1)) } diff --git a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala index d3257f7ea1a20..e1a112a0a6b35 100644 --- a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala @@ -227,7 +227,7 @@ class FileMessageSetTest extends BaseMessageSetTestCases { offsetSeq = offsets, messages = messagesV1:_*) - // Down converion + // Down conversion // down conversion for non-compressed messages var fileMessageSet = new FileMessageSet(tempFile()) fileMessageSet.append(messageSetV1) diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 199f8b90cc271..91a44493d3b92 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -37,8 +37,6 @@ class LogManagerTest { logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) logProps.put(LogConfig.SegmentIndexBytesProp, 4096: java.lang.Integer) logProps.put(LogConfig.RetentionMsProp, maxLogAgeMs: java.lang.Integer) - // We need to use magic value 1 here because some tests requires no message format conversion. - logProps.put(LogConfig.MessageFormatVersionProp, "v1") val logConfig = LogConfig(logProps) var logDir: File = null var logManager: LogManager = null diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index c73a06647f1fe..426b5e825e84c 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -134,7 +134,7 @@ 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, "v1") + 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) @@ -164,7 +164,7 @@ class LogTest extends JUnitSuite { 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, "v1") + 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 @@ -350,7 +350,7 @@ class LogTest extends JUnitSuite { 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, "v1") + logProps.put(LogConfig.MessageFormatVersionProp, ApiVersion.latestVersion.toString()) val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) try { diff --git a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala index ecde7325a84af..c25a28111dd7d 100644 --- a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala @@ -21,6 +21,7 @@ import java.nio._ import java.util.concurrent.atomic.AtomicLong import kafka.utils.TestUtils +import org.apache.kafka.common.errors.InvalidTimestampException import org.junit.Assert._ import org.junit.Test @@ -184,30 +185,24 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { val now = System.currentTimeMillis() assertEquals("message set size should not change", messages.size, validatedMessages.size) - for (messageAndOffset <- validatedMessages) { - messageAndOffset.message.ensureValid() - assertTrue(messageAndOffset.message.timestamp >= startTime && messageAndOffset.message.timestamp <= now) - assertEquals(LogAppendTime, messageAndOffset.message.timestampType) - } + validatedMessages.foreach({case messageAndOffset => validateLogAppendTime(messageAndOffset.message)}) assertEquals("message set size should not change", compressedMessagesWithRecompresion.size, validatedCompressedMessages.size) - for (messageAndOffset <- validatedCompressedMessages) { - messageAndOffset.message.ensureValid() - assertTrue(s"Timestamp of message ${messageAndOffset.message}} should be between $startTime and $now", - messageAndOffset.message.timestamp >= startTime && messageAndOffset.message.timestamp <= now) - assertEquals(LogAppendTime, messageAndOffset.message.timestampType) - } + 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) - for (messageAndOffset <- validatedCompressedMessagesWithoutRecompression) { - messageAndOffset.message.ensureValid() - assertTrue(s"Timestamp of message ${messageAndOffset.message}} should be between $startTime and $now", - messageAndOffset.message.timestamp >= startTime && messageAndOffset.message.timestamp <= now) - assertEquals(LogAppendTime, messageAndOffset.message.timestampType) - } + 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(LogAppendTime, message.timestampType) + } } @Test @@ -258,7 +253,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { messageTimestampDiffMaxMs = 1000L) fail("Should throw InvalidMessageException.") } catch { - case e: InvalidMessageException => + case e: InvalidTimestampException => } try { @@ -270,7 +265,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { messageTimestampDiffMaxMs = 1000L) fail("Should throw InvalidMessageException.") } catch { - case e: InvalidMessageException => + case e: InvalidTimestampException => } } diff --git a/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala b/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala index cf1afeff7e080..53b85eff35961 100644 --- a/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala @@ -42,17 +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, Message.NoTimestamp, Message.MagicValue_V0), - new Message(bytes2k, Message.NoTimestamp, Message.MagicValue_V0), - new Message(bytes3k, Message.NoTimestamp, Message.MagicValue_V0)) + 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) { diff --git a/core/src/test/scala/unit/kafka/message/MessageTest.scala b/core/src/test/scala/unit/kafka/message/MessageTest.scala index 3f4433ec3fc4d..7ab708c7bcd53 100755 --- a/core/src/test/scala/unit/kafka/message/MessageTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageTest.scala @@ -107,47 +107,34 @@ class MessageTest extends JUnitSuite { @Test def testMessageFormatConversion() { + + def convertAndVerify(v: MessageTestVal, fromMessageFormat: Byte, toMessageFormat: Byte) { + assertEquals("Message should be the same when convert to the same version.", + v.message.toFormatVersion(fromMessageFormat), v.message) + val convertedMessage = v.message.toFormatVersion(toMessageFormat) + assertEquals("Size difference is not expected value", convertedMessage.size - v.message.size, + Message.headerSizeDiff(fromMessageFormat, toMessageFormat)) + assertTrue("Message should still be valid", convertedMessage.isValid) + assertEquals("Timestamp should be NoTimestamp", convertedMessage.timestamp, Message.NoTimestamp) + assertEquals(s"Magic value should be $toMessageFormat now", convertedMessage.magic, toMessageFormat) + if (convertedMessage.hasKey) + assertEquals("Message key should not change", convertedMessage.key, ByteBuffer.wrap(v.key)) + else + assertNull(convertedMessage.key) + if(v.payload == null) { + assertTrue(convertedMessage.isNull) + assertEquals("Payload should be null", null, convertedMessage.payload) + } else { + assertEquals("Message payload should not change", convertedMessage.payload, ByteBuffer.wrap(v.payload)) + } + assertEquals("Compression codec should not change", convertedMessage.compressionCodec, v.codec) + } + for (v <- messages) { if (v.magicValue == Message.MagicValue_V0) { - assertEquals("Message should be the same when convert to the same version.", - v.message.toFormatVersion(Message.MagicValue_V0), v.message) - val messageV1 = v.message.toFormatVersion(Message.MagicValue_V1) - assertEquals("Size difference is not expected value", messageV1.size - v.message.size, - Message.headerSizeDiff(Message.MagicValue_V0, Message.MagicValue_V1)) - assertTrue("Message should still be valid", messageV1.isValid) - assertEquals("Timestamp should be NoTimestamp", messageV1.timestamp, Message.NoTimestamp) - assertEquals("Magic value should be 1 now", messageV1.magic, Message.MagicValue_V1) - if (messageV1.hasKey) - assertEquals("Message key should not change", messageV1.key, ByteBuffer.wrap(v.key)) - else - assertNull(messageV1.key) - if(v.payload == null) { - assertTrue(messageV1.isNull) - assertEquals("Payload should be null", null, messageV1.payload) - } else { - assertEquals("Message payload should not change", messageV1.payload, ByteBuffer.wrap(v.payload)) - } - assertEquals("Compression codec should not change", messageV1.compressionCodec, v.codec) + convertAndVerify(v, Message.MagicValue_V0, Message.MagicValue_V1) } else if (v.magicValue == Message.MagicValue_V1) { - assertEquals("Message should be the same when convert to the same version.", - v.message.toFormatVersion(Message.MagicValue_V1), v.message) - val messageV0 = v.message.toFormatVersion(Message.MagicValue_V0) - assertEquals("Size difference is not expected value", messageV0.size - v.message.size, - Message.headerSizeDiff(Message.MagicValue_V1, Message.MagicValue_V0)) - assertTrue("Message should still be valid", messageV0.isValid) - assertEquals("Message should have NoTimestamp", Message.NoTimestamp, messageV0.timestamp) - assertEquals("Magic value should be 1 now", messageV0.magic, Message.MagicValue_V0) - if (messageV0.hasKey) - assertEquals("Message key should not change", messageV0.key, ByteBuffer.wrap(v.key)) - else - assertNull(messageV0.key) - if(v.payload == null) { - assertTrue(messageV0.isNull) - assertEquals("Payload should be null", null, messageV0.payload) - } else { - assertEquals("Message payload should not change", messageV0.payload, ByteBuffer.wrap(v.payload)) - } - assertEquals("Compression codec should not change", messageV0.compressionCodec, v.codec) + convertAndVerify(v, Message.MagicValue_V1, Message.MagicValue_V0) } } } @@ -159,7 +146,7 @@ class MessageTest extends JUnitSuite { @Test(expected = classOf[IllegalArgumentException]) def testInValidTimestamp() { - new Message("hello".getBytes, -3L, Message.MagicValue_V0) + new Message("hello".getBytes, -3L, Message.MagicValue_V1) } @Test(expected = classOf[IllegalArgumentException]) diff --git a/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala b/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala index 35b4d3a3d05bc..8c0862ec16164 100644 --- a/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala @@ -34,7 +34,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, timestamp = Message.NoTimestamp, timestampType = CreateTime, magicValue = Message.MagicValue_V0) { output => + writer.write(key = key, codec = codec, timestamp = Message.NoTimestamp, timestampType = CreateTime, magicValue = Message.MagicValue_V1) { output => val out = if (codec == NoCompressionCodec) output else CompressionFactory(codec, output) try { val p = rnd.nextInt(bytes.length) @@ -101,7 +101,7 @@ class MessageWriterTest extends JUnitSuite { def testWithNoCompressionAttribute(): Unit = { val bytes = mkRandomArray(4096) val actual = mkMessageWithWriter(bytes = bytes, codec = NoCompressionCodec) - val expected = new Message(bytes, Message.NoTimestamp, NoCompressionCodec, Message.MagicValue_V0) + val expected = new Message(bytes, Message.NoTimestamp, NoCompressionCodec, Message.MagicValue_V1) assertEquals(expected.buffer, actual.buffer) } @@ -109,7 +109,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), Message.NoTimestamp, SnappyCompressionCodec, Message.MagicValue_V0) + val expected = new Message(compress(bytes, SnappyCompressionCodec), Message.NoTimestamp, SnappyCompressionCodec, Message.MagicValue_V1) assertEquals( decompress(toArray(expected.payload), SnappyCompressionCodec).toSeq, @@ -122,7 +122,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, timestamp = Message.NoTimestamp, codec = NoCompressionCodec, magicValue = Message.MagicValue_V0) + val expected = new Message(bytes = bytes, key = key, timestamp = Message.NoTimestamp, codec = NoCompressionCodec, magicValue = Message.MagicValue_V1) assertEquals(expected.buffer, actual.buffer) } diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index b1120bd511084..ba6b7b510398f 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -282,13 +282,13 @@ class KafkaConfigTest { assertEquals(ApiVersion.latestVersion, conf.interBrokerProtocolVersion) props.put(KafkaConfig.InterBrokerProtocolVersionProp,"0.8.2.0") - props.put(KafkaConfig.MessageFormatVersionProp, "v0") + props.put(KafkaConfig.MessageFormatVersionProp, "0.9.0") val conf2 = KafkaConfig.fromProps(props) 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.MessageFormatVersionProp, "v0") + props.put(KafkaConfig.MessageFormatVersionProp, "0.9.0") val conf3 = KafkaConfig.fromProps(props) assertEquals(KAFKA_0_8_2, conf3.interBrokerProtocolVersion) diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index 94b9f52239477..5c2092c1af58b 100755 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -206,7 +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", "v1") + props.put("message.format.version", "0.10.0") props } diff --git a/docs/upgrade.html b/docs/upgrade.html index 527c2c1cab3ca..0a4b45a464098 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -17,14 +17,15 @@

    1.5 Upgrading From Previous Versions

    -

    Upgrading from 0.9.x to 0.10.0.0

    -0.10.0.0 has potential performance impact during upgrade and protocol changes (please review before upgrading). -Because new protocols are introduced, it is important to upgrade your Kafka clusters before upgrading your clients. +

    Upgrading from 0.8.x and 0.9.x to 0.10.0.0

    +0.10.0.0 has potential performance impact during upgrade and +potential breaking changes (please review before upgrading). Because new protocols +are introduced, it is important to upgrade your Kafka clusters before upgrading your clients.

    For a rolling upgrade:

      -
    1. Update server.properties file on all brokers and add the following properties: inter.broker.protocol.version=0.9.0.0, message.format.version=v0
    2. +
    3. Update server.properties file on all brokers and add the following properties: inter.broker.protocol.version=CURRENT_KAFKA_VERSION(e.g. 0.8.2, 0.9.0.0), message.format.version=CURRENT_KAFKA_VERSION(e.g. 0.8.2, 0.9.0.0)
    4. Upgrade the brokers. This can be done a broker at a time by simply bringing it down, updating the code, and restarting it.
    5. Once the entire cluster is upgraded, bump the protocol version by editing inter.broker.protocol.version and setting it to 0.10.0.0.
    6. Restart the brokers one by one for the new protocol version to take effect.
    7. @@ -52,15 +53,17 @@
      to 0.10.0.0 consumer.

      - Note: By setting the message format version, one certifies all the existing messages is on or below that - message format version. Otherwise consumers before 0.10.0.0 might break. + Note: 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 v1, one should not change it back to v0 since it may break the consumer on versions before 0.10.0.

      -
      protocol change for 0.10.0.0
      +
      potential breaking changes in 0.10.0.0
        -
      • Message format v1 is added to include a timestamp field in the messages and use relative offsets for compressed messages.
      • -
      • ProduceRequest/Response v2 is added to support message format v1
      • -
      • FetchRequest/Response v2 is added to support message format v1
      • +
      • Message format v1 is added and used by default to include a timestamp field in the messages and use relative offsets for compressed messages.
      • +
      • ProduceRequest/Response v2 is added and used by default to support message format v1
      • +
      • FetchRequest/Response v2 is added and used by default to support message format v1
      • +
      • MessageFormatter interface change: def writeTo(key: Array[Byte], value: Array[Byte], timestamp: Long, timestampType: TimestampType, output: PrintStream)

      Upgrading from 0.8.0, 0.8.1.X or 0.8.2.X to 0.9.0.0

      From 89958b95bf032e943a15c3d2291638af5dcd36bb Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Mon, 15 Feb 2016 22:24:45 -0800 Subject: [PATCH 23/33] Addressed Ismael's comments --- .../clients/consumer/ConsumerRecord.java | 6 ++++++ .../kafka/clients/producer/KafkaProducer.java | 9 +++++---- .../clients/producer/ProducerRecord.java | 17 ++++++++-------- .../kafka/common/protocol/Protocol.java | 2 +- .../common/requests/ProduceResponse.java | 4 ++-- core/src/main/scala/kafka/log/Log.scala | 2 +- .../kafka/message/ByteBufferMessageSet.scala | 20 +++++++++---------- .../main/scala/kafka/message/Message.scala | 5 +++-- 8 files changed, 37 insertions(+), 28 deletions(-) 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 8aa3c03530b37..42e0a906ae9da 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 @@ -91,10 +91,16 @@ public long offset() { return offset; } + /** + * The timestamp of this record + */ public long timestamp() { return timestamp; } + /** + * The timestamp type of this record + */ public TimestampType timestampType() { return timestampType; } 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 0bcc4ed5781ba..4f732be916d79 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 @@ -360,10 +360,11 @@ public Future send(ProducerRecord record) { * response after each one. *

      * 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#CreateTime} - * is used by the topic, the timestamp will be the user provided timestamp or the record send time if user did not - * specify a timestamp for the record. If {@link org.apache.kafka.common.record.TimestampType#LogAppendTime} is - * used for the topic, the timestamp will be the Kafka broker local time when the message is appended. + * it was assigned and the timestamp of the record. If + * {@link org.apache.kafka.common.record.TimestampType#CreateTime CreateTime} is used by the topic, the timestamp + * will be the user provided timestamp or the record send time if user did not specify a timestamp for the record. + * If {@link org.apache.kafka.common.record.TimestampType#LogAppendTime LogAppendTime} is used for the topic, the + * timestamp will be the Kafka broker local time when the message is appended. *

      * 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() 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 f4dcce7be146b..6c1c5e55d42e4 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 @@ -20,19 +20,20 @@ * 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. *

      - * The record also has an associated timestamp. If 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. + * 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. *

    8. * If the topic is configured to use {@link org.apache.kafka.common.record.TimestampType#CreateTime CreateTime} * the timestamp in the producer record will be used by broker. *
    9. *
    10. * If the topic is configured to use {@link org.apache.kafka.common.record.TimestampType#LogAppendTime LogAppendTime} - * the timestamp in the producer record will be overwritten by broker with broker local time when broker append the + * 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. *
    11. *

      - * In either of the above case, the timestamp that has actually been used will be returned to user in + * 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 { @@ -44,7 +45,7 @@ public final class ProducerRecord { private final Long timestamp; /** - * Creates a record to be sent to a specified topic and partition + * Creates a record to be sent to a specified topic and partition with a specified timestamp * * @param topic The topic the record will be appended to * @param partition The partition to which the record should be sent @@ -98,14 +99,14 @@ public ProducerRecord(String topic, V 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; @@ -126,7 +127,7 @@ public Long timestamp() { } /** - * The partition to which the record will be sent (or null if no partition was specified) + * @return The partition to which the record will be sent (or null if no partition was specified) */ public Integer partition() { return partition; 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 e2b1cbf0337c6..57a837daeceea 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 @@ -137,7 +137,7 @@ public class Protocol { " due to quota violation. (Zero if the request did not violate any quota.)", 0)); /** - * PRODUCE_RESPONSE_V2 added the timestamp in per partition response status. + * 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. */ 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 b4a2d422e0374..ee2df59707f13 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 @@ -69,7 +69,7 @@ public ProduceResponse(Map responses) { } /** - * Constructor for latest version + * Constructor for the latest version * @param responses Produced data grouped by topic-partition * @param throttleTime Time in milliseconds the response was throttled */ @@ -78,7 +78,7 @@ public ProduceResponse(Map responses, int thr } /** - * Constructor for specific version + * 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. diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index c0119a6707b15..c42960d6578aa 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -333,7 +333,7 @@ class Log(val dir: File, if (assignOffsets) { // assign offsets to the message set val offset = new AtomicLong(nextOffsetMetadata.messageOffset) - val now = System.currentTimeMillis() + val now = SystemTime.milliseconds try { validMessages = validMessages.validateMessagesAndAssignOffsets(offset, now, diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 31bc9aea058c3..f2accf546c6cd 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -280,10 +280,10 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi // This constructor is only used internally private[kafka] def this(compressionCodec: CompressionCodec, - offsetCounter: AtomicLong, - wrapperMessageTimestamp: Option[Long], - timestampType: TimestampType, - messages: Message*) { + offsetCounter: AtomicLong, + wrapperMessageTimestamp: Option[Long], + timestampType: TimestampType, + messages: Message*) { this(ByteBufferMessageSet.create(new OffsetAssigner(offsetCounter, messages.size), compressionCodec, wrapperMessageTimestamp, timestampType, messages:_*)) } @@ -391,9 +391,9 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi * starting from 0. * 3. When magic value = 1, validate and maybe overwrite timestamps of messages. * - * This method will convert the messages based on the following scenarios: - * A. Magic value of a message = 0 and messageFormatVersion is 0 - * B. Magic value of a message = 1 and messageFormatVersion is 1 + * 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 avoids re-compression. @@ -500,8 +500,8 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi } // We create this method to save memory copy operation. It reads from the original message set and directly - // write the converted messages into new message set buffer. Hence we don't need to allocate memory for each message - // during message conversion. + // 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, @@ -572,7 +572,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi /** * This method validates the timestamps of a message. - * If the message is using create time, this method checks if it is with acceptable range. + * If the message is using create time, this method checks if it is within acceptable range. */ private def validateTimestamp(message: Message, now: Long, diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala index fc65d13126156..5a7ef7d107d78 100755 --- a/core/src/main/scala/kafka/message/Message.scala +++ b/core/src/main/scala/kafka/message/Message.scala @@ -147,12 +147,13 @@ class Message(val buffer: ByteBuffer, /** * 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. Default to CreateTime - * @param key The key of the message (null, if none) + * @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], From c3ca748783da8e9036daec8f4e19ab259f7daea9 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Tue, 16 Feb 2016 12:56:33 -0800 Subject: [PATCH 24/33] Addressed Ismael's comments. --- .../kafka/clients/producer/KafkaProducer.java | 4 +- .../clients/producer/ProducerRecord.java | 4 +- .../producer/internals/RecordBatch.java | 5 +- .../kafka/common/record/MemoryRecords.java | 8 ++- .../apache/kafka/common/record/Record.java | 10 +-- .../kafka/common/record/TimestampType.java | 21 +++++- .../kafka/common/requests/ProduceRequest.java | 1 - .../clients/consumer/MockConsumerTest.java | 4 +- .../internals/ConsumerInterceptorsTest.java | 6 +- .../kafka/test/MockConsumerInterceptor.java | 2 +- .../connect/runtime/WorkerSinkTaskTest.java | 2 +- .../runtime/WorkerSinkTaskThreadedTest.java | 4 +- .../storage/KafkaConfigStorageTest.java | 24 +++---- .../storage/KafkaOffsetBackingStoreTest.java | 16 ++--- .../kafka/connect/util/KafkaBasedLogTest.java | 16 ++--- .../scala/kafka/consumer/BaseConsumer.scala | 5 +- .../coordinator/GroupMetadataManager.scala | 1 + core/src/main/scala/kafka/log/Log.scala | 3 +- core/src/main/scala/kafka/log/LogConfig.scala | 3 +- .../kafka/message/ByteBufferMessageSet.scala | 23 ++++--- .../main/scala/kafka/message/Message.scala | 14 ++-- .../kafka/message/MessageAndMetadata.scala | 3 +- .../scala/kafka/message/MessageWriter.scala | 1 + .../scala/kafka/message/TimestampType.scala | 68 ------------------- .../scala/kafka/tools/ConsoleConsumer.scala | 7 +- .../main/scala/kafka/tools/MirrorMaker.scala | 4 +- .../kafka/api/BaseConsumerTest.scala | 4 +- .../kafka/api/BaseProducerSendTest.scala | 15 ++-- .../kafka/api/PlaintextConsumerTest.scala | 4 +- .../message/ByteBufferMessageSetTest.scala | 37 +++++----- .../kafka/message/MessageWriterTest.scala | 3 +- .../unit/kafka/producer/ProducerTest.scala | 9 +-- .../processor/internals/RecordQueue.java | 2 +- .../internals/PartitionGroupTest.java | 12 ++-- .../internals/ProcessorStateManagerTest.java | 6 +- .../processor/internals/RecordQueueTest.java | 18 ++--- .../processor/internals/StandbyTaskTest.java | 18 ++--- .../processor/internals/StreamTaskTest.java | 30 ++++---- .../test/ProcessorTopologyTestDriver.java | 2 +- 39 files changed, 190 insertions(+), 229 deletions(-) delete mode 100644 core/src/main/scala/kafka/message/TimestampType.scala 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 4f732be916d79..caf953bb76aa0 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 @@ -361,9 +361,9 @@ public Future send(ProducerRecord record) { *

      * 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#CreateTime CreateTime} is used by the topic, the timestamp + * {@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 user did not specify a timestamp for the record. - * If {@link org.apache.kafka.common.record.TimestampType#LogAppendTime LogAppendTime} is used for the topic, the + * 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. *

      * Since the send call is asynchronous it returns a {@link java.util.concurrent.Future Future} for the 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 6c1c5e55d42e4..9110e6bc65b64 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 @@ -24,11 +24,11 @@ * record with its current time. The timestamp eventually used by Kafka depends on the timestamp type configured for * the topic. *

    12. - * If the topic is configured to use {@link org.apache.kafka.common.record.TimestampType#CreateTime CreateTime} + * 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 broker. *
    13. *
    14. - * If the topic is configured to use {@link org.apache.kafka.common.record.TimestampType#LogAppendTime LogAppendTime} + * 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. *
    15. 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 37e067b4071f0..362f267b44ac1 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 @@ -14,7 +14,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.concurrent.atomic.AtomicLong; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.RecordMetadata; @@ -36,7 +35,7 @@ public final class RecordBatch { public int recordCount = 0; public int maxRecordSize = 0; - private final AtomicLong offsetCounter = new AtomicLong(0); + private Long offsetCounter = 0L; public volatile int attempts = 0; public final long createdMs; public long drainedMs; @@ -68,7 +67,7 @@ public FutureRecordMetadata tryAppend(long timestamp, byte[] key, byte[] value, if (!this.records.hasRoomFor(key, value)) { return null; } else { - this.records.append(offsetCounter.getAndIncrement(), timestamp, 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, timestamp); 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 88d8e3028489a..01da1e21b55e5 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,8 +16,8 @@ import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.ArrayDeque; import java.util.Iterator; -import java.util.LinkedList; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.utils.AbstractIterator; @@ -216,7 +216,7 @@ public static class RecordsIterator extends AbstractIterator { private RecordsIterator innerIter; // The variables for inner iterator - private final LinkedList logEntries; + private final ArrayDeque logEntries; private final long absoluteBaseOffset; public RecordsIterator(ByteBuffer buffer, CompressionType type, boolean shallow) { @@ -238,7 +238,7 @@ private RecordsIterator(LogEntry entry) { // 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 LinkedList<>(); + this.logEntries = new ArrayDeque<>(); long wrapperRecordTimestamp = entry.record().timestamp(); while (true) { try { @@ -292,6 +292,8 @@ protected LogEntry makeNext() { // which will de-compress the payload to a set of messages; // since we assume nested compression is not allowed, the deep iterator // would not try to further decompress underlying messages + // There will be at least one element in the inner iterator, so we don't + // need to call hasNext() here. innerIter = new RecordsIterator(entry); return innerIter.next(); } 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 388de93893dd4..8390dc7eff054 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 @@ -64,7 +64,7 @@ public final class Record { /** * The current "magic" value */ - public static final byte CURRENT_MAGIC_VALUE = 1; + 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 @@ -324,15 +324,15 @@ public byte attributes() { /** * 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 = LogAppendTime and WrapperRecordTimestamp is not null - Compressed message using LogAppendTime - * 3. wrapperRecordTimestampType = CreateTime and wrapperRecordTimestamp is not null - Compressed message using CreateTime + * 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.LogAppendTime && wrapperRecordTimestamp != null) + if (wrapperRecordTimestampType == TimestampType.LOG_APPEND_TIME && wrapperRecordTimestamp != null) return wrapperRecordTimestamp; // Case 1, 3 else @@ -345,7 +345,7 @@ public long timestamp() { */ public TimestampType timestampType() { if (magic() == 0) - return TimestampType.NoTimestampType; + return TimestampType.NO_TIMESTAMP_TYPE; else return wrapperRecordTimestampType == null ? TimestampType.getTimestampType(attributes()) : wrapperRecordTimestampType; } diff --git a/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java b/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java index 34073c289d59a..ab12a35079581 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java +++ b/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java @@ -17,11 +17,13 @@ package org.apache.kafka.common.record; +import java.util.NoSuchElementException; + /** * The timestamp type of the records. */ public enum TimestampType { - NoTimestampType(-1, "NoTimestampType"), CreateTime(0, "CreateTime"), LogAppendTime(1, "LogAppendTime"); + NO_TIMESTAMP_TYPE(-1, "NoTimestampType"), CREATE_TIME(0, "CreateTime"), LOG_APPEND_TIME(1, "LogAppendTime"); public final int value; public final String name; @@ -32,14 +34,27 @@ public enum TimestampType { public static TimestampType getTimestampType(byte attributes) { int timestampType = (attributes & Record.TIMESTAMP_TYPE_MASK) >> Record.TIMESTAMP_TYPE_ATTRIBUTE_OFFSET; - return timestampType == 0 ? CreateTime : LogAppendTime; + return timestampType == 0 ? CREATE_TIME : LOG_APPEND_TIME; } public static byte setTimestampType(byte attributes, TimestampType timestampType) { - return timestampType == CreateTime ? + 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; 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 146c8b39a7713..c7d41e664c9b2 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 @@ -106,7 +106,6 @@ public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { case 0: return new ProduceResponse(responseMap); case 1: - return new ProduceResponse(responseMap, ProduceResponse.DEFAULT_THROTTLE_TIME, versionId); case 2: return new ProduceResponse(responseMap, ProduceResponse.DEFAULT_THROTTLE_TIME, versionId); default: 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 af8aa50d37765..3ef5c8bea17c1 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 @@ -43,8 +43,8 @@ public void testSimpleMock() { beginningOffsets.put(new TopicPartition("test", 1), 0L); consumer.updateBeginningOffsets(beginningOffsets); consumer.seek(new TopicPartition("test", 0), 0); - ConsumerRecord rec1 = new ConsumerRecord("test", 0, 0, 0L, TimestampType.CreateTime, "key1", "value1"); - ConsumerRecord rec2 = new ConsumerRecord("test", 0, 1, 0L, TimestampType.CreateTime, "key2", "value2"); + ConsumerRecord rec1 = new ConsumerRecord("test", 0, 0, 0L, TimestampType.CREATE_TIME, "key1", "value1"); + ConsumerRecord rec2 = new ConsumerRecord("test", 0, 1, 0L, TimestampType.CREATE_TIME, "key2", "value2"); consumer.addRecord(rec1); consumer.addRecord(rec2); ConsumerRecords recs = consumer.poll(1); 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 d0748c167a45b..25843c7ae621b 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 @@ -44,7 +44,7 @@ public class ConsumerInterceptorsTest { private final TopicPartition filterTopicPart1 = new TopicPartition("test5", filterPartition1); private final TopicPartition filterTopicPart2 = new TopicPartition("test6", filterPartition2); private final ConsumerRecord consumerRecord = - new ConsumerRecord<>(topic, partition, 0, 0L, TimestampType.CreateTime, 1, 1); + new ConsumerRecord<>(topic, partition, 0, 0L, TimestampType.CREATE_TIME, 1, 1); private int onCommitCount = 0; private int onConsumeCount = 0; @@ -117,9 +117,9 @@ public void testOnConsumeChain() { List> list1 = new ArrayList<>(); list1.add(consumerRecord); List> list2 = new ArrayList<>(); - list2.add(new ConsumerRecord<>(filterTopicPart1.topic(), filterTopicPart1.partition(), 0, 0L, TimestampType.CreateTime, 1, 1)); + list2.add(new ConsumerRecord<>(filterTopicPart1.topic(), filterTopicPart1.partition(), 0, 0L, TimestampType.CREATE_TIME, 1, 1)); List> list3 = new ArrayList<>(); - list3.add(new ConsumerRecord<>(filterTopicPart2.topic(), filterTopicPart2.partition(), 0, 0L, TimestampType.CreateTime, 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); 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 ea875cd2a762b..3246578f78edf 100644 --- a/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java +++ b/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java @@ -57,7 +57,7 @@ public ConsumerRecords onConsume(ConsumerRecords List> lst = new ArrayList<>(); for (ConsumerRecord record: records.records(tp)) { lst.add(new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), - 0L, TimestampType.CreateTime, record.key(), record.value().toUpperCase())); + 0L, TimestampType.CREATE_TIME, record.key(), record.value().toUpperCase())); } recordMap.put(tp, lst); } 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 6c8f59ca84e45..978e3a1ddd21c 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 @@ -291,7 +291,7 @@ private void expectConsumerPoll(final int numMessages) { public ConsumerRecords answer() throws Throwable { List> records = new ArrayList<>(); for (int i = 0; i < numMessages; i++) - records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned + i, 0L, TimestampType.CreateTime, 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 ? 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 3615c43462a2c..e202209058458 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 @@ -520,7 +520,7 @@ public ConsumerRecords answer() throws Throwable { Collections.singletonMap( new TopicPartition(TOPIC, PARTITION), Arrays.asList( - new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, 0L, TimestampType.CreateTime, RAW_KEY, RAW_VALUE) + new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, 0L, TimestampType.CREATE_TIME, RAW_KEY, RAW_VALUE) ))); recordsReturned++; return records; @@ -548,7 +548,7 @@ public ConsumerRecords answer() throws Throwable { Collections.singletonMap( new TopicPartition(TOPIC, PARTITION), Arrays.asList( - new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, 0L, TimestampType.CreateTime, RAW_KEY, RAW_VALUE) + new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, 0L, TimestampType.CREATE_TIME, RAW_KEY, RAW_VALUE) ))); recordsReturned++; return records; 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 3db3ff9d1f0a2..e878e127f0e95 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 @@ -289,14 +289,14 @@ public void testRestore() throws Exception { expectConfigure(); // Overwrite each type at least once to ensure we see the latest data after loading List> existingRecords = Arrays.asList( - new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CreateTime, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)), - new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CreateTime, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(1)), - new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CreateTime, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)), - new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CreateTime, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(3)), - new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CreateTime, 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, 0L, TimestampType.CreateTime, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5)), - new ConsumerRecord<>(TOPIC, 0, 6, 0L, TimestampType.CreateTime, 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 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)); @@ -343,12 +343,12 @@ public void testPutTaskConfigsDoesNotResolveAllInconsistencies() throws Exceptio expectConfigure(); List> existingRecords = Arrays.asList( - new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CreateTime, 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, 0L, TimestampType.CreateTime, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)), - new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CreateTime, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(4)), - new ConsumerRecord<>(TOPIC, 0, 5, 0L, TimestampType.CreateTime, 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 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)); @@ -484,7 +484,7 @@ private void expectReadToEnd(final LinkedHashMap serializedConfi public Future answer() throws Throwable { TestFuture future = new TestFuture(); for (Map.Entry entry : serializedConfigs.entrySet()) - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, logOffset++, 0L, TimestampType.CreateTime, 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; } 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 a803eab2acaa2..61763a80c703e 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 @@ -126,10 +126,10 @@ public void testStartStop() throws Exception { public void testReloadOnStart() throws Exception { expectConfigure(); expectStart(Arrays.asList( - new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CreateTime, TP0_KEY.array(), TP0_VALUE.array()), - new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CreateTime, TP1_KEY.array(), TP1_VALUE.array()), - new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CreateTime, TP0_KEY.array(), TP0_VALUE_NEW.array()), - new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CreateTime, 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(); @@ -177,8 +177,8 @@ public Object answer() throws Throwable { PowerMock.expectLastCall().andAnswer(new IAnswer() { @Override public Object answer() throws Throwable { - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CreateTime, TP0_KEY.array(), TP0_VALUE.array())); - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CreateTime, 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; } @@ -190,8 +190,8 @@ public Object answer() throws Throwable { PowerMock.expectLastCall().andAnswer(new IAnswer() { @Override public Object answer() throws Throwable { - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CreateTime, TP0_KEY.array(), TP0_VALUE_NEW.array())); - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CreateTime, 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; } 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 fac6038b20ecd..b2246f54b1f13 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 @@ -183,7 +183,7 @@ public void run() { consumer.schedulePollTask(new Runnable() { @Override public void run() { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CreateTime, TP0_KEY, TP0_VALUE)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, TP0_KEY, TP0_VALUE)); } }); consumer.scheduleNopPollTask(); @@ -191,7 +191,7 @@ public void run() { consumer.schedulePollTask(new Runnable() { @Override public void run() { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CreateTime, TP1_KEY, TP1_VALUE)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, TP1_KEY, TP1_VALUE)); } }); consumer.schedulePollTask(new Runnable() { @@ -298,16 +298,16 @@ public void run() { consumer.schedulePollTask(new Runnable() { @Override public void run() { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CreateTime, TP0_KEY, TP0_VALUE)); - consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CreateTime, TP0_KEY, TP0_VALUE_NEW)); - consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CreateTime, 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, 0L, TimestampType.CreateTime, TP1_KEY, TP1_VALUE_NEW)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CREATE_TIME, TP1_KEY, TP1_VALUE_NEW)); } }); @@ -363,8 +363,8 @@ public void run() { consumer.schedulePollTask(new Runnable() { @Override public void run() { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CreateTime, TP0_KEY, TP0_VALUE_NEW)); - consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CreateTime, 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)); } }); diff --git a/core/src/main/scala/kafka/consumer/BaseConsumer.scala b/core/src/main/scala/kafka/consumer/BaseConsumer.scala index 76b5d7af01ba0..999a375a20b13 100644 --- a/core/src/main/scala/kafka/consumer/BaseConsumer.scala +++ b/core/src/main/scala/kafka/consumer/BaseConsumer.scala @@ -21,8 +21,9 @@ import java.util.Properties import java.util.regex.Pattern import kafka.common.StreamEndException -import kafka.message.{NoTimestampType, TimestampType, Message} +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 @@ -40,7 +41,7 @@ case class BaseConsumerRecord(topic: String, partition: Int, offset: Long, timestamp: Long = Message.NoTimestamp, - timestampType: TimestampType = NoTimestampType, + timestampType: TimestampType = TimestampType.NO_TIMESTAMP_TYPE, key: Array[Byte], value: Array[Byte]) diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index 975cb0efc3e23..7c91db8e7d9b4 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -27,6 +27,7 @@ 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 diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index c42960d6578aa..f8c0b77ef2b2d 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -28,6 +28,7 @@ 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 @@ -348,7 +349,7 @@ class Log(val dir: File, } 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 == LogAppendTime) + if (config.messageTimestampType == TimestampType.LOG_APPEND_TIME) appendInfo.timestamp = now } else { // we are taking the offsets we are given diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index 08d6e997388f8..2c5e578dd971b 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -20,9 +20,10 @@ package kafka.log import java.util.Properties import kafka.api.ApiVersion -import kafka.message.{BrokerCompressionCodec, Message, TimestampType} +import kafka.message.{BrokerCompressionCodec, Message} import kafka.server.KafkaConfig import org.apache.kafka.common.config.{AbstractConfig, ConfigDef} +import org.apache.kafka.common.record.TimestampType import org.apache.kafka.common.utils.Utils object Defaults { diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index f2accf546c6cd..d822817e5f7b6 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -26,6 +26,7 @@ import java.io._ 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 @@ -261,17 +262,17 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi def this(compressionCodec: CompressionCodec, messages: Message*) { this(ByteBufferMessageSet.create(new OffsetAssigner(new AtomicLong(0), messages.size), compressionCodec, - None, CreateTime, messages:_*)) + None, TimestampType.CREATE_TIME, messages:_*)) } def this(compressionCodec: CompressionCodec, offsetCounter: AtomicLong, messages: Message*) { this(ByteBufferMessageSet.create(new OffsetAssigner(offsetCounter, messages.size), compressionCodec, - None, CreateTime, messages:_*)) + None, TimestampType.CREATE_TIME, messages:_*)) } def this(compressionCodec: CompressionCodec, offsetSeq: Seq[Long], messages: Message*) { this(ByteBufferMessageSet.create(new OffsetAssigner(offsetSeq), compressionCodec, - None, CreateTime, messages:_*)) + None, TimestampType.CREATE_TIME, messages:_*)) } def this(messages: Message*) { @@ -456,7 +457,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi val wrapperMessageTimestamp = { if (messageFormatVersion == Message.MagicValue_V0) Some(Message.NoTimestamp) - else if (messageFormatVersion > Message.MagicValue_V0 && messageTimestampType == CreateTime) + else if (messageFormatVersion > Message.MagicValue_V0 && messageTimestampType == TimestampType.CREATE_TIME) Some(maxTimestamp) else // Log append time Some(now) @@ -478,13 +479,13 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi val attributeOffset = MessageSet.LogOverhead + Message.AttributesOffset val timestamp = buffer.getLong(timestampOffset) val attributes = buffer.get(attributeOffset) - if (messageTimestampType == CreateTime && timestamp == maxTimestamp) + if (messageTimestampType == TimestampType.CREATE_TIME && timestamp == maxTimestamp) // We don't need to recompute crc if the timestamp is not updated. crcUpdateNeeded = false - else if (messageTimestampType == LogAppendTime) { + else if (messageTimestampType == TimestampType.LOG_APPEND_TIME) { // Set timestamp type and timestamp buffer.putLong(timestampOffset, now) - buffer.put(attributeOffset, TimestampType.setTimestampType(attributes, LogAppendTime)) + buffer.put(attributeOffset, TimestampType.setTimestampType(attributes, TimestampType.LOG_APPEND_TIME)) } if (crcUpdateNeeded) { @@ -553,9 +554,9 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi validateMessageKey(message, compactedTopic) if (message.magic > Message.MagicValue_V0) { validateTimestamp(message, now, timestampType, timestampDiffMaxMs) - if (timestampType == LogAppendTime) { + if (timestampType == TimestampType.LOG_APPEND_TIME) { message.buffer.putLong(Message.TimestampOffset, now) - message.buffer.put(Message.AttributesOffset, TimestampType.setTimestampType(message.attributes, LogAppendTime)) + message.buffer.put(Message.AttributesOffset, TimestampType.setTimestampType(message.attributes, TimestampType.LOG_APPEND_TIME)) Utils.writeUnsignedInt(message.buffer, Message.CrcOffset, message.computeChecksum()) } } @@ -578,10 +579,10 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi now: Long, timestampType: TimestampType, timestampDiffMaxMs: Long) { - if (timestampType == CreateTime && math.abs(message.timestamp - now) > timestampDiffMaxMs) + 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 == LogAppendTime) + 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.") } diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala index 5a7ef7d107d78..42b30674922c6 100755 --- a/core/src/main/scala/kafka/message/Message.scala +++ b/core/src/main/scala/kafka/message/Message.scala @@ -19,6 +19,8 @@ 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 @@ -149,7 +151,7 @@ class Message(val buffer: ByteBuffer, * @param bytes The payload of the message * @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. Default to CreateTime + * @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 @@ -206,7 +208,7 @@ class Message(val buffer: ByteBuffer, } def this(bytes: Array[Byte], key: Array[Byte], timestamp: Long, codec: CompressionCodec, magicValue: Byte) = - this(bytes = bytes, key = key, timestamp = timestamp, timestampType = CreateTime, codec = codec, payloadOffset = 0, payloadSize = -1, magicValue = magicValue) + this(bytes = bytes, key = key, timestamp = timestamp, timestampType = TimestampType.CREATE_TIME, codec = codec, payloadOffset = 0, payloadSize = -1, magicValue = magicValue) def this(bytes: Array[Byte], timestamp: Long, codec: CompressionCodec, magicValue: Byte) = this(bytes = bytes, key = null, timestamp = timestamp, codec = codec, magicValue = magicValue) @@ -306,7 +308,7 @@ class Message(val buffer: ByteBuffer, if (magic == MagicValue_V0) Message.NoTimestamp // Case 2 - else if (wrapperMessageTimestampType.exists(_ == LogAppendTime) && wrapperMessageTimestamp.isDefined) + else if (wrapperMessageTimestampType.exists(_ == TimestampType.LOG_APPEND_TIME) && wrapperMessageTimestamp.isDefined) wrapperMessageTimestamp.get else // case 1, 3 buffer.getLong(Message.TimestampOffset) @@ -317,7 +319,7 @@ class Message(val buffer: ByteBuffer, */ def timestampType = { if (magic == MagicValue_V0) - NoTimestampType + TimestampType.NO_TIMESTAMP_TYPE else wrapperMessageTimestampType.getOrElse(TimestampType.getTimestampType(attributes)) } @@ -365,7 +367,7 @@ class Message(val buffer: ByteBuffer, byteBuffer.put(Message.MagicValue_V1) byteBuffer.put(TimestampType.setTimestampType(attributes, timestampType)) // Up-conversion, insert the timestamp field - if (timestampType == LogAppendTime) + if (timestampType == TimestampType.LOG_APPEND_TIME) byteBuffer.putLong(now) else byteBuffer.putLong(Message.NoTimestamp) @@ -374,7 +376,7 @@ class Message(val buffer: ByteBuffer, // Down-conversion, reserve CRC and update magic byte byteBuffer.position(Message.MagicOffset) byteBuffer.put(Message.MagicValue_V0) - byteBuffer.put(TimestampType.setTimestampType(attributes, CreateTime)) + 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) } diff --git a/core/src/main/scala/kafka/message/MessageAndMetadata.scala b/core/src/main/scala/kafka/message/MessageAndMetadata.scala index 9c079fcc57372..ac9ef77d515e7 100755 --- a/core/src/main/scala/kafka/message/MessageAndMetadata.scala +++ b/core/src/main/scala/kafka/message/MessageAndMetadata.scala @@ -18,6 +18,7 @@ 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, @@ -25,7 +26,7 @@ case class MessageAndMetadata[K, V](topic: String, private val rawMessage: Message, offset: Long, timestamp: Long = Message.NoTimestamp, - timestampType: TimestampType = CreateTime, + timestampType: TimestampType = TimestampType.CREATE_TIME, keyDecoder: Decoder[K], valueDecoder: Decoder[V]) { /** diff --git a/core/src/main/scala/kafka/message/MessageWriter.scala b/core/src/main/scala/kafka/message/MessageWriter.scala index 16043062729a4..660772c156707 100755 --- a/core/src/main/scala/kafka/message/MessageWriter.scala +++ b/core/src/main/scala/kafka/message/MessageWriter.scala @@ -20,6 +20,7 @@ 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) { diff --git a/core/src/main/scala/kafka/message/TimestampType.scala b/core/src/main/scala/kafka/message/TimestampType.scala deleted file mode 100644 index 67399486fa0ed..0000000000000 --- a/core/src/main/scala/kafka/message/TimestampType.scala +++ /dev/null @@ -1,68 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.message - -/** - * The timestamp type of the messages. - */ -object TimestampType { - - def getTimestampType(attribute: Byte) = { - (attribute & Message.TimestampTypeMask) >> Message.TimestampTypeAttributeBitOffset match { - case 0 => CreateTime - case 1 => LogAppendTime - } - } - - def setTimestampType(attribute: Byte, timestampType: TimestampType): Byte = { - if (timestampType == CreateTime) - (attribute & ~Message.TimestampTypeMask).toByte - else - (attribute | Message.TimestampTypeMask).toByte - } - - def forName(name: String): TimestampType = { - name match { - case CreateTime.name => CreateTime - case LogAppendTime.name => LogAppendTime - case NoTimestampType.name => NoTimestampType - } - } - -} - -sealed trait TimestampType { - def value: Int - def name: String - override def toString = name -} - -case object NoTimestampType extends TimestampType { - val value = -1 - val name = "NoTimestampType" -} - -case object CreateTime extends TimestampType { - val value = 0 - val name = "CreateTime" -} - -case object LogAppendTime extends TimestampType { - val value = 1 - val name = "LogAppendTime" -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala index 177d57bf52f95..8937740022c60 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 @@ -357,7 +358,7 @@ class DefaultMessageFormatter extends MessageFormatter { } def writeTo(key: Array[Byte], value: Array[Byte], timestamp: Long, timestampType: TimestampType, output: PrintStream) { - if (timestampType != NoTimestampType) { + if (timestampType != TimestampType.NO_TIMESTAMP_TYPE) { output.write(s"$timestampType:$timestamp".getBytes) output.write(keySeparator) } @@ -379,7 +380,7 @@ class LoggingMessageFormatter extends MessageFormatter { if(logger.isInfoEnabled) logger.info(s"key:${if (key == null) "null" else new String(key)}, " + s"value:${if (value == null) "null" else new String(value)}, " + - {if (timestampType != NoTimestampType) s"$timestampType:$timestamp" else ""}) + {if (timestampType != TimestampType.NO_TIMESTAMP_TYPE) s"$timestampType:$timestamp" else ""}) } } @@ -402,7 +403,7 @@ class ChecksumMessageFormatter extends MessageFormatter { def writeTo(key: Array[Byte], value: Array[Byte], timestamp: Long, timestampType: TimestampType, output: PrintStream) { val chksum = - if (timestampType != NoTimestampType) + 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 diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index 61177de79f6aa..cd5952efeb30e 100755 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -28,7 +28,7 @@ import joptsimple.OptionParser import kafka.client.ClientUtils import kafka.consumer.{BaseConsumerRecord, ConsumerIterator, BaseConsumer, Blacklist, ConsumerConfig, ConsumerThreadId, ConsumerTimeoutException, TopicFilter, Whitelist, ZookeeperConsumerConnector} import kafka.javaapi.consumer.ConsumerRebalanceListener -import kafka.message.{TimestampType, MessageAndMetadata} +import kafka.message.MessageAndMetadata import kafka.metrics.KafkaMetricsGroup import kafka.serializer.DefaultDecoder import kafka.utils.{CommandLineUtils, CoreUtils, Logging} @@ -551,7 +551,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { record.partition, record.offset, record.timestamp(), - TimestampType.forName(record.timestampType().name), + record.timestampType(), record.key, record.value) } diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index 93873818fa099..f15c005210b5e 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -288,7 +288,7 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { startingOffset: Int, startingKeyAndValueIndex: Int = 0, startingTimestamp: Long = 0L, - timestampType: TimestampType = TimestampType.CreateTime, + timestampType: TimestampType = TimestampType.CREATE_TIME, tp: TopicPartition = tp) { val records = consumeRecords(consumer, numRecords) val now = System.currentTimeMillis() @@ -297,7 +297,7 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { val offset = startingOffset + i assertEquals(tp.topic(), record.topic()) assertEquals(tp.partition(), record.partition()) - if (timestampType == TimestampType.CreateTime) { + if (timestampType == TimestampType.CREATE_TIME) { assertEquals(timestampType, record.timestampType()) val timestamp = startingTimestamp + i assertEquals(timestamp.toLong, record.timestamp()) diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala index 88934960589a2..807b8bb9055ca 100644 --- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala @@ -27,10 +27,11 @@ import kafka.message.Message import kafka.server.KafkaConfig import kafka.utils.TestUtils import org.apache.kafka.clients.producer._ -import org.apache.kafka.common.errors.{InvalidTimestampException, CorruptRecordException, 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 { @@ -149,13 +150,13 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { 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.CreateTime) + sendAndVerifyTimestamp(producer, TimestampType.CREATE_TIME) } @Test def testSendNonCompressedMessageWithCreateTime() { val producer = createProducer(brokerList = brokerList, lingerMs = Long.MaxValue) - sendAndVerifyTimestamp(producer, TimestampType.CreateTime) + sendAndVerifyTimestamp(producer, TimestampType.CREATE_TIME) } @Test @@ -163,13 +164,13 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { 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.LogAppendTime) + sendAndVerifyTimestamp(producer, TimestampType.LOG_APPEND_TIME) } @Test def testSendNonCompressedMessageWithLogApendTime() { val producer = createProducer(brokerList = brokerList, lingerMs = Long.MaxValue) - sendAndVerifyTimestamp(producer, TimestampType.LogAppendTime) + sendAndVerifyTimestamp(producer, TimestampType.LOG_APPEND_TIME) } private def sendAndVerifyTimestamp(producer: KafkaProducer[Array[Byte], Array[Byte]], timestampType: TimestampType) { @@ -186,7 +187,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { if (exception == null) { assertEquals(offset, metadata.offset()) assertEquals(topic, metadata.topic()) - if (timestampType == TimestampType.CreateTime) + if (timestampType == TimestampType.CREATE_TIME) assertEquals(baseTimestamp + timestampDiff, metadata.timestamp()) else assertTrue(metadata.timestamp() >= startTime && metadata.timestamp() <= System.currentTimeMillis()) @@ -202,7 +203,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { try { // create topic val topicProps = new Properties() - if (timestampType == TimestampType.LogAppendTime) + if (timestampType == TimestampType.LOG_APPEND_TIME) topicProps.setProperty(LogConfig.MessageTimestampTypeProp, "LogAppendTime") else topicProps.setProperty(LogConfig.MessageTimestampTypeProp, "CreateTime") diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index b969c26febab9..3d7cad38677c5 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -699,14 +699,14 @@ class PlaintextConsumerTest extends BaseConsumerTest { 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.LogAppendTime) + 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.LogAppendTime) + startingTimestamp = startTime, timestampType = TimestampType.LOG_APPEND_TIME) } def runMultiConsumerSessionTimeoutTest(closeConsumer: Boolean): Unit = { diff --git a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala index c25a28111dd7d..77f5d65f82a71 100644 --- a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala @@ -22,6 +22,7 @@ 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 @@ -164,7 +165,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { sourceCodec = NoCompressionCodec, targetCodec = NoCompressionCodec, messageFormatVersion = 1, - messageTimestampType = LogAppendTime, + messageTimestampType = TimestampType.LOG_APPEND_TIME, messageTimestampDiffMaxMs = 1000L) val validatedCompressedMessages = @@ -172,7 +173,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { sourceCodec = DefaultCompressionCodec, targetCodec = DefaultCompressionCodec, messageFormatVersion = 1, - messageTimestampType = LogAppendTime, + messageTimestampType = TimestampType.LOG_APPEND_TIME, messageTimestampDiffMaxMs = 1000L) val validatedCompressedMessagesWithoutRecompression = @@ -180,7 +181,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { sourceCodec = DefaultCompressionCodec, targetCodec = DefaultCompressionCodec, messageFormatVersion = 1, - messageTimestampType = LogAppendTime, + messageTimestampType = TimestampType.LOG_APPEND_TIME, messageTimestampDiffMaxMs = 1000L) val now = System.currentTimeMillis() @@ -201,7 +202,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { message.ensureValid() assertTrue(s"Timestamp of message $message should be between $startTime and $now", message.timestamp >= startTime && message.timestamp <= now) - assertEquals(LogAppendTime, message.timestampType) + assertEquals(TimestampType.LOG_APPEND_TIME, message.timestampType) } } @@ -215,7 +216,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { sourceCodec = NoCompressionCodec, targetCodec = NoCompressionCodec, messageFormatVersion = 1, - messageTimestampType = CreateTime, + messageTimestampType = TimestampType.CREATE_TIME, messageTimestampDiffMaxMs = 1000L) val validatedCompressedMessages = @@ -223,18 +224,18 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { sourceCodec = DefaultCompressionCodec, targetCodec = DefaultCompressionCodec, messageFormatVersion = 1, - messageTimestampType = CreateTime, + messageTimestampType = TimestampType.CREATE_TIME, messageTimestampDiffMaxMs = 1000L) for (messageAndOffset <- validatedMessages) { messageAndOffset.message.ensureValid() assertEquals(messageAndOffset.message.timestamp, now) - assertEquals(messageAndOffset.message.timestampType, CreateTime) + assertEquals(messageAndOffset.message.timestampType, TimestampType.CREATE_TIME) } for (messageAndOffset <- validatedCompressedMessages) { messageAndOffset.message.ensureValid() assertEquals(messageAndOffset.message.timestamp, now) - assertEquals(messageAndOffset.message.timestampType, CreateTime) + assertEquals(messageAndOffset.message.timestampType, TimestampType.CREATE_TIME) } } @@ -249,7 +250,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { sourceCodec = NoCompressionCodec, targetCodec = NoCompressionCodec, messageFormatVersion = 1, - messageTimestampType = CreateTime, + messageTimestampType = TimestampType.CREATE_TIME, messageTimestampDiffMaxMs = 1000L) fail("Should throw InvalidMessageException.") } catch { @@ -261,7 +262,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { sourceCodec = DefaultCompressionCodec, targetCodec = DefaultCompressionCodec, messageFormatVersion = 1, - messageTimestampType = CreateTime, + messageTimestampType = TimestampType.CREATE_TIME, messageTimestampDiffMaxMs = 1000L) fail("Should throw InvalidMessageException.") } catch { @@ -280,7 +281,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { sourceCodec = NoCompressionCodec, targetCodec = NoCompressionCodec, messageFormatVersion = 0, - messageTimestampType = CreateTime, + messageTimestampType = TimestampType.CREATE_TIME, messageTimestampDiffMaxMs = 1000L), offset) // check compressed messages @@ -289,7 +290,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { sourceCodec = DefaultCompressionCodec, targetCodec = DefaultCompressionCodec, messageFormatVersion = 0, - messageTimestampType = CreateTime, + messageTimestampType = TimestampType.CREATE_TIME, messageTimestampDiffMaxMs = 1000L), offset) } @@ -306,7 +307,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { val messageWithOffset = messages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(offset), sourceCodec = NoCompressionCodec, targetCodec = NoCompressionCodec, - messageTimestampType = CreateTime, + messageTimestampType = TimestampType.CREATE_TIME, messageTimestampDiffMaxMs = 5000L) checkOffsets(messageWithOffset, offset) @@ -315,7 +316,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { val compressedMessagesWithOffset = compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new AtomicLong(offset), sourceCodec = DefaultCompressionCodec, targetCodec = DefaultCompressionCodec, - messageTimestampType = CreateTime, + messageTimestampType = TimestampType.CREATE_TIME, messageTimestampDiffMaxMs = 5000L) checkOffsets(compressedMessagesWithOffset, offset) } @@ -332,7 +333,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { sourceCodec = NoCompressionCodec, targetCodec = NoCompressionCodec, messageFormatVersion = 1, - messageTimestampType = LogAppendTime, + messageTimestampType = TimestampType.LOG_APPEND_TIME, messageTimestampDiffMaxMs = 1000L), offset) // check compressed messages @@ -341,7 +342,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { sourceCodec = DefaultCompressionCodec, targetCodec = DefaultCompressionCodec, messageFormatVersion = 1, - messageTimestampType = LogAppendTime, + messageTimestampType = TimestampType.LOG_APPEND_TIME, messageTimestampDiffMaxMs = 1000L), offset) // Check down conversion @@ -355,7 +356,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { sourceCodec = NoCompressionCodec, targetCodec = NoCompressionCodec, messageFormatVersion = 0, - messageTimestampType = CreateTime, + messageTimestampType = TimestampType.CREATE_TIME, messageTimestampDiffMaxMs = 5000L), offset) // check compressed messages @@ -364,7 +365,7 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { sourceCodec = DefaultCompressionCodec, targetCodec = DefaultCompressionCodec, messageFormatVersion = 0, - messageTimestampType = CreateTime, + messageTimestampType = TimestampType.CREATE_TIME, messageTimestampDiffMaxMs = 5000L), offset) } diff --git a/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala b/core/src/test/scala/unit/kafka/message/MessageWriterTest.scala index 8c0862ec16164..6f0ee1d57b728 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, timestamp = Message.NoTimestamp, timestampType = CreateTime, magicValue = Message.MagicValue_V1) { 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) diff --git a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala index 0bc81991ad567..12b3583d01e5a 100755 --- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala @@ -25,12 +25,13 @@ import kafka.admin.AdminUtils import kafka.api.FetchRequestBuilder import kafka.common.FailedToSendMessageException import kafka.consumer.SimpleConsumer -import kafka.message.{CreateTime, Message} +import kafka.message.Message import kafka.serializer.StringEncoder 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} @@ -187,14 +188,14 @@ class ProducerTest extends ZooKeeperTestHarness with Logging{ 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(CreateTime, messageSet(0).message.timestampType) + 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(CreateTime, messageSet(1).message.timestampType) + assertEquals(TimestampType.CREATE_TIME, messageSet(1).message.timestampType) assertEquals(Message.MagicValue_V1, messageSet(1).message.magic) producer1.close() @@ -277,7 +278,7 @@ class ProducerTest extends ZooKeeperTestHarness with Logging{ assertTrue(ByteBuffer.wrap("test1".getBytes).equals(message.payload)) assertTrue(ByteBuffer.wrap("test".getBytes).equals(message.key)) assertTrue(message.timestamp >= startTime && message.timestamp < endTime) - assertEquals(CreateTime, message.timestampType) + assertEquals(TimestampType.CREATE_TIME, message.timestampType) assertEquals(Message.MagicValue_V1, message.magic) assertFalse("Message set should have another message", messageSet1.hasNext) } catch { 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 79ff5ad7eb568..078720412f4be 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 @@ -78,7 +78,7 @@ public int addRawRecords(Iterable> rawRecords, Ti Object value = source.deserializeValue(rawRecord.topic(), rawRecord.value()); ConsumerRecord record = new ConsumerRecord<>(rawRecord.topic(), rawRecord.partition(), - rawRecord.offset(), rawRecord.timestamp(), TimestampType.CreateTime, key, value); + rawRecord.offset(), rawRecord.timestamp(), TimestampType.CREATE_TIME, key, value); long timestamp = timestampExtractor.extract(record); StampedRecord stampedRecord = new StampedRecord(record, timestamp); 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 02e394fc3057e..9d0c0e22e75c6 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 @@ -60,17 +60,17 @@ public void testTimeTracking() { // add three 3 records with timestamp 1, 3, 5 to partition-1 List> list1 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 3, 0L, TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 5, 0L, TimestampType.CreateTime, 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> list2 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 2, 0L, TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 4, 0L, TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 6, 0L, TimestampType.CreateTime, 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); 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 d131fbdb67067..916079dcae57a 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 @@ -85,7 +85,7 @@ public void reset() { public void bufferRecord(ConsumerRecord record) { recordBuffer.add( new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), 0L, - TimestampType.CreateTime, + TimestampType.CREATE_TIME, serializer.serialize(record.topic(), record.key()), serializer.serialize(record.topic(), record.value()))); endOffset = record.offset(); @@ -269,7 +269,7 @@ public void testRegisterPersistentStore() throws IOException { int key = i * 10; expectedKeys.add(key); restoreConsumer.bufferRecord( - new ConsumerRecord<>(persistentStoreTopicName, 2, 0L, offset, TimestampType.CreateTime, key, 0) + new ConsumerRecord<>(persistentStoreTopicName, 2, 0L, offset, TimestampType.CREATE_TIME, key, 0) ); } @@ -322,7 +322,7 @@ public void testRegisterNonPersistentStore() throws IOException { int key = i; expectedKeys.add(i); restoreConsumer.bufferRecord( - new ConsumerRecord<>(nonPersistentStoreTopicName, 2, 0L, offset, TimestampType.CreateTime, key, 0) + new ConsumerRecord<>(nonPersistentStoreTopicName, 2, 0L, offset, TimestampType.CREATE_TIME, key, 0) ); } 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 605c08872ddb0..614e2c7d90812 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 @@ -51,9 +51,9 @@ public void testTimeTracking() { // add three 3 out-of-order records with timestamp 2, 1, 3 List> list1 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 2, 0L, TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 3, 0L, TimestampType.CreateTime, 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); @@ -73,9 +73,9 @@ public void testTimeTracking() { // add three 3 out-of-order records with timestamp 4, 1, 2 // now with 3, 4, 1, 2 List> list2 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 4, 0L, TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 2, 0L, TimestampType.CreateTime, 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); @@ -100,9 +100,9 @@ public void testTimeTracking() { // add three more records with 4, 5, 6 List> list3 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 4, 0L, TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 5, 0L, TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 6, 0L, TimestampType.CreateTime, 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); 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 a2d69cc4cbb95..e0be58757db1f 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 @@ -153,7 +153,7 @@ public void testUpdateNonPersistentStore() throws Exception { restoreStateConsumer.assign(new ArrayList<>(task.changeLogPartitions())); task.update(partition1, - records(new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, 0L, TimestampType.CreateTime, recordKey, recordValue)) + records(new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, 0L, TimestampType.CREATE_TIME, recordKey, recordValue)) ); } finally { @@ -172,9 +172,9 @@ public void testUpdate() throws Exception { restoreStateConsumer.assign(new ArrayList<>(task.changeLogPartitions())); for (ConsumerRecord record : Arrays.asList( - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 10, 0L, TimestampType.CreateTime, 1, 100), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 20, 0L, TimestampType.CreateTime, 2, 100), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 30, 0L, TimestampType.CreateTime, 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); } @@ -235,11 +235,11 @@ public void testUpdateKTable() throws Exception { restoreStateConsumer.assign(new ArrayList<>(task.changeLogPartitions())); for (ConsumerRecord record : Arrays.asList( - new ConsumerRecord<>(ktable.topic(), ktable.partition(), 10, 0L, TimestampType.CreateTime, 1, 100), - new ConsumerRecord<>(ktable.topic(), ktable.partition(), 20, 0L, TimestampType.CreateTime, 2, 100), - new ConsumerRecord<>(ktable.topic(), ktable.partition(), 30, 0L, TimestampType.CreateTime, 3, 100), - new ConsumerRecord<>(ktable.topic(), ktable.partition(), 40, 0L, TimestampType.CreateTime, 4, 100), - new ConsumerRecord<>(ktable.topic(), ktable.partition(), 50, 0L, TimestampType.CreateTime, 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); } 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 ad2f5b96fc3fb..94f0ce3e31bcc 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 @@ -108,15 +108,15 @@ public void testProcessOrder() throws Exception { 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, 0L, TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, 0L, TimestampType.CreateTime, 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, 0L, TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, TimestampType.CreateTime, 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()); @@ -159,15 +159,15 @@ public void testPauseResume() throws Exception { 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, 0L, TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, TimestampType.CreateTime, 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, 0L, TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 55, 0L, TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 65, 0L, TimestampType.CreateTime, 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()); @@ -178,9 +178,9 @@ public void testPauseResume() throws Exception { assertTrue(consumer.paused().contains(partition2)); task.addRecords(partition1, records( - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, 0L, TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 40, 0L, TimestampType.CreateTime, recordKey, recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 50, 0L, TimestampType.CreateTime, 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()); 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 60cc5bc08df54..e414d8044af5e 100644 --- a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java @@ -201,7 +201,7 @@ public void process(String topicName, byte[] key, byte[] value) { } // Add the record ... long offset = offsetsByTopicPartition.get(tp).incrementAndGet(); - task.addRecords(tp, records(new ConsumerRecord(tp.topic(), tp.partition(), offset, 0L, TimestampType.CreateTime, key, value))); + task.addRecords(tp, records(new ConsumerRecord(tp.topic(), tp.partition(), offset, 0L, TimestampType.CREATE_TIME, key, value))); producer.clear(); // Process the record ... task.process(); From 085958e41ffce463be04a60f410c2d4435e015b2 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Tue, 16 Feb 2016 17:07:06 -0800 Subject: [PATCH 25/33] Added documentation for changing internal offset topic configuration after rolling upgrade. --- docs/upgrade.html | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/upgrade.html b/docs/upgrade.html index 0a4b45a464098..820f178e4f54c 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -29,6 +29,8 @@

      Upgrading from 0.8.x and 0.9.x to 0.10
    16. Upgrade the brokers. This can be done a broker at a time by simply bringing it down, updating the code, and restarting it.
    17. Once the entire cluster is upgraded, bump the protocol version by editing inter.broker.protocol.version and setting it to 0.10.0.0.
    18. Restart the brokers one by one for the new protocol version to take effect.
    19. +
    20. Set message.format.version=0.10.0 for topic "__consumer_offsets" using bin/kafka-topic.sh. + This configures internal offset topic to use message format v1. This is optional but recommended.
    21. Note: 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. From 9bd7f883bf5095bcb2b3003a233c36722ed24468 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Tue, 16 Feb 2016 22:11:47 -0800 Subject: [PATCH 26/33] rebased on trunk --- core/src/main/scala/kafka/server/ReplicaManager.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 8873638905154..2ca1e61a8c085 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -29,16 +29,18 @@ import kafka.log.{LogAppendInfo, LogManager} 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, +InvalidTimestampException} import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.errors._ import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests.{LeaderAndIsrRequest, StopReplicaRequest, UpdateMetadataRequest} +import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.utils.{Time => JTime} -import scala.collection.JavaConverters._ import scala.collection._ +import scala.collection.JavaConverters._ /* * Result metadata of a log append operation on the log From 095d88d6d0774c59513168d75ed5ce58cc0ec834 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 17 Feb 2016 10:27:36 +0000 Subject: [PATCH 27/33] Minor comment improvements --- .../kafka/clients/producer/KafkaProducer.java | 6 +++--- .../clients/producer/ProducerRecord.java | 8 +++---- .../clients/producer/RecordMetadata.java | 6 +++--- .../src/main/scala/kafka/api/ApiVersion.scala | 21 ++++++++++--------- 4 files changed, 21 insertions(+), 20 deletions(-) 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 caf953bb76aa0..a066512c9d51a 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 @@ -362,9 +362,9 @@ public Future send(ProducerRecord record) { * 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 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. + * 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. *

      * 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() 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 9110e6bc65b64..85b4d8ddf1c0c 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 @@ -24,11 +24,11 @@ * record with its current time. The timestamp eventually used by Kafka depends on the timestamp type configured for * the topic. *

    22. - * 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 broker. + * 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. *
    23. *
    24. - * If the topic is configured to use {@link org.apache.kafka.common.record.TimestampType#LOG_APPEND_TIME LogAppendTime} + * 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. *
    25. @@ -45,7 +45,7 @@ public final class ProducerRecord { private final Long timestamp; /** - * Creates a record to be sent to a specified topic and partition with a specified timestamp + * 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 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 917f99b1d6166..d9ea239b3d5e5 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 @@ -25,9 +25,9 @@ public final class RecordMetadata { private final long offset; // The timestamp of the message. - // If LogAppendTime is used for the topic, the timestamp will be a timestamp returned by broker. - // If CreateTime is used for the topic, the timestamp is the timestamp in the corresponding ProducerRecord if - // user provided one, otherwise it will be the producer local time when the producer record was handed to the + // 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; diff --git a/core/src/main/scala/kafka/api/ApiVersion.scala b/core/src/main/scala/kafka/api/ApiVersion.scala index aa6dd56719f86..bec678f484c79 100644 --- a/core/src/main/scala/kafka/api/ApiVersion.scala +++ b/core/src/main/scala/kafka/api/ApiVersion.scala @@ -27,15 +27,16 @@ import kafka.message.Message * 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, to facilitate people deploying code from - * trunk, we introduce internal versions since 0.10.0. For example, the first time that we introduce a version change - * in 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. 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 0.10.0 release. + * 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 @@ -101,4 +102,4 @@ 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 -} \ No newline at end of file +} From 648a457382f2eaab6ba836ed3710e2c2281cac95 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 17 Feb 2016 10:29:10 +0000 Subject: [PATCH 28/33] Use `long` instead of `Long` for `offsetCounter` And move it to be along other `private` fields. --- .../apache/kafka/clients/producer/internals/RecordBatch.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 362f267b44ac1..af9095d76d742 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 @@ -35,7 +35,6 @@ public final class RecordBatch { public int recordCount = 0; public int maxRecordSize = 0; - private Long offsetCounter = 0L; public volatile int attempts = 0; public final long createdMs; public long drainedMs; @@ -45,6 +44,7 @@ public final class RecordBatch { public final ProduceRequestResult produceFuture; public long lastAppendTime; private final List thunks; + private long offsetCounter = 0L; private boolean retry; public RecordBatch(TopicPartition tp, MemoryRecords records, long now) { @@ -156,4 +156,4 @@ public boolean inRetry() { public void setRetry() { this.retry = true; } -} \ No newline at end of file +} From d0772a3f394677769c9dedc92d7eadb499c1fa1f Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 17 Feb 2016 10:32:33 +0000 Subject: [PATCH 29/33] Minor code style improvements --- .../org/apache/kafka/common/requests/ProduceResponse.java | 2 +- core/src/main/scala/kafka/consumer/BaseConsumer.scala | 4 ++-- .../main/scala/kafka/coordinator/GroupMetadataManager.scala | 6 ++++-- core/src/main/scala/kafka/server/ReplicaManager.scala | 2 +- core/src/main/scala/kafka/tools/MirrorMaker.scala | 4 ++-- 5 files changed, 10 insertions(+), 8 deletions(-) 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 ee2df59707f13..58175e1bf367f 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 @@ -74,7 +74,7 @@ public ProduceResponse(Map responses) { * @param throttleTime Time in milliseconds the response was throttled */ public ProduceResponse(Map responses, int throttleTime) { - this(responses, throttleTime, (int) ProtoUtils.latestVersion(ApiKeys.PRODUCE.id)); + this(responses, throttleTime, ProtoUtils.latestVersion(ApiKeys.PRODUCE.id)); } /** diff --git a/core/src/main/scala/kafka/consumer/BaseConsumer.scala b/core/src/main/scala/kafka/consumer/BaseConsumer.scala index 999a375a20b13..3774e735089ae 100644 --- a/core/src/main/scala/kafka/consumer/BaseConsumer.scala +++ b/core/src/main/scala/kafka/consumer/BaseConsumer.scala @@ -71,8 +71,8 @@ class NewShinyConsumer(topic: Option[String], whitelist: Option[String], consume BaseConsumerRecord(record.topic, record.partition, record.offset, - record.timestamp(), - TimestampType.forName(record.timestampType().name), + record.timestamp, + record.timestampType, record.key, record.value) } diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index 7c91db8e7d9b4..e994b3593f7e8 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -626,9 +626,11 @@ class GroupMetadataManager(val brokerId: Int, config.offsetsTopicNumPartitions } - private def getMessageFormatVersion(partition: Int) = { + private def getMessageFormatVersion(partition: Int): Byte = { val groupMetadataTopicAndPartition = new TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, partition) - replicaManager.getMessageFormatVersion(groupMetadataTopicAndPartition).get + replicaManager.getMessageFormatVersion(groupMetadataTopicAndPartition).getOrElse { + throw new IllegalArgumentException(s"Message format version for partition $groupMetadataTopicPartitionCount not found") + } } /** diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 2ca1e61a8c085..16b8c3aa69c1b 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -573,7 +573,7 @@ class ReplicaManager(val config: KafkaConfig, } } - def getMessageFormatVersion(topicAndPartition: TopicAndPartition) = { + def getMessageFormatVersion(topicAndPartition: TopicAndPartition): Option[Byte] = { getReplica(topicAndPartition.topic, topicAndPartition.partition).flatMap(_.log.map(_.config.messageFormatVersion)) } diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index cd5952efeb30e..95b0aad5344a3 100755 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -550,8 +550,8 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { BaseConsumerRecord(record.topic, record.partition, record.offset, - record.timestamp(), - record.timestampType(), + record.timestamp, + record.timestampType, record.key, record.value) } From a4313f4375c6e4bfbddbd94f91871fdf77121d01 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 17 Feb 2016 10:47:28 +0000 Subject: [PATCH 30/33] Use `Time` interface in `GroupCoordinator` and `GroupMetadataManager` --- .../kafka/coordinator/GroupCoordinator.scala | 36 +++++-------------- .../coordinator/GroupMetadataManager.scala | 21 +++++------ .../main/scala/kafka/server/KafkaServer.scala | 2 +- .../GroupCoordinatorResponseTest.scala | 3 +- 4 files changed, 23 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala index 2f1b8428bbe5b..c86e87ba484a9 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) - } } diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index e994b3593f7e8..b3e1bc1a0776f 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -30,6 +30,7 @@ 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._ @@ -48,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] @@ -143,7 +144,7 @@ class GroupMetadataManager(val brokerId: Int, // retry removing this group. val groupPartition = partitionFor(group.groupId) val tombstone = new Message(bytes = null, key = GroupMetadataManager.groupMetadataKey(group.groupId), - timestamp = SystemTime.milliseconds, magicValue = getMessageFormatVersion(groupPartition)) + timestamp = time.milliseconds(), magicValue = getMessageFormatVersion(groupPartition)) val partitionOpt = replicaManager.getPartition(GroupCoordinator.GroupMetadataTopicName, groupPartition) partitionOpt.foreach { partition => @@ -171,7 +172,7 @@ class GroupMetadataManager(val brokerId: Int, val message = new Message( key = GroupMetadataManager.groupMetadataKey(group.groupId), bytes = GroupMetadataManager.groupMetadataValue(group, groupAssignment), - timestamp = SystemTime.milliseconds, + timestamp = time.milliseconds(), magicValue = getMessageFormatVersion(partitionFor(group.groupId)) ) @@ -255,7 +256,7 @@ class GroupMetadataManager(val brokerId: Int, new Message( key = GroupMetadataManager.offsetCommitKey(groupId, topicAndPartition.topic, topicAndPartition.partition), bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata), - timestamp = SystemTime.milliseconds, + timestamp = time.milliseconds(), magicValue = getMessageFormatVersion(partitionFor(groupId)) ) }.toSeq @@ -363,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) => @@ -442,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) } @@ -537,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) => @@ -556,7 +557,7 @@ class GroupMetadataManager(val brokerId: Int, val commitKey = GroupMetadataManager.offsetCommitKey(groupTopicAndPartition.group, groupTopicAndPartition.topicPartition.topic, groupTopicAndPartition.topicPartition.partition) - (offsetsPartition, new Message(bytes = null, key = commitKey, timestamp = SystemTime.milliseconds, + (offsetsPartition, new Message(bytes = null, key = commitKey, timestamp = time.milliseconds(), magicValue = getMessageFormatVersion(offsetsPartition))) }.groupBy { case (partition, tombstone) => partition } @@ -586,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 = { diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 33217bd2294d6..e3e185f412e17 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -200,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.*/ diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala index 9282c4991a8ee..587abd5a49cdc 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala @@ -24,6 +24,7 @@ import kafka.common.{OffsetAndMetadata, TopicAndPartition} 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} @@ -88,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 From 200d69bc621b5f5e225f00ad0da4af859ba68aef Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Thu, 18 Feb 2016 09:46:43 -0800 Subject: [PATCH 31/33] Addressed Jun's comments --- .../scala/kafka/admin/ConfigCommand.scala | 18 +++--- .../main/scala/kafka/admin/TopicCommand.scala | 27 +++++---- .../src/main/scala/kafka/api/ApiVersion.scala | 1 - .../main/scala/kafka/log/FileMessageSet.scala | 2 +- .../src/main/scala/kafka/log/LogCleaner.scala | 2 +- core/src/main/scala/kafka/log/LogConfig.scala | 2 +- .../kafka/message/ByteBufferMessageSet.scala | 21 ++++--- .../main/scala/kafka/message/Message.scala | 8 --- .../main/scala/kafka/message/MessageSet.scala | 6 +- .../main/scala/kafka/server/KafkaApis.scala | 4 +- .../main/scala/kafka/server/KafkaConfig.scala | 13 +++-- .../scala/kafka/tools/ConsoleConsumer.scala | 6 +- .../unit/kafka/log/FileMessageSetTest.scala | 57 ++++++------------- .../unit/kafka/message/MessageTest.scala | 6 +- .../unit/kafka/server/KafkaConfigTest.scala | 2 +- docs/upgrade.html | 36 ++++++------ 16 files changed, 93 insertions(+), 118 deletions(-) diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index 82a6612dca479..5cc06abc80dda 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 it is not compatible with the inter broker protocol configuration in the Kafka cluster.") + } props } diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index c17b5bc4f7a29..0bab6f139d903 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 it is not compatible with the inter broker protocol configuration of the Kafka cluster.") + } props } diff --git a/core/src/main/scala/kafka/api/ApiVersion.scala b/core/src/main/scala/kafka/api/ApiVersion.scala index bec678f484c79..6b5fb7c6eb972 100644 --- a/core/src/main/scala/kafka/api/ApiVersion.scala +++ b/core/src/main/scala/kafka/api/ApiVersion.scala @@ -97,7 +97,6 @@ case object KAFKA_0_9_0 extends ApiVersion { val id: Int = 3 } -// This is a between-release protocol version case object KAFKA_0_10_0_IV0 extends ApiVersion { val version: String = "0.10.0-IV0" val messageFormatVersion: Byte = Message.MagicValue_V1 diff --git a/core/src/main/scala/kafka/log/FileMessageSet.scala b/core/src/main/scala/kafka/log/FileMessageSet.scala index b47a643275b58..fe31ad4cca46a 100755 --- a/core/src/main/scala/kafka/log/FileMessageSet.scala +++ b/core/src/main/scala/kafka/log/FileMessageSet.scala @@ -259,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)) diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 06b598842e26d..a3aff1559cfdc 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -483,7 +483,7 @@ private[log] class Cleaner(val id: Int, ByteBufferMessageSet.writeMessage(buffer, messageOffset.message, messageOffset.offset) MessageSet.messageSetSize(messagesIterable) } else { - val magicAndTimestamp = MessageSet.validateMagicValuesAndGetTimestamp(messages.map(_.message)) + val magicAndTimestamp = MessageSet.magicAndLargestTimestamp(messages.map(_.message)) val firstAbsoluteOffset = messages.head.offset var offset = -1L val timestampType = messages.head.message.timestampType diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index 2c5e578dd971b..a8fffbd26f657 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -146,7 +146,7 @@ object LogConfig { 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) diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index d822817e5f7b6..2867c78161e2d 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -50,7 +50,7 @@ object ByteBufferMessageSet { } else { val magicAndTimestamp = wrapperMessageTimestamp match { case Some(ts) => MagicAndTimestamp(messages.head.magic, ts) - case None => MessageSet.validateMagicValuesAndGetTimestamp(messages) + case None => MessageSet.magicAndLargestTimestamp(messages) } var offset = -1L val messageWriter = new MessageWriter(math.min(math.max(MessageSet.messageSetSize(messages) / 2, 1024), 1 << 16)) @@ -61,7 +61,7 @@ object ByteBufferMessageSet { offset = offsetAssignor.nextAbsoluteOffset if (message.magic != magicAndTimestamp.magic) throw new IllegalArgumentException("Messages in the message set must have same magic value") - // Use relative offset if magic value is greater than 0 + // Use inner offset if magic value is greater than 0 if (magicAndTimestamp.magic > Message.MagicValue_V0) output.writeLong(offsetAssignor.toInnerOffset(offset)) else @@ -122,7 +122,7 @@ object ByteBufferMessageSet { // read record size val size = compressed.readInt() - if (size < MinHeaderSize) + if (size < MinMessageOverhead) throw new InvalidMessageException("Message found with corrupt size (" + size + ") in deep iterator") // read the record into an intermediate record buffer @@ -144,7 +144,7 @@ object ByteBufferMessageSet { override def makeNext(): MessageAndOffset = { messageAndOffsets match { - // Using relative offset and timestamps + // Using inner offset and timestamps case Some(innerMessageAndOffsets) => if (innerMessageAndOffsets.isEmpty) allDone() @@ -155,7 +155,7 @@ object ByteBufferMessageSet { val absoluteOffset = wrapperMessageOffset + relativeOffset new MessageAndOffset(message, absoluteOffset) } - // Not using relative offset and timestamps + // Not using inner offset and timestamps case None => try { readMessageFromStream() @@ -217,7 +217,7 @@ private class OffsetAssigner(offsets: Seq[Long]) { * 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 is set and used. + * 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: @@ -231,12 +231,12 @@ private class OffsetAssigner(offsets: Seq[Long]) { * type to CreateTime, otherwise the messages will be rejected by broker. * * - * The way absolute offset calculated is following: + * 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 stream compressing way. + * 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 @@ -343,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 @@ -397,7 +397,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi * 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 avoids re-compression. + * operations and avoid re-compression. */ private[kafka] def validateMessagesAndAssignOffsets(offsetCounter: AtomicLong, now: Long = System.currentTimeMillis(), @@ -509,7 +509,6 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi timestampType: TimestampType, messageTimestampDiffMaxMs: Long, toMagicValue: Byte): ByteBufferMessageSet = { - // Get message count, shallow iterator is in-place val sizeInBytesAfterConversion = shallowValidBytes + this.internalIterator(isShallow = true).foldLeft(0)( (sizeDiff, messageAndOffset) => sizeDiff + Message.headerSizeDiff(messageAndOffset.message.magic, toMagicValue)) val newBuffer = ByteBuffer.allocate(sizeInBytesAfterConversion) diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala index 42b30674922c6..9ca880810f4a3 100755 --- a/core/src/main/scala/kafka/message/Message.scala +++ b/core/src/main/scala/kafka/message/Message.scala @@ -64,14 +64,6 @@ object Message { */ val MinMessageOverhead = KeyOffset_V0 + ValueSizeLength - /** - * The minimum valid size for the message header - * The MinHeaderSize does not include the TimestampLength for backward compatibility. - * 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 message format V0. - */ - val MinHeaderSize = MessageHeaderSizeMap.values.min - /** * The "magic" value * When magic value is 0, the message uses absolute offset and does not have a timestamp field. diff --git a/core/src/main/scala/kafka/message/MessageSet.scala b/core/src/main/scala/kafka/message/MessageSet.scala index c3169cd790a40..014788ab4b70e 100644 --- a/core/src/main/scala/kafka/message/MessageSet.scala +++ b/core/src/main/scala/kafka/message/MessageSet.scala @@ -55,10 +55,10 @@ 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 max timestamp - * of the inner messages. + * 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 validateMagicValuesAndGetTimestamp(messages: Seq[Message]): MagicAndTimestamp = { + def magicAndLargestTimestamp(messages: Seq[Message]): MagicAndTimestamp = { val firstMagicValue = messages.head.magic var largestTimestamp: Long = Message.NoTimestamp for (message <- messages) { diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 7cf16a8d74c54..bd02630ae2db3 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -442,7 +442,7 @@ class KafkaApis(val requestChannel: RequestChannel, // 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(_ > 0) && + 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)) @@ -484,7 +484,7 @@ class KafkaApis(val requestChannel: RequestChannel, quotaManagers(ApiKeys.FETCH.id).recordAndMaybeThrottle(fetchRequest.clientId, FetchResponse.responseSize(responsePartitionData.groupBy(_._1.topic), fetchRequest.versionId), - fetchResponseCallback) + fetchResponseCallback) } } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 719e9f7fa3085..34121a8c03568 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -19,7 +19,7 @@ package kafka.server import java.util.Properties -import kafka.api.{KAFKA_0_10_0_IV0, ApiVersion} +import kafka.api.ApiVersion import kafka.cluster.EndPoint import kafka.consumer.ConsumerConfig import kafka.coordinator.OffsetConfig @@ -424,12 +424,13 @@ object KafkaConfig { 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-IV0. 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." + "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 = "Set maximum allowed time difference between broker local time and message's timestamp. " + - "This configuration only works when message.timestamp.type=CreateTime. A message will be rejected if its timestamp exceeds this threshold." + 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" @@ -569,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) diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala index 8937740022c60..fe2ce9f9e987d 100755 --- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala @@ -378,9 +378,9 @@ class LoggingMessageFormatter extends MessageFormatter { 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)}, " + - s"value:${if (value == null) "null" else new String(value)}, " + - {if (timestampType != TimestampType.NO_TIMESTAMP_TYPE) s"$timestampType:$timestamp" else ""}) + 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)}") } } diff --git a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala index e1a112a0a6b35..731126c31f224 100644 --- a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala @@ -233,32 +233,14 @@ class FileMessageSetTest extends BaseMessageSetTestCases { fileMessageSet.append(messageSetV1) fileMessageSet.flush() var convertedMessageSet = fileMessageSet.toMessageFormat(Message.MagicValue_V0) - var i = 0 - for (messageAndOffset <- convertedMessageSet) { - val message = messageAndOffset.message - val offset = messageAndOffset.offset - assertEquals("magic byte should be 0", Message.MagicValue_V0, message.magic) - assertEquals("offset should not change", offsets(i), offset) - assertEquals("key should not change", messagesV0(i).key, message.key) - assertEquals("payload should not change", messagesV0(i).payload, message.payload) - i += 1 - } + verifyConvertedMessageSet(convertedMessageSet, Message.MagicValue_V0) + // down conversion for compressed messages fileMessageSet = new FileMessageSet(tempFile()) fileMessageSet.append(compressedMessageSetV1) fileMessageSet.flush() - // down conversion for compressed messages convertedMessageSet = fileMessageSet.toMessageFormat(Message.MagicValue_V0) - i = 0 - for (messageAndOffset <- convertedMessageSet) { - val message = messageAndOffset.message - val offset = messageAndOffset.offset - assertEquals("magic byte should be 0", Message.MagicValue_V0, message.magic) - assertEquals("offset should not change", offsets(i), offset) - assertEquals("key should not change", messagesV0(i).key, message.key) - assertEquals("payload should not change", messagesV0(i).payload, message.payload) - i += 1 - } + 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 @@ -266,31 +248,24 @@ class FileMessageSetTest extends BaseMessageSetTestCases { fileMessageSet.append(messageSetV0) fileMessageSet.flush() convertedMessageSet = fileMessageSet.toMessageFormat(Message.MagicValue_V1) - i = 0 - for (messageAndOffset <- convertedMessageSet) { - val message = messageAndOffset.message - val offset = messageAndOffset.offset - assertEquals("magic byte should be 1", Message.MagicValue_V1, message.magic) - assertEquals("offset should not change", offsets(i), offset) - assertEquals("key should not change", messagesV0(i).key, message.key) - assertEquals("payload should not change", messagesV0(i).payload, message.payload) - i += 1 - } + verifyConvertedMessageSet(convertedMessageSet, Message.MagicValue_V1) + // up conversion for compressed messages fileMessageSet = new FileMessageSet(tempFile()) fileMessageSet.append(compressedMessageSetV0) fileMessageSet.flush() - // up conversion for compressed messages convertedMessageSet = fileMessageSet.toMessageFormat(Message.MagicValue_V1) - i = 0 - for (messageAndOffset <- convertedMessageSet) { - val message = messageAndOffset.message - val offset = messageAndOffset.offset - assertEquals("magic byte should be 1", Message.MagicValue_V1, message.magic) - assertEquals("offset should not change", offsets(i), offset) - assertEquals("key should not change", messagesV0(i).key, message.key) - assertEquals("payload should not change", messagesV0(i).payload, message.payload) - i += 1 + 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 + } } } } diff --git a/core/src/test/scala/unit/kafka/message/MessageTest.scala b/core/src/test/scala/unit/kafka/message/MessageTest.scala index 7ab708c7bcd53..3c8a41fca7886 100755 --- a/core/src/test/scala/unit/kafka/message/MessageTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageTest.scala @@ -140,17 +140,17 @@ class MessageTest extends JUnitSuite { } @Test(expected = classOf[IllegalArgumentException]) - def testInValidTimestampAndMagicValueCombination() { + def testInvalidTimestampAndMagicValueCombination() { new Message("hello".getBytes, 0L, Message.MagicValue_V0) } @Test(expected = classOf[IllegalArgumentException]) - def testInValidTimestamp() { + def testInvalidTimestamp() { new Message("hello".getBytes, -3L, Message.MagicValue_V1) } @Test(expected = classOf[IllegalArgumentException]) - def testInValidMagicByte() { + def testInvalidMagicByte() { new Message("hello".getBytes, 0L, 2.toByte) } diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index ba6b7b510398f..836f2ec923f06 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -462,7 +462,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") diff --git a/docs/upgrade.html b/docs/upgrade.html index 820f178e4f54c..b3f24d0b402c1 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -17,7 +17,7 @@

      1.5 Upgrading From Previous Versions

      -

      Upgrading from 0.8.x and 0.9.x to 0.10.0.0

      +

      Upgrading from 0.8.x or 0.9.x to 0.10.0.0

      0.10.0.0 has potential performance impact during upgrade and potential breaking changes (please review before upgrading). Because new protocols are introduced, it is important to upgrade your Kafka clusters before upgrading your clients. @@ -25,12 +25,13 @@

      Upgrading from 0.8.x and 0.9.x to 0.10

      For a rolling upgrade:

        -
      1. Update server.properties file on all brokers and add the following properties: inter.broker.protocol.version=CURRENT_KAFKA_VERSION(e.g. 0.8.2, 0.9.0.0), message.format.version=CURRENT_KAFKA_VERSION(e.g. 0.8.2, 0.9.0.0)
      2. +
      3. Update server.properties file on all brokers and add the following properties: 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 potential performance impact during upgrade for the details. +
      4. Upgrade the brokers. This can be done a broker at a time by simply bringing it down, updating the code, and restarting it.
      5. Once the entire cluster is upgraded, bump the protocol version by editing inter.broker.protocol.version and setting it to 0.10.0.0.
      6. Restart the brokers one by one for the new protocol version to take effect.
      7. -
      8. Set message.format.version=0.10.0 for topic "__consumer_offsets" using bin/kafka-topic.sh. - This configures internal offset topic to use message format v1. This is optional but recommended.

      Note: 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. @@ -39,32 +40,31 @@

      Upgrading from 0.8.x and 0.9.x to 0.10
      potential performance impact in 0.10.0.0 during upgrade

      - Message format v1 is added to include a new timestamp field and use relative offsets for compressed messages in 0.10.0.0. + Message format 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 in 0.10.0 in v1. If a consumer client is on a version before 0.10.0.0, it only understands - message format v0. In this case, the broker is able to convert messages of format v1 to v0 before sending a response - to the consumer on an older version. However, the broker can't use zero-copy transfer in this case. + 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 format 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, one can set the message format to v0 after - upgrading the broker to 0.10.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 v1 on the broker. + 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.

      - 0.10.0.0 clients can handle both message format v0 and v1. Hence there is no performance impact for 0.10.0.0 clients - if message.format.version is set to v0 on broker. Broker will not convert the messages but just return message v0 - to 0.10.0.0 consumer. + For clients that are upgraded to 0.10.0.0, there is no performance impact.

      Note: 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 v1, one should not change it back to v0 since it may break the consumer on versions before 0.10.0. + 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.

      potential breaking changes in 0.10.0.0
        -
      • Message format v1 is added and used by default to include a timestamp field in the messages and use relative offsets for compressed messages.
      • -
      • ProduceRequest/Response v2 is added and used by default to support message format v1
      • -
      • FetchRequest/Response v2 is added and used by default to support message format v1
      • +
      • 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.
      • +
      • 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.
      • +
      • ProduceRequest/Response v2 is added and used by default to support message format 0.10.0
      • +
      • FetchRequest/Response v2 is added and used by default to support message format 0.10.0
      • MessageFormatter interface change: def writeTo(key: Array[Byte], value: Array[Byte], timestamp: Long, timestampType: TimestampType, output: PrintStream)
      From 99a4a278b9cdfe8599e3b09f46fa7b585c110ce5 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Thu, 18 Feb 2016 13:47:47 -0800 Subject: [PATCH 32/33] Addressed Ismael's comments. --- core/src/main/scala/kafka/message/Message.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala index 9ca880810f4a3..51aa11a9dd19d 100755 --- a/core/src/main/scala/kafka/message/Message.scala +++ b/core/src/main/scala/kafka/message/Message.scala @@ -42,14 +42,10 @@ object Message { // Only message format version 1 has the timestamp field. val TimestampOffset = AttributesOffset + AttributesLength val TimestampLength = 8 - // Message format version 0 does not have timestamp field. val KeySizeOffset_V0 = AttributesOffset + AttributesLength - // Message format version 1 has timestamp field. val KeySizeOffset_V1 = TimestampOffset + TimestampLength val KeySizeLength = 4 - // Message format version 0 does not have timestamp field. val KeyOffset_V0 = KeySizeOffset_V0 + KeySizeLength - // Message format version 1 has timestamp field. val KeyOffset_V1 = KeySizeOffset_V1 + KeySizeLength val ValueSizeLength = 4 From 22c74cd18b4865920dd2aac7da4c907b33932bb4 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Thu, 18 Feb 2016 19:26:44 -0800 Subject: [PATCH 33/33] Addressed Jun's comments --- .../apache/kafka/common/protocol/Protocol.java | 2 +- .../main/scala/kafka/admin/ConfigCommand.scala | 2 +- .../main/scala/kafka/admin/TopicCommand.scala | 2 +- .../main/scala/kafka/server/ConfigHandler.scala | 2 +- .../main/scala/kafka/server/KafkaConfig.scala | 2 +- .../unit/kafka/log/FileMessageSetTest.scala | 2 +- .../unit/kafka/server/KafkaConfigTest.scala | 10 ++++++---- docs/upgrade.html | 17 +++++++++-------- 8 files changed, 21 insertions(+), 18 deletions(-) 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 57a837daeceea..3787d2cecf23c 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 @@ -112,7 +112,7 @@ public class Protocol { INT64)))))))); /** * The body of PRODUCE_REQUEST_V1 is the same as PRODUCE_REQUEST_V0. - * The version number is bumped up to indicate the client support quota throttle time field in the response. + * 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; /** diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index 5cc06abc80dda..276689a0a736f 100644 --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -119,7 +119,7 @@ object ConfigCommand { 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 it is not compatible with the inter broker protocol configuration in the Kafka cluster.") + 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 } diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index 0bab6f139d903..d4212c551f549 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -238,7 +238,7 @@ object TopicCommand extends Logging { 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 it is not compatible with the inter broker protocol configuration of the Kafka cluster.") + 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 } diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala index f32675d1a08ab..9343fdebeedb4 100644 --- a/core/src/main/scala/kafka/server/ConfigHandler.scala +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -49,7 +49,7 @@ class TopicConfigHandler(private val logManager: LogManager, kafkaConfig: KafkaC // Validate the compatibility of message format version. Option(topicConfig.getProperty(LogConfig.MessageFormatVersionProp)) match { case Some(versionString) => - if (kafkaConfig.interBrokerProtocolVersion.messageFormatVersion < ApiVersion(versionString).messageFormatVersion) { + 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}") diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 34121a8c03568..ae810ebfed967 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -978,7 +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.messageFormatVersion >= ApiVersion(messageFormatVersion).messageFormatVersion, + require(interBrokerProtocolVersion.onOrAfter(ApiVersion(messageFormatVersion)), s"message.format.version $messageFormatVersion cannot be used when inter.broker.protocol.version is set to $interBrokerProtocolVersion") } } diff --git a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala index 731126c31f224..0179166be5502 100644 --- a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala @@ -217,7 +217,7 @@ class FileMessageSetTest extends BaseMessageSetTestCases { 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)) + new Message("goodbye".getBytes, "k2".getBytes, 2L, Message.MagicValue_V1)) val messageSetV1 = new ByteBufferMessageSet( compressionCodec = NoCompressionCodec, offsetSeq = offsets, diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 836f2ec923f06..7fe9ffc8be288 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -281,14 +281,16 @@ class KafkaConfigTest { val conf = KafkaConfig.fromProps(props) assertEquals(ApiVersion.latestVersion, conf.interBrokerProtocolVersion) - props.put(KafkaConfig.InterBrokerProtocolVersionProp,"0.8.2.0") - props.put(KafkaConfig.MessageFormatVersionProp, "0.9.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_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.MessageFormatVersionProp, "0.9.0") + 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_0_8_2, conf3.interBrokerProtocolVersion) diff --git a/docs/upgrade.html b/docs/upgrade.html index b3f24d0b402c1..f6d67eb3f99ef 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -25,7 +25,7 @@

      Upgrading from 0.8.x or 0.9.x to 0.10.

      For a rolling upgrade:

        -
      1. Update server.properties file on all brokers and add the following properties: inter.broker.protocol.version=CURRENT_KAFKA_VERSION(e.g. 0.8.2, 0.9.0.0). +
      2. 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 potential performance impact during upgrade for the details.
      3. @@ -40,10 +40,10 @@

        Upgrading from 0.8.x or 0.9.x to 0.10.
        potential performance impact in 0.10.0.0 during upgrade

        - Message format 0.10.0 now includes a new timestamp field and uses relative offsets for compressed messages. + 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 format 0.10.0 to earlier format + 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 @@ -61,11 +61,12 @@

        potential breaking changes in 0.10.0.0
          -
        • 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.
        • -
        • 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.
        • -
        • ProduceRequest/Response v2 is added and used by default to support message format 0.10.0
        • -
        • FetchRequest/Response v2 is added and used by default to support message format 0.10.0
        • -
        • MessageFormatter interface change: def writeTo(key: Array[Byte], value: Array[Byte], timestamp: Long, timestampType: TimestampType, output: PrintStream)
        • +
        • 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.
        • +
        • 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.
        • +
        • ProduceRequest/Response v2 is added and used by default to support message format 0.10.0
        • +
        • FetchRequest/Response v2 is added and used by default to support message format 0.10.0
        • +
        • MessageFormatter interface changed from void writeTo(byte[] key, byte[] value, PrintStream output) to + void writeTo(byte[] key, byte[] value, long timestamp, TimestampType timestampType, PrintStream output)

        Upgrading from 0.8.0, 0.8.1.X or 0.8.2.X to 0.9.0.0