From 6765ae099677618ba69eb8c83d7d877a2c14e524 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 29 Oct 2014 16:45:54 -0700 Subject: [PATCH 01/20] Protocol --- .../org/apache/kafka/common/protocol/Protocol.java | 185 ++++++++++++--------- .../kafka/common/requests/OffsetCommitRequest.java | 41 ++++- 2 files changed, 144 insertions(+), 82 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 101f382..45fedef 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 @@ -88,92 +88,119 @@ public class Protocol { /* Produce api */ - public static final Schema TOPIC_PRODUCE_DATA_V0 = new Schema(new Field("topic", STRING), - new Field("data", - new ArrayOf(new Schema(new Field("partition", - INT32), - new Field("record_set", - BYTES))))); - - public static final Schema PRODUCE_REQUEST_V0 = new Schema(new Field("acks", - INT16, - "The number of nodes that should replicate the produce before returning. -1 indicates the full ISR."), - new Field("timeout", - INT32, - "The time to await a response in ms."), - new Field("topic_data", - new ArrayOf(TOPIC_PRODUCE_DATA_V0))); - - public static final Schema PRODUCE_RESPONSE_V0 = new Schema(new Field("responses", - new ArrayOf(new Schema(new Field("topic", - STRING), - new Field("partition_responses", - new ArrayOf(new Schema(new Field("partition", - INT32), - new Field("error_code", - INT16), - new Field("base_offset", - INT64)))))))); - - public static final Schema[] PRODUCE_REQUEST = new Schema[] {PRODUCE_REQUEST_V0}; - public static final Schema[] PRODUCE_RESPONSE = new Schema[] {PRODUCE_RESPONSE_V0}; + public static Schema TOPIC_PRODUCE_DATA_V0 = new Schema(new Field("topic", STRING), + new Field("data", new ArrayOf(new Schema(new Field("partition", INT32), + new Field("record_set", BYTES))))); + + public static Schema PRODUCE_REQUEST_V0 = new Schema(new Field("acks", + INT16, + "The number of nodes that should replicate the produce before returning. -1 indicates the full ISR."), + new Field("timeout", INT32, "The time to await a response in ms."), + new Field("topic_data", new ArrayOf(TOPIC_PRODUCE_DATA_V0))); + + public static Schema PRODUCE_RESPONSE_V0 = new Schema(new Field("responses", + new ArrayOf(new Schema(new Field("topic", STRING), + new Field("partition_responses", + new ArrayOf(new Schema(new Field("partition", + INT32), + new Field("error_code", + INT16), + new Field("base_offset", + INT64)))))))); + + public static Schema[] PRODUCE_REQUEST = new Schema[] { PRODUCE_REQUEST_V0 }; + public static Schema[] PRODUCE_RESPONSE = new Schema[] { PRODUCE_RESPONSE_V0 }; /* Offset commit api */ - public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V0 = new Schema(new Field("partition", - INT32, - "Topic partition id."), - new Field("offset", - INT64, - "Message offset to be committed."), - new Field("timestamp", - INT64, - "Timestamp of the commit"), - new Field("metadata", - STRING, - "Any associated metadata the client wants to keep.")); - - public static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V0 = new Schema(new Field("topic", - STRING, - "Topic to commit."), - new Field("partitions", - new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V0), - "Partitions to commit offsets.")); + public static Schema OFFSET_COMMIT_REQUEST_PARTITION_V0 = new Schema(new Field("partition", + INT32, + "Topic partition id."), + new Field("offset", + INT64, + "Message offset to be committed."), + new Field("timestamp", + INT64, + "Timestamp of the commit"), + new Field("metadata", + STRING, + "Any associated metadata the client wants to keep.")); + + public static Schema OFFSET_COMMIT_REQUEST_PARTITION_V1 = new Schema(new Field("partition", + INT32, + "Topic partition id."), + new Field("offset", + INT64, + "Message offset to be committed."), + new Field("metadata", + STRING, + "Any associated metadata the client wants to keep.")); - public static final Schema OFFSET_COMMIT_REQUEST_V0 = new Schema(new Field("group_id", + public static Schema OFFSET_COMMIT_REQUEST_TOPIC_V0 = new Schema(new Field("topic", STRING, - "The consumer group id."), - new Field("topics", - new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0), - "Topics to commit offsets.")); + "Topic to commit."), + new Field("partitions", + new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V0), + "Partitions to commit offsets.")); - public static final Schema OFFSET_COMMIT_REQUEST_V1 = new Schema(new Field("group_id", - STRING, - "The consumer group id."), - new Field("group_generation_id", - INT32, - "The generation of the consumer group."), - new Field("consumer_id", + public static Schema OFFSET_COMMIT_REQUEST_TOPIC_V1 = new Schema(new Field("topic", STRING, - "The consumer id assigned by the group coordinator."), - new Field("topics", - new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0), - "Topics to commit offsets.")); - - public static final Schema OFFSET_COMMIT_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", - INT32, - "Topic partition id."), - new Field("error_code", INT16)); - - public static final Schema OFFSET_COMMIT_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), - new Field("partition_responses", - new ArrayOf(OFFSET_COMMIT_RESPONSE_PARTITION_V0))); - - public static final Schema OFFSET_COMMIT_RESPONSE_V0 = new Schema(new Field("responses", - new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0))); - - public static final Schema[] OFFSET_COMMIT_REQUEST = new Schema[] {OFFSET_COMMIT_REQUEST_V0, OFFSET_COMMIT_REQUEST_V1}; + "Topic to commit."), + new Field("partitions", + new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V1), + "Partitions to commit offsets.")); + + public static Schema OFFSET_COMMIT_REQUEST_V0 = new Schema(new Field("group_id", + STRING, + "The consumer group id."), + new Field("topics", + new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0), + "Topics to commit offsets.")); + + public static Schema OFFSET_COMMIT_REQUEST_V1 = new Schema(new Field("group_id", + STRING, + "The consumer group id."), + new Field("group_generation_id", + INT32, + "The generation of the consumer group."), + new Field("consumer_id", + STRING, + "The consumer id assigned by the group coordinator."), + new Field("topics", + new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0), + "Topics to commit offsets.")); + + public static Schema OFFSET_COMMIT_REQUEST_V2 = new Schema(new Field("group_id", + STRING, + "The consumer group id."), + new Field("group_generation_id", + INT32, + "The generation of the consumer group."), + new Field("consumer_id", + STRING, + "The consumer id assigned by the group coordinator."), + new Field("retention_time", + INT64, + "Time period in seconds to retain the offset."), + new Field("topics", + new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V1), + "Topics to commit offsets.")); + + public static Schema OFFSET_COMMIT_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", + INT32, + "Topic partition id."), + new Field("error_code", + INT16)); + + public static Schema OFFSET_COMMIT_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), + new Field("partition_responses", + new ArrayOf(OFFSET_COMMIT_RESPONSE_PARTITION_V0))); + + public static Schema OFFSET_COMMIT_RESPONSE_V0 = new Schema(new Field("responses", + new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0))); + + public static Schema[] OFFSET_COMMIT_REQUEST = new Schema[] { OFFSET_COMMIT_REQUEST_V0, OFFSET_COMMIT_REQUEST_V1 }; /* The response types for both V0 and V1 of OFFSET_COMMIT_REQUEST are the same. */ - public static final Schema[] OFFSET_COMMIT_RESPONSE = new Schema[] {OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V0}; + public static Schema[] OFFSET_COMMIT_RESPONSE = new Schema[] { OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V0}; /* Offset fetch api */ public static final Schema OFFSET_FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition", diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java index 94e9d37..8078d0e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java @@ -29,12 +29,12 @@ import org.apache.kafka.common.utils.CollectionUtils; * This wrapper supports both v0 and v1 of OffsetCommitRequest. */ public class OffsetCommitRequest extends AbstractRequestResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_COMMIT.id); private static final String GROUP_ID_KEY_NAME = "group_id"; private static final String GENERATION_ID_KEY_NAME = "group_generation_id"; private static final String CONSUMER_ID_KEY_NAME = "consumer_id"; private static final String TOPICS_KEY_NAME = "topics"; + private static final String RETENTION_KEY_NAME = "retention_time"; // topic level field names private static final String TOPIC_KEY_NAME = "topic"; @@ -48,10 +48,13 @@ public class OffsetCommitRequest extends AbstractRequestResponse { public static final int DEFAULT_GENERATION_ID = -1; public static final String DEFAULT_CONSUMER_ID = ""; + public static final long DEFAULT_TIMESTAMP = -1; + public static final long DEFAULT_RETENTION = -1; private final String groupId; private final int generationId; private final String consumerId; + private final long retentionTime; private final Map offsetData; public static final class PartitionData { @@ -78,6 +81,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { this.groupId = groupId; this.generationId = DEFAULT_GENERATION_ID; this.consumerId = DEFAULT_CONSUMER_ID; + this.retentionTime = DEFAULT_RETENTION; this.offsetData = offsetData; } @@ -97,6 +101,28 @@ public class OffsetCommitRequest extends AbstractRequestResponse { this.groupId = groupId; this.generationId = generationId; this.consumerId = consumerId; + this.retentionTime = DEFAULT_RETENTION; + this.offsetData = offsetData; + } + + /** + * Constructor for version 2. + * @param groupId + * @param generationId + * @param consumerId + * @param offsetData + */ + public OffsetCommitRequest(String groupId, int generationId, String consumerId, long retentionTime, Map offsetData) { + super(new Struct(CURRENT_SCHEMA)); + + initCommonFields(groupId, offsetData); + struct.set(GENERATION_ID_KEY_NAME, generationId); + struct.set(CONSUMER_ID_KEY_NAME, consumerId); + struct.set(RETENTION_KEY_NAME, retentionTime); + this.groupId = groupId; + this.generationId = generationId; + this.consumerId = consumerId; + this.retentionTime = retentionTime; this.offsetData = offsetData; } @@ -114,7 +140,8 @@ public class OffsetCommitRequest extends AbstractRequestResponse { Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset); - partitionData.set(TIMESTAMP_KEY_NAME, fetchPartitionData.timestamp); + if (fetchPartitionData.timestamp != DEFAULT_TIMESTAMP) + partitionData.set(TIMESTAMP_KEY_NAME, fetchPartitionData.timestamp); partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata); partitionArray.add(partitionData); } @@ -134,7 +161,9 @@ public class OffsetCommitRequest extends AbstractRequestResponse { Struct partitionResponse = (Struct) partitionResponseObj; int partition = partitionResponse.getInt(PARTITION_KEY_NAME); long offset = partitionResponse.getLong(COMMIT_OFFSET_KEY_NAME); - long timestamp = partitionResponse.getLong(TIMESTAMP_KEY_NAME); + long timestamp = DEFAULT_TIMESTAMP; + if (partitionResponse.hasField(TIMESTAMP_KEY_NAME)) + timestamp = partitionResponse.getLong(TIMESTAMP_KEY_NAME); String metadata = partitionResponse.getString(METADATA_KEY_NAME); PartitionData partitionData = new PartitionData(offset, timestamp, metadata); offsetData.put(new TopicPartition(topic, partition), partitionData); @@ -152,6 +181,12 @@ public class OffsetCommitRequest extends AbstractRequestResponse { consumerId = struct.getString(CONSUMER_ID_KEY_NAME); else consumerId = DEFAULT_CONSUMER_ID; + + // This field only exists in v2 + if (struct.hasField(RETENTION_KEY_NAME)) + retentionTime = struct.getLong(RETENTION_KEY_NAME); + else + retentionTime = DEFAULT_RETENTION; } public String groupId() { -- 1.7.12.4 From 0eeba3cd0baa70ec1e21512fd37a5c61ea183c1e Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 30 Oct 2014 11:37:25 -0700 Subject: [PATCH 02/20] v1 --- .../org/apache/kafka/common/protocol/Protocol.java | 2 +- .../kafka/common/requests/OffsetCommitRequest.java | 49 ++++++++++---------- .../main/scala/kafka/api/OffsetCommitRequest.scala | 53 +++++++++++++++------- .../kafka/common/OffsetMetadataAndError.scala | 2 +- core/src/main/scala/kafka/server/KafkaApis.scala | 1 + .../main/scala/kafka/server/OffsetManager.scala | 15 +++++- .../api/RequestResponseSerializationTest.scala | 19 +++++++- 7 files changed, 96 insertions(+), 45 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 45fedef..622a9eb 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 @@ -180,7 +180,7 @@ public class Protocol { "The consumer id assigned by the group coordinator."), new Field("retention_time", INT64, - "Time period in seconds to retain the offset."), + "Time period in ms to retain the offset."), new Field("topics", new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V1), "Topics to commit offsets.")); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java index 8078d0e..762ff51 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java @@ -34,7 +34,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { private static final String GENERATION_ID_KEY_NAME = "group_generation_id"; private static final String CONSUMER_ID_KEY_NAME = "consumer_id"; private static final String TOPICS_KEY_NAME = "topics"; - private static final String RETENTION_KEY_NAME = "retention_time"; + private static final String RETENTION_TIME_KEY_NAME = "retention_time"; // topic level field names private static final String TOPIC_KEY_NAME = "topic"; @@ -49,7 +49,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { public static final int DEFAULT_GENERATION_ID = -1; public static final String DEFAULT_CONSUMER_ID = ""; public static final long DEFAULT_TIMESTAMP = -1; - public static final long DEFAULT_RETENTION = -1; + public static final long DEFAULT_RETENTION_TIME = -1; private final String groupId; private final int generationId; @@ -81,7 +81,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { this.groupId = groupId; this.generationId = DEFAULT_GENERATION_ID; this.consumerId = DEFAULT_CONSUMER_ID; - this.retentionTime = DEFAULT_RETENTION; + this.retentionTime = DEFAULT_RETENTION_TIME; this.offsetData = offsetData; } @@ -92,6 +92,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { * @param consumerId * @param offsetData */ + @Deprecated public OffsetCommitRequest(String groupId, int generationId, String consumerId, Map offsetData) { super(new Struct(CURRENT_SCHEMA)); @@ -101,7 +102,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { this.groupId = groupId; this.generationId = generationId; this.consumerId = consumerId; - this.retentionTime = DEFAULT_RETENTION; + this.retentionTime = DEFAULT_RETENTION_TIME; this.offsetData = offsetData; } @@ -118,7 +119,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { initCommonFields(groupId, offsetData); struct.set(GENERATION_ID_KEY_NAME, generationId); struct.set(CONSUMER_ID_KEY_NAME, consumerId); - struct.set(RETENTION_KEY_NAME, retentionTime); + struct.set(RETENTION_TIME_KEY_NAME, retentionTime); this.groupId = groupId; this.generationId = generationId; this.consumerId = consumerId; @@ -153,6 +154,26 @@ public class OffsetCommitRequest extends AbstractRequestResponse { public OffsetCommitRequest(Struct struct) { super(struct); + + groupId = struct.getString(GROUP_ID_KEY_NAME); + // This field only exists in v1. + if (struct.hasField(GENERATION_ID_KEY_NAME)) + generationId = struct.getInt(GENERATION_ID_KEY_NAME); + else + generationId = DEFAULT_GENERATION_ID; + + // This field only exists in v1. + if (struct.hasField(CONSUMER_ID_KEY_NAME)) + consumerId = struct.getString(CONSUMER_ID_KEY_NAME); + else + consumerId = DEFAULT_CONSUMER_ID; + + // This field only exists in v2 + if (struct.hasField(RETENTION_TIME_KEY_NAME)) + retentionTime = struct.getLong(RETENTION_TIME_KEY_NAME); + else + retentionTime = DEFAULT_RETENTION_TIME; + offsetData = new HashMap(); for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) { Struct topicResponse = (Struct) topicResponseObj; @@ -169,24 +190,6 @@ public class OffsetCommitRequest extends AbstractRequestResponse { offsetData.put(new TopicPartition(topic, partition), partitionData); } } - groupId = struct.getString(GROUP_ID_KEY_NAME); - // This field only exists in v1. - if (struct.hasField(GENERATION_ID_KEY_NAME)) - generationId = struct.getInt(GENERATION_ID_KEY_NAME); - else - generationId = DEFAULT_GENERATION_ID; - - // This field only exists in v1. - if (struct.hasField(CONSUMER_ID_KEY_NAME)) - consumerId = struct.getString(CONSUMER_ID_KEY_NAME); - else - consumerId = DEFAULT_CONSUMER_ID; - - // This field only exists in v2 - if (struct.hasField(RETENTION_KEY_NAME)) - retentionTime = struct.getLong(RETENTION_KEY_NAME); - else - retentionTime = DEFAULT_RETENTION; } public String groupId() { diff --git a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala index 050615c..cfc5dde 100644 --- a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala @@ -26,7 +26,7 @@ import kafka.network.RequestChannel.Response import scala.collection._ object OffsetCommitRequest extends Logging { - val CurrentVersion: Short = 1 + val CurrentVersion: Short = 2 val DefaultClientId = "" def readFrom(buffer: ByteBuffer): OffsetCommitRequest = { @@ -34,8 +34,8 @@ object OffsetCommitRequest extends Logging { // Read values from the envelope val versionId = buffer.getShort - assert(versionId == 0 || versionId == 1, - "Version " + versionId + " is invalid for OffsetCommitRequest. Valid versions are 0 or 1.") + assert(versionId == 0 || versionId == 1 || versionId == 2, + "Version " + versionId + " is invalid for OffsetCommitRequest. Valid versions are 0, 1 or 2.") val correlationId = buffer.getInt val clientId = readShortString(buffer) @@ -43,14 +43,20 @@ object OffsetCommitRequest extends Logging { // Read the OffsetRequest val consumerGroupId = readShortString(buffer) - // version 1 specific fields + // version 1 and 2 specific fields var groupGenerationId: Int = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_GENERATION_ID var consumerId: String = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_CONSUMER_ID - if (versionId == 1) { + if (versionId >= 1) { groupGenerationId = buffer.getInt consumerId = readShortString(buffer) } + // version 2 specific fields + var retentionMs: Long = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME + if (versionId == 2) { + retentionMs = buffer.getLong + } + val topicCount = buffer.getInt val pairs = (1 to topicCount).flatMap(_ => { val topic = readShortString(buffer) @@ -59,14 +65,18 @@ object OffsetCommitRequest extends Logging { val partitionId = buffer.getInt val offset = buffer.getLong val timestamp = { - val given = buffer.getLong - if (given == -1L) now else given + if (versionId <= 1) { + val given = buffer.getLong + if (given == -1L) now else given + } + else + -1L } val metadata = readShortString(buffer) (TopicAndPartition(topic, partitionId), OffsetAndMetadata(offset, metadata, timestamp)) }) }) - OffsetCommitRequest(consumerGroupId, immutable.Map(pairs:_*), versionId, correlationId, clientId, groupGenerationId, consumerId) + OffsetCommitRequest(consumerGroupId, immutable.Map(pairs:_*), versionId, correlationId, clientId, groupGenerationId, consumerId, retentionMs) } } @@ -76,11 +86,12 @@ case class OffsetCommitRequest(groupId: String, correlationId: Int = 0, clientId: String = OffsetCommitRequest.DefaultClientId, groupGenerationId: Int = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_GENERATION_ID, - consumerId: String = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_CONSUMER_ID) + consumerId: String = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_CONSUMER_ID, + retentionMs: Long = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME) extends RequestOrResponse(Some(RequestKeys.OffsetCommitKey)) { - assert(versionId == 0 || versionId == 1, - "Version " + versionId + " is invalid for OffsetCommitRequest. Valid versions are 0 or 1.") + assert(versionId == 0 || versionId == 1 || versionId == 2, + "Version " + versionId + " is invalid for OffsetCommitRequest. Valid versions are 0, 1 or 2.") lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic) @@ -93,11 +104,17 @@ case class OffsetCommitRequest(groupId: String, // Write OffsetCommitRequest writeShortString(buffer, groupId) // consumer group - // version 1 specific data - if (versionId == 1) { + // version 1 and 2 specific data + if (versionId >= 1) { buffer.putInt(groupGenerationId) writeShortString(buffer, consumerId) } + + // version 2 specific data + if (versionId == 2) { + buffer.putLong(retentionMs) + } + buffer.putInt(requestInfoGroupedByTopic.size) // number of topics requestInfoGroupedByTopic.foreach( t1 => { // topic -> Map[TopicAndPartition, OffsetMetadataAndError] writeShortString(buffer, t1._1) // topic @@ -105,7 +122,9 @@ case class OffsetCommitRequest(groupId: String, t1._2.foreach( t2 => { buffer.putInt(t2._1.partition) buffer.putLong(t2._2.offset) - buffer.putLong(t2._2.timestamp) + // version 0 and 1 specific data + if (versionId <= 1) + buffer.putLong(t2._2.timestamp) writeShortString(buffer, t2._2.metadata) }) }) @@ -116,7 +135,8 @@ case class OffsetCommitRequest(groupId: String, 4 + /* correlationId */ shortStringLength(clientId) + shortStringLength(groupId) + - (if (versionId == 1) 4 /* group generation id */ + shortStringLength(consumerId) else 0) + + (if (versionId >= 1) 4 /* group generation id */ + shortStringLength(consumerId) else 0) + + (if (versionId == 2) 8 /* retention time */ else 0) + 4 + /* topic count */ requestInfoGroupedByTopic.foldLeft(0)((count, topicAndOffsets) => { val (topic, offsets) = topicAndOffsets @@ -127,7 +147,7 @@ case class OffsetCommitRequest(groupId: String, innerCount + 4 /* partition */ + 8 /* offset */ + - 8 /* timestamp */ + + (if (versionId <= 1) 8 else 0) /* timestamp */ + shortStringLength(offsetAndMetadata._2.metadata) }) }) @@ -149,6 +169,7 @@ case class OffsetCommitRequest(groupId: String, offsetCommitRequest.append("; GroupId: " + groupId) offsetCommitRequest.append("; GroupGenerationId: " + groupGenerationId) offsetCommitRequest.append("; ConsumerId: " + consumerId) + offsetCommitRequest.append("; RetentionMs: " + retentionMs) if(details) offsetCommitRequest.append("; RequestInfo: " + requestInfo.mkString(",")) offsetCommitRequest.toString() diff --git a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala index 1584a92..f2c3dbe 100644 --- a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala +++ b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala @@ -19,7 +19,7 @@ package kafka.common case class OffsetAndMetadata(offset: Long, metadata: String = OffsetAndMetadata.NoMetadata, - timestamp: Long = -1L) { + var timestamp: Long = -1L) { override def toString = "OffsetAndMetadata[%d,%s%s]" .format(offset, if (metadata != null && metadata.length > 0) metadata else "NO_METADATA", diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 35af98f..8fed969 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -154,6 +154,7 @@ class KafkaApis(val requestChannel: RequestChannel, offsetCommitRequest.groupId, offsetCommitRequest.consumerId, offsetCommitRequest.groupGenerationId, + offsetCommitRequest.retentionMs, offsetCommitRequest.requestInfo, sendResponseCallback) } diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index d05e14d..f6ce6f2 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -214,6 +214,7 @@ class OffsetManager(val config: OffsetManagerConfig, def storeOffsets(groupId: String, consumerId: String, generationId: Int, + retentionMs: Long, offsetMetadata: immutable.Map[TopicAndPartition, OffsetAndMetadata], responseCallback: immutable.Map[TopicAndPartition, Short] => Unit) { // check if there are any non-existent topics @@ -229,10 +230,16 @@ class OffsetManager(val config: OffsetManagerConfig, } // construct the message set to append + val timestamp = + if (retentionMs == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME) + org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME + else + retentionMs + SystemTime.milliseconds + val messages = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => new Message( key = OffsetManager.offsetCommitKey(groupId, topicAndPartition.topic, topicAndPartition.partition), - bytes = OffsetManager.offsetCommitValue(offsetAndMetadata) + bytes = OffsetManager.offsetCommitValue(offsetAndMetadata, timestamp) ) }.toSeq @@ -496,12 +503,16 @@ object OffsetManager { * Generates the payload for offset commit message from given offset and metadata * * @param offsetAndMetadata consumer's current offset and metadata + * @param timestamp global timestamp of the offset * @return payload for offset commit message */ - def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata): Array[Byte] = { + def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata, timestamp: Long): Array[Byte] = { val value = new Struct(CURRENT_SCHEMA.valueSchema) value.set(VALUE_OFFSET_FIELD, offsetAndMetadata.offset) value.set(VALUE_METADATA_FIELD, offsetAndMetadata.metadata) + // maybe override the timestamp with the global timestamp + if (timestamp != org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME) + offsetAndMetadata.timestamp = timestamp value.set(VALUE_TIMESTAMP_FIELD, offsetAndMetadata.timestamp) val byteBuffer = ByteBuffer.allocate(2 /* version */ + value.sizeOf) diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index fba852a..17beda0 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -151,8 +151,21 @@ object SerializationTestUtils { new TopicMetadataResponse(brokers, Seq(topicmetaData1, topicmetaData2), 1) } + def createTestOffsetCommitRequestV2: OffsetCommitRequest = { + new OffsetCommitRequest( + groupId = "group 1", + retentionMs = SystemTime.milliseconds, + requestInfo=collection.immutable.Map( + TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L, metadata="some metadata"), + TopicAndPartition(topic1, 1) -> OffsetAndMetadata(offset=100L, metadata=OffsetAndMetadata.NoMetadata) + )) + } + def createTestOffsetCommitRequestV1: OffsetCommitRequest = { - new OffsetCommitRequest("group 1", collection.immutable.Map( + new OffsetCommitRequest( + versionId = 1, + groupId = "group 1", + requestInfo = collection.immutable.Map( TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L, metadata="some metadata", timestamp=SystemTime.milliseconds), TopicAndPartition(topic1, 1) -> OffsetAndMetadata(offset=100L, metadata=OffsetAndMetadata.NoMetadata, timestamp=SystemTime.milliseconds) )) @@ -232,6 +245,7 @@ class RequestResponseSerializationTest extends JUnitSuite { private val topicMetadataResponse = SerializationTestUtils.createTestTopicMetadataResponse private val offsetCommitRequestV0 = SerializationTestUtils.createTestOffsetCommitRequestV0 private val offsetCommitRequestV1 = SerializationTestUtils.createTestOffsetCommitRequestV1 + private val offsetCommitRequestV2 = SerializationTestUtils.createTestOffsetCommitRequestV2 private val offsetCommitResponse = SerializationTestUtils.createTestOffsetCommitResponse private val offsetFetchRequest = SerializationTestUtils.createTestOffsetFetchRequest private val offsetFetchResponse = SerializationTestUtils.createTestOffsetFetchResponse @@ -250,7 +264,8 @@ class RequestResponseSerializationTest extends JUnitSuite { collection.immutable.Seq(leaderAndIsrRequest, leaderAndIsrResponse, stopReplicaRequest, stopReplicaResponse, producerRequest, producerResponse, fetchRequest, offsetRequest, offsetResponse, topicMetadataRequest, - topicMetadataResponse, offsetCommitRequestV0, offsetCommitRequestV1, + topicMetadataResponse, + offsetCommitRequestV0, offsetCommitRequestV1, offsetCommitRequestV2, offsetCommitResponse, offsetFetchRequest, offsetFetchResponse, consumerMetadataRequest, consumerMetadataResponse, consumerMetadataResponseNoCoordinator, heartbeatRequest, -- 1.7.12.4 From 7057be78a66655d73c7743594989d6614d0a3209 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 30 Oct 2014 11:40:53 -0700 Subject: [PATCH 03/20] last pass --- core/src/main/scala/kafka/server/OffsetManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index f6ce6f2..8fafa81 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -510,7 +510,7 @@ object OffsetManager { val value = new Struct(CURRENT_SCHEMA.valueSchema) value.set(VALUE_OFFSET_FIELD, offsetAndMetadata.offset) value.set(VALUE_METADATA_FIELD, offsetAndMetadata.metadata) - // maybe override the timestamp with the global timestamp + // maybe override the timestamp with the global value if (timestamp != org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME) offsetAndMetadata.timestamp = timestamp value.set(VALUE_TIMESTAMP_FIELD, offsetAndMetadata.timestamp) -- 1.7.12.4 From 061425ba0d446227a19f7565840e72c6225bae2d Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 30 Oct 2014 11:42:54 -0700 Subject: [PATCH 04/20] minor TODO --- core/src/main/scala/kafka/common/OffsetMetadataAndError.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala index f2c3dbe..a2cfeb0 100644 --- a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala +++ b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala @@ -20,6 +20,8 @@ package kafka.common case class OffsetAndMetadata(offset: Long, metadata: String = OffsetAndMetadata.NoMetadata, var timestamp: Long = -1L) { + // TODO: timestamp needs to be removed after we deprecate OffsetCommitRequest version 0 and 1 + override def toString = "OffsetAndMetadata[%d,%s%s]" .format(offset, if (metadata != null && metadata.length > 0) metadata else "NO_METADATA", -- 1.7.12.4 From 74f0e7c012e7de86139716e343f1e5234b10d70b Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 6 Nov 2014 15:35:07 -0800 Subject: [PATCH 05/20] Incorporate Joel's comments --- .../org/apache/kafka/common/protocol/Protocol.java | 6 +- .../kafka/common/requests/OffsetCommitRequest.java | 21 ++++-- .../kafka/common/requests/RequestResponseTest.java | 4 +- .../main/scala/kafka/api/OffsetCommitRequest.scala | 34 ++++++---- core/src/main/scala/kafka/server/KafkaApis.scala | 13 +++- core/src/main/scala/kafka/server/KafkaServer.scala | 1 + .../main/scala/kafka/server/OffsetManager.scala | 26 ++++---- .../scala/unit/kafka/server/OffsetCommitTest.scala | 74 +++++++++++++++++++++- 8 files changed, 136 insertions(+), 43 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 622a9eb..401e914 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 @@ -198,9 +198,9 @@ public class Protocol { public static Schema OFFSET_COMMIT_RESPONSE_V0 = new Schema(new Field("responses", new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0))); - public static Schema[] OFFSET_COMMIT_REQUEST = new Schema[] { OFFSET_COMMIT_REQUEST_V0, OFFSET_COMMIT_REQUEST_V1 }; - /* The response types for both V0 and V1 of OFFSET_COMMIT_REQUEST are the same. */ - public static Schema[] OFFSET_COMMIT_RESPONSE = new Schema[] { OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V0}; + public static Schema[] OFFSET_COMMIT_REQUEST = new Schema[] { OFFSET_COMMIT_REQUEST_V0, OFFSET_COMMIT_REQUEST_V1, OFFSET_COMMIT_REQUEST_V2 }; + /* The response types for both V0, V1 and V2 of OFFSET_COMMIT_REQUEST are the same. */ + public static Schema[] OFFSET_COMMIT_RESPONSE = new Schema[] { OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V0 }; /* Offset fetch api */ public static final Schema OFFSET_FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition", diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java index 762ff51..bd6dc42 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java @@ -49,7 +49,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { public static final int DEFAULT_GENERATION_ID = -1; public static final String DEFAULT_CONSUMER_ID = ""; public static final long DEFAULT_TIMESTAMP = -1; - public static final long DEFAULT_RETENTION_TIME = -1; + public static final long DEFAULT_RETENTION_TIME = Long.MAX_VALUE; private final String groupId; private final int generationId; @@ -62,11 +62,18 @@ public class OffsetCommitRequest extends AbstractRequestResponse { public final long timestamp; public final String metadata; + @Deprecated public PartitionData(long offset, long timestamp, String metadata) { this.offset = offset; this.timestamp = timestamp; this.metadata = metadata; } + + public PartitionData(long offset, String metadata) { + this.offset = offset; + this.timestamp = DEFAULT_TIMESTAMP; + this.metadata = metadata; + } } /** @@ -111,6 +118,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { * @param groupId * @param generationId * @param consumerId + * @param retentionTime * @param offsetData */ public OffsetCommitRequest(String groupId, int generationId, String consumerId, long retentionTime, Map offsetData) { @@ -182,11 +190,14 @@ public class OffsetCommitRequest extends AbstractRequestResponse { Struct partitionResponse = (Struct) partitionResponseObj; int partition = partitionResponse.getInt(PARTITION_KEY_NAME); long offset = partitionResponse.getLong(COMMIT_OFFSET_KEY_NAME); - long timestamp = DEFAULT_TIMESTAMP; - if (partitionResponse.hasField(TIMESTAMP_KEY_NAME)) - timestamp = partitionResponse.getLong(TIMESTAMP_KEY_NAME); String metadata = partitionResponse.getString(METADATA_KEY_NAME); - PartitionData partitionData = new PartitionData(offset, timestamp, metadata); + PartitionData partitionData; + if (partitionResponse.hasField(TIMESTAMP_KEY_NAME)) { + long timestamp = partitionResponse.getLong(TIMESTAMP_KEY_NAME); + partitionData = new PartitionData(offset, timestamp, metadata); + } else { + partitionData = new PartitionData(offset, metadata); + } offsetData.put(new TopicPartition(topic, partition), partitionData); } } 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 13237fd..61a767a 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 @@ -140,8 +140,8 @@ public class RequestResponseTest { private AbstractRequestResponse createOffsetCommitRequest() { Map commitData = new HashMap(); - commitData.put(new TopicPartition("test", 0), new OffsetCommitRequest.PartitionData(100, 1000000, "")); - return new OffsetCommitRequest("group1", 100, "consumer1", commitData); + commitData.put(new TopicPartition("test", 0), new OffsetCommitRequest.PartitionData(100, "")); + return new OffsetCommitRequest("group1", 100, "consumer1", 1000000, commitData); } private AbstractRequestResponse createOffsetCommitResponse() { diff --git a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala index cfc5dde..c1e59bf 100644 --- a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala @@ -44,18 +44,26 @@ object OffsetCommitRequest extends Logging { val consumerGroupId = readShortString(buffer) // version 1 and 2 specific fields - var groupGenerationId: Int = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_GENERATION_ID - var consumerId: String = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_CONSUMER_ID - if (versionId >= 1) { - groupGenerationId = buffer.getInt - consumerId = readShortString(buffer) - } + val groupGenerationId: Int = + if (versionId >= 1) { + buffer.getInt + } else { + org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_GENERATION_ID + } + val consumerId: String = + if (versionId >= 1) { + readShortString(buffer) + } else { + org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_CONSUMER_ID + } // version 2 specific fields - var retentionMs: Long = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME - if (versionId == 2) { - retentionMs = buffer.getLong - } + val retentionMs: Long = + if (versionId == 2) { + buffer.getLong + } else { + org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME + } val topicCount = buffer.getInt val pairs = (1 to topicCount).flatMap(_ => { @@ -65,10 +73,8 @@ object OffsetCommitRequest extends Logging { val partitionId = buffer.getInt val offset = buffer.getLong val timestamp = { - if (versionId <= 1) { - val given = buffer.getLong - if (given == -1L) now else given - } + if (versionId <= 1) + buffer.getLong else -1L } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 8fed969..558194e 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -149,12 +149,23 @@ class KafkaApis(val requestChannel: RequestChannel, val response = OffsetCommitResponse(commitStatus, offsetCommitRequest.correlationId) requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } + + // compute the retention time based on the request version: + // if it is before v2, we can use the default retention period config + val offsetRetention = + if (offsetCommitRequest.versionId <= 1) { + offsetManager.config.offsetsRetentionMs + } else { + offsetCommitRequest.retentionMs + } + + // call offset manager to store offsets offsetManager.storeOffsets( offsetCommitRequest.groupId, offsetCommitRequest.consumerId, offsetCommitRequest.groupGenerationId, - offsetCommitRequest.retentionMs, + offsetRetention, offsetCommitRequest.requestInfo, sendResponseCallback) } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index dddef93..4db3384 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -402,6 +402,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg maxMetadataSize = config.offsetMetadataMaxSize, loadBufferSize = config.offsetsLoadBufferSize, offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L, + offsetsRetentionCheckIntervalMs = config.offsetsRetentionCheckIntervalMs, offsetsTopicNumPartitions = config.offsetsTopicPartitions, offsetsTopicReplicationFactor = config.offsetsTopicReplicationFactor, offsetCommitTimeoutMs = config.offsetCommitTimeoutMs, diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 8fafa81..34edfca 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -120,9 +120,9 @@ class OffsetManager(val config: OffsetManagerConfig, debug("Compacting offsets cache.") val startMs = SystemTime.milliseconds - val staleOffsets = offsetsCache.filter(startMs - _._2.timestamp > config.offsetsRetentionMs) + val staleOffsets = offsetsCache.filter(_._2.timestamp < startMs) - debug("Found %d stale offsets (older than %d ms).".format(staleOffsets.size, config.offsetsRetentionMs)) + debug("Found %d stale offsets (expiration timestamp passed already).".format(staleOffsets.size)) // delete the stale offsets from the table and generate tombstone messages to remove them from the log val tombstonesForPartition = staleOffsets.map { case(groupTopicAndPartition, offsetAndMetadata) => @@ -229,17 +229,14 @@ class OffsetManager(val config: OffsetManagerConfig, validateOffsetMetadataLength(offsetAndMetadata.metadata) || nonExistentTopics.contains(topicAndPartition) } - // construct the message set to append - val timestamp = - if (retentionMs == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME) - org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME - else - retentionMs + SystemTime.milliseconds + // compute the global expiration timestamp + val expirationTimestamp = retentionMs + SystemTime.milliseconds + // construct the message set to append val messages = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => new Message( key = OffsetManager.offsetCommitKey(groupId, topicAndPartition.topic, topicAndPartition.partition), - bytes = OffsetManager.offsetCommitValue(offsetAndMetadata, timestamp) + bytes = OffsetManager.offsetCommitValue(offsetAndMetadata, expirationTimestamp) ) }.toSeq @@ -503,18 +500,17 @@ object OffsetManager { * Generates the payload for offset commit message from given offset and metadata * * @param offsetAndMetadata consumer's current offset and metadata - * @param timestamp global timestamp of the offset + * @param expirationTimestamp global expiration time of the offset * @return payload for offset commit message */ - def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata, timestamp: Long): Array[Byte] = { + def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata, expirationTimestamp: Long): Array[Byte] = { val value = new Struct(CURRENT_SCHEMA.valueSchema) value.set(VALUE_OFFSET_FIELD, offsetAndMetadata.offset) value.set(VALUE_METADATA_FIELD, offsetAndMetadata.metadata) - // maybe override the timestamp with the global value - if (timestamp != org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME) - offsetAndMetadata.timestamp = timestamp + // if the expiration timestamp is not specified, used the global value + if (offsetAndMetadata.timestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) + offsetAndMetadata.timestamp = expirationTimestamp value.set(VALUE_TIMESTAMP_FIELD, offsetAndMetadata.timestamp) - val byteBuffer = ByteBuffer.allocate(2 /* version */ + value.sizeOf) byteBuffer.putShort(CURRENT_OFFSET_SCHEMA_VERSION) value.writeTo(byteBuffer) diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index e4d0435..4d7a5dd 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -33,12 +33,13 @@ import scala.collection._ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { val random: Random = new Random() + val brokerPort: Int = 9099 + val group = "test-group" + val retentionCheckInterval: Long = 100L var logDir: File = null var topicLogDir: File = null var server: KafkaServer = null var logSize: Int = 100 - val brokerPort: Int = 9099 - val group = "test-group" var simpleConsumer: SimpleConsumer = null var time: Time = new MockTime() @@ -68,7 +69,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { Utils.rm(logDir) super.tearDown() } - + /* @Test def testUpdateOffsets() { val topic = "topic" @@ -204,6 +205,73 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { val commitResponse1 = simpleConsumer.commitOffsets(commitRequest1) assertEquals(ErrorMapping.OffsetMetadataTooLargeCode, commitResponse1.commitStatus.get(topicAndPartition).get) + } + */ + @Test + def testOffsetExpiration() { + // set up topic partition + val topic = "topic" + val topicPartition = TopicAndPartition(topic, 0) + createTopic(zkClient, topic, servers = Seq(server), numPartitions = 1) + + val fetchRequest = OffsetFetchRequest(group, Seq(TopicAndPartition(topic, 0))) + + // v0 version commit request with timestamp set to -1 + // should not expire + val commitRequest0 = OffsetCommitRequest( + groupId =group, + requestInfo = immutable.Map(topicPartition -> OffsetAndMetadata(1L, "metadata", -1)), + versionId = 0 + ) + assertEquals(ErrorMapping.NoError, simpleConsumer.commitOffsets(commitRequest0).commitStatus.get(topicPartition).get) + Thread.sleep(retentionCheckInterval * 2) + assertEquals(1L, simpleConsumer.fetchOffsets(fetchRequest).requestInfo.get(topicPartition).get.offset) + + // v1 version commit request with timestamp set to -1 + // should not expire + val commitRequest1 = OffsetCommitRequest( + groupId = group, + requestInfo = immutable.Map(topicPartition -> OffsetAndMetadata(2L, "metadata", -1)), + versionId = 1 + ) + assertEquals(ErrorMapping.NoError, simpleConsumer.commitOffsets(commitRequest1).commitStatus.get(topicPartition).get) + Thread.sleep(retentionCheckInterval * 2) + assertEquals(2L, simpleConsumer.fetchOffsets(fetchRequest).requestInfo.get(topicPartition).get.offset) + + // v1 version commit request with timestamp set to now + // should expire + val commitRequest2 = OffsetCommitRequest( + groupId = group, + requestInfo = immutable.Map(topicPartition -> OffsetAndMetadata(3L, "metadata", SystemTime.milliseconds)), + versionId = 1 + ) + assertEquals(ErrorMapping.NoError, simpleConsumer.commitOffsets(commitRequest2).commitStatus.get(topicPartition).get) + Thread.sleep(retentionCheckInterval * 2) + assertEquals(-1L, simpleConsumer.fetchOffsets(fetchRequest).requestInfo.get(topicPartition).get.offset) + + // v2 version commit request with retention time set to 1 hour + // should not expire + val commitRequest3 = OffsetCommitRequest( + groupId = group, + requestInfo = immutable.Map(topicPartition -> OffsetAndMetadata(4L, "metadata", -1)), + versionId = 2, + retentionMs = 1000 * 60 * 60 + ) + assertEquals(ErrorMapping.NoError, simpleConsumer.commitOffsets(commitRequest3).commitStatus.get(topicPartition).get) + Thread.sleep(retentionCheckInterval * 2) + assertEquals(4L, simpleConsumer.fetchOffsets(fetchRequest).requestInfo.get(topicPartition).get.offset) + + // v2 version commit request with retention time set to 0 second + // should expire + val commitRequest4 = OffsetCommitRequest( + groupId = "test-group", + requestInfo = immutable.Map(TopicAndPartition(topic, 0) -> OffsetAndMetadata(5L, "metadata", -1)), + versionId = 2, + retentionMs = 0 + ) + assertEquals(ErrorMapping.NoError, simpleConsumer.commitOffsets(commitRequest4).commitStatus.get(topicPartition).get) + Thread.sleep(retentionCheckInterval * 2) + assertEquals(-1L, simpleConsumer.fetchOffsets(fetchRequest).requestInfo.get(topicPartition).get.offset) } -- 1.7.12.4 From af9f63a13a9aa77cdedd88edd15a86371b680d9a Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 7 Nov 2014 16:54:23 -0800 Subject: [PATCH 06/20] incorporate Joel's comments round two --- .../kafka/common/requests/OffsetCommitRequest.java | 10 ++-- .../main/scala/kafka/api/OffsetCommitRequest.scala | 2 +- .../main/scala/kafka/api/OffsetFetchRequest.scala | 9 ++-- .../kafka/common/OffsetMetadataAndError.scala | 63 +++++++++++++--------- .../consumer/ZookeeperConsumerConnector.scala | 2 +- .../main/scala/kafka/server/OffsetManager.scala | 8 +-- 6 files changed, 59 insertions(+), 35 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java index bd6dc42..a1aa98c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java @@ -59,6 +59,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { public static final class PartitionData { public final long offset; + @Deprecated public final long timestamp; public final String metadata; @@ -70,9 +71,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { } public PartitionData(long offset, String metadata) { - this.offset = offset; - this.timestamp = DEFAULT_TIMESTAMP; - this.metadata = metadata; + this(offset, DEFAULT_TIMESTAMP, metadata); } } @@ -192,6 +191,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { long offset = partitionResponse.getLong(COMMIT_OFFSET_KEY_NAME); String metadata = partitionResponse.getString(METADATA_KEY_NAME); PartitionData partitionData; + // This field only exists in v0 and v1 if (partitionResponse.hasField(TIMESTAMP_KEY_NAME)) { long timestamp = partitionResponse.getLong(TIMESTAMP_KEY_NAME); partitionData = new PartitionData(offset, timestamp, metadata); @@ -215,6 +215,10 @@ public class OffsetCommitRequest extends AbstractRequestResponse { return consumerId; } + public long retentionTime() { + return retentionTime; + } + public Map offsetData() { return offsetData; } diff --git a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala index c1e59bf..875ccb8 100644 --- a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import kafka.api.ApiUtils._ import kafka.utils.{SystemTime, Logging} import kafka.network.{RequestChannel, BoundedByteBufferSend} -import kafka.common.{OffsetAndMetadata, ErrorMapping, TopicAndPartition} +import kafka.common.{OffsetMetadata, OffsetAndMetadata, ErrorMapping, TopicAndPartition} import kafka.network.RequestChannel.Response import scala.collection._ diff --git a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala index c7604b9..d696284 100644 --- a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala @@ -23,7 +23,11 @@ import kafka.api.ApiUtils._ import kafka.utils.Logging import kafka.network.{BoundedByteBufferSend, RequestChannel} import kafka.network.RequestChannel.Response -import kafka.common.{OffsetAndMetadata, ErrorMapping, OffsetMetadataAndError, TopicAndPartition} +import kafka.common._ +import kafka.common.TopicAndPartition +import kafka.network.RequestChannel.Response +import scala.Some + object OffsetFetchRequest extends Logging { val CurrentVersion: Short = 0 val DefaultClientId = "" @@ -91,8 +95,7 @@ case class OffsetFetchRequest(groupId: String, override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { val responseMap = requestInfo.map { case (topicAndPartition) => (topicAndPartition, OffsetMetadataAndError( - offset = OffsetAndMetadata.InvalidOffset, - error = ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) + ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) )) }.toMap val errorResponse = OffsetFetchResponse(requestInfo=responseMap, correlationId=correlationId) diff --git a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala index a2cfeb0..b42482b 100644 --- a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala +++ b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala @@ -17,42 +17,57 @@ package kafka.common -case class OffsetAndMetadata(offset: Long, - metadata: String = OffsetAndMetadata.NoMetadata, - var timestamp: Long = -1L) { - // TODO: timestamp needs to be removed after we deprecate OffsetCommitRequest version 0 and 1 - - override def toString = "OffsetAndMetadata[%d,%s%s]" - .format(offset, - if (metadata != null && metadata.length > 0) metadata else "NO_METADATA", - if (timestamp == -1) "" else "," + timestamp.toString) +case class OffsetMetadata(offset: Long, + metadata: String = OffsetMetadata.NoMetadata) { + override def toString = "OffsetMetadata[%d,%s]" + .format(offset, + if (metadata != null && metadata.length > 0) metadata else "NO_METADATA") } -object OffsetAndMetadata { +object OffsetMetadata { val InvalidOffset: Long = -1L val NoMetadata: String = "" - val InvalidTime: Long = -1L + + val InvalidOffsetMetadata = OffsetMetadata(OffsetMetadata.InvalidOffset, OffsetMetadata.NoMetadata) } -case class OffsetMetadataAndError(offset: Long, - metadata: String = OffsetAndMetadata.NoMetadata, - error: Short = ErrorMapping.NoError) { +// TODO: the following class will be replaced by OffsetMetadata after we deprecate OffsetCommitRequest version 0 and 1 +case class OffsetAndMetadata(offsetMetadata: OffsetMetadata, + timestamp: Long = -1) { + def offset() = offsetMetadata.offset - def this(offsetMetadata: OffsetAndMetadata, error: Short) = - this(offsetMetadata.offset, offsetMetadata.metadata, error) + def metadata() = offsetMetadata.metadata - def this(error: Short) = - this(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, error) + override def toString = "[%s,ExpirationTime %d]".format(offsetMetadata, timestamp) +} - def asTuple = (offset, metadata, error) +object OffsetAndMetadata { + def apply(offset: Long, metadata: String, timestamp: Long) = new OffsetAndMetadata(OffsetMetadata(offset, metadata), timestamp) - override def toString = "OffsetMetadataAndError[%d,%s,%d]".format(offset, metadata, error) + def apply(offset: Long) = new OffsetAndMetadata(OffsetMetadata(offset, OffsetMetadata.NoMetadata)) +} + +case class OffsetMetadataAndError(offsetMetadata: OffsetMetadata, + error: Short = ErrorMapping.NoError) { + def offset() = offsetMetadata.offset + + def metadata() = offsetMetadata.metadata + + override def toString = "[%s,ErrorCode %d]".format(offsetMetadata, error) } object OffsetMetadataAndError { - val NoOffset = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.NoOffsetsCommittedCode) - val OffsetsLoading = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.OffsetsLoadInProgressCode) - val NotOffsetManagerForGroup = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.NotCoordinatorForConsumerCode) - val UnknownTopicOrPartition = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.UnknownTopicOrPartitionCode) + val NoOffset = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, ErrorMapping.NoError) + val OffsetsLoading = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, ErrorMapping.OffsetsLoadInProgressCode) + val UnknownTopicOrPartition = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, ErrorMapping.UnknownTopicOrPartitionCode) + val NotOffsetManagerForGroup = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, ErrorMapping.NotCoordinatorForConsumerCode) + + def apply(offset: Long) = new OffsetMetadataAndError(OffsetMetadata(offset, OffsetMetadata.NoMetadata), ErrorMapping.NoError) + + def apply(error: Short) = new OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, error) + + def apply(offset: Long, metadata: String, error: Short) = new OffsetMetadataAndError(OffsetMetadata(offset, metadata), error) } + + diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index cca815a..b1cf0db 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -400,7 +400,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, val dirs = new ZKGroupTopicDirs(config.groupId, topicPartition.topic) val offsetString = readDataMaybeNull(zkClient, dirs.consumerOffsetDir + "/" + topicPartition.partition)._1 offsetString match { - case Some(offsetStr) => (topicPartition, OffsetMetadataAndError(offsetStr.toLong, OffsetAndMetadata.NoMetadata, ErrorMapping.NoError)) + case Some(offsetStr) => (topicPartition, OffsetMetadataAndError(offsetStr.toLong)) case None => (topicPartition, OffsetMetadataAndError.NoOffset) } } diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 34edfca..51106eb 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -120,9 +120,11 @@ class OffsetManager(val config: OffsetManagerConfig, debug("Compacting offsets cache.") val startMs = SystemTime.milliseconds - val staleOffsets = offsetsCache.filter(_._2.timestamp < startMs) + val staleOffsets = offsetsCache.filter { case (groupTopicPartition, offsetAndMetadata) => + offsetAndMetadata.timestamp < startMs + } - debug("Found %d stale offsets (expiration timestamp passed already).".format(staleOffsets.size)) + debug("Found %d expired offsets.".format(staleOffsets.size)) // delete the stale offsets from the table and generate tombstone messages to remove them from the log val tombstonesForPartition = staleOffsets.map { case(groupTopicAndPartition, offsetAndMetadata) => @@ -509,7 +511,7 @@ object OffsetManager { value.set(VALUE_METADATA_FIELD, offsetAndMetadata.metadata) // if the expiration timestamp is not specified, used the global value if (offsetAndMetadata.timestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) - offsetAndMetadata.timestamp = expirationTimestamp + offsetAndMetadata.copy(timestamp = expirationTimestamp) value.set(VALUE_TIMESTAMP_FIELD, offsetAndMetadata.timestamp) val byteBuffer = ByteBuffer.allocate(2 /* version */ + value.sizeOf) byteBuffer.putShort(CURRENT_OFFSET_SCHEMA_VERSION) -- 1.7.12.4 From 6aac93a809e4c618ab1a46f79a3a2491b3b4766f Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 17 Nov 2014 17:41:19 -0800 Subject: [PATCH 07/20] Joel's comment round three --- .../kafka/common/requests/OffsetCommitRequest.java | 22 +++++++++++++++------- .../main/scala/kafka/api/OffsetCommitRequest.scala | 6 ++---- .../kafka/common/OffsetMetadataAndError.scala | 4 ++-- 3 files changed, 19 insertions(+), 13 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java index a1aa98c..6499232 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java @@ -43,24 +43,32 @@ public class OffsetCommitRequest extends AbstractRequestResponse { // partition level field names private static final String PARTITION_KEY_NAME = "partition"; private static final String COMMIT_OFFSET_KEY_NAME = "offset"; - private static final String TIMESTAMP_KEY_NAME = "timestamp"; private static final String METADATA_KEY_NAME = "metadata"; + @Deprecated + private static final String TIMESTAMP_KEY_NAME = "timestamp"; // for v0, v1 + + // default values for the current version public static final int DEFAULT_GENERATION_ID = -1; public static final String DEFAULT_CONSUMER_ID = ""; - public static final long DEFAULT_TIMESTAMP = -1; public static final long DEFAULT_RETENTION_TIME = Long.MAX_VALUE; + // default values for old versions, + // will be removed after these versions are deprecated + @Deprecated + public static final long DEFAULT_TIMESTAMP = -1; // for V0, V1 + private final String groupId; - private final int generationId; private final String consumerId; + private final int generationId; private final long retentionTime; private final Map offsetData; public static final class PartitionData { - public final long offset; @Deprecated - public final long timestamp; + public final long timestamp; // for V0, V1 + + public final long offset; public final String metadata; @Deprecated @@ -100,7 +108,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { */ @Deprecated public OffsetCommitRequest(String groupId, int generationId, String consumerId, Map offsetData) { - super(new Struct(CURRENT_SCHEMA)); + super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 1))); initCommonFields(groupId, offsetData); struct.set(GENERATION_ID_KEY_NAME, generationId); @@ -148,7 +156,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset); - if (fetchPartitionData.timestamp != DEFAULT_TIMESTAMP) + if (partitionData.hasField(TIMESTAMP_KEY_NAME)) partitionData.set(TIMESTAMP_KEY_NAME, fetchPartitionData.timestamp); partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata); partitionArray.add(partitionData); diff --git a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala index 875ccb8..b0e53ac 100644 --- a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala @@ -30,8 +30,6 @@ object OffsetCommitRequest extends Logging { val DefaultClientId = "" def readFrom(buffer: ByteBuffer): OffsetCommitRequest = { - val now = SystemTime.milliseconds - // Read values from the envelope val versionId = buffer.getShort assert(versionId == 0 || versionId == 1 || versionId == 2, @@ -116,8 +114,8 @@ case class OffsetCommitRequest(groupId: String, writeShortString(buffer, consumerId) } - // version 2 specific data - if (versionId == 2) { + // version 2 or above specific data + if (versionId >= 2) { buffer.putLong(retentionMs) } diff --git a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala index b42482b..46c8ab3 100644 --- a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala +++ b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala @@ -49,9 +49,9 @@ object OffsetAndMetadata { case class OffsetMetadataAndError(offsetMetadata: OffsetMetadata, error: Short = ErrorMapping.NoError) { - def offset() = offsetMetadata.offset + def offset = offsetMetadata.offset - def metadata() = offsetMetadata.metadata + def metadata = offsetMetadata.metadata override def toString = "[%s,ErrorCode %d]".format(offsetMetadata, error) } -- 1.7.12.4 From 2bf2fde0f24cee932e0eeef6f652f1b3a78a0e09 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 21 Nov 2014 14:00:17 -0800 Subject: [PATCH 08/20] Joel round four --- .../java/org/apache/kafka/common/requests/OffsetCommitRequest.java | 2 +- core/src/main/scala/kafka/api/OffsetCommitRequest.scala | 6 +++--- core/src/main/scala/kafka/server/OffsetManager.scala | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java index 6499232..ccae5e4 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java @@ -56,7 +56,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { // default values for old versions, // will be removed after these versions are deprecated @Deprecated - public static final long DEFAULT_TIMESTAMP = -1; // for V0, V1 + public static final long DEFAULT_TIMESTAMP = -1L; // for V0, V1 private final String groupId; private final String consumerId; diff --git a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala index b0e53ac..a180c8a 100644 --- a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala @@ -57,7 +57,7 @@ object OffsetCommitRequest extends Logging { // version 2 specific fields val retentionMs: Long = - if (versionId == 2) { + if (versionId >= 2) { buffer.getLong } else { org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME @@ -74,7 +74,7 @@ object OffsetCommitRequest extends Logging { if (versionId <= 1) buffer.getLong else - -1L + org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP } val metadata = readShortString(buffer) (TopicAndPartition(topic, partitionId), OffsetAndMetadata(offset, metadata, timestamp)) @@ -140,7 +140,7 @@ case class OffsetCommitRequest(groupId: String, shortStringLength(clientId) + shortStringLength(groupId) + (if (versionId >= 1) 4 /* group generation id */ + shortStringLength(consumerId) else 0) + - (if (versionId == 2) 8 /* retention time */ else 0) + + (if (versionId >= 2) 8 /* retention time */ else 0) + 4 + /* topic count */ requestInfoGroupedByTopic.foldLeft(0)((count, topicAndOffsets) => { val (topic, offsets) = topicAndOffsets diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 51106eb..d8ed2e2 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -509,7 +509,7 @@ object OffsetManager { val value = new Struct(CURRENT_SCHEMA.valueSchema) value.set(VALUE_OFFSET_FIELD, offsetAndMetadata.offset) value.set(VALUE_METADATA_FIELD, offsetAndMetadata.metadata) - // if the expiration timestamp is not specified, used the global value + // if the expiration timestamp is not specified, use the global value if (offsetAndMetadata.timestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) offsetAndMetadata.copy(timestamp = expirationTimestamp) value.set(VALUE_TIMESTAMP_FIELD, offsetAndMetadata.timestamp) -- 1.7.12.4 From b10615808407409f4bd74f85ce54bf3ec8236be8 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 25 Nov 2014 15:42:29 -0800 Subject: [PATCH 09/20] dummy --- core/src/main/scala/kafka/server/OffsetManager.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index d8ed2e2..e8e8380 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -232,13 +232,14 @@ class OffsetManager(val config: OffsetManagerConfig, } // compute the global expiration timestamp - val expirationTimestamp = retentionMs + SystemTime.milliseconds + val commitTimestamp = SystemTime.milliseconds + val expirationTimestamp = retentionMs + commitTimestamp // construct the message set to append val messages = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => new Message( key = OffsetManager.offsetCommitKey(groupId, topicAndPartition.topic, topicAndPartition.partition), - bytes = OffsetManager.offsetCommitValue(offsetAndMetadata, expirationTimestamp) + bytes = OffsetManager.offsetCommitValue(offsetAndMetadata, commitTimestamp, expirationTimestamp) ) }.toSeq @@ -502,10 +503,13 @@ object OffsetManager { * Generates the payload for offset commit message from given offset and metadata * * @param offsetAndMetadata consumer's current offset and metadata + * @param commitTimestamp global commit time of the offset * @param expirationTimestamp global expiration time of the offset * @return payload for offset commit message */ - def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata, expirationTimestamp: Long): Array[Byte] = { + def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata, + commitTimestamp: Long, + expirationTimestamp: Long): Array[Byte] = { val value = new Struct(CURRENT_SCHEMA.valueSchema) value.set(VALUE_OFFSET_FIELD, offsetAndMetadata.offset) value.set(VALUE_METADATA_FIELD, offsetAndMetadata.metadata) -- 1.7.12.4 From 6f8b733df90c6d8fea21e382f69131d20ad250b1 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 1 Dec 2014 11:42:07 -0800 Subject: [PATCH 10/20] Joel round four --- .../main/scala/kafka/server/OffsetManager.scala | 53 ++++++++++++++++++---- 1 file changed, 43 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index e8e8380..9470730 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -453,7 +453,7 @@ object OffsetManager { private case class KeyAndValueSchemas(keySchema: Schema, valueSchema: Schema) - private val CURRENT_OFFSET_SCHEMA_VERSION = 0.toShort + private val CURRENT_OFFSET_SCHEMA_VERSION = 1.toShort private val OFFSET_COMMIT_KEY_SCHEMA_V0 = new Schema(new Field("group", STRING), new Field("topic", STRING), @@ -465,12 +465,22 @@ object OffsetManager { private val OFFSET_COMMIT_VALUE_SCHEMA_V0 = new Schema(new Field("offset", INT64), new Field("metadata", STRING, "Associated metadata.", ""), new Field("timestamp", INT64)) + + private val OFFSET_COMMIT_VALUE_SCHEMA_V1 = new Schema(new Field("offset", INT64), + new Field("metadata", STRING, "Associated metadata.", ""), + new Field("commit_timestamp", INT64), + new Field("expire_timestamp", INT64)) + private val VALUE_OFFSET_FIELD = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("offset") private val VALUE_METADATA_FIELD = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("metadata") private val VALUE_TIMESTAMP_FIELD = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("timestamp") + private val VALUE_COMMIT_TIMESTAMP_FIELD = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("commit_timestamp") + private val VALUE_EXPIRE_TIMESTAMP_FIELD = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("expire_timestamp") + // map of versions to schemas - private val OFFSET_SCHEMAS = Map(0 -> KeyAndValueSchemas(OFFSET_COMMIT_KEY_SCHEMA_V0, OFFSET_COMMIT_VALUE_SCHEMA_V0)) + private val OFFSET_SCHEMAS = Map(0 -> KeyAndValueSchemas(OFFSET_COMMIT_KEY_SCHEMA_V0, OFFSET_COMMIT_VALUE_SCHEMA_V0), + 1 -> KeyAndValueSchemas(OFFSET_COMMIT_KEY_SCHEMA_V0, OFFSET_COMMIT_VALUE_SCHEMA_V1)) private val CURRENT_SCHEMA = schemaFor(CURRENT_OFFSET_SCHEMA_VERSION) @@ -510,13 +520,15 @@ object OffsetManager { def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata, commitTimestamp: Long, expirationTimestamp: Long): Array[Byte] = { + // generate commit value with schema version 1 val value = new Struct(CURRENT_SCHEMA.valueSchema) value.set(VALUE_OFFSET_FIELD, offsetAndMetadata.offset) value.set(VALUE_METADATA_FIELD, offsetAndMetadata.metadata) + value.set(VALUE_COMMIT_TIMESTAMP_FIELD, commitTimestamp) // if the expiration timestamp is not specified, use the global value if (offsetAndMetadata.timestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) offsetAndMetadata.copy(timestamp = expirationTimestamp) - value.set(VALUE_TIMESTAMP_FIELD, offsetAndMetadata.timestamp) + value.set(VALUE_EXPIRE_TIMESTAMP_FIELD, offsetAndMetadata.timestamp) val byteBuffer = ByteBuffer.allocate(2 /* version */ + value.sizeOf) byteBuffer.putShort(CURRENT_OFFSET_SCHEMA_VERSION) value.writeTo(byteBuffer) @@ -548,18 +560,39 @@ object OffsetManager { * @return an offset-metadata object from the message */ def readMessageValue(buffer: ByteBuffer): OffsetAndMetadata = { - if(buffer == null) { // tombstone + val structAndVersion = readMessageValueStruct(buffer) + + if (structAndVersion._1 == null) { // tombstone null } else { + val version = structAndVersion._2 + val struct = structAndVersion._1 + val offset = struct.get(VALUE_OFFSET_FIELD).asInstanceOf[Long] + val metadata = struct.get(VALUE_METADATA_FIELD).asInstanceOf[String] + + // handle version 0 and 1 + val timestamp = + if (version == 0) { + struct.get(VALUE_TIMESTAMP_FIELD).asInstanceOf[Long] + } else if (version == 1) { + struct.get(VALUE_EXPIRE_TIMESTAMP_FIELD).asInstanceOf[Long] + } else { + throw new IllegalStateException("Unknown offset message version") + } + + OffsetAndMetadata(offset, metadata, timestamp) + } + } + + def readMessageValueStruct(buffer: ByteBuffer): (Struct, Short) = { + if(buffer == null) { // tombstone + (null, -1) + } else { val version = buffer.getShort() val valueSchema = schemaFor(version).valueSchema val value = valueSchema.read(buffer).asInstanceOf[Struct] - val offset = value.get(VALUE_OFFSET_FIELD).asInstanceOf[Long] - val metadata = value.get(VALUE_METADATA_FIELD).asInstanceOf[String] - val timestamp = value.get(VALUE_TIMESTAMP_FIELD).asInstanceOf[Long] - - OffsetAndMetadata(offset, metadata, timestamp) + (value, version) } } @@ -568,7 +601,7 @@ object OffsetManager { class OffsetsMessageFormatter extends MessageFormatter { def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream) { val formattedKey = if (key == null) "NULL" else OffsetManager.readMessageKey(ByteBuffer.wrap(key)).toString - val formattedValue = if (value == null) "NULL" else OffsetManager.readMessageValue(ByteBuffer.wrap(value)).toString + val formattedValue = if (value == null) "NULL" else OffsetManager.readMessageValueStruct(ByteBuffer.wrap(value))._1.toString output.write(formattedKey.getBytes) output.write("::".getBytes) output.write(formattedValue.getBytes) -- 1.7.12.4 From 1e8b09c93c679e20d1ac35545b848e5cc5de5001 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 1 Dec 2014 18:03:06 -0800 Subject: [PATCH 11/20] unit tests --- .../apache/kafka/common/protocol/types/Struct.java | 6 ++-- .../kafka/common/OffsetMetadataAndError.scala | 11 +++--- .../main/scala/kafka/server/OffsetManager.scala | 40 ++++++++++++---------- .../api/RequestResponseSerializationTest.scala | 35 ++++++++++--------- .../scala/unit/kafka/server/OffsetCommitTest.scala | 8 ++--- 5 files changed, 52 insertions(+), 48 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java index 7672a3a..92de6a9 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java @@ -52,7 +52,7 @@ public class Struct { else if (field.defaultValue != Field.NO_DEFAULT) return field.defaultValue; else - throw new SchemaException("Missing value for field '" + field.name + " which has no default value."); + throw new SchemaException("Missing value for field '" + field.name + "' which has no default value."); } /** @@ -191,7 +191,7 @@ public class Struct { ArrayOf array = (ArrayOf) field.type(); return new Struct((Schema) array.type()); } else { - throw new SchemaException("Field " + field.name + " is not a container type, it is of type " + field.type()); + throw new SchemaException("Field '" + field.name + "' is not a container type, it is of type " + field.type()); } } @@ -234,7 +234,7 @@ public class Struct { */ private void validateField(Field field) { if (this.schema != field.schema) - throw new SchemaException("Attempt to access field '" + field.name + " from a different schema instance."); + throw new SchemaException("Attempt to access field '" + field.name + "' from a different schema instance."); if (field.index > values.length) throw new SchemaException("Invalid field index: " + field.index); } diff --git a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala index 46c8ab3..38893b0 100644 --- a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala +++ b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala @@ -17,8 +17,7 @@ package kafka.common -case class OffsetMetadata(offset: Long, - metadata: String = OffsetMetadata.NoMetadata) { +case class OffsetMetadata(offset: Long, metadata: String = OffsetMetadata.NoMetadata) { override def toString = "OffsetMetadata[%d,%s]" .format(offset, if (metadata != null && metadata.length > 0) metadata else "NO_METADATA") @@ -32,8 +31,7 @@ object OffsetMetadata { } // TODO: the following class will be replaced by OffsetMetadata after we deprecate OffsetCommitRequest version 0 and 1 -case class OffsetAndMetadata(offsetMetadata: OffsetMetadata, - timestamp: Long = -1) { +case class OffsetAndMetadata(offsetMetadata: OffsetMetadata, var timestamp: Long = -1L) { def offset() = offsetMetadata.offset def metadata() = offsetMetadata.metadata @@ -44,11 +42,12 @@ case class OffsetAndMetadata(offsetMetadata: OffsetMetadata, object OffsetAndMetadata { def apply(offset: Long, metadata: String, timestamp: Long) = new OffsetAndMetadata(OffsetMetadata(offset, metadata), timestamp) + def apply(offset: Long, metadata: String) = new OffsetAndMetadata(OffsetMetadata(offset, metadata)) + def apply(offset: Long) = new OffsetAndMetadata(OffsetMetadata(offset, OffsetMetadata.NoMetadata)) } -case class OffsetMetadataAndError(offsetMetadata: OffsetMetadata, - error: Short = ErrorMapping.NoError) { +case class OffsetMetadataAndError(offsetMetadata: OffsetMetadata, error: Short = ErrorMapping.NoError) { def offset = offsetMetadata.offset def metadata = offsetMetadata.metadata diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 9470730..4ebc1d8 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -42,7 +42,6 @@ import java.util.concurrent.TimeUnit import com.yammer.metrics.core.Gauge import org.I0Itec.zkclient.ZkClient - /** * Configuration settings for in-built offset management * @param maxMetadataSize The maximum allowed metadata for any offset commit. @@ -471,12 +470,14 @@ object OffsetManager { new Field("commit_timestamp", INT64), new Field("expire_timestamp", INT64)) - private val VALUE_OFFSET_FIELD = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("offset") - private val VALUE_METADATA_FIELD = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("metadata") - private val VALUE_TIMESTAMP_FIELD = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("timestamp") + private val VALUE_OFFSET_FIELD_V0 = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("offset") + private val VALUE_METADATA_FIELD_V0 = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("metadata") + private val VALUE_TIMESTAMP_FIELD_V0 = OFFSET_COMMIT_VALUE_SCHEMA_V0.get("timestamp") - private val VALUE_COMMIT_TIMESTAMP_FIELD = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("commit_timestamp") - private val VALUE_EXPIRE_TIMESTAMP_FIELD = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("expire_timestamp") + private val VALUE_OFFSET_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("offset") + private val VALUE_METADATA_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("metadata") + private val VALUE_COMMIT_TIMESTAMP_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("commit_timestamp") + private val VALUE_EXPIRE_TIMESTAMP_FIELD_V1 = OFFSET_COMMIT_VALUE_SCHEMA_V1.get("expire_timestamp") // map of versions to schemas private val OFFSET_SCHEMAS = Map(0 -> KeyAndValueSchemas(OFFSET_COMMIT_KEY_SCHEMA_V0, OFFSET_COMMIT_VALUE_SCHEMA_V0), @@ -522,13 +523,13 @@ object OffsetManager { expirationTimestamp: Long): Array[Byte] = { // generate commit value with schema version 1 val value = new Struct(CURRENT_SCHEMA.valueSchema) - value.set(VALUE_OFFSET_FIELD, offsetAndMetadata.offset) - value.set(VALUE_METADATA_FIELD, offsetAndMetadata.metadata) - value.set(VALUE_COMMIT_TIMESTAMP_FIELD, commitTimestamp) + value.set(VALUE_OFFSET_FIELD_V1, offsetAndMetadata.offset) + value.set(VALUE_METADATA_FIELD_V1, offsetAndMetadata.metadata) + value.set(VALUE_COMMIT_TIMESTAMP_FIELD_V1, commitTimestamp) // if the expiration timestamp is not specified, use the global value if (offsetAndMetadata.timestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) - offsetAndMetadata.copy(timestamp = expirationTimestamp) - value.set(VALUE_EXPIRE_TIMESTAMP_FIELD, offsetAndMetadata.timestamp) + offsetAndMetadata.timestamp = expirationTimestamp + value.set(VALUE_EXPIRE_TIMESTAMP_FIELD_V1, offsetAndMetadata.timestamp) val byteBuffer = ByteBuffer.allocate(2 /* version */ + value.sizeOf) byteBuffer.putShort(CURRENT_OFFSET_SCHEMA_VERSION) value.writeTo(byteBuffer) @@ -567,20 +568,23 @@ object OffsetManager { } else { val version = structAndVersion._2 val struct = structAndVersion._1 - val offset = struct.get(VALUE_OFFSET_FIELD).asInstanceOf[Long] - val metadata = struct.get(VALUE_METADATA_FIELD).asInstanceOf[String] // handle version 0 and 1 - val timestamp = if (version == 0) { - struct.get(VALUE_TIMESTAMP_FIELD).asInstanceOf[Long] + val offset = struct.get(VALUE_OFFSET_FIELD_V0).asInstanceOf[Long] + val metadata = struct.get(VALUE_OFFSET_FIELD_V0).asInstanceOf[String] + val timestamp = struct.get(VALUE_TIMESTAMP_FIELD_V0).asInstanceOf[Long] + + OffsetAndMetadata(offset, metadata, timestamp) } else if (version == 1) { - struct.get(VALUE_EXPIRE_TIMESTAMP_FIELD).asInstanceOf[Long] + val offset = struct.get(VALUE_OFFSET_FIELD_V1).asInstanceOf[Long] + val metadata = struct.get(VALUE_OFFSET_FIELD_V1).asInstanceOf[String] + val timestamp = struct.get(VALUE_EXPIRE_TIMESTAMP_FIELD_V1).asInstanceOf[Long] + + OffsetAndMetadata(offset, metadata, timestamp) } else { throw new IllegalStateException("Unknown offset message version") } - - OffsetAndMetadata(offset, metadata, timestamp) } } diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index 17beda0..4cb803f 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -17,20 +17,21 @@ package kafka.api -import org.junit._ -import org.scalatest.junit.JUnitSuite -import junit.framework.Assert._ -import java.nio.ByteBuffer -import kafka.message.{Message, ByteBufferMessageSet} +import kafka.common._ import kafka.cluster.Broker -import kafka.common.{OffsetAndMetadata, ErrorMapping, OffsetMetadataAndError} +import kafka.controller.LeaderIsrAndControllerEpoch +import kafka.message.{Message, ByteBufferMessageSet} import kafka.utils.SystemTime + +import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.requests._ -import org.apache.kafka.common.protocol.ApiKeys + import scala.Some -import kafka.controller.LeaderIsrAndControllerEpoch -import kafka.common.TopicAndPartition -import org.apache.kafka.common.TopicPartition +import java.nio.ByteBuffer + +import org.junit._ +import org.scalatest.junit.JUnitSuite +import junit.framework.Assert._ object SerializationTestUtils { @@ -156,8 +157,8 @@ object SerializationTestUtils { groupId = "group 1", retentionMs = SystemTime.milliseconds, requestInfo=collection.immutable.Map( - TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L, metadata="some metadata"), - TopicAndPartition(topic1, 1) -> OffsetAndMetadata(offset=100L, metadata=OffsetAndMetadata.NoMetadata) + TopicAndPartition(topic1, 0) -> OffsetAndMetadata(42L, "some metadata"), + TopicAndPartition(topic1, 1) -> OffsetAndMetadata(100L, OffsetMetadata.NoMetadata) )) } @@ -166,8 +167,8 @@ object SerializationTestUtils { versionId = 1, groupId = "group 1", requestInfo = collection.immutable.Map( - TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L, metadata="some metadata", timestamp=SystemTime.milliseconds), - TopicAndPartition(topic1, 1) -> OffsetAndMetadata(offset=100L, metadata=OffsetAndMetadata.NoMetadata, timestamp=SystemTime.milliseconds) + TopicAndPartition(topic1, 0) -> OffsetAndMetadata(42L, "some metadata", SystemTime.milliseconds), + TopicAndPartition(topic1, 1) -> OffsetAndMetadata(100L, OffsetMetadata.NoMetadata, SystemTime.milliseconds) )) } @@ -176,8 +177,8 @@ object SerializationTestUtils { versionId = 0, groupId = "group 1", requestInfo = collection.immutable.Map( - TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L, metadata="some metadata", timestamp=SystemTime.milliseconds), - TopicAndPartition(topic1, 1) -> OffsetAndMetadata(offset=100L, metadata=OffsetAndMetadata.NoMetadata, timestamp=SystemTime.milliseconds) + TopicAndPartition(topic1, 0) -> OffsetAndMetadata(42L, "some metadata", SystemTime.milliseconds), + TopicAndPartition(topic1, 1) -> OffsetAndMetadata(100L, OffsetMetadata.NoMetadata, SystemTime.milliseconds) )) } @@ -196,7 +197,7 @@ object SerializationTestUtils { def createTestOffsetFetchResponse: OffsetFetchResponse = { new OffsetFetchResponse(collection.immutable.Map( TopicAndPartition(topic1, 0) -> OffsetMetadataAndError(42L, "some metadata", ErrorMapping.NoError), - TopicAndPartition(topic1, 1) -> OffsetMetadataAndError(100L, OffsetAndMetadata.NoMetadata, ErrorMapping.UnknownTopicOrPartitionCode) + TopicAndPartition(topic1, 1) -> OffsetMetadataAndError(100L, OffsetMetadata.NoMetadata, ErrorMapping.UnknownTopicOrPartitionCode) )) } diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index 4d7a5dd..50aa52d 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -219,8 +219,8 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { // v0 version commit request with timestamp set to -1 // should not expire val commitRequest0 = OffsetCommitRequest( - groupId =group, - requestInfo = immutable.Map(topicPartition -> OffsetAndMetadata(1L, "metadata", -1)), + groupId = group, + requestInfo = immutable.Map(topicPartition -> OffsetAndMetadata(1L, "metadata", -1L)), versionId = 0 ) assertEquals(ErrorMapping.NoError, simpleConsumer.commitOffsets(commitRequest0).commitStatus.get(topicPartition).get) @@ -231,7 +231,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { // should not expire val commitRequest1 = OffsetCommitRequest( groupId = group, - requestInfo = immutable.Map(topicPartition -> OffsetAndMetadata(2L, "metadata", -1)), + requestInfo = immutable.Map(topicPartition -> OffsetAndMetadata(2L, "metadata", -1L)), versionId = 1 ) assertEquals(ErrorMapping.NoError, simpleConsumer.commitOffsets(commitRequest1).commitStatus.get(topicPartition).get) @@ -253,7 +253,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { // should not expire val commitRequest3 = OffsetCommitRequest( groupId = group, - requestInfo = immutable.Map(topicPartition -> OffsetAndMetadata(4L, "metadata", -1)), + requestInfo = immutable.Map(topicPartition -> OffsetAndMetadata(4L, "metadata", -1L)), versionId = 2, retentionMs = 1000 * 60 * 60 ) -- 1.7.12.4 From 887a6904febd023fb248b82a96c0e0e75238fd3d Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 14 Jan 2015 11:52:29 -0800 Subject: [PATCH 12/20] Joel and Jun's comments --- .../kafka/common/requests/OffsetCommitRequest.java | 64 +++++++++++----------- .../kafka/common/requests/RequestResponseTest.java | 4 +- .../kafka/common/OffsetMetadataAndError.scala | 2 +- .../main/scala/kafka/server/OffsetManager.scala | 54 +++++++++--------- .../main/scala/kafka/server/ReplicaManager.scala | 2 +- .../scala/unit/kafka/server/OffsetCommitTest.scala | 4 +- 6 files changed, 67 insertions(+), 63 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java index ccae5e4..24cac49 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java @@ -62,9 +62,9 @@ public class OffsetCommitRequest extends AbstractRequestResponse { private final String consumerId; private final int generationId; private final long retentionTime; - private final Map offsetData; + private final Map offsetData; - public static final class PartitionData { + public static final class PartitionOffset { @Deprecated public final long timestamp; // for V0, V1 @@ -72,13 +72,13 @@ public class OffsetCommitRequest extends AbstractRequestResponse { public final String metadata; @Deprecated - public PartitionData(long offset, long timestamp, String metadata) { + public PartitionOffset(long offset, long timestamp, String metadata) { this.offset = offset; this.timestamp = timestamp; this.metadata = metadata; } - public PartitionData(long offset, String metadata) { + public PartitionOffset(long offset, String metadata) { this(offset, DEFAULT_TIMESTAMP, metadata); } } @@ -89,7 +89,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { * @param offsetData */ @Deprecated - public OffsetCommitRequest(String groupId, Map offsetData) { + public OffsetCommitRequest(String groupId, Map offsetData) { super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 0))); initCommonFields(groupId, offsetData); this.groupId = groupId; @@ -107,7 +107,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { * @param offsetData */ @Deprecated - public OffsetCommitRequest(String groupId, int generationId, String consumerId, Map offsetData) { + public OffsetCommitRequest(String groupId, int generationId, String consumerId, Map offsetData) { super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 1))); initCommonFields(groupId, offsetData); @@ -128,7 +128,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { * @param retentionTime * @param offsetData */ - public OffsetCommitRequest(String groupId, int generationId, String consumerId, long retentionTime, Map offsetData) { + public OffsetCommitRequest(String groupId, int generationId, String consumerId, long retentionTime, Map offsetData) { super(new Struct(CURRENT_SCHEMA)); initCommonFields(groupId, offsetData); @@ -142,23 +142,25 @@ public class OffsetCommitRequest extends AbstractRequestResponse { this.offsetData = offsetData; } - private void initCommonFields(String groupId, Map offsetData) { - Map> topicsData = CollectionUtils.groupDataByTopic(offsetData); + private void initCommonFields(String groupId, Map offsetData) { + Map> topicsData = CollectionUtils.groupDataByTopic(offsetData); struct.set(GROUP_ID_KEY_NAME, groupId); List topicArray = new ArrayList(); - for (Map.Entry> topicEntry : topicsData.entrySet()) { + + for (Map.Entry> topicEntry: topicsData.entrySet()) { Struct topicData = struct.instance(TOPICS_KEY_NAME); topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); List partitionArray = new ArrayList(); - for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { - PartitionData fetchPartitionData = partitionEntry.getValue(); + for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { + PartitionOffset fetchPartitionOffset = partitionEntry.getValue(); Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); - partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset); + partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionOffset.offset); + // Only for v0 and v1 if (partitionData.hasField(TIMESTAMP_KEY_NAME)) - partitionData.set(TIMESTAMP_KEY_NAME, fetchPartitionData.timestamp); - partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata); + partitionData.set(TIMESTAMP_KEY_NAME, fetchPartitionOffset.timestamp); + partitionData.set(METADATA_KEY_NAME, fetchPartitionOffset.metadata); partitionArray.add(partitionData); } topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); @@ -189,24 +191,24 @@ public class OffsetCommitRequest extends AbstractRequestResponse { else retentionTime = DEFAULT_RETENTION_TIME; - offsetData = new HashMap(); - for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) { - Struct topicResponse = (Struct) topicResponseObj; - String topic = topicResponse.getString(TOPIC_KEY_NAME); - for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) { - Struct partitionResponse = (Struct) partitionResponseObj; - int partition = partitionResponse.getInt(PARTITION_KEY_NAME); - long offset = partitionResponse.getLong(COMMIT_OFFSET_KEY_NAME); - String metadata = partitionResponse.getString(METADATA_KEY_NAME); - PartitionData partitionData; + offsetData = new HashMap(); + for (Object topicDataObj : struct.getArray(TOPICS_KEY_NAME)) { + Struct topicData = (Struct) topicDataObj; + String topic = topicData.getString(TOPIC_KEY_NAME); + for (Object partitionDataObj : topicData.getArray(PARTITIONS_KEY_NAME)) { + Struct partitionData = (Struct) partitionDataObj; + int partition = partitionData.getInt(PARTITION_KEY_NAME); + long offset = partitionData.getLong(COMMIT_OFFSET_KEY_NAME); + String metadata = partitionData.getString(METADATA_KEY_NAME); + PartitionOffset partitionOffset; // This field only exists in v0 and v1 - if (partitionResponse.hasField(TIMESTAMP_KEY_NAME)) { - long timestamp = partitionResponse.getLong(TIMESTAMP_KEY_NAME); - partitionData = new PartitionData(offset, timestamp, metadata); + if (partitionData.hasField(TIMESTAMP_KEY_NAME)) { + long timestamp = partitionData.getLong(TIMESTAMP_KEY_NAME); + partitionOffset = new PartitionOffset(offset, timestamp, metadata); } else { - partitionData = new PartitionData(offset, metadata); + partitionOffset = new PartitionOffset(offset, metadata); } - offsetData.put(new TopicPartition(topic, partition), partitionData); + offsetData.put(new TopicPartition(topic, partition), partitionOffset); } } } @@ -227,7 +229,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { return retentionTime; } - public Map offsetData() { + public Map offsetData() { return offsetData; } 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 61a767a..21ebbee 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 @@ -139,8 +139,8 @@ public class RequestResponseTest { } private AbstractRequestResponse createOffsetCommitRequest() { - Map commitData = new HashMap(); - commitData.put(new TopicPartition("test", 0), new OffsetCommitRequest.PartitionData(100, "")); + Map commitData = new HashMap(); + commitData.put(new TopicPartition("test", 0), new OffsetCommitRequest.PartitionOffset(100, "")); return new OffsetCommitRequest("group1", 100, "consumer1", 1000000, commitData); } diff --git a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala index 38893b0..60ce3f7 100644 --- a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala +++ b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala @@ -31,7 +31,7 @@ object OffsetMetadata { } // TODO: the following class will be replaced by OffsetMetadata after we deprecate OffsetCommitRequest version 0 and 1 -case class OffsetAndMetadata(offsetMetadata: OffsetMetadata, var timestamp: Long = -1L) { +case class OffsetAndMetadata(offsetMetadata: OffsetMetadata, timestamp: Long = -1L) { def offset() = offsetMetadata.offset def metadata() = offsetMetadata.metadata diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 4ebc1d8..73cf423 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -236,9 +236,17 @@ class OffsetManager(val config: OffsetManagerConfig, // construct the message set to append val messages = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => + // if the expiration timestamp is not specified, use the global value + val newOffsetAndMetadata = offsetAndMetadata.copy(timestamp = + if (offsetAndMetadata.timestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) + expirationTimestamp + else + offsetAndMetadata.timestamp + ) + new Message( key = OffsetManager.offsetCommitKey(groupId, topicAndPartition.topic, topicAndPartition.partition), - bytes = OffsetManager.offsetCommitValue(offsetAndMetadata, commitTimestamp, expirationTimestamp) + bytes = OffsetManager.offsetCommitValue(newOffsetAndMetadata, commitTimestamp) ) }.toSeq @@ -498,7 +506,7 @@ object OffsetManager { * * @return key for offset commit message */ - def offsetCommitKey(group: String, topic: String, partition: Int, versionId: Short = 0): Array[Byte] = { + private def offsetCommitKey(group: String, topic: String, partition: Int, versionId: Short = 0): Array[Byte] = { val key = new Struct(CURRENT_SCHEMA.keySchema) key.set(KEY_GROUP_FIELD, group) key.set(KEY_TOPIC_FIELD, topic) @@ -515,20 +523,15 @@ object OffsetManager { * * @param offsetAndMetadata consumer's current offset and metadata * @param commitTimestamp global commit time of the offset - * @param expirationTimestamp global expiration time of the offset * @return payload for offset commit message */ - def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata, - commitTimestamp: Long, - expirationTimestamp: Long): Array[Byte] = { + private def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata, + commitTimestamp: Long): Array[Byte] = { // generate commit value with schema version 1 val value = new Struct(CURRENT_SCHEMA.valueSchema) value.set(VALUE_OFFSET_FIELD_V1, offsetAndMetadata.offset) value.set(VALUE_METADATA_FIELD_V1, offsetAndMetadata.metadata) value.set(VALUE_COMMIT_TIMESTAMP_FIELD_V1, commitTimestamp) - // if the expiration timestamp is not specified, use the global value - if (offsetAndMetadata.timestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) - offsetAndMetadata.timestamp = expirationTimestamp value.set(VALUE_EXPIRE_TIMESTAMP_FIELD_V1, offsetAndMetadata.timestamp) val byteBuffer = ByteBuffer.allocate(2 /* version */ + value.sizeOf) byteBuffer.putShort(CURRENT_OFFSET_SCHEMA_VERSION) @@ -542,7 +545,7 @@ object OffsetManager { * @param buffer input byte-buffer * @return an GroupTopicPartition object */ - def readMessageKey(buffer: ByteBuffer): GroupTopicPartition = { + private def readMessageKey(buffer: ByteBuffer): GroupTopicPartition = { val version = buffer.getShort() val keySchema = schemaFor(version).keySchema val key = keySchema.read(buffer).asInstanceOf[Struct] @@ -560,26 +563,23 @@ object OffsetManager { * @param buffer input byte-buffer * @return an offset-metadata object from the message */ - def readMessageValue(buffer: ByteBuffer): OffsetAndMetadata = { + private def readMessageValue(buffer: ByteBuffer): OffsetAndMetadata = { val structAndVersion = readMessageValueStruct(buffer) - if (structAndVersion._1 == null) { // tombstone + if (structAndVersion.value == null) { // tombstone null } else { - val version = structAndVersion._2 - val struct = structAndVersion._1 - // handle version 0 and 1 - if (version == 0) { - val offset = struct.get(VALUE_OFFSET_FIELD_V0).asInstanceOf[Long] - val metadata = struct.get(VALUE_OFFSET_FIELD_V0).asInstanceOf[String] - val timestamp = struct.get(VALUE_TIMESTAMP_FIELD_V0).asInstanceOf[Long] + if (structAndVersion.version == 0) { + val offset = structAndVersion.value.get(VALUE_OFFSET_FIELD_V0).asInstanceOf[Long] + val metadata = structAndVersion.value.get(VALUE_METADATA_FIELD_V0).asInstanceOf[String] + val timestamp = structAndVersion.value.get(VALUE_TIMESTAMP_FIELD_V0).asInstanceOf[Long] OffsetAndMetadata(offset, metadata, timestamp) - } else if (version == 1) { - val offset = struct.get(VALUE_OFFSET_FIELD_V1).asInstanceOf[Long] - val metadata = struct.get(VALUE_OFFSET_FIELD_V1).asInstanceOf[String] - val timestamp = struct.get(VALUE_EXPIRE_TIMESTAMP_FIELD_V1).asInstanceOf[Long] + } else if (structAndVersion.version == 1) { + val offset = structAndVersion.value.get(VALUE_OFFSET_FIELD_V1).asInstanceOf[Long] + val metadata = structAndVersion.value.get(VALUE_OFFSET_FIELD_V1).asInstanceOf[String] + val timestamp = structAndVersion.value.get(VALUE_EXPIRE_TIMESTAMP_FIELD_V1).asInstanceOf[Long] OffsetAndMetadata(offset, metadata, timestamp) } else { @@ -588,7 +588,7 @@ object OffsetManager { } } - def readMessageValueStruct(buffer: ByteBuffer): (Struct, Short) = { + private def readMessageValueStruct(buffer: ByteBuffer): MessageValueStructAndVersion = { if(buffer == null) { // tombstone (null, -1) } else { @@ -596,7 +596,7 @@ object OffsetManager { val valueSchema = schemaFor(version).valueSchema val value = valueSchema.read(buffer).asInstanceOf[Struct] - (value, version) + MessageValueStructAndVersion(value, version) } } @@ -605,7 +605,7 @@ object OffsetManager { class OffsetsMessageFormatter extends MessageFormatter { def writeTo(key: Array[Byte], value: Array[Byte], output: PrintStream) { val formattedKey = if (key == null) "NULL" else OffsetManager.readMessageKey(ByteBuffer.wrap(key)).toString - val formattedValue = if (value == null) "NULL" else OffsetManager.readMessageValueStruct(ByteBuffer.wrap(value))._1.toString + val formattedValue = if (value == null) "NULL" else OffsetManager.readMessageValueStruct(ByteBuffer.wrap(value)).value.toString output.write(formattedKey.getBytes) output.write("::".getBytes) output.write(formattedValue.getBytes) @@ -615,6 +615,8 @@ object OffsetManager { } +case class MessageValueStructAndVersion(value: Struct, version: Short) + case class GroupTopicPartition(group: String, topicPartition: TopicAndPartition) { def this(group: String, topic: String, partition: Int) = diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index c527482..44f0026 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -65,7 +65,7 @@ object ReplicaManager { } class ReplicaManager(val config: KafkaConfig, - time: Time, + private val time: Time, val zkClient: ZkClient, scheduler: Scheduler, val logManager: LogManager, diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index 50aa52d..266ed06 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -69,7 +69,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { Utils.rm(logDir) super.tearDown() } - /* + @Test def testUpdateOffsets() { val topic = "topic" @@ -206,7 +206,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(ErrorMapping.OffsetMetadataTooLargeCode, commitResponse1.commitStatus.get(topicAndPartition).get) } - */ + @Test def testOffsetExpiration() { // set up topic partition -- 1.7.12.4 From abc5ef4dc54bbb50d80d131aa3cbf0d0f81e0077 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 14 Jan 2015 15:49:42 -0800 Subject: [PATCH 13/20] incorporate Joel and Jun's comments --- .../main/scala/kafka/api/OffsetCommitRequest.scala | 2 + core/src/main/scala/kafka/server/KafkaApis.scala | 13 +++++- .../main/scala/kafka/server/OffsetManager.scala | 14 +------ .../scala/unit/kafka/server/OffsetCommitTest.scala | 48 +++++++++++++--------- .../unit/kafka/server/ServerShutdownTest.scala | 4 +- 5 files changed, 47 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala index a180c8a..f5b4a6b 100644 --- a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala @@ -77,9 +77,11 @@ object OffsetCommitRequest extends Logging { org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP } val metadata = readShortString(buffer) + (TopicAndPartition(topic, partitionId), OffsetAndMetadata(offset, metadata, timestamp)) }) }) + OffsetCommitRequest(consumerGroupId, immutable.Map(pairs:_*), versionId, correlationId, clientId, groupGenerationId, consumerId, retentionMs) } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 558194e..667c044 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -159,6 +159,17 @@ class KafkaApis(val requestChannel: RequestChannel, offsetCommitRequest.retentionMs } + // compute the expiration time stamp based on the request version: + // if it is before v2, use the retention time + current time stamp + val commitTimestamp = SystemTime.milliseconds + val offsetData = offsetCommitRequest.requestInfo.mapValues(offsetAndMetadata => + offsetAndMetadata.copy(timestamp = + if (offsetAndMetadata.timestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) + offsetRetention + commitTimestamp + else + offsetAndMetadata.timestamp + ) + ) // call offset manager to store offsets offsetManager.storeOffsets( @@ -166,7 +177,7 @@ class KafkaApis(val requestChannel: RequestChannel, offsetCommitRequest.consumerId, offsetCommitRequest.groupGenerationId, offsetRetention, - offsetCommitRequest.requestInfo, + offsetData, sendResponseCallback) } diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 73cf423..918bbba 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -230,23 +230,13 @@ class OffsetManager(val config: OffsetManagerConfig, validateOffsetMetadataLength(offsetAndMetadata.metadata) || nonExistentTopics.contains(topicAndPartition) } - // compute the global expiration timestamp val commitTimestamp = SystemTime.milliseconds - val expirationTimestamp = retentionMs + commitTimestamp // construct the message set to append val messages = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => - // if the expiration timestamp is not specified, use the global value - val newOffsetAndMetadata = offsetAndMetadata.copy(timestamp = - if (offsetAndMetadata.timestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) - expirationTimestamp - else - offsetAndMetadata.timestamp - ) - new Message( key = OffsetManager.offsetCommitKey(groupId, topicAndPartition.topic, topicAndPartition.partition), - bytes = OffsetManager.offsetCommitValue(newOffsetAndMetadata, commitTimestamp) + bytes = OffsetManager.offsetCommitValue(offsetAndMetadata, commitTimestamp) ) }.toSeq @@ -590,7 +580,7 @@ object OffsetManager { private def readMessageValueStruct(buffer: ByteBuffer): MessageValueStructAndVersion = { if(buffer == null) { // tombstone - (null, -1) + MessageValueStructAndVersion(null, -1) } else { val version = buffer.getShort() val valueSchema = schemaFor(version).valueSchema diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index 266ed06..48c6801 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -17,20 +17,24 @@ package kafka.server -import java.io.File -import kafka.utils._ -import junit.framework.Assert._ -import java.util.Properties +import kafka.api.{ConsumerMetadataRequest, OffsetCommitRequest, OffsetFetchRequest} import kafka.consumer.SimpleConsumer -import org.junit.{After, Before, Test} +import kafka.common.{OffsetMetadata, OffsetMetadataAndError, OffsetAndMetadata, ErrorMapping, TopicAndPartition} +import kafka.utils._ +import kafka.utils.TestUtils._ import kafka.zk.ZooKeeperTestHarness + +import org.junit.{After, Before, Test} import org.scalatest.junit.JUnit3Suite -import kafka.api.{ConsumerMetadataRequest, OffsetCommitRequest, OffsetFetchRequest} -import kafka.utils.TestUtils._ -import kafka.common.{OffsetMetadataAndError, OffsetAndMetadata, ErrorMapping, TopicAndPartition} + +import java.util.Properties +import java.io.File + import scala.util.Random import scala.collection._ +import junit.framework.Assert._ + class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { val random: Random = new Random() val brokerPort: Int = 9099 @@ -90,7 +94,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { val fetchResponse = simpleConsumer.fetchOffsets(fetchRequest) assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(topicAndPartition).get.error) - assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(topicAndPartition).get.metadata) + assertEquals(OffsetMetadata.NoMetadata, fetchResponse.requestInfo.get(topicAndPartition).get.metadata) assertEquals(42L, fetchResponse.requestInfo.get(topicAndPartition).get.offset) // Commit a new offset @@ -156,31 +160,37 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { val fetchResponse = simpleConsumer.fetchOffsets(fetchRequest) assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get.error) + assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.error) - assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.error) assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.error) + + assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.error) assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.error) - assertEquals(ErrorMapping.NoOffsetsCommittedCode, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.error) - assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.error) assertEquals(OffsetMetadataAndError.UnknownTopicOrPartition, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get) + + assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.error) assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get) + + assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.error) assertEquals(OffsetMetadataAndError.UnknownTopicOrPartition, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get) assertEquals("metadata one", fetchResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get.metadata) assertEquals("metadata two", fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.metadata) assertEquals("metadata three", fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.metadata) - assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.metadata) - assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.metadata) - assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.metadata) - assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.metadata) + + assertEquals(OffsetMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.metadata) + assertEquals(OffsetMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.metadata) + assertEquals(OffsetMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.metadata) + assertEquals(OffsetMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.metadata) assertEquals(42L, fetchResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get.offset) assertEquals(43L, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.offset) assertEquals(44L, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.offset) assertEquals(45L, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.offset) - assertEquals(OffsetAndMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.offset) - assertEquals(OffsetAndMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.offset) - assertEquals(OffsetAndMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.offset) + + assertEquals(OffsetMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.offset) + assertEquals(OffsetMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.offset) + assertEquals(OffsetMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.offset) } @Test diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index b46daa4..71317eb 100644 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -163,10 +163,10 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { server.shutdown() server.awaitShutdown() server.shutdown() - assertTrue(true); + assertTrue(true) } catch{ - case ex => fail() + case ex: Throwable => fail() } } } -- 1.7.12.4 From 33b8679930846c76d5c86bcae4aea8fa57480ff1 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 21 Jan 2015 16:35:01 -0800 Subject: [PATCH 14/20] Joel's comments --- core/src/main/scala/kafka/server/KafkaApis.scala | 7 ++++--- core/src/main/scala/kafka/server/OffsetManager.scala | 3 ++- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 667c044..35e7796 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -151,16 +151,17 @@ class KafkaApis(val requestChannel: RequestChannel, } // compute the retention time based on the request version: - // if it is before v2, we can use the default retention period config + // if it is before v2 or not specified by user, we can use the default retention period config val offsetRetention = - if (offsetCommitRequest.versionId <= 1) { + if (offsetCommitRequest.versionId <= 1 || + offsetCommitRequest.retentionMs == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME) { offsetManager.config.offsetsRetentionMs } else { offsetCommitRequest.retentionMs } // compute the expiration time stamp based on the request version: - // if it is before v2, use the retention time + current time stamp + // if it is after v2, use the retention time + current time stamp val commitTimestamp = SystemTime.milliseconds val offsetData = offsetCommitRequest.requestInfo.mapValues(offsetAndMetadata => offsetAndMetadata.copy(timestamp = diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 918bbba..13ca417 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -61,7 +61,7 @@ import org.I0Itec.zkclient.ZkClient */ case class OffsetManagerConfig(maxMetadataSize: Int = OffsetManagerConfig.DefaultMaxMetadataSize, loadBufferSize: Int = OffsetManagerConfig.DefaultLoadBufferSize, - offsetsRetentionMs: Long = 24*60*60000L, + offsetsRetentionMs: Long = OffsetManagerConfig.DefaultOffsetRetentionMs, offsetsRetentionCheckIntervalMs: Long = OffsetManagerConfig.DefaultOffsetsRetentionCheckIntervalMs, offsetsTopicNumPartitions: Int = OffsetManagerConfig.DefaultOffsetsTopicNumPartitions, offsetsTopicSegmentBytes: Int = OffsetManagerConfig.DefaultOffsetsTopicSegmentBytes, @@ -73,6 +73,7 @@ case class OffsetManagerConfig(maxMetadataSize: Int = OffsetManagerConfig.Defaul object OffsetManagerConfig { val DefaultMaxMetadataSize = 4096 val DefaultLoadBufferSize = 5*1024*1024 + val DefaultOffsetRetentionMs = 24*60*60*1000L val DefaultOffsetsRetentionCheckIntervalMs = 600000L val DefaultOffsetsTopicNumPartitions = 50 val DefaultOffsetsTopicSegmentBytes = 100*1024*1024 -- 1.7.12.4 From 405429e10a4bea1cbfbaa9accb7ce58cbc5e2a35 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 22 Jan 2015 18:47:10 -0800 Subject: [PATCH 15/20] Jun's comments --- .../kafka/common/requests/OffsetCommitRequest.java | 2 +- .../main/scala/kafka/api/OffsetCommitRequest.scala | 18 ++++++++---------- .../kafka/common/OffsetMetadataAndError.scala | 10 +++++++--- core/src/main/scala/kafka/server/KafkaApis.scala | 22 +++++++++++++++------- .../main/scala/kafka/server/OffsetManager.scala | 19 ++++++++----------- 5 files changed, 39 insertions(+), 32 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java index 24cac49..2224ad2 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java @@ -51,7 +51,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { // default values for the current version public static final int DEFAULT_GENERATION_ID = -1; public static final String DEFAULT_CONSUMER_ID = ""; - public static final long DEFAULT_RETENTION_TIME = Long.MAX_VALUE; + public static final long DEFAULT_RETENTION_TIME = -1L; // default values for old versions, // will be removed after these versions are deprecated diff --git a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala index f5b4a6b..cf8e6ac 100644 --- a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala @@ -43,25 +43,23 @@ object OffsetCommitRequest extends Logging { // version 1 and 2 specific fields val groupGenerationId: Int = - if (versionId >= 1) { + if (versionId >= 1) buffer.getInt - } else { + else org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_GENERATION_ID - } + val consumerId: String = - if (versionId >= 1) { + if (versionId >= 1) readShortString(buffer) - } else { + else org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_CONSUMER_ID - } // version 2 specific fields val retentionMs: Long = - if (versionId >= 2) { + if (versionId >= 2) buffer.getLong - } else { + else org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME - } val topicCount = buffer.getInt val pairs = (1 to topicCount).flatMap(_ => { @@ -130,7 +128,7 @@ case class OffsetCommitRequest(groupId: String, buffer.putLong(t2._2.offset) // version 0 and 1 specific data if (versionId <= 1) - buffer.putLong(t2._2.timestamp) + buffer.putLong(t2._2.commitTimestamp) writeShortString(buffer, t2._2.metadata) }) }) diff --git a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala index 60ce3f7..139913f 100644 --- a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala +++ b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala @@ -30,16 +30,20 @@ object OffsetMetadata { val InvalidOffsetMetadata = OffsetMetadata(OffsetMetadata.InvalidOffset, OffsetMetadata.NoMetadata) } -// TODO: the following class will be replaced by OffsetMetadata after we deprecate OffsetCommitRequest version 0 and 1 -case class OffsetAndMetadata(offsetMetadata: OffsetMetadata, timestamp: Long = -1L) { +case class OffsetAndMetadata(offsetMetadata: OffsetMetadata, + commitTimestamp: Long = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP, + expireTimestamp: Long = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) { + def offset() = offsetMetadata.offset def metadata() = offsetMetadata.metadata - override def toString = "[%s,ExpirationTime %d]".format(offsetMetadata, timestamp) + override def toString = "[%s,CommitTime %d,ExpirationTime %d]".format(offsetMetadata, commitTimestamp, expireTimestamp) } object OffsetAndMetadata { + def apply(offset: Long, metadata: String, commitTimestamp: Long, expireTimestamp: Long) = new OffsetAndMetadata(OffsetMetadata(offset, metadata), commitTimestamp, expireTimestamp) + def apply(offset: Long, metadata: String, timestamp: Long) = new OffsetAndMetadata(OffsetMetadata(offset, metadata), timestamp) def apply(offset: Long, metadata: String) = new OffsetAndMetadata(OffsetMetadata(offset, metadata)) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 35e7796..5e630c3 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -160,15 +160,23 @@ class KafkaApis(val requestChannel: RequestChannel, offsetCommitRequest.retentionMs } - // compute the expiration time stamp based on the request version: - // if it is after v2, use the retention time + current time stamp - val commitTimestamp = SystemTime.milliseconds + // if the commit time stamp is set to -1, override to current time stamp + // expire time stamp is computed as commit time stamp + retention time + val currentTimestamp = SystemTime.milliseconds + val defaultExpireTimestamp = offsetRetention + currentTimestamp + val offsetData = offsetCommitRequest.requestInfo.mapValues(offsetAndMetadata => - offsetAndMetadata.copy(timestamp = - if (offsetAndMetadata.timestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) - offsetRetention + commitTimestamp + offsetAndMetadata.copy(commitTimestamp = { + if (offsetAndMetadata.commitTimestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) + currentTimestamp + else + offsetAndMetadata.commitTimestamp + }, expireTimestamp = { + if (offsetAndMetadata.commitTimestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) + defaultExpireTimestamp else - offsetAndMetadata.timestamp + offsetRetention + offsetAndMetadata.commitTimestamp + } ) ) diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 13ca417..ac80f20 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -121,7 +121,7 @@ class OffsetManager(val config: OffsetManagerConfig, val startMs = SystemTime.milliseconds val staleOffsets = offsetsCache.filter { case (groupTopicPartition, offsetAndMetadata) => - offsetAndMetadata.timestamp < startMs + offsetAndMetadata.expireTimestamp < startMs } debug("Found %d expired offsets.".format(staleOffsets.size)) @@ -231,13 +231,11 @@ class OffsetManager(val config: OffsetManagerConfig, validateOffsetMetadataLength(offsetAndMetadata.metadata) || nonExistentTopics.contains(topicAndPartition) } - val commitTimestamp = SystemTime.milliseconds - // construct the message set to append val messages = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => new Message( key = OffsetManager.offsetCommitKey(groupId, topicAndPartition.topic, topicAndPartition.partition), - bytes = OffsetManager.offsetCommitValue(offsetAndMetadata, commitTimestamp) + bytes = OffsetManager.offsetCommitValue(offsetAndMetadata) ) }.toSeq @@ -513,17 +511,15 @@ object OffsetManager { * Generates the payload for offset commit message from given offset and metadata * * @param offsetAndMetadata consumer's current offset and metadata - * @param commitTimestamp global commit time of the offset * @return payload for offset commit message */ - private def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata, - commitTimestamp: Long): Array[Byte] = { + private def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata): Array[Byte] = { // generate commit value with schema version 1 val value = new Struct(CURRENT_SCHEMA.valueSchema) value.set(VALUE_OFFSET_FIELD_V1, offsetAndMetadata.offset) value.set(VALUE_METADATA_FIELD_V1, offsetAndMetadata.metadata) - value.set(VALUE_COMMIT_TIMESTAMP_FIELD_V1, commitTimestamp) - value.set(VALUE_EXPIRE_TIMESTAMP_FIELD_V1, offsetAndMetadata.timestamp) + value.set(VALUE_COMMIT_TIMESTAMP_FIELD_V1, offsetAndMetadata.commitTimestamp) + value.set(VALUE_EXPIRE_TIMESTAMP_FIELD_V1, offsetAndMetadata.expireTimestamp) val byteBuffer = ByteBuffer.allocate(2 /* version */ + value.sizeOf) byteBuffer.putShort(CURRENT_OFFSET_SCHEMA_VERSION) value.writeTo(byteBuffer) @@ -570,9 +566,10 @@ object OffsetManager { } else if (structAndVersion.version == 1) { val offset = structAndVersion.value.get(VALUE_OFFSET_FIELD_V1).asInstanceOf[Long] val metadata = structAndVersion.value.get(VALUE_OFFSET_FIELD_V1).asInstanceOf[String] - val timestamp = structAndVersion.value.get(VALUE_EXPIRE_TIMESTAMP_FIELD_V1).asInstanceOf[Long] + val commitTimestamp = structAndVersion.value.get(VALUE_COMMIT_TIMESTAMP_FIELD_V1).asInstanceOf[Long] + val expireTimestamp = structAndVersion.value.get(VALUE_EXPIRE_TIMESTAMP_FIELD_V1).asInstanceOf[Long] - OffsetAndMetadata(offset, metadata, timestamp) + OffsetAndMetadata(offset, metadata, commitTimestamp, expireTimestamp) } else { throw new IllegalStateException("Unknown offset message version") } -- 1.7.12.4 From 0a8f0867907585a1d0ee72550e206e1befa7a7ef Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 23 Jan 2015 16:05:55 -0800 Subject: [PATCH 16/20] minor test fix --- .../test/scala/unit/kafka/server/OffsetCommitTest.scala | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index 48c6801..fcb8fb7 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -226,7 +226,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { val fetchRequest = OffsetFetchRequest(group, Seq(TopicAndPartition(topic, 0))) - // v0 version commit request with timestamp set to -1 + // v0 version commit request with commit timestamp set to -1 // should not expire val commitRequest0 = OffsetCommitRequest( groupId = group, @@ -237,7 +237,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { Thread.sleep(retentionCheckInterval * 2) assertEquals(1L, simpleConsumer.fetchOffsets(fetchRequest).requestInfo.get(topicPartition).get.offset) - // v1 version commit request with timestamp set to -1 + // v1 version commit request with commit timestamp set to -1 // should not expire val commitRequest1 = OffsetCommitRequest( groupId = group, @@ -248,11 +248,11 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { Thread.sleep(retentionCheckInterval * 2) assertEquals(2L, simpleConsumer.fetchOffsets(fetchRequest).requestInfo.get(topicPartition).get.offset) - // v1 version commit request with timestamp set to now + // v1 version commit request with commit timestamp set to now - two days // should expire val commitRequest2 = OffsetCommitRequest( groupId = group, - requestInfo = immutable.Map(topicPartition -> OffsetAndMetadata(3L, "metadata", SystemTime.milliseconds)), + requestInfo = immutable.Map(topicPartition -> OffsetAndMetadata(3L, "metadata", SystemTime.milliseconds - 2*24*60*60*1000L)), versionId = 1 ) assertEquals(ErrorMapping.NoError, simpleConsumer.commitOffsets(commitRequest2).commitStatus.get(topicPartition).get) @@ -265,7 +265,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { groupId = group, requestInfo = immutable.Map(topicPartition -> OffsetAndMetadata(4L, "metadata", -1L)), versionId = 2, - retentionMs = 1000 * 60 * 60 + retentionMs = 1000 * 60 * 60L ) assertEquals(ErrorMapping.NoError, simpleConsumer.commitOffsets(commitRequest3).commitStatus.get(topicPartition).get) Thread.sleep(retentionCheckInterval * 2) @@ -275,9 +275,9 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { // should expire val commitRequest4 = OffsetCommitRequest( groupId = "test-group", - requestInfo = immutable.Map(TopicAndPartition(topic, 0) -> OffsetAndMetadata(5L, "metadata", -1)), + requestInfo = immutable.Map(TopicAndPartition(topic, 0) -> OffsetAndMetadata(5L, "metadata", -1L)), versionId = 2, - retentionMs = 0 + retentionMs = 0L ) assertEquals(ErrorMapping.NoError, simpleConsumer.commitOffsets(commitRequest4).commitStatus.get(topicPartition).get) Thread.sleep(retentionCheckInterval * 2) -- 1.7.12.4 From f0b9fd1918798cd80ac120dfa7770c6b92699fcc Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 6 Feb 2015 11:00:53 -0800 Subject: [PATCH 17/20] address comments --- .../org/apache/kafka/common/protocol/Protocol.java | 2 +- .../kafka/common/requests/OffsetCommitRequest.java | 52 +++++++++++----------- .../kafka/common/requests/RequestResponseTest.java | 4 +- .../main/scala/kafka/api/OffsetFetchRequest.scala | 6 +-- 4 files changed, 32 insertions(+), 32 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 401e914..4c72486 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 @@ -199,7 +199,7 @@ public class Protocol { new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0))); public static Schema[] OFFSET_COMMIT_REQUEST = new Schema[] { OFFSET_COMMIT_REQUEST_V0, OFFSET_COMMIT_REQUEST_V1, OFFSET_COMMIT_REQUEST_V2 }; - /* The response types for both V0, V1 and V2 of OFFSET_COMMIT_REQUEST are the same. */ + /* The response types for V0, V1 and V2 of OFFSET_COMMIT_REQUEST are the same. */ public static Schema[] OFFSET_COMMIT_RESPONSE = new Schema[] { OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V0 }; /* Offset fetch api */ diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java index 2224ad2..b92f670 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java @@ -62,9 +62,9 @@ public class OffsetCommitRequest extends AbstractRequestResponse { private final String consumerId; private final int generationId; private final long retentionTime; - private final Map offsetData; + private final Map offsetData; - public static final class PartitionOffset { + public static final class PartitionData { @Deprecated public final long timestamp; // for V0, V1 @@ -72,13 +72,13 @@ public class OffsetCommitRequest extends AbstractRequestResponse { public final String metadata; @Deprecated - public PartitionOffset(long offset, long timestamp, String metadata) { + public PartitionData(long offset, long timestamp, String metadata) { this.offset = offset; this.timestamp = timestamp; this.metadata = metadata; } - public PartitionOffset(long offset, String metadata) { + public PartitionData(long offset, String metadata) { this(offset, DEFAULT_TIMESTAMP, metadata); } } @@ -89,7 +89,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { * @param offsetData */ @Deprecated - public OffsetCommitRequest(String groupId, Map offsetData) { + public OffsetCommitRequest(String groupId, Map offsetData) { super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 0))); initCommonFields(groupId, offsetData); this.groupId = groupId; @@ -107,7 +107,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { * @param offsetData */ @Deprecated - public OffsetCommitRequest(String groupId, int generationId, String consumerId, Map offsetData) { + public OffsetCommitRequest(String groupId, int generationId, String consumerId, Map offsetData) { super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 1))); initCommonFields(groupId, offsetData); @@ -128,7 +128,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { * @param retentionTime * @param offsetData */ - public OffsetCommitRequest(String groupId, int generationId, String consumerId, long retentionTime, Map offsetData) { + public OffsetCommitRequest(String groupId, int generationId, String consumerId, long retentionTime, Map offsetData) { super(new Struct(CURRENT_SCHEMA)); initCommonFields(groupId, offsetData); @@ -142,25 +142,25 @@ public class OffsetCommitRequest extends AbstractRequestResponse { this.offsetData = offsetData; } - private void initCommonFields(String groupId, Map offsetData) { - Map> topicsData = CollectionUtils.groupDataByTopic(offsetData); + private void initCommonFields(String groupId, Map offsetData) { + Map> topicsData = CollectionUtils.groupDataByTopic(offsetData); struct.set(GROUP_ID_KEY_NAME, groupId); List topicArray = new ArrayList(); - for (Map.Entry> topicEntry: topicsData.entrySet()) { + for (Map.Entry> topicEntry: topicsData.entrySet()) { Struct topicData = struct.instance(TOPICS_KEY_NAME); topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); List partitionArray = new ArrayList(); - for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { - PartitionOffset fetchPartitionOffset = partitionEntry.getValue(); + for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { + PartitionData fetchPartitionData = partitionEntry.getValue(); Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); - partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionOffset.offset); + partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset); // Only for v0 and v1 if (partitionData.hasField(TIMESTAMP_KEY_NAME)) - partitionData.set(TIMESTAMP_KEY_NAME, fetchPartitionOffset.timestamp); - partitionData.set(METADATA_KEY_NAME, fetchPartitionOffset.metadata); + partitionData.set(TIMESTAMP_KEY_NAME, fetchPartitionData.timestamp); + partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata); partitionArray.add(partitionData); } topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray()); @@ -191,22 +191,22 @@ public class OffsetCommitRequest extends AbstractRequestResponse { else retentionTime = DEFAULT_RETENTION_TIME; - offsetData = new HashMap(); + offsetData = new HashMap(); for (Object topicDataObj : struct.getArray(TOPICS_KEY_NAME)) { Struct topicData = (Struct) topicDataObj; String topic = topicData.getString(TOPIC_KEY_NAME); for (Object partitionDataObj : topicData.getArray(PARTITIONS_KEY_NAME)) { - Struct partitionData = (Struct) partitionDataObj; - int partition = partitionData.getInt(PARTITION_KEY_NAME); - long offset = partitionData.getLong(COMMIT_OFFSET_KEY_NAME); - String metadata = partitionData.getString(METADATA_KEY_NAME); - PartitionOffset partitionOffset; + Struct partitionDataStruct = (Struct) partitionDataObj; + int partition = partitionDataStruct.getInt(PARTITION_KEY_NAME); + long offset = partitionDataStruct.getLong(COMMIT_OFFSET_KEY_NAME); + String metadata = partitionDataStruct.getString(METADATA_KEY_NAME); + PartitionData partitionOffset; // This field only exists in v0 and v1 - if (partitionData.hasField(TIMESTAMP_KEY_NAME)) { - long timestamp = partitionData.getLong(TIMESTAMP_KEY_NAME); - partitionOffset = new PartitionOffset(offset, timestamp, metadata); + if (partitionDataStruct.hasField(TIMESTAMP_KEY_NAME)) { + long timestamp = partitionDataStruct.getLong(TIMESTAMP_KEY_NAME); + partitionOffset = new PartitionData(offset, timestamp, metadata); } else { - partitionOffset = new PartitionOffset(offset, metadata); + partitionOffset = new PartitionData(offset, metadata); } offsetData.put(new TopicPartition(topic, partition), partitionOffset); } @@ -229,7 +229,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse { return retentionTime; } - public Map offsetData() { + public Map offsetData() { return offsetData; } 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 21ebbee..61a767a 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 @@ -139,8 +139,8 @@ public class RequestResponseTest { } private AbstractRequestResponse createOffsetCommitRequest() { - Map commitData = new HashMap(); - commitData.put(new TopicPartition("test", 0), new OffsetCommitRequest.PartitionOffset(100, "")); + Map commitData = new HashMap(); + commitData.put(new TopicPartition("test", 0), new OffsetCommitRequest.PartitionData(100, "")); return new OffsetCommitRequest("group1", 100, "consumer1", 1000000, commitData); } diff --git a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala index d696284..67811a7 100644 --- a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala @@ -17,17 +17,17 @@ package kafka.api -import java.nio.ByteBuffer - import kafka.api.ApiUtils._ import kafka.utils.Logging import kafka.network.{BoundedByteBufferSend, RequestChannel} -import kafka.network.RequestChannel.Response import kafka.common._ import kafka.common.TopicAndPartition import kafka.network.RequestChannel.Response + import scala.Some +import java.nio.ByteBuffer + object OffsetFetchRequest extends Logging { val CurrentVersion: Short = 0 val DefaultClientId = "" -- 1.7.12.4 From 8faf1c31234ae8cc4db07423970eb0efc40332b4 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 26 Mar 2015 12:13:41 -0700 Subject: [PATCH 18/20] rebase and fixes --- .../clients/consumer/internals/Coordinator.java | 8 +- .../org/apache/kafka/common/protocol/Protocol.java | 184 ++++++++++----------- core/src/main/scala/kafka/server/KafkaApis.scala | 24 ++- .../main/scala/kafka/server/OffsetManager.scala | 13 +- core/src/test/resources/log4j.properties | 6 +- .../scala/unit/kafka/server/OffsetCommitTest.scala | 1 + 6 files changed, 122 insertions(+), 114 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java index 436f9b2..8d44814 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java @@ -162,8 +162,12 @@ public final class Coordinator { Map offsetData; offsetData = new HashMap(offsets.size()); for (Map.Entry entry : offsets.entrySet()) - offsetData.put(entry.getKey(), new OffsetCommitRequest.PartitionData(entry.getValue(), now, "")); - OffsetCommitRequest req = new OffsetCommitRequest(this.groupId, this.generation, this.consumerId, offsetData); + offsetData.put(entry.getKey(), new OffsetCommitRequest.PartitionData(entry.getValue(), "")); + OffsetCommitRequest req = new OffsetCommitRequest(this.groupId, + this.generation, + this.consumerId, + OffsetCommitRequest.DEFAULT_RETENTION_TIME, + offsetData); // send request and possibly wait for response if it is blocking RequestCompletionHandler handler = new CommitOffsetCompletionHandler(offsets); 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 4c72486..9c4518e 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 @@ -88,17 +88,17 @@ public class Protocol { /* Produce api */ - public static Schema TOPIC_PRODUCE_DATA_V0 = new Schema(new Field("topic", STRING), - new Field("data", new ArrayOf(new Schema(new Field("partition", INT32), + public static final Schema TOPIC_PRODUCE_DATA_V0 = new Schema(new Field("topic", STRING), + new Field("data", new ArrayOf(new Schema(new Field("partition", INT32), new Field("record_set", BYTES))))); - public static Schema PRODUCE_REQUEST_V0 = new Schema(new Field("acks", + public static final Schema PRODUCE_REQUEST_V0 = new Schema(new Field("acks", INT16, "The number of nodes that should replicate the produce before returning. -1 indicates the full ISR."), - new Field("timeout", INT32, "The time to await a response in ms."), - new Field("topic_data", new ArrayOf(TOPIC_PRODUCE_DATA_V0))); + new Field("timeout", INT32, "The time to await a response in ms."), + new Field("topic_data", new ArrayOf(TOPIC_PRODUCE_DATA_V0))); - public static Schema PRODUCE_RESPONSE_V0 = new Schema(new Field("responses", + public static final Schema PRODUCE_RESPONSE_V0 = new Schema(new Field("responses", new ArrayOf(new Schema(new Field("topic", STRING), new Field("partition_responses", new ArrayOf(new Schema(new Field("partition", @@ -108,99 +108,99 @@ public class Protocol { new Field("base_offset", INT64)))))))); - public static Schema[] PRODUCE_REQUEST = new Schema[] { PRODUCE_REQUEST_V0 }; - public static Schema[] PRODUCE_RESPONSE = new Schema[] { PRODUCE_RESPONSE_V0 }; + public static final Schema[] PRODUCE_REQUEST = new Schema[] {PRODUCE_REQUEST_V0}; + public static final Schema[] PRODUCE_RESPONSE = new Schema[] {PRODUCE_RESPONSE_V0}; /* Offset commit api */ - public static Schema OFFSET_COMMIT_REQUEST_PARTITION_V0 = new Schema(new Field("partition", - INT32, - "Topic partition id."), - new Field("offset", - INT64, - "Message offset to be committed."), - new Field("timestamp", - INT64, - "Timestamp of the commit"), - new Field("metadata", - STRING, - "Any associated metadata the client wants to keep.")); - - public static Schema OFFSET_COMMIT_REQUEST_PARTITION_V1 = new Schema(new Field("partition", - INT32, - "Topic partition id."), - new Field("offset", - INT64, - "Message offset to be committed."), - new Field("metadata", - STRING, - "Any associated metadata the client wants to keep.")); + public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V0 = new Schema(new Field("partition", + INT32, + "Topic partition id."), + new Field("offset", + INT64, + "Message offset to be committed."), + new Field("timestamp", + INT64, + "Timestamp of the commit"), + new Field("metadata", + STRING, + "Any associated metadata the client wants to keep.")); - public static Schema OFFSET_COMMIT_REQUEST_TOPIC_V0 = new Schema(new Field("topic", + public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V1 = new Schema(new Field("partition", + INT32, + "Topic partition id."), + new Field("offset", + INT64, + "Message offset to be committed."), + new Field("metadata", + STRING, + "Any associated metadata the client wants to keep.")); + + public static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V0 = new Schema(new Field("topic", + STRING, + "Topic to commit."), + new Field("partitions", + new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V0), + "Partitions to commit offsets.")); + + public static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V1 = new Schema(new Field("topic", + STRING, + "Topic to commit."), + new Field("partitions", + new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V1), + "Partitions to commit offsets.")); + + public static final Schema OFFSET_COMMIT_REQUEST_V0 = new Schema(new Field("group_id", STRING, - "Topic to commit."), - new Field("partitions", - new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V0), - "Partitions to commit offsets.")); + "The consumer group id."), + new Field("topics", + new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0), + "Topics to commit offsets.")); - public static Schema OFFSET_COMMIT_REQUEST_TOPIC_V1 = new Schema(new Field("topic", + public static final Schema OFFSET_COMMIT_REQUEST_V1 = new Schema(new Field("group_id", + STRING, + "The consumer group id."), + new Field("group_generation_id", + INT32, + "The generation of the consumer group."), + new Field("consumer_id", + STRING, + "The consumer id assigned by the group coordinator."), + new Field("topics", + new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0), + "Topics to commit offsets.")); + + public static final Schema OFFSET_COMMIT_REQUEST_V2 = new Schema(new Field("group_id", + STRING, + "The consumer group id."), + new Field("group_generation_id", + INT32, + "The generation of the consumer group."), + new Field("consumer_id", STRING, - "Topic to commit."), - new Field("partitions", - new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V1), - "Partitions to commit offsets.")); - - public static Schema OFFSET_COMMIT_REQUEST_V0 = new Schema(new Field("group_id", - STRING, - "The consumer group id."), - new Field("topics", - new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0), - "Topics to commit offsets.")); - - public static Schema OFFSET_COMMIT_REQUEST_V1 = new Schema(new Field("group_id", - STRING, - "The consumer group id."), - new Field("group_generation_id", - INT32, - "The generation of the consumer group."), - new Field("consumer_id", - STRING, - "The consumer id assigned by the group coordinator."), - new Field("topics", - new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0), - "Topics to commit offsets.")); - - public static Schema OFFSET_COMMIT_REQUEST_V2 = new Schema(new Field("group_id", - STRING, - "The consumer group id."), - new Field("group_generation_id", - INT32, - "The generation of the consumer group."), - new Field("consumer_id", - STRING, - "The consumer id assigned by the group coordinator."), - new Field("retention_time", - INT64, - "Time period in ms to retain the offset."), - new Field("topics", - new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V1), - "Topics to commit offsets.")); - - public static Schema OFFSET_COMMIT_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", - INT32, - "Topic partition id."), - new Field("error_code", - INT16)); - - public static Schema OFFSET_COMMIT_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), - new Field("partition_responses", - new ArrayOf(OFFSET_COMMIT_RESPONSE_PARTITION_V0))); - - public static Schema OFFSET_COMMIT_RESPONSE_V0 = new Schema(new Field("responses", - new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0))); - - public static Schema[] OFFSET_COMMIT_REQUEST = new Schema[] { OFFSET_COMMIT_REQUEST_V0, OFFSET_COMMIT_REQUEST_V1, OFFSET_COMMIT_REQUEST_V2 }; + "The consumer id assigned by the group coordinator."), + new Field("retention_time", + INT64, + "Time period in ms to retain the offset."), + new Field("topics", + new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V1), + "Topics to commit offsets.")); + + public static final Schema OFFSET_COMMIT_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", + INT32, + "Topic partition id."), + new Field("error_code", + INT16)); + + public static final Schema OFFSET_COMMIT_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), + new Field("partition_responses", + new ArrayOf(OFFSET_COMMIT_RESPONSE_PARTITION_V0))); + + public static final Schema OFFSET_COMMIT_RESPONSE_V0 = new Schema(new Field("responses", + new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0))); + + public static final Schema[] OFFSET_COMMIT_REQUEST = new Schema[] {OFFSET_COMMIT_REQUEST_V0, OFFSET_COMMIT_REQUEST_V1, OFFSET_COMMIT_REQUEST_V2}; /* The response types for V0, V1 and V2 of OFFSET_COMMIT_REQUEST are the same. */ - public static Schema[] OFFSET_COMMIT_RESPONSE = new Schema[] { OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V0 }; + public static final Schema[] OFFSET_COMMIT_RESPONSE = new Schema[] {OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V0}; /* Offset fetch api */ public static final Schema OFFSET_FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition", diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 5e630c3..475b907 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -160,23 +160,20 @@ class KafkaApis(val requestChannel: RequestChannel, offsetCommitRequest.retentionMs } - // if the commit time stamp is set to -1, override to current time stamp - // expire time stamp is computed as commit time stamp + retention time + // always override the commit time stamp to the current timestamp; + // however, if the commit time stamp was explicitly specified, set the expire time stamp as commit time stamp + retention time val currentTimestamp = SystemTime.milliseconds val defaultExpireTimestamp = offsetRetention + currentTimestamp val offsetData = offsetCommitRequest.requestInfo.mapValues(offsetAndMetadata => - offsetAndMetadata.copy(commitTimestamp = { - if (offsetAndMetadata.commitTimestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) - currentTimestamp - else - offsetAndMetadata.commitTimestamp - }, expireTimestamp = { - if (offsetAndMetadata.commitTimestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) - defaultExpireTimestamp - else - offsetRetention + offsetAndMetadata.commitTimestamp - } + offsetAndMetadata.copy( + commitTimestamp = currentTimestamp, + expireTimestamp = { + if (offsetAndMetadata.commitTimestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) + defaultExpireTimestamp + else + offsetRetention + offsetAndMetadata.commitTimestamp + } ) ) @@ -185,7 +182,6 @@ class KafkaApis(val requestChannel: RequestChannel, offsetCommitRequest.groupId, offsetCommitRequest.consumerId, offsetCommitRequest.groupGenerationId, - offsetRetention, offsetData, sendResponseCallback) } diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index ac80f20..395b1db 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -216,7 +216,6 @@ class OffsetManager(val config: OffsetManagerConfig, def storeOffsets(groupId: String, consumerId: String, generationId: Int, - retentionMs: Long, offsetMetadata: immutable.Map[TopicAndPartition, OffsetAndMetadata], responseCallback: immutable.Map[TopicAndPartition, Short] => Unit) { // check if there are any non-existent topics @@ -383,8 +382,17 @@ class OffsetManager(val config: OffsetManagerConfig, else trace("Ignoring redundant tombstone for %s.".format(key)) } else { + // special handling for version 0: + // set the expiration time stamp as commit time stamp + server default retention time val value = OffsetManager.readMessageValue(msgAndOffset.message.payload) - putOffset(key, value) + putOffset(key, value.copy ( + expireTimestamp = { + if (value.expireTimestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) + value.commitTimestamp + config.offsetsRetentionMs + else + value.expireTimestamp + } + )) trace("Loaded offset %s for %s.".format(value, key)) } currOffset = msgAndOffset.nextOffset @@ -556,7 +564,6 @@ object OffsetManager { if (structAndVersion.value == null) { // tombstone null } else { - // handle version 0 and 1 if (structAndVersion.version == 0) { val offset = structAndVersion.value.get(VALUE_OFFSET_FIELD_V0).asInstanceOf[Long] val metadata = structAndVersion.value.get(VALUE_METADATA_FIELD_V0).asInstanceOf[String] diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 1b7d5d8..9973dad 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=TRACE, 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.kafka=TRACE +log4j.logger.org.apache.kafka=TRACE # 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/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index fcb8fb7..6c47cbc 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -51,6 +51,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { override def setUp() { super.setUp() val config: Properties = createBrokerConfig(1, brokerPort) + config.setProperty(KafkaConfig.OffsetsRetentionCheckIntervalMsProp, retentionCheckInterval.toString) val logDirPath = config.getProperty("log.dir") logDir = new File(logDirPath) time = new MockTime() -- 1.7.12.4 From a8b5a500cd6e353d92bca439b363a017b96e3744 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 26 Mar 2015 12:15:10 -0700 Subject: [PATCH 19/20] revert log4j --- core/src/test/resources/log4j.properties | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 9973dad..1b7d5d8 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=TRACE, 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=TRACE -log4j.logger.org.apache.kafka=TRACE +log4j.logger.kafka=ERROR +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 -- 1.7.12.4 From 459733b67a3e04430c8b398209a5348946864fd7 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 26 Mar 2015 12:27:06 -0700 Subject: [PATCH 20/20] minor fixes --- core/src/main/scala/kafka/server/KafkaConfig.scala | 2 +- core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala | 1 + 2 files 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 46d21c7..422451a 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -336,7 +336,7 @@ object KafkaConfig { val OffsetsTopicPartitionsDoc = "The number of partitions for the offset commit topic (should not change after deployment)" val OffsetsTopicSegmentBytesDoc = "The offsets topic segment bytes should be kept relatively small in order to facilitate faster log compaction and cache loads" val OffsetsTopicCompressionCodecDoc = "Compression codec for the offsets topic - compression may be used to achieve \"atomic\" commits" - val OffsetsRetentionMinutesDoc = "Offsets older than this retention period will be discarded" + val OffsetsRetentionMinutesDoc = "Log retention window in minutes for offsets topic" val OffsetsRetentionCheckIntervalMsDoc = "Frequency at which to check for stale offsets" val OffsetCommitTimeoutMsDoc = "Offset commit will be delayed until all replicas for the offsets topic receive the commit " + "or this timeout is reached. This is similar to the producer request timeout." diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index 6c47cbc..7654275 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -51,6 +51,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { override def setUp() { super.setUp() val config: Properties = createBrokerConfig(1, brokerPort) + config.setProperty(KafkaConfig.OffsetsTopicReplicationFactorProp, "1") config.setProperty(KafkaConfig.OffsetsRetentionCheckIntervalMsProp, retentionCheckInterval.toString) val logDirPath = config.getProperty("log.dir") logDir = new File(logDirPath) -- 1.7.12.4