From 662d69696b38e4f33dbc81a01d17954d67aef5df Mon Sep 17 00:00:00 2001 From: "Andrey L. Neporada" Date: Fri, 29 Jul 2016 15:28:07 +0300 Subject: [PATCH 01/54] KAFKA-2063: Add possibility to bound fetch response size --- .../clients/consumer/ConsumerConfig.java | 13 ++++ .../kafka/clients/consumer/KafkaConsumer.java | 1 + .../clients/consumer/internals/Fetcher.java | 5 +- .../kafka/common/protocol/Protocol.java | 23 +++++- .../kafka/common/requests/FetchRequest.java | 74 ++++++++++++++++--- .../kafka/common/requests/FetchResponse.java | 10 +-- .../clients/consumer/KafkaConsumerTest.java | 2 + .../consumer/internals/FetcherTest.java | 2 + .../src/main/scala/kafka/api/ApiVersion.scala | 11 ++- .../main/scala/kafka/api/FetchRequest.scala | 47 +++++++++--- .../scala/kafka/consumer/ConsumerConfig.scala | 6 +- .../scala/kafka/server/DelayedFetch.scala | 2 + .../main/scala/kafka/server/KafkaApis.scala | 1 + .../main/scala/kafka/server/KafkaConfig.scala | 7 +- .../kafka/server/ReplicaFetcherThread.scala | 8 +- .../scala/kafka/server/ReplicaManager.scala | 19 ++++- .../kafka/api/AuthorizerIntegrationTest.scala | 2 +- .../unit/kafka/server/KafkaConfigTest.scala | 1 + .../kafka/server/ReplicaManagerTest.scala | 3 + .../unit/kafka/server/SimpleFetchTest.scala | 4 +- 20 files changed, 198 insertions(+), 43 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 4ce908e38c37..b59c3c70ba66 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -112,6 +112,13 @@ public class ConsumerConfig extends AbstractConfig { public static final String FETCH_MIN_BYTES_CONFIG = "fetch.min.bytes"; private static final String FETCH_MIN_BYTES_DOC = "The minimum amount of data the server should return for a fetch request. If insufficient data is available the request will wait for that much data to accumulate before answering the request. The default setting of 1 byte means that fetch requests are answered as soon as a single byte of data is available or the fetch request times out waiting for data to arrive. Setting this to something greater than 1 will cause the server to wait for larger amounts of data to accumulate which can improve server throughput a bit at the cost of some additional latency."; + /** + * fetch.max.bytes + */ + public static final String FETCH_MAX_BYTES_CONFIG = "fetch.max.bytes"; + private static final String FETCH_MAX_BYTES_DOC = "The maximum amount of data the server should return for a fetch request. This is not an absolute maximum - if there is a single message which is larger than fetch.max.bytes, it will still be returned."; + public static final int DEFAULT_FETCH_MAX_BYTES = 50 * 1024 * 1024; + /** * fetch.max.wait.ms */ @@ -265,6 +272,12 @@ public class ConsumerConfig extends AbstractConfig { atLeast(0), Importance.HIGH, FETCH_MIN_BYTES_DOC) + .define(FETCH_MAX_BYTES_CONFIG, + Type.INT, + DEFAULT_FETCH_MAX_BYTES, + atLeast(0), + Importance.MEDIUM, + FETCH_MAX_BYTES_DOC) .define(FETCH_MAX_WAIT_MS_CONFIG, Type.INT, 500, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index ade42438f798..5bf2b7a366c2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -669,6 +669,7 @@ private KafkaConsumer(ConsumerConfig config, } this.fetcher = new Fetcher<>(this.client, config.getInt(ConsumerConfig.FETCH_MIN_BYTES_CONFIG), + config.getInt(ConsumerConfig.FETCH_MAX_BYTES_CONFIG), config.getInt(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG), config.getInt(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG), config.getInt(ConsumerConfig.MAX_POLL_RECORDS_CONFIG), diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 23a8511c2c36..92c9a319b230 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -79,6 +79,7 @@ public class Fetcher { private final ConsumerNetworkClient client; private final Time time; private final int minBytes; + private final int maxBytes; private final int maxWaitMs; private final int fetchSize; private final long retryBackoffMs; @@ -96,6 +97,7 @@ public class Fetcher { public Fetcher(ConsumerNetworkClient client, int minBytes, + int maxBytes, int maxWaitMs, int fetchSize, int maxPollRecords, @@ -113,6 +115,7 @@ public Fetcher(ConsumerNetworkClient client, this.metadata = metadata; this.subscriptions = subscriptions; this.minBytes = minBytes; + this.maxBytes = maxBytes; this.maxWaitMs = maxWaitMs; this.fetchSize = fetchSize; this.maxPollRecords = maxPollRecords; @@ -540,7 +543,7 @@ private Map createFetchRequests() { Map requests = new HashMap<>(); for (Map.Entry> entry : fetchable.entrySet()) { Node node = entry.getKey(); - FetchRequest fetch = new FetchRequest(this.maxWaitMs, this.minBytes, entry.getValue()); + FetchRequest fetch = new FetchRequest(this.maxWaitMs, this.minBytes, entry.getValue(), this.maxBytes); requests.put(node, fetch); } return requests; 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 313477fd3f8f..a88616aaf4f4 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 @@ -432,6 +432,24 @@ public class Protocol { // Only the version number is incremented to indicate the client support message format V1 which uses // relative offset and has timestamp. public static final Schema FETCH_REQUEST_V2 = FETCH_REQUEST_V1; + // FETCH_REQUEST_V3 added top level max_bytes field - the total size of partition data to accumulate in response. + // The partition ordering is now relevant - partitions will be processed in order they appear in request. + public static final Schema FETCH_REQUEST_V3 = new Schema(new Field("replica_id", + INT32, + "Broker id of the follower. For normal consumers, use -1."), + new Field("max_wait_time", + INT32, + "Maximum time in ms to wait for the response."), + new Field("min_bytes", + INT32, + "Minimum bytes to accumulate in the response."), + new Field("max_bytes", + INT32, + "Maximum bytes to accumulate in the response."), + new Field("topics", + new ArrayOf(FETCH_REQUEST_TOPIC_V0), + "Topics to fetch.")); + public static final Schema FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", INT32, "Topic partition id."), @@ -458,9 +476,10 @@ public class Protocol { // record set only includes messages of v0 (magic byte 0). In v2, record set can include messages of v0 and v1 // (magic byte 0 and 1). For details, see ByteBufferMessageSet. public static final Schema FETCH_RESPONSE_V2 = FETCH_RESPONSE_V1; + public static final Schema FETCH_RESPONSE_V3 = FETCH_RESPONSE_V2; - public static final Schema[] FETCH_REQUEST = new Schema[] {FETCH_REQUEST_V0, FETCH_REQUEST_V1, FETCH_REQUEST_V2}; - public static final Schema[] FETCH_RESPONSE = new Schema[] {FETCH_RESPONSE_V0, FETCH_RESPONSE_V1, FETCH_RESPONSE_V2}; + public static final Schema[] FETCH_REQUEST = new Schema[] {FETCH_REQUEST_V0, FETCH_REQUEST_V1, FETCH_REQUEST_V2, FETCH_REQUEST_V3}; + public static final Schema[] FETCH_RESPONSE = new Schema[] {FETCH_RESPONSE_V0, FETCH_RESPONSE_V1, FETCH_RESPONSE_V2, FETCH_RESPONSE_V3}; /* List groups api */ public static final Schema LIST_GROUPS_REQUEST_V0 = new Schema(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java index f8b7fe3a6da7..2b8827253f08 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -14,7 +14,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -24,7 +24,6 @@ import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.utils.CollectionUtils; public class FetchRequest extends AbstractRequest { @@ -33,6 +32,7 @@ public class FetchRequest extends AbstractRequest { private static final String REPLICA_ID_KEY_NAME = "replica_id"; private static final String MAX_WAIT_KEY_NAME = "max_wait_time"; private static final String MIN_BYTES_KEY_NAME = "min_bytes"; + private static final String RESPONSE_MAX_BYTES_KEY_NAME = "max_bytes"; private static final String TOPICS_KEY_NAME = "topics"; // topic level field names @@ -44,10 +44,14 @@ public class FetchRequest extends AbstractRequest { private static final String FETCH_OFFSET_KEY_NAME = "fetch_offset"; private static final String MAX_BYTES_KEY_NAME = "max_bytes"; + // default values for current version + public static final int DEFAULT_RESPONSE_MAX_BYTES = Integer.MAX_VALUE; + private final int replicaId; private final int maxWait; private final int minBytes; - private final Map fetchData; + private final int maxBytes; + private final LinkedHashMap fetchData; public static final class PartitionData { public final long offset; @@ -59,29 +63,66 @@ public PartitionData(long offset, int maxBytes) { } } + private static final class TopicAndPartitionData { + public final String topic; + public final LinkedHashMap partitions; + + public TopicAndPartitionData(String topic) { + this.topic = topic; + this.partitions = new LinkedHashMap<>(); + } + + public static final List groupByTopicOrdered(Map fetchData) { + List topics = new ArrayList<>(); + for (Map.Entry topicEntry : fetchData.entrySet()) { + String topic = topicEntry.getKey().topic(); + int partition = topicEntry.getKey().partition(); + PartitionData partitionData = topicEntry.getValue(); + if (topics.isEmpty() || topics.get(topics.size() - 1).topic != topic) + topics.add(new TopicAndPartitionData(topic)); + topics.get(topics.size() - 1).partitions.put(partition, partitionData); + } + return topics; + } + } + /** * Create a non-replica fetch request */ public FetchRequest(int maxWait, int minBytes, Map fetchData) { - this(CONSUMER_REPLICA_ID, maxWait, minBytes, fetchData); + this(ProtoUtils.latestVersion(ApiKeys.FETCH.id), CONSUMER_REPLICA_ID, maxWait, minBytes, fetchData, DEFAULT_RESPONSE_MAX_BYTES); } + public FetchRequest(int maxWait, int minBytes, Map fetchData, int maxBytes) { + this(ProtoUtils.latestVersion(ApiKeys.FETCH.id), CONSUMER_REPLICA_ID, maxWait, minBytes, fetchData, maxBytes); + } + /** * Create a replica fetch request */ public FetchRequest(int replicaId, int maxWait, int minBytes, Map fetchData) { - super(new Struct(CURRENT_SCHEMA)); - Map> topicsData = CollectionUtils.groupDataByTopic(fetchData); + this(ProtoUtils.latestVersion(ApiKeys.FETCH.id), replicaId, maxWait, minBytes, fetchData, DEFAULT_RESPONSE_MAX_BYTES); + } + + public FetchRequest(int replicaId, int maxWait, int minBytes, Map fetchData, int maxBytes) { + this(ProtoUtils.latestVersion(ApiKeys.FETCH.id), replicaId, maxWait, minBytes, fetchData, maxBytes); + } + + public FetchRequest(int version, int replicaId, int maxWait, int minBytes, Map fetchData, int maxBytes) { + super(new Struct(ProtoUtils.requestSchema(ApiKeys.FETCH.id, version))); + List topicsData = TopicAndPartitionData.groupByTopicOrdered(fetchData); struct.set(REPLICA_ID_KEY_NAME, replicaId); struct.set(MAX_WAIT_KEY_NAME, maxWait); struct.set(MIN_BYTES_KEY_NAME, minBytes); + if (version >= 3) + struct.set(RESPONSE_MAX_BYTES_KEY_NAME, maxBytes); List topicArray = new ArrayList(); - for (Map.Entry> topicEntry : topicsData.entrySet()) { + for (TopicAndPartitionData topicEntry : topicsData) { Struct topicData = struct.instance(TOPICS_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); + topicData.set(TOPIC_KEY_NAME, topicEntry.topic); List partitionArray = new ArrayList(); - for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { + for (Map.Entry partitionEntry : topicEntry.partitions.entrySet()) { PartitionData fetchPartitionData = partitionEntry.getValue(); Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); @@ -96,7 +137,8 @@ public FetchRequest(int replicaId, int maxWait, int minBytes, Map(fetchData); } public FetchRequest(Struct struct) { @@ -104,7 +146,11 @@ public FetchRequest(Struct struct) { replicaId = struct.getInt(REPLICA_ID_KEY_NAME); maxWait = struct.getInt(MAX_WAIT_KEY_NAME); minBytes = struct.getInt(MIN_BYTES_KEY_NAME); - fetchData = new HashMap(); + if (struct.hasField(RESPONSE_MAX_BYTES_KEY_NAME)) + maxBytes = struct.getInt(RESPONSE_MAX_BYTES_KEY_NAME); + else + maxBytes = DEFAULT_RESPONSE_MAX_BYTES; + fetchData = new LinkedHashMap(); for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) { Struct topicResponse = (Struct) topicResponseObj; String topic = topicResponse.getString(TOPIC_KEY_NAME); @@ -121,7 +167,7 @@ public FetchRequest(Struct struct) { @Override public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { - Map responseData = new HashMap(); + Map responseData = new LinkedHashMap(); for (Map.Entry entry: fetchData.entrySet()) { FetchResponse.PartitionData partitionResponse = new FetchResponse.PartitionData(Errors.forException(e).code(), @@ -153,6 +199,10 @@ public int minBytes() { return minBytes; } + public int maxBytes() { + return maxBytes; + } + public Map fetchData() { return fetchData; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java index f28472f3a63f..5254f6bbe86a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java @@ -25,7 +25,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -65,7 +65,7 @@ public class FetchResponse extends AbstractRequestResponse { public static final long INVALID_HIGHWATERMARK = -1L; public static final ByteBuffer EMPTY_RECORD_SET = ByteBuffer.allocate(0); - private final Map responseData; + private final LinkedHashMap responseData; private final int throttleTime; public static final class PartitionData { @@ -87,7 +87,7 @@ public PartitionData(short errorCode, long highWatermark, ByteBuffer recordSet) public FetchResponse(Map responseData) { super(new Struct(ProtoUtils.responseSchema(ApiKeys.FETCH.id, 0))); initCommonFields(responseData); - this.responseData = responseData; + this.responseData = new LinkedHashMap(responseData); this.throttleTime = DEFAULT_THROTTLE_TIME; } @@ -100,13 +100,13 @@ public FetchResponse(Map responseData, int thrott super(new Struct(CURRENT_SCHEMA)); initCommonFields(responseData); struct.set(THROTTLE_TIME_KEY_NAME, throttleTime); - this.responseData = responseData; + this.responseData = new LinkedHashMap(responseData); this.throttleTime = throttleTime; } public FetchResponse(Struct struct) { super(struct); - responseData = new HashMap(); + responseData = new LinkedHashMap(); for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { Struct topicResponse = (Struct) topicResponseObj; String topic = topicResponse.getString(TOPIC_KEY_NAME); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index dbe3d67a717b..dd8753f0fdf5 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -1037,6 +1037,7 @@ private KafkaConsumer newConsumer(Time time, long requestTimeoutMs = 30000; boolean excludeInternalTopics = true; int minBytes = 1; + int maxBytes = Integer.MAX_VALUE; int maxWaitMs = 500; int fetchSize = 1024 * 1024; int maxPollRecords = Integer.MAX_VALUE; @@ -1075,6 +1076,7 @@ private KafkaConsumer newConsumer(Time time, Fetcher fetcher = new Fetcher<>( consumerClient, minBytes, + maxBytes, maxWaitMs, fetchSize, maxPollRecords, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 90ddcb678b28..4d93ead1aad8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -80,6 +80,7 @@ public class FetcherTest { private final String metricGroup = "consumer" + groupId + "-fetch-manager-metrics"; private TopicPartition tp = new TopicPartition(topicName, 0); private int minBytes = 1; + private int maxBytes = Integer.MAX_VALUE; private int maxWaitMs = 0; private int fetchSize = 1000; private long retryBackoffMs = 100; @@ -721,6 +722,7 @@ private Fetcher createFetcher(SubscriptionState subscriptions, int maxPollRecords) { return new Fetcher<>(consumerClient, minBytes, + maxBytes, maxWaitMs, fetchSize, maxPollRecords, diff --git a/core/src/main/scala/kafka/api/ApiVersion.scala b/core/src/main/scala/kafka/api/ApiVersion.scala index d955225a351f..cdd418dad5a5 100644 --- a/core/src/main/scala/kafka/api/ApiVersion.scala +++ b/core/src/main/scala/kafka/api/ApiVersion.scala @@ -54,7 +54,10 @@ object ApiVersion { "0.10.0" -> KAFKA_0_10_0_IV1, // introduced for JoinGroup protocol change in KIP-62 - "0.10.1-IV0" -> KAFKA_0_10_1_IV0 + "0.10.1-IV0" -> KAFKA_0_10_1_IV0, + // 0.10.1-IV1 is introduced for KIP-74(fetch response size limit). + "0.10.1-IV1" -> KAFKA_0_10_1_IV1, + "0.10.1" -> KAFKA_0_10_1_IV1 ) private val versionPattern = "\\.".r @@ -120,3 +123,9 @@ case object KAFKA_0_10_1_IV0 extends ApiVersion { val messageFormatVersion: Byte = Message.MagicValue_V1 val id: Int = 6 } + +case object KAFKA_0_10_1_IV1 extends ApiVersion { + val version: String = "0.10.1-IV1" + val messageFormatVersion: Byte = Message.MagicValue_V1 + val id: Int = 7 +} diff --git a/core/src/main/scala/kafka/api/FetchRequest.scala b/core/src/main/scala/kafka/api/FetchRequest.scala index f74bd1c983d3..adcaf401a55c 100644 --- a/core/src/main/scala/kafka/api/FetchRequest.scala +++ b/core/src/main/scala/kafka/api/FetchRequest.scala @@ -28,14 +28,15 @@ import java.util.concurrent.atomic.AtomicInteger import java.nio.ByteBuffer import org.apache.kafka.common.protocol.{ApiKeys, Errors} -import scala.collection.immutable.Map +import scala.collection.immutable.{ListMap, Map} case class PartitionFetchInfo(offset: Long, fetchSize: Int) object FetchRequest { - val CurrentVersion = 2.shortValue + val CurrentVersion = 3.shortValue val DefaultMaxWait = 0 val DefaultMinBytes = 0 + val DefaultMaxBytes = Int.MaxValue val DefaultCorrelationId = 0 def readFrom(buffer: ByteBuffer): FetchRequest = { @@ -45,6 +46,7 @@ object FetchRequest { val replicaId = buffer.getInt val maxWait = buffer.getInt val minBytes = buffer.getInt + val maxBytes = if (versionId < 3) DefaultMaxBytes else buffer.getInt val topicCount = buffer.getInt val pairs = (1 to topicCount).flatMap(_ => { val topic = readShortString(buffer) @@ -56,7 +58,7 @@ object FetchRequest { (TopicAndPartition(topic, partitionId), PartitionFetchInfo(offset, fetchSize)) }) }) - FetchRequest(versionId, correlationId, clientId, replicaId, maxWait, minBytes, Map(pairs:_*)) + FetchRequest(versionId, correlationId, clientId, replicaId, maxWait, minBytes, maxBytes, ListMap(pairs:_*)) } } @@ -66,13 +68,26 @@ case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion, replicaId: Int = Request.OrdinaryConsumerId, maxWait: Int = FetchRequest.DefaultMaxWait, minBytes: Int = FetchRequest.DefaultMinBytes, + maxBytes: Int = FetchRequest.DefaultMaxBytes, requestInfo: Map[TopicAndPartition, PartitionFetchInfo]) extends RequestOrResponse(Some(ApiKeys.FETCH.id)) { /** - * Partitions the request info into a map of maps (one for each topic). - */ - lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic) + * Partitions the request info into a list of lists (one for each topic) while preserving request info ordering + */ + private type PartitionInfoList = Vector[(Int, PartitionFetchInfo)] + private type TopicInfoEntry = (String, PartitionInfoList) + private val requestInfoGroupedByTopic : Vector[TopicInfoEntry] = { + requestInfo.foldLeft(Vector.empty[TopicInfoEntry])((folded, currTopicAndPartition) => { + val (TopicAndPartition(topic, partition), partitionFetchInfo) = currTopicAndPartition + if (folded.isEmpty || folded.last._1 != topic) { + folded :+ (topic, Vector((partition, partitionFetchInfo))) + } else { + val updatedTail = (folded.last._1, folded.last._2 :+ (partition, partitionFetchInfo)) + folded.dropRight(1) :+ updatedTail + } + }) + } /** * Public constructor for the clients @@ -81,13 +96,15 @@ case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion, clientId: String, maxWait: Int, minBytes: Int, + maxBytes: Int, requestInfo: Map[TopicAndPartition, PartitionFetchInfo]) { this(versionId = FetchRequest.CurrentVersion, correlationId = correlationId, clientId = clientId, replicaId = Request.OrdinaryConsumerId, maxWait = maxWait, - minBytes= minBytes, + minBytes = minBytes, + maxBytes = maxBytes, requestInfo = requestInfo) } @@ -98,13 +115,15 @@ case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion, buffer.putInt(replicaId) buffer.putInt(maxWait) buffer.putInt(minBytes) + if (versionId >= 3) + buffer.putInt(maxBytes) buffer.putInt(requestInfoGroupedByTopic.size) // topic count requestInfoGroupedByTopic.foreach { case (topic, partitionFetchInfos) => writeShortString(buffer, topic) buffer.putInt(partitionFetchInfos.size) // partition count partitionFetchInfos.foreach { - case (TopicAndPartition(_, partition), PartitionFetchInfo(offset, fetchSize)) => + case (partition, PartitionFetchInfo(offset, fetchSize)) => buffer.putInt(partition) buffer.putLong(offset) buffer.putInt(fetchSize) @@ -119,6 +138,7 @@ case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion, 4 + /* replicaId */ 4 + /* maxWait */ 4 + /* minBytes */ + (if (versionId >= 3) 4 /* maxBytes */ else 0) + 4 + /* topic count */ requestInfoGroupedByTopic.foldLeft(0)((foldedTopics, currTopic) => { val (topic, partitionFetchInfos) = currTopic @@ -165,6 +185,7 @@ case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion, fetchRequest.append("; ReplicaId: " + replicaId) fetchRequest.append("; MaxWait: " + maxWait + " ms") fetchRequest.append("; MinBytes: " + minBytes + " bytes") + fetchRequest.append("; MaxBytes:" + maxBytes + " bytes") if(details) fetchRequest.append("; RequestInfo: " + requestInfo.mkString(",")) fetchRequest.toString() @@ -179,7 +200,8 @@ class FetchRequestBuilder() { private var replicaId = Request.OrdinaryConsumerId private var maxWait = FetchRequest.DefaultMaxWait private var minBytes = FetchRequest.DefaultMinBytes - private val requestMap = new collection.mutable.HashMap[TopicAndPartition, PartitionFetchInfo] + private var maxBytes = FetchRequest.DefaultMaxBytes + private val requestMap = new collection.mutable.LinkedHashMap[TopicAndPartition, PartitionFetchInfo] def addFetch(topic: String, partition: Int, offset: Long, fetchSize: Int) = { requestMap.put(TopicAndPartition(topic, partition), PartitionFetchInfo(offset, fetchSize)) @@ -209,13 +231,18 @@ class FetchRequestBuilder() { this } + def maxBytes(maxBytes: Int): FetchRequestBuilder = { + this.maxBytes = maxBytes + this + } + def requestVersion(versionId: Short): FetchRequestBuilder = { this.versionId = versionId this } def build() = { - val fetchRequest = FetchRequest(versionId, correlationId.getAndIncrement, clientId, replicaId, maxWait, minBytes, requestMap.toMap) + val fetchRequest = FetchRequest(versionId, correlationId.getAndIncrement, clientId, replicaId, maxWait, minBytes, maxBytes, requestMap.toMap) requestMap.clear() fetchRequest } diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala index e1c792d01c11..fd8983c03cc3 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala @@ -38,6 +38,7 @@ object ConsumerConfig extends Config { val AutoOffsetReset = OffsetRequest.LargestTimeString val ConsumerTimeoutMs = -1 val MinFetchBytes = 1 + val MaxFetchBytes = 50 * 1024 * 1024 val MaxFetchWaitMs = 100 val MirrorTopicsWhitelist = "" val MirrorTopicsBlacklist = "" @@ -119,7 +120,7 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig( /** the socket receive buffer for network requests */ val socketReceiveBufferBytes = props.getInt("socket.receive.buffer.bytes", SocketBufferSize) - /** the number of bytes of messages to attempt to fetch */ + /** the number of bytes of messages to attempt to fetch from each partition */ val fetchMessageMaxBytes = props.getInt("fetch.message.max.bytes", FetchSize) /** the number threads used to fetch data */ @@ -140,6 +141,9 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig( /** the minimum amount of data the server should return for a fetch request. If insufficient data is available the request will block */ val fetchMinBytes = props.getInt("fetch.min.bytes", MinFetchBytes) + /** the maximum amount of data the server should return for a fetch request */ + val fetchMaxBytes = props.getInt("fetch.max.bytes", MaxFetchBytes) + /** the maximum amount of time the server will block before answering the fetch request if there isn't sufficient data to immediately satisfy fetch.min.bytes */ val fetchWaitMaxMs = props.getInt("fetch.wait.max.ms", MaxFetchWaitMs) require(fetchWaitMaxMs <= socketTimeoutMs, "socket.timeout.ms should always be at least fetch.wait.max.ms" + diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index a0ff00de0d43..e8e910300efd 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -37,6 +37,7 @@ case class FetchPartitionStatus(startOffsetMetadata: LogOffsetMetadata, fetchInf * The fetch metadata maintained by the delayed fetch operation */ case class FetchMetadata(fetchMinBytes: Int, + fetchMaxBytes: Int, fetchOnlyLeader: Boolean, fetchOnlyCommitted: Boolean, isFromFollower: Boolean, @@ -130,6 +131,7 @@ class DelayedFetch(delayMs: Long, override def onComplete() { val logReadResults = replicaManager.readFromLocalLog(fetchMetadata.fetchOnlyLeader, fetchMetadata.fetchOnlyCommitted, + fetchMetadata.fetchMaxBytes, fetchMetadata.fetchPartitionStatus.mapValues(status => status.fetchInfo)) val fetchPartitionData = logReadResults.mapValues(result => diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 0a5258e9fa5b..bc48ce368651 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -505,6 +505,7 @@ class KafkaApis(val requestChannel: RequestChannel, fetchRequest.maxWait.toLong, fetchRequest.replicaId, fetchRequest.minBytes, + fetchRequest.maxBytes, authorizedRequestInfo, sendResponseCallback) } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 531ee625c4d1..84bec8006331 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -115,6 +115,7 @@ object Defaults { val ReplicaFetchMaxBytes = ConsumerConfig.FetchSize val ReplicaFetchWaitMaxMs = 500 val ReplicaFetchMinBytes = 1 + val ReplicaFetchResponseMaxBytes = 10 * 1024 * 1024 val NumReplicaFetchers = 1 val ReplicaFetchBackoffMs = 1000 val ReplicaHighWatermarkCheckpointIntervalMs = 5000L @@ -280,6 +281,7 @@ object KafkaConfig { val ReplicaFetchMaxBytesProp = "replica.fetch.max.bytes" val ReplicaFetchWaitMaxMsProp = "replica.fetch.wait.max.ms" val ReplicaFetchMinBytesProp = "replica.fetch.min.bytes" + val ReplicaFetchResponseMaxBytesProp = "replica.fetch.response.max.bytes" val ReplicaFetchBackoffMsProp = "replica.fetch.backoff.ms" val NumReplicaFetchersProp = "num.replica.fetchers" val ReplicaHighWatermarkCheckpointIntervalMsProp = "replica.high.watermark.checkpoint.interval.ms" @@ -475,10 +477,11 @@ object KafkaConfig { " the leader will remove the follower from isr" val ReplicaSocketTimeoutMsDoc = "The socket timeout for network requests. Its value should be at least replica.fetch.wait.max.ms" val ReplicaSocketReceiveBufferBytesDoc = "The socket receive buffer for network requests" - val ReplicaFetchMaxBytesDoc = "The number of bytes of messages to attempt to fetch" + val ReplicaFetchMaxBytesDoc = "The number of bytes of messages to attempt to fetch for each partition" val ReplicaFetchWaitMaxMsDoc = "max wait time for each fetcher request issued by follower replicas. This value should always be less than the " + "replica.lag.time.max.ms at all times to prevent frequent shrinking of ISR for low throughput topics" val ReplicaFetchMinBytesDoc = "Minimum bytes expected for each fetch response. If not enough bytes, wait up to replicaMaxWaitTimeMs" + val ReplicaFetchResponseMaxBytesDoc = "Maximum bytes expected for entire fetch response." val NumReplicaFetchersDoc = "Number of fetcher threads used to replicate messages from a source broker. " + "Increasing this value can increase the degree of I/O parallelism in the follower broker." val ReplicaFetchBackoffMsDoc = "The amount of time to sleep when fetch partition error occurs." @@ -662,6 +665,7 @@ object KafkaConfig { .define(ReplicaFetchWaitMaxMsProp, INT, Defaults.ReplicaFetchWaitMaxMs, HIGH, ReplicaFetchWaitMaxMsDoc) .define(ReplicaFetchBackoffMsProp, INT, Defaults.ReplicaFetchBackoffMs, atLeast(0), MEDIUM, ReplicaFetchBackoffMsDoc) .define(ReplicaFetchMinBytesProp, INT, Defaults.ReplicaFetchMinBytes, HIGH, ReplicaFetchMinBytesDoc) + .define(ReplicaFetchResponseMaxBytesProp, INT, Defaults.ReplicaFetchResponseMaxBytes, atLeast(0), MEDIUM, ReplicaFetchResponseMaxBytesDoc) .define(NumReplicaFetchersProp, INT, Defaults.NumReplicaFetchers, HIGH, NumReplicaFetchersDoc) .define(ReplicaHighWatermarkCheckpointIntervalMsProp, LONG, Defaults.ReplicaHighWatermarkCheckpointIntervalMs, HIGH, ReplicaHighWatermarkCheckpointIntervalMsDoc) .define(FetchPurgatoryPurgeIntervalRequestsProp, INT, Defaults.FetchPurgatoryPurgeIntervalRequests, MEDIUM, FetchPurgatoryPurgeIntervalRequestsDoc) @@ -866,6 +870,7 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean) extends Abstra val replicaFetchMaxBytes = getInt(KafkaConfig.ReplicaFetchMaxBytesProp) val replicaFetchWaitMaxMs = getInt(KafkaConfig.ReplicaFetchWaitMaxMsProp) val replicaFetchMinBytes = getInt(KafkaConfig.ReplicaFetchMinBytesProp) + val replicaFetchResponseMaxBytes = getInt(KafkaConfig.ReplicaFetchResponseMaxBytesProp) val replicaFetchBackoffMs = getInt(KafkaConfig.ReplicaFetchBackoffMsProp) val numReplicaFetchers = getInt(KafkaConfig.NumReplicaFetchersProp) val replicaHighWatermarkCheckpointIntervalMs = getLong(KafkaConfig.ReplicaHighWatermarkCheckpointIntervalMsProp) diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index ef602e4ff3f8..df44d07e7d68 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -23,7 +23,7 @@ import kafka.admin.AdminUtils import kafka.cluster.BrokerEndPoint import kafka.log.LogConfig import kafka.message.ByteBufferMessageSet -import kafka.api.{KAFKA_0_10_0_IV0, KAFKA_0_9_0} +import kafka.api.{KAFKA_0_9_0, KAFKA_0_10_0_IV0, KAFKA_0_10_1_IV1} import kafka.common.{KafkaStorageException, TopicAndPartition} import ReplicaFetcherThread._ import org.apache.kafka.clients.{ManualMetadataUpdater, NetworkClient, ClientRequest, ClientResponse} @@ -56,13 +56,15 @@ class ReplicaFetcherThread(name: String, type PD = PartitionData private val fetchRequestVersion: Short = - if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_10_0_IV0) 2 + if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_10_1_IV1) 3 + else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_10_0_IV0) 2 else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_9_0) 1 else 0 private val socketTimeout: Int = brokerConfig.replicaSocketTimeoutMs private val replicaId = brokerConfig.brokerId private val maxWait = brokerConfig.replicaFetchWaitMaxMs private val minBytes = brokerConfig.replicaFetchMinBytes + private val maxBytes = brokerConfig.replicaFetchResponseMaxBytes private val fetchSize = brokerConfig.replicaFetchMaxBytes private def clientId = name @@ -275,7 +277,7 @@ class ReplicaFetcherThread(name: String, requestMap(new TopicPartition(topic, partition)) = new JFetchRequest.PartitionData(partitionFetchState.offset, fetchSize) } - new FetchRequest(new JFetchRequest(replicaId, maxWait, minBytes, requestMap.asJava)) + new FetchRequest(new JFetchRequest(fetchRequestVersion, replicaId, maxWait, minBytes, requestMap.asJava, maxBytes)) } } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index f1cc694cd607..43cca45ef157 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -457,6 +457,7 @@ class ReplicaManager(val config: KafkaConfig, def fetchMessages(timeout: Long, replicaId: Int, fetchMinBytes: Int, + fetchMaxBytes: Int, fetchInfo: immutable.Map[TopicAndPartition, PartitionFetchInfo], responseCallback: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) { val isFromFollower = replicaId >= 0 @@ -464,7 +465,7 @@ class ReplicaManager(val config: KafkaConfig, val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(replicaId) // read from local logs - val logReadResults = readFromLocalLog(fetchOnlyFromLeader, fetchOnlyCommitted, fetchInfo) + val logReadResults = readFromLocalLog(fetchOnlyFromLeader, fetchOnlyCommitted, fetchMaxBytes, fetchInfo) // if the fetch comes from the follower, // update its corresponding log end offset @@ -489,7 +490,7 @@ class ReplicaManager(val config: KafkaConfig, val fetchPartitionStatus = logReadResults.map { case (topicAndPartition, result) => (topicAndPartition, FetchPartitionStatus(result.info.fetchOffsetMetadata, fetchInfo.get(topicAndPartition).get)) } - val fetchMetadata = FetchMetadata(fetchMinBytes, fetchOnlyFromLeader, fetchOnlyCommitted, isFromFollower, fetchPartitionStatus) + val fetchMetadata = FetchMetadata(fetchMinBytes, fetchMaxBytes, fetchOnlyFromLeader, fetchOnlyCommitted, isFromFollower, fetchPartitionStatus) val delayedFetch = new DelayedFetch(timeout, fetchMetadata, this, responseCallback) // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation @@ -507,15 +508,17 @@ class ReplicaManager(val config: KafkaConfig, */ def readFromLocalLog(fetchOnlyFromLeader: Boolean, readOnlyCommitted: Boolean, + fetchMaxBytes: Int, readPartitionInfo: Map[TopicAndPartition, PartitionFetchInfo]): Map[TopicAndPartition, LogReadResult] = { + var limitBytes = fetchMaxBytes readPartitionInfo.map { case (TopicAndPartition(topic, partition), PartitionFetchInfo(offset, fetchSize)) => BrokerTopicStats.getBrokerTopicStats(topic).totalFetchRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats().totalFetchRequestRate.mark() val partitionDataAndOffsetInfo = try { - trace("Fetching log segment for topic %s, partition %d, offset %d, size %d".format(topic, partition, offset, fetchSize)) + trace("Fetching log segment for topic %s, partition %d, offset %d, partition fetch size %d, remaining response limit %d".format(topic, partition, offset, fetchSize, limitBytes)) // decide whether to only fetch from leader val localReplica = if (fetchOnlyFromLeader) @@ -538,13 +541,21 @@ class ReplicaManager(val config: KafkaConfig, val initialLogEndOffset = localReplica.logEndOffset val logReadInfo = localReplica.log match { case Some(log) => - val adjustedFetchSize = if (Topic.isInternal(topic) && !readOnlyCommitted) Math.max(fetchSize, log.config.maxMessageSize) else fetchSize + val adjustedFetchSize = + if (Topic.isInternal(topic) && !readOnlyCommitted) + Math.max(fetchSize, log.config.maxMessageSize) + else if (limitBytes > 0) + fetchSize + else + 0 log.read(offset, adjustedFetchSize, maxOffsetOpt) case None => error("Leader for partition [%s,%d] does not have a local log".format(topic, partition)) FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty) } + limitBytes = math.max(0, limitBytes - logReadInfo.messageSet.sizeInBytes) + val readToEndOfLog = initialLogEndOffset.messageOffset - logReadInfo.fetchOffsetMetadata.messageOffset <= 0 LogReadResult(logReadInfo, localReplica.highWatermark.messageOffset, fetchSize, readToEndOfLog, None) diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 1a5f187eaba5..cc2b3c3ec6b0 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -176,7 +176,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } private def createFetchRequest = { - new requests.FetchRequest(5000, 100, Map(tp -> new requests.FetchRequest.PartitionData(0, 100)).asJava) + new requests.FetchRequest(5000, 100, Int.MaxValue, Map(tp -> new requests.FetchRequest.PartitionData(0, 100)).asJava) } private def createListOffsetsRequest = { diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 9ee3d3289261..042dabfc8e6e 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -498,6 +498,7 @@ class KafkaConfigTest { case KafkaConfig.ReplicaFetchMaxBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") case KafkaConfig.ReplicaFetchWaitMaxMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") case KafkaConfig.ReplicaFetchMinBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") + case KafkaConfig.ReplicaFetchResponseMaxBytesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") case KafkaConfig.NumReplicaFetchersProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") case KafkaConfig.ReplicaHighWatermarkCheckpointIntervalMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") case KafkaConfig.FetchPurgatoryPurgeIntervalRequestsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index bfb66b99201e..61f7b501f1dd 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -171,6 +171,7 @@ class ReplicaManagerTest { timeout = 1000, replicaId = -1, fetchMinBytes = 100000, + fetchMaxBytes = Int.MaxValue, fetchInfo = collection.immutable.Map(new TopicAndPartition(topic, 0) -> new PartitionFetchInfo(0, 100000)), responseCallback = fetchCallback) @@ -240,6 +241,7 @@ class ReplicaManagerTest { timeout = 1000, replicaId = 1, fetchMinBytes = 0, + fetchMaxBytes = Int.MaxValue, fetchInfo = collection.immutable.Map(new TopicAndPartition(topic, 0) -> new PartitionFetchInfo(1, 100000)), responseCallback = fetchCallback) @@ -254,6 +256,7 @@ class ReplicaManagerTest { timeout = 1000, replicaId = -1, fetchMinBytes = 0, + fetchMaxBytes = Int.MaxValue, fetchInfo = collection.immutable.Map(new TopicAndPartition(topic, 0) -> new PartitionFetchInfo(1, 100000)), responseCallback = fetchCallback) diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index 774169832e97..0e1f68d48a71 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -148,9 +148,9 @@ class SimpleFetchTest { val initialAllTopicsCount = BrokerTopicStats.getBrokerAllTopicsStats().totalFetchRequestRate.count() assertEquals("Reading committed data should return messages only up to high watermark", messagesToHW, - replicaManager.readFromLocalLog(true, true, fetchInfo).get(topicAndPartition).get.info.messageSet.head.message) + replicaManager.readFromLocalLog(true, true, Int.MaxValue, fetchInfo).get(topicAndPartition).get.info.messageSet.head.message) assertEquals("Reading any data can return messages up to the end of the log", messagesToLEO, - replicaManager.readFromLocalLog(true, false, fetchInfo).get(topicAndPartition).get.info.messageSet.head.message) + replicaManager.readFromLocalLog(true, false, Int.MaxValue, fetchInfo).get(topicAndPartition).get.info.messageSet.head.message) assertEquals("Counts should increment after fetch", initialTopicCount+2, BrokerTopicStats.getBrokerTopicStats(topic).totalFetchRequestRate.count()) assertEquals("Counts should increment after fetch", initialAllTopicsCount+2, BrokerTopicStats.getBrokerAllTopicsStats().totalFetchRequestRate.count()) From 283557027260c2d139e66eb3a9e191f6e9f726ab Mon Sep 17 00:00:00 2001 From: "Andrey L. Neporada" Date: Mon, 12 Sep 2016 14:43:01 +0300 Subject: [PATCH 02/54] LOGBROKER-2063: Change interfaces of FetchRequest/Response to Vector instead of Map --- .../main/scala/kafka/api/FetchRequest.scala | 15 ++++---- .../main/scala/kafka/api/FetchResponse.scala | 36 ++++++++++++++----- .../consumer/ConsumerFetcherThread.scala | 6 ++-- .../scala/kafka/javaapi/FetchRequest.scala | 6 ++-- .../kafka/server/AbstractFetcherThread.scala | 4 +-- .../main/scala/kafka/server/KafkaApis.scala | 10 +++--- .../kafka/server/ReplicaFetcherThread.scala | 2 +- .../RequestResponseSerializationTest.scala | 8 ++--- .../kafka/integration/PrimitiveApiTest.scala | 6 ++-- .../server/AbstractFetcherThreadTest.scala | 4 +-- 10 files changed, 58 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/kafka/api/FetchRequest.scala b/core/src/main/scala/kafka/api/FetchRequest.scala index adcaf401a55c..94598f124279 100644 --- a/core/src/main/scala/kafka/api/FetchRequest.scala +++ b/core/src/main/scala/kafka/api/FetchRequest.scala @@ -58,7 +58,7 @@ object FetchRequest { (TopicAndPartition(topic, partitionId), PartitionFetchInfo(offset, fetchSize)) }) }) - FetchRequest(versionId, correlationId, clientId, replicaId, maxWait, minBytes, maxBytes, ListMap(pairs:_*)) + FetchRequest(versionId, correlationId, clientId, replicaId, maxWait, minBytes, maxBytes, Vector(pairs:_*)) } } @@ -69,9 +69,10 @@ case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion, maxWait: Int = FetchRequest.DefaultMaxWait, minBytes: Int = FetchRequest.DefaultMinBytes, maxBytes: Int = FetchRequest.DefaultMaxBytes, - requestInfo: Map[TopicAndPartition, PartitionFetchInfo]) + requestInfo: Vector[(TopicAndPartition, PartitionFetchInfo)]) extends RequestOrResponse(Some(ApiKeys.FETCH.id)) { + lazy val requestInfoMapByTopic = requestInfo.toMap /** * Partitions the request info into a list of lists (one for each topic) while preserving request info ordering */ @@ -97,7 +98,7 @@ case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion, maxWait: Int, minBytes: Int, maxBytes: Int, - requestInfo: Map[TopicAndPartition, PartitionFetchInfo]) { + requestInfo: Seq[(TopicAndPartition, PartitionFetchInfo)]) { this(versionId = FetchRequest.CurrentVersion, correlationId = correlationId, clientId = clientId, @@ -105,7 +106,7 @@ case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion, maxWait = maxWait, minBytes = minBytes, maxBytes = maxBytes, - requestInfo = requestInfo) + requestInfo = requestInfo.toVector) } def writeTo(buffer: ByteBuffer) { @@ -201,10 +202,10 @@ class FetchRequestBuilder() { private var maxWait = FetchRequest.DefaultMaxWait private var minBytes = FetchRequest.DefaultMinBytes private var maxBytes = FetchRequest.DefaultMaxBytes - private val requestMap = new collection.mutable.LinkedHashMap[TopicAndPartition, PartitionFetchInfo] + private val requestMap = new collection.mutable.ArrayBuffer[(TopicAndPartition, PartitionFetchInfo)] def addFetch(topic: String, partition: Int, offset: Long, fetchSize: Int) = { - requestMap.put(TopicAndPartition(topic, partition), PartitionFetchInfo(offset, fetchSize)) + requestMap.append((TopicAndPartition(topic, partition), PartitionFetchInfo(offset, fetchSize))) this } @@ -242,7 +243,7 @@ class FetchRequestBuilder() { } def build() = { - val fetchRequest = FetchRequest(versionId, correlationId.getAndIncrement, clientId, replicaId, maxWait, minBytes, maxBytes, requestMap.toMap) + val fetchRequest = FetchRequest(versionId, correlationId.getAndIncrement, clientId, replicaId, maxWait, minBytes, maxBytes, requestMap.toVector) requestMap.clear() fetchRequest } diff --git a/core/src/main/scala/kafka/api/FetchResponse.scala b/core/src/main/scala/kafka/api/FetchResponse.scala index 1066d7f9b4bb..a039dae882b3 100644 --- a/core/src/main/scala/kafka/api/FetchResponse.scala +++ b/core/src/main/scala/kafka/api/FetchResponse.scala @@ -100,7 +100,7 @@ object TopicData { val partitionData = FetchResponsePartitionData.readFrom(buffer) (partitionId, partitionData) }) - TopicData(topic, Map(topicPartitionDataPairs:_*)) + TopicData(topic, Vector(topicPartitionDataPairs:_*)) } def headerSize(topic: String) = @@ -108,9 +108,11 @@ object TopicData { 4 /* partition count */ } -case class TopicData(topic: String, partitionData: Map[Int, FetchResponsePartitionData]) { +case class TopicData(topic: String, partitionData: Vector[(Int, FetchResponsePartitionData)]) { val sizeInBytes = - TopicData.headerSize(topic) + partitionData.values.foldLeft(0)(_ + _.sizeInBytes + 4) + TopicData.headerSize(topic) + partitionData.foldLeft(0)((folded, data) => { + folded + data._2.sizeInBytes + 4 + } /*_ + _.sizeInBytes + 4*/) val headerSize = TopicData.headerSize(topic) } @@ -173,7 +175,22 @@ object FetchResponse { (TopicAndPartition(topicData.topic, partitionId), partitionData) } }) - FetchResponse(correlationId, Map(pairs:_*), requestVersion, throttleTime) + FetchResponse(correlationId, Vector(pairs:_*), requestVersion, throttleTime) + } + + type FetchResponseEntry = (TopicAndPartition, FetchResponsePartitionData) + + def groupByTopicOrdered(data: Seq[(TopicAndPartition, FetchResponsePartitionData)]): Vector[(String, Vector[FetchResponseEntry])] = { + data.foldLeft(Vector.empty[(String, Vector[FetchResponseEntry])])((folded, currFetchResponse) => { + val (topicPartition, fetchResponsePartitionData) = currFetchResponse + val TopicAndPartition(topic, partition) = topicPartition + if (folded.isEmpty || folded.last._1 != topic) { + folded :+ (topic, Vector((topicPartition, fetchResponsePartitionData))) + } else { + val updatedTail = (folded.last._1, folded.last._2 :+ (topicPartition, fetchResponsePartitionData)) + folded.dropRight(1) :+ updatedTail + } + }) } // Returns the size of the response header @@ -185,7 +202,7 @@ object FetchResponse { } // Returns the size of entire fetch response in bytes (including the header size) - def responseSize(dataGroupedByTopic: Map[String, Map[TopicAndPartition, FetchResponsePartitionData]], + def responseSize(dataGroupedByTopic: Vector[(String, Vector[FetchResponseEntry])], requestVersion: Int): Int = { headerSize(requestVersion) + dataGroupedByTopic.foldLeft(0) { case (folded, (topic, partitionDataMap)) => @@ -198,7 +215,7 @@ object FetchResponse { } case class FetchResponse(correlationId: Int, - data: Map[TopicAndPartition, FetchResponsePartitionData], + data: Vector[(TopicAndPartition, FetchResponsePartitionData)], requestVersion: Int = 0, throttleTimeMs: Int = 0) extends RequestOrResponse() { @@ -206,7 +223,8 @@ case class FetchResponse(correlationId: Int, /** * Partitions the data into a map of maps (one for each topic). */ - lazy val dataGroupedByTopic = data.groupBy{ case (topicAndPartition, fetchData) => topicAndPartition.topic } + lazy val dataByTopicAndPartition = data.toMap + lazy val dataGroupedByTopic = FetchResponse.groupByTopicOrdered(data) val headerSizeInBytes = FetchResponse.headerSize(requestVersion) lazy val sizeInBytes = FetchResponse.responseSize(dataGroupedByTopic, requestVersion) @@ -234,7 +252,7 @@ case class FetchResponse(correlationId: Int, private def partitionDataFor(topic: String, partition: Int): FetchResponsePartitionData = { val topicAndPartition = TopicAndPartition(topic, partition) - data.get(topicAndPartition) match { + dataByTopicAndPartition.get(topicAndPartition) match { case Some(partitionData) => partitionData case _ => throw new IllegalArgumentException( @@ -247,7 +265,7 @@ case class FetchResponse(correlationId: Int, def highWatermark(topic: String, partition: Int) = partitionDataFor(topic, partition).hw - def hasError = data.values.exists(_.error != Errors.NONE.code) + def hasError = dataByTopicAndPartition.values.exists(_.error != Errors.NONE.code) def errorCode(topic: String, partition: Int) = partitionDataFor(topic, partition).error } diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala index 8801ff8ee825..5bb60dfbd21c 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala @@ -94,7 +94,7 @@ class ConsumerFetcherThread(name: String, consumerFetcherManager.addPartitionsWithError(partitions) } - protected def buildFetchRequest(partitionMap: collection.Map[TopicAndPartition, PartitionFetchState]): FetchRequest = { + protected def buildFetchRequest(partitionMap: collection.Seq[(TopicAndPartition, PartitionFetchState)]): FetchRequest = { partitionMap.foreach { case ((topicAndPartition, partitionFetchState)) => if (partitionFetchState.isActive) fetchRequestBuilder.addFetch(topicAndPartition.topic, topicAndPartition.partition, partitionFetchState.offset, @@ -105,7 +105,7 @@ class ConsumerFetcherThread(name: String, } protected def fetch(fetchRequest: FetchRequest): collection.Map[TopicAndPartition, PartitionData] = - simpleConsumer.fetch(fetchRequest.underlying).data.map { case (key, value) => + simpleConsumer.fetch(fetchRequest.underlying).dataByTopicAndPartition.map { case (key, value) => key -> new PartitionData(value) } @@ -115,7 +115,7 @@ object ConsumerFetcherThread { class FetchRequest(val underlying: kafka.api.FetchRequest) extends AbstractFetcherThread.FetchRequest { def isEmpty: Boolean = underlying.requestInfo.isEmpty - def offset(topicAndPartition: TopicAndPartition): Long = underlying.requestInfo(topicAndPartition).offset + def offset(topicAndPartition: TopicAndPartition): Long = underlying.requestInfoMapByTopic(topicAndPartition).offset } class PartitionData(val underlying: FetchResponsePartitionData) extends AbstractFetcherThread.PartitionData { diff --git a/core/src/main/scala/kafka/javaapi/FetchRequest.scala b/core/src/main/scala/kafka/javaapi/FetchRequest.scala index 406007746993..020ad758c9b5 100644 --- a/core/src/main/scala/kafka/javaapi/FetchRequest.scala +++ b/core/src/main/scala/kafka/javaapi/FetchRequest.scala @@ -29,9 +29,9 @@ class FetchRequest(correlationId: Int, requestInfo: java.util.Map[TopicAndPartition, PartitionFetchInfo]) { val underlying = { - val scalaMap: Map[TopicAndPartition, PartitionFetchInfo] = { + val scalaMap: Seq[(TopicAndPartition, PartitionFetchInfo)] = { import scala.collection.JavaConversions._ - (requestInfo: mutable.Map[TopicAndPartition, PartitionFetchInfo]).toMap + (requestInfo: mutable.Map[TopicAndPartition, PartitionFetchInfo]).toSeq } kafka.api.FetchRequest( correlationId = correlationId, @@ -39,7 +39,7 @@ class FetchRequest(correlationId: Int, replicaId = Request.OrdinaryConsumerId, maxWait = maxWait, minBytes = minBytes, - requestInfo = scalaMap + requestInfo = scalaMap.toVector ) } diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index 4aba66714070..7ee9344795ce 100755 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -67,7 +67,7 @@ abstract class AbstractFetcherThread(name: String, // deal with partitions with errors, potentially due to leadership changes def handlePartitionsWithErrors(partitions: Iterable[TopicAndPartition]) - protected def buildFetchRequest(partitionMap: Map[TopicAndPartition, PartitionFetchState]): REQ + protected def buildFetchRequest(partitionMap: Seq[(TopicAndPartition, PartitionFetchState)]): REQ protected def fetch(fetchRequest: REQ): Map[TopicAndPartition, PD] @@ -86,7 +86,7 @@ abstract class AbstractFetcherThread(name: String, override def doWork() { val fetchRequest = inLock(partitionMapLock) { - val fetchRequest = buildFetchRequest(partitionMap) + val fetchRequest = buildFetchRequest(partitionMap.toSeq) if (fetchRequest.isEmpty) { trace("There are no active partitions. Back off for %d ms before sending a fetch request".format(fetchBackOffMs)) partitionMapCond.await(fetchBackOffMs, TimeUnit.MILLISECONDS) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index bc48ce368651..d5ad2a55d203 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -434,8 +434,8 @@ class KafkaApis(val requestChannel: RequestChannel, case (topicAndPartition, _) => authorize(request.session, Read, new Resource(auth.Topic, topicAndPartition.topic)) } - val unauthorizedPartitionData = unauthorizedRequestInfo.mapValues { _ => - FetchResponsePartitionData(Errors.TOPIC_AUTHORIZATION_FAILED.code, -1, MessageSet.Empty) + val unauthorizedPartitionData = unauthorizedRequestInfo.map { case (tp, _) => + (tp, FetchResponsePartitionData(Errors.TOPIC_AUTHORIZATION_FAILED.code, -1, MessageSet.Empty)) } // the callback for sending a fetch response @@ -478,7 +478,7 @@ class KafkaApis(val requestChannel: RequestChannel, def fetchResponseCallback(delayTimeMs: Int) { trace(s"Sending fetch response to client ${fetchRequest.clientId} of " + s"${convertedPartitionData.values.map(_.messages.sizeInBytes).sum} bytes") - val response = FetchResponse(fetchRequest.correlationId, mergedPartitionData, fetchRequest.versionId, delayTimeMs) + val response = FetchResponse(fetchRequest.correlationId, mergedPartitionData.toVector, fetchRequest.versionId, delayTimeMs) requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, response))) } @@ -491,7 +491,7 @@ class KafkaApis(val requestChannel: RequestChannel, fetchResponseCallback(0) } else { quotaManagers(ApiKeys.FETCH.id).recordAndMaybeThrottle(fetchRequest.clientId, - FetchResponse.responseSize(mergedPartitionData.groupBy(_._1.topic), + FetchResponse.responseSize(FetchResponse.groupByTopicOrdered(mergedPartitionData.toSeq), fetchRequest.versionId), fetchResponseCallback) } @@ -506,7 +506,7 @@ class KafkaApis(val requestChannel: RequestChannel, fetchRequest.replicaId, fetchRequest.minBytes, fetchRequest.maxBytes, - authorizedRequestInfo, + authorizedRequestInfo.toMap, sendResponseCallback) } } diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index df44d07e7d68..a0eb23d03013 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -269,7 +269,7 @@ class ReplicaFetcherThread(name: String, } } - protected def buildFetchRequest(partitionMap: Map[TopicAndPartition, PartitionFetchState]): FetchRequest = { + protected def buildFetchRequest(partitionMap: Seq[(TopicAndPartition, PartitionFetchState)]): FetchRequest = { val requestMap = mutable.Map.empty[TopicPartition, JFetchRequest.PartitionData] partitionMap.foreach { case ((TopicAndPartition(topic, partition), partitionFetchState)) => diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index ca9dac424db2..f8fbae7b0084 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -127,11 +127,11 @@ object SerializationTestUtils { ), ProducerRequest.CurrentVersion, 100) def createTestFetchRequest: FetchRequest = { - new FetchRequest(requestInfo = requestInfos) + new FetchRequest(requestInfo = requestInfos.toVector) } def createTestFetchResponse: FetchResponse = { - FetchResponse(1, topicDataFetchResponse) + FetchResponse(1, topicDataFetchResponse.toVector) } def createTestOffsetRequest = new OffsetRequest( @@ -267,11 +267,11 @@ class RequestResponseSerializationTest extends JUnitSuite { def testFetchResponseVersion() { val oldClientResponse = FetchResponse(1, Map( TopicAndPartition("t1", 0) -> new FetchResponsePartitionData(messages = new ByteBufferMessageSet(new Message("first message".getBytes))) - ), 0) + ).toVector, 0) val newClientResponse = FetchResponse(1, Map( TopicAndPartition("t1", 0) -> new FetchResponsePartitionData(messages = new ByteBufferMessageSet(new Message("first message".getBytes))) - ), 1, 100) + ).toVector, 1, 100) // new response should have 4 bytes more than the old response since delayTime is an INT32 assertEquals(oldClientResponse.sizeInBytes + 4, newClientResponse.sizeInBytes) diff --git a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala index 140f61522579..3998a21265d7 100755 --- a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala +++ b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala @@ -60,7 +60,7 @@ class PrimitiveApiTest extends ProducerConsumerTestHarness { @Test def testEmptyFetchRequest() { val partitionRequests = immutable.Map[TopicAndPartition, PartitionFetchInfo]() - val request = new FetchRequest(requestInfo = partitionRequests) + val request = new FetchRequest(requestInfo = partitionRequests.toVector) val fetched = consumer.fetch(request) assertTrue(!fetched.hasError && fetched.data.isEmpty) } @@ -149,7 +149,7 @@ class PrimitiveApiTest extends ProducerConsumerTestHarness { try { val request = builder.build() val response = consumer.fetch(request) - response.data.values.foreach(pdata => ErrorMapping.maybeThrowException(pdata.error)) + response.data.foreach(pdata => ErrorMapping.maybeThrowException(pdata._2.error)) fail("Expected exception when fetching message with invalid offset") } catch { case e: OffsetOutOfRangeException => // This is good. @@ -165,7 +165,7 @@ class PrimitiveApiTest extends ProducerConsumerTestHarness { try { val request = builder.build() val response = consumer.fetch(request) - response.data.values.foreach(pdata => ErrorMapping.maybeThrowException(pdata.error)) + response.data.foreach(pdata => ErrorMapping.maybeThrowException(pdata._2.error)) fail("Expected exception when fetching message with invalid partition") } catch { case e: UnknownTopicOrPartitionException => // This is good. diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala index b95f2bf05f3d..0b13bab6cbe6 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala @@ -119,8 +119,8 @@ class AbstractFetcherThreadTest { fetchRequest.offsets.mapValues(_ => new DummyPartitionData) } - override protected def buildFetchRequest(partitionMap: collection.Map[TopicAndPartition, PartitionFetchState]): DummyFetchRequest = { - new DummyFetchRequest(partitionMap.mapValues(_.offset)) + override protected def buildFetchRequest(partitionMap: collection.Seq[(TopicAndPartition, PartitionFetchState)]): DummyFetchRequest = { + new DummyFetchRequest(partitionMap.map(kv => (kv._1, kv._2.offset)).toMap) } } From 4c053981a2317e191e00e157e7c1d64d422da070 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 12 Sep 2016 20:28:01 -0700 Subject: [PATCH 03/54] KAFKA-2311; Make KafkaConsumer's ensureNotClosed method thread-safe Here is the patch on github ijuma. Acquiring the consumer lock (the single thread access controls) requires that the consumer be open. I changed the closed variable to be volatile so that another thread's writes will visible to the reading thread. Additionally, there was an additional check if the consumer was closed after the lock was acquired. This check is no longer necessary. This is my original work and I license it to the project under the project's open source license. Author: Tim Brooks Reviewers: Jason Gustafson Closes #1637 from tbrooks8/KAFKA-2311 --- .../java/org/apache/kafka/clients/consumer/KafkaConsumer.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 5bf2b7a366c2..59f19876bd7f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -508,7 +508,7 @@ public class KafkaConsumer implements Consumer { private final Metadata metadata; private final long retryBackoffMs; private final long requestTimeoutMs; - private boolean closed = false; + private volatile boolean closed = false; // currentThread holds the threadId of the current thread accessing KafkaConsumer // and is used to prevent multi-threaded access @@ -1398,7 +1398,6 @@ public Set paused() { public void close() { acquire(); try { - if (closed) return; close(false); } finally { release(); From b621ba115c0fcc45e66653d92a20ec4862cab9c8 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Tue, 13 Sep 2016 17:33:54 -0700 Subject: [PATCH 04/54] KAFKA-4158; Reset quota to default value if quota override is deleted Author: Dong Lin Reviewers: Joel Koshy , Jiangjie Qin Closes #1851 from lindong28/KAFKA-4158 --- .../kafka/server/ClientQuotaManager.scala | 8 ++++++++ .../scala/kafka/server/ConfigHandler.scala | 4 ++++ .../server/DynamicConfigChangeTest.scala | 19 +++++++++++++++++-- 3 files changed, 29 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index c99ba9760342..5e9008089cf9 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -242,6 +242,14 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, .quota(quota) } + /** + * Reset quotas to the default value for the given clientId + * @param clientId client to override + */ + def resetQuota(clientId: String) = { + updateQuota(clientId, defaultQuota) + } + /** * Overrides quotas per clientId * @param clientId client to override diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala index ab1d7825e183..d07fdd814b67 100644 --- a/core/src/main/scala/kafka/server/ConfigHandler.scala +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -82,11 +82,15 @@ class ClientIdConfigHandler(private val quotaManagers: Map[Short, ClientQuotaMan if (clientConfig.containsKey(ClientConfigOverride.ProducerOverride)) { quotaManagers(ApiKeys.PRODUCE.id).updateQuota(clientId, new Quota(clientConfig.getProperty(ClientConfigOverride.ProducerOverride).toLong, true)) + } else { + quotaManagers(ApiKeys.PRODUCE.id).resetQuota(clientId) } if (clientConfig.containsKey(ClientConfigOverride.ConsumerOverride)) { quotaManagers(ApiKeys.FETCH.id).updateQuota(clientId, new Quota(clientConfig.getProperty(ClientConfigOverride.ConsumerOverride).toLong, true)) + } else { + quotaManagers(ApiKeys.FETCH.id).resetQuota(clientId) } } } diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index af979e44767f..a9df9294f0ae 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -65,10 +65,9 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { props.put(ClientConfigOverride.ProducerOverride, "1000") props.put(ClientConfigOverride.ConsumerOverride, "2000") AdminUtils.changeClientIdConfig(zkUtils, clientId, props) + val quotaManagers: Map[Short, ClientQuotaManager] = servers.head.apis.quotaManagers TestUtils.retry(10000) { - val configHandler = this.servers.head.dynamicConfigHandlers(ConfigType.Client).asInstanceOf[ClientIdConfigHandler] - val quotaManagers: Map[Short, ClientQuotaManager] = servers.head.apis.quotaManagers val overrideProducerQuota = quotaManagers.get(ApiKeys.PRODUCE.id).get.quota(clientId) val overrideConsumerQuota = quotaManagers.get(ApiKeys.FETCH.id).get.quota(clientId) @@ -77,6 +76,22 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { assertEquals(s"ClientId $clientId must have overridden consumer quota of 2000", Quota.upperBound(2000), overrideConsumerQuota) } + + val defaultProducerQuota = servers.head.apis.config.producerQuotaBytesPerSecondDefault.doubleValue + val defaultConsumerQuota = servers.head.apis.config.consumerQuotaBytesPerSecondDefault.doubleValue + assertNotEquals("defaultProducerQuota should be different from 1000", 1000, defaultProducerQuota) + assertNotEquals("defaultConsumerQuota should be different from 2000", 2000, defaultConsumerQuota) + AdminUtils.changeClientIdConfig(zkUtils, clientId, new Properties()) + + TestUtils.retry(10000) { + val producerQuota = quotaManagers.get(ApiKeys.PRODUCE.id).get.quota(clientId) + val consumerQuota = quotaManagers.get(ApiKeys.FETCH.id).get.quota(clientId) + + assertEquals(s"ClientId $clientId must have reset producer quota to " + defaultProducerQuota, + Quota.upperBound(defaultProducerQuota), producerQuota) + assertEquals(s"ClientId $clientId must have reset consumer quota to " + defaultConsumerQuota, + Quota.upperBound(defaultConsumerQuota), consumerQuota) + } } @Test From f98fa67f81e0f89d47e649b8a6d2fa9790f9a7ad Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 14 Sep 2016 09:16:29 -0700 Subject: [PATCH 05/54] MINOR: Give a name to the coordinator heartbeat thread Followed the same naming pattern as the producer sender thread. Author: Ismael Juma Reviewers: Jason Gustafson Closes #1854 from ijuma/heartbeat-thread-name --- .../kafka/clients/consumer/internals/AbstractCoordinator.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index 7ae79c067407..2f362394b2af 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -793,6 +793,10 @@ private class HeartbeatThread extends Thread { private boolean closed = false; private AtomicReference failed = new AtomicReference<>(null); + HeartbeatThread() { + super("kafka-coordinator-heartbeat-thread" + (groupId.isEmpty() ? "" : " | " + groupId)); + } + public void enable() { synchronized (AbstractCoordinator.this) { this.enabled = true; From a6b3a8edd9cccf478bad12fc014daa06a476816d Mon Sep 17 00:00:00 2001 From: David Chen Date: Wed, 14 Sep 2016 10:38:40 -0700 Subject: [PATCH 06/54] KAFKA-4162: Fixed typo "rebalance" Author: David Chen Reviewers: Ewen Cheslack-Postava Closes #1853 from mvj3/KAFKA-4162 --- README.md | 2 +- .../kafka/connect/runtime/distributed/DistributedHerder.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 91c09805e749..063defe9c53e 100644 --- a/README.md +++ b/README.md @@ -15,7 +15,7 @@ Java 7 should be used for building in order to support both Java 7 and Java 8 at Now everything else will work. ### Building a jar and running it ### - ./gradlew jar + ./gradlew jar Follow instructions in http://kafka.apache.org/documentation.html#quickstart diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java index afcd283f4964..ef617a76068b 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java @@ -1013,7 +1013,7 @@ public void onAssigned(ConnectProtocol.Assignment assignment, int generation) { rebalanceResolved = false; } - // Delete the statuses of all connectors removed prior to the start of this reblaance. This has to + // Delete the statuses of all connectors removed prior to the start of this rebalance. This has to // be done after the rebalance completes to avoid race conditions as the previous generation attempts // to change the state to UNASSIGNED after tasks have been stopped. if (isLeader()) From 62a73948960a1c2c28f281f9b31135e0d9c8432f Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Wed, 14 Sep 2016 18:04:58 -0700 Subject: [PATCH 07/54] KAFKA-4172; Ensure fetch responses contain the requested partitions Author: Jason Gustafson Reviewers: Ismael Juma Closes #1857 from hachikuji/KAFKA-4172 --- .../clients/consumer/internals/Fetcher.java | 15 +++++ .../clients/consumer/KafkaConsumerTest.java | 62 +++++++++++++++---- 2 files changed, 65 insertions(+), 12 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 92c9a319b230..b3fe4185333f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -144,6 +144,12 @@ public boolean hasInFlightFetches() { return numInFlightFetches.get() > 0; } + private boolean matchesRequestedPartitions(FetchRequest request, FetchResponse response) { + Set requestedPartitions = request.fetchData().keySet(); + Set fetchedPartitions = response.responseData().keySet(); + return fetchedPartitions.equals(requestedPartitions); + } + /** * Set-up a fetch request for any node that we have assigned partitions for which doesn't already have * an in-flight fetch or pending fetch data. @@ -161,6 +167,15 @@ public void onSuccess(ClientResponse resp) { numInFlightFetches.decrementAndGet(); FetchResponse response = new FetchResponse(resp.responseBody()); + if (!matchesRequestedPartitions(request, response)) { + // obviously we expect the broker to always send us valid responses, so this check + // is mainly for test cases where mock fetch responses must be manually crafted. + log.warn("Ignoring fetch response containing partitions {} since it does not match " + + "the requested partitions {}", response.responseData().keySet(), + request.fetchData().keySet()); + return; + } + Set partitions = new HashSet<>(response.responseData().keySet()); FetchResponseMetricAggregator metricAggregator = new FetchResponseMetricAggregator(sensors, partitions); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index dd8753f0fdf5..17bb9b55ad65 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -73,6 +73,8 @@ import java.util.regex.Pattern; import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; +import static java.util.Collections.singletonMap; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -139,7 +141,7 @@ public void testInvalidSocketReceiveBufferSize() throws Exception { public void testSubscription() { KafkaConsumer consumer = newConsumer(); - consumer.subscribe(Collections.singletonList(topic)); + consumer.subscribe(singletonList(topic)); assertEquals(singleton(topic), consumer.subscription()); assertTrue(consumer.assignment().isEmpty()); @@ -147,7 +149,7 @@ public void testSubscription() { assertTrue(consumer.subscription().isEmpty()); assertTrue(consumer.assignment().isEmpty()); - consumer.assign(Collections.singletonList(tp0)); + consumer.assign(singletonList(tp0)); assertTrue(consumer.subscription().isEmpty()); assertEquals(singleton(tp0), consumer.assignment()); @@ -175,7 +177,7 @@ public void testSubscriptionOnNullTopic() { String nullTopic = null; try { - consumer.subscribe(Collections.singletonList(nullTopic)); + consumer.subscribe(singletonList(nullTopic)); } finally { consumer.close(); } @@ -187,7 +189,7 @@ public void testSubscriptionOnEmptyTopic() { String emptyTopic = " "; try { - consumer.subscribe(Collections.singletonList(emptyTopic)); + consumer.subscribe(singletonList(emptyTopic)); } finally { consumer.close(); } @@ -299,7 +301,7 @@ public void testInterceptorConstructorClose() throws Exception { public void testPause() { KafkaConsumer consumer = newConsumer(); - consumer.assign(Collections.singletonList(tp0)); + consumer.assign(singletonList(tp0)); assertEquals(singleton(tp0), consumer.assignment()); assertTrue(consumer.paused().isEmpty()); @@ -569,6 +571,41 @@ public void testWakeupWithFetchDataAvailable() { assertEquals(5, records.count()); } + @Test + public void fetchResponseWithUnexpectedPartitionIsIgnored() { + int rebalanceTimeoutMs = 60000; + int sessionTimeoutMs = 30000; + int heartbeatIntervalMs = 3000; + + // adjust auto commit interval lower than heartbeat so we don't need to deal with + // a concurrent heartbeat request + int autoCommitIntervalMs = 1000; + + Time time = new MockTime(); + MockClient client = new MockClient(time); + Cluster cluster = TestUtils.singletonCluster(singletonMap(topic, 1)); + Node node = cluster.nodes().get(0); + client.setNode(node); + Metadata metadata = new Metadata(0, Long.MAX_VALUE); + metadata.update(cluster, time.milliseconds()); + PartitionAssignor assignor = new RangeAssignor(); + + final KafkaConsumer consumer = newConsumer(time, client, metadata, assignor, + rebalanceTimeoutMs, sessionTimeoutMs, heartbeatIntervalMs, true, autoCommitIntervalMs); + + consumer.subscribe(singletonList(topic), getConsumerRebalanceListener(consumer)); + + prepareRebalance(client, node, assignor, singletonList(tp0), null); + + Map fetches1 = new HashMap<>(); + fetches1.put(tp0, new FetchInfo(0, 1)); + fetches1.put(t2p0, new FetchInfo(0, 10)); // not assigned and not fetched + client.prepareResponseFrom(fetchResponse(fetches1), node); + + ConsumerRecords records = consumer.poll(0); + assertEquals(0, records.count()); + } + /** * Verify that when a consumer changes its topic subscription its assigned partitions * do not immediately change, and the latest consumed offsets of its to-be-revoked @@ -631,12 +668,17 @@ public void testSubscriptionChangesWithAutoCommitEnabled() { ConsumerRecords records = consumer.poll(0); + // clear out the prefetch so it doesn't interfere with the rest of the test + fetches1.put(tp0, new FetchInfo(1, 0)); + fetches1.put(t2p0, new FetchInfo(10, 0)); + client.respondFrom(fetchResponse(fetches1), node); + client.poll(0, time.milliseconds()); + // verify that the fetch occurred as expected assertEquals(11, records.count()); assertEquals(1L, consumer.position(tp0)); assertEquals(10L, consumer.position(t2p0)); - // subscription change consumer.subscribe(Arrays.asList(topic, topic3), getConsumerRebalanceListener(consumer)); @@ -655,12 +697,10 @@ public void testSubscriptionChangesWithAutoCommitEnabled() { // mock rebalance responses prepareRebalance(client, node, assignor, Arrays.asList(tp0, t3p0), coordinator); - // mock a response to the outstanding fetch so that we have data available on the next poll + // mock a response to the next fetch from the new assignment Map fetches2 = new HashMap<>(); fetches2.put(tp0, new FetchInfo(1, 1)); fetches2.put(t3p0, new FetchInfo(0, 100)); - client.respondFrom(fetchResponse(fetches2), node); - client.poll(0, time.milliseconds()); client.prepareResponse(fetchResponse(fetches2)); records = consumer.poll(0); @@ -679,14 +719,12 @@ public void testSubscriptionChangesWithAutoCommitEnabled() { assertTrue(consumer.assignment().size() == 2); assertTrue(consumer.assignment().contains(tp0) && consumer.assignment().contains(t3p0)); - // mock the offset commit response for to be revoked partitions Map partitionOffsets2 = new HashMap<>(); partitionOffsets2.put(tp0, 2L); partitionOffsets2.put(t3p0, 100L); commitReceived = prepareOffsetCommitResponse(client, coordinator, partitionOffsets2); - // unsubscribe consumer.unsubscribe(); // verify that subscription and assignment are both cleared @@ -993,7 +1031,7 @@ private Struct listOffsetsResponse(Map offsets, short erro Map partitionData = new HashMap<>(); for (Map.Entry partitionOffset : offsets.entrySet()) { partitionData.put(partitionOffset.getKey(), new ListOffsetResponse.PartitionData(error, - Collections.singletonList(partitionOffset.getValue()))); + singletonList(partitionOffset.getValue()))); } return new ListOffsetResponse(partitionData).toStruct(); } From a2e3ee5c42621c06bb40ce729ecc31834c4c6491 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 13 Sep 2016 12:55:43 +0100 Subject: [PATCH 08/54] Refactor constructors for a.k.c.r.FetchRequest and introduce factory methods for the replica case Use diamond in a couple of places. --- .../clients/consumer/internals/Fetcher.java | 2 +- .../kafka/common/requests/FetchRequest.java | 38 ++++++++++++------- .../kafka/server/ReplicaFetcherThread.scala | 6 ++- 3 files changed, 31 insertions(+), 15 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index b3fe4185333f..9b0f98808474 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -558,7 +558,7 @@ private Map createFetchRequests() { Map requests = new HashMap<>(); for (Map.Entry> entry : fetchable.entrySet()) { Node node = entry.getKey(); - FetchRequest fetch = new FetchRequest(this.maxWaitMs, this.minBytes, entry.getValue(), this.maxBytes); + FetchRequest fetch = new FetchRequest(this.maxWaitMs, this.minBytes, this.maxBytes, entry.getValue()); requests.put(node, fetch); } return requests; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java index 2b8827253f08..9c167e713ddc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -87,28 +87,40 @@ public static final List groupByTopicOrdered(Map fetchData) { - this(ProtoUtils.latestVersion(ApiKeys.FETCH.id), CONSUMER_REPLICA_ID, maxWait, minBytes, fetchData, DEFAULT_RESPONSE_MAX_BYTES); - } - public FetchRequest(int maxWait, int minBytes, Map fetchData, int maxBytes) { - this(ProtoUtils.latestVersion(ApiKeys.FETCH.id), CONSUMER_REPLICA_ID, maxWait, minBytes, fetchData, maxBytes); + // Any of 0, 1 or 2 would do here + this(2, CONSUMER_REPLICA_ID, maxWait, minBytes, DEFAULT_RESPONSE_MAX_BYTES, fetchData); } + /** + * Create a non-replica fetch request for the current version. + */ + public FetchRequest(int maxWait, int minBytes, int maxBytes, Map fetchData) { + this(ProtoUtils.latestVersion(ApiKeys.FETCH.id), CONSUMER_REPLICA_ID, maxWait, minBytes, maxBytes, fetchData); + } /** - * Create a replica fetch request + * Create a replica fetch request for versions 0, 1 or 2 (the actual version is determined by the RequestHeader). */ - public FetchRequest(int replicaId, int maxWait, int minBytes, Map fetchData) { - this(ProtoUtils.latestVersion(ApiKeys.FETCH.id), replicaId, maxWait, minBytes, fetchData, DEFAULT_RESPONSE_MAX_BYTES); + @Deprecated + public static FetchRequest fromReplica(int replicaId, int maxWait, int minBytes, + Map fetchData) { + // Any of 0, 1 or 2 would do here + return new FetchRequest(2, replicaId, maxWait, minBytes, DEFAULT_RESPONSE_MAX_BYTES, fetchData); } - public FetchRequest(int replicaId, int maxWait, int minBytes, Map fetchData, int maxBytes) { - this(ProtoUtils.latestVersion(ApiKeys.FETCH.id), replicaId, maxWait, minBytes, fetchData, maxBytes); + /** + * Create a replica fetch request for the current version. + */ + public static FetchRequest fromReplica(int replicaId, int maxWait, int minBytes, int maxBytes, + Map fetchData) { + return new FetchRequest(ProtoUtils.latestVersion(ApiKeys.FETCH.id), replicaId, maxWait, minBytes, maxBytes, fetchData); } - public FetchRequest(int version, int replicaId, int maxWait, int minBytes, Map fetchData, int maxBytes) { + private FetchRequest(int version, int replicaId, int maxWait, int minBytes, int maxBytes, Map fetchData) { super(new Struct(ProtoUtils.requestSchema(ApiKeys.FETCH.id, version))); List topicsData = TopicAndPartitionData.groupByTopicOrdered(fetchData); @@ -138,7 +150,7 @@ public FetchRequest(int version, int replicaId, int maxWait, int minBytes, Map(fetchData); + this.fetchData = new LinkedHashMap<>(fetchData); } public FetchRequest(Struct struct) { @@ -150,7 +162,7 @@ public FetchRequest(Struct struct) { maxBytes = struct.getInt(RESPONSE_MAX_BYTES_KEY_NAME); else maxBytes = DEFAULT_RESPONSE_MAX_BYTES; - fetchData = new LinkedHashMap(); + fetchData = new LinkedHashMap<>(); for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) { Struct topicResponse = (Struct) topicResponseObj; String topic = topicResponse.getString(TOPIC_KEY_NAME); diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index a0eb23d03013..b7c027f0d54f 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -277,7 +277,11 @@ class ReplicaFetcherThread(name: String, requestMap(new TopicPartition(topic, partition)) = new JFetchRequest.PartitionData(partitionFetchState.offset, fetchSize) } - new FetchRequest(new JFetchRequest(fetchRequestVersion, replicaId, maxWait, minBytes, requestMap.asJava, maxBytes)) + val request = + if (fetchRequestVersion >= 3) JFetchRequest.fromReplica(replicaId, maxWait, minBytes, maxBytes, requestMap) + else JFetchRequest.fromReplica(replicaId, maxWait, minBytes, requestMap) + + new FetchRequest(request) } } From 62a4b1252072bd46660a8a6b4254f71bf87ed6b2 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 13 Sep 2016 12:56:21 +0100 Subject: [PATCH 09/54] Add missing cases in `FetchRequest.getErrorResponse` and equality bug --- .../java/org/apache/kafka/common/requests/FetchRequest.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java index 9c167e713ddc..19bb29e5f6fc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -78,7 +78,7 @@ public static final List groupByTopicOrdered(Map Date: Tue, 13 Sep 2016 13:13:30 +0100 Subject: [PATCH 10/54] Fix reference to assignFromUser in javadoc --- .../kafka/clients/consumer/internals/SubscriptionState.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java index dd0bce9edd62..f3316d2a8613 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java @@ -27,7 +27,7 @@ /** * A class for tracking the topics, partitions, and offsets for the consumer. A partition - * is "assigned" either directly with {@link #assignFromUser(Collection)} (manual assignment) + * is "assigned" either directly with {@link #assignFromUser(Set)} (manual assignment) * or with {@link #assignFromSubscribed(Collection)} (automatic assignment from subscription). * * Once assigned, the partition is not considered "fetchable" until its initial position has From 20129a07bb27cd2dab9be24ce8d7dcda07e8d98c Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 13 Sep 2016 16:57:53 +0100 Subject: [PATCH 11/54] Use `TopicPartition` instead of `TopicAndPartition` in *FetcherThread classes This will make it easier to reuse a shared Java class for request round-robin. --- .../consumer/ConsumerFetcherManager.scala | 9 ++- .../consumer/ConsumerFetcherThread.scala | 34 ++++++----- .../kafka/server/AbstractFetcherManager.scala | 8 ++- .../kafka/server/AbstractFetcherThread.scala | 60 ++++++++++--------- .../kafka/server/ReplicaFetcherThread.scala | 60 +++++++++---------- .../server/AbstractFetcherThreadTest.scala | 22 +++---- 6 files changed, 103 insertions(+), 90 deletions(-) diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala index e73faf238ef8..f217882cfd7b 100755 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala @@ -32,6 +32,8 @@ import kafka.common.TopicAndPartition import kafka.client.ClientUtils import java.util.concurrent.atomic.AtomicInteger +import org.apache.kafka.common.TopicPartition + /** * Usage: * Once ConsumerFetcherManager is created, startConnections() and stopAllConnections() can be called repeatedly @@ -115,9 +117,12 @@ class ConsumerFetcherManager(private val consumerIdString: String, } override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): AbstractFetcherThread = { + val topicPartitionMap = partitionMap.map { case (TopicAndPartition(topic, partition), topicInfo) => + new TopicPartition(topic, partition) -> topicInfo + } new ConsumerFetcherThread( "ConsumerFetcherThread-%s-%d-%d".format(consumerIdString, fetcherId, sourceBroker.id), - config, sourceBroker, partitionMap, this) + config, sourceBroker, topicPartitionMap, this) } def startConnections(topicInfos: Iterable[PartitionTopicInfo], cluster: Cluster) { @@ -163,4 +168,4 @@ class ConsumerFetcherManager(private val consumerIdString: String, } } } -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala index 5bb60dfbd21c..24766b58ba15 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala @@ -25,11 +25,12 @@ import kafka.common.{ErrorMapping, TopicAndPartition} import scala.collection.JavaConverters import JavaConverters._ import ConsumerFetcherThread._ +import org.apache.kafka.common.TopicPartition class ConsumerFetcherThread(name: String, val config: ConsumerConfig, sourceBroker: BrokerEndPoint, - partitionMap: Map[TopicAndPartition, PartitionTopicInfo], + partitionMap: Map[TopicPartition, PartitionTopicInfo], val consumerFetcherManager: ConsumerFetcherManager) extends AbstractFetcherThread(name = name, clientId = config.clientId, @@ -66,47 +67,48 @@ class ConsumerFetcherThread(name: String, } // process fetched data - def processPartitionData(topicAndPartition: TopicAndPartition, fetchOffset: Long, partitionData: PartitionData) { - val pti = partitionMap(topicAndPartition) + def processPartitionData(topicPartition: TopicPartition, fetchOffset: Long, partitionData: PartitionData) { + val pti = partitionMap(topicPartition) if (pti.getFetchOffset != fetchOffset) throw new RuntimeException("Offset doesn't match for partition [%s,%d] pti offset: %d fetch offset: %d" - .format(topicAndPartition.topic, topicAndPartition.partition, pti.getFetchOffset, fetchOffset)) + .format(topicPartition.topic, topicPartition.partition, pti.getFetchOffset, fetchOffset)) pti.enqueue(partitionData.underlying.messages.asInstanceOf[ByteBufferMessageSet]) } // handle a partition whose offset is out of range and return a new fetch offset - def handleOffsetOutOfRange(topicAndPartition: TopicAndPartition): Long = { + def handleOffsetOutOfRange(topicPartition: TopicPartition): Long = { val startTimestamp = config.autoOffsetReset match { case OffsetRequest.SmallestTimeString => OffsetRequest.EarliestTime case OffsetRequest.LargestTimeString => OffsetRequest.LatestTime case _ => OffsetRequest.LatestTime } + val topicAndPartition = new TopicAndPartition(topicPartition.topic, topicPartition.partition) val newOffset = simpleConsumer.earliestOrLatestOffset(topicAndPartition, startTimestamp, Request.OrdinaryConsumerId) - val pti = partitionMap(topicAndPartition) + val pti = partitionMap(topicPartition) pti.resetFetchOffset(newOffset) pti.resetConsumeOffset(newOffset) newOffset } // any logic for partitions whose leader has changed - def handlePartitionsWithErrors(partitions: Iterable[TopicAndPartition]) { + def handlePartitionsWithErrors(partitions: Iterable[TopicPartition]) { removePartitions(partitions.toSet) - consumerFetcherManager.addPartitionsWithError(partitions) + consumerFetcherManager.addPartitionsWithError(partitions.map(tp => new TopicAndPartition(tp.topic, tp.partition))) } - protected def buildFetchRequest(partitionMap: collection.Seq[(TopicAndPartition, PartitionFetchState)]): FetchRequest = { - partitionMap.foreach { case ((topicAndPartition, partitionFetchState)) => + protected def buildFetchRequest(partitionMap: collection.Seq[(TopicPartition, PartitionFetchState)]): FetchRequest = { + partitionMap.foreach { case ((topicPartition, partitionFetchState)) => if (partitionFetchState.isActive) - fetchRequestBuilder.addFetch(topicAndPartition.topic, topicAndPartition.partition, partitionFetchState.offset, + fetchRequestBuilder.addFetch(topicPartition.topic, topicPartition.partition, partitionFetchState.offset, fetchSize) } new FetchRequest(fetchRequestBuilder.build()) } - protected def fetch(fetchRequest: FetchRequest): collection.Map[TopicAndPartition, PartitionData] = - simpleConsumer.fetch(fetchRequest.underlying).dataByTopicAndPartition.map { case (key, value) => - key -> new PartitionData(value) + protected def fetch(fetchRequest: FetchRequest): collection.Map[TopicPartition, PartitionData] = + simpleConsumer.fetch(fetchRequest.underlying).dataByTopicAndPartition.map { case (TopicAndPartition(t, p), value) => + new TopicPartition(t, p) -> new PartitionData(value) } } @@ -115,7 +117,9 @@ object ConsumerFetcherThread { class FetchRequest(val underlying: kafka.api.FetchRequest) extends AbstractFetcherThread.FetchRequest { def isEmpty: Boolean = underlying.requestInfo.isEmpty - def offset(topicAndPartition: TopicAndPartition): Long = underlying.requestInfoMapByTopic(topicAndPartition).offset + def offset(topicPartition: TopicPartition): Long = { + underlying.requestInfoMapByTopic(new TopicAndPartition(topicPartition.topic, topicPartition.partition)).offset + } } class PartitionData(val underlying: FetchResponsePartitionData) extends AbstractFetcherThread.PartitionData { diff --git a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala index ec405169bfbc..b7beb208de7b 100755 --- a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala @@ -25,6 +25,7 @@ import kafka.cluster.BrokerEndPoint import kafka.metrics.KafkaMetricsGroup import kafka.common.TopicAndPartition import com.yammer.metrics.core.Gauge +import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.utils.Utils abstract class AbstractFetcherManager(protected val name: String, clientId: String, numFetchers: Int = 1) @@ -85,8 +86,9 @@ abstract class AbstractFetcherManager(protected val name: String, clientId: Stri fetcherThread.start } - fetcherThreadMap(brokerAndFetcherId).addPartitions(partitionAndOffsets.map { case (topicAndPartition, brokerAndInitOffset) => - topicAndPartition -> brokerAndInitOffset.initOffset + fetcherThreadMap(brokerAndFetcherId).addPartitions(partitionAndOffsets.map { + case (TopicAndPartition(t, p), brokerAndInitOffset) => + new TopicPartition(t, p) -> brokerAndInitOffset.initOffset }) } } @@ -98,7 +100,7 @@ abstract class AbstractFetcherManager(protected val name: String, clientId: Stri def removeFetcherForPartitions(partitions: Set[TopicAndPartition]) { mapLock synchronized { for ((key, fetcher) <- fetcherThreadMap) { - fetcher.removePartitions(partitions) + fetcher.removePartitions(partitions.map { case TopicAndPartition(t, p) => new TopicPartition(t, p) }) } } info("Removed fetcher for partitions %s".format(partitions.mkString(","))) diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index 7ee9344795ce..759a919114b1 100755 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -34,6 +34,7 @@ import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicLong import com.yammer.metrics.core.Gauge +import org.apache.kafka.common.TopicPartition /** * Abstract class for fetching data from multiple partitions from the same broker. @@ -48,7 +49,7 @@ abstract class AbstractFetcherThread(name: String, type REQ <: FetchRequest type PD <: PartitionData - private val partitionMap = new mutable.HashMap[TopicAndPartition, PartitionFetchState] // a (topic, partition) -> partitionFetchState map + private val partitionMap = new mutable.HashMap[TopicPartition, PartitionFetchState] // a (topic, partition) -> partitionFetchState map private val partitionMapLock = new ReentrantLock private val partitionMapCond = partitionMapLock.newCondition() @@ -59,17 +60,17 @@ abstract class AbstractFetcherThread(name: String, /* callbacks to be defined in subclass */ // process fetched data - def processPartitionData(topicAndPartition: TopicAndPartition, fetchOffset: Long, partitionData: PD) + def processPartitionData(topicPartition: TopicPartition, fetchOffset: Long, partitionData: PD) // handle a partition whose offset is out of range and return a new fetch offset - def handleOffsetOutOfRange(topicAndPartition: TopicAndPartition): Long + def handleOffsetOutOfRange(topicPartition: TopicPartition): Long // deal with partitions with errors, potentially due to leadership changes - def handlePartitionsWithErrors(partitions: Iterable[TopicAndPartition]) + def handlePartitionsWithErrors(partitions: Iterable[TopicPartition]) - protected def buildFetchRequest(partitionMap: Seq[(TopicAndPartition, PartitionFetchState)]): REQ + protected def buildFetchRequest(partitionMap: Seq[(TopicPartition, PartitionFetchState)]): REQ - protected def fetch(fetchRequest: REQ): Map[TopicAndPartition, PD] + protected def fetch(fetchRequest: REQ): Map[TopicPartition, PD] override def shutdown(){ initiateShutdown() @@ -99,8 +100,8 @@ abstract class AbstractFetcherThread(name: String, } private def processFetchRequest(fetchRequest: REQ) { - val partitionsWithError = new mutable.HashSet[TopicAndPartition] - var responseData: Map[TopicAndPartition, PD] = Map.empty + val partitionsWithError = new mutable.HashSet[TopicPartition] + var responseData: Map[TopicPartition, PD] = Map.empty try { trace("Issuing to broker %d of fetch request %s".format(sourceBroker.id, fetchRequest)) @@ -122,11 +123,12 @@ abstract class AbstractFetcherThread(name: String, // process fetched data inLock(partitionMapLock) { - responseData.foreach { case (topicAndPartition, partitionData) => - val TopicAndPartition(topic, partitionId) = topicAndPartition - partitionMap.get(topicAndPartition).foreach(currentPartitionFetchState => + responseData.foreach { case (topicPartition, partitionData) => + val topic = topicPartition.topic + val partitionId = topicPartition.partition + partitionMap.get(topicPartition).foreach(currentPartitionFetchState => // we append to the log if the current offset is defined and it is the same as the offset requested during fetch - if (fetchRequest.offset(topicAndPartition) == currentPartitionFetchState.offset) { + if (fetchRequest.offset(topicPartition) == currentPartitionFetchState.offset) { Errors.forCode(partitionData.errorCode) match { case Errors.NONE => try { @@ -136,11 +138,11 @@ abstract class AbstractFetcherThread(name: String, case Some(m: MessageAndOffset) => m.nextOffset case None => currentPartitionFetchState.offset } - partitionMap.put(topicAndPartition, new PartitionFetchState(newOffset)) + partitionMap.put(topicPartition, new PartitionFetchState(newOffset)) fetcherLagStats.getAndMaybePut(topic, partitionId).lag = Math.max(0L, partitionData.highWatermark - newOffset) fetcherStats.byteRate.mark(validBytes) // Once we hand off the partition data to the subclass, we can't mess with it any more in this thread - processPartitionData(topicAndPartition, currentPartitionFetchState.offset, partitionData) + processPartitionData(topicPartition, currentPartitionFetchState.offset, partitionData) } catch { case ime: CorruptRecordException => // we log the error and continue. This ensures two things @@ -154,20 +156,20 @@ abstract class AbstractFetcherThread(name: String, } case Errors.OFFSET_OUT_OF_RANGE => try { - val newOffset = handleOffsetOutOfRange(topicAndPartition) - partitionMap.put(topicAndPartition, new PartitionFetchState(newOffset)) + val newOffset = handleOffsetOutOfRange(topicPartition) + partitionMap.put(topicPartition, new PartitionFetchState(newOffset)) error("Current offset %d for partition [%s,%d] out of range; reset offset to %d" .format(currentPartitionFetchState.offset, topic, partitionId, newOffset)) } catch { case e: Throwable => error("Error getting offset for partition [%s,%d] to broker %d".format(topic, partitionId, sourceBroker.id), e) - partitionsWithError += topicAndPartition + partitionsWithError += topicPartition } case _ => if (isRunning.get) { error("Error for partition [%s,%d] to broker %d:%s".format(topic, partitionId, sourceBroker.id, partitionData.exception.get)) - partitionsWithError += topicAndPartition + partitionsWithError += topicPartition } } }) @@ -181,22 +183,22 @@ abstract class AbstractFetcherThread(name: String, } } - def addPartitions(partitionAndOffsets: Map[TopicAndPartition, Long]) { + def addPartitions(partitionAndOffsets: Map[TopicPartition, Long]) { partitionMapLock.lockInterruptibly() try { - for ((topicAndPartition, offset) <- partitionAndOffsets) { + for ((topicPartition, offset) <- partitionAndOffsets) { // If the partitionMap already has the topic/partition, then do not update the map with the old offset - if (!partitionMap.contains(topicAndPartition)) + if (!partitionMap.contains(topicPartition)) partitionMap.put( - topicAndPartition, - if (PartitionTopicInfo.isOffsetInvalid(offset)) new PartitionFetchState(handleOffsetOutOfRange(topicAndPartition)) + topicPartition, + if (PartitionTopicInfo.isOffsetInvalid(offset)) new PartitionFetchState(handleOffsetOutOfRange(topicPartition)) else new PartitionFetchState(offset) )} partitionMapCond.signalAll() } finally partitionMapLock.unlock() } - def delayPartitions(partitions: Iterable[TopicAndPartition], delay: Long) { + def delayPartitions(partitions: Iterable[TopicPartition], delay: Long) { partitionMapLock.lockInterruptibly() try { for (partition <- partitions) { @@ -209,12 +211,12 @@ abstract class AbstractFetcherThread(name: String, } finally partitionMapLock.unlock() } - def removePartitions(topicAndPartitions: Set[TopicAndPartition]) { + def removePartitions(topicPartitions: Set[TopicPartition]) { partitionMapLock.lockInterruptibly() try { - topicAndPartitions.foreach { topicAndPartition => - partitionMap.remove(topicAndPartition) - fetcherLagStats.unregister(topicAndPartition.topic, topicAndPartition.partition) + topicPartitions.foreach { topicPartition => + partitionMap.remove(topicPartition) + fetcherLagStats.unregister(topicPartition.topic, topicPartition.partition) } } finally partitionMapLock.unlock() } @@ -231,7 +233,7 @@ object AbstractFetcherThread { trait FetchRequest { def isEmpty: Boolean - def offset(topicAndPartition: TopicAndPartition): Long + def offset(topicPartition: TopicPartition): Long } trait PartitionData { diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index b7c027f0d54f..e5c121c0c3c5 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -30,7 +30,6 @@ import org.apache.kafka.clients.{ManualMetadataUpdater, NetworkClient, ClientReq import org.apache.kafka.common.network.{LoginType, Selectable, ChannelBuilders, NetworkReceive, Selector, Mode} import org.apache.kafka.common.requests.{ListOffsetResponse, FetchResponse, RequestSend, AbstractRequest, ListOffsetRequest} import org.apache.kafka.common.requests.{FetchRequest => JFetchRequest} -import org.apache.kafka.common.security.ssl.SslFactory import org.apache.kafka.common.{Node, TopicPartition} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.{Errors, ApiKeys} @@ -112,22 +111,23 @@ class ReplicaFetcherThread(name: String, } // process fetched data - def processPartitionData(topicAndPartition: TopicAndPartition, fetchOffset: Long, partitionData: PartitionData) { + def processPartitionData(topicPartition: TopicPartition, fetchOffset: Long, partitionData: PartitionData) { try { - val TopicAndPartition(topic, partitionId) = topicAndPartition + val topic = topicPartition.topic + val partitionId = topicPartition.partition val replica = replicaMgr.getReplica(topic, partitionId).get val messageSet = partitionData.toByteBufferMessageSet - warnIfMessageOversized(messageSet, topicAndPartition) + warnIfMessageOversized(messageSet, topicPartition) if (fetchOffset != replica.logEndOffset.messageOffset) - throw new RuntimeException("Offset mismatch for partition %s: fetched offset = %d, log end offset = %d.".format(topicAndPartition, fetchOffset, replica.logEndOffset.messageOffset)) + throw new RuntimeException("Offset mismatch for partition %s: fetched offset = %d, log end offset = %d.".format(topicPartition, fetchOffset, replica.logEndOffset.messageOffset)) if (logger.isTraceEnabled) trace("Follower %d has replica log end offset %d for partition %s. Received %d messages and leader hw %d" - .format(replica.brokerId, replica.logEndOffset.messageOffset, topicAndPartition, messageSet.sizeInBytes, partitionData.highWatermark)) + .format(replica.brokerId, replica.logEndOffset.messageOffset, topicPartition, messageSet.sizeInBytes, partitionData.highWatermark)) replica.log.get.append(messageSet, assignOffsets = false) if (logger.isTraceEnabled) trace("Follower %d has replica log end offset %d after appending %d bytes of messages for partition %s" - .format(replica.brokerId, replica.logEndOffset.messageOffset, messageSet.sizeInBytes, topicAndPartition)) + .format(replica.brokerId, replica.logEndOffset.messageOffset, messageSet.sizeInBytes, topicPartition)) val followerHighWatermark = replica.logEndOffset.messageOffset.min(partitionData.highWatermark) // for the follower replica, we do not need to keep // its segment base offset the physical position, @@ -138,14 +138,14 @@ class ReplicaFetcherThread(name: String, .format(replica.brokerId, topic, partitionId, followerHighWatermark)) } catch { case e: KafkaStorageException => - fatal(s"Disk error while replicating data for $topicAndPartition", e) + fatal(s"Disk error while replicating data for $topicPartition", e) Runtime.getRuntime.halt(1) } } - def warnIfMessageOversized(messageSet: ByteBufferMessageSet, topicAndPartition: TopicAndPartition): Unit = { + def warnIfMessageOversized(messageSet: ByteBufferMessageSet, topicPartition: TopicPartition): Unit = { if (messageSet.sizeInBytes > 0 && messageSet.validBytes <= 0) - error(s"Replication is failing due to a message that is greater than replica.fetch.max.bytes for partition $topicAndPartition. " + + error(s"Replication is failing due to a message that is greater than replica.fetch.max.bytes for partition $topicPartition. " + "This generally occurs when the max.message.bytes has been overridden to exceed this value and a suitably large " + "message has also been sent. To fix this problem increase replica.fetch.max.bytes in your broker config to be " + "equal or larger than your settings for max.message.bytes, both at a broker and topic level.") @@ -154,8 +154,9 @@ class ReplicaFetcherThread(name: String, /** * Handle a partition whose offset is out of range and return a new fetch offset. */ - def handleOffsetOutOfRange(topicAndPartition: TopicAndPartition): Long = { - val replica = replicaMgr.getReplica(topicAndPartition.topic, topicAndPartition.partition).get + def handleOffsetOutOfRange(topicPartition: TopicPartition): Long = { + val topicAndPartition = TopicAndPartition(topicPartition.topic, topicPartition.partition) + val replica = replicaMgr.getReplica(topicPartition.topic, topicPartition.partition).get /** * Unclean leader election: A follower goes down, in the meanwhile the leader keeps appending messages. The follower comes back up @@ -167,7 +168,7 @@ class ReplicaFetcherThread(name: String, * * There is a potential for a mismatch between the logs of the two replicas here. We don't fix this mismatch as of now. */ - val leaderEndOffset: Long = earliestOrLatestOffset(topicAndPartition, ListOffsetRequest.LATEST_TIMESTAMP, + val leaderEndOffset: Long = earliestOrLatestOffset(topicPartition, ListOffsetRequest.LATEST_TIMESTAMP, brokerConfig.brokerId) if (leaderEndOffset < replica.logEndOffset.messageOffset) { @@ -175,16 +176,16 @@ class ReplicaFetcherThread(name: String, // This situation could only happen if the unclean election configuration for a topic changes while a replica is down. Otherwise, // we should never encounter this situation since a non-ISR leader cannot be elected if disallowed by the broker configuration. if (!LogConfig.fromProps(brokerConfig.originals, AdminUtils.fetchEntityConfig(replicaMgr.zkUtils, - ConfigType.Topic, topicAndPartition.topic)).uncleanLeaderElectionEnable) { + ConfigType.Topic, topicPartition.topic)).uncleanLeaderElectionEnable) { // Log a fatal error and shutdown the broker to ensure that data loss does not unexpectedly occur. - fatal("Exiting because log truncation is not allowed for partition %s,".format(topicAndPartition) + + fatal("Exiting because log truncation is not allowed for partition %s,".format(topicPartition) + " Current leader %d's latest offset %d is less than replica %d's latest offset %d" .format(sourceBroker.id, leaderEndOffset, brokerConfig.brokerId, replica.logEndOffset.messageOffset)) System.exit(1) } warn("Replica %d for partition %s reset its fetch offset from %d to current leader %d's latest offset %d" - .format(brokerConfig.brokerId, topicAndPartition, replica.logEndOffset.messageOffset, sourceBroker.id, leaderEndOffset)) + .format(brokerConfig.brokerId, topicPartition, replica.logEndOffset.messageOffset, sourceBroker.id, leaderEndOffset)) replicaMgr.logManager.truncateTo(Map(topicAndPartition -> leaderEndOffset)) leaderEndOffset } else { @@ -210,10 +211,10 @@ class ReplicaFetcherThread(name: String, * and the current leader's log start offset. * */ - val leaderStartOffset: Long = earliestOrLatestOffset(topicAndPartition, ListOffsetRequest.EARLIEST_TIMESTAMP, + val leaderStartOffset: Long = earliestOrLatestOffset(topicPartition, ListOffsetRequest.EARLIEST_TIMESTAMP, brokerConfig.brokerId) warn("Replica %d for partition %s reset its fetch offset from %d to current leader %d's start offset %d" - .format(brokerConfig.brokerId, topicAndPartition, replica.logEndOffset.messageOffset, sourceBroker.id, leaderStartOffset)) + .format(brokerConfig.brokerId, topicPartition, replica.logEndOffset.messageOffset, sourceBroker.id, leaderStartOffset)) val offsetToFetch = Math.max(leaderStartOffset, replica.logEndOffset.messageOffset) // Only truncate log when current leader's log start offset is greater than follower's log end offset. if (leaderStartOffset > replica.logEndOffset.messageOffset) @@ -223,14 +224,14 @@ class ReplicaFetcherThread(name: String, } // any logic for partitions whose leader has changed - def handlePartitionsWithErrors(partitions: Iterable[TopicAndPartition]) { + def handlePartitionsWithErrors(partitions: Iterable[TopicPartition]) { delayPartitions(partitions, brokerConfig.replicaFetchBackoffMs.toLong) } - protected def fetch(fetchRequest: FetchRequest): Map[TopicAndPartition, PartitionData] = { + protected def fetch(fetchRequest: FetchRequest): Map[TopicPartition, PartitionData] = { val clientResponse = sendRequest(ApiKeys.FETCH, Some(fetchRequestVersion), fetchRequest.underlying) new FetchResponse(clientResponse.responseBody).responseData.asScala.map { case (key, value) => - TopicAndPartition(key.topic, key.partition) -> new PartitionData(value) + key -> new PartitionData(value) } } @@ -254,8 +255,7 @@ class ReplicaFetcherThread(name: String, } - private def earliestOrLatestOffset(topicAndPartition: TopicAndPartition, earliestOrLatest: Long, consumerId: Int): Long = { - val topicPartition = new TopicPartition(topicAndPartition.topic, topicAndPartition.partition) + private def earliestOrLatestOffset(topicPartition: TopicPartition, earliestOrLatest: Long, consumerId: Int): Long = { val partitions = Map( topicPartition -> new ListOffsetRequest.PartitionData(earliestOrLatest, 1) ) @@ -269,17 +269,17 @@ class ReplicaFetcherThread(name: String, } } - protected def buildFetchRequest(partitionMap: Seq[(TopicAndPartition, PartitionFetchState)]): FetchRequest = { + protected def buildFetchRequest(partitionMap: Seq[(TopicPartition, PartitionFetchState)]): FetchRequest = { val requestMap = mutable.Map.empty[TopicPartition, JFetchRequest.PartitionData] - partitionMap.foreach { case ((TopicAndPartition(topic, partition), partitionFetchState)) => + partitionMap.foreach { case (topicPartition, partitionFetchState) => if (partitionFetchState.isActive) - requestMap(new TopicPartition(topic, partition)) = new JFetchRequest.PartitionData(partitionFetchState.offset, fetchSize) + requestMap(topicPartition) = new JFetchRequest.PartitionData(partitionFetchState.offset, fetchSize) } val request = - if (fetchRequestVersion >= 3) JFetchRequest.fromReplica(replicaId, maxWait, minBytes, maxBytes, requestMap) - else JFetchRequest.fromReplica(replicaId, maxWait, minBytes, requestMap) + if (fetchRequestVersion >= 3) JFetchRequest.fromReplica(replicaId, maxWait, minBytes, maxBytes, requestMap.asJava) + else JFetchRequest.fromReplica(replicaId, maxWait, minBytes, requestMap.asJava) new FetchRequest(request) } @@ -290,8 +290,8 @@ object ReplicaFetcherThread { private[server] class FetchRequest(val underlying: JFetchRequest) extends AbstractFetcherThread.FetchRequest { def isEmpty: Boolean = underlying.fetchData.isEmpty - def offset(topicAndPartition: TopicAndPartition): Long = - underlying.fetchData.asScala(new TopicPartition(topicAndPartition.topic, topicAndPartition.partition)).offset + def offset(topicPartition: TopicPartition): Long = + underlying.fetchData.asScala(topicPartition).offset } private[server] class PartitionData(val underlying: FetchResponse.PartitionData) extends AbstractFetcherThread.PartitionData { diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala index 0b13bab6cbe6..4c3f15637a04 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala @@ -19,10 +19,10 @@ package kafka.server import com.yammer.metrics.Metrics import kafka.cluster.BrokerEndPoint -import kafka.common.TopicAndPartition import kafka.message.ByteBufferMessageSet import kafka.server.AbstractFetcherThread.{FetchRequest, PartitionData} import kafka.utils.TestUtils +import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.Errors import org.junit.Assert.{assertFalse, assertTrue} import org.junit.{Before, Test} @@ -39,7 +39,7 @@ class AbstractFetcherThreadTest { @Test def testMetricsRemovedOnShutdown() { - val partition = new TopicAndPartition("topic", 0) + val partition = new TopicPartition("topic", 0) val fetcherThread = new DummyFetcherThread("dummy", "client", new BrokerEndPoint(0, "localhost", 9092)) fetcherThread.start() @@ -60,7 +60,7 @@ class AbstractFetcherThreadTest { @Test def testConsumerLagRemovedWithPartition() { - val partition = new TopicAndPartition("topic", 0) + val partition = new TopicPartition("topic", 0) val fetcherThread = new DummyFetcherThread("dummy", "client", new BrokerEndPoint(0, "localhost", 9092)) fetcherThread.start() @@ -83,10 +83,10 @@ class AbstractFetcherThreadTest { private def allMetricsNames = Metrics.defaultRegistry().allMetrics().asScala.keySet.map(_.getName) - class DummyFetchRequest(val offsets: collection.Map[TopicAndPartition, Long]) extends FetchRequest { + class DummyFetchRequest(val offsets: collection.Map[TopicPartition, Long]) extends FetchRequest { override def isEmpty: Boolean = offsets.isEmpty - override def offset(topicAndPartition: TopicAndPartition): Long = offsets(topicAndPartition) + override def offset(topicAndPartition: TopicPartition): Long = offsets(topicAndPartition) } class DummyPartitionData extends PartitionData { @@ -107,20 +107,20 @@ class AbstractFetcherThreadTest { type REQ = DummyFetchRequest type PD = PartitionData - override def processPartitionData(topicAndPartition: TopicAndPartition, + override def processPartitionData(topicAndPartition: TopicPartition, fetchOffset: Long, partitionData: PartitionData): Unit = {} - override def handleOffsetOutOfRange(topicAndPartition: TopicAndPartition): Long = 0L + override def handleOffsetOutOfRange(topicAndPartition: TopicPartition): Long = 0L - override def handlePartitionsWithErrors(partitions: Iterable[TopicAndPartition]): Unit = {} + override def handlePartitionsWithErrors(partitions: Iterable[TopicPartition]): Unit = {} - override protected def fetch(fetchRequest: DummyFetchRequest): collection.Map[TopicAndPartition, DummyPartitionData] = { + override protected def fetch(fetchRequest: DummyFetchRequest): collection.Map[TopicPartition, DummyPartitionData] = { fetchRequest.offsets.mapValues(_ => new DummyPartitionData) } - override protected def buildFetchRequest(partitionMap: collection.Seq[(TopicAndPartition, PartitionFetchState)]): DummyFetchRequest = { - new DummyFetchRequest(partitionMap.map(kv => (kv._1, kv._2.offset)).toMap) + override protected def buildFetchRequest(partitionMap: collection.Seq[(TopicPartition, PartitionFetchState)]): DummyFetchRequest = { + new DummyFetchRequest(partitionMap.map { case (k, v) => (k, v.offset) }.toMap) } } From df15bb73dea722c80d2ee352b660cd6ef01ca4de Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 14 Sep 2016 01:49:06 +0100 Subject: [PATCH 12/54] Implement round-robin for `Fetcher` and `*FetcherThread` classes --- .../clients/consumer/internals/Fetcher.java | 20 ++- .../consumer/internals/SubscriptionState.java | 80 ++++++---- .../kafka/common/internals/FetchBuilder.java | 150 ++++++++++++++++++ .../kafka/common/requests/FetchRequest.java | 4 +- .../common/requests/RequestResponseTest.java | 4 +- .../kafka/server/AbstractFetcherThread.scala | 78 +++++---- .../kafka/server/ReplicaFetcherThread.scala | 9 +- .../kafka/api/AuthorizerIntegrationTest.scala | 5 +- 8 files changed, 272 insertions(+), 78 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/internals/FetchBuilder.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 9b0f98808474..bbcfdedaf7c3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -62,6 +62,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Locale; import java.util.Map; @@ -517,8 +518,8 @@ private void handleListOffsetResponse(TopicPartition topicPartition, } } - private Set fetchablePartitions() { - Set fetchable = subscriptions.fetchablePartitions(); + private List fetchablePartitions() { + List fetchable = subscriptions.fetchablePartitions(); if (nextInLineRecords != null && !nextInLineRecords.isEmpty()) fetchable.remove(nextInLineRecords.partition); for (CompletedFetch completedFetch : completedFetches) @@ -533,16 +534,16 @@ private Set fetchablePartitions() { private Map createFetchRequests() { // create the fetch info Cluster cluster = metadata.fetch(); - Map> fetchable = new HashMap<>(); + Map> fetchable = new LinkedHashMap<>(); for (TopicPartition partition : fetchablePartitions()) { Node node = cluster.leaderFor(partition); if (node == null) { metadata.requestUpdate(); } else if (this.client.pendingRequestCount(node) == 0) { // if there is a leader and no in-flight requests, issue a new fetch - Map fetch = fetchable.get(node); + LinkedHashMap fetch = fetchable.get(node); if (fetch == null) { - fetch = new HashMap<>(); + fetch = new LinkedHashMap<>(); fetchable.put(node, fetch); } @@ -556,7 +557,7 @@ private Map createFetchRequests() { // create the fetches Map requests = new HashMap<>(); - for (Map.Entry> entry : fetchable.entrySet()) { + for (Map.Entry> entry : fetchable.entrySet()) { Node node = entry.getKey(); FetchRequest fetch = new FetchRequest(this.maxWaitMs, this.minBytes, this.maxBytes, entry.getValue()); requests.put(node, fetch); @@ -574,6 +575,7 @@ private PartitionRecords parseFetchedData(CompletedFetch completedFetch) { int bytes = 0; int recordsCount = 0; PartitionRecords parsedRecords = null; + boolean emptyResponse = false; try { if (!subscriptions.isFetchable(tp)) { @@ -604,6 +606,9 @@ private PartitionRecords parseFetchedData(CompletedFetch completedFetch) { } } + if (!skippedRecords && bytes == 0) + emptyResponse = true; + recordsCount = parsed.size(); this.sensors.recordTopicFetchMetrics(tp.topic(), bytes, recordsCount); @@ -651,6 +656,9 @@ private PartitionRecords parseFetchedData(CompletedFetch completedFetch) { completedFetch.metricAggregator.record(tp, bytes, recordsCount); } + if (!emptyResponse) + subscriptions.movePartitionToEnd(tp); + return parsedRecords; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java index f3316d2a8613..0cd3f4535779 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java @@ -16,11 +16,14 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.internals.FetchBuilder; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.regex.Pattern; @@ -68,8 +71,8 @@ private enum SubscriptionType { /* the list of topics the group has subscribed to (set only for the leader on join group completion) */ private final Set groupSubscription; - /* the list of partitions currently assigned */ - private final Map assignment; + /* the partitions that are currently assigned, note that the order of partition matters (see FetchBuilder for more details) */ + private final FetchBuilder assignment; /* do we need to request the latest committed offsets from the coordinator? */ private boolean needsFetchCommittedOffsets; @@ -84,7 +87,7 @@ public SubscriptionState(OffsetResetStrategy defaultResetStrategy) { this.defaultResetStrategy = defaultResetStrategy; this.subscription = Collections.emptySet(); this.userAssignment = Collections.emptySet(); - this.assignment = new HashMap<>(); + this.assignment = new FetchBuilder<>(); this.groupSubscription = new HashSet<>(); this.needsFetchCommittedOffsets = true; // initialize to true for the consumers to fetch offset upon starting up this.subscribedPattern = null; @@ -156,13 +159,17 @@ public void resetGroupSubscription() { public void assignFromUser(Set partitions) { setSubscriptionType(SubscriptionType.USER_ASSIGNED); - if (!this.assignment.keySet().equals(partitions)) { + if (!this.assignment.partitionSet().equals(partitions)) { this.userAssignment = partitions; - for (TopicPartition partition : partitions) - if (!assignment.containsKey(partition)) - addAssignedPartition(partition); - this.assignment.keySet().retainAll(this.userAssignment); + Map partitionToState = new HashMap<>(); + for (TopicPartition partition : partitions) { + TopicPartitionState state = assignment.stateValue(partition); + if (state == null) + state = new TopicPartitionState(); + partitionToState.put(partition, state); + } + this.assignment.set(partitionToState); this.needsFetchCommittedOffsets = true; } } @@ -179,13 +186,18 @@ public void assignFromSubscribed(Collection assignments) { if (!this.subscription.contains(tp.topic())) throw new IllegalArgumentException("Assigned partition " + tp + " for non-subscribed topic."); - // after rebalancing, we always reinitialize the assignment state - this.assignment.clear(); - for (TopicPartition tp: assignments) - addAssignedPartition(tp); + // after rebalancing, we always reinitialize the assignment value + this.assignment.set(partitionToStateMap(assignments)); this.needsFetchCommittedOffsets = true; } + private Map partitionToStateMap(Collection assignments) { + Map map = new HashMap<>(assignments.size()); + for (TopicPartition tp : assignments) + map.put(tp, new TopicPartitionState()); + return map; + } + public void subscribe(Pattern pattern, ConsumerRebalanceListener listener) { if (listener == null) throw new IllegalArgumentException("RebalanceListener cannot be null"); @@ -218,11 +230,9 @@ public Set subscription() { public Set pausedPartitions() { HashSet paused = new HashSet<>(); - for (Map.Entry entry : assignment.entrySet()) { - final TopicPartition tp = entry.getKey(); - final TopicPartitionState state = entry.getValue(); - if (state.paused) { - paused.add(tp); + for (FetchBuilder.PartitionState state : assignment.partitionStates()) { + if (state.value().paused) { + paused.add(state.topicPartition()); } } return paused; @@ -243,7 +253,7 @@ public Set groupSubscription() { } private TopicPartitionState assignedState(TopicPartition tp) { - TopicPartitionState state = this.assignment.get(tp); + TopicPartitionState state = this.assignment.stateValue(tp); if (state == null) throw new IllegalStateException("No current assignment for partition " + tp); return state; @@ -274,14 +284,14 @@ public void seek(TopicPartition tp, long offset) { } public Set assignedPartitions() { - return this.assignment.keySet(); + return this.assignment.partitionSet(); } - public Set fetchablePartitions() { - Set fetchable = new HashSet<>(); - for (Map.Entry entry : assignment.entrySet()) { - if (entry.getValue().isFetchable()) - fetchable.add(entry.getKey()); + public List fetchablePartitions() { + List fetchable = new ArrayList<>(); + for (FetchBuilder.PartitionState state : assignment.partitionStates()) { + if (state.value().isFetchable()) + fetchable.add(state.topicPartition()); } return fetchable; } @@ -300,10 +310,9 @@ public Long position(TopicPartition tp) { public Map allConsumed() { Map allConsumed = new HashMap<>(); - for (Map.Entry entry : assignment.entrySet()) { - TopicPartitionState state = entry.getValue(); - if (state.hasValidPosition()) - allConsumed.put(entry.getKey(), new OffsetAndMetadata(state.position)); + for (FetchBuilder.PartitionState state : assignment.partitionStates()) { + if (state.value().hasValidPosition()) + allConsumed.put(state.topicPartition(), new OffsetAndMetadata(state.value().position)); } return allConsumed; } @@ -329,7 +338,7 @@ public OffsetResetStrategy resetStrategy(TopicPartition partition) { } public boolean hasAllFetchPositions() { - for (TopicPartitionState state : assignment.values()) + for (TopicPartitionState state : assignment.partitionStateValues()) if (!state.hasValidPosition()) return false; return true; @@ -337,14 +346,15 @@ public boolean hasAllFetchPositions() { public Set missingFetchPositions() { Set missing = new HashSet<>(); - for (Map.Entry entry : assignment.entrySet()) - if (!entry.getValue().hasValidPosition()) - missing.add(entry.getKey()); + for (FetchBuilder.PartitionState state : assignment.partitionStates()) { + if (!state.value().hasValidPosition()) + missing.add(state.topicPartition()); + } return missing; } public boolean isAssigned(TopicPartition tp) { - return assignment.containsKey(tp); + return assignment.contains(tp); } public boolean isPaused(TopicPartition tp) { @@ -363,8 +373,8 @@ public void resume(TopicPartition tp) { assignedState(tp).resume(); } - private void addAssignedPartition(TopicPartition tp) { - this.assignment.put(tp, new TopicPartitionState()); + public void movePartitionToEnd(TopicPartition tp) { + assignment.moveToEnd(tp); } public ConsumerRebalanceListener listener() { diff --git a/clients/src/main/java/org/apache/kafka/common/internals/FetchBuilder.java b/clients/src/main/java/org/apache/kafka/common/internals/FetchBuilder.java new file mode 100644 index 000000000000..6292bef5ec38 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/internals/FetchBuilder.java @@ -0,0 +1,150 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.common.internals; + +import org.apache.kafka.common.TopicPartition; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +//TODO Better name? +public class FetchBuilder { + + private final List> list = new ArrayList<>(); + private final Map map = new HashMap<>(); + + public FetchBuilder() {} + + public void moveToEnd(TopicPartition topicPartition) { + for (int i = 0; i < list.size(); ++i) { + PartitionState state = list.get(i); + if (state.topicPartition.equals(topicPartition)) { + list.remove(i); + list.add(state); + break; + } + } + } + + public void updateAndMoveToEnd(TopicPartition topicPartition, S state) { + PartitionState partitionState = new PartitionState<>(topicPartition, state); + S prev = map.put(topicPartition, state); + if (prev != null) { + for (int i = 0; i < list.size(); ++i) { + if (list.get(i).topicPartition.equals(topicPartition)) { + list.remove(i); + list.add(partitionState); + break; + } + } + } else + list.add(partitionState); + } + + public void remove(TopicPartition topicPartition) { + map.remove(topicPartition); + for (Iterator> it = list.iterator(); it.hasNext(); ) { + PartitionState state = it.next(); + if (state.topicPartition.equals(topicPartition)) { + it.remove(); + break; + } + } + } + + public Set partitionSet() { + return map.keySet(); + } + + public void clear() { + map.clear(); + list.clear(); + } + + public boolean contains(TopicPartition topicPartition) { + return map.containsKey(topicPartition); + } + + public List> partitionStates() { + return Collections.unmodifiableList(list); + } + + public List partitionStateValues() { + List result = new ArrayList<>(list.size()); + for (PartitionState state : list) + result.add(state.value); + return result; + } + + public S stateValue(TopicPartition topicPartition) { + return map.get(topicPartition); + } + + public int size() { + return map.size(); + } + + public void set(Map partitionToState) { + map.clear(); + list.clear(); + update(partitionToState); + } + + private void update(Map partitionToState) { + Map> topicToPartitions = new HashMap<>(); + for (TopicPartition tp : partitionToState.keySet()) { + List partitions = topicToPartitions.get(tp.topic()); + if (partitions == null) { + partitions = new ArrayList<>(); + topicToPartitions.put(tp.topic(), partitions); + } + partitions.add(tp); + } + for (Map.Entry> entry : topicToPartitions.entrySet()) { + for (TopicPartition tp : entry.getValue()) { + S state = partitionToState.get(tp); + map.put(tp, state); + list.add(new PartitionState<>(tp, state)); + } + } + } + + public static class PartitionState { + private final TopicPartition topicPartition; + private final S value; + + public PartitionState(TopicPartition topicPartition, S state) { + this.topicPartition = topicPartition; + this.value = state; + } + + public S value() { + return value; + } + + public TopicPartition topicPartition() { + return topicPartition; + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java index 19bb29e5f6fc..034050c2c84d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -98,7 +98,7 @@ public FetchRequest(int maxWait, int minBytes, Map fetchData) { + public FetchRequest(int maxWait, int minBytes, int maxBytes, LinkedHashMap fetchData) { this(ProtoUtils.latestVersion(ApiKeys.FETCH.id), CONSUMER_REPLICA_ID, maxWait, minBytes, maxBytes, fetchData); } @@ -116,7 +116,7 @@ public static FetchRequest fromReplica(int replicaId, int maxWait, int minBytes, * Create a replica fetch request for the current version. */ public static FetchRequest fromReplica(int replicaId, int maxWait, int minBytes, int maxBytes, - Map fetchData) { + LinkedHashMap fetchData) { return new FetchRequest(ProtoUtils.latestVersion(ApiKeys.FETCH.id), replicaId, maxWait, minBytes, maxBytes, fetchData); } 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 766c7453f49f..2d30d4d120f1 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 @@ -217,11 +217,13 @@ private AbstractRequestResponse createGroupCoordinatorResponse() { return new GroupCoordinatorResponse(Errors.NONE.code(), new Node(10, "host1", 2014)); } + //TODO Add fetch tests for various versions including for error response + @SuppressWarnings("deprecation") private AbstractRequest createFetchRequest() { Map fetchData = new HashMap<>(); fetchData.put(new TopicPartition("test1", 0), new FetchRequest.PartitionData(100, 1000000)); fetchData.put(new TopicPartition("test2", 0), new FetchRequest.PartitionData(200, 1000000)); - return new FetchRequest(-1, 100, 100000, fetchData); + return new FetchRequest(100, 100000, fetchData); } private AbstractRequestResponse createFetchResponse() { diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index 759a919114b1..933603166ff1 100755 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -21,20 +21,23 @@ import java.util.concurrent.locks.ReentrantLock import kafka.cluster.BrokerEndPoint import kafka.consumer.PartitionTopicInfo -import kafka.message.{MessageAndOffset, ByteBufferMessageSet} -import kafka.utils.{Pool, ShutdownableThread, DelayedItem} -import kafka.common.{KafkaException, ClientIdAndBroker, TopicAndPartition} +import kafka.message.{ByteBufferMessageSet} +import kafka.utils.{DelayedItem, Pool, ShutdownableThread} +import kafka.common.{ClientIdAndBroker, KafkaException} import kafka.metrics.KafkaMetricsGroup import kafka.utils.CoreUtils.inLock import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.protocol.Errors import AbstractFetcherThread._ -import scala.collection.{mutable, Set, Map} + +import scala.collection.{Map, Set, mutable} +import scala.collection.JavaConverters._ import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicLong import com.yammer.metrics.core.Gauge import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.internals.FetchBuilder /** * Abstract class for fetching data from multiple partitions from the same broker. @@ -49,7 +52,7 @@ abstract class AbstractFetcherThread(name: String, type REQ <: FetchRequest type PD <: PartitionData - private val partitionMap = new mutable.HashMap[TopicPartition, PartitionFetchState] // a (topic, partition) -> partitionFetchState map + private val partitionMap = new FetchBuilder[PartitionFetchState] private val partitionMapLock = new ReentrantLock private val partitionMapCond = partitionMapLock.newCondition() @@ -87,7 +90,9 @@ abstract class AbstractFetcherThread(name: String, override def doWork() { val fetchRequest = inLock(partitionMapLock) { - val fetchRequest = buildFetchRequest(partitionMap.toSeq) + val fetchRequest = buildFetchRequest(partitionMap.partitionStates.asScala.map { state => + state.topicPartition -> state.value + }) if (fetchRequest.isEmpty) { trace("There are no active partitions. Back off for %d ms before sending a fetch request".format(fetchBackOffMs)) partitionMapCond.await(fetchBackOffMs, TimeUnit.MILLISECONDS) @@ -100,7 +105,13 @@ abstract class AbstractFetcherThread(name: String, } private def processFetchRequest(fetchRequest: REQ) { - val partitionsWithError = new mutable.HashSet[TopicPartition] + val partitionsWithError = mutable.Set[TopicPartition]() + + def updatePartitionsWithError(partition: TopicPartition): Unit = { + partitionsWithError += partition + partitionMap.moveToEnd(partition) + } + var responseData: Map[TopicPartition, PD] = Map.empty try { @@ -111,8 +122,10 @@ abstract class AbstractFetcherThread(name: String, if (isRunning.get) { warn(s"Error in fetch $fetchRequest", t) inLock(partitionMapLock) { - partitionsWithError ++= partitionMap.keys + partitionMap.partitionSet.asScala.foreach(updatePartitionsWithError) // there is an error occurred while fetching partitions, sleep a while + // note that `ReplicaFetcherThread.handlePartitionsWithError` will also introduce the same delay for every + // partition with error effectively doubling the delay. It would be good to improve this. partitionMapCond.await(fetchBackOffMs, TimeUnit.MILLISECONDS) } } @@ -126,21 +139,23 @@ abstract class AbstractFetcherThread(name: String, responseData.foreach { case (topicPartition, partitionData) => val topic = topicPartition.topic val partitionId = topicPartition.partition - partitionMap.get(topicPartition).foreach(currentPartitionFetchState => + Option(partitionMap.stateValue(topicPartition)).foreach(currentPartitionFetchState => // we append to the log if the current offset is defined and it is the same as the offset requested during fetch if (fetchRequest.offset(topicPartition) == currentPartitionFetchState.offset) { Errors.forCode(partitionData.errorCode) match { case Errors.NONE => try { val messages = partitionData.toByteBufferMessageSet - val validBytes = messages.validBytes val newOffset = messages.shallowIterator.toSeq.lastOption match { - case Some(m: MessageAndOffset) => m.nextOffset - case None => currentPartitionFetchState.offset + case Some(m) => + partitionMap.updateAndMoveToEnd(topicPartition, new PartitionFetchState(m.nextOffset)) + fetcherStats.byteRate.mark(messages.validBytes) + m.nextOffset + case None => + currentPartitionFetchState.offset } - partitionMap.put(topicPartition, new PartitionFetchState(newOffset)) + fetcherLagStats.getAndMaybePut(topic, partitionId).lag = Math.max(0L, partitionData.highWatermark - newOffset) - fetcherStats.byteRate.mark(validBytes) // Once we hand off the partition data to the subclass, we can't mess with it any more in this thread processPartitionData(topicPartition, currentPartitionFetchState.offset, partitionData) } catch { @@ -157,19 +172,19 @@ abstract class AbstractFetcherThread(name: String, case Errors.OFFSET_OUT_OF_RANGE => try { val newOffset = handleOffsetOutOfRange(topicPartition) - partitionMap.put(topicPartition, new PartitionFetchState(newOffset)) + partitionMap.updateAndMoveToEnd(topicPartition, new PartitionFetchState(newOffset)) error("Current offset %d for partition [%s,%d] out of range; reset offset to %d" .format(currentPartitionFetchState.offset, topic, partitionId, newOffset)) } catch { case e: Throwable => error("Error getting offset for partition [%s,%d] to broker %d".format(topic, partitionId, sourceBroker.id), e) - partitionsWithError += topicPartition + updatePartitionsWithError(topicPartition) } case _ => if (isRunning.get) { error("Error for partition [%s,%d] to broker %d:%s".format(topic, partitionId, sourceBroker.id, partitionData.exception.get)) - partitionsWithError += topicPartition + updatePartitionsWithError(topicPartition) } } }) @@ -186,14 +201,19 @@ abstract class AbstractFetcherThread(name: String, def addPartitions(partitionAndOffsets: Map[TopicPartition, Long]) { partitionMapLock.lockInterruptibly() try { - for ((topicPartition, offset) <- partitionAndOffsets) { - // If the partitionMap already has the topic/partition, then do not update the map with the old offset - if (!partitionMap.contains(topicPartition)) - partitionMap.put( - topicPartition, - if (PartitionTopicInfo.isOffsetInvalid(offset)) new PartitionFetchState(handleOffsetOutOfRange(topicPartition)) - else new PartitionFetchState(offset) - )} + // If the partitionMap already has the topic/partition, then do not update the map with the old offset + val newPartitionToState = partitionAndOffsets.filter { case (tp, _) => + !partitionMap.contains(tp) + }.map { case (tp, offset) => + val fetchState = + if (PartitionTopicInfo.isOffsetInvalid(offset)) new PartitionFetchState(handleOffsetOutOfRange(tp)) + else new PartitionFetchState(offset) + tp -> fetchState + } + val existingPartitionToState = partitionMap.partitionStates.asScala.map { state => + state.topicPartition -> state.value + }.toMap + partitionMap.set((existingPartitionToState ++ newPartitionToState).asJava) partitionMapCond.signalAll() } finally partitionMapLock.unlock() } @@ -202,9 +222,9 @@ abstract class AbstractFetcherThread(name: String, partitionMapLock.lockInterruptibly() try { for (partition <- partitions) { - partitionMap.get(partition).foreach (currentPartitionFetchState => + Option(partitionMap.stateValue(partition)).foreach (currentPartitionFetchState => if (currentPartitionFetchState.isActive) - partitionMap.put(partition, new PartitionFetchState(currentPartitionFetchState.offset, new DelayedItem(delay))) + partitionMap.updateAndMoveToEnd(partition, new PartitionFetchState(currentPartitionFetchState.offset, new DelayedItem(delay))) ) } partitionMapCond.signalAll() @@ -318,13 +338,13 @@ case class ClientIdTopicPartition(clientId: String, topic: String, partitionId: } /** - * case class to keep partition offset and its state(active , inactive) + * case class to keep partition offset and its state(active, inactive) */ case class PartitionFetchState(offset: Long, delay: DelayedItem) { def this(offset: Long) = this(offset, new DelayedItem(0)) - def isActive: Boolean = { delay.getDelay(TimeUnit.MILLISECONDS) == 0 } + def isActive: Boolean = delay.getDelay(TimeUnit.MILLISECONDS) == 0 override def toString = "%d-%b".format(offset, isActive) } diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index e5c121c0c3c5..e85a49163324 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -18,6 +18,7 @@ package kafka.server import java.net.SocketTimeoutException +import java.util import kafka.admin.AdminUtils import kafka.cluster.BrokerEndPoint @@ -270,16 +271,16 @@ class ReplicaFetcherThread(name: String, } protected def buildFetchRequest(partitionMap: Seq[(TopicPartition, PartitionFetchState)]): FetchRequest = { - val requestMap = mutable.Map.empty[TopicPartition, JFetchRequest.PartitionData] + val requestMap = new util.LinkedHashMap[TopicPartition, JFetchRequest.PartitionData] partitionMap.foreach { case (topicPartition, partitionFetchState) => if (partitionFetchState.isActive) - requestMap(topicPartition) = new JFetchRequest.PartitionData(partitionFetchState.offset, fetchSize) + requestMap.put(topicPartition, new JFetchRequest.PartitionData(partitionFetchState.offset, fetchSize)) } val request = - if (fetchRequestVersion >= 3) JFetchRequest.fromReplica(replicaId, maxWait, minBytes, maxBytes, requestMap.asJava) - else JFetchRequest.fromReplica(replicaId, maxWait, minBytes, requestMap.asJava) + if (fetchRequestVersion >= 3) JFetchRequest.fromReplica(replicaId, maxWait, minBytes, maxBytes, requestMap) + else JFetchRequest.fromReplica(replicaId, maxWait, minBytes, requestMap) new FetchRequest(request) } diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index cc2b3c3ec6b0..6d3b098ebc9a 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -13,6 +13,7 @@ package kafka.api import java.nio.ByteBuffer +import java.util import java.util.concurrent.ExecutionException import java.util.{ArrayList, Collections, Properties} @@ -176,7 +177,9 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } private def createFetchRequest = { - new requests.FetchRequest(5000, 100, Int.MaxValue, Map(tp -> new requests.FetchRequest.PartitionData(0, 100)).asJava) + val partitionMap = new util.LinkedHashMap[TopicPartition, requests.FetchRequest.PartitionData] + partitionMap.put(tp, new requests.FetchRequest.PartitionData(0, 100)) + new requests.FetchRequest(5000, 100, Int.MaxValue, partitionMap) } private def createListOffsetsRequest = { From e114821806bc0544baab012631859fc7d6e62bd2 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 00:09:44 +0100 Subject: [PATCH 13/54] Flesh out tests in `RequestResponseTest` --- .../common/requests/RequestResponseTest.java | 25 +++++++++++++------ 1 file changed, 18 insertions(+), 7 deletions(-) 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 2d30d4d120f1..f211bb2cf080 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 @@ -30,6 +30,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -49,8 +50,8 @@ public void testSerialization() throws Exception { createControlledShutdownRequest(), createControlledShutdownResponse(), createControlledShutdownRequest().getErrorResponse(1, new UnknownServerException()), - createFetchRequest(), - createFetchRequest().getErrorResponse(1, new UnknownServerException()), + createFetchRequest(3), + createFetchRequest(3).getErrorResponse(3, new UnknownServerException()), createFetchResponse(), createHeartBeatRequest(), createHeartBeatRequest().getErrorResponse(0, new UnknownServerException()), @@ -114,7 +115,7 @@ public void testSerialization() throws Exception { createMetadataResponse(0); createMetadataRequest(Arrays.asList("topic1")).getErrorResponse(0, new UnknownServerException()); - checkSerialization(createFetchRequest().getErrorResponse(0, new UnknownServerException()), 0); + checkOlderFetchVersions(); checkSerialization(createOffsetCommitRequest(0), 0); checkSerialization(createOffsetCommitRequest(0).getErrorResponse(0, new UnknownServerException()), 0); checkSerialization(createOffsetCommitRequest(1), 1); @@ -127,6 +128,14 @@ public void testSerialization() throws Exception { checkSerialization(createUpdateMetadataRequest(1, null).getErrorResponse(1, new UnknownServerException()), 1); } + private void checkOlderFetchVersions() throws Exception { + int latestVersion = ProtoUtils.latestVersion(ApiKeys.FETCH.id); + for (int i = 0; i < latestVersion; ++i) { + checkSerialization(createFetchRequest(i).getErrorResponse(i, new UnknownServerException()), i); + checkSerialization(createFetchRequest(i), i); + } + } + private void checkSerialization(AbstractRequestResponse req, Integer version) throws Exception { ByteBuffer buffer = ByteBuffer.allocate(req.sizeOf()); req.writeTo(buffer); @@ -217,13 +226,15 @@ private AbstractRequestResponse createGroupCoordinatorResponse() { return new GroupCoordinatorResponse(Errors.NONE.code(), new Node(10, "host1", 2014)); } - //TODO Add fetch tests for various versions including for error response @SuppressWarnings("deprecation") - private AbstractRequest createFetchRequest() { - Map fetchData = new HashMap<>(); + private AbstractRequest createFetchRequest(int version) { + LinkedHashMap fetchData = new LinkedHashMap<>(); fetchData.put(new TopicPartition("test1", 0), new FetchRequest.PartitionData(100, 1000000)); fetchData.put(new TopicPartition("test2", 0), new FetchRequest.PartitionData(200, 1000000)); - return new FetchRequest(100, 100000, fetchData); + if (version < 3) + return new FetchRequest(100, 100000, fetchData); + else + return new FetchRequest(100, 1000, 1000000, fetchData); } private AbstractRequestResponse createFetchResponse() { From 60643b6e4201fc9c7566ff6b3e9f8b3e8152b702 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 00:10:10 +0100 Subject: [PATCH 14/54] Flesh out config descriptions in `ConsumerConfig` --- .../apache/kafka/clients/consumer/ConsumerConfig.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index b59c3c70ba66..972916228dcd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -116,7 +116,10 @@ public class ConsumerConfig extends AbstractConfig { * fetch.max.bytes */ public static final String FETCH_MAX_BYTES_CONFIG = "fetch.max.bytes"; - private static final String FETCH_MAX_BYTES_DOC = "The maximum amount of data the server should return for a fetch request. This is not an absolute maximum - if there is a single message which is larger than fetch.max.bytes, it will still be returned."; + private static final String FETCH_MAX_BYTES_DOC = "The maximum amount of data the server should return for a fetch request. " + + "This is not an absolute maximum - if the first message in the first partition of the fetch is larger than " + + "this value, the message will still be returned to ensure that the consumer can make progress. Also note " + + "that the consumer performs multiple fetches in parallel."; public static final int DEFAULT_FETCH_MAX_BYTES = 50 * 1024 * 1024; /** @@ -132,7 +135,10 @@ public class ConsumerConfig extends AbstractConfig { * max.partition.fetch.bytes */ public static final String MAX_PARTITION_FETCH_BYTES_CONFIG = "max.partition.fetch.bytes"; - private static final String MAX_PARTITION_FETCH_BYTES_DOC = "The maximum amount of data per-partition the server will return. The maximum total memory used for a request will be #partitions * max.partition.fetch.bytes. This size must be at least as large as the maximum message size the server allows or else it is possible for the producer to send messages larger than the consumer can fetch. If that happens, the consumer can get stuck trying to fetch a large message on a certain partition."; + private static final String MAX_PARTITION_FETCH_BYTES_DOC = "The maximum amount of data per-partition the server " + + "will return. If the first message in the first partition of the fetch is larger than this limit, the message will still " + + "be returned to ensure that the consumer can make progress. See " + FETCH_MAX_BYTES_CONFIG + " for limiting " + + "the request size."; public static final int DEFAULT_MAX_PARTITION_FETCH_BYTES = 1 * 1024 * 1024; /** send.buffer.bytes */ From a2b8b1bc327b22bbd12ebb7add94506b9051f425 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 02:23:06 +0100 Subject: [PATCH 15/54] Various compatibility and ordering changes to fetch request/response classes --- .../clients/consumer/internals/Fetcher.java | 2 +- .../kafka/common/requests/FetchRequest.java | 9 +- .../kafka/common/requests/FetchResponse.java | 83 +++++++++++-------- .../main/scala/kafka/api/FetchRequest.scala | 71 +++++++++++----- .../main/scala/kafka/api/FetchResponse.scala | 45 ++++------ .../consumer/ConsumerFetcherThread.scala | 11 +-- .../scala/kafka/javaapi/FetchRequest.scala | 48 +++++++---- .../kafka/server/AbstractFetcherThread.scala | 4 +- .../main/scala/kafka/server/KafkaApis.scala | 7 +- .../kafka/server/ReplicaFetcherThread.scala | 4 +- .../server/AbstractFetcherThreadTest.scala | 8 +- 11 files changed, 164 insertions(+), 128 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index bbcfdedaf7c3..7c385f83389e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -156,7 +156,7 @@ private boolean matchesRequestedPartitions(FetchRequest request, FetchResponse r * an in-flight fetch or pending fetch data. */ public void sendFetches() { - for (Map.Entry fetchEntry: createFetchRequests().entrySet()) { + for (Map.Entry fetchEntry : createFetchRequests().entrySet()) { final FetchRequest request = fetchEntry.getValue(); final Node fetchTarget = fetchEntry.getKey(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java index 034050c2c84d..4980f8639635 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -72,7 +72,7 @@ public TopicAndPartitionData(String topic) { this.partitions = new LinkedHashMap<>(); } - public static final List groupByTopicOrdered(Map fetchData) { + public static final List groupByTopicOrdered(LinkedHashMap fetchData) { List topics = new ArrayList<>(); for (Map.Entry topicEntry : fetchData.entrySet()) { String topic = topicEntry.getKey().topic(); @@ -92,7 +92,7 @@ public static final List groupByTopicOrdered(Map fetchData) { // Any of 0, 1 or 2 would do here - this(2, CONSUMER_REPLICA_ID, maxWait, minBytes, DEFAULT_RESPONSE_MAX_BYTES, fetchData); + this(2, CONSUMER_REPLICA_ID, maxWait, minBytes, DEFAULT_RESPONSE_MAX_BYTES, new LinkedHashMap<>(fetchData)); } /** @@ -109,7 +109,7 @@ public FetchRequest(int maxWait, int minBytes, int maxBytes, LinkedHashMap fetchData) { // Any of 0, 1 or 2 would do here - return new FetchRequest(2, replicaId, maxWait, minBytes, DEFAULT_RESPONSE_MAX_BYTES, fetchData); + return new FetchRequest(2, replicaId, maxWait, minBytes, DEFAULT_RESPONSE_MAX_BYTES, new LinkedHashMap<>(fetchData)); } /** @@ -120,7 +120,8 @@ public static FetchRequest fromReplica(int replicaId, int maxWait, int minBytes, return new FetchRequest(ProtoUtils.latestVersion(ApiKeys.FETCH.id), replicaId, maxWait, minBytes, maxBytes, fetchData); } - private FetchRequest(int version, int replicaId, int maxWait, int minBytes, int maxBytes, Map fetchData) { + private FetchRequest(int version, int replicaId, int maxWait, int minBytes, int maxBytes, + LinkedHashMap fetchData) { super(new Struct(ProtoUtils.requestSchema(ApiKeys.FETCH.id, version))); List topicsData = TopicAndPartitionData.groupByTopicOrdered(fetchData); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java index 5254f6bbe86a..ee1c3debdf6e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java @@ -85,52 +85,38 @@ public PartitionData(short errorCode, long highWatermark, ByteBuffer recordSet) * @param responseData fetched data grouped by topic-partition */ public FetchResponse(Map responseData) { - super(new Struct(ProtoUtils.responseSchema(ApiKeys.FETCH.id, 0))); - initCommonFields(responseData); - this.responseData = new LinkedHashMap(responseData); - this.throttleTime = DEFAULT_THROTTLE_TIME; + this(0, new LinkedHashMap<>(responseData), DEFAULT_THROTTLE_TIME); } - /** - * Constructor for Version 1 - * @param responseData fetched data grouped by topic-partition - * @param throttleTime Time in milliseconds the response was throttled - */ + /** + * Constructor for Version 1 and 2 + * @param responseData fetched data grouped by topic-partition + * @param throttleTime Time in milliseconds the response was throttled + */ public FetchResponse(Map responseData, int throttleTime) { - super(new Struct(CURRENT_SCHEMA)); - initCommonFields(responseData); - struct.set(THROTTLE_TIME_KEY_NAME, throttleTime); - this.responseData = new LinkedHashMap(responseData); - this.throttleTime = throttleTime; + // the schema for versions 1 and 2 is the same, so we pick 2 here + this(2, new LinkedHashMap<>(responseData), throttleTime); } - public FetchResponse(Struct struct) { - super(struct); - responseData = new LinkedHashMap(); - for (Object topicResponseObj : struct.getArray(RESPONSES_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); - short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME); - long highWatermark = partitionResponse.getLong(HIGH_WATERMARK_KEY_NAME); - ByteBuffer recordSet = partitionResponse.getBytes(RECORD_SET_KEY_NAME); - PartitionData partitionData = new PartitionData(errorCode, highWatermark, recordSet); - responseData.put(new TopicPartition(topic, partition), partitionData); - } - } - this.throttleTime = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME; + /** + * Constructor for Version 3 + * @param responseData fetched data grouped by topic-partition + * @param throttleTime Time in milliseconds the response was throttled + */ + public FetchResponse(LinkedHashMap responseData, int throttleTime) { + this(3, responseData, throttleTime); } - private void initCommonFields(Map responseData) { - Map> topicsData = CollectionUtils.groupDataByTopic(responseData); + private FetchResponse(int version, LinkedHashMap responseData, int throttleTime) { + super(new Struct(ProtoUtils.responseSchema(ApiKeys.FETCH.id, version))); - List topicArray = new ArrayList(); + //FIXME Need to fix this to maintain ordering + Map> topicsData = CollectionUtils.groupDataByTopic(responseData); + List topicArray = new ArrayList<>(); for (Map.Entry> topicEntry: topicsData.entrySet()) { Struct topicData = struct.instance(RESPONSES_KEY_NAME); topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); - List partitionArray = new ArrayList(); + List partitionArray = new ArrayList<>(); for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { PartitionData fetchPartitionData = partitionEntry.getValue(); Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); @@ -144,10 +130,35 @@ private void initCommonFields(Map responseData) { topicArray.add(topicData); } struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); + + if (version >= 1) + struct.set(THROTTLE_TIME_KEY_NAME, throttleTime); + + this.responseData = responseData; + this.throttleTime = throttleTime; } + public FetchResponse(Struct struct) { + super(struct); + LinkedHashMap responseData = new LinkedHashMap<>(); + for (Object topicResponseObj : struct.getArray(RESPONSES_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); + short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME); + long highWatermark = partitionResponse.getLong(HIGH_WATERMARK_KEY_NAME); + ByteBuffer recordSet = partitionResponse.getBytes(RECORD_SET_KEY_NAME); + PartitionData partitionData = new PartitionData(errorCode, highWatermark, recordSet); + responseData.put(new TopicPartition(topic, partition), partitionData); + } + } + this.responseData = responseData; + this.throttleTime = struct.hasField(THROTTLE_TIME_KEY_NAME) ? struct.getInt(THROTTLE_TIME_KEY_NAME) : DEFAULT_THROTTLE_TIME; + } - public Map responseData() { + public LinkedHashMap responseData() { return responseData; } diff --git a/core/src/main/scala/kafka/api/FetchRequest.scala b/core/src/main/scala/kafka/api/FetchRequest.scala index 94598f124279..05ee6af501bb 100644 --- a/core/src/main/scala/kafka/api/FetchRequest.scala +++ b/core/src/main/scala/kafka/api/FetchRequest.scala @@ -23,16 +23,21 @@ import kafka.common.TopicAndPartition import kafka.consumer.ConsumerConfig import kafka.network.RequestChannel import kafka.message.MessageSet - import java.util.concurrent.atomic.AtomicInteger import java.nio.ByteBuffer + +import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.{ApiKeys, Errors} -import scala.collection.immutable.{ListMap, Map} +import scala.collection.mutable.ArrayBuffer +import scala.util.Random case class PartitionFetchInfo(offset: Long, fetchSize: Int) object FetchRequest { + + private val random = new Random + val CurrentVersion = 3.shortValue val DefaultMaxWait = 0 val DefaultMinBytes = 0 @@ -60,6 +65,20 @@ object FetchRequest { }) FetchRequest(versionId, correlationId, clientId, replicaId, maxWait, minBytes, maxBytes, Vector(pairs:_*)) } + + def shuffle(requestInfo: Seq[(TopicAndPartition, PartitionFetchInfo)]): Seq[(TopicAndPartition, PartitionFetchInfo)] = + random.shuffle(requestInfo) + + def batchByTopic[T](s: Seq[(TopicAndPartition, T)]): Seq[(String, Seq[(Int, T)])] = { + val result = new ArrayBuffer[(String, ArrayBuffer[(Int, T)])] + s.foreach { case (TopicAndPartition(t, p), value) => + if (result.isEmpty || result.last._1 != t) + result += (t -> new ArrayBuffer) + result.last._2 += (p -> value) + } + result + } + } case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion, @@ -69,36 +88,24 @@ case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion, maxWait: Int = FetchRequest.DefaultMaxWait, minBytes: Int = FetchRequest.DefaultMinBytes, maxBytes: Int = FetchRequest.DefaultMaxBytes, - requestInfo: Vector[(TopicAndPartition, PartitionFetchInfo)]) + requestInfo: Seq[(TopicAndPartition, PartitionFetchInfo)]) extends RequestOrResponse(Some(ApiKeys.FETCH.id)) { - lazy val requestInfoMapByTopic = requestInfo.toMap /** * Partitions the request info into a list of lists (one for each topic) while preserving request info ordering */ - private type PartitionInfoList = Vector[(Int, PartitionFetchInfo)] - private type TopicInfoEntry = (String, PartitionInfoList) - private val requestInfoGroupedByTopic : Vector[TopicInfoEntry] = { - requestInfo.foldLeft(Vector.empty[TopicInfoEntry])((folded, currTopicAndPartition) => { - val (TopicAndPartition(topic, partition), partitionFetchInfo) = currTopicAndPartition - if (folded.isEmpty || folded.last._1 != topic) { - folded :+ (topic, Vector((partition, partitionFetchInfo))) - } else { - val updatedTail = (folded.last._1, folded.last._2 :+ (partition, partitionFetchInfo)) - folded.dropRight(1) :+ updatedTail - } - }) - } + private type PartitionInfos = Seq[(Int, PartitionFetchInfo)] + private lazy val requestInfoGroupedByTopic: Seq[(String, PartitionInfos)] = FetchRequest.batchByTopic(requestInfo) - /** - * Public constructor for the clients - */ + /** Public constructor for the clients */ + @deprecated("The order of partitions in `requestInfo` is relevant, so this constructor is deprecated in favour of the " + + "one that takes a Seq", since = "0.10.1.0") def this(correlationId: Int, clientId: String, maxWait: Int, minBytes: Int, maxBytes: Int, - requestInfo: Seq[(TopicAndPartition, PartitionFetchInfo)]) { + requestInfo: Map[TopicAndPartition, PartitionFetchInfo]) { this(versionId = FetchRequest.CurrentVersion, correlationId = correlationId, clientId = clientId, @@ -106,7 +113,24 @@ case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion, maxWait = maxWait, minBytes = minBytes, maxBytes = maxBytes, - requestInfo = requestInfo.toVector) + requestInfo = FetchRequest.shuffle(requestInfo.toSeq)) + } + + /** Public constructor for the clients */ + def this(correlationId: Int, + clientId: String, + maxWait: Int, + minBytes: Int, + maxBytes: Int, + requestInfo: Seq[(TopicAndPartition, PartitionFetchInfo)]) { + this(versionId = FetchRequest.CurrentVersion, + correlationId = correlationId, + clientId = clientId, + replicaId = Request.OrdinaryConsumerId, + maxWait = maxWait, + minBytes = minBytes, + maxBytes = maxBytes, + requestInfo = requestInfo) } def writeTo(buffer: ByteBuffer) { @@ -243,7 +267,8 @@ class FetchRequestBuilder() { } def build() = { - val fetchRequest = FetchRequest(versionId, correlationId.getAndIncrement, clientId, replicaId, maxWait, minBytes, maxBytes, requestMap.toVector) + val fetchRequest = FetchRequest(versionId, correlationId.getAndIncrement, clientId, replicaId, maxWait, minBytes, + maxBytes, new ArrayBuffer() ++ requestMap) requestMap.clear() fetchRequest } diff --git a/core/src/main/scala/kafka/api/FetchResponse.scala b/core/src/main/scala/kafka/api/FetchResponse.scala index a039dae882b3..d99bbcd79d59 100644 --- a/core/src/main/scala/kafka/api/FetchResponse.scala +++ b/core/src/main/scala/kafka/api/FetchResponse.scala @@ -28,6 +28,7 @@ import org.apache.kafka.common.network.{Send, MultiSend} import org.apache.kafka.common.protocol.Errors import scala.collection._ +import JavaConverters._ object FetchResponsePartitionData { def readFrom(buffer: ByteBuffer): FetchResponsePartitionData = { @@ -100,7 +101,7 @@ object TopicData { val partitionData = FetchResponsePartitionData.readFrom(buffer) (partitionId, partitionData) }) - TopicData(topic, Vector(topicPartitionDataPairs:_*)) + TopicData(topic, Seq(topicPartitionDataPairs:_*)) } def headerSize(topic: String) = @@ -108,7 +109,7 @@ object TopicData { 4 /* partition count */ } -case class TopicData(topic: String, partitionData: Vector[(Int, FetchResponsePartitionData)]) { +case class TopicData(topic: String, partitionData: Seq[(Int, FetchResponsePartitionData)]) { val sizeInBytes = TopicData.headerSize(topic) + partitionData.foldLeft(0)((folded, data) => { folded + data._2.sizeInBytes + 4 @@ -170,28 +171,17 @@ object FetchResponse { val topicCount = buffer.getInt val pairs = (1 to topicCount).flatMap(_ => { val topicData = TopicData.readFrom(buffer) - topicData.partitionData.map { - case (partitionId, partitionData) => - (TopicAndPartition(topicData.topic, partitionId), partitionData) + topicData.partitionData.map { case (partitionId, partitionData) => + (TopicAndPartition(topicData.topic, partitionId), partitionData) } }) FetchResponse(correlationId, Vector(pairs:_*), requestVersion, throttleTime) } - type FetchResponseEntry = (TopicAndPartition, FetchResponsePartitionData) - - def groupByTopicOrdered(data: Seq[(TopicAndPartition, FetchResponsePartitionData)]): Vector[(String, Vector[FetchResponseEntry])] = { - data.foldLeft(Vector.empty[(String, Vector[FetchResponseEntry])])((folded, currFetchResponse) => { - val (topicPartition, fetchResponsePartitionData) = currFetchResponse - val TopicAndPartition(topic, partition) = topicPartition - if (folded.isEmpty || folded.last._1 != topic) { - folded :+ (topic, Vector((topicPartition, fetchResponsePartitionData))) - } else { - val updatedTail = (folded.last._1, folded.last._2 :+ (topicPartition, fetchResponsePartitionData)) - folded.dropRight(1) :+ updatedTail - } - }) - } + type FetchResponseEntry = (Int, FetchResponsePartitionData) + + def batchByTopic(data: Seq[(TopicAndPartition, FetchResponsePartitionData)]): Seq[(String, Seq[FetchResponseEntry])] = + FetchRequest.batchByTopic(data) // Returns the size of the response header def headerSize(requestVersion: Int): Int = { @@ -202,12 +192,12 @@ object FetchResponse { } // Returns the size of entire fetch response in bytes (including the header size) - def responseSize(dataGroupedByTopic: Vector[(String, Vector[FetchResponseEntry])], + def responseSize(dataGroupedByTopic: Seq[(String, Seq[FetchResponseEntry])], requestVersion: Int): Int = { headerSize(requestVersion) + dataGroupedByTopic.foldLeft(0) { case (folded, (topic, partitionDataMap)) => val topicData = TopicData(topic, partitionDataMap.map { - case (topicAndPartition, partitionData) => (topicAndPartition.partition, partitionData) + case (partitionId, partitionData) => (partitionId, partitionData) }) folded + topicData.sizeInBytes } @@ -215,7 +205,7 @@ object FetchResponse { } case class FetchResponse(correlationId: Int, - data: Vector[(TopicAndPartition, FetchResponsePartitionData)], + data: Seq[(TopicAndPartition, FetchResponsePartitionData)], requestVersion: Int = 0, throttleTimeMs: Int = 0) extends RequestOrResponse() { @@ -223,8 +213,8 @@ case class FetchResponse(correlationId: Int, /** * Partitions the data into a map of maps (one for each topic). */ - lazy val dataByTopicAndPartition = data.toMap - lazy val dataGroupedByTopic = FetchResponse.groupByTopicOrdered(data) + private lazy val dataByTopicAndPartition = data.toMap + lazy val dataGroupedByTopic = FetchResponse.batchByTopic(data) val headerSizeInBytes = FetchResponse.headerSize(requestVersion) lazy val sizeInBytes = FetchResponse.responseSize(dataGroupedByTopic, requestVersion) @@ -292,10 +282,9 @@ class FetchResponseSend(val dest: String, val fetchResponse: FetchResponse) exte fetchResponse.writeHeaderTo(buffer) buffer.rewind() - private val sends = new MultiSend(dest, JavaConversions.seqAsJavaList(fetchResponse.dataGroupedByTopic.toList.map { - case(topic, data) => new TopicDataSend(dest, TopicData(topic, - data.map{case(topicAndPartition, message) => (topicAndPartition.partition, message)})) - })) + private val sends = new MultiSend(dest, fetchResponse.dataGroupedByTopic.map { + case (topic, data) => new TopicDataSend(dest, TopicData(topic, data)): Send + }.asJava) override def writeTo(channel: GatheringByteChannel): Long = { if (completed) diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala index 24766b58ba15..41ab64916d2d 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala @@ -106,8 +106,8 @@ class ConsumerFetcherThread(name: String, new FetchRequest(fetchRequestBuilder.build()) } - protected def fetch(fetchRequest: FetchRequest): collection.Map[TopicPartition, PartitionData] = - simpleConsumer.fetch(fetchRequest.underlying).dataByTopicAndPartition.map { case (TopicAndPartition(t, p), value) => + protected def fetch(fetchRequest: FetchRequest): Seq[(TopicPartition, PartitionData)] = + simpleConsumer.fetch(fetchRequest.underlying).data.map { case (TopicAndPartition(t, p), value) => new TopicPartition(t, p) -> new PartitionData(value) } @@ -116,10 +116,11 @@ class ConsumerFetcherThread(name: String, object ConsumerFetcherThread { class FetchRequest(val underlying: kafka.api.FetchRequest) extends AbstractFetcherThread.FetchRequest { + private lazy val tpToOffset: Map[TopicPartition, Long] = underlying.requestInfo.map { case (tp, fetchInfo) => + new TopicPartition(tp.topic, tp.partition) -> fetchInfo.offset + }.toMap def isEmpty: Boolean = underlying.requestInfo.isEmpty - def offset(topicPartition: TopicPartition): Long = { - underlying.requestInfoMapByTopic(new TopicAndPartition(topicPartition.topic, topicPartition.partition)).offset - } + def offset(topicPartition: TopicPartition): Long = tpToOffset(topicPartition) } class PartitionData(val underlying: FetchResponsePartitionData) extends AbstractFetcherThread.PartitionData { diff --git a/core/src/main/scala/kafka/javaapi/FetchRequest.scala b/core/src/main/scala/kafka/javaapi/FetchRequest.scala index 020ad758c9b5..fb9fa8e9a1af 100644 --- a/core/src/main/scala/kafka/javaapi/FetchRequest.scala +++ b/core/src/main/scala/kafka/javaapi/FetchRequest.scala @@ -17,32 +17,44 @@ package kafka.javaapi -import java.nio.ByteBuffer +import java.util + import kafka.common.TopicAndPartition -import kafka.api.{Request, PartitionFetchInfo} -import scala.collection.mutable +import kafka.api.{PartitionFetchInfo, Request} + +import scala.collection.JavaConverters._ + +object FetchRequest { + private def seqToLinkedHashMap[K, V](s: Seq[(K, V)]): util.LinkedHashMap[K, V] = { + val map = new util.LinkedHashMap[K, V] + s.foreach { case (k, v) => map.put(k, v) } + map + } +} class FetchRequest(correlationId: Int, clientId: String, maxWait: Int, minBytes: Int, - requestInfo: java.util.Map[TopicAndPartition, PartitionFetchInfo]) { - - val underlying = { - val scalaMap: Seq[(TopicAndPartition, PartitionFetchInfo)] = { - import scala.collection.JavaConversions._ - (requestInfo: mutable.Map[TopicAndPartition, PartitionFetchInfo]).toSeq - } - kafka.api.FetchRequest( - correlationId = correlationId, - clientId = clientId, - replicaId = Request.OrdinaryConsumerId, - maxWait = maxWait, - minBytes = minBytes, - requestInfo = scalaMap.toVector - ) + requestInfo: util.LinkedHashMap[TopicAndPartition, PartitionFetchInfo]) { + + @deprecated("The order of partitions in `requestInfo` is relevant, so this constructor is deprecated in favour of the " + + "one that takes a LinkedHashMap", since = "0.10.1.0") + def this(correlationId: Int, clientId: String, maxWait: Int, minBytes: Int, + requestInfo: java.util.Map[TopicAndPartition, PartitionFetchInfo]) { + this(correlationId, clientId, maxWait, minBytes, + FetchRequest.seqToLinkedHashMap(kafka.api.FetchRequest.shuffle(requestInfo.asScala.toSeq))) } + val underlying = kafka.api.FetchRequest( + correlationId = correlationId, + clientId = clientId, + replicaId = Request.OrdinaryConsumerId, + maxWait = maxWait, + minBytes = minBytes, + requestInfo = requestInfo.asScala.toBuffer + ) + override def toString = underlying.toString override def equals(other: Any) = canEqual(other) && { diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index 933603166ff1..5bafbaebf4ad 100755 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -73,7 +73,7 @@ abstract class AbstractFetcherThread(name: String, protected def buildFetchRequest(partitionMap: Seq[(TopicPartition, PartitionFetchState)]): REQ - protected def fetch(fetchRequest: REQ): Map[TopicPartition, PD] + protected def fetch(fetchRequest: REQ): Seq[(TopicPartition, PD)] override def shutdown(){ initiateShutdown() @@ -112,7 +112,7 @@ abstract class AbstractFetcherThread(name: String, partitionMap.moveToEnd(partition) } - var responseData: Map[TopicPartition, PD] = Map.empty + var responseData: Seq[(TopicPartition, PD)] = Seq.empty try { trace("Issuing to broker %d of fetch request %s".format(sourceBroker.id, fetchRequest)) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index d5ad2a55d203..04768b08303a 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -490,10 +490,9 @@ class KafkaApis(val requestChannel: RequestChannel, if (fetchRequest.isFromFollower) { fetchResponseCallback(0) } else { - quotaManagers(ApiKeys.FETCH.id).recordAndMaybeThrottle(fetchRequest.clientId, - FetchResponse.responseSize(FetchResponse.groupByTopicOrdered(mergedPartitionData.toSeq), - fetchRequest.versionId), - fetchResponseCallback) + val responseSize = FetchResponse.responseSize(FetchResponse.batchByTopic(mergedPartitionData.toSeq), + fetchRequest.versionId) + quotaManagers(ApiKeys.FETCH.id).recordAndMaybeThrottle(fetchRequest.clientId, responseSize, fetchResponseCallback) } } diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index e85a49163324..339740d2cb0c 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -229,9 +229,9 @@ class ReplicaFetcherThread(name: String, delayPartitions(partitions, brokerConfig.replicaFetchBackoffMs.toLong) } - protected def fetch(fetchRequest: FetchRequest): Map[TopicPartition, PartitionData] = { + protected def fetch(fetchRequest: FetchRequest): Seq[(TopicPartition, PartitionData)] = { val clientResponse = sendRequest(ApiKeys.FETCH, Some(fetchRequestVersion), fetchRequest.underlying) - new FetchResponse(clientResponse.responseBody).responseData.asScala.map { case (key, value) => + new FetchResponse(clientResponse.responseBody).responseData.asScala.toSeq.map { case (key, value) => key -> new PartitionData(value) } } diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala index 4c3f15637a04..6a97d5d9ca27 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala @@ -115,13 +115,11 @@ class AbstractFetcherThreadTest { override def handlePartitionsWithErrors(partitions: Iterable[TopicPartition]): Unit = {} - override protected def fetch(fetchRequest: DummyFetchRequest): collection.Map[TopicPartition, DummyPartitionData] = { - fetchRequest.offsets.mapValues(_ => new DummyPartitionData) - } + override protected def fetch(fetchRequest: DummyFetchRequest): Seq[(TopicPartition, DummyPartitionData)] = + fetchRequest.offsets.mapValues(_ => new DummyPartitionData).toSeq - override protected def buildFetchRequest(partitionMap: collection.Seq[(TopicPartition, PartitionFetchState)]): DummyFetchRequest = { + override protected def buildFetchRequest(partitionMap: collection.Seq[(TopicPartition, PartitionFetchState)]): DummyFetchRequest = new DummyFetchRequest(partitionMap.map { case (k, v) => (k, v.offset) }.toMap) - } } } From d89bf00738d9c522e39e0ec9cba34de2514acc5c Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 12:05:47 +0100 Subject: [PATCH 16/54] Introduce `minOneMessage` parameter to `Log.read` with a `minOneMessage` --- .../main/scala/kafka/log/FileMessageSet.scala | 10 ++--- core/src/main/scala/kafka/log/Log.scala | 5 ++- .../src/main/scala/kafka/log/LogSegment.scala | 35 ++++++++++------- .../unit/kafka/log/FileMessageSetTest.scala | 38 ++++++++++--------- .../scala/unit/kafka/log/LogSegmentTest.scala | 2 +- 5 files changed, 52 insertions(+), 38 deletions(-) diff --git a/core/src/main/scala/kafka/log/FileMessageSet.scala b/core/src/main/scala/kafka/log/FileMessageSet.scala index 57630429e341..b477b273973c 100755 --- a/core/src/main/scala/kafka/log/FileMessageSet.scala +++ b/core/src/main/scala/kafka/log/FileMessageSet.scala @@ -128,11 +128,11 @@ class FileMessageSet private[kafka](@volatile var file: File, /** * Search forward for the file position of the last offset that is greater than or equal to the target offset - * and return its physical position. If no such offsets are found, return null. + * and return its physical position and the size of the message. If no such offsets are found, return null. * @param targetOffset The offset to search for. * @param startingPosition The starting position in the file to begin searching from. */ - def searchForOffset(targetOffset: Long, startingPosition: Int): OffsetPosition = { + def searchForOffsetWithSize(targetOffset: Long, startingPosition: Int): (OffsetPosition, Int) = { var position = startingPosition val buffer = ByteBuffer.allocate(MessageSet.LogOverhead) val size = sizeInBytes() @@ -144,11 +144,11 @@ class FileMessageSet private[kafka](@volatile var file: File, .format(targetOffset, startingPosition, file.getAbsolutePath)) buffer.rewind() val offset = buffer.getLong() - if(offset >= targetOffset) - return OffsetPosition(offset, position) val messageSize = buffer.getInt() - if(messageSize < Message.MinMessageOverhead) + if (messageSize < Message.MinMessageOverhead) throw new IllegalStateException("Invalid message size: " + messageSize) + if (offset >= targetOffset) + return (OffsetPosition(offset, position), messageSize + MessageSet.LogOverhead) position += MessageSet.LogOverhead + messageSize } null diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 894beabf526e..cfd047297d87 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -517,11 +517,12 @@ class Log(val dir: File, * @param startOffset The offset to begin reading at * @param maxLength The maximum number of bytes to read * @param maxOffset The offset to read up to, exclusive. (i.e. this offset NOT included in the resulting message set) + * @param minOneMessage If this is true, the first message will be returned even if it exceeds `maxLength` (if one exists) * * @throws OffsetOutOfRangeException If startOffset is beyond the log end offset or before the base offset of the first segment. * @return The fetch data information including fetch starting offset metadata and messages read. */ - def read(startOffset: Long, maxLength: Int, maxOffset: Option[Long] = None): FetchDataInfo = { + def read(startOffset: Long, maxLength: Int, maxOffset: Option[Long] = None, minOneMessage: Boolean = false): FetchDataInfo = { trace("Reading %d bytes from offset %d in log %s of length %d bytes".format(maxLength, startOffset, name, size)) // Because we don't use lock for reading, the synchronization is a little bit tricky. @@ -558,7 +559,7 @@ class Log(val dir: File, entry.getValue.size } } - val fetchInfo = entry.getValue.read(startOffset, maxOffset, maxLength, maxPosition) + val fetchInfo = entry.getValue.read(startOffset, maxOffset, maxLength, maxPosition, minOneMessage) if(fetchInfo == null) { entry = segments.higherEntry(entry.getKey) } else { diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index ccc247258b09..62acb3c612a3 100755 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -119,12 +119,13 @@ class LogSegment(val log: FileMessageSet, * @param offset The offset we want to translate * @param startingFilePosition A lower bound on the file position from which to begin the search. This is purely an optimization and * when omitted, the search will begin at the position in the offset index. - * @return The position in the log storing the message with the least offset >= the requested offset or null if no message meets this criteria. + * @return The position in the log storing the message with the least offset >= the requested offset and the size of the + * message or null if no message meets this criteria. */ @threadsafe - private[log] def translateOffset(offset: Long, startingFilePosition: Int = 0): OffsetPosition = { + private[log] def translateOffset(offset: Long, startingFilePosition: Int = 0): (OffsetPosition, Int) = { val mapping = index.lookup(offset) - log.searchForOffset(offset, max(mapping.position, startingFilePosition)) + log.searchForOffsetWithSize(offset, max(mapping.position, startingFilePosition)) } /** @@ -135,33 +136,40 @@ class LogSegment(val log: FileMessageSet, * @param maxSize The maximum number of bytes to include in the message set we read * @param maxOffset An optional maximum offset for the message set we read * @param maxPosition The maximum position in the log segment that should be exposed for read + * @param minOneMessage If this is true, the first message will be returned even if it exceeds `maxSize` (if one exists) * * @return The fetched data and the offset metadata of the first message whose offset is >= startOffset, * or null if the startOffset is larger than the largest offset in this log */ @threadsafe - def read(startOffset: Long, maxOffset: Option[Long], maxSize: Int, maxPosition: Long = size): FetchDataInfo = { + def read(startOffset: Long, maxOffset: Option[Long], maxSize: Int, maxPosition: Long = size, + minOneMessage: Boolean = false): FetchDataInfo = { if(maxSize < 0) throw new IllegalArgumentException("Invalid max size for log read (%d)".format(maxSize)) val logSize = log.sizeInBytes // this may change, need to save a consistent copy - val startPosition = translateOffset(startOffset) + val startOffsetAndSize = translateOffset(startOffset) // if the start position is already off the end of the log, return null - if(startPosition == null) + if (startOffsetAndSize == null) return null + val (startPosition, messageSize) = startOffsetAndSize val offsetMetadata = new LogOffsetMetadata(startOffset, this.baseOffset, startPosition.position) // if the size is zero, still return a log segment but with zero size - if(maxSize == 0) + if (maxSize == 0 && !minOneMessage) return FetchDataInfo(offsetMetadata, MessageSet.Empty) + val maxLength = + if (minOneMessage) math.max(maxSize, messageSize) + else maxSize + // calculate the length of the message set to read based on whether or not they gave us a maxOffset val length = maxOffset match { case None => // no max offset, just read until the max position - min((maxPosition - startPosition.position).toInt, maxSize) + min((maxPosition - startPosition.position).toInt, maxLength) case Some(offset) => // there is a max offset, translate it to a file position and use that to calculate the max read size; // when the leader of a partition changes, it's possible for the new leader's high watermark to be less than the @@ -171,11 +179,11 @@ class LogSegment(val log: FileMessageSet, return FetchDataInfo(offsetMetadata, MessageSet.Empty) val mapping = translateOffset(offset, startPosition.position) val endPosition = - if(mapping == null) + if (mapping == null) logSize // the max offset is off the end of the log, use the end of the file else - mapping.position - min(min(maxPosition, endPosition) - startPosition.position, maxSize).toInt + mapping._1.position + min(min(maxPosition, endPosition) - startPosition.position, maxLength).toInt } FetchDataInfo(offsetMetadata, log.read(startPosition.position, length)) @@ -260,14 +268,15 @@ class LogSegment(val log: FileMessageSet, @nonthreadsafe def truncateTo(offset: Long): Int = { val mapping = translateOffset(offset) - if(mapping == null) + if (mapping == null) return 0 index.truncateTo(offset) timeIndex.truncateTo(offset) // after truncation, reset and allocate more space for the (new currently active) index index.resize(index.maxIndexSize) timeIndex.resize(timeIndex.maxIndexSize) - val bytesTruncated = log.truncateTo(mapping.position) + val (offsetPosition, _) = mapping + val bytesTruncated = log.truncateTo(offsetPosition.position) if(log.sizeInBytes == 0) { created = time.milliseconds rollingBasedTimestamp = None diff --git a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala index 82496f2a8498..8702474923a9 100644 --- a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala @@ -108,24 +108,28 @@ class FileMessageSetTest extends BaseMessageSetTestCases { // append a new message with a high offset val lastMessage = new Message("test".getBytes) messageSet.append(new ByteBufferMessageSet(NoCompressionCodec, new LongRef(50), lastMessage)) + val messages = messageSet.toSeq var position = 0 - assertEquals("Should be able to find the first message by its offset", - OffsetPosition(0L, position), - messageSet.searchForOffset(0, 0)) - position += MessageSet.entrySize(messageSet.head.message) + val message1Size = MessageSet.entrySize(messages.head.message) + assertEquals("Should be able to find the first message by its offset", + (OffsetPosition(0L, position), message1Size), + messageSet.searchForOffsetWithSize(0, 0)) + position += message1Size + val message2Size = MessageSet.entrySize(messages(1).message) assertEquals("Should be able to find second message when starting from 0", - OffsetPosition(1L, position), - messageSet.searchForOffset(1, 0)) + (OffsetPosition(1L, position), message2Size), + messageSet.searchForOffsetWithSize(1, 0)) assertEquals("Should be able to find second message starting from its offset", - OffsetPosition(1L, position), - messageSet.searchForOffset(1, position)) - position += MessageSet.entrySize(messageSet.tail.head.message) + MessageSet.entrySize(messageSet.tail.tail.head.message) + (OffsetPosition(1L, position), message2Size), + messageSet.searchForOffsetWithSize(1, position)) + position += message2Size + MessageSet.entrySize(messages(2).message) + val message4Size = MessageSet.entrySize(messages(3).message) assertEquals("Should be able to find fourth message from a non-existant offset", - OffsetPosition(50L, position), - messageSet.searchForOffset(3, position)) - assertEquals("Should be able to find fourth message by correct offset", - OffsetPosition(50L, position), - messageSet.searchForOffset(50, position)) + (OffsetPosition(50L, position), message4Size), + messageSet.searchForOffsetWithSize(3, position)) + assertEquals("Should be able to find fourth message by correct offset", + (OffsetPosition(50L, position), message4Size), + messageSet.searchForOffsetWithSize(50, position)) } /** @@ -134,7 +138,7 @@ class FileMessageSetTest extends BaseMessageSetTestCases { @Test def testIteratorWithLimits() { val message = messageSet.toList(1) - val start = messageSet.searchForOffset(1, 0).position + val start = messageSet.searchForOffsetWithSize(1, 0)._1.position val size = message.message.size + 12 val slice = messageSet.read(start, size) assertEquals(List(message), slice.toList) @@ -148,7 +152,7 @@ class FileMessageSetTest extends BaseMessageSetTestCases { @Test def testTruncate() { val message = messageSet.toList.head - val end = messageSet.searchForOffset(1, 0).position + val end = messageSet.searchForOffsetWithSize(1, 0)._1.position messageSet.truncateTo(end) assertEquals(List(message), messageSet.toList) assertEquals(MessageSet.entrySize(message.message), messageSet.sizeInBytes) @@ -272,7 +276,7 @@ class FileMessageSetTest extends BaseMessageSetTestCases { @Test def testFormatConversionWithPartialMessage() { val message = messageSet.toList(1) - val start = messageSet.searchForOffset(1, 0).position + val start = messageSet.searchForOffsetWithSize(1, 0)._1.position val size = message.message.size + 12 val slice = messageSet.read(start, size - 1) val messageV0 = slice.toMessageFormat(Message.MagicValue_V0) diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala index 64140e824ed6..49feebdcc087 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala @@ -284,7 +284,7 @@ class LogSegmentTest { seg.append(i, Message.NoTimestamp, -1L, messages(i, i.toString)) val offsetToBeginCorruption = TestUtils.random.nextInt(messagesAppended) // start corrupting somewhere in the middle of the chosen record all the way to the end - val position = seg.log.searchForOffset(offsetToBeginCorruption, 0).position + TestUtils.random.nextInt(15) + val position = seg.log.searchForOffsetWithSize(offsetToBeginCorruption, 0)._1.position + TestUtils.random.nextInt(15) TestUtils.writeNonsenseToFile(seg.log.file, position, seg.log.file.length.toInt - position) seg.recover(64*1024) assertEquals("Should have truncated off bad messages.", (0 until offsetToBeginCorruption).toList, seg.log.map(_.offset).toList) From 8572cdf947c1b0efa8c6a6c20106af36e187d918 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 12:07:40 +0100 Subject: [PATCH 17/54] Use `Seq` instead of `Map` in `ReplicaManager` and improve handling of messages larger than fetch size We now always return the first message of the first partition (if there is one) irrespective of its size. --- .../scala/kafka/server/DelayedFetch.scala | 15 +- .../main/scala/kafka/server/KafkaApis.scala | 10 +- .../scala/kafka/server/ReplicaManager.scala | 156 +++++++++--------- .../kafka/api/PlaintextConsumerTest.scala | 36 ++-- .../kafka/server/ReplicaManagerTest.scala | 20 +-- .../unit/kafka/server/SimpleFetchTest.scala | 13 +- 6 files changed, 136 insertions(+), 114 deletions(-) diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index e8e910300efd..01708234e9f9 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -41,7 +41,7 @@ case class FetchMetadata(fetchMinBytes: Int, fetchOnlyLeader: Boolean, fetchOnlyCommitted: Boolean, isFromFollower: Boolean, - fetchPartitionStatus: Map[TopicAndPartition, FetchPartitionStatus]) { + fetchPartitionStatus: Seq[(TopicAndPartition, FetchPartitionStatus)]) { override def toString = "[minBytes: " + fetchMinBytes + ", " + "onlyLeader:" + fetchOnlyLeader + ", " @@ -55,7 +55,7 @@ case class FetchMetadata(fetchMinBytes: Int, class DelayedFetch(delayMs: Long, fetchMetadata: FetchMetadata, replicaManager: ReplicaManager, - responseCallback: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) + responseCallback: Seq[(TopicAndPartition, FetchResponsePartitionData)] => Unit) extends DelayedOperation(delayMs) { /** @@ -129,13 +129,16 @@ class DelayedFetch(delayMs: Long, * Upon completion, read whatever data is available and pass to the complete callback */ override def onComplete() { - val logReadResults = replicaManager.readFromLocalLog(fetchMetadata.fetchOnlyLeader, + val logReadResults = replicaManager.readFromLocalLog( + fetchMetadata.fetchOnlyLeader, fetchMetadata.fetchOnlyCommitted, fetchMetadata.fetchMaxBytes, - fetchMetadata.fetchPartitionStatus.mapValues(status => status.fetchInfo)) + fetchMetadata.fetchPartitionStatus.map { case (tp, status) => tp -> status.fetchInfo } + ) - val fetchPartitionData = logReadResults.mapValues(result => - FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)) + val fetchPartitionData = logReadResults.map { case (tp, result) => + tp -> FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet) + } responseCallback(fetchPartitionData) } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 04768b08303a..43e40e5951f9 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -439,7 +439,7 @@ class KafkaApis(val requestChannel: RequestChannel, } // the callback for sending a fetch response - def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData]) { + def sendResponseCallback(responsePartitionData: Seq[(TopicAndPartition, FetchResponsePartitionData)]) { val convertedPartitionData = // Need to down-convert message when consumer only takes magic value 0. @@ -477,8 +477,8 @@ class KafkaApis(val requestChannel: RequestChannel, def fetchResponseCallback(delayTimeMs: Int) { trace(s"Sending fetch response to client ${fetchRequest.clientId} of " + - s"${convertedPartitionData.values.map(_.messages.sizeInBytes).sum} bytes") - val response = FetchResponse(fetchRequest.correlationId, mergedPartitionData.toVector, fetchRequest.versionId, delayTimeMs) + s"${convertedPartitionData.map { case (_, v) => v.messages.sizeInBytes }.sum} bytes") + val response = FetchResponse(fetchRequest.correlationId, mergedPartitionData.toSeq, fetchRequest.versionId, delayTimeMs) requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, response))) } @@ -497,7 +497,7 @@ class KafkaApis(val requestChannel: RequestChannel, } if (authorizedRequestInfo.isEmpty) - sendResponseCallback(Map.empty) + sendResponseCallback(Seq.empty) else { // call the replica manager to fetch messages from the local replica replicaManager.fetchMessages( @@ -505,7 +505,7 @@ class KafkaApis(val requestChannel: RequestChannel, fetchRequest.replicaId, fetchRequest.minBytes, fetchRequest.maxBytes, - authorizedRequestInfo.toMap, + authorizedRequestInfo, sendResponseCallback) } } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 43cca45ef157..d15323f2e252 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -458,14 +458,14 @@ class ReplicaManager(val config: KafkaConfig, replicaId: Int, fetchMinBytes: Int, fetchMaxBytes: Int, - fetchInfo: immutable.Map[TopicAndPartition, PartitionFetchInfo], - responseCallback: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) { + fetchInfos: Seq[(TopicAndPartition, PartitionFetchInfo)], + responseCallback: Seq[(TopicAndPartition, FetchResponsePartitionData)] => Unit) { val isFromFollower = replicaId >= 0 val fetchOnlyFromLeader: Boolean = replicaId != Request.DebuggingConsumerId val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(replicaId) // read from local logs - val logReadResults = readFromLocalLog(fetchOnlyFromLeader, fetchOnlyCommitted, fetchMaxBytes, fetchInfo) + val logReadResults = readFromLocalLog(fetchOnlyFromLeader, fetchOnlyCommitted, fetchMaxBytes, fetchInfos) // if the fetch comes from the follower, // update its corresponding log end offset @@ -473,28 +473,34 @@ class ReplicaManager(val config: KafkaConfig, updateFollowerLogReadResults(replicaId, logReadResults) // check if this fetch request can be satisfied right away - val bytesReadable = logReadResults.values.map(_.info.messageSet.sizeInBytes).sum - val errorReadingData = logReadResults.values.foldLeft(false) ((errorIncurred, readResult) => + val logReadResultValues = logReadResults.map { case (_, v) => v } + val bytesReadable = logReadResultValues.map(_.info.messageSet.sizeInBytes).sum + val errorReadingData = logReadResultValues.foldLeft(false) ((errorIncurred, readResult) => errorIncurred || (readResult.errorCode != Errors.NONE.code)) // respond immediately if 1) fetch request does not want to wait // 2) fetch request does not require any data // 3) has enough data to respond // 4) some error happens while reading data - if(timeout <= 0 || fetchInfo.size <= 0 || bytesReadable >= fetchMinBytes || errorReadingData) { - val fetchPartitionData = logReadResults.mapValues(result => - FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)) + if (timeout <= 0 || fetchInfos.isEmpty || bytesReadable >= fetchMinBytes || errorReadingData) { + val fetchPartitionData = logReadResults.map { case (tp, result) => + tp -> FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet) + } responseCallback(fetchPartitionData) } else { // construct the fetch results from the read results val fetchPartitionStatus = logReadResults.map { case (topicAndPartition, result) => - (topicAndPartition, FetchPartitionStatus(result.info.fetchOffsetMetadata, fetchInfo.get(topicAndPartition).get)) + val fetchInfo = fetchInfos.collectFirst { + case (tp, v) if tp == topicAndPartition => v + }.getOrElse(sys.error(s"Partition $topicAndPartition not found in fetchInfos")) + (topicAndPartition, FetchPartitionStatus(result.info.fetchOffsetMetadata, fetchInfo)) } - val fetchMetadata = FetchMetadata(fetchMinBytes, fetchMaxBytes, fetchOnlyFromLeader, fetchOnlyCommitted, isFromFollower, fetchPartitionStatus) + val fetchMetadata = FetchMetadata(fetchMinBytes, fetchMaxBytes, fetchOnlyFromLeader, fetchOnlyCommitted, isFromFollower, + fetchPartitionStatus) val delayedFetch = new DelayedFetch(timeout, fetchMetadata, this, responseCallback) // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation - val delayedFetchKeys = fetchPartitionStatus.keys.map(new TopicPartitionOperationKey(_)).toSeq + val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => new TopicPartitionOperationKey(tp) } // try to complete the request immediately, otherwise put it into the purgatory; // this is because while the delayed fetch operation is being created, new requests @@ -504,80 +510,82 @@ class ReplicaManager(val config: KafkaConfig, } /** - * Read from a single topic/partition at the given offset upto maxSize bytes + * Read from multiple topic partitions at the given offset up to maxSize bytes */ def readFromLocalLog(fetchOnlyFromLeader: Boolean, readOnlyCommitted: Boolean, fetchMaxBytes: Int, - readPartitionInfo: Map[TopicAndPartition, PartitionFetchInfo]): Map[TopicAndPartition, LogReadResult] = { + readPartitionInfo: Seq[(TopicAndPartition, PartitionFetchInfo)]): Seq[(TopicAndPartition, LogReadResult)] = { + + def read(tp: TopicAndPartition, fetchInfo: PartitionFetchInfo, limitBytes: Int, minOneMessage: Boolean): LogReadResult = { + val TopicAndPartition(topic, partition) = tp + val PartitionFetchInfo(offset, fetchSize) = fetchInfo - var limitBytes = fetchMaxBytes - readPartitionInfo.map { case (TopicAndPartition(topic, partition), PartitionFetchInfo(offset, fetchSize)) => BrokerTopicStats.getBrokerTopicStats(topic).totalFetchRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats().totalFetchRequestRate.mark() - val partitionDataAndOffsetInfo = - try { - trace("Fetching log segment for topic %s, partition %d, offset %d, partition fetch size %d, remaining response limit %d".format(topic, partition, offset, fetchSize, limitBytes)) - - // decide whether to only fetch from leader - val localReplica = if (fetchOnlyFromLeader) - getLeaderReplicaIfLocal(topic, partition) - else - getReplicaOrException(topic, partition) - - // decide whether to only fetch committed data (i.e. messages below high watermark) - val maxOffsetOpt = if (readOnlyCommitted) - Some(localReplica.highWatermark.messageOffset) - else - None - - /* Read the LogOffsetMetadata prior to performing the read from the log. - * We use the LogOffsetMetadata to determine if a particular replica is in-sync or not. - * Using the log end offset after performing the read can lead to a race condition - * where data gets appended to the log immediately after the replica has consumed from it - * This can cause a replica to always be out of sync. - */ - val initialLogEndOffset = localReplica.logEndOffset - val logReadInfo = localReplica.log match { - case Some(log) => - val adjustedFetchSize = - if (Topic.isInternal(topic) && !readOnlyCommitted) - Math.max(fetchSize, log.config.maxMessageSize) - else if (limitBytes > 0) - fetchSize - else - 0 - log.read(offset, adjustedFetchSize, maxOffsetOpt) - case None => - error("Leader for partition [%s,%d] does not have a local log".format(topic, partition)) - FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty) - } - - limitBytes = math.max(0, limitBytes - logReadInfo.messageSet.sizeInBytes) + try { + trace("Fetching log segment for topic %s, partition %d, offset %d, partition fetch size %d, remaining response limit %d".format(topic, partition, offset, fetchSize, limitBytes)) - val readToEndOfLog = initialLogEndOffset.messageOffset - logReadInfo.fetchOffsetMetadata.messageOffset <= 0 + // decide whether to only fetch from leader + val localReplica = if (fetchOnlyFromLeader) + getLeaderReplicaIfLocal(topic, partition) + else + getReplicaOrException(topic, partition) - LogReadResult(logReadInfo, localReplica.highWatermark.messageOffset, fetchSize, readToEndOfLog, None) - } catch { - // NOTE: Failed fetch requests metric is not incremented for known exceptions since it - // is supposed to indicate un-expected failure of a broker in handling a fetch request - case utpe: UnknownTopicOrPartitionException => - LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, false, Some(utpe)) - case nle: NotLeaderForPartitionException => - LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, false, Some(nle)) - case rnae: ReplicaNotAvailableException => - LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, false, Some(rnae)) - case oor : OffsetOutOfRangeException => - LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, false, Some(oor)) - case e: Throwable => - BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark() - BrokerTopicStats.getBrokerAllTopicsStats().failedFetchRequestRate.mark() - error("Error processing fetch operation on partition [%s,%d] offset %d".format(topic, partition, offset), e) - LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, false, Some(e)) + // decide whether to only fetch committed data (i.e. messages below high watermark) + val maxOffsetOpt = if (readOnlyCommitted) + Some(localReplica.highWatermark.messageOffset) + else + None + + /* Read the LogOffsetMetadata prior to performing the read from the log. + * We use the LogOffsetMetadata to determine if a particular replica is in-sync or not. + * Using the log end offset after performing the read can lead to a race condition + * where data gets appended to the log immediately after the replica has consumed from it + * This can cause a replica to always be out of sync. + */ + val initialLogEndOffset = localReplica.logEndOffset + val logReadInfo = localReplica.log match { + case Some(log) => + val adjustedFetchSize = math.min(fetchSize, limitBytes) + log.read(offset, adjustedFetchSize, maxOffsetOpt, minOneMessage) + case None => + error("Leader for partition [%s,%d] does not have a local log".format(topic, partition)) + FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty) } - (TopicAndPartition(topic, partition), partitionDataAndOffsetInfo) + + val readToEndOfLog = initialLogEndOffset.messageOffset - logReadInfo.fetchOffsetMetadata.messageOffset <= 0 + + LogReadResult(logReadInfo, localReplica.highWatermark.messageOffset, fetchSize, readToEndOfLog, None) + } catch { + // NOTE: Failed fetch requests metric is not incremented for known exceptions since it + // is supposed to indicate un-expected failure of a broker in handling a fetch request + case utpe: UnknownTopicOrPartitionException => + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, false, Some(utpe)) + case nle: NotLeaderForPartitionException => + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, false, Some(nle)) + case rnae: ReplicaNotAvailableException => + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, false, Some(rnae)) + case oor : OffsetOutOfRangeException => + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, false, Some(oor)) + case e: Throwable => + BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark() + BrokerTopicStats.getBrokerAllTopicsStats().failedFetchRequestRate.mark() + error("Error processing fetch operation on partition [%s,%d] offset %d".format(topic, partition, offset), e) + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, false, Some(e)) + } + } + + var limitBytes = fetchMaxBytes + val result = new mutable.ArrayBuffer[(TopicAndPartition, LogReadResult)] + readPartitionInfo.foreach { case (tp, fetchInfo) => + val minOneMessage = tp == readPartitionInfo.head._1 + val readResult = read(tp, fetchInfo, limitBytes, minOneMessage) + limitBytes = math.max(0, limitBytes - readResult.info.messageSet.sizeInBytes) + result += (tp -> readResult) } + result } def getMessageFormatVersion(topicAndPartition: TopicAndPartition): Option[Byte] = @@ -870,7 +878,7 @@ class ReplicaManager(val config: KafkaConfig, allPartitions.values.foreach(partition => partition.maybeShrinkIsr(config.replicaLagTimeMaxMs)) } - private def updateFollowerLogReadResults(replicaId: Int, readResults: Map[TopicAndPartition, LogReadResult]) { + private def updateFollowerLogReadResults(replicaId: Int, readResults: Seq[(TopicAndPartition, LogReadResult)]) { debug("Recording follower broker %d log read results: %s ".format(replicaId, readResults)) readResults.foreach { case (topicAndPartition, readResult) => getPartition(topicAndPartition.topic, topicAndPartition.partition) match { diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 243f9138f72f..74d62921b680 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -539,26 +539,38 @@ class PlaintextConsumerTest extends BaseConsumerTest { } @Test - def testFetchRecordTooLarge() { + def testFetchRecordLargerThanFetchMaxBytes() { val maxFetchBytes = 10 * 1024 - this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, maxFetchBytes.toString) + this.consumerConfig.setProperty(ConsumerConfig.FETCH_MAX_BYTES_CONFIG, maxFetchBytes.toString) + checkLargeRecord(maxFetchBytes + 1) + } + + private def checkLargeRecord(producerRecordSize: Int): Unit = { val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) consumers += consumer0 // produce a record that is larger than the configured fetch size - val record = new ProducerRecord[Array[Byte], Array[Byte]](tp.topic(), tp.partition(), "key".getBytes, new Array[Byte](maxFetchBytes + 1)) + val record = new ProducerRecord[Array[Byte], Array[Byte]](tp.topic(), tp.partition(), "key".getBytes, + new Array[Byte](producerRecordSize)) this.producers.head.send(record) - // consuming a too-large record should fail + // consuming a record that is too large should succeed since KIP-74 consumer0.assign(List(tp).asJava) - val e = intercept[RecordTooLargeException] { - consumer0.poll(20000) - } - val oversizedPartitions = e.recordTooLargePartitions() - assertNotNull(oversizedPartitions) - assertEquals(1, oversizedPartitions.size) - // the oversized message is at offset 0 - assertEquals(0L, oversizedPartitions.get(tp)) + val records = consumer0.poll(20000) + assertEquals(1, records.count) + val consumerRecord = records.iterator().next() + assertEquals(0L, consumerRecord.offset) + assertEquals(tp.topic(), consumerRecord.topic()) + assertEquals(tp.partition(), consumerRecord.partition()) + assertArrayEquals(record.key(), consumerRecord.key()) + assertArrayEquals(record.value(), consumerRecord.value()) + } + + @Test + def testFetchRecordLargerThanMaxPartitionFetchBytes() { + val maxPartitionFetchBytes = 10 * 1024 + this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, maxPartitionFetchBytes.toString) + checkLargeRecord(maxPartitionFetchBytes + 1) } @Test diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 61f7b501f1dd..f754feb5de34 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -136,8 +136,8 @@ class ReplicaManagerTest { } var fetchCallbackFired = false - def fetchCallback(responseStatus: Map[TopicAndPartition, FetchResponsePartitionData]) = { - assertEquals("Should give NotLeaderForPartitionException", Errors.NOT_LEADER_FOR_PARTITION.code, responseStatus.values.head.error) + def fetchCallback(responseStatus: Seq[(TopicAndPartition, FetchResponsePartitionData)]) = { + assertEquals("Should give NotLeaderForPartitionException", Errors.NOT_LEADER_FOR_PARTITION.code, responseStatus.map(_._2).head.error) fetchCallbackFired = true } @@ -172,7 +172,7 @@ class ReplicaManagerTest { replicaId = -1, fetchMinBytes = 100000, fetchMaxBytes = Int.MaxValue, - fetchInfo = collection.immutable.Map(new TopicAndPartition(topic, 0) -> new PartitionFetchInfo(0, 100000)), + fetchInfos = Seq(new TopicAndPartition(topic, 0) -> new PartitionFetchInfo(0, 100000)), responseCallback = fetchCallback) // Make this replica the follower @@ -203,8 +203,8 @@ class ReplicaManagerTest { EasyMock.expect(metadataCache.getAliveBrokers).andReturn(aliveBrokers).anyTimes() EasyMock.replay(metadataCache) - val brokerList : java.util.List[Integer] = Seq[Integer](0, 1, 2).asJava - val brokerSet : java.util.Set[Integer] = Set[Integer](0, 1, 2).asJava + val brokerList: java.util.List[Integer] = Seq[Integer](0, 1, 2).asJava + val brokerSet: java.util.Set[Integer] = Set[Integer](0, 1, 2).asJava val partition = rm.getOrCreatePartition(topic, 0) partition.getOrCreateReplica(0) @@ -230,9 +230,9 @@ class ReplicaManagerTest { var fetchCallbackFired = false var fetchError = 0 var fetchedMessages: MessageSet = null - def fetchCallback(responseStatus: Map[TopicAndPartition, FetchResponsePartitionData]) = { - fetchError = responseStatus.values.head.error - fetchedMessages = responseStatus.values.head.messages + def fetchCallback(responseStatus: Seq[(TopicAndPartition, FetchResponsePartitionData)]) = { + fetchError = responseStatus.map(_._2).head.error + fetchedMessages = responseStatus.map(_._2).head.messages fetchCallbackFired = true } @@ -242,7 +242,7 @@ class ReplicaManagerTest { replicaId = 1, fetchMinBytes = 0, fetchMaxBytes = Int.MaxValue, - fetchInfo = collection.immutable.Map(new TopicAndPartition(topic, 0) -> new PartitionFetchInfo(1, 100000)), + fetchInfos = Seq(new TopicAndPartition(topic, 0) -> new PartitionFetchInfo(1, 100000)), responseCallback = fetchCallback) @@ -257,7 +257,7 @@ class ReplicaManagerTest { replicaId = -1, fetchMinBytes = 0, fetchMaxBytes = Int.MaxValue, - fetchInfo = collection.immutable.Map(new TopicAndPartition(topic, 0) -> new PartitionFetchInfo(1, 100000)), + fetchInfos = Seq(new TopicAndPartition(topic, 0) -> new PartitionFetchInfo(1, 100000)), responseCallback = fetchCallback) assertTrue(fetchCallbackFired) diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index 0e1f68d48a71..f773ea2ff70a 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -26,12 +26,11 @@ import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.utils.{MockTime => JMockTime} import org.junit.{Test, After, Before} -import java.util.{Properties, Collections} +import java.util.{Properties} import java.util.concurrent.atomic.AtomicBoolean import collection.JavaConversions._ import org.easymock.EasyMock -import org.I0Itec.zkclient.ZkClient import org.junit.Assert._ class SimpleFetchTest { @@ -62,7 +61,7 @@ class SimpleFetchTest { val partitionId = 0 val topicAndPartition = TopicAndPartition(topic, partitionId) - val fetchInfo = Collections.singletonMap(topicAndPartition, PartitionFetchInfo(0, fetchSize)).toMap + val fetchInfo = Seq(topicAndPartition -> PartitionFetchInfo(0, fetchSize)) var replicaManager: ReplicaManager = null @@ -80,12 +79,12 @@ class SimpleFetchTest { val log = EasyMock.createMock(classOf[Log]) EasyMock.expect(log.logEndOffset).andReturn(leaderLEO).anyTimes() EasyMock.expect(log.logEndOffsetMetadata).andReturn(new LogOffsetMetadata(leaderLEO)).anyTimes() - EasyMock.expect(log.read(0, fetchSize, Some(partitionHW))).andReturn( + EasyMock.expect(log.read(0, fetchSize, Some(partitionHW), true)).andReturn( new FetchDataInfo( new LogOffsetMetadata(0L, 0L, 0), new ByteBufferMessageSet(messagesToHW) )).anyTimes() - EasyMock.expect(log.read(0, fetchSize, None)).andReturn( + EasyMock.expect(log.read(0, fetchSize, None, true)).andReturn( new FetchDataInfo( new LogOffsetMetadata(0L, 0L, 0), new ByteBufferMessageSet(messagesToLEO) @@ -148,9 +147,9 @@ class SimpleFetchTest { val initialAllTopicsCount = BrokerTopicStats.getBrokerAllTopicsStats().totalFetchRequestRate.count() assertEquals("Reading committed data should return messages only up to high watermark", messagesToHW, - replicaManager.readFromLocalLog(true, true, Int.MaxValue, fetchInfo).get(topicAndPartition).get.info.messageSet.head.message) + replicaManager.readFromLocalLog(true, true, Int.MaxValue, fetchInfo).find(_._1 == topicAndPartition).get._2.info.messageSet.head.message) assertEquals("Reading any data can return messages up to the end of the log", messagesToLEO, - replicaManager.readFromLocalLog(true, false, Int.MaxValue, fetchInfo).get(topicAndPartition).get.info.messageSet.head.message) + replicaManager.readFromLocalLog(true, false, Int.MaxValue, fetchInfo).find(_._1 == topicAndPartition).get._2.info.messageSet.head.message) assertEquals("Counts should increment after fetch", initialTopicCount+2, BrokerTopicStats.getBrokerTopicStats(topic).totalFetchRequestRate.count()) assertEquals("Counts should increment after fetch", initialAllTopicsCount+2, BrokerTopicStats.getBrokerAllTopicsStats().totalFetchRequestRate.count()) From 52402f20011bfbc8449493f4dfda21fbf8377205 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 12:08:07 +0100 Subject: [PATCH 18/54] Fix comment in `DEFAULT_RESPONSE_MAX_BYTES` --- .../java/org/apache/kafka/common/requests/FetchRequest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java index 4980f8639635..f224cb57aa1b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -44,7 +44,7 @@ public class FetchRequest extends AbstractRequest { private static final String FETCH_OFFSET_KEY_NAME = "fetch_offset"; private static final String MAX_BYTES_KEY_NAME = "max_bytes"; - // default values for current version + // default values for older versions where a request level limit did not exist public static final int DEFAULT_RESPONSE_MAX_BYTES = Integer.MAX_VALUE; private final int replicaId; From c0503f4caadc3f3c644c715826f4e0d4055b7d4f Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 13:17:05 +0100 Subject: [PATCH 19/54] Mention broker configs for limiting message size --- .../kafka/clients/consumer/ConsumerConfig.java | 14 ++++++++------ core/src/main/scala/kafka/server/KafkaConfig.scala | 10 ++++++++-- 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 972916228dcd..9f474a81138e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -117,9 +117,10 @@ public class ConsumerConfig extends AbstractConfig { */ public static final String FETCH_MAX_BYTES_CONFIG = "fetch.max.bytes"; private static final String FETCH_MAX_BYTES_DOC = "The maximum amount of data the server should return for a fetch request. " + - "This is not an absolute maximum - if the first message in the first partition of the fetch is larger than " + - "this value, the message will still be returned to ensure that the consumer can make progress. Also note " + - "that the consumer performs multiple fetches in parallel."; + "This is not an absolute maximum, if the first message in the first partition of the fetch is larger than " + + "this value, the message will still be returned to ensure that the consumer can make progress. " + + "The maximum message size accepted by the broker is defined via message.max.bytes (broker config) or " + + "max.message.bytes (topic config). Note that the consumer performs multiple fetches in parallel."; public static final int DEFAULT_FETCH_MAX_BYTES = 50 * 1024 * 1024; /** @@ -136,9 +137,10 @@ public class ConsumerConfig extends AbstractConfig { */ public static final String MAX_PARTITION_FETCH_BYTES_CONFIG = "max.partition.fetch.bytes"; private static final String MAX_PARTITION_FETCH_BYTES_DOC = "The maximum amount of data per-partition the server " + - "will return. If the first message in the first partition of the fetch is larger than this limit, the message will still " + - "be returned to ensure that the consumer can make progress. See " + FETCH_MAX_BYTES_CONFIG + " for limiting " + - "the request size."; + "will return. If the first message in the first partition of the fetch is larger than this limit, the " + + "message will still be returned to ensure that the consumer can make progress. The maximum message size " + + "accepted by the broker is defined via message.max.bytes (broker config) or " + + "max.message.bytes (topic config). See " + FETCH_MAX_BYTES_CONFIG + " for limiting the consumer request size"; public static final int DEFAULT_MAX_PARTITION_FETCH_BYTES = 1 * 1024 * 1024; /** send.buffer.bytes */ diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 84bec8006331..3b67f0f38293 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -477,11 +477,17 @@ object KafkaConfig { " the leader will remove the follower from isr" val ReplicaSocketTimeoutMsDoc = "The socket timeout for network requests. Its value should be at least replica.fetch.wait.max.ms" val ReplicaSocketReceiveBufferBytesDoc = "The socket receive buffer for network requests" - val ReplicaFetchMaxBytesDoc = "The number of bytes of messages to attempt to fetch for each partition" + val ReplicaFetchMaxBytesDoc = "The number of bytes of messages to attempt to fetch for each partition. This is not an absolute maximum, " + + "if the first message in the first partition of the fetch is larger than this value, the message will still be returned " + + "to ensure that progress can be made. The maximum message size accepted by the broker is defined via " + + "message.max.bytes (broker config) or max.message.bytes (topic config)." val ReplicaFetchWaitMaxMsDoc = "max wait time for each fetcher request issued by follower replicas. This value should always be less than the " + "replica.lag.time.max.ms at all times to prevent frequent shrinking of ISR for low throughput topics" val ReplicaFetchMinBytesDoc = "Minimum bytes expected for each fetch response. If not enough bytes, wait up to replicaMaxWaitTimeMs" - val ReplicaFetchResponseMaxBytesDoc = "Maximum bytes expected for entire fetch response." + val ReplicaFetchResponseMaxBytesDoc = "Maximum bytes expected for the entire fetch response. This is not an absolute maximum, " + + "if the first message in the first partition of the fetch is larger than this value, the message will still be returned " + + "to ensure that progress can be made. The maximum message size accepted by the broker is defined via " + + "message.max.bytes (broker config) or max.message.bytes (topic config)." val NumReplicaFetchersDoc = "Number of fetcher threads used to replicate messages from a source broker. " + "Increasing this value can increase the degree of I/O parallelism in the follower broker." val ReplicaFetchBackoffMsDoc = "The amount of time to sleep when fetch partition error occurs." From b38e096041d58dd83e51d772569dc2ca0e0b4fbf Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 13:23:06 +0100 Subject: [PATCH 20/54] Allow `message.max.bytes` to be smaller than `replica.fetch.max.bytes` --- core/src/main/scala/kafka/server/KafkaConfig.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 3b67f0f38293..bffb1928d422 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -1036,7 +1036,6 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean) extends Abstra require(logCleanerDedupeBufferSize / logCleanerThreads > 1024 * 1024, "log.cleaner.dedupe.buffer.size must be at least 1MB per cleaner thread.") require(replicaFetchWaitMaxMs <= replicaSocketTimeoutMs, "replica.socket.timeout.ms should always be at least replica.fetch.wait.max.ms" + " to prevent unnecessary socket timeouts") - require(replicaFetchMaxBytes >= messageMaxBytes, "replica.fetch.max.bytes should be equal or greater than message.max.bytes") require(replicaFetchWaitMaxMs <= replicaLagTimeMaxMs, "replica.fetch.wait.max.ms should always be at least replica.lag.time.max.ms" + " to prevent frequent changes in ISR") require(offsetCommitRequiredAcks >= -1 && offsetCommitRequiredAcks <= offsetsTopicReplicationFactor, From e8b01055282ec6a26b60a1ddb027c4012c445391 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 13:34:39 +0100 Subject: [PATCH 21/54] Add validator for `ReplicaFetchMaxBytesProp` and reduce priority to medium --- core/src/main/scala/kafka/server/KafkaConfig.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index bffb1928d422..0a95d3216ca0 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -667,7 +667,7 @@ object KafkaConfig { .define(ReplicaLagTimeMaxMsProp, LONG, Defaults.ReplicaLagTimeMaxMs, HIGH, ReplicaLagTimeMaxMsDoc) .define(ReplicaSocketTimeoutMsProp, INT, Defaults.ReplicaSocketTimeoutMs, HIGH, ReplicaSocketTimeoutMsDoc) .define(ReplicaSocketReceiveBufferBytesProp, INT, Defaults.ReplicaSocketReceiveBufferBytes, HIGH, ReplicaSocketReceiveBufferBytesDoc) - .define(ReplicaFetchMaxBytesProp, INT, Defaults.ReplicaFetchMaxBytes, HIGH, ReplicaFetchMaxBytesDoc) + .define(ReplicaFetchMaxBytesProp, INT, Defaults.ReplicaFetchMaxBytes, atLeast(0), MEDIUM, ReplicaFetchMaxBytesDoc) .define(ReplicaFetchWaitMaxMsProp, INT, Defaults.ReplicaFetchWaitMaxMs, HIGH, ReplicaFetchWaitMaxMsDoc) .define(ReplicaFetchBackoffMsProp, INT, Defaults.ReplicaFetchBackoffMs, atLeast(0), MEDIUM, ReplicaFetchBackoffMsDoc) .define(ReplicaFetchMinBytesProp, INT, Defaults.ReplicaFetchMinBytes, HIGH, ReplicaFetchMinBytesDoc) From 7ecde2117c85f0e478975f969d359aa8369a103f Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 13:38:32 +0100 Subject: [PATCH 22/54] Rename `groupByTopicOrdered` to `batchByTopic` to be consistent --- .../java/org/apache/kafka/common/requests/FetchRequest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java index f224cb57aa1b..c0e343c8a794 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -72,7 +72,7 @@ public TopicAndPartitionData(String topic) { this.partitions = new LinkedHashMap<>(); } - public static final List groupByTopicOrdered(LinkedHashMap fetchData) { + public static final List batchByTopic(LinkedHashMap fetchData) { List topics = new ArrayList<>(); for (Map.Entry topicEntry : fetchData.entrySet()) { String topic = topicEntry.getKey().topic(); @@ -123,7 +123,7 @@ public static FetchRequest fromReplica(int replicaId, int maxWait, int minBytes, private FetchRequest(int version, int replicaId, int maxWait, int minBytes, int maxBytes, LinkedHashMap fetchData) { super(new Struct(ProtoUtils.requestSchema(ApiKeys.FETCH.id, version))); - List topicsData = TopicAndPartitionData.groupByTopicOrdered(fetchData); + List topicsData = TopicAndPartitionData.batchByTopic(fetchData); struct.set(REPLICA_ID_KEY_NAME, replicaId); struct.set(MAX_WAIT_KEY_NAME, maxWait); From ee686f5e18848f33264a888067dac365815b61cd Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 13:40:29 +0100 Subject: [PATCH 23/54] Combine "max_bytes` key name constants --- .../apache/kafka/common/requests/FetchRequest.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java index c0e343c8a794..de7d495a42e5 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -32,9 +32,11 @@ public class FetchRequest extends AbstractRequest { private static final String REPLICA_ID_KEY_NAME = "replica_id"; private static final String MAX_WAIT_KEY_NAME = "max_wait_time"; private static final String MIN_BYTES_KEY_NAME = "min_bytes"; - private static final String RESPONSE_MAX_BYTES_KEY_NAME = "max_bytes"; private static final String TOPICS_KEY_NAME = "topics"; + // request and partition level name + private static final String MAX_BYTES_KEY_NAME = "max_bytes"; + // topic level field names private static final String TOPIC_KEY_NAME = "topic"; private static final String PARTITIONS_KEY_NAME = "partitions"; @@ -42,7 +44,6 @@ public class FetchRequest extends AbstractRequest { // partition level field names private static final String PARTITION_KEY_NAME = "partition"; private static final String FETCH_OFFSET_KEY_NAME = "fetch_offset"; - private static final String MAX_BYTES_KEY_NAME = "max_bytes"; // default values for older versions where a request level limit did not exist public static final int DEFAULT_RESPONSE_MAX_BYTES = Integer.MAX_VALUE; @@ -129,7 +130,7 @@ private FetchRequest(int version, int replicaId, int maxWait, int minBytes, int struct.set(MAX_WAIT_KEY_NAME, maxWait); struct.set(MIN_BYTES_KEY_NAME, minBytes); if (version >= 3) - struct.set(RESPONSE_MAX_BYTES_KEY_NAME, maxBytes); + struct.set(MAX_BYTES_KEY_NAME, maxBytes); List topicArray = new ArrayList(); for (TopicAndPartitionData topicEntry : topicsData) { Struct topicData = struct.instance(TOPICS_KEY_NAME); @@ -159,8 +160,8 @@ public FetchRequest(Struct struct) { replicaId = struct.getInt(REPLICA_ID_KEY_NAME); maxWait = struct.getInt(MAX_WAIT_KEY_NAME); minBytes = struct.getInt(MIN_BYTES_KEY_NAME); - if (struct.hasField(RESPONSE_MAX_BYTES_KEY_NAME)) - maxBytes = struct.getInt(RESPONSE_MAX_BYTES_KEY_NAME); + if (struct.hasField(MAX_BYTES_KEY_NAME)) + maxBytes = struct.getInt(MAX_BYTES_KEY_NAME); else maxBytes = DEFAULT_RESPONSE_MAX_BYTES; fetchData = new LinkedHashMap<>(); From 22c039c8c82071cc5bfe44b1994587663f36b301 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 13:41:32 +0100 Subject: [PATCH 24/54] Remove `final` from static method --- .../java/org/apache/kafka/common/requests/FetchRequest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java index de7d495a42e5..4fffa5df9584 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -73,7 +73,7 @@ public TopicAndPartitionData(String topic) { this.partitions = new LinkedHashMap<>(); } - public static final List batchByTopic(LinkedHashMap fetchData) { + public static List batchByTopic(LinkedHashMap fetchData) { List topics = new ArrayList<>(); for (Map.Entry topicEntry : fetchData.entrySet()) { String topic = topicEntry.getKey().topic(); From 387c4ac1c8ea6a3d6ad45a4c04d436bda98a5fd6 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 14:06:51 +0100 Subject: [PATCH 25/54] Use `batchByTopic` in `FetchResponse` too --- .../kafka/common/requests/FetchRequest.java | 16 ++++++++-------- .../kafka/common/requests/FetchResponse.java | 11 +++++------ 2 files changed, 13 insertions(+), 14 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java index 4fffa5df9584..913b6e6c6882 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -64,21 +64,21 @@ public PartitionData(long offset, int maxBytes) { } } - private static final class TopicAndPartitionData { + static final class TopicAndPartitionData { public final String topic; - public final LinkedHashMap partitions; + public final LinkedHashMap partitions; public TopicAndPartitionData(String topic) { this.topic = topic; this.partitions = new LinkedHashMap<>(); } - public static List batchByTopic(LinkedHashMap fetchData) { - List topics = new ArrayList<>(); - for (Map.Entry topicEntry : fetchData.entrySet()) { + public static List> batchByTopic(LinkedHashMap data) { + List> topics = new ArrayList<>(); + for (Map.Entry topicEntry : data.entrySet()) { String topic = topicEntry.getKey().topic(); int partition = topicEntry.getKey().partition(); - PartitionData partitionData = topicEntry.getValue(); + T partitionData = topicEntry.getValue(); if (topics.isEmpty() || !topics.get(topics.size() - 1).topic.equals(topic)) topics.add(new TopicAndPartitionData(topic)); topics.get(topics.size() - 1).partitions.put(partition, partitionData); @@ -124,7 +124,7 @@ public static FetchRequest fromReplica(int replicaId, int maxWait, int minBytes, private FetchRequest(int version, int replicaId, int maxWait, int minBytes, int maxBytes, LinkedHashMap fetchData) { super(new Struct(ProtoUtils.requestSchema(ApiKeys.FETCH.id, version))); - List topicsData = TopicAndPartitionData.batchByTopic(fetchData); + List> topicsData = TopicAndPartitionData.batchByTopic(fetchData); struct.set(REPLICA_ID_KEY_NAME, replicaId); struct.set(MAX_WAIT_KEY_NAME, maxWait); @@ -132,7 +132,7 @@ private FetchRequest(int version, int replicaId, int maxWait, int minBytes, int if (version >= 3) struct.set(MAX_BYTES_KEY_NAME, maxBytes); List topicArray = new ArrayList(); - for (TopicAndPartitionData topicEntry : topicsData) { + for (TopicAndPartitionData topicEntry : topicsData) { Struct topicData = struct.instance(TOPICS_KEY_NAME); topicData.set(TOPIC_KEY_NAME, topicEntry.topic); List partitionArray = new ArrayList(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java index ee1c3debdf6e..905cf127ec83 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java @@ -21,7 +21,7 @@ import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.utils.CollectionUtils; +import org.apache.kafka.common.requests.FetchRequest.TopicAndPartitionData; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -110,14 +110,13 @@ public FetchResponse(LinkedHashMap responseData, private FetchResponse(int version, LinkedHashMap responseData, int throttleTime) { super(new Struct(ProtoUtils.responseSchema(ApiKeys.FETCH.id, version))); - //FIXME Need to fix this to maintain ordering - Map> topicsData = CollectionUtils.groupDataByTopic(responseData); + List> topicsData = FetchRequest.TopicAndPartitionData.batchByTopic(responseData); List topicArray = new ArrayList<>(); - for (Map.Entry> topicEntry: topicsData.entrySet()) { + for (TopicAndPartitionData topicEntry: topicsData) { Struct topicData = struct.instance(RESPONSES_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); + topicData.set(TOPIC_KEY_NAME, topicEntry.topic); List partitionArray = new ArrayList<>(); - for (Map.Entry partitionEntry : topicEntry.getValue().entrySet()) { + for (Map.Entry partitionEntry : topicEntry.partitions.entrySet()) { PartitionData fetchPartitionData = partitionEntry.getValue(); Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey()); From d58398876677c879f1f7a714df714fbb98d8f4c6 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 15:36:39 +0100 Subject: [PATCH 26/54] Add tests to verify that we only return a large message if it's the first message in the first partition --- .../kafka/api/PlaintextConsumerTest.scala | 39 +++++++++++++++++++ 1 file changed, 39 insertions(+) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 74d62921b680..dfac2cf1a9af 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -566,6 +566,45 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertArrayEquals(record.value(), consumerRecord.value()) } + /** We should only return a large record if it's the first record in the first partition of the fetch request */ + @Test + def testFetchHonoursFetchSizeIfLargeRecordNotFirst(): Unit = { + val maxFetchBytes = 10 * 1024 + this.consumerConfig.setProperty(ConsumerConfig.FETCH_MAX_BYTES_CONFIG, maxFetchBytes.toString) + checkFetchHonoursSizeIfLargeRecordNotFirst(maxFetchBytes) + } + + private def checkFetchHonoursSizeIfLargeRecordNotFirst(largeProducerRecordSize: Int): Unit = { + val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) + consumers += consumer0 + + val smallRecord = new ProducerRecord[Array[Byte], Array[Byte]](tp.topic(), tp.partition(), "small".getBytes, + "value".getBytes) + val largeRecord = new ProducerRecord[Array[Byte], Array[Byte]](tp.topic(), tp.partition(), "large".getBytes, + new Array[Byte](largeProducerRecordSize)) + this.producers.head.send(smallRecord) + this.producers.head.send(largeRecord) + + // consuming a record that is too large should succeed since KIP-74 + consumer0.assign(List(tp).asJava) + val records = consumer0.poll(20000) + assertEquals(1, records.count) + val consumerRecord = records.iterator().next() + assertEquals(0L, consumerRecord.offset) + assertEquals(tp.topic(), consumerRecord.topic()) + assertEquals(tp.partition(), consumerRecord.partition()) + assertArrayEquals(smallRecord.key(), consumerRecord.key()) + assertArrayEquals(smallRecord.value(), consumerRecord.value()) + } + + /** We should only return a large record if it's the first record in the first partition of the fetch request */ + @Test + def testFetchHonoursMaxPartitionFetchBytesIfLargeRecordNotFirst(): Unit = { + val maxPartitionFetchBytes = 10 * 1024 + this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, maxPartitionFetchBytes.toString) + checkFetchHonoursSizeIfLargeRecordNotFirst(maxPartitionFetchBytes) + } + @Test def testFetchRecordLargerThanMaxPartitionFetchBytes() { val maxPartitionFetchBytes = 10 * 1024 From 1ae52e5d255d087ddb89fce8fa9089417a7f71f6 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 15:58:24 +0100 Subject: [PATCH 27/54] Rename `partitionMap` to `fetchBuilder` in `AbstractFetcherThread` --- .../kafka/server/AbstractFetcherThread.scala | 30 +++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index 5bafbaebf4ad..edb2736782b9 100755 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -21,7 +21,7 @@ import java.util.concurrent.locks.ReentrantLock import kafka.cluster.BrokerEndPoint import kafka.consumer.PartitionTopicInfo -import kafka.message.{ByteBufferMessageSet} +import kafka.message.ByteBufferMessageSet import kafka.utils.{DelayedItem, Pool, ShutdownableThread} import kafka.common.{ClientIdAndBroker, KafkaException} import kafka.metrics.KafkaMetricsGroup @@ -52,7 +52,7 @@ abstract class AbstractFetcherThread(name: String, type REQ <: FetchRequest type PD <: PartitionData - private val partitionMap = new FetchBuilder[PartitionFetchState] + private val fetchBuilder = new FetchBuilder[PartitionFetchState] private val partitionMapLock = new ReentrantLock private val partitionMapCond = partitionMapLock.newCondition() @@ -90,7 +90,7 @@ abstract class AbstractFetcherThread(name: String, override def doWork() { val fetchRequest = inLock(partitionMapLock) { - val fetchRequest = buildFetchRequest(partitionMap.partitionStates.asScala.map { state => + val fetchRequest = buildFetchRequest(fetchBuilder.partitionStates.asScala.map { state => state.topicPartition -> state.value }) if (fetchRequest.isEmpty) { @@ -109,7 +109,7 @@ abstract class AbstractFetcherThread(name: String, def updatePartitionsWithError(partition: TopicPartition): Unit = { partitionsWithError += partition - partitionMap.moveToEnd(partition) + fetchBuilder.moveToEnd(partition) } var responseData: Seq[(TopicPartition, PD)] = Seq.empty @@ -122,7 +122,7 @@ abstract class AbstractFetcherThread(name: String, if (isRunning.get) { warn(s"Error in fetch $fetchRequest", t) inLock(partitionMapLock) { - partitionMap.partitionSet.asScala.foreach(updatePartitionsWithError) + fetchBuilder.partitionSet.asScala.foreach(updatePartitionsWithError) // there is an error occurred while fetching partitions, sleep a while // note that `ReplicaFetcherThread.handlePartitionsWithError` will also introduce the same delay for every // partition with error effectively doubling the delay. It would be good to improve this. @@ -139,7 +139,7 @@ abstract class AbstractFetcherThread(name: String, responseData.foreach { case (topicPartition, partitionData) => val topic = topicPartition.topic val partitionId = topicPartition.partition - Option(partitionMap.stateValue(topicPartition)).foreach(currentPartitionFetchState => + Option(fetchBuilder.stateValue(topicPartition)).foreach(currentPartitionFetchState => // we append to the log if the current offset is defined and it is the same as the offset requested during fetch if (fetchRequest.offset(topicPartition) == currentPartitionFetchState.offset) { Errors.forCode(partitionData.errorCode) match { @@ -148,7 +148,7 @@ abstract class AbstractFetcherThread(name: String, val messages = partitionData.toByteBufferMessageSet val newOffset = messages.shallowIterator.toSeq.lastOption match { case Some(m) => - partitionMap.updateAndMoveToEnd(topicPartition, new PartitionFetchState(m.nextOffset)) + fetchBuilder.updateAndMoveToEnd(topicPartition, new PartitionFetchState(m.nextOffset)) fetcherStats.byteRate.mark(messages.validBytes) m.nextOffset case None => @@ -172,7 +172,7 @@ abstract class AbstractFetcherThread(name: String, case Errors.OFFSET_OUT_OF_RANGE => try { val newOffset = handleOffsetOutOfRange(topicPartition) - partitionMap.updateAndMoveToEnd(topicPartition, new PartitionFetchState(newOffset)) + fetchBuilder.updateAndMoveToEnd(topicPartition, new PartitionFetchState(newOffset)) error("Current offset %d for partition [%s,%d] out of range; reset offset to %d" .format(currentPartitionFetchState.offset, topic, partitionId, newOffset)) } catch { @@ -203,17 +203,17 @@ abstract class AbstractFetcherThread(name: String, try { // If the partitionMap already has the topic/partition, then do not update the map with the old offset val newPartitionToState = partitionAndOffsets.filter { case (tp, _) => - !partitionMap.contains(tp) + !fetchBuilder.contains(tp) }.map { case (tp, offset) => val fetchState = if (PartitionTopicInfo.isOffsetInvalid(offset)) new PartitionFetchState(handleOffsetOutOfRange(tp)) else new PartitionFetchState(offset) tp -> fetchState } - val existingPartitionToState = partitionMap.partitionStates.asScala.map { state => + val existingPartitionToState = fetchBuilder.partitionStates.asScala.map { state => state.topicPartition -> state.value }.toMap - partitionMap.set((existingPartitionToState ++ newPartitionToState).asJava) + fetchBuilder.set((existingPartitionToState ++ newPartitionToState).asJava) partitionMapCond.signalAll() } finally partitionMapLock.unlock() } @@ -222,9 +222,9 @@ abstract class AbstractFetcherThread(name: String, partitionMapLock.lockInterruptibly() try { for (partition <- partitions) { - Option(partitionMap.stateValue(partition)).foreach (currentPartitionFetchState => + Option(fetchBuilder.stateValue(partition)).foreach (currentPartitionFetchState => if (currentPartitionFetchState.isActive) - partitionMap.updateAndMoveToEnd(partition, new PartitionFetchState(currentPartitionFetchState.offset, new DelayedItem(delay))) + fetchBuilder.updateAndMoveToEnd(partition, new PartitionFetchState(currentPartitionFetchState.offset, new DelayedItem(delay))) ) } partitionMapCond.signalAll() @@ -235,7 +235,7 @@ abstract class AbstractFetcherThread(name: String, partitionMapLock.lockInterruptibly() try { topicPartitions.foreach { topicPartition => - partitionMap.remove(topicPartition) + fetchBuilder.remove(topicPartition) fetcherLagStats.unregister(topicPartition.topic, topicPartition.partition) } } finally partitionMapLock.unlock() @@ -243,7 +243,7 @@ abstract class AbstractFetcherThread(name: String, def partitionCount() = { partitionMapLock.lockInterruptibly() - try partitionMap.size + try fetchBuilder.size finally partitionMapLock.unlock() } From cdc9ee31e619cda4c3da2fc1a0f843bbb1490be6 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 16:58:36 +0100 Subject: [PATCH 28/54] Add test for `FetchBuilder` and document it --- .../kafka/common/internals/FetchBuilder.java | 56 ++++- .../common/internals/FetchBuilderTest.java | 219 ++++++++++++++++++ 2 files changed, 271 insertions(+), 4 deletions(-) create mode 100644 clients/src/test/java/org/apache/kafka/common/internals/FetchBuilderTest.java diff --git a/clients/src/main/java/org/apache/kafka/common/internals/FetchBuilder.java b/clients/src/main/java/org/apache/kafka/common/internals/FetchBuilder.java index 6292bef5ec38..f45d609f81bf 100644 --- a/clients/src/main/java/org/apache/kafka/common/internals/FetchBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/internals/FetchBuilder.java @@ -23,11 +23,20 @@ import java.util.Collections; import java.util.HashMap; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; -//TODO Better name? +/** + * This builder is a useful building block for doing fetch requests where topic partitions have to be rotated via + * round-robin to ensure fairness and some level of determinism given the existence of a limit on the fetch response + * size. Because the serialization of fetch requests is more efficient if all partitions for the same topic are grouped + * together, we do such grouping in the method `set`. + * + * As partitions are moved to the end, the same topic may be repeated more than once. + */ public class FetchBuilder { private final List> list = new ArrayList<>(); @@ -72,6 +81,9 @@ public void remove(TopicPartition topicPartition) { } } + /** + * Returns the partitions in random order. + */ public Set partitionSet() { return map.keySet(); } @@ -85,10 +97,16 @@ public boolean contains(TopicPartition topicPartition) { return map.containsKey(topicPartition); } + /** + * Returns the partition states in order. + */ public List> partitionStates() { return Collections.unmodifiableList(list); } + /** + * Returns the partition state values in order. + */ public List partitionStateValues() { List result = new ArrayList<>(list.size()); for (PartitionState state : list) @@ -104,6 +122,12 @@ public int size() { return map.size(); } + /** + * Update the builder to have the received map as its state (i.e. the previous state is cleared). The builder will + * "batch by topic", so if we have a, b and c, each with two partitions, we may end up with something like the + * following (the order of topics and partitions within topics is dependent on the iteration order of the received + * map): a0, a1, b1, b0, c0, c1. + */ public void set(Map partitionToState) { map.clear(); list.clear(); @@ -111,7 +135,7 @@ public void set(Map partitionToState) { } private void update(Map partitionToState) { - Map> topicToPartitions = new HashMap<>(); + Map> topicToPartitions = new LinkedHashMap<>(); for (TopicPartition tp : partitionToState.keySet()) { List partitions = topicToPartitions.get(tp.topic()); if (partitions == null) { @@ -134,17 +158,41 @@ public static class PartitionState { private final S value; public PartitionState(TopicPartition topicPartition, S state) { - this.topicPartition = topicPartition; - this.value = state; + this.topicPartition = Objects.requireNonNull(topicPartition); + this.value = Objects.requireNonNull(state); } public S value() { return value; } + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + PartitionState that = (PartitionState) o; + + return topicPartition.equals(that.topicPartition) && value.equals(that.value); + } + + @Override + public int hashCode() { + int result = topicPartition.hashCode(); + result = 31 * result + value.hashCode(); + return result; + } + public TopicPartition topicPartition() { return topicPartition; } + + @Override + public String toString() { + return "PartitionState(" + topicPartition + "=" + value + ')'; + } } } diff --git a/clients/src/test/java/org/apache/kafka/common/internals/FetchBuilderTest.java b/clients/src/test/java/org/apache/kafka/common/internals/FetchBuilderTest.java new file mode 100644 index 000000000000..a109f0897395 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/internals/FetchBuilderTest.java @@ -0,0 +1,219 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.common.internals; + +import org.apache.kafka.common.TopicPartition; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class FetchBuilderTest { + + @Test + public void testSet() { + FetchBuilder builder = new FetchBuilder<>(); + LinkedHashMap map = createMap(); + builder.set(map); + LinkedHashMap expected = new LinkedHashMap<>(); + expected.put(new TopicPartition("foo", 2), "foo 2"); + expected.put(new TopicPartition("foo", 0), "foo 0"); + expected.put(new TopicPartition("blah", 2), "blah 2"); + expected.put(new TopicPartition("blah", 1), "blah 1"); + expected.put(new TopicPartition("baz", 2), "baz 2"); + expected.put(new TopicPartition("baz", 3), "baz 3"); + checkState(builder, expected); + + builder.set(new LinkedHashMap()); + checkState(builder, new LinkedHashMap()); + } + + private LinkedHashMap createMap() { + LinkedHashMap map = new LinkedHashMap<>(); + map.put(new TopicPartition("foo", 2), "foo 2"); + map.put(new TopicPartition("blah", 2), "blah 2"); + map.put(new TopicPartition("blah", 1), "blah 1"); + map.put(new TopicPartition("baz", 2), "baz 2"); + map.put(new TopicPartition("foo", 0), "foo 0"); + map.put(new TopicPartition("baz", 3), "baz 3"); + return map; + } + + private void checkState(FetchBuilder builder, LinkedHashMap expected) { + assertEquals(expected.keySet(), builder.partitionSet()); + assertEquals(expected.size(), builder.size()); + List> states = new ArrayList<>(); + for (Map.Entry entry : expected.entrySet()) { + states.add(new FetchBuilder.PartitionState<>(entry.getKey(), entry.getValue())); + assertTrue(builder.contains(entry.getKey())); + } + assertEquals(states, builder.partitionStates()); + } + + @Test + public void testMoveToEnd() { + FetchBuilder builder = new FetchBuilder<>(); + LinkedHashMap map = createMap(); + builder.set(map); + + builder.moveToEnd(new TopicPartition("baz", 2)); + LinkedHashMap expected = new LinkedHashMap<>(); + expected.put(new TopicPartition("foo", 2), "foo 2"); + expected.put(new TopicPartition("foo", 0), "foo 0"); + expected.put(new TopicPartition("blah", 2), "blah 2"); + expected.put(new TopicPartition("blah", 1), "blah 1"); + expected.put(new TopicPartition("baz", 3), "baz 3"); + expected.put(new TopicPartition("baz", 2), "baz 2"); + checkState(builder, expected); + + builder.moveToEnd(new TopicPartition("foo", 2)); + expected = new LinkedHashMap<>(); + expected.put(new TopicPartition("foo", 0), "foo 0"); + expected.put(new TopicPartition("blah", 2), "blah 2"); + expected.put(new TopicPartition("blah", 1), "blah 1"); + expected.put(new TopicPartition("baz", 3), "baz 3"); + expected.put(new TopicPartition("baz", 2), "baz 2"); + expected.put(new TopicPartition("foo", 2), "foo 2"); + checkState(builder, expected); + + // no-op + builder.moveToEnd(new TopicPartition("foo", 2)); + checkState(builder, expected); + + // partition doesn't exist + builder.moveToEnd(new TopicPartition("baz", 5)); + checkState(builder, expected); + + // topic doesn't exist + builder.moveToEnd(new TopicPartition("aaa", 2)); + checkState(builder, expected); + } + + @Test + public void testUpdateAndMoveToEnd() { + FetchBuilder builder = new FetchBuilder<>(); + LinkedHashMap map = createMap(); + builder.set(map); + + builder.updateAndMoveToEnd(new TopicPartition("foo", 0), "foo 0 updated"); + LinkedHashMap expected = new LinkedHashMap<>(); + expected.put(new TopicPartition("foo", 2), "foo 2"); + expected.put(new TopicPartition("blah", 2), "blah 2"); + expected.put(new TopicPartition("blah", 1), "blah 1"); + expected.put(new TopicPartition("baz", 2), "baz 2"); + expected.put(new TopicPartition("baz", 3), "baz 3"); + expected.put(new TopicPartition("foo", 0), "foo 0 updated"); + checkState(builder, expected); + + builder.updateAndMoveToEnd(new TopicPartition("baz", 2), "baz 2 updated"); + expected = new LinkedHashMap<>(); + expected.put(new TopicPartition("foo", 2), "foo 2"); + expected.put(new TopicPartition("blah", 2), "blah 2"); + expected.put(new TopicPartition("blah", 1), "blah 1"); + expected.put(new TopicPartition("baz", 3), "baz 3"); + expected.put(new TopicPartition("foo", 0), "foo 0 updated"); + expected.put(new TopicPartition("baz", 2), "baz 2 updated"); + checkState(builder, expected); + + // partition doesn't exist + builder.updateAndMoveToEnd(new TopicPartition("baz", 5), "baz 5 new"); + expected = new LinkedHashMap<>(); + expected.put(new TopicPartition("foo", 2), "foo 2"); + expected.put(new TopicPartition("blah", 2), "blah 2"); + expected.put(new TopicPartition("blah", 1), "blah 1"); + expected.put(new TopicPartition("baz", 3), "baz 3"); + expected.put(new TopicPartition("foo", 0), "foo 0 updated"); + expected.put(new TopicPartition("baz", 2), "baz 2 updated"); + expected.put(new TopicPartition("baz", 5), "baz 5 new"); + checkState(builder, expected); + + // topic doesn't exist + builder.updateAndMoveToEnd(new TopicPartition("aaa", 2), "aaa 2 new"); + expected = new LinkedHashMap<>(); + expected.put(new TopicPartition("foo", 2), "foo 2"); + expected.put(new TopicPartition("blah", 2), "blah 2"); + expected.put(new TopicPartition("blah", 1), "blah 1"); + expected.put(new TopicPartition("baz", 3), "baz 3"); + expected.put(new TopicPartition("foo", 0), "foo 0 updated"); + expected.put(new TopicPartition("baz", 2), "baz 2 updated"); + expected.put(new TopicPartition("baz", 5), "baz 5 new"); + expected.put(new TopicPartition("aaa", 2), "aaa 2 new"); + checkState(builder, expected); + } + + @Test + public void testPartitionValues() { + FetchBuilder builder = new FetchBuilder<>(); + LinkedHashMap map = createMap(); + builder.set(map); + List expected = new ArrayList<>(); + expected.add("foo 2"); + expected.add("foo 0"); + expected.add("blah 2"); + expected.add("blah 1"); + expected.add("baz 2"); + expected.add("baz 3"); + assertEquals(expected, builder.partitionStateValues()); + } + + @Test + public void testClear() { + FetchBuilder builder = new FetchBuilder<>(); + LinkedHashMap map = createMap(); + builder.set(map); + builder.clear(); + checkState(builder, new LinkedHashMap()); + } + + @Test + public void testRemove() { + FetchBuilder builder = new FetchBuilder<>(); + LinkedHashMap map = createMap(); + builder.set(map); + + builder.remove(new TopicPartition("foo", 2)); + LinkedHashMap expected = new LinkedHashMap<>(); + expected.put(new TopicPartition("foo", 0), "foo 0"); + expected.put(new TopicPartition("blah", 2), "blah 2"); + expected.put(new TopicPartition("blah", 1), "blah 1"); + expected.put(new TopicPartition("baz", 2), "baz 2"); + expected.put(new TopicPartition("baz", 3), "baz 3"); + checkState(builder, expected); + + builder.remove(new TopicPartition("blah", 1)); + expected = new LinkedHashMap<>(); + expected.put(new TopicPartition("foo", 0), "foo 0"); + expected.put(new TopicPartition("blah", 2), "blah 2"); + expected.put(new TopicPartition("baz", 2), "baz 2"); + expected.put(new TopicPartition("baz", 3), "baz 3"); + checkState(builder, expected); + + builder.remove(new TopicPartition("baz", 3)); + expected = new LinkedHashMap<>(); + expected.put(new TopicPartition("foo", 0), "foo 0"); + expected.put(new TopicPartition("blah", 2), "blah 2"); + expected.put(new TopicPartition("baz", 2), "baz 2"); + checkState(builder, expected); + } + +} From 0146fb7311e9532fcefe0081a2a48e1aa4918f52 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 17:39:14 +0100 Subject: [PATCH 29/54] Remove `list` from `FetchBuilder` and change `map` to be a `LinkedHashMap` --- .../kafka/common/internals/FetchBuilder.java | 54 +++++-------------- 1 file changed, 13 insertions(+), 41 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/internals/FetchBuilder.java b/clients/src/main/java/org/apache/kafka/common/internals/FetchBuilder.java index f45d609f81bf..8db4c221beb8 100644 --- a/clients/src/main/java/org/apache/kafka/common/internals/FetchBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/internals/FetchBuilder.java @@ -20,9 +20,6 @@ import org.apache.kafka.common.TopicPartition; import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -39,46 +36,23 @@ */ public class FetchBuilder { - private final List> list = new ArrayList<>(); - private final Map map = new HashMap<>(); + private final LinkedHashMap map = new LinkedHashMap<>(); public FetchBuilder() {} public void moveToEnd(TopicPartition topicPartition) { - for (int i = 0; i < list.size(); ++i) { - PartitionState state = list.get(i); - if (state.topicPartition.equals(topicPartition)) { - list.remove(i); - list.add(state); - break; - } - } + S state = map.remove(topicPartition); + if (state != null) + map.put(topicPartition, state); } public void updateAndMoveToEnd(TopicPartition topicPartition, S state) { - PartitionState partitionState = new PartitionState<>(topicPartition, state); - S prev = map.put(topicPartition, state); - if (prev != null) { - for (int i = 0; i < list.size(); ++i) { - if (list.get(i).topicPartition.equals(topicPartition)) { - list.remove(i); - list.add(partitionState); - break; - } - } - } else - list.add(partitionState); + map.remove(topicPartition); + map.put(topicPartition, state); } public void remove(TopicPartition topicPartition) { map.remove(topicPartition); - for (Iterator> it = list.iterator(); it.hasNext(); ) { - PartitionState state = it.next(); - if (state.topicPartition.equals(topicPartition)) { - it.remove(); - break; - } - } } /** @@ -90,7 +64,6 @@ public Set partitionSet() { public void clear() { map.clear(); - list.clear(); } public boolean contains(TopicPartition topicPartition) { @@ -101,17 +74,18 @@ public boolean contains(TopicPartition topicPartition) { * Returns the partition states in order. */ public List> partitionStates() { - return Collections.unmodifiableList(list); + List> result = new ArrayList<>(); + for (Map.Entry entry : map.entrySet()) { + result.add(new PartitionState(entry.getKey(), entry.getValue())); + } + return result; } /** * Returns the partition state values in order. */ public List partitionStateValues() { - List result = new ArrayList<>(list.size()); - for (PartitionState state : list) - result.add(state.value); - return result; + return new ArrayList<>(map.values()); } public S stateValue(TopicPartition topicPartition) { @@ -130,12 +104,11 @@ public int size() { */ public void set(Map partitionToState) { map.clear(); - list.clear(); update(partitionToState); } private void update(Map partitionToState) { - Map> topicToPartitions = new LinkedHashMap<>(); + LinkedHashMap> topicToPartitions = new LinkedHashMap<>(); for (TopicPartition tp : partitionToState.keySet()) { List partitions = topicToPartitions.get(tp.topic()); if (partitions == null) { @@ -148,7 +121,6 @@ private void update(Map partitionToState) { for (TopicPartition tp : entry.getValue()) { S state = partitionToState.get(tp); map.put(tp, state); - list.add(new PartitionState<>(tp, state)); } } } From e5ccb0e578063eef5b1c2f28d6f7e4d0800f4871 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 17:49:56 +0100 Subject: [PATCH 30/54] Fix import checkstyle error --- .../java/org/apache/kafka/common/requests/FetchResponse.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java index 905cf127ec83..111d1972888d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java @@ -21,7 +21,6 @@ import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.requests.FetchRequest.TopicAndPartitionData; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -110,9 +109,9 @@ public FetchResponse(LinkedHashMap responseData, private FetchResponse(int version, LinkedHashMap responseData, int throttleTime) { super(new Struct(ProtoUtils.responseSchema(ApiKeys.FETCH.id, version))); - List> topicsData = FetchRequest.TopicAndPartitionData.batchByTopic(responseData); + List> topicsData = FetchRequest.TopicAndPartitionData.batchByTopic(responseData); List topicArray = new ArrayList<>(); - for (TopicAndPartitionData topicEntry: topicsData) { + for (FetchRequest.TopicAndPartitionData topicEntry: topicsData) { Struct topicData = struct.instance(RESPONSES_KEY_NAME); topicData.set(TOPIC_KEY_NAME, topicEntry.topic); List partitionArray = new ArrayList<>(); From 7a5bbc803e3703bb1043e4f79115957fc38a4744 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 21:33:03 +0100 Subject: [PATCH 31/54] Clean-ups in `FetchRequest` --- .../org/apache/kafka/common/requests/FetchRequest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java index 913b6e6c6882..dcdfd9c685da 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -92,7 +92,7 @@ public static List> batchByTopic(LinkedHashMap fetchData) { - // Any of 0, 1 or 2 would do here + // Any of 0, 1 or 2 would do here since the schemas for these versions are identical this(2, CONSUMER_REPLICA_ID, maxWait, minBytes, DEFAULT_RESPONSE_MAX_BYTES, new LinkedHashMap<>(fetchData)); } @@ -109,7 +109,7 @@ public FetchRequest(int maxWait, int minBytes, int maxBytes, LinkedHashMap fetchData) { - // Any of 0, 1 or 2 would do here + // Any of 0, 1 or 2 would do here since the schemas for these versions are identical return new FetchRequest(2, replicaId, maxWait, minBytes, DEFAULT_RESPONSE_MAX_BYTES, new LinkedHashMap<>(fetchData)); } @@ -152,7 +152,7 @@ private FetchRequest(int version, int replicaId, int maxWait, int minBytes, int this.maxWait = maxWait; this.minBytes = minBytes; this.maxBytes = maxBytes; - this.fetchData = new LinkedHashMap<>(fetchData); + this.fetchData = fetchData; } public FetchRequest(Struct struct) { @@ -181,7 +181,7 @@ public FetchRequest(Struct struct) { @Override public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { - Map responseData = new LinkedHashMap(); + Map responseData = new LinkedHashMap<>(); for (Map.Entry entry: fetchData.entrySet()) { FetchResponse.PartitionData partitionResponse = new FetchResponse.PartitionData(Errors.forException(e).code(), From 1950fac412067db286556574e664fe60543c4c9e Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 22:06:10 +0100 Subject: [PATCH 32/54] Remove `emptyResponse` boolean in `Fetcher` and add clarifying comment --- .../apache/kafka/clients/consumer/internals/Fetcher.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 7c385f83389e..ed202d35a83d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -575,7 +575,6 @@ private PartitionRecords parseFetchedData(CompletedFetch completedFetch) { int bytes = 0; int recordsCount = 0; PartitionRecords parsedRecords = null; - boolean emptyResponse = false; try { if (!subscriptions.isFetchable(tp)) { @@ -606,9 +605,6 @@ private PartitionRecords parseFetchedData(CompletedFetch completedFetch) { } } - if (!skippedRecords && bytes == 0) - emptyResponse = true; - recordsCount = parsed.size(); this.sensors.recordTopicFetchMetrics(tp.topic(), bytes, recordsCount); @@ -656,7 +652,9 @@ private PartitionRecords parseFetchedData(CompletedFetch completedFetch) { completedFetch.metricAggregator.record(tp, bytes, recordsCount); } - if (!emptyResponse) + // we move the partition to the end if we received some bytes or if there was an error. This way, it's more + // likely that partitions for the same topic can remain together (allowing for more efficient serialization). + if (bytes > 0 || partition.errorCode != Errors.NONE.code()) subscriptions.movePartitionToEnd(tp); return parsedRecords; From 6447db1af1bd6785531737ef4c35d04f20345a7e Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 22:32:45 +0100 Subject: [PATCH 33/54] Add `testReadWithMinMessage` to `LogTest` --- .../test/scala/unit/kafka/log/LogTest.scala | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 4935aae9c8fc..d99fa14fe888 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -246,6 +246,36 @@ class LogTest extends JUnitSuite { assertEquals("A read should now return the last message in the log", log.logEndOffset - 1, log.read(1, 200, None).messageSet.head.offset) } + @Test + def testReadWithMinMessage() { + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) + val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray + val messages = messageIds.map(id => new Message(id.toString.getBytes)) + + // now test the case that we give the offsets and use non-sequential offsets + for (i <- 0 until messages.length) + log.append(new ByteBufferMessageSet(NoCompressionCodec, new LongRef(messageIds(i)), messages = messages(i)), + assignOffsets = false) + + for (i <- 50 until messageIds.max) { + val idx = messageIds.indexWhere(_ >= i) + val reads = Seq( + log.read(i, 1, minOneMessage = true), + log.read(i, 100, minOneMessage = true), + log.read(i, 100, Some(10000), minOneMessage = true) + ).map(_.messageSet.head) + reads.foreach { read => + assertEquals("Offset read should match message id.", messageIds(idx), read.offset) + assertEquals("Message should match appended.", messages(idx), read.message) + } + + assertEquals(Seq.empty, log.read(i, 1, Some(1), minOneMessage = true).messageSet.toIndexedSeq) + } + + } + /** * Test reading at the boundary of the log, specifically * - reading from the logEndOffset should give an empty message set From 9f6bbdbf1bc0e4fe78a733276f3b7d5c749f98b0 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 22:43:14 +0100 Subject: [PATCH 34/54] Minor clean-ups in `searchOffsetWithSize` and `LogSegment.read` As per Jason's suggestions. --- core/src/main/scala/kafka/log/FileMessageSet.scala | 4 +++- core/src/main/scala/kafka/log/LogSegment.scala | 10 +++++----- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/kafka/log/FileMessageSet.scala b/core/src/main/scala/kafka/log/FileMessageSet.scala index b477b273973c..c76653af21d5 100755 --- a/core/src/main/scala/kafka/log/FileMessageSet.scala +++ b/core/src/main/scala/kafka/log/FileMessageSet.scala @@ -128,7 +128,9 @@ class FileMessageSet private[kafka](@volatile var file: File, /** * Search forward for the file position of the last offset that is greater than or equal to the target offset - * and return its physical position and the size of the message. If no such offsets are found, return null. + * and return its physical position and the size of the message (including log overhead) at the returned offset. If + * no such offsets are found, return null. + * * @param targetOffset The offset to search for. * @param startingPosition The starting position in the file to begin searching from. */ diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index 62acb3c612a3..0a5452577a05 100755 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -154,22 +154,22 @@ class LogSegment(val log: FileMessageSet, if (startOffsetAndSize == null) return null - val (startPosition, messageSize) = startOffsetAndSize + val (startPosition, messageSetSize) = startOffsetAndSize val offsetMetadata = new LogOffsetMetadata(startOffset, this.baseOffset, startPosition.position) // if the size is zero, still return a log segment but with zero size if (maxSize == 0 && !minOneMessage) return FetchDataInfo(offsetMetadata, MessageSet.Empty) - val maxLength = - if (minOneMessage) math.max(maxSize, messageSize) + val adjustedMaxSize = + if (minOneMessage) math.max(maxSize, messageSetSize) else maxSize // calculate the length of the message set to read based on whether or not they gave us a maxOffset val length = maxOffset match { case None => // no max offset, just read until the max position - min((maxPosition - startPosition.position).toInt, maxLength) + min((maxPosition - startPosition.position).toInt, adjustedMaxSize) case Some(offset) => // there is a max offset, translate it to a file position and use that to calculate the max read size; // when the leader of a partition changes, it's possible for the new leader's high watermark to be less than the @@ -183,7 +183,7 @@ class LogSegment(val log: FileMessageSet, logSize // the max offset is off the end of the log, use the end of the file else mapping._1.position - min(min(maxPosition, endPosition) - startPosition.position, maxLength).toInt + min(min(maxPosition, endPosition) - startPosition.position, adjustedMaxSize).toInt } FetchDataInfo(offsetMetadata, log.read(startPosition.position, length)) From 28c6cb4a4839e2f1950d2ca0dd52a180a469ad18 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Sep 2016 23:02:17 +0100 Subject: [PATCH 35/54] Replace usages of `TopicAndPartition` with `TopicPartition` in `*FetcherManager` classes --- .../consumer/ConsumerFetcherManager.scala | 29 +++++++------------ .../consumer/ConsumerFetcherThread.scala | 2 +- .../kafka/server/AbstractFetcherManager.scala | 15 ++++------ .../scala/kafka/server/ReplicaManager.scala | 10 +++---- 4 files changed, 23 insertions(+), 33 deletions(-) diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala index f217882cfd7b..5b5fe0d09d7e 100755 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala @@ -17,10 +17,10 @@ package kafka.consumer -import org.I0Itec.zkclient.ZkClient import kafka.server.{BrokerAndInitialOffset, AbstractFetcherThread, AbstractFetcherManager} import kafka.cluster.{BrokerEndPoint, Cluster} import org.apache.kafka.common.protocol.SecurityProtocol +import org.apache.kafka.common.TopicPartition import scala.collection.immutable import collection.mutable.HashMap import scala.collection.mutable @@ -28,12 +28,9 @@ import java.util.concurrent.locks.ReentrantLock import kafka.utils.CoreUtils.inLock import kafka.utils.ZkUtils import kafka.utils.{ShutdownableThread, SystemTime} -import kafka.common.TopicAndPartition import kafka.client.ClientUtils import java.util.concurrent.atomic.AtomicInteger -import org.apache.kafka.common.TopicPartition - /** * Usage: * Once ConsumerFetcherManager is created, startConnections() and stopAllConnections() can be called repeatedly @@ -44,9 +41,9 @@ class ConsumerFetcherManager(private val consumerIdString: String, private val zkUtils : ZkUtils) extends AbstractFetcherManager("ConsumerFetcherManager-%d".format(SystemTime.milliseconds), config.clientId, config.numConsumerFetchers) { - private var partitionMap: immutable.Map[TopicAndPartition, PartitionTopicInfo] = null + private var partitionMap: immutable.Map[TopicPartition, PartitionTopicInfo] = null private var cluster: Cluster = null - private val noLeaderPartitionSet = new mutable.HashSet[TopicAndPartition] + private val noLeaderPartitionSet = new mutable.HashSet[TopicPartition] private val lock = new ReentrantLock private val cond = lock.newCondition() private var leaderFinderThread: ShutdownableThread = null @@ -55,7 +52,7 @@ class ConsumerFetcherManager(private val consumerIdString: String, private class LeaderFinderThread(name: String) extends ShutdownableThread(name) { // thread responsible for adding the fetcher to the right broker when leader is available override def doWork() { - val leaderForPartitionsMap = new HashMap[TopicAndPartition, BrokerEndPoint] + val leaderForPartitionsMap = new HashMap[TopicPartition, BrokerEndPoint] lock.lock() try { while (noLeaderPartitionSet.isEmpty) { @@ -74,7 +71,7 @@ class ConsumerFetcherManager(private val consumerIdString: String, topicsMetadata.foreach { tmd => val topic = tmd.topic tmd.partitionsMetadata.foreach { pmd => - val topicAndPartition = TopicAndPartition(topic, pmd.partitionId) + val topicAndPartition = new TopicPartition(topic, pmd.partitionId) if(pmd.leader.isDefined && noLeaderPartitionSet.contains(topicAndPartition)) { val leaderBroker = pmd.leader.get leaderForPartitionsMap.put(topicAndPartition, leaderBroker) @@ -94,9 +91,8 @@ class ConsumerFetcherManager(private val consumerIdString: String, } try { - addFetcherForPartitions(leaderForPartitionsMap.map{ - case (topicAndPartition, broker) => - topicAndPartition -> BrokerAndInitialOffset(broker, partitionMap(topicAndPartition).getFetchOffset())} + addFetcherForPartitions(leaderForPartitionsMap.map { case (topicPartition, broker) => + topicPartition -> BrokerAndInitialOffset(broker, partitionMap(topicPartition).getFetchOffset())} ) } catch { case t: Throwable => { @@ -117,12 +113,9 @@ class ConsumerFetcherManager(private val consumerIdString: String, } override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): AbstractFetcherThread = { - val topicPartitionMap = partitionMap.map { case (TopicAndPartition(topic, partition), topicInfo) => - new TopicPartition(topic, partition) -> topicInfo - } new ConsumerFetcherThread( "ConsumerFetcherThread-%s-%d-%d".format(consumerIdString, fetcherId, sourceBroker.id), - config, sourceBroker, topicPartitionMap, this) + config, sourceBroker, partitionMap, this) } def startConnections(topicInfos: Iterable[PartitionTopicInfo], cluster: Cluster) { @@ -130,9 +123,9 @@ class ConsumerFetcherManager(private val consumerIdString: String, leaderFinderThread.start() inLock(lock) { - partitionMap = topicInfos.map(tpi => (TopicAndPartition(tpi.topic, tpi.partitionId), tpi)).toMap + partitionMap = topicInfos.map(tpi => (new TopicPartition(tpi.topic, tpi.partitionId), tpi)).toMap this.cluster = cluster - noLeaderPartitionSet ++= topicInfos.map(tpi => TopicAndPartition(tpi.topic, tpi.partitionId)) + noLeaderPartitionSet ++= topicInfos.map(tpi => new TopicPartition(tpi.topic, tpi.partitionId)) cond.signalAll() } } @@ -159,7 +152,7 @@ class ConsumerFetcherManager(private val consumerIdString: String, info("All connections stopped") } - def addPartitionsWithError(partitionList: Iterable[TopicAndPartition]) { + def addPartitionsWithError(partitionList: Iterable[TopicPartition]) { debug("adding partitions with error %s".format(partitionList)) inLock(lock) { if (partitionMap != null) { diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala index 41ab64916d2d..f657b81b2fb1 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala @@ -93,7 +93,7 @@ class ConsumerFetcherThread(name: String, // any logic for partitions whose leader has changed def handlePartitionsWithErrors(partitions: Iterable[TopicPartition]) { removePartitions(partitions.toSet) - consumerFetcherManager.addPartitionsWithError(partitions.map(tp => new TopicAndPartition(tp.topic, tp.partition))) + consumerFetcherManager.addPartitionsWithError(partitions) } protected def buildFetchRequest(partitionMap: collection.Seq[(TopicPartition, PartitionFetchState)]): FetchRequest = { diff --git a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala index b7beb208de7b..8f6b84fb1144 100755 --- a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala @@ -23,7 +23,6 @@ import scala.collection.Map import kafka.utils.Logging import kafka.cluster.BrokerEndPoint import kafka.metrics.KafkaMetricsGroup -import kafka.common.TopicAndPartition import com.yammer.metrics.core.Gauge import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.utils.Utils @@ -72,7 +71,7 @@ abstract class AbstractFetcherManager(protected val name: String, clientId: Stri // to be defined in subclass to create a specific fetcher def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): AbstractFetcherThread - def addFetcherForPartitions(partitionAndOffsets: Map[TopicAndPartition, BrokerAndInitialOffset]) { + def addFetcherForPartitions(partitionAndOffsets: Map[TopicPartition, BrokerAndInitialOffset]) { mapLock synchronized { val partitionsPerFetcher = partitionAndOffsets.groupBy{ case(topicAndPartition, brokerAndInitialOffset) => BrokerAndFetcherId(brokerAndInitialOffset.broker, getFetcherId(topicAndPartition.topic, topicAndPartition.partition))} @@ -86,9 +85,8 @@ abstract class AbstractFetcherManager(protected val name: String, clientId: Stri fetcherThread.start } - fetcherThreadMap(brokerAndFetcherId).addPartitions(partitionAndOffsets.map { - case (TopicAndPartition(t, p), brokerAndInitOffset) => - new TopicPartition(t, p) -> brokerAndInitOffset.initOffset + fetcherThreadMap(brokerAndFetcherId).addPartitions(partitionAndOffsets.map { case (tp, brokerAndInitOffset) => + tp -> brokerAndInitOffset.initOffset }) } } @@ -97,11 +95,10 @@ abstract class AbstractFetcherManager(protected val name: String, clientId: Stri "[" + topicAndPartition + ", initOffset " + brokerAndInitialOffset.initOffset + " to broker " + brokerAndInitialOffset.broker + "] "})) } - def removeFetcherForPartitions(partitions: Set[TopicAndPartition]) { + def removeFetcherForPartitions(partitions: Set[TopicPartition]) { mapLock synchronized { - for ((key, fetcher) <- fetcherThreadMap) { - fetcher.removePartitions(partitions.map { case TopicAndPartition(t, p) => new TopicPartition(t, p) }) - } + for ((key, fetcher) <- fetcherThreadMap) + fetcher.removePartitions(partitions) } info("Removed fetcher for partitions %s".format(partitions.mkString(","))) } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index d15323f2e252..7bfa30bd57b9 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -263,8 +263,8 @@ class ReplicaManager(val config: KafkaConfig, val partitions = stopReplicaRequest.partitions.asScala controllerEpoch = stopReplicaRequest.controllerEpoch // First stop fetchers for all partitions, then stop the corresponding replicas - replicaFetcherManager.removeFetcherForPartitions(partitions.map(r => TopicAndPartition(r.topic, r.partition))) - for(topicPartition <- partitions){ + replicaFetcherManager.removeFetcherForPartitions(partitions) + for (topicPartition <- partitions){ val errorCode = stopReplica(topicPartition.topic, topicPartition.partition, stopReplicaRequest.deletePartitions) responseMap.put(topicPartition, errorCode) } @@ -715,7 +715,7 @@ class ReplicaManager(val config: KafkaConfig, try { // First stop fetchers for all the partitions - replicaFetcherManager.removeFetcherForPartitions(partitionState.keySet.map(new TopicAndPartition(_))) + replicaFetcherManager.removeFetcherForPartitions(partitionState.keySet.map(p => new TopicPartition(p.topic, p.partitionId))) // Update the partition information to be the leader partitionState.foreach{ case (partition, partitionStateInfo) => if (partition.makeLeader(controllerId, partitionStateInfo, correlationId)) @@ -814,7 +814,7 @@ class ReplicaManager(val config: KafkaConfig, } } - replicaFetcherManager.removeFetcherForPartitions(partitionsToMakeFollower.map(new TopicAndPartition(_))) + replicaFetcherManager.removeFetcherForPartitions(partitionsToMakeFollower.map(p => new TopicPartition(p.topic, p.partitionId))) partitionsToMakeFollower.foreach { partition => stateChangeLogger.trace(("Broker %d stopped fetchers as part of become-follower request from controller " + "%d epoch %d with correlation id %d for partition %s") @@ -844,7 +844,7 @@ class ReplicaManager(val config: KafkaConfig, else { // we do not need to check if the leader exists again since this has been done at the beginning of this process val partitionsToMakeFollowerWithLeaderAndOffset = partitionsToMakeFollower.map(partition => - new TopicAndPartition(partition) -> BrokerAndInitialOffset( + new TopicPartition(partition.topic, partition.partitionId) -> BrokerAndInitialOffset( metadataCache.getAliveBrokers.find(_.id == partition.leaderReplicaIdOpt.get).get.getBrokerEndPoint(config.interBrokerSecurityProtocol), partition.getReplica().get.logEndOffset.messageOffset)).toMap replicaFetcherManager.addFetcherForPartitions(partitionsToMakeFollowerWithLeaderAndOffset) From 55481b42aa78885fba2fe04372e40e94afd6dc4e Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 16 Sep 2016 00:25:25 +0100 Subject: [PATCH 36/54] Rename `FetchBuilder` to `PartitionStates` --- .../consumer/internals/SubscriptionState.java | 14 +-- ...FetchBuilder.java => PartitionStates.java} | 8 +- ...lderTest.java => PartitionStatesTest.java} | 100 +++++++++--------- .../kafka/server/AbstractFetcherThread.scala | 30 +++--- 4 files changed, 76 insertions(+), 76 deletions(-) rename clients/src/main/java/org/apache/kafka/common/internals/{FetchBuilder.java => PartitionStates.java} (95%) rename clients/src/test/java/org/apache/kafka/common/internals/{FetchBuilderTest.java => PartitionStatesTest.java} (72%) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java index 0cd3f4535779..9029417c2927 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java @@ -16,7 +16,7 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.internals.FetchBuilder; +import org.apache.kafka.common.internals.PartitionStates; import java.util.ArrayList; import java.util.Collection; @@ -72,7 +72,7 @@ private enum SubscriptionType { private final Set groupSubscription; /* the partitions that are currently assigned, note that the order of partition matters (see FetchBuilder for more details) */ - private final FetchBuilder assignment; + private final PartitionStates assignment; /* do we need to request the latest committed offsets from the coordinator? */ private boolean needsFetchCommittedOffsets; @@ -87,7 +87,7 @@ public SubscriptionState(OffsetResetStrategy defaultResetStrategy) { this.defaultResetStrategy = defaultResetStrategy; this.subscription = Collections.emptySet(); this.userAssignment = Collections.emptySet(); - this.assignment = new FetchBuilder<>(); + this.assignment = new PartitionStates<>(); this.groupSubscription = new HashSet<>(); this.needsFetchCommittedOffsets = true; // initialize to true for the consumers to fetch offset upon starting up this.subscribedPattern = null; @@ -230,7 +230,7 @@ public Set subscription() { public Set pausedPartitions() { HashSet paused = new HashSet<>(); - for (FetchBuilder.PartitionState state : assignment.partitionStates()) { + for (PartitionStates.PartitionState state : assignment.partitionStates()) { if (state.value().paused) { paused.add(state.topicPartition()); } @@ -289,7 +289,7 @@ public Set assignedPartitions() { public List fetchablePartitions() { List fetchable = new ArrayList<>(); - for (FetchBuilder.PartitionState state : assignment.partitionStates()) { + for (PartitionStates.PartitionState state : assignment.partitionStates()) { if (state.value().isFetchable()) fetchable.add(state.topicPartition()); } @@ -310,7 +310,7 @@ public Long position(TopicPartition tp) { public Map allConsumed() { Map allConsumed = new HashMap<>(); - for (FetchBuilder.PartitionState state : assignment.partitionStates()) { + for (PartitionStates.PartitionState state : assignment.partitionStates()) { if (state.value().hasValidPosition()) allConsumed.put(state.topicPartition(), new OffsetAndMetadata(state.value().position)); } @@ -346,7 +346,7 @@ public boolean hasAllFetchPositions() { public Set missingFetchPositions() { Set missing = new HashSet<>(); - for (FetchBuilder.PartitionState state : assignment.partitionStates()) { + for (PartitionStates.PartitionState state : assignment.partitionStates()) { if (!state.value().hasValidPosition()) missing.add(state.topicPartition()); } diff --git a/clients/src/main/java/org/apache/kafka/common/internals/FetchBuilder.java b/clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java similarity index 95% rename from clients/src/main/java/org/apache/kafka/common/internals/FetchBuilder.java rename to clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java index 8db4c221beb8..7dd292be993d 100644 --- a/clients/src/main/java/org/apache/kafka/common/internals/FetchBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java @@ -27,18 +27,18 @@ import java.util.Set; /** - * This builder is a useful building block for doing fetch requests where topic partitions have to be rotated via + * This class is a useful building block for doing fetch requests where topic partitions have to be rotated via * round-robin to ensure fairness and some level of determinism given the existence of a limit on the fetch response * size. Because the serialization of fetch requests is more efficient if all partitions for the same topic are grouped * together, we do such grouping in the method `set`. * * As partitions are moved to the end, the same topic may be repeated more than once. */ -public class FetchBuilder { +public class PartitionStates { private final LinkedHashMap map = new LinkedHashMap<>(); - public FetchBuilder() {} + public PartitionStates() {} public void moveToEnd(TopicPartition topicPartition) { S state = map.remove(topicPartition); @@ -76,7 +76,7 @@ public boolean contains(TopicPartition topicPartition) { public List> partitionStates() { List> result = new ArrayList<>(); for (Map.Entry entry : map.entrySet()) { - result.add(new PartitionState(entry.getKey(), entry.getValue())); + result.add(new PartitionState<>(entry.getKey(), entry.getValue())); } return result; } diff --git a/clients/src/test/java/org/apache/kafka/common/internals/FetchBuilderTest.java b/clients/src/test/java/org/apache/kafka/common/internals/PartitionStatesTest.java similarity index 72% rename from clients/src/test/java/org/apache/kafka/common/internals/FetchBuilderTest.java rename to clients/src/test/java/org/apache/kafka/common/internals/PartitionStatesTest.java index a109f0897395..66c7abc80247 100644 --- a/clients/src/test/java/org/apache/kafka/common/internals/FetchBuilderTest.java +++ b/clients/src/test/java/org/apache/kafka/common/internals/PartitionStatesTest.java @@ -28,13 +28,13 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -public class FetchBuilderTest { +public class PartitionStatesTest { @Test public void testSet() { - FetchBuilder builder = new FetchBuilder<>(); + PartitionStates states = new PartitionStates<>(); LinkedHashMap map = createMap(); - builder.set(map); + states.set(map); LinkedHashMap expected = new LinkedHashMap<>(); expected.put(new TopicPartition("foo", 2), "foo 2"); expected.put(new TopicPartition("foo", 0), "foo 0"); @@ -42,10 +42,10 @@ public void testSet() { expected.put(new TopicPartition("blah", 1), "blah 1"); expected.put(new TopicPartition("baz", 2), "baz 2"); expected.put(new TopicPartition("baz", 3), "baz 3"); - checkState(builder, expected); + checkState(states, expected); - builder.set(new LinkedHashMap()); - checkState(builder, new LinkedHashMap()); + states.set(new LinkedHashMap()); + checkState(states, new LinkedHashMap()); } private LinkedHashMap createMap() { @@ -59,24 +59,24 @@ private LinkedHashMap createMap() { return map; } - private void checkState(FetchBuilder builder, LinkedHashMap expected) { - assertEquals(expected.keySet(), builder.partitionSet()); - assertEquals(expected.size(), builder.size()); - List> states = new ArrayList<>(); + private void checkState(PartitionStates states, LinkedHashMap expected) { + assertEquals(expected.keySet(), states.partitionSet()); + assertEquals(expected.size(), states.size()); + List> statesList = new ArrayList<>(); for (Map.Entry entry : expected.entrySet()) { - states.add(new FetchBuilder.PartitionState<>(entry.getKey(), entry.getValue())); - assertTrue(builder.contains(entry.getKey())); + statesList.add(new PartitionStates.PartitionState<>(entry.getKey(), entry.getValue())); + assertTrue(states.contains(entry.getKey())); } - assertEquals(states, builder.partitionStates()); + assertEquals(statesList, states.partitionStates()); } @Test public void testMoveToEnd() { - FetchBuilder builder = new FetchBuilder<>(); + PartitionStates states = new PartitionStates<>(); LinkedHashMap map = createMap(); - builder.set(map); + states.set(map); - builder.moveToEnd(new TopicPartition("baz", 2)); + states.moveToEnd(new TopicPartition("baz", 2)); LinkedHashMap expected = new LinkedHashMap<>(); expected.put(new TopicPartition("foo", 2), "foo 2"); expected.put(new TopicPartition("foo", 0), "foo 0"); @@ -84,9 +84,9 @@ public void testMoveToEnd() { expected.put(new TopicPartition("blah", 1), "blah 1"); expected.put(new TopicPartition("baz", 3), "baz 3"); expected.put(new TopicPartition("baz", 2), "baz 2"); - checkState(builder, expected); + checkState(states, expected); - builder.moveToEnd(new TopicPartition("foo", 2)); + states.moveToEnd(new TopicPartition("foo", 2)); expected = new LinkedHashMap<>(); expected.put(new TopicPartition("foo", 0), "foo 0"); expected.put(new TopicPartition("blah", 2), "blah 2"); @@ -94,28 +94,28 @@ public void testMoveToEnd() { expected.put(new TopicPartition("baz", 3), "baz 3"); expected.put(new TopicPartition("baz", 2), "baz 2"); expected.put(new TopicPartition("foo", 2), "foo 2"); - checkState(builder, expected); + checkState(states, expected); // no-op - builder.moveToEnd(new TopicPartition("foo", 2)); - checkState(builder, expected); + states.moveToEnd(new TopicPartition("foo", 2)); + checkState(states, expected); // partition doesn't exist - builder.moveToEnd(new TopicPartition("baz", 5)); - checkState(builder, expected); + states.moveToEnd(new TopicPartition("baz", 5)); + checkState(states, expected); // topic doesn't exist - builder.moveToEnd(new TopicPartition("aaa", 2)); - checkState(builder, expected); + states.moveToEnd(new TopicPartition("aaa", 2)); + checkState(states, expected); } @Test public void testUpdateAndMoveToEnd() { - FetchBuilder builder = new FetchBuilder<>(); + PartitionStates states = new PartitionStates<>(); LinkedHashMap map = createMap(); - builder.set(map); + states.set(map); - builder.updateAndMoveToEnd(new TopicPartition("foo", 0), "foo 0 updated"); + states.updateAndMoveToEnd(new TopicPartition("foo", 0), "foo 0 updated"); LinkedHashMap expected = new LinkedHashMap<>(); expected.put(new TopicPartition("foo", 2), "foo 2"); expected.put(new TopicPartition("blah", 2), "blah 2"); @@ -123,9 +123,9 @@ public void testUpdateAndMoveToEnd() { expected.put(new TopicPartition("baz", 2), "baz 2"); expected.put(new TopicPartition("baz", 3), "baz 3"); expected.put(new TopicPartition("foo", 0), "foo 0 updated"); - checkState(builder, expected); + checkState(states, expected); - builder.updateAndMoveToEnd(new TopicPartition("baz", 2), "baz 2 updated"); + states.updateAndMoveToEnd(new TopicPartition("baz", 2), "baz 2 updated"); expected = new LinkedHashMap<>(); expected.put(new TopicPartition("foo", 2), "foo 2"); expected.put(new TopicPartition("blah", 2), "blah 2"); @@ -133,10 +133,10 @@ public void testUpdateAndMoveToEnd() { expected.put(new TopicPartition("baz", 3), "baz 3"); expected.put(new TopicPartition("foo", 0), "foo 0 updated"); expected.put(new TopicPartition("baz", 2), "baz 2 updated"); - checkState(builder, expected); + checkState(states, expected); // partition doesn't exist - builder.updateAndMoveToEnd(new TopicPartition("baz", 5), "baz 5 new"); + states.updateAndMoveToEnd(new TopicPartition("baz", 5), "baz 5 new"); expected = new LinkedHashMap<>(); expected.put(new TopicPartition("foo", 2), "foo 2"); expected.put(new TopicPartition("blah", 2), "blah 2"); @@ -145,10 +145,10 @@ public void testUpdateAndMoveToEnd() { expected.put(new TopicPartition("foo", 0), "foo 0 updated"); expected.put(new TopicPartition("baz", 2), "baz 2 updated"); expected.put(new TopicPartition("baz", 5), "baz 5 new"); - checkState(builder, expected); + checkState(states, expected); // topic doesn't exist - builder.updateAndMoveToEnd(new TopicPartition("aaa", 2), "aaa 2 new"); + states.updateAndMoveToEnd(new TopicPartition("aaa", 2), "aaa 2 new"); expected = new LinkedHashMap<>(); expected.put(new TopicPartition("foo", 2), "foo 2"); expected.put(new TopicPartition("blah", 2), "blah 2"); @@ -158,14 +158,14 @@ public void testUpdateAndMoveToEnd() { expected.put(new TopicPartition("baz", 2), "baz 2 updated"); expected.put(new TopicPartition("baz", 5), "baz 5 new"); expected.put(new TopicPartition("aaa", 2), "aaa 2 new"); - checkState(builder, expected); + checkState(states, expected); } @Test public void testPartitionValues() { - FetchBuilder builder = new FetchBuilder<>(); + PartitionStates states = new PartitionStates<>(); LinkedHashMap map = createMap(); - builder.set(map); + states.set(map); List expected = new ArrayList<>(); expected.add("foo 2"); expected.add("foo 0"); @@ -173,47 +173,47 @@ public void testPartitionValues() { expected.add("blah 1"); expected.add("baz 2"); expected.add("baz 3"); - assertEquals(expected, builder.partitionStateValues()); + assertEquals(expected, states.partitionStateValues()); } @Test public void testClear() { - FetchBuilder builder = new FetchBuilder<>(); + PartitionStates states = new PartitionStates<>(); LinkedHashMap map = createMap(); - builder.set(map); - builder.clear(); - checkState(builder, new LinkedHashMap()); + states.set(map); + states.clear(); + checkState(states, new LinkedHashMap()); } @Test public void testRemove() { - FetchBuilder builder = new FetchBuilder<>(); + PartitionStates states = new PartitionStates<>(); LinkedHashMap map = createMap(); - builder.set(map); + states.set(map); - builder.remove(new TopicPartition("foo", 2)); + states.remove(new TopicPartition("foo", 2)); LinkedHashMap expected = new LinkedHashMap<>(); expected.put(new TopicPartition("foo", 0), "foo 0"); expected.put(new TopicPartition("blah", 2), "blah 2"); expected.put(new TopicPartition("blah", 1), "blah 1"); expected.put(new TopicPartition("baz", 2), "baz 2"); expected.put(new TopicPartition("baz", 3), "baz 3"); - checkState(builder, expected); + checkState(states, expected); - builder.remove(new TopicPartition("blah", 1)); + states.remove(new TopicPartition("blah", 1)); expected = new LinkedHashMap<>(); expected.put(new TopicPartition("foo", 0), "foo 0"); expected.put(new TopicPartition("blah", 2), "blah 2"); expected.put(new TopicPartition("baz", 2), "baz 2"); expected.put(new TopicPartition("baz", 3), "baz 3"); - checkState(builder, expected); + checkState(states, expected); - builder.remove(new TopicPartition("baz", 3)); + states.remove(new TopicPartition("baz", 3)); expected = new LinkedHashMap<>(); expected.put(new TopicPartition("foo", 0), "foo 0"); expected.put(new TopicPartition("blah", 2), "blah 2"); expected.put(new TopicPartition("baz", 2), "baz 2"); - checkState(builder, expected); + checkState(states, expected); } } diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index edb2736782b9..99dde1b2dcf6 100755 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -37,7 +37,7 @@ import java.util.concurrent.atomic.AtomicLong import com.yammer.metrics.core.Gauge import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.internals.FetchBuilder +import org.apache.kafka.common.internals.PartitionStates /** * Abstract class for fetching data from multiple partitions from the same broker. @@ -52,7 +52,7 @@ abstract class AbstractFetcherThread(name: String, type REQ <: FetchRequest type PD <: PartitionData - private val fetchBuilder = new FetchBuilder[PartitionFetchState] + private val partitionStates = new PartitionStates[PartitionFetchState] private val partitionMapLock = new ReentrantLock private val partitionMapCond = partitionMapLock.newCondition() @@ -90,7 +90,7 @@ abstract class AbstractFetcherThread(name: String, override def doWork() { val fetchRequest = inLock(partitionMapLock) { - val fetchRequest = buildFetchRequest(fetchBuilder.partitionStates.asScala.map { state => + val fetchRequest = buildFetchRequest(partitionStates.partitionStates.asScala.map { state => state.topicPartition -> state.value }) if (fetchRequest.isEmpty) { @@ -109,7 +109,7 @@ abstract class AbstractFetcherThread(name: String, def updatePartitionsWithError(partition: TopicPartition): Unit = { partitionsWithError += partition - fetchBuilder.moveToEnd(partition) + partitionStates.moveToEnd(partition) } var responseData: Seq[(TopicPartition, PD)] = Seq.empty @@ -122,7 +122,7 @@ abstract class AbstractFetcherThread(name: String, if (isRunning.get) { warn(s"Error in fetch $fetchRequest", t) inLock(partitionMapLock) { - fetchBuilder.partitionSet.asScala.foreach(updatePartitionsWithError) + partitionStates.partitionSet.asScala.foreach(updatePartitionsWithError) // there is an error occurred while fetching partitions, sleep a while // note that `ReplicaFetcherThread.handlePartitionsWithError` will also introduce the same delay for every // partition with error effectively doubling the delay. It would be good to improve this. @@ -139,7 +139,7 @@ abstract class AbstractFetcherThread(name: String, responseData.foreach { case (topicPartition, partitionData) => val topic = topicPartition.topic val partitionId = topicPartition.partition - Option(fetchBuilder.stateValue(topicPartition)).foreach(currentPartitionFetchState => + Option(partitionStates.stateValue(topicPartition)).foreach(currentPartitionFetchState => // we append to the log if the current offset is defined and it is the same as the offset requested during fetch if (fetchRequest.offset(topicPartition) == currentPartitionFetchState.offset) { Errors.forCode(partitionData.errorCode) match { @@ -148,7 +148,7 @@ abstract class AbstractFetcherThread(name: String, val messages = partitionData.toByteBufferMessageSet val newOffset = messages.shallowIterator.toSeq.lastOption match { case Some(m) => - fetchBuilder.updateAndMoveToEnd(topicPartition, new PartitionFetchState(m.nextOffset)) + partitionStates.updateAndMoveToEnd(topicPartition, new PartitionFetchState(m.nextOffset)) fetcherStats.byteRate.mark(messages.validBytes) m.nextOffset case None => @@ -172,7 +172,7 @@ abstract class AbstractFetcherThread(name: String, case Errors.OFFSET_OUT_OF_RANGE => try { val newOffset = handleOffsetOutOfRange(topicPartition) - fetchBuilder.updateAndMoveToEnd(topicPartition, new PartitionFetchState(newOffset)) + partitionStates.updateAndMoveToEnd(topicPartition, new PartitionFetchState(newOffset)) error("Current offset %d for partition [%s,%d] out of range; reset offset to %d" .format(currentPartitionFetchState.offset, topic, partitionId, newOffset)) } catch { @@ -203,17 +203,17 @@ abstract class AbstractFetcherThread(name: String, try { // If the partitionMap already has the topic/partition, then do not update the map with the old offset val newPartitionToState = partitionAndOffsets.filter { case (tp, _) => - !fetchBuilder.contains(tp) + !partitionStates.contains(tp) }.map { case (tp, offset) => val fetchState = if (PartitionTopicInfo.isOffsetInvalid(offset)) new PartitionFetchState(handleOffsetOutOfRange(tp)) else new PartitionFetchState(offset) tp -> fetchState } - val existingPartitionToState = fetchBuilder.partitionStates.asScala.map { state => + val existingPartitionToState = partitionStates.partitionStates.asScala.map { state => state.topicPartition -> state.value }.toMap - fetchBuilder.set((existingPartitionToState ++ newPartitionToState).asJava) + partitionStates.set((existingPartitionToState ++ newPartitionToState).asJava) partitionMapCond.signalAll() } finally partitionMapLock.unlock() } @@ -222,9 +222,9 @@ abstract class AbstractFetcherThread(name: String, partitionMapLock.lockInterruptibly() try { for (partition <- partitions) { - Option(fetchBuilder.stateValue(partition)).foreach (currentPartitionFetchState => + Option(partitionStates.stateValue(partition)).foreach (currentPartitionFetchState => if (currentPartitionFetchState.isActive) - fetchBuilder.updateAndMoveToEnd(partition, new PartitionFetchState(currentPartitionFetchState.offset, new DelayedItem(delay))) + partitionStates.updateAndMoveToEnd(partition, new PartitionFetchState(currentPartitionFetchState.offset, new DelayedItem(delay))) ) } partitionMapCond.signalAll() @@ -235,7 +235,7 @@ abstract class AbstractFetcherThread(name: String, partitionMapLock.lockInterruptibly() try { topicPartitions.foreach { topicPartition => - fetchBuilder.remove(topicPartition) + partitionStates.remove(topicPartition) fetcherLagStats.unregister(topicPartition.topic, topicPartition.partition) } } finally partitionMapLock.unlock() @@ -243,7 +243,7 @@ abstract class AbstractFetcherThread(name: String, def partitionCount() = { partitionMapLock.lockInterruptibly() - try fetchBuilder.size + try partitionStates.size finally partitionMapLock.unlock() } From f21b814f69eab4cfe1bd4397ef0aef28c557fbe8 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 16 Sep 2016 07:36:31 +0100 Subject: [PATCH 37/54] Fixed ConcurrentModificationException by returning a copy in `partitionSet` --- .../org/apache/kafka/common/internals/PartitionStates.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java b/clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java index 7dd292be993d..816017653c38 100644 --- a/clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java +++ b/clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.TopicPartition; import java.util.ArrayList; +import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -59,7 +60,7 @@ public void remove(TopicPartition topicPartition) { * Returns the partitions in random order. */ public Set partitionSet() { - return map.keySet(); + return new HashSet<>(map.keySet()); } public void clear() { From fc670474fea42aa505a1ec830bfb3e1d5617bb23 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 16 Sep 2016 11:25:41 +0100 Subject: [PATCH 38/54] Improve logging in `ReplicaManager.readFromLocalLog` --- core/src/main/scala/kafka/server/ReplicaManager.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index efc015e243b2..e42c585dabe4 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -529,7 +529,9 @@ class ReplicaManager(val config: KafkaConfig, BrokerTopicStats.getBrokerAllTopicsStats().totalFetchRequestRate.mark() try { - trace("Fetching log segment for topic %s, partition %d, offset %d, partition fetch size %d, remaining response limit %d".format(topic, partition, offset, fetchSize, limitBytes)) + trace(s"Fetching log segment for partition $tp, offset ${offset}, partition fetch size ${fetchSize}, " + + s"remaining response limit ${limitBytes}" + + (if (minOneMessage) s", ignoring size limits for first message" else "")) // decide whether to only fetch from leader val localReplica = if (fetchOnlyFromLeader) @@ -558,12 +560,12 @@ class ReplicaManager(val config: KafkaConfig, val fetch = log.read(offset, adjustedFetchSize, maxOffsetOpt, minOneMessage) // If the partition is marked as throttled, and we are over-quota then exclude it - if (quota.isThrottled(TopicAndPartition(topic, partition)) && quota.isQuotaExceeded) + if (quota.isThrottled(tp) && quota.isQuotaExceeded) FetchDataInfo(fetch.fetchOffsetMetadata, MessageSet.Empty) else fetch case None => - error("Leader for partition [%s,%d] does not have a local log".format(topic, partition)) + error(s"Leader for partition $tp does not have a local log") FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty) } @@ -584,7 +586,7 @@ class ReplicaManager(val config: KafkaConfig, case e: Throwable => BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats().failedFetchRequestRate.mark() - error("Error processing fetch operation on partition [%s,%d] offset %d".format(topic, partition, offset), e) + error(s"Error processing fetch operation on partition ${tp}, offset $offset", e) LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, false, Some(e)) } } From 9fa75e255cf48bc08b23810dac5def327b3fe3fc Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 16 Sep 2016 12:14:42 +0100 Subject: [PATCH 39/54] Improve behaviour when `maxSize` is non-zero but smaller than the message size It is now consistent with maxSize=0. --- .../src/main/scala/kafka/log/LogSegment.scala | 4 ++-- .../test/scala/unit/kafka/log/LogTest.scala | 24 +++++++++++++++++++ 2 files changed, 26 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index 0a5452577a05..aca5b3622761 100755 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -157,8 +157,8 @@ class LogSegment(val log: FileMessageSet, val (startPosition, messageSetSize) = startOffsetAndSize val offsetMetadata = new LogOffsetMetadata(startOffset, this.baseOffset, startPosition.position) - // if the size is zero, still return a log segment but with zero size - if (maxSize == 0 && !minOneMessage) + // return a log segment but with zero size in the case below + if (!minOneMessage && maxSize < messageSetSize) return FetchDataInfo(offsetMetadata, MessageSet.Empty) val adjustedMaxSize = diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index d99fa14fe888..abc9a9647854 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -276,6 +276,30 @@ class LogTest extends JUnitSuite { } + @Test + def testReadWithTooSmallMaxLength() { + val logProps = new Properties() + logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer) + val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) + val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray + val messages = messageIds.map(id => new Message(id.toString.getBytes)) + + // now test the case that we give the offsets and use non-sequential offsets + for (i <- 0 until messages.length) + log.append(new ByteBufferMessageSet(NoCompressionCodec, new LongRef(messageIds(i)), messages = messages(i)), + assignOffsets = false) + + for (i <- 50 until messageIds.max) { + val messageSets = Seq( + log.read(i, 1), + log.read(i, 0) + ).map(_.messageSet) + messageSets.foreach { messageSet => + assertEquals(MessageSet.Empty, messageSet) + } + } + } + /** * Test reading at the boundary of the log, specifically * - reading from the logEndOffset should give an empty message set From 965b73ceb0b0e65d67c5d508b261ea59ff217d6e Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 16 Sep 2016 12:15:30 +0100 Subject: [PATCH 40/54] Add `testLowMaxFetchSizeForRequestAndPartition` Include a few test clean-ups --- .../kafka/api/BaseConsumerTest.scala | 30 ++++---- .../kafka/api/PlaintextConsumerTest.scala | 68 ++++++++++++++++--- 2 files changed, 73 insertions(+), 25 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index c13bf5804ebd..102b7cf8b416 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -19,16 +19,15 @@ import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.common.record.TimestampType import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.apache.kafka.common.{PartitionInfo, TopicPartition} -import kafka.utils.{TestUtils, Logging, ShutdownableThread} +import kafka.utils.{Logging, ShutdownableThread, TestUtils} import kafka.common.Topic import kafka.server.KafkaConfig -import java.util.ArrayList import org.junit.Assert._ import org.junit.{Before, Test} import scala.collection.JavaConverters._ -import scala.collection.mutable.Buffer +import scala.collection.mutable.{ArrayBuffer, Buffer} import org.apache.kafka.clients.producer.KafkaProducer import org.apache.kafka.common.errors.WakeupException @@ -134,19 +133,22 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { } } - protected def sendRecords(numRecords: Int): Unit = { + protected def sendRecords(numRecords: Int): Seq[ProducerRecord[Array[Byte], Array[Byte]]] = sendRecords(numRecords, tp) - } - protected def sendRecords(numRecords: Int, tp: TopicPartition) { + protected def sendRecords(numRecords: Int, tp: TopicPartition): Seq[ProducerRecord[Array[Byte], Array[Byte]]] = sendRecords(this.producers.head, numRecords, tp) - } - protected def sendRecords(producer: KafkaProducer[Array[Byte], Array[Byte]], numRecords: Int, tp: TopicPartition) { - (0 until numRecords).foreach { i => - producer.send(new ProducerRecord(tp.topic(), tp.partition(), i.toLong, s"key $i".getBytes, s"value $i".getBytes)) + protected def sendRecords(producer: KafkaProducer[Array[Byte], Array[Byte]], numRecords: Int, + tp: TopicPartition): Seq[ProducerRecord[Array[Byte], Array[Byte]]] = { + val records = (0 until numRecords).map { i => + val record = new ProducerRecord(tp.topic(), tp.partition(), i.toLong, s"key $i".getBytes, s"value $i".getBytes) + producer.send(record) + record } producer.flush() + + records } protected def consumeAndVerifyRecords(consumer: Consumer[Array[Byte], Array[Byte]], @@ -160,7 +162,7 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { val records = consumeRecords(consumer, numRecords, maxPollRecords = maxPollRecords) val now = System.currentTimeMillis() for (i <- 0 until numRecords) { - val record = records.get(i) + val record = records(i) val offset = startingOffset + i assertEquals(tp.topic, record.topic) assertEquals(tp.partition, record.partition) @@ -183,15 +185,15 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { protected def consumeRecords[K, V](consumer: Consumer[K, V], numRecords: Int, - maxPollRecords: Int = Int.MaxValue): ArrayList[ConsumerRecord[K, V]] = { - val records = new ArrayList[ConsumerRecord[K, V]] + maxPollRecords: Int = Int.MaxValue): ArrayBuffer[ConsumerRecord[K, V]] = { + val records = new ArrayBuffer[ConsumerRecord[K, V]] val maxIters = numRecords * 300 var iters = 0 while (records.size < numRecords) { val polledRecords = consumer.poll(50).asScala assertTrue(polledRecords.size <= maxPollRecords) for (record <- polledRecords) - records.add(record) + records += record if (iters > maxIters) throw new IllegalStateException("Failed to consume the expected records after " + iters + " iterations.") iters += 1 diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index dfac2cf1a9af..f4398dbb23e8 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -15,7 +15,6 @@ package kafka.api import java.util import java.util.Properties - import java.util.regex.Pattern import kafka.log.LogConfig @@ -23,11 +22,11 @@ import kafka.server.KafkaConfig import kafka.utils.TestUtils import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.producer.KafkaProducer -import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer, ByteArraySerializer} -import org.apache.kafka.test.{MockProducerInterceptor, MockConsumerInterceptor} +import org.apache.kafka.common.serialization.{ByteArraySerializer, StringDeserializer, StringSerializer} +import org.apache.kafka.test.{MockConsumerInterceptor, MockProducerInterceptor} import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.errors.{InvalidTopicException, RecordTooLargeException} +import org.apache.kafka.common.errors.{InvalidTopicException} import org.apache.kafka.common.record.{CompressionType, TimestampType} import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.junit.Assert._ @@ -550,7 +549,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { consumers += consumer0 // produce a record that is larger than the configured fetch size - val record = new ProducerRecord[Array[Byte], Array[Byte]](tp.topic(), tp.partition(), "key".getBytes, + val record = new ProducerRecord(tp.topic(), tp.partition(), "key".getBytes, new Array[Byte](producerRecordSize)) this.producers.head.send(record) @@ -578,14 +577,14 @@ class PlaintextConsumerTest extends BaseConsumerTest { val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) consumers += consumer0 - val smallRecord = new ProducerRecord[Array[Byte], Array[Byte]](tp.topic(), tp.partition(), "small".getBytes, + val smallRecord = new ProducerRecord(tp.topic(), tp.partition(), "small".getBytes, "value".getBytes) - val largeRecord = new ProducerRecord[Array[Byte], Array[Byte]](tp.topic(), tp.partition(), "large".getBytes, + val largeRecord = new ProducerRecord(tp.topic(), tp.partition(), "large".getBytes, new Array[Byte](largeProducerRecordSize)) this.producers.head.send(smallRecord) this.producers.head.send(largeRecord) - // consuming a record that is too large should succeed since KIP-74 + // we should only get the small record in the first `poll` consumer0.assign(List(tp).asJava) val records = consumer0.poll(20000) assertEquals(1, records.count) @@ -612,6 +611,53 @@ class PlaintextConsumerTest extends BaseConsumerTest { checkLargeRecord(maxPartitionFetchBytes + 1) } + /** Test that we consume all partitions if fetch max bytes and max.partition.fetch.bytes are low */ + @Test + def testLowMaxFetchSizeForRequestAndPartition(): Unit = { + // one of the effects of this is that there will be some log reads where `0 > remaining limit bytes < message size` + // and we don't return the message because it's not the first message in the first partition of the fetch + // this behaves a little different than when remaining limit bytes is 0 and it's important to test it + this.consumerConfig.setProperty(ConsumerConfig.FETCH_MAX_BYTES_CONFIG, "500") + this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, "100") + val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) + consumers += consumer0 + + val topic1 = "topic1" + val topic2 = "topic2" + val topic3 = "topic3" + val partitionCount = 30 + val topics = Seq(topic1, topic2, topic3) + topics.foreach { topicName => + TestUtils.createTopic(zkUtils, topicName, partitionCount, serverCount, servers) + } + + val partitions = topics.flatMap { topic => + (0 until partitionCount).map(new TopicPartition(topic, _)) + } + + assertEquals(0, consumer0.assignment().size) + + consumer0.subscribe(List(topic1, topic2, topic3).asJava) + + TestUtils.waitUntilTrue(() => { + consumer0.poll(50) + consumer0.assignment() == partitions.toSet.asJava + }, s"Expected partitions ${partitions.asJava} but actually got ${consumer0.assignment}") + + val producerRecords = partitions.flatMap(sendRecords(partitionCount, _)) + val consumerRecords = consumeRecords(consumer0, producerRecords.size) + + val expected = producerRecords.map { record => + (record.topic, record.partition, new String(record.key), new String(record.value), record.timestamp) + }.toSet + + val actual = consumerRecords.map { record => + (record.topic, record.partition, new String(record.key), new String(record.value), record.timestamp) + }.toSet + + assertEquals(expected, actual) + } + @Test def testRoundRobinAssignment() { // 1 consumer using round-robin assignment @@ -760,7 +806,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { // consume and verify that values are modified by interceptors val records = consumeRecords(testConsumer, numRecords) for (i <- 0 until numRecords) { - val record = records.get(i) + val record = records(i) assertEquals(s"key $i", new String(record.key())) assertEquals(s"value $i$appendStr".toUpperCase(Locale.ROOT), new String(record.value())) } @@ -844,7 +890,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) producerProps.put(ProducerConfig.INTERCEPTOR_CLASSES_CONFIG, "org.apache.kafka.test.MockProducerInterceptor") producerProps.put("mock.interceptor.append", appendStr) - val testProducer = new KafkaProducer[Array[Byte], Array[Byte]](producerProps, new ByteArraySerializer(), new ByteArraySerializer()) + val testProducer = new KafkaProducer(producerProps, new ByteArraySerializer(), new ByteArraySerializer()) producers += testProducer // producing records should succeed @@ -860,7 +906,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { // consume and verify that values are not modified by interceptors -- their exceptions are caught and logged, but not propagated val records = consumeRecords(testConsumer, 1) - val record = records.get(0) + val record = records.head assertEquals(s"value will not be modified", new String(record.value())) } From f5eb8d53b337189f822caa06a167d3e34fa6402f Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 16 Sep 2016 12:20:03 +0100 Subject: [PATCH 41/54] Update `Fetcher` not to try and detect `RecordTooLargeException`s The behaviour in brokers changed so it should never happen. --- .../clients/consumer/internals/Fetcher.java | 17 --------------- .../consumer/internals/FetcherTest.java | 21 ------------------- 2 files changed, 38 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index ed202d35a83d..202c0ad89fef 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -26,7 +26,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.InvalidMetadataException; import org.apache.kafka.common.errors.InvalidTopicException; -import org.apache.kafka.common.errors.RecordTooLargeException; import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.errors.TimeoutException; @@ -594,14 +593,11 @@ private PartitionRecords parseFetchedData(CompletedFetch completedFetch) { ByteBuffer buffer = partition.recordSet; MemoryRecords records = MemoryRecords.readableRecords(buffer); List> parsed = new ArrayList<>(); - boolean skippedRecords = false; for (LogEntry logEntry : records) { // Skip the messages earlier than current position. if (logEntry.offset() >= position) { parsed.add(parseRecord(tp, logEntry)); bytes += logEntry.size(); - } else { - skippedRecords = true; } } @@ -613,19 +609,6 @@ private PartitionRecords parseFetchedData(CompletedFetch completedFetch) { parsedRecords = new PartitionRecords<>(fetchOffset, tp, parsed); ConsumerRecord record = parsed.get(parsed.size() - 1); this.sensors.recordsFetchLag.record(partition.highWatermark - record.offset()); - } else if (buffer.limit() > 0 && !skippedRecords) { - // we did not read a single message from a non-empty buffer - // because that message's size is larger than fetch size, in this case - // record this exception - Map recordTooLargePartitions = Collections.singletonMap(tp, fetchOffset); - throw new RecordTooLargeException("There are some messages at [Partition=Offset]: " - + recordTooLargePartitions - + " whose size is larger than the fetch size " - + this.fetchSize - + " and hence cannot be ever returned." - + " Increase the fetch size on the client (using max.partition.fetch.bytes)," - + " or decrease the maximum message size the broker will allow (using message.max.bytes).", - recordTooLargePartitions); } } else if (partition.errorCode == Errors.NOT_LEADER_FOR_PARTITION.code() || partition.errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 4d93ead1aad8..4998382c7454 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -31,7 +31,6 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.InvalidTopicException; -import org.apache.kafka.common.errors.RecordTooLargeException; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.TopicAuthorizationException; @@ -64,7 +63,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Random; import static java.util.Collections.singleton; import static org.junit.Assert.assertEquals; @@ -316,25 +314,6 @@ public void testFetchNonContinuousRecords() { assertEquals(30L, consumerRecords.get(2).offset()); } - @Test(expected = RecordTooLargeException.class) - public void testFetchRecordTooLarge() { - subscriptions.assignFromUser(singleton(tp)); - subscriptions.seek(tp, 0); - - // prepare large record - MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE); - byte[] bytes = new byte[this.fetchSize]; - new Random().nextBytes(bytes); - records.append(1L, 0L, null, bytes); - records.close(); - - // resize the limit of the buffer to pretend it is only fetch-size large - fetcher.sendFetches(); - client.prepareResponse(fetchResponse((ByteBuffer) records.buffer().limit(this.fetchSize), Errors.NONE.code(), 100L, 0)); - consumerClient.poll(0); - fetcher.fetchedRecords(); - } - @Test public void testUnauthorizedTopic() { subscriptions.assignFromUser(singleton(tp)); From 8c653529822f0d6ee81a3e217201fc8fb75a6ef3 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 16 Sep 2016 13:03:34 +0100 Subject: [PATCH 42/54] Accept large messages from the first non-empty partition (instead of the first partition) As suggested by Jun --- .../org/apache/kafka/clients/consumer/ConsumerConfig.java | 4 ++-- core/src/main/scala/kafka/server/KafkaConfig.scala | 4 ++-- core/src/main/scala/kafka/server/ReplicaManager.scala | 7 +++++-- .../integration/kafka/api/PlaintextConsumerTest.scala | 2 +- 4 files changed, 10 insertions(+), 7 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 9f474a81138e..ae791b0d46db 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -117,7 +117,7 @@ public class ConsumerConfig extends AbstractConfig { */ public static final String FETCH_MAX_BYTES_CONFIG = "fetch.max.bytes"; private static final String FETCH_MAX_BYTES_DOC = "The maximum amount of data the server should return for a fetch request. " + - "This is not an absolute maximum, if the first message in the first partition of the fetch is larger than " + + "This is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than " + "this value, the message will still be returned to ensure that the consumer can make progress. " + "The maximum message size accepted by the broker is defined via message.max.bytes (broker config) or " + "max.message.bytes (topic config). Note that the consumer performs multiple fetches in parallel."; @@ -137,7 +137,7 @@ public class ConsumerConfig extends AbstractConfig { */ public static final String MAX_PARTITION_FETCH_BYTES_CONFIG = "max.partition.fetch.bytes"; private static final String MAX_PARTITION_FETCH_BYTES_DOC = "The maximum amount of data per-partition the server " + - "will return. If the first message in the first partition of the fetch is larger than this limit, the " + + "will return. If the first message in the first non-empty partition of the fetch is larger than this limit, the " + "message will still be returned to ensure that the consumer can make progress. The maximum message size " + "accepted by the broker is defined via message.max.bytes (broker config) or " + "max.message.bytes (topic config). See " + FETCH_MAX_BYTES_CONFIG + " for limiting the consumer request size"; diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index ba0f974f5a06..d9d30766bb4f 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -483,14 +483,14 @@ object KafkaConfig { val ReplicaSocketTimeoutMsDoc = "The socket timeout for network requests. Its value should be at least replica.fetch.wait.max.ms" val ReplicaSocketReceiveBufferBytesDoc = "The socket receive buffer for network requests" val ReplicaFetchMaxBytesDoc = "The number of bytes of messages to attempt to fetch for each partition. This is not an absolute maximum, " + - "if the first message in the first partition of the fetch is larger than this value, the message will still be returned " + + "if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned " + "to ensure that progress can be made. The maximum message size accepted by the broker is defined via " + "message.max.bytes (broker config) or max.message.bytes (topic config)." val ReplicaFetchWaitMaxMsDoc = "max wait time for each fetcher request issued by follower replicas. This value should always be less than the " + "replica.lag.time.max.ms at all times to prevent frequent shrinking of ISR for low throughput topics" val ReplicaFetchMinBytesDoc = "Minimum bytes expected for each fetch response. If not enough bytes, wait up to replicaMaxWaitTimeMs" val ReplicaFetchResponseMaxBytesDoc = "Maximum bytes expected for the entire fetch response. This is not an absolute maximum, " + - "if the first message in the first partition of the fetch is larger than this value, the message will still be returned " + + "if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned " + "to ensure that progress can be made. The maximum message size accepted by the broker is defined via " + "message.max.bytes (broker config) or max.message.bytes (topic config)." val NumReplicaFetchersDoc = "Number of fetcher threads used to replicate messages from a source broker. " + diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index e42c585dabe4..72d9a88e8d6b 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -593,10 +593,13 @@ class ReplicaManager(val config: KafkaConfig, var limitBytes = fetchMaxBytes val result = new mutable.ArrayBuffer[(TopicAndPartition, LogReadResult)] + var minOneMessage = true readPartitionInfo.foreach { case (tp, fetchInfo) => - val minOneMessage = tp == readPartitionInfo.head._1 val readResult = read(tp, fetchInfo, limitBytes, minOneMessage) - limitBytes = math.max(0, limitBytes - readResult.info.messageSet.sizeInBytes) + val messageSetSize = readResult.info.messageSet.sizeInBytes + if (messageSetSize > 0) + minOneMessage = false + limitBytes = math.max(0, limitBytes - messageSetSize) result += (tp -> readResult) } result diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index f4398dbb23e8..b05e55964658 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -565,7 +565,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertArrayEquals(record.value(), consumerRecord.value()) } - /** We should only return a large record if it's the first record in the first partition of the fetch request */ + /** We should only return a large record if it's the first record in the first non-empty partition of the fetch request */ @Test def testFetchHonoursFetchSizeIfLargeRecordNotFirst(): Unit = { val maxFetchBytes = 10 * 1024 From f61cdaa9f115512e43e3d33b118e5908f4b539fd Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 16 Sep 2016 13:04:13 +0100 Subject: [PATCH 43/54] Use `minOneMessage = true` when reading the log from `GroupMetadataManager` --- .../src/main/scala/kafka/coordinator/GroupMetadataManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index cf8ae917b7ac..1dc2a499d457 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -410,7 +410,7 @@ class GroupMetadataManager(val brokerId: Int, while (currOffset < getHighWatermark(offsetsPartition) && !shuttingDown.get()) { buffer.clear() - val messages = log.read(currOffset, config.loadBufferSize).messageSet.asInstanceOf[FileMessageSet] + val messages = log.read(currOffset, config.loadBufferSize, minOneMessage = true).messageSet.asInstanceOf[FileMessageSet] messages.readInto(buffer, 0) val messageSet = new ByteBufferMessageSet(buffer) messageSet.foreach { msgAndOffset => From 5c14cbcf45b1d25d4a54227a28690741d370c4ff Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 16 Sep 2016 13:04:21 +0100 Subject: [PATCH 44/54] Remove unused import. --- core/src/main/scala/kafka/api/FetchRequest.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/kafka/api/FetchRequest.scala b/core/src/main/scala/kafka/api/FetchRequest.scala index 05ee6af501bb..f100d4b63a4d 100644 --- a/core/src/main/scala/kafka/api/FetchRequest.scala +++ b/core/src/main/scala/kafka/api/FetchRequest.scala @@ -26,7 +26,6 @@ import kafka.message.MessageSet import java.util.concurrent.atomic.AtomicInteger import java.nio.ByteBuffer -import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.{ApiKeys, Errors} import scala.collection.mutable.ArrayBuffer From eef8fe4ef073e5e4ed83aefa0168c29f00c783f6 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 16 Sep 2016 13:14:21 +0100 Subject: [PATCH 45/54] Remove `warnIfMessageOversized` from `ReplicaFetcherThread` --- .../main/scala/kafka/server/ReplicaFetcherThread.scala | 9 --------- 1 file changed, 9 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 8b8a1a53b34f..66710a5d834c 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -120,7 +120,6 @@ class ReplicaFetcherThread(name: String, val partitionId = topicPartition.partition val replica = replicaMgr.getReplica(topic, partitionId).get val messageSet = partitionData.toByteBufferMessageSet - warnIfMessageOversized(messageSet, topicPartition) if (fetchOffset != replica.logEndOffset.messageOffset) throw new RuntimeException("Offset mismatch for partition %s: fetched offset = %d, log end offset = %d.".format(topicPartition, fetchOffset, replica.logEndOffset.messageOffset)) @@ -148,14 +147,6 @@ class ReplicaFetcherThread(name: String, } } - def warnIfMessageOversized(messageSet: ByteBufferMessageSet, topicPartition: TopicPartition): Unit = { - if (messageSet.sizeInBytes > 0 && messageSet.validBytes <= 0) - error(s"Replication is failing due to a message that is greater than replica.fetch.max.bytes for partition $topicPartition. " + - "This generally occurs when the max.message.bytes has been overridden to exceed this value and a suitably large " + - "message has also been sent. To fix this problem increase replica.fetch.max.bytes in your broker config to be " + - "equal or larger than your settings for max.message.bytes, both at a broker and topic level.") - } - /** * Handle a partition whose offset is out of range and return a new fetch offset. */ From 4e7e6044589e8efb0b9cf43592b81e547d30834c Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 16 Sep 2016 13:19:48 +0100 Subject: [PATCH 46/54] Remove message size warnings from `TopicCommand` and `ConfigCommand` --- .../scala/kafka/admin/ConfigCommand.scala | 18 +------ .../main/scala/kafka/admin/TopicCommand.scala | 48 ------------------- 2 files changed, 2 insertions(+), 64 deletions(-) diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index ebf9e61a224b..a4ca6fc75f11 100644 --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -69,7 +69,6 @@ object ConfigCommand { val configsToBeDeleted = parseConfigsToBeDeleted(opts) val entityType = opts.options.valueOf(opts.entityType) val entityName = opts.options.valueOf(opts.entityName) - warnOnMaxMessagesChange(configsToBeAdded, opts.options.has(opts.forceOpt)) // compile the final set of configs val configs = utils.fetchEntityConfig(zkUtils, entityType, entityName) @@ -85,22 +84,9 @@ object ConfigCommand { println(s"Updated config for EntityType:$entityType => EntityName:'$entityName'.") } - def warnOnMaxMessagesChange(configs: Properties, force: Boolean): Unit = { - val maxMessageBytes = configs.get(LogConfig.MaxMessageBytesProp) match { - case n: String => n.toInt - case _ => -1 - } - if (maxMessageBytes > Defaults.MaxMessageSize){ - error(TopicCommand.longMessageSizeWarning(maxMessageBytes)) - if (!force) - TopicCommand.askToProceed - } - } - private def parseBroker(broker: String): Int = { - try { - broker.toInt - }catch { + try broker.toInt + catch { case e: NumberFormatException => throw new IllegalArgumentException(s"Error parsing broker $broker. The broker's Entity Name must be a single integer value") } diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index 421486c07aa6..a35a98977124 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -97,13 +97,11 @@ object TopicCommand extends Logging { try { if (opts.options.has(opts.replicaAssignmentOpt)) { val assignment = parseReplicaAssignment(opts.options.valueOf(opts.replicaAssignmentOpt)) - warnOnMaxMessagesChange(configs, assignment.valuesIterator.next().length, opts.options.has(opts.forceOpt)) AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, assignment, configs, update = false) } else { CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.partitionsOpt, opts.replicationFactorOpt) val partitions = opts.options.valueOf(opts.partitionsOpt).intValue val replicas = opts.options.valueOf(opts.replicationFactorOpt).intValue - warnOnMaxMessagesChange(configs, replicas, opts.options.has(opts.forceOpt)) val rackAwareMode = if (opts.options.has(opts.disableRackAware)) RackAwareMode.Disabled else RackAwareMode.Enforced AdminUtils.createTopic(zkUtils, topic, partitions, replicas, configs, rackAwareMode) @@ -357,20 +355,6 @@ object TopicCommand extends Logging { CommandLineUtils.checkInvalidArgs(parser, options, ifNotExistsOpt, allTopicLevelOpts -- Set(createOpt)) } } - def warnOnMaxMessagesChange(configs: Properties, replicas: Integer, force: Boolean): Unit = { - val maxMessageBytes = configs.get(LogConfig.MaxMessageBytesProp) match { - case n: String => n.toInt - case _ => -1 - } - if (maxMessageBytes > Defaults.MaxMessageSize) - if (replicas > 1) { - error(longMessageSizeWarning(maxMessageBytes)) - if (!force) - askToProceed - } - else - warn(shortMessageSizeWarning(maxMessageBytes)) - } def askToProceed: Unit = { println("Are you sure you want to continue? [y/n]") @@ -380,37 +364,5 @@ object TopicCommand extends Logging { } } - def shortMessageSizeWarning(maxMessageBytes: Int): String = { - "\n\n" + - "*****************************************************************************************************\n" + - "*** WARNING: you are creating a topic where the max.message.bytes is greater than the broker's ***\n" + - "*** default max.message.bytes. This operation is potentially dangerous. Consumers will get ***\n" + - s"*** failures if their fetch.message.max.bytes (old consumer) or ${NewConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG} ***\n"+ - "*** (new consumer) < the value you are using. ***\n" + - "*****************************************************************************************************\n" + - s"- value set here: $maxMessageBytes\n" + - s"- Default Old Consumer fetch.message.max.bytes: ${OldConsumerConfig.FetchSize}\n" + - s"- Default New Consumer ${NewConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG}: ${NewConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES}\n" + - s"- Default Broker max.message.bytes: ${kafka.server.Defaults.MessageMaxBytes}\n\n" - } - - def longMessageSizeWarning(maxMessageBytes: Int): String = { - "\n\n" + - "*****************************************************************************************************\n" + - "*** WARNING: you are creating a topic where the max.message.bytes is greater than the broker's ***\n" + - "*** default max.message.bytes. This operation is dangerous. There are two potential side effects: ***\n" + - "*** - Consumers will get failures if their fetch.message.max.bytes (old consumer) or ***\n" + - s"*** ${NewConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG} (new consumer) < the value you are using ***\n" + - "*** - Producer requests larger than replica.fetch.max.bytes will not replicate and hence have ***\n" + - "*** a higher risk of data loss ***\n" + - "*** You should ensure both of these settings are greater than the value set here before using ***\n" + - "*** this topic. ***\n" + - "*****************************************************************************************************\n" + - s"- value set here: $maxMessageBytes\n" + - s"- Default Broker replica.fetch.max.bytes: ${kafka.server.Defaults.ReplicaFetchMaxBytes}\n" + - s"- Default Broker max.message.bytes: ${kafka.server.Defaults.MessageMaxBytes}\n" + - s"- Default Old Consumer fetch.message.max.bytes: ${OldConsumerConfig.FetchSize}\n" + - s"- Default New Consumer ${NewConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG}: ${NewConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES}\n\n" - } } From d206a590c8034ec8554392171dced97fdcf87dc9 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 16 Sep 2016 14:50:44 +0100 Subject: [PATCH 47/54] Add test to verify that oversized messages produced with ack=all succeed Oversized in the context of replication. --- .../api/ProducerFailureHandlingTest.scala | 94 ++++++++++--------- 1 file changed, 48 insertions(+), 46 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index c8fcba6c297f..457a90983b01 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -18,14 +18,14 @@ package kafka.api import java.util.concurrent.{ExecutionException, TimeoutException} -import java.util.{Properties} +import java.util.Properties + import kafka.common.Topic -import kafka.consumer.SimpleConsumer import kafka.integration.KafkaServerTestHarness +import kafka.log.LogConfig import kafka.server.KafkaConfig -import kafka.utils.{ShutdownableThread, TestUtils} +import kafka.utils.{TestUtils} import org.apache.kafka.clients.producer._ -import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback import org.apache.kafka.common.errors.{InvalidTopicException, NotEnoughReplicasAfterAppendException, NotEnoughReplicasException} import org.junit.Assert._ import org.junit.{After, Before, Test} @@ -33,12 +33,16 @@ import org.junit.{After, Before, Test} class ProducerFailureHandlingTest extends KafkaServerTestHarness { private val producerBufferSize = 30000 private val serverMessageMaxBytes = producerBufferSize/2 + private val replicaFetchMaxPartitionBytes = serverMessageMaxBytes + 200 + private val replicaFetchMaxResponseBytes = replicaFetchMaxPartitionBytes + 200 val numServers = 2 val overridingProps = new Properties() overridingProps.put(KafkaConfig.AutoCreateTopicsEnableProp, false.toString) overridingProps.put(KafkaConfig.MessageMaxBytesProp, serverMessageMaxBytes.toString) + overridingProps.put(KafkaConfig.ReplicaFetchMaxBytesProp, replicaFetchMaxPartitionBytes.toString) + overridingProps.put(KafkaConfig.ReplicaFetchResponseMaxBytesDoc, replicaFetchMaxResponseBytes.toString) // Set a smaller value for the number of partitions for the offset commit topic (__consumer_offset topic) // so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long overridingProps.put(KafkaConfig.OffsetsTopicPartitionsProp, 1.toString) @@ -46,13 +50,10 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { def generateConfigs() = TestUtils.createBrokerConfigs(numServers, zkConnect, false).map(KafkaConfig.fromProps(_, overridingProps)) - private var consumer1: SimpleConsumer = null - private var consumer2: SimpleConsumer = null - - private var producer1: KafkaProducer[Array[Byte],Array[Byte]] = null - private var producer2: KafkaProducer[Array[Byte],Array[Byte]] = null - private var producer3: KafkaProducer[Array[Byte],Array[Byte]] = null - private var producer4: KafkaProducer[Array[Byte],Array[Byte]] = null + private var producer1: KafkaProducer[Array[Byte], Array[Byte]] = null + private var producer2: KafkaProducer[Array[Byte], Array[Byte]] = null + private var producer3: KafkaProducer[Array[Byte], Array[Byte]] = null + private var producer4: KafkaProducer[Array[Byte], Array[Byte]] = null private val topic1 = "topic-1" private val topic2 = "topic-2" @@ -88,7 +89,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { TestUtils.createTopic(zkUtils, topic1, 1, numServers, servers) // send a too-large record - val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1)) + val record = new ProducerRecord(topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1)) assertEquals("Returned metadata should have offset -1", producer1.send(record).get.offset, -1L) } @@ -101,19 +102,48 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { TestUtils.createTopic(zkUtils, topic1, 1, numServers, servers) // send a too-large record - val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1)) + val record = new ProducerRecord(topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1)) intercept[ExecutionException] { producer2.send(record).get } } + private def checkTooLargeRecordForReplicationWithAckAll(maxFetchSize: Int) { + val maxMessageSize = maxFetchSize + 100 + val topicConfig = new Properties + topicConfig.setProperty(LogConfig.MinInSyncReplicasProp, numServers.toString) + topicConfig.setProperty(LogConfig.MaxMessageBytesProp, maxMessageSize.toString) + + // create topic + val topic10 = "topic10" + TestUtils.createTopic(zkUtils, topic10, servers.size, numServers, servers, topicConfig) + + // send a record that is too large for replication, but within the broker max message limit + val record = new ProducerRecord(topic10, null, "key".getBytes, new Array[Byte](maxMessageSize - 50)) + val recordMetadata = producer3.send(record).get + + assertEquals(topic10, recordMetadata.topic()) + } + + /** This should succeed as the replica fetcher thread can handle oversized messages since KIP-74 */ + @Test + def testPartitionTooLargeForReplicationWithAckAll() { + checkTooLargeRecordForReplicationWithAckAll(replicaFetchMaxPartitionBytes) + } + + /** This should succeed as the replica fetcher thread can handle oversized messages since KIP-74 */ + @Test + def testResponseTooLargeForReplicationWithAckAll() { + checkTooLargeRecordForReplicationWithAckAll(replicaFetchMaxResponseBytes) + } + /** * With non-exist-topic the future metadata should return ExecutionException caused by TimeoutException */ @Test def testNonExistentTopic() { // send a record with non-exist topic - val record = new ProducerRecord[Array[Byte],Array[Byte]](topic2, null, "key".getBytes, "value".getBytes) + val record = new ProducerRecord(topic2, null, "key".getBytes, "value".getBytes) intercept[ExecutionException] { producer1.send(record).get } @@ -138,7 +168,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { producer4 = TestUtils.createNewProducer("localhost:8686,localhost:4242", acks = 1, maxBlockMs = 10000L, bufferSize = producerBufferSize) // send a record with incorrect broker list - val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, "value".getBytes) + val record = new ProducerRecord(topic1, null, "key".getBytes, "value".getBytes) intercept[ExecutionException] { producer4.send(record).get } @@ -212,7 +242,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { TestUtils.createTopic(zkUtils, topicName, 1, numServers, servers, topicProps) - val record = new ProducerRecord[Array[Byte],Array[Byte]](topicName, null, "key".getBytes, "value".getBytes) + val record = new ProducerRecord(topicName, null, "key".getBytes, "value".getBytes) try { producer3.send(record).get fail("Expected exception when producing to topic with fewer brokers than min.insync.replicas") @@ -228,11 +258,11 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { def testNotEnoughReplicasAfterBrokerShutdown() { val topicName = "minisrtest2" val topicProps = new Properties() - topicProps.put("min.insync.replicas",numServers.toString) + topicProps.put("min.insync.replicas", numServers.toString) TestUtils.createTopic(zkUtils, topicName, 1, numServers, servers,topicProps) - val record = new ProducerRecord[Array[Byte],Array[Byte]](topicName, null, "key".getBytes, "value".getBytes) + val record = new ProducerRecord(topicName, null, "key".getBytes, "value".getBytes) // this should work with all brokers up and running producer3.send(record).get @@ -256,32 +286,4 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { servers.head.startup() } - private class ProducerScheduler extends ShutdownableThread("daemon-producer", false) - { - val numRecords = 1000 - var sent = 0 - var failed = false - - val producer = TestUtils.createNewProducer(brokerList, bufferSize = producerBufferSize, retries = 10) - - override def doWork(): Unit = { - val responses = - for (i <- sent+1 to sent+numRecords) - yield producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, null, i.toString.getBytes), - new ErrorLoggingCallback(topic1, null, null, true)) - val futures = responses.toList - - try { - futures.map(_.get) - sent += numRecords - } catch { - case e : Exception => failed = true - } - } - - override def shutdown(){ - super.shutdown() - producer.close() - } - } } From 03694d0aff564e62db14379ac46613cfc2ef1847 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 16 Sep 2016 23:05:53 +0100 Subject: [PATCH 48/54] Introduce FetchRequestTest --- .../unit/kafka/server/FetchRequestTest.scala | 203 ++++++++++++++++++ 1 file changed, 203 insertions(+) create mode 100644 core/src/test/scala/unit/kafka/server/FetchRequestTest.scala diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala new file mode 100644 index 000000000000..1a09c9928a76 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala @@ -0,0 +1,203 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.server + +import java.util +import java.util.Properties + +import kafka.log.LogConfig +import kafka.utils.TestUtils +import kafka.utils.TestUtils._ +import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.record.{LogEntry, MemoryRecords} +import org.apache.kafka.common.requests.{FetchRequest, FetchResponse} +import org.apache.kafka.common.serialization.StringSerializer +import org.junit.Test +import org.junit.Assert._ + +import scala.collection.JavaConverters._ +import scala.util.Random + +/** + * Subclasses of `BaseConsumerTest` exercise the consumer and fetch request/response. This class + * complements those classes with tests that require lower-level access to the protocol. + */ +class FetchRequestTest extends BaseRequestTest { + + private var producer: KafkaProducer[String, String] = null + + override def setUp() { + super.setUp() + producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers), + retries = 5, keySerializer = new StringSerializer, valueSerializer = new StringSerializer) + } + + override def tearDown() { + producer.close() + super.tearDown() + } + + private def createFetchRequest(maxResponseBytes: Int, maxPartitionBytes: Int, topicPartitions: Seq[TopicPartition], + offsetMap: Map[TopicPartition, Long] = Map.empty): FetchRequest = { + val partitionMap = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData] + topicPartitions.foreach { tp => + partitionMap.put(tp, new FetchRequest.PartitionData(offsetMap.getOrElse(tp, 0), maxPartitionBytes)) + } + new FetchRequest(Int.MaxValue, 0, maxResponseBytes, partitionMap) + } + + private def sendFetchRequest(leaderId: Int, request: FetchRequest): FetchResponse = { + val response = send(request, ApiKeys.FETCH, destination = brokerSocketServer(leaderId)) + FetchResponse.parse(response) + } + + @Test + def testBrokerRespectsPartitionsOrderAndSizeLimits(): Unit = { + val messagesPerPartition = 9 + val maxResponseBytes = 800 + val maxPartitionBytes = 200 + + def createFetchRequest(topicPartitions: Seq[TopicPartition], offsetMap: Map[TopicPartition, Long] = Map.empty): FetchRequest = + this.createFetchRequest(maxResponseBytes, maxPartitionBytes, topicPartitions, offsetMap) + + val topicPartitionToLeader = createTopics(numTopics = 5, numPartitions = 6) + val random = new Random(0) + val topicPartitions = topicPartitionToLeader.keySet + produceData(topicPartitions, messagesPerPartition) + + val leaderId = servers.head.config.brokerId + val partitionsForLeader = topicPartitionToLeader.toVector.collect { + case (tp, partitionLeaderId) if partitionLeaderId == leaderId => tp + } + + val partitionsWithLargeMessages = partitionsForLeader.takeRight(2) + val partitionWithLargeMessage1 = partitionsWithLargeMessages.head + val partitionWithLargeMessage2 = partitionsWithLargeMessages(1) + producer.send(new ProducerRecord(partitionWithLargeMessage1.topic, partitionWithLargeMessage1.partition, + "larger than partition limit", new String(new Array[Byte](maxPartitionBytes + 1)))).get + producer.send(new ProducerRecord(partitionWithLargeMessage2.topic, partitionWithLargeMessage2.partition, + "larger than response limit", new String(new Array[Byte](maxResponseBytes + 1)))).get + + val partitionsWithoutLargeMessages = partitionsForLeader.filterNot(partitionsWithLargeMessages.contains) + + // 1. Partitions with large messages at the end + val shuffledTopicPartitions1 = random.shuffle(partitionsWithoutLargeMessages) ++ partitionsWithLargeMessages + val fetchRequest1 = createFetchRequest(shuffledTopicPartitions1) + val fetchResponse1 = sendFetchRequest(leaderId, fetchRequest1) + checkFetchResponse(shuffledTopicPartitions1, fetchResponse1, maxPartitionBytes, maxResponseBytes, messagesPerPartition) + + // 2. Same as 1, but shuffled again + val shuffledTopicPartitions2 = random.shuffle(partitionsWithoutLargeMessages) ++ partitionsWithLargeMessages + val fetchRequest2 = createFetchRequest(shuffledTopicPartitions2) + val fetchResponse2 = sendFetchRequest(leaderId, fetchRequest2) + checkFetchResponse(shuffledTopicPartitions2, fetchResponse2, maxPartitionBytes, maxResponseBytes, messagesPerPartition) + + // 3. Partition with message larger than the partition limit at the start of the list + val shuffledTopicPartitions3 = Seq(partitionWithLargeMessage1, partitionWithLargeMessage2) ++ + random.shuffle(partitionsWithoutLargeMessages) + val fetchRequest3 = createFetchRequest(shuffledTopicPartitions3, Map(partitionWithLargeMessage1 -> messagesPerPartition)) + val fetchResponse3 = sendFetchRequest(leaderId, fetchRequest3) + assertEquals(shuffledTopicPartitions3, fetchResponse3.responseData.keySet.asScala.toSeq) + val responseSize3 = fetchResponse3.responseData.asScala.values.map { partitionData => + logEntries(partitionData).map(_.size).sum + }.sum + assertTrue(responseSize3 <= maxResponseBytes) + val partitionData3 = fetchResponse3.responseData.get(partitionWithLargeMessage1) + assertEquals(Errors.NONE.code, partitionData3.errorCode) + assertTrue(partitionData3.highWatermark > 0) + val size3 = logEntries(partitionData3).map(_.size).sum + assertTrue(s"Expected $size3 to be smaller than $maxResponseBytes", size3 <= maxResponseBytes) + assertTrue(s"Expected $size3 to be larger than $maxPartitionBytes", size3 > maxPartitionBytes) + + // 4. Partition with message larger than the response limit at the start of the list + val shuffledTopicPartitions4 = Seq(partitionWithLargeMessage2, partitionWithLargeMessage1) ++ + random.shuffle(partitionsWithoutLargeMessages) + val fetchRequest4 = createFetchRequest(shuffledTopicPartitions4, Map(partitionWithLargeMessage2 -> messagesPerPartition)) + val fetchResponse4 = sendFetchRequest(leaderId, fetchRequest4) + assertEquals(shuffledTopicPartitions4, fetchResponse4.responseData.keySet.asScala.toSeq) + val nonEmptyPartitions4 = fetchResponse4.responseData.asScala.toSeq.collect { + case (tp, partitionData) if logEntries(partitionData).map(_.size).sum > 0 => tp + } + assertEquals(Seq(partitionWithLargeMessage2), nonEmptyPartitions4) + val partitionData4 = fetchResponse4.responseData.get(partitionWithLargeMessage2) + assertEquals(Errors.NONE.code, partitionData4.errorCode) + assertTrue(partitionData4.highWatermark > 0) + val size4 = logEntries(partitionData4).map(_.size).sum + assertTrue(s"Expected $size4 to be larger than $maxResponseBytes}", size4 > maxResponseBytes) + } + + private def logEntries(partitionData: FetchResponse.PartitionData): Seq[LogEntry] = { + val memoryRecords = MemoryRecords.readableRecords(partitionData.recordSet) + memoryRecords.iterator.asScala.toIndexedSeq + } + + private def checkFetchResponse(expectedPartitions: Seq[TopicPartition], fetchResponse: FetchResponse, + maxPartitionBytes: Int, maxResponseBytes: Int, numMessagesPerPartition: Int): Unit = { + assertEquals(expectedPartitions, fetchResponse.responseData.keySet.asScala.toSeq) + var emptyResponseSeen = false + var responseSize = 0 + + expectedPartitions.foreach { tp => + val partitionData = fetchResponse.responseData.get(tp) + assertEquals(Errors.NONE.code, partitionData.errorCode) + assertTrue(partitionData.highWatermark > 0) + val messages = logEntries(partitionData) + assertTrue(messages.size < numMessagesPerPartition) + val size = messages.map(_.size).sum + responseSize += size + if (size == 0 && !emptyResponseSeen) + emptyResponseSeen = true + else if (size != 0 && !emptyResponseSeen) { + assertTrue(size <= maxPartitionBytes) + // tolerance below may have to be tweaked if the message sizes in the test change + assertTrue(size > maxPartitionBytes - 50) + } + else if (size != 0 && emptyResponseSeen) + fail(s"Expected partition with size 0, but found $tp with size $size") + } + + assertTrue(responseSize <= maxResponseBytes) + // tolerance below may have to be tweaked if the message sizes in the test change + assertTrue(responseSize > maxResponseBytes - 50) + } + + private def createTopics(numTopics: Int, numPartitions: Int): Map[TopicPartition, Int] = { + val topics = (0 until numPartitions).map(t => s"topic${t}") + val topicConfig = new Properties + topicConfig.setProperty(LogConfig.MinInSyncReplicasProp, 2.toString) + topics.flatMap { topic => + val partitionToLeader = createTopic(zkUtils, topic, numPartitions = numPartitions, replicationFactor = 2, + servers = servers, topicConfig = topicConfig) + partitionToLeader.map { case (partition, leader) => new TopicPartition(topic, partition) -> leader.get } + }.toMap + } + + private def produceData(topicPartitions: Iterable[TopicPartition], numMessagesPerPartition: Int): Seq[ProducerRecord[String, String]] = { + val records = for { + tp <- topicPartitions.toSeq + messageIndex <- 0 until numMessagesPerPartition + } yield { + val suffix = s"${tp}-${messageIndex}" + new ProducerRecord(tp.topic, tp.partition, s"key $suffix", s"value $suffix") + } + records.map(producer.send).foreach(_.get) + records + } + +} From 7534e9eb1fa8fbc89bc88753f10c95e8f76ba8b6 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Sat, 17 Sep 2016 01:40:35 +0100 Subject: [PATCH 49/54] Preserve behaviour of fetch request version 2 and add test to verify --- .../src/main/scala/kafka/log/LogSegment.scala | 17 ++--- .../scala/kafka/server/DelayedFetch.scala | 2 + .../scala/kafka/server/FetchDataInfo.scala | 3 +- .../main/scala/kafka/server/KafkaApis.scala | 1 + .../scala/kafka/server/ReplicaManager.scala | 25 ++++--- .../kafka/api/PlaintextConsumerTest.scala | 2 +- .../test/scala/unit/kafka/log/LogTest.scala | 18 +++-- .../unit/kafka/server/FetchRequestTest.scala | 68 +++++++++++++------ .../server/ReplicaManagerQuotasTest.scala | 6 +- .../kafka/server/ReplicaManagerTest.scala | 3 + .../unit/kafka/server/SimpleFetchTest.scala | 4 +- 11 files changed, 98 insertions(+), 51 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index aca5b3622761..44a761132421 100755 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -144,7 +144,7 @@ class LogSegment(val log: FileMessageSet, @threadsafe def read(startOffset: Long, maxOffset: Option[Long], maxSize: Int, maxPosition: Long = size, minOneMessage: Boolean = false): FetchDataInfo = { - if(maxSize < 0) + if (maxSize < 0) throw new IllegalArgumentException("Invalid max size for log read (%d)".format(maxSize)) val logSize = log.sizeInBytes // this may change, need to save a consistent copy @@ -157,14 +157,14 @@ class LogSegment(val log: FileMessageSet, val (startPosition, messageSetSize) = startOffsetAndSize val offsetMetadata = new LogOffsetMetadata(startOffset, this.baseOffset, startPosition.position) - // return a log segment but with zero size in the case below - if (!minOneMessage && maxSize < messageSetSize) - return FetchDataInfo(offsetMetadata, MessageSet.Empty) - val adjustedMaxSize = if (minOneMessage) math.max(maxSize, messageSetSize) else maxSize + // return a log segment but with zero size in the case below + if (adjustedMaxSize == 0) + return FetchDataInfo(offsetMetadata, MessageSet.Empty) + // calculate the length of the message set to read based on whether or not they gave us a maxOffset val length = maxOffset match { case None => @@ -175,8 +175,8 @@ class LogSegment(val log: FileMessageSet, // when the leader of a partition changes, it's possible for the new leader's high watermark to be less than the // true high watermark in the previous leader for a short window. In this window, if a consumer fetches on an // offset between new leader's high watermark and the log end offset, we want to return an empty response. - if(offset < startOffset) - return FetchDataInfo(offsetMetadata, MessageSet.Empty) + if (offset < startOffset) + return FetchDataInfo(offsetMetadata, MessageSet.Empty, messageSetIncomplete = false) val mapping = translateOffset(offset, startPosition.position) val endPosition = if (mapping == null) @@ -186,7 +186,8 @@ class LogSegment(val log: FileMessageSet, min(min(maxPosition, endPosition) - startPosition.position, adjustedMaxSize).toInt } - FetchDataInfo(offsetMetadata, log.read(startPosition.position, length)) + FetchDataInfo(offsetMetadata, log.read(startPosition.position, length), + messageSetIncomplete = adjustedMaxSize < messageSetSize) } /** diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index e9a45d133090..4b17e81e9a46 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -38,6 +38,7 @@ case class FetchPartitionStatus(startOffsetMetadata: LogOffsetMetadata, fetchInf */ case class FetchMetadata(fetchMinBytes: Int, fetchMaxBytes: Int, + hardMaxBytesLimit: Boolean, fetchOnlyLeader: Boolean, fetchOnlyCommitted: Boolean, isFromFollower: Boolean, @@ -141,6 +142,7 @@ class DelayedFetch(delayMs: Long, fetchMetadata.fetchOnlyLeader, fetchMetadata.fetchOnlyCommitted, fetchMetadata.fetchMaxBytes, + fetchMetadata.hardMaxBytesLimit, fetchMetadata.fetchPartitionStatus.map { case (tp, status) => tp -> status.fetchInfo }, quota ) diff --git a/core/src/main/scala/kafka/server/FetchDataInfo.scala b/core/src/main/scala/kafka/server/FetchDataInfo.scala index 1a8a60494100..0a4d6b861452 100644 --- a/core/src/main/scala/kafka/server/FetchDataInfo.scala +++ b/core/src/main/scala/kafka/server/FetchDataInfo.scala @@ -19,4 +19,5 @@ package kafka.server import kafka.message.MessageSet -case class FetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata, messageSet: MessageSet) +case class FetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata, messageSet: MessageSet, + messageSetIncomplete: Boolean = false) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index c0be3e06bdf9..391b4148f5f5 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -507,6 +507,7 @@ class KafkaApis(val requestChannel: RequestChannel, fetchRequest.replicaId, fetchRequest.minBytes, fetchRequest.maxBytes, + fetchRequest.versionId <= 2, authorizedRequestInfo, replicationQuota(fetchRequest), sendResponseCallback) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 72d9a88e8d6b..12f8768fc140 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -25,15 +25,12 @@ import kafka.api._ import kafka.cluster.{Partition, Replica} import kafka.common._ import kafka.controller.KafkaController -import kafka.log.{LogAppendInfo, LogManager} +import kafka.log.{FileMessageSet, LogAppendInfo, LogManager} import kafka.message.{ByteBufferMessageSet, InvalidMessageException, Message, MessageSet} import kafka.metrics.KafkaMetricsGroup import kafka.server.QuotaFactory.UnboundedQuota import kafka.utils._ -import org.apache.kafka.common.errors.{ControllerMovedException, CorruptRecordException, InvalidTimestampException, - InvalidTopicException, NotLeaderForPartitionException, OffsetOutOfRangeException, - RecordBatchTooLargeException, RecordTooLargeException, ReplicaNotAvailableException, - UnknownTopicOrPartitionException} +import org.apache.kafka.common.errors.{ControllerMovedException, CorruptRecordException, InvalidTimestampException, InvalidTopicException, NotLeaderForPartitionException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException, ReplicaNotAvailableException, UnknownTopicOrPartitionException} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors @@ -460,6 +457,7 @@ class ReplicaManager(val config: KafkaConfig, replicaId: Int, fetchMinBytes: Int, fetchMaxBytes: Int, + hardMaxBytesLimit: Boolean, fetchInfos: Seq[(TopicAndPartition, PartitionFetchInfo)], quota: ReplicaQuota = UnboundedQuota, responseCallback: Seq[(TopicAndPartition, FetchResponsePartitionData)] => Unit) { @@ -468,7 +466,8 @@ class ReplicaManager(val config: KafkaConfig, val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(replicaId) // read from local logs - val logReadResults = readFromLocalLog(fetchOnlyFromLeader, fetchOnlyCommitted, fetchMaxBytes, fetchInfos, quota) + val logReadResults = readFromLocalLog(fetchOnlyFromLeader, fetchOnlyCommitted, fetchMaxBytes, hardMaxBytesLimit, + fetchInfos, quota) // if the fetch comes from the follower, // update its corresponding log end offset @@ -498,8 +497,8 @@ class ReplicaManager(val config: KafkaConfig, }.getOrElse(sys.error(s"Partition $topicAndPartition not found in fetchInfos")) (topicAndPartition, FetchPartitionStatus(result.info.fetchOffsetMetadata, fetchInfo)) } - val fetchMetadata = FetchMetadata(fetchMinBytes, fetchMaxBytes, fetchOnlyFromLeader, fetchOnlyCommitted, isFromFollower, - fetchPartitionStatus) + val fetchMetadata = FetchMetadata(fetchMinBytes, fetchMaxBytes, hardMaxBytesLimit, fetchOnlyFromLeader, + fetchOnlyCommitted, isFromFollower, fetchPartitionStatus) val delayedFetch = new DelayedFetch(timeout, fetchMetadata, this, quota, responseCallback) // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation @@ -518,6 +517,7 @@ class ReplicaManager(val config: KafkaConfig, def readFromLocalLog(fetchOnlyFromLeader: Boolean, readOnlyCommitted: Boolean, fetchMaxBytes: Int, + hardMaxBytesLimit: Boolean, readPartitionInfo: Seq[(TopicAndPartition, PartitionFetchInfo)], quota: ReplicaQuota): Seq[(TopicAndPartition, LogReadResult)] = { @@ -531,7 +531,7 @@ class ReplicaManager(val config: KafkaConfig, try { trace(s"Fetching log segment for partition $tp, offset ${offset}, partition fetch size ${fetchSize}, " + s"remaining response limit ${limitBytes}" + - (if (minOneMessage) s", ignoring size limits for first message" else "")) + (if (minOneMessage) s", ignoring response/partition size limits" else "")) // decide whether to only fetch from leader val localReplica = if (fetchOnlyFromLeader) @@ -562,6 +562,10 @@ class ReplicaManager(val config: KafkaConfig, // If the partition is marked as throttled, and we are over-quota then exclude it if (quota.isThrottled(tp) && quota.isQuotaExceeded) FetchDataInfo(fetch.fetchOffsetMetadata, MessageSet.Empty) + // For FetchRequest version 3, we replace incomplete message sets with an empty one as consumers can make + // progress in such cases and don't need to report a `RecordTooLargeException` + else if (!hardMaxBytesLimit && fetch.messageSetIncomplete) + FetchDataInfo(fetch.fetchOffsetMetadata, MessageSet.Empty) else fetch case None => @@ -593,10 +597,11 @@ class ReplicaManager(val config: KafkaConfig, var limitBytes = fetchMaxBytes val result = new mutable.ArrayBuffer[(TopicAndPartition, LogReadResult)] - var minOneMessage = true + var minOneMessage = !hardMaxBytesLimit readPartitionInfo.foreach { case (tp, fetchInfo) => val readResult = read(tp, fetchInfo, limitBytes, minOneMessage) val messageSetSize = readResult.info.messageSet.sizeInBytes + // Once we read from a non-empty partition, we stop ignoring request and partition level size limits if (messageSetSize > 0) minOneMessage = false limitBytes = math.max(0, limitBytes - messageSetSize) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index b05e55964658..b86c0b7452e7 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -615,7 +615,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { @Test def testLowMaxFetchSizeForRequestAndPartition(): Unit = { // one of the effects of this is that there will be some log reads where `0 > remaining limit bytes < message size` - // and we don't return the message because it's not the first message in the first partition of the fetch + // and we don't return the message because it's not the first message in the first non-empty partition of the fetch // this behaves a little different than when remaining limit bytes is 0 and it's important to test it this.consumerConfig.setProperty(ConsumerConfig.FETCH_MAX_BYTES_CONFIG, "500") this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, "100") diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index abc9a9647854..791a09463324 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -290,13 +290,17 @@ class LogTest extends JUnitSuite { assignOffsets = false) for (i <- 50 until messageIds.max) { - val messageSets = Seq( - log.read(i, 1), - log.read(i, 0) - ).map(_.messageSet) - messageSets.foreach { messageSet => - assertEquals(MessageSet.Empty, messageSet) - } + assertEquals(MessageSet.Empty, log.read(i, 0).messageSet) + + // we return an incomplete message instead of an empty one for the case below + // we use this mechanism to tell consumers of the fetch request version 2 and below that the message size is + // larger than the fetch size + // in fetch request version 3, we no longer need this as we return oversized messages from the first non-empty + // partition + val fetchInfo = log.read(i, 1) + assertTrue(fetchInfo.messageSetIncomplete) + assertTrue(fetchInfo.messageSet.isInstanceOf[FileMessageSet]) + assertEquals(1, fetchInfo.messageSet.sizeInBytes) } } diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala index 1a09c9928a76..bb0e06003f81 100644 --- a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala @@ -24,7 +24,7 @@ import kafka.utils.TestUtils import kafka.utils.TestUtils._ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.protocol.{ApiKeys, Errors, ProtoUtils} import org.apache.kafka.common.record.{LogEntry, MemoryRecords} import org.apache.kafka.common.requests.{FetchRequest, FetchResponse} import org.apache.kafka.common.serialization.StringSerializer @@ -53,17 +53,22 @@ class FetchRequestTest extends BaseRequestTest { super.tearDown() } + private def createFetchRequest(maxResponseBytes: Int, maxPartitionBytes: Int, topicPartitions: Seq[TopicPartition], - offsetMap: Map[TopicPartition, Long] = Map.empty): FetchRequest = { + offsetMap: Map[TopicPartition, Long] = Map.empty): FetchRequest = + new FetchRequest(Int.MaxValue, 0, maxResponseBytes, createPartitionMap(maxPartitionBytes, topicPartitions, offsetMap)) + + private def createPartitionMap(maxPartitionBytes: Int, topicPartitions: Seq[TopicPartition], + offsetMap: Map[TopicPartition, Long] = Map.empty): util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData] = { val partitionMap = new util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData] topicPartitions.foreach { tp => partitionMap.put(tp, new FetchRequest.PartitionData(offsetMap.getOrElse(tp, 0), maxPartitionBytes)) } - new FetchRequest(Int.MaxValue, 0, maxResponseBytes, partitionMap) + partitionMap } - private def sendFetchRequest(leaderId: Int, request: FetchRequest): FetchResponse = { - val response = send(request, ApiKeys.FETCH, destination = brokerSocketServer(leaderId)) + private def sendFetchRequest(leaderId: Int, request: FetchRequest, version: Option[Short] = None): FetchResponse = { + val response = send(request, ApiKeys.FETCH, version, destination = brokerSocketServer(leaderId)) FetchResponse.parse(response) } @@ -71,7 +76,7 @@ class FetchRequestTest extends BaseRequestTest { def testBrokerRespectsPartitionsOrderAndSizeLimits(): Unit = { val messagesPerPartition = 9 val maxResponseBytes = 800 - val maxPartitionBytes = 200 + val maxPartitionBytes = 190 def createFetchRequest(topicPartitions: Seq[TopicPartition], offsetMap: Map[TopicPartition, Long] = Map.empty): FetchRequest = this.createFetchRequest(maxResponseBytes, maxPartitionBytes, topicPartitions, offsetMap) @@ -124,6 +129,7 @@ class FetchRequestTest extends BaseRequestTest { val size3 = logEntries(partitionData3).map(_.size).sum assertTrue(s"Expected $size3 to be smaller than $maxResponseBytes", size3 <= maxResponseBytes) assertTrue(s"Expected $size3 to be larger than $maxPartitionBytes", size3 > maxPartitionBytes) + assertTrue(maxPartitionBytes < MemoryRecords.readableRecords(partitionData3.recordSet).sizeInBytes) // 4. Partition with message larger than the response limit at the start of the list val shuffledTopicPartitions4 = Seq(partitionWithLargeMessage2, partitionWithLargeMessage1) ++ @@ -139,7 +145,23 @@ class FetchRequestTest extends BaseRequestTest { assertEquals(Errors.NONE.code, partitionData4.errorCode) assertTrue(partitionData4.highWatermark > 0) val size4 = logEntries(partitionData4).map(_.size).sum - assertTrue(s"Expected $size4 to be larger than $maxResponseBytes}", size4 > maxResponseBytes) + assertTrue(s"Expected $size4 to be larger than $maxResponseBytes", size4 > maxResponseBytes) + assertTrue(maxResponseBytes < MemoryRecords.readableRecords(partitionData4.recordSet).sizeInBytes) + } + + @Test + def testFetchRequestV2WithOversizedMessage(): Unit = { + val maxPartitionBytes = 200 + val (topicPartition, leaderId) = createTopics(numTopics = 1, numPartitions = 1).head + producer.send(new ProducerRecord(topicPartition.topic, topicPartition.partition, + "key", new String(new Array[Byte](maxPartitionBytes + 1)))).get + val fetchRequest = new FetchRequest(Int.MaxValue, 0, createPartitionMap(maxPartitionBytes, Seq(topicPartition))) + val fetchResponse = sendFetchRequest(leaderId, fetchRequest, Some(2)) + val partitionData = fetchResponse.responseData.get(topicPartition) + assertEquals(Errors.NONE.code, partitionData.errorCode) + assertTrue(partitionData.highWatermark > 0) + assertEquals(maxPartitionBytes, MemoryRecords.readableRecords(partitionData.recordSet).sizeInBytes) + assertEquals(0, logEntries(partitionData).map(_.size).sum) } private def logEntries(partitionData: FetchResponse.PartitionData): Seq[LogEntry] = { @@ -152,29 +174,37 @@ class FetchRequestTest extends BaseRequestTest { assertEquals(expectedPartitions, fetchResponse.responseData.keySet.asScala.toSeq) var emptyResponseSeen = false var responseSize = 0 + var responseBufferSize = 0 expectedPartitions.foreach { tp => val partitionData = fetchResponse.responseData.get(tp) assertEquals(Errors.NONE.code, partitionData.errorCode) assertTrue(partitionData.highWatermark > 0) - val messages = logEntries(partitionData) + + val memoryRecords = MemoryRecords.readableRecords(partitionData.recordSet) + responseBufferSize += memoryRecords.sizeInBytes + + val messages = memoryRecords.iterator.asScala.toIndexedSeq assertTrue(messages.size < numMessagesPerPartition) - val size = messages.map(_.size).sum - responseSize += size - if (size == 0 && !emptyResponseSeen) + val messagesSize = messages.map(_.size).sum + responseSize += messagesSize + if (messagesSize == 0 && !emptyResponseSeen) { + assertEquals(0, memoryRecords.sizeInBytes) emptyResponseSeen = true - else if (size != 0 && !emptyResponseSeen) { - assertTrue(size <= maxPartitionBytes) - // tolerance below may have to be tweaked if the message sizes in the test change - assertTrue(size > maxPartitionBytes - 50) } - else if (size != 0 && emptyResponseSeen) - fail(s"Expected partition with size 0, but found $tp with size $size") + else if (messagesSize != 0 && !emptyResponseSeen) { + assertTrue(messagesSize <= maxPartitionBytes) + assertEquals(maxPartitionBytes, memoryRecords.sizeInBytes) + } + else if (messagesSize != 0 && emptyResponseSeen) + fail(s"Expected partition with size 0, but found $tp with size $messagesSize") + else if (memoryRecords.sizeInBytes != 0 && emptyResponseSeen) + fail(s"Expected partition buffer with size 0, but found $tp with size ${memoryRecords.sizeInBytes}") + } + assertEquals(maxResponseBytes - maxResponseBytes % maxPartitionBytes, responseBufferSize) assertTrue(responseSize <= maxResponseBytes) - // tolerance below may have to be tweaked if the message sizes in the test change - assertTrue(responseSize > maxResponseBytes - 50) } private def createTopics(numTopics: Int, numPartitions: Int): Map[TopicPartition, Int] = { diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala index 82d0fd4ad64d..f8f8ddad35a1 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala @@ -54,7 +54,7 @@ class ReplicaManagerQuotasTest { expect(quota.isQuotaExceeded()).andReturn(true).once() replay(quota) - val fetch = replicaManager.readFromLocalLog(true, true, Int.MaxValue, fetchInfo, quota) + val fetch = replicaManager.readFromLocalLog(true, true, Int.MaxValue, false, fetchInfo, quota) assertEquals("Given two partitions, with only one throttled, we should get the first", 1, fetch.find(_._1 == topicAndPartition1).get._2.info.messageSet.size) @@ -71,7 +71,7 @@ class ReplicaManagerQuotasTest { expect(quota.isQuotaExceeded()).andReturn(true).once() replay(quota) - val fetch = replicaManager.readFromLocalLog(true, true, Int.MaxValue, fetchInfo, quota) + val fetch = replicaManager.readFromLocalLog(true, true, Int.MaxValue, false, fetchInfo, quota) assertEquals("Given two partitions, with both throttled, we should get no messages", 0, fetch.find(_._1 == topicAndPartition1).get._2.info.messageSet.size) assertEquals("Given two partitions, with both throttled, we should get no messages", 0, @@ -87,7 +87,7 @@ class ReplicaManagerQuotasTest { expect(quota.isQuotaExceeded()).andReturn(false).once() replay(quota) - val fetch = replicaManager.readFromLocalLog(true, true, Int.MaxValue, fetchInfo, quota) + val fetch = replicaManager.readFromLocalLog(true, true, Int.MaxValue, false, fetchInfo, quota) assertEquals("Given two partitions, with both non-throttled, we should get both messages", 1, fetch.find(_._1 == topicAndPartition1).get._2.info.messageSet.size) assertEquals("Given two partitions, with both non-throttled, we should get both messages", 1, diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 0a0b45817b10..ca8d7129d09b 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -172,6 +172,7 @@ class ReplicaManagerTest { replicaId = -1, fetchMinBytes = 100000, fetchMaxBytes = Int.MaxValue, + hardMaxBytesLimit = false, fetchInfos = Seq(new TopicAndPartition(topic, 0) -> new PartitionFetchInfo(0, 100000)), responseCallback = fetchCallback) @@ -242,6 +243,7 @@ class ReplicaManagerTest { replicaId = 1, fetchMinBytes = 0, fetchMaxBytes = Int.MaxValue, + hardMaxBytesLimit = false, fetchInfos = Seq(new TopicAndPartition(topic, 0) -> new PartitionFetchInfo(1, 100000)), responseCallback = fetchCallback) @@ -257,6 +259,7 @@ class ReplicaManagerTest { replicaId = -1, fetchMinBytes = 0, fetchMaxBytes = Int.MaxValue, + hardMaxBytesLimit = false, fetchInfos = Seq(new TopicAndPartition(topic, 0) -> new PartitionFetchInfo(1, 100000)), responseCallback = fetchCallback) diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index 96d953f732fd..71c2b41bd93f 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -149,9 +149,9 @@ class SimpleFetchTest { val initialAllTopicsCount = BrokerTopicStats.getBrokerAllTopicsStats().totalFetchRequestRate.count() assertEquals("Reading committed data should return messages only up to high watermark", messagesToHW, - replicaManager.readFromLocalLog(true, true, Int.MaxValue, fetchInfo, UnboundedQuota).find(_._1 == topicAndPartition).get._2.info.messageSet.head.message) + replicaManager.readFromLocalLog(true, true, Int.MaxValue, false, fetchInfo, UnboundedQuota).find(_._1 == topicAndPartition).get._2.info.messageSet.head.message) assertEquals("Reading any data can return messages up to the end of the log", messagesToLEO, - replicaManager.readFromLocalLog(true, false, Int.MaxValue, fetchInfo, UnboundedQuota).find(_._1 == topicAndPartition).get._2.info.messageSet.head.message) + replicaManager.readFromLocalLog(true, false, Int.MaxValue, false, fetchInfo, UnboundedQuota).find(_._1 == topicAndPartition).get._2.info.messageSet.head.message) assertEquals("Counts should increment after fetch", initialTopicCount+2, BrokerTopicStats.getBrokerTopicStats(topic).totalFetchRequestRate.count()) assertEquals("Counts should increment after fetch", initialAllTopicsCount+2, BrokerTopicStats.getBrokerAllTopicsStats().totalFetchRequestRate.count()) From 6f1df09912bf4da9a59e4cb5c76c2753e5f4103b Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Sat, 17 Sep 2016 21:52:58 +0100 Subject: [PATCH 50/54] Rename `messageSetIncomplete` to `firstMessageSetIncomplete` As per Jun's suggestion. --- core/src/main/scala/kafka/log/LogSegment.scala | 4 ++-- core/src/main/scala/kafka/server/FetchDataInfo.scala | 2 +- core/src/main/scala/kafka/server/ReplicaManager.scala | 2 +- core/src/test/scala/unit/kafka/log/LogTest.scala | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index 44a761132421..0eb433048051 100755 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -176,7 +176,7 @@ class LogSegment(val log: FileMessageSet, // true high watermark in the previous leader for a short window. In this window, if a consumer fetches on an // offset between new leader's high watermark and the log end offset, we want to return an empty response. if (offset < startOffset) - return FetchDataInfo(offsetMetadata, MessageSet.Empty, messageSetIncomplete = false) + return FetchDataInfo(offsetMetadata, MessageSet.Empty, firstMessageSetIncomplete = false) val mapping = translateOffset(offset, startPosition.position) val endPosition = if (mapping == null) @@ -187,7 +187,7 @@ class LogSegment(val log: FileMessageSet, } FetchDataInfo(offsetMetadata, log.read(startPosition.position, length), - messageSetIncomplete = adjustedMaxSize < messageSetSize) + firstMessageSetIncomplete = adjustedMaxSize < messageSetSize) } /** diff --git a/core/src/main/scala/kafka/server/FetchDataInfo.scala b/core/src/main/scala/kafka/server/FetchDataInfo.scala index 0a4d6b861452..9d6d4378a36b 100644 --- a/core/src/main/scala/kafka/server/FetchDataInfo.scala +++ b/core/src/main/scala/kafka/server/FetchDataInfo.scala @@ -20,4 +20,4 @@ package kafka.server import kafka.message.MessageSet case class FetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata, messageSet: MessageSet, - messageSetIncomplete: Boolean = false) + firstMessageSetIncomplete: Boolean = false) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 12f8768fc140..b4db22c2bc74 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -564,7 +564,7 @@ class ReplicaManager(val config: KafkaConfig, FetchDataInfo(fetch.fetchOffsetMetadata, MessageSet.Empty) // For FetchRequest version 3, we replace incomplete message sets with an empty one as consumers can make // progress in such cases and don't need to report a `RecordTooLargeException` - else if (!hardMaxBytesLimit && fetch.messageSetIncomplete) + else if (!hardMaxBytesLimit && fetch.firstMessageSetIncomplete) FetchDataInfo(fetch.fetchOffsetMetadata, MessageSet.Empty) else fetch diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 791a09463324..e496853e7e42 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -298,7 +298,7 @@ class LogTest extends JUnitSuite { // in fetch request version 3, we no longer need this as we return oversized messages from the first non-empty // partition val fetchInfo = log.read(i, 1) - assertTrue(fetchInfo.messageSetIncomplete) + assertTrue(fetchInfo.firstMessageSetIncomplete) assertTrue(fetchInfo.messageSet.isInstanceOf[FileMessageSet]) assertEquals(1, fetchInfo.messageSet.sizeInBytes) } From 31e158969ea49ae51536e1f9cad3c1a789b35ced Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Sat, 17 Sep 2016 21:54:26 +0100 Subject: [PATCH 51/54] Group exception clauses since the result is the same As per Jason's suggestion --- .../main/scala/kafka/server/ReplicaManager.scala | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index b4db22c2bc74..2c843e868095 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -25,7 +25,7 @@ import kafka.api._ import kafka.cluster.{Partition, Replica} import kafka.common._ import kafka.controller.KafkaController -import kafka.log.{FileMessageSet, LogAppendInfo, LogManager} +import kafka.log.{LogAppendInfo, LogManager} import kafka.message.{ByteBufferMessageSet, InvalidMessageException, Message, MessageSet} import kafka.metrics.KafkaMetricsGroup import kafka.server.QuotaFactory.UnboundedQuota @@ -579,14 +579,11 @@ class ReplicaManager(val config: KafkaConfig, } catch { // NOTE: Failed fetch requests metric is not incremented for known exceptions since it // is supposed to indicate un-expected failure of a broker in handling a fetch request - case utpe: UnknownTopicOrPartitionException => - LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, false, Some(utpe)) - case nle: NotLeaderForPartitionException => - LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, false, Some(nle)) - case rnae: ReplicaNotAvailableException => - LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, false, Some(rnae)) - case oor : OffsetOutOfRangeException => - LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, false, Some(oor)) + case e@ (_: UnknownTopicOrPartitionException | + _: NotLeaderForPartitionException | + _: ReplicaNotAvailableException | + _: OffsetOutOfRangeException) => + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, false, Some(e)) case e: Throwable => BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats().failedFetchRequestRate.mark() From bf6b4dbea57f682b7f907ad8778be65471509be1 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Sat, 17 Sep 2016 23:16:35 +0100 Subject: [PATCH 52/54] Mention that the way we group topics and partitions deviates from the optimal over time --- .../org/apache/kafka/common/internals/PartitionStates.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java b/clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java index 816017653c38..49823c099c6b 100644 --- a/clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java +++ b/clients/src/main/java/org/apache/kafka/common/internals/PartitionStates.java @@ -33,7 +33,10 @@ * size. Because the serialization of fetch requests is more efficient if all partitions for the same topic are grouped * together, we do such grouping in the method `set`. * - * As partitions are moved to the end, the same topic may be repeated more than once. + * As partitions are moved to the end, the same topic may be repeated more than once. In the optimal case, a single + * topic would "wrap around" and appear twice. However, as partitions are fetched in different orders and partition + * leadership changes, we will deviate from the optimal. If this turns out to be an issue in practice, we can improve + * it by tracking the partitions per node or calling `set` every so often. */ public class PartitionStates { From 3ba807b27b97ddf82adfef1394cb9ef1449900e6 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Sat, 17 Sep 2016 23:23:59 +0100 Subject: [PATCH 53/54] Restore replication warning due to oversized messages if fetch request <= 2 Suggested by Jun --- .../scala/kafka/server/ReplicaFetcherThread.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 66710a5d834c..78e00df75be5 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -121,6 +121,8 @@ class ReplicaFetcherThread(name: String, val replica = replicaMgr.getReplica(topic, partitionId).get val messageSet = partitionData.toByteBufferMessageSet + maybeWarnIfMessageOversized(messageSet, topicPartition) + if (fetchOffset != replica.logEndOffset.messageOffset) throw new RuntimeException("Offset mismatch for partition %s: fetched offset = %d, log end offset = %d.".format(topicPartition, fetchOffset, replica.logEndOffset.messageOffset)) if (logger.isTraceEnabled) @@ -147,6 +149,15 @@ class ReplicaFetcherThread(name: String, } } + def maybeWarnIfMessageOversized(messageSet: ByteBufferMessageSet, topicPartition: TopicPartition): Unit = { + // oversized messages don't cause replication to fail from fetch request version 3 (KIP-74) + if (fetchRequestVersion <= 2 && messageSet.sizeInBytes > 0 && messageSet.validBytes <= 0) + error(s"Replication is failing due to a message that is greater than replica.fetch.max.bytes for partition $topicPartition. " + + "This generally occurs when the max.message.bytes has been overridden to exceed this value and a suitably large " + + "message has also been sent. To fix this problem increase replica.fetch.max.bytes in your broker config to be " + + "equal or larger than your settings for max.message.bytes, both at a broker and topic level.") + } + /** * Handle a partition whose offset is out of range and return a new fetch offset. */ From 2edf038b055e058b6af70c8312d1d044b128de2a Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Sat, 17 Sep 2016 23:42:04 +0100 Subject: [PATCH 54/54] Improve protocol description for fetch request version 3 --- .../java/org/apache/kafka/common/protocol/Protocol.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 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 6d1259ff7990..d64cf6d12a49 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 @@ -457,10 +457,12 @@ public class Protocol { "Minimum bytes to accumulate in the response."), new Field("max_bytes", INT32, - "Maximum bytes to accumulate in the response."), + "Maximum bytes to accumulate in the response. Note that this is not an absolute maximum, " + + "if the first message in the first non-empty partition of the fetch is larger than this " + + "value, the message will still be returned to ensure that progress can be made."), new Field("topics", new ArrayOf(FETCH_REQUEST_TOPIC_V0), - "Topics to fetch.")); + "Topics to fetch in the order provided.")); public static final Schema FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", INT32,