From 795e262b9f7c9c046b397e670f35698f17d2c13d Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Tue, 30 Sep 2014 08:13:45 -0700 Subject: [PATCH 001/491] bump up release version to 0.8.3 --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index 8f6f2bf7bbd27..a04a6c8b98c89 100644 --- a/gradle.properties +++ b/gradle.properties @@ -14,7 +14,7 @@ # limitations under the License. group=org.apache.kafka -version=0.8.2-SNAPSHOT +version=0.8.3-SNAPSHOT scalaVersion=2.10.1 task=build From e05704c40b51b399e2143faa3dbb668daca9890c Mon Sep 17 00:00:00 2001 From: Joe Stein Date: Fri, 3 Oct 2014 22:04:36 -0400 Subject: [PATCH 002/491] KAFKA-1662 gradle release issue permgen space patch by Sriharsha Chintalapani reviewed by Joe Sein --- gradle.properties | 1 + 1 file changed, 1 insertion(+) diff --git a/gradle.properties b/gradle.properties index a04a6c8b98c89..5d3155fd44614 100644 --- a/gradle.properties +++ b/gradle.properties @@ -17,6 +17,7 @@ group=org.apache.kafka version=0.8.3-SNAPSHOT scalaVersion=2.10.1 task=build +org.gradle.jvmargs=-XX:MaxPermSize=512m -Xmx1024m mavenUrl= mavenUsername= From 9cb59f319e26a5ad0b2646f9718bdb90deb21656 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Sat, 4 Oct 2014 16:34:16 -0700 Subject: [PATCH 003/491] KAFKA-1057 Trim whitespaces from user specified configs; reviewed by Neha Narkhede --- core/src/main/scala/kafka/utils/VerifiableProperties.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/utils/VerifiableProperties.scala b/core/src/main/scala/kafka/utils/VerifiableProperties.scala index 2f95d540c5329..2ffc7f452dc7a 100644 --- a/core/src/main/scala/kafka/utils/VerifiableProperties.scala +++ b/core/src/main/scala/kafka/utils/VerifiableProperties.scala @@ -35,7 +35,7 @@ class VerifiableProperties(val props: Properties) extends Logging { def getProperty(name: String): String = { val value = props.getProperty(name) referenceSet.add(name) - return value + if(value == null) value else value.trim() } /** From 40948a33c93ab7b5bdc62c7cf680538e920afa4d Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Sat, 4 Oct 2014 18:09:16 -0700 Subject: [PATCH 004/491] KAFKA-1663 Controller unable to shutdown after a soft failure; reviewed by Neha Narkhede --- .../main/scala/kafka/controller/TopicDeletionManager.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala index 219c4136e905a..6ee3de2cb46d0 100644 --- a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala +++ b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala @@ -89,7 +89,6 @@ class TopicDeletionManager(controller: KafkaController, def start() { if (isDeleteTopicEnabled) { deleteTopicsThread = new DeleteTopicsThread() - deleteTopicStateChanged.set(true) deleteTopicsThread.start() } } @@ -214,7 +213,7 @@ class TopicDeletionManager(controller: KafkaController, */ private def awaitTopicDeletionNotification() { inLock(deleteLock) { - while(!deleteTopicsThread.isRunning.get() && !deleteTopicStateChanged.compareAndSet(true, false)) { + while(deleteTopicsThread.isRunning.get() && !deleteTopicStateChanged.compareAndSet(true, false)) { debug("Waiting for signal to start or continue topic deletion") deleteTopicsCond.await() } @@ -429,4 +428,3 @@ class TopicDeletionManager(controller: KafkaController, } } } - From f452c426bb562ffc5cc2db38665591d157d5c80a Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Mon, 6 Oct 2014 07:45:23 -0700 Subject: [PATCH 005/491] kafka-1673; potential java.lang.IllegalStateException in BufferPool.allocate(); patched by Jun Rao; reviewed by Jay Kreps --- .../clients/producer/internals/BufferPool.java | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java index 169a6568bf7ef..aa91e1444a49c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java @@ -125,14 +125,11 @@ public ByteBuffer allocate(int size) throws InterruptedException { // loop over and over until we have a buffer or have reserved // enough memory to allocate one while (accumulated < size) { - try { - long startWait = time.nanoseconds(); - moreMemory.await(300, TimeUnit.MILLISECONDS); - long endWait = time.nanoseconds(); - this.waitTime.record(endWait - startWait, time.milliseconds()); - } catch (InterruptedException e) { - // This should never happen. Just let it go. - } + long startWait = time.nanoseconds(); + moreMemory.await(); + long endWait = time.nanoseconds(); + this.waitTime.record(endWait - startWait, time.milliseconds()); + // check if we can satisfy this request from the free list, // otherwise allocate memory if (accumulated == 0 && size == this.poolableSize && !this.free.isEmpty()) { From 88595bfb5229bda0ad7c3de2ce3b04f95a9f38a7 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Mon, 6 Oct 2014 17:16:07 -0700 Subject: [PATCH 006/491] KAFKA-1668 Print an error if you try to alter a topic that doesn't exist. --- core/src/main/scala/kafka/admin/TopicCommand.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index 3b2166aa4e2e8..7672c5aab4fba 100644 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -96,6 +96,9 @@ object TopicCommand { def alterTopic(zkClient: ZkClient, opts: TopicCommandOptions) { val topics = getTopics(zkClient, opts) + if (topics.length == 0) { + println("Topic %s does not exist".format(opts.options.valueOf(opts.topicOpt))) + } topics.foreach { topic => val configs = AdminUtils.fetchTopicConfig(zkClient, topic) if(opts.options.has(opts.configOpt) || opts.options.has(opts.deleteConfigOpt)) { From 1a6c9207c7c19f04b91b455e86bbf4e779e7e7a2 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 6 Oct 2014 17:21:30 -0700 Subject: [PATCH 007/491] kafka-1468; (followup) add back inadventently removed commit; patched by Guozhang Wang; reviewed by Jun Rao --- core/src/main/scala/kafka/server/RequestPurgatory.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/RequestPurgatory.scala index cf3ed4c8f197d..9d76234bc2c81 100644 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ b/core/src/main/scala/kafka/server/RequestPurgatory.scala @@ -166,7 +166,7 @@ abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInt * bookkeeping logic. */ private class Watchers { - private val requests = new util.ArrayList[T] + private val requests = new util.LinkedList[T] // return the size of the watch list def watched() = requests.size() From 68b9f7716df1d994a9d43bec6bc42c90e66f1e99 Mon Sep 17 00:00:00 2001 From: Anton Karamanov Date: Tue, 7 Oct 2014 18:22:31 -0700 Subject: [PATCH 008/491] kafka-1644; Inherit FetchResponse from RequestOrResponse; patched by Anton Karamanov; reviewed by Jun Rao --- core/src/main/scala/kafka/api/FetchResponse.scala | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/api/FetchResponse.scala b/core/src/main/scala/kafka/api/FetchResponse.scala index af9308737bf78..8d085a1f18f80 100644 --- a/core/src/main/scala/kafka/api/FetchResponse.scala +++ b/core/src/main/scala/kafka/api/FetchResponse.scala @@ -152,7 +152,8 @@ object FetchResponse { case class FetchResponse(correlationId: Int, - data: Map[TopicAndPartition, FetchResponsePartitionData]) { + data: Map[TopicAndPartition, FetchResponsePartitionData]) + extends RequestOrResponse() { /** * Partitions the data into a map of maps (one for each topic). @@ -168,6 +169,16 @@ case class FetchResponse(correlationId: Int, folded + topicData.sizeInBytes }) + /* + * FetchResponse uses [sendfile](http://man7.org/linux/man-pages/man2/sendfile.2.html) + * api for data transfer, so `writeTo` aren't actually being used. + * It is implemented as an empty function to comform to `RequestOrResponse.writeTo` + * abstract method signature. + */ + def writeTo(buffer: ByteBuffer): Unit = throw new UnsupportedOperationException + + override def describe(details: Boolean): String = toString + private def partitionDataFor(topic: String, partition: Int): FetchResponsePartitionData = { val topicAndPartition = TopicAndPartition(topic, partition) data.get(topicAndPartition) match { From c940470e32916e2dbbe8f95bd295950a3681c5b7 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Thu, 9 Oct 2014 08:05:32 -0700 Subject: [PATCH 009/491] kafka-1670; Corrupt log files for segment.bytes values close to Int.MaxInt; patched by Sriharsha Chintalapani; reviewed by Jay Kreps and Jun Rao --- .../errors/RecordBatchTooLargeException.java | 39 +++++++++++++++++++ .../apache/kafka/common/protocol/Errors.java | 3 +- .../scala/kafka/common/ErrorMapping.scala | 5 ++- .../MessageSetSizeTooLargeException.scala | 22 +++++++++++ core/src/main/scala/kafka/log/Log.scala | 27 ++++++++++--- .../scala/unit/kafka/log/LogManagerTest.scala | 2 +- .../test/scala/unit/kafka/log/LogTest.scala | 34 +++++++++++----- .../unit/kafka/server/LogOffsetTest.scala | 8 ++-- 8 files changed, 117 insertions(+), 23 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/RecordBatchTooLargeException.java create mode 100644 core/src/main/scala/kafka/common/MessageSetSizeTooLargeException.scala diff --git a/clients/src/main/java/org/apache/kafka/common/errors/RecordBatchTooLargeException.java b/clients/src/main/java/org/apache/kafka/common/errors/RecordBatchTooLargeException.java new file mode 100644 index 0000000000000..f3f3f27d34dfb --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/RecordBatchTooLargeException.java @@ -0,0 +1,39 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.common.errors; + +/** + * This record batch is larger than the maximum allowable size + */ +public class RecordBatchTooLargeException extends ApiException { + + private static final long serialVersionUID = 1L; + + public RecordBatchTooLargeException() { + super(); + } + + public RecordBatchTooLargeException(String message, Throwable cause) { + super(message, cause); + } + + public RecordBatchTooLargeException(String message) { + super(message); + } + + public RecordBatchTooLargeException(Throwable cause) { + super(cause); + } + +} + diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index d434f420ad634..d5f5de32bbed6 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -43,7 +43,8 @@ public enum Errors { OFFSET_METADATA_TOO_LARGE(12, new OffsetMetadataTooLarge("The metadata field of the offset request was too large.")), NETWORK_EXCEPTION(13, new NetworkException("The server disconnected before a response was received.")), // TODO: errorCode 14, 15, 16 - INVALID_TOPIC_EXCEPTION(17, new InvalidTopicException("The request attempted to perform an operation on an invalid topic.")); + INVALID_TOPIC_EXCEPTION(17, new InvalidTopicException("The request attempted to perform an operation on an invalid topic.")), + RECORD_LIST_TOO_LARGE(18, new RecordBatchTooLargeException("The request included message batch larger than the configured segment size on the server.")); private static Map, Errors> classToError = new HashMap, Errors>(); private static Map codeToError = new HashMap(); diff --git a/core/src/main/scala/kafka/common/ErrorMapping.scala b/core/src/main/scala/kafka/common/ErrorMapping.scala index 3fae7910e4ce1..a190607a3bb60 100644 --- a/core/src/main/scala/kafka/common/ErrorMapping.scala +++ b/core/src/main/scala/kafka/common/ErrorMapping.scala @@ -19,7 +19,6 @@ package kafka.common import kafka.message.InvalidMessageException import java.nio.ByteBuffer -import java.lang.Throwable import scala.Predef._ /** @@ -47,6 +46,7 @@ object ErrorMapping { val ConsumerCoordinatorNotAvailableCode: Short = 15 val NotCoordinatorForConsumerCode: Short = 16 val InvalidTopicCode : Short = 17 + val MessageSetSizeTooLargeCode: Short = 18 private val exceptionToCode = Map[Class[Throwable], Short]( @@ -65,7 +65,8 @@ object ErrorMapping { classOf[OffsetsLoadInProgressException].asInstanceOf[Class[Throwable]] -> OffsetsLoadInProgressCode, classOf[ConsumerCoordinatorNotAvailableException].asInstanceOf[Class[Throwable]] -> ConsumerCoordinatorNotAvailableCode, classOf[NotCoordinatorForConsumerException].asInstanceOf[Class[Throwable]] -> NotCoordinatorForConsumerCode, - classOf[InvalidTopicException].asInstanceOf[Class[Throwable]] -> InvalidTopicCode + classOf[InvalidTopicException].asInstanceOf[Class[Throwable]] -> InvalidTopicCode, + classOf[MessageSetSizeTooLargeException].asInstanceOf[Class[Throwable]] -> MessageSetSizeTooLargeCode ).withDefaultValue(UnknownCode) /* invert the mapping */ diff --git a/core/src/main/scala/kafka/common/MessageSetSizeTooLargeException.scala b/core/src/main/scala/kafka/common/MessageSetSizeTooLargeException.scala new file mode 100644 index 0000000000000..94a616ed3972a --- /dev/null +++ b/core/src/main/scala/kafka/common/MessageSetSizeTooLargeException.scala @@ -0,0 +1,22 @@ +/** + * 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.common + +class MessageSetSizeTooLargeException(message: String) extends RuntimeException(message) { + def this() = this(null) +} diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 0ddf97bd30311..a123cdc52f341 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -252,9 +252,6 @@ class Log(val dir: File, lock synchronized { appendInfo.firstOffset = nextOffsetMetadata.messageOffset - // maybe roll the log if this segment is full - val segment = maybeRoll() - if(assignOffsets) { // assign offsets to the message set val offset = new AtomicLong(nextOffsetMetadata.messageOffset) @@ -282,6 +279,16 @@ class Log(val dir: File, } } + // check messages set size may be exceed config.segmentSize + if(validMessages.sizeInBytes > config.segmentSize) { + throw new MessageSetSizeTooLargeException("Message set size is %d bytes which exceeds the maximum configured segment size of %d." + .format(validMessages.sizeInBytes, config.segmentSize)) + } + + + // maybe roll the log if this segment is full + val segment = maybeRoll(validMessages.sizeInBytes) + // now append to the log segment.append(appendInfo.firstOffset, validMessages) @@ -489,12 +496,20 @@ class Log(val dir: File, def logEndOffset: Long = nextOffsetMetadata.messageOffset /** - * Roll the log over to a new empty log segment if necessary + * Roll the log over to a new empty log segment if necessary. + * + * @param messagesSize The messages set size in bytes + * logSegment will be rolled if one of the following conditions met + *
    + *
  1. The logSegment is full + *
  2. The maxTime has elapsed + *
  3. The index is full + *
* @return The currently active segment after (perhaps) rolling to a new segment */ - private def maybeRoll(): LogSegment = { + private def maybeRoll(messagesSize: Int): LogSegment = { val segment = activeSegment - if (segment.size > config.segmentSize || + if (segment.size > config.segmentSize - messagesSize || segment.size > 0 && time.milliseconds - segment.created > config.segmentMs || segment.index.isFull) { debug("Rolling new log segment in %s (log_size = %d/%d, index_size = %d/%d, age_ms = %d/%d)." diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 59bd8a981b3fb..90cd53033fafa 100644 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -114,7 +114,7 @@ class LogManagerTest extends JUnit3Suite { val setSize = TestUtils.singleMessageSet("test".getBytes()).sizeInBytes logManager.shutdown() - val config = logConfig.copy(segmentSize = 10 * (setSize - 1), retentionSize = 5L * 10L * setSize + 10L) + val config = logConfig.copy(segmentSize = 10 * setSize, retentionSize = 5L * 10L * setSize + 10L) logManager = createLogManager() logManager.startup diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 577d102fc2eb6..a0cbd3bbbeeab 100644 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -18,15 +18,13 @@ package kafka.log import java.io._ -import java.util.ArrayList import java.util.concurrent.atomic._ import junit.framework.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.{After, Before, Test} import kafka.message._ -import kafka.common.{MessageSizeTooLargeException, OffsetOutOfRangeException} +import kafka.common.{MessageSizeTooLargeException, OffsetOutOfRangeException, MessageSetSizeTooLargeException} import kafka.utils._ -import scala.Some import kafka.server.KafkaConfig class LogTest extends JUnitSuite { @@ -239,7 +237,7 @@ class LogTest extends JUnitSuite { @Test def testCompressedMessages() { /* this log should roll after every messageset */ - val log = new Log(logDir, logConfig.copy(segmentSize = 10), recoveryPoint = 0L, time.scheduler, time = time) + val log = new Log(logDir, logConfig.copy(segmentSize = 100), recoveryPoint = 0L, time.scheduler, time = time) /* append 2 compressed message sets, each with two messages giving offsets 0, 1, 2, 3 */ log.append(new ByteBufferMessageSet(DefaultCompressionCodec, new Message("hello".getBytes), new Message("there".getBytes))) @@ -286,7 +284,26 @@ class LogTest extends JUnitSuite { } /** - * We have a max size limit on message appends, check that it is properly enforced by appending a message larger than the + * MessageSet size shouldn't exceed the config.segmentSize, check that it is properly enforced by + * appending a message set larger than the config.segmentSize setting and checking that an exception is thrown. + */ + @Test + def testMessageSetSizeCheck() { + val messageSet = new ByteBufferMessageSet(NoCompressionCodec, new Message ("You".getBytes), new Message("bethe".getBytes)) + // append messages to log + val configSegmentSize = messageSet.sizeInBytes - 1 + val log = new Log(logDir, logConfig.copy(segmentSize = configSegmentSize), recoveryPoint = 0L, time.scheduler, time = time) + + try { + log.append(messageSet) + fail("message set should throw MessageSetSizeTooLargeException.") + } catch { + case e: MessageSetSizeTooLargeException => // this is good + } + } + + /** + * We have a max size limit on message appends, check that it is properly enforced by appending a message larger than the * setting and checking that an exception is thrown. */ @Test @@ -305,10 +322,9 @@ class LogTest extends JUnitSuite { log.append(second) fail("Second message set should throw MessageSizeTooLargeException.") } catch { - case e: MessageSizeTooLargeException => // this is good + case e: MessageSizeTooLargeException => // this is good } } - /** * Append a bunch of messages to a log and then re-open it both with and without recovery and check that the log re-initializes correctly. */ @@ -375,7 +391,7 @@ class LogTest extends JUnitSuite { val set = TestUtils.singleMessageSet("test".getBytes()) val setSize = set.sizeInBytes val msgPerSeg = 10 - val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages + val segmentSize = msgPerSeg * setSize // each segment will be 10 messages // create a log val log = new Log(logDir, logConfig.copy(segmentSize = segmentSize), recoveryPoint = 0L, scheduler = time.scheduler, time = time) @@ -429,7 +445,7 @@ class LogTest extends JUnitSuite { val set = TestUtils.singleMessageSet("test".getBytes()) val setSize = set.sizeInBytes val msgPerSeg = 10 - val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages + val segmentSize = msgPerSeg * setSize // each segment will be 10 messages val config = logConfig.copy(segmentSize = segmentSize) val log = new Log(logDir, config, recoveryPoint = 0L, scheduler = time.scheduler, time = time) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index 9556ed92c61ff..c06ee756bf0fe 100644 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -92,7 +92,7 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { log.flush() val offsets = server.apis.fetchOffsets(logManager, TopicAndPartition(topic, part), OffsetRequest.LatestTime, 10) - assertEquals(Seq(20L, 16L, 12L, 8L, 4L, 0L), offsets) + assertEquals(Seq(20L, 18L, 15L, 12L, 9L, 6L, 3L, 0), offsets) waitUntilTrue(() => isLeaderLocalOnBroker(topic, part, server), "Leader should be elected") val topicAndPartition = TopicAndPartition(topic, part) @@ -101,7 +101,7 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { replicaId = 0) val consumerOffsets = simpleConsumer.getOffsetsBefore(offsetRequest).partitionErrorAndOffsets(topicAndPartition).offsets - assertEquals(Seq(20L, 16L, 12L, 8L, 4L, 0L), consumerOffsets) + assertEquals(Seq(20L, 18L, 15L, 12L, 9L, 6L, 3L, 0), consumerOffsets) // try to fetch using latest offset val fetchResponse = simpleConsumer.fetch( @@ -155,14 +155,14 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { val now = time.milliseconds + 30000 // pretend it is the future to avoid race conditions with the fs val offsets = server.apis.fetchOffsets(logManager, TopicAndPartition(topic, part), now, 10) - assertEquals(Seq(20L, 16L, 12L, 8L, 4L, 0L), offsets) + assertEquals(Seq(20L, 18L, 15L, 12L, 9L, 6L, 3L, 0L), offsets) waitUntilTrue(() => isLeaderLocalOnBroker(topic, part, server), "Leader should be elected") val topicAndPartition = TopicAndPartition(topic, part) val offsetRequest = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(now, 10)), replicaId = 0) val consumerOffsets = simpleConsumer.getOffsetsBefore(offsetRequest).partitionErrorAndOffsets(topicAndPartition).offsets - assertEquals(Seq(20L, 16L, 12L, 8L, 4L, 0L), consumerOffsets) + assertEquals(Seq(20L, 18L, 15L, 12L, 9L, 6L, 3L, 0L), consumerOffsets) } @Test From a7e49298b02ae35c227a40deae63debeae91138f Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Thu, 9 Oct 2014 14:29:21 -0700 Subject: [PATCH 010/491] kafka-1681; Newly elected KafkaController might not start deletion of pending topics; patched by Sriharsha Chintalapani; reviewed by Jun Rao --- core/src/main/scala/kafka/controller/TopicDeletionManager.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala index 6ee3de2cb46d0..e56f22d739906 100644 --- a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala +++ b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala @@ -89,6 +89,8 @@ class TopicDeletionManager(controller: KafkaController, def start() { if (isDeleteTopicEnabled) { deleteTopicsThread = new DeleteTopicsThread() + if (topicsToBeDeleted.size > 0) + deleteTopicStateChanged.set(true) deleteTopicsThread.start() } } From a314461fa04b3249608607d1040c8e0cf8724bac Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Thu, 9 Oct 2014 14:51:08 -0700 Subject: [PATCH 011/491] kafka-1558; AdminUtils.deleteTopic does not work; patched by Sriharsha Chintalapani; reviewed by Jun Rao --- core/src/main/scala/kafka/controller/KafkaController.scala | 3 +-- .../main/scala/kafka/controller/PartitionStateMachine.scala | 4 ++-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 8ab4a1b8072c9..381c52fcc73c7 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -1300,8 +1300,7 @@ class PreferredReplicaElectionListener(controller: KafkaController) extends IZkD error("Skipping preferred replica election for partitions %s since the respective topics are being deleted" .format(partitionsForTopicsToBeDeleted)) } - else - controller.onPreferredReplicaElection(partitions -- partitionsForTopicsToBeDeleted) + controller.onPreferredReplicaElection(partitions -- partitionsForTopicsToBeDeleted) } } diff --git a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala index e20b63a6ec1c1..2f0694bc5cdfc 100644 --- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala +++ b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala @@ -484,8 +484,6 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { topicsToBeDeleted --= nonExistentTopics if(topicsToBeDeleted.size > 0) { info("Starting topic deletion for topics " + topicsToBeDeleted.mkString(",")) - // add topic to deletion list - controller.deleteTopicManager.enqueueTopicsForDeletion(topicsToBeDeleted) // mark topic ineligible for deletion if other state changes are in progress topicsToBeDeleted.foreach { topic => val preferredReplicaElectionInProgress = @@ -495,6 +493,8 @@ class PartitionStateMachine(controller: KafkaController) extends Logging { if(preferredReplicaElectionInProgress || partitionReassignmentInProgress) controller.deleteTopicManager.markTopicIneligibleForDeletion(Set(topic)) } + // add topic to deletion list + controller.deleteTopicManager.enqueueTopicsForDeletion(topicsToBeDeleted) } } } From 58e58529b350a3da860b1f51fdfa356dfc42761f Mon Sep 17 00:00:00 2001 From: Mayuresh Gharat Date: Thu, 9 Oct 2014 16:34:40 -0700 Subject: [PATCH 012/491] KAFKA-1648; Robin consumer balance throws an NPE when there are no topics --- .../kafka/consumer/PartitionAssignor.scala | 62 ++++++++++--------- .../consumer/PartitionAssignorTest.scala | 2 +- 2 files changed, 33 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/kafka/consumer/PartitionAssignor.scala b/core/src/main/scala/kafka/consumer/PartitionAssignor.scala index 8ea7368dc394a..e6ff7683a0df4 100644 --- a/core/src/main/scala/kafka/consumer/PartitionAssignor.scala +++ b/core/src/main/scala/kafka/consumer/PartitionAssignor.scala @@ -71,39 +71,41 @@ class RoundRobinAssignor() extends PartitionAssignor with Logging { def assign(ctx: AssignmentContext) = { val partitionOwnershipDecision = collection.mutable.Map[TopicAndPartition, ConsumerThreadId]() - // check conditions (a) and (b) - val (headTopic, headThreadIdSet) = (ctx.consumersForTopic.head._1, ctx.consumersForTopic.head._2.toSet) - ctx.consumersForTopic.foreach { case (topic, threadIds) => - val threadIdSet = threadIds.toSet - require(threadIdSet == headThreadIdSet, - "Round-robin assignment is allowed only if all consumers in the group subscribe to the same topics, " + - "AND if the stream counts across topics are identical for a given consumer instance.\n" + - "Topic %s has the following available consumer streams: %s\n".format(topic, threadIdSet) + - "Topic %s has the following available consumer streams: %s\n".format(headTopic, headThreadIdSet)) - } + if (ctx.consumersForTopic.size > 0) { + // check conditions (a) and (b) + val (headTopic, headThreadIdSet) = (ctx.consumersForTopic.head._1, ctx.consumersForTopic.head._2.toSet) + ctx.consumersForTopic.foreach { case (topic, threadIds) => + val threadIdSet = threadIds.toSet + require(threadIdSet == headThreadIdSet, + "Round-robin assignment is allowed only if all consumers in the group subscribe to the same topics, " + + "AND if the stream counts across topics are identical for a given consumer instance.\n" + + "Topic %s has the following available consumer streams: %s\n".format(topic, threadIdSet) + + "Topic %s has the following available consumer streams: %s\n".format(headTopic, headThreadIdSet)) + } - val threadAssignor = Utils.circularIterator(headThreadIdSet.toSeq.sorted) + val threadAssignor = Utils.circularIterator(headThreadIdSet.toSeq.sorted) + + info("Starting round-robin assignment with consumers " + ctx.consumers) + val allTopicPartitions = ctx.partitionsForTopic.flatMap { case (topic, partitions) => + info("Consumer %s rebalancing the following partitions for topic %s: %s" + .format(ctx.consumerId, topic, partitions)) + partitions.map(partition => { + TopicAndPartition(topic, partition) + }) + }.toSeq.sortWith((topicPartition1, topicPartition2) => { + /* + * Randomize the order by taking the hashcode to reduce the likelihood of all partitions of a given topic ending + * up on one consumer (if it has a high enough stream count). + */ + topicPartition1.toString.hashCode < topicPartition2.toString.hashCode + }) - info("Starting round-robin assignment with consumers " + ctx.consumers) - val allTopicPartitions = ctx.partitionsForTopic.flatMap { case(topic, partitions) => - info("Consumer %s rebalancing the following partitions for topic %s: %s" - .format(ctx.consumerId, topic, partitions)) - partitions.map(partition => { - TopicAndPartition(topic, partition) + allTopicPartitions.foreach(topicPartition => { + val threadId = threadAssignor.next() + if (threadId.consumer == ctx.consumerId) + partitionOwnershipDecision += (topicPartition -> threadId) }) - }.toSeq.sortWith((topicPartition1, topicPartition2) => { - /* - * Randomize the order by taking the hashcode to reduce the likelihood of all partitions of a given topic ending - * up on one consumer (if it has a high enough stream count). - */ - topicPartition1.toString.hashCode < topicPartition2.toString.hashCode - }) - - allTopicPartitions.foreach(topicPartition => { - val threadId = threadAssignor.next() - if (threadId.consumer == ctx.consumerId) - partitionOwnershipDecision += (topicPartition -> threadId) - }) + } partitionOwnershipDecision } diff --git a/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala b/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala index 9ceae222ca5bf..24954de66ccc5 100644 --- a/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala @@ -87,7 +87,7 @@ private object PartitionAssignorTest extends Logging { private val MaxConsumerCount = 10 private val MaxStreamCount = 8 private val MaxTopicCount = 100 - private val MinTopicCount = 20 + private val MinTopicCount = 0 private val MaxPartitionCount = 120 private val MinPartitionCount = 8 From 043190c601f37e42c32189f3df5ddd986e57da95 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Thu, 9 Oct 2014 18:24:53 -0700 Subject: [PATCH 013/491] kafka-1555; provide strong consistency with reasonable availability; patched by Gwen Shapira; reviewed by Joel Koshy and Jun Rao --- .../clients/producer/ProducerConfig.java | 17 +++-- .../apache/kafka/common/config/ConfigDef.java | 42 +++++++++++++ ...NotEnoughReplicasAfterAppendException.java | 43 +++++++++++++ .../errors/NotEnoughReplicasException.java | 40 ++++++++++++ .../apache/kafka/common/protocol/Errors.java | 7 ++- .../main/scala/kafka/cluster/Partition.scala | 27 +++++++- .../scala/kafka/common/ErrorMapping.scala | 20 +++--- ...otEnoughReplicasAfterAppendException.scala | 27 ++++++++ .../common/NotEnoughReplicasException.scala | 25 ++++++++ core/src/main/scala/kafka/log/LogConfig.scala | 62 ++++++++++++------- .../kafka/producer/SyncProducerConfig.scala | 14 +++-- .../main/scala/kafka/server/KafkaApis.scala | 6 +- .../main/scala/kafka/server/KafkaConfig.scala | 5 ++ .../api/ProducerFailureHandlingTest.scala | 57 ++++++++++++++++- .../unit/kafka/producer/ProducerTest.scala | 27 ++++---- .../kafka/producer/SyncProducerTest.scala | 22 +++++++ .../scala/unit/kafka/utils/TestUtils.scala | 10 ++- 17 files changed, 384 insertions(+), 67 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasException.java create mode 100644 core/src/main/scala/kafka/common/NotEnoughReplicasAfterAppendException.scala create mode 100644 core/src/main/scala/kafka/common/NotEnoughReplicasException.scala diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index f9de4af426449..79d57f9bf3160 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -14,7 +14,10 @@ import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; import static org.apache.kafka.common.config.ConfigDef.Range.between; +import static org.apache.kafka.common.config.ConfigDef.ValidString.in; +import java.util.Arrays; +import java.util.List; import java.util.Map; import org.apache.kafka.common.config.AbstractConfig; @@ -77,7 +80,8 @@ public class ProducerConfig extends AbstractConfig { /** acks */ public static final String ACKS_CONFIG = "acks"; - private static final String ACKS_DOC = "The number of acknowledgments the producer requires the leader to have received before considering a request complete. This controls the " + " durability of records that are sent. The following settings are common: " + private static final String ACKS_DOC = "The number of acknowledgments the producer requires the leader to have received before considering a request complete. This controls the " + + " durability of records that are sent. The following settings are common: " + "
    " + "
  • acks=0 If set to zero then the producer will not wait for any acknowledgment from the" + " server at all. The record will be immediately added to the socket buffer and considered sent. No guarantee can be" @@ -89,9 +93,7 @@ public class ProducerConfig extends AbstractConfig { + " acknowledging the record but before the followers have replicated it then the record will be lost." + "
  • acks=all This means the leader will wait for the full set of in-sync replicas to" + " acknowledge the record. This guarantees that the record will not be lost as long as at least one in-sync replica" - + " remains alive. This is the strongest available guarantee." - + "
  • Other settings such as acks=2 are also possible, and will require the given number of" - + " acknowledgements but this is generally less useful."; + + " remains alive. This is the strongest available guarantee."; /** timeout.ms */ public static final String TIMEOUT_CONFIG = "timeout.ms"; @@ -175,7 +177,12 @@ public class ProducerConfig extends AbstractConfig { config = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, BOOSTRAP_SERVERS_DOC) .define(BUFFER_MEMORY_CONFIG, Type.LONG, 32 * 1024 * 1024L, atLeast(0L), Importance.HIGH, BUFFER_MEMORY_DOC) .define(RETRIES_CONFIG, Type.INT, 0, between(0, Integer.MAX_VALUE), Importance.HIGH, RETRIES_DOC) - .define(ACKS_CONFIG, Type.STRING, "1", Importance.HIGH, ACKS_DOC) + .define(ACKS_CONFIG, + Type.STRING, + "1", + in(Arrays.asList("all","-1", "0", "1")), + Importance.HIGH, + ACKS_DOC) .define(COMPRESSION_TYPE_CONFIG, Type.STRING, "none", Importance.HIGH, COMPRESSION_TYPE_DOC) .define(BATCH_SIZE_CONFIG, Type.INT, 16384, atLeast(0), Importance.MEDIUM, BATCH_SIZE_DOC) .define(TIMEOUT_CONFIG, Type.INT, 30 * 1000, atLeast(0), Importance.MEDIUM, TIMEOUT_DOC) diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java index addc906160883..227309e8c62f9 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java @@ -268,6 +268,48 @@ else if (max == null) } } + public static class ValidString implements Validator { + List validStrings; + + private ValidString(List validStrings) { + this.validStrings = validStrings; + } + + public static ValidString in(List validStrings) { + return new ValidString(validStrings); + } + + @Override + public void ensureValid(String name, Object o) { + + String s = (String) o; + + if (!validStrings.contains(s)) { + throw new ConfigException(name,o,"String must be one of:" +join(validStrings)); + } + + } + + public String toString() { + return "[" + join(validStrings) + "]"; + } + + private String join(List list) + { + StringBuilder sb = new StringBuilder(); + boolean first = true; + for (String item : list) + { + if (first) + first = false; + else + sb.append(","); + sb.append(item); + } + return sb.toString(); + } + } + private static class ConfigKey { public final String name; public final Type type; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java b/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java new file mode 100644 index 0000000000000..75c80a97e4308 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.errors; + +/** + * Number of insync replicas for the partition is lower than min.insync.replicas + * This exception is raised when the low ISR size is discovered *after* the message + * was already appended to the log. Producer retries will cause duplicates. + */ +public class NotEnoughReplicasAfterAppendException extends RetriableException { + private static final long serialVersionUID = 1L; + + public NotEnoughReplicasAfterAppendException() { + super(); + } + + public NotEnoughReplicasAfterAppendException(String message, Throwable cause) { + super(message,cause); + } + + public NotEnoughReplicasAfterAppendException(String message) { + super(message); + } + + public NotEnoughReplicasAfterAppendException(Throwable cause) { + super(cause); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasException.java b/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasException.java new file mode 100644 index 0000000000000..486d5155bbb1f --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasException.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.errors; + +/** + * Number of insync replicas for the partition is lower than min.insync.replicas + */ +public class NotEnoughReplicasException extends RetriableException { + private static final long serialVersionUID = 1L; + + public NotEnoughReplicasException() { + super(); + } + + public NotEnoughReplicasException(String message, Throwable cause) { + super(message, cause); + } + + public NotEnoughReplicasException(String message) { + super(message); + } + + public NotEnoughReplicasException(Throwable cause) { + super(cause); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index d5f5de32bbed6..3316b6a109831 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -25,7 +25,7 @@ /** * This class contains all the client-server errors--those errors that must be sent from the server to the client. These * are thus part of the protocol. The names can be changed but the error code cannot. - * + * * Do not add exceptions that occur only on the client or only on the server here. */ public enum Errors { @@ -44,8 +44,9 @@ public enum Errors { NETWORK_EXCEPTION(13, new NetworkException("The server disconnected before a response was received.")), // TODO: errorCode 14, 15, 16 INVALID_TOPIC_EXCEPTION(17, new InvalidTopicException("The request attempted to perform an operation on an invalid topic.")), - RECORD_LIST_TOO_LARGE(18, new RecordBatchTooLargeException("The request included message batch larger than the configured segment size on the server.")); - + RECORD_LIST_TOO_LARGE(18, new RecordBatchTooLargeException("The request included message batch larger than the configured segment size on the server.")), + NOT_ENOUGH_REPLICAS(19, new NotEnoughReplicasException("Messages are rejected since there are fewer in-sync replicas than required.")), + NOT_ENOUGH_REPLICAS_AFTER_APPEND(20, new NotEnoughReplicasAfterAppendException("Messages are written to the log, but to fewer in-sync replicas than required.")); private static Map, Errors> classToError = new HashMap, Errors>(); private static Map codeToError = new HashMap(); static { diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index ff106b47e6ee1..e88ecf224a4da 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -269,14 +269,26 @@ class Partition(val topic: String, else true /* also count the local (leader) replica */ }) + val minIsr = leaderReplica.log.get.config.minInSyncReplicas + trace("%d/%d acks satisfied for %s-%d".format(numAcks, requiredAcks, topic, partitionId)) - if ((requiredAcks < 0 && leaderReplica.highWatermark.messageOffset >= requiredOffset) || - (requiredAcks > 0 && numAcks >= requiredAcks)) { + if (requiredAcks < 0 && leaderReplica.highWatermark.messageOffset >= requiredOffset ) { /* * requiredAcks < 0 means acknowledge after all replicas in ISR * are fully caught up to the (local) leader's offset * corresponding to this produce request. + * + * minIsr means that the topic is configured not to accept messages + * if there are not enough replicas in ISR + * in this scenario the request was already appended locally and + * then added to the purgatory before the ISR was shrunk */ + if (minIsr <= curInSyncReplicas.size) { + (true, ErrorMapping.NoError) + } else { + (true, ErrorMapping.NotEnoughReplicasAfterAppendCode) + } + } else if (requiredAcks > 0 && numAcks >= requiredAcks) { (true, ErrorMapping.NoError) } else (false, ErrorMapping.NoError) @@ -350,12 +362,21 @@ class Partition(val topic: String, stuckReplicas ++ slowReplicas } - def appendMessagesToLeader(messages: ByteBufferMessageSet) = { + def appendMessagesToLeader(messages: ByteBufferMessageSet, requiredAcks: Int=0) = { inReadLock(leaderIsrUpdateLock) { val leaderReplicaOpt = leaderReplicaIfLocal() leaderReplicaOpt match { case Some(leaderReplica) => val log = leaderReplica.log.get + val minIsr = log.config.minInSyncReplicas + val inSyncSize = inSyncReplicas.size + + // Avoid writing to leader if there are not enough insync replicas to make it safe + if (inSyncSize < minIsr && requiredAcks == -1) { + throw new NotEnoughReplicasException("Number of insync replicas for partition [%s,%d] is [%d], below required minimum [%d]" + .format(topic,partitionId,minIsr,inSyncSize)) + } + val info = log.append(messages, assignOffsets = true) // probably unblock some follower fetch requests since log end offset has been updated replicaManager.unblockDelayedFetchRequests(new TopicPartitionRequestKey(this.topic, this.partitionId)) diff --git a/core/src/main/scala/kafka/common/ErrorMapping.scala b/core/src/main/scala/kafka/common/ErrorMapping.scala index a190607a3bb60..880ab4a004f07 100644 --- a/core/src/main/scala/kafka/common/ErrorMapping.scala +++ b/core/src/main/scala/kafka/common/ErrorMapping.scala @@ -5,7 +5,7 @@ * 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 @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import scala.Predef._ /** - * A bi-directional mapping between error codes and exceptions x + * A bi-directional mapping between error codes and exceptions */ object ErrorMapping { val EmptyByteBuffer = ByteBuffer.allocate(0) @@ -47,8 +47,10 @@ object ErrorMapping { val NotCoordinatorForConsumerCode: Short = 16 val InvalidTopicCode : Short = 17 val MessageSetSizeTooLargeCode: Short = 18 + val NotEnoughReplicasCode : Short = 19 + val NotEnoughReplicasAfterAppendCode: Short = 20 - private val exceptionToCode = + private val exceptionToCode = Map[Class[Throwable], Short]( classOf[OffsetOutOfRangeException].asInstanceOf[Class[Throwable]] -> OffsetOutOfRangeCode, classOf[InvalidMessageException].asInstanceOf[Class[Throwable]] -> InvalidMessageCode, @@ -66,15 +68,17 @@ object ErrorMapping { classOf[ConsumerCoordinatorNotAvailableException].asInstanceOf[Class[Throwable]] -> ConsumerCoordinatorNotAvailableCode, classOf[NotCoordinatorForConsumerException].asInstanceOf[Class[Throwable]] -> NotCoordinatorForConsumerCode, classOf[InvalidTopicException].asInstanceOf[Class[Throwable]] -> InvalidTopicCode, - classOf[MessageSetSizeTooLargeException].asInstanceOf[Class[Throwable]] -> MessageSetSizeTooLargeCode + classOf[MessageSetSizeTooLargeException].asInstanceOf[Class[Throwable]] -> MessageSetSizeTooLargeCode, + classOf[NotEnoughReplicasException].asInstanceOf[Class[Throwable]] -> NotEnoughReplicasCode, + classOf[NotEnoughReplicasAfterAppendException].asInstanceOf[Class[Throwable]] -> NotEnoughReplicasAfterAppendCode ).withDefaultValue(UnknownCode) - + /* invert the mapping */ - private val codeToException = + private val codeToException = (Map[Short, Class[Throwable]]() ++ exceptionToCode.iterator.map(p => (p._2, p._1))).withDefaultValue(classOf[UnknownException]) - + def codeFor(exception: Class[Throwable]): Short = exceptionToCode(exception) - + def maybeThrowException(code: Short) = if(code != 0) throw codeToException(code).newInstance() diff --git a/core/src/main/scala/kafka/common/NotEnoughReplicasAfterAppendException.scala b/core/src/main/scala/kafka/common/NotEnoughReplicasAfterAppendException.scala new file mode 100644 index 0000000000000..c4f9def6162e9 --- /dev/null +++ b/core/src/main/scala/kafka/common/NotEnoughReplicasAfterAppendException.scala @@ -0,0 +1,27 @@ +/* + * 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.common + +/** + * Number of insync replicas for the partition is lower than min.insync.replicas + * This exception is raised when the low ISR size is discovered *after* the message + * was already appended to the log. Producer retries will cause duplicates. + */ +class NotEnoughReplicasAfterAppendException(message: String) extends RuntimeException(message) { + def this() = this(null) +} diff --git a/core/src/main/scala/kafka/common/NotEnoughReplicasException.scala b/core/src/main/scala/kafka/common/NotEnoughReplicasException.scala new file mode 100644 index 0000000000000..bfbe0ee4a5a15 --- /dev/null +++ b/core/src/main/scala/kafka/common/NotEnoughReplicasException.scala @@ -0,0 +1,25 @@ +/* + * 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.common + +/** + * Message was rejected because number of insync replicas for the partition is lower than min.insync.replicas + */ +class NotEnoughReplicasException(message: String) extends RuntimeException(message) { + def this() = this(null) +} diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index 5746ad4767589..d2cc9e3d6b7a4 100644 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -36,6 +36,7 @@ object Defaults { val MinCleanableDirtyRatio = 0.5 val Compact = false val UncleanLeaderElectionEnable = true + val MinInSyncReplicas = 1 } /** @@ -53,7 +54,9 @@ object Defaults { * @param minCleanableRatio The ratio of bytes that are available for cleaning to the bytes already cleaned * @param compact Should old segments in this log be deleted or deduplicated? * @param uncleanLeaderElectionEnable Indicates whether unclean leader election is enabled; actually a controller-level property - * but included here for topic-specific configuration validation purposes + * but included here for topic-specific configuration validation purposes + * @param minInSyncReplicas If number of insync replicas drops below this number, we stop accepting writes with -1 (or all) required acks + * */ case class LogConfig(val segmentSize: Int = Defaults.SegmentSize, val segmentMs: Long = Defaults.SegmentMs, @@ -68,8 +71,9 @@ case class LogConfig(val segmentSize: Int = Defaults.SegmentSize, val deleteRetentionMs: Long = Defaults.DeleteRetentionMs, val minCleanableRatio: Double = Defaults.MinCleanableDirtyRatio, val compact: Boolean = Defaults.Compact, - val uncleanLeaderElectionEnable: Boolean = Defaults.UncleanLeaderElectionEnable) { - + val uncleanLeaderElectionEnable: Boolean = Defaults.UncleanLeaderElectionEnable, + val minInSyncReplicas: Int = Defaults.MinInSyncReplicas) { + def toProps: Properties = { val props = new Properties() import LogConfig._ @@ -87,9 +91,9 @@ case class LogConfig(val segmentSize: Int = Defaults.SegmentSize, props.put(MinCleanableDirtyRatioProp, minCleanableRatio.toString) props.put(CleanupPolicyProp, if(compact) "compact" else "delete") props.put(UncleanLeaderElectionEnableProp, uncleanLeaderElectionEnable.toString) + props.put(MinInSyncReplicasProp, minInSyncReplicas.toString) props } - } object LogConfig { @@ -107,13 +111,14 @@ object LogConfig { val MinCleanableDirtyRatioProp = "min.cleanable.dirty.ratio" val CleanupPolicyProp = "cleanup.policy" val UncleanLeaderElectionEnableProp = "unclean.leader.election.enable" - - val ConfigNames = Set(SegmentBytesProp, - SegmentMsProp, - SegmentIndexBytesProp, - FlushMessagesProp, - FlushMsProp, - RetentionBytesProp, + val MinInSyncReplicasProp = "min.insync.replicas" + + val ConfigNames = Set(SegmentBytesProp, + SegmentMsProp, + SegmentIndexBytesProp, + FlushMessagesProp, + FlushMsProp, + RetentionBytesProp, RententionMsProp, MaxMessageBytesProp, IndexIntervalBytesProp, @@ -121,9 +126,9 @@ object LogConfig { DeleteRetentionMsProp, MinCleanableDirtyRatioProp, CleanupPolicyProp, - UncleanLeaderElectionEnableProp) - - + UncleanLeaderElectionEnableProp, + MinInSyncReplicasProp) + /** * Parse the given properties instance into a LogConfig object */ @@ -144,9 +149,10 @@ object LogConfig { compact = props.getProperty(CleanupPolicyProp, if(Defaults.Compact) "compact" else "delete") .trim.toLowerCase != "delete", uncleanLeaderElectionEnable = props.getProperty(UncleanLeaderElectionEnableProp, - Defaults.UncleanLeaderElectionEnable.toString).toBoolean) + Defaults.UncleanLeaderElectionEnable.toString).toBoolean, + minInSyncReplicas = props.getProperty(MinInSyncReplicasProp,Defaults.MinInSyncReplicas.toString).toInt) } - + /** * Create a log config instance using the given properties and defaults */ @@ -155,7 +161,7 @@ object LogConfig { props.putAll(overrides) fromProps(props) } - + /** * Check that property names are valid */ @@ -164,15 +170,27 @@ object LogConfig { for(name <- props.keys) require(LogConfig.ConfigNames.contains(name), "Unknown configuration \"%s\".".format(name)) } - + /** * Check that the given properties contain only valid log config names, and that all values can be parsed. */ def validate(props: Properties) { validateNames(props) + validateMinInSyncReplicas(props) LogConfig.fromProps(LogConfig().toProps, props) // check that we can parse the values } - -} - - \ No newline at end of file + + /** + * Check that MinInSyncReplicas is reasonable + * Unfortunately, we can't validate its smaller than number of replicas + * since we don't have this information here + */ + private def validateMinInSyncReplicas(props: Properties) { + val minIsr = props.getProperty(MinInSyncReplicasProp) + if (minIsr != null && minIsr.toInt < 1) { + throw new InvalidConfigException("Wrong value " + minIsr + " of min.insync.replicas in topic configuration; " + + " Valid values are at least 1") + } + } + +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/producer/SyncProducerConfig.scala b/core/src/main/scala/kafka/producer/SyncProducerConfig.scala index 69b2d0c11bb14..a08ce00a0aae7 100644 --- a/core/src/main/scala/kafka/producer/SyncProducerConfig.scala +++ b/core/src/main/scala/kafka/producer/SyncProducerConfig.scala @@ -42,11 +42,15 @@ trait SyncProducerConfigShared { val clientId = props.getString("client.id", SyncProducerConfig.DefaultClientId) /* - * The required acks of the producer requests - negative value means ack - * after the replicas in ISR have caught up to the leader's offset - * corresponding to this produce request. + * The number of acknowledgments the producer requires the leader to have received before considering a request complete. + * This controls the durability of the messages sent by the producer. + * + * request.required.acks = 0 - means the producer will not wait for any acknowledgement from the leader. + * request.required.acks = 1 - means the leader will write the message to its local log and immediately acknowledge + * request.required.acks = -1 - means the leader will wait for acknowledgement from all in-sync replicas before acknowledging the write */ - val requestRequiredAcks = props.getShort("request.required.acks", SyncProducerConfig.DefaultRequiredAcks) + + val requestRequiredAcks = props.getShortInRange("request.required.acks", SyncProducerConfig.DefaultRequiredAcks,(-1,1)) /* * The ack timeout of the producer requests. Value must be non-negative and non-zero @@ -59,4 +63,4 @@ object SyncProducerConfig { val DefaultClientId = "" val DefaultRequiredAcks : Short = 0 val DefaultAckTimeoutMs = 10000 -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index c584b559416b3..67f2833804cb1 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -248,7 +248,7 @@ class KafkaApis(val requestChannel: RequestChannel, val partitionOpt = replicaManager.getPartition(topicAndPartition.topic, topicAndPartition.partition) val info = partitionOpt match { case Some(partition) => - partition.appendMessagesToLeader(messages.asInstanceOf[ByteBufferMessageSet]) + partition.appendMessagesToLeader(messages.asInstanceOf[ByteBufferMessageSet],producerRequest.requiredAcks) case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d" .format(topicAndPartition, brokerId)) } @@ -284,6 +284,10 @@ class KafkaApis(val requestChannel: RequestChannel, warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( producerRequest.correlationId, producerRequest.clientId, topicAndPartition, nle.getMessage)) new ProduceResult(topicAndPartition, nle) + case nere: NotEnoughReplicasException => + warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( + producerRequest.correlationId, producerRequest.clientId, topicAndPartition, nere.getMessage)) + new ProduceResult(topicAndPartition, nere) case e: Throwable => BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).failedProduceRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats.failedProduceRequestRate.mark() diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 165c816a9f4c9..90af698b01ec8 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -199,6 +199,11 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* enable auto creation of topic on the server */ val autoCreateTopicsEnable = props.getBoolean("auto.create.topics.enable", true) + /* define the minimum number of replicas in ISR needed to satisfy a produce request with required.acks=-1 (or all) */ + val minInSyncReplicas = props.getIntInRange("min.insync.replicas",1,(1,Int.MaxValue)) + + + /*********** Replication configuration ***********/ /* the socket timeout for controller-to-broker channels */ diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index 39f777baebb6c..209a409cb47eb 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -18,12 +18,12 @@ package kafka.api import kafka.common.Topic -import org.apache.kafka.common.errors.InvalidTopicException +import org.apache.kafka.common.errors.{InvalidTopicException,NotEnoughReplicasException} import org.scalatest.junit.JUnit3Suite import org.junit.Test import org.junit.Assert._ -import java.util.Random +import java.util.{Properties, Random} import java.lang.Integer import java.util.concurrent.{TimeoutException, TimeUnit, ExecutionException} @@ -302,6 +302,59 @@ class ProducerFailureHandlingTest extends JUnit3Suite with KafkaServerTestHarnes producer1.send(new ProducerRecord(Topic.InternalTopics.head, "test".getBytes, "test".getBytes)).get } + @Test + def testNotEnoughReplicas() { + val topicName = "minisrtest" + val topicProps = new Properties(); + topicProps.put("min.insync.replicas","3"); + + + TestUtils.createTopic(zkClient, topicName, 1, 2, servers,topicProps) + + + 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") + } catch { + case e: ExecutionException => + if (!e.getCause.isInstanceOf[NotEnoughReplicasException]) { + fail("Expected NotEnoughReplicasException when producing to topic with fewer brokers than min.insync.replicas") + } + } + } + + @Test + def testNotEnoughReplicasAfterBrokerShutdown() { + val topicName = "minisrtest2" + val topicProps = new Properties(); + topicProps.put("min.insync.replicas","2"); + + + TestUtils.createTopic(zkClient, topicName, 1, 2, servers,topicProps) + + + val record = new ProducerRecord(topicName, null, "key".getBytes, "value".getBytes) + // This should work + producer3.send(record).get + + //shut down one broker + servers.head.shutdown() + servers.head.awaitShutdown() + try { + producer3.send(record).get + fail("Expected exception when producing to topic with fewer brokers than min.insync.replicas") + } catch { + case e: ExecutionException => + if (!e.getCause.isInstanceOf[NotEnoughReplicasException]) { + fail("Expected NotEnoughReplicasException when producing to topic with fewer brokers than min.insync.replicas") + } + } + + servers.head.startup() + + } + private class ProducerScheduler extends ShutdownableThread("daemon-producer", false) { val numRecords = 1000 diff --git a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala index dd71d81041e1b..ce65dab4910d9 100644 --- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala @@ -17,6 +17,7 @@ package kafka.producer +import org.apache.kafka.common.config.ConfigException import org.scalatest.TestFailedException import org.scalatest.junit.JUnit3Suite import kafka.consumer.SimpleConsumer @@ -143,7 +144,7 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ @Test def testSendToNewTopic() { val props1 = new util.Properties() - props1.put("request.required.acks", "2") + props1.put("request.required.acks", "-1") val topic = "new-topic" // create topic with 1 partition and await leadership @@ -181,24 +182,20 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ // no need to retry since the send will always fail props2.put("message.send.max.retries", "0") - val producer2 = TestUtils.createProducer[String, String]( - brokerList = TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)), - encoder = classOf[StringEncoder].getName, - keyEncoder = classOf[StringEncoder].getName, - partitioner = classOf[StaticPartitioner].getName, - producerProps = props2) - try { - producer2.send(new KeyedMessage[String, String](topic, "test", "test2")) - fail("Should have timed out for 3 acks.") + val producer2 = TestUtils.createProducer[String, String]( + brokerList = TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[StringEncoder].getName, + partitioner = classOf[StaticPartitioner].getName, + producerProps = props2) + producer2.close + fail("we don't support request.required.acks greater than 1") } catch { - case se: FailedToSendMessageException => - // this is expected + case iae: IllegalArgumentException => // this is expected case e: Throwable => fail("Not expected", e) - } - finally { - producer2.close() + } } diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala index 24deea06753e5..fb61d552f2320 100644 --- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala @@ -18,6 +18,7 @@ package kafka.producer import java.net.SocketTimeoutException +import java.util.Properties import junit.framework.Assert import kafka.admin.AdminUtils import kafka.integration.KafkaServerTestHarness @@ -113,6 +114,7 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { Assert.assertEquals(0, response2.status(TopicAndPartition("test", 0)).offset) } + @Test def testMessageSizeTooLargeWithAckZero() { val server = servers.head @@ -225,4 +227,24 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { val response = producer.send(emptyRequest) Assert.assertTrue(response == null) } + + @Test + def testNotEnoughReplicas() { + val topicName = "minisrtest" + val server = servers.head + + val props = TestUtils.getSyncProducerConfig(server.socketServer.port) + props.put("request.required.acks", "-1") + + val producer = new SyncProducer(new SyncProducerConfig(props)) + val topicProps = new Properties(); + topicProps.put("min.insync.replicas","2"); + AdminUtils.createTopic(zkClient, topicName, 1, 1,topicProps) + TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topicName, 0) + + val response = producer.send(TestUtils.produceRequest(topicName, 0, + new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = new Message(messageBytes)),-1)) + + Assert.assertEquals(ErrorMapping.NotEnoughReplicasCode, response.status(TopicAndPartition(topicName, 0)).error) + } } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 2dbdd3c2f03ca..dd3640f47b26a 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -168,10 +168,14 @@ object TestUtils extends Logging { * Wait until the leader is elected and the metadata is propagated to all brokers. * Return the leader for each partition. */ - def createTopic(zkClient: ZkClient, topic: String, numPartitions: Int = 1, replicationFactor: Int = 1, - servers: Seq[KafkaServer]) : scala.collection.immutable.Map[Int, Option[Int]] = { + def createTopic(zkClient: ZkClient, + topic: String, + numPartitions: Int = 1, + replicationFactor: Int = 1, + servers: Seq[KafkaServer], + topicConfig: Properties = new Properties) : scala.collection.immutable.Map[Int, Option[Int]] = { // create topic - AdminUtils.createTopic(zkClient, topic, numPartitions, replicationFactor) + AdminUtils.createTopic(zkClient, topic, numPartitions, replicationFactor, topicConfig) // wait until the update metadata request for new topic reaches all servers (0 until numPartitions).map { case i => TestUtils.waitUntilMetadataIsPropagated(servers, topic, i) From 09b2b3cbe07e8cdea42434fa6c54af0a064a93a2 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Thu, 9 Oct 2014 21:08:38 -0700 Subject: [PATCH 014/491] kafka-1699; autoRebalanceScheduler.shutdown() causes deadlock while controller shutting down; patched by Sriharsha Chintalapani; reviewed by Jun Rao --- core/src/main/scala/kafka/controller/KafkaController.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 381c52fcc73c7..51a5bad24f1e7 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -345,12 +345,13 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt if (deleteTopicManager != null) deleteTopicManager.shutdown() + // shutdown leader rebalance scheduler + if (config.autoLeaderRebalanceEnable) + autoRebalanceScheduler.shutdown() + inLock(controllerContext.controllerLock) { // de-register partition ISR listener for on-going partition reassignment task deregisterReassignedPartitionsIsrChangeListeners() - // shutdown leader rebalance scheduler - if (config.autoLeaderRebalanceEnable) - autoRebalanceScheduler.shutdown() // shutdown partition state machine partitionStateMachine.shutdown() // shutdown replica state machine From 1733c0bfb06a86eba7b16db3c80cabc91fc04b22 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Sun, 12 Oct 2014 08:44:59 -0700 Subject: [PATCH 015/491] kafka-1670 (followup to fix system tests); Corrupt log files for segment.bytes values close to Int.MaxInt; patched by Sriharsha Chintalapani; reviewed by Jun Rao --- .../testcase_15001/testcase_5001_properties.json | 12 ++++++------ .../testcase_15002/testcase_5002_properties.json | 14 +++++++------- .../testcase_15003/testcase_5003_properties.json | 14 +++++++------- .../testcase_15004/testcase_5004_properties.json | 14 +++++++------- .../testcase_15005/testcase_5005_properties.json | 14 +++++++------- .../testcase_15006/testcase_5006_properties.json | 14 +++++++------- .../testcase_5001/testcase_5001_properties.json | 14 +++++++------- .../testcase_5002/testcase_5002_properties.json | 14 +++++++------- .../testcase_5003/testcase_5003_properties.json | 14 +++++++------- .../testcase_5004/testcase_5004_properties.json | 14 +++++++------- .../testcase_5005/testcase_5005_properties.json | 14 +++++++------- .../testcase_5006/testcase_5006_properties.json | 14 +++++++------- .../testcase_7001/testcase_7001_properties.json | 8 ++++---- .../testcase_7002/testcase_7002_properties.json | 8 ++++---- .../testcase_0001/testcase_0001_properties.json | 8 ++++---- .../testcase_0002/testcase_0002_properties.json | 8 ++++---- .../testcase_0003/testcase_0003_properties.json | 8 ++++---- .../testcase_0004/testcase_0004_properties.json | 8 ++++---- .../testcase_0005/testcase_0005_properties.json | 8 ++++---- .../testcase_0006/testcase_0006_properties.json | 8 ++++---- .../testcase_0007/testcase_0007_properties.json | 8 ++++---- .../testcase_0008/testcase_0008_properties.json | 8 ++++---- .../testcase_0009/testcase_0009_properties.json | 8 ++++---- .../testcase_0101/testcase_0101_properties.json | 8 ++++---- .../testcase_0102/testcase_0102_properties.json | 8 ++++---- .../testcase_0103/testcase_0103_properties.json | 8 ++++---- .../testcase_0104/testcase_0104_properties.json | 8 ++++---- .../testcase_0105/testcase_0105_properties.json | 8 ++++---- .../testcase_0106/testcase_0106_properties.json | 8 ++++---- .../testcase_0107/testcase_0107_properties.json | 8 ++++---- .../testcase_0108/testcase_0108_properties.json | 8 ++++---- .../testcase_0109/testcase_0109_properties.json | 8 ++++---- .../testcase_1/testcase_1_properties.json | 8 ++++---- .../testcase_10101/testcase_0101_properties.json | 8 ++++---- .../testcase_10102/testcase_0102_properties.json | 8 ++++---- .../testcase_10103/testcase_0103_properties.json | 8 ++++---- .../testcase_10104/testcase_0104_properties.json | 8 ++++---- .../testcase_10105/testcase_0105_properties.json | 8 ++++---- .../testcase_10106/testcase_0106_properties.json | 8 ++++---- .../testcase_10107/testcase_0107_properties.json | 8 ++++---- .../testcase_10108/testcase_0108_properties.json | 8 ++++---- .../testcase_10109/testcase_0109_properties.json | 8 ++++---- 42 files changed, 203 insertions(+), 203 deletions(-) diff --git a/system_test/mirror_maker_testsuite/testcase_15001/testcase_5001_properties.json b/system_test/mirror_maker_testsuite/testcase_15001/testcase_5001_properties.json index 287cab9a9b6b9..9dd3477e70a53 100644 --- a/system_test/mirror_maker_testsuite/testcase_15001/testcase_5001_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_15001/testcase_5001_properties.json @@ -54,7 +54,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_4_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -65,7 +65,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_5_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -76,7 +76,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_6_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -87,7 +87,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_7_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -98,7 +98,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_8_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -109,7 +109,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_9_logs", "default.replication.factor": "3", "num.partitions": "5", diff --git a/system_test/mirror_maker_testsuite/testcase_15002/testcase_5002_properties.json b/system_test/mirror_maker_testsuite/testcase_15002/testcase_5002_properties.json index 5457eb1d26526..d6495e54d6845 100644 --- a/system_test/mirror_maker_testsuite/testcase_15002/testcase_5002_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_15002/testcase_5002_properties.json @@ -7,7 +7,7 @@ "06":"At the end it verifies the log size and contents", "07":"Use a consumer to verify no message loss in TARGET cluster.", "08":"Producer dimensions : mode:sync, acks:-1, comp:0", - "09":"Log segment size : 10240" + "09":"Log segment size : 20480" }, "testcase_args": { "bounce_leader": "false", @@ -54,7 +54,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_4_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -65,7 +65,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_5_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -76,7 +76,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_6_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -87,7 +87,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_7_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -98,7 +98,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_8_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -109,7 +109,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_9_logs", "default.replication.factor": "3", "num.partitions": "5", diff --git a/system_test/mirror_maker_testsuite/testcase_15003/testcase_5003_properties.json b/system_test/mirror_maker_testsuite/testcase_15003/testcase_5003_properties.json index 98fefee5838a2..842c70eaf33fd 100644 --- a/system_test/mirror_maker_testsuite/testcase_15003/testcase_5003_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_15003/testcase_5003_properties.json @@ -7,7 +7,7 @@ "06":"At the end it verifies the log size and contents", "07":"Use a consumer to verify no message loss in TARGET cluster.", "08":"Producer dimensions : mode:async, acks:-1, comp:1", - "09":"Log segment size : 10240" + "09":"Log segment size : 20480" }, "testcase_args": { "bounce_leader": "false", @@ -55,7 +55,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_4_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -66,7 +66,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_5_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -77,7 +77,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_6_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -88,7 +88,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_7_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -99,7 +99,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_8_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -110,7 +110,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_9_logs", "default.replication.factor": "3", "num.partitions": "5", diff --git a/system_test/mirror_maker_testsuite/testcase_15004/testcase_5004_properties.json b/system_test/mirror_maker_testsuite/testcase_15004/testcase_5004_properties.json index 6067b1263d0a5..48864e61afb9c 100644 --- a/system_test/mirror_maker_testsuite/testcase_15004/testcase_5004_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_15004/testcase_5004_properties.json @@ -7,7 +7,7 @@ "06":"At the end it verifies the log size and contents", "07":"Use a consumer to verify no message loss in TARGET cluster.", "08":"Producer dimensions : mode:async, acks:1, comp:1", - "09":"Log segment size : 10240" + "09":"Log segment size : 20480" }, "testcase_args": { "bounce_leader": "false", @@ -55,7 +55,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_4_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -66,7 +66,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_5_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -77,7 +77,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_6_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -88,7 +88,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_7_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -99,7 +99,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_8_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -110,7 +110,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_9_logs", "default.replication.factor": "3", "num.partitions": "5", diff --git a/system_test/mirror_maker_testsuite/testcase_15005/testcase_5005_properties.json b/system_test/mirror_maker_testsuite/testcase_15005/testcase_5005_properties.json index 58483ad375cd1..92b2a6b4ffeb8 100644 --- a/system_test/mirror_maker_testsuite/testcase_15005/testcase_5005_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_15005/testcase_5005_properties.json @@ -7,7 +7,7 @@ "06":"At the end it verifies the log size and contents", "07":"Use a consumer to verify no message loss in TARGET cluster.", "08":"Producer dimensions : mode:async, acks:-1, comp:1", - "09":"Log segment size : 10240" + "09":"Log segment size : 20480" }, "testcase_args": { "bounce_leader": "false", @@ -55,7 +55,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_4_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -66,7 +66,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_5_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -77,7 +77,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_6_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -88,7 +88,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_7_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -99,7 +99,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_8_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -110,7 +110,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_9_logs", "default.replication.factor": "3", "num.partitions": "5", diff --git a/system_test/mirror_maker_testsuite/testcase_15006/testcase_5006_properties.json b/system_test/mirror_maker_testsuite/testcase_15006/testcase_5006_properties.json index 1d9190ce7c1b5..7d5019c6aa7fe 100644 --- a/system_test/mirror_maker_testsuite/testcase_15006/testcase_5006_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_15006/testcase_5006_properties.json @@ -7,7 +7,7 @@ "06":"At the end it verifies the log size and contents", "07":"Use a consumer to verify no message loss in TARGET cluster.", "08":"Producer dimensions : mode:async, acks:1, comp:1", - "09":"Log segment size : 10240" + "09":"Log segment size : 20480" }, "testcase_args": { "bounce_leader": "false", @@ -55,7 +55,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_4_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -66,7 +66,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_5_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -77,7 +77,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_6_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -88,7 +88,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_7_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -99,7 +99,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_8_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -110,7 +110,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_9_logs", "default.replication.factor": "3", "num.partitions": "5", diff --git a/system_test/mirror_maker_testsuite/testcase_5001/testcase_5001_properties.json b/system_test/mirror_maker_testsuite/testcase_5001/testcase_5001_properties.json index 4a0da6ea45201..0891836421909 100644 --- a/system_test/mirror_maker_testsuite/testcase_5001/testcase_5001_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5001/testcase_5001_properties.json @@ -7,7 +7,7 @@ "06":"At the end it verifies the log size and contents", "07":"Use a consumer to verify no message loss in TARGET cluster.", "08":"Producer dimensions : mode:sync, acks:-1, comp:0", - "09":"Log segment size : 10240" + "09":"Log segment size : 20480" }, "testcase_args": { "bounce_leader": "false", @@ -54,7 +54,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_4_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -65,7 +65,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_5_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -76,7 +76,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_6_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -87,7 +87,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_7_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -98,7 +98,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_8_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -109,7 +109,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_9_logs", "default.replication.factor": "3", "num.partitions": "5", diff --git a/system_test/mirror_maker_testsuite/testcase_5002/testcase_5002_properties.json b/system_test/mirror_maker_testsuite/testcase_5002/testcase_5002_properties.json index d74e97de642ba..56e481255cbbb 100644 --- a/system_test/mirror_maker_testsuite/testcase_5002/testcase_5002_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5002/testcase_5002_properties.json @@ -7,7 +7,7 @@ "06":"At the end it verifies the log size and contents", "07":"Use a consumer to verify no message loss in TARGET cluster.", "08":"Producer dimensions : mode:sync, acks:-1, comp:0", - "09":"Log segment size : 10240" + "09":"Log segment size : 20480" }, "testcase_args": { "bounce_leader": "false", @@ -54,7 +54,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_4_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -65,7 +65,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_5_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -76,7 +76,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_6_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -87,7 +87,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_7_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -98,7 +98,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_8_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -109,7 +109,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_9_logs", "default.replication.factor": "3", "num.partitions": "5", diff --git a/system_test/mirror_maker_testsuite/testcase_5003/testcase_5003_properties.json b/system_test/mirror_maker_testsuite/testcase_5003/testcase_5003_properties.json index e33acf192769d..8f8c47af4e797 100644 --- a/system_test/mirror_maker_testsuite/testcase_5003/testcase_5003_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5003/testcase_5003_properties.json @@ -7,7 +7,7 @@ "06":"At the end it verifies the log size and contents", "07":"Use a consumer to verify no message loss in TARGET cluster.", "08":"Producer dimensions : mode:async, acks:-1, comp:1", - "09":"Log segment size : 10240" + "09":"Log segment size : 20480" }, "testcase_args": { "bounce_leader": "false", @@ -55,7 +55,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_4_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -66,7 +66,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_5_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -77,7 +77,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_6_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -88,7 +88,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_7_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -99,7 +99,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_8_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -110,7 +110,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_9_logs", "default.replication.factor": "3", "num.partitions": "5", diff --git a/system_test/mirror_maker_testsuite/testcase_5004/testcase_5004_properties.json b/system_test/mirror_maker_testsuite/testcase_5004/testcase_5004_properties.json index 5c39bcf50527d..baa639b688888 100644 --- a/system_test/mirror_maker_testsuite/testcase_5004/testcase_5004_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5004/testcase_5004_properties.json @@ -7,7 +7,7 @@ "06":"At the end it verifies the log size and contents", "07":"Use a consumer to verify no message loss in TARGET cluster.", "08":"Producer dimensions : mode:async, acks:1, comp:1", - "09":"Log segment size : 10240" + "09":"Log segment size : 20480" }, "testcase_args": { "bounce_leader": "false", @@ -55,7 +55,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_4_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -66,7 +66,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_5_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -77,7 +77,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_6_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -88,7 +88,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_7_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -99,7 +99,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_8_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -110,7 +110,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_9_logs", "default.replication.factor": "3", "num.partitions": "5", diff --git a/system_test/mirror_maker_testsuite/testcase_5005/testcase_5005_properties.json b/system_test/mirror_maker_testsuite/testcase_5005/testcase_5005_properties.json index 697af656fb58d..8c383756d88ca 100644 --- a/system_test/mirror_maker_testsuite/testcase_5005/testcase_5005_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5005/testcase_5005_properties.json @@ -7,7 +7,7 @@ "06":"At the end it verifies the log size and contents", "07":"Use a consumer to verify no message loss in TARGET cluster.", "08":"Producer dimensions : mode:async, acks:-1, comp:1", - "09":"Log segment size : 10240" + "09":"Log segment size : 20480" }, "testcase_args": { "bounce_leader": "false", @@ -55,7 +55,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_4_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -66,7 +66,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_5_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -77,7 +77,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_6_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -88,7 +88,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_7_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -99,7 +99,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_8_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -110,7 +110,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_9_logs", "default.replication.factor": "3", "num.partitions": "5", diff --git a/system_test/mirror_maker_testsuite/testcase_5006/testcase_5006_properties.json b/system_test/mirror_maker_testsuite/testcase_5006/testcase_5006_properties.json index a610a60d2b18c..fb275330bd188 100644 --- a/system_test/mirror_maker_testsuite/testcase_5006/testcase_5006_properties.json +++ b/system_test/mirror_maker_testsuite/testcase_5006/testcase_5006_properties.json @@ -7,7 +7,7 @@ "06":"At the end it verifies the log size and contents", "07":"Use a consumer to verify no message loss in TARGET cluster.", "08":"Producer dimensions : mode:async, acks:1, comp:1", - "09":"Log segment size : 10240" + "09":"Log segment size : 20480" }, "testcase_args": { "bounce_leader": "false", @@ -55,7 +55,7 @@ "entity_id": "4", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_4_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -66,7 +66,7 @@ "entity_id": "5", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_5_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -77,7 +77,7 @@ "entity_id": "6", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_6_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -88,7 +88,7 @@ "entity_id": "7", "port": "9094", "broker.id": "4", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_7_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -99,7 +99,7 @@ "entity_id": "8", "port": "9095", "broker.id": "5", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_8_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -110,7 +110,7 @@ "entity_id": "9", "port": "9096", "broker.id": "6", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_9_logs", "default.replication.factor": "3", "num.partitions": "5", diff --git a/system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json b/system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json index 02af3e86357db..1f0b7180a9d0a 100644 --- a/system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json +++ b/system_test/offset_management_testsuite/testcase_7001/testcase_7001_properties.json @@ -30,7 +30,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -41,7 +41,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -52,7 +52,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -63,7 +63,7 @@ "entity_id": "4", "port": "9094", "broker.id": "4", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_4_logs", "default.replication.factor": "3", "num.partitions": "5", diff --git a/system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json b/system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json index fdab69b41dbc9..c5866a2ecc42e 100644 --- a/system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json +++ b/system_test/offset_management_testsuite/testcase_7002/testcase_7002_properties.json @@ -30,7 +30,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -41,7 +41,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -52,7 +52,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "5", @@ -63,7 +63,7 @@ "entity_id": "4", "port": "9094", "broker.id": "4", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_4_logs", "default.replication.factor": "3", "num.partitions": "5", diff --git a/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json b/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json index eaaa4ed869995..308f1937bbdc0 100644 --- a/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json +++ b/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:sync, acks:-1, comp:0", - "07":"Log segment size : 10240" + "07":"Log segment size : 20480 }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_0002/testcase_0002_properties.json b/system_test/replication_testsuite/testcase_0002/testcase_0002_properties.json index 0ffbf67084b67..3f9e7d05db17e 100644 --- a/system_test/replication_testsuite/testcase_0002/testcase_0002_properties.json +++ b/system_test/replication_testsuite/testcase_0002/testcase_0002_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:sync, acks:-1, comp:1", - "07":"Log segment size : 10240" + "07":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_0003/testcase_0003_properties.json b/system_test/replication_testsuite/testcase_0003/testcase_0003_properties.json index e2fb579b9c0b1..b10c6266a786a 100644 --- a/system_test/replication_testsuite/testcase_0003/testcase_0003_properties.json +++ b/system_test/replication_testsuite/testcase_0003/testcase_0003_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:sync, acks:1, comp:1", - "07":"Log segment size : 10240" + "07":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_0004/testcase_0004_properties.json b/system_test/replication_testsuite/testcase_0004/testcase_0004_properties.json index 62fbe089fb9fe..22c08023f3d9e 100644 --- a/system_test/replication_testsuite/testcase_0004/testcase_0004_properties.json +++ b/system_test/replication_testsuite/testcase_0004/testcase_0004_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:async, acks:-1, comp:1", - "07":"Log segment size : 10240" + "07":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_0005/testcase_0005_properties.json b/system_test/replication_testsuite/testcase_0005/testcase_0005_properties.json index 02ad59d085e3c..1317d8460f82c 100644 --- a/system_test/replication_testsuite/testcase_0005/testcase_0005_properties.json +++ b/system_test/replication_testsuite/testcase_0005/testcase_0005_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:async, acks:1, comp:1", - "07":"Log segment size : 10240" + "07":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_0006/testcase_0006_properties.json b/system_test/replication_testsuite/testcase_0006/testcase_0006_properties.json index b64304f4d0e4e..d31346506a19d 100644 --- a/system_test/replication_testsuite/testcase_0006/testcase_0006_properties.json +++ b/system_test/replication_testsuite/testcase_0006/testcase_0006_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:sync, acks:-1, comp:1", - "07":"Log segment size : 10240" + "07":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_0007/testcase_0007_properties.json b/system_test/replication_testsuite/testcase_0007/testcase_0007_properties.json index e8507099a8418..bd00f130d7fc6 100644 --- a/system_test/replication_testsuite/testcase_0007/testcase_0007_properties.json +++ b/system_test/replication_testsuite/testcase_0007/testcase_0007_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:async, acks:-1, comp:1", - "07":"Log segment size : 10240" + "07":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_0008/testcase_0008_properties.json b/system_test/replication_testsuite/testcase_0008/testcase_0008_properties.json index 47217cf66c758..ac5c13b09a84f 100644 --- a/system_test/replication_testsuite/testcase_0008/testcase_0008_properties.json +++ b/system_test/replication_testsuite/testcase_0008/testcase_0008_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:sync, acks:1, comp:1", - "07":"Log segment size : 10240" + "07":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_0009/testcase_0009_properties.json b/system_test/replication_testsuite/testcase_0009/testcase_0009_properties.json index 3ddaad422bc86..6dadd7a288664 100644 --- a/system_test/replication_testsuite/testcase_0009/testcase_0009_properties.json +++ b/system_test/replication_testsuite/testcase_0009/testcase_0009_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:async, acks:1, comp:1", - "07":"Log segment size : 10240" + "07":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -29,7 +29,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -40,7 +40,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -51,7 +51,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_0101/testcase_0101_properties.json b/system_test/replication_testsuite/testcase_0101/testcase_0101_properties.json index 62381518a1b52..614cb1cf01d3c 100644 --- a/system_test/replication_testsuite/testcase_0101/testcase_0101_properties.json +++ b/system_test/replication_testsuite/testcase_0101/testcase_0101_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:sync, acks:-1, comp:0", - "10":"Log segment size : 10240" + "10":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_0102/testcase_0102_properties.json b/system_test/replication_testsuite/testcase_0102/testcase_0102_properties.json index 69203c3f5b412..fbf01772545f0 100644 --- a/system_test/replication_testsuite/testcase_0102/testcase_0102_properties.json +++ b/system_test/replication_testsuite/testcase_0102/testcase_0102_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:sync, acks:-1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_0103/testcase_0103_properties.json b/system_test/replication_testsuite/testcase_0103/testcase_0103_properties.json index 6bc54ced174b0..317a6e3d33840 100644 --- a/system_test/replication_testsuite/testcase_0103/testcase_0103_properties.json +++ b/system_test/replication_testsuite/testcase_0103/testcase_0103_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:sync, acks:1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_0104/testcase_0104_properties.json b/system_test/replication_testsuite/testcase_0104/testcase_0104_properties.json index 4b3cfe70f6178..d1a790b34fa5b 100644 --- a/system_test/replication_testsuite/testcase_0104/testcase_0104_properties.json +++ b/system_test/replication_testsuite/testcase_0104/testcase_0104_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:async, acks:-1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_0105/testcase_0105_properties.json b/system_test/replication_testsuite/testcase_0105/testcase_0105_properties.json index a913a6c8d2de0..8f4f8bfac7146 100644 --- a/system_test/replication_testsuite/testcase_0105/testcase_0105_properties.json +++ b/system_test/replication_testsuite/testcase_0105/testcase_0105_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:async, acks:1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_0106/testcase_0106_properties.json b/system_test/replication_testsuite/testcase_0106/testcase_0106_properties.json index 55b0bdbb1e81c..a307f855c9449 100644 --- a/system_test/replication_testsuite/testcase_0106/testcase_0106_properties.json +++ b/system_test/replication_testsuite/testcase_0106/testcase_0106_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:sync, acks:-1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_0107/testcase_0107_properties.json b/system_test/replication_testsuite/testcase_0107/testcase_0107_properties.json index 608ed87132189..827319eb6c6db 100644 --- a/system_test/replication_testsuite/testcase_0107/testcase_0107_properties.json +++ b/system_test/replication_testsuite/testcase_0107/testcase_0107_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:async, acks:-1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_0108/testcase_0108_properties.json b/system_test/replication_testsuite/testcase_0108/testcase_0108_properties.json index f1e9caf373079..ae015cd0eb82b 100644 --- a/system_test/replication_testsuite/testcase_0108/testcase_0108_properties.json +++ b/system_test/replication_testsuite/testcase_0108/testcase_0108_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:sync, acks:1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_0109/testcase_0109_properties.json b/system_test/replication_testsuite/testcase_0109/testcase_0109_properties.json index 26fa2ac751bea..1951a1220956f 100644 --- a/system_test/replication_testsuite/testcase_0109/testcase_0109_properties.json +++ b/system_test/replication_testsuite/testcase_0109/testcase_0109_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:async, acks:1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_1/testcase_1_properties.json b/system_test/replication_testsuite/testcase_1/testcase_1_properties.json index 2398722977419..0c6d7a316cc6b 100644 --- a/system_test/replication_testsuite/testcase_1/testcase_1_properties.json +++ b/system_test/replication_testsuite/testcase_1/testcase_1_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:sync, acks:-1, comp:0", - "10":"Log segment size : 10240" + "10":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "log_filename": "kafka_server_9091.log", "config_filename": "kafka_server_9091.properties" @@ -41,7 +41,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "log_filename": "kafka_server_9092.log", "config_filename": "kafka_server_9092.properties" @@ -50,7 +50,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "log_filename": "kafka_server_9093.log", "config_filename": "kafka_server_9093.properties" diff --git a/system_test/replication_testsuite/testcase_10101/testcase_0101_properties.json b/system_test/replication_testsuite/testcase_10101/testcase_0101_properties.json index 85e4b61751b12..3f8e5870799b7 100644 --- a/system_test/replication_testsuite/testcase_10101/testcase_0101_properties.json +++ b/system_test/replication_testsuite/testcase_10101/testcase_0101_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:sync, acks:-1, comp:0", - "10":"Log segment size : 10240" + "10":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_10102/testcase_0102_properties.json b/system_test/replication_testsuite/testcase_10102/testcase_0102_properties.json index 0d2f59fbcaa30..c96352d5e18e6 100644 --- a/system_test/replication_testsuite/testcase_10102/testcase_0102_properties.json +++ b/system_test/replication_testsuite/testcase_10102/testcase_0102_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:sync, acks:-1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_10103/testcase_0103_properties.json b/system_test/replication_testsuite/testcase_10103/testcase_0103_properties.json index 34acfa9d5c684..55fa39e148818 100644 --- a/system_test/replication_testsuite/testcase_10103/testcase_0103_properties.json +++ b/system_test/replication_testsuite/testcase_10103/testcase_0103_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:sync, acks:1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_10104/testcase_0104_properties.json b/system_test/replication_testsuite/testcase_10104/testcase_0104_properties.json index 4145345fc433a..15827eb994ae8 100644 --- a/system_test/replication_testsuite/testcase_10104/testcase_0104_properties.json +++ b/system_test/replication_testsuite/testcase_10104/testcase_0104_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:async, acks:-1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_10105/testcase_0105_properties.json b/system_test/replication_testsuite/testcase_10105/testcase_0105_properties.json index 2eecc768e7e97..d1fa1ade07c53 100644 --- a/system_test/replication_testsuite/testcase_10105/testcase_0105_properties.json +++ b/system_test/replication_testsuite/testcase_10105/testcase_0105_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:async, acks:1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "1", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "1", diff --git a/system_test/replication_testsuite/testcase_10106/testcase_0106_properties.json b/system_test/replication_testsuite/testcase_10106/testcase_0106_properties.json index 744174eab1c82..675c76f656efa 100644 --- a/system_test/replication_testsuite/testcase_10106/testcase_0106_properties.json +++ b/system_test/replication_testsuite/testcase_10106/testcase_0106_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:sync, acks:-1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_10107/testcase_0107_properties.json b/system_test/replication_testsuite/testcase_10107/testcase_0107_properties.json index e881b138c1fc0..afc221cdc8bf7 100644 --- a/system_test/replication_testsuite/testcase_10107/testcase_0107_properties.json +++ b/system_test/replication_testsuite/testcase_10107/testcase_0107_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:async, acks:-1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_10108/testcase_0108_properties.json b/system_test/replication_testsuite/testcase_10108/testcase_0108_properties.json index 7b48fdb441ad5..5df72f3d52a6c 100644 --- a/system_test/replication_testsuite/testcase_10108/testcase_0108_properties.json +++ b/system_test/replication_testsuite/testcase_10108/testcase_0108_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:sync, acks:1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", diff --git a/system_test/replication_testsuite/testcase_10109/testcase_0109_properties.json b/system_test/replication_testsuite/testcase_10109/testcase_0109_properties.json index a98ae0390bf55..9b156681ec485 100644 --- a/system_test/replication_testsuite/testcase_10109/testcase_0109_properties.json +++ b/system_test/replication_testsuite/testcase_10109/testcase_0109_properties.json @@ -8,7 +8,7 @@ "07":"At the end it verifies the log size and contents", "08":"Use a consumer to verify no message loss.", "09":"Producer dimensions : mode:async, acks:1, comp:1", - "10":"Log segment size : 10240" + "10":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", @@ -32,7 +32,7 @@ "entity_id": "1", "port": "9091", "broker.id": "1", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_1_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -43,7 +43,7 @@ "entity_id": "2", "port": "9092", "broker.id": "2", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_2_logs", "default.replication.factor": "3", "num.partitions": "3", @@ -54,7 +54,7 @@ "entity_id": "3", "port": "9093", "broker.id": "3", - "log.segment.bytes": "10240", + "log.segment.bytes": "20480", "log.dir": "/tmp/kafka_server_3_logs", "default.replication.factor": "3", "num.partitions": "3", From 7062ed7db36f913328a9124797c47f70e3461025 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Sun, 12 Oct 2014 15:50:31 -0700 Subject: [PATCH 016/491] KAFKA-1692 Include client ID in new producer IO thread name; reviewed by Neha Narkhede --- .../java/org/apache/kafka/clients/producer/KafkaProducer.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index f58b8508d3f81..32f444ebbd278 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -139,7 +139,8 @@ private KafkaProducer(ProducerConfig config) { config.getInt(ProducerConfig.TIMEOUT_CONFIG), this.metrics, new SystemTime()); - this.ioThread = new KafkaThread("kafka-producer-network-thread", this.sender, true); + String ioThreadName = "kafka-producer-network-thread" + (clientId.length() > 0 ? " | " + clientId : ""); + this.ioThread = new KafkaThread(ioThreadName, this.sender, true); this.ioThread.start(); this.errors = this.metrics.sensor("errors"); From 0d65f043fed14c482a788d403c6a05544c0dd01b Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Sun, 12 Oct 2014 16:15:40 -0700 Subject: [PATCH 017/491] KAFKA-1471 Add producer unit tests for LZ4 and LZ4HC compression codecs; patched by James Oliver; reviewed by Neha Narkhede --- .../org/apache/kafka/clients/producer/ProducerConfig.java | 4 ++-- core/src/main/scala/kafka/tools/ConsoleProducer.scala | 4 ++-- core/src/main/scala/kafka/tools/PerfConfig.scala | 2 +- .../scala/integration/kafka/api/ProducerCompressionTest.scala | 3 +++ core/src/test/scala/unit/kafka/message/MessageTest.scala | 2 +- 5 files changed, 9 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 79d57f9bf3160..bf4ed66791b9a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -153,8 +153,8 @@ public class ProducerConfig extends AbstractConfig { /** compression.type */ public static final String COMPRESSION_TYPE_CONFIG = "compression.type"; - private static final String COMPRESSION_TYPE_DOC = "The compression type for all data generated by the producer. The default is none (i.e. no compression). Valid " + " values are none, gzip, or snappy. Compression is of full batches of data, " - + " so the efficacy of batching will also impact the compression ratio (more batching means better compression)."; + private static final String COMPRESSION_TYPE_DOC = "The compression type for all data generated by the producer. The default is none (i.e. no compression). Valid " + " values are none, gzip, snappy, lz4, or lz4hc. " + + "Compression is of full batches of data, so the efficacy of batching will also impact the compression ratio (more batching means better compression)."; /** metrics.sample.window.ms */ public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = "metrics.sample.window.ms"; diff --git a/core/src/main/scala/kafka/tools/ConsoleProducer.scala b/core/src/main/scala/kafka/tools/ConsoleProducer.scala index 8e9ba0b284671..b024a693c23cb 100644 --- a/core/src/main/scala/kafka/tools/ConsoleProducer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleProducer.scala @@ -113,8 +113,8 @@ object ConsoleProducer { .describedAs("broker-list") .ofType(classOf[String]) val syncOpt = parser.accepts("sync", "If set message send requests to the brokers are synchronously, one at a time as they arrive.") - val compressionCodecOpt = parser.accepts("compression-codec", "The compression codec: either 'gzip' or 'snappy'." + - "If specified without value, than it defaults to 'gzip'") + val compressionCodecOpt = parser.accepts("compression-codec", "The compression codec: either 'none', 'gzip', 'snappy', 'lz4', or 'lz4hc'." + + "If specified without value, then it defaults to 'gzip'") .withOptionalArg() .describedAs("compression-codec") .ofType(classOf[String]) diff --git a/core/src/main/scala/kafka/tools/PerfConfig.scala b/core/src/main/scala/kafka/tools/PerfConfig.scala index 129cc013f68d2..c72002976d904 100644 --- a/core/src/main/scala/kafka/tools/PerfConfig.scala +++ b/core/src/main/scala/kafka/tools/PerfConfig.scala @@ -53,7 +53,7 @@ class PerfConfig(args: Array[String]) { .defaultsTo(200) val compressionCodecOpt = parser.accepts("compression-codec", "If set, messages are sent compressed") .withRequiredArg - .describedAs("supported codec: NoCompressionCodec as 0, GZIPCompressionCodec as 1, SnappyCompressionCodec as 2") + .describedAs("supported codec: NoCompressionCodec as 0, GZIPCompressionCodec as 1, SnappyCompressionCodec as 2, LZ4CompressionCodec as 3, LZ4HCCompressionCodec as 4") .ofType(classOf[java.lang.Integer]) .defaultsTo(0) val helpOpt = parser.accepts("help", "Print usage.") diff --git a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala index 17e2c6e9dfd78..c95485170fd8b 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala @@ -121,8 +121,11 @@ object ProducerCompressionTest { @Parameters def parameters: Collection[Array[String]] = { val list = new ArrayList[Array[String]]() + list.add(Array("none")) list.add(Array("gzip")) list.add(Array("snappy")) + list.add(Array("lz4")) + list.add(Array("lz4hc")) list } } diff --git a/core/src/test/scala/unit/kafka/message/MessageTest.scala b/core/src/test/scala/unit/kafka/message/MessageTest.scala index 4837585d03535..958c1a60069ad 100644 --- a/core/src/test/scala/unit/kafka/message/MessageTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageTest.scala @@ -39,7 +39,7 @@ class MessageTest extends JUnitSuite { def setUp(): Unit = { val keys = Array(null, "key".getBytes, "".getBytes) val vals = Array("value".getBytes, "".getBytes, null) - val codecs = Array(NoCompressionCodec, GZIPCompressionCodec) + val codecs = Array(NoCompressionCodec, GZIPCompressionCodec, SnappyCompressionCodec, LZ4CompressionCodec, LZ4HCCompressionCodec) for(k <- keys; v <- vals; codec <- codecs) messages += new MessageTestVal(k, v, codec, new Message(v, k, codec)) } From be2e8a769efe35dc5c360545e6220a9fe652cced Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Mon, 13 Oct 2014 11:36:21 -0700 Subject: [PATCH 018/491] KAFKA-1305. Controller can hang on controlled shutdown with auto leader balance enabled; reviewed by Neha Narkhede and Jun Rao --- core/src/main/scala/kafka/server/KafkaConfig.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 90af698b01ec8..7fcbc16da8986 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -210,7 +210,7 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro val controllerSocketTimeoutMs = props.getInt("controller.socket.timeout.ms", 30000) /* the buffer size for controller-to-broker-channels */ - val controllerMessageQueueSize= props.getInt("controller.message.queue.size", 10) + val controllerMessageQueueSize= props.getInt("controller.message.queue.size", Int.MaxValue) /* default replication factors for automatically created topics */ val defaultReplicationFactor = props.getInt("default.replication.factor", 1) @@ -256,7 +256,7 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* Enables auto leader balancing. A background thread checks and triggers leader * balance if required at regular intervals */ - val autoLeaderRebalanceEnable = props.getBoolean("auto.leader.rebalance.enable", false) + val autoLeaderRebalanceEnable = props.getBoolean("auto.leader.rebalance.enable", true) /* the ratio of leader imbalance allowed per broker. The controller would trigger a leader balance if it goes above * this value per broker. The value is specified in percentage. */ From d5041bc79fd656fe361c7f6643f9f26c2e8f22fe Mon Sep 17 00:00:00 2001 From: Alexis Midon Date: Mon, 13 Oct 2014 16:16:14 -0700 Subject: [PATCH 019/491] kafka-1702; Messages silently Lost by producer; patched by Alexis Midon; reviewed by Jun Rao --- .../producer/async/DefaultEventHandler.scala | 102 +++++++++--------- 1 file changed, 53 insertions(+), 49 deletions(-) diff --git a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala index 33470fff21bd4..821901e4f434d 100644 --- a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala +++ b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala @@ -95,27 +95,28 @@ class DefaultEventHandler[K,V](config: ProducerConfig, val partitionedDataOpt = partitionAndCollate(messages) partitionedDataOpt match { case Some(partitionedData) => - val failedProduceRequests = new ArrayBuffer[KeyedMessage[K,Message]] - try { - for ((brokerid, messagesPerBrokerMap) <- partitionedData) { - if (logger.isTraceEnabled) - messagesPerBrokerMap.foreach(partitionAndEvent => - trace("Handling event for Topic: %s, Broker: %d, Partitions: %s".format(partitionAndEvent._1, brokerid, partitionAndEvent._2))) - val messageSetPerBroker = groupMessagesToSet(messagesPerBrokerMap) - - val failedTopicPartitions = send(brokerid, messageSetPerBroker) - failedTopicPartitions.foreach(topicPartition => { - messagesPerBrokerMap.get(topicPartition) match { - case Some(data) => failedProduceRequests.appendAll(data) - case None => // nothing - } - }) + val failedProduceRequests = new ArrayBuffer[KeyedMessage[K, Message]] + for ((brokerid, messagesPerBrokerMap) <- partitionedData) { + if (logger.isTraceEnabled) { + messagesPerBrokerMap.foreach(partitionAndEvent => + trace("Handling event for Topic: %s, Broker: %d, Partitions: %s".format(partitionAndEvent._1, brokerid, partitionAndEvent._2))) + } + val messageSetPerBrokerOpt = groupMessagesToSet(messagesPerBrokerMap) + messageSetPerBrokerOpt match { + case Some(messageSetPerBroker) => + val failedTopicPartitions = send(brokerid, messageSetPerBroker) + failedTopicPartitions.foreach(topicPartition => { + messagesPerBrokerMap.get(topicPartition) match { + case Some(data) => failedProduceRequests.appendAll(data) + case None => // nothing + } + }) + case None => // failed to group messages + messagesPerBrokerMap.values.foreach(m => failedProduceRequests.appendAll(m)) } - } catch { - case t: Throwable => error("Failed to send messages", t) } failedProduceRequests - case None => // all produce requests failed + case None => // failed to collate messages messages } } @@ -290,43 +291,46 @@ class DefaultEventHandler[K,V](config: ProducerConfig, } } - private def groupMessagesToSet(messagesPerTopicAndPartition: collection.mutable.Map[TopicAndPartition, Seq[KeyedMessage[K,Message]]]) = { + private def groupMessagesToSet(messagesPerTopicAndPartition: collection.mutable.Map[TopicAndPartition, Seq[KeyedMessage[K, Message]]]) = { /** enforce the compressed.topics config here. - * If the compression codec is anything other than NoCompressionCodec, - * Enable compression only for specified topics if any - * If the list of compressed topics is empty, then enable the specified compression codec for all topics - * If the compression codec is NoCompressionCodec, compression is disabled for all topics + * If the compression codec is anything other than NoCompressionCodec, + * Enable compression only for specified topics if any + * If the list of compressed topics is empty, then enable the specified compression codec for all topics + * If the compression codec is NoCompressionCodec, compression is disabled for all topics */ - - val messagesPerTopicPartition = messagesPerTopicAndPartition.map { case (topicAndPartition, messages) => - val rawMessages = messages.map(_.message) - ( topicAndPartition, - config.compressionCodec match { - case NoCompressionCodec => - debug("Sending %d messages with no compression to %s".format(messages.size, topicAndPartition)) - new ByteBufferMessageSet(NoCompressionCodec, rawMessages: _*) - case _ => - config.compressedTopics.size match { - case 0 => - debug("Sending %d messages with compression codec %d to %s" - .format(messages.size, config.compressionCodec.codec, topicAndPartition)) - new ByteBufferMessageSet(config.compressionCodec, rawMessages: _*) - case _ => - if(config.compressedTopics.contains(topicAndPartition.topic)) { + try { + val messagesPerTopicPartition = messagesPerTopicAndPartition.map { case (topicAndPartition, messages) => + val rawMessages = messages.map(_.message) + (topicAndPartition, + config.compressionCodec match { + case NoCompressionCodec => + debug("Sending %d messages with no compression to %s".format(messages.size, topicAndPartition)) + new ByteBufferMessageSet(NoCompressionCodec, rawMessages: _*) + case _ => + config.compressedTopics.size match { + case 0 => debug("Sending %d messages with compression codec %d to %s" .format(messages.size, config.compressionCodec.codec, topicAndPartition)) new ByteBufferMessageSet(config.compressionCodec, rawMessages: _*) - } - else { - debug("Sending %d messages to %s with no compression as it is not in compressed.topics - %s" - .format(messages.size, topicAndPartition, config.compressedTopics.toString)) - new ByteBufferMessageSet(NoCompressionCodec, rawMessages: _*) - } - } - } - ) + case _ => + if (config.compressedTopics.contains(topicAndPartition.topic)) { + debug("Sending %d messages with compression codec %d to %s" + .format(messages.size, config.compressionCodec.codec, topicAndPartition)) + new ByteBufferMessageSet(config.compressionCodec, rawMessages: _*) + } + else { + debug("Sending %d messages to %s with no compression as it is not in compressed.topics - %s" + .format(messages.size, topicAndPartition, config.compressedTopics.toString)) + new ByteBufferMessageSet(NoCompressionCodec, rawMessages: _*) + } + } + } + ) + } + Some(messagesPerTopicPartition) + } catch { + case t: Throwable => error("Failed to group messages", t); None } - messagesPerTopicPartition } def close() { From 22643bfc2b683b426eb41d4b66cf90c3b7ec7f0d Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Tue, 14 Oct 2014 21:16:48 -0700 Subject: [PATCH 020/491] KAFKA-979 Add jitter for time based rolling; reviewed by Neha Narkhede and Joel Koshy --- core/src/main/scala/kafka/log/Log.scala | 8 ++++-- .../src/main/scala/kafka/log/LogCleaner.scala | 2 +- core/src/main/scala/kafka/log/LogConfig.scala | 12 +++++++++ .../src/main/scala/kafka/log/LogSegment.scala | 6 +++-- .../main/scala/kafka/server/KafkaConfig.scala | 16 +++++++++++- .../main/scala/kafka/server/KafkaServer.scala | 3 ++- .../scala/unit/kafka/log/LogSegmentTest.scala | 2 +- .../test/scala/unit/kafka/log/LogTest.scala | 26 +++++++++++++++++++ 8 files changed, 67 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index a123cdc52f341..157d67369baab 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -141,6 +141,7 @@ class Log(val dir: File, startOffset = start, indexIntervalBytes = config.indexInterval, maxIndexSize = config.maxIndexSize, + rollJitterMs = config.randomSegmentJitter, time = time) if(!hasIndex) { error("Could not find index file corresponding to log file %s, rebuilding index...".format(segment.log.file.getAbsolutePath)) @@ -156,6 +157,7 @@ class Log(val dir: File, startOffset = 0, indexIntervalBytes = config.indexInterval, maxIndexSize = config.maxIndexSize, + rollJitterMs = config.randomSegmentJitter, time = time)) } else { recoverLog() @@ -510,7 +512,7 @@ class Log(val dir: File, private def maybeRoll(messagesSize: Int): LogSegment = { val segment = activeSegment if (segment.size > config.segmentSize - messagesSize || - segment.size > 0 && time.milliseconds - segment.created > config.segmentMs || + segment.size > 0 && time.milliseconds - segment.created > config.segmentMs - segment.rollJitterMs || segment.index.isFull) { debug("Rolling new log segment in %s (log_size = %d/%d, index_size = %d/%d, age_ms = %d/%d)." .format(name, @@ -519,7 +521,7 @@ class Log(val dir: File, segment.index.entries, segment.index.maxEntries, time.milliseconds - segment.created, - config.segmentMs)) + config.segmentMs - segment.rollJitterMs)) roll() } else { segment @@ -550,6 +552,7 @@ class Log(val dir: File, startOffset = newOffset, indexIntervalBytes = config.indexInterval, maxIndexSize = config.maxIndexSize, + rollJitterMs = config.randomSegmentJitter, time = time) val prev = addSegment(segment) if(prev != null) @@ -642,6 +645,7 @@ class Log(val dir: File, newOffset, indexIntervalBytes = config.indexInterval, maxIndexSize = config.maxIndexSize, + rollJitterMs = config.randomSegmentJitter, time = time)) updateLogEndOffset(newOffset) this.recoveryPoint = math.min(newOffset, this.recoveryPoint) diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index c20de4ad4734c..f8fcb843c80ee 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -340,7 +340,7 @@ private[log] class Cleaner(val id: Int, indexFile.delete() val messages = new FileMessageSet(logFile) val index = new OffsetIndex(indexFile, segments.head.baseOffset, segments.head.index.maxIndexSize) - val cleaned = new LogSegment(messages, index, segments.head.baseOffset, segments.head.indexIntervalBytes, time) + val cleaned = new LogSegment(messages, index, segments.head.baseOffset, segments.head.indexIntervalBytes, log.config.randomSegmentJitter, time) try { // clean segments into the new destination segment diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index d2cc9e3d6b7a4..e48922a97727d 100644 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -18,12 +18,15 @@ package kafka.log import java.util.Properties +import org.apache.kafka.common.utils.Utils + import scala.collection._ import kafka.common._ object Defaults { val SegmentSize = 1024 * 1024 val SegmentMs = Long.MaxValue + val SegmentJitterMs = 0L val FlushInterval = Long.MaxValue val FlushMs = Long.MaxValue val RetentionSize = Long.MaxValue @@ -43,6 +46,7 @@ object Defaults { * Configuration settings for a log * @param segmentSize The soft maximum for the size of a segment file in the log * @param segmentMs The soft maximum on the amount of time before a new log segment is rolled + * @param segmentJitterMs The maximum random jitter subtracted from segmentMs to avoid thundering herds of segment rolling * @param flushInterval The number of messages that can be written to the log before a flush is forced * @param flushMs The amount of time the log can have dirty data before a flush is forced * @param retentionSize The approximate total number of bytes this log can use @@ -60,6 +64,7 @@ object Defaults { */ case class LogConfig(val segmentSize: Int = Defaults.SegmentSize, val segmentMs: Long = Defaults.SegmentMs, + val segmentJitterMs: Long = Defaults.SegmentJitterMs, val flushInterval: Long = Defaults.FlushInterval, val flushMs: Long = Defaults.FlushMs, val retentionSize: Long = Defaults.RetentionSize, @@ -79,6 +84,7 @@ case class LogConfig(val segmentSize: Int = Defaults.SegmentSize, import LogConfig._ props.put(SegmentBytesProp, segmentSize.toString) props.put(SegmentMsProp, segmentMs.toString) + props.put(SegmentJitterMsProp, segmentJitterMs.toString) props.put(SegmentIndexBytesProp, maxIndexSize.toString) props.put(FlushMessagesProp, flushInterval.toString) props.put(FlushMsProp, flushMs.toString) @@ -94,11 +100,15 @@ case class LogConfig(val segmentSize: Int = Defaults.SegmentSize, props.put(MinInSyncReplicasProp, minInSyncReplicas.toString) props } + + def randomSegmentJitter: Long = + if (segmentJitterMs == 0) 0 else Utils.abs(scala.util.Random.nextInt()) % math.min(segmentJitterMs, segmentMs) } object LogConfig { val SegmentBytesProp = "segment.bytes" val SegmentMsProp = "segment.ms" + val SegmentJitterMsProp = "segment.jitter.ms" val SegmentIndexBytesProp = "segment.index.bytes" val FlushMessagesProp = "flush.messages" val FlushMsProp = "flush.ms" @@ -115,6 +125,7 @@ object LogConfig { val ConfigNames = Set(SegmentBytesProp, SegmentMsProp, + SegmentJitterMsProp, SegmentIndexBytesProp, FlushMessagesProp, FlushMsProp, @@ -135,6 +146,7 @@ object LogConfig { def fromProps(props: Properties): LogConfig = { new LogConfig(segmentSize = props.getProperty(SegmentBytesProp, Defaults.SegmentSize.toString).toInt, segmentMs = props.getProperty(SegmentMsProp, Defaults.SegmentMs.toString).toLong, + segmentJitterMs = props.getProperty(SegmentJitterMsProp, Defaults.SegmentJitterMs.toString).toLong, maxIndexSize = props.getProperty(SegmentIndexBytesProp, Defaults.MaxIndexSize.toString).toInt, flushInterval = props.getProperty(FlushMessagesProp, Defaults.FlushInterval.toString).toLong, flushMs = props.getProperty(FlushMsProp, Defaults.FlushMs.toString).toLong, diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index 7597d309f37a0..ac9643423a28d 100644 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -44,18 +44,20 @@ class LogSegment(val log: FileMessageSet, val index: OffsetIndex, val baseOffset: Long, val indexIntervalBytes: Int, + val rollJitterMs: Long, time: Time) extends Logging { var created = time.milliseconds - + /* the number of bytes since we last added an entry in the offset index */ private var bytesSinceLastIndexEntry = 0 - def this(dir: File, startOffset: Long, indexIntervalBytes: Int, maxIndexSize: Int, time: Time) = + def this(dir: File, startOffset: Long, indexIntervalBytes: Int, maxIndexSize: Int, rollJitterMs: Long, time: Time) = this(new FileMessageSet(file = Log.logFilename(dir, startOffset)), new OffsetIndex(file = Log.indexFilename(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize), startOffset, indexIntervalBytes, + rollJitterMs, time) /* Return the size in bytes of this log segment */ diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 7fcbc16da8986..6e26c5436feb4 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -57,7 +57,18 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro millisInHour * props.getIntInRange("log.roll.hours", 24*7, (1, Int.MaxValue)) } } - + + private def getLogRollTimeJitterMillis(): Long = { + val millisInHour = 60L * 60L * 1000L + + if(props.containsKey("log.roll.jitter.ms")) { + props.getIntInRange("log.roll.jitter.ms", (0, Int.MaxValue)) + } + else { + millisInHour * props.getIntInRange("log.roll.jitter.hours", 0, (0, Int.MaxValue)) + } + } + /*********** General Configuration ***********/ /* the broker id for this server */ @@ -131,6 +142,9 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* the maximum time before a new log segment is rolled out */ val logRollTimeMillis = getLogRollTimeMillis + /* the maximum jitter to subtract from logRollTimeMillis */ + val logRollTimeJitterMillis = getLogRollTimeJitterMillis + /* the number of hours to keep a log file before deleting it */ val logRetentionTimeMillis = getLogRetentionTimeMillis diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 3e9e91f2b456b..07c0a078ffa51 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -309,8 +309,9 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg def getLogManager(): LogManager = logManager private def createLogManager(zkClient: ZkClient, brokerState: BrokerState): LogManager = { - val defaultLogConfig = LogConfig(segmentSize = config.logSegmentBytes, + val defaultLogConfig = LogConfig(segmentSize = config.logSegmentBytes, segmentMs = config.logRollTimeMillis, + segmentJitterMs = config.logRollTimeJitterMillis, flushInterval = config.logFlushIntervalMessages, flushMs = config.logFlushIntervalMs.toLong, retentionSize = config.logRetentionBytes, diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala index 7b97e6a80753a..03fb3512c4a44 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala @@ -39,7 +39,7 @@ class LogSegmentTest extends JUnit3Suite { val idxFile = TestUtils.tempFile() idxFile.delete() val idx = new OffsetIndex(idxFile, offset, 1000) - val seg = new LogSegment(ms, idx, offset, 10, SystemTime) + val seg = new LogSegment(ms, idx, offset, 10, 0, SystemTime) segments += seg seg } diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index a0cbd3bbbeeab..d670ba76acd54 100644 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -87,6 +87,32 @@ class LogTest extends JUnitSuite { assertEquals("Appending an empty message set should not roll log even if succient time has passed.", numSegments, log.numberOfSegments) } + /** + * Test for jitter s for time based log roll. This test appends messages then changes the time + * using the mock clock to force the log to roll and checks the number of segments. + */ + @Test + def testTimeBasedLogRollJitter() { + val set = TestUtils.singleMessageSet("test".getBytes()) + val maxJitter = 20 * 60L + + // create a log + val log = new Log(logDir, + logConfig.copy(segmentMs = 1 * 60 * 60L, segmentJitterMs = maxJitter), + recoveryPoint = 0L, + scheduler = time.scheduler, + time = time) + assertEquals("Log begins with a single empty segment.", 1, log.numberOfSegments) + log.append(set) + + time.sleep(log.config.segmentMs - maxJitter) + log.append(set) + assertEquals("Log does not roll on this append because it occurs earlier than max jitter", 1, log.numberOfSegments); + time.sleep(maxJitter - log.activeSegment.rollJitterMs + 1) + log.append(set) + assertEquals("Log should roll after segmentMs adjusted by random jitter", 2, log.numberOfSegments) + } + /** * Test that appending more than the maximum segment size rolls the log */ From de432a09e632f78df9e580b51277f81582c3f026 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Tue, 14 Oct 2014 21:46:53 -0700 Subject: [PATCH 021/491] KAFKA-1637 SimpleConsumer.fetchOffset returns wrong error code when no offset exists for topic/partition/consumer group; reviewed by Neha Narkhede --- .../kafka/common/OffsetMetadataAndError.scala | 1 + .../main/scala/kafka/server/KafkaApis.scala | 8 +++++++- .../unit/kafka/server/OffsetCommitTest.scala | 19 ++++++++++++++++--- 3 files changed, 24 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala index 1586243d20d6a..4cabffeacea09 100644 --- a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala +++ b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala @@ -51,5 +51,6 @@ object OffsetMetadataAndError { val NoOffset = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.NoError) val OffsetsLoading = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.OffsetsLoadInProgressCode) val NotOffsetManagerForGroup = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.NotCoordinatorForConsumerCode) + val UnknownTopicOrPartition = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.UnknownTopicOrPartitionCode) } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 67f2833804cb1..6ad64d2fae8dd 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -505,7 +505,13 @@ class KafkaApis(val requestChannel: RequestChannel, def handleOffsetFetchRequest(request: RequestChannel.Request) { val offsetFetchRequest = request.requestObj.asInstanceOf[OffsetFetchRequest] - val status = offsetManager.getOffsets(offsetFetchRequest.groupId, offsetFetchRequest.requestInfo).toMap + // Missing + val (missingTopicPartitions, availableTopicPartitions) = offsetFetchRequest.requestInfo.partition(topicAndPartition => + replicaManager.getPartition(topicAndPartition.topic, topicAndPartition.partition).isEmpty + ) + val missingStatus = missingTopicPartitions.map(topicAndPartition => (topicAndPartition, OffsetMetadataAndError.UnknownTopicOrPartition)).toMap + val availableStatus = offsetManager.getOffsets(offsetFetchRequest.groupId, availableTopicPartitions).toMap + val status = missingStatus ++ availableStatus val response = OffsetFetchResponse(status, offsetFetchRequest.correlationId) diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index 2d9325045ac1a..8c5364fa97da1 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -116,7 +116,13 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { val topic1 = "topic-1" val topic2 = "topic-2" val topic3 = "topic-3" - val topic4 = "topic-4" + val topic4 = "topic-4" // Topic that group never consumes + val topic5 = "topic-5" // Non-existent topic + + createTopic(zkClient, topic1, servers = Seq(server), numPartitions = 1) + createTopic(zkClient, topic2, servers = Seq(server), numPartitions = 2) + createTopic(zkClient, topic3, servers = Seq(server), numPartitions = 1) + createTopic(zkClient, topic4, servers = Seq(server), numPartitions = 1) val commitRequest = OffsetCommitRequest("test-group", immutable.Map( TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L, metadata="metadata one"), @@ -136,7 +142,8 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { TopicAndPartition(topic3, 0), TopicAndPartition(topic2, 1), TopicAndPartition(topic3, 1), // An unknown partition - TopicAndPartition(topic4, 0) // An unknown topic + TopicAndPartition(topic4, 0), // An unused topic + TopicAndPartition(topic5, 0) // An unknown topic )) val fetchResponse = simpleConsumer.fetchOffsets(fetchRequest) @@ -144,8 +151,12 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.error) assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.error) assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.error) - assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get) + assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.error) + assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.error) + assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.error) + assertEquals(OffsetMetadataAndError.UnknownTopicOrPartition, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get) assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get) + assertEquals(OffsetMetadataAndError.UnknownTopicOrPartition, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get) assertEquals("metadata one", fetchResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get.metadata) assertEquals("metadata two", fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.metadata) @@ -153,6 +164,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.metadata) assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.metadata) assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.metadata) + assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.metadata) assertEquals(42L, fetchResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get.offset) assertEquals(43L, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.offset) @@ -160,6 +172,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(45L, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.offset) assertEquals(OffsetAndMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.offset) assertEquals(OffsetAndMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.offset) + assertEquals(OffsetAndMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.offset) } @Test From 841387b23ac30707dd7f24a79de441c3302458d4 Mon Sep 17 00:00:00 2001 From: Joel Koshy Date: Tue, 14 Oct 2014 22:31:50 -0700 Subject: [PATCH 022/491] Revert "KAFKA-1637 SimpleConsumer.fetchOffset returns wrong error code when no offset exists for topic/partition/consumer group; reviewed by Neha Narkhede" This reverts commit de432a09e632f78df9e580b51277f81582c3f026. --- .../kafka/common/OffsetMetadataAndError.scala | 1 - .../main/scala/kafka/server/KafkaApis.scala | 8 +------- .../unit/kafka/server/OffsetCommitTest.scala | 19 +++---------------- 3 files changed, 4 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala index 4cabffeacea09..1586243d20d6a 100644 --- a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala +++ b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala @@ -51,6 +51,5 @@ object OffsetMetadataAndError { val NoOffset = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.NoError) val OffsetsLoading = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.OffsetsLoadInProgressCode) val NotOffsetManagerForGroup = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.NotCoordinatorForConsumerCode) - val UnknownTopicOrPartition = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.UnknownTopicOrPartitionCode) } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 6ad64d2fae8dd..67f2833804cb1 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -505,13 +505,7 @@ class KafkaApis(val requestChannel: RequestChannel, def handleOffsetFetchRequest(request: RequestChannel.Request) { val offsetFetchRequest = request.requestObj.asInstanceOf[OffsetFetchRequest] - // Missing - val (missingTopicPartitions, availableTopicPartitions) = offsetFetchRequest.requestInfo.partition(topicAndPartition => - replicaManager.getPartition(topicAndPartition.topic, topicAndPartition.partition).isEmpty - ) - val missingStatus = missingTopicPartitions.map(topicAndPartition => (topicAndPartition, OffsetMetadataAndError.UnknownTopicOrPartition)).toMap - val availableStatus = offsetManager.getOffsets(offsetFetchRequest.groupId, availableTopicPartitions).toMap - val status = missingStatus ++ availableStatus + val status = offsetManager.getOffsets(offsetFetchRequest.groupId, offsetFetchRequest.requestInfo).toMap val response = OffsetFetchResponse(status, offsetFetchRequest.correlationId) diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index 8c5364fa97da1..2d9325045ac1a 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -116,13 +116,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { val topic1 = "topic-1" val topic2 = "topic-2" val topic3 = "topic-3" - val topic4 = "topic-4" // Topic that group never consumes - val topic5 = "topic-5" // Non-existent topic - - createTopic(zkClient, topic1, servers = Seq(server), numPartitions = 1) - createTopic(zkClient, topic2, servers = Seq(server), numPartitions = 2) - createTopic(zkClient, topic3, servers = Seq(server), numPartitions = 1) - createTopic(zkClient, topic4, servers = Seq(server), numPartitions = 1) + val topic4 = "topic-4" val commitRequest = OffsetCommitRequest("test-group", immutable.Map( TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L, metadata="metadata one"), @@ -142,8 +136,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { TopicAndPartition(topic3, 0), TopicAndPartition(topic2, 1), TopicAndPartition(topic3, 1), // An unknown partition - TopicAndPartition(topic4, 0), // An unused topic - TopicAndPartition(topic5, 0) // An unknown topic + TopicAndPartition(topic4, 0) // An unknown topic )) val fetchResponse = simpleConsumer.fetchOffsets(fetchRequest) @@ -151,12 +144,8 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.error) assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.error) assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.error) - assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.error) - assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.error) - assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.error) - assertEquals(OffsetMetadataAndError.UnknownTopicOrPartition, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get) + assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get) assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get) - assertEquals(OffsetMetadataAndError.UnknownTopicOrPartition, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get) assertEquals("metadata one", fetchResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get.metadata) assertEquals("metadata two", fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.metadata) @@ -164,7 +153,6 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.metadata) assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.metadata) assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.metadata) - assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.metadata) assertEquals(42L, fetchResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get.offset) assertEquals(43L, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.offset) @@ -172,7 +160,6 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(45L, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.offset) assertEquals(OffsetAndMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.offset) assertEquals(OffsetAndMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.offset) - assertEquals(OffsetAndMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.offset) } @Test From 4271ecbf06d0ffe16db3222b4e75247641a4c35f Mon Sep 17 00:00:00 2001 From: Neha Narkhede Date: Wed, 15 Oct 2014 17:07:30 -0700 Subject: [PATCH 023/491] KAFKA-1637 SimpleConsumer.fetchOffset returns wrong error code when no offset exists for topic/partition/consumer group; reviewed by Neha Narkhede and Joel Koshy --- .../kafka/common/OffsetMetadataAndError.scala | 1 + .../main/scala/kafka/server/KafkaApis.scala | 7 ++++++- .../unit/kafka/server/OffsetCommitTest.scala | 19 ++++++++++++++++--- 3 files changed, 23 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala index 1586243d20d6a..4cabffeacea09 100644 --- a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala +++ b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala @@ -51,5 +51,6 @@ object OffsetMetadataAndError { val NoOffset = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.NoError) val OffsetsLoading = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.OffsetsLoadInProgressCode) val NotOffsetManagerForGroup = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.NotCoordinatorForConsumerCode) + val UnknownTopicOrPartition = OffsetMetadataAndError(OffsetAndMetadata.InvalidOffset, OffsetAndMetadata.NoMetadata, ErrorMapping.UnknownTopicOrPartitionCode) } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 67f2833804cb1..85498b4a1368d 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -505,7 +505,12 @@ class KafkaApis(val requestChannel: RequestChannel, def handleOffsetFetchRequest(request: RequestChannel.Request) { val offsetFetchRequest = request.requestObj.asInstanceOf[OffsetFetchRequest] - val status = offsetManager.getOffsets(offsetFetchRequest.groupId, offsetFetchRequest.requestInfo).toMap + val (unknownTopicPartitions, knownTopicPartitions) = offsetFetchRequest.requestInfo.partition(topicAndPartition => + metadataCache.getPartitionInfo(topicAndPartition.topic, topicAndPartition.partition).isEmpty + ) + val unknownStatus = unknownTopicPartitions.map(topicAndPartition => (topicAndPartition, OffsetMetadataAndError.UnknownTopicOrPartition)).toMap + val knownStatus = offsetManager.getOffsets(offsetFetchRequest.groupId, knownTopicPartitions).toMap + val status = unknownStatus ++ knownStatus val response = OffsetFetchResponse(status, offsetFetchRequest.correlationId) diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index 2d9325045ac1a..8c5364fa97da1 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -116,7 +116,13 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { val topic1 = "topic-1" val topic2 = "topic-2" val topic3 = "topic-3" - val topic4 = "topic-4" + val topic4 = "topic-4" // Topic that group never consumes + val topic5 = "topic-5" // Non-existent topic + + createTopic(zkClient, topic1, servers = Seq(server), numPartitions = 1) + createTopic(zkClient, topic2, servers = Seq(server), numPartitions = 2) + createTopic(zkClient, topic3, servers = Seq(server), numPartitions = 1) + createTopic(zkClient, topic4, servers = Seq(server), numPartitions = 1) val commitRequest = OffsetCommitRequest("test-group", immutable.Map( TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L, metadata="metadata one"), @@ -136,7 +142,8 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { TopicAndPartition(topic3, 0), TopicAndPartition(topic2, 1), TopicAndPartition(topic3, 1), // An unknown partition - TopicAndPartition(topic4, 0) // An unknown topic + TopicAndPartition(topic4, 0), // An unused topic + TopicAndPartition(topic5, 0) // An unknown topic )) val fetchResponse = simpleConsumer.fetchOffsets(fetchRequest) @@ -144,8 +151,12 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.error) assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 0)).get.error) assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.error) - assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get) + assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.error) + assertEquals(ErrorMapping.NoError, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.error) + assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.error) + assertEquals(OffsetMetadataAndError.UnknownTopicOrPartition, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get) assertEquals(OffsetMetadataAndError.NoOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get) + assertEquals(OffsetMetadataAndError.UnknownTopicOrPartition, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get) assertEquals("metadata one", fetchResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get.metadata) assertEquals("metadata two", fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.metadata) @@ -153,6 +164,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.metadata) assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.metadata) assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.metadata) + assertEquals(OffsetAndMetadata.NoMetadata, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.metadata) assertEquals(42L, fetchResponse.requestInfo.get(TopicAndPartition(topic1, 0)).get.offset) assertEquals(43L, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 0)).get.offset) @@ -160,6 +172,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(45L, fetchResponse.requestInfo.get(TopicAndPartition(topic2, 1)).get.offset) assertEquals(OffsetAndMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic3, 1)).get.offset) assertEquals(OffsetAndMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic4, 0)).get.offset) + assertEquals(OffsetAndMetadata.InvalidOffset, fetchResponse.requestInfo.get(TopicAndPartition(topic5, 0)).get.offset) } @Test From 37356bfee09bf0dffc93d85270c342e03c36ca44 Mon Sep 17 00:00:00 2001 From: James Oliver Date: Fri, 17 Oct 2014 10:07:34 -0700 Subject: [PATCH 024/491] kafka-1493; Use a well-documented LZ4 compression format and remove redundant LZ4HC option; patched by James Oliver; reviewed by Jun Rao --- .../clients/producer/ProducerConfig.java | 2 +- .../message/KafkaLZ4BlockInputStream.java | 233 +++++++++++ .../message/KafkaLZ4BlockOutputStream.java | 387 ++++++++++++++++++ .../kafka/common/record/CompressionType.java | 6 +- .../kafka/common/record/Compressor.java | 21 +- .../org/apache/kafka/common/utils/Utils.java | 60 +++ config/producer.properties | 4 +- .../kafka/message/CompressionCodec.scala | 7 - .../kafka/message/CompressionFactory.scala | 13 +- .../scala/kafka/tools/ConsoleProducer.scala | 2 +- .../main/scala/kafka/tools/PerfConfig.scala | 2 +- .../kafka/api/ProducerCompressionTest.scala | 5 +- .../message/MessageCompressionTest.scala | 12 - .../unit/kafka/message/MessageTest.scala | 2 +- 14 files changed, 698 insertions(+), 58 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/message/KafkaLZ4BlockInputStream.java create mode 100644 clients/src/main/java/org/apache/kafka/common/message/KafkaLZ4BlockOutputStream.java diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index bf4ed66791b9a..9095caf0db1e4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -153,7 +153,7 @@ public class ProducerConfig extends AbstractConfig { /** compression.type */ public static final String COMPRESSION_TYPE_CONFIG = "compression.type"; - private static final String COMPRESSION_TYPE_DOC = "The compression type for all data generated by the producer. The default is none (i.e. no compression). Valid " + " values are none, gzip, snappy, lz4, or lz4hc. " + private static final String COMPRESSION_TYPE_DOC = "The compression type for all data generated by the producer. The default is none (i.e. no compression). Valid " + " values are none, gzip, snappy, or lz4. " + "Compression is of full batches of data, so the efficacy of batching will also impact the compression ratio (more batching means better compression)."; /** metrics.sample.window.ms */ diff --git a/clients/src/main/java/org/apache/kafka/common/message/KafkaLZ4BlockInputStream.java b/clients/src/main/java/org/apache/kafka/common/message/KafkaLZ4BlockInputStream.java new file mode 100644 index 0000000000000..5be72fef1f976 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/message/KafkaLZ4BlockInputStream.java @@ -0,0 +1,233 @@ +/** + * 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.message; + +import static org.apache.kafka.common.message.KafkaLZ4BlockOutputStream.LZ4_FRAME_INCOMPRESSIBLE_MASK; +import static org.apache.kafka.common.message.KafkaLZ4BlockOutputStream.LZ4_MAX_HEADER_LENGTH; +import static org.apache.kafka.common.message.KafkaLZ4BlockOutputStream.MAGIC; + +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; + +import org.apache.kafka.common.message.KafkaLZ4BlockOutputStream.BD; +import org.apache.kafka.common.message.KafkaLZ4BlockOutputStream.FLG; +import org.apache.kafka.common.utils.Utils; + +import net.jpountz.lz4.LZ4Exception; +import net.jpountz.lz4.LZ4Factory; +import net.jpountz.lz4.LZ4SafeDecompressor; +import net.jpountz.xxhash.XXHash32; +import net.jpountz.xxhash.XXHashFactory; + +/** + * A partial implementation of the v1.4.1 LZ4 Frame format. + * + * @see LZ4 Framing Format Spec + */ +public final class KafkaLZ4BlockInputStream extends FilterInputStream { + + public static final String PREMATURE_EOS = "Stream ended prematurely"; + public static final String NOT_SUPPORTED = "Stream unsupported"; + public static final String BLOCK_HASH_MISMATCH = "Block checksum mismatch"; + public static final String DESCRIPTOR_HASH_MISMATCH = "Stream frame descriptor corrupted"; + + private final LZ4SafeDecompressor decompressor; + private final XXHash32 checksum; + private final byte[] buffer; + private final byte[] compressedBuffer; + private final int maxBlockSize; + private FLG flg; + private BD bd; + private int bufferOffset; + private int bufferSize; + private boolean finished; + + /** + * Create a new {@link InputStream} that will decompress data using the LZ4 algorithm. + * + * @param in The stream to decompress + * @throws IOException + */ + public KafkaLZ4BlockInputStream(InputStream in) throws IOException { + super(in); + decompressor = LZ4Factory.fastestInstance().safeDecompressor(); + checksum = XXHashFactory.fastestInstance().hash32(); + readHeader(); + maxBlockSize = bd.getBlockMaximumSize(); + buffer = new byte[maxBlockSize]; + compressedBuffer = new byte[maxBlockSize]; + bufferOffset = 0; + bufferSize = 0; + finished = false; + } + + /** + * Reads the magic number and frame descriptor from the underlying {@link InputStream}. + * + * @throws IOException + */ + private void readHeader() throws IOException { + byte[] header = new byte[LZ4_MAX_HEADER_LENGTH]; + + // read first 6 bytes into buffer to check magic and FLG/BD descriptor flags + bufferOffset = 6; + if (in.read(header, 0, bufferOffset) != bufferOffset) { + throw new IOException(PREMATURE_EOS); + } + + if (MAGIC != Utils.readUnsignedIntLE(header, bufferOffset-6)) { + throw new IOException(NOT_SUPPORTED); + } + flg = FLG.fromByte(header[bufferOffset-2]); + bd = BD.fromByte(header[bufferOffset-1]); + // TODO read uncompressed content size, update flg.validate() + // TODO read dictionary id, update flg.validate() + + // check stream descriptor hash + byte hash = (byte) ((checksum.hash(header, 0, bufferOffset, 0) >> 8) & 0xFF); + header[bufferOffset++] = (byte) in.read(); + if (hash != header[bufferOffset-1]) { + throw new IOException(DESCRIPTOR_HASH_MISMATCH); + } + } + + /** + * Decompresses (if necessary) buffered data, optionally computes and validates a XXHash32 checksum, + * and writes the result to a buffer. + * + * @throws IOException + */ + private void readBlock() throws IOException { + int blockSize = Utils.readUnsignedIntLE(in); + + // Check for EndMark + if (blockSize == 0) { + finished = true; + // TODO implement content checksum, update flg.validate() + return; + } else if (blockSize > maxBlockSize) { + throw new IOException(String.format("Block size %s exceeded max: %s", blockSize, maxBlockSize)); + } + + boolean compressed = (blockSize & LZ4_FRAME_INCOMPRESSIBLE_MASK) == 0; + byte[] bufferToRead; + if (compressed) { + bufferToRead = compressedBuffer; + } else { + blockSize &= ~LZ4_FRAME_INCOMPRESSIBLE_MASK; + bufferToRead = buffer; + bufferSize = blockSize; + } + + if (in.read(bufferToRead, 0, blockSize) != blockSize) { + throw new IOException(PREMATURE_EOS); + } + + // verify checksum + if (flg.isBlockChecksumSet() && Utils.readUnsignedIntLE(in) != checksum.hash(bufferToRead, 0, blockSize, 0)) { + throw new IOException(BLOCK_HASH_MISMATCH); + } + + if (compressed) { + try { + bufferSize = decompressor.decompress(compressedBuffer, 0, blockSize, buffer, 0, maxBlockSize); + } catch (LZ4Exception e) { + throw new IOException(e); + } + } + + bufferOffset = 0; + } + + @Override + public int read() throws IOException { + if (finished) { + return -1; + } + if (available() == 0) { + readBlock(); + } + if (finished) { + return -1; + } + int value = buffer[bufferOffset++] & 0xFF; + + return value; + } + + @Override + public int read(byte b[], int off, int len) throws IOException { + net.jpountz.util.Utils.checkRange(b, off, len); + if (finished) { + return -1; + } + if (available() == 0) { + readBlock(); + } + if (finished) { + return -1; + } + len = Math.min(len, available()); + System.arraycopy(buffer, bufferOffset, b, off, len); + bufferOffset += len; + return len; + } + + @Override + public long skip(long n) throws IOException { + if (finished) { + return 0; + } + if (available() == 0) { + readBlock(); + } + if (finished) { + return 0; + } + n = Math.min(n, available()); + bufferOffset += n; + return n; + } + + @Override + public int available() throws IOException { + return bufferSize - bufferOffset; + } + + @Override + public void close() throws IOException { + in.close(); + } + + @Override + public synchronized void mark(int readlimit) { + throw new RuntimeException("mark not supported"); + } + + @Override + public synchronized void reset() throws IOException { + throw new RuntimeException("reset not supported"); + } + + @Override + public boolean markSupported() { + return false; + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/message/KafkaLZ4BlockOutputStream.java b/clients/src/main/java/org/apache/kafka/common/message/KafkaLZ4BlockOutputStream.java new file mode 100644 index 0000000000000..e5b9e433e14ef --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/message/KafkaLZ4BlockOutputStream.java @@ -0,0 +1,387 @@ +/** + * 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.message; + +import java.io.FilterOutputStream; +import java.io.IOException; +import java.io.OutputStream; + +import org.apache.kafka.common.utils.Utils; + +import net.jpountz.lz4.LZ4Compressor; +import net.jpountz.lz4.LZ4Factory; +import net.jpountz.xxhash.XXHash32; +import net.jpountz.xxhash.XXHashFactory; + +/** + * A partial implementation of the v1.4.1 LZ4 Frame format. + * + * @see LZ4 Framing Format Spec + */ +public final class KafkaLZ4BlockOutputStream extends FilterOutputStream { + + public static final int MAGIC = 0x184D2204; + public static final int LZ4_MAX_HEADER_LENGTH = 19; + public static final int LZ4_FRAME_INCOMPRESSIBLE_MASK = 0x80000000; + + public static final String CLOSED_STREAM = "The stream is already closed"; + + public static final int BLOCKSIZE_64KB = 4; + public static final int BLOCKSIZE_256KB = 5; + public static final int BLOCKSIZE_1MB = 6; + public static final int BLOCKSIZE_4MB = 7; + + private final LZ4Compressor compressor; + private final XXHash32 checksum; + private final FLG flg; + private final BD bd; + private final byte[] buffer; + private final byte[] compressedBuffer; + private final int maxBlockSize; + private int bufferOffset; + private boolean finished; + + /** + * Create a new {@link OutputStream} that will compress data using the LZ4 algorithm. + * + * @param out The output stream to compress + * @param blockSize Default: 4. The block size used during compression. 4=64kb, 5=256kb, 6=1mb, 7=4mb. All other values will generate an exception + * @param blockChecksum Default: false. When true, a XXHash32 checksum is computed and appended to the stream for every block of data + * @throws IOException + */ + public KafkaLZ4BlockOutputStream(OutputStream out, int blockSize, boolean blockChecksum) throws IOException { + super(out); + compressor = LZ4Factory.fastestInstance().fastCompressor(); + checksum = XXHashFactory.fastestInstance().hash32(); + bd = new BD(blockSize); + flg = new FLG(blockChecksum); + bufferOffset = 0; + maxBlockSize = bd.getBlockMaximumSize(); + buffer = new byte[maxBlockSize]; + compressedBuffer = new byte[compressor.maxCompressedLength(maxBlockSize)]; + finished = false; + writeHeader(); + } + + /** + * Create a new {@link OutputStream} that will compress data using the LZ4 algorithm. + * + * @param out The stream to compress + * @param blockSize Default: 4. The block size used during compression. 4=64kb, 5=256kb, 6=1mb, 7=4mb. All other values will generate an exception + * @throws IOException + */ + public KafkaLZ4BlockOutputStream(OutputStream out, int blockSize) throws IOException { + this(out, blockSize, false); + } + + /** + * Create a new {@link OutputStream} that will compress data using the LZ4 algorithm. + * + * @param out The output stream to compress + * @throws IOException + */ + public KafkaLZ4BlockOutputStream(OutputStream out) throws IOException { + this(out, BLOCKSIZE_64KB); + } + + /** + * Writes the magic number and frame descriptor to the underlying {@link OutputStream}. + * + * @throws IOException + */ + private void writeHeader() throws IOException { + Utils.writeUnsignedIntLE(buffer, 0, MAGIC); + bufferOffset = 4; + buffer[bufferOffset++] = flg.toByte(); + buffer[bufferOffset++] = bd.toByte(); + // TODO write uncompressed content size, update flg.validate() + // TODO write dictionary id, update flg.validate() + // compute checksum on all descriptor fields + int hash = (checksum.hash(buffer, 0, bufferOffset, 0) >> 8) & 0xFF; + buffer[bufferOffset++] = (byte) hash; + // write out frame descriptor + out.write(buffer, 0, bufferOffset); + bufferOffset = 0; + } + + /** + * Compresses buffered data, optionally computes an XXHash32 checksum, and writes + * the result to the underlying {@link OutputStream}. + * + * @throws IOException + */ + private void writeBlock() throws IOException { + if (bufferOffset == 0) { + return; + } + + int compressedLength = compressor.compress(buffer, 0, bufferOffset, compressedBuffer, 0); + byte[] bufferToWrite = compressedBuffer; + int compressMethod = 0; + + // Store block uncompressed if compressed length is greater (incompressible) + if (compressedLength >= bufferOffset) { + bufferToWrite = buffer; + compressedLength = bufferOffset; + compressMethod = LZ4_FRAME_INCOMPRESSIBLE_MASK; + } + + // Write content + Utils.writeUnsignedIntLE(out, compressedLength | compressMethod); + out.write(bufferToWrite, 0, compressedLength); + + // Calculate and write block checksum + if (flg.isBlockChecksumSet()) { + int hash = checksum.hash(bufferToWrite, 0, compressedLength, 0); + Utils.writeUnsignedIntLE(out, hash); + } + bufferOffset = 0; + } + + /** + * Similar to the {@link #writeBlock()} method. Writes a 0-length block + * (without block checksum) to signal the end of the block stream. + * + * @throws IOException + */ + private void writeEndMark() throws IOException { + Utils.writeUnsignedIntLE(out, 0); + // TODO implement content checksum, update flg.validate() + finished = true; + } + + @Override + public void write(int b) throws IOException { + ensureNotFinished(); + if (bufferOffset == maxBlockSize) { + writeBlock(); + } + buffer[bufferOffset++] = (byte) b; + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + net.jpountz.util.Utils.checkRange(b, off, len); + ensureNotFinished(); + + int bufferRemainingLength = maxBlockSize - bufferOffset; + // while b will fill the buffer + while (len > bufferRemainingLength) { + // fill remaining space in buffer + System.arraycopy(b, off, buffer, bufferOffset, bufferRemainingLength); + bufferOffset = maxBlockSize; + writeBlock(); + // compute new offset and length + off += bufferRemainingLength; + len -= bufferRemainingLength; + bufferRemainingLength = maxBlockSize; + } + + System.arraycopy(b, off, buffer, bufferOffset, len); + bufferOffset += len; + } + + @Override + public void flush() throws IOException { + if (!finished) { + writeBlock(); + } + if (out != null) { + out.flush(); + } + } + + /** + * A simple state check to ensure the stream is still open. + */ + private void ensureNotFinished() { + if (finished) { + throw new IllegalStateException(CLOSED_STREAM); + } + } + + @Override + public void close() throws IOException { + if (!finished) { + writeEndMark(); + flush(); + finished = true; + } + if (out != null) { + out.close(); + out = null; + } + } + + public static class FLG { + + private static final int VERSION = 1; + + private final int presetDictionary; + private final int reserved1; + private final int contentChecksum; + private final int contentSize; + private final int blockChecksum; + private final int blockIndependence; + private final int version; + + public FLG() { + this(false); + } + + public FLG(boolean blockChecksum) { + this(0, 0, 0, 0, blockChecksum ? 1 : 0, 1, VERSION); + } + + private FLG(int presetDictionary, int reserved1, int contentChecksum, + int contentSize, int blockChecksum, int blockIndependence, int version) { + this.presetDictionary = presetDictionary; + this.reserved1 = reserved1; + this.contentChecksum = contentChecksum; + this.contentSize = contentSize; + this.blockChecksum = blockChecksum; + this.blockIndependence = blockIndependence; + this.version = version; + validate(); + } + + public static FLG fromByte(byte flg) { + int presetDictionary = (flg >>> 0) & 1; + int reserved1 = (flg >>> 1) & 1; + int contentChecksum = (flg >>> 2) & 1; + int contentSize = (flg >>> 3) & 1; + int blockChecksum = (flg >>> 4) & 1; + int blockIndependence = (flg >>> 5) & 1; + int version = (flg >>> 6) & 3; + + return new FLG(presetDictionary, reserved1, contentChecksum, + contentSize, blockChecksum, blockIndependence, version); + } + + public byte toByte() { + return (byte) ( + ((presetDictionary & 1) << 0) + | ((reserved1 & 1) << 1) + | ((contentChecksum & 1) << 2) + | ((contentSize & 1) << 3) + | ((blockChecksum & 1) << 4) + | ((blockIndependence & 1) << 5) + | ((version & 3) << 6) ); + } + + private void validate() { + if (presetDictionary != 0) { + throw new RuntimeException("Preset dictionary is unsupported"); + } + if (reserved1 != 0) { + throw new RuntimeException("Reserved1 field must be 0"); + } + if (contentChecksum != 0) { + throw new RuntimeException("Content checksum is unsupported"); + } + if (contentSize != 0) { + throw new RuntimeException("Content size is unsupported"); + } + if (blockIndependence != 1) { + throw new RuntimeException("Dependent block stream is unsupported"); + } + if (version != VERSION) { + throw new RuntimeException(String.format("Version %d is unsupported", version)); + } + } + + public boolean isPresetDictionarySet() { + return presetDictionary == 1; + } + + public boolean isContentChecksumSet() { + return contentChecksum == 1; + } + + public boolean isContentSizeSet() { + return contentSize == 1; + } + + public boolean isBlockChecksumSet() { + return blockChecksum == 1; + } + + public boolean isBlockIndependenceSet() { + return blockIndependence == 1; + } + + public int getVersion() { + return version; + } + } + + public static class BD { + + private final int reserved2; + private final int blockSizeValue; + private final int reserved3; + + public BD() { + this(0, BLOCKSIZE_64KB, 0); + } + + public BD(int blockSizeValue) { + this(0, blockSizeValue, 0); + } + + private BD(int reserved2, int blockSizeValue, int reserved3) { + this.reserved2 = reserved2; + this.blockSizeValue = blockSizeValue; + this.reserved3 = reserved3; + validate(); + } + + public static BD fromByte(byte bd) { + int reserved2 = (bd >>> 0) & 15; + int blockMaximumSize = (bd >>> 4) & 7; + int reserved3 = (bd >>> 7) & 1; + + return new BD(reserved2, blockMaximumSize, reserved3); + } + + private void validate() { + if (reserved2 != 0) { + throw new RuntimeException("Reserved2 field must be 0"); + } + if (blockSizeValue < 4 || blockSizeValue > 7) { + throw new RuntimeException("Block size value must be between 4 and 7"); + } + if (reserved3 != 0) { + throw new RuntimeException("Reserved3 field must be 0"); + } + } + + // 2^(2n+8) + public int getBlockMaximumSize() { + return (1 << ((2 * blockSizeValue) + 8)); + } + + public byte toByte() { + return (byte) ( + ((reserved2 & 15) << 0) + | ((blockSizeValue & 7) << 4) + | ((reserved3 & 1) << 7) ); + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java b/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java index 5227b2d7ab803..65a7e4323793d 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java +++ b/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java @@ -20,7 +20,7 @@ * The compression type to use */ public enum CompressionType { - NONE(0, "none", 1.0f), GZIP(1, "gzip", 0.5f), SNAPPY(2, "snappy", 0.5f), LZ4(3, "lz4", 0.5f), LZ4HC(4, "lz4hc", 0.5f); + NONE(0, "none", 1.0f), GZIP(1, "gzip", 0.5f), SNAPPY(2, "snappy", 0.5f), LZ4(3, "lz4", 0.5f); public final int id; public final String name; @@ -42,8 +42,6 @@ public static CompressionType forId(int id) { return SNAPPY; case 3: return LZ4; - case 4: - return LZ4HC; default: throw new IllegalArgumentException("Unknown compression type id: " + id); } @@ -58,8 +56,6 @@ else if (SNAPPY.name.equals(name)) return SNAPPY; else if (LZ4.name.equals(name)) return LZ4; - else if (LZ4HC.name.equals(name)) - return LZ4HC; else throw new IllegalArgumentException("Unknown compression name: " + name); } diff --git a/clients/src/main/java/org/apache/kafka/common/record/Compressor.java b/clients/src/main/java/org/apache/kafka/common/record/Compressor.java index 0323f5f7032dc..d684e6833bd81 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/Compressor.java +++ b/clients/src/main/java/org/apache/kafka/common/record/Compressor.java @@ -218,27 +218,13 @@ static public DataOutputStream wrapForOutput(ByteBufferOutputStream buffer, Comp } case LZ4: try { - Class LZ4BlockOutputStream = Class.forName("net.jpountz.lz4.LZ4BlockOutputStream"); - OutputStream stream = (OutputStream) LZ4BlockOutputStream.getConstructor(OutputStream.class) + Class outputStreamClass = Class.forName("org.apache.kafka.common.message.KafkaLZ4BlockOutputStream"); + OutputStream stream = (OutputStream) outputStreamClass.getConstructor(OutputStream.class) .newInstance(buffer); return new DataOutputStream(stream); } catch (Exception e) { throw new KafkaException(e); } - case LZ4HC: - try { - Class factoryClass = Class.forName("net.jpountz.lz4.LZ4Factory"); - Class compressorClass = Class.forName("net.jpountz.lz4.LZ4Compressor"); - Class lz4BlockOutputStream = Class.forName("net.jpountz.lz4.LZ4BlockOutputStream"); - Object factory = factoryClass.getMethod("fastestInstance").invoke(null); - Object compressor = factoryClass.getMethod("highCompressor").invoke(factory); - OutputStream stream = (OutputStream) lz4BlockOutputStream - .getConstructor(OutputStream.class, Integer.TYPE, compressorClass) - .newInstance(buffer, 1 << 16, compressor); - return new DataOutputStream(stream); - } catch (Exception e) { - throw new KafkaException(e); - } default: throw new IllegalArgumentException("Unknown compression type: " + type); } @@ -266,10 +252,9 @@ static public DataInputStream wrapForInput(ByteBufferInputStream buffer, Compres throw new KafkaException(e); } case LZ4: - case LZ4HC: // dynamically load LZ4 class to avoid runtime dependency try { - Class inputStreamClass = Class.forName("net.jpountz.lz4.LZ4BlockInputStream"); + Class inputStreamClass = Class.forName("org.apache.kafka.common.message.KafkaLZ4BlockInputStream"); InputStream stream = (InputStream) inputStreamClass.getConstructor(InputStream.class) .newInstance(buffer); return new DataInputStream(stream); diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index a0827f576e8c3..527dd0f9c47fc 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -12,6 +12,9 @@ */ package org.apache.kafka.common.utils; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import java.io.UnsupportedEncodingException; import java.nio.ByteBuffer; import java.util.regex.Matcher; @@ -74,6 +77,34 @@ public static long readUnsignedInt(ByteBuffer buffer, int index) { return buffer.getInt(index) & 0xffffffffL; } + /** + * Read an unsigned integer stored in little-endian format from the {@link InputStream}. + * + * @param in The stream to read from + * @return The integer read (MUST BE TREATED WITH SPECIAL CARE TO AVOID SIGNEDNESS) + */ + public static int readUnsignedIntLE(InputStream in) throws IOException { + return (in.read() << 8*0) + | (in.read() << 8*1) + | (in.read() << 8*2) + | (in.read() << 8*3); + } + + /** + * Read an unsigned integer stored in little-endian format from a byte array + * at a given offset. + * + * @param buffer The byte array to read from + * @param offset The position in buffer to read from + * @return The integer read (MUST BE TREATED WITH SPECIAL CARE TO AVOID SIGNEDNESS) + */ + public static int readUnsignedIntLE(byte[] buffer, int offset) { + return (buffer[offset++] << 8*0) + | (buffer[offset++] << 8*1) + | (buffer[offset++] << 8*2) + | (buffer[offset] << 8*3); + } + /** * Write the given long value as a 4 byte unsigned integer. Overflow is ignored. * @@ -95,6 +126,35 @@ public static void writeUnsignedInt(ByteBuffer buffer, int index, long value) { buffer.putInt(index, (int) (value & 0xffffffffL)); } + /** + * Write an unsigned integer in little-endian format to the {@link OutputStream}. + * + * @param out The stream to write to + * @param value The value to write + */ + public static void writeUnsignedIntLE(OutputStream out, int value) throws IOException { + out.write(value >>> 8*0); + out.write(value >>> 8*1); + out.write(value >>> 8*2); + out.write(value >>> 8*3); + } + + /** + * Write an unsigned integer in little-endian format to a byte array + * at a given offset. + * + * @param buffer The byte array to write to + * @param offset The position in buffer to write to + * @param value The value to write + */ + public static void writeUnsignedIntLE(byte[] buffer, int offset, int value) { + buffer[offset++] = (byte) (value >>> 8*0); + buffer[offset++] = (byte) (value >>> 8*1); + buffer[offset++] = (byte) (value >>> 8*2); + buffer[offset] = (byte) (value >>> 8*3); + } + + /** * Get the absolute value of the given number. If the number is Int.MinValue return 0. This is different from * java.lang.Math.abs or scala.math.abs in that they return Int.MinValue (!). diff --git a/config/producer.properties b/config/producer.properties index 39d65d7c6c21f..47ae3e241301e 100644 --- a/config/producer.properties +++ b/config/producer.properties @@ -26,8 +26,8 @@ metadata.broker.list=localhost:9092 # specifies whether the messages are sent asynchronously (async) or synchronously (sync) producer.type=sync -# specify the compression codec for all data generated: none, gzip, snappy, lz4, lz4hc. -# the old config values work as well: 0, 1, 2, 3, 4 for none, gzip, snappy, lz4, lz4hc, respectivally +# specify the compression codec for all data generated: none, gzip, snappy, lz4. +# the old config values work as well: 0, 1, 2, 3 for none, gzip, snappy, lz4, respectively compression.codec=none # message encoder diff --git a/core/src/main/scala/kafka/message/CompressionCodec.scala b/core/src/main/scala/kafka/message/CompressionCodec.scala index de0a0fade5387..9439d2bc29a0c 100644 --- a/core/src/main/scala/kafka/message/CompressionCodec.scala +++ b/core/src/main/scala/kafka/message/CompressionCodec.scala @@ -24,7 +24,6 @@ object CompressionCodec { case GZIPCompressionCodec.codec => GZIPCompressionCodec case SnappyCompressionCodec.codec => SnappyCompressionCodec case LZ4CompressionCodec.codec => LZ4CompressionCodec - case LZ4HCCompressionCodec.codec => LZ4HCCompressionCodec case _ => throw new kafka.common.UnknownCodecException("%d is an unknown compression codec".format(codec)) } } @@ -34,7 +33,6 @@ object CompressionCodec { case GZIPCompressionCodec.name => GZIPCompressionCodec case SnappyCompressionCodec.name => SnappyCompressionCodec case LZ4CompressionCodec.name => LZ4CompressionCodec - case LZ4HCCompressionCodec.name => LZ4HCCompressionCodec case _ => throw new kafka.common.UnknownCodecException("%s is an unknown compression codec".format(name)) } } @@ -62,11 +60,6 @@ case object LZ4CompressionCodec extends CompressionCodec { val name = "lz4" } -case object LZ4HCCompressionCodec extends CompressionCodec { - val codec = 4 - val name = "lz4hc" -} - case object NoCompressionCodec extends CompressionCodec { val codec = 0 val name = "none" diff --git a/core/src/main/scala/kafka/message/CompressionFactory.scala b/core/src/main/scala/kafka/message/CompressionFactory.scala index 8420e13d0d868..c721040bd0420 100644 --- a/core/src/main/scala/kafka/message/CompressionFactory.scala +++ b/core/src/main/scala/kafka/message/CompressionFactory.scala @@ -22,6 +22,8 @@ import java.util.zip.GZIPOutputStream import java.util.zip.GZIPInputStream import java.io.InputStream +import org.apache.kafka.common.message.{KafkaLZ4BlockInputStream, KafkaLZ4BlockOutputStream} + object CompressionFactory { def apply(compressionCodec: CompressionCodec, stream: OutputStream): OutputStream = { @@ -32,11 +34,7 @@ object CompressionFactory { import org.xerial.snappy.SnappyOutputStream new SnappyOutputStream(stream) case LZ4CompressionCodec => - import net.jpountz.lz4.LZ4BlockOutputStream - new LZ4BlockOutputStream(stream) - case LZ4HCCompressionCodec => - import net.jpountz.lz4.{LZ4BlockOutputStream, LZ4Factory} - new LZ4BlockOutputStream(stream, 1 << 16, LZ4Factory.fastestInstance().highCompressor()) + new KafkaLZ4BlockOutputStream(stream) case _ => throw new kafka.common.UnknownCodecException("Unknown Codec: " + compressionCodec) } @@ -49,9 +47,8 @@ object CompressionFactory { case SnappyCompressionCodec => import org.xerial.snappy.SnappyInputStream new SnappyInputStream(stream) - case LZ4CompressionCodec | LZ4HCCompressionCodec => - import net.jpountz.lz4.LZ4BlockInputStream - new LZ4BlockInputStream(stream) + case LZ4CompressionCodec => + new KafkaLZ4BlockInputStream(stream) case _ => throw new kafka.common.UnknownCodecException("Unknown Codec: " + compressionCodec) } diff --git a/core/src/main/scala/kafka/tools/ConsoleProducer.scala b/core/src/main/scala/kafka/tools/ConsoleProducer.scala index b024a693c23cb..397d80da08c92 100644 --- a/core/src/main/scala/kafka/tools/ConsoleProducer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleProducer.scala @@ -113,7 +113,7 @@ object ConsoleProducer { .describedAs("broker-list") .ofType(classOf[String]) val syncOpt = parser.accepts("sync", "If set message send requests to the brokers are synchronously, one at a time as they arrive.") - val compressionCodecOpt = parser.accepts("compression-codec", "The compression codec: either 'none', 'gzip', 'snappy', 'lz4', or 'lz4hc'." + + val compressionCodecOpt = parser.accepts("compression-codec", "The compression codec: either 'none', 'gzip', 'snappy', or 'lz4'." + "If specified without value, then it defaults to 'gzip'") .withOptionalArg() .describedAs("compression-codec") diff --git a/core/src/main/scala/kafka/tools/PerfConfig.scala b/core/src/main/scala/kafka/tools/PerfConfig.scala index c72002976d904..d073acf9adf4b 100644 --- a/core/src/main/scala/kafka/tools/PerfConfig.scala +++ b/core/src/main/scala/kafka/tools/PerfConfig.scala @@ -53,7 +53,7 @@ class PerfConfig(args: Array[String]) { .defaultsTo(200) val compressionCodecOpt = parser.accepts("compression-codec", "If set, messages are sent compressed") .withRequiredArg - .describedAs("supported codec: NoCompressionCodec as 0, GZIPCompressionCodec as 1, SnappyCompressionCodec as 2, LZ4CompressionCodec as 3, LZ4HCCompressionCodec as 4") + .describedAs("supported codec: NoCompressionCodec as 0, GZIPCompressionCodec as 1, SnappyCompressionCodec as 2, LZ4CompressionCodec as 3") .ofType(classOf[java.lang.Integer]) .defaultsTo(0) val helpOpt = parser.accepts("help", "Print usage.") diff --git a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala index c95485170fd8b..6379f2b60af79 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala @@ -47,7 +47,7 @@ class ProducerCompressionTest(compression: String) extends JUnit3Suite with ZooK private val config = new KafkaConfig(props) private val topic = "topic" - private val numRecords = 100 + private val numRecords = 2000 @Before override def setUp() { @@ -73,6 +73,8 @@ class ProducerCompressionTest(compression: String) extends JUnit3Suite with ZooK val props = new Properties() props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromConfigs(Seq(config))) props.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, compression) + props.put(ProducerConfig.BATCH_SIZE_CONFIG, "66000") + props.put(ProducerConfig.LINGER_MS_CONFIG, "200") var producer = new KafkaProducer(props) val consumer = new SimpleConsumer("localhost", port, 100, 1024*1024, "") @@ -125,7 +127,6 @@ object ProducerCompressionTest { list.add(Array("gzip")) list.add(Array("snappy")) list.add(Array("lz4")) - list.add(Array("lz4hc")) list } } diff --git a/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala b/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala index 6f0addcea64f1..0bb275d0dc840 100644 --- a/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala @@ -32,8 +32,6 @@ class MessageCompressionTest extends JUnitSuite { codecs += SnappyCompressionCodec if(isLZ4Available) codecs += LZ4CompressionCodec - if (izLZ4HCAvailable) - codecs += LZ4HCCompressionCodec for(codec <- codecs) testSimpleCompressDecompress(codec) } @@ -74,14 +72,4 @@ class MessageCompressionTest extends JUnitSuite { case e: UnsatisfiedLinkError => false } } - - def izLZ4HCAvailable(): Boolean = { - try { - val lz4hc = new net.jpountz.lz4.LZ4BlockOutputStream(new ByteArrayOutputStream(), 1 << 16, - net.jpountz.lz4.LZ4Factory.fastestInstance().highCompressor()) - true - } catch { - case e: UnsatisfiedLinkError => false - } - } } diff --git a/core/src/test/scala/unit/kafka/message/MessageTest.scala b/core/src/test/scala/unit/kafka/message/MessageTest.scala index 958c1a60069ad..7b74a0d315470 100644 --- a/core/src/test/scala/unit/kafka/message/MessageTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageTest.scala @@ -39,7 +39,7 @@ class MessageTest extends JUnitSuite { def setUp(): Unit = { val keys = Array(null, "key".getBytes, "".getBytes) val vals = Array("value".getBytes, "".getBytes, null) - val codecs = Array(NoCompressionCodec, GZIPCompressionCodec, SnappyCompressionCodec, LZ4CompressionCodec, LZ4HCCompressionCodec) + val codecs = Array(NoCompressionCodec, GZIPCompressionCodec, SnappyCompressionCodec, LZ4CompressionCodec) for(k <- keys; v <- vals; codec <- codecs) messages += new MessageTestVal(k, v, codec, new Message(v, k, codec)) } From 5f1ad107b71318144af07a0fd9098b560c4efa82 Mon Sep 17 00:00:00 2001 From: Ivan Lyutov Date: Fri, 17 Oct 2014 12:30:41 -0700 Subject: [PATCH 025/491] kafka-1671; uploaded archives are missing for Scala version 2.11; patched by Ivan Lyutov; reviewed by Jun Rao --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 2e488a1ab0437..ee87e0f40ad05 100644 --- a/build.gradle +++ b/build.gradle @@ -34,7 +34,7 @@ ext { } def isVerificationRequired(project) { - project.gradle.startParameter.taskNames.contains("uploadArchives") + project.gradle.startParameter.taskNames.any { it.contains("upload") } } apply from: file('wrapper.gradle') From 5152fb7e8eea720046ce844ca5153d2766ee002e Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Fri, 17 Oct 2014 15:04:31 -0700 Subject: [PATCH 026/491] KAFKA-1108 When controlled shutdown attempt fails, the reason is not always logged; reviewed by Neha Narkhede --- core/src/main/scala/kafka/server/KafkaServer.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 07c0a078ffa51..4de812374e8fb 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -220,7 +220,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg channel.send(request) response = channel.receive() - val shutdownResponse = ControlledShutdownResponse.readFrom(response.buffer) if (shutdownResponse.errorCode == ErrorMapping.NoError && shutdownResponse.partitionsRemaining != null && shutdownResponse.partitionsRemaining.size == 0) { @@ -236,6 +235,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg case ioe: java.io.IOException => channel.disconnect() channel = null + warn("Error during controlled shutdown, possibly because leader movement took longer than the configured socket.timeout.ms: %s".format(ioe.getMessage)) // ignore and try again } } From 2d322a7bd12966e4a66f04b75add619a481ff22d Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Mon, 20 Oct 2014 11:07:44 -0700 Subject: [PATCH 027/491] kafka-1717; remove netty dependency through ZK 3.4.x; patched by Jun Rao; reviewed by Sriharsha Chintalapani and Neha Narkhede --- build.gradle | 1 + 1 file changed, 1 insertion(+) diff --git a/build.gradle b/build.gradle index ee87e0f40ad05..c3e6bb839ad65 100644 --- a/build.gradle +++ b/build.gradle @@ -235,6 +235,7 @@ project(':core') { compile.exclude module: 'jmxri' compile.exclude module: 'jmxtools' compile.exclude module: 'mail' + compile.exclude module: 'netty' } tasks.create(name: "copyDependantLibs", type: Copy) { From 4b095760cb38c726c49e6f245ffd7dc8ccf49863 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Mon, 20 Oct 2014 18:02:00 -0700 Subject: [PATCH 028/491] kafka-1698; Validator.ensureValid() only validates default config value; patched by Ewen Cheslack-Postava; reviewed by Jun Rao --- .../apache/kafka/common/config/ConfigDef.java | 2 + .../kafka/common/config/ConfigDefTest.java | 41 +++++++++++++++++++ 2 files changed, 43 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java index 227309e8c62f9..c4cea2cc072f4 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java @@ -124,6 +124,8 @@ else if (key.defaultValue == NO_DEFAULT_VALUE) throw new ConfigException("Missing required configuration \"" + key.name + "\" which has no default value."); else value = key.defaultValue; + if (key.validator != null) + key.validator.ensureValid(key.name, value); values.put(key.name, value); } return values; diff --git a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java index 09a82feeb7cae..7869e467ab624 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java @@ -16,12 +16,15 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; +import java.util.Arrays; import java.util.HashMap; import java.util.Map; import java.util.Properties; import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.Validator; import org.apache.kafka.common.config.ConfigDef.Range; +import org.apache.kafka.common.config.ConfigDef.ValidString; import org.apache.kafka.common.config.ConfigDef.Type; import org.junit.Test; @@ -97,4 +100,42 @@ private void testBadInputs(Type type, Object... values) { } } } + + @Test(expected = ConfigException.class) + public void testInvalidDefaultRange() { + ConfigDef def = new ConfigDef().define("name", Type.INT, -1, Range.between(0,10), Importance.HIGH, "docs"); + } + + @Test(expected = ConfigException.class) + public void testInvalidDefaultString() { + ConfigDef def = new ConfigDef().define("name", Type.STRING, "bad", ValidString.in(Arrays.asList("valid", "values")), Importance.HIGH, "docs"); + } + + @Test + public void testValidators() { + testValidators(Type.INT, Range.between(0,10), 5, new Object[]{1, 5, 9}, new Object[]{-1, 11}); + testValidators(Type.STRING, ValidString.in(Arrays.asList("good", "values", "default")), "default", + new Object[]{"good", "values", "default"}, new Object[]{"bad", "inputs"}); + } + + private void testValidators(Type type, Validator validator, Object defaultVal, Object[] okValues, Object[] badValues) { + ConfigDef def = new ConfigDef().define("name", type, defaultVal, validator, Importance.HIGH, "docs"); + + for (Object value : okValues) { + Map m = new HashMap(); + m.put("name", value); + def.parse(m); + } + + for (Object value : badValues) { + Map m = new HashMap(); + m.put("name", value); + try { + def.parse(m); + fail("Expected a config exception due to invalid value " + value); + } catch (ConfigException e) { + // this is good + } + } + } } From e8923ae33ac90170381d8c4a59a5eb810ee9f36d Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Wed, 22 Oct 2014 09:59:17 -0700 Subject: [PATCH 029/491] KAFKA-1653 Duplicate broker ids allowed in replica assignment; reviewed by Neha Narkhede --- ...referredReplicaLeaderElectionCommand.scala | 11 ++++++--- .../admin/ReassignPartitionsCommand.scala | 24 ++++++++++++++++--- .../main/scala/kafka/admin/TopicCommand.scala | 4 ++++ .../kafka/tools/StateChangeLogMerger.scala | 7 +++++- core/src/main/scala/kafka/utils/Utils.scala | 10 ++++++++ core/src/main/scala/kafka/utils/ZkUtils.scala | 15 ++++++++---- 6 files changed, 59 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala b/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala index c7918483c0204..79b5e0aeb4af7 100644 --- a/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala +++ b/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala @@ -78,12 +78,17 @@ object PreferredReplicaLeaderElectionCommand extends Logging { case Some(m) => m.asInstanceOf[Map[String, Any]].get("partitions") match { case Some(partitionsList) => - val partitions = partitionsList.asInstanceOf[List[Map[String, Any]]] - partitions.map { p => + val partitionsRaw = partitionsList.asInstanceOf[List[Map[String, Any]]] + val partitions = partitionsRaw.map { p => val topic = p.get("topic").get.asInstanceOf[String] val partition = p.get("partition").get.asInstanceOf[Int] TopicAndPartition(topic, partition) - }.toSet + } + val duplicatePartitions = Utils.duplicates(partitions) + val partitionsSet = partitions.toSet + if (duplicatePartitions.nonEmpty) + throw new AdminOperationException("Preferred replica election data contains duplicate partitions: %s".format(duplicatePartitions.mkString(","))) + partitionsSet case None => throw new AdminOperationException("Preferred replica election data is empty") } case None => throw new AdminOperationException("Preferred replica election data is empty") diff --git a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala index 691d69a49a240..979992b68af37 100644 --- a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala +++ b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala @@ -81,8 +81,14 @@ object ReassignPartitionsCommand extends Logging { CommandLineUtils.printUsageAndDie(opts.parser, "If --generate option is used, command must include both --topics-to-move-json-file and --broker-list options") val topicsToMoveJsonFile = opts.options.valueOf(opts.topicsToMoveJsonFileOpt) val brokerListToReassign = opts.options.valueOf(opts.brokerListOpt).split(',').map(_.toInt) + val duplicateReassignments = Utils.duplicates(brokerListToReassign) + if (duplicateReassignments.nonEmpty) + throw new AdminCommandFailedException("Broker list contains duplicate entries: %s".format(duplicateReassignments.mkString(","))) val topicsToMoveJsonString = Utils.readFileAsString(topicsToMoveJsonFile) val topicsToReassign = ZkUtils.parseTopicsData(topicsToMoveJsonString) + val duplicateTopicsToReassign = Utils.duplicates(topicsToReassign) + if (duplicateTopicsToReassign.nonEmpty) + throw new AdminCommandFailedException("List of topics to reassign contains duplicate entries: %s".format(duplicateTopicsToReassign.mkString(","))) val topicPartitionsToReassign = ZkUtils.getReplicaAssignmentForTopics(zkClient, topicsToReassign) var partitionsToBeReassigned : Map[TopicAndPartition, Seq[Int]] = new mutable.HashMap[TopicAndPartition, List[Int]]() @@ -103,17 +109,29 @@ object ReassignPartitionsCommand extends Logging { CommandLineUtils.printUsageAndDie(opts.parser, "If --execute option is used, command must include --reassignment-json-file that was output " + "during the --generate option") val reassignmentJsonFile = opts.options.valueOf(opts.reassignmentJsonFileOpt) val reassignmentJsonString = Utils.readFileAsString(reassignmentJsonFile) - val partitionsToBeReassigned = ZkUtils.parsePartitionReassignmentData(reassignmentJsonString) + val partitionsToBeReassigned = ZkUtils.parsePartitionReassignmentDataWithoutDedup(reassignmentJsonString) if (partitionsToBeReassigned.isEmpty) throw new AdminCommandFailedException("Partition reassignment data file %s is empty".format(reassignmentJsonFile)) - val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, partitionsToBeReassigned) + val duplicateReassignedPartitions = Utils.duplicates(partitionsToBeReassigned.map{ case(tp,replicas) => tp}) + if (duplicateReassignedPartitions.nonEmpty) + throw new AdminCommandFailedException("Partition reassignment contains duplicate topic partitions: %s".format(duplicateReassignedPartitions.mkString(","))) + val duplicateEntries= partitionsToBeReassigned + .map{ case(tp,replicas) => (tp, Utils.duplicates(replicas))} + .filter{ case (tp,duplicatedReplicas) => duplicatedReplicas.nonEmpty } + if (duplicateEntries.nonEmpty) { + val duplicatesMsg = duplicateEntries + .map{ case (tp,duplicateReplicas) => "%s contains multiple entries for %s".format(tp, duplicateReplicas.mkString(",")) } + .mkString(". ") + throw new AdminCommandFailedException("Partition replica lists may not contain duplicate entries: %s".format(duplicatesMsg)) + } + val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, partitionsToBeReassigned.toMap) // before starting assignment, output the current replica assignment to facilitate rollback val currentPartitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, partitionsToBeReassigned.map(_._1.topic).toSeq) println("Current partition replica assignment\n\n%s\n\nSave this to use as the --reassignment-json-file option during rollback" .format(ZkUtils.getPartitionReassignmentZkData(currentPartitionReplicaAssignment))) // start the reassignment if(reassignPartitionsCommand.reassignPartitions()) - println("Successfully started reassignment of partitions %s".format(ZkUtils.getPartitionReassignmentZkData(partitionsToBeReassigned))) + println("Successfully started reassignment of partitions %s".format(ZkUtils.getPartitionReassignmentZkData(partitionsToBeReassigned.toMap))) else println("Failed to reassign partitions %s".format(partitionsToBeReassigned)) } diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index 7672c5aab4fba..0b2735e7fc42e 100644 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -19,6 +19,7 @@ package kafka.admin import joptsimple._ import java.util.Properties +import kafka.common.AdminCommandFailedException import kafka.utils._ import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.exception.ZkNodeExistsException @@ -225,6 +226,9 @@ object TopicCommand { val ret = new mutable.HashMap[Int, List[Int]]() for (i <- 0 until partitionList.size) { val brokerList = partitionList(i).split(":").map(s => s.trim().toInt) + val duplicateBrokers = Utils.duplicates(brokerList) + if (duplicateBrokers.nonEmpty) + throw new AdminCommandFailedException("Partition replica lists may not contain duplicate entries: %s".format(duplicateBrokers.mkString(","))) ret.put(i, brokerList.toList) if (ret(i).size != ret(0).size) throw new AdminOperationException("Partition " + i + " has different replication factor: " + brokerList) diff --git a/core/src/main/scala/kafka/tools/StateChangeLogMerger.scala b/core/src/main/scala/kafka/tools/StateChangeLogMerger.scala index d298e7e81acc7..b34b8c7d7d4ca 100644 --- a/core/src/main/scala/kafka/tools/StateChangeLogMerger.scala +++ b/core/src/main/scala/kafka/tools/StateChangeLogMerger.scala @@ -22,7 +22,7 @@ import scala.util.matching.Regex import collection.mutable import java.util.Date import java.text.SimpleDateFormat -import kafka.utils.{Logging, CommandLineUtils} +import kafka.utils.{Utils, Logging, CommandLineUtils} import kafka.common.Topic import java.io.{BufferedOutputStream, OutputStream} @@ -115,6 +115,11 @@ object StateChangeLogMerger extends Logging { } if (options.has(partitionsOpt)) { partitions = options.valueOf(partitionsOpt).split(",").toList.map(_.toInt) + val duplicatePartitions = Utils.duplicates(partitions) + if (duplicatePartitions.nonEmpty) { + System.err.println("The list of partitions contains repeated entries: %s".format(duplicatePartitions.mkString(","))) + System.exit(1) + } } startDate = dateFormat.parse(options.valueOf(startTimeOpt).replace('\"', ' ').trim) endDate = dateFormat.parse(options.valueOf(endTimeOpt).replace('\"', ' ').trim) diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala index 29d5a17d4a03c..23aefb4715b17 100644 --- a/core/src/main/scala/kafka/utils/Utils.scala +++ b/core/src/main/scala/kafka/utils/Utils.scala @@ -566,4 +566,14 @@ object Utils extends Logging { case c => c }.mkString } + + /** + * Returns a list of duplicated items + */ + def duplicates[T](s: Traversable[T]): Iterable[T] = { + s.groupBy(identity) + .map{ case (k,l) => (k,l.size)} + .filter{ case (k,l) => (l > 1) } + .keys + } } diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index a7b1fdcb50d5c..56e3e88e0cc6d 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -575,23 +575,28 @@ object ZkUtils extends Logging { } } - def parsePartitionReassignmentData(jsonData: String): Map[TopicAndPartition, Seq[Int]] = { - val reassignedPartitions: mutable.Map[TopicAndPartition, Seq[Int]] = mutable.Map() + // Parses without deduplicating keys so the the data can be checked before allowing reassignment to proceed + def parsePartitionReassignmentDataWithoutDedup(jsonData: String): Seq[(TopicAndPartition, Seq[Int])] = { Json.parseFull(jsonData) match { case Some(m) => m.asInstanceOf[Map[String, Any]].get("partitions") match { case Some(partitionsSeq) => - partitionsSeq.asInstanceOf[Seq[Map[String, Any]]].foreach(p => { + partitionsSeq.asInstanceOf[Seq[Map[String, Any]]].map(p => { val topic = p.get("topic").get.asInstanceOf[String] val partition = p.get("partition").get.asInstanceOf[Int] val newReplicas = p.get("replicas").get.asInstanceOf[Seq[Int]] - reassignedPartitions += TopicAndPartition(topic, partition) -> newReplicas + TopicAndPartition(topic, partition) -> newReplicas }) case None => + Seq.empty } case None => + Seq.empty } - reassignedPartitions + } + + def parsePartitionReassignmentData(jsonData: String): Map[TopicAndPartition, Seq[Int]] = { + parsePartitionReassignmentDataWithoutDedup(jsonData).toMap } def parseTopicsData(jsonData: String): Seq[String] = { From cda1f73d0498d66aafc8c6e57f84e0cd3c69928a Mon Sep 17 00:00:00 2001 From: Muneyuki Noguchi Date: Thu, 23 Oct 2014 14:52:07 -0700 Subject: [PATCH 030/491] kafka-1727; Fix comment about message format; patched by Muneyuki Noguchi; reviewed by Jun Rao --- core/src/main/scala/kafka/message/Message.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala index d2a7293c7be40..d3024807a84e1 100644 --- a/core/src/main/scala/kafka/message/Message.scala +++ b/core/src/main/scala/kafka/message/Message.scala @@ -70,7 +70,7 @@ object Message { * A message. The format of an N byte message is the following: * * 1. 4 byte CRC32 of the message - * 2. 1 byte "magic" identifier to allow format changes, value is 2 currently + * 2. 1 byte "magic" identifier to allow format changes, value is 0 currently * 3. 1 byte "attributes" identifier to allow annotations on the message independent of the version (e.g. compression enabled, type of codec used) * 4. 4 byte key length, containing length K * 5. K byte key From 35f589bb4654b49035c27780717f560e74400444 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 23 Oct 2014 15:42:10 -0700 Subject: [PATCH 031/491] KAFKA-1641; Reset first dirty offset for compaction to earliest offset if the checkpointed offset is invalid; reviewed by Joel Koshy --- .../scala/kafka/log/LogCleanerManager.scala | 30 +++++++++++++++---- 1 file changed, 24 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala index e8ced6a592250..bcfef77ed53f9 100644 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -75,13 +75,31 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To def grabFilthiestLog(): Option[LogToClean] = { inLock(lock) { val lastClean = allCleanerCheckpoints() - val dirtyLogs = logs.filter(l => l._2.config.compact) // skip any logs marked for delete rather than dedupe - .filterNot(l => inProgress.contains(l._1)) // skip any logs already in-progress - .map(l => LogToClean(l._1, l._2, // create a LogToClean instance for each - lastClean.getOrElse(l._1, l._2.logSegments.head.baseOffset))) - .filter(l => l.totalBytes > 0) // skip any empty logs + val dirtyLogs = logs.filter { + case (topicAndPartition, log) => log.config.compact // skip any logs marked for delete rather than dedupe + }.filterNot { + case (topicAndPartition, log) => inProgress.contains(topicAndPartition) // skip any logs already in-progress + }.map { + case (topicAndPartition, log) => // create a LogToClean instance for each + // if the log segments are abnormally truncated and hence the checkpointed offset + // is no longer valid, reset to the log starting offset and log the error event + val logStartOffset = log.logSegments.head.baseOffset + val firstDirtyOffset = { + val offset = lastClean.getOrElse(topicAndPartition, logStartOffset) + if (offset < logStartOffset) { + error("Resetting first dirty offset to log start offset %d since the checkpointed offset %d is invalid." + .format(logStartOffset, offset)) + logStartOffset + } else { + offset + } + } + LogToClean(topicAndPartition, log, firstDirtyOffset) + }.filter(ltc => ltc.totalBytes > 0) // skip any empty logs + this.dirtiestLogCleanableRatio = if (!dirtyLogs.isEmpty) dirtyLogs.max.cleanableRatio else 0 - val cleanableLogs = dirtyLogs.filter(l => l.cleanableRatio > l.log.config.minCleanableRatio) // and must meet the minimum threshold for dirty byte ratio + // and must meet the minimum threshold for dirty byte ratio + val cleanableLogs = dirtyLogs.filter(ltc => ltc.cleanableRatio > ltc.log.config.minCleanableRatio) if(cleanableLogs.isEmpty) { None } else { From b77099626e8fc258abed67dc10b2816347ba0f2b Mon Sep 17 00:00:00 2001 From: Geoffrey Anderson Date: Thu, 23 Oct 2014 16:12:20 -0700 Subject: [PATCH 032/491] kafka-1700; examples directory - README and shell scripts are out of date; patched by Geoffrey Anderson; reviewed by Jun Rao --- examples/README | 13 +------ examples/bin/java-producer-consumer-demo.sh | 43 ++------------------- examples/bin/java-simple-consumer-demo.sh | 43 ++------------------- 3 files changed, 7 insertions(+), 92 deletions(-) diff --git a/examples/README b/examples/README index 61de2868de29e..53db6969b2e2d 100644 --- a/examples/README +++ b/examples/README @@ -1,17 +1,6 @@ This directory contains examples of client code that uses kafka. -The default target for ant is kafka.examples.KafkaConsumerProducerDemo which sends and receives -messages from Kafka server. - -In order to run demo from SBT: - 1. Start Zookeeper and the Kafka server - 2. ./sbt from top-level kafka directory - 3. Switch to the kafka java examples project -> project kafka-examples - 4. execute run -> run - 5. For simple consumer demo, select option 1 - For unlimited producer-consumer run, select option 2 - -To run the demo using scripts: +To run the demo: 1. Start Zookeeper and the Kafka server 2. For simple consumer demo, run bin/java-simple-consumer-demo.sh diff --git a/examples/bin/java-producer-consumer-demo.sh b/examples/bin/java-producer-consumer-demo.sh index 29e01c2dcf823..fd25e5955397e 100755 --- a/examples/bin/java-producer-consumer-demo.sh +++ b/examples/bin/java-producer-consumer-demo.sh @@ -16,44 +16,7 @@ base_dir=$(dirname $0)/../.. -for file in $base_dir/project/boot/scala-2.8.0/lib/*.jar; -do - if [ ${file##*/} != "sbt-launch.jar" ]; then - CLASSPATH=$CLASSPATH:$file - fi -done - -for file in $base_dir/core/lib_managed/scala_2.8.0/compile/*.jar; -do - CLASSPATH=$CLASSPATH:$file -done - -for file in $base_dir/core/lib/*.jar; -do - CLASSPATH=$CLASSPATH:$file -done - -for file in $base_dir/core/target/scala_2.8.0/*.jar; -do - CLASSPATH=$CLASSPATH:$file -done - -for file in $base_dir/examples/target/scala_2.8.0/*.jar; -do - CLASSPATH=$CLASSPATH:$file -done - -echo $CLASSPATH - -if [ -z "$KAFKA_PERF_OPTS" ]; then - KAFKA_OPTS="-Xmx512M -server -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.port=3333 -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false" -fi - -if [ -z "$JAVA_HOME" ]; then - JAVA="java" -else - JAVA="$JAVA_HOME/bin/java" +if [ "x$KAFKA_HEAP_OPTS" = "x" ]; then + export KAFKA_HEAP_OPTS="-Xmx512M" fi - -$JAVA $KAFKA_OPTS -cp $CLASSPATH kafka.examples.KafkaConsumerProducerDemo $@ - +exec $base_dir/bin/kafka-run-class.sh kafka.examples.KafkaConsumerProducerDemo $@ diff --git a/examples/bin/java-simple-consumer-demo.sh b/examples/bin/java-simple-consumer-demo.sh index 4716a098c7d40..c4f103e827e69 100755 --- a/examples/bin/java-simple-consumer-demo.sh +++ b/examples/bin/java-simple-consumer-demo.sh @@ -16,44 +16,7 @@ base_dir=$(dirname $0)/../.. -for file in $base_dir/project/boot/scala-2.8.0/lib/*.jar; -do - if [ ${file##*/} != "sbt-launch.jar" ]; then - CLASSPATH=$CLASSPATH:$file - fi -done - -for file in $base_dir/core/lib_managed/scala_2.8.0/compile/*.jar; -do - CLASSPATH=$CLASSPATH:$file -done - -for file in $base_dir/core/lib/*.jar; -do - CLASSPATH=$CLASSPATH:$file -done - -for file in $base_dir/core/target/scala_2.8.0/*.jar; -do - CLASSPATH=$CLASSPATH:$file -done - -for file in $base_dir/examples/target/scala_2.8.0/*.jar; -do - CLASSPATH=$CLASSPATH:$file -done - -echo $CLASSPATH - -if [ -z "$KAFKA_PERF_OPTS" ]; then - KAFKA_OPTS="-Xmx512M -server -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.port=3333 -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false" -fi - -if [ -z "$JAVA_HOME" ]; then - JAVA="java" -else - JAVA="$JAVA_HOME/bin/java" +if [ "x$KAFKA_HEAP_OPTS" = "x" ]; then + export KAFKA_HEAP_OPTS="-Xmx512M" fi - -$JAVA $KAFKA_OPTS -cp $CLASSPATH kafka.examples.SimpleConsumerDemo $@ - +exec $base_dir/bin/kafka-run-class.sh kafka.examples.SimpleConsumerDemo $@ From e402c7be723e5f189a8f2b4f91a28f3b26ea9112 Mon Sep 17 00:00:00 2001 From: Joe Crobak Date: Thu, 23 Oct 2014 17:06:49 -0700 Subject: [PATCH 033/491] kafka-1711; WARN Property topic is not valid when running console producer; patched by Joe Crobak; reviewed by Jun Rao --- core/src/main/scala/kafka/tools/ConsoleProducer.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/tools/ConsoleProducer.scala b/core/src/main/scala/kafka/tools/ConsoleProducer.scala index 397d80da08c92..1061cc74fac69 100644 --- a/core/src/main/scala/kafka/tools/ConsoleProducer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleProducer.scala @@ -35,9 +35,11 @@ object ConsoleProducer { val config = new ProducerConfig(args) val reader = Class.forName(config.readerClass).newInstance().asInstanceOf[MessageReader] val props = new Properties - props.put("topic", config.topic) props.putAll(config.cmdLineProps) - reader.init(System.in, props) + + val readerProps = new Properties(props) + readerProps.put("topic", config.topic) + reader.init(System.in, readerProps) try { val producer = From 12ce4b1e1e0e6ec7fd3585baf3ed531ecc16a5d9 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Fri, 24 Oct 2014 09:04:14 -0700 Subject: [PATCH 034/491] KAFKA-1725 Configuration file bugs in system tests add noise to output and break a few tests; reviewed by Neha Narkhede --- ...se_5001_properties.json => testcase_15001_properties.json} | 0 ...se_5002_properties.json => testcase_15002_properties.json} | 0 ...se_5003_properties.json => testcase_15003_properties.json} | 0 ...se_5004_properties.json => testcase_15004_properties.json} | 0 ...se_5005_properties.json => testcase_15005_properties.json} | 0 ...se_5006_properties.json => testcase_15006_properties.json} | 0 .../testcase_0001/testcase_0001_properties.json | 2 +- ...se_0101_properties.json => testcase_10101_properties.json} | 0 ...se_0102_properties.json => testcase_10102_properties.json} | 0 ...se_0103_properties.json => testcase_10103_properties.json} | 0 ...se_0104_properties.json => testcase_10104_properties.json} | 0 ...se_0105_properties.json => testcase_10105_properties.json} | 0 ...se_0106_properties.json => testcase_10106_properties.json} | 0 ...se_0107_properties.json => testcase_10107_properties.json} | 0 ...se_0108_properties.json => testcase_10108_properties.json} | 0 ...se_0109_properties.json => testcase_10109_properties.json} | 0 ...se_0110_properties.json => testcase_10110_properties.json} | 0 ...se_0131_properties.json => testcase_10131_properties.json} | 0 ...se_0132_properties.json => testcase_10132_properties.json} | 0 ...se_0133_properties.json => testcase_10133_properties.json} | 0 ...se_0134_properties.json => testcase_10134_properties.json} | 0 system_test/utils/metrics.py | 2 ++ topics.json | 4 ++++ 23 files changed, 7 insertions(+), 1 deletion(-) rename system_test/mirror_maker_testsuite/testcase_15001/{testcase_5001_properties.json => testcase_15001_properties.json} (100%) rename system_test/mirror_maker_testsuite/testcase_15002/{testcase_5002_properties.json => testcase_15002_properties.json} (100%) rename system_test/mirror_maker_testsuite/testcase_15003/{testcase_5003_properties.json => testcase_15003_properties.json} (100%) rename system_test/mirror_maker_testsuite/testcase_15004/{testcase_5004_properties.json => testcase_15004_properties.json} (100%) rename system_test/mirror_maker_testsuite/testcase_15005/{testcase_5005_properties.json => testcase_15005_properties.json} (100%) rename system_test/mirror_maker_testsuite/testcase_15006/{testcase_5006_properties.json => testcase_15006_properties.json} (100%) rename system_test/replication_testsuite/testcase_10101/{testcase_0101_properties.json => testcase_10101_properties.json} (100%) rename system_test/replication_testsuite/testcase_10102/{testcase_0102_properties.json => testcase_10102_properties.json} (100%) rename system_test/replication_testsuite/testcase_10103/{testcase_0103_properties.json => testcase_10103_properties.json} (100%) rename system_test/replication_testsuite/testcase_10104/{testcase_0104_properties.json => testcase_10104_properties.json} (100%) rename system_test/replication_testsuite/testcase_10105/{testcase_0105_properties.json => testcase_10105_properties.json} (100%) rename system_test/replication_testsuite/testcase_10106/{testcase_0106_properties.json => testcase_10106_properties.json} (100%) rename system_test/replication_testsuite/testcase_10107/{testcase_0107_properties.json => testcase_10107_properties.json} (100%) rename system_test/replication_testsuite/testcase_10108/{testcase_0108_properties.json => testcase_10108_properties.json} (100%) rename system_test/replication_testsuite/testcase_10109/{testcase_0109_properties.json => testcase_10109_properties.json} (100%) rename system_test/replication_testsuite/testcase_10110/{testcase_0110_properties.json => testcase_10110_properties.json} (100%) rename system_test/replication_testsuite/testcase_10131/{testcase_0131_properties.json => testcase_10131_properties.json} (100%) rename system_test/replication_testsuite/testcase_10132/{testcase_0132_properties.json => testcase_10132_properties.json} (100%) rename system_test/replication_testsuite/testcase_10133/{testcase_0133_properties.json => testcase_10133_properties.json} (100%) rename system_test/replication_testsuite/testcase_10134/{testcase_0134_properties.json => testcase_10134_properties.json} (100%) create mode 100644 topics.json diff --git a/system_test/mirror_maker_testsuite/testcase_15001/testcase_5001_properties.json b/system_test/mirror_maker_testsuite/testcase_15001/testcase_15001_properties.json similarity index 100% rename from system_test/mirror_maker_testsuite/testcase_15001/testcase_5001_properties.json rename to system_test/mirror_maker_testsuite/testcase_15001/testcase_15001_properties.json diff --git a/system_test/mirror_maker_testsuite/testcase_15002/testcase_5002_properties.json b/system_test/mirror_maker_testsuite/testcase_15002/testcase_15002_properties.json similarity index 100% rename from system_test/mirror_maker_testsuite/testcase_15002/testcase_5002_properties.json rename to system_test/mirror_maker_testsuite/testcase_15002/testcase_15002_properties.json diff --git a/system_test/mirror_maker_testsuite/testcase_15003/testcase_5003_properties.json b/system_test/mirror_maker_testsuite/testcase_15003/testcase_15003_properties.json similarity index 100% rename from system_test/mirror_maker_testsuite/testcase_15003/testcase_5003_properties.json rename to system_test/mirror_maker_testsuite/testcase_15003/testcase_15003_properties.json diff --git a/system_test/mirror_maker_testsuite/testcase_15004/testcase_5004_properties.json b/system_test/mirror_maker_testsuite/testcase_15004/testcase_15004_properties.json similarity index 100% rename from system_test/mirror_maker_testsuite/testcase_15004/testcase_5004_properties.json rename to system_test/mirror_maker_testsuite/testcase_15004/testcase_15004_properties.json diff --git a/system_test/mirror_maker_testsuite/testcase_15005/testcase_5005_properties.json b/system_test/mirror_maker_testsuite/testcase_15005/testcase_15005_properties.json similarity index 100% rename from system_test/mirror_maker_testsuite/testcase_15005/testcase_5005_properties.json rename to system_test/mirror_maker_testsuite/testcase_15005/testcase_15005_properties.json diff --git a/system_test/mirror_maker_testsuite/testcase_15006/testcase_5006_properties.json b/system_test/mirror_maker_testsuite/testcase_15006/testcase_15006_properties.json similarity index 100% rename from system_test/mirror_maker_testsuite/testcase_15006/testcase_5006_properties.json rename to system_test/mirror_maker_testsuite/testcase_15006/testcase_15006_properties.json diff --git a/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json b/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json index 308f1937bbdc0..250ffe08ae4c2 100644 --- a/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json +++ b/system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json @@ -5,7 +5,7 @@ "04":"At the end it verifies the log size and contents", "05":"Use a consumer to verify no message loss.", "06":"Producer dimensions : mode:sync, acks:-1, comp:0", - "07":"Log segment size : 20480 + "07":"Log segment size : 20480" }, "testcase_args": { "broker_type": "leader", diff --git a/system_test/replication_testsuite/testcase_10101/testcase_0101_properties.json b/system_test/replication_testsuite/testcase_10101/testcase_10101_properties.json similarity index 100% rename from system_test/replication_testsuite/testcase_10101/testcase_0101_properties.json rename to system_test/replication_testsuite/testcase_10101/testcase_10101_properties.json diff --git a/system_test/replication_testsuite/testcase_10102/testcase_0102_properties.json b/system_test/replication_testsuite/testcase_10102/testcase_10102_properties.json similarity index 100% rename from system_test/replication_testsuite/testcase_10102/testcase_0102_properties.json rename to system_test/replication_testsuite/testcase_10102/testcase_10102_properties.json diff --git a/system_test/replication_testsuite/testcase_10103/testcase_0103_properties.json b/system_test/replication_testsuite/testcase_10103/testcase_10103_properties.json similarity index 100% rename from system_test/replication_testsuite/testcase_10103/testcase_0103_properties.json rename to system_test/replication_testsuite/testcase_10103/testcase_10103_properties.json diff --git a/system_test/replication_testsuite/testcase_10104/testcase_0104_properties.json b/system_test/replication_testsuite/testcase_10104/testcase_10104_properties.json similarity index 100% rename from system_test/replication_testsuite/testcase_10104/testcase_0104_properties.json rename to system_test/replication_testsuite/testcase_10104/testcase_10104_properties.json diff --git a/system_test/replication_testsuite/testcase_10105/testcase_0105_properties.json b/system_test/replication_testsuite/testcase_10105/testcase_10105_properties.json similarity index 100% rename from system_test/replication_testsuite/testcase_10105/testcase_0105_properties.json rename to system_test/replication_testsuite/testcase_10105/testcase_10105_properties.json diff --git a/system_test/replication_testsuite/testcase_10106/testcase_0106_properties.json b/system_test/replication_testsuite/testcase_10106/testcase_10106_properties.json similarity index 100% rename from system_test/replication_testsuite/testcase_10106/testcase_0106_properties.json rename to system_test/replication_testsuite/testcase_10106/testcase_10106_properties.json diff --git a/system_test/replication_testsuite/testcase_10107/testcase_0107_properties.json b/system_test/replication_testsuite/testcase_10107/testcase_10107_properties.json similarity index 100% rename from system_test/replication_testsuite/testcase_10107/testcase_0107_properties.json rename to system_test/replication_testsuite/testcase_10107/testcase_10107_properties.json diff --git a/system_test/replication_testsuite/testcase_10108/testcase_0108_properties.json b/system_test/replication_testsuite/testcase_10108/testcase_10108_properties.json similarity index 100% rename from system_test/replication_testsuite/testcase_10108/testcase_0108_properties.json rename to system_test/replication_testsuite/testcase_10108/testcase_10108_properties.json diff --git a/system_test/replication_testsuite/testcase_10109/testcase_0109_properties.json b/system_test/replication_testsuite/testcase_10109/testcase_10109_properties.json similarity index 100% rename from system_test/replication_testsuite/testcase_10109/testcase_0109_properties.json rename to system_test/replication_testsuite/testcase_10109/testcase_10109_properties.json diff --git a/system_test/replication_testsuite/testcase_10110/testcase_0110_properties.json b/system_test/replication_testsuite/testcase_10110/testcase_10110_properties.json similarity index 100% rename from system_test/replication_testsuite/testcase_10110/testcase_0110_properties.json rename to system_test/replication_testsuite/testcase_10110/testcase_10110_properties.json diff --git a/system_test/replication_testsuite/testcase_10131/testcase_0131_properties.json b/system_test/replication_testsuite/testcase_10131/testcase_10131_properties.json similarity index 100% rename from system_test/replication_testsuite/testcase_10131/testcase_0131_properties.json rename to system_test/replication_testsuite/testcase_10131/testcase_10131_properties.json diff --git a/system_test/replication_testsuite/testcase_10132/testcase_0132_properties.json b/system_test/replication_testsuite/testcase_10132/testcase_10132_properties.json similarity index 100% rename from system_test/replication_testsuite/testcase_10132/testcase_0132_properties.json rename to system_test/replication_testsuite/testcase_10132/testcase_10132_properties.json diff --git a/system_test/replication_testsuite/testcase_10133/testcase_0133_properties.json b/system_test/replication_testsuite/testcase_10133/testcase_10133_properties.json similarity index 100% rename from system_test/replication_testsuite/testcase_10133/testcase_0133_properties.json rename to system_test/replication_testsuite/testcase_10133/testcase_10133_properties.json diff --git a/system_test/replication_testsuite/testcase_10134/testcase_0134_properties.json b/system_test/replication_testsuite/testcase_10134/testcase_10134_properties.json similarity index 100% rename from system_test/replication_testsuite/testcase_10134/testcase_0134_properties.json rename to system_test/replication_testsuite/testcase_10134/testcase_10134_properties.json diff --git a/system_test/utils/metrics.py b/system_test/utils/metrics.py index d98d3cdeab00b..3e663483202a1 100644 --- a/system_test/utils/metrics.py +++ b/system_test/utils/metrics.py @@ -106,6 +106,8 @@ def ensure_valid_headers(headers, attributes): " headers: {0}".format(",".join(headers))) def plot_graphs(inputCsvFiles, labels, title, xLabel, yLabel, attribute, outputGraphFile): + if not inputCsvFiles: return + # create empty plot fig=plt.figure() fig.subplots_adjust(bottom=0.2) diff --git a/topics.json b/topics.json new file mode 100644 index 0000000000000..ff011ed381e78 --- /dev/null +++ b/topics.json @@ -0,0 +1,4 @@ +{"topics": + [{"topic": "foo"}], + "version":1 + } From 43cb192b59ab676d35c38f9144fbc98a954a11d3 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Fri, 24 Oct 2014 09:26:39 -0700 Subject: [PATCH 035/491] KAFKA-1719 Make mirror maker exit when one consumer/producer thread exits; reviewed by Neha Narkhede, Joel Koshy and Guozhang Wang --- .../main/scala/kafka/tools/MirrorMaker.scala | 74 ++++++++++++------- 1 file changed, 48 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index b8698ee1469c8..f399105087588 100644 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -17,26 +17,27 @@ package kafka.tools -import kafka.utils.{SystemTime, Utils, CommandLineUtils, Logging} import kafka.consumer._ -import kafka.serializer._ -import kafka.producer.{OldProducer, NewShinyProducer, BaseProducer} import kafka.metrics.KafkaMetricsGroup - +import kafka.producer.{BaseProducer, NewShinyProducer, OldProducer} +import kafka.serializer._ +import kafka.utils._ import org.apache.kafka.clients.producer.ProducerRecord +import java.util.Random +import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} +import java.util.concurrent.{BlockingQueue, ArrayBlockingQueue, CountDownLatch, TimeUnit} + import scala.collection.JavaConversions._ import joptsimple.OptionParser -import java.util.Random -import java.util.concurrent.atomic.AtomicInteger -import java.util.concurrent.{TimeUnit, BlockingQueue, ArrayBlockingQueue, CountDownLatch} object MirrorMaker extends Logging { private var connectors: Seq[ZookeeperConsumerConnector] = null private var consumerThreads: Seq[ConsumerThread] = null private var producerThreads: Seq[ProducerThread] = null + private val isShuttingdown: AtomicBoolean = new AtomicBoolean(false) private val shutdownMessage : ProducerRecord = new ProducerRecord("shutdown", "shutdown".getBytes) @@ -89,10 +90,10 @@ object MirrorMaker extends Logging { .ofType(classOf[String]) val blacklistOpt = parser.accepts("blacklist", - "Blacklist of topics to mirror.") - .withRequiredArg() - .describedAs("Java regex (String)") - .ofType(classOf[String]) + "Blacklist of topics to mirror.") + .withRequiredArg() + .describedAs("Java regex (String)") + .ofType(classOf[String]) val helpOpt = parser.accepts("help", "Print this message.") @@ -173,19 +174,21 @@ object MirrorMaker extends Logging { } def cleanShutdown() { - if (connectors != null) connectors.foreach(_.shutdown) - if (consumerThreads != null) consumerThreads.foreach(_.awaitShutdown) - if (producerThreads != null) { - producerThreads.foreach(_.shutdown) - producerThreads.foreach(_.awaitShutdown) + if (isShuttingdown.compareAndSet(false, true)) { + if (connectors != null) connectors.foreach(_.shutdown) + if (consumerThreads != null) consumerThreads.foreach(_.awaitShutdown) + if (producerThreads != null) { + producerThreads.foreach(_.shutdown) + producerThreads.foreach(_.awaitShutdown) + } + info("Kafka mirror maker shutdown successfully") } - info("Kafka mirror maker shutdown successfully") } - class DataChannel(capacity: Int, numProducers: Int, numConsumers: Int) extends KafkaMetricsGroup { + class DataChannel(capacity: Int, numInputs: Int, numOutputs: Int) extends KafkaMetricsGroup { - val queues = new Array[BlockingQueue[ProducerRecord]](numConsumers) - for (i <- 0 until numConsumers) + val queues = new Array[BlockingQueue[ProducerRecord]](numOutputs) + for (i <- 0 until numOutputs) queues(i) = new ArrayBlockingQueue[ProducerRecord](capacity) private val counter = new AtomicInteger(new Random().nextInt()) @@ -203,17 +206,21 @@ object MirrorMaker extends Logging { // Otherwise use the queue based on the key value so that same key-ed messages go to the same queue val queueId = if(record.key() != null) { - Utils.abs(java.util.Arrays.hashCode(record.key())) % numConsumers + Utils.abs(java.util.Arrays.hashCode(record.key())) % numOutputs } else { - Utils.abs(counter.getAndIncrement()) % numConsumers + Utils.abs(counter.getAndIncrement()) % numOutputs } + put(record, queueId) + } + + def put(record: ProducerRecord, queueId: Int) { val queue = queues(queueId) var putSucceed = false while (!putSucceed) { val startPutTime = SystemTime.nanoseconds putSucceed = queue.offer(record, 500, TimeUnit.MILLISECONDS) - waitPut.mark((SystemTime.nanoseconds - startPutTime) / numProducers) + waitPut.mark((SystemTime.nanoseconds - startPutTime) / numInputs) } channelSizeHist.update(queue.size) } @@ -224,7 +231,7 @@ object MirrorMaker extends Logging { while (data == null) { val startTakeTime = SystemTime.nanoseconds data = queue.poll(500, TimeUnit.MILLISECONDS) - waitTake.mark((SystemTime.nanoseconds - startTakeTime) / numConsumers) + waitTake.mark((SystemTime.nanoseconds - startTakeTime) / numOutputs) } channelSizeHist.update(queue.size) data @@ -238,6 +245,7 @@ object MirrorMaker extends Logging { private val shutdownLatch = new CountDownLatch(1) private val threadName = "mirrormaker-consumer-" + threadId + private var isCleanShutdown: Boolean = true this.logIdent = "[%s] ".format(threadName) this.setName(threadName) @@ -250,11 +258,18 @@ object MirrorMaker extends Logging { mirrorDataChannel.put(data) } } catch { - case e: Throwable => + case e: Throwable => { fatal("Stream unexpectedly exited.", e) + isCleanShutdown = false + } } finally { shutdownLatch.countDown() info("Consumer thread stopped") + // If it exits accidentally, stop the entire mirror maker. + if (!isCleanShutdown) { + fatal("Consumer thread exited abnormally, stopping the whole mirror maker.") + System.exit(-1) + } } } @@ -273,6 +288,7 @@ object MirrorMaker extends Logging { val threadId: Int) extends Thread with Logging with KafkaMetricsGroup { private val threadName = "mirrormaker-producer-" + threadId private val shutdownComplete: CountDownLatch = new CountDownLatch(1) + private var isCleanShutdown: Boolean = true this.logIdent = "[%s] ".format(threadName) setName(threadName) @@ -292,17 +308,23 @@ object MirrorMaker extends Logging { } catch { case t: Throwable => { fatal("Producer thread failure due to ", t) + isCleanShutdown = false } } finally { shutdownComplete.countDown info("Producer thread stopped") + // If it exits accidentally, stop the entire mirror maker. + if (!isCleanShutdown) { + fatal("Producer thread exited abnormally, stopping the whole mirror maker.") + System.exit(-1) + } } } def shutdown { try { info("Producer thread " + threadName + " shutting down") - dataChannel.put(shutdownMessage) + dataChannel.put(shutdownMessage, threadId) } catch { case ie: InterruptedException => { From 20f5b01feacd1e95d558073b7be511cb82888b3e Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Mon, 27 Oct 2014 16:23:24 -0700 Subject: [PATCH 036/491] KAFKA-1732 DumpLogSegments tool fails when path has a '.'; reviewed by Neha Narkhede --- core/src/main/scala/kafka/tools/DumpLogSegments.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index 8e9d47b8d4adc..fe2cc11b75f37 100644 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -104,8 +104,7 @@ object DumpLogSegments { misMatchesForIndexFilesMap: mutable.HashMap[String, List[(Long, Long)]], maxMessageSize: Int) { val startOffset = file.getName().split("\\.")(0).toLong - val logFileName = file.getAbsolutePath.split("\\.")(0) + Log.LogFileSuffix - val logFile = new File(logFileName) + val logFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.LogFileSuffix) val messageSet = new FileMessageSet(logFile, false) val index = new OffsetIndex(file = file, baseOffset = startOffset) for(i <- 0 until index.entries) { From 89831204c092f3a417bf41945925a2e9a0ec828e Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 29 Oct 2014 17:11:03 -0700 Subject: [PATCH 037/491] KAFKA-1583; Encapsulate replicated log implementation details into ReplicaManager and refactor KafkaApis; reviewed by Joel Koshy and Jun Rao --- .../main/scala/kafka/api/FetchRequest.scala | 1 - .../main/scala/kafka/api/FetchResponse.scala | 12 +- .../scala/kafka/api/OffsetCommitRequest.scala | 24 +- .../scala/kafka/api/ProducerRequest.scala | 1 - .../scala/kafka/api/ProducerResponse.scala | 3 +- .../main/scala/kafka/cluster/Partition.scala | 50 ++- .../scala/kafka/common/ErrorMapping.scala | 2 + core/src/main/scala/kafka/log/Log.scala | 26 +- .../kafka/network/BoundedByteBufferSend.scala | 4 +- .../scala/kafka/server/DelayedFetch.scala | 125 +++--- .../scala/kafka/server/DelayedProduce.scala | 135 ++++--- .../kafka/server/FetchRequestPurgatory.scala | 69 ---- .../main/scala/kafka/server/KafkaApis.scala | 325 +++++---------- .../scala/kafka/server/OffsetManager.scala | 93 ++++- .../server/ProducerRequestPurgatory.scala | 69 ---- .../scala/kafka/server/ReplicaManager.scala | 370 ++++++++++++------ .../scala/kafka/server/RequestPurgatory.scala | 336 ++++++++-------- .../main/scala/kafka/utils/DelayedItem.scala | 8 +- .../api/ProducerFailureHandlingTest.scala | 18 +- .../kafka/producer/SyncProducerTest.scala | 4 +- .../server/HighwatermarkPersistenceTest.scala | 7 + .../unit/kafka/server/ISRExpirationTest.scala | 17 +- .../kafka/server/ReplicaManagerTest.scala | 6 + .../kafka/server/RequestPurgatoryTest.scala | 103 ++--- .../kafka/server/ServerShutdownTest.scala | 2 +- .../unit/kafka/server/SimpleFetchTest.scala | 270 ++++--------- 26 files changed, 1006 insertions(+), 1074 deletions(-) delete mode 100644 core/src/main/scala/kafka/server/FetchRequestPurgatory.scala delete mode 100644 core/src/main/scala/kafka/server/ProducerRequestPurgatory.scala diff --git a/core/src/main/scala/kafka/api/FetchRequest.scala b/core/src/main/scala/kafka/api/FetchRequest.scala index 59c09155dd25f..b038c15186c0c 100644 --- a/core/src/main/scala/kafka/api/FetchRequest.scala +++ b/core/src/main/scala/kafka/api/FetchRequest.scala @@ -30,7 +30,6 @@ import scala.collection.immutable.Map case class PartitionFetchInfo(offset: Long, fetchSize: Int) - object FetchRequest { val CurrentVersion = 0.shortValue val DefaultMaxWait = 0 diff --git a/core/src/main/scala/kafka/api/FetchResponse.scala b/core/src/main/scala/kafka/api/FetchResponse.scala index 8d085a1f18f80..75aaf57fb76ec 100644 --- a/core/src/main/scala/kafka/api/FetchResponse.scala +++ b/core/src/main/scala/kafka/api/FetchResponse.scala @@ -25,6 +25,8 @@ import kafka.message.{MessageSet, ByteBufferMessageSet} import kafka.network.{MultiSend, Send} import kafka.api.ApiUtils._ +import scala.collection._ + object FetchResponsePartitionData { def readFrom(buffer: ByteBuffer): FetchResponsePartitionData = { val error = buffer.getShort @@ -150,10 +152,8 @@ object FetchResponse { } } - -case class FetchResponse(correlationId: Int, - data: Map[TopicAndPartition, FetchResponsePartitionData]) - extends RequestOrResponse() { +case class FetchResponse(correlationId: Int, data: Map[TopicAndPartition, FetchResponsePartitionData]) + extends RequestOrResponse() { /** * Partitions the data into a map of maps (one for each topic). @@ -171,8 +171,8 @@ case class FetchResponse(correlationId: Int, /* * FetchResponse uses [sendfile](http://man7.org/linux/man-pages/man2/sendfile.2.html) - * api for data transfer, so `writeTo` aren't actually being used. - * It is implemented as an empty function to comform to `RequestOrResponse.writeTo` + * api for data transfer through the FetchResponseSend, so `writeTo` aren't actually being used. + * It is implemented as an empty function to conform to `RequestOrResponse.writeTo` * abstract method signature. */ def writeTo(buffer: ByteBuffer): Unit = throw new UnsupportedOperationException diff --git a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala index 861a6cf11dc6b..050615c72efe7 100644 --- a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala @@ -78,28 +78,12 @@ case class OffsetCommitRequest(groupId: String, groupGenerationId: Int = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_GENERATION_ID, consumerId: String = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_CONSUMER_ID) extends RequestOrResponse(Some(RequestKeys.OffsetCommitKey)) { + assert(versionId == 0 || versionId == 1, "Version " + versionId + " is invalid for OffsetCommitRequest. Valid versions are 0 or 1.") lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic) - def filterLargeMetadata(maxMetadataSize: Int) = - requestInfo.filter(info => info._2.metadata == null || info._2.metadata.length <= maxMetadataSize) - - def responseFor(errorCode: Short, offsetMetadataMaxSize: Int) = { - val commitStatus = requestInfo.map {info => - (info._1, if (info._2.metadata != null && info._2.metadata.length > offsetMetadataMaxSize) - ErrorMapping.OffsetMetadataTooLargeCode - else if (errorCode == ErrorMapping.UnknownTopicOrPartitionCode) - ErrorMapping.ConsumerCoordinatorNotAvailableCode - else if (errorCode == ErrorMapping.NotLeaderForPartitionCode) - ErrorMapping.NotCoordinatorForConsumerCode - else - errorCode) - }.toMap - OffsetCommitResponse(commitStatus, correlationId) - } - def writeTo(buffer: ByteBuffer) { // Write envelope buffer.putShort(versionId) @@ -150,8 +134,10 @@ case class OffsetCommitRequest(groupId: String, override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { val errorCode = ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) - val errorResponse = responseFor(errorCode, Int.MaxValue) - requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + val commitStatus = requestInfo.mapValues(_ => errorCode) + val commitResponse = OffsetCommitResponse(commitStatus, correlationId) + + requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(commitResponse))) } override def describe(details: Boolean): String = { diff --git a/core/src/main/scala/kafka/api/ProducerRequest.scala b/core/src/main/scala/kafka/api/ProducerRequest.scala index b2366e7eedcac..570b2da1d8650 100644 --- a/core/src/main/scala/kafka/api/ProducerRequest.scala +++ b/core/src/main/scala/kafka/api/ProducerRequest.scala @@ -153,7 +153,6 @@ case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion, producerRequest.toString() } - def emptyData(){ data.clear() } diff --git a/core/src/main/scala/kafka/api/ProducerResponse.scala b/core/src/main/scala/kafka/api/ProducerResponse.scala index a286272c834b6..5d1fac4cb8943 100644 --- a/core/src/main/scala/kafka/api/ProducerResponse.scala +++ b/core/src/main/scala/kafka/api/ProducerResponse.scala @@ -43,8 +43,7 @@ object ProducerResponse { case class ProducerResponseStatus(var error: Short, offset: Long) -case class ProducerResponse(correlationId: Int, - status: Map[TopicAndPartition, ProducerResponseStatus]) +case class ProducerResponse(correlationId: Int, status: Map[TopicAndPartition, ProducerResponseStatus]) extends RequestOrResponse() { /** diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index e88ecf224a4da..1be57008e983f 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -230,7 +230,33 @@ class Partition(val topic: String, } } - def updateLeaderHWAndMaybeExpandIsr(replicaId: Int) { + /** + * Update the log end offset of a certain replica of this partition + */ + def updateReplicaLEO(replicaId: Int, offset: LogOffsetMetadata) = { + getReplica(replicaId) match { + case Some(replica) => + replica.logEndOffset = offset + + // check if we need to expand ISR to include this replica + // if it is not in the ISR yet + maybeExpandIsr(replicaId) + + debug("Recorded replica %d log end offset (LEO) position %d for partition %s." + .format(replicaId, offset.messageOffset, TopicAndPartition(topic, partitionId))) + case None => + throw new NotAssignedReplicaException(("Leader %d failed to record follower %d's position %d since the replica" + + " is not recognized to be one of the assigned replicas %s for partition [%s,%d]").format(localBrokerId, replicaId, + offset.messageOffset, assignedReplicas().map(_.brokerId).mkString(","), topic, partitionId)) + } + } + + /** + * Check and maybe expand the ISR of the partition. + * + * This function can be triggered when a replica's LEO has incremented + */ + def maybeExpandIsr(replicaId: Int) { inWriteLock(leaderIsrUpdateLock) { // check if this replica needs to be added to the ISR leaderReplicaIfLocal() match { @@ -252,7 +278,10 @@ class Partition(val topic: String, updateIsr(newInSyncReplicas) replicaManager.isrExpandRate.mark() } + // check if the HW of the partition can now be incremented + // since the replica maybe now be in the ISR and its LEO has just incremented maybeIncrementLeaderHW(leaderReplica) + case None => // nothing to do if no longer leader } } @@ -272,6 +301,7 @@ class Partition(val topic: String, val minIsr = leaderReplica.log.get.config.minInSyncReplicas trace("%d/%d acks satisfied for %s-%d".format(numAcks, requiredAcks, topic, partitionId)) + if (requiredAcks < 0 && leaderReplica.highWatermark.messageOffset >= requiredOffset ) { /* * requiredAcks < 0 means acknowledge after all replicas in ISR @@ -298,8 +328,14 @@ class Partition(val topic: String, } /** - * There is no need to acquire the leaderIsrUpdate lock here since all callers of this private API acquire that lock - * @param leaderReplica + * Check and maybe increment the high watermark of the partition; + * this function can be triggered when + * + * 1. Partition ISR changed + * 2. Any replica's LEO changed + * + * Note There is no need to acquire the leaderIsrUpdate lock here + * since all callers of this private API acquire that lock */ private def maybeIncrementLeaderHW(leaderReplica: Replica) { val allLogEndOffsets = inSyncReplicas.map(_.logEndOffset) @@ -310,8 +346,8 @@ class Partition(val topic: String, debug("High watermark for partition [%s,%d] updated to %s".format(topic, partitionId, newHighWatermark)) // some delayed requests may be unblocked after HW changed val requestKey = new TopicPartitionRequestKey(this.topic, this.partitionId) - replicaManager.unblockDelayedFetchRequests(requestKey) - replicaManager.unblockDelayedProduceRequests(requestKey) + replicaManager.tryCompleteDelayedFetch(requestKey) + replicaManager.tryCompleteDelayedProduce(requestKey) } else { debug("Skipping update high watermark since Old hw %s is larger than new hw %s for partition [%s,%d]. All leo's are %s" .format(oldHighWatermark, newHighWatermark, topic, partitionId, allLogEndOffsets.mkString(","))) @@ -362,7 +398,7 @@ class Partition(val topic: String, stuckReplicas ++ slowReplicas } - def appendMessagesToLeader(messages: ByteBufferMessageSet, requiredAcks: Int=0) = { + def appendMessagesToLeader(messages: ByteBufferMessageSet, requiredAcks: Int = 0) = { inReadLock(leaderIsrUpdateLock) { val leaderReplicaOpt = leaderReplicaIfLocal() leaderReplicaOpt match { @@ -379,7 +415,7 @@ class Partition(val topic: String, val info = log.append(messages, assignOffsets = true) // probably unblock some follower fetch requests since log end offset has been updated - replicaManager.unblockDelayedFetchRequests(new TopicPartitionRequestKey(this.topic, this.partitionId)) + replicaManager.tryCompleteDelayedFetch(new TopicPartitionRequestKey(this.topic, this.partitionId)) // we may need to increment high watermark since ISR could be down to 1 maybeIncrementLeaderHW(leaderReplica) info diff --git a/core/src/main/scala/kafka/common/ErrorMapping.scala b/core/src/main/scala/kafka/common/ErrorMapping.scala index 880ab4a004f07..eedc2f5f21dd8 100644 --- a/core/src/main/scala/kafka/common/ErrorMapping.scala +++ b/core/src/main/scala/kafka/common/ErrorMapping.scala @@ -84,4 +84,6 @@ object ErrorMapping { throw codeToException(code).newInstance() def exceptionFor(code: Short) : Throwable = codeToException(code).newInstance() + + def exceptionNameFor(code: Short) : String = codeToException(code).getName() } diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 157d67369baab..37b4a85cca06a 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -31,6 +31,21 @@ import scala.collection.JavaConversions import com.yammer.metrics.core.Gauge +object LogAppendInfo { + val UnknownLogAppendInfo = LogAppendInfo(-1, -1, NoCompressionCodec, -1, -1, false) +} + +/** + * Struct to hold various quantities we compute about each message set before appending to the log + * @param firstOffset The first offset in the message set + * @param lastOffset The last offset in the message set + * @param shallowCount The number of shallow messages + * @param validBytes The number of valid bytes + * @param codec The codec used in the message set + * @param offsetsMonotonic Are the offsets in this message set monotonically increasing + */ +case class LogAppendInfo(var firstOffset: Long, var lastOffset: Long, codec: CompressionCodec, shallowCount: Int, validBytes: Int, offsetsMonotonic: Boolean) + /** * An append-only log for storing messages. @@ -310,17 +325,6 @@ class Log(val dir: File, } } - /** - * Struct to hold various quantities we compute about each message set before appending to the log - * @param firstOffset The first offset in the message set - * @param lastOffset The last offset in the message set - * @param shallowCount The number of shallow messages - * @param validBytes The number of valid bytes - * @param codec The codec used in the message set - * @param offsetsMonotonic Are the offsets in this message set monotonically increasing - */ - case class LogAppendInfo(var firstOffset: Long, var lastOffset: Long, codec: CompressionCodec, shallowCount: Int, validBytes: Int, offsetsMonotonic: Boolean) - /** * Validate the following: *
      diff --git a/core/src/main/scala/kafka/network/BoundedByteBufferSend.scala b/core/src/main/scala/kafka/network/BoundedByteBufferSend.scala index a624359fb2059..55ecac285e00a 100644 --- a/core/src/main/scala/kafka/network/BoundedByteBufferSend.scala +++ b/core/src/main/scala/kafka/network/BoundedByteBufferSend.scala @@ -25,7 +25,7 @@ import kafka.api.RequestOrResponse @nonthreadsafe private[kafka] class BoundedByteBufferSend(val buffer: ByteBuffer) extends Send { - private var sizeBuffer = ByteBuffer.allocate(4) + private val sizeBuffer = ByteBuffer.allocate(4) // Avoid possibility of overflow for 2GB-4 byte buffer if(buffer.remaining > Int.MaxValue - sizeBuffer.limit) @@ -53,7 +53,7 @@ private[kafka] class BoundedByteBufferSend(val buffer: ByteBuffer) extends Send def writeTo(channel: GatheringByteChannel): Int = { expectIncomplete() - var written = channel.write(Array(sizeBuffer, buffer)) + val written = channel.write(Array(sizeBuffer, buffer)) // if we are done, mark it off if(!buffer.hasRemaining) complete = true diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index e0f14e25af03e..1ccbb4b6fdbbd 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -17,75 +17,110 @@ package kafka.server -import kafka.network.RequestChannel -import kafka.api.{FetchResponse, FetchRequest} -import kafka.common.{UnknownTopicOrPartitionException, NotLeaderForPartitionException, TopicAndPartition} +import kafka.api.FetchResponsePartitionData +import kafka.api.PartitionFetchInfo +import kafka.common.UnknownTopicOrPartitionException +import kafka.common.NotLeaderForPartitionException +import kafka.common.TopicAndPartition -import scala.collection.immutable.Map -import scala.collection.Seq +import scala.collection._ + +case class FetchPartitionStatus(startOffsetMetadata: LogOffsetMetadata, fetchInfo: PartitionFetchInfo) { + + override def toString = "[startOffsetMetadata: " + startOffsetMetadata + ", " + + "fetchInfo: " + fetchInfo + "]" +} /** - * A delayed fetch request, which is satisfied (or more - * accurately, unblocked) -- if: - * Case A: This broker is no longer the leader for some partitions it tries to fetch - * - should return whatever data is available for the rest partitions. - * Case B: This broker is does not know of some partitions it tries to fetch - * - should return whatever data is available for the rest partitions. - * Case C: The fetch offset locates not on the last segment of the log - * - should return all the data on that segment. - * Case D: The accumulated bytes from all the fetching partitions exceeds the minimum bytes - * - should return whatever data is available. + * The fetch metadata maintained by the delayed produce request */ +case class FetchMetadata(fetchMinBytes: Int, + fetchOnlyLeader: Boolean, + fetchOnlyCommitted: Boolean, + fetchPartitionStatus: Map[TopicAndPartition, FetchPartitionStatus]) { + + override def toString = "[minBytes: " + fetchMinBytes + ", " + + "onlyLeader:" + fetchOnlyLeader + ", " + "onlyCommitted: " + fetchOnlyCommitted + ", " + "partitionStatus: " + fetchPartitionStatus + "]" +} -class DelayedFetch(override val keys: Seq[TopicPartitionRequestKey], - override val request: RequestChannel.Request, - override val delayMs: Long, - val fetch: FetchRequest, - private val partitionFetchOffsets: Map[TopicAndPartition, LogOffsetMetadata]) - extends DelayedRequest(keys, request, delayMs) { +/** + * A delayed fetch request that can be created by the replica manager and watched + * in the fetch request purgatory + */ +class DelayedFetch(delayMs: Long, + fetchMetadata: FetchMetadata, + replicaManager: ReplicaManager, + responseCallback: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) + extends DelayedRequest(delayMs) { - def isSatisfied(replicaManager: ReplicaManager) : Boolean = { + /** + * The request can be completed if: + * + * Case A: This broker is no longer the leader for some partitions it tries to fetch + * Case B: This broker does not know of some partitions it tries to fetch + * Case C: The fetch offset locates not on the last segment of the log + * Case D: The accumulated bytes from all the fetching partitions exceeds the minimum bytes + * + * Upon completion, should return whatever data is available for each valid partition + */ + override def tryComplete() : Boolean = { var accumulatedSize = 0 - val fromFollower = fetch.isFromFollower - partitionFetchOffsets.foreach { - case (topicAndPartition, fetchOffset) => + fetchMetadata.fetchPartitionStatus.foreach { + case (topicAndPartition, fetchStatus) => + val fetchOffset = fetchStatus.startOffsetMetadata try { if (fetchOffset != LogOffsetMetadata.UnknownOffsetMetadata) { val replica = replicaManager.getLeaderReplicaIfLocal(topicAndPartition.topic, topicAndPartition.partition) val endOffset = - if (fromFollower) - replica.logEndOffset - else + if (fetchMetadata.fetchOnlyCommitted) replica.highWatermark + else + replica.logEndOffset if (endOffset.offsetOnOlderSegment(fetchOffset)) { - // Case C, this can happen when the new follower replica fetching on a truncated leader - debug("Satisfying fetch request %s since it is fetching later segments of partition %s.".format(fetch, topicAndPartition)) - return true + // Case C, this can happen when the new fetch operation is on a truncated leader + debug("Satisfying fetch %s since it is fetching later segments of partition %s.".format(fetchMetadata, topicAndPartition)) + return forceComplete() } else if (fetchOffset.offsetOnOlderSegment(endOffset)) { - // Case C, this can happen when the folloer replica is lagging too much - debug("Satisfying fetch request %s immediately since it is fetching older segments.".format(fetch)) - return true + // Case C, this can happen when the fetch operation is falling behind the current segment + // or the partition has just rolled a new segment + debug("Satisfying fetch %s immediately since it is fetching older segments.".format(fetchMetadata)) + return forceComplete() } else if (fetchOffset.precedes(endOffset)) { - accumulatedSize += endOffset.positionDiff(fetchOffset) + // we need take the partition fetch size as upper bound when accumulating the bytes + accumulatedSize += math.min(endOffset.positionDiff(fetchOffset), fetchStatus.fetchInfo.fetchSize) } } } catch { - case utpe: UnknownTopicOrPartitionException => // Case A - debug("Broker no longer know of %s, satisfy %s immediately".format(topicAndPartition, fetch)) - return true - case nle: NotLeaderForPartitionException => // Case B - debug("Broker is no longer the leader of %s, satisfy %s immediately".format(topicAndPartition, fetch)) - return true + case utpe: UnknownTopicOrPartitionException => // Case B + debug("Broker no longer know of %s, satisfy %s immediately".format(topicAndPartition, fetchMetadata)) + return forceComplete() + case nle: NotLeaderForPartitionException => // Case A + debug("Broker is no longer the leader of %s, satisfy %s immediately".format(topicAndPartition, fetchMetadata)) + return forceComplete() } } // Case D - accumulatedSize >= fetch.minBytes + if (accumulatedSize >= fetchMetadata.fetchMinBytes) + forceComplete() + else + false } - def respond(replicaManager: ReplicaManager): FetchResponse = { - val topicData = replicaManager.readMessageSets(fetch) - FetchResponse(fetch.correlationId, topicData.mapValues(_.data)) + /** + * Upon completion, read whatever data is available and pass to the complete callback + */ + override def onComplete() { + val logReadResults = replicaManager.readFromLocalLog(fetchMetadata.fetchOnlyLeader, + fetchMetadata.fetchOnlyCommitted, + fetchMetadata.fetchPartitionStatus.mapValues(status => status.fetchInfo)) + + val fetchPartitionData = logReadResults.mapValues(result => + FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)) + + responseCallback(fetchPartitionData) } } \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index 9481508fc2d61..8049e07e5d6d6 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -17,99 +17,106 @@ package kafka.server -import kafka.api._ + +import kafka.api.ProducerResponseStatus import kafka.common.ErrorMapping import kafka.common.TopicAndPartition -import kafka.utils.Logging -import kafka.network.RequestChannel import scala.Some -import scala.collection.immutable.Map -import scala.collection.Seq - -/** A delayed produce request, which is satisfied (or more - * accurately, unblocked) -- if for every partition it produce to: - * Case A: This broker is not the leader: unblock - should return error. - * Case B: This broker is the leader: - * B.1 - If there was a localError (when writing to the local log): unblock - should return error - * B.2 - else, at least requiredAcks replicas should be caught up to this request. - */ +import scala.collection._ -class DelayedProduce(override val keys: Seq[TopicPartitionRequestKey], - override val request: RequestChannel.Request, - override val delayMs: Long, - val produce: ProducerRequest, - val partitionStatus: Map[TopicAndPartition, DelayedProduceResponseStatus], - val offsetCommitRequestOpt: Option[OffsetCommitRequest] = None) - extends DelayedRequest(keys, request, delayMs) with Logging { +case class ProducePartitionStatus(requiredOffset: Long, responseStatus: ProducerResponseStatus) { + @volatile var acksPending = false - // first update the acks pending variable according to the error code - partitionStatus foreach { case (topicAndPartition, delayedStatus) => - if (delayedStatus.responseStatus.error == ErrorMapping.NoError) { - // Timeout error state will be cleared when required acks are received - delayedStatus.acksPending = true - delayedStatus.responseStatus.error = ErrorMapping.RequestTimedOutCode - } else { - delayedStatus.acksPending = false - } + override def toString = "[acksPending: %b, error: %d, startOffset: %d, requiredOffset: %d]" + .format(acksPending, responseStatus.error, responseStatus.offset, requiredOffset) +} - trace("Initial partition status for %s is %s".format(topicAndPartition, delayedStatus)) - } +/** + * The produce metadata maintained by the delayed produce request + */ +case class ProduceMetadata(produceRequiredAcks: Short, + produceStatus: Map[TopicAndPartition, ProducePartitionStatus]) { - def respond(offsetManager: OffsetManager): RequestOrResponse = { - val responseStatus = partitionStatus.mapValues(status => status.responseStatus) + override def toString = "[requiredAcks: %d, partitionStatus: %s]" + .format(produceRequiredAcks, produceStatus) +} - val errorCode = responseStatus.find { case (_, status) => - status.error != ErrorMapping.NoError - }.map(_._2.error).getOrElse(ErrorMapping.NoError) +/** + * A delayed produce request that can be created by the replica manager and watched + * in the produce request purgatory + */ +class DelayedProduce(delayMs: Long, + produceMetadata: ProduceMetadata, + replicaManager: ReplicaManager, + responseCallback: Map[TopicAndPartition, ProducerResponseStatus] => Unit) + extends DelayedRequest(delayMs) { - if (errorCode == ErrorMapping.NoError) { - offsetCommitRequestOpt.foreach(ocr => offsetManager.putOffsets(ocr.groupId, ocr.requestInfo) ) + // first update the acks pending variable according to the error code + produceMetadata.produceStatus.foreach { case (topicAndPartition, status) => + if (status.responseStatus.error == ErrorMapping.NoError) { + // Timeout error state will be cleared when required acks are received + status.acksPending = true + status.responseStatus.error = ErrorMapping.RequestTimedOutCode + } else { + status.acksPending = false } - val response = offsetCommitRequestOpt.map(_.responseFor(errorCode, offsetManager.config.maxMetadataSize)) - .getOrElse(ProducerResponse(produce.correlationId, responseStatus)) - - response + trace("Initial partition status for %s is %s".format(topicAndPartition, status)) } - def isSatisfied(replicaManager: ReplicaManager) = { + /** + * The delayed produce request can be completed if every partition + * it produces to is satisfied by one of the following: + * + * Case A: This broker is no longer the leader: set an error in response + * Case B: This broker is the leader: + * B.1 - If there was a local error thrown while checking if at least requiredAcks + * replicas have caught up to this request: set an error in response + * B.2 - Otherwise, set the response with no error. + */ + override def tryComplete(): Boolean = { // check for each partition if it still has pending acks - partitionStatus.foreach { case (topicAndPartition, fetchPartitionStatus) => - trace("Checking producer request satisfaction for %s, acksPending = %b" - .format(topicAndPartition, fetchPartitionStatus.acksPending)) + produceMetadata.produceStatus.foreach { case (topicAndPartition, status) => + trace("Checking produce satisfaction for %s, current status %s" + .format(topicAndPartition, status)) // skip those partitions that have already been satisfied - if (fetchPartitionStatus.acksPending) { + if (status.acksPending) { val partitionOpt = replicaManager.getPartition(topicAndPartition.topic, topicAndPartition.partition) val (hasEnough, errorCode) = partitionOpt match { case Some(partition) => partition.checkEnoughReplicasReachOffset( - fetchPartitionStatus.requiredOffset, - produce.requiredAcks) + status.requiredOffset, + produceMetadata.produceRequiredAcks) case None => + // Case A (false, ErrorMapping.UnknownTopicOrPartitionCode) } if (errorCode != ErrorMapping.NoError) { - fetchPartitionStatus.acksPending = false - fetchPartitionStatus.responseStatus.error = errorCode + // Case B.1 + status.acksPending = false + status.responseStatus.error = errorCode } else if (hasEnough) { - fetchPartitionStatus.acksPending = false - fetchPartitionStatus.responseStatus.error = ErrorMapping.NoError + // Case B.2 + status.acksPending = false + status.responseStatus.error = ErrorMapping.NoError } } } - // unblocked if there are no partitions with pending acks - val satisfied = ! partitionStatus.exists(p => p._2.acksPending) - satisfied + // check if each partition has satisfied at lease one of case A and case B + if (! produceMetadata.produceStatus.values.exists(p => p.acksPending)) + forceComplete() + else + false } -} - -case class DelayedProduceResponseStatus(val requiredOffset: Long, - val responseStatus: ProducerResponseStatus) { - @volatile var acksPending = false - override def toString = - "acksPending:%b, error: %d, startOffset: %d, requiredOffset: %d".format( - acksPending, responseStatus.error, responseStatus.offset, requiredOffset) + /** + * Upon completion, return the current response status along with the error code per partition + */ + override def onComplete() { + val responseStatus = produceMetadata.produceStatus.mapValues(status => status.responseStatus) + responseCallback(responseStatus) + } } + diff --git a/core/src/main/scala/kafka/server/FetchRequestPurgatory.scala b/core/src/main/scala/kafka/server/FetchRequestPurgatory.scala deleted file mode 100644 index ed13188912535..0000000000000 --- a/core/src/main/scala/kafka/server/FetchRequestPurgatory.scala +++ /dev/null @@ -1,69 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.server - -import kafka.metrics.KafkaMetricsGroup -import kafka.network.RequestChannel -import kafka.api.FetchResponseSend - -import java.util.concurrent.TimeUnit - -/** - * The purgatory holding delayed fetch requests - */ -class FetchRequestPurgatory(replicaManager: ReplicaManager, requestChannel: RequestChannel) - extends RequestPurgatory[DelayedFetch](replicaManager.config.brokerId, replicaManager.config.fetchPurgatoryPurgeIntervalRequests) { - this.logIdent = "[FetchRequestPurgatory-%d] ".format(replicaManager.config.brokerId) - - private class DelayedFetchRequestMetrics(forFollower: Boolean) extends KafkaMetricsGroup { - private val metricPrefix = if (forFollower) "Follower" else "Consumer" - - val expiredRequestMeter = newMeter(metricPrefix + "ExpiresPerSecond", "requests", TimeUnit.SECONDS) - } - - private val aggregateFollowerFetchRequestMetrics = new DelayedFetchRequestMetrics(forFollower = true) - private val aggregateNonFollowerFetchRequestMetrics = new DelayedFetchRequestMetrics(forFollower = false) - - private def recordDelayedFetchExpired(forFollower: Boolean) { - val metrics = if (forFollower) aggregateFollowerFetchRequestMetrics - else aggregateNonFollowerFetchRequestMetrics - - metrics.expiredRequestMeter.mark() - } - - /** - * Check if a specified delayed fetch request is satisfied - */ - def checkSatisfied(delayedFetch: DelayedFetch): Boolean = delayedFetch.isSatisfied(replicaManager) - - /** - * When a delayed fetch request expires just answer it with whatever data is present - */ - def expire(delayedFetch: DelayedFetch) { - debug("Expiring fetch request %s.".format(delayedFetch.fetch)) - val fromFollower = delayedFetch.fetch.isFromFollower - recordDelayedFetchExpired(fromFollower) - respond(delayedFetch) - } - - // TODO: purgatory should not be responsible for sending back the responses - def respond(delayedFetch: DelayedFetch) { - val response = delayedFetch.respond(replicaManager) - requestChannel.sendResponse(new RequestChannel.Response(delayedFetch.request, new FetchResponseSend(response))) - } -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 85498b4a1368d..968b0c4f809ea 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -20,7 +20,6 @@ package kafka.server import kafka.api._ import kafka.common._ import kafka.log._ -import kafka.message._ import kafka.network._ import kafka.admin.AdminUtils import kafka.network.RequestChannel.Response @@ -42,12 +41,8 @@ class KafkaApis(val requestChannel: RequestChannel, val config: KafkaConfig, val controller: KafkaController) extends Logging { - val producerRequestPurgatory = new ProducerRequestPurgatory(replicaManager, offsetManager, requestChannel) - val fetchRequestPurgatory = new FetchRequestPurgatory(replicaManager, requestChannel) - // TODO: the following line will be removed in 0.9 - replicaManager.initWithRequestPurgatory(producerRequestPurgatory, fetchRequestPurgatory) - var metadataCache = new MetadataCache this.logIdent = "[KafkaApi-%d] ".format(brokerId) + val metadataCache = new MetadataCache /** * Top-level method that handles all requests and multiplexes to the right api @@ -56,7 +51,7 @@ class KafkaApis(val requestChannel: RequestChannel, try{ trace("Handling request: " + request.requestObj + " from client: " + request.remoteAddress) request.requestId match { - case RequestKeys.ProduceKey => handleProducerOrOffsetCommitRequest(request) + case RequestKeys.ProduceKey => handleProducerRequest(request) case RequestKeys.FetchKey => handleFetchRequest(request) case RequestKeys.OffsetsKey => handleOffsetRequest(request) case RequestKeys.MetadataKey => handleTopicMetadataRequest(request) @@ -64,7 +59,7 @@ class KafkaApis(val requestChannel: RequestChannel, case RequestKeys.StopReplicaKey => handleStopReplicaRequest(request) case RequestKeys.UpdateMetadataKey => handleUpdateMetadataRequest(request) case RequestKeys.ControlledShutdownKey => handleControlledShutdownRequest(request) - case RequestKeys.OffsetCommitKey => handleProducerOrOffsetCommitRequest(request) + case RequestKeys.OffsetCommitKey => handleOffsetCommitRequest(request) case RequestKeys.OffsetFetchKey => handleOffsetFetchRequest(request) case RequestKeys.ConsumerMetadataKey => handleConsumerMetadataRequest(request) case requestId => throw new KafkaException("Unknown api code " + requestId) @@ -123,179 +118,87 @@ class KafkaApis(val requestChannel: RequestChannel, requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(controlledShutdownResponse))) } - private def producerRequestFromOffsetCommit(offsetCommitRequest: OffsetCommitRequest) = { - val msgs = offsetCommitRequest.filterLargeMetadata(config.offsetMetadataMaxSize).map { - case (topicAndPartition, offset) => - new Message( - bytes = OffsetManager.offsetCommitValue(offset), - key = OffsetManager.offsetCommitKey(offsetCommitRequest.groupId, topicAndPartition.topic, topicAndPartition.partition) - ) - }.toSeq - - val producerData = mutable.Map( - TopicAndPartition(OffsetManager.OffsetsTopicName, offsetManager.partitionFor(offsetCommitRequest.groupId)) -> - new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, msgs:_*) - ) - - val request = ProducerRequest( - correlationId = offsetCommitRequest.correlationId, - clientId = offsetCommitRequest.clientId, - requiredAcks = config.offsetCommitRequiredAcks, - ackTimeoutMs = config.offsetCommitTimeoutMs, - data = producerData) - trace("Created producer request %s for offset commit request %s.".format(request, offsetCommitRequest)) - request - } /** - * Handle a produce request or offset commit request (which is really a specialized producer request) + * Handle an offset commit request */ - def handleProducerOrOffsetCommitRequest(request: RequestChannel.Request) { - val (produceRequest, offsetCommitRequestOpt) = - if (request.requestId == RequestKeys.OffsetCommitKey) { - val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest] - (producerRequestFromOffsetCommit(offsetCommitRequest), Some(offsetCommitRequest)) - } else { - (request.requestObj.asInstanceOf[ProducerRequest], None) - } - - val sTime = SystemTime.milliseconds - val localProduceResults = appendToLocalLog(produceRequest, offsetCommitRequestOpt.nonEmpty) - debug("Produce to local log in %d ms".format(SystemTime.milliseconds - sTime)) - - val firstErrorCode = localProduceResults.find(_.errorCode != ErrorMapping.NoError).map(_.errorCode).getOrElse(ErrorMapping.NoError) - - val numPartitionsInError = localProduceResults.count(_.error.isDefined) - if(produceRequest.requiredAcks == 0) { - // no operation needed if producer request.required.acks = 0; however, if there is any exception in handling the request, since - // no response is expected by the producer the handler will send a close connection response to the socket server - // to close the socket so that the producer client will know that some exception has happened and will refresh its metadata - if (numPartitionsInError != 0) { - info(("Send the close connection response due to error handling produce request " + - "[clientId = %s, correlationId = %s, topicAndPartition = %s] with Ack=0") - .format(produceRequest.clientId, produceRequest.correlationId, produceRequest.topicPartitionMessageSizeMap.keySet.mkString(","))) - requestChannel.closeConnection(request.processor, request) - } else { - - if (firstErrorCode == ErrorMapping.NoError) - offsetCommitRequestOpt.foreach(ocr => offsetManager.putOffsets(ocr.groupId, ocr.requestInfo)) - - if (offsetCommitRequestOpt.isDefined) { - val response = offsetCommitRequestOpt.get.responseFor(firstErrorCode, config.offsetMetadataMaxSize) - requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) - } else - requestChannel.noOperation(request.processor, request) - } - } else if (produceRequest.requiredAcks == 1 || - produceRequest.numPartitions <= 0 || - numPartitionsInError == produceRequest.numPartitions) { - - if (firstErrorCode == ErrorMapping.NoError) { - offsetCommitRequestOpt.foreach(ocr => offsetManager.putOffsets(ocr.groupId, ocr.requestInfo) ) + def handleOffsetCommitRequest(request: RequestChannel.Request) { + val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest] + + // the callback for sending the response + def sendResponseCallback(commitStatus: immutable.Map[TopicAndPartition, Short]) { + commitStatus.foreach { case (topicAndPartition, errorCode) => + // we only print warnings for known errors here; only replica manager could see an unknown + // exception while trying to write the offset message to the local log, and it will log + // an error message and write the error code in this case; hence it can be ignored here + if (errorCode != ErrorMapping.NoError && errorCode != ErrorMapping.UnknownCode) { + debug("Offset commit request with correlation id %d from client %s on partition %s failed due to %s" + .format(offsetCommitRequest.correlationId, offsetCommitRequest.clientId, + topicAndPartition, ErrorMapping.exceptionNameFor(errorCode))) + } } - val statuses = localProduceResults.map(r => r.key -> ProducerResponseStatus(r.errorCode, r.start)).toMap - val response = offsetCommitRequestOpt.map(_.responseFor(firstErrorCode, config.offsetMetadataMaxSize)) - .getOrElse(ProducerResponse(produceRequest.correlationId, statuses)) - + val response = OffsetCommitResponse(commitStatus, offsetCommitRequest.correlationId) requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) - } else { - // create a list of (topic, partition) pairs to use as keys for this delayed request - val producerRequestKeys = produceRequest.data.keys.map( - topicAndPartition => new TopicPartitionRequestKey(topicAndPartition)).toSeq - val statuses = localProduceResults.map(r => - r.key -> DelayedProduceResponseStatus(r.end + 1, ProducerResponseStatus(r.errorCode, r.start))).toMap - val delayedRequest = new DelayedProduce( - producerRequestKeys, - request, - produceRequest.ackTimeoutMs.toLong, - produceRequest, - statuses, - offsetCommitRequestOpt) - - // add the produce request for watch if it's not satisfied, otherwise send the response back - val satisfiedByMe = producerRequestPurgatory.checkAndMaybeWatch(delayedRequest) - if (satisfiedByMe) - producerRequestPurgatory.respond(delayedRequest) } - // we do not need the data anymore - produceRequest.emptyData() - } - - case class ProduceResult(key: TopicAndPartition, start: Long, end: Long, error: Option[Throwable] = None) { - def this(key: TopicAndPartition, throwable: Throwable) = - this(key, -1L, -1L, Some(throwable)) - - def errorCode = error match { - case None => ErrorMapping.NoError - case Some(error) => ErrorMapping.codeFor(error.getClass.asInstanceOf[Class[Throwable]]) - } + // call offset manager to store offsets + offsetManager.storeOffsets( + offsetCommitRequest.groupId, + offsetCommitRequest.consumerId, + offsetCommitRequest.groupGenerationId, + offsetCommitRequest.requestInfo, + sendResponseCallback) } /** - * Helper method for handling a parsed producer request + * Handle a produce request */ - private def appendToLocalLog(producerRequest: ProducerRequest, isOffsetCommit: Boolean): Iterable[ProduceResult] = { - val partitionAndData: Map[TopicAndPartition, MessageSet] = producerRequest.data - trace("Append [%s] to local log ".format(partitionAndData.toString)) - partitionAndData.map {case (topicAndPartition, messages) => - try { - if (Topic.InternalTopics.contains(topicAndPartition.topic) && - !(isOffsetCommit && topicAndPartition.topic == OffsetManager.OffsetsTopicName)) { - throw new InvalidTopicException("Cannot append to internal topic %s".format(topicAndPartition.topic)) - } - val partitionOpt = replicaManager.getPartition(topicAndPartition.topic, topicAndPartition.partition) - val info = partitionOpt match { - case Some(partition) => - partition.appendMessagesToLeader(messages.asInstanceOf[ByteBufferMessageSet],producerRequest.requiredAcks) - case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d" - .format(topicAndPartition, brokerId)) + def handleProducerRequest(request: RequestChannel.Request) { + val produceRequest = request.requestObj.asInstanceOf[ProducerRequest] + + // the callback for sending the response + def sendResponseCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { + var errorInResponse = false + responseStatus.foreach { case (topicAndPartition, status) => + // we only print warnings for known errors here; if it is unknown, it will cause + // an error message in the replica manager + if (status.error != ErrorMapping.NoError && status.error != ErrorMapping.UnknownCode) { + debug("Produce request with correlation id %d from client %s on partition %s failed due to %s" + .format(produceRequest.correlationId, produceRequest.clientId, + topicAndPartition, ErrorMapping.exceptionNameFor(status.error))) + errorInResponse = true } + } - val numAppendedMessages = if (info.firstOffset == -1L || info.lastOffset == -1L) 0 else (info.lastOffset - info.firstOffset + 1) + if (produceRequest.requiredAcks == 0) { + // no operation needed if producer request.required.acks = 0; however, if there is any error in handling + // the request, since no response is expected by the producer, the server will close socket server so that + // the producer client will know that some error has happened and will refresh its metadata + if (errorInResponse) { + info("Close connection due to error handling produce request with correlation id %d from client id %s with ack=0" + .format(produceRequest.correlationId, produceRequest.clientId)) + requestChannel.closeConnection(request.processor, request) + } else { + requestChannel.noOperation(request.processor, request) + } + } else { + val response = ProducerResponse(produceRequest.correlationId, responseStatus) + requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + } + } - // update stats for successfully appended bytes and messages as bytesInRate and messageInRate - BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesInRate.mark(messages.sizeInBytes) - BrokerTopicStats.getBrokerAllTopicsStats.bytesInRate.mark(messages.sizeInBytes) - BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).messagesInRate.mark(numAppendedMessages) - BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(numAppendedMessages) + // call the replica manager to append messages to the replicas + replicaManager.appendMessages( + produceRequest.ackTimeoutMs.toLong, + produceRequest.requiredAcks, + produceRequest.data, + sendResponseCallback) - trace("%d bytes written to log %s-%d beginning at offset %d and ending at offset %d" - .format(messages.size, topicAndPartition.topic, topicAndPartition.partition, info.firstOffset, info.lastOffset)) - ProduceResult(topicAndPartition, info.firstOffset, info.lastOffset) - } catch { - // NOTE: Failed produce requests is not incremented for UnknownTopicOrPartitionException and NotLeaderForPartitionException - // since failed produce requests metric is supposed to indicate failure of a broker in handling a produce request - // for a partition it is the leader for - case e: KafkaStorageException => - fatal("Halting due to unrecoverable I/O error while handling produce request: ", e) - Runtime.getRuntime.halt(1) - null - case ite: InvalidTopicException => - warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( - producerRequest.correlationId, producerRequest.clientId, topicAndPartition, ite.getMessage)) - new ProduceResult(topicAndPartition, ite) - case utpe: UnknownTopicOrPartitionException => - warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( - producerRequest.correlationId, producerRequest.clientId, topicAndPartition, utpe.getMessage)) - new ProduceResult(topicAndPartition, utpe) - case nle: NotLeaderForPartitionException => - warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( - producerRequest.correlationId, producerRequest.clientId, topicAndPartition, nle.getMessage)) - new ProduceResult(topicAndPartition, nle) - case nere: NotEnoughReplicasException => - warn("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( - producerRequest.correlationId, producerRequest.clientId, topicAndPartition, nere.getMessage)) - new ProduceResult(topicAndPartition, nere) - case e: Throwable => - BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).failedProduceRequestRate.mark() - BrokerTopicStats.getBrokerAllTopicsStats.failedProduceRequestRate.mark() - error("Error processing ProducerRequest with correlation id %d from client %s on partition %s" - .format(producerRequest.correlationId, producerRequest.clientId, topicAndPartition), e) - new ProduceResult(topicAndPartition, e) - } - } + // if the request is put into the purgatory, it will have a held reference + // and hence cannot be garbage collected; hence we clear its data here in + // order to let GC re-claim its memory since it is already appended to log + produceRequest.emptyData() } /** @@ -303,59 +206,38 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleFetchRequest(request: RequestChannel.Request) { val fetchRequest = request.requestObj.asInstanceOf[FetchRequest] - val dataRead = replicaManager.readMessageSets(fetchRequest) - - // if the fetch request comes from the follower, - // update its corresponding log end offset - if(fetchRequest.isFromFollower) - recordFollowerLogEndOffsets(fetchRequest.replicaId, dataRead.mapValues(_.offset)) - - // check if this fetch request can be satisfied right away - val bytesReadable = dataRead.values.map(_.data.messages.sizeInBytes).sum - val errorReadingData = dataRead.values.foldLeft(false)((errorIncurred, dataAndOffset) => - errorIncurred || (dataAndOffset.data.error != ErrorMapping.NoError)) - // send the data 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(fetchRequest.maxWait <= 0 || - fetchRequest.numPartitions <= 0 || - bytesReadable >= fetchRequest.minBytes || - errorReadingData) { - debug("Returning fetch response %s for fetch request with correlation id %d to client %s" - .format(dataRead.values.map(_.data.error).mkString(","), fetchRequest.correlationId, fetchRequest.clientId)) - val response = new FetchResponse(fetchRequest.correlationId, dataRead.mapValues(_.data)) - requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(response))) - } else { - debug("Putting fetch request with correlation id %d from client %s into purgatory".format(fetchRequest.correlationId, - fetchRequest.clientId)) - // create a list of (topic, partition) pairs to use as keys for this delayed request - val delayedFetchKeys = fetchRequest.requestInfo.keys.toSeq.map(new TopicPartitionRequestKey(_)) - val delayedFetch = new DelayedFetch(delayedFetchKeys, request, fetchRequest.maxWait, fetchRequest, - dataRead.mapValues(_.offset)) - - // add the fetch request for watch if it's not satisfied, otherwise send the response back - val satisfiedByMe = fetchRequestPurgatory.checkAndMaybeWatch(delayedFetch) - if (satisfiedByMe) - fetchRequestPurgatory.respond(delayedFetch) - } - } - private def recordFollowerLogEndOffsets(replicaId: Int, offsets: Map[TopicAndPartition, LogOffsetMetadata]) { - debug("Record follower log end offsets: %s ".format(offsets)) - offsets.foreach { - case (topicAndPartition, offset) => - replicaManager.updateReplicaLEOAndPartitionHW(topicAndPartition.topic, - topicAndPartition.partition, replicaId, offset) + // the callback for sending the response + def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData]) { + responsePartitionData.foreach { case (topicAndPartition, data) => + // we only print warnings for known errors here; if it is unknown, it will cause + // an error message in the replica manager already and hence can be ignored here + if (data.error != ErrorMapping.NoError && data.error != ErrorMapping.UnknownCode) { + debug("Fetch request with correlation id %d from client %s on partition %s failed due to %s" + .format(fetchRequest.correlationId, fetchRequest.clientId, + topicAndPartition, ErrorMapping.exceptionNameFor(data.error))) + } + + // record the bytes out metrics only when the response is being sent + BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesOutRate.mark(data.messages.sizeInBytes) + BrokerTopicStats.getBrokerAllTopicsStats().bytesOutRate.mark(data.messages.sizeInBytes) + } - // for producer requests with ack > 1, we need to check - // if they can be unblocked after some follower's log end offsets have moved - replicaManager.unblockDelayedProduceRequests(new TopicPartitionRequestKey(topicAndPartition)) + val response = FetchResponse(fetchRequest.correlationId, responsePartitionData) + requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(response))) } + + // call the replica manager to fetch messages from the local replica + replicaManager.fetchMessages( + fetchRequest.maxWait.toLong, + fetchRequest.replicaId, + fetchRequest.minBytes, + fetchRequest.requestInfo, + sendResponseCallback) } /** - * Service the offset request API + * Handle an offset request */ def handleOffsetRequest(request: RequestChannel.Request) { val offsetRequest = request.requestObj.asInstanceOf[OffsetRequest] @@ -387,15 +269,15 @@ class KafkaApis(val requestChannel: RequestChannel, // NOTE: UnknownTopicOrPartitionException and NotLeaderForPartitionException are special cased since these error messages // are typically transient and there is no value in logging the entire stack trace for the same case utpe: UnknownTopicOrPartitionException => - warn("Offset request with correlation id %d from client %s on partition %s failed due to %s".format( + debug("Offset request with correlation id %d from client %s on partition %s failed due to %s".format( offsetRequest.correlationId, offsetRequest.clientId, topicAndPartition, utpe.getMessage)) (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(utpe.getClass.asInstanceOf[Class[Throwable]]), Nil) ) case nle: NotLeaderForPartitionException => - warn("Offset request with correlation id %d from client %s on partition %s failed due to %s".format( + debug("Offset request with correlation id %d from client %s on partition %s failed due to %s".format( offsetRequest.correlationId, offsetRequest.clientId, topicAndPartition,nle.getMessage)) (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(nle.getClass.asInstanceOf[Class[Throwable]]), Nil) ) case e: Throwable => - warn("Error while responding to offset request", e) + error("Error while responding to offset request", e) (topicAndPartition, PartitionOffsetsResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), Nil) ) } }) @@ -415,7 +297,7 @@ class KafkaApis(val requestChannel: RequestChannel, } } - def fetchOffsetsBefore(log: Log, timestamp: Long, maxNumOffsets: Int): Seq[Long] = { + private def fetchOffsetsBefore(log: Log, timestamp: Long, maxNumOffsets: Int): Seq[Long] = { val segsArray = log.logSegments.toArray var offsetTimeArray: Array[(Long, Long)] = null if(segsArray.last.size > 0) @@ -488,7 +370,7 @@ class KafkaApis(val requestChannel: RequestChannel, } /** - * Service the topic metadata request API + * Handle a topic metadata request */ def handleTopicMetadataRequest(request: RequestChannel.Request) { val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest] @@ -500,7 +382,7 @@ class KafkaApis(val requestChannel: RequestChannel, } /* - * Service the Offset fetch API + * Handle an offset fetch request */ def handleOffsetFetchRequest(request: RequestChannel.Request) { val offsetFetchRequest = request.requestObj.asInstanceOf[OffsetFetchRequest] @@ -520,7 +402,7 @@ class KafkaApis(val requestChannel: RequestChannel, } /* - * Service the consumer metadata API + * Handle a consumer metadata request */ def handleConsumerMetadataRequest(request: RequestChannel.Request) { val consumerMetadataRequest = request.requestObj.asInstanceOf[ConsumerMetadataRequest] @@ -545,9 +427,8 @@ class KafkaApis(val requestChannel: RequestChannel, } def close() { - debug("Shutting down.") - fetchRequestPurgatory.shutdown() - producerRequestPurgatory.shutdown() + // TODO currently closing the API is an no-op since the API no longer maintain any modules + // maybe removing the closing call in the end when KafkaAPI becomes a pure stateless layer debug("Shut down complete.") } } diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 43eb2a35bb54d..2957bc435102b 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -40,6 +40,7 @@ import java.util.concurrent.TimeUnit import com.yammer.metrics.core.Gauge import org.I0Itec.zkclient.ZkClient +import kafka.api.ProducerResponseStatus /** @@ -144,6 +145,8 @@ class OffsetManager(val config: OffsetManagerConfig, trace("Marked %d offsets in %s for deletion.".format(messages.size, appendPartition)) try { + // do not need to require acks since even if the tombsone is lost, + // it will be appended again in the next purge cycle partition.appendMessagesToLeader(new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*)) tombstones.size } @@ -192,13 +195,91 @@ class OffsetManager(val config: OffsetManagerConfig, offsetsCache.put(key, offsetAndMetadata) } - def putOffsets(group: String, offsets: Map[TopicAndPartition, OffsetAndMetadata]) { - // this method is called _after_ the offsets have been durably appended to the commit log, so there is no need to - // check for current leadership as we do for the offset fetch - trace("Putting offsets %s for group %s in offsets partition %d.".format(offsets, group, partitionFor(group))) - offsets.foreach { case (topicAndPartition, offsetAndMetadata) => - putOffset(GroupTopicPartition(group, topicAndPartition), offsetAndMetadata) + /* + * Check if the offset metadata length is valid + */ + def validateOffsetMetadataLength(metadata: String) : Boolean = { + metadata == null || metadata.length() <= config.maxMetadataSize + } + + /** + * Store offsets by appending it to the replicated log and then inserting to cache + */ + // TODO: generation id and consumer id is needed by coordinator to do consumer checking in the future + def storeOffsets(groupName: String, + consumerId: String, + generationId: Int, + offsetMetadata: immutable.Map[TopicAndPartition, OffsetAndMetadata], + responseCallback: immutable.Map[TopicAndPartition, Short] => Unit) { + + // first filter out partitions with offset metadata size exceeding limit + // TODO: in the future we may want to only support atomic commit and hence fail the whole commit + val filteredOffsetMetadata = offsetMetadata.filter { case (topicAndPartition, offsetAndMetadata) => + validateOffsetMetadataLength(offsetAndMetadata.metadata) } + + // construct the message set to append + val messages = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => + new Message( + key = OffsetManager.offsetCommitKey(groupName, topicAndPartition.topic, topicAndPartition.partition), + bytes = OffsetManager.offsetCommitValue(offsetAndMetadata) + ) + }.toSeq + + val offsetTopicPartition = TopicAndPartition(OffsetManager.OffsetsTopicName, partitionFor(groupName)) + + val offsetsAndMetadataMessageSet = Map(offsetTopicPartition -> + new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*)) + + // set the callback function to insert offsets into cache after log append completed + def putCacheCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { + // the append response should only contain the topics partition + if (responseStatus.size != 1 || ! responseStatus.contains(offsetTopicPartition)) + throw new IllegalStateException("Append status %s should only have one partition %s" + .format(responseStatus, offsetTopicPartition)) + + // construct the commit response status and insert + // the offset and metadata to cache iff the append status has no error + val status = responseStatus(offsetTopicPartition) + + val responseCode = + if (status.error == ErrorMapping.NoError) { + filteredOffsetMetadata.foreach { case (topicAndPartition, offsetAndMetadata) => + putOffset(GroupTopicPartition(groupName, topicAndPartition), offsetAndMetadata) + } + ErrorMapping.NoError + } else { + debug("Offset commit %s from group %s consumer %s with generation %d failed when appending to log due to %s" + .format(filteredOffsetMetadata, groupName, consumerId, generationId, ErrorMapping.exceptionNameFor(status.error))) + + // transform the log append error code to the corresponding the commit status error code + if (status.error == ErrorMapping.UnknownTopicOrPartitionCode) + ErrorMapping.ConsumerCoordinatorNotAvailableCode + else if (status.error == ErrorMapping.NotLeaderForPartitionCode) + ErrorMapping.NotCoordinatorForConsumerCode + else + status.error + } + + + // compute the final error codes for the commit response + val commitStatus = offsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => + if (validateOffsetMetadataLength(offsetAndMetadata.metadata)) + (topicAndPartition, responseCode) + else + (topicAndPartition, ErrorMapping.OffsetMetadataTooLargeCode) + } + + // finally trigger the callback logic passed from the API layer + responseCallback(commitStatus) + } + + // call replica manager to append the offset messages + replicaManager.appendMessages( + config.offsetCommitTimeoutMs.toLong, + config.offsetCommitRequiredAcks, + offsetsAndMetadataMessageSet, + putCacheCallback) } /** diff --git a/core/src/main/scala/kafka/server/ProducerRequestPurgatory.scala b/core/src/main/scala/kafka/server/ProducerRequestPurgatory.scala deleted file mode 100644 index d4a7d4a79b442..0000000000000 --- a/core/src/main/scala/kafka/server/ProducerRequestPurgatory.scala +++ /dev/null @@ -1,69 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.server - -import kafka.metrics.KafkaMetricsGroup -import kafka.utils.Pool -import kafka.network.{BoundedByteBufferSend, RequestChannel} - -import java.util.concurrent.TimeUnit - -/** - * The purgatory holding delayed producer requests - */ -class ProducerRequestPurgatory(replicaManager: ReplicaManager, offsetManager: OffsetManager, requestChannel: RequestChannel) - extends RequestPurgatory[DelayedProduce](replicaManager.config.brokerId, replicaManager.config.producerPurgatoryPurgeIntervalRequests) { - this.logIdent = "[ProducerRequestPurgatory-%d] ".format(replicaManager.config.brokerId) - - private class DelayedProducerRequestMetrics(keyLabel: String = DelayedRequestKey.globalLabel) extends KafkaMetricsGroup { - val expiredRequestMeter = newMeter(keyLabel + "ExpiresPerSecond", "requests", TimeUnit.SECONDS) - } - - private val producerRequestMetricsForKey = { - val valueFactory = (k: DelayedRequestKey) => new DelayedProducerRequestMetrics(k.keyLabel + "-") - new Pool[DelayedRequestKey, DelayedProducerRequestMetrics](Some(valueFactory)) - } - - private val aggregateProduceRequestMetrics = new DelayedProducerRequestMetrics - - private def recordDelayedProducerKeyExpired(key: DelayedRequestKey) { - val keyMetrics = producerRequestMetricsForKey.getAndMaybePut(key) - List(keyMetrics, aggregateProduceRequestMetrics).foreach(_.expiredRequestMeter.mark()) - } - - /** - * Check if a specified delayed fetch request is satisfied - */ - def checkSatisfied(delayedProduce: DelayedProduce) = delayedProduce.isSatisfied(replicaManager) - - /** - * When a delayed produce request expires answer it with possible time out error codes - */ - def expire(delayedProduce: DelayedProduce) { - debug("Expiring produce request %s.".format(delayedProduce.produce)) - for ((topicPartition, responseStatus) <- delayedProduce.partitionStatus if responseStatus.acksPending) - recordDelayedProducerKeyExpired(new TopicPartitionRequestKey(topicPartition)) - respond(delayedProduce) - } - - // TODO: purgatory should not be responsible for sending back the responses - def respond(delayedProduce: DelayedProduce) { - val response = delayedProduce.respond(offsetManager) - requestChannel.sendResponse(new RequestChannel.Response(delayedProduce.request, new BoundedByteBufferSend(response))) - } -} diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 78b7514cc1095..02fa3821271e9 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -20,11 +20,11 @@ import kafka.api._ import kafka.common._ import kafka.utils._ import kafka.cluster.{Broker, Partition, Replica} -import kafka.log.LogManager +import kafka.log.{LogAppendInfo, LogManager} import kafka.metrics.KafkaMetricsGroup import kafka.controller.KafkaController import kafka.common.TopicAndPartition -import kafka.message.MessageSet +import kafka.message.{ByteBufferMessageSet, MessageSet} import java.util.concurrent.atomic.AtomicBoolean import java.io.{IOException, File} @@ -39,14 +39,31 @@ import scala.Some import org.I0Itec.zkclient.ZkClient import com.yammer.metrics.core.Gauge -object ReplicaManager { - val HighWatermarkFilename = "replication-offset-checkpoint" +/* + * Result metadata of a log append operation on the log + */ +case class LogAppendResult(info: LogAppendInfo, error: Option[Throwable] = None) { + def errorCode = error match { + case None => ErrorMapping.NoError + case Some(e) => ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) + } } -case class PartitionDataAndOffset(data: FetchResponsePartitionData, offset: LogOffsetMetadata) +/* + * Result metadata of a log read operation on the log + */ +case class LogReadResult(info: FetchDataInfo, hw: Long, readSize: Int, error: Option[Throwable] = None) { + def errorCode = error match { + case None => ErrorMapping.NoError + case Some(e) => ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]) + } +} +object ReplicaManager { + val HighWatermarkFilename = "replication-offset-checkpoint" +} -class ReplicaManager(val config: KafkaConfig, +class ReplicaManager(val config: KafkaConfig, time: Time, val zkClient: ZkClient, scheduler: Scheduler, @@ -64,8 +81,9 @@ class ReplicaManager(val config: KafkaConfig, this.logIdent = "[Replica Manager on Broker " + localBrokerId + "]: " val stateChangeLogger = KafkaController.stateChangeLogger - var producerRequestPurgatory: ProducerRequestPurgatory = null - var fetchRequestPurgatory: FetchRequestPurgatory = null + val producerRequestPurgatory = new RequestPurgatory[DelayedProduce](config.brokerId, config.producerPurgatoryPurgeIntervalRequests) + val fetchRequestPurgatory = new RequestPurgatory[DelayedFetch](config.brokerId, config.fetchPurgatoryPurgeIntervalRequests) + newGauge( "LeaderCount", @@ -100,37 +118,27 @@ class ReplicaManager(val config: KafkaConfig, } /** - * Initialize the replica manager with the request purgatory + * Try to complete some delayed produce requests with the request key; + * this can be triggered when: * - * TODO: will be removed in 0.9 where we refactor server structure - */ - - def initWithRequestPurgatory(producerRequestPurgatory: ProducerRequestPurgatory, fetchRequestPurgatory: FetchRequestPurgatory) { - this.producerRequestPurgatory = producerRequestPurgatory - this.fetchRequestPurgatory = fetchRequestPurgatory - } - - /** - * Unblock some delayed produce requests with the request key + * 1. The partition HW has changed (for acks = -1) + * 2. A follower replica's fetch operation is received (for acks > 1) */ - def unblockDelayedProduceRequests(key: DelayedRequestKey) { - val satisfied = producerRequestPurgatory.update(key) - debug("Request key %s unblocked %d producer requests." - .format(key.keyLabel, satisfied.size)) - - // send any newly unblocked responses - satisfied.foreach(producerRequestPurgatory.respond(_)) + def tryCompleteDelayedProduce(key: DelayedRequestKey) { + val completed = producerRequestPurgatory.checkAndComplete(key) + debug("Request key %s unblocked %d producer requests.".format(key.keyLabel, completed)) } /** - * Unblock some delayed fetch requests with the request key + * Try to complete some delayed fetch requests with the request key; + * this can be triggered when: + * + * 1. The partition HW has changed (for regular fetch) + * 2. A new message set is appended to the local log (for follower fetch) */ - def unblockDelayedFetchRequests(key: DelayedRequestKey) { - val satisfied = fetchRequestPurgatory.update(key) - debug("Request key %s unblocked %d fetch requests.".format(key.keyLabel, satisfied.size)) - - // send any newly unblocked responses - satisfied.foreach(fetchRequestPurgatory.respond(_)) + def tryCompleteDelayedFetch(key: DelayedRequestKey) { + val completed = fetchRequestPurgatory.checkAndComplete(key) + debug("Request key %s unblocked %d fetch requests.".format(key.keyLabel, completed)) } def startup() { @@ -237,74 +245,205 @@ class ReplicaManager(val config: KafkaConfig, } /** - * Read from all the offset details given and return a map of - * (topic, partition) -> PartitionData + * Append messages to leader replicas of the partition, and wait for them to be replicated to other replicas; + * the callback function will be triggered either when timeout or the required acks are satisfied */ - def readMessageSets(fetchRequest: FetchRequest) = { - val isFetchFromFollower = fetchRequest.isFromFollower - fetchRequest.requestInfo.map - { - case (TopicAndPartition(topic, partition), PartitionFetchInfo(offset, fetchSize)) => - val partitionDataAndOffsetInfo = - try { - val (fetchInfo, highWatermark) = readMessageSet(topic, partition, offset, fetchSize, fetchRequest.replicaId) - BrokerTopicStats.getBrokerTopicStats(topic).bytesOutRate.mark(fetchInfo.messageSet.sizeInBytes) - BrokerTopicStats.getBrokerAllTopicsStats.bytesOutRate.mark(fetchInfo.messageSet.sizeInBytes) - if (isFetchFromFollower) { - debug("Partition [%s,%d] received fetch request from follower %d" - .format(topic, partition, fetchRequest.replicaId)) - } - new PartitionDataAndOffset(new FetchResponsePartitionData(ErrorMapping.NoError, highWatermark, fetchInfo.messageSet), fetchInfo.fetchOffset) - } catch { - // NOTE: Failed fetch requests is not incremented for UnknownTopicOrPartitionException and NotLeaderForPartitionException - // since failed fetch requests metric is supposed to indicate failure of a broker in handling a fetch request - // for a partition it is the leader for - case utpe: UnknownTopicOrPartitionException => - warn("Fetch request with correlation id %d from client %s on partition [%s,%d] failed due to %s".format( - fetchRequest.correlationId, fetchRequest.clientId, topic, partition, utpe.getMessage)) - new PartitionDataAndOffset(new FetchResponsePartitionData(ErrorMapping.codeFor(utpe.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty), LogOffsetMetadata.UnknownOffsetMetadata) - case nle: NotLeaderForPartitionException => - warn("Fetch request with correlation id %d from client %s on partition [%s,%d] failed due to %s".format( - fetchRequest.correlationId, fetchRequest.clientId, topic, partition, nle.getMessage)) - new PartitionDataAndOffset(new FetchResponsePartitionData(ErrorMapping.codeFor(nle.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty), LogOffsetMetadata.UnknownOffsetMetadata) - case t: Throwable => - BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark() - BrokerTopicStats.getBrokerAllTopicsStats.failedFetchRequestRate.mark() - error("Error when processing fetch request for partition [%s,%d] offset %d from %s with correlation id %d. Possible cause: %s" - .format(topic, partition, offset, if (isFetchFromFollower) "follower" else "consumer", fetchRequest.correlationId, t.getMessage)) - new PartitionDataAndOffset(new FetchResponsePartitionData(ErrorMapping.codeFor(t.getClass.asInstanceOf[Class[Throwable]]), -1L, MessageSet.Empty), LogOffsetMetadata.UnknownOffsetMetadata) - } - (TopicAndPartition(topic, partition), partitionDataAndOffsetInfo) + def appendMessages(timeout: Long, + requiredAcks: Short, + messagesPerPartition: Map[TopicAndPartition, MessageSet], + responseCallback: Map[TopicAndPartition, ProducerResponseStatus] => Unit) { + + val sTime = SystemTime.milliseconds + val localProduceResults = appendToLocalLog(messagesPerPartition, requiredAcks) + debug("Produce to local log in %d ms".format(SystemTime.milliseconds - sTime)) + + val produceStatus = localProduceResults.map{ case (topicAndPartition, result) => + topicAndPartition -> + ProducePartitionStatus( + result.info.lastOffset + 1, // required offset + ProducerResponseStatus(result.errorCode, result.info.firstOffset)) // response status + } + + if(requiredAcks == 0 || + requiredAcks == 1 || + messagesPerPartition.size <= 0 || + localProduceResults.values.count(_.error.isDefined) == messagesPerPartition.size) { + // in case of the following we can respond immediately: + // + // 1. required acks = 0 or 1 + // 2. there is no data to append + // 3. all partition appends have failed + val produceResponseStatus = produceStatus.mapValues(status => status.responseStatus) + responseCallback(produceResponseStatus) + } else { + // create delayed produce operation + val produceMetadata = ProduceMetadata(requiredAcks, produceStatus) + val delayedProduce = new DelayedProduce(timeout, produceMetadata, this, responseCallback) + + // create a list of (topic, partition) pairs to use as keys for this delayed request + val producerRequestKeys = messagesPerPartition.keys.map(new TopicPartitionRequestKey(_)).toSeq + + // try to complete the request immediately, otherwise put it into the purgatory + // this is because while the delayed request is being created, new requests may + // arrive which can make this request completable. + producerRequestPurgatory.tryCompleteElseWatch(delayedProduce, producerRequestKeys) + } + } + + /** + * Append the messages to the local replica logs + */ + private def appendToLocalLog(messagesPerPartition: Map[TopicAndPartition, MessageSet], + requiredAcks: Short): Map[TopicAndPartition, LogAppendResult] = { + trace("Append [%s] to local log ".format(messagesPerPartition)) + messagesPerPartition.map { case (topicAndPartition, messages) => + try { + val partitionOpt = getPartition(topicAndPartition.topic, topicAndPartition.partition) + val info = partitionOpt match { + case Some(partition) => + partition.appendMessagesToLeader(messages.asInstanceOf[ByteBufferMessageSet], requiredAcks) + case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d" + .format(topicAndPartition, localBrokerId)) + } + + val numAppendedMessages = + if (info.firstOffset == -1L || info.lastOffset == -1L) + 0 + else + info.lastOffset - info.firstOffset + 1 + + // update stats for successfully appended bytes and messages as bytesInRate and messageInRate + BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesInRate.mark(messages.sizeInBytes) + BrokerTopicStats.getBrokerAllTopicsStats.bytesInRate.mark(messages.sizeInBytes) + BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).messagesInRate.mark(numAppendedMessages) + BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(numAppendedMessages) + + trace("%d bytes written to log %s-%d beginning at offset %d and ending at offset %d" + .format(messages.size, topicAndPartition.topic, topicAndPartition.partition, info.firstOffset, info.lastOffset)) + (topicAndPartition, LogAppendResult(info)) + } catch { + // NOTE: Failed produce requests metric is not incremented for known exceptions + // it is supposed to indicate un-expected failures of a broker in handling a produce request + case e: KafkaStorageException => + fatal("Halting due to unrecoverable I/O error while handling produce request: ", e) + Runtime.getRuntime.halt(1) + (topicAndPartition, null) + case utpe: UnknownTopicOrPartitionException => + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(utpe))) + case nle: NotLeaderForPartitionException => + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(nle))) + case e: Throwable => + BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).failedProduceRequestRate.mark() + BrokerTopicStats.getBrokerAllTopicsStats.failedProduceRequestRate.mark() + error("Error processing append operation on partition %s".format(topicAndPartition), e) + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(e))) + } + } + } + + /** + * Fetch messages from the leader replica, and wait until enough data can be fetched and return; + * the callback function will be triggered either when timeout or required fetch info is satisfied + */ + def fetchMessages(timeout: Long, + replicaId: Int, + fetchMinBytes: Int, + fetchInfo: Map[TopicAndPartition, PartitionFetchInfo], + responseCallback: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) { + + val fetchOnlyFromLeader: Boolean = replicaId != Request.DebuggingConsumerId + val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(replicaId) + + // read from local logs + val logReadResults = readFromLocalLog(fetchOnlyFromLeader, fetchOnlyCommitted, fetchInfo) + + // if the fetch comes from the follower, + // update its corresponding log end offset + if(Request.isValidBrokerId(replicaId)) + updateFollowerLEOs(replicaId, logReadResults.mapValues(_.info.fetchOffset)) + + // 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) => + errorIncurred || (readResult.errorCode != ErrorMapping.NoError)) + + // 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)) + responseCallback(fetchPartitionData) + } else { + // construct the fetch results from the read results + val fetchPartitionStatus = logReadResults.map { case (topicAndPartition, result) => + (topicAndPartition, FetchPartitionStatus(result.info.fetchOffset, fetchInfo.get(topicAndPartition).get)) + } + val fetchMetadata = FetchMetadata(fetchMinBytes, fetchOnlyFromLeader, fetchOnlyCommitted, fetchPartitionStatus) + val delayedFetch = new DelayedFetch(timeout, fetchMetadata, this, responseCallback) + + // create a list of (topic, partition) pairs to use as keys for this delayed request + val delayedFetchKeys = fetchPartitionStatus.keys.map(new TopicPartitionRequestKey(_)).toSeq + + // try to complete the request immediately, otherwise put it into the purgatory; + // this is because while the delayed request is being created, new requests may + // arrive which can make this request completable. + fetchRequestPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys) } } /** * Read from a single topic/partition at the given offset upto maxSize bytes */ - private def readMessageSet(topic: String, - partition: Int, - offset: Long, - maxSize: Int, - fromReplicaId: Int): (FetchDataInfo, Long) = { - // check if the current broker is the leader for the partitions - val localReplica = if(fromReplicaId == Request.DebuggingConsumerId) - getReplicaOrException(topic, partition) - else - getLeaderReplicaIfLocal(topic, partition) - trace("Fetching log segment for topic, partition, offset, size = " + (topic, partition, offset, maxSize)) - val maxOffsetOpt = - if (Request.isValidBrokerId(fromReplicaId)) - None - else - Some(localReplica.highWatermark.messageOffset) - val fetchInfo = localReplica.log match { - case Some(log) => - log.read(offset, maxSize, maxOffsetOpt) - case None => - error("Leader for partition [%s,%d] does not have a local log".format(topic, partition)) - FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty) + def readFromLocalLog(fetchOnlyFromLeader: Boolean, + readOnlyCommitted: Boolean, + readPartitionInfo: Map[TopicAndPartition, PartitionFetchInfo]): Map[TopicAndPartition, LogReadResult] = { + + readPartitionInfo.map { case (TopicAndPartition(topic, partition), PartitionFetchInfo(offset, fetchSize)) => + val partitionDataAndOffsetInfo = + try { + trace("Fetching log segment for topic %s, partition %d, offset %d, size %d".format(topic, partition, offset, fetchSize)) + + // 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 on log + val logReadInfo = localReplica.log match { + case Some(log) => + log.read(offset, fetchSize, maxOffsetOpt) + case None => + error("Leader for partition [%s,%d] does not have a local log".format(topic, partition)) + FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty) + } + + LogReadResult(logReadInfo, localReplica.highWatermark.messageOffset, fetchSize, 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, Some(utpe)) + case nle: NotLeaderForPartitionException => + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, Some(nle)) + case rnae: ReplicaNotAvailableException => + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, Some(rnae)) + 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)) + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, Some(e)) + } + (TopicAndPartition(topic, partition), partitionDataAndOffsetInfo) } - (fetchInfo, localReplica.highWatermark.messageOffset) } def maybeUpdateMetadataCache(updateMetadataRequest: UpdateMetadataRequest, metadataCache: MetadataCache) { @@ -557,32 +696,27 @@ class ReplicaManager(val config: KafkaConfig, allPartitions.values.foreach(partition => partition.maybeShrinkIsr(config.replicaLagTimeMaxMs, config.replicaLagMaxMessages)) } - def updateReplicaLEOAndPartitionHW(topic: String, partitionId: Int, replicaId: Int, offset: LogOffsetMetadata) = { - getPartition(topic, partitionId) match { - case Some(partition) => - partition.getReplica(replicaId) match { - case Some(replica) => - replica.logEndOffset = offset - // check if we need to update HW and expand Isr - partition.updateLeaderHWAndMaybeExpandIsr(replicaId) - debug("Recorded follower %d position %d for partition [%s,%d].".format(replicaId, offset.messageOffset, topic, partitionId)) - case None => - throw new NotAssignedReplicaException(("Leader %d failed to record follower %d's position %d since the replica" + - " is not recognized to be one of the assigned replicas %s for partition [%s,%d]").format(localBrokerId, replicaId, - offset.messageOffset, partition.assignedReplicas().map(_.brokerId).mkString(","), topic, partitionId)) - - } - case None => - warn("While recording the follower position, the partition [%s,%d] hasn't been created, skip updating leader HW".format(topic, partitionId)) + private def updateFollowerLEOs(replicaId: Int, offsets: Map[TopicAndPartition, LogOffsetMetadata]) { + debug("Recording follower broker %d log end offsets: %s ".format(replicaId, offsets)) + offsets.foreach { case (topicAndPartition, offset) => + getPartition(topicAndPartition.topic, topicAndPartition.partition) match { + case Some(partition) => + partition.updateReplicaLEO(replicaId, offset) + + // for producer requests with ack > 1, we need to check + // if they can be unblocked after some follower's log end offsets have moved + tryCompleteDelayedProduce(new TopicPartitionRequestKey(topicAndPartition)) + case None => + warn("While recording the replica LEO, the partition %s hasn't been created.".format(topicAndPartition)) + } } } private def getLeaderPartitions() : List[Partition] = { allPartitions.values.filter(_.leaderReplicaIfLocal().isDefined).toList } - /** - * Flushes the highwatermark value for all partitions to the highwatermark file - */ + + // Flushes the highwatermark value for all partitions to the highwatermark file def checkpointHighWatermarks() { val replicas = allPartitions.values.map(_.getReplica(config.brokerId)).collect{case Some(replica) => replica} val replicasByDir = replicas.filter(_.log.isDefined).groupBy(_.log.get.dir.getParentFile.getAbsolutePath) @@ -598,10 +732,14 @@ class ReplicaManager(val config: KafkaConfig, } } - def shutdown() { - info("Shut down") + // High watermark do not need to be checkpointed only when under unit tests + def shutdown(checkpointHW: Boolean = true) { + info("Shutting down") replicaFetcherManager.shutdown() - checkpointHighWatermarks() + fetchRequestPurgatory.shutdown() + producerRequestPurgatory.shutdown() + if (checkpointHW) + checkpointHighWatermarks() info("Shut down completely") } } diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/RequestPurgatory.scala index 9d76234bc2c81..323b12e765f98 100644 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ b/core/src/main/scala/kafka/server/RequestPurgatory.scala @@ -17,7 +17,6 @@ package kafka.server -import kafka.network._ import kafka.utils._ import kafka.metrics.KafkaMetricsGroup @@ -30,50 +29,75 @@ import com.yammer.metrics.core.Gauge /** - * A request whose processing needs to be delayed for at most the given delayMs - * The associated keys are used for bookeeping, and represent the "trigger" that causes this request to check if it is satisfied, - * for example a key could be a (topic, partition) pair. + * An operation whose processing needs to be delayed for at most the given delayMs. For example + * a delayed produce operation could be waiting for specified number of acks; or + * a delayed fetch operation could be waiting for a given number of bytes to accumulate. + * + * The logic upon completing a delayed operation is defined in onComplete() and will be called exactly once. + * Once an operation is completed, isCompleted() will return true. onComplete() can be triggered by either + * forceComplete(), which forces calling onComplete() after delayMs if the operation is not yet completed, + * or tryComplete(), which first checks if the operation can be completed or not now, and if yes calls + * forceComplete(). + * + * A subclass of DelayedRequest needs to provide an implementation of both onComplete() and tryComplete(). */ -class DelayedRequest(val keys: Seq[Any], val request: RequestChannel.Request, delayMs: Long) extends DelayedItem[RequestChannel.Request](request, delayMs) { - val satisfied = new AtomicBoolean(false) +abstract class DelayedRequest(delayMs: Long) extends DelayedItem(delayMs) { + private val completed = new AtomicBoolean(false) + + /* + * Force completing the delayed operation, if not already completed. + * This function can be triggered when + * + * 1. The operation has been verified to be completable inside tryComplete() + * 2. The operation has expired and hence needs to be completed right now + * + * Return true iff the operation is completed by the caller + */ + def forceComplete(): Boolean = { + if (completed.compareAndSet(false, true)) { + onComplete() + true + } else { + false + } + } + + /** + * Check if the delayed operation is already completed + */ + def isCompleted(): Boolean = completed.get() + + /** + * Process for completing an operation; This function needs to be defined in subclasses + * and will be called exactly once in forceComplete() + */ + def onComplete(): Unit + + /* + * Try to complete the delayed operation by first checking if the operation + * can be completed by now. If yes execute the completion logic by calling + * forceComplete() and return true iff forceComplete returns true; otherwise return false + * + * Note that concurrent threads can check if an operation can be completed or not, + * but only the first thread will succeed in completing the operation and return + * true, others will still return false + * + * this function needs to be defined in subclasses + */ + def tryComplete(): Boolean } /** - * A helper class for dealing with asynchronous requests with a timeout. A DelayedRequest has a request to delay - * and also a list of keys that can trigger the action. Implementations can add customized logic to control what it means for a given - * request to be satisfied. For example it could be that we are waiting for user-specified number of acks on a given (topic, partition) - * to be able to respond to a request or it could be that we are waiting for a given number of bytes to accumulate on a given request - * to be able to respond to that request (in the simple case we might wait for at least one byte to avoid busy waiting). - * - * For us the key is generally a (topic, partition) pair. - * By calling - * val isSatisfiedByMe = checkAndMaybeWatch(delayedRequest) - * we will check if a request is satisfied already, and if not add the request for watch on all its keys. - * - * It is up to the user to then call - * val satisfied = update(key, request) - * when a request relevant to the given key occurs. This triggers bookeeping logic and returns back any requests satisfied by this - * new request. - * - * An implementation provides extends two helper functions - * def checkSatisfied(request: R, delayed: T): Boolean - * this function returns true if the given request (in combination with whatever previous requests have happened) satisfies the delayed - * request delayed. This method will likely also need to do whatever bookkeeping is necessary. - * - * The second function is - * def expire(delayed: T) - * this function handles delayed requests that have hit their time limit without being satisfied. - * + * A helper purgatory class for bookkeeping delayed operations with a timeout, and expiring timed out operations. */ -abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: Int = 1000) +class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: Int = 1000) extends Logging with KafkaMetricsGroup { /* a list of requests watching each key */ private val watchersForKey = new Pool[Any, Watchers](Some((key: Any) => new Watchers)) /* background thread expiring requests that have been waiting too long */ - private val expiredRequestReaper = new ExpiredRequestReaper - private val expirationThread = Utils.newThread(name="request-expiration-task", runnable=expiredRequestReaper, daemon=false) + private val expirationReaper = new ExpiredOperationReaper newGauge( "PurgatorySize", @@ -89,230 +113,182 @@ abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInt } ) - expirationThread.start() + expirationReaper.start() /** - * Try to add the request for watch on all keys. Return true iff the request is - * satisfied and the satisfaction is done by the caller. + * Check if the operation can be completed, if not watch it based on the given watch keys + * + * Note that a delayed operation can be watched on multiple keys. It is possible that + * an operation is completed after it has been added to the watch list for some, but + * not all of the keys. In this case, the operation is considered completed and won't + * be added to the watch list of the remaining keys. The expiration reaper thread will + * remove this operation from any watcher list in which the operation exists. * - * Requests can be watched on only a few of the keys if it is found satisfied when - * trying to add it to each one of the keys. In this case the request is still treated as satisfied - * and hence no longer watched. Those already added elements will be later purged by the expire reaper. + * @param operation the delayed operation to be checked + * @param watchKeys keys for bookkeeping the operation + * @return true iff the delayed operations can be completed by the caller */ - def checkAndMaybeWatch(delayedRequest: T): Boolean = { - for(key <- delayedRequest.keys) { - val lst = watchersFor(key) - if(!lst.checkAndMaybeAdd(delayedRequest)) { - if(delayedRequest.satisfied.compareAndSet(false, true)) - return true - else - return false + def tryCompleteElseWatch(operation: T, watchKeys: Seq[Any]): Boolean = { + for(key <- watchKeys) { + // if the operation is already completed, stopping adding it to + // any further lists and return false + if (operation.isCompleted()) + return false + val watchers = watchersFor(key) + // if the operation can by completed by myself, stop adding it to + // any further lists and return true immediately + if(operation synchronized operation.tryComplete()) { + return true + } else { + watchers.watch(operation) } } - // if it is indeed watched, add to the expire queue also - expiredRequestReaper.enqueue(delayedRequest) + // if it cannot be completed by now and hence is watched, add to the expire queue also + if (! operation.isCompleted()) { + expirationReaper.enqueue(operation) + } false } /** - * Update any watchers and return a list of newly satisfied requests. + * Check if some some delayed requests can be completed with the given watch key, + * and if yes complete them. + * + * @return the number of completed requests during this process */ - def update(key: Any): Seq[T] = { - val w = watchersForKey.get(key) - if(w == null) - Seq.empty + def checkAndComplete(key: Any): Int = { + val watchers = watchersForKey.get(key) + if(watchers == null) + 0 else - w.collectSatisfiedRequests() + watchers.tryCompleteWatched() } - /* - * Return the size of the watched lists in the purgatory, which is the size of watch lists. - * Since an operation may still be in the watch lists even when it has been completed, - * this number may be larger than the number of real operations watched + /** + * Return the total size of watch lists the purgatory. Since an operation may be watched + * on multiple lists, and some of its watched entries may still be in the watch lists + * even when it has been completed, this number may be larger than the number of real operations watched */ def watched() = watchersForKey.values.map(_.watched).sum - /* - * Return the number of requests in the expiry reaper's queue + /** + * Return the number of delayed operations in the expiry queue */ - def delayed() = expiredRequestReaper.delayed() + def delayed() = expirationReaper.delayed /* - * Return the watch list for the given watch key + * Return the watch list of the given key */ private def watchersFor(key: Any) = watchersForKey.getAndMaybePut(key) - - /** - * Check if this delayed request is already satisfied - */ - protected def checkSatisfied(request: T): Boolean - - /** - * Handle an expired delayed request - */ - protected def expire(delayed: T) /** * Shutdown the expire reaper thread */ def shutdown() { - expiredRequestReaper.shutdown() + expirationReaper.shutdown() } /** - * A linked list of DelayedRequests watching some key with some associated - * bookkeeping logic. + * A linked list of watched delayed operations based on some key */ private class Watchers { private val requests = new util.LinkedList[T] - // return the size of the watch list - def watched() = requests.size() + def watched = requests.size() - // potentially add the element to watch if it is not satisfied yet - def checkAndMaybeAdd(t: T): Boolean = { + // add the element to watch + def watch(t: T) { synchronized { - // if it is already satisfied, do not add to the watch list - if (t.satisfied.get) - return false - // synchronize on the delayed request to avoid any race condition - // with expire and update threads on client-side. - if(t synchronized checkSatisfied(t)) { - return false - } requests.add(t) - return true } } - // traverse the list and purge satisfied elements - def purgeSatisfied(): Int = { + // traverse the list and try to complete some watched elements + def tryCompleteWatched(): Int = { + var completed = 0 synchronized { val iter = requests.iterator() - var purged = 0 while(iter.hasNext) { val curr = iter.next - if(curr.satisfied.get()) { + if (curr.isCompleted()) { + // another thread has completed this request, just remove it iter.remove() - purged += 1 + } else { + if(curr synchronized curr.tryComplete()) { + iter.remove() + completed += 1 + } } } - purged } + completed } - // traverse the list and try to satisfy watched elements - def collectSatisfiedRequests(): Seq[T] = { - val response = new mutable.ArrayBuffer[T] + // traverse the list and purge elements that are already completed by others + def purgeCompleted(): Int = { + var purged = 0 synchronized { val iter = requests.iterator() - while(iter.hasNext) { + while (iter.hasNext) { val curr = iter.next - if(curr.satisfied.get) { - // another thread has satisfied this request, remove it + if(curr.isCompleted()) { iter.remove() - } else { - // synchronize on curr to avoid any race condition with expire - // on client-side. - val satisfied = curr synchronized checkSatisfied(curr) - if(satisfied) { - iter.remove() - val updated = curr.satisfied.compareAndSet(false, true) - if(updated == true) { - response += curr - } - } + purged += 1 } } } - response + purged } } /** - * Runnable to expire requests that have sat unfullfilled past their deadline + * A background reaper to expire delayed operations that have timed out */ - private class ExpiredRequestReaper extends Runnable with Logging { - this.logIdent = "ExpiredRequestReaper-%d ".format(brokerId) - private val running = new AtomicBoolean(true) - private val shutdownLatch = new CountDownLatch(1) + private class ExpiredOperationReaper extends ShutdownableThread( + "ExpirationReaper-%d".format(brokerId), + false) { + /* The queue storing all delayed operations */ private val delayedQueue = new DelayQueue[T] + /* + * Return the number of delayed operations kept by the reaper + */ def delayed() = delayedQueue.size() - - /** Main loop for the expiry thread */ - def run() { - while(running.get) { - try { - val curr = pollExpired() - if (curr != null) { - curr synchronized { - expire(curr) - } - } - // see if we need to purge the watch lists - if (RequestPurgatory.this.watched() >= purgeInterval) { - debug("Begin purging watch lists") - val numPurgedFromWatchers = watchersForKey.values.map(_.purgeSatisfied()).sum - debug("Purged %d elements from watch lists.".format(numPurgedFromWatchers)) - } - // see if we need to purge the delayed request queue - if (delayed() >= purgeInterval) { - debug("Begin purging delayed queue") - val purged = purgeSatisfied() - debug("Purged %d requests from delayed queue.".format(purged)) - } - } catch { - case e: Exception => - error("Error in long poll expiry thread: ", e) - } - } - shutdownLatch.countDown() - } - /** Add a request to be expired */ + /* + * Add an operation to be expired + */ def enqueue(t: T) { delayedQueue.add(t) } - /** Shutdown the expiry thread*/ - def shutdown() { - debug("Shutting down.") - running.set(false) - shutdownLatch.await() - debug("Shut down complete.") - } - /** - * Get the next expired event + * Try to get the next expired event and force completing it */ - private def pollExpired(): T = { - while(true) { - val curr = delayedQueue.poll(200L, TimeUnit.MILLISECONDS) - if (curr == null) - return null.asInstanceOf[T] - val updated = curr.satisfied.compareAndSet(false, true) - if(updated) { - return curr + private def expireNext() { + val curr = delayedQueue.poll(200L, TimeUnit.MILLISECONDS) + if (curr != null.asInstanceOf[T]) { + // if there is an expired operation, try to force complete it + if (curr synchronized curr.forceComplete()) { + debug("Force complete expired delayed operation %s".format(curr)) } } - throw new RuntimeException("This should not happen") } /** * Delete all satisfied events from the delay queue and the watcher lists */ - private def purgeSatisfied(): Int = { + private def purgeCompleted(): Int = { var purged = 0 // purge the delayed queue val iter = delayedQueue.iterator() - while(iter.hasNext) { + while (iter.hasNext) { val curr = iter.next() - if(curr.satisfied.get) { + if (curr.isCompleted()) { iter.remove() purged += 1 } @@ -320,6 +296,22 @@ abstract class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInt purged } - } + override def doWork() { + // try to get the next expired operation and force completing it + expireNext() + // see if we need to purge the watch lists + if (RequestPurgatory.this.watched() >= purgeInterval) { + debug("Begin purging watch lists") + val purged = watchersForKey.values.map(_.purgeCompleted()).sum + debug("Purged %d elements from watch lists.".format(purged)) + } + // see if we need to purge the delayed request queue + if (delayed() >= purgeInterval) { + debug("Begin purging delayed queue") + val purged = purgeCompleted() + debug("Purged %d operations from delayed queue.".format(purged)) + } + } + } } diff --git a/core/src/main/scala/kafka/utils/DelayedItem.scala b/core/src/main/scala/kafka/utils/DelayedItem.scala index d7276494072f1..a4e0dabc858bc 100644 --- a/core/src/main/scala/kafka/utils/DelayedItem.scala +++ b/core/src/main/scala/kafka/utils/DelayedItem.scala @@ -20,7 +20,7 @@ package kafka.utils import java.util.concurrent._ import scala.math._ -class DelayedItem[T](val item: T, delay: Long, unit: TimeUnit) extends Delayed with Logging { +class DelayedItem(delay: Long, unit: TimeUnit) extends Delayed with Logging { val createdMs = SystemTime.milliseconds val delayMs = { @@ -29,8 +29,8 @@ class DelayedItem[T](val item: T, delay: Long, unit: TimeUnit) extends Delayed w else given } - def this(item: T, delayMs: Long) = - this(item, delayMs, TimeUnit.MILLISECONDS) + def this(delayMs: Long) = + this(delayMs, TimeUnit.MILLISECONDS) /** * The remaining delay time @@ -41,7 +41,7 @@ class DelayedItem[T](val item: T, delay: Long, unit: TimeUnit) extends Delayed w } def compareTo(d: Delayed): Int = { - val delayed = d.asInstanceOf[DelayedItem[T]] + val delayed = d.asInstanceOf[DelayedItem] val myEnd = createdMs + delayMs val yourEnd = delayed.createdMs + delayed.delayMs diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index 209a409cb47eb..8531f533f3a64 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -305,13 +305,11 @@ class ProducerFailureHandlingTest extends JUnit3Suite with KafkaServerTestHarnes @Test def testNotEnoughReplicas() { val topicName = "minisrtest" - val topicProps = new Properties(); - topicProps.put("min.insync.replicas","3"); - + val topicProps = new Properties() + topicProps.put("min.insync.replicas","3") TestUtils.createTopic(zkClient, topicName, 1, 2, servers,topicProps) - val record = new ProducerRecord(topicName, null, "key".getBytes, "value".getBytes) try { producer3.send(record).get @@ -327,18 +325,16 @@ class ProducerFailureHandlingTest extends JUnit3Suite with KafkaServerTestHarnes @Test def testNotEnoughReplicasAfterBrokerShutdown() { val topicName = "minisrtest2" - val topicProps = new Properties(); - topicProps.put("min.insync.replicas","2"); - + val topicProps = new Properties() + topicProps.put("min.insync.replicas","2") TestUtils.createTopic(zkClient, topicName, 1, 2, servers,topicProps) - val record = new ProducerRecord(topicName, null, "key".getBytes, "value".getBytes) - // This should work + // this should work with all brokers up and running producer3.send(record).get - //shut down one broker + // shut down one broker servers.head.shutdown() servers.head.awaitShutdown() try { @@ -351,8 +347,8 @@ class ProducerFailureHandlingTest extends JUnit3Suite with KafkaServerTestHarnes } } + // restart the server servers.head.startup() - } private class ProducerScheduler extends ShutdownableThread("daemon-producer", false) diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala index fb61d552f2320..d60d8e0f49443 100644 --- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala @@ -237,8 +237,8 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { props.put("request.required.acks", "-1") val producer = new SyncProducer(new SyncProducerConfig(props)) - val topicProps = new Properties(); - topicProps.put("min.insync.replicas","2"); + val topicProps = new Properties() + topicProps.put("min.insync.replicas","2") AdminUtils.createTopic(zkClient, topicName, 1, 1,topicProps) TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topicName, 0) diff --git a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala index 03a424d45215e..8913fc1d59f71 100644 --- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala +++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala @@ -74,6 +74,9 @@ class HighwatermarkPersistenceTest extends JUnit3Suite { fooPartition0Hw = hwmFor(replicaManager, topic, 0) assertEquals(leaderReplicaPartition0.highWatermark.messageOffset, fooPartition0Hw) EasyMock.verify(zkClient) + + // shutdown the replica manager upon test completion + replicaManager.shutdown(false) } def testHighWatermarkPersistenceMultiplePartitions() { @@ -130,6 +133,10 @@ class HighwatermarkPersistenceTest extends JUnit3Suite { topic1Partition0Hw = hwmFor(replicaManager, topic1, 0) assertEquals(10L, topic1Partition0Hw) EasyMock.verify(zkClient) + + // shutdown the replica manager upon test completion + replicaManager.shutdown(false) + } def hwmFor(replicaManager: ReplicaManager, topic: String, partition: Int): Long = { diff --git a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala index cd302aa51eb83..a703d2715048c 100644 --- a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala +++ b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala @@ -36,8 +36,21 @@ class IsrExpirationTest extends JUnit3Suite { }) val topic = "foo" + val time = new MockTime + + var replicaManager: ReplicaManager = null + + override def setUp() { + super.setUp() + replicaManager = new ReplicaManager(configs.head, time, null, null, null, new AtomicBoolean(false)) + } + + override def tearDown() { + replicaManager.shutdown(false) + super.tearDown() + } + def testIsrExpirationForStuckFollowers() { - val time = new MockTime val log = getLogWithLogEndOffset(15L, 2) // set logEndOffset for leader to 15L // create one partition and all replicas @@ -61,7 +74,6 @@ class IsrExpirationTest extends JUnit3Suite { } def testIsrExpirationForSlowFollowers() { - val time = new MockTime // create leader replica val log = getLogWithLogEndOffset(15L, 1) // add one partition @@ -82,7 +94,6 @@ class IsrExpirationTest extends JUnit3Suite { private def getPartitionWithAllReplicasInIsr(topic: String, partitionId: Int, time: Time, config: KafkaConfig, localLog: Log): Partition = { val leaderId=config.brokerId - val replicaManager = new ReplicaManager(config, time, null, null, null, new AtomicBoolean(false)) val partition = replicaManager.getOrCreatePartition(topic, partitionId) val leaderReplica = new Replica(leaderId, partition, time, 0, Some(localLog)) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index a9c4ddc78df0b..faa907131ed0a 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -42,6 +42,9 @@ class ReplicaManagerTest extends JUnit3Suite { val partition = rm.getOrCreatePartition(topic, 1) partition.getOrCreateReplica(1) rm.checkpointHighWatermarks() + + // shutdown the replica manager upon test completion + rm.shutdown(false) } @Test @@ -56,5 +59,8 @@ class ReplicaManagerTest extends JUnit3Suite { val partition = rm.getOrCreatePartition(topic, 1) partition.getOrCreateReplica(1) rm.checkpointHighWatermarks() + + // shutdown the replica manager upon test completion + rm.shutdown(false) } } diff --git a/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala b/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala index a577f4a8bf420..a7720d579ea15 100644 --- a/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala @@ -17,24 +17,18 @@ package kafka.server -import scala.collection._ import org.junit.Test +import org.scalatest.junit.JUnit3Suite import junit.framework.Assert._ -import kafka.message._ -import kafka.api._ import kafka.utils.TestUtils -import org.scalatest.junit.JUnit3Suite - class RequestPurgatoryTest extends JUnit3Suite { - val producerRequest1 = TestUtils.produceRequest("test", 0, new ByteBufferMessageSet(new Message("hello1".getBytes))) - val producerRequest2 = TestUtils.produceRequest("test", 0, new ByteBufferMessageSet(new Message("hello2".getBytes))) - var purgatory: MockRequestPurgatory = null + var purgatory: RequestPurgatory[MockDelayedRequest] = null override def setUp() { super.setUp() - purgatory = new MockRequestPurgatory(5) + purgatory = new RequestPurgatory[MockDelayedRequest](0, 5) } override def tearDown() { @@ -44,58 +38,59 @@ class RequestPurgatoryTest extends JUnit3Suite { @Test def testRequestSatisfaction() { - val r1 = new DelayedRequest(Array("test1"), null, 100000L) - val r2 = new DelayedRequest(Array("test2"), null, 100000L) - assertEquals("With no waiting requests, nothing should be satisfied", 0, purgatory.update("test1").size) - assertFalse("r1 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r1)) - assertEquals("Still nothing satisfied", 0, purgatory.update("test1").size) - assertFalse("r2 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r2)) - assertEquals("Still nothing satisfied", 0, purgatory.update("test2").size) - purgatory.satisfied += r1 - assertEquals("r1 satisfied", mutable.ArrayBuffer(r1), purgatory.update("test1")) - assertEquals("Nothing satisfied", 0, purgatory.update("test1").size) - purgatory.satisfied += r2 - assertEquals("r2 satisfied", mutable.ArrayBuffer(r2), purgatory.update("test2")) - assertEquals("Nothing satisfied", 0, purgatory.update("test2").size) + val r1 = new MockDelayedRequest(100000L) + val r2 = new MockDelayedRequest(100000L) + assertEquals("With no waiting requests, nothing should be satisfied", 0, purgatory.checkAndComplete("test1")) + assertFalse("r1 not satisfied and hence watched", purgatory.tryCompleteElseWatch(r1, Array("test1"))) + assertEquals("Still nothing satisfied", 0, purgatory.checkAndComplete("test1")) + assertFalse("r2 not satisfied and hence watched", purgatory.tryCompleteElseWatch(r2, Array("test2"))) + assertEquals("Still nothing satisfied", 0, purgatory.checkAndComplete("test2")) + r1.completable = true + assertEquals("r1 satisfied", 1, purgatory.checkAndComplete("test1")) + assertEquals("Nothing satisfied", 0, purgatory.checkAndComplete("test1")) + r2.completable = true + assertEquals("r2 satisfied", 1, purgatory.checkAndComplete("test2")) + assertEquals("Nothing satisfied", 0, purgatory.checkAndComplete("test2")) } @Test def testRequestExpiry() { val expiration = 20L - val r1 = new DelayedRequest(Array("test1"), null, expiration) - val r2 = new DelayedRequest(Array("test1"), null, 200000L) + val r1 = new MockDelayedRequest(expiration) + val r2 = new MockDelayedRequest(200000L) val start = System.currentTimeMillis - assertFalse("r1 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r1)) - assertFalse("r2 not satisfied and hence watched", purgatory.checkAndMaybeWatch(r2)) - purgatory.awaitExpiration(r1) + assertFalse("r1 not satisfied and hence watched", purgatory.tryCompleteElseWatch(r1, Array("test1"))) + assertFalse("r2 not satisfied and hence watched", purgatory.tryCompleteElseWatch(r2, Array("test2"))) + r1.awaitExpiration() val elapsed = System.currentTimeMillis - start - assertTrue("r1 expired", purgatory.expired.contains(r1)) - assertTrue("r2 hasn't expired", !purgatory.expired.contains(r2)) + assertTrue("r1 completed due to expiration", r1.isCompleted()) + assertFalse("r2 hasn't completed", r2.isCompleted()) assertTrue("Time for expiration %d should at least %d".format(elapsed, expiration), elapsed >= expiration) } @Test def testRequestPurge() { - val r1 = new DelayedRequest(Array("test1"), null, 100000L) - val r12 = new DelayedRequest(Array("test1", "test2"), null, 100000L) - val r23 = new DelayedRequest(Array("test2", "test3"), null, 100000L) - purgatory.checkAndMaybeWatch(r1) - purgatory.checkAndMaybeWatch(r12) - purgatory.checkAndMaybeWatch(r23) + val r1 = new MockDelayedRequest(100000L) + val r2 = new MockDelayedRequest(100000L) + purgatory.tryCompleteElseWatch(r1, Array("test1")) + purgatory.tryCompleteElseWatch(r2, Array("test1", "test2")) + purgatory.tryCompleteElseWatch(r1, Array("test2", "test3")) assertEquals("Purgatory should have 5 watched elements", 5, purgatory.watched()) assertEquals("Purgatory should have 3 total delayed requests", 3, purgatory.delayed()) - // satisfy one of the requests, it should then be purged from the watch list with purge interval 5 - r12.satisfied.set(true) + // complete one of the operations, it should + // eventually be purged from the watch list with purge interval 5 + r2.completable = true + r2.tryComplete() TestUtils.waitUntilTrue(() => purgatory.watched() == 3, - "Purgatory should have 3 watched elements instead of " + + purgatory.watched(), 1000L) + "Purgatory should have 3 watched elements instead of " + purgatory.watched(), 1000L) TestUtils.waitUntilTrue(() => purgatory.delayed() == 3, "Purgatory should still have 3 total delayed requests instead of " + purgatory.delayed(), 1000L) // add two more requests, then the satisfied request should be purged from the delayed queue with purge interval 5 - purgatory.checkAndMaybeWatch(r1) - purgatory.checkAndMaybeWatch(r1) + purgatory.tryCompleteElseWatch(r1, Array("test1")) + purgatory.tryCompleteElseWatch(r1, Array("test1")) TestUtils.waitUntilTrue(() => purgatory.watched() == 5, "Purgatory should have 5 watched elements instead of " + purgatory.watched(), 1000L) @@ -103,19 +98,25 @@ class RequestPurgatoryTest extends JUnit3Suite { "Purgatory should have 4 total delayed requests instead of " + purgatory.delayed(), 1000L) } - class MockRequestPurgatory(purge: Int) extends RequestPurgatory[DelayedRequest](purgeInterval = purge) { - val satisfied = mutable.Set[DelayedRequest]() - val expired = mutable.Set[DelayedRequest]() - def awaitExpiration(delayed: DelayedRequest) = { - delayed synchronized { - delayed.wait() + class MockDelayedRequest(delayMs: Long) extends DelayedRequest(delayMs) { + var completable = false + + def awaitExpiration() { + synchronized { + wait() } } - def checkSatisfied(delayed: DelayedRequest): Boolean = satisfied.contains(delayed) - def expire(delayed: DelayedRequest) { - expired += delayed - delayed synchronized { - delayed.notify() + + override def tryComplete() = { + if (completable) + forceComplete() + else + false + } + + override def onComplete() { + synchronized { + notify() } } } diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index 3804a114e97c8..1bfb501b2f29c 100644 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -106,7 +106,7 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { val newProps = TestUtils.createBrokerConfig(0, port) newProps.setProperty("delete.topic.enable", "true") val newConfig = new KafkaConfig(newProps) - var server = new KafkaServer(newConfig) + val server = new KafkaServer(newConfig) server.startup() server.shutdown() server.awaitShutdown() diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index 09ed8f5a7a414..ccf5e2e36260b 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -17,18 +17,21 @@ package kafka.server import kafka.api._ -import kafka.cluster.{Partition, Replica} -import kafka.common.{ErrorMapping, TopicAndPartition} +import kafka.utils._ +import kafka.cluster.Replica +import kafka.common.TopicAndPartition import kafka.log.Log import kafka.message.{ByteBufferMessageSet, Message} -import kafka.network.RequestChannel -import kafka.utils.{ZkUtils, Time, TestUtils, MockTime} import scala.Some +import java.util.Collections +import java.util.concurrent.atomic.AtomicBoolean +import collection.JavaConversions._ import org.easymock.EasyMock import org.I0Itec.zkclient.ZkClient import org.scalatest.junit.JUnit3Suite +import junit.framework.Assert._ class SimpleFetchTest extends JUnit3Suite { @@ -37,215 +40,102 @@ class SimpleFetchTest extends JUnit3Suite { override val replicaFetchWaitMaxMs = 100 override val replicaLagMaxMessages = 10L }) - val topic = "foo" - val partitionId = 0 - /** - * The scenario for this test is that there is one topic, "test-topic", one broker "0" that has - * one partition with one follower replica on broker "1". The leader replica on "0" - * has HW of "5" and LEO of "20". The follower on broker "1" has a local replica - * with a HW matching the leader's ("5") and LEO of "15", meaning it's not in-sync - * but is still in ISR (hasn't yet expired from ISR). - * - * When a normal consumer fetches data, it should only see data up to the HW of the leader, - * in this case up an offset of "5". - */ - def testNonReplicaSeesHwWhenFetching() { - /* setup */ - val time = new MockTime - val leo = 20L - val hw = 5 - val fetchSize = 100 - val messages = new Message("test-message".getBytes()) + // set the replica manager with the partition + val time = new MockTime + val leaderLEO = 20L + val followerLEO = 15L + val partitionHW = 5 - // create nice mock since we don't particularly care about zkclient calls - val zkClient = EasyMock.createNiceMock(classOf[ZkClient]) - EasyMock.expect(zkClient.exists(ZkUtils.ControllerEpochPath)).andReturn(false) - EasyMock.replay(zkClient) + val fetchSize = 100 + val messagesToHW = new Message("messageToHW".getBytes()) + val messagesToLEO = new Message("messageToLEO".getBytes()) - val log = EasyMock.createMock(classOf[kafka.log.Log]) - EasyMock.expect(log.logEndOffset).andReturn(leo).anyTimes() - EasyMock.expect(log) - EasyMock.expect(log.read(0, fetchSize, Some(hw))).andReturn( - new FetchDataInfo( - new LogOffsetMetadata(0L, 0L, leo.toInt), - new ByteBufferMessageSet(messages) - )).anyTimes() - EasyMock.replay(log) + val topic = "test-topic" + val partitionId = 0 + val topicAndPartition = TopicAndPartition(topic, partitionId) - val logManager = EasyMock.createMock(classOf[kafka.log.LogManager]) - EasyMock.expect(logManager.getLog(TopicAndPartition(topic, partitionId))).andReturn(Some(log)).anyTimes() - EasyMock.replay(logManager) + val fetchInfo = Collections.singletonMap(topicAndPartition, PartitionFetchInfo(0, fetchSize)).toMap - val replicaManager = EasyMock.createMock(classOf[kafka.server.ReplicaManager]) - EasyMock.expect(replicaManager.config).andReturn(configs.head) - EasyMock.expect(replicaManager.logManager).andReturn(logManager) - EasyMock.expect(replicaManager.replicaFetcherManager).andReturn(EasyMock.createMock(classOf[ReplicaFetcherManager])) - EasyMock.expect(replicaManager.zkClient).andReturn(zkClient) - EasyMock.expect(replicaManager.readMessageSets(EasyMock.anyObject())).andReturn({ - val fetchInfo = log.read(0, fetchSize, Some(hw)) - val partitionData = new FetchResponsePartitionData(ErrorMapping.NoError, hw.toLong, fetchInfo.messageSet) - Map(TopicAndPartition(topic, partitionId) -> new PartitionDataAndOffset(partitionData, fetchInfo.fetchOffset)) - }).anyTimes() - EasyMock.replay(replicaManager) - - val partition = getPartitionWithAllReplicasInISR(topic, partitionId, time, configs.head.brokerId, log, hw, replicaManager) - partition.getReplica(configs(1).brokerId).get.logEndOffset = new LogOffsetMetadata(leo - 5L, 0L, leo.toInt - 5) - - EasyMock.reset(replicaManager) - EasyMock.expect(replicaManager.config).andReturn(configs.head).anyTimes() - EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(topic, partitionId)).andReturn(partition.leaderReplicaIfLocal().get).anyTimes() - EasyMock.expect(replicaManager.initWithRequestPurgatory(EasyMock.anyObject(), EasyMock.anyObject())) - EasyMock.expect(replicaManager.readMessageSets(EasyMock.anyObject())).andReturn({ - val fetchInfo = log.read(0, fetchSize, Some(hw)) - val partitionData = new FetchResponsePartitionData(ErrorMapping.NoError, hw.toLong, fetchInfo.messageSet) - Map(TopicAndPartition(topic, partitionId) -> new PartitionDataAndOffset(partitionData, fetchInfo.fetchOffset)) - }).anyTimes() - - EasyMock.replay(replicaManager) - - val offsetManager = EasyMock.createMock(classOf[kafka.server.OffsetManager]) - - val controller = EasyMock.createMock(classOf[kafka.controller.KafkaController]) - - // start a request channel with 2 processors and a queue size of 5 (this is more or less arbitrary) - // don't provide replica or leader callbacks since they will not be tested here - val requestChannel = new RequestChannel(2, 5) - val apis = new KafkaApis(requestChannel, replicaManager, offsetManager, zkClient, configs.head.brokerId, configs.head, controller) - - val partitionStateInfo = EasyMock.createNiceMock(classOf[PartitionStateInfo]) - apis.metadataCache.addOrUpdatePartitionInfo(topic, partitionId, partitionStateInfo) - EasyMock.replay(partitionStateInfo) - // This request (from a follower) wants to read up to 2*HW but should only get back up to HW bytes into the log - val goodFetch = new FetchRequestBuilder() - .replicaId(Request.OrdinaryConsumerId) - .addFetch(topic, partitionId, 0, fetchSize) - .build() - val goodFetchBB = TestUtils.createRequestByteBuffer(goodFetch) - - // send the request - apis.handleFetchRequest(new RequestChannel.Request(processor=1, requestKey=5, buffer=goodFetchBB, startTimeMs=1)) - - // make sure the log only reads bytes between 0->HW (5) - EasyMock.verify(log) - } + var replicaManager: ReplicaManager = null - /** - * The scenario for this test is that there is one topic, "test-topic", on broker "0" that has - * one partition with one follower replica on broker "1". The leader replica on "0" - * has HW of "5" and LEO of "20". The follower on broker "1" has a local replica - * with a HW matching the leader's ("5") and LEO of "15", meaning it's not in-sync - * but is still in ISR (hasn't yet expired from ISR). - * - * When the follower from broker "1" fetches data, it should see data upto the log end offset ("20") - */ - def testReplicaSeesLeoWhenFetching() { - /* setup */ - val time = new MockTime - val leo = 20 - val hw = 5 - - val messages = new Message("test-message".getBytes()) - - val followerReplicaId = configs(1).brokerId - val followerLEO = 15 + override def setUp() { + super.setUp() + // create nice mock since we don't particularly care about zkclient calls val zkClient = EasyMock.createNiceMock(classOf[ZkClient]) - EasyMock.expect(zkClient.exists(ZkUtils.ControllerEpochPath)).andReturn(false) EasyMock.replay(zkClient) - val log = EasyMock.createMock(classOf[kafka.log.Log]) - EasyMock.expect(log.logEndOffset).andReturn(leo).anyTimes() - EasyMock.expect(log.read(followerLEO, Integer.MAX_VALUE, None)).andReturn( + // create nice mock since we don't particularly care about scheduler calls + val scheduler = EasyMock.createNiceMock(classOf[KafkaScheduler]) + EasyMock.replay(scheduler) + + // create the log which takes read with either HW max offset or none max offset + val log = EasyMock.createMock(classOf[Log]) + EasyMock.expect(log.logEndOffset).andReturn(leaderLEO).anyTimes() + EasyMock.expect(log.read(0, fetchSize, Some(partitionHW))).andReturn( new FetchDataInfo( - new LogOffsetMetadata(followerLEO, 0L, followerLEO), - new ByteBufferMessageSet(messages) + new LogOffsetMetadata(0L, 0L, 0), + new ByteBufferMessageSet(messagesToHW) + )).anyTimes() + EasyMock.expect(log.read(0, fetchSize, None)).andReturn( + new FetchDataInfo( + new LogOffsetMetadata(0L, 0L, 0), + new ByteBufferMessageSet(messagesToLEO) )).anyTimes() EasyMock.replay(log) + // create the log manager that is aware of this mock log val logManager = EasyMock.createMock(classOf[kafka.log.LogManager]) - EasyMock.expect(logManager.getLog(TopicAndPartition(topic, 0))).andReturn(Some(log)).anyTimes() + EasyMock.expect(logManager.getLog(topicAndPartition)).andReturn(Some(log)).anyTimes() EasyMock.replay(logManager) - val replicaManager = EasyMock.createMock(classOf[kafka.server.ReplicaManager]) - EasyMock.expect(replicaManager.config).andReturn(configs.head) - EasyMock.expect(replicaManager.logManager).andReturn(logManager) - EasyMock.expect(replicaManager.replicaFetcherManager).andReturn(EasyMock.createMock(classOf[ReplicaFetcherManager])) - EasyMock.expect(replicaManager.zkClient).andReturn(zkClient) - EasyMock.expect(replicaManager.readMessageSets(EasyMock.anyObject())).andReturn({ - val fetchInfo = log.read(followerLEO, Integer.MAX_VALUE, None) - val partitionData = new FetchResponsePartitionData(ErrorMapping.NoError, hw.toLong, fetchInfo.messageSet) - Map(TopicAndPartition(topic, partitionId) -> new PartitionDataAndOffset(partitionData, fetchInfo.fetchOffset)) - }).anyTimes() - EasyMock.replay(replicaManager) - - val partition = getPartitionWithAllReplicasInISR(topic, partitionId, time, configs.head.brokerId, log, hw, replicaManager) - partition.getReplica(followerReplicaId).get.logEndOffset = new LogOffsetMetadata(followerLEO.asInstanceOf[Long], 0L, followerLEO) - - EasyMock.reset(replicaManager) - EasyMock.expect(replicaManager.config).andReturn(configs.head).anyTimes() - EasyMock.expect(replicaManager.updateReplicaLEOAndPartitionHW(topic, partitionId, followerReplicaId, new LogOffsetMetadata(followerLEO.asInstanceOf[Long], 0L, followerLEO))) - EasyMock.expect(replicaManager.getReplica(topic, partitionId, followerReplicaId)).andReturn(partition.inSyncReplicas.find(_.brokerId == configs(1).brokerId)) - EasyMock.expect(replicaManager.getLeaderReplicaIfLocal(topic, partitionId)).andReturn(partition.leaderReplicaIfLocal().get).anyTimes() - EasyMock.expect(replicaManager.initWithRequestPurgatory(EasyMock.anyObject(), EasyMock.anyObject())) - EasyMock.expect(replicaManager.readMessageSets(EasyMock.anyObject())).andReturn({ - val fetchInfo = log.read(followerLEO, Integer.MAX_VALUE, None) - val partitionData = new FetchResponsePartitionData(ErrorMapping.NoError, hw.toLong, fetchInfo.messageSet) - Map(TopicAndPartition(topic, partitionId) -> new PartitionDataAndOffset(partitionData, fetchInfo.fetchOffset)) - }).anyTimes() - EasyMock.expect(replicaManager.unblockDelayedProduceRequests(EasyMock.anyObject())).anyTimes() - EasyMock.replay(replicaManager) - - val offsetManager = EasyMock.createMock(classOf[kafka.server.OffsetManager]) - - val controller = EasyMock.createMock(classOf[kafka.controller.KafkaController]) - - val requestChannel = new RequestChannel(2, 5) - val apis = new KafkaApis(requestChannel, replicaManager, offsetManager, zkClient, configs.head.brokerId, configs.head, controller) - val partitionStateInfo = EasyMock.createNiceMock(classOf[PartitionStateInfo]) - apis.metadataCache.addOrUpdatePartitionInfo(topic, partitionId, partitionStateInfo) - EasyMock.replay(partitionStateInfo) - - /** - * This fetch, coming from a replica, requests all data at offset "15". Because the request is coming - * from a follower, the leader should oblige and read beyond the HW. - */ - val bigFetch = new FetchRequestBuilder() - .replicaId(followerReplicaId) - .addFetch(topic, partitionId, followerLEO, Integer.MAX_VALUE) - .build() - - val fetchRequestBB = TestUtils.createRequestByteBuffer(bigFetch) - - // send the request - apis.handleFetchRequest(new RequestChannel.Request(processor=0, requestKey=5, buffer=fetchRequestBB, startTimeMs=1)) - - /** - * Make sure the log satisfies the fetch from a follower by reading data beyond the HW, mainly all bytes after - * an offset of 15 - */ - EasyMock.verify(log) - } + // create the replica manager + replicaManager = new ReplicaManager(configs.head, time, zkClient, scheduler, logManager, new AtomicBoolean(false)) + + // add the partition with two replicas, both in ISR + val partition = replicaManager.getOrCreatePartition(topic, partitionId) - private def getPartitionWithAllReplicasInISR(topic: String, partitionId: Int, time: Time, leaderId: Int, - localLog: Log, leaderHW: Long, replicaManager: ReplicaManager): Partition = { - val partition = new Partition(topic, partitionId, time, replicaManager) - val leaderReplica = new Replica(leaderId, partition, time, 0, Some(localLog)) + // create the leader replica with the local log + val leaderReplica = new Replica(configs(0).brokerId, partition, time, 0, Some(log)) + leaderReplica.highWatermark = new LogOffsetMetadata(partitionHW) + partition.leaderReplicaIdOpt = Some(leaderReplica.brokerId) - val allReplicas = getFollowerReplicas(partition, leaderId, time) :+ leaderReplica + // create the follower replica with defined log end offset + val followerReplica= new Replica(configs(1).brokerId, partition, time) + followerReplica.logEndOffset = new LogOffsetMetadata(followerLEO, 0L, followerLEO.toInt) + + // add both of them to ISR + val allReplicas = List(leaderReplica, followerReplica) allReplicas.foreach(partition.addReplicaIfNotExists(_)) - // set in sync replicas for this partition to all the assigned replicas partition.inSyncReplicas = allReplicas.toSet - // set the leader and its hw and the hw update time - partition.leaderReplicaIdOpt = Some(leaderId) - leaderReplica.highWatermark = new LogOffsetMetadata(leaderHW) - partition } - private def getFollowerReplicas(partition: Partition, leaderId: Int, time: Time): Seq[Replica] = { - configs.filter(_.brokerId != leaderId).map { config => - new Replica(config.brokerId, partition, time) - } + override def tearDown() { + replicaManager.shutdown(false) + super.tearDown() } + /** + * The scenario for this test is that there is one topic that has one partition + * with one leader replica on broker "0" and one follower replica on broker "1" + * inside the replica manager's metadata. + * + * The leader replica on "0" has HW of "5" and LEO of "20". The follower on + * broker "1" has a local replica with a HW matching the leader's ("5") and + * LEO of "15", meaning it's not in-sync but is still in ISR (hasn't yet expired from ISR). + * + * When a fetch operation with read committed data turned on is received, the replica manager + * should only return data up to the HW of the partition; when a fetch operation with read + * committed data turned off is received, the replica manager could return data up to the LEO + * of the local leader replica's log. + */ + def testReadFromLog() { + + 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) + + 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) + } } From 1ed9cf6d03603518d950f7e9a5f122c4ed5d7cee Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Thu, 30 Oct 2014 16:06:13 -0700 Subject: [PATCH 038/491] KAFKA-1647; Create replicas on follower transition even if leader is unavailable, otherwise replication offset checkpoints (high water marks) can be lost on hard kills and restarts; reviewed by Joel Koshy, Neha Narkhede, Jun Rao and Guozhang Wang --- .../src/main/scala/kafka/server/ReplicaManager.scala | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 02fa3821271e9..3007a6d89b637 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -64,8 +64,8 @@ object ReplicaManager { } class ReplicaManager(val config: KafkaConfig, - time: Time, - val zkClient: ZkClient, + time: Time, + val zkClient: ZkClient, scheduler: Scheduler, val logManager: LogManager, val isShuttingDown: AtomicBoolean ) extends Logging with KafkaMetricsGroup { @@ -621,6 +621,7 @@ class ReplicaManager(val config: KafkaConfig, val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch val newLeaderBrokerId = leaderIsrAndControllerEpoch.leaderAndIsr.leader leaders.find(_.id == newLeaderBrokerId) match { + // Only change partition state when the leader is available case Some(leaderBroker) => if (partition.makeFollower(controllerId, partitionStateInfo, correlationId, offsetManager)) partitionsToMakeFollower += partition @@ -632,10 +633,13 @@ class ReplicaManager(val config: KafkaConfig, case None => // The leader broker should always be present in the leaderAndIsrRequest. // If not, we should record the error message and abort the transition process for this partition - stateChangeLogger.error(("Broker %d aborted the become-follower state change with correlation id %d from " + - "controller %d epoch %d for partition [%s,%d] since new leader %d is not currently available") + stateChangeLogger.error(("Broker %d received LeaderAndIsrRequest with correlation id %d from controller" + + " %d epoch %d for partition [%s,%d] but cannot become follower since the new leader %d is unavailable.") .format(localBrokerId, correlationId, controllerId, leaderIsrAndControllerEpoch.controllerEpoch, partition.topic, partition.partitionId, newLeaderBrokerId)) + // Create the local replica even if the leader is unavailable. This is required to ensure that we include + // the partition's high watermark in the checkpoint file (see KAFKA-1647) + partition.getOrCreateReplica() } } From 58e3f99e244d6c4b9ad4166cc5fcf75561d87dab Mon Sep 17 00:00:00 2001 From: Marc Chung Date: Mon, 3 Nov 2014 21:17:21 -0800 Subject: [PATCH 039/491] kafka-1733; Producer.send will block indeterminately when broker is unavailable; patched by Marc Chung; reviewed by Jun Rao --- .../main/scala/kafka/network/BlockingChannel.scala | 11 +++++++---- core/src/main/scala/kafka/producer/SyncProducer.scala | 2 +- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/network/BlockingChannel.scala b/core/src/main/scala/kafka/network/BlockingChannel.scala index eb7bb14d94cb3..6e2a38eee8e56 100644 --- a/core/src/main/scala/kafka/network/BlockingChannel.scala +++ b/core/src/main/scala/kafka/network/BlockingChannel.scala @@ -42,7 +42,8 @@ class BlockingChannel( val host: String, private var readChannel: ReadableByteChannel = null private var writeChannel: GatheringByteChannel = null private val lock = new Object() - + private val connectTimeoutMs = readTimeoutMs + def connect() = lock synchronized { if(!connected) { try { @@ -55,19 +56,21 @@ class BlockingChannel( val host: String, channel.socket.setSoTimeout(readTimeoutMs) channel.socket.setKeepAlive(true) channel.socket.setTcpNoDelay(true) - channel.connect(new InetSocketAddress(host, port)) + channel.socket.connect(new InetSocketAddress(host, port), connectTimeoutMs) writeChannel = channel readChannel = Channels.newChannel(channel.socket().getInputStream) connected = true // settings may not match what we requested above - val msg = "Created socket with SO_TIMEOUT = %d (requested %d), SO_RCVBUF = %d (requested %d), SO_SNDBUF = %d (requested %d)." + val msg = "Created socket with SO_TIMEOUT = %d (requested %d), SO_RCVBUF = %d (requested %d), SO_SNDBUF = %d (requested %d), connectTimeoutMs = %d." debug(msg.format(channel.socket.getSoTimeout, readTimeoutMs, channel.socket.getReceiveBufferSize, readBufferSize, channel.socket.getSendBufferSize, - writeBufferSize)) + writeBufferSize, + connectTimeoutMs)) + } catch { case e: Throwable => disconnect() } diff --git a/core/src/main/scala/kafka/producer/SyncProducer.scala b/core/src/main/scala/kafka/producer/SyncProducer.scala index 42c950375098b..35e9e8cdb385a 100644 --- a/core/src/main/scala/kafka/producer/SyncProducer.scala +++ b/core/src/main/scala/kafka/producer/SyncProducer.scala @@ -42,7 +42,7 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { val brokerInfo = "host_%s-port_%s".format(config.host, config.port) val producerRequestStats = ProducerRequestStatsRegistry.getProducerRequestStats(config.clientId) - trace("Instantiating Scala Sync Producer") + trace("Instantiating Scala Sync Producer with properties: %s".format(config.props)) private def verifyRequest(request: RequestOrResponse) = { /** From 4bb020212aa0260a750b69dc18856cf25c1e7011 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Tue, 4 Nov 2014 10:09:22 -0800 Subject: [PATCH 040/491] KAFKA-1706; Add a byte bounded blocking queue utility; reviewed by Joel Koshy --- .../utils/ByteBoundedBlockingQueue.scala | 219 ++++++++++++++++++ .../utils/ByteBoundedBlockingQueueTest.scala | 99 ++++++++ 2 files changed, 318 insertions(+) create mode 100644 core/src/main/scala/kafka/utils/ByteBoundedBlockingQueue.scala create mode 100644 core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala diff --git a/core/src/main/scala/kafka/utils/ByteBoundedBlockingQueue.scala b/core/src/main/scala/kafka/utils/ByteBoundedBlockingQueue.scala new file mode 100644 index 0000000000000..6a85d7e494f6c --- /dev/null +++ b/core/src/main/scala/kafka/utils/ByteBoundedBlockingQueue.scala @@ -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 kafka.utils + +import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.{TimeUnit, LinkedBlockingQueue} + +/** + * A blocking queue that have size limits on both number of elements and number of bytes. + */ +class ByteBoundedBlockingQueue[E] (val queueNumMessageCapacity: Int, val queueByteCapacity: Int, sizeFunction: Option[(E) => Int]) + extends Iterable[E] { + private val queue = new LinkedBlockingQueue[E] (queueNumMessageCapacity) + private var currentByteSize = new AtomicInteger() + private val putLock = new Object + + /** + * Please refer to [[java.util.concurrent.BlockingQueue#offer]] + * An element can be enqueued provided the current size (in number of elements) is within the configured + * capacity and the current size in bytes of the queue is within the configured byte capacity. i.e., the + * element may be enqueued even if adding it causes the queue's size in bytes to exceed the byte capacity. + * @param e the element to put into the queue + * @param timeout the amount of time to wait before the expire the operation + * @param unit the time unit of timeout parameter, default to millisecond + * @return true if the element is put into queue, false if it is not + * @throws NullPointerException if element is null + * @throws InterruptedException if interrupted during waiting + */ + def offer(e: E, timeout: Long, unit: TimeUnit = TimeUnit.MICROSECONDS): Boolean = { + if (e == null) throw new NullPointerException("Putting null element into queue.") + val startTime = SystemTime.nanoseconds + val expireTime = startTime + unit.toNanos(timeout) + putLock synchronized { + var timeoutNanos = expireTime - SystemTime.nanoseconds + while (currentByteSize.get() >= queueByteCapacity && timeoutNanos > 0) { + // ensure that timeoutNanos > 0, otherwise (per javadoc) we have to wait until the next notify + putLock.wait(timeoutNanos / 1000000, (timeoutNanos % 1000000).toInt) + timeoutNanos = expireTime - SystemTime.nanoseconds + } + // only proceed if queue has capacity and not timeout + timeoutNanos = expireTime - SystemTime.nanoseconds + if (currentByteSize.get() < queueByteCapacity && timeoutNanos > 0) { + val success = queue.offer(e, timeoutNanos, TimeUnit.NANOSECONDS) + // only increase queue byte size if put succeeds + if (success) + currentByteSize.addAndGet(sizeFunction.get(e)) + // wake up another thread in case multiple threads are waiting + if (currentByteSize.get() < queueByteCapacity) + putLock.notify() + success + } else { + false + } + } + } + + /** + * Please refer to [[java.util.concurrent.BlockingQueue#offer]]. + * Put an element to the tail of the queue, return false immediately if queue is full + * @param e The element to put into queue + * @return true on succeed, false on failure + * @throws NullPointerException if element is null + * @throws InterruptedException if interrupted during waiting + */ + def offer(e: E): Boolean = { + if (e == null) throw new NullPointerException("Putting null element into queue.") + putLock synchronized { + if (currentByteSize.get() >= queueByteCapacity) { + false + } else { + val success = queue.offer(e) + if (success) + currentByteSize.addAndGet(sizeFunction.get(e)) + // wake up another thread in case multiple threads are waiting + if (currentByteSize.get() < queueByteCapacity) + putLock.notify() + success + } + } + } + + /** + * Please refer to [[java.util.concurrent.BlockingQueue#put]]. + * Put an element to the tail of the queue, block if queue is full + * @param e The element to put into queue + * @return true on succeed, false on failure + * @throws NullPointerException if element is null + * @throws InterruptedException if interrupted during waiting + */ + def put(e: E): Boolean = { + if (e == null) throw new NullPointerException("Putting null element into queue.") + putLock synchronized { + if (currentByteSize.get() >= queueByteCapacity) + putLock.wait() + val success = queue.offer(e) + if (success) + currentByteSize.addAndGet(sizeFunction.get(e)) + // wake up another thread in case multiple threads are waiting + if (currentByteSize.get() < queueByteCapacity) + putLock.notify() + success + } + } + + /** + * Please refer to [[java.util.concurrent.BlockingQueue#poll]] + * Get an element from the head of queue. Wait for some time if the queue is empty. + * @param timeout the amount of time to wait if the queue is empty + * @param unit the unit type + * @return the first element in the queue, null if queue is empty + */ + def poll(timeout: Long, unit: TimeUnit): E = { + val e = queue.poll(timeout, unit) + // only wake up waiting threads if the queue size drop under queueByteCapacity + if (e != null && + currentByteSize.getAndAdd(-sizeFunction.get(e)) > queueByteCapacity && + currentByteSize.get() < queueByteCapacity) + putLock.synchronized(putLock.notify()) + e + } + + /** + * Please refer to [[java.util.concurrent.BlockingQueue#poll]] + * Get an element from the head of queue. + * @return the first element in the queue, null if queue is empty + */ + def poll(): E = { + val e = queue.poll() + // only wake up waiting threads if the queue size drop under queueByteCapacity + if (e != null && + currentByteSize.getAndAdd(-sizeFunction.get(e)) > queueByteCapacity && + currentByteSize.get() < queueByteCapacity) + putLock.synchronized(putLock.notify()) + e + } + + /** + * Please refer to [[java.util.concurrent.BlockingQueue#take]] + * Get an element from the head of the queue, block if the queue is empty + * @return the first element in the queue, null if queue is empty + */ + def take(): E = { + val e = queue.take() + // only wake up waiting threads if the queue size drop under queueByteCapacity + if (currentByteSize.getAndAdd(-sizeFunction.get(e)) > queueByteCapacity && + currentByteSize.get() < queueByteCapacity) + putLock.synchronized(putLock.notify()) + e + } + + /** + * Iterator for the queue + * @return Iterator for the queue + */ + override def iterator = new Iterator[E] () { + private val iter = queue.iterator() + private var curr: E = null.asInstanceOf[E] + + def hasNext: Boolean = iter.hasNext + + def next(): E = { + curr = iter.next() + curr + } + + def remove() { + if (curr == null) + throw new IllegalStateException("Iterator does not have a current element.") + iter.remove() + if (currentByteSize.addAndGet(-sizeFunction.get(curr)) < queueByteCapacity) + putLock.synchronized(putLock.notify()) + } + } + + /** + * get the number of elements in the queue + * @return number of elements in the queue + */ + override def size() = queue.size() + + /** + * get the current byte size in the queue + * @return current queue size in bytes + */ + def byteSize() = { + val currSize = currentByteSize.get() + // There is a potential race where after an element is put into the queue and before the size is added to + // currentByteSize, it was taken out of the queue and the size was deducted from the currentByteSize, + // in that case, currentByteSize would become negative, in that case, just put the queue size to be 0. + if (currSize > 0) currSize else 0 + } + + /** + * get the number of unused slots in the queue + * @return the number of unused slots in the queue + */ + def remainingSize = queue.remainingCapacity() + + /** + * get the remaining bytes capacity of the queue + * @return the remaining bytes capacity of the queue + */ + def remainingByteSize = math.max(0, queueByteCapacity - currentByteSize.get()) +} diff --git a/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala b/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala new file mode 100644 index 0000000000000..fe8d2aeb642bc --- /dev/null +++ b/core/src/test/scala/unit/kafka/utils/ByteBoundedBlockingQueueTest.scala @@ -0,0 +1,99 @@ +/** + * 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 unit.kafka.utils + +import java.util.concurrent.TimeUnit + +import junit.framework.Assert._ +import org.junit.{Test} +import kafka.utils.ByteBoundedBlockingQueue + +class ByteBoundedBlockingQueueTest { + val sizeFunction = (a: String) => a.length + val queue = new ByteBoundedBlockingQueue[String](5, 15, Some(sizeFunction)) + + @Test + def testByteBoundedBlockingQueue() { + assertEquals(5, queue.remainingSize) + assertEquals(15, queue.remainingByteSize) + + //offer a message whose size is smaller than remaining capacity + val m0 = new String("0123456789") + assertEquals(true, queue.offer(m0)) + assertEquals(1, queue.size()) + assertEquals(10, queue.byteSize()) + assertEquals(4, queue.remainingSize) + assertEquals(5, queue.remainingByteSize) + + // offer a message where remaining capacity < message size < capacity limit + val m1 = new String("1234567890") + assertEquals(true, queue.offer(m1)) + assertEquals(2, queue.size()) + assertEquals(20, queue.byteSize()) + assertEquals(3, queue.remainingSize) + assertEquals(0, queue.remainingByteSize) + + // offer a message using timeout, should fail because no space is left + val m2 = new String("2345678901") + assertEquals(false, queue.offer(m2, 10, TimeUnit.MILLISECONDS)) + assertEquals(2, queue.size()) + assertEquals(20, queue.byteSize()) + assertEquals(3, queue.remainingSize) + assertEquals(0, queue.remainingByteSize) + + // take an element out of the queue + assertEquals("0123456789", queue.take()) + assertEquals(1, queue.size()) + assertEquals(10, queue.byteSize()) + assertEquals(4, queue.remainingSize) + assertEquals(5, queue.remainingByteSize) + + // add 5 small elements into the queue, first 4 should succeed, the 5th one should fail + // test put() + assertEquals(true, queue.put("a")) + assertEquals(true, queue.offer("b")) + assertEquals(true, queue.offer("c")) + assertEquals(4, queue.size()) + assertEquals(13, queue.byteSize()) + assertEquals(1, queue.remainingSize) + assertEquals(2, queue.remainingByteSize) + + assertEquals(true, queue.offer("d")) + assertEquals(5, queue.size()) + assertEquals(14, queue.byteSize()) + assertEquals(0, queue.remainingSize) + assertEquals(1, queue.remainingByteSize) + + assertEquals(false, queue.offer("e")) + assertEquals(5, queue.size()) + assertEquals(14, queue.byteSize()) + assertEquals(0, queue.remainingSize) + assertEquals(1, queue.remainingByteSize) + + // try take 6 elements out of the queue, the last poll() should fail as there is no element anymore + // test take() + assertEquals("1234567890", queue.poll(10, TimeUnit.MILLISECONDS)) + // test poll + assertEquals("a", queue.poll()) + assertEquals("b", queue.poll()) + assertEquals("c", queue.poll()) + assertEquals("d", queue.poll()) + assertEquals(null, queue.poll(10, TimeUnit.MILLISECONDS)) + } + +} From 06e1a6552e2c1de08c185359f3114eb12e61cfae Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Thu, 6 Nov 2014 18:13:14 -0800 Subject: [PATCH 041/491] KAFKA-1747 TestcaseEnv improperly shares state between instances; reviewed by Neha Narkhede --- system_test/utils/testcase_env.py | 117 +++++++++++++++--------------- 1 file changed, 58 insertions(+), 59 deletions(-) diff --git a/system_test/utils/testcase_env.py b/system_test/utils/testcase_env.py index b3c29105c0434..1d2fb5762f531 100644 --- a/system_test/utils/testcase_env.py +++ b/system_test/utils/testcase_env.py @@ -28,68 +28,67 @@ import system_test_utils class TestcaseEnv(): + def __init__(self, systemTestEnv, classInstance): + self.systemTestEnv = systemTestEnv - # ================================ - # Generic testcase environment - # ================================ - - # dictionary of entity_id to ppid for Zookeeper entities - # key: entity_id - # val: ppid of Zookeeper associated to that entity_id - # { 0: 12345, 1: 12389, ... } - entityZkParentPidDict = {} - - # dictionary of entity_id to ppid for broker entities - # key: entity_id - # val: ppid of broker associated to that entity_id - # { 0: 12345, 1: 12389, ... } - entityBrokerParentPidDict = {} - - # dictionary of entity_id to ppid for mirror-maker entities - # key: entity_id - # val: ppid of broker associated to that entity_id - # { 0: 12345, 1: 12389, ... } - entityMirrorMakerParentPidDict = {} - - # dictionary of entity_id to ppid for console-consumer entities - # key: entity_id - # val: ppid of console consumer associated to that entity_id - # { 0: 12345, 1: 12389, ... } - entityConsoleConsumerParentPidDict = {} - - # dictionary of entity_id to ppid for migration tool entities - # key: entity_id - # val: ppid of broker associated to that entity_id - # { 0: 12345, 1: 12389, ... } - entityMigrationToolParentPidDict = {} - - # dictionary of entity_id to list of JMX ppid - # key: entity_id - # val: list of JMX ppid associated to that entity_id - # { 1: [1234, 1235, 1236], 2: [2234, 2235, 2236], ... } - entityJmxParentPidDict = {} - - # dictionary of hostname-topic-ppid for consumer - # key: hostname - # val: dict of topic-ppid - # { host1: { test1 : 12345 }, host1: { test2 : 12389 }, ... } - consumerHostParentPidDict = {} - - # dictionary of hostname-topic-ppid for producer - # key: hostname - # val: dict of topic-ppid - # { host1: { test1 : 12345 }, host1: { test2 : 12389 }, ... } - producerHostParentPidDict = {} - - # list of testcase configs - testcaseConfigsList = [] - - # dictionary to keep track of testcase arguments such as replica_factor, num_partition - testcaseArgumentsDict = {} + # ================================ + # Generic testcase environment + # ================================ + # dictionary of entity_id to ppid for Zookeeper entities + # key: entity_id + # val: ppid of Zookeeper associated to that entity_id + # { 0: 12345, 1: 12389, ... } + self.entityZkParentPidDict = {} + + # dictionary of entity_id to ppid for broker entities + # key: entity_id + # val: ppid of broker associated to that entity_id + # { 0: 12345, 1: 12389, ... } + self.entityBrokerParentPidDict = {} + + # dictionary of entity_id to ppid for mirror-maker entities + # key: entity_id + # val: ppid of broker associated to that entity_id + # { 0: 12345, 1: 12389, ... } + self.entityMirrorMakerParentPidDict = {} + + # dictionary of entity_id to ppid for console-consumer entities + # key: entity_id + # val: ppid of console consumer associated to that entity_id + # { 0: 12345, 1: 12389, ... } + self.entityConsoleConsumerParentPidDict = {} + + # dictionary of entity_id to ppid for migration tool entities + # key: entity_id + # val: ppid of broker associated to that entity_id + # { 0: 12345, 1: 12389, ... } + self.entityMigrationToolParentPidDict = {} + + # dictionary of entity_id to list of JMX ppid + # key: entity_id + # val: list of JMX ppid associated to that entity_id + # { 1: [1234, 1235, 1236], 2: [2234, 2235, 2236], ... } + self.entityJmxParentPidDict = {} + + # dictionary of hostname-topic-ppid for consumer + # key: hostname + # val: dict of topic-ppid + # { host1: { test1 : 12345 }, host1: { test2 : 12389 }, ... } + self.consumerHostParentPidDict = {} + + # dictionary of hostname-topic-ppid for producer + # key: hostname + # val: dict of topic-ppid + # { host1: { test1 : 12345 }, host1: { test2 : 12389 }, ... } + self.producerHostParentPidDict = {} + + # list of testcase configs + self.testcaseConfigsList = [] + + # dictionary to keep track of testcase arguments such as replica_factor, num_partition + self.testcaseArgumentsDict = {} - def __init__(self, systemTestEnv, classInstance): - self.systemTestEnv = systemTestEnv # gather the test case related info and add to an SystemTestEnv object self.testcaseResultsDict = {} From cbdcd5f108d783e9d0e758fbf0d8479968db3a68 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Thu, 6 Nov 2014 18:16:13 -0800 Subject: [PATCH 042/491] KAFKA-1746 System tests don't handle errors well; reviewed by Neha Narkhede --- system_test/mirror_maker_testsuite/mirror_maker_test.py | 1 + .../offset_management_testsuite/offset_management_test.py | 1 + system_test/replication_testsuite/replica_basic_test.py | 2 ++ system_test/system_test_runner.py | 2 +- system_test/utils/kafka_system_test_utils.py | 3 +++ 5 files changed, 8 insertions(+), 1 deletion(-) diff --git a/system_test/mirror_maker_testsuite/mirror_maker_test.py b/system_test/mirror_maker_testsuite/mirror_maker_test.py index c0117c64cbb76..48f9ff6b2810f 100644 --- a/system_test/mirror_maker_testsuite/mirror_maker_test.py +++ b/system_test/mirror_maker_testsuite/mirror_maker_test.py @@ -315,6 +315,7 @@ def runTest(self): except Exception as e: self.log_message("Exception while running test {0}".format(e)) traceback.print_exc() + self.testcaseEnv.validationStatusDict["Test completed"] = "FAILED" finally: if not skipThisTestCase and not self.systemTestEnv.printTestDescriptionsOnly: diff --git a/system_test/offset_management_testsuite/offset_management_test.py b/system_test/offset_management_testsuite/offset_management_test.py index 12b5cd25140e1..aa389105aa427 100644 --- a/system_test/offset_management_testsuite/offset_management_test.py +++ b/system_test/offset_management_testsuite/offset_management_test.py @@ -290,6 +290,7 @@ def runTest(self): except Exception as e: self.log_message("Exception while running test {0}".format(e)) traceback.print_exc() + self.testcaseEnv.validationStatusDict["Test completed"] = "FAILED" finally: if not skipThisTestCase and not self.systemTestEnv.printTestDescriptionsOnly: diff --git a/system_test/replication_testsuite/replica_basic_test.py b/system_test/replication_testsuite/replica_basic_test.py index 660006cc253bb..16a24a407051a 100644 --- a/system_test/replication_testsuite/replica_basic_test.py +++ b/system_test/replication_testsuite/replica_basic_test.py @@ -451,6 +451,8 @@ def runTest(self): except Exception as e: self.log_message("Exception while running test {0}".format(e)) traceback.print_exc() + self.testcaseEnv.validationStatusDict["Test completed"] = "FAILED" + finally: if not skipThisTestCase and not self.systemTestEnv.printTestDescriptionsOnly: diff --git a/system_test/system_test_runner.py b/system_test/system_test_runner.py index ee7aa25233355..5078d4479fab7 100644 --- a/system_test/system_test_runner.py +++ b/system_test/system_test_runner.py @@ -199,6 +199,6 @@ def main(): # main entry point # ========================= -main() +sys.exit(main()) diff --git a/system_test/utils/kafka_system_test_utils.py b/system_test/utils/kafka_system_test_utils.py index 1093b660ebd0c..41d511cbc310f 100644 --- a/system_test/utils/kafka_system_test_utils.py +++ b/system_test/utils/kafka_system_test_utils.py @@ -2400,6 +2400,9 @@ def validate_index_log(systemTestEnv, testcaseEnv, clusterName="source"): logger.debug("#### error found [" + line + "]", extra=d) failureCount += 1 showMismatchedIndexOffset = True + if subproc.wait() != 0: + logger.debug("#### error found [DumpLogSegments exited abnormally]", extra=d) + failureCount += 1 if failureCount == 0: validationStatusDict["Validate index log in cluster [" + clusterName + "]"] = "PASSED" From f20e5108a271e1fc37ca6752773efa74e2fef67f Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Fri, 7 Nov 2014 10:46:21 -0800 Subject: [PATCH 043/491] kafka-1738; Partitions for topic not created after restart from forced shutdown; patched by Jun Rao; reviewed by Neha Narkhede --- .../scala/kafka/controller/ControllerChannelManager.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index ecbfa0f328ba6..eb492f0044974 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -130,10 +130,11 @@ class RequestSendThread(val controllerId: Int, // removeBroker which will invoke shutdown() on this thread. At that point, we will stop retrying. try { channel.send(request) + receive = channel.receive() isSendSuccessful = true } catch { case e: Throwable => // if the send was not successful, reconnect to broker and resend the message - error(("Controller %d epoch %d failed to send request %s to broker %s. " + + warn(("Controller %d epoch %d fails to send request %s to broker %s. " + "Reconnecting to broker.").format(controllerId, controllerContext.epoch, request.toString, toBroker.toString()), e) channel.disconnect() @@ -143,7 +144,6 @@ class RequestSendThread(val controllerId: Int, Utils.swallow(Thread.sleep(300)) } } - receive = channel.receive() var response: RequestOrResponse = null request.requestId.get match { case RequestKeys.LeaderAndIsrKey => @@ -162,7 +162,7 @@ class RequestSendThread(val controllerId: Int, } } catch { case e: Throwable => - warn("Controller %d fails to send a request to broker %s".format(controllerId, toBroker.toString()), e) + error("Controller %d fails to send a request to broker %s".format(controllerId, toBroker.toString()), e) // If there is any socket error (eg, socket timeout), the channel is no longer usable and needs to be recreated. channel.disconnect() } From 9b3da9fbfd60e3a0482ec7bd615dc9806cd9941c Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Sun, 9 Nov 2014 20:23:46 -0800 Subject: [PATCH 044/491] KAFKA-1739 Remove testComplexCompressDecompress in MessageCompressionTest; reviewed by Neha Narkhede --- .../unit/kafka/message/MessageCompressionTest.scala | 9 --------- 1 file changed, 9 deletions(-) diff --git a/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala b/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala index 0bb275d0dc840..76987d4fa68fd 100644 --- a/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala +++ b/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala @@ -44,15 +44,6 @@ class MessageCompressionTest extends JUnitSuite { assertEquals(messages, decompressed) } - @Test - def testComplexCompressDecompress() { - val messages = List(new Message("hi there".getBytes), new Message("I am fine".getBytes), new Message("I am not so well today".getBytes)) - val message = new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec, messages = messages.slice(0, 2):_*) - val complexMessages = List(message.shallowIterator.next.message):::messages.slice(2,3) - val complexMessage = new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec, messages = complexMessages:_*) - val decompressedMessages = complexMessage.iterator.map(_.message).toList - assertEquals(messages, decompressedMessages) - } def isSnappyAvailable(): Boolean = { try { From 53160bb07c9f77ed21f486ed448aff0277c139b0 Mon Sep 17 00:00:00 2001 From: Onur Karaman Date: Tue, 11 Nov 2014 18:42:22 -0800 Subject: [PATCH 045/491] KAFKA-1742; ControllerContext removeTopic does not correctly update state; reviewed by Joel Koshy, Guozhang Wang and Neha Narkhede --- core/src/main/scala/kafka/controller/KafkaController.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 51a5bad24f1e7..66df6d2fbdbdd 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -115,8 +115,8 @@ class ControllerContext(val zkClient: ZkClient, } def removeTopic(topic: String) = { - partitionLeadershipInfo = partitionLeadershipInfo.dropWhile(p => p._1.topic.equals(topic)) - partitionReplicaAssignment = partitionReplicaAssignment.dropWhile(p => p._1.topic.equals(topic)) + partitionLeadershipInfo = partitionLeadershipInfo.filter{ case (topicAndPartition, _) => topicAndPartition.topic != topic } + partitionReplicaAssignment = partitionReplicaAssignment.filter{ case (topicAndPartition, _) => topicAndPartition.topic != topic } allTopics -= topic } } From 2cd9ae74544efe049393e1a6bf60533aea956d7d Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 12 Nov 2014 15:54:50 -0800 Subject: [PATCH 046/491] KAFKA-1762; Update max-inflight-requests doc-string in producer config to note risk of reordering in the presence of retries --- .../org/apache/kafka/clients/producer/ProducerConfig.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 9095caf0db1e4..72d3ddd0c29bf 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -171,7 +171,9 @@ public class ProducerConfig extends AbstractConfig { /** max.in.flight.requests.per.connection */ public static final String MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION = "max.in.flight.requests.per.connection"; - private static final String MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_DOC = "The maximum number of unacknowledged requests the client will send on a single connection before blocking."; + private static final String MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_DOC = "The maximum number of unacknowledged requests the client will send on a single connection before blocking." + + " Note that if this setting is set to be greater than 1 and there are failed sends, there is a risk of" + + " message re-ordering due to retries (i.e., if retries are enabled)."; static { config = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, BOOSTRAP_SERVERS_DOC) From d99af88eafefbfd1c537a64f8107cd9041346015 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Thu, 13 Nov 2014 18:10:35 -0800 Subject: [PATCH 047/491] KAFKA-1764; ZookeeperConsumerConnector should not put multiple shutdown commands to the same data chunk queue; reviewed by Joel Koshy and Guozhang Wang --- core/src/main/scala/kafka/consumer/ConsumerIterator.scala | 1 - .../main/scala/kafka/consumer/ZookeeperConsumerConnector.scala | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala index ac491b4da2583..78fbf75651583 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala @@ -71,7 +71,6 @@ class ConsumerIterator[K, V](private val channel: BlockingQueue[FetchedDataChunk } if(currentDataChunk eq ZookeeperConsumerConnector.shutdownCommand) { debug("Received the shutdown command") - channel.offer(currentDataChunk) return allDone } else { currentTopicInfo = currentDataChunk.topicInfo diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index fbc680fde21b0..2402b45f646bf 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -256,7 +256,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } private def sendShutdownToAllQueues() = { - for (queue <- topicThreadIdAndQueues.values) { + for (queue <- topicThreadIdAndQueues.values.toSet) { debug("Clearing up queue") queue.clear() queue.put(ZookeeperConsumerConnector.shutdownCommand) From 8c35030e4d24e603e5dbd53862ca6ae4fe59b4e9 Mon Sep 17 00:00:00 2001 From: Joel Koshy Date: Thu, 13 Nov 2014 20:58:03 -0800 Subject: [PATCH 048/491] Revert "KAFKA-1764; ZookeeperConsumerConnector should not put multiple shutdown commands to the same data chunk queue; reviewed by Joel Koshy and Guozhang Wang" This reverts commit d99af88eafefbfd1c537a64f8107cd9041346015. --- core/src/main/scala/kafka/consumer/ConsumerIterator.scala | 1 + .../main/scala/kafka/consumer/ZookeeperConsumerConnector.scala | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala index 78fbf75651583..ac491b4da2583 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala @@ -71,6 +71,7 @@ class ConsumerIterator[K, V](private val channel: BlockingQueue[FetchedDataChunk } if(currentDataChunk eq ZookeeperConsumerConnector.shutdownCommand) { debug("Received the shutdown command") + channel.offer(currentDataChunk) return allDone } else { currentTopicInfo = currentDataChunk.topicInfo diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index 2402b45f646bf..fbc680fde21b0 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -256,7 +256,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } private def sendShutdownToAllQueues() = { - for (queue <- topicThreadIdAndQueues.values.toSet) { + for (queue <- topicThreadIdAndQueues.values) { debug("Clearing up queue") queue.clear() queue.put(ZookeeperConsumerConnector.shutdownCommand) From ca2cf97a6b151566f091a5dd016b93dfdaf87628 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Fri, 14 Nov 2014 10:02:45 -0800 Subject: [PATCH 049/491] KAFKA-1764; ZookeeperConsumerConnector should not put multiple shutdown commands to the same data chunk queue; reviewed by Joel Koshy and Guozhang Wang --- core/src/main/scala/kafka/consumer/ConsumerIterator.scala | 1 - .../main/scala/kafka/consumer/ZookeeperConsumerConnector.scala | 2 +- .../test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala | 2 +- 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala index ac491b4da2583..78fbf75651583 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala @@ -71,7 +71,6 @@ class ConsumerIterator[K, V](private val channel: BlockingQueue[FetchedDataChunk } if(currentDataChunk eq ZookeeperConsumerConnector.shutdownCommand) { debug("Received the shutdown command") - channel.offer(currentDataChunk) return allDone } else { currentTopicInfo = currentDataChunk.topicInfo diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index fbc680fde21b0..f476973eeff65 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -256,7 +256,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } private def sendShutdownToAllQueues() = { - for (queue <- topicThreadIdAndQueues.values) { + for (queue <- topicThreadIdAndQueues.values.toSet[BlockingQueue[FetchedDataChunk]]) { debug("Clearing up queue") queue.clear() queue.put(ZookeeperConsumerConnector.shutdownCommand) diff --git a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala index 151ba7ccd006f..c0355cc0135c6 100644 --- a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala @@ -80,7 +80,7 @@ class ConsumerIteratorTest extends JUnit3Suite with KafkaServerTestHarness { val receivedMessages = (0 until 5).map(i => iter.next.message).toList assertFalse(iter.hasNext) - assertEquals(1, queue.size) // This is only the shutdown command. + assertEquals(0, queue.size) // Shutdown command has been consumed. assertEquals(5, receivedMessages.size) val unconsumed = messageSet.filter(_.offset >= consumedOffset).map(m => Utils.readString(m.message.payload)) assertEquals(unconsumed, receivedMessages) From 7d89867c05a7cca6e76cc48c47f8e703d68a9a43 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Fri, 14 Nov 2014 14:30:04 -0800 Subject: [PATCH 050/491] kafka-1642; [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost; patched by Ewen Cheslack-Postava; reviewed by Guozhang Wang and Jun Rao --- .../clients/ClusterConnectionStates.java | 21 ++++++ .../org/apache/kafka/clients/KafkaClient.java | 10 +++ .../apache/kafka/clients/NetworkClient.java | 13 ++++ .../producer/internals/RecordAccumulator.java | 18 ++++-- .../clients/producer/internals/Sender.java | 13 +++- .../org/apache/kafka/clients/MockClient.java | 5 ++ .../producer/RecordAccumulatorTest.java | 64 +++++++++++++++---- 7 files changed, 124 insertions(+), 20 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java index d304660f29246..8aece7e81a804 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java @@ -56,6 +56,27 @@ public boolean isBlackedOut(int node, long now) { return state.state == ConnectionState.DISCONNECTED && now - state.lastConnectAttemptMs < this.reconnectBackoffMs; } + /** + * Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When + * disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled + * connections. + * @param node The node to check + * @param now The current time in ms + */ + public long connectionDelay(int node, long now) { + NodeConnectionState state = nodeState.get(node); + if (state == null) return 0; + long timeWaited = now - state.lastConnectAttemptMs; + if (state.state == ConnectionState.DISCONNECTED) { + return Math.max(this.reconnectBackoffMs - timeWaited, 0); + } + else { + // When connecting or connected, we should be able to delay indefinitely since other events (connection or + // data acked) will cause a wakeup once data can be sent. + return Long.MAX_VALUE; + } + } + /** * Enter the connecting state for the given node. * @param node The id of the node we are connecting to diff --git a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java index 29658d4a15f11..397695568d3fd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java @@ -40,6 +40,16 @@ public interface KafkaClient { */ public boolean ready(Node node, long now); + /** + * Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When + * disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled + * connections. + * @param node The node to check + * @param now The current timestamp + * @return The number of milliseconds to wait. + */ + public long connectionDelay(Node node, long now); + /** * Initiate the sending of the given requests and return any completed responses. Requests can only be sent on ready * connections. diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index eea270abb16f4..525b95e98010c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -118,6 +118,19 @@ public boolean ready(Node node, long now) { return false; } + /** + * Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When + * disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled + * connections. + * @param node The node to check + * @param now The current timestamp + * @return The number of milliseconds to wait. + */ + @Override + public long connectionDelay(Node node, long now) { + return connectionStates.connectionDelay(node.id(), now); + } + /** * Check if the node with the given id is ready to send more requests. * @param node The given node id diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index c5d470011d334..c15485d1af304 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -183,9 +183,9 @@ public void reenqueue(RecordBatch batch, long now) { } /** - * Get a list of nodes whose partitions are ready to be sent, and the time to when any partition will be ready if no - * partitions are ready yet; If the ready nodes list is non-empty, the timeout value will be 0. Also return the flag - * for whether there are any unknown leaders for the accumulated partition batches. + * Get a list of nodes whose partitions are ready to be sent, and the earliest time at which any non-sendable + * partition will be ready; Also return the flag for whether there are any unknown leaders for the accumulated + * partition batches. *

      * A destination node is ready to send data if ANY one of its partition is not backing off the send and ANY of the * following are true : @@ -219,11 +219,17 @@ public ReadyCheckResult ready(Cluster cluster, long nowMs) { long timeToWaitMs = backingOff ? retryBackoffMs : lingerMs; long timeLeftMs = Math.max(timeToWaitMs - waitedTimeMs, 0); boolean full = deque.size() > 1 || batch.records.isFull(); - boolean expired = waitedTimeMs >= lingerMs; + boolean expired = waitedTimeMs >= timeToWaitMs; boolean sendable = full || expired || exhausted || closed; - if (sendable && !backingOff) + if (sendable && !backingOff) { readyNodes.add(leader); - nextReadyCheckDelayMs = Math.min(timeLeftMs, nextReadyCheckDelayMs); + } + else { + // Note that this results in a conservative estimate since an un-sendable partition may have + // a leader that will later be found to have sendable data. However, this is good enough + // since we'll just wake up and then sleep again for the remaining time. + nextReadyCheckDelayMs = Math.min(timeLeftMs, nextReadyCheckDelayMs); + } } } } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 8ebe7ed82c938..84a7a07269c51 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -152,10 +152,13 @@ public void run(long now) { // remove any nodes we aren't ready to send to Iterator iter = result.readyNodes.iterator(); + long notReadyTimeout = Long.MAX_VALUE; while (iter.hasNext()) { Node node = iter.next(); - if (!this.client.ready(node, now)) + if (!this.client.ready(node, now)) { iter.remove(); + notReadyTimeout = Math.min(notReadyTimeout, this.client.connectionDelay(node, now)); + } } // create produce requests @@ -163,16 +166,22 @@ public void run(long now) { List requests = createProduceRequests(batches, now); sensors.updateProduceRequestMetrics(requests); + // If we have any nodes that are ready to send + have sendable data, poll with 0 timeout so this can immediately + // loop and try sending more data. Otherwise, the timeout is determined by nodes that have partitions with data + // that isn't yet sendable (e.g. lingering, backing off). Note that this specifically does not include nodes + // with sendable data that aren't ready to send since they would cause busy looping. + long pollTimeout = Math.min(result.nextReadyCheckDelayMs, notReadyTimeout); if (result.readyNodes.size() > 0) { log.trace("Nodes with data ready to send: {}", result.readyNodes); log.trace("Created {} produce requests: {}", requests.size(), requests); + pollTimeout = 0; } // if some partitions are already ready to be sent, the select time would be 0; // otherwise if some partition already has some data accumulated but not ready yet, // the select time will be the time difference between now and its linger expiry time; // otherwise the select time will be the time difference between now and the metadata expiry time; - List responses = this.client.poll(requests, result.nextReadyCheckDelayMs, now); + List responses = this.client.poll(requests, pollTimeout, now); for (ClientResponse response : responses) { if (response.wasDisconnected()) handleDisconnect(response, now); diff --git a/clients/src/test/java/org/apache/kafka/clients/MockClient.java b/clients/src/test/java/org/apache/kafka/clients/MockClient.java index aae8d4a1e9827..47b5d4ac1f2a5 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java @@ -41,6 +41,11 @@ public boolean ready(Node node, long now) { return found; } + @Override + public long connectionDelay(Node node, long now) { + return 0; + } + public void disconnect(Integer node) { Iterator iter = requests.iterator(); while (iter.hasNext()) { diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java index 0762b35abba05..2c9932401d573 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java @@ -14,6 +14,7 @@ import static java.util.Arrays.asList; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertFalse; import java.nio.ByteBuffer; @@ -43,16 +44,20 @@ public class RecordAccumulatorTest { private String topic = "test"; private int partition1 = 0; private int partition2 = 1; - private Node node = new Node(0, "localhost", 1111); + private int partition3 = 2; + private Node node1 = new Node(0, "localhost", 1111); + private Node node2 = new Node(1, "localhost", 1112); private TopicPartition tp1 = new TopicPartition(topic, partition1); private TopicPartition tp2 = new TopicPartition(topic, partition2); - private PartitionInfo part1 = new PartitionInfo(topic, partition1, node, null, null); - private PartitionInfo part2 = new PartitionInfo(topic, partition2, node, null, null); + private TopicPartition tp3 = new TopicPartition(topic, partition3); + private PartitionInfo part1 = new PartitionInfo(topic, partition1, node1, null, null); + private PartitionInfo part2 = new PartitionInfo(topic, partition2, node1, null, null); + private PartitionInfo part3 = new PartitionInfo(topic, partition3, node2, null, null); private MockTime time = new MockTime(); private byte[] key = "key".getBytes(); private byte[] value = "value".getBytes(); private int msgSize = Records.LOG_OVERHEAD + Record.recordSize(key, value); - private Cluster cluster = new Cluster(Collections.singleton(node), Arrays.asList(part1, part2)); + private Cluster cluster = new Cluster(Arrays.asList(node1, node2), Arrays.asList(part1, part2, part3)); private Metrics metrics = new Metrics(time); @Test @@ -65,8 +70,8 @@ public void testFull() throws Exception { assertEquals("No partitions should be ready.", 0, accum.ready(cluster, now).readyNodes.size()); } accum.append(tp1, key, value, CompressionType.NONE, null); - assertEquals("Our partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()).readyNodes); - List batches = accum.drain(cluster, Collections.singleton(node), Integer.MAX_VALUE, 0).get(node.id()); + assertEquals("Our partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes); + List batches = accum.drain(cluster, Collections.singleton(node1), Integer.MAX_VALUE, 0).get(node1.id()); assertEquals(1, batches.size()); RecordBatch batch = batches.get(0); Iterator iter = batch.records.iterator(); @@ -83,7 +88,7 @@ public void testAppendLarge() throws Exception { int batchSize = 512; RecordAccumulator accum = new RecordAccumulator(batchSize, 10 * 1024, 0L, 100L, false, metrics, time); accum.append(tp1, key, new byte[2 * batchSize], CompressionType.NONE, null); - assertEquals("Our partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()).readyNodes); + assertEquals("Our partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes); } @Test @@ -93,8 +98,8 @@ public void testLinger() throws Exception { accum.append(tp1, key, value, CompressionType.NONE, null); assertEquals("No partitions should be ready", 0, accum.ready(cluster, time.milliseconds()).readyNodes.size()); time.sleep(10); - assertEquals("Our partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()).readyNodes); - List batches = accum.drain(cluster, Collections.singleton(node), Integer.MAX_VALUE, 0).get(node.id()); + assertEquals("Our partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes); + List batches = accum.drain(cluster, Collections.singleton(node1), Integer.MAX_VALUE, 0).get(node1.id()); assertEquals(1, batches.size()); RecordBatch batch = batches.get(0); Iterator iter = batch.records.iterator(); @@ -113,9 +118,9 @@ public void testPartialDrain() throws Exception { for (int i = 0; i < appends; i++) accum.append(tp, key, value, CompressionType.NONE, null); } - assertEquals("Partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()).readyNodes); + assertEquals("Partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes); - List batches = accum.drain(cluster, Collections.singleton(node), 1024, 0).get(node.id()); + List batches = accum.drain(cluster, Collections.singleton(node1), 1024, 0).get(node1.id()); assertEquals("But due to size bound only one partition should have been retrieved", 1, batches.size()); } @@ -145,7 +150,7 @@ public void run() { long now = time.milliseconds(); while (read < numThreads * msgs) { Set nodes = accum.ready(cluster, now).readyNodes; - List batches = accum.drain(cluster, nodes, 5 * 1024, 0).get(node.id()); + List batches = accum.drain(cluster, nodes, 5 * 1024, 0).get(node1.id()); if (batches != null) { for (RecordBatch batch : batches) { for (LogEntry entry : batch.records) @@ -159,4 +164,39 @@ public void run() { t.join(); } + + @Test + public void testNextReadyCheckDelay() throws Exception { + // Next check time will use lingerMs since this test won't trigger any retries/backoff + long lingerMs = 10L; + RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, lingerMs, 100L, false, metrics, time); + // Just short of going over the limit so we trigger linger time + int appends = 1024 / msgSize; + + // Partition on node1 only + for (int i = 0; i < appends; i++) + accum.append(tp1, key, value, CompressionType.NONE, null); + RecordAccumulator.ReadyCheckResult result = accum.ready(cluster, time.milliseconds()); + assertEquals("No nodes should be ready.", 0, result.readyNodes.size()); + assertEquals("Next check time should be the linger time", lingerMs, result.nextReadyCheckDelayMs); + + time.sleep(lingerMs / 2); + + // Add partition on node2 only + for (int i = 0; i < appends; i++) + accum.append(tp3, key, value, CompressionType.NONE, null); + result = accum.ready(cluster, time.milliseconds()); + assertEquals("No nodes should be ready.", 0, result.readyNodes.size()); + assertEquals("Next check time should be defined by node1, half remaining linger time", lingerMs / 2, result.nextReadyCheckDelayMs); + + // Add data for another partition on node1, enough to make data sendable immediately + for (int i = 0; i < appends+1; i++) + accum.append(tp2, key, value, CompressionType.NONE, null); + result = accum.ready(cluster, time.milliseconds()); + assertEquals("Node1 should be ready", Collections.singleton(node1), result.readyNodes); + // Note this can actually be < linger time because it may use delays from partitions that aren't sendable + // but have leaders with other sendable data. + assertTrue("Next check time should be defined by node2, at most linger time", result.nextReadyCheckDelayMs <= lingerMs); + } + } From 26ad54c109db9eca01df926255017ece3eb65645 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Fri, 14 Nov 2014 15:15:14 -0800 Subject: [PATCH 051/491] kafka-1721; Snappy compressor is not thread safe; patched by Ewen Cheslack-Postava; reviewed by Jun Rao --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index c3e6bb839ad65..17b996984efda 100644 --- a/build.gradle +++ b/build.gradle @@ -343,7 +343,7 @@ project(':clients') { dependencies { compile "org.slf4j:slf4j-api:1.7.6" - compile 'org.xerial.snappy:snappy-java:1.1.1.3' + compile 'org.xerial.snappy:snappy-java:1.1.1.6' compile 'net.jpountz.lz4:lz4:1.2.0' testCompile 'com.novocode:junit-interface:0.9' From 6f4b8a10115a7373917cf2a55b74aa8a14c547b9 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Mon, 17 Nov 2014 18:53:16 -0800 Subject: [PATCH 052/491] kafka-17693; javadoc should only include client facing packages; patched by Jun Rao; reviewed by Neha Narkhede --- build.gradle | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/build.gradle b/build.gradle index 17b996984efda..11eb11355efdd 100644 --- a/build.gradle +++ b/build.gradle @@ -361,4 +361,10 @@ project(':clients') { exceptionFormat = 'full' } } + + javadoc { + include "**/org/apache/kafka/clients/producer/*" + include "**/org/apache/kafka/common/errors/*" + } + } From 1c7d783dd648d0e691ad80cb1357c52d3f0af11a Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Tue, 18 Nov 2014 19:03:51 -0800 Subject: [PATCH 053/491] kafka-1743; ConsumerConnector.commitOffsets in 0.8.2 is not backward compatible; patched by Manikumar Reddy; reviewed by Jun Rao --- .../main/scala/kafka/consumer/ConsumerConnector.scala | 7 ++++++- .../kafka/consumer/ZookeeperConsumerConnector.scala | 11 ++++++++--- .../javaapi/consumer/ZookeeperConsumerConnector.scala | 2 +- .../consumer/ZookeeperConsumerConnectorTest.scala | 4 ++-- 4 files changed, 17 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/kafka/consumer/ConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ConsumerConnector.scala index 07677c1c26768..62c0686e816d2 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConnector.scala @@ -70,7 +70,12 @@ trait ConsumerConnector { /** * Commit the offsets of all broker partitions connected by this connector. */ - def commitOffsets(retryOnFailure: Boolean = true) + def commitOffsets(retryOnFailure: Boolean) + + /** + * KAFKA-1743: This method added for backward compatibility. + */ + def commitOffsets /** * Shut down the connector diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index f476973eeff65..4b6dc072b2d89 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -198,7 +198,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } sendShutdownToAllQueues() if (config.autoCommitEnable) - commitOffsets() + commitOffsets(true) if (zkClient != null) { zkClient.close() zkClient = null @@ -285,7 +285,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } } - def commitOffsets(isAutoCommit: Boolean = true) { + def commitOffsets(isAutoCommit: Boolean) { var retriesRemaining = 1 + (if (isAutoCommit) config.offsetsCommitMaxRetries else 0) // no retries for commits from auto-commit var done = false @@ -373,6 +373,11 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } } + /** + * KAFKA-1743: This method added for backward compatibility. + */ + def commitOffsets { commitOffsets(true) } + private def fetchOffsetFromZooKeeper(topicPartition: TopicAndPartition) = { val dirs = new ZKGroupTopicDirs(config.groupId, topicPartition.topic) val offsetString = readDataMaybeNull(zkClient, dirs.consumerOffsetDir + "/" + topicPartition.partition)._1 @@ -712,7 +717,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, * successfully and the fetchers restart to fetch more data chunks **/ if (config.autoCommitEnable) - commitOffsets() + commitOffsets(true) case None => } } diff --git a/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala index 1f98db5d692ad..9d5a47fb8e04d 100644 --- a/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala @@ -108,7 +108,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, createMessageStreamsByFilter(topicFilter, 1, new DefaultDecoder(), new DefaultDecoder()) def commitOffsets() { - underlying.commitOffsets() + underlying.commitOffsets(true) } def commitOffsets(retryOnFailure: Boolean) { diff --git a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala index e1d87112a2a58..9b0521c85fa66 100644 --- a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala @@ -114,7 +114,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar assertEquals(expected_1, actual_1) // commit consumed offsets - zkConsumerConnector1.commitOffsets() + zkConsumerConnector1.commitOffsets(true) // create a consumer val consumerConfig2 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer2)) { @@ -202,7 +202,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar assertEquals(expected_1, actual_1) // commit consumed offsets - zkConsumerConnector1.commitOffsets() + zkConsumerConnector1.commitOffsets(true) // create a consumer val consumerConfig2 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer2)) { From 457744a820d806e546edebbd8ffd33f6772e519f Mon Sep 17 00:00:00 2001 From: Vladimir Tretyakov Date: Wed, 19 Nov 2014 17:57:41 -0800 Subject: [PATCH 054/491] kafka-1481; Stop using dashes AND underscores as separators in MBean names; patched by Vladimir Tretyakov; reviewed by Joel Koshy and Jun Rao --- build.gradle | 6 + .../main/scala/kafka/cluster/Partition.scala | 13 +- .../src/main/scala/kafka/common/AppInfo.scala | 66 +++++++++ .../kafka/common/ClientIdAndBroker.scala | 14 +- .../scala/kafka/common/ClientIdAndTopic.scala | 14 +- .../consumer/ConsumerFetcherThread.scala | 2 +- .../kafka/consumer/ConsumerTopicStats.scala | 17 ++- .../FetchRequestAndResponseStats.scala | 28 ++-- .../scala/kafka/consumer/SimpleConsumer.scala | 5 +- .../consumer/ZookeeperConsumerConnector.scala | 37 +++-- core/src/main/scala/kafka/log/Log.scala | 38 +++-- .../kafka/metrics/KafkaMetricsGroup.scala | 135 +++++++++++------- .../scala/kafka/network/RequestChannel.scala | 27 ++-- .../scala/kafka/network/SocketServer.scala | 2 +- .../main/scala/kafka/producer/Producer.scala | 3 +- .../kafka/producer/ProducerRequestStats.scala | 23 +-- .../scala/kafka/producer/ProducerStats.scala | 7 +- .../kafka/producer/ProducerTopicStats.scala | 23 +-- .../scala/kafka/producer/SyncProducer.scala | 12 +- .../producer/async/ProducerSendThread.scala | 5 +- .../kafka/server/AbstractFetcherManager.scala | 31 ++-- .../kafka/server/AbstractFetcherThread.scala | 36 ++--- .../scala/kafka/server/DelayedFetch.scala | 1 - .../scala/kafka/server/DelayedProduce.scala | 4 +- .../kafka/server/DelayedRequestKey.scala | 38 ----- .../kafka/server/KafkaRequestHandler.scala | 25 ++-- .../main/scala/kafka/server/KafkaServer.scala | 1 - .../kafka/server/KafkaServerStartable.scala | 2 + .../scala/kafka/server/ReplicaManager.scala | 15 +- .../kafka/tools/ProducerPerformance.scala | 1 - .../ZookeeperConsumerConnectorTest.scala | 101 +++---------- .../unit/kafka/metrics/MetricsTest.scala | 72 ++++++++++ .../scala/unit/kafka/utils/TestUtils.scala | 74 +++++++++- 33 files changed, 540 insertions(+), 338 deletions(-) create mode 100644 core/src/main/scala/kafka/common/AppInfo.scala delete mode 100644 core/src/main/scala/kafka/server/DelayedRequestKey.scala create mode 100644 core/src/test/scala/unit/kafka/metrics/MetricsTest.scala diff --git a/build.gradle b/build.gradle index 11eb11355efdd..030af63553afb 100644 --- a/build.gradle +++ b/build.gradle @@ -263,6 +263,12 @@ project(':core') { dependsOn 'copyDependantLibs' } + jar.manifest { + attributes( + 'Version': "${version}" + ) + } + task testJar(type: Jar) { classifier = 'test' from sourceSets.test.output diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 1be57008e983f..b9fde2aacbb1f 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -21,7 +21,7 @@ import kafka.admin.AdminUtils import kafka.utils._ import kafka.api.{PartitionStateInfo, LeaderAndIsr} import kafka.log.LogConfig -import kafka.server.{TopicPartitionRequestKey, LogOffsetMetadata, OffsetManager, ReplicaManager} +import kafka.server.{LogOffsetMetadata, OffsetManager, ReplicaManager} import kafka.metrics.KafkaMetricsGroup import kafka.controller.KafkaController import kafka.message.ByteBufferMessageSet @@ -29,7 +29,6 @@ import kafka.message.ByteBufferMessageSet import java.io.IOException import java.util.concurrent.locks.ReentrantReadWriteLock import kafka.utils.Utils.{inReadLock,inWriteLock} -import scala.Some import scala.collection.immutable.Set import com.yammer.metrics.core.Gauge @@ -62,13 +61,13 @@ class Partition(val topic: String, private def isReplicaLocal(replicaId: Int) : Boolean = (replicaId == localBrokerId) - newGauge( - topic + "-" + partitionId + "-UnderReplicated", + newGauge("UnderReplicated", new Gauge[Int] { def value = { if (isUnderReplicated) 1 else 0 } - } + }, + Map("topic" -> topic, "partition" -> partitionId.toString) ) def isUnderReplicated(): Boolean = { @@ -345,7 +344,7 @@ class Partition(val topic: String, leaderReplica.highWatermark = newHighWatermark debug("High watermark for partition [%s,%d] updated to %s".format(topic, partitionId, newHighWatermark)) // some delayed requests may be unblocked after HW changed - val requestKey = new TopicPartitionRequestKey(this.topic, this.partitionId) + val requestKey = new TopicAndPartition(this.topic, this.partitionId) replicaManager.tryCompleteDelayedFetch(requestKey) replicaManager.tryCompleteDelayedProduce(requestKey) } else { @@ -415,7 +414,7 @@ class Partition(val topic: String, val info = log.append(messages, assignOffsets = true) // probably unblock some follower fetch requests since log end offset has been updated - replicaManager.tryCompleteDelayedFetch(new TopicPartitionRequestKey(this.topic, this.partitionId)) + replicaManager.tryCompleteDelayedFetch(new TopicAndPartition(this.topic, this.partitionId)) // we may need to increment high watermark since ISR could be down to 1 maybeIncrementLeaderHW(leaderReplica) info diff --git a/core/src/main/scala/kafka/common/AppInfo.scala b/core/src/main/scala/kafka/common/AppInfo.scala new file mode 100644 index 0000000000000..d642ca555f83c --- /dev/null +++ b/core/src/main/scala/kafka/common/AppInfo.scala @@ -0,0 +1,66 @@ +/** + * 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.common + +import java.net.URL +import java.util.jar.{Attributes, Manifest} + +import com.yammer.metrics.core.Gauge +import kafka.metrics.KafkaMetricsGroup + +object AppInfo extends KafkaMetricsGroup { + private var isRegistered = false + private val lock = new Object() + + def registerInfo(): Unit = { + lock.synchronized { + if (isRegistered) { + return + } + } + + try { + val clazz = AppInfo.getClass + val className = clazz.getSimpleName + ".class" + val classPath = clazz.getResource(className).toString + if (!classPath.startsWith("jar")) { + // Class not from JAR + return + } + val manifestPath = classPath.substring(0, classPath.lastIndexOf("!") + 1) + "/META-INF/MANIFEST.MF" + + val mf = new Manifest + mf.read(new URL(manifestPath).openStream()) + val version = mf.getMainAttributes.get(new Attributes.Name("Version")).toString + + newGauge("Version", + new Gauge[String] { + def value = { + version + } + }) + + lock.synchronized { + isRegistered = true + } + } catch { + case e: Exception => + warn("Can't read Kafka version from MANIFEST.MF. Possible cause: %s".format(e)) + } + } +} diff --git a/core/src/main/scala/kafka/common/ClientIdAndBroker.scala b/core/src/main/scala/kafka/common/ClientIdAndBroker.scala index 93223a9c93b55..3b09041d33ac4 100644 --- a/core/src/main/scala/kafka/common/ClientIdAndBroker.scala +++ b/core/src/main/scala/kafka/common/ClientIdAndBroker.scala @@ -8,7 +8,7 @@ package kafka.common * (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 + * 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, @@ -21,6 +21,14 @@ package kafka.common * Convenience case class since (clientId, brokerInfo) pairs are used to create * SyncProducer Request Stats and SimpleConsumer Request and Response Stats. */ -case class ClientIdAndBroker(clientId: String, brokerInfo: String) { - override def toString = "%s-%s".format(clientId, brokerInfo) + +trait ClientIdBroker { +} + +case class ClientIdAndBroker(clientId: String, brokerHost: String, brokerPort: Int) extends ClientIdBroker { + override def toString = "%s-%s-%d".format(clientId, brokerHost, brokerPort) +} + +case class ClientIdAllBrokers(clientId: String) extends ClientIdBroker { + override def toString = "%s-%s".format(clientId, "AllBrokers") } diff --git a/core/src/main/scala/kafka/common/ClientIdAndTopic.scala b/core/src/main/scala/kafka/common/ClientIdAndTopic.scala index 7acf9e76bdd87..5825aad2c8d1a 100644 --- a/core/src/main/scala/kafka/common/ClientIdAndTopic.scala +++ b/core/src/main/scala/kafka/common/ClientIdAndTopic.scala @@ -1,5 +1,3 @@ -package kafka.common - /** * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -17,11 +15,21 @@ package kafka.common * limitations under the License. */ +package kafka.common + /** * Convenience case class since (clientId, topic) pairs are used in the creation * of many Stats objects. */ -case class ClientIdAndTopic(clientId: String, topic: String) { +trait ClientIdTopic { +} + +case class ClientIdAndTopic(clientId: String, topic: String) extends ClientIdTopic { override def toString = "%s-%s".format(clientId, topic) } +case class ClientIdAllTopics(clientId: String) extends ClientIdTopic { + override def toString = "%s-%s".format(clientId, "AllTopics") +} + + diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala index f8c1b4e674f75..ee6139c901082 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala @@ -30,7 +30,7 @@ class ConsumerFetcherThread(name: String, partitionMap: Map[TopicAndPartition, PartitionTopicInfo], val consumerFetcherManager: ConsumerFetcherManager) extends AbstractFetcherThread(name = name, - clientId = config.clientId + "-" + name, + clientId = config.clientId, sourceBroker = sourceBroker, socketTimeout = config.socketTimeoutMs, socketBufferSize = config.socketReceiveBufferBytes, diff --git a/core/src/main/scala/kafka/consumer/ConsumerTopicStats.scala b/core/src/main/scala/kafka/consumer/ConsumerTopicStats.scala index f63e6c59bb1e8..01797ff766a7f 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerTopicStats.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerTopicStats.scala @@ -20,12 +20,17 @@ package kafka.consumer import kafka.utils.{Pool, threadsafe, Logging} import java.util.concurrent.TimeUnit import kafka.metrics.KafkaMetricsGroup -import kafka.common.ClientIdAndTopic +import kafka.common.{ClientIdTopic, ClientIdAllTopics, ClientIdAndTopic} @threadsafe -class ConsumerTopicMetrics(metricId: ClientIdAndTopic) extends KafkaMetricsGroup { - val messageRate = newMeter(metricId + "MessagesPerSec", "messages", TimeUnit.SECONDS) - val byteRate = newMeter(metricId + "BytesPerSec", "bytes", TimeUnit.SECONDS) +class ConsumerTopicMetrics(metricId: ClientIdTopic) extends KafkaMetricsGroup { + val tags = metricId match { + case ClientIdAndTopic(clientId, topic) => Map("clientId" -> clientId, "topic" -> topic) + case ClientIdAllTopics(clientId) => Map("clientId" -> clientId) + } + + val messageRate = newMeter("MessagesPerSec", "messages", TimeUnit.SECONDS, tags) + val byteRate = newMeter("BytesPerSec", "bytes", TimeUnit.SECONDS, tags) } /** @@ -35,12 +40,12 @@ class ConsumerTopicMetrics(metricId: ClientIdAndTopic) extends KafkaMetricsGroup class ConsumerTopicStats(clientId: String) extends Logging { private val valueFactory = (k: ClientIdAndTopic) => new ConsumerTopicMetrics(k) private val stats = new Pool[ClientIdAndTopic, ConsumerTopicMetrics](Some(valueFactory)) - private val allTopicStats = new ConsumerTopicMetrics(new ClientIdAndTopic(clientId, "AllTopics")) // to differentiate from a topic named AllTopics + private val allTopicStats = new ConsumerTopicMetrics(new ClientIdAllTopics(clientId)) // to differentiate from a topic named AllTopics def getConsumerAllTopicStats(): ConsumerTopicMetrics = allTopicStats def getConsumerTopicStats(topic: String): ConsumerTopicMetrics = { - stats.getAndMaybePut(new ClientIdAndTopic(clientId, topic + "-")) + stats.getAndMaybePut(new ClientIdAndTopic(clientId, topic)) } } diff --git a/core/src/main/scala/kafka/consumer/FetchRequestAndResponseStats.scala b/core/src/main/scala/kafka/consumer/FetchRequestAndResponseStats.scala index 5243f415288c6..3df55e13001ce 100644 --- a/core/src/main/scala/kafka/consumer/FetchRequestAndResponseStats.scala +++ b/core/src/main/scala/kafka/consumer/FetchRequestAndResponseStats.scala @@ -19,13 +19,21 @@ package kafka.consumer import java.util.concurrent.TimeUnit -import kafka.common.ClientIdAndBroker +import kafka.common.{ClientIdAllBrokers, ClientIdBroker, ClientIdAndBroker} import kafka.metrics.{KafkaMetricsGroup, KafkaTimer} import kafka.utils.Pool -class FetchRequestAndResponseMetrics(metricId: ClientIdAndBroker) extends KafkaMetricsGroup { - val requestTimer = new KafkaTimer(newTimer(metricId + "FetchRequestRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS)) - val requestSizeHist = newHistogram(metricId + "FetchResponseSize") +class FetchRequestAndResponseMetrics(metricId: ClientIdBroker) extends KafkaMetricsGroup { + val tags = metricId match { + case ClientIdAndBroker(clientId, brokerHost, brokerPort) => + Map("clientId" -> clientId, "brokerHost" -> brokerHost, + "brokerPort" -> brokerPort.toString) + case ClientIdAllBrokers(clientId) => + Map("clientId" -> clientId) + } + + val requestTimer = new KafkaTimer(newTimer("FetchRequestRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS, tags)) + val requestSizeHist = newHistogram("FetchResponseSize", biased = true, tags) } /** @@ -33,14 +41,14 @@ class FetchRequestAndResponseMetrics(metricId: ClientIdAndBroker) extends KafkaM * @param clientId ClientId of the given consumer */ class FetchRequestAndResponseStats(clientId: String) { - private val valueFactory = (k: ClientIdAndBroker) => new FetchRequestAndResponseMetrics(k) - private val stats = new Pool[ClientIdAndBroker, FetchRequestAndResponseMetrics](Some(valueFactory)) - private val allBrokersStats = new FetchRequestAndResponseMetrics(new ClientIdAndBroker(clientId, "AllBrokers")) + private val valueFactory = (k: ClientIdBroker) => new FetchRequestAndResponseMetrics(k) + private val stats = new Pool[ClientIdBroker, FetchRequestAndResponseMetrics](Some(valueFactory)) + private val allBrokersStats = new FetchRequestAndResponseMetrics(new ClientIdAllBrokers(clientId)) def getFetchRequestAndResponseAllBrokersStats(): FetchRequestAndResponseMetrics = allBrokersStats - def getFetchRequestAndResponseStats(brokerInfo: String): FetchRequestAndResponseMetrics = { - stats.getAndMaybePut(new ClientIdAndBroker(clientId, brokerInfo + "-")) + def getFetchRequestAndResponseStats(brokerHost: String, brokerPort: Int): FetchRequestAndResponseMetrics = { + stats.getAndMaybePut(new ClientIdAndBroker(clientId, brokerHost, brokerPort)) } } @@ -56,7 +64,7 @@ object FetchRequestAndResponseStatsRegistry { } def removeConsumerFetchRequestAndResponseStats(clientId: String) { - val pattern = (clientId + "-ConsumerFetcherThread.*").r + val pattern = (".*" + clientId + ".*").r val keys = globalStats.keys for (key <- keys) { pattern.findFirstIn(key) match { diff --git a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala index d349a3000feb9..e53ee51638b93 100644 --- a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala @@ -36,7 +36,6 @@ class SimpleConsumer(val host: String, ConsumerConfig.validateClientId(clientId) private val lock = new Object() private val blockingChannel = new BlockingChannel(host, port, bufferSize, BlockingChannel.UseDefaultBufferSize, soTimeout) - val brokerInfo = "host_%s-port_%s".format(host, port) private val fetchRequestAndResponseStats = FetchRequestAndResponseStatsRegistry.getFetchRequestAndResponseStats(clientId) private var isClosed = false @@ -106,7 +105,7 @@ class SimpleConsumer(val host: String, */ def fetch(request: FetchRequest): FetchResponse = { var response: Receive = null - val specificTimer = fetchRequestAndResponseStats.getFetchRequestAndResponseStats(brokerInfo).requestTimer + val specificTimer = fetchRequestAndResponseStats.getFetchRequestAndResponseStats(host, port).requestTimer val aggregateTimer = fetchRequestAndResponseStats.getFetchRequestAndResponseAllBrokersStats.requestTimer aggregateTimer.time { specificTimer.time { @@ -115,7 +114,7 @@ class SimpleConsumer(val host: String, } val fetchResponse = FetchResponse.readFrom(response.buffer) val fetchedSize = fetchResponse.sizeInBytes - fetchRequestAndResponseStats.getFetchRequestAndResponseStats(brokerInfo).requestSizeHist.update(fetchedSize) + fetchRequestAndResponseStats.getFetchRequestAndResponseStats(host, port).requestSizeHist.update(fetchedSize) fetchRequestAndResponseStats.getFetchRequestAndResponseAllBrokersStats.requestSizeHist.update(fetchedSize) fetchResponse } diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index 4b6dc072b2d89..3e1718bc7ca6c 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -104,9 +104,9 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private var wildcardTopicWatcher: ZookeeperTopicEventWatcher = null // useful for tracking migration of consumers to store offsets in kafka - private val kafkaCommitMeter = newMeter(config.clientId + "-KafkaCommitsPerSec", "commits", TimeUnit.SECONDS) - private val zkCommitMeter = newMeter(config.clientId + "-ZooKeeperCommitsPerSec", "commits", TimeUnit.SECONDS) - private val rebalanceTimer = new KafkaTimer(newTimer(config.clientId + "-RebalanceRateAndTime", TimeUnit.MILLISECONDS, TimeUnit.SECONDS)) + private val kafkaCommitMeter = newMeter("KafkaCommitsPerSec", "commits", TimeUnit.SECONDS, Map("clientId" -> config.clientId)) + private val zkCommitMeter = newMeter("ZooKeeperCommitsPerSec", "commits", TimeUnit.SECONDS, Map("clientId" -> config.clientId)) + private val rebalanceTimer = new KafkaTimer(newTimer("RebalanceRateAndTime", TimeUnit.MILLISECONDS, TimeUnit.SECONDS, Map("clientId" -> config.clientId))) val consumerIdString = { var consumerUuid : String = null @@ -138,6 +138,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } KafkaMetricsReporter.startReporters(config.props) + AppInfo.registerInfo() def this(config: ConsumerConfig) = this(config, true) @@ -521,14 +522,15 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private var isWatcherTriggered = false private val lock = new ReentrantLock private val cond = lock.newCondition() - + @volatile private var allTopicsOwnedPartitionsCount = 0 - newGauge(config.clientId + "-" + config.groupId + "-AllTopicsOwnedPartitionsCount", new Gauge[Int] { - def value() = allTopicsOwnedPartitionsCount - }) + newGauge("OwnedPartitionsCount", + new Gauge[Int] { + def value() = allTopicsOwnedPartitionsCount + }, + Map("clientId" -> config.clientId, "groupId" -> config.groupId)) - private def ownedPartitionsCountMetricName(topic: String) = - "%s-%s-%s-OwnedPartitionsCount".format(config.clientId, config.groupId, topic) + private def ownedPartitionsCountMetricTags(topic: String) = Map("clientId" -> config.clientId, "groupId" -> config.groupId, "topic" -> topic) private val watcherExecutorThread = new Thread(consumerIdString + "_watcher_executor") { override def run() { @@ -581,7 +583,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, for(partition <- infos.keys) { deletePartitionOwnershipFromZK(topic, partition) } - removeMetric(ownedPartitionsCountMetricName(topic)) + removeMetric("OwnedPartitionsCount", ownedPartitionsCountMetricTags(topic)) localTopicRegistry.remove(topic) } allTopicsOwnedPartitionsCount = 0 @@ -684,9 +686,11 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, partitionOwnershipDecision.view.groupBy { case(topicPartition, consumerThreadId) => topicPartition.topic } .foreach { case (topic, partitionThreadPairs) => - newGauge(ownedPartitionsCountMetricName(topic), new Gauge[Int] { - def value() = partitionThreadPairs.size - }) + newGauge("OwnedPartitionsCount", + new Gauge[Int] { + def value() = partitionThreadPairs.size + }, + ownedPartitionsCountMetricTags(topic)) } topicRegistry = currentTopicRegistry @@ -868,10 +872,13 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, topicThreadIdAndQueues.put(topicThreadId, q) debug("Adding topicThreadId %s and queue %s to topicThreadIdAndQueues data structure".format(topicThreadId, q.toString)) newGauge( - config.clientId + "-" + config.groupId + "-" + topicThreadId._1 + "-" + topicThreadId._2 + "-FetchQueueSize", + "FetchQueueSize", new Gauge[Int] { def value = q.size - } + }, + Map("clientId" -> config.clientId, + "topic" -> topicThreadId._1, + "threadId" -> topicThreadId._2.threadId.toString) ) }) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 37b4a85cca06a..4fae2f0d339b2 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -88,17 +88,31 @@ class Log(val dir: File, info("Completed load of log %s with log end offset %d".format(name, logEndOffset)) - newGauge(name + "-" + "NumLogSegments", - new Gauge[Int] { def value = numberOfSegments }) - - newGauge(name + "-" + "LogStartOffset", - new Gauge[Long] { def value = logStartOffset }) - - newGauge(name + "-" + "LogEndOffset", - new Gauge[Long] { def value = logEndOffset }) - - newGauge(name + "-" + "Size", - new Gauge[Long] {def value = size}) + val tags = Map("topic" -> topicAndPartition.topic, "partition" -> topicAndPartition.partition.toString) + + newGauge("NumLogSegments", + new Gauge[Int] { + def value = numberOfSegments + }, + tags) + + newGauge("LogStartOffset", + new Gauge[Long] { + def value = logStartOffset + }, + tags) + + newGauge("LogEndOffset", + new Gauge[Long] { + def value = logEndOffset + }, + tags) + + newGauge("Size", + new Gauge[Long] { + def value = size + }, + tags) /** The name of this log */ def name = dir.getName() @@ -168,7 +182,7 @@ class Log(val dir: File, if(logSegments.size == 0) { // no existing segments, create a new mutable segment beginning at offset 0 - segments.put(0, new LogSegment(dir = dir, + segments.put(0L, new LogSegment(dir = dir, startOffset = 0, indexIntervalBytes = config.indexInterval, maxIndexSize = config.maxIndexSize, diff --git a/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala b/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala index 2313a57d02c06..e9e49180f6de4 100644 --- a/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala +++ b/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala @@ -6,7 +6,7 @@ * (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 + * 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, @@ -35,29 +35,52 @@ trait KafkaMetricsGroup extends Logging { * Creates a new MetricName object for gauges, meters, etc. created for this * metrics group. * @param name Descriptive name of the metric. + * @param tags Additional attributes which mBean will have. * @return Sanitized metric name object. */ - private def metricName(name: String) = { + private def metricName(name: String, tags: scala.collection.Map[String, String] = Map.empty) = { val klass = this.getClass val pkg = if (klass.getPackage == null) "" else klass.getPackage.getName val simpleName = klass.getSimpleName.replaceAll("\\$$", "") - new MetricName(pkg, simpleName, name) + + explicitMetricName(pkg, simpleName, name, tags) } - def newGauge[T](name: String, metric: Gauge[T]) = - Metrics.defaultRegistry().newGauge(metricName(name), metric) - def newMeter(name: String, eventType: String, timeUnit: TimeUnit) = - Metrics.defaultRegistry().newMeter(metricName(name), eventType, timeUnit) + private def explicitMetricName(group: String, typeName: String, name: String, tags: scala.collection.Map[String, String] = Map.empty) = { + val nameBuilder: StringBuilder = new StringBuilder + + nameBuilder.append(group) + + nameBuilder.append(":type=") + + nameBuilder.append(typeName) - def newHistogram(name: String, biased: Boolean = true) = - Metrics.defaultRegistry().newHistogram(metricName(name), biased) + if (name.length > 0) { + nameBuilder.append(",name=") + nameBuilder.append(name) + } + + KafkaMetricsGroup.toMBeanName(tags).map(mbeanName => nameBuilder.append(",").append(mbeanName)) + + new MetricName(group, typeName, name, null, nameBuilder.toString()) + } - def newTimer(name: String, durationUnit: TimeUnit, rateUnit: TimeUnit) = - Metrics.defaultRegistry().newTimer(metricName(name), durationUnit, rateUnit) + def newGauge[T](name: String, metric: Gauge[T], tags: scala.collection.Map[String, String] = Map.empty) = + Metrics.defaultRegistry().newGauge(metricName(name, tags), metric) + + def newMeter(name: String, eventType: String, timeUnit: TimeUnit, tags: scala.collection.Map[String, String] = Map.empty) = + Metrics.defaultRegistry().newMeter(metricName(name, tags), eventType, timeUnit) + + def newHistogram(name: String, biased: Boolean = true, tags: scala.collection.Map[String, String] = Map.empty) = + Metrics.defaultRegistry().newHistogram(metricName(name, tags), biased) + + def newTimer(name: String, durationUnit: TimeUnit, rateUnit: TimeUnit, tags: scala.collection.Map[String, String] = Map.empty) = + Metrics.defaultRegistry().newTimer(metricName(name, tags), durationUnit, rateUnit) + + def removeMetric(name: String, tags: scala.collection.Map[String, String] = Map.empty) = + Metrics.defaultRegistry().removeMetric(metricName(name, tags)) - def removeMetric(name: String) = - Metrics.defaultRegistry().removeMetric(metricName(name)) } @@ -68,72 +91,75 @@ object KafkaMetricsGroup extends KafkaMetricsGroup with Logging { */ private val consumerMetricNameList: immutable.List[MetricName] = immutable.List[MetricName]( // kafka.consumer.ZookeeperConsumerConnector - new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "-FetchQueueSize"), - new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "-KafkaCommitsPerSec"), - new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "-ZooKeeperCommitsPerSec"), - new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "-RebalanceRateAndTime"), - new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "-OwnedPartitionsCount"), - new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "AllTopicsOwnedPartitionsCount"), + new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "FetchQueueSize"), + new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "KafkaCommitsPerSec"), + new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "ZooKeeperCommitsPerSec"), + new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "RebalanceRateAndTime"), + new MetricName("kafka.consumer", "ZookeeperConsumerConnector", "OwnedPartitionsCount"), // kafka.consumer.ConsumerFetcherManager - new MetricName("kafka.consumer", "ConsumerFetcherManager", "-MaxLag"), - new MetricName("kafka.consumer", "ConsumerFetcherManager", "-MinFetchRate"), + new MetricName("kafka.consumer", "ConsumerFetcherManager", "MaxLag"), + new MetricName("kafka.consumer", "ConsumerFetcherManager", "MinFetchRate"), // kafka.server.AbstractFetcherThread <-- kafka.consumer.ConsumerFetcherThread - new MetricName("kafka.server", "FetcherLagMetrics", "-ConsumerLag"), + new MetricName("kafka.server", "FetcherLagMetrics", "ConsumerLag"), // kafka.consumer.ConsumerTopicStats <-- kafka.consumer.{ConsumerIterator, PartitionTopicInfo} - new MetricName("kafka.consumer", "ConsumerTopicMetrics", "-MessagesPerSec"), - new MetricName("kafka.consumer", "ConsumerTopicMetrics", "-AllTopicsMessagesPerSec"), + new MetricName("kafka.consumer", "ConsumerTopicMetrics", "MessagesPerSec"), // kafka.consumer.ConsumerTopicStats - new MetricName("kafka.consumer", "ConsumerTopicMetrics", "-BytesPerSec"), - new MetricName("kafka.consumer", "ConsumerTopicMetrics", "-AllTopicsBytesPerSec"), + new MetricName("kafka.consumer", "ConsumerTopicMetrics", "BytesPerSec"), // kafka.server.AbstractFetcherThread <-- kafka.consumer.ConsumerFetcherThread - new MetricName("kafka.server", "FetcherStats", "-BytesPerSec"), - new MetricName("kafka.server", "FetcherStats", "-RequestsPerSec"), + new MetricName("kafka.server", "FetcherStats", "BytesPerSec"), + new MetricName("kafka.server", "FetcherStats", "RequestsPerSec"), // kafka.consumer.FetchRequestAndResponseStats <-- kafka.consumer.SimpleConsumer - new MetricName("kafka.consumer", "FetchRequestAndResponseMetrics", "-FetchResponseSize"), - new MetricName("kafka.consumer", "FetchRequestAndResponseMetrics", "-FetchRequestRateAndTimeMs"), - new MetricName("kafka.consumer", "FetchRequestAndResponseMetrics", "-AllBrokersFetchResponseSize"), - new MetricName("kafka.consumer", "FetchRequestAndResponseMetrics", "-AllBrokersFetchRequestRateAndTimeMs"), + new MetricName("kafka.consumer", "FetchRequestAndResponseMetrics", "FetchResponseSize"), + new MetricName("kafka.consumer", "FetchRequestAndResponseMetrics", "FetchRequestRateAndTimeMs"), /** * ProducerRequestStats <-- SyncProducer * metric for SyncProducer in fetchTopicMetaData() needs to be removed when consumer is closed. */ - new MetricName("kafka.producer", "ProducerRequestMetrics", "-ProducerRequestRateAndTimeMs"), - new MetricName("kafka.producer", "ProducerRequestMetrics", "-ProducerRequestSize"), - new MetricName("kafka.producer", "ProducerRequestMetrics", "-AllBrokersProducerRequestRateAndTimeMs"), - new MetricName("kafka.producer", "ProducerRequestMetrics", "-AllBrokersProducerRequestSize") + new MetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestRateAndTimeMs"), + new MetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestSize") ) - private val producerMetricNameList: immutable.List[MetricName] = immutable.List[MetricName] ( + private val producerMetricNameList: immutable.List[MetricName] = immutable.List[MetricName]( // kafka.producer.ProducerStats <-- DefaultEventHandler <-- Producer - new MetricName("kafka.producer", "ProducerStats", "-SerializationErrorsPerSec"), - new MetricName("kafka.producer", "ProducerStats", "-ResendsPerSec"), - new MetricName("kafka.producer", "ProducerStats", "-FailedSendsPerSec"), + new MetricName("kafka.producer", "ProducerStats", "SerializationErrorsPerSec"), + new MetricName("kafka.producer", "ProducerStats", "ResendsPerSec"), + new MetricName("kafka.producer", "ProducerStats", "FailedSendsPerSec"), // kafka.producer.ProducerSendThread - new MetricName("kafka.producer.async", "ProducerSendThread", "-ProducerQueueSize"), + new MetricName("kafka.producer.async", "ProducerSendThread", "ProducerQueueSize"), // kafka.producer.ProducerTopicStats <-- kafka.producer.{Producer, async.DefaultEventHandler} - new MetricName("kafka.producer", "ProducerTopicMetrics", "-MessagesPerSec"), - new MetricName("kafka.producer", "ProducerTopicMetrics", "-DroppedMessagesPerSec"), - new MetricName("kafka.producer", "ProducerTopicMetrics", "-BytesPerSec"), - new MetricName("kafka.producer", "ProducerTopicMetrics", "-AllTopicsMessagesPerSec"), - new MetricName("kafka.producer", "ProducerTopicMetrics", "-AllTopicsDroppedMessagesPerSec"), - new MetricName("kafka.producer", "ProducerTopicMetrics", "-AllTopicsBytesPerSec"), + new MetricName("kafka.producer", "ProducerTopicMetrics", "MessagesPerSec"), + new MetricName("kafka.producer", "ProducerTopicMetrics", "DroppedMessagesPerSec"), + new MetricName("kafka.producer", "ProducerTopicMetrics", "BytesPerSec"), // kafka.producer.ProducerRequestStats <-- SyncProducer - new MetricName("kafka.producer", "ProducerRequestMetrics", "-ProducerRequestRateAndTimeMs"), - new MetricName("kafka.producer", "ProducerRequestMetrics", "-ProducerRequestSize"), - new MetricName("kafka.producer", "ProducerRequestMetrics", "-AllBrokersProducerRequestRateAndTimeMs"), - new MetricName("kafka.producer", "ProducerRequestMetrics", "-AllBrokersProducerRequestSize") + new MetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestRateAndTimeMs"), + new MetricName("kafka.producer", "ProducerRequestMetrics", "ProducerRequestSize") ) + private def toMBeanName(tags: collection.Map[String, String]): Option[String] = { + val filteredTags = tags + .filter { case (tagKey, tagValue) => tagValue != ""} + if (filteredTags.nonEmpty) { + val tagsString = filteredTags + .map { case (key, value) => "%s=%s".format(key, value)} + .mkString(",") + + Some(tagsString) + } + else { + None + } + } + def removeAllConsumerMetrics(clientId: String) { FetchRequestAndResponseStatsRegistry.removeConsumerFetchRequestAndResponseStats(clientId) ConsumerTopicStatsRegistry.removeConsumerTopicStat(clientId) @@ -150,18 +176,19 @@ object KafkaMetricsGroup extends KafkaMetricsGroup with Logging { private def removeAllMetricsInList(metricNameList: immutable.List[MetricName], clientId: String) { metricNameList.foreach(metric => { - val pattern = (clientId + ".*" + metric.getName +".*").r + val pattern = (".*clientId=" + clientId + ".*").r val registeredMetrics = scala.collection.JavaConversions.asScalaSet(Metrics.defaultRegistry().allMetrics().keySet()) for (registeredMetric <- registeredMetrics) { if (registeredMetric.getGroup == metric.getGroup && + registeredMetric.getName == metric.getName && registeredMetric.getType == metric.getType) { - pattern.findFirstIn(registeredMetric.getName) match { + pattern.findFirstIn(registeredMetric.getMBeanName) match { case Some(_) => { val beforeRemovalSize = Metrics.defaultRegistry().allMetrics().keySet().size Metrics.defaultRegistry().removeMetric(registeredMetric) val afterRemovalSize = Metrics.defaultRegistry().allMetrics().keySet().size trace("Removing metric %s. Metrics registry size reduced from %d to %d".format( - registeredMetric, beforeRemovalSize, afterRemovalSize)) + registeredMetric, beforeRemovalSize, afterRemovalSize)) } case _ => } diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index 4560d8fb7dbfe..7b1db3dbbb2c0 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -125,12 +125,12 @@ class RequestChannel(val numProcessors: Int, val queueSize: Int) extends KafkaMe def value = responseQueues.foldLeft(0) {(total, q) => total + q.size()} }) - for(i <- 0 until numProcessors) { - newGauge( - "Processor-" + i + "-ResponseQueueSize", + for (i <- 0 until numProcessors) { + newGauge("ResponseQueueSize", new Gauge[Int] { def value = responseQueues(i).size() - } + }, + Map("processor" -> i.toString) ) } @@ -187,24 +187,25 @@ class RequestChannel(val numProcessors: Int, val queueSize: Int) extends KafkaMe object RequestMetrics { val metricsMap = new scala.collection.mutable.HashMap[String, RequestMetrics] - val consumerFetchMetricName = RequestKeys.nameForKey(RequestKeys.FetchKey) + "-Consumer" - val followFetchMetricName = RequestKeys.nameForKey(RequestKeys.FetchKey) + "-Follower" + val consumerFetchMetricName = RequestKeys.nameForKey(RequestKeys.FetchKey) + "Consumer" + val followFetchMetricName = RequestKeys.nameForKey(RequestKeys.FetchKey) + "Follower" (RequestKeys.keyToNameAndDeserializerMap.values.map(e => e._1) ++ List(consumerFetchMetricName, followFetchMetricName)).foreach(name => metricsMap.put(name, new RequestMetrics(name))) } class RequestMetrics(name: String) extends KafkaMetricsGroup { - val requestRate = newMeter(name + "-RequestsPerSec", "requests", TimeUnit.SECONDS) + val tags = Map("request" -> name) + val requestRate = newMeter("RequestsPerSec", "requests", TimeUnit.SECONDS, tags) // time a request spent in a request queue - val requestQueueTimeHist = newHistogram(name + "-RequestQueueTimeMs") + val requestQueueTimeHist = newHistogram("RequestQueueTimeMs", biased = true, tags) // time a request takes to be processed at the local broker - val localTimeHist = newHistogram(name + "-LocalTimeMs") + val localTimeHist = newHistogram("LocalTimeMs", biased = true, tags) // time a request takes to wait on remote brokers (only relevant to fetch and produce requests) - val remoteTimeHist = newHistogram(name + "-RemoteTimeMs") + val remoteTimeHist = newHistogram("RemoteTimeMs", biased = true, tags) // time a response spent in a response queue - val responseQueueTimeHist = newHistogram(name + "-ResponseQueueTimeMs") + val responseQueueTimeHist = newHistogram("ResponseQueueTimeMs", biased = true, tags) // time to send the response to the requester - val responseSendTimeHist = newHistogram(name + "-ResponseSendTimeMs") - val totalTimeHist = newHistogram(name + "-TotalTimeMs") + val responseSendTimeHist = newHistogram("ResponseSendTimeMs", biased = true, tags) + val totalTimeHist = newHistogram("TotalTimeMs", biased = true, tags) } diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index cee76b323e5f3..e451592fe3581 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -67,7 +67,7 @@ class SocketServer(val brokerId: Int, time, maxRequestSize, aggregateIdleMeter, - newMeter("NetworkProcessor-" + i + "-IdlePercent", "percent", TimeUnit.NANOSECONDS), + newMeter("IdlePercent", "percent", TimeUnit.NANOSECONDS, Map("networkProcessor" -> i.toString)), numProcessorThreads, requestChannel, quotas, diff --git a/core/src/main/scala/kafka/producer/Producer.scala b/core/src/main/scala/kafka/producer/Producer.scala index cd634f653caa9..e38d2fa7ec873 100644 --- a/core/src/main/scala/kafka/producer/Producer.scala +++ b/core/src/main/scala/kafka/producer/Producer.scala @@ -19,7 +19,7 @@ package kafka.producer import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.{LinkedBlockingQueue, TimeUnit} -import kafka.common.QueueFullException +import kafka.common.{AppInfo, QueueFullException} import kafka.metrics._ import kafka.producer.async.{DefaultEventHandler, EventHandler, ProducerSendThread} import kafka.serializer.Encoder @@ -53,6 +53,7 @@ class Producer[K,V](val config: ProducerConfig, private val producerTopicStats = ProducerTopicStatsRegistry.getProducerTopicStats(config.clientId) KafkaMetricsReporter.startReporters(config.props) + AppInfo.registerInfo() def this(config: ProducerConfig) = this(config, diff --git a/core/src/main/scala/kafka/producer/ProducerRequestStats.scala b/core/src/main/scala/kafka/producer/ProducerRequestStats.scala index 1c46d729d8259..026e93a2f1dcc 100644 --- a/core/src/main/scala/kafka/producer/ProducerRequestStats.scala +++ b/core/src/main/scala/kafka/producer/ProducerRequestStats.scala @@ -19,11 +19,16 @@ package kafka.producer import kafka.metrics.{KafkaTimer, KafkaMetricsGroup} import java.util.concurrent.TimeUnit import kafka.utils.Pool -import kafka.common.ClientIdAndBroker +import kafka.common.{ClientIdAllBrokers, ClientIdBroker, ClientIdAndBroker} -class ProducerRequestMetrics(metricId: ClientIdAndBroker) extends KafkaMetricsGroup { - val requestTimer = new KafkaTimer(newTimer(metricId + "ProducerRequestRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS)) - val requestSizeHist = newHistogram(metricId + "ProducerRequestSize") +class ProducerRequestMetrics(metricId: ClientIdBroker) extends KafkaMetricsGroup { + val tags = metricId match { + case ClientIdAndBroker(clientId, brokerHost, brokerPort) => Map("clientId" -> clientId, "brokerHost" -> brokerHost, "brokerPort" -> brokerPort.toString) + case ClientIdAllBrokers(clientId) => Map("clientId" -> clientId) + } + + val requestTimer = new KafkaTimer(newTimer("ProducerRequestRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS, tags)) + val requestSizeHist = newHistogram("ProducerRequestSize", biased = true, tags) } /** @@ -31,14 +36,14 @@ class ProducerRequestMetrics(metricId: ClientIdAndBroker) extends KafkaMetricsGr * @param clientId ClientId of the given producer */ class ProducerRequestStats(clientId: String) { - private val valueFactory = (k: ClientIdAndBroker) => new ProducerRequestMetrics(k) - private val stats = new Pool[ClientIdAndBroker, ProducerRequestMetrics](Some(valueFactory)) - private val allBrokersStats = new ProducerRequestMetrics(new ClientIdAndBroker(clientId, "AllBrokers")) + private val valueFactory = (k: ClientIdBroker) => new ProducerRequestMetrics(k) + private val stats = new Pool[ClientIdBroker, ProducerRequestMetrics](Some(valueFactory)) + private val allBrokersStats = new ProducerRequestMetrics(new ClientIdAllBrokers(clientId)) def getProducerRequestAllBrokersStats(): ProducerRequestMetrics = allBrokersStats - def getProducerRequestStats(brokerInfo: String): ProducerRequestMetrics = { - stats.getAndMaybePut(new ClientIdAndBroker(clientId, brokerInfo + "-")) + def getProducerRequestStats(brokerHost: String, brokerPort: Int): ProducerRequestMetrics = { + stats.getAndMaybePut(new ClientIdAndBroker(clientId, brokerHost, brokerPort)) } } diff --git a/core/src/main/scala/kafka/producer/ProducerStats.scala b/core/src/main/scala/kafka/producer/ProducerStats.scala index 35e3aae2f81e5..1d0fa888c99a5 100644 --- a/core/src/main/scala/kafka/producer/ProducerStats.scala +++ b/core/src/main/scala/kafka/producer/ProducerStats.scala @@ -21,9 +21,10 @@ import java.util.concurrent.TimeUnit import kafka.utils.Pool class ProducerStats(clientId: String) extends KafkaMetricsGroup { - val serializationErrorRate = newMeter(clientId + "-SerializationErrorsPerSec", "errors", TimeUnit.SECONDS) - val resendRate = newMeter(clientId + "-ResendsPerSec", "resends", TimeUnit.SECONDS) - val failedSendRate = newMeter(clientId + "-FailedSendsPerSec", "failed sends", TimeUnit.SECONDS) + val tags: Map[String, String] = Map("clientId" -> clientId) + val serializationErrorRate = newMeter("SerializationErrorsPerSec", "errors", TimeUnit.SECONDS, tags) + val resendRate = newMeter("ResendsPerSec", "resends", TimeUnit.SECONDS, tags) + val failedSendRate = newMeter("FailedSendsPerSec", "failed sends", TimeUnit.SECONDS, tags) } /** diff --git a/core/src/main/scala/kafka/producer/ProducerTopicStats.scala b/core/src/main/scala/kafka/producer/ProducerTopicStats.scala index 9bb1419dcc45f..97594c8313672 100644 --- a/core/src/main/scala/kafka/producer/ProducerTopicStats.scala +++ b/core/src/main/scala/kafka/producer/ProducerTopicStats.scala @@ -17,16 +17,21 @@ package kafka.producer import kafka.metrics.KafkaMetricsGroup -import kafka.common.ClientIdAndTopic +import kafka.common.{ClientIdTopic, ClientIdAllTopics, ClientIdAndTopic} import kafka.utils.{Pool, threadsafe} import java.util.concurrent.TimeUnit @threadsafe -class ProducerTopicMetrics(metricId: ClientIdAndTopic) extends KafkaMetricsGroup { - val messageRate = newMeter(metricId + "MessagesPerSec", "messages", TimeUnit.SECONDS) - val byteRate = newMeter(metricId + "BytesPerSec", "bytes", TimeUnit.SECONDS) - val droppedMessageRate = newMeter(metricId + "DroppedMessagesPerSec", "drops", TimeUnit.SECONDS) +class ProducerTopicMetrics(metricId: ClientIdTopic) extends KafkaMetricsGroup { + val tags = metricId match { + case ClientIdAndTopic(clientId, topic) => Map("clientId" -> clientId, "topic" -> topic) + case ClientIdAllTopics(clientId) => Map("clientId" -> clientId) + } + + val messageRate = newMeter("MessagesPerSec", "messages", TimeUnit.SECONDS, tags) + val byteRate = newMeter("BytesPerSec", "bytes", TimeUnit.SECONDS, tags) + val droppedMessageRate = newMeter("DroppedMessagesPerSec", "drops", TimeUnit.SECONDS, tags) } /** @@ -34,14 +39,14 @@ class ProducerTopicMetrics(metricId: ClientIdAndTopic) extends KafkaMetricsGroup * @param clientId The clientId of the given producer client. */ class ProducerTopicStats(clientId: String) { - private val valueFactory = (k: ClientIdAndTopic) => new ProducerTopicMetrics(k) - private val stats = new Pool[ClientIdAndTopic, ProducerTopicMetrics](Some(valueFactory)) - private val allTopicsStats = new ProducerTopicMetrics(new ClientIdAndTopic(clientId, "AllTopics")) // to differentiate from a topic named AllTopics + private val valueFactory = (k: ClientIdTopic) => new ProducerTopicMetrics(k) + private val stats = new Pool[ClientIdTopic, ProducerTopicMetrics](Some(valueFactory)) + private val allTopicsStats = new ProducerTopicMetrics(new ClientIdAllTopics(clientId)) // to differentiate from a topic named AllTopics def getProducerAllTopicsStats(): ProducerTopicMetrics = allTopicsStats def getProducerTopicStats(topic: String): ProducerTopicMetrics = { - stats.getAndMaybePut(new ClientIdAndTopic(clientId, topic + "-")) + stats.getAndMaybePut(new ClientIdAndTopic(clientId, topic)) } } diff --git a/core/src/main/scala/kafka/producer/SyncProducer.scala b/core/src/main/scala/kafka/producer/SyncProducer.scala index 35e9e8cdb385a..0f09951329a8a 100644 --- a/core/src/main/scala/kafka/producer/SyncProducer.scala +++ b/core/src/main/scala/kafka/producer/SyncProducer.scala @@ -5,7 +5,7 @@ * 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 @@ -39,7 +39,6 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { @volatile private var shutdown: Boolean = false private val blockingChannel = new BlockingChannel(config.host, config.port, BlockingChannel.UseDefaultBufferSize, config.sendBufferBytes, config.requestTimeoutMs) - val brokerInfo = "host_%s-port_%s".format(config.host, config.port) val producerRequestStats = ProducerRequestStatsRegistry.getProducerRequestStats(config.clientId) trace("Instantiating Scala Sync Producer with properties: %s".format(config.props)) @@ -93,11 +92,11 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { */ def send(producerRequest: ProducerRequest): ProducerResponse = { val requestSize = producerRequest.sizeInBytes - producerRequestStats.getProducerRequestStats(brokerInfo).requestSizeHist.update(requestSize) + producerRequestStats.getProducerRequestStats(config.host, config.port).requestSizeHist.update(requestSize) producerRequestStats.getProducerRequestAllBrokersStats.requestSizeHist.update(requestSize) var response: Receive = null - val specificTimer = producerRequestStats.getProducerRequestStats(brokerInfo).requestTimer + val specificTimer = producerRequestStats.getProducerRequestStats(config.host, config.port).requestTimer val aggregateTimer = producerRequestStats.getProducerRequestAllBrokersStats.requestTimer aggregateTimer.time { specificTimer.time { @@ -134,7 +133,7 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { case e: Exception => error("Error on disconnect: ", e) } } - + private def connect(): BlockingChannel = { if (!blockingChannel.isConnected && !shutdown) { try { @@ -156,5 +155,4 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { connect() } } -} - +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala b/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala index 42e9c741c2dce..2ccf82a6f1ed6 100644 --- a/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala +++ b/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala @@ -34,10 +34,11 @@ class ProducerSendThread[K,V](val threadName: String, private val shutdownLatch = new CountDownLatch(1) private val shutdownCommand = new KeyedMessage[K,V]("shutdown", null.asInstanceOf[K], null.asInstanceOf[V]) - newGauge(clientId + "-ProducerQueueSize", + newGauge("ProducerQueueSize", new Gauge[Int] { def value = queue.size - }) + }, + Map("clientId" -> clientId)) override def run { try { diff --git a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala index 9390edf37dae9..20c00cb8cc235 100644 --- a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala @@ -26,7 +26,7 @@ import kafka.metrics.KafkaMetricsGroup import kafka.common.TopicAndPartition import com.yammer.metrics.core.Gauge -abstract class AbstractFetcherManager(protected val name: String, metricPrefix: String, numFetchers: Int = 1) +abstract class AbstractFetcherManager(protected val name: String, clientId: String, numFetchers: Int = 1) extends Logging with KafkaMetricsGroup { // map of (source broker_id, fetcher_id per source broker) => fetcher private val fetcherThreadMap = new mutable.HashMap[BrokerAndFetcherId, AbstractFetcherThread] @@ -34,7 +34,7 @@ abstract class AbstractFetcherManager(protected val name: String, metricPrefix: this.logIdent = "[" + name + "] " newGauge( - metricPrefix + "-MaxLag", + "MaxLag", new Gauge[Long] { // current max lag across all fetchers/topics/partitions def value = fetcherThreadMap.foldLeft(0L)((curMaxAll, fetcherThreadMapEntry) => { @@ -42,24 +42,25 @@ abstract class AbstractFetcherManager(protected val name: String, metricPrefix: curMaxThread.max(fetcherLagStatsEntry._2.lag) }).max(curMaxAll) }) - } + }, + Map("clientId" -> clientId) ) newGauge( - metricPrefix + "-MinFetchRate", - { - new Gauge[Double] { - // current min fetch rate across all fetchers/topics/partitions - def value = { - val headRate: Double = - fetcherThreadMap.headOption.map(_._2.fetcherStats.requestRate.oneMinuteRate).getOrElse(0) - - fetcherThreadMap.foldLeft(headRate)((curMinAll, fetcherThreadMapEntry) => { - fetcherThreadMapEntry._2.fetcherStats.requestRate.oneMinuteRate.min(curMinAll) - }) - } + "MinFetchRate", { + new Gauge[Double] { + // current min fetch rate across all fetchers/topics/partitions + def value = { + val headRate: Double = + fetcherThreadMap.headOption.map(_._2.fetcherStats.requestRate.oneMinuteRate).getOrElse(0) + + fetcherThreadMap.foldLeft(headRate)((curMinAll, fetcherThreadMapEntry) => { + fetcherThreadMapEntry._2.fetcherStats.requestRate.oneMinuteRate.min(curMinAll) + }) } } + }, + Map("clientId" -> clientId) ) private def getFetcherId(topic: String, partitionId: Int) : Int = { diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index 2e9532e820b5b..8c281d4668f92 100644 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -26,9 +26,7 @@ import kafka.utils.Utils.inLock import kafka.message.{InvalidMessageException, ByteBufferMessageSet, MessageAndOffset} import kafka.metrics.KafkaMetricsGroup -import scala.collection.mutable -import scala.collection.Set -import scala.collection.Map +import scala.collection.{mutable, Set, Map} import java.util.concurrent.TimeUnit import java.util.concurrent.locks.ReentrantLock import java.util.concurrent.atomic.AtomicLong @@ -46,8 +44,7 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke private val partitionMapLock = new ReentrantLock private val partitionMapCond = partitionMapLock.newCondition() val simpleConsumer = new SimpleConsumer(sourceBroker.host, sourceBroker.port, socketTimeout, socketBufferSize, clientId) - private val brokerInfo = "host_%s-port_%s".format(sourceBroker.host, sourceBroker.port) - private val metricId = new ClientIdAndBroker(clientId, brokerInfo) + private val metricId = new ClientIdAndBroker(clientId, sourceBroker.host, sourceBroker.port) val fetcherStats = new FetcherStats(metricId) val fetcherLagStats = new FetcherLagStats(metricId) val fetchRequestBuilder = new FetchRequestBuilder(). @@ -204,13 +201,15 @@ abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroke } } -class FetcherLagMetrics(metricId: ClientIdBrokerTopicPartition) extends KafkaMetricsGroup { +class FetcherLagMetrics(metricId: ClientIdTopicPartition) extends KafkaMetricsGroup { private[this] val lagVal = new AtomicLong(-1L) - newGauge( - metricId + "-ConsumerLag", + newGauge("ConsumerLag", new Gauge[Long] { def value = lagVal.get - } + }, + Map("clientId" -> metricId.clientId, + "topic" -> metricId.topic, + "partition" -> metricId.partitionId.toString) ) def lag_=(newLag: Long) { @@ -221,20 +220,25 @@ class FetcherLagMetrics(metricId: ClientIdBrokerTopicPartition) extends KafkaMet } class FetcherLagStats(metricId: ClientIdAndBroker) { - private val valueFactory = (k: ClientIdBrokerTopicPartition) => new FetcherLagMetrics(k) - val stats = new Pool[ClientIdBrokerTopicPartition, FetcherLagMetrics](Some(valueFactory)) + private val valueFactory = (k: ClientIdTopicPartition) => new FetcherLagMetrics(k) + val stats = new Pool[ClientIdTopicPartition, FetcherLagMetrics](Some(valueFactory)) def getFetcherLagStats(topic: String, partitionId: Int): FetcherLagMetrics = { - stats.getAndMaybePut(new ClientIdBrokerTopicPartition(metricId.clientId, metricId.brokerInfo, topic, partitionId)) + stats.getAndMaybePut(new ClientIdTopicPartition(metricId.clientId, topic, partitionId)) } } class FetcherStats(metricId: ClientIdAndBroker) extends KafkaMetricsGroup { - val requestRate = newMeter(metricId + "-RequestsPerSec", "requests", TimeUnit.SECONDS) - val byteRate = newMeter(metricId + "-BytesPerSec", "bytes", TimeUnit.SECONDS) + val tags = Map("clientId" -> metricId.clientId, + "brokerHost" -> metricId.brokerHost, + "brokerPort" -> metricId.brokerPort.toString) + + val requestRate = newMeter("RequestsPerSec", "requests", TimeUnit.SECONDS, tags) + + val byteRate = newMeter("BytesPerSec", "bytes", TimeUnit.SECONDS, tags) } -case class ClientIdBrokerTopicPartition(clientId: String, brokerInfo: String, topic: String, partitionId: Int) { - override def toString = "%s-%s-%s-%d".format(clientId, brokerInfo, topic, partitionId) +case class ClientIdTopicPartition(clientId: String, topic: String, partitionId: Int) { + override def toString = "%s-%s-%d".format(clientId, topic, partitionId) } diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index 1ccbb4b6fdbbd..1e2e56f87a3c2 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -44,7 +44,6 @@ case class FetchMetadata(fetchMinBytes: Int, "onlyCommitted: " + fetchOnlyCommitted + ", " "partitionStatus: " + fetchPartitionStatus + "]" } - /** * A delayed fetch request that can be created by the replica manager and watched * in the fetch request purgatory diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index 8049e07e5d6d6..1603066d33f82 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -22,7 +22,6 @@ import kafka.api.ProducerResponseStatus import kafka.common.ErrorMapping import kafka.common.TopicAndPartition -import scala.Some import scala.collection._ case class ProducePartitionStatus(requiredOffset: Long, responseStatus: ProducerResponseStatus) { @@ -118,5 +117,4 @@ class DelayedProduce(delayMs: Long, val responseStatus = produceMetadata.produceStatus.mapValues(status => status.responseStatus) responseCallback(responseStatus) } -} - +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/DelayedRequestKey.scala b/core/src/main/scala/kafka/server/DelayedRequestKey.scala deleted file mode 100644 index 628ef59564b9b..0000000000000 --- a/core/src/main/scala/kafka/server/DelayedRequestKey.scala +++ /dev/null @@ -1,38 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.server - -import kafka.common.TopicAndPartition - -/** - * Keys used for delayed request metrics recording - */ -trait DelayedRequestKey { - def keyLabel: String -} - -object DelayedRequestKey { - val globalLabel = "All" -} - -case class TopicPartitionRequestKey(topic: String, partition: Int) extends DelayedRequestKey { - - def this(topicAndPartition: TopicAndPartition) = this(topicAndPartition.topic, topicAndPartition.partition) - - override def keyLabel = "%s-%d".format(topic, partition) -} diff --git a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala index 00bcc06716fbb..e4053fbe8ef78 100644 --- a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala +++ b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala @@ -93,23 +93,28 @@ class KafkaRequestHandlerPool(val brokerId: Int, } } -class BrokerTopicMetrics(name: String) extends KafkaMetricsGroup { - val messagesInRate = newMeter(name + "MessagesInPerSec", "messages", TimeUnit.SECONDS) - val bytesInRate = newMeter(name + "BytesInPerSec", "bytes", TimeUnit.SECONDS) - val bytesOutRate = newMeter(name + "BytesOutPerSec", "bytes", TimeUnit.SECONDS) - val bytesRejectedRate = newMeter(name + "BytesRejectedPerSec", "bytes", TimeUnit.SECONDS) - val failedProduceRequestRate = newMeter(name + "FailedProduceRequestsPerSec", "requests", TimeUnit.SECONDS) - val failedFetchRequestRate = newMeter(name + "FailedFetchRequestsPerSec", "requests", TimeUnit.SECONDS) +class BrokerTopicMetrics(name: Option[String]) extends KafkaMetricsGroup { + val tags: scala.collection.Map[String, String] = name match { + case None => scala.collection.Map.empty + case Some(topic) => Map("topic" -> topic) + } + + val messagesInRate = newMeter("MessagesInPerSec", "messages", TimeUnit.SECONDS, tags) + val bytesInRate = newMeter("BytesInPerSec", "bytes", TimeUnit.SECONDS, tags) + val bytesOutRate = newMeter("BytesOutPerSec", "bytes", TimeUnit.SECONDS, tags) + val bytesRejectedRate = newMeter("BytesRejectedPerSec", "bytes", TimeUnit.SECONDS, tags) + val failedProduceRequestRate = newMeter("FailedProduceRequestsPerSec", "requests", TimeUnit.SECONDS, tags) + val failedFetchRequestRate = newMeter("FailedFetchRequestsPerSec", "requests", TimeUnit.SECONDS, tags) } object BrokerTopicStats extends Logging { - private val valueFactory = (k: String) => new BrokerTopicMetrics(k) + private val valueFactory = (k: String) => new BrokerTopicMetrics(Some(k)) private val stats = new Pool[String, BrokerTopicMetrics](Some(valueFactory)) - private val allTopicsStats = new BrokerTopicMetrics("AllTopics") + private val allTopicsStats = new BrokerTopicMetrics(None) def getBrokerAllTopicsStats(): BrokerTopicMetrics = allTopicsStats def getBrokerTopicStats(topic: String): BrokerTopicMetrics = { - stats.getAndMaybePut(topic + "-") + stats.getAndMaybePut(topic) } } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 4de812374e8fb..1bf7d10cef23a 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -25,7 +25,6 @@ import kafka.utils._ import java.util.concurrent._ import atomic.{AtomicInteger, AtomicBoolean} import java.io.File -import java.net.BindException import org.I0Itec.zkclient.ZkClient import kafka.controller.{ControllerStats, KafkaController} import kafka.cluster.Broker diff --git a/core/src/main/scala/kafka/server/KafkaServerStartable.scala b/core/src/main/scala/kafka/server/KafkaServerStartable.scala index cd64bbe56c99d..1c1b75b4137a8 100644 --- a/core/src/main/scala/kafka/server/KafkaServerStartable.scala +++ b/core/src/main/scala/kafka/server/KafkaServerStartable.scala @@ -17,6 +17,7 @@ package kafka.server +import kafka.common.AppInfo import kafka.utils.Logging @@ -26,6 +27,7 @@ class KafkaServerStartable(val serverConfig: KafkaConfig) extends Logging { def startup() { try { server.startup() + AppInfo.registerInfo() } catch { case e: Throwable => diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 3007a6d89b637..f043f042e4ce2 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -34,7 +34,6 @@ import scala.collection._ import scala.collection.mutable.HashMap import scala.collection.Map import scala.collection.Set -import scala.Some import org.I0Itec.zkclient.ZkClient import com.yammer.metrics.core.Gauge @@ -124,9 +123,9 @@ class ReplicaManager(val config: KafkaConfig, * 1. The partition HW has changed (for acks = -1) * 2. A follower replica's fetch operation is received (for acks > 1) */ - def tryCompleteDelayedProduce(key: DelayedRequestKey) { + def tryCompleteDelayedProduce(key: TopicAndPartition) { val completed = producerRequestPurgatory.checkAndComplete(key) - debug("Request key %s unblocked %d producer requests.".format(key.keyLabel, completed)) + debug("Request key %s unblocked %d producer requests.".format(key, completed)) } /** @@ -136,9 +135,9 @@ class ReplicaManager(val config: KafkaConfig, * 1. The partition HW has changed (for regular fetch) * 2. A new message set is appended to the local log (for follower fetch) */ - def tryCompleteDelayedFetch(key: DelayedRequestKey) { + def tryCompleteDelayedFetch(key: TopicAndPartition) { val completed = fetchRequestPurgatory.checkAndComplete(key) - debug("Request key %s unblocked %d fetch requests.".format(key.keyLabel, completed)) + debug("Request key %s unblocked %d fetch requests.".format(key, completed)) } def startup() { @@ -281,7 +280,7 @@ class ReplicaManager(val config: KafkaConfig, val delayedProduce = new DelayedProduce(timeout, produceMetadata, this, responseCallback) // create a list of (topic, partition) pairs to use as keys for this delayed request - val producerRequestKeys = messagesPerPartition.keys.map(new TopicPartitionRequestKey(_)).toSeq + val producerRequestKeys = messagesPerPartition.keys.toSeq // try to complete the request immediately, otherwise put it into the purgatory // this is because while the delayed request is being created, new requests may @@ -384,7 +383,7 @@ class ReplicaManager(val config: KafkaConfig, val delayedFetch = new DelayedFetch(timeout, fetchMetadata, this, responseCallback) // create a list of (topic, partition) pairs to use as keys for this delayed request - val delayedFetchKeys = fetchPartitionStatus.keys.map(new TopicPartitionRequestKey(_)).toSeq + val delayedFetchKeys = fetchPartitionStatus.keys.toSeq // try to complete the request immediately, otherwise put it into the purgatory; // this is because while the delayed request is being created, new requests may @@ -709,7 +708,7 @@ class ReplicaManager(val config: KafkaConfig, // for producer requests with ack > 1, we need to check // if they can be unblocked after some follower's log end offsets have moved - tryCompleteDelayedProduce(new TopicPartitionRequestKey(topicAndPartition)) + tryCompleteDelayedProduce(topicAndPartition) case None => warn("While recording the replica LEO, the partition %s hasn't been created.".format(topicAndPartition)) } diff --git a/core/src/main/scala/kafka/tools/ProducerPerformance.scala b/core/src/main/scala/kafka/tools/ProducerPerformance.scala index f61c7c701fd85..f2dc4ed2f04f0 100644 --- a/core/src/main/scala/kafka/tools/ProducerPerformance.scala +++ b/core/src/main/scala/kafka/tools/ProducerPerformance.scala @@ -28,7 +28,6 @@ import java.util.concurrent.atomic.AtomicLong import java.util._ import java.text.SimpleDateFormat import java.math.BigInteger -import scala.collection.immutable.List import org.apache.log4j.Logger diff --git a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala index 9b0521c85fa66..8c4687b2c96fd 100644 --- a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala @@ -26,8 +26,7 @@ import kafka.message._ import kafka.serializer._ import org.I0Itec.zkclient.ZkClient import kafka.utils._ -import kafka.producer.{ProducerConfig, KeyedMessage, Producer} -import java.util.{Collections, Properties} +import java.util.Collections import org.apache.log4j.{Logger, Level} import kafka.utils.TestUtils._ import kafka.common.MessageStreamsExistException @@ -89,8 +88,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar zkConsumerConnector0.shutdown // send some messages to each broker - val sentMessages1 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages) ++ - sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages) + val sentMessages1 = sendMessagesToPartition(configs, topic, 0, nMessages) ++ + sendMessagesToPartition(configs, topic, 1, nMessages) // wait to make sure the topic and partition have a leader for the successful case waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) @@ -123,8 +122,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val zkConsumerConnector2 = new ZookeeperConsumerConnector(consumerConfig2, true) val topicMessageStreams2 = zkConsumerConnector2.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) // send some messages to each broker - val sentMessages2 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages) ++ - sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages) + val sentMessages2 = sendMessagesToPartition(configs, topic, 0, nMessages) ++ + sendMessagesToPartition(configs, topic, 1, nMessages) waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) @@ -144,8 +143,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val zkConsumerConnector3 = new ZookeeperConsumerConnector(consumerConfig3, true) val topicMessageStreams3 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int]()) // send some messages to each broker - val sentMessages3 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages) ++ - sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages) + val sentMessages3 = sendMessagesToPartition(configs, topic, 0, nMessages) ++ + sendMessagesToPartition(configs, topic, 1, nMessages) waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) @@ -178,8 +177,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar requestHandlerLogger.setLevel(Level.FATAL) // send some messages to each broker - val sentMessages1 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, GZIPCompressionCodec) ++ - sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, GZIPCompressionCodec) + val sentMessages1 = sendMessagesToPartition(configs, topic, 0, nMessages, GZIPCompressionCodec) ++ + sendMessagesToPartition(configs, topic, 1, nMessages, GZIPCompressionCodec) waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) @@ -211,8 +210,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val zkConsumerConnector2 = new ZookeeperConsumerConnector(consumerConfig2, true) val topicMessageStreams2 = zkConsumerConnector2.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) // send some messages to each broker - val sentMessages2 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, GZIPCompressionCodec) ++ - sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, GZIPCompressionCodec) + val sentMessages2 = sendMessagesToPartition(configs, topic, 0, nMessages, GZIPCompressionCodec) ++ + sendMessagesToPartition(configs, topic, 1, nMessages, GZIPCompressionCodec) waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) @@ -232,8 +231,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val zkConsumerConnector3 = new ZookeeperConsumerConnector(consumerConfig3, true) val topicMessageStreams3 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int](), new StringDecoder(), new StringDecoder()) // send some messages to each broker - val sentMessages3 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, GZIPCompressionCodec) ++ - sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, GZIPCompressionCodec) + val sentMessages3 = sendMessagesToPartition(configs, topic, 0, nMessages, GZIPCompressionCodec) ++ + sendMessagesToPartition(configs, topic, 1, nMessages, GZIPCompressionCodec) waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1) @@ -254,8 +253,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar def testCompressionSetConsumption() { // send some messages to each broker - val sentMessages = sendMessagesToBrokerPartition(configs.head, topic, 0, 200, DefaultCompressionCodec) ++ - sendMessagesToBrokerPartition(configs.last, topic, 1, 200, DefaultCompressionCodec) + val sentMessages = sendMessagesToPartition(configs, topic, 0, 200, DefaultCompressionCodec) ++ + sendMessagesToPartition(configs, topic, 1, 200, DefaultCompressionCodec) TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0) TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1) @@ -280,8 +279,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar requestHandlerLogger.setLevel(Level.FATAL) // send some messages to each broker - val sentMessages = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, NoCompressionCodec) ++ - sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, NoCompressionCodec) + val sentMessages = sendMessagesToPartition(configs, topic, 0, nMessages, NoCompressionCodec) ++ + sendMessagesToPartition(configs, topic, 1, nMessages, NoCompressionCodec) TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0) TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1) @@ -321,7 +320,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = servers) // send some messages to each broker - val sentMessages1 = sendMessages(configs.head, nMessages, "batch1", NoCompressionCodec, 1) + val sentMessages1 = sendMessages(configs, topic, "producer1", nMessages, "batch1", NoCompressionCodec, 1) // create a consumer val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1)) @@ -345,70 +344,6 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar zkClient.close() } - def sendMessagesToBrokerPartition(config: KafkaConfig, - topic: String, - partition: Int, - numMessages: Int, - compression: CompressionCodec = NoCompressionCodec): List[String] = { - val header = "test-%d-%d".format(config.brokerId, partition) - val props = new Properties() - props.put("compression.codec", compression.codec.toString) - val producer: Producer[Int, String] = - createProducer(TestUtils.getBrokerListStrFromConfigs(configs), - encoder = classOf[StringEncoder].getName, - keyEncoder = classOf[IntEncoder].getName, - partitioner = classOf[FixedValuePartitioner].getName, - producerProps = props) - - val ms = 0.until(numMessages).map(x => header + config.brokerId + "-" + partition + "-" + x) - producer.send(ms.map(m => new KeyedMessage[Int, String](topic, partition, m)):_*) - debug("Sent %d messages to broker %d for partition [%s,%d]".format(ms.size, config.brokerId, topic, partition)) - producer.close() - ms.toList - } - - def sendMessages(config: KafkaConfig, - messagesPerNode: Int, - header: String, - compression: CompressionCodec, - numParts: Int): List[String]= { - var messages: List[String] = Nil - val props = new Properties() - props.put("compression.codec", compression.codec.toString) - val producer: Producer[Int, String] = - createProducer(brokerList = TestUtils.getBrokerListStrFromConfigs(configs), - encoder = classOf[StringEncoder].getName, - keyEncoder = classOf[IntEncoder].getName, - partitioner = classOf[FixedValuePartitioner].getName, - producerProps = props) - - for (partition <- 0 until numParts) { - val ms = 0.until(messagesPerNode).map(x => header + config.brokerId + "-" + partition + "-" + x) - producer.send(ms.map(m => new KeyedMessage[Int, String](topic, partition, m)):_*) - messages ++= ms - debug("Sent %d messages to broker %d for partition [%s,%d]".format(ms.size, config.brokerId, topic, partition)) - } - producer.close() - messages - } - - def getMessages(nMessagesPerThread: Int, - topicMessageStreams: Map[String,List[KafkaStream[String, String]]]): List[String]= { - var messages: List[String] = Nil - for((topic, messageStreams) <- topicMessageStreams) { - for (messageStream <- messageStreams) { - val iterator = messageStream.iterator - for(i <- 0 until nMessagesPerThread) { - assertTrue(iterator.hasNext) - val message = iterator.next.message - messages ::= message - debug("received message: " + message) - } - } - } - messages.reverse - } - def getZKChildrenValues(path : String) : Seq[Tuple2[String,String]] = { val children = zkClient.getChildren(path) Collections.sort(children) diff --git a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala new file mode 100644 index 0000000000000..3cf23b3d6d446 --- /dev/null +++ b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala @@ -0,0 +1,72 @@ +/** + * 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.consumer + +import com.yammer.metrics.Metrics +import junit.framework.Assert._ +import kafka.integration.KafkaServerTestHarness +import kafka.server._ +import scala.collection._ +import org.scalatest.junit.JUnit3Suite +import kafka.message._ +import kafka.serializer._ +import kafka.utils._ +import kafka.utils.TestUtils._ + +class MetricsTest extends JUnit3Suite with KafkaServerTestHarness with Logging { + val zookeeperConnect = TestZKUtils.zookeeperConnect + val numNodes = 2 + val numParts = 2 + val topic = "topic1" + val configs = + for (props <- TestUtils.createBrokerConfigs(numNodes)) + yield new KafkaConfig(props) { + override val zkConnect = zookeeperConnect + override val numPartitions = numParts + } + val nMessages = 2 + + override def tearDown() { + super.tearDown() + } + + def testMetricsLeak() { + // create topic topic1 with 1 partition on broker 0 + createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = servers) + // force creation not client's specific metrics. + createAndShutdownStep("group0", "consumer0", "producer0") + + val countOfStaticMetrics = Metrics.defaultRegistry().allMetrics().keySet().size + + for (i <- 0 to 5) { + createAndShutdownStep("group" + i % 3, "consumer" + i % 2, "producer" + i % 2) + assertEquals(countOfStaticMetrics, Metrics.defaultRegistry().allMetrics().keySet().size) + } + } + + def createAndShutdownStep(group: String, consumerId: String, producerId: String): Unit = { + val sentMessages1 = sendMessages(configs, topic, producerId, nMessages, "batch1", NoCompressionCodec, 1) + // create a consumer + val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumerId)) + val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) + val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) + val receivedMessages1 = getMessages(nMessages, topicMessageStreams1) + + zkConsumerConnector1.shutdown() + } +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index dd3640f47b26a..0da774d0ed015 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -26,7 +26,6 @@ import java.util.Properties import org.apache.kafka.common.utils.Utils._ -import collection.mutable.Map import collection.mutable.ListBuffer import org.I0Itec.zkclient.ZkClient @@ -36,7 +35,7 @@ import kafka.producer._ import kafka.message._ import kafka.api._ import kafka.cluster.Broker -import kafka.consumer.ConsumerConfig +import kafka.consumer.{KafkaStream, ConsumerConfig} import kafka.serializer.{StringEncoder, DefaultEncoder, Encoder} import kafka.common.TopicAndPartition import kafka.admin.AdminUtils @@ -47,6 +46,8 @@ import junit.framework.AssertionFailedError import junit.framework.Assert._ import org.apache.kafka.clients.producer.KafkaProducer +import scala.collection.Map + /** * Utility functions to help with testing */ @@ -483,7 +484,7 @@ object TestUtils extends Logging { val data = topics.flatMap(topic => partitions.map(partition => (TopicAndPartition(topic, partition), message)) ) - new ProducerRequest(correlationId, clientId, acks.toShort, timeout, Map(data:_*)) + new ProducerRequest(correlationId, clientId, acks.toShort, timeout, collection.mutable.Map(data:_*)) } def makeLeaderForPartition(zkClient: ZkClient, topic: String, @@ -720,6 +721,73 @@ object TestUtils extends Logging { time = time, brokerState = new BrokerState()) } + + def sendMessagesToPartition(configs: Seq[KafkaConfig], + topic: String, + partition: Int, + numMessages: Int, + compression: CompressionCodec = NoCompressionCodec): List[String] = { + val header = "test-%d".format(partition) + val props = new Properties() + props.put("compression.codec", compression.codec.toString) + val producer: Producer[Int, String] = + createProducer(TestUtils.getBrokerListStrFromConfigs(configs), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[IntEncoder].getName, + partitioner = classOf[FixedValuePartitioner].getName, + producerProps = props) + + val ms = 0.until(numMessages).map(x => header + "-" + x) + producer.send(ms.map(m => new KeyedMessage[Int, String](topic, partition, m)):_*) + debug("Sent %d messages for partition [%s,%d]".format(ms.size, topic, partition)) + producer.close() + ms.toList + } + + def sendMessages(configs: Seq[KafkaConfig], + topic: String, + producerId: String, + messagesPerNode: Int, + header: String, + compression: CompressionCodec, + numParts: Int): List[String]= { + var messages: List[String] = Nil + val props = new Properties() + props.put("compression.codec", compression.codec.toString) + props.put("client.id", producerId) + val producer: Producer[Int, String] = + createProducer(brokerList = TestUtils.getBrokerListStrFromConfigs(configs), + encoder = classOf[StringEncoder].getName, + keyEncoder = classOf[IntEncoder].getName, + partitioner = classOf[FixedValuePartitioner].getName, + producerProps = props) + + for (partition <- 0 until numParts) { + val ms = 0.until(messagesPerNode).map(x => header + "-" + partition + "-" + x) + producer.send(ms.map(m => new KeyedMessage[Int, String](topic, partition, m)):_*) + messages ++= ms + debug("Sent %d messages for partition [%s,%d]".format(ms.size, topic, partition)) + } + producer.close() + messages + } + + def getMessages(nMessagesPerThread: Int, + topicMessageStreams: Map[String, List[KafkaStream[String, String]]]): List[String] = { + var messages: List[String] = Nil + for ((topic, messageStreams) <- topicMessageStreams) { + for (messageStream <- messageStreams) { + val iterator = messageStream.iterator + for (i <- 0 until nMessagesPerThread) { + assertTrue(iterator.hasNext) + val message = iterator.next.message + messages ::= message + debug("received message: " + message) + } + } + } + messages.reverse + } } object TestZKUtils { From f0fd70929a1ad607aa1fbf6ed9b40b16ab95b807 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Fri, 21 Nov 2014 14:25:18 -0800 Subject: [PATCH 055/491] KAFKA-1770; clarify descriptive comments of UnknownTopicOrPartitionException; reviewed by Guozhang Wang --- .../scala/kafka/common/UnknownTopicOrPartitionException.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/common/UnknownTopicOrPartitionException.scala b/core/src/main/scala/kafka/common/UnknownTopicOrPartitionException.scala index 781e551e5b78b..f382d16de9c39 100644 --- a/core/src/main/scala/kafka/common/UnknownTopicOrPartitionException.scala +++ b/core/src/main/scala/kafka/common/UnknownTopicOrPartitionException.scala @@ -17,7 +17,9 @@ package kafka.common /** - * Indicates an unknown topic or a partition id not between 0 and numPartitions-1 + * Indicates one of the following situation: + * 1. Producer does not have the partition metadata for this id upon sending messages + * 2. Broker does not have the specified partition by id upon receiving messages */ class UnknownTopicOrPartitionException(message: String) extends RuntimeException(message) { def this() = this(null) From 409c367ceb5f9eae395cb346dd3fa02b8ee8ee70 Mon Sep 17 00:00:00 2001 From: Jarek Jarcec Cecho Date: Sun, 23 Nov 2014 16:40:58 -0800 Subject: [PATCH 056/491] KAFKA-1654 Provide a way to override server configuration from command line; reviewed by Neha Narkhede --- bin/kafka-server-start.sh | 2 +- core/src/main/scala/kafka/Kafka.scala | 38 ++++-- .../scala/unit/kafka/KafkaConfigTest.scala | 120 ++++++++++++++++++ 3 files changed, 150 insertions(+), 10 deletions(-) create mode 100644 core/src/test/scala/unit/kafka/KafkaConfigTest.scala diff --git a/bin/kafka-server-start.sh b/bin/kafka-server-start.sh index 1737d0c1cb086..dc01d46a13f16 100755 --- a/bin/kafka-server-start.sh +++ b/bin/kafka-server-start.sh @@ -16,7 +16,7 @@ if [ $# -lt 1 ]; then - echo "USAGE: $0 [-daemon] server.properties" + echo "USAGE: $0 [-daemon] server.properties [--override property=value]*" exit 1 fi base_dir=$(dirname $0) diff --git a/core/src/main/scala/kafka/Kafka.scala b/core/src/main/scala/kafka/Kafka.scala index 2e94fee2fa35a..77a49e12af6f8 100644 --- a/core/src/main/scala/kafka/Kafka.scala +++ b/core/src/main/scala/kafka/Kafka.scala @@ -17,22 +17,42 @@ package kafka - +import scala.collection.JavaConversions._ +import joptsimple.OptionParser import metrics.KafkaMetricsReporter import server.{KafkaConfig, KafkaServerStartable, KafkaServer} -import utils.{Utils, Logging} +import kafka.utils.{CommandLineUtils, Utils, Logging} object Kafka extends Logging { - def main(args: Array[String]): Unit = { - if (args.length != 1) { - println("USAGE: java [options] %s server.properties".format(classOf[KafkaServer].getSimpleName())) - System.exit(1) + def getKafkaConfigFromArgs(args: Array[String]): KafkaConfig = { + val optionParser = new OptionParser + val overrideOpt = optionParser.accepts("override", "Optional property that should override values set in server.properties file") + .withRequiredArg() + .ofType(classOf[String]) + + if (args.length == 0) { + CommandLineUtils.printUsageAndDie(optionParser, "USAGE: java [options] %s server.properties [--override property=value]*".format(classOf[KafkaServer].getSimpleName())) + } + + val props = Utils.loadProps(args(0)) + + if(args.length > 1) { + val options = optionParser.parse(args.slice(1, args.length): _*) + + if(options.nonOptionArguments().size() > 0) { + CommandLineUtils.printUsageAndDie(optionParser, "Found non argument parameters: " + options.nonOptionArguments().toArray.mkString(",")) + } + + props.putAll(CommandLineUtils.parseKeyValueArgs(options.valuesOf(overrideOpt))) } - + + new KafkaConfig(props) + } + + def main(args: Array[String]): Unit = { try { - val props = Utils.loadProps(args(0)) - val serverConfig = new KafkaConfig(props) + val serverConfig = getKafkaConfigFromArgs(args) KafkaMetricsReporter.startReporters(serverConfig.props) val kafkaServerStartable = new KafkaServerStartable(serverConfig) diff --git a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala new file mode 100644 index 0000000000000..4d36b8b1173f6 --- /dev/null +++ b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala @@ -0,0 +1,120 @@ +/** + * 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 unit.kafka + +import java.io.{FileOutputStream, File} +import java.security.Permission + +import kafka.Kafka +import org.junit.{After, Before, Test} +import junit.framework.Assert._ + +class KafkaTest { + + val originalSecurityManager: SecurityManager = System.getSecurityManager + + class ExitCalled extends SecurityException { + } + + private class NoExitSecurityManager extends SecurityManager { + override def checkExit(status: Int): Unit = { + throw new ExitCalled + } + + override def checkPermission(perm : Permission): Unit = { + } + + override def checkPermission(perm : Permission, context: Object): Unit = { + } + } + + @Before + def setSecurityManager() : Unit = { + System.setSecurityManager(new NoExitSecurityManager) + } + + @After + def setOriginalSecurityManager() : Unit = { + System.setSecurityManager(originalSecurityManager) + } + + @Test + def testGetKafkaConfigFromArgs(): Unit = { + val propertiesFile = prepareDefaultConfig() + + // We should load configuration file without any arguments + val config1 = Kafka.getKafkaConfigFromArgs(Array(propertiesFile)) + assertEquals(1, config1.brokerId) + + // We should be able to override given property on command line + val config2 = Kafka.getKafkaConfigFromArgs(Array(propertiesFile, "--override", "broker.id=2")) + assertEquals(2, config2.brokerId) + + // We should be also able to set completely new property + val config3 = Kafka.getKafkaConfigFromArgs(Array(propertiesFile, "--override", "port=1987")) + assertEquals(1, config3.brokerId) + assertEquals(1987, config3.port) + + // We should be also able to set several properties + val config4 = Kafka.getKafkaConfigFromArgs(Array(propertiesFile, "--override", "port=1987", "--override", "broker.id=2")) + assertEquals(2, config4.brokerId) + assertEquals(1987, config4.port) + } + + @Test(expected = classOf[ExitCalled]) + def testGetKafkaConfigFromArgsWrongSetValue(): Unit = { + val propertiesFile = prepareDefaultConfig() + Kafka.getKafkaConfigFromArgs(Array(propertiesFile, "--override", "a=b=c")) + } + + @Test(expected = classOf[ExitCalled]) + def testGetKafkaConfigFromArgsNonArgsAtTheEnd(): Unit = { + val propertiesFile = prepareDefaultConfig() + Kafka.getKafkaConfigFromArgs(Array(propertiesFile, "--override", "broker.id=1", "broker.id=2")) + } + + @Test(expected = classOf[ExitCalled]) + def testGetKafkaConfigFromArgsNonArgsOnly(): Unit = { + val propertiesFile = prepareDefaultConfig() + Kafka.getKafkaConfigFromArgs(Array(propertiesFile, "broker.id=1", "broker.id=2")) + } + + @Test(expected = classOf[ExitCalled]) + def testGetKafkaConfigFromArgsNonArgsAtTheBegging(): Unit = { + val propertiesFile = prepareDefaultConfig() + Kafka.getKafkaConfigFromArgs(Array(propertiesFile, "broker.id=1", "--override", "broker.id=2")) + } + + def prepareDefaultConfig(): String = { + prepareConfig(Array("broker.id=1", "zookeeper.connect=somewhere")) + } + + def prepareConfig(lines : Array[String]): String = { + val file = File.createTempFile("kafkatest", ".properties") + file.deleteOnExit() + + val writer = new FileOutputStream(file) + lines.foreach { l => + writer.write(l.getBytes) + writer.write("\n".getBytes) + } + + writer.close + + file.getAbsolutePath + } +} From 72601f783e24217734a3923c6318e12a0f600d0f Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 24 Nov 2014 11:44:48 -0800 Subject: [PATCH 057/491] KAFKA-1624; bump up default scala version to 2.11.4 to compile with java 8; reviewed by Joe Stein, Gwen Shapira and Joel Koshy --- bin/kafka-run-class.sh | 2 +- bin/windows/kafka-run-class.bat | 2 +- build.gradle | 14 +++++++------- gradle.properties | 2 +- scala.gradle | 4 ++-- 5 files changed, 12 insertions(+), 12 deletions(-) diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index 36c742b67a725..ce3a4d06a27f6 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -32,7 +32,7 @@ if [ ! -d "$LOG_DIR" ]; then fi if [ -z "$SCALA_VERSION" ]; then - SCALA_VERSION=2.10.1 + SCALA_VERSION=2.10.4 fi if [ -z "$SCALA_BINARY_VERSION" ]; then diff --git a/bin/windows/kafka-run-class.bat b/bin/windows/kafka-run-class.bat index 8e9780e2eb74a..9df3d2b45236b 100644 --- a/bin/windows/kafka-run-class.bat +++ b/bin/windows/kafka-run-class.bat @@ -28,7 +28,7 @@ popd set CLASSPATH= IF ["%SCALA_VERSION%"] EQU [""] ( - set SCALA_VERSION=2.10.1 + set SCALA_VERSION=2.10.4 ) IF ["%SCALA_BINARY_VERSION%"] EQU [""] ( diff --git a/build.gradle b/build.gradle index 030af63553afb..18f86e4c8a106 100644 --- a/build.gradle +++ b/build.gradle @@ -137,7 +137,7 @@ subprojects { } } -for ( sv in ['2_9_1', '2_9_2', '2_10_1', '2_11'] ) { +for ( sv in ['2_9_1', '2_9_2', '2_10_4', '2_11'] ) { String svInDot = sv.replaceAll( "_", ".") tasks.create(name: "jar_core_${sv}", type: GradleBuild) { @@ -177,20 +177,20 @@ for ( sv in ['2_9_1', '2_9_2', '2_10_1', '2_11'] ) { } } -tasks.create(name: "jarAll", dependsOn: ['jar_core_2_9_1', 'jar_core_2_9_2', 'jar_core_2_10_1', 'jar_core_2_11', 'clients:jar', 'examples:jar', 'contrib:hadoop-consumer:jar', 'contrib:hadoop-producer:jar']) { +tasks.create(name: "jarAll", dependsOn: ['jar_core_2_9_1', 'jar_core_2_9_2', 'jar_core_2_10_4', 'jar_core_2_11', 'clients:jar', 'examples:jar', 'contrib:hadoop-consumer:jar', 'contrib:hadoop-producer:jar']) { } -tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_9_1', 'srcJar_2_9_2', 'srcJar_2_10_1', 'srcJar_2_11', 'clients:srcJar', 'examples:srcJar', 'contrib:hadoop-consumer:srcJar', 'contrib:hadoop-producer:srcJar']) { } +tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_9_1', 'srcJar_2_9_2', 'srcJar_2_10_4', 'srcJar_2_11', 'clients:srcJar', 'examples:srcJar', 'contrib:hadoop-consumer:srcJar', 'contrib:hadoop-producer:srcJar']) { } -tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_9_1', 'docsJar_2_9_2', 'docsJar_2_10_1', 'docsJar_2_11', 'clients:docsJar', 'examples:docsJar', 'contrib:hadoop-consumer:docsJar', 'contrib:hadoop-producer:docsJar']) { } +tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_9_1', 'docsJar_2_9_2', 'docsJar_2_10_4', 'docsJar_2_11', 'clients:docsJar', 'examples:docsJar', 'contrib:hadoop-consumer:docsJar', 'contrib:hadoop-producer:docsJar']) { } -tasks.create(name: "testAll", dependsOn: ['test_core_2_9_1', 'test_core_2_9_2', 'test_core_2_10_1', 'test_core_2_11', 'clients:test']) { +tasks.create(name: "testAll", dependsOn: ['test_core_2_9_1', 'test_core_2_9_2', 'test_core_2_10_4', 'test_core_2_11', 'clients:test']) { } -tasks.create(name: "releaseTarGzAll", dependsOn: ['releaseTarGz_2_9_1', 'releaseTarGz_2_9_2', 'releaseTarGz_2_10_1', 'releaseTarGz_2_11']) { +tasks.create(name: "releaseTarGzAll", dependsOn: ['releaseTarGz_2_9_1', 'releaseTarGz_2_9_2', 'releaseTarGz_2_10_4', 'releaseTarGz_2_11']) { } -tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_9_1', 'uploadCoreArchives_2_9_2', 'uploadCoreArchives_2_10_1', 'uploadCoreArchives_2_11', 'clients:uploadArchives', 'examples:uploadArchives', 'contrib:hadoop-consumer:uploadArchives', 'contrib:hadoop-producer:uploadArchives']) { +tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_9_1', 'uploadCoreArchives_2_9_2', 'uploadCoreArchives_2_10_4', 'uploadCoreArchives_2_11', 'clients:uploadArchives', 'examples:uploadArchives', 'contrib:hadoop-consumer:uploadArchives', 'contrib:hadoop-producer:uploadArchives']) { } project(':core') { diff --git a/gradle.properties b/gradle.properties index 5d3155fd44614..19150cf2561b3 100644 --- a/gradle.properties +++ b/gradle.properties @@ -15,7 +15,7 @@ group=org.apache.kafka version=0.8.3-SNAPSHOT -scalaVersion=2.10.1 +scalaVersion=2.10.4 task=build org.gradle.jvmargs=-XX:MaxPermSize=512m -Xmx1024m diff --git a/scala.gradle b/scala.gradle index 6adf9af7dbbe7..3c6f38ad035b3 100644 --- a/scala.gradle +++ b/scala.gradle @@ -1,7 +1,7 @@ if (!hasProperty('scalaVersion')) { - ext.scalaVersion = '2.10.1' + ext.scalaVersion = '2.10.4' } -ext.defaultScalaVersion = '2.10.1' +ext.defaultScalaVersion = '2.10.4' if (scalaVersion.startsWith('2.10')) { ext.baseScalaVersion = '2.10' } else if (scalaVersion.startsWith('2.11')) { From 9f8b8dad2b7ad31c9595f559f6d9e7d07d2f696d Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 24 Nov 2014 14:38:49 -0800 Subject: [PATCH 058/491] KAFKA-1580; disallow non-admin clients to produce to internal (e.g. offsets) topics; reviewed by Joel Koshy --- .../main/scala/kafka/admin/AdminUtils.scala | 21 +++-- .../main/scala/kafka/server/KafkaApis.scala | 5 ++ .../scala/kafka/server/OffsetManager.scala | 13 +-- .../scala/kafka/server/ReplicaManager.scala | 88 +++++++++++-------- .../api/ProducerFailureHandlingTest.scala | 23 ++--- 5 files changed, 87 insertions(+), 63 deletions(-) diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 94c53320b768b..28b12c7b89a56 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -17,25 +17,30 @@ package kafka.admin -import java.util.Random -import java.util.Properties -import kafka.api.{TopicMetadata, PartitionMetadata} +import kafka.common._ import kafka.cluster.Broker import kafka.log.LogConfig import kafka.utils.{Logging, ZkUtils, Json} -import org.I0Itec.zkclient.ZkClient -import org.I0Itec.zkclient.exception.ZkNodeExistsException +import kafka.api.{TopicMetadata, PartitionMetadata} + +import java.util.Random +import java.util.Properties +import scala.Some +import scala.Predef._ import scala.collection._ import mutable.ListBuffer import scala.collection.mutable -import kafka.common._ -import scala.Predef._ import collection.Map -import scala.Some import collection.Set +import org.I0Itec.zkclient.ZkClient +import org.I0Itec.zkclient.exception.ZkNodeExistsException + object AdminUtils extends Logging { val rand = new Random + + val AdminClientId = "__admin_client" + val TopicConfigChangeZnodePrefix = "config_change_" /** diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 968b0c4f809ea..2a1c0326b6e69 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -188,10 +188,15 @@ class KafkaApis(val requestChannel: RequestChannel, } } + // only allow appending to internal topic partitions + // if the client is not from admin + val internalTopicsAllowed = produceRequest.clientId == AdminUtils.AdminClientId + // call the replica manager to append messages to the replicas replicaManager.appendMessages( produceRequest.ackTimeoutMs.toLong, produceRequest.requiredAcks, + internalTopicsAllowed, produceRequest.data, sendResponseCallback) diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 2957bc435102b..3c79428962604 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -29,6 +29,7 @@ import kafka.message._ import kafka.metrics.KafkaMetricsGroup import kafka.common.TopicAndPartition import kafka.tools.MessageFormatter +import kafka.api.ProducerResponseStatus import scala.Some import scala.collection._ @@ -40,7 +41,6 @@ import java.util.concurrent.TimeUnit import com.yammer.metrics.core.Gauge import org.I0Itec.zkclient.ZkClient -import kafka.api.ProducerResponseStatus /** @@ -206,7 +206,7 @@ class OffsetManager(val config: OffsetManagerConfig, * Store offsets by appending it to the replicated log and then inserting to cache */ // TODO: generation id and consumer id is needed by coordinator to do consumer checking in the future - def storeOffsets(groupName: String, + def storeOffsets(groupId: String, consumerId: String, generationId: Int, offsetMetadata: immutable.Map[TopicAndPartition, OffsetAndMetadata], @@ -221,12 +221,12 @@ class OffsetManager(val config: OffsetManagerConfig, // construct the message set to append val messages = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => new Message( - key = OffsetManager.offsetCommitKey(groupName, topicAndPartition.topic, topicAndPartition.partition), + key = OffsetManager.offsetCommitKey(groupId, topicAndPartition.topic, topicAndPartition.partition), bytes = OffsetManager.offsetCommitValue(offsetAndMetadata) ) }.toSeq - val offsetTopicPartition = TopicAndPartition(OffsetManager.OffsetsTopicName, partitionFor(groupName)) + val offsetTopicPartition = TopicAndPartition(OffsetManager.OffsetsTopicName, partitionFor(groupId)) val offsetsAndMetadataMessageSet = Map(offsetTopicPartition -> new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*)) @@ -245,12 +245,12 @@ class OffsetManager(val config: OffsetManagerConfig, val responseCode = if (status.error == ErrorMapping.NoError) { filteredOffsetMetadata.foreach { case (topicAndPartition, offsetAndMetadata) => - putOffset(GroupTopicPartition(groupName, topicAndPartition), offsetAndMetadata) + putOffset(GroupTopicPartition(groupId, topicAndPartition), offsetAndMetadata) } ErrorMapping.NoError } else { debug("Offset commit %s from group %s consumer %s with generation %d failed when appending to log due to %s" - .format(filteredOffsetMetadata, groupName, consumerId, generationId, ErrorMapping.exceptionNameFor(status.error))) + .format(filteredOffsetMetadata, groupId, consumerId, generationId, ErrorMapping.exceptionNameFor(status.error))) // transform the log append error code to the corresponding the commit status error code if (status.error == ErrorMapping.UnknownTopicOrPartitionCode) @@ -278,6 +278,7 @@ class OffsetManager(val config: OffsetManagerConfig, replicaManager.appendMessages( config.offsetCommitTimeoutMs.toLong, config.offsetCommitRequiredAcks, + true, // allow appending to internal offset topic offsetsAndMetadataMessageSet, putCacheCallback) } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index f043f042e4ce2..b3566b0bc33fe 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -249,11 +249,12 @@ class ReplicaManager(val config: KafkaConfig, */ def appendMessages(timeout: Long, requiredAcks: Short, + internalTopicsAllowed: Boolean, messagesPerPartition: Map[TopicAndPartition, MessageSet], responseCallback: Map[TopicAndPartition, ProducerResponseStatus] => Unit) { val sTime = SystemTime.milliseconds - val localProduceResults = appendToLocalLog(messagesPerPartition, requiredAcks) + val localProduceResults = appendToLocalLog(internalTopicsAllowed, messagesPerPartition, requiredAcks) debug("Produce to local log in %d ms".format(SystemTime.milliseconds - sTime)) val produceStatus = localProduceResults.map{ case (topicAndPartition, result) => @@ -292,50 +293,59 @@ class ReplicaManager(val config: KafkaConfig, /** * Append the messages to the local replica logs */ - private def appendToLocalLog(messagesPerPartition: Map[TopicAndPartition, MessageSet], + private def appendToLocalLog(internalTopicsAllowed: Boolean, + messagesPerPartition: Map[TopicAndPartition, MessageSet], requiredAcks: Short): Map[TopicAndPartition, LogAppendResult] = { trace("Append [%s] to local log ".format(messagesPerPartition)) messagesPerPartition.map { case (topicAndPartition, messages) => - try { - val partitionOpt = getPartition(topicAndPartition.topic, topicAndPartition.partition) - val info = partitionOpt match { - case Some(partition) => - partition.appendMessagesToLeader(messages.asInstanceOf[ByteBufferMessageSet], requiredAcks) - case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d" - .format(topicAndPartition, localBrokerId)) - } + // reject appending to internal topics if it is not allowed + if (Topic.InternalTopics.contains(topicAndPartition.topic) && !internalTopicsAllowed) { - val numAppendedMessages = - if (info.firstOffset == -1L || info.lastOffset == -1L) - 0 - else - info.lastOffset - info.firstOffset + 1 + (topicAndPartition, LogAppendResult( + LogAppendInfo.UnknownLogAppendInfo, + Some(new InvalidTopicException("Cannot append to internal topic %s".format(topicAndPartition.topic))))) + } else { + try { + val partitionOpt = getPartition(topicAndPartition.topic, topicAndPartition.partition) + val info = partitionOpt match { + case Some(partition) => + partition.appendMessagesToLeader(messages.asInstanceOf[ByteBufferMessageSet], requiredAcks) + case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d" + .format(topicAndPartition, localBrokerId)) + } + + val numAppendedMessages = + if (info.firstOffset == -1L || info.lastOffset == -1L) + 0 + else + info.lastOffset - info.firstOffset + 1 - // update stats for successfully appended bytes and messages as bytesInRate and messageInRate - BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesInRate.mark(messages.sizeInBytes) - BrokerTopicStats.getBrokerAllTopicsStats.bytesInRate.mark(messages.sizeInBytes) - BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).messagesInRate.mark(numAppendedMessages) - BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(numAppendedMessages) + // update stats for successfully appended bytes and messages as bytesInRate and messageInRate + BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesInRate.mark(messages.sizeInBytes) + BrokerTopicStats.getBrokerAllTopicsStats.bytesInRate.mark(messages.sizeInBytes) + BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).messagesInRate.mark(numAppendedMessages) + BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(numAppendedMessages) - trace("%d bytes written to log %s-%d beginning at offset %d and ending at offset %d" - .format(messages.size, topicAndPartition.topic, topicAndPartition.partition, info.firstOffset, info.lastOffset)) - (topicAndPartition, LogAppendResult(info)) - } catch { - // NOTE: Failed produce requests metric is not incremented for known exceptions - // it is supposed to indicate un-expected failures of a broker in handling a produce request - case e: KafkaStorageException => - fatal("Halting due to unrecoverable I/O error while handling produce request: ", e) - Runtime.getRuntime.halt(1) - (topicAndPartition, null) - case utpe: UnknownTopicOrPartitionException => - (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(utpe))) - case nle: NotLeaderForPartitionException => - (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(nle))) - case e: Throwable => - BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).failedProduceRequestRate.mark() - BrokerTopicStats.getBrokerAllTopicsStats.failedProduceRequestRate.mark() - error("Error processing append operation on partition %s".format(topicAndPartition), e) - (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(e))) + trace("%d bytes written to log %s-%d beginning at offset %d and ending at offset %d" + .format(messages.size, topicAndPartition.topic, topicAndPartition.partition, info.firstOffset, info.lastOffset)) + (topicAndPartition, LogAppendResult(info)) + } catch { + // NOTE: Failed produce requests metric is not incremented for known exceptions + // it is supposed to indicate un-expected failures of a broker in handling a produce request + case e: KafkaStorageException => + fatal("Halting due to unrecoverable I/O error while handling produce request: ", e) + Runtime.getRuntime.halt(1) + (topicAndPartition, null) + case utpe: UnknownTopicOrPartitionException => + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(utpe))) + case nle: NotLeaderForPartitionException => + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(nle))) + case e: Throwable => + BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).failedProduceRequestRate.mark() + BrokerTopicStats.getBrokerAllTopicsStats.failedProduceRequestRate.mark() + error("Error processing append operation on partition %s".format(topicAndPartition), e) + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(e))) + } } } } diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index 8531f533f3a64..a913fe59ba6f7 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -15,27 +15,27 @@ * limitations under the License. */ -package kafka.api +package kafka.api.test -import kafka.common.Topic -import org.apache.kafka.common.errors.{InvalidTopicException,NotEnoughReplicasException} -import org.scalatest.junit.JUnit3Suite import org.junit.Test import org.junit.Assert._ -import java.util.{Properties, Random} import java.lang.Integer +import java.util.{Properties, Random} import java.util.concurrent.{TimeoutException, TimeUnit, ExecutionException} +import kafka.api.FetchRequestBuilder +import kafka.common.Topic +import kafka.consumer.SimpleConsumer import kafka.server.KafkaConfig -import kafka.utils.{TestZKUtils, ShutdownableThread, TestUtils} import kafka.integration.KafkaServerTestHarness -import kafka.consumer.SimpleConsumer +import kafka.utils.{TestZKUtils, ShutdownableThread, TestUtils} import org.apache.kafka.common.KafkaException +import org.apache.kafka.common.errors.{InvalidTopicException, NotEnoughReplicasException} import org.apache.kafka.clients.producer._ -class ProducerFailureHandlingTest extends JUnit3Suite with KafkaServerTestHarness { +class ProducerFailureHandlingTest extends KafkaServerTestHarness { private val producerBufferSize = 30000 private val serverMessageMaxBytes = producerBufferSize/2 @@ -297,9 +297,12 @@ class ProducerFailureHandlingTest extends JUnit3Suite with KafkaServerTestHarnes assertEquals("Should have fetched " + scheduler.sent + " unique messages", scheduler.sent, uniqueMessageSize) } - @Test(expected = classOf[InvalidTopicException]) + @Test def testCannotSendToInternalTopic() { - producer1.send(new ProducerRecord(Topic.InternalTopics.head, "test".getBytes, "test".getBytes)).get + val thrown = intercept[ExecutionException] { + producer2.send(new ProducerRecord(Topic.InternalTopics.head, "test".getBytes, "test".getBytes)).get + } + assertTrue(thrown.getCause.isInstanceOf[InvalidTopicException]) } @Test From 834b6419806750ff555e8ffd104caa420004e84e Mon Sep 17 00:00:00 2001 From: Dmytro Kostiuchenko Date: Tue, 25 Nov 2014 14:36:31 -0800 Subject: [PATCH 059/491] kafka-1667; topic-level configuration not validated; patched by Dmytro Kostiuchenko; reviewed by Jun Rao --- .../apache/kafka/common/config/ConfigDef.java | 9 ++ .../main/scala/kafka/admin/TopicCommand.scala | 2 +- core/src/main/scala/kafka/log/LogConfig.scala | 144 +++++++++++------- core/src/main/scala/kafka/utils/Utils.scala | 26 ++++ .../test/scala/kafka/log/LogConfigTest.scala | 93 +++++++++++ .../UncleanLeaderElectionTest.scala | 4 +- 6 files changed, 218 insertions(+), 60 deletions(-) create mode 100644 core/src/test/scala/kafka/log/LogConfigTest.scala diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java index c4cea2cc072f4..347e25280c3b3 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java @@ -19,6 +19,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; /** * This class is used for specifying the set of expected configurations, their type, their defaults, their @@ -48,6 +49,14 @@ public class ConfigDef { private final Map configKeys = new HashMap(); + /** + * Returns unmodifiable set of properties names defined in this {@linkplain ConfigDef} + * @return new unmodifiable {@link Set} instance containing the keys + */ + public Set names() { + return Collections.unmodifiableSet(configKeys.keySet()); + } + /** * Define a new configuration * @param name The name of the config parameter diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index 0b2735e7fc42e..285c0333ff435 100644 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -256,7 +256,7 @@ object TopicCommand { .ofType(classOf[String]) val nl = System.getProperty("line.separator") val configOpt = parser.accepts("config", "A topic configuration override for the topic being created or altered." + - "The following is a list of valid configurations: " + nl + LogConfig.ConfigNames.map("\t" + _).mkString(nl) + nl + + "The following is a list of valid configurations: " + nl + LogConfig.configNames.map("\t" + _).mkString(nl) + nl + "See the Kafka documentation for full details on the topic configs.") .withRequiredArg .describedAs("name=value") diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index e48922a97727d..f2fbc5565f4cc 100644 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -21,7 +21,7 @@ import java.util.Properties import org.apache.kafka.common.utils.Utils import scala.collection._ -import kafka.common._ +import org.apache.kafka.common.config.ConfigDef object Defaults { val SegmentSize = 1024 * 1024 @@ -44,21 +44,20 @@ object Defaults { /** * Configuration settings for a log - * @param segmentSize The soft maximum for the size of a segment file in the log + * @param segmentSize The hard maximum for the size of a segment file in the log * @param segmentMs The soft maximum on the amount of time before a new log segment is rolled * @param segmentJitterMs The maximum random jitter subtracted from segmentMs to avoid thundering herds of segment rolling * @param flushInterval The number of messages that can be written to the log before a flush is forced * @param flushMs The amount of time the log can have dirty data before a flush is forced * @param retentionSize The approximate total number of bytes this log can use - * @param retentionMs The age approximate maximum age of the last segment that is retained + * @param retentionMs The approximate maximum age of the last segment that is retained * @param maxIndexSize The maximum size of an index file * @param indexInterval The approximate number of bytes between index entries * @param fileDeleteDelayMs The time to wait before deleting a file from the filesystem * @param deleteRetentionMs The time to retain delete markers in the log. Only applicable for logs that are being compacted. * @param minCleanableRatio The ratio of bytes that are available for cleaning to the bytes already cleaned * @param compact Should old segments in this log be deleted or deduplicated? - * @param uncleanLeaderElectionEnable Indicates whether unclean leader election is enabled; actually a controller-level property - * but included here for topic-specific configuration validation purposes + * @param uncleanLeaderElectionEnable Indicates whether unclean leader election is enabled * @param minInSyncReplicas If number of insync replicas drops below this number, we stop accepting writes with -1 (or all) required acks * */ @@ -106,6 +105,10 @@ case class LogConfig(val segmentSize: Int = Defaults.SegmentSize, } object LogConfig { + + val Delete = "delete" + val Compact = "compact" + val SegmentBytesProp = "segment.bytes" val SegmentMsProp = "segment.ms" val SegmentJitterMsProp = "segment.jitter.ms" @@ -123,46 +126,84 @@ object LogConfig { val UncleanLeaderElectionEnableProp = "unclean.leader.election.enable" val MinInSyncReplicasProp = "min.insync.replicas" - val ConfigNames = Set(SegmentBytesProp, - SegmentMsProp, - SegmentJitterMsProp, - SegmentIndexBytesProp, - FlushMessagesProp, - FlushMsProp, - RetentionBytesProp, - RententionMsProp, - MaxMessageBytesProp, - IndexIntervalBytesProp, - FileDeleteDelayMsProp, - DeleteRetentionMsProp, - MinCleanableDirtyRatioProp, - CleanupPolicyProp, - UncleanLeaderElectionEnableProp, - MinInSyncReplicasProp) + val SegmentSizeDoc = "The hard maximum for the size of a segment file in the log" + val SegmentMsDoc = "The soft maximum on the amount of time before a new log segment is rolled" + val SegmentJitterMsDoc = "The maximum random jitter subtracted from segmentMs to avoid thundering herds of segment" + + " rolling" + val FlushIntervalDoc = "The number of messages that can be written to the log before a flush is forced" + val FlushMsDoc = "The amount of time the log can have dirty data before a flush is forced" + val RetentionSizeDoc = "The approximate total number of bytes this log can use" + val RetentionMsDoc = "The approximate maximum age of the last segment that is retained" + val MaxIndexSizeDoc = "The maximum size of an index file" + val MaxMessageSizeDoc = "The maximum size of a message" + val IndexIntervalDoc = "The approximate number of bytes between index entries" + val FileDeleteDelayMsDoc = "The time to wait before deleting a file from the filesystem" + val DeleteRetentionMsDoc = "The time to retain delete markers in the log. Only applicable for logs that are being" + + " compacted." + val MinCleanableRatioDoc = "The ratio of bytes that are available for cleaning to the bytes already cleaned" + val CompactDoc = "Should old segments in this log be deleted or deduplicated?" + val UncleanLeaderElectionEnableDoc = "Indicates whether unclean leader election is enabled" + val MinInSyncReplicasDoc = "If number of insync replicas drops below this number, we stop accepting writes with" + + " -1 (or all) required acks" + + private val configDef = { + import ConfigDef.Range._ + import ConfigDef.ValidString._ + import ConfigDef.Type._ + import ConfigDef.Importance._ + import java.util.Arrays.asList + + new ConfigDef() + .define(SegmentBytesProp, INT, Defaults.SegmentSize, atLeast(0), MEDIUM, SegmentSizeDoc) + .define(SegmentMsProp, LONG, Defaults.SegmentMs, atLeast(0), MEDIUM, SegmentMsDoc) + .define(SegmentJitterMsProp, LONG, Defaults.SegmentJitterMs, atLeast(0), MEDIUM, SegmentJitterMsDoc) + .define(SegmentIndexBytesProp, INT, Defaults.MaxIndexSize, atLeast(0), MEDIUM, MaxIndexSizeDoc) + .define(FlushMessagesProp, LONG, Defaults.FlushInterval, atLeast(0), MEDIUM, FlushIntervalDoc) + .define(FlushMsProp, LONG, Defaults.FlushMs, atLeast(0), MEDIUM, FlushMsDoc) + // can be negative. See kafka.log.LogManager.cleanupSegmentsToMaintainSize + .define(RetentionBytesProp, LONG, Defaults.RetentionSize, MEDIUM, RetentionSizeDoc) + .define(RententionMsProp, LONG, Defaults.RetentionMs, atLeast(0), MEDIUM, RetentionMsDoc) + .define(MaxMessageBytesProp, INT, Defaults.MaxMessageSize, atLeast(0), MEDIUM, MaxMessageSizeDoc) + .define(IndexIntervalBytesProp, INT, Defaults.IndexInterval, atLeast(0), MEDIUM, IndexIntervalDoc) + .define(DeleteRetentionMsProp, LONG, Defaults.DeleteRetentionMs, atLeast(0), MEDIUM, DeleteRetentionMsDoc) + .define(FileDeleteDelayMsProp, LONG, Defaults.FileDeleteDelayMs, atLeast(0), MEDIUM, FileDeleteDelayMsDoc) + .define(MinCleanableDirtyRatioProp, DOUBLE, Defaults.MinCleanableDirtyRatio, between(0, 1), MEDIUM, + MinCleanableRatioDoc) + .define(CleanupPolicyProp, STRING, if (Defaults.Compact) Compact else Delete, in(asList(Compact, Delete)), MEDIUM, + CompactDoc) + // we validate true/false explicitly to fail in case of typo + .define(UncleanLeaderElectionEnableProp, STRING, Defaults.UncleanLeaderElectionEnable.toString, + in(asList(true.toString, false.toString)), MEDIUM, UncleanLeaderElectionEnableDoc) + .define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), MEDIUM, MinInSyncReplicasDoc) + } + + def configNames() = { + import JavaConversions._ + configDef.names().toList.sorted + } /** * Parse the given properties instance into a LogConfig object */ def fromProps(props: Properties): LogConfig = { - new LogConfig(segmentSize = props.getProperty(SegmentBytesProp, Defaults.SegmentSize.toString).toInt, - segmentMs = props.getProperty(SegmentMsProp, Defaults.SegmentMs.toString).toLong, - segmentJitterMs = props.getProperty(SegmentJitterMsProp, Defaults.SegmentJitterMs.toString).toLong, - maxIndexSize = props.getProperty(SegmentIndexBytesProp, Defaults.MaxIndexSize.toString).toInt, - flushInterval = props.getProperty(FlushMessagesProp, Defaults.FlushInterval.toString).toLong, - flushMs = props.getProperty(FlushMsProp, Defaults.FlushMs.toString).toLong, - retentionSize = props.getProperty(RetentionBytesProp, Defaults.RetentionSize.toString).toLong, - retentionMs = props.getProperty(RententionMsProp, Defaults.RetentionMs.toString).toLong, - maxMessageSize = props.getProperty(MaxMessageBytesProp, Defaults.MaxMessageSize.toString).toInt, - indexInterval = props.getProperty(IndexIntervalBytesProp, Defaults.IndexInterval.toString).toInt, - fileDeleteDelayMs = props.getProperty(FileDeleteDelayMsProp, Defaults.FileDeleteDelayMs.toString).toInt, - deleteRetentionMs = props.getProperty(DeleteRetentionMsProp, Defaults.DeleteRetentionMs.toString).toLong, - minCleanableRatio = props.getProperty(MinCleanableDirtyRatioProp, - Defaults.MinCleanableDirtyRatio.toString).toDouble, - compact = props.getProperty(CleanupPolicyProp, if(Defaults.Compact) "compact" else "delete") - .trim.toLowerCase != "delete", - uncleanLeaderElectionEnable = props.getProperty(UncleanLeaderElectionEnableProp, - Defaults.UncleanLeaderElectionEnable.toString).toBoolean, - minInSyncReplicas = props.getProperty(MinInSyncReplicasProp,Defaults.MinInSyncReplicas.toString).toInt) + import kafka.utils.Utils.evaluateDefaults + val parsed = configDef.parse(evaluateDefaults(props)) + new LogConfig(segmentSize = parsed.get(SegmentBytesProp).asInstanceOf[Int], + segmentMs = parsed.get(SegmentMsProp).asInstanceOf[Long], + segmentJitterMs = parsed.get(SegmentJitterMsProp).asInstanceOf[Long], + maxIndexSize = parsed.get(SegmentIndexBytesProp).asInstanceOf[Int], + flushInterval = parsed.get(FlushMessagesProp).asInstanceOf[Long], + flushMs = parsed.get(FlushMsProp).asInstanceOf[Long], + retentionSize = parsed.get(RetentionBytesProp).asInstanceOf[Long], + retentionMs = parsed.get(RententionMsProp).asInstanceOf[Long], + maxMessageSize = parsed.get(MaxMessageBytesProp).asInstanceOf[Int], + indexInterval = parsed.get(IndexIntervalBytesProp).asInstanceOf[Int], + fileDeleteDelayMs = parsed.get(FileDeleteDelayMsProp).asInstanceOf[Long], + deleteRetentionMs = parsed.get(DeleteRetentionMsProp).asInstanceOf[Long], + minCleanableRatio = parsed.get(MinCleanableDirtyRatioProp).asInstanceOf[Double], + compact = parsed.get(CleanupPolicyProp).asInstanceOf[String].toLowerCase != Delete, + uncleanLeaderElectionEnable = parsed.get(UncleanLeaderElectionEnableProp).asInstanceOf[String].toBoolean, + minInSyncReplicas = parsed.get(MinInSyncReplicasProp).asInstanceOf[Int]) } /** @@ -179,30 +220,17 @@ object LogConfig { */ def validateNames(props: Properties) { import JavaConversions._ + val names = configDef.names() for(name <- props.keys) - require(LogConfig.ConfigNames.contains(name), "Unknown configuration \"%s\".".format(name)) + require(names.contains(name), "Unknown configuration \"%s\".".format(name)) } /** - * Check that the given properties contain only valid log config names, and that all values can be parsed. + * Check that the given properties contain only valid log config names and that all values can be parsed and are valid */ def validate(props: Properties) { validateNames(props) - validateMinInSyncReplicas(props) - LogConfig.fromProps(LogConfig().toProps, props) // check that we can parse the values - } - - /** - * Check that MinInSyncReplicas is reasonable - * Unfortunately, we can't validate its smaller than number of replicas - * since we don't have this information here - */ - private def validateMinInSyncReplicas(props: Properties) { - val minIsr = props.getProperty(MinInSyncReplicasProp) - if (minIsr != null && minIsr.toInt < 1) { - throw new InvalidConfigException("Wrong value " + minIsr + " of min.insync.replicas in topic configuration; " + - " Valid values are at least 1") - } + configDef.parse(props) } -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala index 23aefb4715b17..58685cc47b4c4 100644 --- a/core/src/main/scala/kafka/utils/Utils.scala +++ b/core/src/main/scala/kafka/utils/Utils.scala @@ -505,6 +505,32 @@ object Utils extends Logging { props.store(writer, "") writer.toString } + + /** + * Turn {@linkplain java.util.Properties} with default values into a {@linkplain java.util.Map}. Following example + * illustrates difference from the cast + *

      +   * val defaults = new Properties()
      +   * defaults.put("foo", "bar")
      +   * val props = new Properties(defaults)
      +   *
      +   * props.getProperty("foo") // "bar"
      +   * props.get("foo") // null
      +   * evaluateDefaults(props).get("foo") // "bar"
      +   * 
      + * + * @param props properties to evaluate + * @return new java.util.Map instance + */ + def evaluateDefaults(props: Properties): java.util.Map[String, String] = { + import java.util._ + import JavaConversions.asScalaSet + val evaluated = new HashMap[String, String]() + for (name <- props.stringPropertyNames()) { + evaluated.put(name, props.getProperty(name)) + } + evaluated + } /** * Read some properties with the given default values diff --git a/core/src/test/scala/kafka/log/LogConfigTest.scala b/core/src/test/scala/kafka/log/LogConfigTest.scala new file mode 100644 index 0000000000000..99b0df7b69c5e --- /dev/null +++ b/core/src/test/scala/kafka/log/LogConfigTest.scala @@ -0,0 +1,93 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log + +import org.apache.kafka.common.config.ConfigException +import org.scalatest.junit.JUnit3Suite +import org.junit.{Assert, Test} +import java.util.Properties + +class LogConfigTest extends JUnit3Suite { + + @Test + def testFromPropsDefaults() { + val defaults = new Properties() + defaults.put(LogConfig.SegmentBytesProp, "4242") + val props = new Properties(defaults) + + val config = LogConfig.fromProps(props) + + Assert.assertEquals(4242, config.segmentSize) + Assert.assertEquals("LogConfig defaults should be retained", Defaults.MaxMessageSize, config.maxMessageSize) + } + + @Test + def testFromPropsEmpty() { + val p = new Properties() + val config = LogConfig.fromProps(p) + Assert.assertEquals(LogConfig(), config) + } + + @Test + def testFromPropsToProps() { + import scala.util.Random._ + val expected = new Properties() + LogConfig.configNames().foreach((name) => { + name match { + case LogConfig.UncleanLeaderElectionEnableProp => expected.setProperty(name, randFrom("true", "false")) + case LogConfig.CleanupPolicyProp => expected.setProperty(name, randFrom(LogConfig.Compact, LogConfig.Delete)) + case LogConfig.MinCleanableDirtyRatioProp => expected.setProperty(name, "%.1f".format(nextDouble * .9 + .1)) + case LogConfig.MinInSyncReplicasProp => expected.setProperty(name, (nextInt(Int.MaxValue - 1) + 1).toString) + case LogConfig.RetentionBytesProp => expected.setProperty(name, nextInt().toString) + case positiveIntProperty => expected.setProperty(name, nextInt(Int.MaxValue).toString) + } + }) + + val actual = LogConfig.fromProps(expected).toProps + Assert.assertEquals(expected, actual) + } + + @Test + def testFromPropsInvalid() { + LogConfig.configNames().foreach((name) => { + name match { + case LogConfig.UncleanLeaderElectionEnableProp => return + case LogConfig.RetentionBytesProp => assertPropertyInvalid(name, "not_a_number") + case LogConfig.CleanupPolicyProp => assertPropertyInvalid(name, "true", "foobar"); + case LogConfig.MinCleanableDirtyRatioProp => assertPropertyInvalid(name, "not_a_number", "-0.1", "1.2") + case LogConfig.MinInSyncReplicasProp => assertPropertyInvalid(name, "not_a_number", "0", "-1") + case positiveIntProperty => assertPropertyInvalid(name, "not_a_number", "-1") + } + }) + } + + private def assertPropertyInvalid(name: String, values: AnyRef*) { + values.foreach((value) => { + val props = new Properties + props.setProperty(name, value.toString) + intercept[ConfigException] { + LogConfig.fromProps(props) + } + }) + } + + private def randFrom[T](choices: T*): T = { + import scala.util.Random + choices(Random.nextInt(choices.size)) + } +} diff --git a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala index f44568cb25edf..ba3bcdcd1de98 100644 --- a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala @@ -17,6 +17,8 @@ package kafka.integration +import org.apache.kafka.common.config.ConfigException + import scala.collection.mutable.MutableList import scala.util.Random import org.apache.log4j.{Level, Logger} @@ -155,7 +157,7 @@ class UncleanLeaderElectionTest extends JUnit3Suite with ZooKeeperTestHarness { val topicProps = new Properties() topicProps.put("unclean.leader.election.enable", "invalid") - intercept[IllegalArgumentException] { + intercept[ConfigException] { AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1)), topicProps) } } From 4408f487ae911e29dbc6ef4d94010a6fad702109 Mon Sep 17 00:00:00 2001 From: Jean-Francois Im Date: Mon, 1 Dec 2014 10:09:58 -0800 Subject: [PATCH 060/491] KAFKA-1783; Add missing slash in documentation for consumer's ZK path; reviewed by Guozhang Wang --- .../main/scala/kafka/consumer/ZookeeperConsumerConnector.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index 3e1718bc7ca6c..da29a8cb46109 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -46,7 +46,7 @@ import scala.collection._ * * Directories: * 1. Consumer id registry: - * /consumers/[group_id]/ids[consumer_id] -> topic1,...topicN + * /consumers/[group_id]/ids/[consumer_id] -> topic1,...topicN * A consumer has a unique consumer id within a consumer group. A consumer registers its id as an ephemeral znode * and puts all topics that it subscribes to as the value of the znode. The znode is deleted when the client is gone. * A consumer subscribes to event changes of the consumer id registry within its group. From 8a719e037201e343e22a3d2479651e781ddaccc1 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Mon, 1 Dec 2014 16:02:47 -0800 Subject: [PATCH 061/491] kafka-1799; ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG doesn't work; patched by Manikumar Reddy; reviewed by Jun Rao --- .../org/apache/kafka/common/config/AbstractConfig.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java index 8d88610534155..3d4ab7228926f 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java @@ -144,7 +144,12 @@ public List getConfiguredInstances(String key, Class t) { List klasses = getList(key); List objects = new ArrayList(); for (String klass : klasses) { - Class c = getClass(klass); + Class c; + try { + c = Class.forName(klass); + } catch (ClassNotFoundException e) { + throw new ConfigException(key, klass, "Class " + klass + " could not be found."); + } if (c == null) return null; Object o = Utils.newInstance(c); From 74bc8860c3c3d582f88581fc7d9edffdfa2dc3a5 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Mon, 1 Dec 2014 16:04:01 -0800 Subject: [PATCH 062/491] kafka-1799; (add missing test file) ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG doesn't work; patched by Manikumar Reddy; reviewed by Jun Rao --- .../common/config/AbstractConfigTest.java | 100 ++++++++++++++++++ 1 file changed, 100 insertions(+) create mode 100644 clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java diff --git a/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java b/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java new file mode 100644 index 0000000000000..3cfd36d992f5f --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java @@ -0,0 +1,100 @@ +/** + * 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.config; + +import static org.junit.Assert.fail; + +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.Type; +import org.apache.kafka.common.metrics.KafkaMetric; +import org.apache.kafka.common.metrics.MetricsReporter; +import org.junit.Test; + +public class AbstractConfigTest { + + @Test + public void testConfiguredInstances() { + testValidInputs(""); + testValidInputs("org.apache.kafka.common.config.AbstractConfigTest$TestMetricsReporter"); + testValidInputs("org.apache.kafka.common.config.AbstractConfigTest$TestMetricsReporter,org.apache.kafka.common.config.AbstractConfigTest$TestMetricsReporter"); + testInvalidInputs(","); + testInvalidInputs("org.apache.kafka.clients.producer.unknown-metrics-reporter"); + testInvalidInputs("test1,test2"); + testInvalidInputs("org.apache.kafka.common.config.AbstractConfigTest$TestMetricsReporter,"); + } + + private void testValidInputs(String configValue) { + Properties props = new Properties(); + props.put(TestConfig.METRIC_REPORTER_CLASSES_CONFIG, configValue); + TestConfig config = new TestConfig(props); + try { + config.getConfiguredInstances(TestConfig.METRIC_REPORTER_CLASSES_CONFIG, + MetricsReporter.class); + } catch (ConfigException e) { + fail("No exceptions are expected here, valid props are :" + props); + } + } + + private void testInvalidInputs(String configValue) { + Properties props = new Properties(); + props.put(TestConfig.METRIC_REPORTER_CLASSES_CONFIG, configValue); + TestConfig config = new TestConfig(props); + try { + config.getConfiguredInstances(TestConfig.METRIC_REPORTER_CLASSES_CONFIG, + MetricsReporter.class); + fail("Expected a config exception due to invalid props :" + props); + } catch (ConfigException e) { + // this is good + } + } + + private static class TestConfig extends AbstractConfig { + + private static final ConfigDef config; + + public static final String METRIC_REPORTER_CLASSES_CONFIG = "metric.reporters"; + private static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters."; + + static { + config = new ConfigDef().define(METRIC_REPORTER_CLASSES_CONFIG, + Type.LIST, "", Importance.LOW, METRIC_REPORTER_CLASSES_DOC); + } + + public TestConfig(Map props) { + super(config, props); + } + } + + public static class TestMetricsReporter implements MetricsReporter { + + @Override + public void configure(Map configs) { + } + + @Override + public void init(List metrics) { +} + + @Override + public void metricChange(KafkaMetric metric) { + } + + @Override + public void close() { + } + } +} From b24f9c0890302a5149aaaf26dc933bd0a842f1be Mon Sep 17 00:00:00 2001 From: dokovan Date: Mon, 1 Dec 2014 16:24:31 -0800 Subject: [PATCH 063/491] kafka-1798; ConfigDef.parseType() should throw exception on invalid boolean value; patched by dokovan; reviewed by Jun Rao --- .../java/org/apache/kafka/common/config/ConfigDef.java | 10 ++++++++-- .../org/apache/kafka/common/config/ConfigDefTest.java | 9 ++++++++- 2 files changed, 16 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java index 347e25280c3b3..98cb79b701918 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java @@ -154,8 +154,14 @@ private Object parseType(String name, Object value, Type type) { trimmed = ((String) value).trim(); switch (type) { case BOOLEAN: - if (value instanceof String) - return Boolean.parseBoolean(trimmed); + if (value instanceof String) { + if (trimmed.equalsIgnoreCase("true")) + return true; + else if (trimmed.equalsIgnoreCase("false")) + return false; + else + throw new ConfigException(name, value, "Expected value to be either true or false"); + } else if (value instanceof Boolean) return value; else diff --git a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java index 7869e467ab624..3c442a27a7ba3 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java @@ -38,7 +38,9 @@ public void testBasicTypes() { .define("d", Type.LIST, Importance.HIGH, "docs") .define("e", Type.DOUBLE, Importance.HIGH, "docs") .define("f", Type.CLASS, Importance.HIGH, "docs") - .define("g", Type.BOOLEAN, Importance.HIGH, "docs"); + .define("g", Type.BOOLEAN, Importance.HIGH, "docs") + .define("h", Type.BOOLEAN, Importance.HIGH, "docs") + .define("i", Type.BOOLEAN, Importance.HIGH, "docs"); Properties props = new Properties(); props.put("a", "1 "); @@ -47,6 +49,8 @@ public void testBasicTypes() { props.put("e", 42.5d); props.put("f", String.class.getName()); props.put("g", "true"); + props.put("h", "FalSE"); + props.put("i", "TRUE"); Map vals = def.parse(props); assertEquals(1, vals.get("a")); @@ -56,6 +60,8 @@ public void testBasicTypes() { assertEquals(42.5d, vals.get("e")); assertEquals(String.class, vals.get("f")); assertEquals(true, vals.get("g")); + assertEquals(false, vals.get("h")); + assertEquals(true, vals.get("i")); } @Test(expected = ConfigException.class) @@ -85,6 +91,7 @@ public void testBadInputs() { testBadInputs(Type.DOUBLE, "hello", null, new Object()); testBadInputs(Type.STRING, new Object()); testBadInputs(Type.LIST, 53, new Object()); + testBadInputs(Type.BOOLEAN, "hello", "truee", "fals"); } private void testBadInputs(Type type, Object... values) { From 723fa4c40d2d04a8557ef9a0ae1626e56f30a9a3 Mon Sep 17 00:00:00 2001 From: Balaji Seshadri Date: Tue, 2 Dec 2014 15:17:20 -0800 Subject: [PATCH 064/491] KAFKA-328 Write unit test for kafka server startup and shutdown API; reviewed by Neha Narkhede --- .../unit/kafka/server/ServerShutdownTest.scala | 14 ++++++++++++++ .../unit/kafka/server/ServerStartupTest.scala | 16 ++++++++++++++-- 2 files changed, 28 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index 1bfb501b2f29c..67918f2842d47 100644 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -146,4 +146,18 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { .map(_.asInstanceOf[Thread]) .count(t => !t.isDaemon && t.isAlive && t.getClass.getCanonicalName.toLowerCase.startsWith("kafka"))) } + + def testConsecutiveShutdown(){ + val server = new KafkaServer(config) + try { + server.startup() + server.shutdown() + server.awaitShutdown() + server.shutdown() + assertTrue(true); + } + catch{ + case ex => fail() + } + } } diff --git a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala index a0ed4855f2550..8fe7cd496f74a 100644 --- a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala @@ -18,7 +18,6 @@ package kafka.server import org.scalatest.junit.JUnit3Suite -import kafka.zk import kafka.utils.ZkUtils import kafka.utils.Utils import kafka.utils.TestUtils @@ -36,7 +35,6 @@ class ServerStartupTest extends JUnit3Suite with ZooKeeperTestHarness { val props = TestUtils.createBrokerConfig(brokerId, TestUtils.choosePort()) val zooKeeperConnect = props.get("zookeeper.connect") props.put("zookeeper.connect", zooKeeperConnect + zookeeperChroot) - server = TestUtils.createServer(new KafkaConfig(props)) } @@ -51,4 +49,18 @@ class ServerStartupTest extends JUnit3Suite with ZooKeeperTestHarness { assertTrue(pathExists) } + def testServerStartupConsecutively() { + server.shutdown() + try { + intercept[IllegalStateException]{ + server.startup() + server.startup() + } + } + finally { + server.shutdown() + } + + } + } \ No newline at end of file From 15f9e2762d32a0345a5ca13a1e981be646856b41 Mon Sep 17 00:00:00 2001 From: Raman Gupta Date: Tue, 2 Dec 2014 16:35:02 -0800 Subject: [PATCH 065/491] kafka-1801; Remove non-functional variable definition in log4j.properties; patched by Raman Gupta; reviewed by Jun Rao --- config/log4j.properties | 2 -- 1 file changed, 2 deletions(-) diff --git a/config/log4j.properties b/config/log4j.properties index 95022543debed..c51ab8b6b20d2 100644 --- a/config/log4j.properties +++ b/config/log4j.properties @@ -13,8 +13,6 @@ # See the License for the specific language governing permissions and # limitations under the License. -kafka.logs.dir=logs - log4j.rootLogger=INFO, stdout log4j.appender.stdout=org.apache.log4j.ConsoleAppender From 7e9368baf2e1cd457293b815a665e5d3e4418dfb Mon Sep 17 00:00:00 2001 From: Dave Parfitt Date: Tue, 2 Dec 2014 17:53:21 -0800 Subject: [PATCH 066/491] kafka-1803; UncleanLeaderElectionEnableProp in LogConfig should be of boolean; patched by Dave Parfitt; reviewed by Jun Rao --- core/src/main/scala/kafka/log/LogConfig.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index f2fbc5565f4cc..ca7a99e99f641 100644 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -171,9 +171,8 @@ object LogConfig { MinCleanableRatioDoc) .define(CleanupPolicyProp, STRING, if (Defaults.Compact) Compact else Delete, in(asList(Compact, Delete)), MEDIUM, CompactDoc) - // we validate true/false explicitly to fail in case of typo - .define(UncleanLeaderElectionEnableProp, STRING, Defaults.UncleanLeaderElectionEnable.toString, - in(asList(true.toString, false.toString)), MEDIUM, UncleanLeaderElectionEnableDoc) + .define(UncleanLeaderElectionEnableProp, BOOLEAN, Defaults.UncleanLeaderElectionEnable, + MEDIUM, UncleanLeaderElectionEnableDoc) .define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), MEDIUM, MinInSyncReplicasDoc) } @@ -202,7 +201,7 @@ object LogConfig { deleteRetentionMs = parsed.get(DeleteRetentionMsProp).asInstanceOf[Long], minCleanableRatio = parsed.get(MinCleanableDirtyRatioProp).asInstanceOf[Double], compact = parsed.get(CleanupPolicyProp).asInstanceOf[String].toLowerCase != Delete, - uncleanLeaderElectionEnable = parsed.get(UncleanLeaderElectionEnableProp).asInstanceOf[String].toBoolean, + uncleanLeaderElectionEnable = parsed.get(UncleanLeaderElectionEnableProp).asInstanceOf[Boolean], minInSyncReplicas = parsed.get(MinInSyncReplicasProp).asInstanceOf[Int]) } From 3cc10d5ff9bf73cff2045685f89d71fee92a41f4 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 4 Dec 2014 13:51:44 -0800 Subject: [PATCH 067/491] KAFKA-1720; Renamed Delayed Operations after KAFKA-1583; reviewed by Gwen Shapira and Joel Koshy --- .../main/scala/kafka/cluster/Partition.scala | 14 +++--- .../scala/kafka/server/DelayedFetch.scala | 10 ++-- ...Purgatory.scala => DelayedOperation.scala} | 47 +++++++++---------- .../kafka/server/DelayedOperationKey.scala | 38 +++++++++++++++ .../scala/kafka/server/DelayedProduce.scala | 14 +++--- .../scala/kafka/server/ReplicaManager.scala | 42 ++++++++--------- ...yTest.scala => DelayedOperationTest.scala} | 26 +++++----- system_test/metrics.json | 4 +- 8 files changed, 116 insertions(+), 79 deletions(-) rename core/src/main/scala/kafka/server/{RequestPurgatory.scala => DelayedOperation.scala} (86%) create mode 100644 core/src/main/scala/kafka/server/DelayedOperationKey.scala rename core/src/test/scala/unit/kafka/server/{RequestPurgatoryTest.scala => DelayedOperationTest.scala} (82%) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index b9fde2aacbb1f..b230e9a1fb1a3 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -17,18 +17,18 @@ package kafka.cluster import kafka.common._ -import kafka.admin.AdminUtils import kafka.utils._ +import kafka.utils.Utils.{inReadLock,inWriteLock} +import kafka.admin.AdminUtils import kafka.api.{PartitionStateInfo, LeaderAndIsr} import kafka.log.LogConfig -import kafka.server.{LogOffsetMetadata, OffsetManager, ReplicaManager} +import kafka.server.{TopicPartitionOperationKey, LogOffsetMetadata, OffsetManager, ReplicaManager} import kafka.metrics.KafkaMetricsGroup import kafka.controller.KafkaController import kafka.message.ByteBufferMessageSet import java.io.IOException import java.util.concurrent.locks.ReentrantReadWriteLock -import kafka.utils.Utils.{inReadLock,inWriteLock} import scala.collection.immutable.Set import com.yammer.metrics.core.Gauge @@ -232,7 +232,7 @@ class Partition(val topic: String, /** * Update the log end offset of a certain replica of this partition */ - def updateReplicaLEO(replicaId: Int, offset: LogOffsetMetadata) = { + def updateReplicaLEO(replicaId: Int, offset: LogOffsetMetadata) { getReplica(replicaId) match { case Some(replica) => replica.logEndOffset = offset @@ -343,8 +343,8 @@ class Partition(val topic: String, if(oldHighWatermark.precedes(newHighWatermark)) { leaderReplica.highWatermark = newHighWatermark debug("High watermark for partition [%s,%d] updated to %s".format(topic, partitionId, newHighWatermark)) - // some delayed requests may be unblocked after HW changed - val requestKey = new TopicAndPartition(this.topic, this.partitionId) + // some delayed operations may be unblocked after HW changed + val requestKey = new TopicPartitionOperationKey(this.topic, this.partitionId) replicaManager.tryCompleteDelayedFetch(requestKey) replicaManager.tryCompleteDelayedProduce(requestKey) } else { @@ -414,7 +414,7 @@ class Partition(val topic: String, val info = log.append(messages, assignOffsets = true) // probably unblock some follower fetch requests since log end offset has been updated - replicaManager.tryCompleteDelayedFetch(new TopicAndPartition(this.topic, this.partitionId)) + replicaManager.tryCompleteDelayedFetch(new TopicPartitionOperationKey(this.topic, this.partitionId)) // we may need to increment high watermark since ISR could be down to 1 maybeIncrementLeaderHW(leaderReplica) info diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index 1e2e56f87a3c2..dd602ee2e65c2 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -32,7 +32,7 @@ case class FetchPartitionStatus(startOffsetMetadata: LogOffsetMetadata, fetchInf } /** - * The fetch metadata maintained by the delayed produce request + * The fetch metadata maintained by the delayed fetch operation */ case class FetchMetadata(fetchMinBytes: Int, fetchOnlyLeader: Boolean, @@ -45,17 +45,17 @@ case class FetchMetadata(fetchMinBytes: Int, "partitionStatus: " + fetchPartitionStatus + "]" } /** - * A delayed fetch request that can be created by the replica manager and watched - * in the fetch request purgatory + * A delayed fetch operation that can be created by the replica manager and watched + * in the fetch operation purgatory */ class DelayedFetch(delayMs: Long, fetchMetadata: FetchMetadata, replicaManager: ReplicaManager, responseCallback: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) - extends DelayedRequest(delayMs) { + extends DelayedOperation(delayMs) { /** - * The request can be completed if: + * The operation can be completed if: * * Case A: This broker is no longer the leader for some partitions it tries to fetch * Case B: This broker does not know of some partitions it tries to fetch diff --git a/core/src/main/scala/kafka/server/RequestPurgatory.scala b/core/src/main/scala/kafka/server/DelayedOperation.scala similarity index 86% rename from core/src/main/scala/kafka/server/RequestPurgatory.scala rename to core/src/main/scala/kafka/server/DelayedOperation.scala index 323b12e765f98..fc06b01cad3a0 100644 --- a/core/src/main/scala/kafka/server/RequestPurgatory.scala +++ b/core/src/main/scala/kafka/server/DelayedOperation.scala @@ -39,9 +39,9 @@ import com.yammer.metrics.core.Gauge * or tryComplete(), which first checks if the operation can be completed or not now, and if yes calls * forceComplete(). * - * A subclass of DelayedRequest needs to provide an implementation of both onComplete() and tryComplete(). + * A subclass of DelayedOperation needs to provide an implementation of both onComplete() and tryComplete(). */ -abstract class DelayedRequest(delayMs: Long) extends DelayedItem(delayMs) { +abstract class DelayedOperation(delayMs: Long) extends DelayedItem(delayMs) { private val completed = new AtomicBoolean(false) /* @@ -51,7 +51,10 @@ abstract class DelayedRequest(delayMs: Long) extends DelayedItem(delayMs) { * 1. The operation has been verified to be completable inside tryComplete() * 2. The operation has expired and hence needs to be completed right now * - * Return true iff the operation is completed by the caller + * Return true iff the operation is completed by the caller: note that + * concurrent threads can try to complete the same operation, but only + * the first thread will succeed in completing the operation and return + * true, others will still return false */ def forceComplete(): Boolean = { if (completed.compareAndSet(false, true)) { @@ -68,8 +71,8 @@ abstract class DelayedRequest(delayMs: Long) extends DelayedItem(delayMs) { def isCompleted(): Boolean = completed.get() /** - * Process for completing an operation; This function needs to be defined in subclasses - * and will be called exactly once in forceComplete() + * Process for completing an operation; This function needs to be defined + * in subclasses and will be called exactly once in forceComplete() */ def onComplete(): Unit @@ -78,11 +81,7 @@ abstract class DelayedRequest(delayMs: Long) extends DelayedItem(delayMs) { * can be completed by now. If yes execute the completion logic by calling * forceComplete() and return true iff forceComplete returns true; otherwise return false * - * Note that concurrent threads can check if an operation can be completed or not, - * but only the first thread will succeed in completing the operation and return - * true, others will still return false - * - * this function needs to be defined in subclasses + * This function needs to be defined in subclasses */ def tryComplete(): Boolean } @@ -90,13 +89,13 @@ abstract class DelayedRequest(delayMs: Long) extends DelayedItem(delayMs) { /** * A helper purgatory class for bookkeeping delayed operations with a timeout, and expiring timed out operations. */ -class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: Int = 1000) +class DelayedOperationPurgatory[T <: DelayedOperation](brokerId: Int = 0, purgeInterval: Int = 1000) extends Logging with KafkaMetricsGroup { - /* a list of requests watching each key */ + /* a list of operation watching keys */ private val watchersForKey = new Pool[Any, Watchers](Some((key: Any) => new Watchers)) - /* background thread expiring requests that have been waiting too long */ + /* background thread expiring operations that have timed out */ private val expirationReaper = new ExpiredOperationReaper newGauge( @@ -107,7 +106,7 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In ) newGauge( - "NumDelayedRequests", + "NumDelayedOperations", new Gauge[Int] { def value = delayed() } @@ -153,10 +152,10 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In } /** - * Check if some some delayed requests can be completed with the given watch key, + * Check if some some delayed operations can be completed with the given watch key, * and if yes complete them. * - * @return the number of completed requests during this process + * @return the number of completed operations during this process */ def checkAndComplete(key: Any): Int = { val watchers = watchersForKey.get(key) @@ -194,14 +193,14 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In * A linked list of watched delayed operations based on some key */ private class Watchers { - private val requests = new util.LinkedList[T] + private val operations = new util.LinkedList[T] - def watched = requests.size() + def watched = operations.size() // add the element to watch def watch(t: T) { synchronized { - requests.add(t) + operations.add(t) } } @@ -209,11 +208,11 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In def tryCompleteWatched(): Int = { var completed = 0 synchronized { - val iter = requests.iterator() + val iter = operations.iterator() while(iter.hasNext) { val curr = iter.next if (curr.isCompleted()) { - // another thread has completed this request, just remove it + // another thread has completed this operation, just remove it iter.remove() } else { if(curr synchronized curr.tryComplete()) { @@ -230,7 +229,7 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In def purgeCompleted(): Int = { var purged = 0 synchronized { - val iter = requests.iterator() + val iter = operations.iterator() while (iter.hasNext) { val curr = iter.next if(curr.isCompleted()) { @@ -301,12 +300,12 @@ class RequestPurgatory[T <: DelayedRequest](brokerId: Int = 0, purgeInterval: In // try to get the next expired operation and force completing it expireNext() // see if we need to purge the watch lists - if (RequestPurgatory.this.watched() >= purgeInterval) { + if (DelayedOperationPurgatory.this.watched() >= purgeInterval) { debug("Begin purging watch lists") val purged = watchersForKey.values.map(_.purgeCompleted()).sum debug("Purged %d elements from watch lists.".format(purged)) } - // see if we need to purge the delayed request queue + // see if we need to purge the delayed operation queue if (delayed() >= purgeInterval) { debug("Begin purging delayed queue") val purged = purgeCompleted() diff --git a/core/src/main/scala/kafka/server/DelayedOperationKey.scala b/core/src/main/scala/kafka/server/DelayedOperationKey.scala new file mode 100644 index 0000000000000..fb7e9ed5c16dd --- /dev/null +++ b/core/src/main/scala/kafka/server/DelayedOperationKey.scala @@ -0,0 +1,38 @@ +/** + * 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 kafka.common.TopicAndPartition + +/** + * Keys used for delayed operation metrics recording + */ +trait DelayedOperationKey { + def keyLabel: String +} + +object DelayedOperationKey { + val globalLabel = "All" +} + +case class TopicPartitionOperationKey(topic: String, partition: Int) extends DelayedOperationKey { + + def this(topicAndPartition: TopicAndPartition) = this(topicAndPartition.topic, topicAndPartition.partition) + + override def keyLabel = "%s-%d".format(topic, partition) +} diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index 1603066d33f82..c229088eb4f3d 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -32,7 +32,7 @@ case class ProducePartitionStatus(requiredOffset: Long, responseStatus: Producer } /** - * The produce metadata maintained by the delayed produce request + * The produce metadata maintained by the delayed produce operation */ case class ProduceMetadata(produceRequiredAcks: Short, produceStatus: Map[TopicAndPartition, ProducePartitionStatus]) { @@ -42,14 +42,14 @@ case class ProduceMetadata(produceRequiredAcks: Short, } /** - * A delayed produce request that can be created by the replica manager and watched - * in the produce request purgatory + * A delayed produce operation that can be created by the replica manager and watched + * in the produce operation purgatory */ class DelayedProduce(delayMs: Long, produceMetadata: ProduceMetadata, replicaManager: ReplicaManager, responseCallback: Map[TopicAndPartition, ProducerResponseStatus] => Unit) - extends DelayedRequest(delayMs) { + extends DelayedOperation(delayMs) { // first update the acks pending variable according to the error code produceMetadata.produceStatus.foreach { case (topicAndPartition, status) => @@ -65,13 +65,13 @@ class DelayedProduce(delayMs: Long, } /** - * The delayed produce request can be completed if every partition + * The delayed produce operation can be completed if every partition * it produces to is satisfied by one of the following: * * Case A: This broker is no longer the leader: set an error in response * Case B: This broker is the leader: * B.1 - If there was a local error thrown while checking if at least requiredAcks - * replicas have caught up to this request: set an error in response + * replicas have caught up to this operation: set an error in response * B.2 - Otherwise, set the response with no error. */ override def tryComplete(): Boolean = { @@ -117,4 +117,4 @@ class DelayedProduce(delayMs: Long, val responseStatus = produceMetadata.produceStatus.mapValues(status => status.responseStatus) responseCallback(responseStatus) } -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index b3566b0bc33fe..e58fbb922e93b 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -80,8 +80,8 @@ class ReplicaManager(val config: KafkaConfig, this.logIdent = "[Replica Manager on Broker " + localBrokerId + "]: " val stateChangeLogger = KafkaController.stateChangeLogger - val producerRequestPurgatory = new RequestPurgatory[DelayedProduce](config.brokerId, config.producerPurgatoryPurgeIntervalRequests) - val fetchRequestPurgatory = new RequestPurgatory[DelayedFetch](config.brokerId, config.fetchPurgatoryPurgeIntervalRequests) + val delayedProducePurgatory = new DelayedOperationPurgatory[DelayedProduce](config.brokerId, config.producerPurgatoryPurgeIntervalRequests) + val delayedFetchPurgatory = new DelayedOperationPurgatory[DelayedFetch](config.brokerId, config.fetchPurgatoryPurgeIntervalRequests) newGauge( @@ -123,9 +123,9 @@ class ReplicaManager(val config: KafkaConfig, * 1. The partition HW has changed (for acks = -1) * 2. A follower replica's fetch operation is received (for acks > 1) */ - def tryCompleteDelayedProduce(key: TopicAndPartition) { - val completed = producerRequestPurgatory.checkAndComplete(key) - debug("Request key %s unblocked %d producer requests.".format(key, completed)) + def tryCompleteDelayedProduce(key: DelayedOperationKey) { + val completed = delayedProducePurgatory.checkAndComplete(key) + debug("Request key %s unblocked %d producer requests.".format(key.keyLabel, completed)) } /** @@ -135,9 +135,9 @@ class ReplicaManager(val config: KafkaConfig, * 1. The partition HW has changed (for regular fetch) * 2. A new message set is appended to the local log (for follower fetch) */ - def tryCompleteDelayedFetch(key: TopicAndPartition) { - val completed = fetchRequestPurgatory.checkAndComplete(key) - debug("Request key %s unblocked %d fetch requests.".format(key, completed)) + def tryCompleteDelayedFetch(key: DelayedOperationKey) { + val completed = delayedFetchPurgatory.checkAndComplete(key) + debug("Request key %s unblocked %d fetch requests.".format(key.keyLabel, completed)) } def startup() { @@ -280,13 +280,13 @@ class ReplicaManager(val config: KafkaConfig, val produceMetadata = ProduceMetadata(requiredAcks, produceStatus) val delayedProduce = new DelayedProduce(timeout, produceMetadata, this, responseCallback) - // create a list of (topic, partition) pairs to use as keys for this delayed request - val producerRequestKeys = messagesPerPartition.keys.toSeq + // create a list of (topic, partition) pairs to use as keys for this delayed produce operation + val producerRequestKeys = messagesPerPartition.keys.map(new TopicPartitionOperationKey(_)).toSeq // try to complete the request immediately, otherwise put it into the purgatory - // this is because while the delayed request is being created, new requests may - // arrive which can make this request completable. - producerRequestPurgatory.tryCompleteElseWatch(delayedProduce, producerRequestKeys) + // this is because while the delayed produce operation is being created, new + // requests may arrive and hence make this operation completable. + delayedProducePurgatory.tryCompleteElseWatch(delayedProduce, producerRequestKeys) } } @@ -392,13 +392,13 @@ class ReplicaManager(val config: KafkaConfig, val fetchMetadata = FetchMetadata(fetchMinBytes, fetchOnlyFromLeader, fetchOnlyCommitted, fetchPartitionStatus) val delayedFetch = new DelayedFetch(timeout, fetchMetadata, this, responseCallback) - // create a list of (topic, partition) pairs to use as keys for this delayed request - val delayedFetchKeys = fetchPartitionStatus.keys.toSeq + // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation + val delayedFetchKeys = fetchPartitionStatus.keys.map(new TopicPartitionOperationKey(_)).toSeq // try to complete the request immediately, otherwise put it into the purgatory; - // this is because while the delayed request is being created, new requests may - // arrive which can make this request completable. - fetchRequestPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys) + // this is because while the delayed fetch operation is being created, new requests + // may arrive and hence make this operation completable. + delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys) } } @@ -718,7 +718,7 @@ class ReplicaManager(val config: KafkaConfig, // for producer requests with ack > 1, we need to check // if they can be unblocked after some follower's log end offsets have moved - tryCompleteDelayedProduce(topicAndPartition) + tryCompleteDelayedProduce(new TopicPartitionOperationKey(topicAndPartition)) case None => warn("While recording the replica LEO, the partition %s hasn't been created.".format(topicAndPartition)) } @@ -749,8 +749,8 @@ class ReplicaManager(val config: KafkaConfig, def shutdown(checkpointHW: Boolean = true) { info("Shutting down") replicaFetcherManager.shutdown() - fetchRequestPurgatory.shutdown() - producerRequestPurgatory.shutdown() + delayedFetchPurgatory.shutdown() + delayedProducePurgatory.shutdown() if (checkpointHW) checkpointHighWatermarks() info("Shut down completely") diff --git a/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala b/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala similarity index 82% rename from core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala rename to core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala index a7720d579ea15..93f52d3222fc1 100644 --- a/core/src/test/scala/unit/kafka/server/RequestPurgatoryTest.scala +++ b/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala @@ -22,13 +22,13 @@ import org.scalatest.junit.JUnit3Suite import junit.framework.Assert._ import kafka.utils.TestUtils -class RequestPurgatoryTest extends JUnit3Suite { +class DelayedOperationTest extends JUnit3Suite { - var purgatory: RequestPurgatory[MockDelayedRequest] = null + var purgatory: DelayedOperationPurgatory[MockDelayedOperation] = null override def setUp() { super.setUp() - purgatory = new RequestPurgatory[MockDelayedRequest](0, 5) + purgatory = new DelayedOperationPurgatory[MockDelayedOperation](0, 5) } override def tearDown() { @@ -38,8 +38,8 @@ class RequestPurgatoryTest extends JUnit3Suite { @Test def testRequestSatisfaction() { - val r1 = new MockDelayedRequest(100000L) - val r2 = new MockDelayedRequest(100000L) + val r1 = new MockDelayedOperation(100000L) + val r2 = new MockDelayedOperation(100000L) assertEquals("With no waiting requests, nothing should be satisfied", 0, purgatory.checkAndComplete("test1")) assertFalse("r1 not satisfied and hence watched", purgatory.tryCompleteElseWatch(r1, Array("test1"))) assertEquals("Still nothing satisfied", 0, purgatory.checkAndComplete("test1")) @@ -56,8 +56,8 @@ class RequestPurgatoryTest extends JUnit3Suite { @Test def testRequestExpiry() { val expiration = 20L - val r1 = new MockDelayedRequest(expiration) - val r2 = new MockDelayedRequest(200000L) + val r1 = new MockDelayedOperation(expiration) + val r2 = new MockDelayedOperation(200000L) val start = System.currentTimeMillis assertFalse("r1 not satisfied and hence watched", purgatory.tryCompleteElseWatch(r1, Array("test1"))) assertFalse("r2 not satisfied and hence watched", purgatory.tryCompleteElseWatch(r2, Array("test2"))) @@ -70,14 +70,14 @@ class RequestPurgatoryTest extends JUnit3Suite { @Test def testRequestPurge() { - val r1 = new MockDelayedRequest(100000L) - val r2 = new MockDelayedRequest(100000L) + val r1 = new MockDelayedOperation(100000L) + val r2 = new MockDelayedOperation(100000L) purgatory.tryCompleteElseWatch(r1, Array("test1")) purgatory.tryCompleteElseWatch(r2, Array("test1", "test2")) purgatory.tryCompleteElseWatch(r1, Array("test2", "test3")) assertEquals("Purgatory should have 5 watched elements", 5, purgatory.watched()) - assertEquals("Purgatory should have 3 total delayed requests", 3, purgatory.delayed()) + assertEquals("Purgatory should have 3 total delayed operations", 3, purgatory.delayed()) // complete one of the operations, it should // eventually be purged from the watch list with purge interval 5 @@ -86,7 +86,7 @@ class RequestPurgatoryTest extends JUnit3Suite { TestUtils.waitUntilTrue(() => purgatory.watched() == 3, "Purgatory should have 3 watched elements instead of " + purgatory.watched(), 1000L) TestUtils.waitUntilTrue(() => purgatory.delayed() == 3, - "Purgatory should still have 3 total delayed requests instead of " + purgatory.delayed(), 1000L) + "Purgatory should still have 3 total delayed operations instead of " + purgatory.delayed(), 1000L) // add two more requests, then the satisfied request should be purged from the delayed queue with purge interval 5 purgatory.tryCompleteElseWatch(r1, Array("test1")) @@ -95,10 +95,10 @@ class RequestPurgatoryTest extends JUnit3Suite { TestUtils.waitUntilTrue(() => purgatory.watched() == 5, "Purgatory should have 5 watched elements instead of " + purgatory.watched(), 1000L) TestUtils.waitUntilTrue(() => purgatory.delayed() == 4, - "Purgatory should have 4 total delayed requests instead of " + purgatory.delayed(), 1000L) + "Purgatory should have 4 total delayed operations instead of " + purgatory.delayed(), 1000L) } - class MockDelayedRequest(delayMs: Long) extends DelayedRequest(delayMs) { + class MockDelayedOperation(delayMs: Long) extends DelayedOperation(delayMs) { var completable = false def awaitExpiration() { diff --git a/system_test/metrics.json b/system_test/metrics.json index cd3fc142176b8..30dabe596871f 100644 --- a/system_test/metrics.json +++ b/system_test/metrics.json @@ -78,13 +78,13 @@ { "graph_name": "ProducePurgatoryQueueSize", "y_label": "size", - "bean_name": "kafka.server:type=ProducerRequestPurgatory,name=NumDelayedRequests", + "bean_name": "kafka.server:type=ProducerRequestPurgatory,name=NumDelayedOperations", "attributes": "Value" }, { "graph_name": "FetchPurgatoryQueueSize", "y_label": "size", - "bean_name": "kafka.server:type=FetchRequestPurgatory,name=NumDelayedRequests", + "bean_name": "kafka.server:type=FetchRequestPurgatory,name=NumDelayedOperations", "attributes": "Value" }, { From 09e2fd651e0a98a96107f0ee1087c785a54b4e20 Mon Sep 17 00:00:00 2001 From: Joe Stein Date: Fri, 5 Dec 2014 08:37:11 -0500 Subject: [PATCH 068/491] KAFKA-1173 Using Vagrant to get up and running with Apache Kafka patch by Ewen Cheslack-Postava reviewed by Joe Stein --- .gitignore | 7 +- README.md | 4 ++ Vagrantfile | 168 ++++++++++++++++++++++++++++++++++++++++++++++ vagrant/README.md | 126 ++++++++++++++++++++++++++++++++++ vagrant/base.sh | 43 ++++++++++++ vagrant/broker.sh | 38 +++++++++++ vagrant/zk.sh | 42 ++++++++++++ 7 files changed, 427 insertions(+), 1 deletion(-) create mode 100644 Vagrantfile create mode 100644 vagrant/README.md create mode 100644 vagrant/base.sh create mode 100644 vagrant/broker.sh create mode 100644 vagrant/zk.sh diff --git a/.gitignore b/.gitignore index 99b32a6770e3d..06a64184eaa53 100644 --- a/.gitignore +++ b/.gitignore @@ -20,4 +20,9 @@ TAGS .settings .gradle kafka.ipr -kafka.iws \ No newline at end of file +kafka.iws +.vagrant +Vagrantfile.local + +config/server-* +config/zookeeper-* diff --git a/README.md b/README.md index 9aca90664b2a8..11dfdf9379ad1 100644 --- a/README.md +++ b/README.md @@ -94,6 +94,10 @@ Please note for this to work you should create/update `~/.gradle/gradle.properti ### Determining how transitive dependencies are added ### ./gradlew core:dependencies --configuration runtime +### Running in Vagrant ### + +See [vagrant/README.md](vagrant/README.md). + ### Contribution ### Apache Kafka is interested in building the community; we would welcome any thoughts or [patches](https://issues.apache.org/jira/browse/KAFKA). You can reach us [on the Apache mailing lists](http://kafka.apache.org/contact.html). diff --git a/Vagrantfile b/Vagrantfile new file mode 100644 index 0000000000000..55c67ddda4581 --- /dev/null +++ b/Vagrantfile @@ -0,0 +1,168 @@ +# 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. +# -*- mode: ruby -*- +# vi: set ft=ruby : + +require 'socket' + +# Vagrantfile API/syntax version. Don't touch unless you know what you're doing! +VAGRANTFILE_API_VERSION = "2" + +# General config +enable_dns = false +num_zookeepers = 1 +num_brokers = 3 +num_workers = 0 # Generic workers that get the code, but don't start any services +ram_megabytes = 1280 + +# EC2 +ec2_access_key = ENV['AWS_ACCESS_KEY'] +ec2_secret_key = ENV['AWS_SECRET_KEY'] +ec2_keypair_name = nil +ec2_keypair_file = nil + +ec2_region = "us-east-1" +ec2_az = nil # Uses set by AWS +ec2_ami = "ami-9eaa1cf6" +ec2_instance_type = "m3.medium" +ec2_user = "ubuntu" +ec2_security_groups = nil +ec2_subnet_id = nil +# Only override this by setting it to false if you're running in a VPC and you +# are running Vagrant from within that VPC as well. +ec2_associate_public_ip = nil + +local_config_file = File.join(File.dirname(__FILE__), "Vagrantfile.local") +if File.exists?(local_config_file) then + eval(File.read(local_config_file), binding, "Vagrantfile.local") +end + +# TODO(ksweeney): RAM requirements are not empirical and can probably be significantly lowered. +Vagrant.configure(VAGRANTFILE_API_VERSION) do |config| + config.hostmanager.enabled = true + config.hostmanager.manage_host = enable_dns + config.hostmanager.include_offline = false + + ## Provider-specific global configs + config.vm.provider :virtualbox do |vb,override| + override.vm.box = "ubuntu/trusty64" + + override.hostmanager.ignore_private_ip = false + + # Brokers started with the standard script currently set Xms and Xmx to 1G, + # plus we need some extra head room. + vb.customize ["modifyvm", :id, "--memory", ram_megabytes.to_s] + + if Vagrant.has_plugin?("vagrant-cachier") + config.cache.scope = :box + # Besides the defaults, we use a custom cache to handle the Oracle JDK + # download, which downloads via wget during an apt install. Because of the + # way the installer ends up using its cache directory, we need to jump + # through some hoops instead of just specifying a cache directly -- we + # share to a temporary location and the provisioning scripts symlink data + # to the right location. + config.cache.enable :generic, { + "oracle-jdk7" => { cache_dir: "/tmp/oracle-jdk7-installer-cache" }, + } + end + end + + config.vm.provider :aws do |aws,override| + # The "box" is specified as an AMI + override.vm.box = "dummy" + override.vm.box_url = "https://github.com/mitchellh/vagrant-aws/raw/master/dummy.box" + + override.hostmanager.ignore_private_ip = true + + override.ssh.username = ec2_user + override.ssh.private_key_path = ec2_keypair_file + + aws.access_key_id = ec2_access_key + aws.secret_access_key = ec2_secret_key + aws.keypair_name = ec2_keypair_name + + aws.region = ec2_region + aws.availability_zone = ec2_az + aws.instance_type = ec2_instance_type + aws.ami = ec2_ami + aws.security_groups = ec2_security_groups + aws.subnet_id = ec2_subnet_id + # If a subnet is specified, default to turning on a public IP unless the + # user explicitly specifies the option. Without a public IP, Vagrant won't + # be able to SSH into the hosts unless Vagrant is also running in the VPC. + if ec2_associate_public_ip.nil? + aws.associate_public_ip = true unless ec2_subnet_id.nil? + else + aws.associate_public_ip = ec2_associate_public_ip + end + + # Exclude some directories that can grow very large from syncing + config.vm.synced_folder ".", "/vagrant", type: "rsync", :rsync_excludes => ['.git', 'core/data/', 'logs/', 'system_test/'] + end + + def name_node(node, name) + node.vm.hostname = name + node.vm.provider :aws do |aws| + aws.tags = { 'Name' => "kafka-vagrant-" + Socket.gethostname + "-" + name } + end + end + + def assign_local_ip(node, ip_address) + node.vm.provider :virtualbox do |vb,override| + override.vm.network :private_network, ip: ip_address + end + end + + ## Cluster definition + zookeepers = [] + (1..num_zookeepers).each { |i| + name = "zk" + i.to_s + zookeepers.push(name) + config.vm.define name do |zookeeper| + name_node(zookeeper, name) + ip_address = "192.168.50." + (10 + i).to_s + assign_local_ip(zookeeper, ip_address) + zookeeper.vm.provision "shell", path: "vagrant/base.sh" + zookeeper.vm.provision "shell", path: "vagrant/zk.sh", :args => [i.to_s, num_zookeepers] + end + } + + (1..num_brokers).each { |i| + name = "broker" + i.to_s + config.vm.define name do |broker| + name_node(broker, name) + ip_address = "192.168.50." + (50 + i).to_s + assign_local_ip(broker, ip_address) + # We need to be careful about what we list as the publicly routable + # address since this is registered in ZK and handed out to clients. If + # host DNS isn't setup, we shouldn't use hostnames -- IP addresses must be + # used to support clients running on the host. + zookeeper_connect = zookeepers.map{ |zk_addr| zk_addr + ":2181"}.join(",") + broker.vm.provision "shell", path: "vagrant/base.sh" + broker.vm.provision "shell", path: "vagrant/broker.sh", :args => [i.to_s, enable_dns ? name : ip_address, zookeeper_connect] + end + } + + (1..num_workers).each { |i| + name = "worker" + i.to_s + config.vm.define name do |worker| + name_node(worker, name) + ip_address = "192.168.50." + (100 + i).to_s + assign_local_ip(worker, ip_address) + worker.vm.provision "shell", path: "vagrant/base.sh" + end + } + +end diff --git a/vagrant/README.md b/vagrant/README.md new file mode 100644 index 0000000000000..73cf0390bc4c7 --- /dev/null +++ b/vagrant/README.md @@ -0,0 +1,126 @@ +# Apache Kafka # + +Using Vagrant to get up and running. + +1) Install Virtual Box [https://www.virtualbox.org/](https://www.virtualbox.org/) +2) Install Vagrant >= 1.6.4 [http://www.vagrantup.com/](http://www.vagrantup.com/) +3) Install Vagrant Plugins: + + # Required + $ vagrant plugin install vagrant-hostmanager + # Optional + $ vagrant plugin install vagrant-cachier # Caches & shares package downloads across VMs + +In the main Kafka folder, do a normal Kafka build: + + $ gradle + $ ./gradlew jar + +You can override default settings in `Vagrantfile.local`, which is a Ruby file +that is ignored by git and imported into the Vagrantfile. +One setting you likely want to enable +in `Vagrantfile.local` is `enable_dns = true` to put hostnames in the host's +/etc/hosts file. You probably want this to avoid having to use IP addresses when +addressing the cluster from outside the VMs, e.g. if you run a client on the +host. It's disabled by default since it requires `sudo` access, mucks with your +system state, and breaks with naming conflicts if you try to run multiple +clusters concurrently. + +Now bring up the cluster: + + $ vagrant up --no-provision && vagrant provision + +We separate out the two steps (bringing up the base VMs and configuring them) +due to current limitations in ZooKeeper (ZOOKEEPER-1506) that require us to +collect IPs for all nodes before starting ZooKeeper nodes. + +Once this completes: + +* Zookeeper will be running on 192.168.50.11 (and `zk1` if you used enable_dns) +* Broker 1 on 192.168.50.51 (and `broker1` if you used enable_dns) +* Broker 2 on 192.168.50.52 (and `broker2` if you used enable_dns) +* Broker 3 on 192.168.50.53 (and `broker3` if you used enable_dns) + +To log into one of the machines: + + vagrant ssh + +You can access the brokers and zookeeper by their IP or hostname, e.g. + + # Specify ZooKeeper node 1 by it's IP: 192.168.50.11 + bin/kafka-topics.sh --create --zookeeper 192.168.50.11:2181 --replication-factor 3 --partitions 1 --topic sandbox + + # Specify brokers by their hostnames: broker1, broker2, broker3 + bin/kafka-console-producer.sh --broker-list broker1:9092,broker2:9092,broker3:9092 --topic sandbox + + # Specify ZooKeeper node by its hostname: zk1 + bin/kafka-console-consumer.sh --zookeeper zk1:2181 --topic sandbox --from-beginning + +If you need to update the running cluster, you can re-run the provisioner (the +step that installs software and configures services): + + vagrant provision + +Note that this doesn't currently ensure a fresh start -- old cluster state will +still remain intact after everything restarts. This can be useful for updating +the cluster to your most recent development version. + +Finally, you can clean up the cluster by destroying all the VMs: + + vagrant destroy + +## Configuration ## + +You can override some default settings by specifying the values in +`Vagrantfile.local`. It is interpreted as a Ruby file, although you'll probably +only ever need to change a few simple configuration variables. Some values you +might want to override: + +* `enable_dns` - Register each VM with a hostname in /etc/hosts on the + hosts. Hostnames are always set in the /etc/hosts in the VMs, so this is only + necessary if you want to address them conveniently from the host for tasks + that aren't provided by Vagrant. +* `num_zookeepers` - Size of zookeeper cluster +* `num_brokers` - Number of broker instances to run + + +## Using Other Providers ## + +### EC2 ### + +Install the `vagrant-aws` plugin to provide EC2 support: + + $ vagrant plugin install vagrant-aws + +Next, configure parameters in `Vagrantfile.local`. A few are *required*: +`enable_dns`, `ec2_access_key`, `ec2_secret_key`, `ec2_keypair_name`, `ec2_keypair_file`, and +`ec2_security_groups`. A couple of important notes: + +1. You definitely want to use `enable_dns` if you plan to run clients outside of + the cluster (e.g. from your local host). If you don't, you'll need to go + lookup `vagrant ssh-config`. + +2. You'll have to setup a reasonable security group yourself. You'll need to + open ports for Zookeeper (2888 & 3888 between ZK nodes, 2181 for clients) and + Kafka (9092). Beware that opening these ports to all sources (e.g. so you can + run producers/consumers locally) will allow anyone to access your Kafka + cluster. All other settings have reasonable defaults for setting up an + Ubuntu-based cluster, but you may want to customize instance type, region, + AMI, etc. + +3. `ec2_access_key` and `ec2_secret_key` will use the environment variables + `AWS_ACCESS_KEY` and `AWS_SECRET_KEY` respectively if they are set and not + overridden in `Vagrantfile.local`. + +4. If you're launching into a VPC, you must specify `ec2_subnet_id` (the subnet + in which to launch the nodes) and `ec2_security_groups` must be a list of + security group IDs instead of names, e.g. `sg-34fd3551` instead of + `kafka-test-cluster`. + +Now start things up, but specify the aws provider: + + $ vagrant up --provider=aws --no-parallel --no-provision && vagrant provision + +Your instances should get tagged with a name including your hostname to make +them identifiable and make it easier to track instances in the AWS management +console. diff --git a/vagrant/base.sh b/vagrant/base.sh new file mode 100644 index 0000000000000..6f28dfed67877 --- /dev/null +++ b/vagrant/base.sh @@ -0,0 +1,43 @@ +# 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. + +#!/bin/bash + +set -e + +if [ -z `which javac` ]; then + apt-get -y update + apt-get install -y software-properties-common python-software-properties + add-apt-repository -y ppa:webupd8team/java + apt-get -y update + + # Try to share cache. See Vagrantfile for details + mkdir -p /var/cache/oracle-jdk7-installer + if [ -e "/tmp/oracle-jdk7-installer-cache/" ]; then + find /tmp/oracle-jdk7-installer-cache/ -not -empty -exec cp '{}' /var/cache/oracle-jdk7-installer/ \; + fi + + /bin/echo debconf shared/accepted-oracle-license-v1-1 select true | /usr/bin/debconf-set-selections + apt-get -y install oracle-java7-installer oracle-java7-set-default + + if [ -e "/tmp/oracle-jdk7-installer-cache/" ]; then + cp -R /var/cache/oracle-jdk7-installer/* /tmp/oracle-jdk7-installer-cache + fi +fi + +chmod a+rw /opt +if [ ! -e /opt/kafka ]; then + ln -s /vagrant /opt/kafka +fi diff --git a/vagrant/broker.sh b/vagrant/broker.sh new file mode 100644 index 0000000000000..63f2d4f30c5a4 --- /dev/null +++ b/vagrant/broker.sh @@ -0,0 +1,38 @@ +# 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. + +#!/bin/bash + +# Usage: brokers.sh + +set -e + +BROKER_ID=$1 +PUBLIC_ADDRESS=$2 +PUBLIC_ZOOKEEPER_ADDRESSES=$3 + +cd /opt/kafka + +sed \ + -e 's/broker.id=0/'broker.id=$BROKER_ID'/' \ + -e 's/#advertised.host.name=/'advertised.host.name=$PUBLIC_ADDRESS'/' \ + -e 's/zookeeper.connect=localhost:2181/'zookeeper.connect=$PUBLIC_ZOOKEEPER_ADDRESSES'/' \ + /opt/kafka/config/server.properties > /opt/kafka/config/server-$BROKER_ID.properties + +echo "Killing server" +bin/kafka-server-stop.sh || true +sleep 5 # Because kafka-server-stop.sh doesn't actually wait +echo "Starting server" +bin/kafka-server-start.sh /opt/kafka/config/server-$BROKER_ID.properties 1>> /tmp/broker.log 2>> /tmp/broker.log & diff --git a/vagrant/zk.sh b/vagrant/zk.sh new file mode 100644 index 0000000000000..15517f826461d --- /dev/null +++ b/vagrant/zk.sh @@ -0,0 +1,42 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +#!/bin/bash + +# Usage: zk.sh + +set -e + +ZKID=$1 +NUM_ZK=$2 + +cd /opt/kafka + +cp /opt/kafka/config/zookeeper.properties /opt/kafka/config/zookeeper-$ZKID.properties +echo "initLimit=5" >> /opt/kafka/config/zookeeper-$ZKID.properties +echo "syncLimit=2" >> /opt/kafka/config/zookeeper-$ZKID.properties +echo "quorumListenOnAllIPs=true" >> /opt/kafka/config/zookeeper-$ZKID.properties +for i in `seq 1 $NUM_ZK`; do + echo "server.${i}=zk${i}:2888:3888" >> /opt/kafka/config/zookeeper-$ZKID.properties +done + +mkdir -p /tmp/zookeeper +echo "$ZKID" > /tmp/zookeeper/myid + +echo "Killing ZooKeeper" +bin/zookeeper-server-stop.sh || true +sleep 5 # Because kafka-server-stop.sh doesn't actually wait +echo "Starting ZooKeeper" +bin/zookeeper-server-start.sh config/zookeeper-$ZKID.properties 1>> /tmp/zk.log 2>> /tmp/zk.log & From 4fc74958e3569f6179c11fddace99b674c81f370 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Fri, 5 Dec 2014 09:18:26 -0800 Subject: [PATCH 069/491] KAFKA-1807 Improve accuracy of ProducerPerformance target throughput; reviewed by Neha Narkhede --- .../kafka/clients/tools/ProducerPerformance.java | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java b/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java index ac8615082e21c..28175fb71edbe 100644 --- a/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java +++ b/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java @@ -55,6 +55,7 @@ public static void main(String[] args) throws Exception { long sleepTime = NS_PER_SEC / throughput; long sleepDeficitNs = 0; Stats stats = new Stats(numRecords, 5000); + long start = System.currentTimeMillis(); for (int i = 0; i < numRecords; i++) { long sendStart = System.currentTimeMillis(); Callback cb = stats.nextCompletion(sendStart, payload.length, stats); @@ -66,12 +67,15 @@ public static void main(String[] args) throws Exception { * and then make up the whole deficit in one longer sleep. */ if (throughput > 0) { - sleepDeficitNs += sleepTime; - if (sleepDeficitNs >= MIN_SLEEP_NS) { - long sleepMs = sleepDeficitNs / 1000000; - long sleepNs = sleepDeficitNs - sleepMs * 1000000; - Thread.sleep(sleepMs, (int) sleepNs); - sleepDeficitNs = 0; + float elapsed = (sendStart - start)/1000.f; + if (elapsed > 0 && i/elapsed > throughput) { + sleepDeficitNs += sleepTime; + if (sleepDeficitNs >= MIN_SLEEP_NS) { + long sleepMs = sleepDeficitNs / 1000000; + long sleepNs = sleepDeficitNs - sleepMs * 1000000; + Thread.sleep(sleepMs, (int) sleepNs); + sleepDeficitNs = 0; + } } } } From 2801629964882015a9148e1c0ade22da46376faa Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Fri, 5 Dec 2014 15:18:25 -0800 Subject: [PATCH 070/491] KAFKA-1650; avoid data loss when mirror maker shutdown uncleanly; reviewed by Guozhang Wang --- .../consumer/ZookeeperConsumerConnector.scala | 47 +- .../consumer/ConsumerRebalanceListener.java | 42 ++ .../consumer/ZookeeperConsumerConnector.scala | 4 + .../main/scala/kafka/tools/MirrorMaker.scala | 414 +++++++++++++++--- .../utils/ByteBoundedBlockingQueue.scala | 11 + .../ZookeeperConsumerConnectorTest.scala | 60 ++- 6 files changed, 493 insertions(+), 85 deletions(-) create mode 100644 core/src/main/scala/kafka/javaapi/consumer/ConsumerRebalanceListener.java diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index da29a8cb46109..e991d2187d032 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -28,6 +28,7 @@ import kafka.api._ import kafka.client.ClientUtils import kafka.cluster._ import kafka.common._ +import kafka.javaapi.consumer.ConsumerRebalanceListener import kafka.metrics._ import kafka.network.BlockingChannel import kafka.serializer._ @@ -39,6 +40,7 @@ import org.I0Itec.zkclient.{IZkChildListener, IZkDataListener, IZkStateListener, import org.apache.zookeeper.Watcher.Event.KeeperState import scala.collection._ +import scala.collection.JavaConversions._ /** @@ -102,6 +104,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private val offsetsChannelLock = new Object private var wildcardTopicWatcher: ZookeeperTopicEventWatcher = null + private var consumerRebalanceListener: ConsumerRebalanceListener = null // useful for tracking migration of consumers to store offsets in kafka private val kafkaCommitMeter = newMeter("KafkaCommitsPerSec", "commits", TimeUnit.SECONDS, Map("clientId" -> config.clientId)) @@ -161,6 +164,13 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, wildcardStreamsHandler.streams } + def setConsumerRebalanceListener(listener: ConsumerRebalanceListener) { + if (messageStreamCreated.get()) + throw new MessageStreamsExistException(this.getClass.getSimpleName + + " can only set consumer rebalance listener before creating streams",null) + consumerRebalanceListener = listener + } + private def createFetcher() { if (enableFetcher) fetcher = Some(new ConsumerFetcherManager(consumerIdString, config, zkClient)) @@ -286,17 +296,27 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } } + /** + * KAFKA-1743: This method added for backward compatibility. + */ + def commitOffsets { commitOffsets(true) } + def commitOffsets(isAutoCommit: Boolean) { + commitOffsets(isAutoCommit, null) + } + + def commitOffsets(isAutoCommit: Boolean, + topicPartitionOffsets: immutable.Map[TopicAndPartition, OffsetAndMetadata]) { var retriesRemaining = 1 + (if (isAutoCommit) config.offsetsCommitMaxRetries else 0) // no retries for commits from auto-commit var done = false while (!done) { val committed = offsetsChannelLock synchronized { // committed when we receive either no error codes or only MetadataTooLarge errors - val offsetsToCommit = immutable.Map(topicRegistry.flatMap { case (topic, partitionTopicInfos) => + val offsetsToCommit = if (topicPartitionOffsets == null) {immutable.Map(topicRegistry.flatMap { case (topic, partitionTopicInfos) => partitionTopicInfos.map { case (partition, info) => TopicAndPartition(info.topic, info.partitionId) -> OffsetAndMetadata(info.getConsumeOffset()) } - }.toSeq:_*) + }.toSeq:_*)} else topicPartitionOffsets if (offsetsToCommit.size > 0) { if (config.offsetsStorage == "zookeeper") { @@ -374,11 +394,6 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } } - /** - * KAFKA-1743: This method added for backward compatibility. - */ - def commitOffsets { commitOffsets(true) } - private def fetchOffsetFromZooKeeper(topicPartition: TopicAndPartition) = { val dirs = new ZKGroupTopicDirs(config.groupId, topicPartition.topic) val offsetString = readDataMaybeNull(zkClient, dirs.consumerOffsetDir + "/" + topicPartition.partition)._1 @@ -653,9 +668,18 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, * partitions in parallel. So, not stopping the fetchers leads to duplicate data. */ closeFetchers(cluster, kafkaMessageAndMetadataStreams, myTopicThreadIdsMap) - + if (consumerRebalanceListener != null) { + info("Calling beforeReleasingPartitions() from rebalance listener.") + consumerRebalanceListener.beforeReleasingPartitions( + if (topicRegistry.size == 0) + new java.util.HashMap[String, java.util.Set[java.lang.Integer]] + else + mapAsJavaMap(topicRegistry.map(topics => + topics._1 -> topics._2.keys + ).toMap).asInstanceOf[java.util.Map[String, java.util.Set[java.lang.Integer]]] + ) + } releasePartitionOwnership(topicRegistry) - val assignmentContext = new AssignmentContext(group, consumerIdString, config.excludeInternalTopics, zkClient) val partitionOwnershipDecision = partitionAssignor.assign(assignmentContext) val currentTopicRegistry = new Pool[String, Pool[Int, PartitionTopicInfo]]( @@ -711,7 +735,6 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, case Some(f) => f.stopConnections clearFetcherQueues(allPartitionInfos, cluster, queuesToBeCleared, messageStreams) - info("Committing all offsets after clearing the fetcher queues") /** * here, we need to commit offsets before stopping the consumer from returning any more messages * from the current data chunk. Since partition ownership is not yet released, this commit offsets @@ -720,8 +743,10 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, * by the consumer, there will be no more messages returned by this iterator until the rebalancing finishes * successfully and the fetchers restart to fetch more data chunks **/ - if (config.autoCommitEnable) + if (config.autoCommitEnable) { + info("Committing all offsets after clearing the fetcher queues") commitOffsets(true) + } case None => } } diff --git a/core/src/main/scala/kafka/javaapi/consumer/ConsumerRebalanceListener.java b/core/src/main/scala/kafka/javaapi/consumer/ConsumerRebalanceListener.java new file mode 100644 index 0000000000000..facf509841918 --- /dev/null +++ b/core/src/main/scala/kafka/javaapi/consumer/ConsumerRebalanceListener.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.javaapi.consumer; + +import kafka.common.TopicAndPartition; +import kafka.consumer.ConsumerThreadId; + +import java.util.Map; +import java.util.Set; + +/** + * This listener is used for execution of tasks defined by user when a consumer rebalance + * occurs in {@link kafka.consumer.ZookeeperConsumerConnector} + */ +public interface ConsumerRebalanceListener { + + /** + * This method is called after all the fetcher threads are stopped but before the + * ownership of partitions are released. Depending on whether auto offset commit is + * enabled or not, offsets may or may not have been committed. + * This listener is initially added to prevent duplicate messages on consumer rebalance + * in mirror maker, where offset auto commit is disabled to prevent data loss. It could + * also be used in more general cases. + */ + public void beforeReleasingPartitions(Map> partitionOwnership); + +} diff --git a/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala index 9d5a47fb8e04d..9baad34a9793e 100644 --- a/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala @@ -115,6 +115,10 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, underlying.commitOffsets(retryOnFailure) } + def setConsumerRebalanceListener(consumerRebalanceListener: ConsumerRebalanceListener) { + underlying.setConsumerRebalanceListener(consumerRebalanceListener) + } + def shutdown() { underlying.shutdown } diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index f399105087588..b06ff6000183b 100644 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -17,29 +17,66 @@ package kafka.tools +import com.yammer.metrics.core._ +import kafka.common.{TopicAndPartition, OffsetAndMetadata} +import kafka.javaapi.consumer.ConsumerRebalanceListener +import kafka.utils._ import kafka.consumer._ -import kafka.metrics.KafkaMetricsGroup -import kafka.producer.{BaseProducer, NewShinyProducer, OldProducer} import kafka.serializer._ -import kafka.utils._ -import org.apache.kafka.clients.producer.ProducerRecord - -import java.util.Random -import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} -import java.util.concurrent.{BlockingQueue, ArrayBlockingQueue, CountDownLatch, TimeUnit} +import kafka.producer.{OldProducer, NewShinyProducer} +import kafka.metrics.KafkaMetricsGroup +import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback +import org.apache.kafka.clients.producer.{RecordMetadata, ProducerRecord} +import org.apache.kafka.common.KafkaException import scala.collection.JavaConversions._ import joptsimple.OptionParser +import java.util.Properties +import java.util.concurrent.atomic.{AtomicInteger, AtomicBoolean} +import java.util.concurrent._ -object MirrorMaker extends Logging { +/** + * The mirror maker consists of three major modules: + * Consumer Threads - The consumer threads consume messages from source Kafka cluster through + * ZookeeperConsumerConnector and put them into corresponding data channel queue based on hash value + * of source topic-partitionId string. This guarantees the message order in source partition is + * preserved. + * Producer Threads - Producer threads take messages out of data channel queues and send them to target cluster. Each + * producer thread is bound to one data channel queue, so that the message order is preserved. + * Data Channel - The data channel has multiple queues. The number of queue is same as number of producer threads. + * + * If new producer is used, the offset will be committed based on the new producer's callback. An offset map is + * maintained and updated on each send() callback. A separate offset commit thread will commit the offset periodically. + * @note For mirror maker, MaxInFlightRequests of producer should be set to 1 for producer if the order of the messages + * needs to be preserved. Mirror maker also depends on the in-order delivery to guarantee no data loss. + * We are not force it to be 1 because in some use cases throughput might be important whereas out of order or + * minor data loss is acceptable. + */ +object MirrorMaker extends Logging with KafkaMetricsGroup { - private var connectors: Seq[ZookeeperConsumerConnector] = null + private var connector: ZookeeperConsumerConnector = null private var consumerThreads: Seq[ConsumerThread] = null private var producerThreads: Seq[ProducerThread] = null private val isShuttingdown: AtomicBoolean = new AtomicBoolean(false) - - private val shutdownMessage : ProducerRecord = new ProducerRecord("shutdown", "shutdown".getBytes) + private var offsetCommitThread: OffsetCommitThread = null + + private val valueFactory = (k: TopicAndPartition) => new Pool[Int, Long] + private val topicPartitionOffsetMap: Pool[TopicAndPartition, Pool[Int, Long]] = + new Pool[TopicAndPartition, Pool[Int,Long]](Some(valueFactory)) + // Track the messages unacked for consumer rebalance + private var numMessageUnacked: AtomicInteger = new AtomicInteger(0) + private var consumerRebalanceListener: MirrorMakerConsumerRebalanceListener = null + // This is to indicate whether the rebalance is going on so the producer callback knows if + // the rebalance latch needs to be pulled. + private var inRebalance: AtomicBoolean = new AtomicBoolean(false) + + private val shutdownMessage : MirrorMakerRecord = new MirrorMakerRecord("shutdown", 0, 0, null, "shutdown".getBytes) + + newGauge("MirrorMaker-Unacked-Messages", + new Gauge[Int] { + def value = numMessageUnacked.get() + }) def main(args: Array[String]) { @@ -47,12 +84,12 @@ object MirrorMaker extends Logging { val parser = new OptionParser val consumerConfigOpt = parser.accepts("consumer.config", - "Consumer config to consume from a source cluster. " + - "You may specify multiple of these.") + "Embedded consumer config for consuming from the source cluster.") .withRequiredArg() .describedAs("config file") .ofType(classOf[String]) + // Please see note about MaxInflightRequests val producerConfigOpt = parser.accepts("producer.config", "Embedded producer config.") .withRequiredArg() @@ -83,6 +120,13 @@ object MirrorMaker extends Logging { .ofType(classOf[java.lang.Integer]) .defaultsTo(10000) + val bufferByteSizeOpt = parser.accepts("queue.byte.size", + "Maximum bytes that can be buffered in each data channel queue") + .withRequiredArg() + .describedAs("Data channel queue size in terms of number of bytes") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(100000000) + val whitelistOpt = parser.accepts("whitelist", "Whitelist of topics to mirror.") .withRequiredArg() @@ -95,6 +139,13 @@ object MirrorMaker extends Logging { .describedAs("Java regex (String)") .ofType(classOf[String]) + val offsetCommitIntervalMsOpt = parser.accepts("offset.commit.interval.ms", + "Offset commit interval in ms") + .withRequiredArg() + .describedAs("offset commit interval in millisecond") + .ofType(classOf[java.lang.Integer]) + .defaultsTo(60000) + val helpOpt = parser.accepts("help", "Print this message.") if(args.length == 0) @@ -116,15 +167,28 @@ object MirrorMaker extends Logging { val numProducers = options.valueOf(numProducersOpt).intValue() val numStreams = options.valueOf(numStreamsOpt).intValue() val bufferSize = options.valueOf(bufferSizeOpt).intValue() + val bufferByteSize = options.valueOf(bufferByteSizeOpt).intValue() + val offsetCommitIntervalMs = options.valueOf(offsetCommitIntervalMsOpt).intValue() - // create consumer streams - connectors = options.valuesOf(consumerConfigOpt).toList - .map(cfg => new ConsumerConfig(Utils.loadProps(cfg))) - .map(new ZookeeperConsumerConnector(_)) - val numConsumers = connectors.size * numStreams + // create consumer connector + val consumerConfigProps = Utils.loadProps(options.valuesOf(consumerConfigOpt).head) + val consumerConfig = new ConsumerConfig(consumerConfigProps) + connector = new ZookeeperConsumerConnector(consumerConfig) // create a data channel btw the consumers and the producers - val mirrorDataChannel = new DataChannel(bufferSize, numConsumers, numProducers) + val mirrorDataChannel = new DataChannel(bufferSize, bufferByteSize, numInputs = numStreams, numOutputs = numProducers) + + // set consumer rebalance listener + // Customized consumer rebalance listener should extend MirrorMakerConsumerRebalanceListener + // and take datachannel as argument. + val customRebalanceListenerClass = consumerConfigProps.getProperty("consumer.rebalance.listener") + consumerRebalanceListener = { + if (customRebalanceListenerClass == null) { + new MirrorMakerConsumerRebalanceListener(mirrorDataChannel) + } else + Utils.createObject[MirrorMakerConsumerRebalanceListener](customRebalanceListenerClass, mirrorDataChannel) + } + connector.setConsumerRebalanceListener(consumerRebalanceListener) // create producer threads val useNewProducer = options.has(useNewProducerOpt) @@ -134,12 +198,25 @@ object MirrorMaker extends Logging { producerProps.setProperty("client.id", clientId + "-" + i) val producer = if (useNewProducer) - new NewShinyProducer(producerProps) + new MirrorMakerNewProducer(producerProps) else - new OldProducer(producerProps) + new MirrorMakerOldProducer(producerProps) new ProducerThread(mirrorDataChannel, producer, i) }) + // create offset commit thread + if (useNewProducer) { + /** + * The offset commit thread periodically commit consumed offsets to the source cluster. With the new producer, + * the offsets are updated upon the returned future metadata of the send() call; with the old producer, + * the offsets are updated upon the consumer's iterator advances. By doing this, it is guaranteed no data + * loss even when mirror maker is uncleanly shutdown with the new producer, while with the old producer + * messages inside the data channel could be lost upon mirror maker unclean shutdown. + */ + offsetCommitThread = new OffsetCommitThread(offsetCommitIntervalMs) + offsetCommitThread.start() + } + // create consumer threads val filterSpec = if (options.has(whitelistOpt)) new Whitelist(options.valueOf(whitelistOpt)) @@ -148,14 +225,14 @@ object MirrorMaker extends Logging { var streams: Seq[KafkaStream[Array[Byte], Array[Byte]]] = Nil try { - streams = connectors.map(_.createMessageStreamsByFilter(filterSpec, numStreams, new DefaultDecoder(), new DefaultDecoder())).flatten + streams = connector.createMessageStreamsByFilter(filterSpec, numStreams, new DefaultDecoder(), new DefaultDecoder()) } catch { case t: Throwable => fatal("Unable to create stream - shutting down mirror maker.") - connectors.foreach(_.shutdown) + connector.shutdown() } consumerThreads = streams.zipWithIndex.map(streamAndIndex => new ConsumerThread(streamAndIndex._1, mirrorDataChannel, streamAndIndex._2)) - assert(consumerThreads.size == numConsumers) + assert(consumerThreads.size == numStreams) Runtime.getRuntime.addShutdownHook(new Thread() { override def run() { @@ -170,28 +247,51 @@ object MirrorMaker extends Logging { // since the consumer threads can hit a timeout/other exception; // but in this case the producer should still be able to shutdown // based on the shutdown message in the channel - producerThreads.foreach(_.awaitShutdown) + producerThreads.foreach(_.awaitShutdown()) } def cleanShutdown() { if (isShuttingdown.compareAndSet(false, true)) { - if (connectors != null) connectors.foreach(_.shutdown) - if (consumerThreads != null) consumerThreads.foreach(_.awaitShutdown) + info("Start clean shutdown.") + // Consumer threads will exit when isCleanShutdown is set. + info("Shutting down consumer threads.") + if (consumerThreads != null) { + consumerThreads.foreach(_.shutdown()) + consumerThreads.foreach(_.awaitShutdown()) + } + // After consumer threads exit, shutdown producer. + info("Shutting down producer threads.") if (producerThreads != null) { - producerThreads.foreach(_.shutdown) - producerThreads.foreach(_.awaitShutdown) + producerThreads.foreach(_.shutdown()) + producerThreads.foreach(_.awaitShutdown()) + } + // offset commit thread should only be shutdown after producer threads are shutdown, so we don't lose offsets. + info("Shutting down offset commit thread.") + if (offsetCommitThread != null) { + offsetCommitThread.shutdown() + offsetCommitThread.awaitShutdown() } + // connector can only be shutdown after offsets are committed. + info("Shutting down consumer connectors.") + if (connector != null) + connector.shutdown() info("Kafka mirror maker shutdown successfully") } } - class DataChannel(capacity: Int, numInputs: Int, numOutputs: Int) extends KafkaMetricsGroup { - - val queues = new Array[BlockingQueue[ProducerRecord]](numOutputs) - for (i <- 0 until numOutputs) - queues(i) = new ArrayBlockingQueue[ProducerRecord](capacity) - - private val counter = new AtomicInteger(new Random().nextInt()) + class DataChannel(messageCapacity: Int, byteCapacity: Int, numInputs: Int, numOutputs: Int) + extends KafkaMetricsGroup { + + val queues = new Array[ByteBoundedBlockingQueue[MirrorMakerRecord]](numOutputs) + val channelSizeHists = new Array[Histogram](numOutputs) + val channelByteSizeHists = new Array[Histogram](numOutputs) + val sizeFunction = (record: MirrorMakerRecord) => record.size + for (i <- 0 until numOutputs) { + queues(i) = new ByteBoundedBlockingQueue[MirrorMakerRecord](messageCapacity, byteCapacity, Some(sizeFunction)) + channelSizeHists(i) = newHistogram("MirrorMaker-DataChannel-queue-%d-NumMessages".format(i)) + channelByteSizeHists(i) = newHistogram("MirrorMaker-DataChannel-queue-%d-Bytes".format(i)) + } + private val channelRecordSizeHist = newHistogram("MirrorMaker-DataChannel-Record-Size") // We use a single meter for aggregated wait percentage for the data channel. // Since meter is calculated as total_recorded_value / time_window and @@ -199,21 +299,16 @@ object MirrorMaker extends Logging { // time should be discounted by # threads. private val waitPut = newMeter("MirrorMaker-DataChannel-WaitOnPut", "percent", TimeUnit.NANOSECONDS) private val waitTake = newMeter("MirrorMaker-DataChannel-WaitOnTake", "percent", TimeUnit.NANOSECONDS) - private val channelSizeHist = newHistogram("MirrorMaker-DataChannel-Size") - def put(record: ProducerRecord) { - // If the key of the message is empty, use round-robin to select the queue - // Otherwise use the queue based on the key value so that same key-ed messages go to the same queue + def put(record: MirrorMakerRecord) { + // Use hash of source topic-partition to decide which queue to put the message in. The benefit is that + // we can maintain the message order for both keyed and non-keyed messages. val queueId = - if(record.key() != null) { - Utils.abs(java.util.Arrays.hashCode(record.key())) % numOutputs - } else { - Utils.abs(counter.getAndIncrement()) % numOutputs - } + Utils.abs(java.util.Arrays.hashCode((record.sourceTopic + record.sourcePartition).toCharArray)) % numOutputs put(record, queueId) } - def put(record: ProducerRecord, queueId: Int) { + def put(record: MirrorMakerRecord, queueId: Int) { val queue = queues(queueId) var putSucceed = false @@ -222,20 +317,27 @@ object MirrorMaker extends Logging { putSucceed = queue.offer(record, 500, TimeUnit.MILLISECONDS) waitPut.mark((SystemTime.nanoseconds - startPutTime) / numInputs) } - channelSizeHist.update(queue.size) + channelSizeHists(queueId).update(queue.size()) + channelByteSizeHists(queueId).update(queue.byteSize()) + channelRecordSizeHist.update(sizeFunction(record)) } - def take(queueId: Int): ProducerRecord = { + def take(queueId: Int): MirrorMakerRecord = { val queue = queues(queueId) - var data: ProducerRecord = null + var data: MirrorMakerRecord = null while (data == null) { val startTakeTime = SystemTime.nanoseconds data = queue.poll(500, TimeUnit.MILLISECONDS) waitTake.mark((SystemTime.nanoseconds - startTakeTime) / numOutputs) } - channelSizeHist.update(queue.size) + channelSizeHists(queueId).update(queue.size()) + channelByteSizeHists(queueId).update(queue.byteSize()) data } + + def clear() { + queues.foreach(queue => queue.clear()) + } } class ConsumerThread(stream: KafkaStream[Array[Byte], Array[Byte]], @@ -245,34 +347,44 @@ object MirrorMaker extends Logging { private val shutdownLatch = new CountDownLatch(1) private val threadName = "mirrormaker-consumer-" + threadId - private var isCleanShutdown: Boolean = true this.logIdent = "[%s] ".format(threadName) + private var shutdownFlag: Boolean = false this.setName(threadName) override def run() { info("Starting mirror maker consumer thread " + threadName) try { - for (msgAndMetadata <- stream) { - val data = new ProducerRecord(msgAndMetadata.topic, msgAndMetadata.key, msgAndMetadata.message) + val iter = stream.iterator() + while (!shutdownFlag && iter.hasNext()) { + val msgAndMetadata = iter.next() + val data = new MirrorMakerRecord(msgAndMetadata.topic, + msgAndMetadata.partition, + msgAndMetadata.offset, + msgAndMetadata.key(), + msgAndMetadata.message()) mirrorDataChannel.put(data) } } catch { case e: Throwable => { fatal("Stream unexpectedly exited.", e) - isCleanShutdown = false } } finally { shutdownLatch.countDown() info("Consumer thread stopped") + // If it exits accidentally, stop the entire mirror maker. - if (!isCleanShutdown) { + if (!isShuttingdown.get()) { fatal("Consumer thread exited abnormally, stopping the whole mirror maker.") System.exit(-1) } } } + def shutdown() { + shutdownFlag = true + } + def awaitShutdown() { try { shutdownLatch.await() @@ -284,44 +396,44 @@ object MirrorMaker extends Logging { } class ProducerThread (val dataChannel: DataChannel, - val producer: BaseProducer, + val producer: MirrorMakerBaseProducer, val threadId: Int) extends Thread with Logging with KafkaMetricsGroup { private val threadName = "mirrormaker-producer-" + threadId private val shutdownComplete: CountDownLatch = new CountDownLatch(1) - private var isCleanShutdown: Boolean = true this.logIdent = "[%s] ".format(threadName) setName(threadName) - override def run { + override def run() { info("Starting mirror maker producer thread " + threadName) try { while (true) { - val data: ProducerRecord = dataChannel.take(threadId) - trace("Sending message with value size %d".format(data.value().size)) + val data: MirrorMakerRecord = dataChannel.take(threadId) + trace("Sending message with value size %d".format(data.value.size)) if(data eq shutdownMessage) { info("Received shutdown message") return } - producer.send(data.topic(), data.key(), data.value()) + producer.send(new TopicAndPartition(data.sourceTopic, data.sourcePartition), + data.sourceOffset, + data.key, + data.value) } } catch { - case t: Throwable => { + case t: Throwable => fatal("Producer thread failure due to ", t) - isCleanShutdown = false - } } finally { - shutdownComplete.countDown + shutdownComplete.countDown() info("Producer thread stopped") - // If it exits accidentally, stop the entire mirror maker. - if (!isCleanShutdown) { + // if it exits accidentally, stop the entire mirror maker + if (!isShuttingdown.get()) { fatal("Producer thread exited abnormally, stopping the whole mirror maker.") System.exit(-1) } } } - def shutdown { + def shutdown() { try { info("Producer thread " + threadName + " shutting down") dataChannel.put(shutdownMessage, threadId) @@ -333,10 +445,10 @@ object MirrorMaker extends Logging { } } - def awaitShutdown { + def awaitShutdown() { try { - shutdownComplete.await - producer.close + shutdownComplete.await() + producer.close() info("Producer thread shutdown complete") } catch { case ie: InterruptedException => { @@ -345,5 +457,163 @@ object MirrorMaker extends Logging { } } } + + class OffsetCommitThread(commitIntervalMs: Int) extends Thread with Logging with KafkaMetricsGroup { + private val threadName = "mirrormaker-offset-commit-thread" + private val shutdownComplete: CountDownLatch = new CountDownLatch(1) + this.logIdent = "[%s]".format(threadName) + var shutdownFlag: Boolean = false + var commitCounter: Int = 0 + + this.setName(threadName) + + newGauge("MirrorMaker-Offset-Commit-Counter", + new Gauge[Int] { + def value = commitCounter + }) + + /** + * Use the connector to commit all the offsets. + */ + override def run() { + info("Starting mirror maker offset commit thread") + try { + while (!shutdownFlag) { + Thread.sleep(commitIntervalMs) + commitOffset() + } + } catch { + case t: Throwable => fatal("Exits due to", t) + } finally { + swallow(commitOffset()) + shutdownComplete.countDown() + info("Offset commit thread exited") + if (!isShuttingdown.get()) { + fatal("Offset commit thread exited abnormally, stopping the whole mirror maker.") + System.exit(-1) + } + } + } + + def commitOffset() { + val offsetsToCommit = collection.immutable.Map(topicPartitionOffsetMap.map { + case (topicPartition, partitionOffsetMap) => + topicPartition -> OffsetAndMetadata(getOffsetToCommit(partitionOffsetMap), null) + }.toSeq: _*) + trace("committing offset: %s".format(offsetsToCommit)) + if (connector == null) { + warn("No consumer connector available to commit offset.") + } else { + connector.commitOffsets(isAutoCommit = false, offsetsToCommit) + commitCounter += 1 + } + } + + private def getOffsetToCommit(offsetsMap: Pool[Int, Long]): Long = { + val offsets = offsetsMap.map(_._2).toSeq.sorted + val iter = offsets.iterator + var offsetToCommit = iter.next() + while (iter.hasNext && offsetToCommit + 1 == iter.next()) + offsetToCommit += 1 + // The committed offset will be the first offset of un-consumed message, hence we need to increment by one. + offsetToCommit + 1 + } + + def shutdown() { + shutdownFlag = true + } + + def awaitShutdown() { + try { + shutdownComplete.await() + info("Offset commit thread shutdown complete") + } catch { + case ie: InterruptedException => { + warn("Shutdown of the offset commit thread interrupted") + } + } + } + } + + private[kafka] trait MirrorMakerBaseProducer { + def send(topicPartition: TopicAndPartition, offset: Long, key: Array[Byte], value: Array[Byte]) + def close() + } + + private class MirrorMakerNewProducer (val producerProps: Properties) + extends NewShinyProducer(producerProps) with MirrorMakerBaseProducer { + + override def send(topicPartition: TopicAndPartition, offset: Long, key: Array[Byte], value: Array[Byte]) { + val record = new ProducerRecord(topicPartition.topic, key, value) + if(sync) { + topicPartitionOffsetMap.getAndMaybePut(topicPartition).put(this.producer.send(record).get().partition(), offset) + } else { + this.producer.send(record, + new MirrorMakerProducerCallback(topicPartition, offset, key, value)) + numMessageUnacked.incrementAndGet() + } + } + } + + private class MirrorMakerOldProducer (val producerProps: Properties) + extends OldProducer(producerProps) with MirrorMakerBaseProducer { + + override def send(topicPartition: TopicAndPartition, offset: Long, key: Array[Byte], value: Array[Byte]) { + super.send(topicPartition.topic, key, value) + } + + override def close() { + super.close() + } + } + + private class MirrorMakerProducerCallback (val topicPartition: TopicAndPartition, + val offset: Long, + val key: Array[Byte], + val value: Array[Byte]) + extends ErrorLoggingCallback(topicPartition.topic, key, value, false) { + + override def onCompletion(metadata: RecordMetadata, exception: Exception) { + if (exception != null) { + // Use default call back to log error + super.onCompletion(metadata, exception) + } else { + trace("updating offset:[%s] -> %d".format(topicPartition, offset)) + topicPartitionOffsetMap.getAndMaybePut(topicPartition).put(metadata.partition(), offset) + } + // Notify the rebalance callback only when all the messages handed to producer are acked. + // There is a very slight chance that 1 message is held by producer thread and not handed to producer. + // That message might have duplicate. We are not handling that here. + if (numMessageUnacked.decrementAndGet() == 0 && inRebalance.get()) { + inRebalance synchronized {inRebalance.notify()} + } + } + } + + class MirrorMakerConsumerRebalanceListener (dataChannel: DataChannel) extends ConsumerRebalanceListener { + + override def beforeReleasingPartitions(partitionOwnership: java.util.Map[String, java.util.Set[java.lang.Integer]]) { + info("Clearing data channel.") + dataChannel.clear() + info("Waiting until all the messages are acked.") + inRebalance synchronized { + inRebalance.set(true) + while (numMessageUnacked.get() > 0) + inRebalance.wait() + } + info("Committing offsets.") + offsetCommitThread.commitOffset() + inRebalance.set(true) + } + } + + private[kafka] class MirrorMakerRecord (val sourceTopic: String, + val sourcePartition: Int, + val sourceOffset: Long, + val key: Array[Byte], + val value: Array[Byte]) { + def size = value.length + {if (key == null) 0 else key.length} + } + } diff --git a/core/src/main/scala/kafka/utils/ByteBoundedBlockingQueue.scala b/core/src/main/scala/kafka/utils/ByteBoundedBlockingQueue.scala index 6a85d7e494f6c..26149af943c1d 100644 --- a/core/src/main/scala/kafka/utils/ByteBoundedBlockingQueue.scala +++ b/core/src/main/scala/kafka/utils/ByteBoundedBlockingQueue.scala @@ -216,4 +216,15 @@ class ByteBoundedBlockingQueue[E] (val queueNumMessageCapacity: Int, val queueBy * @return the remaining bytes capacity of the queue */ def remainingByteSize = math.max(0, queueByteCapacity - currentByteSize.get()) + + /** + * remove all the items in the queue + */ + def clear() { + putLock synchronized { + queue.clear() + currentByteSize.set(0) + putLock.notify() + } + } } diff --git a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala index 8c4687b2c96fd..3ccccbdbed07c 100644 --- a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala @@ -19,17 +19,20 @@ package kafka.consumer import junit.framework.Assert._ import kafka.integration.KafkaServerTestHarness +import kafka.javaapi.consumer.ConsumerRebalanceListener import kafka.server._ import scala.collection._ +import scala.collection.JavaConversions._ import org.scalatest.junit.JUnit3Suite import kafka.message._ import kafka.serializer._ import org.I0Itec.zkclient.ZkClient import kafka.utils._ -import java.util.Collections +import kafka.producer.{KeyedMessage, Producer} +import java.util.{Collections, Properties} import org.apache.log4j.{Logger, Level} import kafka.utils.TestUtils._ -import kafka.common.MessageStreamsExistException +import kafka.common.{TopicAndPartition, MessageStreamsExistException} class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with Logging { @@ -344,6 +347,49 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar zkClient.close() } + def testConsumerRebalanceListener() { + // Send messages to create topic + sendMessagesToPartition(configs, topic, 0, nMessages) + sendMessagesToPartition(configs, topic, 1, nMessages) + + val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1)) + val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) + // Register consumer rebalance listener + val rebalanceListener1 = new TestConsumerRebalanceListener() + zkConsumerConnector1.setConsumerRebalanceListener(rebalanceListener1) + val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) + + // Check if rebalance listener is fired + assertEquals(true, rebalanceListener1.listenerCalled) + assertEquals(null, rebalanceListener1.partitionOwnership.get(topic)) + // reset the flag + rebalanceListener1.listenerCalled = false + + val actual_1 = getZKChildrenValues(dirs.consumerOwnerDir) + val expected_1 = List(("0", "group1_consumer1-0"), + ("1", "group1_consumer1-0")) + assertEquals(expected_1, actual_1) + + val consumerConfig2 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer2)) + val zkConsumerConnector2 = new ZookeeperConsumerConnector(consumerConfig2, true) + // Register consumer rebalance listener + val rebalanceListener2 = new TestConsumerRebalanceListener() + zkConsumerConnector2.setConsumerRebalanceListener(rebalanceListener2) + val topicMessageStreams2 = zkConsumerConnector2.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) + + val actual_2 = getZKChildrenValues(dirs.consumerOwnerDir) + val expected_2 = List(("0", "group1_consumer1-0"), + ("1", "group1_consumer2-0")) + assertEquals(expected_2, actual_2) + + // Check if rebalance listener is fired + assertEquals(true, rebalanceListener1.listenerCalled) + assertEquals(Set[Int](0, 1), rebalanceListener1.partitionOwnership.get(topic)) + assertEquals(true, rebalanceListener2.listenerCalled) + assertEquals(null, rebalanceListener2.partitionOwnership.get(topic)) + + } + def getZKChildrenValues(path : String) : Seq[Tuple2[String,String]] = { val children = zkClient.getChildren(path) Collections.sort(children) @@ -355,4 +401,14 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar (partition, zkClient.readData(path + "/" + partition).asInstanceOf[String])) } + private class TestConsumerRebalanceListener extends ConsumerRebalanceListener { + var listenerCalled: Boolean = false + var partitionOwnership: java.util.Map[String, java.util.Set[java.lang.Integer]] = null + + override def beforeReleasingPartitions(partitionOwnership: java.util.Map[String, java.util.Set[java.lang.Integer]]) { + listenerCalled = true + this.partitionOwnership = partitionOwnership + } + } + } From e8ffbd0fee0bc715ad0fe6c9afe85715f84d8e51 Mon Sep 17 00:00:00 2001 From: Joe Stein Date: Fri, 12 Dec 2014 03:46:29 -0500 Subject: [PATCH 071/491] KAFKA-1812 Allow IpV6 in configuration with parseCsvMap patch by Jeff Holoman reviewed by Gwen Shapira and Joe Stein --- core/src/main/scala/kafka/utils/Utils.scala | 105 +++++++++--------- .../scala/unit/kafka/utils/UtilsTest.scala | 36 ++++++ 2 files changed, 91 insertions(+), 50 deletions(-) diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala index 58685cc47b4c4..738c1af9ef5de 100644 --- a/core/src/main/scala/kafka/utils/Utils.scala +++ b/core/src/main/scala/kafka/utils/Utils.scala @@ -5,7 +5,7 @@ * 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 @@ -33,10 +33,10 @@ import kafka.common.KafkaStorageException /** * General helper functions! - * + * * This is for general helper functions that aren't specific to Kafka logic. Things that should have been included in - * the standard library etc. - * + * the standard library etc. + * * If you are making a new helper function and want to add it to this class please ensure the following: * 1. It has documentation * 2. It is the most general possible utility, not just the thing you needed in one particular place @@ -68,18 +68,18 @@ object Utils extends Logging { * @param runnable The runnable to execute in the background * @return The unstarted thread */ - def daemonThread(name: String, runnable: Runnable): Thread = + def daemonThread(name: String, runnable: Runnable): Thread = newThread(name, runnable, true) - + /** * Create a daemon thread * @param name The name of the thread * @param fun The runction to execute in the thread * @return The unstarted thread */ - def daemonThread(name: String, fun: () => Unit): Thread = + def daemonThread(name: String, fun: () => Unit): Thread = daemonThread(name, runnable(fun)) - + /** * Create a new thread * @param name The name of the thread @@ -88,16 +88,16 @@ object Utils extends Logging { * @return The unstarted thread */ def newThread(name: String, runnable: Runnable, daemon: Boolean): Thread = { - val thread = new Thread(runnable, name) + val thread = new Thread(runnable, name) thread.setDaemon(daemon) thread.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() { def uncaughtException(t: Thread, e: Throwable) { error("Uncaught exception in thread '" + t.getName + "':", e) - } + } }) thread } - + /** * Create a new thread * @param runnable The work for the thread to do @@ -114,7 +114,7 @@ object Utils extends Logging { }) thread } - + /** * Read the given byte buffer into a byte array */ @@ -161,7 +161,7 @@ object Utils extends Logging { else new FileInputStream(file).getChannel() } - + /** * Do the given action and log any exceptions thrown without rethrowing them * @param log The log method to use for logging. E.g. logger.warn @@ -174,7 +174,7 @@ object Utils extends Logging { case e: Throwable => log(e.getMessage(), e) } } - + /** * Test if two byte buffers are equal. In this case equality means having * the same bytes from the current position to the limit @@ -191,7 +191,7 @@ object Utils extends Logging { return false return true } - + /** * Translate the given buffer into a string * @param buffer The buffer to translate @@ -202,7 +202,7 @@ object Utils extends Logging { buffer.get(bytes) new String(bytes, encoding) } - + /** * Print an error message and shutdown the JVM * @param message The error message @@ -211,19 +211,19 @@ object Utils extends Logging { System.err.println(message) System.exit(1) } - + /** * Recursively delete the given file/directory and any subfiles (if any exist) * @param file The root file at which to begin deleting */ def rm(file: String): Unit = rm(new File(file)) - + /** * Recursively delete the list of files/directories and any subfiles (if any exist) * @param a sequence of files to be deleted */ def rm(files: Seq[String]): Unit = files.map(f => rm(new File(f))) - + /** * Recursively delete the given file/directory and any subfiles (if any exist) * @param file The root file at which to begin deleting @@ -242,7 +242,7 @@ object Utils extends Logging { file.delete() } } - + /** * Register the given mbean with the platform mbean server, * unregistering any mbean that was there before. Note, @@ -270,7 +270,7 @@ object Utils extends Logging { } } } - + /** * Unregister the mbean with the given name, if there is one registered * @param name The mbean name to unregister @@ -283,16 +283,16 @@ object Utils extends Logging { mbs.unregisterMBean(objName) } } - + /** - * Read an unsigned integer from the current position in the buffer, + * Read an unsigned integer from the current position in the buffer, * incrementing the position by 4 bytes * @param buffer The buffer to read from * @return The integer read, as a long to avoid signedness */ - def readUnsignedInt(buffer: ByteBuffer): Long = + def readUnsignedInt(buffer: ByteBuffer): Long = buffer.getInt() & 0xffffffffL - + /** * Read an unsigned integer from the given position without modifying the buffers * position @@ -300,33 +300,33 @@ object Utils extends Logging { * @param index the index from which to read the integer * @return The integer read, as a long to avoid signedness */ - def readUnsignedInt(buffer: ByteBuffer, index: Int): Long = + def readUnsignedInt(buffer: ByteBuffer, index: Int): Long = buffer.getInt(index) & 0xffffffffL - + /** * Write the given long value as a 4 byte unsigned integer. Overflow is ignored. * @param buffer The buffer to write to * @param value The value to write */ - def writetUnsignedInt(buffer: ByteBuffer, value: Long): Unit = + def writetUnsignedInt(buffer: ByteBuffer, value: Long): Unit = buffer.putInt((value & 0xffffffffL).asInstanceOf[Int]) - + /** * Write the given long value as a 4 byte unsigned integer. Overflow is ignored. * @param buffer The buffer to write to * @param index The position in the buffer at which to begin writing * @param value The value to write */ - def writeUnsignedInt(buffer: ByteBuffer, index: Int, value: Long): Unit = + def writeUnsignedInt(buffer: ByteBuffer, index: Int, value: Long): Unit = buffer.putInt(index, (value & 0xffffffffL).asInstanceOf[Int]) - + /** * Compute the CRC32 of the byte array * @param bytes The array to compute the checksum for * @return The CRC32 */ def crc32(bytes: Array[Byte]): Long = crc32(bytes, 0, bytes.length) - + /** * Compute the CRC32 of the segment of the byte array given by the specificed size and offset * @param bytes The bytes to checksum @@ -339,7 +339,7 @@ object Utils extends Logging { crc.update(bytes, offset, size) crc.getValue() } - + /** * Compute the hash code for the given items */ @@ -356,7 +356,7 @@ object Utils extends Logging { } return h } - + /** * Group the given values by keys extracted with the given function */ @@ -368,12 +368,12 @@ object Utils extends Logging { case Some(l: List[V]) => m.put(k, v :: l) case None => m.put(k, List(v)) } - } + } m } - + /** - * Read some bytes into the provided buffer, and return the number of bytes read. If the + * Read some bytes into the provided buffer, and return the number of bytes read. If the * channel has been closed or we get -1 on the read for any reason, throw an EOFException */ def read(channel: ReadableByteChannel, buffer: ByteBuffer): Int = { @@ -381,8 +381,8 @@ object Utils extends Logging { case -1 => throw new EOFException("Received -1 when reading from channel, socket has likely been closed.") case n: Int => n } - } - + } + /** * Throw an exception if the given value is null, else return it. You can use this like: * val myValue = Utils.notNull(expressionThatShouldntBeNull) @@ -407,15 +407,20 @@ object Utils extends Logging { /** * This method gets comma separated values which contains key,value pairs and returns a map of * key value pairs. the format of allCSVal is key1:val1, key2:val2 .... + * Also supports strings with multiple ":" such as IpV6 addresses, taking the last occurrence + * of the ":" in the pair as the split, eg a:b:c:val1, d:e:f:val2 => a:b:c -> val1, d:e:f -> val2 */ def parseCsvMap(str: String): Map[String, String] = { val map = new mutable.HashMap[String, String] - if("".equals(str)) - return map - val keyVals = str.split("\\s*,\\s*").map(s => s.split("\\s*:\\s*")) - keyVals.map(pair => (pair(0), pair(1))).toMap + if ("".equals(str)) + return map + val keyVals = str.split("\\s*,\\s*").map(s => { + val lio = s.lastIndexOf(":") + Pair(s.substring(0,lio).trim, s.substring(lio + 1).trim) + }) + keyVals.toMap } - + /** * Parse a comma separated string into a sequence of strings. * Whitespace surrounding the comma will be removed. @@ -467,7 +472,7 @@ object Utils extends Logging { stream.close() } } - + /** * Get the absolute value of the given number. If the number is Int.MinValue return 0. * This is different from java.lang.Math.abs or scala.math.abs in that they return Int.MinValue (!). @@ -496,7 +501,7 @@ object Utils extends Logging { throw new KafkaStorageException("Failed to create file %s.".format(path)) f } - + /** * Turn a properties map into a string */ @@ -531,7 +536,7 @@ object Utils extends Logging { } evaluated } - + /** * Read some properties with the given default values */ @@ -541,7 +546,7 @@ object Utils extends Logging { props.load(reader) props } - + /** * Read a big-endian integer from a byte array */ @@ -551,7 +556,7 @@ object Utils extends Logging { ((bytes(offset + 2) & 0xFF) << 8) | (bytes(offset + 3) & 0xFF) } - + /** * Execute the given function inside the lock */ @@ -590,7 +595,7 @@ object Utils extends Logging { */ case c if ((c >= '\u0000' && c <= '\u001f') || (c >= '\u007f' && c <= '\u009f')) => "\\u%04x".format(c: Int) case c => c - }.mkString + }.mkString } /** diff --git a/core/src/test/scala/unit/kafka/utils/UtilsTest.scala b/core/src/test/scala/unit/kafka/utils/UtilsTest.scala index 0d0f0e2fba367..066553cad290c 100644 --- a/core/src/test/scala/unit/kafka/utils/UtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/UtilsTest.scala @@ -104,6 +104,42 @@ class UtilsTest extends JUnitSuite { assertTrue(emptyStringList.equals(emptyList)) } + @Test + def testCsvMap() { + val emptyString: String = "" + val emptyMap = Utils.parseCsvMap(emptyString) + val emptyStringMap = Map.empty[String, String] + assertTrue(emptyMap != null) + assertTrue(emptyStringMap.equals(emptyStringMap)) + + val kvPairsIpV6: String = "a:b:c:v,a:b:c:v" + val ipv6Map = Utils.parseCsvMap(kvPairsIpV6) + for (m <- ipv6Map) { + assertTrue(m._1.equals("a:b:c")) + assertTrue(m._2.equals("v")) + } + + val singleEntry:String = "key:value" + val singleMap = Utils.parseCsvMap(singleEntry) + val value = singleMap.getOrElse("key", 0) + assertTrue(value.equals("value")) + + val kvPairsIpV4: String = "192.168.2.1/30:allow, 192.168.2.1/30:allow" + val ipv4Map = Utils.parseCsvMap(kvPairsIpV4) + for (m <- ipv4Map) { + assertTrue(m._1.equals("192.168.2.1/30")) + assertTrue(m._2.equals("allow")) + } + + val kvPairsSpaces: String = "key:value , key: value" + val spaceMap = Utils.parseCsvMap(kvPairsSpaces) + for (m <- spaceMap) { + assertTrue(m._1.equals("key")) + assertTrue(m._2.equals("value")) + } + } + + @Test def testInLock() { val lock = new ReentrantLock() From 0ef631bc694d0c6071450c43501ca319a58859f4 Mon Sep 17 00:00:00 2001 From: Chris Cope Date: Fri, 12 Dec 2014 11:37:57 -0800 Subject: [PATCH 072/491] kafka-1815; ServerShutdownTest fails in trunk; patched by Chris Cope; reviewed by Jun Rao --- .../consumer/ZookeeperConsumerConnectorTest.scala | 3 ++- .../scala/unit/kafka/server/ServerShutdownTest.scala | 12 ++++++++++-- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala index 3ccccbdbed07c..a17e8532c44aa 100644 --- a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala @@ -387,7 +387,8 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar assertEquals(Set[Int](0, 1), rebalanceListener1.partitionOwnership.get(topic)) assertEquals(true, rebalanceListener2.listenerCalled) assertEquals(null, rebalanceListener2.partitionOwnership.get(topic)) - + zkConsumerConnector1.shutdown() + zkConsumerConnector2.shutdown() } def getZKChildrenValues(path : String) : Seq[Tuple2[String,String]] = { diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index 67918f2842d47..ba1e48e4300c9 100644 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -141,10 +141,18 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { verifyNonDaemonThreadsStatus } + private[this] def isNonDaemonKafkaThread(t: Thread): Boolean = { + val threadName = Option(t.getClass.getCanonicalName) + .getOrElse(t.getClass.getName()) + .toLowerCase + + !t.isDaemon && t.isAlive && threadName.startsWith("kafka") + } + def verifyNonDaemonThreadsStatus() { assertEquals(0, Thread.getAllStackTraces.keySet().toArray - .map(_.asInstanceOf[Thread]) - .count(t => !t.isDaemon && t.isAlive && t.getClass.getCanonicalName.toLowerCase.startsWith("kafka"))) + .map{ _.asInstanceOf[Thread] } + .count(isNonDaemonKafkaThread)) } def testConsecutiveShutdown(){ From 523b36589e942cb99a95debd2c45e795ae533d08 Mon Sep 17 00:00:00 2001 From: Anatoly Fayngelerin Date: Fri, 12 Dec 2014 11:51:41 -0800 Subject: [PATCH 073/491] kafka-1813; Build fails for scala 2.9.2; patched by Anatoly Fayngelerin; reviewed by Jun Rao --- core/src/main/scala/kafka/tools/MirrorMaker.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index b06ff6000183b..77d951d13b8d8 100644 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -504,7 +504,10 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { if (connector == null) { warn("No consumer connector available to commit offset.") } else { - connector.commitOffsets(isAutoCommit = false, offsetsToCommit) + connector.commitOffsets( + isAutoCommit = false, + topicPartitionOffsets = offsetsToCommit + ) commitCounter += 1 } } From ae0bb84fa7e599774cd984196dc62b3dc84b13ca Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Tue, 16 Dec 2014 20:33:11 -0800 Subject: [PATCH 074/491] KAFKA-742: Existing directories under the Kafka data directory without any data cause process to not start; reviewed by Neha Narkhede --- core/src/main/scala/kafka/log/Log.scala | 22 +++++- .../src/main/scala/kafka/log/LogManager.scala | 2 +- .../test/scala/unit/kafka/log/LogTest.scala | 75 +++++++++++++++++++ 3 files changed, 95 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 4fae2f0d339b2..024506cd00556 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -84,7 +84,7 @@ class Log(val dir: File, /* Calculate the offset of the next message */ @volatile var nextOffsetMetadata = new LogOffsetMetadata(activeSegment.nextOffset(), activeSegment.baseOffset, activeSegment.size.toInt) - val topicAndPartition: TopicAndPartition = Log.parseTopicPartitionName(name) + val topicAndPartition: TopicAndPartition = Log.parseTopicPartitionName(dir) info("Completed load of log %s with log end offset %d".format(name, logEndOffset)) @@ -832,9 +832,25 @@ object Log { /** * Parse the topic and partition out of the directory name of a log */ - def parseTopicPartitionName(name: String): TopicAndPartition = { + def parseTopicPartitionName(dir: File): TopicAndPartition = { + val name: String = dir.getName + if (name == null || name.isEmpty || !name.contains('-')) { + throwException(dir) + } val index = name.lastIndexOf('-') - TopicAndPartition(name.substring(0,index), name.substring(index+1).toInt) + val topic: String = name.substring(0, index) + val partition: String = name.substring(index + 1) + if (topic.length < 1 || partition.length < 1) { + throwException(dir) + } + TopicAndPartition(topic, partition.toInt) + } + + def throwException(dir: File) { + throw new KafkaException("Found directory " + dir.getCanonicalPath + ", " + + "'" + dir.getName + "' is not in the form of topic-partition\n" + + "If a directory does not contain Kafka topic data it should not exist in Kafka's log " + + "directory") } } diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 4d2924d04bc4b..4ebaae00ca4b8 100644 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -134,7 +134,7 @@ class LogManager(val logDirs: Array[File], Utils.runnable { debug("Loading log '" + logDir.getName + "'") - val topicPartition = Log.parseTopicPartitionName(logDir.getName) + val topicPartition = Log.parseTopicPartitionName(logDir) val config = topicConfigs.getOrElse(topicPartition.topic, defaultConfig) val logRecoveryPoint = recoveryPoints.getOrElse(topicPartition, 0L) diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index d670ba76acd54..c2dd8eb69da8c 100644 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -688,4 +688,79 @@ class LogTest extends JUnitSuite { assertEquals(recoveryPoint, log.logEndOffset) cleanShutdownFile.delete() } + + @Test + def testParseTopicPartitionName() { + val topic: String = "test_topic" + val partition:String = "143" + val dir: File = new File(logDir + topicPartitionName(topic, partition)) + val topicAndPartition = Log.parseTopicPartitionName(dir); + assertEquals(topic, topicAndPartition.asTuple._1) + assertEquals(partition.toInt, topicAndPartition.asTuple._2) + } + + @Test + def testParseTopicPartitionNameForEmptyName() { + try { + val dir: File = new File("") + val topicAndPartition = Log.parseTopicPartitionName(dir); + fail("KafkaException should have been thrown for dir: " + dir.getCanonicalPath) + } catch { + case e: Exception => // its GOOD! + } + } + + @Test + def testParseTopicPartitionNameForNull() { + try { + val dir: File = null + val topicAndPartition = Log.parseTopicPartitionName(dir); + fail("KafkaException should have been thrown for dir: " + dir) + } catch { + case e: Exception => // its GOOD! + } + } + + @Test + def testParseTopicPartitionNameForMissingSeparator() { + val topic: String = "test_topic" + val partition:String = "1999" + val dir: File = new File(logDir + File.separator + topic + partition) + try { + val topicAndPartition = Log.parseTopicPartitionName(dir); + fail("KafkaException should have been thrown for dir: " + dir.getCanonicalPath) + } catch { + case e: Exception => // its GOOD! + } + } + + @Test + def testParseTopicPartitionNameForMissingTopic() { + val topic: String = "" + val partition:String = "1999" + val dir: File = new File(logDir + topicPartitionName(topic, partition)) + try { + val topicAndPartition = Log.parseTopicPartitionName(dir); + fail("KafkaException should have been thrown for dir: " + dir.getCanonicalPath) + } catch { + case e: Exception => // its GOOD! + } + } + + @Test + def testParseTopicPartitionNameForMissingPartition() { + val topic: String = "test_topic" + val partition:String = "" + val dir: File = new File(logDir + topicPartitionName(topic, partition)) + try { + val topicAndPartition = Log.parseTopicPartitionName(dir); + fail("KafkaException should have been thrown for dir: " + dir.getCanonicalPath) + } catch { + case e: Exception => // its GOOD! + } + } + + def topicPartitionName(topic: String, partition: String): String = { + File.separator + topic + "-" + partition + } } From 92d1d4cd31e1045f0e000e8d2d777c73f7529743 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Wed, 17 Dec 2014 16:29:09 -0800 Subject: [PATCH 075/491] kafka-1797; add the serializer/deserializer api to the new java client; patched by Jun Rao; reviewed by Neha Narkhede --- .../consumer/ByteArrayDeserializer.java | 34 +++++ .../kafka/clients/consumer/Consumer.java | 4 +- .../clients/consumer/ConsumerConfig.java | 14 +- .../consumer/ConsumerRebalanceCallback.java | 4 +- .../clients/consumer/ConsumerRecord.java | 16 +-- .../clients/consumer/ConsumerRecords.java | 14 +- .../kafka/clients/consumer/Deserializer.java | 38 ++++++ .../kafka/clients/consumer/KafkaConsumer.java | 124 +++++++++++------- .../kafka/clients/consumer/MockConsumer.java | 6 +- .../clients/producer/ByteArraySerializer.java | 34 +++++ .../kafka/clients/producer/KafkaProducer.java | 76 +++++++++-- .../kafka/clients/producer/MockProducer.java | 14 +- .../kafka/clients/producer/Producer.java | 6 +- .../clients/producer/ProducerConfig.java | 15 ++- .../clients/producer/ProducerRecord.java | 20 +-- .../kafka/clients/producer/Serializer.java | 38 ++++++ .../producer/internals/Partitioner.java | 2 +- .../clients/tools/ProducerPerformance.java | 4 +- .../errors/DeserializationException.java | 47 +++++++ .../common/errors/SerializationException.java | 46 +++++++ .../clients/producer/MockProducerTest.java | 6 +- .../clients/producer/PartitionerTest.java | 12 +- .../scala/kafka/producer/BaseProducer.scala | 4 +- .../kafka/producer/KafkaLog4jAppender.scala | 6 +- .../main/scala/kafka/tools/MirrorMaker.scala | 3 +- .../scala/kafka/tools/ReplayLogProducer.scala | 4 +- .../kafka/tools/TestEndToEndLatency.scala | 4 +- .../scala/kafka/tools/TestLogCleaning.scala | 6 +- .../kafka/api/ProducerCompressionTest.scala | 4 +- .../api/ProducerFailureHandlingTest.scala | 32 ++--- .../kafka/api/ProducerSendTest.scala | 16 +-- .../scala/unit/kafka/utils/TestUtils.scala | 4 +- 32 files changed, 499 insertions(+), 158 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/ByteArrayDeserializer.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/Deserializer.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/producer/ByteArraySerializer.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/producer/Serializer.java create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/DeserializationException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/SerializationException.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ByteArrayDeserializer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ByteArrayDeserializer.java new file mode 100644 index 0000000000000..514cbd2c27a8d --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ByteArrayDeserializer.java @@ -0,0 +1,34 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package org.apache.kafka.clients.consumer; + +import java.util.Map; + +public class ByteArrayDeserializer implements Deserializer { + + @Override + public void configure(Map configs) { + // nothing to do + } + + @Override + public byte[] deserialize(String topic, byte[] data, boolean isKey) { + return data; + } + + @Override + public void close() { + // nothing to do + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java index 227f5646ee708..1bce50185273d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java @@ -23,7 +23,7 @@ * @see KafkaConsumer * @see MockConsumer */ -public interface Consumer extends Closeable { +public interface Consumer extends Closeable { /** * Incrementally subscribe to the given list of topics. This API is mutually exclusive to @@ -63,7 +63,7 @@ public interface Consumer extends Closeable { * of data is controlled by {@link ConsumerConfig#FETCH_MIN_BYTES_CONFIG} and {@link ConsumerConfig#FETCH_MAX_WAIT_MS_CONFIG}. * If no data is available for timeout ms, returns an empty list */ - public Map poll(long timeout); + public Map> poll(long timeout); /** * Commits offsets returned on the last {@link #poll(long) poll()} for the subscribed list of topics and partitions. 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 46efc0c8483ac..1d64f08762b0c 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 @@ -151,6 +151,14 @@ public class ConsumerConfig extends AbstractConfig { public static final String METRIC_REPORTER_CLASSES_CONFIG = "metric.reporters"; private static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters. Implementing the MetricReporter interface allows " + "plugging in classes that will be notified of new metric creation. The JmxReporter is always included to register JMX statistics."; + /** key.deserializer */ + public static final String KEY_DESERIALIZER_CLASS_CONFIG = "key.deserializer"; + private static final String KEY_DESERIALIZER_CLASS_DOC = "Deserializer class for key that implements the Deserializer interface."; + + /** value.deserializer */ + public static final String VALUE_DESERIALIZER_CLASS_CONFIG = "value.deserializer"; + private static final String VALUE_DESERIALIZER_CLASS_DOC = "Deserializer class for value that implements the Deserializer interface."; + static { /* TODO: add config docs */ config = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, "blah blah") @@ -176,8 +184,10 @@ public class ConsumerConfig extends AbstractConfig { Importance.LOW, METRICS_SAMPLE_WINDOW_MS_DOC) .define(METRICS_NUM_SAMPLES_CONFIG, Type.INT, 2, atLeast(1), Importance.LOW, METRICS_NUM_SAMPLES_DOC) - .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, "", Importance.LOW, METRIC_REPORTER_CLASSES_DOC); - + .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, "", Importance.LOW, METRIC_REPORTER_CLASSES_DOC) + .define(KEY_DESERIALIZER_CLASS_CONFIG, Type.CLASS, "org.apache.kafka.clients.consumer.ByteArrayDeserializer", Importance.HIGH, KEY_DESERIALIZER_CLASS_DOC) + .define(VALUE_DESERIALIZER_CLASS_CONFIG, Type.CLASS, "org.apache.kafka.clients.consumer.ByteArrayDeserializer", Importance.HIGH, VALUE_DESERIALIZER_CLASS_DOC); + } ConsumerConfig(Map props) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java index f026ae41ce820..e4cf7d1cfa01c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java @@ -35,7 +35,7 @@ public interface ConsumerRebalanceCallback { * For examples on usage of this API, see Usage Examples section of {@link KafkaConsumer KafkaConsumer} * @param partitions The list of partitions that are assigned to the consumer after rebalance */ - public void onPartitionsAssigned(Consumer consumer, Collection partitions); + public void onPartitionsAssigned(Consumer consumer, Collection partitions); /** * A callback method the user can implement to provide handling of offset commits to a customized store on the @@ -46,5 +46,5 @@ public interface ConsumerRebalanceCallback { * For examples on usage of this API, see Usage Examples section of {@link KafkaConsumer KafkaConsumer} * @param partitions The list of partitions that were assigned to the consumer on the last rebalance */ - public void onPartitionsRevoked(Consumer consumer, Collection partitions); + public void onPartitionsRevoked(Consumer consumer, Collection partitions); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java index 436d8a479166e..16af70a5de52c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java @@ -18,10 +18,10 @@ * A key/value pair to be received from Kafka. This consists of a topic name and a partition number, from which the * record is being received and an offset that points to the record in a Kafka partition. */ -public final class ConsumerRecord { +public final class ConsumerRecord { private final TopicPartition partition; - private final byte[] key; - private final byte[] value; + private final K key; + private final V value; private final long offset; private volatile Exception error; @@ -34,7 +34,7 @@ public final class ConsumerRecord { * @param value The record contents * @param offset The offset of this record in the corresponding Kafka partition */ - public ConsumerRecord(String topic, int partitionId, byte[] key, byte[] value, long offset) { + public ConsumerRecord(String topic, int partitionId, K key, V value, long offset) { this(topic, partitionId, key, value, offset, null); } @@ -46,7 +46,7 @@ public ConsumerRecord(String topic, int partitionId, byte[] key, byte[] value, l * @param value The record contents * @param offset The offset of this record in the corresponding Kafka partition */ - public ConsumerRecord(String topic, int partitionId, byte[] value, long offset) { + public ConsumerRecord(String topic, int partitionId, V value, long offset) { this(topic, partitionId, null, value, offset); } @@ -60,7 +60,7 @@ public ConsumerRecord(String topic, int partitionId, Exception error) { this(topic, partitionId, null, null, -1L, error); } - private ConsumerRecord(String topic, int partitionId, byte[] key, byte[] value, long offset, Exception error) { + private ConsumerRecord(String topic, int partitionId, K key, V value, long offset, Exception error) { if (topic == null) throw new IllegalArgumentException("Topic cannot be null"); this.partition = new TopicPartition(topic, partitionId); @@ -95,7 +95,7 @@ public TopicPartition topicAndPartition() { * The key (or null if no key is specified) * @throws Exception The exception thrown while fetching this record. */ - public byte[] key() throws Exception { + public K key() throws Exception { if (this.error != null) throw this.error; return key; @@ -105,7 +105,7 @@ public byte[] key() throws Exception { * The value * @throws Exception The exception thrown while fetching this record. */ - public byte[] value() throws Exception { + public V value() throws Exception { if (this.error != null) throw this.error; return value; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java index 2ecfc8aaea90a..bdf4b26942d5a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java @@ -21,12 +21,12 @@ * A container that holds the list {@link ConsumerRecord} per partition for a particular topic. There is one for every topic returned by a * {@link Consumer#poll(long)} operation. */ -public class ConsumerRecords { +public class ConsumerRecords { private final String topic; - private final Map> recordsPerPartition; + private final Map>> recordsPerPartition; - public ConsumerRecords(String topic, Map> records) { + public ConsumerRecords(String topic, Map>> records) { this.topic = topic; this.recordsPerPartition = records; } @@ -36,16 +36,16 @@ public ConsumerRecords(String topic, Map> records) * specified, returns records for all partitions * @return The list of {@link ConsumerRecord}s associated with the given partitions. */ - public List records(int... partitions) { - List recordsToReturn = new ArrayList(); + public List> records(int... partitions) { + List> recordsToReturn = new ArrayList>(); if(partitions.length == 0) { // return records for all partitions - for(Entry> record : recordsPerPartition.entrySet()) { + for(Entry>> record : recordsPerPartition.entrySet()) { recordsToReturn.addAll(record.getValue()); } } else { for(int partition : partitions) { - List recordsForThisPartition = recordsPerPartition.get(partition); + List> recordsForThisPartition = recordsPerPartition.get(partition); recordsToReturn.addAll(recordsForThisPartition); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Deserializer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Deserializer.java new file mode 100644 index 0000000000000..fa857d4debbc9 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Deserializer.java @@ -0,0 +1,38 @@ +/** + * 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.clients.consumer; + +import org.apache.kafka.common.Configurable; + +/** + * + * @param Type to be deserialized into. + * + * A class that implements this interface is expected to have a constructor with no parameter. + */ +public interface Deserializer extends Configurable { + /** + * + * @param topic Topic associated with the data + * @param data Serialized bytes + * @param isKey Is data for key or value + * @return deserialized typed data + */ + public T deserialize(String topic, byte[] data, boolean isKey); + + /** + * Close this deserializer + */ + public void close(); +} \ No newline at end of file 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 fe93afa24fc20..a43b1600c707d 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 @@ -12,19 +12,6 @@ */ package org.apache.kafka.clients.consumer; -import java.net.InetSocketAddress; -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.Properties; -import java.util.Set; -import java.util.Map.Entry; -import java.util.concurrent.Future; - -import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.Metric; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.JmxReporter; @@ -36,6 +23,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.net.InetSocketAddress; +import java.util.*; + /** * A Kafka client that consumes records from a Kafka cluster. *

      @@ -50,12 +40,12 @@ * a convenience method to demonstrate the different use cases of the consumer APIs. Here is a sample implementation of such a process() method. *

        * {@code
      - * private Map process(Map records) {
      + * private Map process(Map records) {
        *     Map processedOffsets = new HashMap();
      - *     for(Entry recordMetadata : records.entrySet()) {
      - *          List recordsPerTopic = recordMetadata.getValue().records();
      + *     for(Entry> recordMetadata : records.entrySet()) {
      + *          List> recordsPerTopic = recordMetadata.getValue().records();
        *          for(int i = 0;i < recordsPerTopic.size();i++) {
      - *               ConsumerRecord record = recordsPerTopic.get(i);
      + *               ConsumerRecord record = recordsPerTopic.get(i);
        *               // process record
        *               try {
        *               	processedOffsets.put(record.topicAndpartition(), record.offset());
      @@ -80,11 +70,11 @@
        * props.put("session.timeout.ms", "1000");
        * props.put("enable.auto.commit", "true");
        * props.put("auto.commit.interval.ms", "10000");
      - * KafkaConsumer consumer = new KafkaConsumer(props);
      + * KafkaConsumer consumer = new KafkaConsumer(props);
        * consumer.subscribe("foo", "bar");
        * boolean isRunning = true;
        * while(isRunning) {
      - *   Map records = consumer.poll(100);
      + *   Map> records = consumer.poll(100);
        *   process(records);
        * }
        * consumer.close();
      @@ -102,14 +92,14 @@
        * props.put("group.id", "test");
        * props.put("session.timeout.ms", "1000");
        * props.put("enable.auto.commit", "false");
      - * KafkaConsumer consumer = new KafkaConsumer(props);
      + * KafkaConsumer consumer = new KafkaConsumer(props);
        * consumer.subscribe("foo", "bar");
        * int commitInterval = 100;
        * int numRecords = 0;
        * boolean isRunning = true;
        * Map consumedOffsets = new HashMap();
        * while(isRunning) {
      - *     Map records = consumer.poll(100);
      + *     Map> records = consumer.poll(100);
        *     try {
        *         Map lastConsumedOffsets = process(records);
        *         consumedOffsets.putAll(lastConsumedOffsets);
      @@ -156,16 +146,17 @@
        * props.put("group.id", "test");
        * props.put("session.timeout.ms", "1000");
        * props.put("enable.auto.commit", "false");
      - * KafkaConsumer consumer = new KafkaConsumer(props,
      + * KafkaConsumer consumer = new KafkaConsumer(
      + *                                            props,
        *                                            new ConsumerRebalanceCallback() {
        *                                                boolean rewindOffsets = true;  // should be retrieved from external application config
      - *                                                public void onPartitionsAssigned(Consumer consumer, Collection partitions) {
      + *                                                public void onPartitionsAssigned(Consumer consumer, Collection partitions) {
        *                                                    Map latestCommittedOffsets = consumer.committed(partitions);
        *                                                    if(rewindOffsets)
        *                                                        Map newOffsets = rewindOffsets(latestCommittedOffsets, 100);
        *                                                    consumer.seek(newOffsets);
        *                                                }
      - *                                                public void onPartitionsRevoked(Consumer consumer, Collection partitions) {
      + *                                                public void onPartitionsRevoked(Consumer consumer, Collection partitions) {
        *                                                    consumer.commit(true);
        *                                                }
        *                                                // this API rewinds every partition back by numberOfMessagesToRewindBackTo messages 
      @@ -183,7 +174,7 @@
        * boolean isRunning = true;
        * Map consumedOffsets = new HashMap();
        * while(isRunning) {
      - *     Map records = consumer.poll(100);
      + *     Map> records = consumer.poll(100);
        *     Map lastConsumedOffsets = process(records);
        *     consumedOffsets.putAll(lastConsumedOffsets);
        *     numRecords += records.size();
      @@ -211,13 +202,14 @@
        * props.put("group.id", "test");
        * props.put("session.timeout.ms", "1000");
        * props.put("enable.auto.commit", "false"); // since enable.auto.commit only applies to Kafka based offset storage
      - * KafkaConsumer consumer = new KafkaConsumer(props,
      + * KafkaConsumer consumer = new KafkaConsumer(
      + *                                            props,
        *                                            new ConsumerRebalanceCallback() {
      - *                                                public void onPartitionsAssigned(Consumer consumer, Collection partitions) {
      + *                                                public void onPartitionsAssigned(Consumer consumer, Collection partitions) {
        *                                                    Map lastCommittedOffsets = getLastCommittedOffsetsFromCustomStore(partitions);
        *                                                    consumer.seek(lastCommittedOffsets);
        *                                                }
      - *                                                public void onPartitionsRevoked(Consumer consumer, Collection partitions) {
      + *                                                public void onPartitionsRevoked(Consumer consumer, Collection partitions) {
        *                                                    Map offsets = getLastConsumedOffsets(partitions);
        *                                                    commitOffsetsToCustomStore(offsets); 
        *                                                }
      @@ -234,7 +226,7 @@
        * boolean isRunning = true;
        * Map consumedOffsets = new HashMap();
        * while(isRunning) {
      - *     Map records = consumer.poll(100);
      + *     Map> records = consumer.poll(100);
        *     Map lastConsumedOffsets = process(records);
        *     consumedOffsets.putAll(lastConsumedOffsets);
        *     numRecords += records.size();
      @@ -259,7 +251,7 @@
        * props.put("group.id", "test");
        * props.put("enable.auto.commit", "true");
        * props.put("auto.commit.interval.ms", "10000");
      - * KafkaConsumer consumer = new KafkaConsumer(props);
      + * KafkaConsumer consumer = new KafkaConsumer(props);
        * // subscribe to some partitions of topic foo
        * TopicPartition partition0 = new TopicPartition("foo", 0);
        * TopicPartition partition1 = new TopicPartition("foo", 1);
      @@ -276,7 +268,7 @@
        * boolean isRunning = true;
        * Map consumedOffsets = new HashMap();
        * while(isRunning) {
      - *     Map records = consumer.poll(100);
      + *     Map> records = consumer.poll(100);
        *     Map lastConsumedOffsets = process(records);
        *     consumedOffsets.putAll(lastConsumedOffsets);
        *     for(TopicPartition partition : partitions) {
      @@ -298,7 +290,7 @@
        * {@code  
        * Properties props = new Properties();
        * props.put("metadata.broker.list", "localhost:9092");
      - * KafkaConsumer consumer = new KafkaConsumer(props);
      + * KafkaConsumer consumer = new KafkaConsumer(props);
        * // subscribe to some partitions of topic foo
        * TopicPartition partition0 = new TopicPartition("foo", 0);
        * TopicPartition partition1 = new TopicPartition("foo", 1);
      @@ -314,7 +306,7 @@
        * boolean isRunning = true;
        * Map consumedOffsets = new HashMap();
        * while(isRunning) {
      - *     Map records = consumer.poll(100);
      + *     Map> records = consumer.poll(100);
        *     Map lastConsumedOffsets = process(records);
        *     consumedOffsets.putAll(lastConsumedOffsets);
        *     // commit offsets for partitions 0,1 for topic foo to custom store
      @@ -331,7 +323,7 @@
        * }
        * 
      */ -public class KafkaConsumer implements Consumer { +public class KafkaConsumer implements Consumer { private static final Logger log = LoggerFactory.getLogger(KafkaConsumer.class); @@ -340,7 +332,9 @@ public class KafkaConsumer implements Consumer { private final Metrics metrics; private final Set subscribedTopics; private final Set subscribedPartitions; - + private final Deserializer keyDeserializer; + private final Deserializer valueDeserializer; + /** * A consumer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings * are documented here. Values can be @@ -351,7 +345,7 @@ public class KafkaConsumer implements Consumer { * @param configs The consumer configs */ public KafkaConsumer(Map configs) { - this(new ConsumerConfig(configs), null); + this(new ConsumerConfig(configs), null, null, null); } /** @@ -364,7 +358,24 @@ public KafkaConsumer(Map configs) { * every rebalance operation. */ public KafkaConsumer(Map configs, ConsumerRebalanceCallback callback) { - this(new ConsumerConfig(configs), callback); + this(new ConsumerConfig(configs), callback, null, null); + } + + /** + * A consumer is instantiated by providing a set of key-value pairs as configuration, a {@link ConsumerRebalanceCallback} + * implementation, a key and a value {@link Deserializer}. + *

      + * Valid configuration strings are documented at {@link ConsumerConfig} + * @param configs The consumer configs + * @param callback A callback interface that the user can implement to manage customized offsets on the start and end of + * every rebalance operation. + * @param keyDeserializer The deserializer for key that implements {@link Deserializer}. The configure() method won't + * be called in the consumer when the deserializer is passed in directly. + * @param valueDeserializer The deserializer for value that implements {@link Deserializer}. The configure() method + * won't be called in the consumer when the deserializer is passed in directly. + */ + public KafkaConsumer(Map configs, ConsumerRebalanceCallback callback, Deserializer keyDeserializer, Deserializer valueDeserializer) { + this(new ConsumerConfig(configs), callback, keyDeserializer, valueDeserializer); } /** @@ -372,12 +383,12 @@ public KafkaConsumer(Map configs, ConsumerRebalanceCallback call * Valid configuration strings are documented at {@link ConsumerConfig} */ public KafkaConsumer(Properties properties) { - this(new ConsumerConfig(properties), null); + this(new ConsumerConfig(properties), null, null, null); } /** * A consumer is instantiated by providing a {@link java.util.Properties} object as configuration and a - * {@link ConsumerRebalanceCallback} implementation. + * {@link ConsumerRebalanceCallback} implementation. *

      * Valid configuration strings are documented at {@link ConsumerConfig} * @param properties The consumer configuration properties @@ -385,14 +396,27 @@ public KafkaConsumer(Properties properties) { * every rebalance operation. */ public KafkaConsumer(Properties properties, ConsumerRebalanceCallback callback) { - this(new ConsumerConfig(properties), callback); + this(new ConsumerConfig(properties), callback, null, null); } - private KafkaConsumer(ConsumerConfig config) { - this(config, null); + /** + * A consumer is instantiated by providing a {@link java.util.Properties} object as configuration and a + * {@link ConsumerRebalanceCallback} implementation, a key and a value {@link Deserializer}. + *

      + * Valid configuration strings are documented at {@link ConsumerConfig} + * @param properties The consumer configuration properties + * @param callback A callback interface that the user can implement to manage customized offsets on the start and end of + * every rebalance operation. + * @param keyDeserializer The deserializer for key that implements {@link Deserializer}. The configure() method won't + * be called in the consumer when the deserializer is passed in directly. + * @param valueDeserializer The deserializer for value that implements {@link Deserializer}. The configure() method + * won't be called in the consumer when the deserializer is passed in directly. + */ + public KafkaConsumer(Properties properties, ConsumerRebalanceCallback callback, Deserializer keyDeserializer, Deserializer valueDeserializer) { + this(new ConsumerConfig(properties), callback, keyDeserializer, valueDeserializer); } - private KafkaConsumer(ConsumerConfig config, ConsumerRebalanceCallback callback) { + private KafkaConsumer(ConsumerConfig config, ConsumerRebalanceCallback callback, Deserializer keyDeserializer, Deserializer valueDeserializer) { log.trace("Starting the Kafka consumer"); subscribedTopics = new HashSet(); subscribedPartitions = new HashSet(); @@ -402,6 +426,18 @@ private KafkaConsumer(ConsumerConfig config, ConsumerRebalanceCallback callback) this.metadataFetchTimeoutMs = config.getLong(ConsumerConfig.METADATA_FETCH_TIMEOUT_CONFIG); this.totalMemorySize = config.getLong(ConsumerConfig.TOTAL_BUFFER_MEMORY_CONFIG); List addresses = ClientUtils.parseAndValidateAddresses(config.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); + + if (keyDeserializer == null) + this.keyDeserializer = config.getConfiguredInstance(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + Deserializer.class); + else + this.keyDeserializer = keyDeserializer; + if (valueDeserializer == null) + this.valueDeserializer = config.getConfiguredInstance(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + Deserializer.class); + else + this.valueDeserializer = valueDeserializer; + config.logUnused(); log.debug("Kafka consumer started"); } @@ -488,7 +524,7 @@ public void unsubscribe(TopicPartition... partitions) { * @return map of topic to records since the last fetch for the subscribed list of topics and partitions */ @Override - public Map poll(long timeout) { + public Map> poll(long timeout) { // TODO Auto-generated method stub return null; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java index c3aad3b4d6b67..8cab16c0a0bdb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java @@ -33,7 +33,7 @@ * The consumer runs in the user thread and multiplexes I/O over TCP connections to each of the brokers it * needs to communicate with. Failure to close the consumer after use will leak these resources. */ -public class MockConsumer implements Consumer { +public class MockConsumer implements Consumer { private final Set subscribedPartitions; private final Set subscribedTopics; @@ -90,10 +90,10 @@ public void unsubscribe(TopicPartition... partitions) { } @Override - public Map poll(long timeout) { + public Map> poll(long timeout) { // hand out one dummy record, 1 per topic Map> records = new HashMap>(); - Map recordMetadata = new HashMap(); + Map> recordMetadata = new HashMap>(); for(TopicPartition partition : subscribedPartitions) { // get the last consumed offset long messageSequence = consumedOffsets.get(partition); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ByteArraySerializer.java b/clients/src/main/java/org/apache/kafka/clients/producer/ByteArraySerializer.java new file mode 100644 index 0000000000000..9005b74a328c9 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ByteArraySerializer.java @@ -0,0 +1,34 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package org.apache.kafka.clients.producer; + +import java.util.Map; + +public class ByteArraySerializer implements Serializer { + + @Override + public void configure(Map configs) { + // nothing to do + } + + @Override + public byte[] serialize(String topic, byte[] data, boolean isKey) { + return data; + } + + @Override + public void close() { + // nothing to do + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 32f444ebbd278..f61efb35db7e0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -59,7 +59,7 @@ * The producer manages a single background thread that does I/O as well as a TCP connection to each of the brokers it * needs to communicate with. Failure to close the producer after use will leak these resources. */ -public class KafkaProducer implements Producer { +public class KafkaProducer implements Producer { private static final Logger log = LoggerFactory.getLogger(KafkaProducer.class); @@ -75,26 +75,59 @@ public class KafkaProducer implements Producer { private final CompressionType compressionType; private final Sensor errors; private final Time time; + private final Serializer keySerializer; + private final Serializer valueSerializer; /** * A producer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings * are documented here. Values can be * either strings or Objects of the appropriate type (for example a numeric configuration would accept either the * string "42" or the integer 42). + * @param configs The producer configs + * */ public KafkaProducer(Map configs) { - this(new ProducerConfig(configs)); + this(new ProducerConfig(configs), null, null); + } + + /** + * A producer is instantiated by providing a set of key-value pairs as configuration, a key and a value {@link Serializer}. + * Valid configuration strings are documented here. + * Values can be either strings or Objects of the appropriate type (for example a numeric configuration would accept + * either the string "42" or the integer 42). + * @param configs The producer configs + * @param keySerializer The serializer for key that implements {@link Serializer}. The configure() method won't be + * called in the producer when the serializer is passed in directly. + * @param valueSerializer The serializer for value that implements {@link Serializer}. The configure() method won't + * be called in the producer when the serializer is passed in directly. + */ + public KafkaProducer(Map configs, Serializer keySerializer, Serializer valueSerializer) { + this(new ProducerConfig(configs), keySerializer, valueSerializer); } /** * A producer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings * are documented here. + * @param properties The producer configs */ public KafkaProducer(Properties properties) { - this(new ProducerConfig(properties)); + this(new ProducerConfig(properties), null, null); } - private KafkaProducer(ProducerConfig config) { + /** + * A producer is instantiated by providing a set of key-value pairs as configuration, a key and a value {@link Serializer}. + * Valid configuration strings are documented here. + * @param properties The producer configs + * @param keySerializer The serializer for key that implements {@link Serializer}. The configure() method won't be + * called in the producer when the serializer is passed in directly. + * @param valueSerializer The serializer for value that implements {@link Serializer}. The configure() method won't + * be called in the producer when the serializer is passed in directly. + */ + public KafkaProducer(Properties properties, Serializer keySerializer, Serializer valueSerializer) { + this(new ProducerConfig(properties), keySerializer, valueSerializer); + } + + private KafkaProducer(ProducerConfig config, Serializer keySerializer, Serializer valueSerializer) { log.trace("Starting the Kafka producer"); this.time = new SystemTime(); MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ProducerConfig.METRICS_NUM_SAMPLES_CONFIG)) @@ -145,6 +178,17 @@ private KafkaProducer(ProducerConfig config) { this.errors = this.metrics.sensor("errors"); + if (keySerializer == null) + this.keySerializer = config.getConfiguredInstance(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + Serializer.class); + else + this.keySerializer = keySerializer; + if (valueSerializer == null) + this.valueSerializer = config.getConfiguredInstance(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + Serializer.class); + else + this.valueSerializer = valueSerializer; + config.logUnused(); log.debug("Kafka producer started"); } @@ -159,9 +203,10 @@ private static int parseAcks(String acksString) { /** * Asynchronously send a record to a topic. Equivalent to {@link #send(ProducerRecord, Callback) send(record, null)} + * @param record The record to be sent */ @Override - public Future send(ProducerRecord record) { + public Future send(ProducerRecord record) { return send(record, null); } @@ -183,14 +228,14 @@ public Future send(ProducerRecord record) { * If you want to simulate a simple blocking call you can do the following: * *

      -     *   producer.send(new ProducerRecord("the-topic", "key, "value")).get();
      +     *   producer.send(new ProducerRecord("the-topic", "key".getBytes(), "value".getBytes())).get();
            * 
      *

      * Those desiring fully non-blocking usage can make use of the {@link Callback} parameter to provide a callback that * will be invoked when the request is complete. * *

      -     *   ProducerRecord record = new ProducerRecord("the-topic", "key, "value");
      +     *   ProducerRecord record = new ProducerRecord("the-topic", "key".getBytes(), "value".getBytes());
            *   producer.send(myRecord,
            *                 new Callback() {
            *                     public void onCompletion(RecordMetadata metadata, Exception e) {
      @@ -205,8 +250,8 @@ public Future send(ProducerRecord record) {
            * following example callback1 is guaranteed to execute before callback2:
            * 
            * 
      -     * producer.send(new ProducerRecord(topic, partition, key, value), callback1);
      -     * producer.send(new ProducerRecord(topic, partition, key2, value2), callback2);
      +     * producer.send(new ProducerRecord(topic, partition, key, value), callback1);
      +     * producer.send(new ProducerRecord(topic, partition, key2, value2), callback2);
            * 
      *

      * Note that callbacks will generally execute in the I/O thread of the producer and so should be reasonably fast or @@ -226,16 +271,19 @@ public Future send(ProducerRecord record) { * indicates no callback) */ @Override - public Future send(ProducerRecord record, Callback callback) { + public Future send(ProducerRecord record, Callback callback) { try { // first make sure the metadata for the topic is available waitOnMetadata(record.topic(), this.metadataFetchTimeoutMs); - int partition = partitioner.partition(record, metadata.fetch()); - int serializedSize = Records.LOG_OVERHEAD + Record.recordSize(record.key(), record.value()); + byte[] serializedKey = keySerializer.serialize(record.topic(), record.key(), true); + byte[] serializedValue = valueSerializer.serialize(record.topic(), record.value(), false); + ProducerRecord serializedRecord = new ProducerRecord(record.topic(), record.partition(), serializedKey, serializedValue); + int partition = partitioner.partition(serializedRecord, metadata.fetch()); + int serializedSize = Records.LOG_OVERHEAD + Record.recordSize(serializedKey, serializedValue); ensureValidRecordSize(serializedSize); TopicPartition tp = new TopicPartition(record.topic(), partition); log.trace("Sending record {} with callback {} to topic {} partition {}", record, callback, record.topic(), partition); - RecordAccumulator.RecordAppendResult result = accumulator.append(tp, record.key(), record.value(), compressionType, callback); + RecordAccumulator.RecordAppendResult result = accumulator.append(tp, serializedKey, serializedValue, compressionType, callback); if (result.batchIsFull || result.newBatchCreated) { log.trace("Waking up the sender since topic {} partition {} is either full or getting a new batch", record.topic(), partition); this.sender.wakeup(); @@ -324,6 +372,8 @@ public void close() { throw new KafkaException(e); } this.metrics.close(); + this.keySerializer.close(); + this.valueSerializer.close(); log.debug("The Kafka producer has closed."); } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java index c0f1d57e0feb8..34624c3b7a1f2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java @@ -40,11 +40,11 @@ * By default this mock will synchronously complete each send call successfully. However it can be configured to allow * the user to control the completion of the call and supply an optional error for the producer to throw. */ -public class MockProducer implements Producer { +public class MockProducer implements Producer { private final Cluster cluster; private final Partitioner partitioner = new Partitioner(); - private final List sent; + private final List> sent; private final Deque completions; private boolean autoComplete; private Map offsets; @@ -62,7 +62,7 @@ public MockProducer(Cluster cluster, boolean autoComplete) { this.cluster = cluster; this.autoComplete = autoComplete; this.offsets = new HashMap(); - this.sent = new ArrayList(); + this.sent = new ArrayList>(); this.completions = new ArrayDeque(); } @@ -90,7 +90,7 @@ public MockProducer() { * @see #history() */ @Override - public synchronized Future send(ProducerRecord record) { + public synchronized Future send(ProducerRecord record) { return send(record, null); } @@ -100,7 +100,7 @@ public synchronized Future send(ProducerRecord record) { * @see #history() */ @Override - public synchronized Future send(ProducerRecord record, Callback callback) { + public synchronized Future send(ProducerRecord record, Callback callback) { int partition = 0; if (this.cluster.partitionsForTopic(record.topic()) != null) partition = partitioner.partition(record, this.cluster); @@ -147,8 +147,8 @@ public void close() { /** * Get the list of sent records since the last call to {@link #clear()} */ - public synchronized List history() { - return new ArrayList(this.sent); + public synchronized List> history() { + return new ArrayList>(this.sent); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java index 36e8398416036..5baa6062bd9ba 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java @@ -31,7 +31,7 @@ * @see KafkaProducer * @see MockProducer */ -public interface Producer extends Closeable { +public interface Producer extends Closeable { /** * Send the given record asynchronously and return a future which will eventually contain the response information. @@ -39,12 +39,12 @@ public interface Producer extends Closeable { * @param record The record to send * @return A future which will eventually contain the response information */ - public Future send(ProducerRecord record); + public Future send(ProducerRecord record); /** * Send a record and invoke the given callback when the record has been acknowledged by the server */ - public Future send(ProducerRecord record, Callback callback); + public Future send(ProducerRecord record, Callback callback); /** * Get a list of partitions for the given topic for custom partition assignment. The partition metadata will change diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 72d3ddd0c29bf..a893d88c2f4e2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -17,7 +17,6 @@ import static org.apache.kafka.common.config.ConfigDef.ValidString.in; import java.util.Arrays; -import java.util.List; import java.util.Map; import org.apache.kafka.common.config.AbstractConfig; @@ -175,6 +174,14 @@ public class ProducerConfig extends AbstractConfig { + " Note that if this setting is set to be greater than 1 and there are failed sends, there is a risk of" + " message re-ordering due to retries (i.e., if retries are enabled)."; + /** key.serializer */ + public static final String KEY_SERIALIZER_CLASS_CONFIG = "key.serializer"; + private static final String KEY_SERIALIZER_CLASS_DOC = "Serializer class for key that implements the Serializer interface."; + + /** value.serializer */ + public static final String VALUE_SERIALIZER_CLASS_CONFIG = "value.serializer"; + private static final String VALUE_SERIALIZER_CLASS_DOC = "Serializer class for value that implements the Serializer interface."; + static { config = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, BOOSTRAP_SERVERS_DOC) .define(BUFFER_MEMORY_CONFIG, Type.LONG, 32 * 1024 * 1024L, atLeast(0L), Importance.HIGH, BUFFER_MEMORY_DOC) @@ -182,7 +189,7 @@ public class ProducerConfig extends AbstractConfig { .define(ACKS_CONFIG, Type.STRING, "1", - in(Arrays.asList("all","-1", "0", "1")), + in(Arrays.asList("all", "-1", "0", "1")), Importance.HIGH, ACKS_DOC) .define(COMPRESSION_TYPE_CONFIG, Type.STRING, "none", Importance.HIGH, COMPRESSION_TYPE_DOC) @@ -221,7 +228,9 @@ public class ProducerConfig extends AbstractConfig { 5, atLeast(1), Importance.LOW, - MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_DOC); + MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_DOC) + .define(KEY_SERIALIZER_CLASS_CONFIG, Type.CLASS, "org.apache.kafka.clients.producer.ByteArraySerializer", Importance.HIGH, KEY_SERIALIZER_CLASS_DOC) + .define(VALUE_SERIALIZER_CLASS_CONFIG, Type.CLASS, "org.apache.kafka.clients.producer.ByteArraySerializer", Importance.HIGH, VALUE_SERIALIZER_CLASS_DOC); } ProducerConfig(Map props) { diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java index c3181b368b6cf..065d4e6c6a496 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java @@ -20,12 +20,12 @@ * specified but a key is present a partition will be chosen using a hash of the key. If neither key nor partition is * present a partition will be assigned in a round-robin fashion. */ -public final class ProducerRecord { +public final class ProducerRecord { private final String topic; private final Integer partition; - private final byte[] key; - private final byte[] value; + private final K key; + private final V value; /** * Creates a record to be sent to a specified topic and partition @@ -35,7 +35,7 @@ public final class ProducerRecord { * @param key The key that will be included in the record * @param value The record contents */ - public ProducerRecord(String topic, Integer partition, byte[] key, byte[] value) { + public ProducerRecord(String topic, Integer partition, K key, V value) { if (topic == null) throw new IllegalArgumentException("Topic cannot be null"); this.topic = topic; @@ -51,7 +51,7 @@ public ProducerRecord(String topic, Integer partition, byte[] key, byte[] value) * @param key The key that will be included in the record * @param value The record contents */ - public ProducerRecord(String topic, byte[] key, byte[] value) { + public ProducerRecord(String topic, K key, V value) { this(topic, null, key, value); } @@ -61,7 +61,7 @@ public ProducerRecord(String topic, byte[] key, byte[] value) { * @param topic The topic this record should be sent to * @param value The record contents */ - public ProducerRecord(String topic, byte[] value) { + public ProducerRecord(String topic, V value) { this(topic, null, value); } @@ -75,14 +75,14 @@ public String topic() { /** * The key (or null if no key is specified) */ - public byte[] key() { + public K key() { return key; } /** * @return The value */ - public byte[] value() { + public V value() { return value; } @@ -95,8 +95,8 @@ public Integer partition() { @Override public String toString() { - String key = this.key == null ? "null" : ("byte[" + this.key.length + "]"); - String value = this.value == null ? "null" : ("byte[" + this.value.length + "]"); + String key = this.key == null ? "null" : this.key.toString(); + String value = this.value == null ? "null" : this.value.toString(); return "ProducerRecord(topic=" + topic + ", partition=" + partition + ", key=" + key + ", value=" + value; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/Serializer.java b/clients/src/main/java/org/apache/kafka/clients/producer/Serializer.java new file mode 100644 index 0000000000000..0378683508396 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/producer/Serializer.java @@ -0,0 +1,38 @@ +/** + * 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.clients.producer; + +import org.apache.kafka.common.Configurable; + +/** + * + * @param Type to be serialized from. + * + * A class that implements this interface is expected to have a constructor with no parameter. + */ +public interface Serializer extends Configurable { + /** + * + * @param topic Topic associated with data + * @param data Typed data + * @param isKey Is data for key or value + * @return bytes of the serialized data + */ + public byte[] serialize(String topic, T data, boolean isKey); + + /** + * Close this serializer + */ + public void close(); +} diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java index 40e8234f87710..483899d2e69b3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java @@ -43,7 +43,7 @@ public class Partitioner { * @param record The record being sent * @param cluster The current cluster metadata */ - public int partition(ProducerRecord record, Cluster cluster) { + public int partition(ProducerRecord record, Cluster cluster) { List partitions = cluster.partitionsForTopic(record.topic()); int numPartitions = partitions.size(); if (record.partition() != null) { diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java b/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java index 28175fb71edbe..1b828007975ef 100644 --- a/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java +++ b/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java @@ -46,12 +46,12 @@ public static void main(String[] args) throws Exception { throw new IllegalArgumentException("Invalid property: " + args[i]); props.put(pieces[0], pieces[1]); } - KafkaProducer producer = new KafkaProducer(props); + KafkaProducer producer = new KafkaProducer(props); /* setup perf test */ byte[] payload = new byte[recordSize]; Arrays.fill(payload, (byte) 1); - ProducerRecord record = new ProducerRecord(topicName, payload); + ProducerRecord record = new ProducerRecord(topicName, payload); long sleepTime = NS_PER_SEC / throughput; long sleepDeficitNs = 0; Stats stats = new Stats(numRecords, 5000); diff --git a/clients/src/main/java/org/apache/kafka/common/errors/DeserializationException.java b/clients/src/main/java/org/apache/kafka/common/errors/DeserializationException.java new file mode 100644 index 0000000000000..a5433398fb978 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/DeserializationException.java @@ -0,0 +1,47 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package org.apache.kafka.common.errors; + +import org.apache.kafka.common.KafkaException; + +/** + * Any exception during deserialization in the consumer + */ +public class DeserializationException extends KafkaException { + + private static final long serialVersionUID = 1L; + + public DeserializationException(String message, Throwable cause) { + super(message, cause); + } + + public DeserializationException(String message) { + super(message); + } + + public DeserializationException(Throwable cause) { + super(cause); + } + + public DeserializationException() { + super(); + } + + /* avoid the expensive and useless stack trace for deserialization exceptions */ + @Override + public Throwable fillInStackTrace() { + return this; + } + +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/errors/SerializationException.java b/clients/src/main/java/org/apache/kafka/common/errors/SerializationException.java new file mode 100644 index 0000000000000..00388d12794f0 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/SerializationException.java @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.common.errors; + +import org.apache.kafka.common.KafkaException; + +/** + * Any exception during serialization in the producer + */ +public class SerializationException extends KafkaException { + + private static final long serialVersionUID = 1L; + + public SerializationException(String message, Throwable cause) { + super(message, cause); + } + + public SerializationException(String message) { + super(message); + } + + public SerializationException(Throwable cause) { + super(cause); + } + + public SerializationException() { + super(); + } + + /* avoid the expensive and useless stack trace for serialization exceptions */ + @Override + public Throwable fillInStackTrace() { + return this; + } + +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java index 9a9411fc900bf..1e2ca03fafa36 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java @@ -37,7 +37,7 @@ public class MockProducerTest { @Test public void testAutoCompleteMock() throws Exception { MockProducer producer = new MockProducer(true); - ProducerRecord record = new ProducerRecord(topic, "key".getBytes(), "value".getBytes()); + ProducerRecord record = new ProducerRecord(topic, "key".getBytes(), "value".getBytes()); Future metadata = producer.send(record); assertTrue("Send should be immediately complete", metadata.isDone()); assertFalse("Send should be successful", isError(metadata)); @@ -51,8 +51,8 @@ public void testAutoCompleteMock() throws Exception { @Test public void testManualCompletion() throws Exception { MockProducer producer = new MockProducer(false); - ProducerRecord record1 = new ProducerRecord("topic", "key1".getBytes(), "value1".getBytes()); - ProducerRecord record2 = new ProducerRecord("topic", "key2".getBytes(), "value2".getBytes()); + ProducerRecord record1 = new ProducerRecord("topic", "key1".getBytes(), "value1".getBytes()); + ProducerRecord record2 = new ProducerRecord("topic", "key2".getBytes(), "value2".getBytes()); Future md1 = producer.send(record1); assertFalse("Send shouldn't have completed", md1.isDone()); Future md2 = producer.send(record2); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java index f06e28ce21e80..1d077fd4c56bf 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java @@ -50,22 +50,22 @@ public class PartitionerTest { public void testUserSuppliedPartitioning() { assertEquals("If the user supplies a partition we should use it.", 0, - partitioner.partition(new ProducerRecord("test", 0, key, value), cluster)); + partitioner.partition(new ProducerRecord("test", 0, key, value), cluster)); } @Test public void testKeyPartitionIsStable() { - int partition = partitioner.partition(new ProducerRecord("test", key, value), cluster); + int partition = partitioner.partition(new ProducerRecord("test", key, value), cluster); assertEquals("Same key should yield same partition", partition, - partitioner.partition(new ProducerRecord("test", key, "value2".getBytes()), cluster)); + partitioner.partition(new ProducerRecord("test", key, "value2".getBytes()), cluster)); } @Test public void testRoundRobinIsStable() { - int startPart = partitioner.partition(new ProducerRecord("test", value), cluster); + int startPart = partitioner.partition(new ProducerRecord("test", value), cluster); for (int i = 1; i <= 100; i++) { - int partition = partitioner.partition(new ProducerRecord("test", value), cluster); + int partition = partitioner.partition(new ProducerRecord("test", value), cluster); assertEquals("Should yield a different partition each call with round-robin partitioner", partition, (startPart + i) % 2); } @@ -74,7 +74,7 @@ public void testRoundRobinIsStable() { @Test public void testRoundRobinWithDownNode() { for (int i = 0; i < partitions.size(); i++) { - int part = partitioner.partition(new ProducerRecord("test", value), cluster); + int part = partitioner.partition(new ProducerRecord("test", value), cluster); assertTrue("We should never choose a leader-less node in round robin", part >= 0 && part < 2); } } diff --git a/core/src/main/scala/kafka/producer/BaseProducer.scala b/core/src/main/scala/kafka/producer/BaseProducer.scala index b0207930dd054..8e007130b2825 100644 --- a/core/src/main/scala/kafka/producer/BaseProducer.scala +++ b/core/src/main/scala/kafka/producer/BaseProducer.scala @@ -33,10 +33,10 @@ class NewShinyProducer(producerProps: Properties) extends BaseProducer { // decide whether to send synchronously based on producer properties val sync = producerProps.getProperty("producer.type", "async").equals("sync") - val producer = new KafkaProducer(producerProps) + val producer = new KafkaProducer[Array[Byte],Array[Byte]](producerProps) override def send(topic: String, key: Array[Byte], value: Array[Byte]) { - val record = new ProducerRecord(topic, key, value) + val record = new ProducerRecord[Array[Byte],Array[Byte]](topic, key, value) if(sync) { this.producer.send(record).get() } else { diff --git a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala index 4b5b823b85477..e194942492324 100644 --- a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala +++ b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala @@ -32,7 +32,7 @@ class KafkaLog4jAppender extends AppenderSkeleton with Logging { var requiredNumAcks: Int = Int.MaxValue var syncSend: Boolean = false - private var producer: KafkaProducer = null + private var producer: KafkaProducer[Array[Byte],Array[Byte]] = null def getTopic: String = topic def setTopic(topic: String) { this.topic = topic } @@ -60,7 +60,7 @@ class KafkaLog4jAppender extends AppenderSkeleton with Logging { throw new MissingConfigException("topic must be specified by the Kafka log4j appender") if(compressionType != null) props.put(org.apache.kafka.clients.producer.ProducerConfig.COMPRESSION_TYPE_CONFIG, compressionType) if(requiredNumAcks != Int.MaxValue) props.put(org.apache.kafka.clients.producer.ProducerConfig.ACKS_CONFIG, requiredNumAcks.toString) - producer = new KafkaProducer(props) + producer = new KafkaProducer[Array[Byte],Array[Byte]](props) LogLog.debug("Kafka producer connected to " + brokerList) LogLog.debug("Logging for topic: " + topic) } @@ -68,7 +68,7 @@ class KafkaLog4jAppender extends AppenderSkeleton with Logging { override def append(event: LoggingEvent) { val message = subAppend(event) LogLog.debug("[" + new Date(event.getTimeStamp).toString + "]" + message) - val response = producer.send(new ProducerRecord(topic, message.getBytes())) + val response = producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic, message.getBytes())) if (syncSend) response.get } diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index 77d951d13b8d8..53cb16c2949e0 100644 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -27,7 +27,6 @@ import kafka.producer.{OldProducer, NewShinyProducer} import kafka.metrics.KafkaMetricsGroup import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback import org.apache.kafka.clients.producer.{RecordMetadata, ProducerRecord} -import org.apache.kafka.common.KafkaException import scala.collection.JavaConversions._ @@ -547,7 +546,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { extends NewShinyProducer(producerProps) with MirrorMakerBaseProducer { override def send(topicPartition: TopicAndPartition, offset: Long, key: Array[Byte], value: Array[Byte]) { - val record = new ProducerRecord(topicPartition.topic, key, value) + val record = new ProducerRecord[Array[Byte],Array[Byte]](topicPartition.topic, key, value) if(sync) { topicPartitionOffsetMap.getAndMaybePut(topicPartition).put(this.producer.send(record).get().partition(), offset) } else { diff --git a/core/src/main/scala/kafka/tools/ReplayLogProducer.scala b/core/src/main/scala/kafka/tools/ReplayLogProducer.scala index 3393a3dd574ac..f541987b2876a 100644 --- a/core/src/main/scala/kafka/tools/ReplayLogProducer.scala +++ b/core/src/main/scala/kafka/tools/ReplayLogProducer.scala @@ -124,7 +124,7 @@ object ReplayLogProducer extends Logging { class ZKConsumerThread(config: Config, stream: KafkaStream[Array[Byte], Array[Byte]]) extends Thread with Logging { val shutdownLatch = new CountDownLatch(1) - val producer = new KafkaProducer(config.producerProps) + val producer = new KafkaProducer[Array[Byte],Array[Byte]](config.producerProps) override def run() { info("Starting consumer thread..") @@ -137,7 +137,7 @@ object ReplayLogProducer extends Logging { stream for (messageAndMetadata <- iter) { try { - val response = producer.send(new ProducerRecord(config.outputTopic, + val response = producer.send(new ProducerRecord[Array[Byte],Array[Byte]](config.outputTopic, messageAndMetadata.key(), messageAndMetadata.message())) if(config.isSync) { response.get() diff --git a/core/src/main/scala/kafka/tools/TestEndToEndLatency.scala b/core/src/main/scala/kafka/tools/TestEndToEndLatency.scala index 67196f30af1cf..2ebc7bf643ea9 100644 --- a/core/src/main/scala/kafka/tools/TestEndToEndLatency.scala +++ b/core/src/main/scala/kafka/tools/TestEndToEndLatency.scala @@ -56,7 +56,7 @@ object TestEndToEndLatency { producerProps.put(ProducerConfig.LINGER_MS_CONFIG, "0") producerProps.put(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "true") producerProps.put(ProducerConfig.ACKS_CONFIG, producerAcks.toString) - val producer = new KafkaProducer(producerProps) + val producer = new KafkaProducer[Array[Byte],Array[Byte]](producerProps) // make sure the consumer fetcher has started before sending data since otherwise // the consumption from the tail will skip the first message and hence be blocked @@ -67,7 +67,7 @@ object TestEndToEndLatency { val latencies = new Array[Long](numMessages) for (i <- 0 until numMessages) { val begin = System.nanoTime - producer.send(new ProducerRecord(topic, message)) + producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic, message)) val received = iter.next val elapsed = System.nanoTime - begin // poor man's progress bar diff --git a/core/src/main/scala/kafka/tools/TestLogCleaning.scala b/core/src/main/scala/kafka/tools/TestLogCleaning.scala index 1d4ea93f2ba8d..b81010ec0fa98 100644 --- a/core/src/main/scala/kafka/tools/TestLogCleaning.scala +++ b/core/src/main/scala/kafka/tools/TestLogCleaning.scala @@ -242,7 +242,7 @@ object TestLogCleaning { val producerProps = new Properties producerProps.setProperty(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "true") producerProps.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerUrl) - val producer = new KafkaProducer(producerProps) + val producer = new KafkaProducer[Array[Byte],Array[Byte]](producerProps) val rand = new Random(1) val keyCount = (messages / dups).toInt val producedFile = File.createTempFile("kafka-log-cleaner-produced-", ".txt") @@ -254,9 +254,9 @@ object TestLogCleaning { val delete = i % 100 < percentDeletes val msg = if(delete) - new ProducerRecord(topic, key.toString.getBytes(), null) + new ProducerRecord[Array[Byte],Array[Byte]](topic, key.toString.getBytes(), null) else - new ProducerRecord(topic, key.toString.getBytes(), i.toString.getBytes()) + new ProducerRecord[Array[Byte],Array[Byte]](topic, key.toString.getBytes(), i.toString.getBytes()) producer.send(msg) producedWriter.write(TestRecord(topic, key, i, delete).toString) producedWriter.newLine() diff --git a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala index 6379f2b60af79..1505fd4464dc9 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala @@ -75,7 +75,7 @@ class ProducerCompressionTest(compression: String) extends JUnit3Suite with ZooK props.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, compression) props.put(ProducerConfig.BATCH_SIZE_CONFIG, "66000") props.put(ProducerConfig.LINGER_MS_CONFIG, "200") - var producer = new KafkaProducer(props) + var producer = new KafkaProducer[Array[Byte],Array[Byte]](props) val consumer = new SimpleConsumer("localhost", port, 100, 1024*1024, "") try { @@ -89,7 +89,7 @@ class ProducerCompressionTest(compression: String) extends JUnit3Suite with ZooK // make sure the returned messages are correct val responses = for (message <- messages) - yield producer.send(new ProducerRecord(topic, null, null, message)) + yield producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic, null, null, message)) val futures = responses.toList for ((future, offset) <- futures zip (0 until numRecords)) { assertEquals(offset.toLong, future.get.offset) diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index a913fe59ba6f7..5ec613cdb50b9 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -52,10 +52,10 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { private var consumer1: SimpleConsumer = null private var consumer2: SimpleConsumer = null - private var producer1: KafkaProducer = null - private var producer2: KafkaProducer = null - private var producer3: KafkaProducer = null - private var producer4: KafkaProducer = 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" @@ -93,7 +93,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { TestUtils.createTopic(zkClient, topic1, 1, 2, servers) // send a too-large record - val record = new ProducerRecord(topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1)) + val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1)) assertEquals("Returned metadata should have offset -1", producer1.send(record).get.offset, -1L) } @@ -106,7 +106,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { TestUtils.createTopic(zkClient, topic1, 1, 2, servers) // send a too-large record - val record = new ProducerRecord(topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1)) + val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1)) intercept[ExecutionException] { producer2.send(record).get } @@ -118,7 +118,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { @Test def testNonExistentTopic() { // send a record with non-exist topic - val record = new ProducerRecord(topic2, null, "key".getBytes, "value".getBytes) + val record = new ProducerRecord[Array[Byte],Array[Byte]](topic2, null, "key".getBytes, "value".getBytes) intercept[ExecutionException] { producer1.send(record).get } @@ -143,7 +143,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { producer4 = TestUtils.createNewProducer("localhost:8686,localhost:4242", acks = 1, blockOnBufferFull = false, bufferSize = producerBufferSize) // send a record with incorrect broker list - val record = new ProducerRecord(topic1, null, "key".getBytes, "value".getBytes) + val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, "value".getBytes) intercept[ExecutionException] { producer4.send(record).get } @@ -160,7 +160,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { TestUtils.createTopic(zkClient, topic1, 1, 2, servers) // first send a message to make sure the metadata is refreshed - val record1 = new ProducerRecord(topic1, null, "key".getBytes, "value".getBytes) + val record1 = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, "value".getBytes) producer1.send(record1).get producer2.send(record1).get @@ -180,7 +180,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { val msgSize = producerBufferSize / tooManyRecords val value = new Array[Byte](msgSize) new Random().nextBytes(value) - val record2 = new ProducerRecord(topic1, null, "key".getBytes, value) + val record2 = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, value) intercept[KafkaException] { for (i <- 1 to tooManyRecords) @@ -201,7 +201,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { TestUtils.createTopic(zkClient, topic1, 1, 2, servers) // create a record with incorrect partition id, send should fail - val record = new ProducerRecord(topic1, new Integer(1), "key".getBytes, "value".getBytes) + val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, new Integer(1), "key".getBytes, "value".getBytes) intercept[IllegalArgumentException] { producer1.send(record) } @@ -221,7 +221,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { // create topic TestUtils.createTopic(zkClient, topic1, 1, 2, servers) - val record = new ProducerRecord(topic1, null, "key".getBytes, "value".getBytes) + val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, "value".getBytes) // first send a message to make sure the metadata is refreshed producer1.send(record).get @@ -300,7 +300,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { @Test def testCannotSendToInternalTopic() { val thrown = intercept[ExecutionException] { - producer2.send(new ProducerRecord(Topic.InternalTopics.head, "test".getBytes, "test".getBytes)).get + producer2.send(new ProducerRecord[Array[Byte],Array[Byte]](Topic.InternalTopics.head, "test".getBytes, "test".getBytes)).get } assertTrue(thrown.getCause.isInstanceOf[InvalidTopicException]) } @@ -313,7 +313,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { TestUtils.createTopic(zkClient, topicName, 1, 2, servers,topicProps) - val record = new ProducerRecord(topicName, null, "key".getBytes, "value".getBytes) + val record = new ProducerRecord[Array[Byte],Array[Byte]](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") @@ -333,7 +333,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { TestUtils.createTopic(zkClient, topicName, 1, 2, servers,topicProps) - val record = new ProducerRecord(topicName, null, "key".getBytes, "value".getBytes) + val record = new ProducerRecord[Array[Byte],Array[Byte]](topicName, null, "key".getBytes, "value".getBytes) // this should work with all brokers up and running producer3.send(record).get @@ -365,7 +365,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { override def doWork(): Unit = { val responses = for (i <- sent+1 to sent+numRecords) - yield producer.send(new ProducerRecord(topic1, null, null, i.toString.getBytes)) + yield producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, null, i.toString.getBytes)) val futures = responses.toList try { diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala index d407af9144ef6..6196060edf9f1 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala @@ -86,24 +86,24 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { TestUtils.createTopic(zkClient, topic, 1, 2, servers) // send a normal record - val record0 = new ProducerRecord(topic, new Integer(0), "key".getBytes, "value".getBytes) + val record0 = new ProducerRecord[Array[Byte],Array[Byte]](topic, new Integer(0), "key".getBytes, "value".getBytes) assertEquals("Should have offset 0", 0L, producer.send(record0, callback).get.offset) // send a record with null value should be ok - val record1 = new ProducerRecord(topic, new Integer(0), "key".getBytes, null) + val record1 = new ProducerRecord[Array[Byte],Array[Byte]](topic, new Integer(0), "key".getBytes, null) assertEquals("Should have offset 1", 1L, producer.send(record1, callback).get.offset) // send a record with null key should be ok - val record2 = new ProducerRecord(topic, new Integer(0), null, "value".getBytes) + val record2 = new ProducerRecord[Array[Byte],Array[Byte]](topic, new Integer(0), null, "value".getBytes) assertEquals("Should have offset 2", 2L, producer.send(record2, callback).get.offset) // send a record with null part id should be ok - val record3 = new ProducerRecord(topic, null, "key".getBytes, "value".getBytes) + val record3 = new ProducerRecord[Array[Byte],Array[Byte]](topic, null, "key".getBytes, "value".getBytes) assertEquals("Should have offset 3", 3L, producer.send(record3, callback).get.offset) // send a record with null topic should fail try { - val record4 = new ProducerRecord(null, new Integer(0), "key".getBytes, "value".getBytes) + val record4 = new ProducerRecord[Array[Byte],Array[Byte]](null, new Integer(0), "key".getBytes, "value".getBytes) producer.send(record4, callback) fail("Should not allow sending a record without topic") } catch { @@ -140,7 +140,7 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { TestUtils.createTopic(zkClient, topic, 1, 2, servers) // non-blocking send a list of records - val record0 = new ProducerRecord(topic, null, "key".getBytes, "value".getBytes) + val record0 = new ProducerRecord[Array[Byte],Array[Byte]](topic, null, "key".getBytes, "value".getBytes) for (i <- 1 to numRecords) producer.send(record0) val response0 = producer.send(record0) @@ -182,7 +182,7 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { val responses = for (i <- 1 to numRecords) - yield producer.send(new ProducerRecord(topic, partition, null, ("value" + i).getBytes)) + yield producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, null, ("value" + i).getBytes)) val futures = responses.toList futures.map(_.get) for (future <- futures) @@ -228,7 +228,7 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { try { // Send a message to auto-create the topic - val record = new ProducerRecord(topic, null, "key".getBytes, "value".getBytes) + val record = new ProducerRecord[Array[Byte],Array[Byte]](topic, null, "key".getBytes, "value".getBytes) assertEquals("Should have offset 0", 0L, producer.send(record).get.offset) // double check that the topic is created with leader elected diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 0da774d0ed015..94d0028d8c490 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -383,7 +383,7 @@ object TestUtils extends Logging { metadataFetchTimeout: Long = 3000L, blockOnBufferFull: Boolean = true, bufferSize: Long = 1024L * 1024L, - retries: Int = 0) : KafkaProducer = { + retries: Int = 0) : KafkaProducer[Array[Byte],Array[Byte]] = { import org.apache.kafka.clients.producer.ProducerConfig val producerProps = new Properties() @@ -395,7 +395,7 @@ object TestUtils extends Logging { producerProps.put(ProducerConfig.RETRIES_CONFIG, retries.toString) producerProps.put(ProducerConfig.RETRY_BACKOFF_MS_CONFIG, "100") producerProps.put(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG, "200") - return new KafkaProducer(producerProps) + return new KafkaProducer[Array[Byte],Array[Byte]](producerProps) } /** From 84521872d18dd90aff6eb47d470893f64749783f Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Thu, 18 Dec 2014 17:05:02 -0800 Subject: [PATCH 076/491] KAFKA-1824 - fix ConsoleProducer so parse.key and key.separator will work again; reviewed by Neha Narkhede --- core/src/main/scala/kafka/tools/ConsoleProducer.scala | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/kafka/tools/ConsoleProducer.scala b/core/src/main/scala/kafka/tools/ConsoleProducer.scala index 1061cc74fac69..ac85949447028 100644 --- a/core/src/main/scala/kafka/tools/ConsoleProducer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleProducer.scala @@ -36,10 +36,8 @@ object ConsoleProducer { val reader = Class.forName(config.readerClass).newInstance().asInstanceOf[MessageReader] val props = new Properties props.putAll(config.cmdLineProps) - - val readerProps = new Properties(props) - readerProps.put("topic", config.topic) - reader.init(System.in, readerProps) + props.put("topic", config.topic) + reader.init(System.in, props) try { val producer = @@ -134,13 +132,13 @@ object ConsoleProducer { .ofType(classOf[java.lang.Integer]) .defaultsTo(100) val sendTimeoutOpt = parser.accepts("timeout", "If set and the producer is running in asynchronous mode, this gives the maximum amount of time" + - " a message will queue awaiting suffient batch size. The value is given in ms.") + " a message will queue awaiting sufficient batch size. The value is given in ms.") .withRequiredArg .describedAs("timeout_ms") .ofType(classOf[java.lang.Integer]) .defaultsTo(1000) val queueSizeOpt = parser.accepts("queue-size", "If set and the producer is running in asynchronous mode, this gives the maximum amount of " + - " messages will queue awaiting suffient batch size.") + " messages will queue awaiting sufficient batch size.") .withRequiredArg .describedAs("queue_size") .ofType(classOf[java.lang.Integer]) From 10c6dec34dae8820bf7ce24839c938135f9a9189 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Tue, 23 Dec 2014 17:04:45 -0800 Subject: [PATCH 077/491] KAFKA-1650; (Follow-up patch) to support no data loss in mirror maker; reviewed by Joel Koshy --- .../kafka/consumer/ConsumerConnector.scala | 15 + .../consumer/ZookeeperConsumerConnector.scala | 56 +-- .../consumer/ZookeeperConsumerConnector.scala | 9 +- .../main/scala/kafka/tools/MirrorMaker.scala | 335 ++++++++++-------- .../scala/kafka/utils/DoublyLinkedList.scala | 126 +++++++ .../scala/unit/kafka/utils/UtilsTest.scala | 50 +++ 6 files changed, 423 insertions(+), 168 deletions(-) create mode 100644 core/src/main/scala/kafka/utils/DoublyLinkedList.scala diff --git a/core/src/main/scala/kafka/consumer/ConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ConsumerConnector.scala index 62c0686e816d2..384be74703050 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConnector.scala @@ -17,6 +17,9 @@ package kafka.consumer +import kafka.common.{OffsetAndMetadata, TopicAndPartition} +import kafka.javaapi.consumer.ConsumerRebalanceListener + import scala.collection._ import kafka.utils.Logging import kafka.serializer._ @@ -76,6 +79,18 @@ trait ConsumerConnector { * KAFKA-1743: This method added for backward compatibility. */ def commitOffsets + + /** + * Commit offsets from an external offsets map. + * @param offsetsToCommit the offsets to be committed. + */ + def commitOffsets(offsetsToCommit: immutable.Map[TopicAndPartition, OffsetAndMetadata], retryOnFailure: Boolean) + + /** + * Wire in a consumer rebalance listener to be executed when consumer rebalance occurs. + * @param listener The consumer rebalance listener to wire in + */ + def setConsumerRebalanceListener(listener: ConsumerRebalanceListener) /** * Shut down the connector diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index e991d2187d032..191a8677444e5 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -302,25 +302,28 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, def commitOffsets { commitOffsets(true) } def commitOffsets(isAutoCommit: Boolean) { - commitOffsets(isAutoCommit, null) + + val offsetsToCommit = + immutable.Map(topicRegistry.flatMap { case (topic, partitionTopicInfos) => + partitionTopicInfos.map { case (partition, info) => + TopicAndPartition(info.topic, info.partitionId) -> OffsetAndMetadata(info.getConsumeOffset()) + } + }.toSeq: _*) + + commitOffsets(offsetsToCommit, isAutoCommit) + } - def commitOffsets(isAutoCommit: Boolean, - topicPartitionOffsets: immutable.Map[TopicAndPartition, OffsetAndMetadata]) { - var retriesRemaining = 1 + (if (isAutoCommit) config.offsetsCommitMaxRetries else 0) // no retries for commits from auto-commit + def commitOffsets(offsetsToCommit: immutable.Map[TopicAndPartition, OffsetAndMetadata], isAutoCommit: Boolean) { + trace("OffsetMap: %s".format(offsetsToCommit)) + var retriesRemaining = 1 + (if (isAutoCommit) 0 else config.offsetsCommitMaxRetries) // no retries for commits from auto-commit var done = false - while (!done) { - val committed = offsetsChannelLock synchronized { // committed when we receive either no error codes or only MetadataTooLarge errors - val offsetsToCommit = if (topicPartitionOffsets == null) {immutable.Map(topicRegistry.flatMap { case (topic, partitionTopicInfos) => - partitionTopicInfos.map { case (partition, info) => - TopicAndPartition(info.topic, info.partitionId) -> OffsetAndMetadata(info.getConsumeOffset()) - } - }.toSeq:_*)} else topicPartitionOffsets - + val committed = offsetsChannelLock synchronized { + // committed when we receive either no error codes or only MetadataTooLarge errors if (offsetsToCommit.size > 0) { if (config.offsetsStorage == "zookeeper") { - offsetsToCommit.foreach { case(topicAndPartition, offsetAndMetadata) => + offsetsToCommit.foreach { case (topicAndPartition, offsetAndMetadata) => commitOffsetToZooKeeper(topicAndPartition, offsetAndMetadata.offset) } true @@ -334,25 +337,25 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, trace("Offset commit response: %s.".format(offsetCommitResponse)) val (commitFailed, retryableIfFailed, shouldRefreshCoordinator, errorCount) = { - offsetCommitResponse.commitStatus.foldLeft(false, false, false, 0) { case(folded, (topicPartition, errorCode)) => + offsetCommitResponse.commitStatus.foldLeft(false, false, false, 0) { case (folded, (topicPartition, errorCode)) => if (errorCode == ErrorMapping.NoError && config.dualCommitEnabled) { - val offset = offsetsToCommit(topicPartition).offset - commitOffsetToZooKeeper(topicPartition, offset) + val offset = offsetsToCommit(topicPartition).offset + commitOffsetToZooKeeper(topicPartition, offset) } (folded._1 || // update commitFailed - errorCode != ErrorMapping.NoError, + errorCode != ErrorMapping.NoError, - folded._2 || // update retryableIfFailed - (only metadata too large is not retryable) - (errorCode != ErrorMapping.NoError && errorCode != ErrorMapping.OffsetMetadataTooLargeCode), + folded._2 || // update retryableIfFailed - (only metadata too large is not retryable) + (errorCode != ErrorMapping.NoError && errorCode != ErrorMapping.OffsetMetadataTooLargeCode), - folded._3 || // update shouldRefreshCoordinator - errorCode == ErrorMapping.NotCoordinatorForConsumerCode || - errorCode == ErrorMapping.ConsumerCoordinatorNotAvailableCode, + folded._3 || // update shouldRefreshCoordinator + errorCode == ErrorMapping.NotCoordinatorForConsumerCode || + errorCode == ErrorMapping.ConsumerCoordinatorNotAvailableCode, - // update error count - folded._4 + (if (errorCode != ErrorMapping.NoError) 1 else 0)) + // update error count + folded._4 + (if (errorCode != ErrorMapping.NoError) 1 else 0)) } } debug(errorCount + " errors in offset commit response.") @@ -381,11 +384,10 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, } } - done = if (isShuttingDown.get() && isAutoCommit) { // should not retry indefinitely if shutting down + done = { retriesRemaining -= 1 retriesRemaining == 0 || committed - } else - true + } if (!done) { debug("Retrying offset commit in %d ms".format(config.offsetsChannelBackoffMs)) diff --git a/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala index 9baad34a9793e..bfd8d37b51e17 100644 --- a/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/javaapi/consumer/ZookeeperConsumerConnector.scala @@ -18,9 +18,8 @@ package kafka.javaapi.consumer import kafka.serializer._ import kafka.consumer._ -import kafka.common.MessageStreamsExistException -import scala.collection.mutable -import scala.collection.JavaConversions +import kafka.common.{OffsetAndMetadata, TopicAndPartition, MessageStreamsExistException} +import scala.collection.{immutable, mutable, JavaConversions} import java.util.concurrent.atomic.AtomicBoolean /** @@ -115,6 +114,10 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, underlying.commitOffsets(retryOnFailure) } + def commitOffsets(offsetsToCommit: java.util.Map[TopicAndPartition, OffsetAndMetadata], retryOnFailure: Boolean) { + underlying.commitOffsets(offsetsToCommit.asInstanceOf[immutable.Map[TopicAndPartition, OffsetAndMetadata]], retryOnFailure) + } + def setConsumerRebalanceListener(consumerRebalanceListener: ConsumerRebalanceListener) { underlying.setConsumerRebalanceListener(consumerRebalanceListener) } diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index 53cb16c2949e0..191542c7b6724 100644 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -23,10 +23,11 @@ import kafka.javaapi.consumer.ConsumerRebalanceListener import kafka.utils._ import kafka.consumer._ import kafka.serializer._ -import kafka.producer.{OldProducer, NewShinyProducer} +import kafka.producer.{KeyedMessage, ProducerConfig} import kafka.metrics.KafkaMetricsGroup import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback -import org.apache.kafka.clients.producer.{RecordMetadata, ProducerRecord} +import org.apache.kafka.clients.producer.{KafkaProducer, RecordMetadata, ProducerRecord} +import org.apache.kafka.common.KafkaException import scala.collection.JavaConversions._ @@ -47,10 +48,14 @@ import java.util.concurrent._ * * If new producer is used, the offset will be committed based on the new producer's callback. An offset map is * maintained and updated on each send() callback. A separate offset commit thread will commit the offset periodically. - * @note For mirror maker, MaxInFlightRequests of producer should be set to 1 for producer if the order of the messages - * needs to be preserved. Mirror maker also depends on the in-order delivery to guarantee no data loss. - * We are not force it to be 1 because in some use cases throughput might be important whereas out of order or - * minor data loss is acceptable. + * @note For mirror maker, the following settings are required to make sure there is no data loss: + * 1. use new producer with following settings + * acks=all + * retries=max integer + * block.on.buffer.full=true + * 2. Consumer Settings + * auto.commit.enable=false + * If --no.data.loss flag is set in option, then those settings are automatically applied. */ object MirrorMaker extends Logging with KafkaMetricsGroup { @@ -58,23 +63,39 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { private var consumerThreads: Seq[ConsumerThread] = null private var producerThreads: Seq[ProducerThread] = null private val isShuttingdown: AtomicBoolean = new AtomicBoolean(false) - private var offsetCommitThread: OffsetCommitThread = null + private val scheduler: KafkaScheduler = new KafkaScheduler(threads = 1) - private val valueFactory = (k: TopicAndPartition) => new Pool[Int, Long] - private val topicPartitionOffsetMap: Pool[TopicAndPartition, Pool[Int, Long]] = - new Pool[TopicAndPartition, Pool[Int,Long]](Some(valueFactory)) + private val unackedOffsetsMap: Pool[TopicAndPartition, UnackedOffsets] = + new Pool[TopicAndPartition, UnackedOffsets](valueFactory = Some((k: TopicAndPartition) => new UnackedOffsets)) // Track the messages unacked for consumer rebalance - private var numMessageUnacked: AtomicInteger = new AtomicInteger(0) - private var consumerRebalanceListener: MirrorMakerConsumerRebalanceListener = null + private var numUnackedMessages: AtomicInteger = new AtomicInteger(0) + private var numSkippedUnackedMessages: AtomicInteger = new AtomicInteger(0) + private var consumerRebalanceListener: ConsumerRebalanceListener = null // This is to indicate whether the rebalance is going on so the producer callback knows if - // the rebalance latch needs to be pulled. - private var inRebalance: AtomicBoolean = new AtomicBoolean(false) + // the flag indicates internal consumer rebalance callback is waiting for all the messages sent to be acked. + private var waitingForMessageAcks: Boolean = false private val shutdownMessage : MirrorMakerRecord = new MirrorMakerRecord("shutdown", 0, 0, null, "shutdown".getBytes) - newGauge("MirrorMaker-Unacked-Messages", + newGauge("MirrorMaker-NumUnackedMessages", new Gauge[Int] { - def value = numMessageUnacked.get() + def value = numUnackedMessages.get() + }) + + // The number of unacked offsets in the unackedOffsetsMap + newGauge("MirrorMaker-UnackedOffsetListsSize", + new Gauge[Int] { + def value = unackedOffsetsMap.iterator.map{ + case(_, unackedOffsets) => unackedOffsets.size + }.sum + }) + + // If a message send failed after retries are exhausted. The offset of the messages will also be removed from + // the unacked offset list to avoid offset commit being stuck on that offset. In this case, the offset of that + // message was not really acked, but was skipped. This metric records the number of skipped offsets. + newGauge("MirrorMaker-NumSkippedOffsets", + new Gauge[Int] { + def value = numSkippedUnackedMessages.get() }) def main(args: Array[String]) { @@ -98,6 +119,9 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { val useNewProducerOpt = parser.accepts("new.producer", "Use the new producer implementation.") + val noDataLossOpt = parser.accepts("no.data.loss", + "Configure the mirror maker to have no data loss.") + val numProducersOpt = parser.accepts("num.producers", "Number of producer instances") .withRequiredArg() @@ -145,6 +169,12 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { .ofType(classOf[java.lang.Integer]) .defaultsTo(60000) + val consumerRebalanceListenerOpt = parser.accepts("consumer.rebalance.listener", + "The consumer rebalance listener to use for mirror maker consumer.") + .withRequiredArg() + .describedAs("A custom rebalance listener of type ConsumerRebalanceListener") + .ofType(classOf[String]) + val helpOpt = parser.accepts("help", "Print this message.") if(args.length == 0) @@ -170,7 +200,11 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { val offsetCommitIntervalMs = options.valueOf(offsetCommitIntervalMsOpt).intValue() // create consumer connector - val consumerConfigProps = Utils.loadProps(options.valuesOf(consumerConfigOpt).head) + val consumerConfigProps = Utils.loadProps(options.valueOf(consumerConfigOpt)) + val noDataLoss = options.has(noDataLossOpt) + // disable consumer auto commit because offset will be committed by offset commit thread. + if (noDataLoss) + consumerConfigProps.setProperty("auto.commit.enable","false") val consumerConfig = new ConsumerConfig(consumerConfigProps) connector = new ZookeeperConsumerConnector(consumerConfig) @@ -178,20 +212,30 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { val mirrorDataChannel = new DataChannel(bufferSize, bufferByteSize, numInputs = numStreams, numOutputs = numProducers) // set consumer rebalance listener - // Customized consumer rebalance listener should extend MirrorMakerConsumerRebalanceListener - // and take datachannel as argument. - val customRebalanceListenerClass = consumerConfigProps.getProperty("consumer.rebalance.listener") - consumerRebalanceListener = { - if (customRebalanceListenerClass == null) { - new MirrorMakerConsumerRebalanceListener(mirrorDataChannel) - } else - Utils.createObject[MirrorMakerConsumerRebalanceListener](customRebalanceListenerClass, mirrorDataChannel) + // custom rebalance listener will be invoked after internal listener finishes its work. + val customRebalanceListenerClass = options.valueOf(consumerRebalanceListenerOpt) + val customRebalanceListener = { + if (customRebalanceListenerClass != null) + Utils.createObject[ConsumerRebalanceListener](customRebalanceListenerClass) + else + null } + consumerRebalanceListener = new InternalRebalanceListener(mirrorDataChannel, Some(customRebalanceListener)) connector.setConsumerRebalanceListener(consumerRebalanceListener) // create producer threads - val useNewProducer = options.has(useNewProducerOpt) val producerProps = Utils.loadProps(options.valueOf(producerConfigOpt)) + val useNewProducer = { + // Override producer settings if no.data.loss is set + if (noDataLoss) { + producerProps.setProperty("retries",Int.MaxValue.toString) + producerProps.setProperty("block.on.buffer.full", "true") + producerProps.setProperty("acks","all") + true + } else { + options.has(useNewProducerOpt) + } + } val clientId = producerProps.getProperty("client.id", "") producerThreads = (0 until numProducers).map(i => { producerProps.setProperty("client.id", clientId + "-" + i) @@ -203,17 +247,17 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { new ProducerThread(mirrorDataChannel, producer, i) }) - // create offset commit thread - if (useNewProducer) { + // start offset commit thread + if (noDataLoss) { /** - * The offset commit thread periodically commit consumed offsets to the source cluster. With the new producer, + * The offset commit thread periodically commit consumed offsets. With the new producer, * the offsets are updated upon the returned future metadata of the send() call; with the old producer, * the offsets are updated upon the consumer's iterator advances. By doing this, it is guaranteed no data * loss even when mirror maker is uncleanly shutdown with the new producer, while with the old producer * messages inside the data channel could be lost upon mirror maker unclean shutdown. */ - offsetCommitThread = new OffsetCommitThread(offsetCommitIntervalMs) - offsetCommitThread.start() + scheduler.startup() + scheduler.schedule("offset-commit", commitOffsets, 0, offsetCommitIntervalMs, TimeUnit.MILLISECONDS) } // create consumer threads @@ -252,7 +296,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { def cleanShutdown() { if (isShuttingdown.compareAndSet(false, true)) { info("Start clean shutdown.") - // Consumer threads will exit when isCleanShutdown is set. + // Shutdown consumer threads. info("Shutting down consumer threads.") if (consumerThreads != null) { consumerThreads.foreach(_.shutdown()) @@ -265,12 +309,10 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { producerThreads.foreach(_.awaitShutdown()) } // offset commit thread should only be shutdown after producer threads are shutdown, so we don't lose offsets. - info("Shutting down offset commit thread.") - if (offsetCommitThread != null) { - offsetCommitThread.shutdown() - offsetCommitThread.awaitShutdown() - } - // connector can only be shutdown after offsets are committed. + scheduler.shutdown() + swallow(commitOffsets()) + + // connector should only be shutdown after offsets are committed. info("Shutting down consumer connectors.") if (connector != null) connector.shutdown() @@ -457,155 +499,134 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { } } - class OffsetCommitThread(commitIntervalMs: Int) extends Thread with Logging with KafkaMetricsGroup { - private val threadName = "mirrormaker-offset-commit-thread" - private val shutdownComplete: CountDownLatch = new CountDownLatch(1) - this.logIdent = "[%s]".format(threadName) - var shutdownFlag: Boolean = false - var commitCounter: Int = 0 - - this.setName(threadName) - - newGauge("MirrorMaker-Offset-Commit-Counter", - new Gauge[Int] { - def value = commitCounter - }) - - /** - * Use the connector to commit all the offsets. - */ - override def run() { - info("Starting mirror maker offset commit thread") - try { - while (!shutdownFlag) { - Thread.sleep(commitIntervalMs) - commitOffset() - } - } catch { - case t: Throwable => fatal("Exits due to", t) - } finally { - swallow(commitOffset()) - shutdownComplete.countDown() - info("Offset commit thread exited") - if (!isShuttingdown.get()) { - fatal("Offset commit thread exited abnormally, stopping the whole mirror maker.") - System.exit(-1) - } - } - } - - def commitOffset() { - val offsetsToCommit = collection.immutable.Map(topicPartitionOffsetMap.map { - case (topicPartition, partitionOffsetMap) => - topicPartition -> OffsetAndMetadata(getOffsetToCommit(partitionOffsetMap), null) + private def commitOffsets() { + try { + info("Committing offsets") + val offsetsToCommit = collection.immutable.Map(unackedOffsetsMap.map { + case (topicPartition, unackedOffsets) => + topicPartition -> OffsetAndMetadata(unackedOffsets.getOffsetToCommit, null) }.toSeq: _*) - trace("committing offset: %s".format(offsetsToCommit)) if (connector == null) { warn("No consumer connector available to commit offset.") } else { - connector.commitOffsets( - isAutoCommit = false, - topicPartitionOffsets = offsetsToCommit - ) - commitCounter += 1 - } - } - - private def getOffsetToCommit(offsetsMap: Pool[Int, Long]): Long = { - val offsets = offsetsMap.map(_._2).toSeq.sorted - val iter = offsets.iterator - var offsetToCommit = iter.next() - while (iter.hasNext && offsetToCommit + 1 == iter.next()) - offsetToCommit += 1 - // The committed offset will be the first offset of un-consumed message, hence we need to increment by one. - offsetToCommit + 1 - } - - def shutdown() { - shutdownFlag = true - } - - def awaitShutdown() { - try { - shutdownComplete.await() - info("Offset commit thread shutdown complete") - } catch { - case ie: InterruptedException => { - warn("Shutdown of the offset commit thread interrupted") - } + connector.commitOffsets(offsetsToCommit, isAutoCommit = false) } + } catch { + case e: OutOfMemoryError => + fatal("Shutting down mirror maker due to error when committing offsets.", e) + System.exit(-1) + case t: Throwable => + warn("Offsets commit failed due to ", t) } } private[kafka] trait MirrorMakerBaseProducer { - def send(topicPartition: TopicAndPartition, offset: Long, key: Array[Byte], value: Array[Byte]) + def send(sourceTopicPartition: TopicAndPartition, sourceOffset: Long, key: Array[Byte], value: Array[Byte]) def close() } - private class MirrorMakerNewProducer (val producerProps: Properties) - extends NewShinyProducer(producerProps) with MirrorMakerBaseProducer { + private class MirrorMakerNewProducer (val producerProps: Properties) extends MirrorMakerBaseProducer { - override def send(topicPartition: TopicAndPartition, offset: Long, key: Array[Byte], value: Array[Byte]) { - val record = new ProducerRecord[Array[Byte],Array[Byte]](topicPartition.topic, key, value) + val sync = producerProps.getProperty("producer.type", "async").equals("sync") + + val producer = new KafkaProducer[Array[Byte], Array[Byte]](producerProps) + + override def send(sourceTopicPartition: TopicAndPartition, sourceOffset: Long, key: Array[Byte], value: Array[Byte]) { + val record = new ProducerRecord(sourceTopicPartition.topic, key, value) if(sync) { - topicPartitionOffsetMap.getAndMaybePut(topicPartition).put(this.producer.send(record).get().partition(), offset) + this.producer.send(record).get() + unackedOffsetsMap.getAndMaybePut(sourceTopicPartition).maybeUpdateMaxOffsetSeen(sourceOffset) } else { - this.producer.send(record, - new MirrorMakerProducerCallback(topicPartition, offset, key, value)) - numMessageUnacked.incrementAndGet() + + val unackedOffsets = unackedOffsetsMap.getAndMaybePut(sourceTopicPartition) + // synchronize to ensure that addOffset precedes removeOffset + unackedOffsets synchronized { + val unackedOffset = new UnackedOffset(sourceOffset) + this.producer.send(record, + new MirrorMakerProducerCallback(sourceTopicPartition, unackedOffset, key, value)) + // add offset to unackedOffsets + unackedOffsets.addOffset(unackedOffset) + numUnackedMessages.incrementAndGet() + } } } + + override def close() { + this.producer.close() + } } - private class MirrorMakerOldProducer (val producerProps: Properties) - extends OldProducer(producerProps) with MirrorMakerBaseProducer { + private class MirrorMakerOldProducer (val producerProps: Properties) extends MirrorMakerBaseProducer { + + // default to byte array partitioner + if (producerProps.getProperty("partitioner.class") == null) + producerProps.setProperty("partitioner.class", classOf[kafka.producer.ByteArrayPartitioner].getName) + val producer = new kafka.producer.Producer[Array[Byte], Array[Byte]](new ProducerConfig(producerProps)) override def send(topicPartition: TopicAndPartition, offset: Long, key: Array[Byte], value: Array[Byte]) { - super.send(topicPartition.topic, key, value) + this.producer.send(new KeyedMessage[Array[Byte], Array[Byte]](topicPartition.topic, key, value)) } override def close() { - super.close() + this.producer.close() } } private class MirrorMakerProducerCallback (val topicPartition: TopicAndPartition, - val offset: Long, + val offset: UnackedOffset, val key: Array[Byte], val value: Array[Byte]) extends ErrorLoggingCallback(topicPartition.topic, key, value, false) { override def onCompletion(metadata: RecordMetadata, exception: Exception) { if (exception != null) { - // Use default call back to log error + // Use default call back to log error. This means the max retries of producer has reached and message + // still could not be sent. In this case we have to remove the offsets from list to let the mirror maker + // move on. The message failed to be sent will be lost in target cluster. + warn("Not be able to send message, offset of "+ topicPartition + " will not advance. Total number" + + "of skipped unacked messages is" + numSkippedUnackedMessages.incrementAndGet()) super.onCompletion(metadata, exception) } else { - trace("updating offset:[%s] -> %d".format(topicPartition, offset)) - topicPartitionOffsetMap.getAndMaybePut(topicPartition).put(metadata.partition(), offset) + trace("Updating offset for %s to %d".format(topicPartition, offset)) } + // remove the offset from the unackedOffsets + val unackedOffsets = unackedOffsetsMap.get(topicPartition) + unackedOffsets.removeOffset(offset) // Notify the rebalance callback only when all the messages handed to producer are acked. - // There is a very slight chance that 1 message is held by producer thread and not handed to producer. + // There is a very slight chance that one message is held by producer thread and not handed to producer. // That message might have duplicate. We are not handling that here. - if (numMessageUnacked.decrementAndGet() == 0 && inRebalance.get()) { - inRebalance synchronized {inRebalance.notify()} + numUnackedMessages synchronized { + if (numUnackedMessages.decrementAndGet() == 0 && waitingForMessageAcks) { + numUnackedMessages.notify() + } } } } - class MirrorMakerConsumerRebalanceListener (dataChannel: DataChannel) extends ConsumerRebalanceListener { + class InternalRebalanceListener (dataChannel: DataChannel, customRebalanceListener: Option[ConsumerRebalanceListener]) + extends ConsumerRebalanceListener { override def beforeReleasingPartitions(partitionOwnership: java.util.Map[String, java.util.Set[java.lang.Integer]]) { info("Clearing data channel.") dataChannel.clear() info("Waiting until all the messages are acked.") - inRebalance synchronized { - inRebalance.set(true) - while (numMessageUnacked.get() > 0) - inRebalance.wait() + numUnackedMessages synchronized { + waitingForMessageAcks = true + while (numUnackedMessages.get() > 0) { + try { + numUnackedMessages.wait() + } catch { + case e: InterruptedException => info("Ignoring interrupt while waiting.") + } + } + waitingForMessageAcks = false } info("Committing offsets.") - offsetCommitThread.commitOffset() - inRebalance.set(true) + commitOffsets() + + // invoke custom consumer rebalance listener + if (customRebalanceListener.isDefined) + customRebalanceListener.get.beforeReleasingPartitions(partitionOwnership) } } @@ -617,5 +638,43 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { def size = value.length + {if (key == null) 0 else key.length} } -} + private class UnackedOffset(offset: Long) extends DoublyLinkedListNode[Long](offset) { + + } + + private class UnackedOffsets { + val offsetList = new DoublyLinkedList[Long] + var maxOffsetSeen: Long = -1L + + def maybeUpdateMaxOffsetSeen(offset: Long) { + this synchronized { + maxOffsetSeen = math.max(maxOffsetSeen, offset) + } + } + def addOffset(offset: DoublyLinkedListNode[Long]) { + this synchronized { + offsetList.add(offset) + maybeUpdateMaxOffsetSeen(offset.element) + } + } + + def removeOffset(offset: DoublyLinkedListNode[Long]) { + offsetList.remove(offset) + } + + def getOffsetToCommit: Long = { + this synchronized { + val smallestUnackedOffset = offsetList.peek() + if (smallestUnackedOffset == null) + // list is empty, commit maxOffsetSeen + 1 + maxOffsetSeen + 1 + else + // commit the smallest unacked offset + smallestUnackedOffset.element + } + } + + def size: Int = offsetList.size + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/utils/DoublyLinkedList.scala b/core/src/main/scala/kafka/utils/DoublyLinkedList.scala new file mode 100644 index 0000000000000..e637ef31d0945 --- /dev/null +++ b/core/src/main/scala/kafka/utils/DoublyLinkedList.scala @@ -0,0 +1,126 @@ +/** + * 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.utils + +/** + * Simple doubly LinkedList node + * @param element The element + * @tparam T The type of element + */ +class DoublyLinkedListNode[T] (val element: T) { + var prev: DoublyLinkedListNode[T] = null + var next: DoublyLinkedListNode[T] = null +} + +/** + * A simple doubly linked list util to allow O(1) remove. + * @tparam T type of element in nodes + */ +@threadsafe +class DoublyLinkedList[T] { + private var head: DoublyLinkedListNode[T] = null + private var tail: DoublyLinkedListNode[T] = null + @volatile private var listSize: Int = 0 + + /** + * Add offset to the tail of the list + * @param node the node to be added to the tail of the list + */ + def add (node: DoublyLinkedListNode[T]) { + this synchronized { + if (head == null) { + // empty list + head = node + tail = node + node.prev = null + node.next = null + } else { + // add to tail + tail.next = node + node.next = null + node.prev = tail + tail = node + } + listSize += 1 + } + } + + /** + * Remove a node from the list. The list will not check if the node is really in the list. + * @param node the node to be removed from the list + */ + def remove (node: DoublyLinkedListNode[T]) { + this synchronized { + if (node ne head) + node.prev.next = node.next + else + head = node.next + + if (node ne tail) + node.next.prev = node.prev + else + tail = node.prev + + node.prev = null + node.next = null + + listSize -= 1 + } + } + + /** + * Remove the first node in the list and return it if the list is not empty. + * @return The first node in the list if the list is not empty. Return Null if the list is empty. + */ + def remove(): DoublyLinkedListNode[T] = { + this synchronized { + if (head != null) { + val node = head + remove(head) + node + } else { + null + } + } + } + + /** + * Get the first node in the list without removing it. + * @return the first node in the list. + */ + def peek(): DoublyLinkedListNode[T] = head + + def size: Int = listSize + + def iterator: Iterator[DoublyLinkedListNode[T]] = { + new IteratorTemplate[DoublyLinkedListNode[T]] { + var current = head + override protected def makeNext(): DoublyLinkedListNode[T] = { + this synchronized { + if (current != null) { + val nextNode = current + current = current.next + nextNode + } else { + allDone() + } + } + } + } + } +} diff --git a/core/src/test/scala/unit/kafka/utils/UtilsTest.scala b/core/src/test/scala/unit/kafka/utils/UtilsTest.scala index 066553cad290c..8c3797a964a27 100644 --- a/core/src/test/scala/unit/kafka/utils/UtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/UtilsTest.scala @@ -150,4 +150,54 @@ class UtilsTest extends JUnitSuite { assertEquals(2, result) assertFalse("Should be unlocked", lock.isLocked) } + + @Test + def testDoublyLinkedList() { + val list = new DoublyLinkedList[Int] + + // test remove from a single-entry list. + list.add(new DoublyLinkedListNode[Int](0)) + list.remove() + assert(list.size == 0) + assert(list.peek() == null) + + // test add + for (i <- 0 to 2) { + list.add(new DoublyLinkedListNode[Int](i)) + } + val toBeRemoved1 = new DoublyLinkedListNode[Int](3) + list.add(toBeRemoved1) + for (i <- 4 to 6) { + list.add(new DoublyLinkedListNode[Int](i)) + } + val toBeRemoved2 = new DoublyLinkedListNode[Int](7) + list.add(toBeRemoved2) + + // test iterator + val iter = list.iterator + for (i <- 0 to 7) { + assert(iter.hasNext) + assert(iter.next().element == i) + } + assert(!iter.hasNext) + + // remove from head + list.remove() + assert(list.peek().element == 1) + // remove from middle + list.remove(toBeRemoved1) + // remove from tail + list.remove(toBeRemoved2) + + // List = [1,2,4,5,6] + val iter2 = list.iterator + for (i <- Array[Int](1,2,4,5,6)) { + assert(iter2.hasNext) + assert(iter2.next().element == i) + } + + // test size + assert(list.size == 5) + } + } From 4471dc08b64809f3cdcbff6b8c830e13df87b117 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Tue, 6 Jan 2015 10:56:32 -0800 Subject: [PATCH 078/491] kafka-1642; (followup patch) [Java New Producer Kafka Trunk] CPU Usage Spike to 100% when network connection is lost; patched by Ewen Cheslack-Postava; patched by Ewen Cheslack-Postava; reviewed by Jun Rao --- .../org/apache/kafka/clients/NetworkClient.java | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 525b95e98010c..6746275d0b259 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -180,9 +180,10 @@ public List poll(List requests, long timeout, lon // should we update our metadata? long timeToNextMetadataUpdate = metadata.timeToNextUpdate(now); - long timeToNextReconnectAttempt = this.lastNoNodeAvailableMs + metadata.refreshBackoff() - now; + long timeToNextReconnectAttempt = Math.max(this.lastNoNodeAvailableMs + metadata.refreshBackoff() - now, 0); + long waitForMetadataFetch = (this.metadataFetchInProgress ? Integer.MAX_VALUE : 0); // if there is no node available to connect, back off refreshing metadata - long metadataTimeout = Math.max(timeToNextMetadataUpdate, timeToNextReconnectAttempt); + long metadataTimeout = Math.max(Math.max(timeToNextMetadataUpdate, timeToNextReconnectAttempt), waitForMetadataFetch); if (!this.metadataFetchInProgress && metadataTimeout == 0) maybeUpdateMetadata(sends, now); @@ -371,6 +372,8 @@ private ClientRequest metadataRequest(long now, int node, Set topics) { * Add a metadata request to the list of sends if we can make one */ private void maybeUpdateMetadata(List sends, long now) { + // Beware that the behavior of this method and the computation of timeouts for poll() are + // highly dependent on the behavior of leastLoadedNode. Node node = this.leastLoadedNode(now); if (node == null) { log.debug("Give up sending metadata request since no node is available"); @@ -391,6 +394,14 @@ private void maybeUpdateMetadata(List sends, long now) { // we don't have a connection to this node right now, make one log.debug("Init connection to node {} for sending metadata request in the next iteration", node.id()); initiateConnect(node, now); + // If initiateConnect failed immediately, this node will be put into blackout and we + // should allow immediately retrying in case there is another candidate node. If it + // is still connecting, the worst case is that we end up setting a longer timeout + // on the next round and then wait for the response. + } else { // connected, but can't send more OR connecting + // In either case, we just need to wait for a network event to let us know the selected + // connection might be usable again. + this.lastNoNodeAvailableMs = now; } } @@ -400,8 +411,8 @@ private void maybeUpdateMetadata(List sends, long now) { private void initiateConnect(Node node, long now) { try { log.debug("Initiating connection to node {} at {}:{}.", node.id(), node.host(), node.port()); - selector.connect(node.id(), new InetSocketAddress(node.host(), node.port()), this.socketSendBuffer, this.socketReceiveBuffer); this.connectionStates.connecting(node.id(), now); + selector.connect(node.id(), new InetSocketAddress(node.host(), node.port()), this.socketSendBuffer, this.socketReceiveBuffer); } catch (IOException e) { /* attempt failed, we'll try again after the backoff */ connectionStates.disconnected(node.id()); From 50b734690a93e58030f69cede8d0a84d1e3f5461 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Tue, 6 Jan 2015 11:07:46 -0800 Subject: [PATCH 079/491] kafka-1797; (follow-up patch) add the serializer/deserializer api to the new java client; patched by Jun Rao; reviewed by Jay Kreps --- build.gradle | 1 + .../clients/consumer/ConsumerConfig.java | 4 +- .../kafka/clients/consumer/KafkaConsumer.java | 37 ++++++++-- .../kafka/clients/producer/KafkaProducer.java | 59 +++++++++++++--- .../clients/producer/ProducerConfig.java | 4 +- .../kafka/common/config/AbstractConfig.java | 13 ++-- .../errors/DeserializationException.java | 47 ------------- .../serialization}/ByteArrayDeserializer.java | 6 +- .../serialization}/ByteArraySerializer.java | 6 +- .../serialization}/Deserializer.java | 21 ++++-- .../serialization}/Serializer.java | 24 ++++--- .../serialization/StringDeserializer.java | 50 ++++++++++++++ .../serialization/StringSerializer.java | 50 ++++++++++++++ .../serialization/SerializationTest.java | 68 +++++++++++++++++++ .../kafka/producer/KafkaLog4jAppender.scala | 2 + .../scala/kafka/tools/ConsoleProducer.scala | 2 + .../main/scala/kafka/tools/MirrorMaker.scala | 10 +-- .../kafka/tools/ProducerPerformance.scala | 2 + .../scala/kafka/tools/ReplayLogProducer.scala | 2 + .../kafka/tools/TestEndToEndLatency.scala | 2 + .../scala/kafka/tools/TestLogCleaning.scala | 2 + .../kafka/api/ProducerCompressionTest.scala | 2 + .../kafka/api/ProducerSendTest.scala | 53 +++++++++++++++ .../scala/unit/kafka/utils/TestUtils.scala | 2 + 24 files changed, 370 insertions(+), 99 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/common/errors/DeserializationException.java rename clients/src/main/java/org/apache/kafka/{clients/consumer => common/serialization}/ByteArrayDeserializer.java (85%) rename clients/src/main/java/org/apache/kafka/{clients/producer => common/serialization}/ByteArraySerializer.java (85%) rename clients/src/main/java/org/apache/kafka/{clients/consumer => common/serialization}/Deserializer.java (70%) rename clients/src/main/java/org/apache/kafka/{clients/producer => common/serialization}/Serializer.java (68%) create mode 100644 clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java create mode 100644 clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java create mode 100644 clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java diff --git a/build.gradle b/build.gradle index 18f86e4c8a106..ba52288031e2a 100644 --- a/build.gradle +++ b/build.gradle @@ -371,6 +371,7 @@ project(':clients') { javadoc { include "**/org/apache/kafka/clients/producer/*" include "**/org/apache/kafka/common/errors/*" + include "**/org/apache/kafka/common/serialization/*" } } 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 1d64f08762b0c..57c1807ccba9f 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 @@ -185,8 +185,8 @@ public class ConsumerConfig extends AbstractConfig { METRICS_SAMPLE_WINDOW_MS_DOC) .define(METRICS_NUM_SAMPLES_CONFIG, Type.INT, 2, atLeast(1), Importance.LOW, METRICS_NUM_SAMPLES_DOC) .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, "", Importance.LOW, METRIC_REPORTER_CLASSES_DOC) - .define(KEY_DESERIALIZER_CLASS_CONFIG, Type.CLASS, "org.apache.kafka.clients.consumer.ByteArrayDeserializer", Importance.HIGH, KEY_DESERIALIZER_CLASS_DOC) - .define(VALUE_DESERIALIZER_CLASS_CONFIG, Type.CLASS, "org.apache.kafka.clients.consumer.ByteArrayDeserializer", Importance.HIGH, VALUE_DESERIALIZER_CLASS_DOC); + .define(KEY_DESERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, KEY_DESERIALIZER_CLASS_DOC) + .define(VALUE_DESERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, VALUE_DESERIALIZER_CLASS_DOC); } 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 a43b1600c707d..a5fedce9ff05c 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 @@ -18,6 +18,7 @@ import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.MetricsReporter; +import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.utils.ClientUtils; import org.apache.kafka.common.utils.SystemTime; import org.slf4j.Logger; @@ -345,7 +346,7 @@ public class KafkaConsumer implements Consumer { * @param configs The consumer configs */ public KafkaConsumer(Map configs) { - this(new ConsumerConfig(configs), null, null, null); + this(configs, null); } /** @@ -358,7 +359,7 @@ public KafkaConsumer(Map configs) { * every rebalance operation. */ public KafkaConsumer(Map configs, ConsumerRebalanceCallback callback) { - this(new ConsumerConfig(configs), callback, null, null); + this(configs, callback, null, null); } /** @@ -375,7 +376,19 @@ public KafkaConsumer(Map configs, ConsumerRebalanceCallback call * won't be called in the consumer when the deserializer is passed in directly. */ public KafkaConsumer(Map configs, ConsumerRebalanceCallback callback, Deserializer keyDeserializer, Deserializer valueDeserializer) { - this(new ConsumerConfig(configs), callback, keyDeserializer, valueDeserializer); + this(new ConsumerConfig(addDeserializerToConfig(configs, keyDeserializer, valueDeserializer)), + callback, keyDeserializer, valueDeserializer); + } + + private static Map addDeserializerToConfig(Map configs, + Deserializer keyDeserializer, Deserializer valueDeserializer) { + Map newConfigs = new HashMap(); + newConfigs.putAll(configs); + if (keyDeserializer != null) + newConfigs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getClass()); + if (keyDeserializer != null) + newConfigs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass()); + return newConfigs; } /** @@ -383,7 +396,7 @@ public KafkaConsumer(Map configs, ConsumerRebalanceCallback call * Valid configuration strings are documented at {@link ConsumerConfig} */ public KafkaConsumer(Properties properties) { - this(new ConsumerConfig(properties), null, null, null); + this(properties, null); } /** @@ -396,7 +409,7 @@ public KafkaConsumer(Properties properties) { * every rebalance operation. */ public KafkaConsumer(Properties properties, ConsumerRebalanceCallback callback) { - this(new ConsumerConfig(properties), callback, null, null); + this(properties, callback, null, null); } /** @@ -413,7 +426,19 @@ public KafkaConsumer(Properties properties, ConsumerRebalanceCallback callback) * won't be called in the consumer when the deserializer is passed in directly. */ public KafkaConsumer(Properties properties, ConsumerRebalanceCallback callback, Deserializer keyDeserializer, Deserializer valueDeserializer) { - this(new ConsumerConfig(properties), callback, keyDeserializer, valueDeserializer); + this(new ConsumerConfig(addDeserializerToConfig(properties, keyDeserializer, valueDeserializer)), + callback, keyDeserializer, valueDeserializer); + } + + private static Properties addDeserializerToConfig(Properties properties, + Deserializer keyDeserializer, Deserializer valueDeserializer) { + Properties newProperties = new Properties(); + newProperties.putAll(properties); + if (keyDeserializer != null) + newProperties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getClass().getName()); + if (keyDeserializer != null) + newProperties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass().getName()); + return newProperties; } private KafkaConsumer(ConsumerConfig config, ConsumerRebalanceCallback callback, Deserializer keyDeserializer, Deserializer valueDeserializer) { diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index f61efb35db7e0..91c672d3a78e3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -13,10 +13,7 @@ package org.apache.kafka.clients.producer; import java.net.InetSocketAddress; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Properties; +import java.util.*; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -34,6 +31,7 @@ import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.errors.ApiException; 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.metrics.JmxReporter; import org.apache.kafka.common.metrics.MetricConfig; @@ -44,6 +42,7 @@ import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.Records; +import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.ClientUtils; import org.apache.kafka.common.utils.KafkaThread; import org.apache.kafka.common.utils.SystemTime; @@ -102,7 +101,19 @@ public KafkaProducer(Map configs) { * be called in the producer when the serializer is passed in directly. */ public KafkaProducer(Map configs, Serializer keySerializer, Serializer valueSerializer) { - this(new ProducerConfig(configs), keySerializer, valueSerializer); + this(new ProducerConfig(addSerializerToConfig(configs, keySerializer, valueSerializer)), + keySerializer, valueSerializer); + } + + private static Map addSerializerToConfig(Map configs, + Serializer keySerializer, Serializer valueSerializer) { + Map newConfigs = new HashMap(); + newConfigs.putAll(configs); + if (keySerializer != null) + newConfigs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializer.getClass()); + if (valueSerializer != null) + newConfigs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializer.getClass()); + return newConfigs; } /** @@ -124,7 +135,19 @@ public KafkaProducer(Properties properties) { * be called in the producer when the serializer is passed in directly. */ public KafkaProducer(Properties properties, Serializer keySerializer, Serializer valueSerializer) { - this(new ProducerConfig(properties), keySerializer, valueSerializer); + this(new ProducerConfig(addSerializerToConfig(properties, keySerializer, valueSerializer)), + keySerializer, valueSerializer); + } + + private static Properties addSerializerToConfig(Properties properties, + Serializer keySerializer, Serializer valueSerializer) { + Properties newProperties = new Properties(); + newProperties.putAll(properties); + if (keySerializer != null) + newProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializer.getClass().getName()); + if (valueSerializer != null) + newProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializer.getClass().getName()); + return newProperties; } private KafkaProducer(ProducerConfig config, Serializer keySerializer, Serializer valueSerializer) { @@ -178,14 +201,18 @@ private KafkaProducer(ProducerConfig config, Serializer keySerializer, Serial this.errors = this.metrics.sensor("errors"); - if (keySerializer == null) + if (keySerializer == null) { this.keySerializer = config.getConfiguredInstance(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, Serializer.class); + this.keySerializer.configure(config.originals(), true); + } else this.keySerializer = keySerializer; - if (valueSerializer == null) + if (valueSerializer == null) { this.valueSerializer = config.getConfiguredInstance(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, Serializer.class); + this.valueSerializer.configure(config.originals(), false); + } else this.valueSerializer = valueSerializer; @@ -275,8 +302,20 @@ public Future send(ProducerRecord record, Callback callback try { // first make sure the metadata for the topic is available waitOnMetadata(record.topic(), this.metadataFetchTimeoutMs); - byte[] serializedKey = keySerializer.serialize(record.topic(), record.key(), true); - byte[] serializedValue = valueSerializer.serialize(record.topic(), record.value(), false); + byte[] serializedKey; + try { + serializedKey = keySerializer.serialize(record.topic(), record.key()); + } catch (ClassCastException cce) { + throw new SerializationException("Can't convert key of class " + record.key().getClass().getName() + + " to the one specified in key.serializer"); + } + byte[] serializedValue; + try { + serializedValue = valueSerializer.serialize(record.topic(), record.value()); + } catch (ClassCastException cce) { + throw new SerializationException("Can't convert value of class " + record.value().getClass().getName() + + " to the one specified in value.serializer"); + } ProducerRecord serializedRecord = new ProducerRecord(record.topic(), record.partition(), serializedKey, serializedValue); int partition = partitioner.partition(serializedRecord, metadata.fetch()); int serializedSize = Records.LOG_OVERHEAD + Record.recordSize(serializedKey, serializedValue); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index a893d88c2f4e2..8b3e565edd1ae 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -229,8 +229,8 @@ public class ProducerConfig extends AbstractConfig { atLeast(1), Importance.LOW, MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_DOC) - .define(KEY_SERIALIZER_CLASS_CONFIG, Type.CLASS, "org.apache.kafka.clients.producer.ByteArraySerializer", Importance.HIGH, KEY_SERIALIZER_CLASS_DOC) - .define(VALUE_SERIALIZER_CLASS_CONFIG, Type.CLASS, "org.apache.kafka.clients.producer.ByteArraySerializer", Importance.HIGH, VALUE_SERIALIZER_CLASS_DOC); + .define(KEY_SERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, KEY_SERIALIZER_CLASS_DOC) + .define(VALUE_SERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, VALUE_SERIALIZER_CLASS_DOC); } ProducerConfig(Map props) { diff --git a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java index 3d4ab7228926f..c4fa058692f50 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java @@ -12,12 +12,7 @@ */ package org.apache.kafka.common.config; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; +import java.util.*; import org.apache.kafka.common.Configurable; import org.apache.kafka.common.KafkaException; @@ -97,6 +92,12 @@ public Set unused() { return keys; } + public Map originals() { + Map copy = new HashMap(); + copy.putAll(originals); + return copy; + } + private void logAll() { StringBuilder b = new StringBuilder(); b.append(getClass().getSimpleName()); diff --git a/clients/src/main/java/org/apache/kafka/common/errors/DeserializationException.java b/clients/src/main/java/org/apache/kafka/common/errors/DeserializationException.java deleted file mode 100644 index a5433398fb978..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/errors/DeserializationException.java +++ /dev/null @@ -1,47 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ - -package org.apache.kafka.common.errors; - -import org.apache.kafka.common.KafkaException; - -/** - * Any exception during deserialization in the consumer - */ -public class DeserializationException extends KafkaException { - - private static final long serialVersionUID = 1L; - - public DeserializationException(String message, Throwable cause) { - super(message, cause); - } - - public DeserializationException(String message) { - super(message); - } - - public DeserializationException(Throwable cause) { - super(cause); - } - - public DeserializationException() { - super(); - } - - /* avoid the expensive and useless stack trace for deserialization exceptions */ - @Override - public Throwable fillInStackTrace() { - return this; - } - -} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ByteArrayDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ByteArrayDeserializer.java similarity index 85% rename from clients/src/main/java/org/apache/kafka/clients/consumer/ByteArrayDeserializer.java rename to clients/src/main/java/org/apache/kafka/common/serialization/ByteArrayDeserializer.java index 514cbd2c27a8d..d89b3ff0509eb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ByteArrayDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ByteArrayDeserializer.java @@ -11,19 +11,19 @@ * specific language governing permissions and limitations under the License. */ -package org.apache.kafka.clients.consumer; +package org.apache.kafka.common.serialization; import java.util.Map; public class ByteArrayDeserializer implements Deserializer { @Override - public void configure(Map configs) { + public void configure(Map configs, boolean isKey) { // nothing to do } @Override - public byte[] deserialize(String topic, byte[] data, boolean isKey) { + public byte[] deserialize(String topic, byte[] data) { return data; } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ByteArraySerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ByteArraySerializer.java similarity index 85% rename from clients/src/main/java/org/apache/kafka/clients/producer/ByteArraySerializer.java rename to clients/src/main/java/org/apache/kafka/common/serialization/ByteArraySerializer.java index 9005b74a328c9..beaef948e38f2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ByteArraySerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ByteArraySerializer.java @@ -11,19 +11,19 @@ * specific language governing permissions and limitations under the License. */ -package org.apache.kafka.clients.producer; +package org.apache.kafka.common.serialization; import java.util.Map; public class ByteArraySerializer implements Serializer { @Override - public void configure(Map configs) { + public void configure(Map configs, boolean isKey) { // nothing to do } @Override - public byte[] serialize(String topic, byte[] data, boolean isKey) { + public byte[] serialize(String topic, byte[] data) { return data; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Deserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java similarity index 70% rename from clients/src/main/java/org/apache/kafka/clients/consumer/Deserializer.java rename to clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java index fa857d4debbc9..3c001d33091c0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/Deserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java @@ -11,9 +11,9 @@ * specific language governing permissions and limitations under the License. */ -package org.apache.kafka.clients.consumer; +package org.apache.kafka.common.serialization; -import org.apache.kafka.common.Configurable; +import java.util.Map; /** * @@ -21,15 +21,22 @@ * * A class that implements this interface is expected to have a constructor with no parameter. */ -public interface Deserializer extends Configurable { +public interface Deserializer { + + /** + * Configure this class. + * @param configs configs in key/value pairs + * @param isKey whether is for key or value + */ + public void configure(Map configs, boolean isKey); + /** * - * @param topic Topic associated with the data - * @param data Serialized bytes - * @param isKey Is data for key or value + * @param topic topic associated with the data + * @param data serialized bytes * @return deserialized typed data */ - public T deserialize(String topic, byte[] data, boolean isKey); + public T deserialize(String topic, byte[] data); /** * Close this deserializer diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/Serializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serializer.java similarity index 68% rename from clients/src/main/java/org/apache/kafka/clients/producer/Serializer.java rename to clients/src/main/java/org/apache/kafka/common/serialization/Serializer.java index 0378683508396..c2fdc23239bd2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/Serializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serializer.java @@ -11,9 +11,9 @@ * specific language governing permissions and limitations under the License. */ -package org.apache.kafka.clients.producer; +package org.apache.kafka.common.serialization; -import org.apache.kafka.common.Configurable; +import java.util.Map; /** * @@ -21,15 +21,21 @@ * * A class that implements this interface is expected to have a constructor with no parameter. */ -public interface Serializer extends Configurable { +public interface Serializer { + + /** + * Configure this class. + * @param configs configs in key/value pairs + * @param isKey whether is for key or value + */ + public void configure(Map configs, boolean isKey); + /** - * - * @param topic Topic associated with data - * @param data Typed data - * @param isKey Is data for key or value - * @return bytes of the serialized data + * @param topic topic associated with data + * @param data typed data + * @return serialized bytes */ - public byte[] serialize(String topic, T data, boolean isKey); + public byte[] serialize(String topic, T data); /** * Close this serializer diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java new file mode 100644 index 0000000000000..a3b3700a1e071 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java @@ -0,0 +1,50 @@ +/** + * 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.serialization; + +import org.apache.kafka.common.errors.SerializationException; + +import java.io.UnsupportedEncodingException; +import java.util.Map; + +/** + * String encoding defaults to UTF8 and can be customized by setting the property key.deserializer.encoding, + * value.deserializer.encoding or deserializer.encoding. The first two take precedence over the last. + */ +public class StringDeserializer implements Deserializer { + private String encoding = "UTF8"; + + @Override + public void configure(Map configs, boolean isKey) { + String propertyName = isKey ? "key.deserializer.encoding" : "value.deserializer.encoding"; + Object encodingValue = configs.get(propertyName); + if (encodingValue == null) + encodingValue = configs.get("deserializer.encoding"); + if (encodingValue != null && encodingValue instanceof String) + encoding = (String) encodingValue; + } + + @Override + public String deserialize(String topic, byte[] data) { + try { + return new String(data, encoding); + } catch (UnsupportedEncodingException e) { + throw new SerializationException("Error when deserializing byte[] to string due to unsupported encoding " + encoding); + } + } + + @Override + public void close() { + // nothing to do + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java new file mode 100644 index 0000000000000..02db47f873698 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java @@ -0,0 +1,50 @@ +/** + * 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.serialization; + +import org.apache.kafka.common.errors.SerializationException; + +import java.io.UnsupportedEncodingException; +import java.util.Map; + +/** + * String encoding defaults to UTF8 and can be customized by setting the property key.serializer.encoding, + * value.serializer.encoding or serializer.encoding. The first two take precedence over the last. + */ +public class StringSerializer implements Serializer { + private String encoding = "UTF8"; + + @Override + public void configure(Map configs, boolean isKey) { + String propertyName = isKey ? "key.serializer.encoding" : "value.serializer.encoding"; + Object encodingValue = configs.get(propertyName); + if (encodingValue == null) + encodingValue = configs.get("serializer.encoding"); + if (encodingValue != null && encodingValue instanceof String) + encoding = (String) encodingValue; + } + + @Override + public byte[] serialize(String topic, String data) { + try { + return data.getBytes(encoding); + } catch (UnsupportedEncodingException e) { + throw new SerializationException("Error when serializing string to byte[] due to unsupported encoding " + encoding); + } + } + + @Override + public void close() { + // nothing to do + } +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java new file mode 100644 index 0000000000000..d550a3137c066 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java @@ -0,0 +1,68 @@ +/** + * 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.serialization; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class SerializationTest { + + private static class SerDeser { + final Serializer serializer; + final Deserializer deserializer; + + public SerDeser(Serializer serializer, Deserializer deserializer) { + this.serializer = serializer; + this.deserializer = deserializer; + } + } + + @Test + public void testStringSerializer() { + String str = "my string"; + String mytopic = "testTopic"; + List encodings = new ArrayList(); + encodings.add("UTF8"); + encodings.add("UTF-16"); + + for ( String encoding : encodings) { + SerDeser serDeser = getStringSerDeser(encoding); + Serializer serializer = serDeser.serializer; + Deserializer deserializer = serDeser.deserializer; + + assertEquals("Should get the original string after serialization and deserialization with encoding " + encoding, + str, deserializer.deserialize(mytopic, serializer.serialize(mytopic, str))); + } + + } + + private SerDeser getStringSerDeser(String encoder) { + Map serializerConfigs = new HashMap(); + serializerConfigs.put("key.serializer.encoding", encoder); + Serializer serializer = new StringSerializer(); + serializer.configure(serializerConfigs, true); + + Map deserializerConfigs = new HashMap(); + deserializerConfigs.put("key.deserializer.encoding", encoder); + Deserializer deserializer = new StringDeserializer(); + deserializer.configure(deserializerConfigs, true); + + return new SerDeser(serializer, deserializer); + } +} diff --git a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala index e194942492324..652dfb8e13017 100644 --- a/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala +++ b/core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala @@ -60,6 +60,8 @@ class KafkaLog4jAppender extends AppenderSkeleton with Logging { throw new MissingConfigException("topic must be specified by the Kafka log4j appender") if(compressionType != null) props.put(org.apache.kafka.clients.producer.ProducerConfig.COMPRESSION_TYPE_CONFIG, compressionType) if(requiredNumAcks != Int.MaxValue) props.put(org.apache.kafka.clients.producer.ProducerConfig.ACKS_CONFIG, requiredNumAcks.toString) + props.put(org.apache.kafka.clients.producer.ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + props.put(org.apache.kafka.clients.producer.ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") producer = new KafkaProducer[Array[Byte],Array[Byte]](props) LogLog.debug("Kafka producer connected to " + brokerList) LogLog.debug("Logging for topic: " + topic) diff --git a/core/src/main/scala/kafka/tools/ConsoleProducer.scala b/core/src/main/scala/kafka/tools/ConsoleProducer.scala index ac85949447028..7fefc2ed6fed1 100644 --- a/core/src/main/scala/kafka/tools/ConsoleProducer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleProducer.scala @@ -59,6 +59,8 @@ object ConsoleProducer { props.put(ProducerConfig.BUFFER_MEMORY_CONFIG, config.maxMemoryBytes.toString) props.put(ProducerConfig.BATCH_SIZE_CONFIG, config.maxPartitionMemoryBytes.toString) props.put(ProducerConfig.CLIENT_ID_CONFIG, "console-producer") + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") new NewShinyProducer(props) } else { diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index 191542c7b6724..e302523212753 100644 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -27,9 +27,6 @@ import kafka.producer.{KeyedMessage, ProducerConfig} import kafka.metrics.KafkaMetricsGroup import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback import org.apache.kafka.clients.producer.{KafkaProducer, RecordMetadata, ProducerRecord} -import org.apache.kafka.common.KafkaException - -import scala.collection.JavaConversions._ import joptsimple.OptionParser import java.util.Properties @@ -240,8 +237,13 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { producerThreads = (0 until numProducers).map(i => { producerProps.setProperty("client.id", clientId + "-" + i) val producer = - if (useNewProducer) + if (useNewProducer) { + producerProps.put(org.apache.kafka.clients.producer.ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.ByteArraySerializer") + producerProps.put(org.apache.kafka.clients.producer.ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.ByteArraySerializer") new MirrorMakerNewProducer(producerProps) + } else new MirrorMakerOldProducer(producerProps) new ProducerThread(mirrorDataChannel, producer, i) diff --git a/core/src/main/scala/kafka/tools/ProducerPerformance.scala b/core/src/main/scala/kafka/tools/ProducerPerformance.scala index f2dc4ed2f04f0..bc25cd2f371c1 100644 --- a/core/src/main/scala/kafka/tools/ProducerPerformance.scala +++ b/core/src/main/scala/kafka/tools/ProducerPerformance.scala @@ -188,6 +188,8 @@ object ProducerPerformance extends Logging { props.put(ProducerConfig.RETRIES_CONFIG, config.producerNumRetries.toString) props.put(ProducerConfig.RETRY_BACKOFF_MS_CONFIG, config.producerRetryBackoffMs.toString) props.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, config.compressionCodec.name) + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") new NewShinyProducer(props) } else { props.put("metadata.broker.list", config.brokerList) diff --git a/core/src/main/scala/kafka/tools/ReplayLogProducer.scala b/core/src/main/scala/kafka/tools/ReplayLogProducer.scala index f541987b2876a..2b8537b89fafa 100644 --- a/core/src/main/scala/kafka/tools/ReplayLogProducer.scala +++ b/core/src/main/scala/kafka/tools/ReplayLogProducer.scala @@ -120,6 +120,8 @@ object ReplayLogProducer extends Logging { import scala.collection.JavaConversions._ val producerProps = CommandLineUtils.parseKeyValueArgs(options.valuesOf(propertyOpt)) producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") } class ZKConsumerThread(config: Config, stream: KafkaStream[Array[Byte], Array[Byte]]) extends Thread with Logging { diff --git a/core/src/main/scala/kafka/tools/TestEndToEndLatency.scala b/core/src/main/scala/kafka/tools/TestEndToEndLatency.scala index 2ebc7bf643ea9..48cff20e760a9 100644 --- a/core/src/main/scala/kafka/tools/TestEndToEndLatency.scala +++ b/core/src/main/scala/kafka/tools/TestEndToEndLatency.scala @@ -56,6 +56,8 @@ object TestEndToEndLatency { producerProps.put(ProducerConfig.LINGER_MS_CONFIG, "0") producerProps.put(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "true") producerProps.put(ProducerConfig.ACKS_CONFIG, producerAcks.toString) + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") val producer = new KafkaProducer[Array[Byte],Array[Byte]](producerProps) // make sure the consumer fetcher has started before sending data since otherwise diff --git a/core/src/main/scala/kafka/tools/TestLogCleaning.scala b/core/src/main/scala/kafka/tools/TestLogCleaning.scala index b81010ec0fa98..af496f7c547a5 100644 --- a/core/src/main/scala/kafka/tools/TestLogCleaning.scala +++ b/core/src/main/scala/kafka/tools/TestLogCleaning.scala @@ -242,6 +242,8 @@ object TestLogCleaning { val producerProps = new Properties producerProps.setProperty(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "true") producerProps.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerUrl) + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") val producer = new KafkaProducer[Array[Byte],Array[Byte]](producerProps) val rand = new Random(1) val keyCount = (messages / dups).toInt diff --git a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala index 1505fd4464dc9..e63558889272b 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala @@ -75,6 +75,8 @@ class ProducerCompressionTest(compression: String) extends JUnit3Suite with ZooK props.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, compression) props.put(ProducerConfig.BATCH_SIZE_CONFIG, "66000") props.put(ProducerConfig.LINGER_MS_CONFIG, "200") + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") var producer = new KafkaProducer[Array[Byte],Array[Byte]](props) val consumer = new SimpleConsumer("localhost", port, 100, 1024*1024, "") diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala index 6196060edf9f1..b15237b76def3 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala @@ -30,6 +30,10 @@ import kafka.consumer.SimpleConsumer import kafka.api.FetchRequestBuilder import kafka.message.Message import kafka.integration.KafkaServerTestHarness +import org.apache.kafka.common.errors.SerializationException +import java.util.Properties +import org.apache.kafka.common.config.ConfigException +import org.apache.kafka.common.serialization.ByteArraySerializer class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { @@ -126,6 +130,55 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { } } + @Test + def testSerializer() { + // send a record with a wrong type should receive a serialization exception + try { + val producer = createNewProducerWithWrongSerializer(brokerList) + val record5 = new ProducerRecord[Array[Byte],Array[Byte]](topic, new Integer(0), "key".getBytes, "value".getBytes) + producer.send(record5) + fail("Should have gotten a SerializationException") + } catch { + case se: SerializationException => // this is ok + } + + try { + createNewProducerWithNoSerializer(brokerList) + fail("Instantiating a producer without specifying a serializer should cause a ConfigException") + } catch { + case ce : ConfigException => // this is ok + } + + // create a producer with explicit serializers should succeed + createNewProducerWithExplicitSerializer(brokerList) + } + + private def createNewProducerWithWrongSerializer(brokerList: String) : KafkaProducer[Array[Byte],Array[Byte]] = { + import org.apache.kafka.clients.producer.ProducerConfig + + val producerProps = new Properties() + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer") + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer") + return new KafkaProducer[Array[Byte],Array[Byte]](producerProps) + } + + private def createNewProducerWithNoSerializer(brokerList: String) : KafkaProducer[Array[Byte],Array[Byte]] = { + import org.apache.kafka.clients.producer.ProducerConfig + + val producerProps = new Properties() + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + return new KafkaProducer[Array[Byte],Array[Byte]](producerProps) + } + + private def createNewProducerWithExplicitSerializer(brokerList: String) : KafkaProducer[Array[Byte],Array[Byte]] = { + import org.apache.kafka.clients.producer.ProducerConfig + + val producerProps = new Properties() + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + return new KafkaProducer[Array[Byte],Array[Byte]](producerProps, new ByteArraySerializer, new ByteArraySerializer) + } + /** * testClose checks the closing behavior * diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 94d0028d8c490..c9e8ba257b77f 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -395,6 +395,8 @@ object TestUtils extends Logging { producerProps.put(ProducerConfig.RETRIES_CONFIG, retries.toString) producerProps.put(ProducerConfig.RETRY_BACKOFF_MS_CONFIG, "100") producerProps.put(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG, "200") + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") return new KafkaProducer[Array[Byte],Array[Byte]](producerProps) } From 517503db2616531b08ee4d08d39c0e1c0bd19e97 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Tue, 6 Jan 2015 12:10:04 -0800 Subject: [PATCH 080/491] kafka-1797; (delta follow-up patch) add the serializer/deserializer api to the new java client; patched by Jun Rao; reviewed by Neha Narkhede --- .../org/apache/kafka/clients/producer/KafkaProducer.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 91c672d3a78e3..a61c56c01c6c6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -76,6 +76,7 @@ public class KafkaProducer implements Producer { private final Time time; private final Serializer keySerializer; private final Serializer valueSerializer; + private final ProducerConfig producerConfig; /** * A producer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings @@ -152,6 +153,7 @@ private static Properties addSerializerToConfig(Properties properties, private KafkaProducer(ProducerConfig config, Serializer keySerializer, Serializer valueSerializer) { log.trace("Starting the Kafka producer"); + this.producerConfig = config; this.time = new SystemTime(); MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ProducerConfig.METRICS_NUM_SAMPLES_CONFIG)) .timeWindow(config.getLong(ProducerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG), @@ -307,14 +309,16 @@ public Future send(ProducerRecord record, Callback callback serializedKey = keySerializer.serialize(record.topic(), record.key()); } catch (ClassCastException cce) { throw new SerializationException("Can't convert key of class " + record.key().getClass().getName() + - " to the one specified in key.serializer"); + " to class " + producerConfig.getClass(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG).getName() + + " specified in key.serializer"); } byte[] serializedValue; try { serializedValue = valueSerializer.serialize(record.topic(), record.value()); } catch (ClassCastException cce) { throw new SerializationException("Can't convert value of class " + record.value().getClass().getName() + - " to the one specified in value.serializer"); + " to class " + producerConfig.getClass(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG).getName() + + " specified in value.serializer"); } ProducerRecord serializedRecord = new ProducerRecord(record.topic(), record.partition(), serializedKey, serializedValue); int partition = partitioner.partition(serializedRecord, metadata.fetch()); From f82518a85001307334132056fe32c28b452cf48b Mon Sep 17 00:00:00 2001 From: Joe Stein Date: Tue, 6 Jan 2015 22:40:27 -0500 Subject: [PATCH 081/491] KAFKA-1512 Fixes for limit the maximum number of connections per ip address patch by Jeff Holoman reviewed by Jay Krepps and Gwen Shapira --- .../scala/kafka/network/SocketServer.scala | 2 +- .../main/scala/kafka/server/KafkaServer.scala | 3 +- .../unit/kafka/network/SocketServerTest.scala | 45 ++++++++++++++----- 3 files changed, 36 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index e451592fe3581..39b1651b680b2 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -47,7 +47,7 @@ class SocketServer(val brokerId: Int, val maxRequestSize: Int = Int.MaxValue, val maxConnectionsPerIp: Int = Int.MaxValue, val connectionsMaxIdleMs: Long, - val maxConnectionsPerIpOverrides: Map[String, Int] = Map[String, Int]()) extends Logging with KafkaMetricsGroup { + val maxConnectionsPerIpOverrides: Map[String, Int] ) extends Logging with KafkaMetricsGroup { this.logIdent = "[Socket Server on Broker " + brokerId + "], " private val time = SystemTime private val processors = new Array[Processor](numProcessorThreads) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 1bf7d10cef23a..1691ad7fc80ca 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -94,7 +94,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg config.socketReceiveBufferBytes, config.socketRequestMaxBytes, config.maxConnectionsPerIp, - config.connectionsMaxIdleMs) + config.connectionsMaxIdleMs, + config.maxConnectionsPerIpOverrides) socketServer.startup() replicaManager = new ReplicaManager(config, time, zkClient, kafkaScheduler, logManager, isShuttingDown) diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 5f4d85254c384..78b431f9c88cc 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -30,6 +30,7 @@ import kafka.common.TopicAndPartition import kafka.message.ByteBufferMessageSet import java.nio.channels.SelectionKey import kafka.utils.TestUtils +import scala.collection.Map class SocketServerTest extends JUnitSuite { @@ -42,7 +43,8 @@ class SocketServerTest extends JUnitSuite { recvBufferSize = 300000, maxRequestSize = 50, maxConnectionsPerIp = 5, - connectionsMaxIdleMs = 60*1000) + connectionsMaxIdleMs = 60*1000, + maxConnectionsPerIpOverrides = Map.empty[String,Int]) server.startup() def sendRequest(socket: Socket, id: Short, request: Array[Byte]) { @@ -71,13 +73,12 @@ class SocketServerTest extends JUnitSuite { channel.sendResponse(new RequestChannel.Response(request.processor, request, send)) } - def connect() = new Socket("localhost", server.port) + def connect(s:SocketServer = server) = new Socket("localhost", s.port) @After def cleanup() { server.shutdown() } - @Test def simpleRequest() { val socket = connect() @@ -141,19 +142,39 @@ class SocketServerTest extends JUnitSuite { // doing a subsequent send should throw an exception as the connection should be closed. sendRequest(socket, 0, bytes) } - + @Test def testMaxConnectionsPerIp() { // make the maximum allowable number of connections and then leak them val conns = (0 until server.maxConnectionsPerIp).map(i => connect()) - // now try one more (should fail) - try { - val conn = connect() - sendRequest(conn, 100, "hello".getBytes) - assertEquals(-1, conn.getInputStream().read()) - } catch { - case e: IOException => // this is good - } + val conn = connect() + conn.setSoTimeout(3000) + assertEquals(-1, conn.getInputStream().read()) + } + + @Test + def testMaxConnectionsPerIPOverrides(): Unit = { + val overrideNum = 6 + val overrides: Map[String, Int] = Map("localhost" -> overrideNum) + val overrideServer: SocketServer = new SocketServer(0, + host = null, + port = kafka.utils.TestUtils.choosePort, + numProcessorThreads = 1, + maxQueuedRequests = 50, + sendBufferSize = 300000, + recvBufferSize = 300000, + maxRequestSize = 50, + maxConnectionsPerIp = 5, + connectionsMaxIdleMs = 60*1000, + maxConnectionsPerIpOverrides = overrides) + overrideServer.startup() + // make the maximum allowable number of connections and then leak them + val conns = ((0 until overrideNum).map(i => connect(overrideServer))) + // now try one more (should fail) + val conn = connect(overrideServer) + conn.setSoTimeout(3000) + assertEquals(-1, conn.getInputStream.read()) + overrideServer.shutdown() } } From a93ef199b2375c422e35d82ac7aa3a2fdacc1e74 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Fri, 9 Jan 2015 11:27:00 -0800 Subject: [PATCH 082/491] kafka-1797; (addressing Manikumar Reddy's comment) add the serializer/deserializer api to the new java client; patched by Jun Rao; reviewed by Manikumar Reddy and Neha Narkhede --- .../kafka/common/serialization/StringDeserializer.java | 5 ++++- .../apache/kafka/common/serialization/StringSerializer.java | 5 ++++- .../apache/kafka/common/serialization/SerializationTest.java | 4 +++- 3 files changed, 11 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java index a3b3700a1e071..9783ea0c53110 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/StringDeserializer.java @@ -37,7 +37,10 @@ public void configure(Map configs, boolean isKey) { @Override public String deserialize(String topic, byte[] data) { try { - return new String(data, encoding); + if (data == null) + return null; + else + return new String(data, encoding); } catch (UnsupportedEncodingException e) { throw new SerializationException("Error when deserializing byte[] to string due to unsupported encoding " + encoding); } diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java index 02db47f873698..636d905ffc729 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/StringSerializer.java @@ -37,7 +37,10 @@ public void configure(Map configs, boolean isKey) { @Override public byte[] serialize(String topic, String data) { try { - return data.getBytes(encoding); + if (data == null) + return null; + else + return data.getBytes(encoding); } catch (UnsupportedEncodingException e) { throw new SerializationException("Error when serializing string to byte[] due to unsupported encoding " + encoding); } diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java index d550a3137c066..b6e1497506675 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java @@ -48,8 +48,10 @@ public void testStringSerializer() { assertEquals("Should get the original string after serialization and deserialization with encoding " + encoding, str, deserializer.deserialize(mytopic, serializer.serialize(mytopic, str))); - } + assertEquals("Should support null in serialization and deserialization with encoding " + encoding, + null, deserializer.deserialize(mytopic, serializer.serialize(mytopic, null))); + } } private SerDeser getStringSerDeser(String encoder) { From e52a6181bf0969f315ac0f0d325eac34d2b4a6ee Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Fri, 9 Jan 2015 11:33:48 -0800 Subject: [PATCH 083/491] kafka-1851; OffsetFetchRequest returns extra partitions when input only contains unknown partitions; patched by Jun Rao; reviewed by Neha Narkhede --- core/src/main/scala/kafka/server/KafkaApis.scala | 6 +++++- .../test/scala/unit/kafka/server/OffsetCommitTest.scala | 9 ++++++++- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 2a1c0326b6e69..c011a1b79bd6c 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -396,7 +396,11 @@ class KafkaApis(val requestChannel: RequestChannel, metadataCache.getPartitionInfo(topicAndPartition.topic, topicAndPartition.partition).isEmpty ) val unknownStatus = unknownTopicPartitions.map(topicAndPartition => (topicAndPartition, OffsetMetadataAndError.UnknownTopicOrPartition)).toMap - val knownStatus = offsetManager.getOffsets(offsetFetchRequest.groupId, knownTopicPartitions).toMap + val knownStatus = + if (knownTopicPartitions.size > 0) + offsetManager.getOffsets(offsetFetchRequest.groupId, knownTopicPartitions).toMap + else + Map.empty[TopicAndPartition, OffsetMetadataAndError] val status = unknownStatus ++ knownStatus val response = OffsetFetchResponse(status, offsetFetchRequest.correlationId) diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index 8c5364fa97da1..4a3a5b264a021 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -79,7 +79,7 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { // create the topic createTopic(zkClient, topic, partitionReplicaAssignment = expectedReplicaAssignment, servers = Seq(server)) - val commitRequest = OffsetCommitRequest("test-group", immutable.Map(topicAndPartition -> OffsetAndMetadata(offset=42L))) + val commitRequest = OffsetCommitRequest(group, immutable.Map(topicAndPartition -> OffsetAndMetadata(offset=42L))) val commitResponse = simpleConsumer.commitOffsets(commitRequest) assertEquals(ErrorMapping.NoError, commitResponse.commitStatus.get(topicAndPartition).get) @@ -109,6 +109,13 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals("some metadata", fetchResponse1.requestInfo.get(topicAndPartition).get.metadata) assertEquals(100L, fetchResponse1.requestInfo.get(topicAndPartition).get.offset) + // Fetch an unknown topic and verify + val unknownTopicAndPartition = TopicAndPartition("unknownTopic", 0) + val fetchRequest2 = OffsetFetchRequest(group, Seq(unknownTopicAndPartition)) + val fetchResponse2 = simpleConsumer.fetchOffsets(fetchRequest2) + + assertEquals(OffsetMetadataAndError.UnknownTopicOrPartition, fetchResponse2.requestInfo.get(unknownTopicAndPartition).get) + assertEquals(1, fetchResponse2.requestInfo.size) } @Test From b1b80860a01cc378cfada3549a3480f0773c3ff8 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Mon, 12 Jan 2015 15:45:13 -0800 Subject: [PATCH 084/491] KAFKA-1070 Auto assign broker id; reviewed by Neha Narkhede --- .../common/GenerateBrokerIdException.scala | 27 +++ .../InconsistentBrokerIdException.scala | 27 +++ .../kafka/log/LogCleanerManager.scala.orig | 203 ++++++++++++++++++ .../server/BrokerMetadataCheckpoint.scala | 83 +++++++ .../main/scala/kafka/server/KafkaConfig.scala | 20 +- .../main/scala/kafka/server/KafkaServer.scala | 87 ++++++-- core/src/main/scala/kafka/utils/ZkUtils.scala | 38 +++- .../server/ServerGenerateBrokerIdTest.scala | 127 +++++++++++ .../scala/unit/kafka/utils/TestUtils.scala | 9 +- 9 files changed, 596 insertions(+), 25 deletions(-) create mode 100644 core/src/main/scala/kafka/common/GenerateBrokerIdException.scala create mode 100644 core/src/main/scala/kafka/common/InconsistentBrokerIdException.scala create mode 100644 core/src/main/scala/kafka/log/LogCleanerManager.scala.orig create mode 100644 core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala create mode 100644 core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala diff --git a/core/src/main/scala/kafka/common/GenerateBrokerIdException.scala b/core/src/main/scala/kafka/common/GenerateBrokerIdException.scala new file mode 100644 index 0000000000000..13784fe50554e --- /dev/null +++ b/core/src/main/scala/kafka/common/GenerateBrokerIdException.scala @@ -0,0 +1,27 @@ +/** + * 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.common + +/** + * Thrown when there is a failure to generate a zookeeper sequenceId to use as brokerId + */ +class GenerateBrokerIdException(message: String, cause: Throwable) extends RuntimeException(message, cause) { + def this(message: String) = this(message, null) + def this(cause: Throwable) = this(null, cause) + def this() = this(null, null) +} diff --git a/core/src/main/scala/kafka/common/InconsistentBrokerIdException.scala b/core/src/main/scala/kafka/common/InconsistentBrokerIdException.scala new file mode 100644 index 0000000000000..0c0d1cd731a4d --- /dev/null +++ b/core/src/main/scala/kafka/common/InconsistentBrokerIdException.scala @@ -0,0 +1,27 @@ +/** + * 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.common + +/** + * Indicates the brokerId stored in logDirs is not consistent across logDirs. + */ +class InconsistentBrokerIdException(message: String, cause: Throwable) extends RuntimeException(message, cause) { + def this(message: String) = this(message, null) + def this(cause: Throwable) = this(null, cause) + def this() = this(null, null) +} diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala.orig b/core/src/main/scala/kafka/log/LogCleanerManager.scala.orig new file mode 100644 index 0000000000000..e8ced6a592250 --- /dev/null +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala.orig @@ -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.log + +import java.io.File +import kafka.metrics.KafkaMetricsGroup +import com.yammer.metrics.core.Gauge +import kafka.utils.{Logging, Pool} +import kafka.server.OffsetCheckpoint +import collection.mutable +import java.util.concurrent.locks.ReentrantLock +import kafka.utils.Utils._ +import java.util.concurrent.TimeUnit +import kafka.common.{LogCleaningAbortedException, TopicAndPartition} + +private[log] sealed trait LogCleaningState +private[log] case object LogCleaningInProgress extends LogCleaningState +private[log] case object LogCleaningAborted extends LogCleaningState +private[log] case object LogCleaningPaused extends LogCleaningState + +/** + * Manage the state of each partition being cleaned. + * If a partition is to be cleaned, it enters the LogCleaningInProgress state. + * While a partition is being cleaned, it can be requested to be aborted and paused. Then the partition first enters + * the LogCleaningAborted state. Once the cleaning task is aborted, the partition enters the LogCleaningPaused state. + * While a partition is in the LogCleaningPaused state, it won't be scheduled for cleaning again, until cleaning is + * requested to be resumed. + */ +private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[TopicAndPartition, Log]) extends Logging with KafkaMetricsGroup { + + override val loggerName = classOf[LogCleaner].getName + + /* the offset checkpoints holding the last cleaned point for each log */ + private val checkpoints = logDirs.map(dir => (dir, new OffsetCheckpoint(new File(dir, "cleaner-offset-checkpoint")))).toMap + + /* the set of logs currently being cleaned */ + private val inProgress = mutable.HashMap[TopicAndPartition, LogCleaningState]() + + /* a global lock used to control all access to the in-progress set and the offset checkpoints */ + private val lock = new ReentrantLock + + /* for coordinating the pausing and the cleaning of a partition */ + private val pausedCleaningCond = lock.newCondition() + + /* a gauge for tracking the cleanable ratio of the dirtiest log */ + @volatile private var dirtiestLogCleanableRatio = 0.0 + newGauge("max-dirty-percent", new Gauge[Int] { def value = (100 * dirtiestLogCleanableRatio).toInt }) + + /** + * @return the position processed for all logs. + */ + def allCleanerCheckpoints(): Map[TopicAndPartition, Long] = + checkpoints.values.flatMap(_.read()).toMap + + /** + * Choose the log to clean next and add it to the in-progress set. We recompute this + * every time off the full set of logs to allow logs to be dynamically added to the pool of logs + * the log manager maintains. + */ + def grabFilthiestLog(): Option[LogToClean] = { + inLock(lock) { + val lastClean = allCleanerCheckpoints() + val dirtyLogs = logs.filter(l => l._2.config.compact) // skip any logs marked for delete rather than dedupe + .filterNot(l => inProgress.contains(l._1)) // skip any logs already in-progress + .map(l => LogToClean(l._1, l._2, // create a LogToClean instance for each + lastClean.getOrElse(l._1, l._2.logSegments.head.baseOffset))) + .filter(l => l.totalBytes > 0) // skip any empty logs + this.dirtiestLogCleanableRatio = if (!dirtyLogs.isEmpty) dirtyLogs.max.cleanableRatio else 0 + val cleanableLogs = dirtyLogs.filter(l => l.cleanableRatio > l.log.config.minCleanableRatio) // and must meet the minimum threshold for dirty byte ratio + if(cleanableLogs.isEmpty) { + None + } else { + val filthiest = cleanableLogs.max + inProgress.put(filthiest.topicPartition, LogCleaningInProgress) + Some(filthiest) + } + } + } + + /** + * Abort the cleaning of a particular partition, if it's in progress. This call blocks until the cleaning of + * the partition is aborted. + * This is implemented by first abortAndPausing and then resuming the cleaning of the partition. + */ + def abortCleaning(topicAndPartition: TopicAndPartition) { + inLock(lock) { + abortAndPauseCleaning(topicAndPartition) + resumeCleaning(topicAndPartition) + info("The cleaning for partition %s is aborted".format(topicAndPartition)) + } + } + + /** + * Abort the cleaning of a particular partition if it's in progress, and pause any future cleaning of this partition. + * This call blocks until the cleaning of the partition is aborted and paused. + * 1. If the partition is not in progress, mark it as paused. + * 2. Otherwise, first mark the state of the partition as aborted. + * 3. The cleaner thread checks the state periodically and if it sees the state of the partition is aborted, it + * throws a LogCleaningAbortedException to stop the cleaning task. + * 4. When the cleaning task is stopped, doneCleaning() is called, which sets the state of the partition as paused. + * 5. abortAndPauseCleaning() waits until the state of the partition is changed to paused. + */ + def abortAndPauseCleaning(topicAndPartition: TopicAndPartition) { + inLock(lock) { + inProgress.get(topicAndPartition) match { + case None => + inProgress.put(topicAndPartition, LogCleaningPaused) + case Some(state) => + state match { + case LogCleaningInProgress => + inProgress.put(topicAndPartition, LogCleaningAborted) + case s => + throw new IllegalStateException("Compaction for partition %s cannot be aborted and paused since it is in %s state." + .format(topicAndPartition, s)) + } + } + while (!isCleaningInState(topicAndPartition, LogCleaningPaused)) + pausedCleaningCond.await(100, TimeUnit.MILLISECONDS) + info("The cleaning for partition %s is aborted and paused".format(topicAndPartition)) + } + } + + /** + * Resume the cleaning of a paused partition. This call blocks until the cleaning of a partition is resumed. + */ + def resumeCleaning(topicAndPartition: TopicAndPartition) { + inLock(lock) { + inProgress.get(topicAndPartition) match { + case None => + throw new IllegalStateException("Compaction for partition %s cannot be resumed since it is not paused." + .format(topicAndPartition)) + case Some(state) => + state match { + case LogCleaningPaused => + inProgress.remove(topicAndPartition) + case s => + throw new IllegalStateException("Compaction for partition %s cannot be resumed since it is in %s state." + .format(topicAndPartition, s)) + } + } + } + info("Compaction for partition %s is resumed".format(topicAndPartition)) + } + + /** + * Check if the cleaning for a partition is in a particular state. The caller is expected to hold lock while making the call. + */ + def isCleaningInState(topicAndPartition: TopicAndPartition, expectedState: LogCleaningState): Boolean = { + inProgress.get(topicAndPartition) match { + case None => return false + case Some(state) => + if (state == expectedState) + return true + else + return false + } + } + + /** + * Check if the cleaning for a partition is aborted. If so, throw an exception. + */ + def checkCleaningAborted(topicAndPartition: TopicAndPartition) { + inLock(lock) { + if (isCleaningInState(topicAndPartition, LogCleaningAborted)) + throw new LogCleaningAbortedException() + } + } + + /** + * Save out the endOffset and remove the given log from the in-progress set, if not aborted. + */ + def doneCleaning(topicAndPartition: TopicAndPartition, dataDir: File, endOffset: Long) { + inLock(lock) { + inProgress(topicAndPartition) match { + case LogCleaningInProgress => + val checkpoint = checkpoints(dataDir) + val offsets = checkpoint.read() + ((topicAndPartition, endOffset)) + checkpoint.write(offsets) + inProgress.remove(topicAndPartition) + case LogCleaningAborted => + inProgress.put(topicAndPartition, LogCleaningPaused) + pausedCleaningCond.signalAll() + case s => + throw new IllegalStateException("In-progress partition %s cannot be in %s state.".format(topicAndPartition, s)) + } + } + } +} diff --git a/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala b/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala new file mode 100644 index 0000000000000..0e542ff0cc3ed --- /dev/null +++ b/core/src/main/scala/kafka/server/BrokerMetadataCheckpoint.scala @@ -0,0 +1,83 @@ +/** + * 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.io._ +import java.util.Properties +import kafka.utils._ + + +case class BrokerMetadata(brokerId: Int) + +/** + * This class saves broker's metadata to a file + */ +class BrokerMetadataCheckpoint(val file: File) extends Logging { + private val lock = new Object() + new File(file + ".tmp").delete() // try to delete any existing temp files for cleanliness + + def write(brokerMetadata: BrokerMetadata) = { + lock synchronized { + try { + val brokerMetaProps = new Properties() + brokerMetaProps.setProperty("version", 0.toString) + brokerMetaProps.setProperty("broker.id", brokerMetadata.brokerId.toString) + val temp = new File(file.getAbsolutePath + ".tmp") + val fileOutputStream = new FileOutputStream(temp) + brokerMetaProps.store(fileOutputStream,"") + fileOutputStream.flush() + fileOutputStream.getFD().sync() + fileOutputStream.close() + // swap new BrokerMetadata file with previous one + if(!temp.renameTo(file)) { + // renameTo() fails on windows if destination file exists. + file.delete() + if(!temp.renameTo(file)) + throw new IOException("File rename from %s to %s failed.".format(temp.getAbsolutePath(), file.getAbsolutePath())) + } + } catch { + case ie: IOException => + error("Failed to write meta.properties due to ",ie) + throw ie + } + } + } + + def read(): Option[BrokerMetadata] = { + lock synchronized { + try { + val brokerMetaProps = new VerifiableProperties(Utils.loadProps(file.getAbsolutePath())) + val version = brokerMetaProps.getIntInRange("version", (0, Int.MaxValue)) + version match { + case 0 => + val brokerId = brokerMetaProps.getIntInRange("broker.id", (0, Int.MaxValue)) + return Some(BrokerMetadata(brokerId)) + case _ => + throw new IOException("Unrecognized version of the server meta.properties file: " + version) + } + } catch { + case e: FileNotFoundException => + warn("No meta.properties file under dir %s".format(file.getAbsolutePath(), e.getMessage)) + None + case e1: Exception => + error("Failed to read meta.properties file under dir %s due to %s".format(file.getAbsolutePath(), e1.getMessage)) + throw e1 + } + } + } +} diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 6e26c5436feb4..bbd3fd75e8396 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -35,13 +35,13 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro private def getLogRetentionTimeMillis(): Long = { val millisInMinute = 60L * 1000L val millisInHour = 60L * millisInMinute - + if(props.containsKey("log.retention.ms")){ props.getIntInRange("log.retention.ms", (1, Int.MaxValue)) } else if(props.containsKey("log.retention.minutes")){ millisInMinute * props.getIntInRange("log.retention.minutes", (1, Int.MaxValue)) - } + } else { millisInHour * props.getIntInRange("log.retention.hours", 24*7, (1, Int.MaxValue)) } @@ -49,7 +49,7 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro private def getLogRollTimeMillis(): Long = { val millisInHour = 60L * 60L * 1000L - + if(props.containsKey("log.roll.ms")){ props.getIntInRange("log.roll.ms", (1, Int.MaxValue)) } @@ -71,8 +71,14 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /*********** General Configuration ***********/ - /* the broker id for this server */ - val brokerId: Int = props.getIntInRange("broker.id", (0, Int.MaxValue)) + /* Max number that can be used for a broker.id */ + val MaxReservedBrokerId = props.getIntInRange("reserved.broker.max.id", 1000, (0, Int.MaxValue)) + + /* The broker id for this server. + * To avoid conflicts between zookeeper generated brokerId and user's config.brokerId + * added MaxReservedBrokerId and zookeeper sequence starts from MaxReservedBrokerId + 1. + */ + var brokerId: Int = if (props.containsKey("broker.id")) props.getIntInRange("broker.id", (0, MaxReservedBrokerId)) else -1 /* the maximum size of message that the server can receive */ val messageMaxBytes = props.getIntInRange("message.max.bytes", 1000000 + MessageSet.LogOverhead, (0, Int.MaxValue)) @@ -117,10 +123,10 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* the maximum number of bytes in a socket request */ val socketRequestMaxBytes: Int = props.getIntInRange("socket.request.max.bytes", 100*1024*1024, (1, Int.MaxValue)) - + /* the maximum number of connections we allow from each ip address */ val maxConnectionsPerIp: Int = props.getIntInRange("max.connections.per.ip", Int.MaxValue, (1, Int.MaxValue)) - + /* per-ip or hostname overrides to the default maximum number of connections */ val maxConnectionsPerIpOverrides = props.getMap("max.connections.per.ip.overrides").map(entry => (entry._1, entry._2.toInt)) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 1691ad7fc80ca..a069eb9272c92 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -25,11 +25,12 @@ import kafka.utils._ import java.util.concurrent._ import atomic.{AtomicInteger, AtomicBoolean} import java.io.File +import collection.mutable import org.I0Itec.zkclient.ZkClient import kafka.controller.{ControllerStats, KafkaController} import kafka.cluster.Broker import kafka.api.{ControlledShutdownResponse, ControlledShutdownRequest} -import kafka.common.ErrorMapping +import kafka.common.{ErrorMapping, InconsistentBrokerIdException, GenerateBrokerIdException} import kafka.network.{Receive, BlockingChannel, SocketServer} import kafka.metrics.KafkaMetricsGroup import com.yammer.metrics.core.Gauge @@ -39,10 +40,11 @@ import com.yammer.metrics.core.Gauge * to start up and shutdown a single Kafka node. */ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logging with KafkaMetricsGroup { - this.logIdent = "[Kafka Server " + config.brokerId + "], " private var isShuttingDown = new AtomicBoolean(false) private var shutdownLatch = new CountDownLatch(1) private var startupComplete = new AtomicBoolean(false) + private var brokerId: Int = -1 + val brokerState: BrokerState = new BrokerState val correlationId: AtomicInteger = new AtomicInteger(0) var socketServer: SocketServer = null @@ -56,6 +58,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg var kafkaController: KafkaController = null val kafkaScheduler = new KafkaScheduler(config.backgroundThreads) var zkClient: ZkClient = null + val brokerMetaPropsFile = "meta.properties" + val brokerMetadataCheckpoints = config.logDirs.map(logDir => (logDir, new BrokerMetadataCheckpoint(new File(logDir + File.separator +brokerMetaPropsFile)))).toMap newGauge( "BrokerState", @@ -77,7 +81,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg /* start scheduler */ kafkaScheduler.startup() - + /* setup zookeeper */ zkClient = initZk() @@ -85,6 +89,10 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg logManager = createLogManager(zkClient, brokerState) logManager.startup() + /* generate brokerId */ + config.brokerId = getBrokerId + this.logIdent = "[Kafka Server " + config.brokerId + "], " + socketServer = new SocketServer(config.brokerId, config.hostName, config.port, @@ -104,26 +112,25 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg offsetManager = createOffsetManager() kafkaController = new KafkaController(config, zkClient, brokerState) - + /* start processing requests */ apis = new KafkaApis(socketServer.requestChannel, replicaManager, offsetManager, zkClient, config.brokerId, config, kafkaController) requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) brokerState.newState(RunningAsBroker) - + Mx4jLoader.maybeLoad() replicaManager.startup() kafkaController.startup() - + topicConfigManager = new TopicConfigManager(zkClient, logManager) topicConfigManager.startup() - + /* tell everyone we are alive */ kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, config.advertisedHostName, config.advertisedPort, config.zkSessionTimeoutMs, zkClient) kafkaHealthcheck.startup() - registerStats() startupComplete.set(true) info("started") @@ -181,10 +188,10 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg info("Starting controlled shutdown") var channel : BlockingChannel = null var prevController : Broker = null - var shutdownSuceeded : Boolean = false + var shutdownSucceeded : Boolean = false try { brokerState.newState(PendingControlledShutdown) - while (!shutdownSuceeded && remainingRetries > 0) { + while (!shutdownSucceeded && remainingRetries > 0) { remainingRetries = remainingRetries - 1 // 1. Find the controller and establish a connection to it. @@ -223,7 +230,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg val shutdownResponse = ControlledShutdownResponse.readFrom(response.buffer) if (shutdownResponse.errorCode == ErrorMapping.NoError && shutdownResponse.partitionsRemaining != null && shutdownResponse.partitionsRemaining.size == 0) { - shutdownSuceeded = true + shutdownSucceeded = true info ("Controlled shutdown succeeded") } else { @@ -239,7 +246,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg // ignore and try again } } - if (!shutdownSuceeded) { + if (!shutdownSucceeded) { Thread.sleep(config.controlledShutdownRetryBackoffMs) warn("Retrying controlled shutdown after the previous attempt failed...") } @@ -251,7 +258,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg channel = null } } - if (!shutdownSuceeded) { + if (!shutdownSucceeded) { warn("Proceeding to do an unclean shutdown as all the controlled shutdown attempts failed") } } @@ -307,7 +314,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg def awaitShutdown(): Unit = shutdownLatch.await() def getLogManager(): LogManager = logManager - + private def createLogManager(zkClient: ZkClient, brokerState: BrokerState): LogManager = { val defaultLogConfig = LogConfig(segmentSize = config.logSegmentBytes, segmentMs = config.logRollTimeMillis, @@ -359,5 +366,55 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg new OffsetManager(offsetManagerConfig, replicaManager, zkClient, kafkaScheduler) } -} + /** + * Generates new brokerId or reads from meta.properties based on following conditions + *

        + *
      1. config has no broker.id provided , generates a broker.id based on Zookeeper's sequence + *
      2. stored broker.id in meta.properties doesn't match in all the log.dirs throws InconsistentBrokerIdException + *
      3. config has broker.id and meta.properties contains broker.id if they don't match throws InconsistentBrokerIdException + *
      4. config has broker.id and there is no meta.properties file, creates new meta.properties and stores broker.id + *
          + * @returns A brokerId. + */ + private def getBrokerId: Int = { + var brokerId = config.brokerId + var logDirsWithoutMetaProps: List[String] = List() + val brokerIdSet = mutable.HashSet[Int]() + + for (logDir <- config.logDirs) { + val brokerMetadataOpt = brokerMetadataCheckpoints(logDir).read() + brokerMetadataOpt match { + case Some(brokerMetadata: BrokerMetadata) => + brokerIdSet.add(brokerMetadata.brokerId) + case None => + logDirsWithoutMetaProps ++= List(logDir) + } + } + + if(brokerIdSet.size > 1) + throw new InconsistentBrokerIdException("Failed to match brokerId across logDirs") + else if(brokerId >= 0 && brokerIdSet.size == 1 && brokerIdSet.last != brokerId) + throw new InconsistentBrokerIdException("Configured brokerId %s doesn't match stored brokerId %s in meta.properties".format(brokerId, brokerIdSet.last)) + else if(brokerIdSet.size == 0 && brokerId < 0) // generate a new brokerId from Zookeeper + brokerId = generateBrokerId + else if(brokerIdSet.size == 1) // pick broker.id from meta.properties + brokerId = brokerIdSet.last + + for(logDir <- logDirsWithoutMetaProps) { + val checkpoint = brokerMetadataCheckpoints(logDir) + checkpoint.write(new BrokerMetadata(brokerId)) + } + + return brokerId + } + private def generateBrokerId: Int = { + try { + ZkUtils.getBrokerSequenceId(zkClient, config.MaxReservedBrokerId) + } catch { + case e: Exception => + error("Failed to generate broker.id due to ", e) + throw new GenerateBrokerIdException("Failed to generate broker.id", e) + } + } +} diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 56e3e88e0cc6d..c14bd455b6642 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -46,6 +46,7 @@ object ZkUtils extends Logging { val ReassignPartitionsPath = "/admin/reassign_partitions" val DeleteTopicsPath = "/admin/delete_topics" val PreferredReplicaLeaderElectionPath = "/admin/preferred_replica_election" + val BrokerSequenceIdPath = "/brokers/seqid" def getTopicPath(topic: String): String = { BrokerTopicsPath + "/" + topic @@ -87,7 +88,8 @@ object ZkUtils extends Logging { } def setupCommonPaths(zkClient: ZkClient) { - for(path <- Seq(ConsumersPath, BrokerIdsPath, BrokerTopicsPath, TopicConfigChangesPath, TopicConfigPath, DeleteTopicsPath)) + for(path <- Seq(ConsumersPath, BrokerIdsPath, BrokerTopicsPath, TopicConfigChangesPath, TopicConfigPath, + DeleteTopicsPath, BrokerSequenceIdPath)) makeSurePersistentPathExists(zkClient, path) } @@ -122,6 +124,14 @@ object ZkUtils extends Logging { } } + /** returns a sequence id generated by updating BrokerSequenceIdPath in Zk. + * users can provide brokerId in the config , inorder to avoid conflicts between zk generated + * seqId and config.brokerId we increment zk seqId by KafkaConfig.MaxReservedBrokerId. + */ + def getBrokerSequenceId(zkClient: ZkClient, MaxReservedBrokerId: Int): Int = { + getSequenceId(zkClient, BrokerSequenceIdPath) + MaxReservedBrokerId + } + /** * Gets the in-sync replicas (ISR) for a specific topic and partition */ @@ -696,6 +706,32 @@ object ZkUtils extends Logging { } } + /** + * This API produces a sequence number by creating / updating given path in zookeeper + * It uses the stat returned by the zookeeper and return the version. Every time + * client updates the path stat.version gets incremented + */ + def getSequenceId(client: ZkClient, path: String): Int = { + try { + val stat = client.writeDataReturnStat(path, "", -1) + return stat.getVersion + } catch { + case e: ZkNoNodeException => { + createParentPath(client, BrokerSequenceIdPath) + try { + client.createPersistent(BrokerSequenceIdPath, "") + return 0 + } catch { + case e: ZkNodeExistsException => + val stat = client.writeDataReturnStat(BrokerSequenceIdPath, "", -1) + return stat.getVersion + case e2: Throwable => throw e2 + } + } + case e2: Throwable => throw e2 + } + } + def getAllTopics(zkClient: ZkClient): Seq[String] = { val topics = ZkUtils.getChildrenParentMayNotExist(zkClient, BrokerTopicsPath) if(topics == null) diff --git a/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala new file mode 100644 index 0000000000000..cf2dd9455a919 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala @@ -0,0 +1,127 @@ +/** + * 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 kafka.zk.ZooKeeperTestHarness +import kafka.utils.{TestUtils, Utils} +import org.junit.Test +import org.scalatest.junit.JUnit3Suite +import junit.framework.Assert._ +import java.io.File + +class ServerGenerateBrokerIdTest extends JUnit3Suite with ZooKeeperTestHarness { + var props1 = TestUtils.createBrokerConfig(-1, TestUtils.choosePort) + var config1 = new KafkaConfig(props1) + var props2 = TestUtils.createBrokerConfig(0, TestUtils.choosePort) + var config2 = new KafkaConfig(props2) + val brokerMetaPropsFile = "meta.properties" + + + @Test + def testAutoGenerateBrokerId() { + var server1 = new KafkaServer(config1) + server1.startup() + server1.shutdown() + assertTrue(verifyBrokerMetadata(config1.logDirs, 1001)) + // restart the server check to see if it uses the brokerId generated previously + server1 = new KafkaServer(config1) + server1.startup() + assertEquals(server1.config.brokerId, 1001) + server1.shutdown() + Utils.rm(server1.config.logDirs) + TestUtils.verifyNonDaemonThreadsStatus + } + + @Test + def testUserConfigAndGeneratedBrokerId() { + // start the server with broker.id as part of config + val server1 = new KafkaServer(config1) + val server2 = new KafkaServer(config2) + val props3 = TestUtils.createBrokerConfig(-1, TestUtils.choosePort) + val config3 = new KafkaConfig(props3) + val server3 = new KafkaServer(config3) + server1.startup() + assertEquals(server1.config.brokerId,1001) + server2.startup() + assertEquals(server2.config.brokerId,0) + server3.startup() + assertEquals(server3.config.brokerId,1002) + server1.shutdown() + server2.shutdown() + server3.shutdown() + assertTrue(verifyBrokerMetadata(server1.config.logDirs,1001)) + assertTrue(verifyBrokerMetadata(server2.config.logDirs,0)) + assertTrue(verifyBrokerMetadata(server3.config.logDirs,1002)) + Utils.rm(server1.config.logDirs) + Utils.rm(server2.config.logDirs) + Utils.rm(server3.config.logDirs) + TestUtils.verifyNonDaemonThreadsStatus + } + + @Test + def testMultipleLogDirsMetaProps() { + // add multiple logDirs and check if the generate brokerId is stored in all of them + val logDirs = props1.getProperty("log.dir")+ "," + TestUtils.tempDir().getAbsolutePath + + "," + TestUtils.tempDir().getAbsolutePath + props1.setProperty("log.dir",logDirs) + config1 = new KafkaConfig(props1) + var server1 = new KafkaServer(config1) + server1.startup() + server1.shutdown() + assertTrue(verifyBrokerMetadata(config1.logDirs, 1001)) + // addition to log.dirs after generation of a broker.id from zk should be copied over + val newLogDirs = props1.getProperty("log.dir") + "," + TestUtils.tempDir().getAbsolutePath + props1.setProperty("log.dir",newLogDirs) + config1 = new KafkaConfig(props1) + server1 = new KafkaServer(config1) + server1.startup() + server1.shutdown() + assertTrue(verifyBrokerMetadata(config1.logDirs, 1001)) + Utils.rm(server1.config.logDirs) + TestUtils.verifyNonDaemonThreadsStatus + } + + @Test + def testConsistentBrokerIdFromUserConfigAndMetaProps() { + // check if configured brokerId and stored brokerId are equal or throw InconsistentBrokerException + var server1 = new KafkaServer(config1) //auto generate broker Id + server1.startup() + server1.shutdown() + server1 = new KafkaServer(config2) // user specified broker id + try { + server1.startup() + } catch { + case e: kafka.common.InconsistentBrokerIdException => //success + } + server1.shutdown() + Utils.rm(server1.config.logDirs) + TestUtils.verifyNonDaemonThreadsStatus + } + + def verifyBrokerMetadata(logDirs: Seq[String], brokerId: Int): Boolean = { + for(logDir <- logDirs) { + val brokerMetadataOpt = (new BrokerMetadataCheckpoint( + new File(logDir + File.separator + brokerMetaPropsFile))).read() + brokerMetadataOpt match { + case Some(brokerMetadata: BrokerMetadata) => + if (brokerMetadata.brokerId != brokerId) return false + case _ => return false + } + } + true + } +} diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index c9e8ba257b77f..ac15d34425795 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -94,7 +94,7 @@ object TestUtils extends Logging { Utils.rm(f) } }) - + f } @@ -154,7 +154,7 @@ object TestUtils extends Logging { def createBrokerConfig(nodeId: Int, port: Int = choosePort(), enableControlledShutdown: Boolean = true): Properties = { val props = new Properties - props.put("broker.id", nodeId.toString) + if (nodeId >= 0) props.put("broker.id", nodeId.toString) props.put("host.name", "localhost") props.put("port", port.toString) props.put("log.dir", TestUtils.tempDir().getAbsolutePath) @@ -700,6 +700,11 @@ object TestUtils extends Logging { ZkUtils.pathExists(zkClient, ZkUtils.ReassignPartitionsPath) } + def verifyNonDaemonThreadsStatus() { + assertEquals(0, Thread.getAllStackTraces.keySet().toArray + .map(_.asInstanceOf[Thread]) + .count(t => !t.isDaemon && t.isAlive && t.getClass.getCanonicalName.toLowerCase.startsWith("kafka"))) + } /** * Create new LogManager instance with default configuration for testing From ad4883a0cd5afc4219e28cdafbd98576eeaee2d1 Mon Sep 17 00:00:00 2001 From: Jaikiran Pai Date: Mon, 12 Jan 2015 15:58:36 -0800 Subject: [PATCH 085/491] KAFKA-1836 metadata.fetch.timeout.ms set to zero blocks forever; reviewed by Neha Narkhede and Ewen Cheslack-Postava --- .../clients/producer/internals/Metadata.java | 13 ++++-- .../kafka/clients/producer/MetadataTest.java | 41 +++++++++++++++---- 2 files changed, 43 insertions(+), 11 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java index 1d30f9edd9533..dcf46581b912c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java @@ -25,7 +25,7 @@ *

          * This class is shared by the client thread (for partitioning) and the background sender thread. * - * Metadata is maintained for only a subset of topics, which can be added to over time. When we request metdata for a + * Metadata is maintained for only a subset of topics, which can be added to over time. When we request metadata for a * topic we don't have any metadata for it will trigger a metadata update. */ public final class Metadata { @@ -99,12 +99,17 @@ public synchronized int requestUpdate() { /** * Wait for metadata update until the current version is larger than the last version we know of */ - public synchronized void awaitUpdate(int lastVerison, long maxWaitMs) { + public synchronized void awaitUpdate(final int lastVersion, final long maxWaitMs) { + if (maxWaitMs < 0) { + throw new IllegalArgumentException("Max time to wait for metadata updates should not be < 0 milli seconds"); + } long begin = System.currentTimeMillis(); long remainingWaitMs = maxWaitMs; - while (this.version <= lastVerison) { + while (this.version <= lastVersion) { try { - wait(remainingWaitMs); + if (remainingWaitMs != 0) { + wait(remainingWaitMs); + } } catch (InterruptedException e) { /* this is fine */ } long elapsed = System.currentTimeMillis() - begin; diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java index 4547bfcb44be4..74605c38cfd7e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java @@ -3,24 +3,23 @@ * 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.clients.producer; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - import org.apache.kafka.clients.producer.internals.Metadata; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.test.TestUtils; import org.junit.Test; +import static org.junit.Assert.*; + public class MetadataTest { private long refreshBackoffMs = 100; @@ -49,13 +48,42 @@ public void testMetadata() throws Exception { assertTrue("Update needed due to stale metadata.", metadata.timeToNextUpdate(time) == 0); } + /** + * Tests that {@link org.apache.kafka.clients.producer.internals.Metadata#awaitUpdate(int, long)} doesn't + * wait forever with a max timeout value of 0 + * + * @throws Exception + * @see https://issues.apache.org/jira/browse/KAFKA-1836 + */ + @Test + public void testMetadataUpdateWaitTime() throws Exception { + long time = 0; + metadata.update(Cluster.empty(), time); + assertFalse("No update needed.", metadata.timeToNextUpdate(time) == 0); + // first try with a max wait time of 0 and ensure that this returns back without waiting forever + try { + metadata.awaitUpdate(metadata.requestUpdate(), 0); + fail("Wait on metadata update was expected to timeout, but it didn't"); + } catch (TimeoutException te) { + // expected + } + // now try with a higher timeout value once + final long TWO_SECOND_WAIT = 2000; + try { + metadata.awaitUpdate(metadata.requestUpdate(), TWO_SECOND_WAIT); + fail("Wait on metadata update was expected to timeout, but it didn't"); + } catch (TimeoutException te) { + // expected + } + } + private Thread asyncFetch(final String topic) { Thread thread = new Thread() { public void run() { while (metadata.fetch().partitionsForTopic(topic) == null) { try { metadata.awaitUpdate(metadata.requestUpdate(), refreshBackoffMs); - } catch(TimeoutException e) { + } catch (TimeoutException e) { // let it go } } @@ -64,5 +92,4 @@ public void run() { thread.start(); return thread; } - } From 14779dddb6a9bb3aefbaa825a62874f89bb47d2c Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Mon, 12 Jan 2015 21:12:26 -0800 Subject: [PATCH 086/491] KAFKA-1819 Cleaner gets confused about deleted and re-created topics; reviewed by Neha Narkhede --- .../src/main/scala/kafka/log/LogCleaner.scala | 11 +++- .../scala/kafka/log/LogCleanerManager.scala | 17 +++-- .../src/main/scala/kafka/log/LogManager.scala | 9 ++- .../unit/kafka/admin/DeleteTopicTest.scala | 64 ++++++++++++++++++- .../kafka/log/LogCleanerIntegrationTest.scala | 14 ++++ 5 files changed, 103 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index f8fcb843c80ee..f8e7cd5fabce7 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -71,8 +71,8 @@ class LogCleaner(val config: CleanerConfig, val logs: Pool[TopicAndPartition, Log], time: Time = SystemTime) extends Logging with KafkaMetricsGroup { - /* for managing the state of partitions being cleaned. */ - private val cleanerManager = new LogCleanerManager(logDirs, logs); + /* for managing the state of partitions being cleaned. package-private to allow access in tests */ + private[log] val cleanerManager = new LogCleanerManager(logDirs, logs); /* a throttle used to limit the I/O of all the cleaner threads to a user-specified maximum rate */ private val throttler = new Throttler(desiredRatePerSec = config.maxIoBytesPerSecond, @@ -129,6 +129,13 @@ class LogCleaner(val config: CleanerConfig, cleanerManager.abortCleaning(topicAndPartition) } + /** + * Update checkpoint file, removing topics and partitions that no longer exist + */ + def updateCheckpoints(dataDir: File) { + cleanerManager.updateCheckpoints(dataDir, update=None); + } + /** * Abort the cleaning of a particular partition if it's in progress, and pause any future cleaning of this partition. * This call blocks until the cleaning of the partition is aborted and paused. diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala index bcfef77ed53f9..fd87d90597981 100644 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -44,9 +44,12 @@ private[log] case object LogCleaningPaused extends LogCleaningState private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[TopicAndPartition, Log]) extends Logging with KafkaMetricsGroup { override val loggerName = classOf[LogCleaner].getName + + // package-private for testing + private[log] val offsetCheckpointFile = "cleaner-offset-checkpoint" /* the offset checkpoints holding the last cleaned point for each log */ - private val checkpoints = logDirs.map(dir => (dir, new OffsetCheckpoint(new File(dir, "cleaner-offset-checkpoint")))).toMap + private val checkpoints = logDirs.map(dir => (dir, new OffsetCheckpoint(new File(dir, offsetCheckpointFile)))).toMap /* the set of logs currently being cleaned */ private val inProgress = mutable.HashMap[TopicAndPartition, LogCleaningState]() @@ -199,6 +202,14 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To } } + def updateCheckpoints(dataDir: File, update: Option[(TopicAndPartition,Long)]) { + inLock(lock) { + val checkpoint = checkpoints(dataDir) + val existing = checkpoint.read().filterKeys(logs.keys) ++ update + checkpoint.write(existing) + } + } + /** * Save out the endOffset and remove the given log from the in-progress set, if not aborted. */ @@ -206,9 +217,7 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To inLock(lock) { inProgress(topicAndPartition) match { case LogCleaningInProgress => - val checkpoint = checkpoints(dataDir) - val offsets = checkpoint.read() + ((topicAndPartition, endOffset)) - checkpoint.write(offsets) + updateCheckpoints(dataDir,Option(topicAndPartition, endOffset)) inProgress.remove(topicAndPartition) case LogCleaningAborted => inProgress.put(topicAndPartition, LogCleaningPaused) diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 4ebaae00ca4b8..47d250af62c1a 100644 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -57,8 +57,9 @@ class LogManager(val logDirs: Array[File], private val dirLocks = lockLogDirs(logDirs) private val recoveryPointCheckpoints = logDirs.map(dir => (dir, new OffsetCheckpoint(new File(dir, RecoveryPointCheckpointFile)))).toMap loadLogs() - - private val cleaner: LogCleaner = + + // public, so we can access this from kafka.admin.DeleteTopicTest + val cleaner: LogCleaner = if(cleanerConfig.enableCleaner) new LogCleaner(cleanerConfig, logDirs, logs, time = time) else @@ -370,8 +371,10 @@ class LogManager(val logDirs: Array[File], } if (removedLog != null) { //We need to wait until there is no more cleaning task on the log to be deleted before actually deleting it. - if (cleaner != null) + if (cleaner != null) { cleaner.abortCleaning(topicAndPartition) + cleaner.updateCheckpoints(removedLog.dir.getParentFile) + } removedLog.delete() info("Deleted log for partition [%s,%d] in %s." .format(topicAndPartition.topic, diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala index 29cc01bcef9ca..33c27678bf8ae 100644 --- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala @@ -16,11 +16,14 @@ */ package kafka.admin +import java.io.File + +import kafka.log.Log import org.scalatest.junit.JUnit3Suite import kafka.zk.ZooKeeperTestHarness import junit.framework.Assert._ import kafka.utils.{ZkUtils, TestUtils} -import kafka.server.{KafkaServer, KafkaConfig} +import kafka.server.{OffsetCheckpoint, KafkaServer, KafkaConfig} import org.junit.Test import kafka.common._ import kafka.producer.{ProducerConfig, Producer} @@ -221,14 +224,50 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { val leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000) assertTrue("Leader should exist for topic test", leaderIdOpt.isDefined) servers.foreach(_.shutdown()) + } + + @Test + def testDeleteTopicWithCleaner() { + val topicName = "test" + val topicAndPartition = TopicAndPartition(topicName, 0) + val topic = topicAndPartition.topic + + val brokerConfigs = TestUtils.createBrokerConfigs(3, false) + brokerConfigs(0).setProperty("delete.topic.enable", "true") + brokerConfigs(0).setProperty("log.cleaner.enable","true") + brokerConfigs(0).setProperty("log.cleanup.policy","compact") + brokerConfigs(0).setProperty("log.segment.bytes","100") + brokerConfigs(0).setProperty("log.segment.delete.delay.ms","1000") + val servers = createTestTopicAndCluster(topic,brokerConfigs) + + // for simplicity, we are validating cleaner offsets on a single broker + val server = servers(0) + val log = server.logManager.getLog(topicAndPartition).get + + // write to the topic to activate cleaner + writeDups(numKeys = 100, numDups = 3,log) + + // wait for cleaner to clean + server.logManager.cleaner.awaitCleaned(topicName,0,0) + // delete topic + AdminUtils.deleteTopic(zkClient, "test") + verifyTopicDeletion("test", servers) + + servers.foreach(_.shutdown()) } private def createTestTopicAndCluster(topic: String): Seq[KafkaServer] = { + + val brokerConfigs = TestUtils.createBrokerConfigs(3, false) + brokerConfigs.foreach(p => p.setProperty("delete.topic.enable", "true") + ) + createTestTopicAndCluster(topic,brokerConfigs) + } + + private def createTestTopicAndCluster(topic: String, brokerConfigs: Seq[Properties]): Seq[KafkaServer] = { val expectedReplicaAssignment = Map(0 -> List(0, 1, 2)) val topicAndPartition = TopicAndPartition(topic, 0) - val brokerConfigs = TestUtils.createBrokerConfigs(3, false) - brokerConfigs.foreach(p => p.setProperty("delete.topic.enable", "true")) // create brokers val servers = brokerConfigs.map(b => TestUtils.createServer(new KafkaConfig(b))) // create the topic @@ -253,5 +292,24 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { // ensure that logs from all replicas are deleted if delete topic is marked successful in zookeeper assertTrue("Replica logs not deleted after delete topic is complete", servers.foldLeft(true)((res, server) => res && server.getLogManager().getLog(topicAndPartition).isEmpty)) + // ensure that topic is removed from all cleaner offsets + TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res,server) => res && + { + val topicAndPartition = TopicAndPartition(topic,0) + val logdir = server.getLogManager().logDirs(0) + val checkpoints = new OffsetCheckpoint(new File(logdir,"cleaner-offset-checkpoint")).read() + !checkpoints.contains(topicAndPartition) + }), + "Cleaner offset for deleted partition should have been removed") + } + + private def writeDups(numKeys: Int, numDups: Int, log: Log): Seq[(Int, Int)] = { + var counter = 0 + for(dup <- 0 until numDups; key <- 0 until numKeys) yield { + val count = counter + log.append(TestUtils.singleMessageSet(payload = counter.toString.getBytes, key = key.toString.getBytes), assignOffsets = true) + counter += 1 + (key, count) + } } } diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 5bfa764638e92..07acd460b1259 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -18,6 +18,8 @@ package kafka.log import java.io.File +import kafka.server.OffsetCheckpoint + import scala.collection._ import org.junit._ import kafka.common.TopicAndPartition @@ -62,6 +64,18 @@ class LogCleanerIntegrationTest extends JUnitSuite { cleaner.awaitCleaned("log", 0, lastCleaned2) val read2 = readFromLog(log) assertEquals("Contents of the map shouldn't change.", appends2.toMap, read2.toMap) + + // simulate deleting a partition, by removing it from logs + // force a checkpoint + // and make sure its gone from checkpoint file + + cleaner.logs.remove(topics(0)) + + cleaner.updateCheckpoints(logDir) + val checkpoints = new OffsetCheckpoint(new File(logDir,cleaner.cleanerManager.offsetCheckpointFile)).read() + + // we expect partition 0 to be gone + assert(!checkpoints.contains(topics(0))) cleaner.shutdown() } From 6f4dea9dbce5cc9f69a917182981b41a56a98a85 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Mon, 12 Jan 2015 18:45:45 -0800 Subject: [PATCH 087/491] kafka-1797; (missed parametric in a few files) add the serializer/deserializer api to the new java client; patched by Jun Rao --- .../org/apache/kafka/clients/producer/KafkaProducer.java | 2 +- .../apache/kafka/clients/producer/MockProducerTest.java | 9 +++------ core/src/main/scala/kafka/tools/MirrorMaker.scala | 2 +- 3 files changed, 5 insertions(+), 8 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index a61c56c01c6c6..d3abeb19b5674 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -320,7 +320,7 @@ public Future send(ProducerRecord record, Callback callback " to class " + producerConfig.getClass(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG).getName() + " specified in value.serializer"); } - ProducerRecord serializedRecord = new ProducerRecord(record.topic(), record.partition(), serializedKey, serializedValue); + ProducerRecord serializedRecord = new ProducerRecord(record.topic(), record.partition(), serializedKey, serializedValue); int partition = partitioner.partition(serializedRecord, metadata.fetch()); int serializedSize = Records.LOG_OVERHEAD + Record.recordSize(serializedKey, serializedValue); ensureValidRecordSize(serializedSize); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java index 1e2ca03fafa36..3676b05eb8b83 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java @@ -25,9 +25,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; -import org.apache.kafka.clients.producer.MockProducer; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.clients.producer.RecordMetadata; import org.junit.Test; public class MockProducerTest { @@ -37,7 +34,7 @@ public class MockProducerTest { @Test public void testAutoCompleteMock() throws Exception { MockProducer producer = new MockProducer(true); - ProducerRecord record = new ProducerRecord(topic, "key".getBytes(), "value".getBytes()); + ProducerRecord record = new ProducerRecord(topic, "key".getBytes(), "value".getBytes()); Future metadata = producer.send(record); assertTrue("Send should be immediately complete", metadata.isDone()); assertFalse("Send should be successful", isError(metadata)); @@ -51,8 +48,8 @@ public void testAutoCompleteMock() throws Exception { @Test public void testManualCompletion() throws Exception { MockProducer producer = new MockProducer(false); - ProducerRecord record1 = new ProducerRecord("topic", "key1".getBytes(), "value1".getBytes()); - ProducerRecord record2 = new ProducerRecord("topic", "key2".getBytes(), "value2".getBytes()); + ProducerRecord record1 = new ProducerRecord("topic", "key1".getBytes(), "value1".getBytes()); + ProducerRecord record2 = new ProducerRecord("topic", "key2".getBytes(), "value2".getBytes()); Future md1 = producer.send(record1); assertFalse("Send shouldn't have completed", md1.isDone()); Future md2 = producer.send(record2); diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index e302523212753..5cbc8103e33a0 100644 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -534,7 +534,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { val producer = new KafkaProducer[Array[Byte], Array[Byte]](producerProps) override def send(sourceTopicPartition: TopicAndPartition, sourceOffset: Long, key: Array[Byte], value: Array[Byte]) { - val record = new ProducerRecord(sourceTopicPartition.topic, key, value) + val record = new ProducerRecord[Array[Byte], Array[Byte]](sourceTopicPartition.topic, key, value) if(sync) { this.producer.send(record).get() unackedOffsetsMap.getAndMaybePut(sourceTopicPartition).maybeUpdateMaxOffsetSeen(sourceOffset) From 688e38ce45a7358a1e0bb359aa9b1a698a841619 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Mon, 12 Jan 2015 22:01:43 -0800 Subject: [PATCH 088/491] KAFKA-1723; make the metrics name in new producer more standard; patched by Manikumar Reddy; reviewed by Jay Kreps and Jun Rao --- build.gradle | 1 + .../kafka/clients/consumer/Consumer.java | 5 +- .../kafka/clients/consumer/KafkaConsumer.java | 3 +- .../kafka/clients/consumer/MockConsumer.java | 3 +- .../kafka/clients/producer/KafkaProducer.java | 19 +- .../kafka/clients/producer/MockProducer.java | 7 +- .../kafka/clients/producer/Producer.java | 4 +- .../producer/internals/BufferPool.java | 28 ++- .../producer/internals/RecordAccumulator.java | 25 ++- .../clients/producer/internals/Sender.java | 81 +++++--- .../java/org/apache/kafka/common/Metric.java | 9 +- .../org/apache/kafka/common/MetricName.java | 179 ++++++++++++++++++ .../kafka/common/metrics/CompoundStat.java | 14 +- .../kafka/common/metrics/JmxReporter.java | 60 +++--- .../kafka/common/metrics/KafkaMetric.java | 18 +- .../apache/kafka/common/metrics/Metrics.java | 58 ++---- .../apache/kafka/common/metrics/Sensor.java | 51 ++--- .../common/metrics/stats/Percentile.java | 18 +- .../common/metrics/stats/Percentiles.java | 2 +- .../apache/kafka/common/network/Selector.java | 93 +++++---- .../clients/producer/BufferPoolTest.java | 14 +- .../producer/RecordAccumulatorTest.java | 16 +- .../kafka/clients/producer/SenderTest.java | 11 +- .../kafka/common/metrics/JmxReporterTest.java | 13 +- .../kafka/common/metrics/MetricsTest.java | 87 +++++---- .../kafka/common/network/SelectorTest.java | 3 +- .../org/apache/kafka/test/MetricsBench.java | 11 +- 27 files changed, 533 insertions(+), 300 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/MetricName.java diff --git a/build.gradle b/build.gradle index ba52288031e2a..c9ac43378c3bf 100644 --- a/build.gradle +++ b/build.gradle @@ -370,6 +370,7 @@ project(':clients') { javadoc { include "**/org/apache/kafka/clients/producer/*" + include "**/org/apache/kafka/common/*" include "**/org/apache/kafka/common/errors/*" include "**/org/apache/kafka/common/serialization/*" } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java index 1bce50185273d..c0c636b3e1ba2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.Metric; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.MetricName; /** * @see KafkaConsumer @@ -111,11 +112,11 @@ public interface Consumer extends Closeable { * @return The offsets for messages that were written to the server before the specified timestamp. */ public Map offsetsBeforeTime(long timestamp, Collection partitions); - + /** * Return a map of metrics maintained by the consumer */ - public Map metrics(); + public Map metrics(); /** * Close this consumer 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 a5fedce9ff05c..76efc216c9e6c 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 @@ -16,6 +16,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.MetricsReporter; import org.apache.kafka.common.serialization.Deserializer; @@ -627,7 +628,7 @@ public Map offsetsBeforeTime(long timestamp, Collection metrics() { + public Map metrics() { return Collections.unmodifiableMap(this.metrics.metrics()); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java index 8cab16c0a0bdb..fa88ac1a8b19b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java @@ -25,6 +25,7 @@ import java.util.Set; import org.apache.kafka.common.Metric; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.MetricName; /** * A mock of the {@link Consumer} interface you can use for testing code that uses Kafka. @@ -179,7 +180,7 @@ public Map offsetsBeforeTime(long timestamp, } @Override - public Map metrics() { + public Map metrics() { return null; } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index d3abeb19b5674..c79149a715fcb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -17,6 +17,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.clients.producer.internals.Metadata; @@ -35,6 +36,7 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.MetricsReporter; import org.apache.kafka.common.metrics.Sensor; @@ -77,6 +79,7 @@ public class KafkaProducer implements Producer { private final Serializer keySerializer; private final Serializer valueSerializer; private final ProducerConfig producerConfig; + private static final AtomicInteger producerAutoId = new AtomicInteger(1); /** * A producer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings @@ -159,7 +162,9 @@ private KafkaProducer(ProducerConfig config, Serializer keySerializer, Serial .timeWindow(config.getLong(ProducerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG), TimeUnit.MILLISECONDS); String clientId = config.getString(ProducerConfig.CLIENT_ID_CONFIG); - String jmxPrefix = "kafka.producer." + (clientId.length() > 0 ? clientId + "." : ""); + if(clientId.length() <= 0) + clientId = "producer-" + producerAutoId.getAndIncrement(); + String jmxPrefix = "kafka.producer"; List reporters = config.getConfiguredInstances(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG, MetricsReporter.class); reporters.add(new JmxReporter(jmxPrefix)); @@ -171,17 +176,20 @@ private KafkaProducer(ProducerConfig config, Serializer keySerializer, Serial this.maxRequestSize = config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG); this.totalMemorySize = config.getLong(ProducerConfig.BUFFER_MEMORY_CONFIG); this.compressionType = CompressionType.forName(config.getString(ProducerConfig.COMPRESSION_TYPE_CONFIG)); + Map metricTags = new LinkedHashMap(); + metricTags.put("client-id", clientId); this.accumulator = new RecordAccumulator(config.getInt(ProducerConfig.BATCH_SIZE_CONFIG), this.totalMemorySize, config.getLong(ProducerConfig.LINGER_MS_CONFIG), retryBackoffMs, config.getBoolean(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG), metrics, - time); + time, + metricTags); List addresses = ClientUtils.parseAndValidateAddresses(config.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); this.metadata.update(Cluster.bootstrap(addresses), time.milliseconds()); - NetworkClient client = new NetworkClient(new Selector(this.metrics, time), + NetworkClient client = new NetworkClient(new Selector(this.metrics, time , "producer", metricTags), this.metadata, clientId, config.getInt(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION), @@ -196,7 +204,8 @@ private KafkaProducer(ProducerConfig config, Serializer keySerializer, Serial config.getInt(ProducerConfig.RETRIES_CONFIG), config.getInt(ProducerConfig.TIMEOUT_CONFIG), this.metrics, - new SystemTime()); + new SystemTime(), + clientId); String ioThreadName = "kafka-producer-network-thread" + (clientId.length() > 0 ? " | " + clientId : ""); this.ioThread = new KafkaThread(ioThreadName, this.sender, true); this.ioThread.start(); @@ -398,7 +407,7 @@ public List partitionsFor(String topic) { } @Override - public Map metrics() { + public Map metrics() { return Collections.unmodifiableMap(this.metrics.metrics()); } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java index 34624c3b7a1f2..904976fadf061 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java @@ -28,10 +28,7 @@ import org.apache.kafka.clients.producer.internals.FutureRecordMetadata; import org.apache.kafka.clients.producer.internals.Partitioner; import org.apache.kafka.clients.producer.internals.ProduceRequestResult; -import org.apache.kafka.common.Cluster; -import org.apache.kafka.common.Metric; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.*; /** @@ -136,7 +133,7 @@ public List partitionsFor(String topic) { return this.cluster.partitionsForTopic(topic); } - public Map metrics() { + public Map metrics() { return Collections.emptyMap(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java index 5baa6062bd9ba..6b2471f878b7d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java @@ -23,11 +23,11 @@ import org.apache.kafka.common.Metric; import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.MetricName; /** * The interface for the {@link KafkaProducer} - * * @see KafkaProducer * @see MockProducer */ @@ -55,7 +55,7 @@ public interface Producer extends Closeable { /** * Return a map of metrics maintained by the producer */ - public Map metrics(); + public Map metrics(); /** * Close this producer diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java index aa91e1444a49c..8d4156d17e949 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java @@ -16,19 +16,21 @@ */ package org.apache.kafka.clients.producer.internals; -import org.apache.kafka.clients.producer.BufferExhaustedException; -import org.apache.kafka.common.metrics.Metrics; -import org.apache.kafka.common.metrics.Sensor; -import org.apache.kafka.common.metrics.stats.Rate; -import org.apache.kafka.common.utils.Time; - import java.nio.ByteBuffer; import java.util.ArrayDeque; import java.util.Deque; +import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; +import org.apache.kafka.clients.producer.BufferExhaustedException; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.Rate; +import org.apache.kafka.common.utils.Time; + /** * A pool of ByteBuffers kept under a given memory limit. This class is fairly specific to the needs of the producer. In @@ -61,8 +63,12 @@ public final class BufferPool { * @param blockOnExhaustion This controls the behavior when the buffer pool is out of memory. If true the * {@link #allocate(int)} call will block and wait for memory to be returned to the pool. If false * {@link #allocate(int)} will throw an exception if the buffer is out of memory. + * @param metrics instance of Metrics + * @param time time instance + * @param metricGrpName logical group name for metrics + * @param metricTags additional key/val attributes for metrics */ - public BufferPool(long memory, int poolableSize, boolean blockOnExhaustion, Metrics metrics, Time time) { + public BufferPool(long memory, int poolableSize, boolean blockOnExhaustion, Metrics metrics, Time time , String metricGrpName , Map metricTags) { this.poolableSize = poolableSize; this.blockOnExhaustion = blockOnExhaustion; this.lock = new ReentrantLock(); @@ -73,9 +79,11 @@ public BufferPool(long memory, int poolableSize, boolean blockOnExhaustion, Metr this.metrics = metrics; this.time = time; this.waitTime = this.metrics.sensor("bufferpool-wait-time"); - this.waitTime.add("bufferpool-wait-ratio", - "The fraction of time an appender waits for space allocation.", - new Rate(TimeUnit.NANOSECONDS)); + MetricName metricName = new MetricName("bufferpool-wait-ratio", + metricGrpName, + "The fraction of time an appender waits for space allocation.", + metricTags); + this.waitTime.add(metricName, new Rate(TimeUnit.NANOSECONDS)); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index c15485d1af304..50889e4ce4b6c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -31,6 +31,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.Measurable; import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; @@ -76,6 +77,7 @@ public final class RecordAccumulator { * memory * @param metrics The metrics * @param time The time instance to use + * @param metricTags additional key/value attributes of the metric */ public RecordAccumulator(int batchSize, long totalSize, @@ -83,35 +85,38 @@ public RecordAccumulator(int batchSize, long retryBackoffMs, boolean blockOnBufferFull, Metrics metrics, - Time time) { + Time time, + Map metricTags) { this.drainIndex = 0; this.closed = false; this.batchSize = batchSize; this.lingerMs = lingerMs; this.retryBackoffMs = retryBackoffMs; this.batches = new CopyOnWriteMap>(); - this.free = new BufferPool(totalSize, batchSize, blockOnBufferFull, metrics, time); + String metricGrpName = "producer-metrics"; + this.free = new BufferPool(totalSize, batchSize, blockOnBufferFull, metrics, time , metricGrpName , metricTags); this.time = time; - registerMetrics(metrics); + registerMetrics(metrics, metricGrpName, metricTags); } - private void registerMetrics(Metrics metrics) { - metrics.addMetric("waiting-threads", - "The number of user threads blocked waiting for buffer memory to enqueue their records", + private void registerMetrics(Metrics metrics, String metricGrpName, Map metricTags) { + + MetricName metricName = new MetricName("waiting-threads", metricGrpName, "The number of user threads blocked waiting for buffer memory to enqueue their records", metricTags); + metrics.addMetric(metricName, new Measurable() { public double measure(MetricConfig config, long now) { return free.queued(); } }); - metrics.addMetric("buffer-total-bytes", - "The maximum amount of buffer memory the client can use (whether or not it is currently used).", + metricName = new MetricName("buffer-total-bytes", metricGrpName, "The maximum amount of buffer memory the client can use (whether or not it is currently used).", metricTags); + metrics.addMetric(metricName, new Measurable() { public double measure(MetricConfig config, long now) { return free.totalMemory(); } }); - metrics.addMetric("buffer-available-bytes", - "The total amount of buffer memory that is not being used (either unallocated or in the free list).", + metricName = new MetricName("buffer-available-bytes", metricGrpName, "The total amount of buffer memory that is not being used (either unallocated or in the free list).", metricTags); + metrics.addMetric(metricName, new Measurable() { public double measure(MetricConfig config, long now) { return free.availableMemory(); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 84a7a07269c51..ccc03d8447ebb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -16,6 +16,7 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -29,6 +30,7 @@ import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.metrics.Measurable; import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.stats.Avg; @@ -82,6 +84,9 @@ public class Sender implements Runnable { /* metrics */ private final SenderMetrics sensors; + /* param clientId of the client */ + private String clientId; + public Sender(KafkaClient client, Metadata metadata, RecordAccumulator accumulator, @@ -90,7 +95,8 @@ public Sender(KafkaClient client, int retries, int requestTimeout, Metrics metrics, - Time time) { + Time time, + String clientId) { this.client = client; this.accumulator = accumulator; this.metadata = metadata; @@ -100,6 +106,7 @@ public Sender(KafkaClient client, this.acks = acks; this.retries = retries; this.time = time; + this.clientId = clientId; this.sensors = new SenderMetrics(metrics); } @@ -324,46 +331,60 @@ private class SenderMetrics { public SenderMetrics(Metrics metrics) { this.metrics = metrics; + Map metricTags = new LinkedHashMap(); + metricTags.put("client-id", clientId); + String metricGrpName = "producer-metrics"; this.batchSizeSensor = metrics.sensor("batch-size"); - this.batchSizeSensor.add("batch-size-avg", "The average number of bytes sent per partition per-request.", new Avg()); - this.batchSizeSensor.add("batch-size-max", "The max number of bytes sent per partition per-request.", new Max()); + MetricName m = new MetricName("batch-size-avg", metricGrpName, "The average number of bytes sent per partition per-request.", metricTags); + this.batchSizeSensor.add(m, new Avg()); + m = new MetricName("batch-size-max", metricGrpName, "The max number of bytes sent per partition per-request.", metricTags); + this.batchSizeSensor.add(m, new Max()); this.compressionRateSensor = metrics.sensor("compression-rate"); - this.compressionRateSensor.add("compression-rate-avg", "The average compression rate of record batches.", new Avg()); + m = new MetricName("compression-rate-avg", metricGrpName, "The average compression rate of record batches.", metricTags); + this.compressionRateSensor.add(m, new Avg()); this.queueTimeSensor = metrics.sensor("queue-time"); - this.queueTimeSensor.add("record-queue-time-avg", - "The average time in ms record batches spent in the record accumulator.", - new Avg()); - this.queueTimeSensor.add("record-queue-time-max", - "The maximum time in ms record batches spent in the record accumulator.", - new Max()); + m = new MetricName("record-queue-time-avg", metricGrpName, "The average time in ms record batches spent in the record accumulator.", metricTags); + this.queueTimeSensor.add(m, new Avg()); + m = new MetricName("record-queue-time-max", metricGrpName, "The maximum time in ms record batches spent in the record accumulator.", metricTags); + this.queueTimeSensor.add(m, new Max()); this.requestTimeSensor = metrics.sensor("request-time"); - this.requestTimeSensor.add("request-latency-avg", "The average request latency in ms", new Avg()); - this.requestTimeSensor.add("request-latency-max", "The maximum request latency in ms", new Max()); + m = new MetricName("request-latency-avg", metricGrpName, "The average request latency in ms", metricTags); + this.requestTimeSensor.add(m, new Avg()); + m = new MetricName("request-latency-max", metricGrpName, "The maximum request latency in ms", metricTags); + this.requestTimeSensor.add(m, new Max()); this.recordsPerRequestSensor = metrics.sensor("records-per-request"); - this.recordsPerRequestSensor.add("record-send-rate", "The average number of records sent per second.", new Rate()); - this.recordsPerRequestSensor.add("records-per-request-avg", "The average number of records per request.", new Avg()); + m = new MetricName("record-send-rate", metricGrpName, "The average number of records sent per second.", metricTags); + this.recordsPerRequestSensor.add(m, new Rate()); + m = new MetricName("records-per-request-avg", metricGrpName, "The average number of records per request.", metricTags); + this.recordsPerRequestSensor.add(m, new Avg()); this.retrySensor = metrics.sensor("record-retries"); - this.retrySensor.add("record-retry-rate", "The average per-second number of retried record sends", new Rate()); + m = new MetricName("record-retry-rate", metricGrpName, "The average per-second number of retried record sends", metricTags); + this.retrySensor.add(m, new Rate()); this.errorSensor = metrics.sensor("errors"); - this.errorSensor.add("record-error-rate", "The average per-second number of record sends that resulted in errors", new Rate()); + m = new MetricName("record-error-rate", metricGrpName, "The average per-second number of record sends that resulted in errors", metricTags); + this.errorSensor.add(m, new Rate()); this.maxRecordSizeSensor = metrics.sensor("record-size-max"); - this.maxRecordSizeSensor.add("record-size-max", "The maximum record size", new Max()); - this.maxRecordSizeSensor.add("record-size-avg", "The average record size", new Avg()); + m = new MetricName("record-size-max", metricGrpName, "The maximum record size", metricTags); + this.maxRecordSizeSensor.add(m, new Max()); + m = new MetricName("record-size-avg", metricGrpName, "The average record size", metricTags); + this.maxRecordSizeSensor.add(m, new Avg()); - this.metrics.addMetric("requests-in-flight", "The current number of in-flight requests awaiting a response.", new Measurable() { + m = new MetricName("requests-in-flight", metricGrpName, "The current number of in-flight requests awaiting a response.", metricTags); + this.metrics.addMetric(m, new Measurable() { public double measure(MetricConfig config, long now) { return client.inFlightRequestCount(); } }); - metrics.addMetric("metadata-age", "The age in seconds of the current producer metadata being used.", new Measurable() { + m = new MetricName("metadata-age", metricGrpName, "The age in seconds of the current producer metadata being used.", metricTags); + metrics.addMetric(m, new Measurable() { public double measure(MetricConfig config, long now) { return (now - metadata.lastUpdate()) / 1000.0; } @@ -376,24 +397,34 @@ public void maybeRegisterTopicMetrics(String topic) { String topicRecordsCountName = "topic." + topic + ".records-per-batch"; Sensor topicRecordCount = this.metrics.getSensor(topicRecordsCountName); if (topicRecordCount == null) { + Map metricTags = new LinkedHashMap(); + metricTags.put("client-id", clientId); + metricTags.put("topic", topic); + String metricGrpName = "producer-topic-metrics"; + topicRecordCount = this.metrics.sensor(topicRecordsCountName); - topicRecordCount.add("topic." + topic + ".record-send-rate", new Rate()); + MetricName m = new MetricName("record-send-rate", metricGrpName , metricTags); + topicRecordCount.add(m, new Rate()); String topicByteRateName = "topic." + topic + ".bytes"; Sensor topicByteRate = this.metrics.sensor(topicByteRateName); - topicByteRate.add("topic." + topic + ".byte-rate", new Rate()); + m = new MetricName("byte-rate", metricGrpName , metricTags); + topicByteRate.add(m, new Rate()); String topicCompressionRateName = "topic." + topic + ".compression-rate"; Sensor topicCompressionRate = this.metrics.sensor(topicCompressionRateName); - topicCompressionRate.add("topic." + topic + ".compression-rate", new Avg()); + m = new MetricName("compression-rate", metricGrpName , metricTags); + topicCompressionRate.add(m, new Avg()); String topicRetryName = "topic." + topic + ".record-retries"; Sensor topicRetrySensor = this.metrics.sensor(topicRetryName); - topicRetrySensor.add("topic." + topic + ".record-retry-rate", new Rate()); + m = new MetricName("record-retry-rate", metricGrpName , metricTags); + topicRetrySensor.add(m, new Rate()); String topicErrorName = "topic." + topic + ".record-errors"; Sensor topicErrorSensor = this.metrics.sensor(topicErrorName); - topicErrorSensor.add("topic." + topic + ".record-error-rate", new Rate()); + m = new MetricName("record-error-rate", metricGrpName , metricTags); + topicErrorSensor.add(m, new Rate()); } } diff --git a/clients/src/main/java/org/apache/kafka/common/Metric.java b/clients/src/main/java/org/apache/kafka/common/Metric.java index b023e8e7c486a..d4ef77e36bbea 100644 --- a/clients/src/main/java/org/apache/kafka/common/Metric.java +++ b/clients/src/main/java/org/apache/kafka/common/Metric.java @@ -22,14 +22,9 @@ public interface Metric { /** - * A unique name for this metric + * A name for this metric */ - public String name(); - - /** - * A description of what is measured...this will be "" if no description was given - */ - public String description(); + public MetricName metricName(); /** * The value of the metric diff --git a/clients/src/main/java/org/apache/kafka/common/MetricName.java b/clients/src/main/java/org/apache/kafka/common/MetricName.java new file mode 100644 index 0000000000000..4e810d56b753b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/MetricName.java @@ -0,0 +1,179 @@ +/** + * 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; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.kafka.common.utils.Utils; + +/** + * The MetricName class encapsulates a metric's name, logical group and its related attributes + *

          + * This class captures the following parameters + *

          + *  name The name of the metric
          + *  group logical group name of the metrics to which this metric belongs.
          + *  description A human-readable description to include in the metric. This is optional.
          + *  tags additional key/value attributes of the metric. This is optional.
          + *   
          + * group, tags parameters can be used to create unique metric names while reporting in JMX or any custom reporting. + * + * Ex: standard JMX MBean can be constructed like domainName:type=group,key1=val1,key2=val2 + * + * Usage looks something like this: + *
          + * // set up metrics:
          + * Metrics metrics = new Metrics(); // this is the global repository of metrics and sensors
          + * Sensor sensor = metrics.sensor("message-sizes");
          + * Map metricTags = new LinkedHashMap();
          + * metricTags.put("client-id", "producer-1");
          + * metricTags.put("topic", "topic");
          + * MetricName metricName = new MetricName("message-size-avg", "producer-metrics", "average message size", metricTags);
          + * sensor.add(metricName, new Avg());
          + * metricName = new MetricName("message-size-max", "producer-metrics",metricTags);
          + * sensor.add(metricName, new Max());
          + *
          + * // as messages are sent we record the sizes
          + * sensor.record(messageSize);
          + * 
          + */ +public final class MetricName { + + private final String name; + private final String group; + private final String description; + private Map tags; + private int hash = 0; + + /** + * @param name The name of the metric + * @param group logical group name of the metrics to which this metric belongs + * @param description A human-readable description to include in the metric + * @param tags additional key/value attributes of the metric + */ + public MetricName(String name, String group, String description, Map tags) { + this.name = Utils.notNull(name); + this.group = Utils.notNull(group); + this.description = Utils.notNull(description); + this.tags = Utils.notNull(tags); + } + + /** + * @param name The name of the metric + * @param group logical group name of the metrics to which this metric belongs + * @param description A human-readable description to include in the metric + * @param keyValue additional key/value attributes of the metric (must come in pairs) + */ + public MetricName(String name, String group, String description, String... keyValue) { + this(name, group, description, getTags(keyValue)); + } + + private static Map getTags(String... keyValue) { + if ((keyValue.length % 2) != 0) + throw new IllegalArgumentException("keyValue needs to be specified in paris"); + Map tags = new HashMap(); + + for (int i=0; i<(keyValue.length / 2); i++) + tags.put(keyValue[i], keyValue[i+1]); + return tags; + } + + /** + * @param name The name of the metric + * @param group logical group name of the metrics to which this metric belongs + * @param tags key/value attributes of the metric + */ + public MetricName(String name, String group, Map tags) { + this(name, group, "", tags); + } + + /** + * @param name The name of the metric + * @param group logical group name of the metrics to which this metric belongs + * @param description A human-readable description to include in the metric + */ + public MetricName(String name, String group, String description) { + this(name, group, description, new HashMap()); + } + + /** + * @param name The name of the metric + * @param group logical group name of the metrics to which this metric belongs + */ + public MetricName(String name, String group) { + this(name, group, "", new HashMap()); + } + + public String name() { + return this.name; + } + + public String group() { + return this.group; + } + + public Map tags() { + return this.tags; + } + + public String description() { + return this.description; + } + + @Override + public int hashCode() { + if (hash != 0) + return hash; + final int prime = 31; + int result = 1; + result = prime * result + ((group == null) ? 0 : group.hashCode()); + result = prime * result + ((name == null) ? 0 : name.hashCode()); + result = prime * result + ((tags == null) ? 0 : tags.hashCode()); + this.hash = result; + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + MetricName other = (MetricName) obj; + if (group == null) { + if (other.group != null) + return false; + } else if (!group.equals(other.group)) + return false; + if (name == null) { + if (other.name != null) + return false; + } else if (!name.equals(other.name)) + return false; + if (tags == null) { + if (other.tags != null) + return false; + } else if (!tags.equals(other.tags)) + return false; + return true; + } + + @Override + public String toString() { + return "MetricName [name=" + name + ", group=" + group + ", description=" + + description + ", tags=" + tags + "]"; + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/CompoundStat.java b/clients/src/main/java/org/apache/kafka/common/metrics/CompoundStat.java index 29185a6a90d00..e0969aad858e4 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/CompoundStat.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/CompoundStat.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.common.metrics; +import org.apache.kafka.common.MetricName; + import java.util.List; /** @@ -28,25 +30,19 @@ public interface CompoundStat extends Stat { public static class NamedMeasurable { - private final String name; - private final String description; + private final MetricName name; private final Measurable stat; - public NamedMeasurable(String name, String description, Measurable stat) { + public NamedMeasurable(MetricName name, Measurable stat) { super(); this.name = name; - this.description = description; this.stat = stat; } - public String name() { + public MetricName name() { return name; } - public String description() { - return description; - } - public Measurable stat() { return stat; } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java b/clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java index 3c312011a7ff7..9c205387acc13 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java @@ -32,6 +32,7 @@ import javax.management.ReflectionException; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.MetricName; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -80,18 +81,39 @@ public void metricChange(KafkaMetric metric) { private KafkaMbean addAttribute(KafkaMetric metric) { try { - String[] names = split(prefix + metric.name()); - String qualifiedName = names[0] + "." + names[1]; - if (!this.mbeans.containsKey(qualifiedName)) - mbeans.put(qualifiedName, new KafkaMbean(names[0], names[1])); - KafkaMbean mbean = this.mbeans.get(qualifiedName); - mbean.setAttribute(names[2], metric); + MetricName metricName = metric.metricName(); + String mBeanName = getMBeanName(metricName); + if (!this.mbeans.containsKey(mBeanName)) + mbeans.put(mBeanName, new KafkaMbean(mBeanName)); + KafkaMbean mbean = this.mbeans.get(mBeanName); + mbean.setAttribute(metricName.name() , metric); return mbean; } catch (JMException e) { - throw new KafkaException("Error creating mbean attribute " + metric.name(), e); + throw new KafkaException("Error creating mbean attribute for metricName :" + metric.metricName(), e); } } + /** + * @param metricName + * @return standard JMX MBean name in the following format + * domainName:type=metricType,key1=val1,key2=val2 + */ + private String getMBeanName(MetricName metricName) { + StringBuilder mBeanName = new StringBuilder(); + mBeanName.append(prefix); + mBeanName.append(":type="); + mBeanName.append(metricName.group()); + for (Map.Entry entry : metricName.tags().entrySet()) { + if(entry.getKey().length() <= 0 || entry.getValue().length() <= 0) + continue; + mBeanName.append(","); + mBeanName.append(entry.getKey()); + mBeanName.append("="); + mBeanName.append(entry.getValue()); + } + return mBeanName.toString(); + } + public void close() { synchronized (lock) { for (KafkaMbean mbean : this.mbeans.values()) @@ -118,29 +140,13 @@ private void reregister(KafkaMbean mbean) { } } - private String[] split(String name) { - int attributeStart = name.lastIndexOf('.'); - if (attributeStart < 0) - throw new IllegalArgumentException("No MBean name in metric name: " + name); - String attributeName = name.substring(attributeStart + 1, name.length()); - String remainder = name.substring(0, attributeStart); - int beanStart = remainder.lastIndexOf('.'); - if (beanStart < 0) - return new String[] { "", remainder, attributeName }; - String packageName = remainder.substring(0, beanStart); - String beanName = remainder.substring(beanStart + 1, remainder.length()); - return new String[] { packageName, beanName, attributeName }; - } - private static class KafkaMbean implements DynamicMBean { - private final String beanName; private final ObjectName objectName; private final Map metrics; - public KafkaMbean(String packageName, String beanName) throws MalformedObjectNameException { - this.beanName = beanName; + public KafkaMbean(String mbeanName) throws MalformedObjectNameException { this.metrics = new HashMap(); - this.objectName = new ObjectName(packageName + ":type=" + beanName); + this.objectName = new ObjectName(mbeanName); } public ObjectName name() { @@ -179,10 +185,10 @@ public MBeanInfo getMBeanInfo() { for (Map.Entry entry : this.metrics.entrySet()) { String attribute = entry.getKey(); KafkaMetric metric = entry.getValue(); - attrs[i] = new MBeanAttributeInfo(attribute, double.class.getName(), metric.description(), true, false, false); + attrs[i] = new MBeanAttributeInfo(attribute, double.class.getName(), metric.metricName().description(), true, false, false); i += 1; } - return new MBeanInfo(beanName, "", attrs, null, null, null); + return new MBeanInfo(this.getClass().getName(), "", attrs, null, null, null); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java b/clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java index a7458b50cb16f..89df1a4ec3e68 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java @@ -17,21 +17,20 @@ package org.apache.kafka.common.metrics; import org.apache.kafka.common.Metric; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.utils.Time; public final class KafkaMetric implements Metric { - private final String name; - private final String description; + private MetricName metricName; private final Object lock; private final Time time; private final Measurable measurable; private MetricConfig config; - KafkaMetric(Object lock, String name, String description, Measurable measurable, MetricConfig config, Time time) { + KafkaMetric(Object lock, MetricName metricName, Measurable measurable, MetricConfig config, Time time) { super(); - this.name = name; - this.description = description; + this.metricName = metricName; this.lock = lock; this.measurable = measurable; this.config = config; @@ -43,13 +42,8 @@ MetricConfig config() { } @Override - public String name() { - return this.name; - } - - @Override - public String description() { - return this.description; + public MetricName metricName() { + return this.metricName; } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java b/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java index 49be4019ac038..b3d3d7c56acb4 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java @@ -17,6 +17,7 @@ import java.util.Map; import java.util.concurrent.ConcurrentMap; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.utils.CopyOnWriteMap; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; @@ -36,8 +37,10 @@ * // set up metrics: * Metrics metrics = new Metrics(); // this is the global repository of metrics and sensors * Sensor sensor = metrics.sensor("message-sizes"); - * sensor.add("kafka.producer.message-sizes.avg", new Avg()); - * sensor.add("kafka.producer.message-sizes.max", new Max()); + * MetricName metricName = new MetricName("message-size-avg", "producer-metrics"); + * sensor.add(metricName, new Avg()); + * metricName = new MetricName("message-size-max", "producer-metrics"); + * sensor.add(metricName, new Max()); * * // as messages are sent we record the sizes * sensor.record(messageSize); @@ -46,7 +49,7 @@ public class Metrics { private final MetricConfig config; - private final ConcurrentMap metrics; + private final ConcurrentMap metrics; private final ConcurrentMap sensors; private final List reporters; private final Time time; @@ -83,7 +86,7 @@ public Metrics(MetricConfig defaultConfig) { public Metrics(MetricConfig defaultConfig, List reporters, Time time) { this.config = defaultConfig; this.sensors = new CopyOnWriteMap(); - this.metrics = new CopyOnWriteMap(); + this.metrics = new CopyOnWriteMap(); this.reporters = Utils.notNull(reporters); this.time = time; for (MetricsReporter reporter : reporters) @@ -139,47 +142,23 @@ public synchronized Sensor sensor(String name, MetricConfig config, Sensor... pa /** * Add a metric to monitor an object that implements measurable. This metric won't be associated with any sensor. * This is a way to expose existing values as metrics. - * @param name The name of the metric + * @param metricName The name of the metric * @param measurable The measurable that will be measured by this metric */ - public void addMetric(String name, Measurable measurable) { - addMetric(name, "", measurable); + public void addMetric(MetricName metricName, Measurable measurable) { + addMetric(metricName, null, measurable); } /** * Add a metric to monitor an object that implements measurable. This metric won't be associated with any sensor. * This is a way to expose existing values as metrics. - * @param name The name of the metric - * @param description A human-readable description to include in the metric - * @param measurable The measurable that will be measured by this metric - */ - public void addMetric(String name, String description, Measurable measurable) { - addMetric(name, description, null, measurable); - } - - /** - * Add a metric to monitor an object that implements measurable. This metric won't be associated with any sensor. - * This is a way to expose existing values as metrics. - * @param name The name of the metric - * @param config The configuration to use when measuring this measurable - * @param measurable The measurable that will be measured by this metric - */ - public void addMetric(String name, MetricConfig config, Measurable measurable) { - addMetric(name, "", config, measurable); - } - - /** - * Add a metric to monitor an object that implements measurable. This metric won't be associated with any sensor. - * This is a way to expose existing values as metrics. - * @param name The name of the metric - * @param description A human-readable description to include in the metric + * @param metricName The name of the metric * @param config The configuration to use when measuring this measurable * @param measurable The measurable that will be measured by this metric */ - public synchronized void addMetric(String name, String description, MetricConfig config, Measurable measurable) { + public synchronized void addMetric(MetricName metricName, MetricConfig config, Measurable measurable) { KafkaMetric m = new KafkaMetric(new Object(), - Utils.notNull(name), - Utils.notNull(description), + Utils.notNull(metricName), Utils.notNull(measurable), config == null ? this.config : config, time); @@ -195,17 +174,18 @@ public synchronized void addReporter(MetricsReporter reporter) { } synchronized void registerMetric(KafkaMetric metric) { - if (this.metrics.containsKey(metric.name())) - throw new IllegalArgumentException("A metric named '" + metric.name() + "' already exists, can't register another one."); - this.metrics.put(metric.name(), metric); + MetricName metricName = metric.metricName(); + if (this.metrics.containsKey(metricName)) + throw new IllegalArgumentException("A metric named '" + metricName + "' already exists, can't register another one."); + this.metrics.put(metricName, metric); for (MetricsReporter reporter : reporters) reporter.metricChange(metric); } /** - * Get all the metrics currently maintained indexed by metric name + * Get all the metrics currently maintained indexed by metricName */ - public Map metrics() { + public Map metrics() { return this.metrics; } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java index 25c1faf2887ea..e53cfaa69f518 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java @@ -18,6 +18,7 @@ import java.util.List; import java.util.Set; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.CompoundStat.NamedMeasurable; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; @@ -112,7 +113,7 @@ private void checkQuotas(long timeMs) { Quota quota = config.quota(); if (quota != null) { if (!quota.acceptable(metric.value(timeMs))) - throw new QuotaViolationException("Metric " + metric.name() + " is in violation of its quota of " + quota.bound()); + throw new QuotaViolationException(metric.metricName() + " is in violation of its quota of " + quota.bound()); } } } @@ -134,55 +135,33 @@ public void add(CompoundStat stat) { public synchronized void add(CompoundStat stat, MetricConfig config) { this.stats.add(Utils.notNull(stat)); for (NamedMeasurable m : stat.stats()) { - KafkaMetric metric = new KafkaMetric(this, m.name(), m.description(), m.stat(), config == null ? this.config : config, time); + KafkaMetric metric = new KafkaMetric(this, m.name(), m.stat(), config == null ? this.config : config, time); this.registry.registerMetric(metric); this.metrics.add(metric); } } /** - * Add a metric with default configuration and no description. Equivalent to - * {@link Sensor#add(String, String, MeasurableStat, MetricConfig) add(name, "", stat, null)} - * - */ - public void add(String name, MeasurableStat stat) { - add(name, stat, null); - } - - /** - * Add a metric with default configuration. Equivalent to - * {@link Sensor#add(String, String, MeasurableStat, MetricConfig) add(name, description, stat, null)} - * - */ - public void add(String name, String description, MeasurableStat stat) { - add(name, description, stat, null); - } - - /** - * Add a metric to this sensor with no description. Equivalent to - * {@link Sensor#add(String, String, MeasurableStat, MetricConfig) add(name, "", stat, config)} - * @param name - * @param stat - * @param config + * Register a metric with this sensor + * @param metricName The name of the metric + * @param stat The statistic to keep */ - public void add(String name, MeasurableStat stat, MetricConfig config) { - add(name, "", stat, config); + public void add(MetricName metricName, MeasurableStat stat) { + add(metricName, stat, null); } /** * Register a metric with this sensor - * @param name The name of the metric - * @param description A description used when reporting the value + * @param metricName The name of the metric * @param stat The statistic to keep * @param config A special configuration for this metric. If null use the sensor default configuration. */ - public synchronized void add(String name, String description, MeasurableStat stat, MetricConfig config) { - KafkaMetric metric = new KafkaMetric(this, - Utils.notNull(name), - Utils.notNull(description), - Utils.notNull(stat), - config == null ? this.config : config, - time); + public synchronized void add(MetricName metricName, MeasurableStat stat, MetricConfig config) { + KafkaMetric metric = new KafkaMetric(new Object(), + Utils.notNull(metricName), + Utils.notNull(stat), + config == null ? this.config : config, + time); this.registry.registerMetric(metric); this.metrics.add(metric); this.stats.add(stat); diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentile.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentile.java index 7365ceb39072a..fb741ae086c71 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentile.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentile.java @@ -16,31 +16,23 @@ */ package org.apache.kafka.common.metrics.stats; +import org.apache.kafka.common.MetricName; + public class Percentile { - private final String name; - private final String description; + private final MetricName name; private final double percentile; - public Percentile(String name, double percentile) { - this(name, "", percentile); - } - - public Percentile(String name, String description, double percentile) { + public Percentile(MetricName name, double percentile) { super(); this.name = name; - this.description = description; this.percentile = percentile; } - public String name() { + public MetricName name() { return this.name; } - public String description() { - return this.description; - } - public double percentile() { return this.percentile; } diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java index c70d577ada8c0..78c93e88fa0b8 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java @@ -59,7 +59,7 @@ public List stats() { List ms = new ArrayList(this.percentiles.length); for (Percentile percentile : this.percentiles) { final double pct = percentile.percentile(); - ms.add(new NamedMeasurable(percentile.name(), percentile.description(), new Measurable() { + ms.add(new NamedMeasurable(percentile.name(), new Measurable() { public double measure(MetricConfig config, long now) { return value(config, now, pct / 100.0); } diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java index 4dd2cdf773f7e..74d695ba39de4 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java @@ -23,6 +23,7 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -31,6 +32,7 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.metrics.Measurable; import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.stats.Avg; @@ -81,17 +83,21 @@ public class Selector implements Selectable { private final List connected; private final Time time; private final SelectorMetrics sensors; + private final String metricGrpPrefix; + private final Map metricTags; /** * Create a new selector */ - public Selector(Metrics metrics, Time time) { + public Selector(Metrics metrics, Time time , String metricGrpPrefix , Map metricTags) { try { this.selector = java.nio.channels.Selector.open(); } catch (IOException e) { throw new KafkaException(e); } this.time = time; + this.metricGrpPrefix = metricGrpPrefix; + this.metricTags = metricTags; this.keys = new HashMap(); this.completedSends = new ArrayList(); this.completedReceives = new ArrayList(); @@ -410,42 +416,52 @@ private class SelectorMetrics { public SelectorMetrics(Metrics metrics) { this.metrics = metrics; + String metricGrpName = metricGrpPrefix + "-metrics"; this.connectionClosed = this.metrics.sensor("connections-closed"); - this.connectionClosed.add("connection-close-rate", "Connections closed per second in the window.", new Rate()); + MetricName metricName = new MetricName("connection-close-rate", metricGrpName, "Connections closed per second in the window.", metricTags); + this.connectionClosed.add(metricName, new Rate()); this.connectionCreated = this.metrics.sensor("connections-created"); - this.connectionCreated.add("connection-creation-rate", "New connections established per second in the window.", new Rate()); + metricName = new MetricName("connection-creation-rate", metricGrpName, "New connections established per second in the window.", metricTags); + this.connectionCreated.add(metricName, new Rate()); this.bytesTransferred = this.metrics.sensor("bytes-sent-received"); - bytesTransferred.add("network-io-rate", - "The average number of network operations (reads or writes) on all connections per second.", - new Rate(new Count())); + metricName = new MetricName("network-io-rate", metricGrpName, "The average number of network operations (reads or writes) on all connections per second.", metricTags); + bytesTransferred.add(metricName, new Rate(new Count())); this.bytesSent = this.metrics.sensor("bytes-sent", bytesTransferred); - this.bytesSent.add("outgoing-byte-rate", "The average number of outgoing bytes sent per second to all servers.", new Rate()); - this.bytesSent.add("request-rate", "The average number of requests sent per second.", new Rate(new Count())); - this.bytesSent.add("request-size-avg", "The average size of all requests in the window..", new Avg()); - this.bytesSent.add("request-size-max", "The maximum size of any request sent in the window.", new Max()); + metricName = new MetricName("outgoing-byte-rate", metricGrpName, "The average number of outgoing bytes sent per second to all servers.", metricTags); + this.bytesSent.add(metricName, new Rate()); + metricName = new MetricName("request-rate", metricGrpName, "The average number of requests sent per second.", metricTags); + this.bytesSent.add(metricName, new Rate(new Count())); + metricName = new MetricName("request-size-avg", metricGrpName, "The average size of all requests in the window..", metricTags); + this.bytesSent.add(metricName, new Avg()); + metricName = new MetricName("request-size-max", metricGrpName, "The maximum size of any request sent in the window.", metricTags); + this.bytesSent.add(metricName, new Max()); this.bytesReceived = this.metrics.sensor("bytes-received", bytesTransferred); - this.bytesReceived.add("incoming-byte-rate", "Bytes/second read off all sockets", new Rate()); - this.bytesReceived.add("response-rate", "Responses received sent per second.", new Rate(new Count())); + metricName = new MetricName("incoming-byte-rate", metricGrpName, "Bytes/second read off all sockets", metricTags); + this.bytesReceived.add(metricName, new Rate()); + metricName = new MetricName("response-rate", metricGrpName, "Responses received sent per second.", metricTags); + this.bytesReceived.add(metricName, new Rate(new Count())); this.selectTime = this.metrics.sensor("select-time"); - this.selectTime.add("select-rate", - "Number of times the I/O layer checked for new I/O to perform per second", - new Rate(new Count())); - this.selectTime.add("io-wait-time-ns-avg", - "The average length of time the I/O thread spent waiting for a socket ready for reads or writes in nanoseconds.", - new Avg()); - this.selectTime.add("io-wait-ratio", "The fraction of time the I/O thread spent waiting.", new Rate(TimeUnit.NANOSECONDS)); + metricName = new MetricName("select-rate", metricGrpName, "Number of times the I/O layer checked for new I/O to perform per second", metricTags); + this.selectTime.add(metricName, new Rate(new Count())); + metricName = new MetricName("io-wait-time-ns-avg", metricGrpName, "The average length of time the I/O thread spent waiting for a socket ready for reads or writes in nanoseconds.", metricTags); + this.selectTime.add(metricName, new Avg()); + metricName = new MetricName("io-wait-ratio", metricGrpName, "The fraction of time the I/O thread spent waiting.", metricTags); + this.selectTime.add(metricName, new Rate(TimeUnit.NANOSECONDS)); this.ioTime = this.metrics.sensor("io-time"); - this.ioTime.add("io-time-ns-avg", "The average length of time for I/O per select call in nanoseconds.", new Avg()); - this.ioTime.add("io-ratio", "The fraction of time the I/O thread spent doing I/O", new Rate(TimeUnit.NANOSECONDS)); + metricName = new MetricName("io-time-ns-avg", metricGrpName, "The average length of time for I/O per select call in nanoseconds.", metricTags); + this.ioTime.add(metricName, new Avg()); + metricName = new MetricName("io-ratio", metricGrpName, "The fraction of time the I/O thread spent doing I/O", metricTags); + this.ioTime.add(metricName, new Rate(TimeUnit.NANOSECONDS)); - this.metrics.addMetric("connection-count", "The current number of active connections.", new Measurable() { + metricName = new MetricName("connection-count", metricGrpName, "The current number of active connections.", metricTags); + this.metrics.addMetric(metricName, new Measurable() { public double measure(MetricConfig config, long now) { return keys.size(); } @@ -459,25 +475,34 @@ public void maybeRegisterNodeMetrics(int node) { String nodeRequestName = "node-" + node + ".bytes-sent"; Sensor nodeRequest = this.metrics.getSensor(nodeRequestName); if (nodeRequest == null) { + String metricGrpName = metricGrpPrefix + "-node-metrics"; + + Map tags = new LinkedHashMap(metricTags); + tags.put("node-id", "node-"+node); + nodeRequest = this.metrics.sensor(nodeRequestName); - nodeRequest.add("node-" + node + ".outgoing-byte-rate", new Rate()); - nodeRequest.add("node-" + node + ".request-rate", - "The average number of requests sent per second.", - new Rate(new Count())); - nodeRequest.add("node-" + node + ".request-size-avg", "The average size of all requests in the window..", new Avg()); - nodeRequest.add("node-" + node + ".request-size-max", "The maximum size of any request sent in the window.", new Max()); + MetricName metricName = new MetricName("outgoing-byte-rate", metricGrpName, tags); + nodeRequest.add(metricName, new Rate()); + metricName = new MetricName("request-rate", metricGrpName, "The average number of requests sent per second.", tags); + nodeRequest.add(metricName, new Rate(new Count())); + metricName = new MetricName("request-size-avg", metricGrpName, "The average size of all requests in the window..", tags); + nodeRequest.add(metricName, new Avg()); + metricName = new MetricName("request-size-max", metricGrpName, "The maximum size of any request sent in the window.", tags); + nodeRequest.add(metricName, new Max()); String nodeResponseName = "node-" + node + ".bytes-received"; Sensor nodeResponse = this.metrics.sensor(nodeResponseName); - nodeResponse.add("node-" + node + ".incoming-byte-rate", new Rate()); - nodeResponse.add("node-" + node + ".response-rate", - "The average number of responses received per second.", - new Rate(new Count())); + metricName = new MetricName("incoming-byte-rate", metricGrpName, tags); + nodeResponse.add(metricName, new Rate()); + metricName = new MetricName("response-rate", metricGrpName, "The average number of responses received per second.", tags); + nodeResponse.add(metricName, new Rate(new Count())); String nodeTimeName = "node-" + node + ".latency"; Sensor nodeRequestTime = this.metrics.sensor(nodeTimeName); - nodeRequestTime.add("node-" + node + ".request-latency-avg", new Avg()); - nodeRequestTime.add("node-" + node + ".request-latency-max", new Max()); + metricName = new MetricName("request-latency-avg", metricGrpName, tags); + nodeRequestTime.add(metricName, new Avg()); + metricName = new MetricName("request-latency-max", metricGrpName, tags); + nodeRequestTime.add(metricName, new Max()); } } } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java index fe3c13f319d48..12368038e1381 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java @@ -24,7 +24,9 @@ import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; @@ -33,6 +35,8 @@ public class BufferPoolTest { private MockTime time = new MockTime(); private Metrics metrics = new Metrics(time); + String metricGroup = "TestMetrics"; + Map metricTags = new LinkedHashMap(); /** * Test the simple non-blocking allocation paths @@ -41,7 +45,7 @@ public class BufferPoolTest { public void testSimple() throws Exception { int totalMemory = 64 * 1024; int size = 1024; - BufferPool pool = new BufferPool(totalMemory, size, false, metrics, time); + BufferPool pool = new BufferPool(totalMemory, size, false, metrics, time, metricGroup, metricTags); ByteBuffer buffer = pool.allocate(size); assertEquals("Buffer size should equal requested size.", size, buffer.limit()); assertEquals("Unallocated memory should have shrunk", totalMemory - size, pool.unallocatedMemory()); @@ -68,7 +72,7 @@ public void testSimple() throws Exception { */ @Test(expected = IllegalArgumentException.class) public void testCantAllocateMoreMemoryThanWeHave() throws Exception { - BufferPool pool = new BufferPool(1024, 512, true, metrics, time); + BufferPool pool = new BufferPool(1024, 512, true, metrics, time, metricGroup, metricTags); ByteBuffer buffer = pool.allocate(1024); assertEquals(1024, buffer.limit()); pool.deallocate(buffer); @@ -77,7 +81,7 @@ public void testCantAllocateMoreMemoryThanWeHave() throws Exception { @Test public void testNonblockingMode() throws Exception { - BufferPool pool = new BufferPool(2, 1, false, metrics, time); + BufferPool pool = new BufferPool(2, 1, false, metrics, time, metricGroup, metricTags); pool.allocate(1); try { pool.allocate(2); @@ -92,7 +96,7 @@ public void testNonblockingMode() throws Exception { */ @Test public void testDelayedAllocation() throws Exception { - BufferPool pool = new BufferPool(5 * 1024, 1024, true, metrics, time); + BufferPool pool = new BufferPool(5 * 1024, 1024, true, metrics, time, metricGroup, metricTags); ByteBuffer buffer = pool.allocate(1024); CountDownLatch doDealloc = asyncDeallocate(pool, buffer); CountDownLatch allocation = asyncAllocate(pool, 5 * 1024); @@ -141,7 +145,7 @@ public void testStressfulSituation() throws Exception { final int iterations = 50000; final int poolableSize = 1024; final int totalMemory = numThreads / 2 * poolableSize; - final BufferPool pool = new BufferPool(totalMemory, poolableSize, true, metrics, time); + final BufferPool pool = new BufferPool(totalMemory, poolableSize, true, metrics, time, metricGroup, metricTags); List threads = new ArrayList(); for (int i = 0; i < numThreads; i++) threads.add(new StressTestThread(pool, iterations)); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java index 2c9932401d573..e2bb8da7154ee 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java @@ -22,7 +22,9 @@ import java.util.Arrays; import java.util.Collections; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; import java.util.Set; import org.apache.kafka.clients.producer.internals.RecordAccumulator; @@ -59,11 +61,13 @@ public class RecordAccumulatorTest { private int msgSize = Records.LOG_OVERHEAD + Record.recordSize(key, value); private Cluster cluster = new Cluster(Arrays.asList(node1, node2), Arrays.asList(part1, part2, part3)); private Metrics metrics = new Metrics(time); + String metricGroup = "TestMetrics"; + Map metricTags = new LinkedHashMap(); @Test public void testFull() throws Exception { long now = time.milliseconds(); - RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, 10L, 100L, false, metrics, time); + RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, 10L, 100L, false, metrics, time, metricTags); int appends = 1024 / msgSize; for (int i = 0; i < appends; i++) { accum.append(tp1, key, value, CompressionType.NONE, null); @@ -86,7 +90,7 @@ public void testFull() throws Exception { @Test public void testAppendLarge() throws Exception { int batchSize = 512; - RecordAccumulator accum = new RecordAccumulator(batchSize, 10 * 1024, 0L, 100L, false, metrics, time); + RecordAccumulator accum = new RecordAccumulator(batchSize, 10 * 1024, 0L, 100L, false, metrics, time, metricTags); accum.append(tp1, key, new byte[2 * batchSize], CompressionType.NONE, null); assertEquals("Our partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes); } @@ -94,7 +98,7 @@ public void testAppendLarge() throws Exception { @Test public void testLinger() throws Exception { long lingerMs = 10L; - RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, lingerMs, 100L, false, metrics, time); + RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, lingerMs, 100L, false, metrics, time, metricTags); accum.append(tp1, key, value, CompressionType.NONE, null); assertEquals("No partitions should be ready", 0, accum.ready(cluster, time.milliseconds()).readyNodes.size()); time.sleep(10); @@ -111,7 +115,7 @@ public void testLinger() throws Exception { @Test public void testPartialDrain() throws Exception { - RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, 10L, 100L, false, metrics, time); + RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, 10L, 100L, false, metrics, time, metricTags); int appends = 1024 / msgSize + 1; List partitions = asList(tp1, tp2); for (TopicPartition tp : partitions) { @@ -129,7 +133,7 @@ public void testStressfulSituation() throws Exception { final int numThreads = 5; final int msgs = 10000; final int numParts = 2; - final RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, 0L, 100L, true, metrics, time); + final RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, 0L, 100L, true, metrics, time, metricTags); List threads = new ArrayList(); for (int i = 0; i < numThreads; i++) { threads.add(new Thread() { @@ -169,7 +173,7 @@ public void run() { public void testNextReadyCheckDelay() throws Exception { // Next check time will use lingerMs since this test won't trigger any retries/backoff long lingerMs = 10L; - RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, lingerMs, 100L, false, metrics, time); + RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, lingerMs, 100L, false, metrics, time, metricTags); // Just short of going over the limit so we trigger linger time int appends = 1024 / msgSize; diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java index ef2ca65cabe97..66cbdf5babed3 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java @@ -16,6 +16,8 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.util.LinkedHashMap; +import java.util.Map; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; @@ -50,7 +52,8 @@ public class SenderTest { private Metadata metadata = new Metadata(0, Long.MAX_VALUE); private Cluster cluster = TestUtils.singletonCluster("test", 1); private Metrics metrics = new Metrics(time); - private RecordAccumulator accumulator = new RecordAccumulator(batchSize, 1024 * 1024, 0L, 0L, false, metrics, time); + Map metricTags = new LinkedHashMap(); + private RecordAccumulator accumulator = new RecordAccumulator(batchSize, 1024 * 1024, 0L, 0L, false, metrics, time, metricTags); private Sender sender = new Sender(client, metadata, this.accumulator, @@ -59,7 +62,8 @@ public class SenderTest { MAX_RETRIES, REQUEST_TIMEOUT_MS, metrics, - time); + time, + "clientId"); @Before public void setup() { @@ -93,7 +97,8 @@ public void testRetries() throws Exception { maxRetries, REQUEST_TIMEOUT_MS, new Metrics(), - time); + time, + "clientId"); // do a successful retry Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null).future; sender.run(time.milliseconds()); // connect diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/JmxReporterTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/JmxReporterTest.java index 2f43c49450e1a..07b1b60d3a9cb 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/JmxReporterTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/JmxReporterTest.java @@ -16,10 +16,7 @@ */ package org.apache.kafka.common.metrics; - -import org.apache.kafka.common.metrics.JmxReporter; -import org.apache.kafka.common.metrics.Metrics; -import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.stats.Avg; import org.apache.kafka.common.metrics.stats.Total; import org.junit.Test; @@ -31,10 +28,10 @@ public void testJmxRegistration() throws Exception { Metrics metrics = new Metrics(); metrics.addReporter(new JmxReporter()); Sensor sensor = metrics.sensor("kafka.requests"); - sensor.add("pack.bean1.avg", new Avg()); - sensor.add("pack.bean2.total", new Total()); + sensor.add(new MetricName("pack.bean1.avg", "grp1"), new Avg()); + sensor.add(new MetricName("pack.bean2.total", "grp2"), new Total()); Sensor sensor2 = metrics.sensor("kafka.blah"); - sensor2.add("pack.bean1.some", new Total()); - sensor2.add("pack.bean2.some", new Total()); + sensor2.add(new MetricName("pack.bean1.some", "grp1"), new Total()); + sensor2.add(new MetricName("pack.bean2.some", "grp1"), new Total()); } } diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java index 19bea0f1fa1eb..998a57c492a51 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java @@ -16,9 +16,12 @@ import static org.junit.Assert.fail; import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.TimeUnit; import org.apache.kafka.common.Metric; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.stats.Avg; import org.apache.kafka.common.metrics.stats.Count; import org.apache.kafka.common.metrics.stats.Max; @@ -38,22 +41,40 @@ public class MetricsTest { MockTime time = new MockTime(); Metrics metrics = new Metrics(new MetricConfig(), Arrays.asList((MetricsReporter) new JmxReporter()), time); + @Test + public void testMetricName() { + MetricName n1 = new MetricName("name", "group", "description", "key1", "value1"); + Map tags = new HashMap(); + tags.put("key1", "value1"); + MetricName n2 = new MetricName("name", "group", "description", tags); + assertEquals("metric names created in two different ways should be equal", n1, n2); + + try { + new MetricName("name", "group", "description", "key1"); + fail("Creating MetricName with an old number of keyValue should fail"); + } catch (IllegalArgumentException e) { + // this is expected + } + } + @Test public void testSimpleStats() throws Exception { ConstantMeasurable measurable = new ConstantMeasurable(); - metrics.addMetric("direct.measurable", measurable); + + metrics.addMetric(new MetricName("direct.measurable", "grp1", "The fraction of time an appender waits for space allocation."), measurable); Sensor s = metrics.sensor("test.sensor"); - s.add("test.avg", new Avg()); - s.add("test.max", new Max()); - s.add("test.min", new Min()); - s.add("test.rate", new Rate(TimeUnit.SECONDS)); - s.add("test.occurences", new Rate(TimeUnit.SECONDS, new Count())); - s.add("test.count", new Count()); - s.add(new Percentiles(100, -100, 100, BucketSizing.CONSTANT, new Percentile("test.median", 50.0), new Percentile("test.perc99_9", - 99.9))); + s.add(new MetricName("test.avg", "grp1"), new Avg()); + s.add(new MetricName("test.max", "grp1"), new Max()); + s.add(new MetricName("test.min", "grp1"), new Min()); + s.add(new MetricName("test.rate", "grp1"), new Rate(TimeUnit.SECONDS)); + s.add(new MetricName("test.occurences", "grp1"), new Rate(TimeUnit.SECONDS, new Count())); + s.add(new MetricName("test.count", "grp1"), new Count()); + s.add(new Percentiles(100, -100, 100, BucketSizing.CONSTANT, + new Percentile(new MetricName("test.median", "grp1"), 50.0), + new Percentile(new MetricName("test.perc99_9", "grp1"),99.9))); Sensor s2 = metrics.sensor("test.sensor2"); - s2.add("s2.total", new Total()); + s2.add(new MetricName("s2.total", "grp1"), new Total()); s2.record(5.0); for (int i = 0; i < 10; i++) @@ -62,27 +83,27 @@ public void testSimpleStats() throws Exception { // pretend 2 seconds passed... time.sleep(2000); - assertEquals("s2 reflects the constant value", 5.0, metrics.metrics().get("s2.total").value(), EPS); - assertEquals("Avg(0...9) = 4.5", 4.5, metrics.metrics().get("test.avg").value(), EPS); - assertEquals("Max(0...9) = 9", 9.0, metrics.metrics().get("test.max").value(), EPS); - assertEquals("Min(0...9) = 0", 0.0, metrics.metrics().get("test.min").value(), EPS); - assertEquals("Rate(0...9) = 22.5", 22.5, metrics.metrics().get("test.rate").value(), EPS); - assertEquals("Occurences(0...9) = 5", 5.0, metrics.metrics().get("test.occurences").value(), EPS); - assertEquals("Count(0...9) = 10", 10.0, metrics.metrics().get("test.count").value(), EPS); + assertEquals("s2 reflects the constant value", 5.0, metrics.metrics().get(new MetricName("s2.total", "grp1")).value(), EPS); + assertEquals("Avg(0...9) = 4.5", 4.5, metrics.metrics().get(new MetricName("test.avg", "grp1")).value(), EPS); + assertEquals("Max(0...9) = 9", 9.0, metrics.metrics().get(new MetricName("test.max", "grp1")).value(), EPS); + assertEquals("Min(0...9) = 0", 0.0, metrics.metrics().get(new MetricName("test.min", "grp1")).value(), EPS); + assertEquals("Rate(0...9) = 22.5", 22.5, metrics.metrics().get(new MetricName("test.rate", "grp1")).value(), EPS); + assertEquals("Occurences(0...9) = 5", 5.0, metrics.metrics().get(new MetricName("test.occurences", "grp1")).value(), EPS); + assertEquals("Count(0...9) = 10", 10.0, metrics.metrics().get(new MetricName("test.count", "grp1")).value(), EPS); } @Test public void testHierarchicalSensors() { Sensor parent1 = metrics.sensor("test.parent1"); - parent1.add("test.parent1.count", new Count()); + parent1.add(new MetricName("test.parent1.count", "grp1"), new Count()); Sensor parent2 = metrics.sensor("test.parent2"); - parent2.add("test.parent2.count", new Count()); + parent2.add(new MetricName("test.parent2.count", "grp1"), new Count()); Sensor child1 = metrics.sensor("test.child1", parent1, parent2); - child1.add("test.child1.count", new Count()); + child1.add(new MetricName("test.child1.count", "grp1"), new Count()); Sensor child2 = metrics.sensor("test.child2", parent1); - child2.add("test.child2.count", new Count()); + child2.add(new MetricName("test.child2.count", "grp1"), new Count()); Sensor grandchild = metrics.sensor("test.grandchild", child1); - grandchild.add("test.grandchild.count", new Count()); + grandchild.add(new MetricName("test.grandchild.count", "grp1"), new Count()); /* increment each sensor one time */ parent1.record(); @@ -150,15 +171,15 @@ public void testOldDataHasNoEffect() { @Test(expected = IllegalArgumentException.class) public void testDuplicateMetricName() { - metrics.sensor("test").add("test", new Avg()); - metrics.sensor("test2").add("test", new Total()); + metrics.sensor("test").add(new MetricName("test", "grp1"), new Avg()); + metrics.sensor("test2").add(new MetricName("test", "grp1"), new Total()); } @Test public void testQuotas() { Sensor sensor = metrics.sensor("test"); - sensor.add("test1.total", new Total(), new MetricConfig().quota(Quota.lessThan(5.0))); - sensor.add("test2.total", new Total(), new MetricConfig().quota(Quota.moreThan(0.0))); + sensor.add(new MetricName("test1.total", "grp1"), new Total(), new MetricConfig().quota(Quota.lessThan(5.0))); + sensor.add(new MetricName("test2.total", "grp1"), new Total(), new MetricConfig().quota(Quota.moreThan(0.0))); sensor.record(5.0); try { sensor.record(1.0); @@ -166,7 +187,7 @@ public void testQuotas() { } catch (QuotaViolationException e) { // this is good } - assertEquals(6.0, metrics.metrics().get("test1.total").value(), EPS); + assertEquals(6.0, metrics.metrics().get(new MetricName("test1.total", "grp1")).value(), EPS); sensor.record(-6.0); try { sensor.record(-1.0); @@ -183,15 +204,15 @@ public void testPercentiles() { 0.0, 100.0, BucketSizing.CONSTANT, - new Percentile("test.p25", 25), - new Percentile("test.p50", 50), - new Percentile("test.p75", 75)); + new Percentile(new MetricName("test.p25", "grp1"), 25), + new Percentile(new MetricName("test.p50", "grp1"), 50), + new Percentile(new MetricName("test.p75", "grp1"), 75)); MetricConfig config = new MetricConfig().eventWindow(50).samples(2); Sensor sensor = metrics.sensor("test", config); sensor.add(percs); - Metric p25 = this.metrics.metrics().get("test.p25"); - Metric p50 = this.metrics.metrics().get("test.p50"); - Metric p75 = this.metrics.metrics().get("test.p75"); + Metric p25 = this.metrics.metrics().get(new MetricName("test.p25", "grp1")); + Metric p50 = this.metrics.metrics().get(new MetricName("test.p50", "grp1")); + Metric p75 = this.metrics.metrics().get(new MetricName("test.p75", "grp1")); // record two windows worth of sequential values for (int i = 0; i < buckets; i++) diff --git a/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java index 5c5e3d40819e4..74c19573a29b0 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java @@ -25,6 +25,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; +import java.util.LinkedHashMap; import java.util.List; import org.apache.kafka.common.metrics.Metrics; @@ -50,7 +51,7 @@ public class SelectorTest { public void setup() throws Exception { this.server = new EchoServer(); this.server.start(); - this.selector = new Selector(new Metrics(), new MockTime()); + this.selector = new Selector(new Metrics(), new MockTime() , "MetricGroup", new LinkedHashMap()); } @After diff --git a/clients/src/test/java/org/apache/kafka/test/MetricsBench.java b/clients/src/test/java/org/apache/kafka/test/MetricsBench.java index 9d98c11482554..633d4bbf214a5 100644 --- a/clients/src/test/java/org/apache/kafka/test/MetricsBench.java +++ b/clients/src/test/java/org/apache/kafka/test/MetricsBench.java @@ -14,6 +14,7 @@ import java.util.Arrays; +import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.metrics.stats.Avg; @@ -31,15 +32,15 @@ public static void main(String[] args) { Sensor parent = metrics.sensor("parent"); Sensor child = metrics.sensor("child", parent); for (Sensor sensor : Arrays.asList(parent, child)) { - sensor.add(sensor.name() + ".avg", new Avg()); - sensor.add(sensor.name() + ".count", new Count()); - sensor.add(sensor.name() + ".max", new Max()); + sensor.add(new MetricName(sensor.name() + ".avg", "grp1"), new Avg()); + sensor.add(new MetricName(sensor.name() + ".count", "grp1"), new Count()); + sensor.add(new MetricName(sensor.name() + ".max", "grp1"), new Max()); sensor.add(new Percentiles(1024, 0.0, iters, BucketSizing.CONSTANT, - new Percentile(sensor.name() + ".median", 50.0), - new Percentile(sensor.name() + ".p_99", 99.0))); + new Percentile(new MetricName(sensor.name() + ".median", "grp1"), 50.0), + new Percentile(new MetricName(sensor.name() + ".p_99", "grp1"), 99.0))); } long start = System.nanoTime(); for (int i = 0; i < iters; i++) From e79ebdfe222288f2c5d997ba7e7c5e18c8c514ed Mon Sep 17 00:00:00 2001 From: Jaikiran Pai Date: Tue, 13 Jan 2015 09:39:20 -0800 Subject: [PATCH 089/491] KAFKA-1854 Allow JIRA username and password to be prompted in the absence of a jira.ini file, during patch submission; reviewed by Neha Narkhede --- kafka-patch-review.py | 31 +++++++++++++++++++++++-------- 1 file changed, 23 insertions(+), 8 deletions(-) diff --git a/kafka-patch-review.py b/kafka-patch-review.py index b7f132f9d210b..959268069cd0c 100644 --- a/kafka-patch-review.py +++ b/kafka-patch-review.py @@ -7,22 +7,31 @@ import datetime import tempfile import commands +import getpass from jira.client import JIRA def get_jira_config(): # read the config file home=jira_home=os.getenv('HOME') home=home.rstrip('/') - jira_config = dict(line.strip().split('=') for line in open(home + '/jira.ini')) - return jira_config + if not (os.path.isfile(home + '/jira.ini')): + jira_user=raw_input('JIRA user :') + jira_pass=getpass.getpass('JIRA password :') + jira_config = {'user':jira_user, 'password':jira_pass} + return jira_config + else: + jira_config = dict(line.strip().split('=') for line in open(home + '/jira.ini')) + return jira_config -def get_jira(): +def get_jira(jira_config): options = { 'server': 'https://issues.apache.org/jira' } - - jira_config = get_jira_config() jira = JIRA(options=options,basic_auth=(jira_config['user'], jira_config['password'])) + # (Force) verify the auth was really done + jira_session=jira.session() + if (jira_session is None): + raise Exception("Failed to login to the JIRA instance") return jira def cmd_exists(cmd): @@ -81,6 +90,15 @@ def main(): p=os.popen(git_remote_update) p.close() + # Get JIRA configuration and login to JIRA to ensure the credentials work, before publishing the patch to the review board + print "Verifying JIRA connection configurations" + try: + jira_config=get_jira_config() + jira=get_jira(jira_config) + except: + print "Failed to login to the JIRA instance", sys.exc_info()[0], sys.exc_info()[1] + sys.exit(1) + rb_command= post_review_tool + " --publish --tracking-branch " + opt.branch + " --target-groups=kafka --bugs-closed=" + opt.jira if opt.debug: rb_command=rb_command + " --debug" @@ -123,7 +141,6 @@ def main(): p.close() print 'Creating diff against', opt.branch, 'and uploading patch to JIRA',opt.jira - jira=get_jira() issue = jira.issue(opt.jira) attachment=open(patch_file) jira.add_attachment(issue,attachment) @@ -146,8 +163,6 @@ def main(): for t in transitions: transitionsMap[t['name']] = t['id'] - jira_config = get_jira_config() - if('Submit Patch' in transitionsMap): jira.transition_issue(issue, transitionsMap['Submit Patch'] , assignee={'name': jira_config['user']} ) From bfb2da3c82ab01f5914d5559ee196d5aa977b17c Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Tue, 13 Jan 2015 09:54:32 -0800 Subject: [PATCH 090/491] trivial change to add byte serializer to ProducerPerformance; patched by Jun Rao --- .../apache/kafka/clients/tools/ProducerPerformance.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java b/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java index 1b828007975ef..689bae9e6ba69 100644 --- a/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java +++ b/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java @@ -15,10 +15,7 @@ import java.util.Arrays; import java.util.Properties; -import org.apache.kafka.clients.producer.Callback; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.clients.producer.*; public class ProducerPerformance { @@ -46,6 +43,8 @@ public static void main(String[] args) throws Exception { throw new IllegalArgumentException("Invalid property: " + args[i]); props.put(pieces[0], pieces[1]); } + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); KafkaProducer producer = new KafkaProducer(props); /* setup perf test */ From a611178408cf8497054ff015caba18cfcff70a60 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Wed, 14 Jan 2015 12:02:50 -0800 Subject: [PATCH 091/491] KAFKA-1723 (delta patch to fix javadoc); make the metrics name in new producer more standard; patched by Manikumar Reddy; reviewed by Jun Rao --- .../kafka/clients/producer/KafkaProducer.java | 22 ++++++++--------- .../org/apache/kafka/common/MetricName.java | 24 ++++++++++++------- 2 files changed, 26 insertions(+), 20 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index c79149a715fcb..fc71710dd5997 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -265,32 +265,32 @@ public Future send(ProducerRecord record) { *

          * If you want to simulate a simple blocking call you can do the following: * - *

          -     *   producer.send(new ProducerRecord("the-topic", "key".getBytes(), "value".getBytes())).get();
          -     * 
          + *
          {@code
          +     * producer.send(new ProducerRecord("the-topic", "key".getBytes(), "value".getBytes())).get();
          +     * }
          *

          * Those desiring fully non-blocking usage can make use of the {@link Callback} parameter to provide a callback that * will be invoked when the request is complete. * - *

          -     *   ProducerRecord record = new ProducerRecord("the-topic", "key".getBytes(), "value".getBytes());
          +     * 
          {@code
          +     * ProducerRecord record = new ProducerRecord("the-topic", "key".getBytes(), "value".getBytes());
                *   producer.send(myRecord,
          -     *                 new Callback() {
          +     *                new Callback() {
                *                     public void onCompletion(RecordMetadata metadata, Exception e) {
                *                         if(e != null)
                *                             e.printStackTrace();
                *                         System.out.println("The offset of the record we just sent is: " + metadata.offset());
                *                     }
          -     *                 });
          -     * 
          + * }); + * }
          * * Callbacks for records being sent to the same partition are guaranteed to execute in order. That is, in the * following example callback1 is guaranteed to execute before callback2: * - *
          -     * producer.send(new ProducerRecord(topic, partition, key, value), callback1);
          +     * 
          {@code
          +     * producer.send(new ProducerRecord(topic, partition, key1, value1), callback1);
                * producer.send(new ProducerRecord(topic, partition, key2, value2), callback2);
          -     * 
          + * }
          *

          * Note that callbacks will generally execute in the I/O thread of the producer and so should be reasonably fast or * they will delay the sending of messages from other threads. If you want to execute blocking or computationally diff --git a/clients/src/main/java/org/apache/kafka/common/MetricName.java b/clients/src/main/java/org/apache/kafka/common/MetricName.java index 4e810d56b753b..7e977e94a8e0b 100644 --- a/clients/src/main/java/org/apache/kafka/common/MetricName.java +++ b/clients/src/main/java/org/apache/kafka/common/MetricName.java @@ -19,34 +19,40 @@ /** * The MetricName class encapsulates a metric's name, logical group and its related attributes - *

          + *

          * This class captures the following parameters *

            *  name The name of the metric
            *  group logical group name of the metrics to which this metric belongs.
            *  description A human-readable description to include in the metric. This is optional.
            *  tags additional key/value attributes of the metric. This is optional.
          - *   
          + *
      * group, tags parameters can be used to create unique metric names while reporting in JMX or any custom reporting. - * + *

      * Ex: standard JMX MBean can be constructed like domainName:type=group,key1=val1,key2=val2 - * + *

      * Usage looks something like this: - *

      + * 
      {@code
        * // set up metrics:
        * Metrics metrics = new Metrics(); // this is the global repository of metrics and sensors
      - * Sensor sensor = metrics.sensor("message-sizes");
      + * Sensor sensor = metrics.sensor("message-sizes");
      + *
        * Map metricTags = new LinkedHashMap();
        * metricTags.put("client-id", "producer-1");
        * metricTags.put("topic", "topic");
      - * MetricName metricName = new MetricName("message-size-avg", "producer-metrics", "average message size", metricTags);
      + *
      + * MetricName metricName = new MetricName("message-size-avg", "producer-metrics", "average message size", metricTags);
        * sensor.add(metricName, new Avg());
      - * metricName = new MetricName("message-size-max", "producer-metrics",metricTags);
      + *
      + * metricName = new MetricName("message-size-max", "producer-metrics", metricTags);
        * sensor.add(metricName, new Max());
        *
      + * metricName = new MetricName("message-size-min", "producer-metrics", "message minimum size", "client-id", "my-client", "topic", "my-topic");
      + * sensor.add(metricName, new Min());
      + *
        * // as messages are sent we record the sizes
        * sensor.record(messageSize);
      - * 
      + * }
      */ public final class MetricName { From 1c8f89bc73ec1844371c812215b255db037e24a5 Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Wed, 14 Jan 2015 15:16:53 -0800 Subject: [PATCH 092/491] KAFKA-1499; Broker-side compression configuration; reviewed by Joel Koshy --- core/src/main/scala/kafka/log/Log.scala | 23 +++-- core/src/main/scala/kafka/log/LogConfig.scala | 20 ++++- .../kafka/message/ByteBufferMessageSet.scala | 14 ++-- .../kafka/message/CompressionCodec.scala | 39 +++++++-- .../main/scala/kafka/server/KafkaConfig.scala | 11 +++ .../main/scala/kafka/server/KafkaServer.scala | 3 +- .../test/scala/kafka/log/LogConfigTest.scala | 2 + .../kafka/log/BrokerCompressionTest.scala | 84 +++++++++++++++++++ .../message/ByteBufferMessageSetTest.scala | 6 +- .../unit/kafka/server/KafkaConfigTest.scala | 28 ++++++- 10 files changed, 201 insertions(+), 29 deletions(-) create mode 100644 core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 024506cd00556..86422bfd5122b 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -32,7 +32,7 @@ import scala.collection.JavaConversions import com.yammer.metrics.core.Gauge object LogAppendInfo { - val UnknownLogAppendInfo = LogAppendInfo(-1, -1, NoCompressionCodec, -1, -1, false) + val UnknownLogAppendInfo = LogAppendInfo(-1, -1, NoCompressionCodec, NoCompressionCodec, -1, -1, false) } /** @@ -41,10 +41,11 @@ object LogAppendInfo { * @param lastOffset The last offset in the message set * @param shallowCount The number of shallow messages * @param validBytes The number of valid bytes - * @param codec The codec used in the message set + * @param sourceCodec The source codec used in the message set(coming from producer) + * @param targetCodec The target codec of the message set(after applying broker compression logic) * @param offsetsMonotonic Are the offsets in this message set monotonically increasing */ -case class LogAppendInfo(var firstOffset: Long, var lastOffset: Long, codec: CompressionCodec, shallowCount: Int, validBytes: Int, offsetsMonotonic: Boolean) +case class LogAppendInfo(var firstOffset: Long, var lastOffset: Long, sourceCodec: CompressionCodec, targetCodec: CompressionCodec, shallowCount: Int, validBytes: Int, offsetsMonotonic: Boolean) /** @@ -287,7 +288,7 @@ class Log(val dir: File, // assign offsets to the message set val offset = new AtomicLong(nextOffsetMetadata.messageOffset) try { - validMessages = validMessages.assignOffsets(offset, appendInfo.codec) + validMessages = validMessages.assignOffsets(offset, appendInfo.sourceCodec, appendInfo.targetCodec) } catch { case e: IOException => throw new KafkaException("Error in validating messages while appending to log '%s'".format(name), e) } @@ -360,7 +361,7 @@ class Log(val dir: File, var shallowMessageCount = 0 var validBytesCount = 0 var firstOffset, lastOffset = -1L - var codec: CompressionCodec = NoCompressionCodec + var sourceCodec: CompressionCodec = NoCompressionCodec var monotonic = true for(messageAndOffset <- messages.shallowIterator) { // update the first offset if on the first message @@ -388,14 +389,18 @@ class Log(val dir: File, shallowMessageCount += 1 validBytesCount += messageSize - + val messageCodec = m.compressionCodec if(messageCodec != NoCompressionCodec) - codec = messageCodec + sourceCodec = messageCodec } - LogAppendInfo(firstOffset, lastOffset, codec, shallowMessageCount, validBytesCount, monotonic) + + //Apply if any broker-side compression + val targetCodec = BrokerCompressionCodec.getTargetCompressionCodec(config.compressionType, sourceCodec) + + LogAppendInfo(firstOffset, lastOffset, sourceCodec, targetCodec, shallowMessageCount, validBytesCount, monotonic) } - + /** * Trim any invalid bytes from the end of this message set (if there are any) * @param messages The message set to trim diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index ca7a99e99f641..2338b4410ca53 100644 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -19,9 +19,11 @@ package kafka.log import java.util.Properties import org.apache.kafka.common.utils.Utils - import scala.collection._ import org.apache.kafka.common.config.ConfigDef +import kafka.common._ +import scala.collection.JavaConversions._ +import kafka.message.BrokerCompressionCodec object Defaults { val SegmentSize = 1024 * 1024 @@ -40,6 +42,7 @@ object Defaults { val Compact = false val UncleanLeaderElectionEnable = true val MinInSyncReplicas = 1 + val CompressionType = "producer" } /** @@ -59,6 +62,7 @@ object Defaults { * @param compact Should old segments in this log be deleted or deduplicated? * @param uncleanLeaderElectionEnable Indicates whether unclean leader election is enabled * @param minInSyncReplicas If number of insync replicas drops below this number, we stop accepting writes with -1 (or all) required acks + * @param compressionType compressionType for a given topic * */ case class LogConfig(val segmentSize: Int = Defaults.SegmentSize, @@ -76,7 +80,8 @@ case class LogConfig(val segmentSize: Int = Defaults.SegmentSize, val minCleanableRatio: Double = Defaults.MinCleanableDirtyRatio, val compact: Boolean = Defaults.Compact, val uncleanLeaderElectionEnable: Boolean = Defaults.UncleanLeaderElectionEnable, - val minInSyncReplicas: Int = Defaults.MinInSyncReplicas) { + val minInSyncReplicas: Int = Defaults.MinInSyncReplicas, + val compressionType: String = Defaults.CompressionType) { def toProps: Properties = { val props = new Properties() @@ -97,6 +102,7 @@ case class LogConfig(val segmentSize: Int = Defaults.SegmentSize, props.put(CleanupPolicyProp, if(compact) "compact" else "delete") props.put(UncleanLeaderElectionEnableProp, uncleanLeaderElectionEnable.toString) props.put(MinInSyncReplicasProp, minInSyncReplicas.toString) + props.put(CompressionTypeProp, compressionType) props } @@ -125,6 +131,7 @@ object LogConfig { val CleanupPolicyProp = "cleanup.policy" val UncleanLeaderElectionEnableProp = "unclean.leader.election.enable" val MinInSyncReplicasProp = "min.insync.replicas" + val CompressionTypeProp = "compression.type" val SegmentSizeDoc = "The hard maximum for the size of a segment file in the log" val SegmentMsDoc = "The soft maximum on the amount of time before a new log segment is rolled" @@ -145,6 +152,10 @@ object LogConfig { val UncleanLeaderElectionEnableDoc = "Indicates whether unclean leader election is enabled" val MinInSyncReplicasDoc = "If number of insync replicas drops below this number, we stop accepting writes with" + " -1 (or all) required acks" + val CompressionTypeDoc = "This parameter allows you to specify the compression logic for a given topic. This config" + + " is used to retain/remove/change the compression set by the producer. This config takes the following options: " + + " uncompressed, gzip, snappy, lz4, producer. uncompressed means that regardless of what the producer sets, the broker" + + " writes the message decompressed. producer means the broker attempts to retain whatever is used by the producer" private val configDef = { import ConfigDef.Range._ @@ -174,6 +185,7 @@ object LogConfig { .define(UncleanLeaderElectionEnableProp, BOOLEAN, Defaults.UncleanLeaderElectionEnable, MEDIUM, UncleanLeaderElectionEnableDoc) .define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), MEDIUM, MinInSyncReplicasDoc) + .define(CompressionTypeProp, STRING, Defaults.CompressionType, in(seqAsJavaList(BrokerCompressionCodec.brokerCompressionOptions)), MEDIUM, CompressionTypeDoc) } def configNames() = { @@ -181,6 +193,7 @@ object LogConfig { configDef.names().toList.sorted } + /** * Parse the given properties instance into a LogConfig object */ @@ -202,7 +215,8 @@ object LogConfig { minCleanableRatio = parsed.get(MinCleanableDirtyRatioProp).asInstanceOf[Double], compact = parsed.get(CleanupPolicyProp).asInstanceOf[String].toLowerCase != Delete, uncleanLeaderElectionEnable = parsed.get(UncleanLeaderElectionEnableProp).asInstanceOf[Boolean], - minInSyncReplicas = parsed.get(MinInSyncReplicasProp).asInstanceOf[Int]) + minInSyncReplicas = parsed.get(MinInSyncReplicasProp).asInstanceOf[Int], + compressionType = parsed.get(CompressionTypeProp).asInstanceOf[String].toLowerCase()) } /** diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index 788c7864bc881..f46ad5cbbbad7 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -194,16 +194,16 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi innerIter.next } } - + } } - + /** * Update the offsets for this message set. This method attempts to do an in-place conversion * if there is no compression, but otherwise recopies the messages */ - private[kafka] def assignOffsets(offsetCounter: AtomicLong, codec: CompressionCodec): ByteBufferMessageSet = { - if(codec == NoCompressionCodec) { + private[kafka] def assignOffsets(offsetCounter: AtomicLong, sourceCodec: CompressionCodec, targetCodec: CompressionCodec): ByteBufferMessageSet = { + if(sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) { // do an in-place conversion var position = 0 buffer.mark() @@ -217,16 +217,16 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi } else { // messages are compressed, crack open the messageset and recompress with correct offset val messages = this.internalIterator(isShallow = false).map(_.message) - new ByteBufferMessageSet(compressionCodec = codec, offsetCounter = offsetCounter, messages = messages.toBuffer:_*) + new ByteBufferMessageSet(compressionCodec = targetCodec, offsetCounter = offsetCounter, messages = messages.toBuffer:_*) } } - + /** * The total number of bytes in this message set, including any partial trailing messages */ def sizeInBytes: Int = buffer.limit - + /** * The total number of bytes in this message set not including any partial, trailing messages */ diff --git a/core/src/main/scala/kafka/message/CompressionCodec.scala b/core/src/main/scala/kafka/message/CompressionCodec.scala index 9439d2bc29a0c..cacde9bca0ee6 100644 --- a/core/src/main/scala/kafka/message/CompressionCodec.scala +++ b/core/src/main/scala/kafka/message/CompressionCodec.scala @@ -38,29 +38,58 @@ object CompressionCodec { } } +object BrokerCompressionCodec { + + val brokerCompressionCodecs = List(UncompressedCodec, SnappyCompressionCodec, LZ4CompressionCodec, GZIPCompressionCodec, ProducerCompressionCodec) + val brokerCompressionOptions = brokerCompressionCodecs.map(codec => codec.name) + + def isValid(compressionType: String): Boolean = brokerCompressionOptions.contains((compressionType.toLowerCase())) + + def getCompressionCodec(compressionType: String): CompressionCodec = { + compressionType.toLowerCase match { + case UncompressedCodec.name => NoCompressionCodec + case _ => CompressionCodec.getCompressionCodec(compressionType) + } + } + + def getTargetCompressionCodec(compressionType: String, producerCompression: CompressionCodec): CompressionCodec = { + if (ProducerCompressionCodec.name.equals(compressionType)) producerCompression + else getCompressionCodec(compressionType) + } +} + sealed trait CompressionCodec { def codec: Int; def name: String } +sealed trait BrokerCompressionCodec { def name: String } -case object DefaultCompressionCodec extends CompressionCodec { +case object DefaultCompressionCodec extends CompressionCodec with BrokerCompressionCodec { val codec = GZIPCompressionCodec.codec val name = GZIPCompressionCodec.name } -case object GZIPCompressionCodec extends CompressionCodec { +case object GZIPCompressionCodec extends CompressionCodec with BrokerCompressionCodec { val codec = 1 val name = "gzip" } -case object SnappyCompressionCodec extends CompressionCodec { +case object SnappyCompressionCodec extends CompressionCodec with BrokerCompressionCodec { val codec = 2 val name = "snappy" } -case object LZ4CompressionCodec extends CompressionCodec { +case object LZ4CompressionCodec extends CompressionCodec with BrokerCompressionCodec { val codec = 3 val name = "lz4" } -case object NoCompressionCodec extends CompressionCodec { +case object NoCompressionCodec extends CompressionCodec with BrokerCompressionCodec { val codec = 0 val name = "none" } + +case object UncompressedCodec extends BrokerCompressionCodec { + val name = "uncompressed" +} + +case object ProducerCompressionCodec extends BrokerCompressionCodec { + val name = "producer" +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index bbd3fd75e8396..9d1adec06a4f1 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -21,6 +21,8 @@ import java.util.Properties import kafka.message.{MessageSet, Message} import kafka.consumer.ConsumerConfig import kafka.utils.{VerifiableProperties, ZKConfig, Utils} +import kafka.message.NoCompressionCodec +import kafka.message.BrokerCompressionCodec /** * Configuration settings for the kafka server @@ -345,4 +347,13 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* Enables delete topic. Delete topic through the admin tool will have no effect if this config is turned off */ val deleteTopicEnable = props.getBoolean("delete.topic.enable", false) + /** + * This parameter allows you to specify the broker-side compression logic. This config is used to + * retain/remove/change the compression set by the producer. This config takes the following options: + * uncompressed, gzip, snappy, lz4, producer. uncompressed means that regardless of what the producer sets, the broker + * writes the message decompressed. producer means the broker attempts to retain whatever is used by the producer" + */ + val compressionType = props.getString("compression.type", "producer").toLowerCase() + require(BrokerCompressionCodec.isValid(compressionType), "compression.type : "+compressionType + " is not valid." + + " Valid options are "+BrokerCompressionCodec.brokerCompressionOptions.mkString(",")) } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index a069eb9272c92..89200da30a049 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -329,7 +329,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg deleteRetentionMs = config.logCleanerDeleteRetentionMs, fileDeleteDelayMs = config.logDeleteDelayMs, minCleanableRatio = config.logCleanerMinCleanRatio, - compact = config.logCleanupPolicy.trim.toLowerCase == "compact") + compact = config.logCleanupPolicy.trim.toLowerCase == "compact", + compressionType = config.compressionType) val defaultProps = defaultLogConfig.toProps val configs = AdminUtils.fetchAllTopicConfigs(zkClient).mapValues(LogConfig.fromProps(defaultProps, _)) // read the log configurations from zookeeper diff --git a/core/src/test/scala/kafka/log/LogConfigTest.scala b/core/src/test/scala/kafka/log/LogConfigTest.scala index 99b0df7b69c5e..fe5bd9d9155a3 100644 --- a/core/src/test/scala/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/kafka/log/LogConfigTest.scala @@ -34,6 +34,7 @@ class LogConfigTest extends JUnit3Suite { Assert.assertEquals(4242, config.segmentSize) Assert.assertEquals("LogConfig defaults should be retained", Defaults.MaxMessageSize, config.maxMessageSize) + Assert.assertEquals("producer", config.compressionType) } @Test @@ -50,6 +51,7 @@ class LogConfigTest extends JUnit3Suite { LogConfig.configNames().foreach((name) => { name match { case LogConfig.UncleanLeaderElectionEnableProp => expected.setProperty(name, randFrom("true", "false")) + case LogConfig.CompressionTypeProp => expected.setProperty(name, randFrom("producer","uncompressed","gzip")) case LogConfig.CleanupPolicyProp => expected.setProperty(name, randFrom(LogConfig.Compact, LogConfig.Delete)) case LogConfig.MinCleanableDirtyRatioProp => expected.setProperty(name, "%.1f".format(nextDouble * .9 + .1)) case LogConfig.MinInSyncReplicasProp => expected.setProperty(name, (nextInt(Int.MaxValue - 1) + 1).toString) diff --git a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala new file mode 100644 index 0000000000000..fa4a8ad0acc9e --- /dev/null +++ b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala @@ -0,0 +1,84 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.log + +import java.io.File +import kafka.utils._ +import kafka.message._ +import org.scalatest.junit.JUnitSuite +import org.junit._ +import org.junit.Assert._ +import org.junit.runner.RunWith +import org.junit.runners.Parameterized +import org.junit.runners.Parameterized.Parameters +import java.util.{ Collection, ArrayList } +import kafka.server.KafkaConfig +import org.apache.kafka.common.record.CompressionType +import scala.collection.JavaConversions._ + +@RunWith(value = classOf[Parameterized]) +class BrokerCompressionTest(messageCompression: String, brokerCompression: String) extends JUnitSuite { + + var logDir: File = null + val time = new MockTime(0) + val logConfig = LogConfig() + + @Before + def setUp() { + logDir = TestUtils.tempDir() + } + + @After + def tearDown() { + Utils.rm(logDir) + } + + /** + * Test broker-side compression configuration + */ + @Test + def testBrokerSideCompression() { + val messageCompressionCode = CompressionCodec.getCompressionCodec(messageCompression) + + /*configure broker-side compression */ + val log = new Log(logDir, logConfig.copy(compressionType = brokerCompression), recoveryPoint = 0L, time.scheduler, time = time) + + /* append two messages */ + log.append(new ByteBufferMessageSet(messageCompressionCode, new Message("hello".getBytes), new Message("there".getBytes))) + + def readMessage(offset: Int) = log.read(offset, 4096).messageSet.head.message + + if (!brokerCompression.equals("producer")) { + val brokerCompressionCode = BrokerCompressionCodec.getCompressionCodec(brokerCompression) + assertEquals("Compression at offset 0 should produce " + brokerCompressionCode.name, brokerCompressionCode, readMessage(0).compressionCodec) + } + else + assertEquals("Compression at offset 0 should produce " + messageCompressionCode.name, messageCompressionCode, readMessage(0).compressionCodec) + + } + +} + +object BrokerCompressionTest { + @Parameters + def parameters: Collection[Array[String]] = { + for (brokerCompression <- BrokerCompressionCodec.brokerCompressionOptions; + messageCompression <- CompressionType.values + ) yield Array(messageCompression.name, brokerCompression) + } +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala index 4e45d965bc423..716254aa33894 100644 --- a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala @@ -147,11 +147,11 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { // check uncompressed offsets checkOffsets(messages, 0) var offset = 1234567 - checkOffsets(messages.assignOffsets(new AtomicLong(offset), NoCompressionCodec), offset) - + checkOffsets(messages.assignOffsets(new AtomicLong(offset), NoCompressionCodec , NoCompressionCodec), offset) + // check compressed messages checkOffsets(compressedMessages, 0) - checkOffsets(compressedMessages.assignOffsets(new AtomicLong(offset), DefaultCompressionCodec), offset) + checkOffsets(compressedMessages.assignOffsets(new AtomicLong(offset), DefaultCompressionCodec , DefaultCompressionCodec), offset) } /* check that offsets are assigned based on byte offset from the given base offset */ diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 2377abe4933e0..82dce80d55395 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -21,6 +21,8 @@ import org.junit.Test import junit.framework.Assert._ import org.scalatest.junit.JUnit3Suite import kafka.utils.TestUtils +import kafka.message.GZIPCompressionCodec +import kafka.message.NoCompressionCodec class KafkaConfigTest extends JUnit3Suite { @@ -180,6 +182,30 @@ class KafkaConfigTest extends JUnit3Suite { assertEquals(24 * 7 * 60L * 60L * 1000L, cfg.logRollTimeMillis ) } - + @Test + def testDefaultCompressionType() { + val props = TestUtils.createBrokerConfig(0, 8181) + val serverConfig = new KafkaConfig(props) + + assertEquals(serverConfig.compressionType, "producer") + } + + @Test + def testValidCompressionType() { + val props = TestUtils.createBrokerConfig(0, 8181) + props.put("compression.type", "gzip") + val serverConfig = new KafkaConfig(props) + + assertEquals(serverConfig.compressionType, "gzip") + } + + @Test + def testInvalidCompressionType() { + val props = TestUtils.createBrokerConfig(0, 8181) + props.put("compression.type", "abc") + intercept[IllegalArgumentException] { + new KafkaConfig(props) + } + } } From 45697ed6c6132475b5dd4dd0069a2a192a75d329 Mon Sep 17 00:00:00 2001 From: Joel Koshy Date: Thu, 15 Jan 2015 08:04:29 -0800 Subject: [PATCH 093/491] KAFKA-1499; Trivial follow-up (fix comments and whitespace) --- core/src/main/scala/kafka/log/Log.scala | 6 +++--- core/src/main/scala/kafka/log/LogConfig.scala | 7 +++---- core/src/main/scala/kafka/message/CompressionCodec.scala | 6 ++++-- core/src/main/scala/kafka/server/KafkaConfig.scala | 9 ++++----- core/src/test/scala/kafka/log/LogConfigTest.scala | 2 +- .../unit/kafka/message/ByteBufferMessageSetTest.scala | 4 ++-- 6 files changed, 17 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 86422bfd5122b..846023bb98d0f 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -41,8 +41,8 @@ object LogAppendInfo { * @param lastOffset The last offset in the message set * @param shallowCount The number of shallow messages * @param validBytes The number of valid bytes - * @param sourceCodec The source codec used in the message set(coming from producer) - * @param targetCodec The target codec of the message set(after applying broker compression logic) + * @param sourceCodec The source codec used in the message set (send by the producer) + * @param targetCodec The target codec of the message set(after applying the broker compression configuration if any) * @param offsetsMonotonic Are the offsets in this message set monotonically increasing */ case class LogAppendInfo(var firstOffset: Long, var lastOffset: Long, sourceCodec: CompressionCodec, targetCodec: CompressionCodec, shallowCount: Int, validBytes: Int, offsetsMonotonic: Boolean) @@ -395,7 +395,7 @@ class Log(val dir: File, sourceCodec = messageCodec } - //Apply if any broker-side compression + // Apply broker-side compression if any val targetCodec = BrokerCompressionCodec.getTargetCompressionCodec(config.compressionType, sourceCodec) LogAppendInfo(firstOffset, lastOffset, sourceCodec, targetCodec, shallowMessageCount, validBytesCount, monotonic) diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index 2338b4410ca53..4631bc78106e6 100644 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -152,10 +152,9 @@ object LogConfig { val UncleanLeaderElectionEnableDoc = "Indicates whether unclean leader election is enabled" val MinInSyncReplicasDoc = "If number of insync replicas drops below this number, we stop accepting writes with" + " -1 (or all) required acks" - val CompressionTypeDoc = "This parameter allows you to specify the compression logic for a given topic. This config" + - " is used to retain/remove/change the compression set by the producer. This config takes the following options: " + - " uncompressed, gzip, snappy, lz4, producer. uncompressed means that regardless of what the producer sets, the broker" + - " writes the message decompressed. producer means the broker attempts to retain whatever is used by the producer" + val CompressionTypeDoc = "Specify the final compression type for a given topic. This configuration accepts the " + + "standard compression codecs ('gzip', 'snappy', lz4). It additionally accepts 'uncompressed' which is equivalent to " + + "no compression; and 'producer' which means retain the original compression codec set by the producer." private val configDef = { import ConfigDef.Range._ diff --git a/core/src/main/scala/kafka/message/CompressionCodec.scala b/core/src/main/scala/kafka/message/CompressionCodec.scala index cacde9bca0ee6..c4aa8ce0897d1 100644 --- a/core/src/main/scala/kafka/message/CompressionCodec.scala +++ b/core/src/main/scala/kafka/message/CompressionCodec.scala @@ -53,8 +53,10 @@ object BrokerCompressionCodec { } def getTargetCompressionCodec(compressionType: String, producerCompression: CompressionCodec): CompressionCodec = { - if (ProducerCompressionCodec.name.equals(compressionType)) producerCompression - else getCompressionCodec(compressionType) + if (ProducerCompressionCodec.name.equals(compressionType)) + producerCompression + else + getCompressionCodec(compressionType) } } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 9d1adec06a4f1..d3d8ac42c29a3 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -348,11 +348,10 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro val deleteTopicEnable = props.getBoolean("delete.topic.enable", false) /** - * This parameter allows you to specify the broker-side compression logic. This config is used to - * retain/remove/change the compression set by the producer. This config takes the following options: - * uncompressed, gzip, snappy, lz4, producer. uncompressed means that regardless of what the producer sets, the broker - * writes the message decompressed. producer means the broker attempts to retain whatever is used by the producer" - */ + * Specify the final compression type for a given topic. This configuration accepts the standard compression codecs + * ('gzip', 'snappy', lz4). It additionally accepts 'uncompressed' which is equivalent to no compression; and + * 'producer' which means retain the original compression codec set by the producer." + */ val compressionType = props.getString("compression.type", "producer").toLowerCase() require(BrokerCompressionCodec.isValid(compressionType), "compression.type : "+compressionType + " is not valid." + " Valid options are "+BrokerCompressionCodec.brokerCompressionOptions.mkString(",")) diff --git a/core/src/test/scala/kafka/log/LogConfigTest.scala b/core/src/test/scala/kafka/log/LogConfigTest.scala index fe5bd9d9155a3..9690f141be752 100644 --- a/core/src/test/scala/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/kafka/log/LogConfigTest.scala @@ -51,7 +51,7 @@ class LogConfigTest extends JUnit3Suite { LogConfig.configNames().foreach((name) => { name match { case LogConfig.UncleanLeaderElectionEnableProp => expected.setProperty(name, randFrom("true", "false")) - case LogConfig.CompressionTypeProp => expected.setProperty(name, randFrom("producer","uncompressed","gzip")) + case LogConfig.CompressionTypeProp => expected.setProperty(name, randFrom("producer", "uncompressed", "gzip")) case LogConfig.CleanupPolicyProp => expected.setProperty(name, randFrom(LogConfig.Compact, LogConfig.Delete)) case LogConfig.MinCleanableDirtyRatioProp => expected.setProperty(name, "%.1f".format(nextDouble * .9 + .1)) case LogConfig.MinInSyncReplicasProp => expected.setProperty(name, (nextInt(Int.MaxValue - 1) + 1).toString) diff --git a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala index 716254aa33894..73a26377eb63a 100644 --- a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala @@ -147,11 +147,11 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { // check uncompressed offsets checkOffsets(messages, 0) var offset = 1234567 - checkOffsets(messages.assignOffsets(new AtomicLong(offset), NoCompressionCodec , NoCompressionCodec), offset) + checkOffsets(messages.assignOffsets(new AtomicLong(offset), NoCompressionCodec, NoCompressionCodec), offset) // check compressed messages checkOffsets(compressedMessages, 0) - checkOffsets(compressedMessages.assignOffsets(new AtomicLong(offset), DefaultCompressionCodec , DefaultCompressionCodec), offset) + checkOffsets(compressedMessages.assignOffsets(new AtomicLong(offset), DefaultCompressionCodec, DefaultCompressionCodec), offset) } /* check that offsets are assigned based on byte offset from the given base offset */ From 602d572f6365f80f14e8c968390972ea9767e263 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Thu, 15 Jan 2015 17:58:32 -0800 Subject: [PATCH 094/491] kafka-1868; ConsoleConsumer shouldn't override dual.commit.enabled to false if not explicitly set; patched by Jun Rao; reviewed by Jeol Koshy --- core/src/main/scala/kafka/tools/ConsoleConsumer.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala index 323fc8566d974..e455cb9a1de22 100644 --- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala @@ -126,10 +126,6 @@ object ConsoleConsumer extends Logging { } consumerProps.put("auto.offset.reset", if(options.has(resetBeginningOpt)) "smallest" else "largest") consumerProps.put("zookeeper.connect", options.valueOf(zkConnectOpt)) - if(!consumerProps.containsKey("dual.commit.enabled")) - consumerProps.put("dual.commit.enabled","false") - if(!consumerProps.containsKey("offsets.storage")) - consumerProps.put("offsets.storage","zookeeper") if (!options.has(deleteConsumerOffsetsOpt) && options.has(resetBeginningOpt) && checkZkPathExists(options.valueOf(zkConnectOpt),"/consumers/" + consumerProps.getProperty("group.id")+ "/offsets")) { From 1769642bb779921267bd57d3d338591dbdf33842 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Fri, 16 Jan 2015 18:34:39 -0800 Subject: [PATCH 095/491] kafka-1870; Cannot commit with simpleConsumer on Zookeeper only with Java API; patched by Jun Rao; reviewed by Jeol Koshy and Sriharsha Chintalapani --- core/src/main/scala/kafka/consumer/SimpleConsumer.scala | 2 ++ core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala | 1 + core/src/main/scala/kafka/javaapi/OffsetFetchRequest.scala | 2 +- .../main/scala/kafka/javaapi/consumer/SimpleConsumer.scala | 4 ++-- 4 files changed, 6 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala index e53ee51638b93..cbef84ac76e62 100644 --- a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala @@ -128,6 +128,7 @@ class SimpleConsumer(val host: String, /** * Commit offsets for a topic + * Version 0 of the request will commit offsets to Zookeeper and version 1 and above will commit offsets to Kafka. * @param request a [[kafka.api.OffsetCommitRequest]] object. * @return a [[kafka.api.OffsetCommitResponse]] object. */ @@ -139,6 +140,7 @@ class SimpleConsumer(val host: String, /** * Fetch offsets for a topic + * Version 0 of the request will fetch offsets from Zookeeper and version 1 and above will fetch offsets from Kafka. * @param request a [[kafka.api.OffsetFetchRequest]] object. * @return a [[kafka.api.OffsetFetchResponse]] object. */ diff --git a/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala b/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala index 27fc1eb9d578e..873f575cc22af 100644 --- a/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala +++ b/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala @@ -32,6 +32,7 @@ class OffsetCommitRequest(groupId: String, kafka.api.OffsetCommitRequest( groupId = groupId, requestInfo = scalaMap, + versionId = 0, // binds to version 0 so that it commits to Zookeeper correlationId = correlationId, clientId = clientId ) diff --git a/core/src/main/scala/kafka/javaapi/OffsetFetchRequest.scala b/core/src/main/scala/kafka/javaapi/OffsetFetchRequest.scala index 5b4f4bbcdd606..1c25aa3332f9e 100644 --- a/core/src/main/scala/kafka/javaapi/OffsetFetchRequest.scala +++ b/core/src/main/scala/kafka/javaapi/OffsetFetchRequest.scala @@ -36,7 +36,7 @@ class OffsetFetchRequest(groupId: String, kafka.api.OffsetFetchRequest( groupId = groupId, requestInfo = scalaSeq, - versionId = versionId, + versionId = 0, // binds to version 0 so that it commits to Zookeeper correlationId = correlationId, clientId = clientId ) diff --git a/core/src/main/scala/kafka/javaapi/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/javaapi/consumer/SimpleConsumer.scala index 0ab0195dc9f66..abf6069d2033b 100644 --- a/core/src/main/scala/kafka/javaapi/consumer/SimpleConsumer.scala +++ b/core/src/main/scala/kafka/javaapi/consumer/SimpleConsumer.scala @@ -80,7 +80,7 @@ class SimpleConsumer(val host: String, } /** - * Commit offsets for a topic + * Commit offsets for a topic to Zookeeper * @param request a [[kafka.javaapi.OffsetCommitRequest]] object. * @return a [[kafka.javaapi.OffsetCommitResponse]] object. */ @@ -90,7 +90,7 @@ class SimpleConsumer(val host: String, } /** - * Fetch offsets for a topic + * Fetch offsets for a topic from Zookeeper * @param request a [[kafka.javaapi.OffsetFetchRequest]] object. * @return a [[kafka.javaapi.OffsetFetchResponse]] object. */ From 5174df53778cb5cb2d6d86e4cec9f3185a2c85db Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Fri, 16 Jan 2015 18:56:32 -0800 Subject: [PATCH 096/491] kafka-1864; Revisit defaults for the internal offsets topic; patched by Jun Rao; reviewed by Jeol Koshy, Neha Narkhede, and Gwen Shapira --- core/src/main/scala/kafka/server/KafkaApis.scala | 11 +++++++++-- core/src/main/scala/kafka/server/KafkaConfig.scala | 6 +++++- core/src/main/scala/kafka/server/OffsetManager.scala | 4 ++-- .../kafka/api/ProducerFailureHandlingTest.scala | 7 +++++++ .../scala/unit/kafka/server/OffsetCommitTest.scala | 7 +++++++ 5 files changed, 30 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index c011a1b79bd6c..ec8d9f7ba4474 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -351,10 +351,17 @@ class KafkaApis(val requestChannel: RequestChannel, if (topic == OffsetManager.OffsetsTopicName || config.autoCreateTopicsEnable) { try { if (topic == OffsetManager.OffsetsTopicName) { - AdminUtils.createTopic(zkClient, topic, config.offsetsTopicPartitions, config.offsetsTopicReplicationFactor, + val aliveBrokers = metadataCache.getAliveBrokers + val offsetsTopicReplicationFactor = + if (aliveBrokers.length > 0) + Math.min(config.offsetsTopicReplicationFactor, aliveBrokers.length) + else + config.offsetsTopicReplicationFactor + AdminUtils.createTopic(zkClient, topic, config.offsetsTopicPartitions, + offsetsTopicReplicationFactor, offsetManager.offsetsTopicConfig) info("Auto creation of topic %s with %d partitions and replication factor %d is successful!" - .format(topic, config.offsetsTopicPartitions, config.offsetsTopicReplicationFactor)) + .format(topic, config.offsetsTopicPartitions, offsetsTopicReplicationFactor)) } else { AdminUtils.createTopic(zkClient, topic, config.numPartitions, config.defaultReplicationFactor) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index d3d8ac42c29a3..88689df718364 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -312,7 +312,11 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro val offsetsLoadBufferSize = props.getIntInRange("offsets.load.buffer.size", OffsetManagerConfig.DefaultLoadBufferSize, (1, Integer.MAX_VALUE)) - /** The replication factor for the offset commit topic (set higher to ensure availability). */ + /** The replication factor for the offsets topic (set higher to ensure availability). To + * ensure that the effective replication factor of the offsets topic is the configured value, + * the number of alive brokers has to be at least the replication factor at the time of the + * first request for the offsets topic. If not, either the offsets topic creation will fail or + * it will get a replication factor of min(alive brokers, configured replication factor) */ val offsetsTopicReplicationFactor: Short = props.getShortInRange("offsets.topic.replication.factor", OffsetManagerConfig.DefaultOffsetsTopicReplicationFactor, (1, Short.MaxValue)) diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 3c79428962604..0bdd42fea931c 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -75,9 +75,9 @@ object OffsetManagerConfig { val DefaultMaxMetadataSize = 4096 val DefaultLoadBufferSize = 5*1024*1024 val DefaultOffsetsRetentionCheckIntervalMs = 600000L - val DefaultOffsetsTopicNumPartitions = 1 + val DefaultOffsetsTopicNumPartitions = 50 val DefaultOffsetsTopicSegmentBytes = 100*1024*1024 - val DefaultOffsetsTopicReplicationFactor = 1.toShort + val DefaultOffsetsTopicReplicationFactor = 3.toShort val DefaultOffsetsTopicCompressionCodec = NoCompressionCodec val DefaultOffsetCommitTimeoutMs = 5000 val DefaultOffsetCommitRequiredAcks = (-1).toShort diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index 5ec613cdb50b9..420a1dd30264c 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -46,6 +46,13 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { override val zkConnect = TestZKUtils.zookeeperConnect override val autoCreateTopicsEnable = false override val messageMaxBytes = serverMessageMaxBytes + // TODO: Currently, when there is no topic in a cluster, the controller doesn't send any UpdateMetadataRequest to + // the broker. As a result, the live broker list in metadataCache is empty. If the number of live brokers is 0, we + // try to create the offset topic with the default offsets.topic.replication.factor of 3. The creation will fail + // since there is not enough live brokers. This causes testCannotSendToInternalTopic() to fail. Temporarily fixing + // the issue by overriding offsets.topic.replication.factor to 1 for now. When we fix KAFKA-1867, we need to + // remove the following config override. + override val offsetsTopicReplicationFactor = 1.asInstanceOf[Short] } diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index 4a3a5b264a021..5b93239cdc26b 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -46,6 +46,13 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { override def setUp() { super.setUp() val config: Properties = createBrokerConfig(1, brokerPort) + // TODO: Currently, when there is no topic in a cluster, the controller doesn't send any UpdateMetadataRequest to + // the broker. As a result, the live broker list in metadataCache is empty. This causes the ConsumerMetadataRequest + // to fail since if the number of live brokers is 0, we try to create the offset topic with the default + // offsets.topic.replication.factor of 3. The creation will fail since there is not enough live brokers. In order + // for the unit test to pass, overriding offsets.topic.replication.factor to 1 for now. When we fix KAFKA-1867, we + // need to remove the following config override. + config.put("offsets.topic.replication.factor", "1") val logDirPath = config.getProperty("log.dir") logDir = new File(logDirPath) time = new MockTime() From 09234f0e5206b8cb55e89cffc4a0ecfbaafc7f86 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Fri, 16 Jan 2015 19:04:11 -0800 Subject: [PATCH 097/491] trivial change to remove -UseCompressedOops option from script --- bin/kafka-run-class.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index ce3a4d06a27f6..22a9865b59394 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -112,7 +112,7 @@ fi # JVM performance options if [ -z "$KAFKA_JVM_PERFORMANCE_OPTS" ]; then - KAFKA_JVM_PERFORMANCE_OPTS="-server -XX:+UseCompressedOops -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:+CMSClassUnloadingEnabled -XX:+CMSScavengeBeforeRemark -XX:+DisableExplicitGC -Djava.awt.headless=true" + KAFKA_JVM_PERFORMANCE_OPTS="-server -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:+CMSClassUnloadingEnabled -XX:+CMSScavengeBeforeRemark -XX:+DisableExplicitGC -Djava.awt.headless=true" fi From 49d7f8ee1cd96a99f44ac48043db9ad8a5336c7d Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Mon, 19 Jan 2015 08:44:53 -0800 Subject: [PATCH 098/491] KAFKA-1723; num.partitions documented default is 1 while actual default is 2; patched by Manikumar Reddy; reviewed by Jun Rao --- config/server.properties | 4 ++-- core/src/main/scala/kafka/server/KafkaConfig.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/config/server.properties b/config/server.properties index b0e4496a8ca73..1614260b71a65 100644 --- a/config/server.properties +++ b/config/server.properties @@ -46,7 +46,7 @@ num.io.threads=8 socket.send.buffer.bytes=102400 # The receive buffer (SO_RCVBUF) used by the socket server -socket.receive.buffer.bytes=65536 +socket.receive.buffer.bytes=102400 # The maximum size of a request that the socket server will accept (protection against OOM) socket.request.max.bytes=104857600 @@ -118,4 +118,4 @@ log.cleaner.enable=false zookeeper.connect=localhost:2181 # Timeout in ms for connecting to zookeeper -zookeeper.connection.timeout.ms=2000 +zookeeper.connection.timeout.ms=6000 diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 88689df718364..6d74983472249 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -100,7 +100,7 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /*********** Socket Server Configuration ***********/ /* the port to listen and accept connections on */ - val port: Int = props.getInt("port", 6667) + val port: Int = props.getInt("port", 9092) /* hostname of broker. If this is set, it will only bind to this address. If this is not set, * it will bind to all interfaces */ From 07cff763633f20f51310e4f123963aaa0c008b2f Mon Sep 17 00:00:00 2001 From: jaikiran pai Date: Mon, 19 Jan 2015 09:05:25 -0800 Subject: [PATCH 099/491] KAFKA-1878; ProducerFailureHandlingTest.testCannotSendToInternalTopic fails with TimeoutException while trying to fetch metadata for topic; patched by jaikiran pai; reviewed by Jun Rao --- .../integration/kafka/api/ProducerFailureHandlingTest.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index 420a1dd30264c..90c0b7a19c7af 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -53,6 +53,9 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { // the issue by overriding offsets.topic.replication.factor to 1 for now. When we fix KAFKA-1867, we need to // remove the following config override. override val offsetsTopicReplicationFactor = 1.asInstanceOf[Short] + // 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 + override val offsetsTopicPartitions = 1 } @@ -309,7 +312,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { val thrown = intercept[ExecutionException] { producer2.send(new ProducerRecord[Array[Byte],Array[Byte]](Topic.InternalTopics.head, "test".getBytes, "test".getBytes)).get } - assertTrue(thrown.getCause.isInstanceOf[InvalidTopicException]) + assertTrue("Unexpected exception while sending to an invalid topic " + thrown.getCause, thrown.getCause.isInstanceOf[InvalidTopicException]) } @Test From 5d648cfdb13dee078bb1b3415f77dbe2d47d7ec4 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Mon, 19 Jan 2015 11:18:23 -0800 Subject: [PATCH 100/491] KAFKA-1876; pom file for scala 2.11 should reference a specific version; patched by Jun Rao; reviewed by Joe Stein and Sriharsha Chintalapani --- README.md | 2 +- build.gradle | 14 +++++++------- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/README.md b/README.md index 11dfdf9379ad1..35e06b1cc6373 100644 --- a/README.md +++ b/README.md @@ -49,7 +49,7 @@ The release file can be found inside ./core/build/distributions/. ### Cleaning the build ### ./gradlew clean -### Running a task on a particular version of Scala (either 2.9.1, 2.9.2, 2.10.1 or 2.11) ### +### Running a task on a particular version of Scala (either 2.9.1, 2.9.2, 2.10.4 or 2.11.5) ### #### (If building a jar with a version other than 2.10, need to set SCALA_BINARY_VERSION variable or change it in bin/kafka-run-class.sh to run quick start.) #### ./gradlew -PscalaVersion=2.9.1 jar ./gradlew -PscalaVersion=2.9.1 test diff --git a/build.gradle b/build.gradle index c9ac43378c3bf..1cbab29ce83e2 100644 --- a/build.gradle +++ b/build.gradle @@ -137,7 +137,7 @@ subprojects { } } -for ( sv in ['2_9_1', '2_9_2', '2_10_4', '2_11'] ) { +for ( sv in ['2_9_1', '2_9_2', '2_10_4', '2_11_5'] ) { String svInDot = sv.replaceAll( "_", ".") tasks.create(name: "jar_core_${sv}", type: GradleBuild) { @@ -177,20 +177,20 @@ for ( sv in ['2_9_1', '2_9_2', '2_10_4', '2_11'] ) { } } -tasks.create(name: "jarAll", dependsOn: ['jar_core_2_9_1', 'jar_core_2_9_2', 'jar_core_2_10_4', 'jar_core_2_11', 'clients:jar', 'examples:jar', 'contrib:hadoop-consumer:jar', 'contrib:hadoop-producer:jar']) { +tasks.create(name: "jarAll", dependsOn: ['jar_core_2_9_1', 'jar_core_2_9_2', 'jar_core_2_10_4', 'jar_core_2_11_5', 'clients:jar', 'examples:jar', 'contrib:hadoop-consumer:jar', 'contrib:hadoop-producer:jar']) { } -tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_9_1', 'srcJar_2_9_2', 'srcJar_2_10_4', 'srcJar_2_11', 'clients:srcJar', 'examples:srcJar', 'contrib:hadoop-consumer:srcJar', 'contrib:hadoop-producer:srcJar']) { } +tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_9_1', 'srcJar_2_9_2', 'srcJar_2_10_4', 'srcJar_2_11_5', 'clients:srcJar', 'examples:srcJar', 'contrib:hadoop-consumer:srcJar', 'contrib:hadoop-producer:srcJar']) { } -tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_9_1', 'docsJar_2_9_2', 'docsJar_2_10_4', 'docsJar_2_11', 'clients:docsJar', 'examples:docsJar', 'contrib:hadoop-consumer:docsJar', 'contrib:hadoop-producer:docsJar']) { } +tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_9_1', 'docsJar_2_9_2', 'docsJar_2_10_4', 'docsJar_2_11_5', 'clients:docsJar', 'examples:docsJar', 'contrib:hadoop-consumer:docsJar', 'contrib:hadoop-producer:docsJar']) { } -tasks.create(name: "testAll", dependsOn: ['test_core_2_9_1', 'test_core_2_9_2', 'test_core_2_10_4', 'test_core_2_11', 'clients:test']) { +tasks.create(name: "testAll", dependsOn: ['test_core_2_9_1', 'test_core_2_9_2', 'test_core_2_10_4', 'test_core_2_11_5', 'clients:test']) { } -tasks.create(name: "releaseTarGzAll", dependsOn: ['releaseTarGz_2_9_1', 'releaseTarGz_2_9_2', 'releaseTarGz_2_10_4', 'releaseTarGz_2_11']) { +tasks.create(name: "releaseTarGzAll", dependsOn: ['releaseTarGz_2_9_1', 'releaseTarGz_2_9_2', 'releaseTarGz_2_10_4', 'releaseTarGz_2_11_5']) { } -tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_9_1', 'uploadCoreArchives_2_9_2', 'uploadCoreArchives_2_10_4', 'uploadCoreArchives_2_11', 'clients:uploadArchives', 'examples:uploadArchives', 'contrib:hadoop-consumer:uploadArchives', 'contrib:hadoop-producer:uploadArchives']) { +tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_9_1', 'uploadCoreArchives_2_9_2', 'uploadCoreArchives_2_10_4', 'uploadCoreArchives_2_11_5', 'clients:uploadArchives', 'examples:uploadArchives', 'contrib:hadoop-consumer:uploadArchives', 'contrib:hadoop-producer:uploadArchives']) { } project(':core') { From fcc2521858efd1bd99e06c319de9489330125a4d Mon Sep 17 00:00:00 2001 From: Joel Koshy Date: Tue, 20 Jan 2015 13:13:50 -0800 Subject: [PATCH 101/491] KAFKA-1823; Fix transient failure in PartitionAssignorTest; reviewed by Guozhang Wang and Neha Narkhede --- .../unit/kafka/consumer/PartitionAssignorTest.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala b/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala index 24954de66ccc5..543070f4fd3e9 100644 --- a/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala @@ -187,10 +187,12 @@ private object PartitionAssignorTest extends Logging { // check for uniform assignment if (verifyAssignmentIsUniform) { val partitionCountForStream = partitionCountPerStream(globalAssignment) - val maxCount = partitionCountForStream.valuesIterator.max - val minCount = partitionCountForStream.valuesIterator.min - assertTrue("Scenario %s: assignment is not uniform (partition counts per stream are in the range [%d, %d])" - .format(scenario, minCount, maxCount), (maxCount - minCount) <= 1) + if (partitionCountForStream.nonEmpty) { + val maxCount = partitionCountForStream.valuesIterator.max + val minCount = partitionCountForStream.valuesIterator.min + assertTrue("Scenario %s: assignment is not uniform (partition counts per stream are in the range [%d, %d])" + .format(scenario, minCount, maxCount), (maxCount - minCount) <= 1) + } } } From a0852d477ec097bd4efe120c21de020fccf6dc49 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Wed, 21 Jan 2015 17:25:54 -0800 Subject: [PATCH 102/491] KAFKA-1848; check consumer shutting down flag inside retry loop; reviewed by Guozhang Wang --- .../consumer/ZookeeperConsumerConnector.scala | 57 +++++++++---------- 1 file changed, 28 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index 191a8677444e5..5487259751ebe 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -613,36 +613,35 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, def syncedRebalance() { rebalanceLock synchronized { rebalanceTimer.time { - if(isShuttingDown.get()) { - return - } else { - for (i <- 0 until config.rebalanceMaxRetries) { - info("begin rebalancing consumer " + consumerIdString + " try #" + i) - var done = false - var cluster: Cluster = null - try { - cluster = getCluster(zkClient) - done = rebalance(cluster) - } catch { - case e: Throwable => - /** occasionally, we may hit a ZK exception because the ZK state is changing while we are iterating. - * For example, a ZK node can disappear between the time we get all children and the time we try to get - * the value of a child. Just let this go since another rebalance will be triggered. - **/ - info("exception during rebalance ", e) - } - info("end rebalancing consumer " + consumerIdString + " try #" + i) - if (done) { - return - } else { - /* Here the cache is at a risk of being stale. To take future rebalancing decisions correctly, we should - * clear the cache */ - info("Rebalancing attempt failed. Clearing the cache before the next rebalancing operation is triggered") - } - // stop all fetchers and clear all the queues to avoid data duplication - closeFetchersForQueues(cluster, kafkaMessageAndMetadataStreams, topicThreadIdAndQueues.map(q => q._2)) - Thread.sleep(config.rebalanceBackoffMs) + for (i <- 0 until config.rebalanceMaxRetries) { + if(isShuttingDown.get()) { + return + } + info("begin rebalancing consumer " + consumerIdString + " try #" + i) + var done = false + var cluster: Cluster = null + try { + cluster = getCluster(zkClient) + done = rebalance(cluster) + } catch { + case e: Throwable => + /** occasionally, we may hit a ZK exception because the ZK state is changing while we are iterating. + * For example, a ZK node can disappear between the time we get all children and the time we try to get + * the value of a child. Just let this go since another rebalance will be triggered. + **/ + info("exception during rebalance ", e) + } + info("end rebalancing consumer " + consumerIdString + " try #" + i) + if (done) { + return + } else { + /* Here the cache is at a risk of being stale. To take future rebalancing decisions correctly, we should + * clear the cache */ + info("Rebalancing attempt failed. Clearing the cache before the next rebalancing operation is triggered") } + // stop all fetchers and clear all the queues to avoid data duplication + closeFetchersForQueues(cluster, kafkaMessageAndMetadataStreams, topicThreadIdAndQueues.map(q => q._2)) + Thread.sleep(config.rebalanceBackoffMs) } } } From 54e53b7312274a3599229a36caa35f0cd931e341 Mon Sep 17 00:00:00 2001 From: Joe Stein Date: Thu, 22 Jan 2015 22:29:33 -0500 Subject: [PATCH 103/491] KAFKA-1891 MirrorMaker hides consumer exception - making troubleshooting challenging patch by Gwen Shapira reviewed by Joe Stein --- core/src/main/scala/kafka/tools/MirrorMaker.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index 5cbc8103e33a0..a11dbc8def148 100644 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -273,7 +273,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { streams = connector.createMessageStreamsByFilter(filterSpec, numStreams, new DefaultDecoder(), new DefaultDecoder()) } catch { case t: Throwable => - fatal("Unable to create stream - shutting down mirror maker.") + fatal("Unable to create stream - shutting down mirror maker.", t) connector.shutdown() } consumerThreads = streams.zipWithIndex.map(streamAndIndex => new ConsumerThread(streamAndIndex._1, mirrorDataChannel, streamAndIndex._2)) From 8cff9119f8d125e55898e2095512b90826522336 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Fri, 23 Jan 2015 09:07:12 -0800 Subject: [PATCH 104/491] KAFKA-1890 Fix bug preventing Mirror Maker from successful rebalance; reviewed by Gwen Shapira and Neha Narkhede --- core/src/main/scala/kafka/tools/MirrorMaker.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index a11dbc8def148..3ee84ed9cbde5 100644 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -213,11 +213,11 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { val customRebalanceListenerClass = options.valueOf(consumerRebalanceListenerOpt) val customRebalanceListener = { if (customRebalanceListenerClass != null) - Utils.createObject[ConsumerRebalanceListener](customRebalanceListenerClass) + Some(Utils.createObject[ConsumerRebalanceListener](customRebalanceListenerClass)) else - null + None } - consumerRebalanceListener = new InternalRebalanceListener(mirrorDataChannel, Some(customRebalanceListener)) + consumerRebalanceListener = new InternalRebalanceListener(mirrorDataChannel, customRebalanceListener) connector.setConsumerRebalanceListener(consumerRebalanceListener) // create producer threads From ff5cbcacc6fc3217ea41c624fb293adeae7bb672 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Fri, 23 Jan 2015 18:23:32 -0800 Subject: [PATCH 105/491] KAFKA-1896; Record size function should check if value is null; reviewed by Guozhang Wang --- core/src/main/scala/kafka/tools/MirrorMaker.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index 3ee84ed9cbde5..81ae205ef7b20 100644 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -637,7 +637,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { val sourceOffset: Long, val key: Array[Byte], val value: Array[Byte]) { - def size = value.length + {if (key == null) 0 else key.length} + def size = {if (value == null) 0 else value.length} + {if (key == null) 0 else key.length} } private class UnackedOffset(offset: Long) extends DoublyLinkedListNode[Long](offset) { From 995d0d369363ef3ad7cf43980746c52768b4df2e Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Sun, 25 Jan 2015 18:43:17 -0800 Subject: [PATCH 106/491] KAFKA-1109 Need to fix GC log configuration code, not able to override KAFKA_GC_LOG_OPTS; reviewed by Neha Narkhede --- bin/kafka-run-class.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index 22a9865b59394..881f578a8f5c7 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -125,7 +125,7 @@ while [ $# -gt 0 ]; do shift 2 ;; -loggc) - if [ -z "$KAFKA_GC_LOG_OPTS"] ; then + if [ -z "$KAFKA_GC_LOG_OPTS" ]; then GC_LOG_ENABLED="true" fi shift From 4aa3dab3de088096461941353ba27cb37f1bd9d1 Mon Sep 17 00:00:00 2001 From: jaikiran pai Date: Sun, 25 Jan 2015 18:54:51 -0800 Subject: [PATCH 107/491] KAFKA-1883 Fix NullPointerException in RequestSendThread; reviewed by Neha Narkhede --- .../controller/ControllerChannelManager.scala | 32 ++++++++++--------- 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index eb492f0044974..fbef34cad16af 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -125,7 +125,7 @@ class RequestSendThread(val controllerId: Int, try { lock synchronized { var isSendSuccessful = false - while(isRunning.get() && !isSendSuccessful) { + while (isRunning.get() && !isSendSuccessful) { // if a broker goes down for a long time, then at some point the controller's zookeeper listener will trigger a // removeBroker which will invoke shutdown() on this thread. At that point, we will stop retrying. try { @@ -136,7 +136,7 @@ class RequestSendThread(val controllerId: Int, case e: Throwable => // if the send was not successful, reconnect to broker and resend the message warn(("Controller %d epoch %d fails to send request %s to broker %s. " + "Reconnecting to broker.").format(controllerId, controllerContext.epoch, - request.toString, toBroker.toString()), e) + request.toString, toBroker.toString()), e) channel.disconnect() connectToBroker(toBroker, channel) isSendSuccessful = false @@ -144,20 +144,22 @@ class RequestSendThread(val controllerId: Int, Utils.swallow(Thread.sleep(300)) } } - var response: RequestOrResponse = null - request.requestId.get match { - case RequestKeys.LeaderAndIsrKey => - response = LeaderAndIsrResponse.readFrom(receive.buffer) - case RequestKeys.StopReplicaKey => - response = StopReplicaResponse.readFrom(receive.buffer) - case RequestKeys.UpdateMetadataKey => - response = UpdateMetadataResponse.readFrom(receive.buffer) - } - stateChangeLogger.trace("Controller %d epoch %d received response %s for a request sent to broker %s" - .format(controllerId, controllerContext.epoch, response.toString, toBroker.toString)) + if (receive != null) { + var response: RequestOrResponse = null + request.requestId.get match { + case RequestKeys.LeaderAndIsrKey => + response = LeaderAndIsrResponse.readFrom(receive.buffer) + case RequestKeys.StopReplicaKey => + response = StopReplicaResponse.readFrom(receive.buffer) + case RequestKeys.UpdateMetadataKey => + response = UpdateMetadataResponse.readFrom(receive.buffer) + } + stateChangeLogger.trace("Controller %d epoch %d received response %s for a request sent to broker %s" + .format(controllerId, controllerContext.epoch, response.toString, toBroker.toString)) - if(callback != null) { - callback(response) + if (callback != null) { + callback(response) + } } } } catch { From 0b171dee1fca16595f635bc9991056712e9e3b1a Mon Sep 17 00:00:00 2001 From: jaikiran pai Date: Sun, 25 Jan 2015 19:15:51 -0800 Subject: [PATCH 108/491] KAFKA-1885 Upgrade junit dependency in core to 4.6 version to allow running individual test methods via gradle command line; reviewed by Neha Narkhede --- README.md | 5 +++++ build.gradle | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 35e06b1cc6373..9bdcf7052434d 100644 --- a/README.md +++ b/README.md @@ -34,6 +34,11 @@ Follow instuctions in http://kafka.apache.org/documentation.html#quickstart ### Running a particular unit test ### ./gradlew -Dtest.single=RequestResponseSerializationTest core:test +### Running a particular test method within a unit test ### + ./gradlew core:test --tests kafka.api.test.ProducerFailureHandlingTest.testCannotSendToInternalTopic + ./gradlew clients:test --tests org.apache.kafka.clients.producer.MetadataTest.testMetadataUpdateWaitTime + + ### Running a particular unit test with log4j output ### change the log4j setting in either clients/src/test/resources/log4j.properties or core/src/test/resources/log4j.properties ./gradlew -i -Dtest.single=RequestResponseSerializationTest core:test diff --git a/build.gradle b/build.gradle index 1cbab29ce83e2..3a509c347200d 100644 --- a/build.gradle +++ b/build.gradle @@ -209,7 +209,7 @@ project(':core') { compile 'com.yammer.metrics:metrics-core:2.2.0' compile 'net.sf.jopt-simple:jopt-simple:3.2' - testCompile 'junit:junit:4.1' + testCompile 'junit:junit:4.6' testCompile 'org.easymock:easymock:3.0' testCompile 'org.objenesis:objenesis:1.2' testCompile project(':clients') From 0f6cc0a058bbd2736f1e638448fc64b58a9f6b41 Mon Sep 17 00:00:00 2001 From: Eric Olander Date: Sun, 25 Jan 2015 19:27:54 -0800 Subject: [PATCH 109/491] KAFKA-1818 KAFKA-1818 clean up code to more idiomatic scala usage; reviewed by Neha Narkhede and Gwen Shapira --- .../scala/kafka/utils/ReplicationUtils.scala | 32 +++++++------------ .../kafka/utils/ReplicationUtilsTest.scala | 10 ++++++ 2 files changed, 22 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/kafka/utils/ReplicationUtils.scala b/core/src/main/scala/kafka/utils/ReplicationUtils.scala index 715767380f7c2..60687332b4c9b 100644 --- a/core/src/main/scala/kafka/utils/ReplicationUtils.scala +++ b/core/src/main/scala/kafka/utils/ReplicationUtils.scala @@ -61,30 +61,22 @@ object ReplicationUtils extends Logging { def getLeaderIsrAndEpochForPartition(zkClient: ZkClient, topic: String, partition: Int):Option[LeaderIsrAndControllerEpoch] = { val leaderAndIsrPath = ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partition) - val leaderAndIsrInfo = ZkUtils.readDataMaybeNull(zkClient, leaderAndIsrPath) - val leaderAndIsrOpt = leaderAndIsrInfo._1 - val stat = leaderAndIsrInfo._2 - leaderAndIsrOpt match { - case Some(leaderAndIsrStr) => parseLeaderAndIsr(leaderAndIsrStr, leaderAndIsrPath, stat) - case None => None - } + val (leaderAndIsrOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, leaderAndIsrPath) + leaderAndIsrOpt.flatMap(leaderAndIsrStr => parseLeaderAndIsr(leaderAndIsrStr, leaderAndIsrPath, stat)) } private def parseLeaderAndIsr(leaderAndIsrStr: String, path: String, stat: Stat) : Option[LeaderIsrAndControllerEpoch] = { - Json.parseFull(leaderAndIsrStr) match { - case Some(m) => - val leaderIsrAndEpochInfo = m.asInstanceOf[Map[String, Any]] - val leader = leaderIsrAndEpochInfo.get("leader").get.asInstanceOf[Int] - val epoch = leaderIsrAndEpochInfo.get("leader_epoch").get.asInstanceOf[Int] - val isr = leaderIsrAndEpochInfo.get("isr").get.asInstanceOf[List[Int]] - val controllerEpoch = leaderIsrAndEpochInfo.get("controller_epoch").get.asInstanceOf[Int] - val zkPathVersion = stat.getVersion - debug("Leader %d, Epoch %d, Isr %s, Zk path version %d for leaderAndIsrPath %s".format(leader, epoch, - isr.toString(), zkPathVersion, path)) - Some(LeaderIsrAndControllerEpoch(LeaderAndIsr(leader, epoch, isr, zkPathVersion), controllerEpoch)) - case None => None - } + Json.parseFull(leaderAndIsrStr).flatMap {m => + val leaderIsrAndEpochInfo = m.asInstanceOf[Map[String, Any]] + val leader = leaderIsrAndEpochInfo.get("leader").get.asInstanceOf[Int] + val epoch = leaderIsrAndEpochInfo.get("leader_epoch").get.asInstanceOf[Int] + val isr = leaderIsrAndEpochInfo.get("isr").get.asInstanceOf[List[Int]] + val controllerEpoch = leaderIsrAndEpochInfo.get("controller_epoch").get.asInstanceOf[Int] + val zkPathVersion = stat.getVersion + debug("Leader %d, Epoch %d, Isr %s, Zk path version %d for leaderAndIsrPath %s".format(leader, epoch, + isr.toString(), zkPathVersion, path)) + Some(LeaderIsrAndControllerEpoch(LeaderAndIsr(leader, epoch, isr, zkPathVersion), controllerEpoch))} } } diff --git a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala index 84e08557de5ac..305498adf416a 100644 --- a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala @@ -17,6 +17,7 @@ package kafka.utils +import kafka.controller.LeaderIsrAndControllerEpoch import kafka.server.{ReplicaFetcherManager, KafkaConfig} import kafka.api.LeaderAndIsr import kafka.zk.ZooKeeperTestHarness @@ -42,6 +43,8 @@ class ReplicationUtilsTest extends JUnit3Suite with ZooKeeperTestHarness { val topicDataMismatch = Json.encode(Map("controller_epoch" -> 1, "leader" -> 1, "versions" -> 2, "leader_epoch" -> 2,"isr" -> List(1,2))) + val topicDataLeaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(LeaderAndIsr(1,leaderEpoch,List(1,2),0), controllerEpoch) + override def setUp() { super.setUp() @@ -92,4 +95,11 @@ class ReplicationUtilsTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(newZkVersion3,-1) } + @Test + def testGetLeaderIsrAndEpochForPartition() { + val leaderIsrAndControllerEpoch = ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partitionId) + assertEquals(topicDataLeaderIsrAndControllerEpoch, leaderIsrAndControllerEpoch.get) + assertEquals(None, ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partitionId + 1)) + } + } From f689e9b1e361cdf47e9966567631c06f614e446d Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Wed, 28 Jan 2015 19:07:39 -0600 Subject: [PATCH 110/491] KAFKA-1902; fix MetricName so that Yammer reporter can work correctly; patched by Jun Rao; reviewed by Manikumar Reddy, Manikumar Reddy and Joel Koshy --- .../kafka/metrics/KafkaMetricsGroup.scala | 27 +++++++++++++++++-- 1 file changed, 25 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala b/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala index e9e49180f6de4..9e31184515697 100644 --- a/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala +++ b/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala @@ -61,9 +61,15 @@ trait KafkaMetricsGroup extends Logging { nameBuilder.append(name) } - KafkaMetricsGroup.toMBeanName(tags).map(mbeanName => nameBuilder.append(",").append(mbeanName)) + val scope: String = KafkaMetricsGroup.toScope(tags).getOrElse(null) + val tagsName = KafkaMetricsGroup.toMBeanName(tags) + tagsName match { + case Some(tn) => + nameBuilder.append(",").append(tn) + case None => + } - new MetricName(group, typeName, name, null, nameBuilder.toString()) + new MetricName(group, typeName, name, scope, nameBuilder.toString()) } def newGauge[T](name: String, metric: Gauge[T], tags: scala.collection.Map[String, String] = Map.empty) = @@ -160,6 +166,23 @@ object KafkaMetricsGroup extends KafkaMetricsGroup with Logging { } } + private def toScope(tags: collection.Map[String, String]): Option[String] = { + val filteredTags = tags + .filter { case (tagKey, tagValue) => tagValue != ""} + if (filteredTags.nonEmpty) { + // convert dot to _ since reporters like Graphite typically use dot to represent hierarchy + val tagsString = filteredTags + .toList.sortWith((t1, t2) => t1._1 < t2._1) + .map { case (key, value) => "%s.%s".format(key, value.replaceAll("\\.", "_"))} + .mkString(".") + + Some(tagsString) + } + else { + None + } + } + def removeAllConsumerMetrics(clientId: String) { FetchRequestAndResponseStatsRegistry.removeConsumerFetchRequestAndResponseStats(clientId) ConsumerTopicStatsRegistry.removeConsumerTopicStat(clientId) From 11ec9bf5a5f150e5c172f88bb3e60b197ee9df5a Mon Sep 17 00:00:00 2001 From: Manikumar Reddy Date: Wed, 28 Jan 2015 19:33:15 -0600 Subject: [PATCH 111/491] KAFKA-1861; Publishing kafka-client:test in order to utilize the helper utils in TestUtils; patched by Manikumar Reddy; reviewed by Jun Rao --- build.gradle | 3 +++ 1 file changed, 3 insertions(+) diff --git a/build.gradle b/build.gradle index 3a509c347200d..a980f61def59a 100644 --- a/build.gradle +++ b/build.gradle @@ -375,4 +375,7 @@ project(':clients') { include "**/org/apache/kafka/common/serialization/*" } + artifacts { + archives testJar + } } From 0699ff2ce60abb466cab5315977a224f1a70a4da Mon Sep 17 00:00:00 2001 From: Jay Kreps Date: Sun, 11 Jan 2015 11:29:48 -0800 Subject: [PATCH 112/491] KAFKA-1760: New consumer. --- build.gradle | 1 + .../apache/kafka/clients/ClientRequest.java | 19 +- .../clients/ClusterConnectionStates.java | 30 +- .../kafka/clients/CommonClientConfigs.java | 58 + .../apache/kafka/clients/ConnectionState.java | 2 +- .../org/apache/kafka/clients/KafkaClient.java | 44 +- .../apache/kafka/clients/NetworkClient.java | 138 +- ...ate.java => RequestCompletionHandler.java} | 18 +- .../kafka/clients/consumer/CommitType.java | 5 + .../kafka/clients/consumer/Consumer.java | 109 +- .../clients/consumer/ConsumerConfig.java | 296 ++- .../consumer/ConsumerRebalanceCallback.java | 89 +- .../clients/consumer/ConsumerRecord.java | 89 +- .../clients/consumer/ConsumerRecords.java | 107 +- .../kafka/clients/consumer/KafkaConsumer.java | 1865 +++++++++++++---- .../kafka/clients/consumer/MockConsumer.java | 233 +- .../NoOffsetForPartitionException.java | 29 + .../clients/consumer/OffsetMetadata.java | 59 - .../clients/consumer/internals/Heartbeat.java | 47 + .../NoOpConsumerRebalanceCallback.java | 30 + .../consumer/internals/SubscriptionState.java | 166 ++ .../kafka/clients/producer/KafkaProducer.java | 3 +- .../kafka/clients/producer/MockProducer.java | 2 +- .../clients/producer/ProducerConfig.java | 64 +- .../clients/producer/internals/Metadata.java | 22 +- .../producer/internals/Partitioner.java | 27 +- .../clients/producer/internals/Sender.java | 155 +- .../java/org/apache/kafka/common/Cluster.java | 14 + .../apache/kafka/common/PartitionInfo.java | 4 +- .../apache/kafka/common/config/ConfigDef.java | 26 +- .../kafka/common/errors/ApiException.java | 2 +- .../kafka/common/network/Selectable.java | 58 +- .../apache/kafka/common/network/Selector.java | 99 +- .../apache/kafka/common/protocol/Errors.java | 59 +- .../kafka/common/protocol/types/Struct.java | 46 +- .../apache/kafka/common/record/LogEntry.java | 4 + .../kafka/common/record/MemoryRecords.java | 48 +- .../requests/ConsumerMetadataRequest.java | 9 +- .../requests/ConsumerMetadataResponse.java | 17 +- .../kafka/common/requests/FetchRequest.java | 74 +- .../kafka/common/requests/FetchResponse.java | 21 +- .../common/requests/HeartbeatRequest.java | 13 +- .../common/requests/HeartbeatResponse.java | 9 +- .../common/requests/JoinGroupRequest.java | 17 +- .../common/requests/JoinGroupResponse.java | 25 +- .../common/requests/ListOffsetRequest.java | 25 +- .../common/requests/ListOffsetResponse.java | 19 +- .../common/requests/MetadataRequest.java | 9 +- .../common/requests/MetadataResponse.java | 33 +- .../common/requests/OffsetCommitRequest.java | 45 +- .../common/requests/OffsetCommitResponse.java | 17 +- .../common/requests/OffsetFetchRequest.java | 17 +- .../common/requests/OffsetFetchResponse.java | 46 +- .../kafka/common/requests/ProduceRequest.java | 21 +- .../common/requests/ProduceResponse.java | 19 +- .../common/serialization/Deserializer.java | 2 +- .../org/apache/kafka/common/utils/Utils.java | 42 + .../org/apache/kafka/clients/MockClient.java | 31 +- .../kafka/clients/NetworkClientTest.java | 35 +- .../clients/consumer/ConsumerExampleTest.java | 297 --- .../clients/consumer/MockConsumerTest.java | 32 + .../internals/SubscriptionStateTest.java | 61 + .../clients/producer/BufferPoolTest.java | 4 +- .../clients/producer/MockProducerTest.java | 2 +- .../clients/producer/PartitionerTest.java | 49 +- .../kafka/clients/producer/SenderTest.java | 6 +- .../kafka/common/config/ConfigDefTest.java | 6 +- .../kafka/common/network/SelectorTest.java | 53 +- .../apache/kafka/common/utils/UtilsTest.java | 10 + .../org/apache/kafka/test/MockSelector.java | 52 +- .../kafka/api/ConsumerMetadataRequest.scala | 2 +- .../kafka/api/ConsumerMetadataResponse.scala | 2 +- .../main/scala/kafka/cluster/Partition.scala | 2 +- .../controller/ControllerChannelManager.scala | 2 +- core/src/main/scala/kafka/log/LogConfig.scala | 4 +- .../main/scala/kafka/server/KafkaApis.scala | 29 + .../scala/kafka/server/ReplicaManager.scala | 2 +- .../scala/kafka/tools/ConsoleConsumer.scala | 2 - .../kafka/tools/ConsumerPerformance.scala | 169 +- .../tools/SimpleConsumerPerformance.scala | 2 +- .../scala/kafka/utils/KafkaScheduler.scala | 56 +- .../integration/kafka/api/ConsumerTest.scala | 286 +++ .../kafka/api/IntegrationTestHarness.scala | 73 + .../RequestResponseSerializationTest.scala | 4 +- .../integration/KafkaServerTestHarness.scala | 39 +- .../kafka/integration/PrimitiveApiTest.scala | 19 +- .../unit/kafka/utils/MockScheduler.scala | 2 + .../unit/kafka/utils/SchedulerTest.scala | 17 + .../scala/unit/kafka/utils/TestUtils.scala | 34 +- 89 files changed, 3984 insertions(+), 1947 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java rename clients/src/main/java/org/apache/kafka/clients/{NodeConnectionState.java => RequestCompletionHandler.java} (66%) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/OffsetMetadata.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java delete mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerExampleTest.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java create mode 100644 core/src/test/scala/integration/kafka/api/ConsumerTest.scala create mode 100644 core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala diff --git a/build.gradle b/build.gradle index a980f61def59a..68443725868c4 100644 --- a/build.gradle +++ b/build.gradle @@ -369,6 +369,7 @@ project(':clients') { } javadoc { + include "**/org/apache/kafka/clients/consumer/*" include "**/org/apache/kafka/clients/producer/*" include "**/org/apache/kafka/common/*" include "**/org/apache/kafka/common/errors/*" diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java index d32c319d8ee4c..ed4c0d98596cc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java @@ -22,24 +22,25 @@ public final class ClientRequest { private final long createdMs; private final boolean expectResponse; private final RequestSend request; - private final Object attachment; + private final RequestCompletionHandler callback; /** * @param createdMs The unix timestamp in milliseconds for the time at which this request was created. * @param expectResponse Should we expect a response message or is this request complete once it is sent? * @param request The request - * @param attachment Associated data with the request + * @param callback A callback to execute when the response has been received (or null if no callback is necessary) */ - public ClientRequest(long createdMs, boolean expectResponse, RequestSend request, Object attachment) { + public ClientRequest(long createdMs, boolean expectResponse, RequestSend request, RequestCompletionHandler callback) { this.createdMs = createdMs; - this.attachment = attachment; + this.callback = callback; this.request = request; this.expectResponse = expectResponse; } @Override public String toString() { - return "ClientRequest(expectResponse=" + expectResponse + ", payload=" + attachment + ", request=" + request + ")"; + return "ClientRequest(expectResponse=" + expectResponse + ", callback=" + callback + ", request=" + request + + ")"; } public boolean expectResponse() { @@ -50,8 +51,12 @@ public RequestSend request() { return request; } - public Object attachment() { - return attachment; + public boolean hasCallback() { + return callback != null; + } + + public RequestCompletionHandler callback() { + return callback; } public long createdTime() { diff --git a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java index 8aece7e81a804..574287d77f7d4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java @@ -119,16 +119,42 @@ public void connected(int node) { public void disconnected(int node) { nodeState(node).state = ConnectionState.DISCONNECTED; } - + /** - * Get the state of our connection to the given state + * Get the state of our connection to the given node * @param node The id of the node * @return The state of our connection */ + public ConnectionState connectionState(int node) { + return nodeState(node).state; + } + + /** + * Get the state of a given node + * @param node The node to fetch the state for + */ private NodeConnectionState nodeState(int node) { NodeConnectionState state = this.nodeState.get(node); if (state == null) throw new IllegalStateException("No entry found for node " + node); return state; } + + /** + * The state of our connection to a node + */ + private static class NodeConnectionState { + + ConnectionState state; + long lastConnectAttemptMs; + + public NodeConnectionState(ConnectionState state, long lastConnectAttempt) { + this.state = state; + this.lastConnectAttemptMs = lastConnectAttempt; + } + + public String toString() { + return "NodeState(" + state + ", " + lastConnectAttemptMs + ")"; + } + } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java new file mode 100644 index 0000000000000..06fcfe62cc1fe --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -0,0 +1,58 @@ +/** + * 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.clients; + +/** + * Some configurations shared by both producer and consumer + */ +public class CommonClientConfigs { + + /* + * NOTE: DO NOT CHANGE EITHER CONFIG NAMES AS THESE ARE PART OF THE PUBLIC API AND CHANGE WILL BREAK USER CODE. + */ + + public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers"; + public static final String BOOSTRAP_SERVERS_DOC = "A list of host/port pairs to use for establishing the initial connection to the Kafka cluster. The client will make use of all servers irrespective of which servers are specified here for bootstrapping—this list only impacts the initial hosts used to discover the full set of servers. This list should be in the form " + + "host1:port1,host2:port2,.... Since these servers are just used for the initial connection to " + + "discover the full cluster membership (which may change dynamically), this list need not contain the full set of " + + "servers (you may want more than one, though, in case a server is down)."; + + public static final String METADATA_MAX_AGE_CONFIG = "metadata.max.age.ms"; + public static final String METADATA_MAX_AGE_DOC = "The period of time in milliseconds after which we force a refresh of metadata even if we haven't seen any partition leadership changes to proactively discover any new brokers or partitions."; + + public static final String SEND_BUFFER_CONFIG = "send.buffer.bytes"; + public static final String SEND_BUFFER_DOC = "The size of the TCP send buffer (SO_SNDBUF) to use when sending data."; + + public static final String RECEIVE_BUFFER_CONFIG = "receive.buffer.bytes"; + public static final String RECEIVE_BUFFER_DOC = "The size of the TCP receive buffer (SO_RCVBUF) to use when reading data."; + + public static final String CLIENT_ID_CONFIG = "client.id"; + public static final String CLIENT_ID_DOC = "An id string to pass to the server when making requests. The purpose of this is to be able to track the source of requests beyond just ip/port by allowing a logical application name to be included in server-side request logging."; + + public static final String RECONNECT_BACKOFF_MS_CONFIG = "reconnect.backoff.ms"; + public static final String RECONNECT_BACKOFF_MS_DOC = "The amount of time to wait before attempting to reconnect to a given host. This avoids repeatedly connecting to a host in a tight loop. This backoff applies to all requests sent by the consumer to the broker."; + + public static final String RETRY_BACKOFF_MS_CONFIG = "retry.backoff.ms"; + public static final String RETRY_BACKOFF_MS_DOC = "The amount of time to wait before attempting to retry a failed fetch request to a given topic partition. This avoids repeated fetching-and-failing in a tight loop."; + + public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = "metrics.sample.window.ms"; + public static final String METRICS_SAMPLE_WINDOW_MS_DOC = "The number of samples maintained to compute metrics."; + + public static final String METRICS_NUM_SAMPLES_CONFIG = "metrics.num.samples"; + public static final String METRICS_NUM_SAMPLES_DOC = "The number of samples maintained to compute metrics."; + + public static final String METRIC_REPORTER_CLASSES_CONFIG = "metric.reporters"; + public static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters. Implementing the MetricReporter interface allows plugging in classes that will be notified of new metric creation. The JmxReporter is always included to register JMX statistics."; + +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java b/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java index ab7e3220f9b76..3867f8ecd5925 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java @@ -15,6 +15,6 @@ /** * The states of a node connection */ -enum ConnectionState { +public enum ConnectionState { DISCONNECTED, CONNECTING, CONNECTED } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java index 397695568d3fd..8a3e55aaff7d8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java @@ -26,6 +26,7 @@ public interface KafkaClient { /** * Check if we are currently ready to send another request to the given node but don't attempt to connect if we * aren't. + * * @param node The node to check * @param now The current timestamp */ @@ -34,6 +35,7 @@ public interface KafkaClient { /** * Initiate a connection to the given node (if necessary), and return true if already connected. The readiness of a * node will change only when poll is invoked. + * * @param node The node to connect to. * @param now The current time * @return true iff we are ready to immediately initiate the sending of another request to the given node. @@ -44,6 +46,7 @@ public interface KafkaClient { * Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When * disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled * connections. + * * @param node The node to check * @param now The current timestamp * @return The number of milliseconds to wait. @@ -51,19 +54,44 @@ public interface KafkaClient { public long connectionDelay(Node node, long now); /** - * Initiate the sending of the given requests and return any completed responses. Requests can only be sent on ready - * connections. - * @param requests The requests to send + * Queue up the given request for sending. Requests can only be sent on ready connections. + * + * @param request The request + * @param now The current time + */ + public void send(ClientRequest request); + + /** + * Do actual reads and writes from sockets. + * * @param timeout The maximum amount of time to wait for responses in ms * @param now The current time in ms * @throws IllegalStateException If a request is sent to an unready node */ - public List poll(List requests, long timeout, long now); + public List poll(long timeout, long now); + + /** + * Complete all in-flight requests for a given node + * + * @param node The node to complete requests for + * @param now The current time in ms + * @return All requests that complete during this time period. + */ + public List completeAll(int node, long now); + + /** + * Complete all in-flight requests + * + * @param now The current time in ms + * @return All requests that complete during this time period. + */ + public List completeAll(long now); /** * Choose the node with the fewest outstanding requests. This method will prefer a node with an existing connection, * but will potentially choose a node for which we don't yet have a connection if all existing connections are in * use. + * * @param now The current time in ms * @return The node with the fewest in-flight requests. */ @@ -74,8 +102,16 @@ public interface KafkaClient { */ public int inFlightRequestCount(); + /** + * Get the total in-flight requests for a particular node + * + * @param nodeId The id of the node + */ + public int inFlightRequestCount(int nodeId); + /** * Generate a request header for the next request + * * @param key The API key of the request */ public RequestHeader nextRequestHeader(ApiKeys key); diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 6746275d0b259..5950191b240f3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -102,6 +102,7 @@ public NetworkClient(Selectable selector, /** * Begin connecting to the given node, return true if we are already connected and ready to send to that node. + * * @param node The node to check * @param now The current timestamp * @return True if we are ready to send to the given node @@ -122,6 +123,7 @@ public boolean ready(Node node, long now) { * Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When * disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled * connections. + * * @param node The node to check * @param now The current timestamp * @return The number of milliseconds to wait. @@ -133,7 +135,8 @@ public long connectionDelay(Node node, long now) { /** * Check if the node with the given id is ready to send more requests. - * @param node The given node id + * + * @param node The node * @param now The current time in ms * @return true if the node is ready */ @@ -141,7 +144,8 @@ public long connectionDelay(Node node, long now) { public boolean isReady(Node node, long now) { int nodeId = node.id(); if (!this.metadataFetchInProgress && this.metadata.timeToNextUpdate(now) == 0) - // if we need to update our metadata now declare all requests unready to make metadata requests first priority + // if we need to update our metadata now declare all requests unready to make metadata requests first + // priority return false; else // otherwise we are ready if we are connected and can send more requests @@ -150,6 +154,7 @@ public boolean isReady(Node node, long now) { /** * Are we connected and ready and able to send more requests to the given node? + * * @param node The node */ private boolean isSendable(int node) { @@ -157,49 +162,106 @@ private boolean isSendable(int node) { } /** - * Initiate the given requests and check for any new responses, waiting up to the specified time. Requests can only - * be sent for ready nodes. - * @param requests The requests to initiate + * Return the state of the connection to the given node + * + * @param node The node to check + * @return The connection state + */ + public ConnectionState connectionState(int node) { + return connectionStates.connectionState(node); + } + + /** + * Queue up the given request for sending. Requests can only be sent out to ready nodes. + * + * @param request The request + * @param now The current time + */ + @Override + public void send(ClientRequest request) { + int nodeId = request.request().destination(); + if (!isSendable(nodeId)) + throw new IllegalStateException("Attempt to send a request to node " + nodeId + " which is not ready."); + + this.inFlightRequests.add(request); + selector.send(request.request()); + } + + /** + * Do actual reads and writes to sockets. + * * @param timeout The maximum amount of time to wait (in ms) for responses if there are none immediately * @param now The current time in milliseconds * @return The list of responses received */ @Override - public List poll(List requests, long timeout, long now) { - List sends = new ArrayList(); - - for (int i = 0; i < requests.size(); i++) { - ClientRequest request = requests.get(i); - int nodeId = request.request().destination(); - if (!isSendable(nodeId)) - throw new IllegalStateException("Attempt to send a request to node " + nodeId + " which is not ready."); - - this.inFlightRequests.add(request); - sends.add(request.request()); - } - + public List poll(long timeout, long now) { // should we update our metadata? long timeToNextMetadataUpdate = metadata.timeToNextUpdate(now); long timeToNextReconnectAttempt = Math.max(this.lastNoNodeAvailableMs + metadata.refreshBackoff() - now, 0); long waitForMetadataFetch = (this.metadataFetchInProgress ? Integer.MAX_VALUE : 0); // if there is no node available to connect, back off refreshing metadata - long metadataTimeout = Math.max(Math.max(timeToNextMetadataUpdate, timeToNextReconnectAttempt), waitForMetadataFetch); + long metadataTimeout = Math.max(Math.max(timeToNextMetadataUpdate, timeToNextReconnectAttempt), + waitForMetadataFetch); if (!this.metadataFetchInProgress && metadataTimeout == 0) - maybeUpdateMetadata(sends, now); - + maybeUpdateMetadata(now); // do the I/O try { - this.selector.poll(Math.min(timeout, metadataTimeout), sends); + this.selector.poll(Math.min(timeout, metadataTimeout)); } catch (IOException e) { log.error("Unexpected error during I/O in producer network thread", e); } + // process completed actions List responses = new ArrayList(); handleCompletedSends(responses, now); handleCompletedReceives(responses, now); handleDisconnections(responses, now); handleConnections(); + // invoke callbacks + for (ClientResponse response : responses) { + if (response.request().hasCallback()) { + try { + response.request().callback().onComplete(response); + } catch (Exception e) { + log.error("Uncaught error in request completion:", e); + } + } + } + + return responses; + } + + /** + * Await all the outstanding responses for requests on the given connection + * + * @param node The node to block on + * @param now The current time in ms + * @return All the collected responses + */ + @Override + public List completeAll(int node, long now) { + try { + this.selector.muteAll(); + this.selector.unmute(node); + List responses = new ArrayList(); + while (inFlightRequestCount(node) > 0) + responses.addAll(poll(Integer.MAX_VALUE, now)); + return responses; + } finally { + this.selector.unmuteAll(); + } + } + + /** + * Wait for all outstanding requests to complete. + */ + @Override + public List completeAll(long now) { + List responses = new ArrayList(); + while (inFlightRequestCount() > 0) + responses.addAll(poll(Integer.MAX_VALUE, now)); return responses; } @@ -211,8 +273,17 @@ public int inFlightRequestCount() { return this.inFlightRequests.inFlightRequestCount(); } + /** + * Get the number of in-flight requests for a given node + */ + @Override + public int inFlightRequestCount(int nodeId) { + return this.inFlightRequests.inFlightRequestCount(nodeId); + } + /** * Generate a request header for the given API key + * * @param key The api key * @return A request header with the appropriate client id and correlation id */ @@ -242,6 +313,7 @@ public void close() { * prefer a node with an existing connection, but will potentially choose a node for which we don't yet have a * connection if all existing connections are in use. This method will never choose a node for which there is no * existing connection and from which we have disconnected within the reconnect backoff period. + * * @return The node with the fewest in-flight requests. */ public Node leastLoadedNode(long now) { @@ -261,12 +333,12 @@ public Node leastLoadedNode(long now) { found = node; } } - return found; } /** * Handle any completed request send. In particular if no response is expected consider the request complete. + * * @param responses The list of responses to update * @param now The current time */ @@ -283,6 +355,7 @@ private void handleCompletedSends(List responses, long now) { /** * Handle any completed receives and update the response list with the responses received. + * * @param responses The list of responses to update * @param now The current time */ @@ -317,6 +390,7 @@ private void handleMetadataResponse(RequestHeader header, Struct body, long now) /** * Handle any disconnected connections + * * @param responses The list of responses that completed with the disconnection * @param now The current time */ @@ -353,10 +427,8 @@ private void handleConnections() { */ private void correlate(RequestHeader requestHeader, ResponseHeader responseHeader) { if (requestHeader.correlationId() != responseHeader.correlationId()) - throw new IllegalStateException("Correlation id for response (" + responseHeader.correlationId() + - ") does not match request (" + - requestHeader.correlationId() + - ")"); + throw new IllegalStateException("Correlation id for response (" + responseHeader.correlationId() + + ") does not match request (" + requestHeader.correlationId() + ")"); } /** @@ -371,7 +443,7 @@ private ClientRequest metadataRequest(long now, int node, Set topics) { /** * Add a metadata request to the list of sends if we can make one */ - private void maybeUpdateMetadata(List sends, long now) { + private void maybeUpdateMetadata(long now) { // Beware that the behavior of this method and the computation of timeouts for poll() are // highly dependent on the behavior of leastLoadedNode. Node node = this.leastLoadedNode(now); @@ -382,17 +454,16 @@ private void maybeUpdateMetadata(List sends, long now) { return; } - log.debug("Trying to send metadata request to node {}", node.id()); if (connectionStates.isConnected(node.id()) && inFlightRequests.canSendMore(node.id())) { Set topics = metadata.topics(); this.metadataFetchInProgress = true; ClientRequest metadataRequest = metadataRequest(now, node.id(), topics); log.debug("Sending metadata request {} to node {}", metadataRequest, node.id()); - sends.add(metadataRequest.request()); + this.selector.send(metadataRequest.request()); this.inFlightRequests.add(metadataRequest); } else if (connectionStates.canConnect(node.id(), now)) { // we don't have a connection to this node right now, make one - log.debug("Init connection to node {} for sending metadata request in the next iteration", node.id()); + log.debug("Initialize connection to node {} for sending metadata request", node.id()); initiateConnect(node, now); // If initiateConnect failed immediately, this node will be put into blackout and we // should allow immediately retrying in case there is another candidate node. If it @@ -412,7 +483,10 @@ private void initiateConnect(Node node, long now) { try { log.debug("Initiating connection to node {} at {}:{}.", node.id(), node.host(), node.port()); this.connectionStates.connecting(node.id(), now); - selector.connect(node.id(), new InetSocketAddress(node.host(), node.port()), this.socketSendBuffer, this.socketReceiveBuffer); + selector.connect(node.id(), + new InetSocketAddress(node.host(), node.port()), + this.socketSendBuffer, + this.socketReceiveBuffer); } catch (IOException e) { /* attempt failed, we'll try again after the backoff */ connectionStates.disconnected(node.id()); diff --git a/clients/src/main/java/org/apache/kafka/clients/NodeConnectionState.java b/clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java similarity index 66% rename from clients/src/main/java/org/apache/kafka/clients/NodeConnectionState.java rename to clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java index 752a979ea0b8b..6fee4e45986d2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NodeConnectionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java @@ -13,19 +13,11 @@ package org.apache.kafka.clients; /** - * The state of our connection to a node + * A callback interface for attaching an action to be executed when a request is complete and the corresponding response + * has been received. This handler will also be invoked if there is a disconnection while handling the request. */ -final class NodeConnectionState { +public interface RequestCompletionHandler { - ConnectionState state; - long lastConnectAttemptMs; + public void onComplete(ClientResponse response); - public NodeConnectionState(ConnectionState state, long lastConnectAttempt) { - this.state = state; - this.lastConnectAttemptMs = lastConnectAttempt; - } - - public String toString() { - return "NodeState(" + state + ", " + lastConnectAttemptMs + ")"; - } -} \ No newline at end of file +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java b/clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java new file mode 100644 index 0000000000000..072cc2e6f92db --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java @@ -0,0 +1,5 @@ +package org.apache.kafka.clients.consumer; + +public enum CommitType { + SYNC, ASYNC +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java index c0c636b3e1ba2..8f587bc0705b6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java @@ -9,14 +9,16 @@ * 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.clients.consumer; import java.io.Closeable; -import java.util.Collection; +import java.util.List; import java.util.Map; +import java.util.Set; import org.apache.kafka.common.Metric; +import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.MetricName; @@ -24,102 +26,85 @@ * @see KafkaConsumer * @see MockConsumer */ -public interface Consumer extends Closeable { +public interface Consumer extends Closeable { + + /** + * @see KafkaConsumer#subscriptions() + */ + public Set subscriptions(); /** - * Incrementally subscribe to the given list of topics. This API is mutually exclusive to - * {@link #subscribe(TopicPartition...) subscribe(partitions)} - * @param topics A variable list of topics that the consumer subscribes to - */ - public void subscribe(String...topics); + * @see KafkaConsumer#subscribe(String...) + */ + public void subscribe(String... topics); /** - * Incrementally subscribes to a specific topic and partition. This API is mutually exclusive to - * {@link #subscribe(String...) subscribe(topics)} - * @param partitions Partitions to subscribe to - */ + * @see KafkaConsumer#subscribe(TopicPartition...) + */ public void subscribe(TopicPartition... partitions); /** - * Unsubscribe from the specific topics. Messages for this topic will not be returned from the next {@link #poll(long) poll()} - * onwards. This should be used in conjunction with {@link #subscribe(String...) subscribe(topics)}. It is an error to - * unsubscribe from a topic that was never subscribed to using {@link #subscribe(String...) subscribe(topics)} - * @param topics Topics to unsubscribe from + * @see KafkaConsumer#unsubscribe(String...) */ public void unsubscribe(String... topics); /** - * Unsubscribe from the specific topic partitions. Messages for these partitions will not be returned from the next - * {@link #poll(long) poll()} onwards. This should be used in conjunction with - * {@link #subscribe(TopicPartition...) subscribe(topic, partitions)}. It is an error to - * unsubscribe from a partition that was never subscribed to using {@link #subscribe(TopicPartition...) subscribe(partitions)} - * @param partitions Partitions to unsubscribe from + * @see KafkaConsumer#unsubscribe(TopicPartition...) */ public void unsubscribe(TopicPartition... partitions); - + /** - * Fetches data for the subscribed list of topics and partitions - * @param timeout The time, in milliseconds, spent waiting in poll if data is not available. If 0, waits indefinitely. Must not be negative - * @return Map of topic to records for the subscribed topics and partitions as soon as data is available for a topic partition. Availability - * of data is controlled by {@link ConsumerConfig#FETCH_MIN_BYTES_CONFIG} and {@link ConsumerConfig#FETCH_MAX_WAIT_MS_CONFIG}. - * If no data is available for timeout ms, returns an empty list + * @see KafkaConsumer#poll(long) */ - public Map> poll(long timeout); + public ConsumerRecords poll(long timeout); /** - * Commits offsets returned on the last {@link #poll(long) poll()} for the subscribed list of topics and partitions. - * @param sync If true, the commit should block until the consumer receives an acknowledgment - * @return An {@link OffsetMetadata} object that contains the partition, offset and a corresponding error code. Returns null - * if the sync flag is set to false + * @see KafkaConsumer#commit(CommitType) */ - public OffsetMetadata commit(boolean sync); + public void commit(CommitType commitType); /** - * Commits the specified offsets for the specified list of topics and partitions to Kafka. - * @param offsets The map of offsets to commit for the given topic partitions - * @param sync If true, commit will block until the consumer receives an acknowledgment - * @return An {@link OffsetMetadata} object that contains the partition, offset and a corresponding error code. Returns null - * if the sync flag is set to false. + * @see KafkaConsumer#commit(Map, CommitType) */ - public OffsetMetadata commit(Map offsets, boolean sync); - + public void commit(Map offsets, CommitType commitType); + /** - * Overrides the fetch positions that the consumer will use on the next fetch request. If the consumer subscribes to a list of topics - * using {@link #subscribe(String...) subscribe(topics)}, an exception will be thrown if the specified topic partition is not owned by - * the consumer. - * @param offsets The map of fetch positions per topic and partition + * @see KafkaConsumer#seek(TopicPartition, long) */ - public void seek(Map offsets); + public void seek(TopicPartition partition, long offset); /** - * Returns the fetch position of the next message for the specified topic partition to be used on the next {@link #poll(long) poll()} - * @param partitions Partitions for which the fetch position will be returned - * @return The position from which data will be fetched for the specified partition on the next {@link #poll(long) poll()} + * @see KafkaConsumer#seekToBeginning(TopicPartition...) */ - public Map position(Collection partitions); - + public void seekToBeginning(TopicPartition... partitions); + /** - * Fetches the last committed offsets for the input list of partitions - * @param partitions The list of partitions to return the last committed offset for - * @return The list of offsets for the specified list of partitions + * @see KafkaConsumer#seekToEnd(TopicPartition...) */ - public Map committed(Collection partitions); - + public void seekToEnd(TopicPartition... partitions); + + /** + * @see KafkaConsumer#position(TopicPartition) + */ + public long position(TopicPartition partition); + /** - * Fetches offsets before a certain timestamp - * @param timestamp The unix timestamp. Value -1 indicates earliest available timestamp. Value -2 indicates latest available timestamp. - * @param partitions The list of partitions for which the offsets are returned - * @return The offsets for messages that were written to the server before the specified timestamp. + * @see KafkaConsumer#committed(TopicPartition) */ - public Map offsetsBeforeTime(long timestamp, Collection partitions); + public long committed(TopicPartition partition); /** - * Return a map of metrics maintained by the consumer + * @see KafkaConsumer#metrics() */ public Map metrics(); /** - * Close this consumer + * @see KafkaConsumer#partitionsFor(String) + */ + public List partitionsFor(String topic); + + /** + * @see KafkaConsumer#close() */ public void close(); 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 57c1807ccba9f..6d4ff7cd2a283 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 @@ -9,13 +9,16 @@ * 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.clients.consumer; import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; +import static org.apache.kafka.common.config.ConfigDef.ValidString.in; import java.util.Map; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceCallback; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; @@ -27,130 +30,121 @@ public class ConsumerConfig extends AbstractConfig { private static final ConfigDef config; - /** - * The identifier of the group this consumer belongs to. This is required if the consumer uses either the - * group management functionality by using {@link Consumer#subscribe(String...) subscribe(topics)}. This is also required - * if the consumer uses the default Kafka based offset management strategy. + /* + * NOTE: DO NOT CHANGE EITHER CONFIG STRINGS OR THEIR JAVA VARIABLE NAMES AS + * THESE ARE PART OF THE PUBLIC API AND CHANGE WILL BREAK USER CODE. */ - public static final String GROUP_ID_CONFIG = "group.id"; - + /** - * The timeout after which, if the {@link Consumer#poll(long) poll(timeout)} is not invoked, the consumer is - * marked dead and a rebalance operation is triggered for the group identified by {@link #GROUP_ID_CONFIG}. Relevant - * if the consumer uses the group management functionality by invoking {@link Consumer#subscribe(String...) subscribe(topics)} + * group.id */ - public static final String SESSION_TIMEOUT_MS = "session.timeout.ms"; + public static final String GROUP_ID_CONFIG = "group.id"; + private static final String GROUP_ID_DOC = "A unique string that identifies the consumer group this consumer belongs to. This property is required if the consumer uses either the group management functionality by using subscribe(topic) or the Kafka-based offset management strategy."; /** - * The number of times a consumer sends a heartbeat to the co-ordinator broker within a {@link #SESSION_TIMEOUT_MS} time window. - * This frequency affects the latency of a rebalance operation since the co-ordinator broker notifies a consumer of a rebalance - * in the heartbeat response. Relevant if the consumer uses the group management functionality by invoking - * {@link Consumer#subscribe(String...) subscribe(topics)} + * session.timeout.ms */ - public static final String HEARTBEAT_FREQUENCY = "heartbeat.frequency"; + public static final String SESSION_TIMEOUT_MS_CONFIG = "session.timeout.ms"; + private static final String SESSION_TIMEOUT_MS_DOC = "The timeout used to detect failures when using Kafka's group management facilities."; /** - * A list of URLs to use for establishing the initial connection to the cluster. This list should be in the form - * host1:port1,host2:port2,.... These urls are just used for the initial connection to discover the - * full cluster membership (which may change dynamically) so this list need not contain the full set of servers (you - * may want more than one, though, in case a server is down). + * bootstrap.servers */ - public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers"; + public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; /** - * If true, periodically commit to Kafka the offsets of messages already returned by the consumer. This committed - * offset will be used when the process fails as the position from which the consumption will begin. + * enable.auto.commit */ public static final String ENABLE_AUTO_COMMIT_CONFIG = "enable.auto.commit"; - + private static final String ENABLE_AUTO_COMMIT_DOC = "If true the consumer's offset will be periodically committed in the background."; + /** - * The friendly name of the partition assignment strategy that the server will use to distribute partition ownership - * amongst consumer instances when group management is used + * auto.commit.interval.ms */ - public static final String PARTITION_ASSIGNMENT_STRATEGY = "partition.assignment.strategy"; - + public static final String AUTO_COMMIT_INTERVAL_MS_CONFIG = "auto.commit.interval.ms"; + private static final String AUTO_COMMIT_INTERVAL_MS_DOC = "The frequency in milliseconds that the consumer offsets are auto-committed to Kafka if enable.auto.commit is set to true."; + /** - * The frequency in milliseconds that the consumer offsets are committed to Kafka. Relevant if {@link #ENABLE_AUTO_COMMIT_CONFIG} - * is turned on. + * partition.assignment.strategy */ - public static final String AUTO_COMMIT_INTERVAL_MS_CONFIG = "auto.commit.interval.ms"; - + public static final String PARTITION_ASSIGNMENT_STRATEGY_CONFIG = "partition.assignment.strategy"; + private static final String PARTITION_ASSIGNMENT_STRATEGY_DOC = "The friendly name of the partition assignment strategy that the server will use to distribute partition ownership amongst consumer instances when group management is used"; + /** - * What to do when there is no initial offset in Kafka or if an offset is out of range: - *
        - *
      • smallest: automatically reset the offset to the smallest offset - *
      • largest: automatically reset the offset to the largest offset - *
      • disable: throw exception to the consumer if no previous offset is found for the consumer's group - *
      • anything else: throw exception to the consumer. - *
      + * auto.offset.reset */ public static final String AUTO_OFFSET_RESET_CONFIG = "auto.offset.reset"; - + private static final String AUTO_OFFSET_RESET_DOC = "What to do when there is no initial offset in Kafka or if the current offset does not exist any more on the server (e.g. because that data has been deleted):
      • smallest: automatically reset the offset to the smallest offset
      • largest: automatically reset the offset to the largest offset
      • none: throw exception to the consumer if no previous offset is found for the consumer's group
      • anything else: throw exception to the consumer.
      "; + /** - * 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. + * fetch.min.bytes */ 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."; + /** - * The maximum amount of time the server will block before answering the fetch request if there isn't sufficient - * data to immediately satisfy {@link #FETCH_MIN_BYTES_CONFIG}. This should be less than or equal to the timeout used in - * {@link KafkaConsumer#poll(long) poll(timeout)} + * fetch.max.wait.ms */ public static final String FETCH_MAX_WAIT_MS_CONFIG = "fetch.max.wait.ms"; - + private static final String FETCH_MAX_WAIT_MS_DOC = "The maximum amount of time the server will block before answering the fetch request if there isn't sufficient data to immediately satisfy the requirement given by fetch.min.bytes."; + + /** metadata.max.age.ms */ + public static final String METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG; + /** - * The maximum amount of time to block waiting to fetch metadata about a topic the first time a record is received - * from that topic. The consumer will throw a TimeoutException if it could not successfully fetch metadata within - * this timeout. + * max.partition.fetch.bytes */ - public static final String METADATA_FETCH_TIMEOUT_CONFIG = "metadata.fetch.timeout.ms"; + 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."; + + /** send.buffer.bytes */ + public static final String SEND_BUFFER_CONFIG = CommonClientConfigs.SEND_BUFFER_CONFIG; + + /** receive.buffer.bytes */ + public static final String RECEIVE_BUFFER_CONFIG = CommonClientConfigs.RECEIVE_BUFFER_CONFIG; /** - * The total memory used by the consumer to buffer records received from the server. This config is meant to control - * the consumer's memory usage, so it is the size of the global fetch buffer that will be shared across all partitions. + * client.id */ - public static final String TOTAL_BUFFER_MEMORY_CONFIG = "total.memory.bytes"; + public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG; /** - * The minimum amount of memory that should be used to fetch at least one message for a partition. This puts a lower - * bound on the consumer's memory utilization when there is at least one message for a partition available on the server. - * 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. + * reconnect.backoff.ms */ - public static final String FETCH_BUFFER_CONFIG = "fetch.buffer.bytes"; - + public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG; + /** - * The id string to pass to the server when making requests. The purpose of this is to be able to track the source - * of requests beyond just ip/port by allowing a logical application name to be included. + * retry.backoff.ms */ - public static final String CLIENT_ID_CONFIG = "client.id"; + public static final String RETRY_BACKOFF_MS_CONFIG = CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG; /** - * The size of the TCP send buffer to use when fetching data + * metrics.sample.window.ms */ - public static final String SOCKET_RECEIVE_BUFFER_CONFIG = "socket.receive.buffer.bytes"; + public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG; /** - * The amount of time to wait before attempting to reconnect to a given host. This avoids repeatedly connecting to a - * host in a tight loop. This backoff applies to all requests sent by the consumer to the broker. + * metrics.num.samples */ - public static final String RECONNECT_BACKOFF_MS_CONFIG = "reconnect.backoff.ms"; + public static final String METRICS_NUM_SAMPLES_CONFIG = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG; - /** metrics.sample.window.ms */ - public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = "metrics.sample.window.ms"; - private static final String METRICS_SAMPLE_WINDOW_MS_DOC = "The metrics system maintains a configurable number of samples over a fixed window size. This configuration " + "controls the size of the window. For example we might maintain two samples each measured over a 30 second period. " - + "When a window expires we erase and overwrite the oldest window."; - - /** metrics.num.samples */ - public static final String METRICS_NUM_SAMPLES_CONFIG = "metrics.num.samples"; - private static final String METRICS_NUM_SAMPLES_DOC = "The number of samples maintained to compute metrics."; + /** + * metric.reporters + */ + public static final String METRIC_REPORTER_CLASSES_CONFIG = CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG; - /** metric.reporters */ - public static final String METRIC_REPORTER_CLASSES_CONFIG = "metric.reporters"; - private static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters. Implementing the MetricReporter interface allows " + "plugging in classes that will be notified of new metric creation. The JmxReporter is always included to register JMX statistics."; + /** + * rebalance.callback.class + */ + public static final String CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG = "rebalance.callback.class"; + private static final String CONSUMER_REBALANCE_CALLBACK_CLASS_DOC = "A user-provided callback to execute when partition assignments change."; + /** + * check.crcs + */ + public static final String CHECK_CRCS_CONFIG = "check.crcs"; + private static final String CHECK_CRCS_DOC = "Automatically check the CRC32 of the records consumed. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may be disabled in cases seeking extreme performance."; + /** key.deserializer */ public static final String KEY_DESERIALIZER_CLASS_CONFIG = "key.deserializer"; private static final String KEY_DESERIALIZER_CLASS_DOC = "Deserializer class for key that implements the Deserializer interface."; @@ -160,38 +154,134 @@ public class ConsumerConfig extends AbstractConfig { private static final String VALUE_DESERIALIZER_CLASS_DOC = "Deserializer class for value that implements the Deserializer interface."; static { - /* TODO: add config docs */ - config = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, "blah blah") - .define(GROUP_ID_CONFIG, Type.STRING, Importance.HIGH, "blah blah") - .define(SESSION_TIMEOUT_MS, Type.LONG, 1000, Importance.HIGH, "blah blah") - .define(HEARTBEAT_FREQUENCY, Type.INT, 3, Importance.MEDIUM, "blah blah") - .define(PARTITION_ASSIGNMENT_STRATEGY, Type.STRING, Importance.MEDIUM, "blah blah") - .define(METADATA_FETCH_TIMEOUT_CONFIG, Type.LONG, 60 * 1000, atLeast(0), Importance.MEDIUM, "blah blah") - .define(ENABLE_AUTO_COMMIT_CONFIG, Type.BOOLEAN, true, Importance.MEDIUM, "blah blah") - .define(AUTO_COMMIT_INTERVAL_MS_CONFIG, Type.LONG, 5000, atLeast(0), Importance.LOW, "blah blah") - .define(CLIENT_ID_CONFIG, Type.STRING, "", Importance.LOW, "blah blah") - .define(TOTAL_BUFFER_MEMORY_CONFIG, Type.LONG, 32 * 1024 * 1024L, atLeast(0L), Importance.LOW, "blah blah") - .define(FETCH_BUFFER_CONFIG, Type.INT, 1 * 1024 * 1024, atLeast(0), Importance.HIGH, "blah blah") - .define(SOCKET_RECEIVE_BUFFER_CONFIG, Type.INT, 128 * 1024, atLeast(0), Importance.LOW, "blah blah") - .define(FETCH_MIN_BYTES_CONFIG, Type.LONG, 1024, atLeast(0), Importance.HIGH, "blah blah") - .define(FETCH_MAX_WAIT_MS_CONFIG, Type.LONG, 500, atLeast(0), Importance.LOW, "blah blah") - .define(RECONNECT_BACKOFF_MS_CONFIG, Type.LONG, 10L, atLeast(0L), Importance.LOW, "blah blah") - .define(AUTO_OFFSET_RESET_CONFIG, Type.STRING, "largest", Importance.MEDIUM, "blah blah") + config = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, + Type.LIST, + Importance.HIGH, + CommonClientConfigs.BOOSTRAP_SERVERS_DOC) + .define(GROUP_ID_CONFIG, Type.STRING, "", Importance.HIGH, GROUP_ID_DOC) + .define(SESSION_TIMEOUT_MS_CONFIG, + Type.LONG, + 30000, + Importance.HIGH, + SESSION_TIMEOUT_MS_DOC) + .define(PARTITION_ASSIGNMENT_STRATEGY_CONFIG, + Type.STRING, + "blah", + Importance.MEDIUM, + PARTITION_ASSIGNMENT_STRATEGY_DOC) + .define(METADATA_MAX_AGE_CONFIG, + Type.LONG, + 5 * 60 * 1000, + atLeast(0), + Importance.LOW, + CommonClientConfigs.METADATA_MAX_AGE_DOC) + .define(ENABLE_AUTO_COMMIT_CONFIG, + Type.BOOLEAN, + true, + Importance.MEDIUM, + ENABLE_AUTO_COMMIT_DOC) + .define(AUTO_COMMIT_INTERVAL_MS_CONFIG, + Type.LONG, + 5000, + atLeast(0), + Importance.LOW, + AUTO_COMMIT_INTERVAL_MS_DOC) + .define(CLIENT_ID_CONFIG, + Type.STRING, + "", + Importance.LOW, + CommonClientConfigs.CLIENT_ID_DOC) + .define(MAX_PARTITION_FETCH_BYTES_CONFIG, + Type.INT, + 1 * 1024 * 1024, + atLeast(0), + Importance.HIGH, + MAX_PARTITION_FETCH_BYTES_DOC) + .define(SEND_BUFFER_CONFIG, + Type.INT, + 128 * 1024, + atLeast(0), + Importance.MEDIUM, + CommonClientConfigs.SEND_BUFFER_DOC) + .define(RECEIVE_BUFFER_CONFIG, + Type.INT, + 32 * 1024, + atLeast(0), + Importance.MEDIUM, + CommonClientConfigs.RECEIVE_BUFFER_DOC) + .define(FETCH_MIN_BYTES_CONFIG, + Type.INT, + 1024, + atLeast(0), + Importance.HIGH, + FETCH_MIN_BYTES_DOC) + .define(FETCH_MAX_WAIT_MS_CONFIG, + Type.INT, + 500, + atLeast(0), + Importance.LOW, + FETCH_MAX_WAIT_MS_DOC) + .define(RECONNECT_BACKOFF_MS_CONFIG, + Type.LONG, + 50L, + atLeast(0L), + Importance.LOW, + CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC) + .define(RETRY_BACKOFF_MS_CONFIG, + Type.LONG, + 100L, + atLeast(0L), + Importance.LOW, + CommonClientConfigs.RETRY_BACKOFF_MS_DOC) + .define(AUTO_OFFSET_RESET_CONFIG, + Type.STRING, + "latest", + in("latest", "earliest", "none"), + Importance.MEDIUM, + AUTO_OFFSET_RESET_DOC) + .define(CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG, + Type.CLASS, + NoOpConsumerRebalanceCallback.class, + Importance.LOW, + CONSUMER_REBALANCE_CALLBACK_CLASS_DOC) + .define(CHECK_CRCS_CONFIG, + Type.BOOLEAN, + true, + Importance.LOW, + CHECK_CRCS_DOC) .define(METRICS_SAMPLE_WINDOW_MS_CONFIG, Type.LONG, 30000, atLeast(0), Importance.LOW, - METRICS_SAMPLE_WINDOW_MS_DOC) - .define(METRICS_NUM_SAMPLES_CONFIG, Type.INT, 2, atLeast(1), Importance.LOW, METRICS_NUM_SAMPLES_DOC) - .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, "", Importance.LOW, METRIC_REPORTER_CLASSES_DOC) - .define(KEY_DESERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, KEY_DESERIALIZER_CLASS_DOC) - .define(VALUE_DESERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, VALUE_DESERIALIZER_CLASS_DOC); - + CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC) + .define(METRICS_NUM_SAMPLES_CONFIG, + Type.INT, + 2, + atLeast(1), + Importance.LOW, + CommonClientConfigs.METRICS_NUM_SAMPLES_DOC) + .define(METRIC_REPORTER_CLASSES_CONFIG, + Type.LIST, + "", + Importance.LOW, + CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) + .define(KEY_DESERIALIZER_CLASS_CONFIG, + Type.CLASS, + Importance.HIGH, + KEY_DESERIALIZER_CLASS_DOC) + .define(VALUE_DESERIALIZER_CLASS_CONFIG, + Type.CLASS, + Importance.HIGH, + VALUE_DESERIALIZER_CLASS_DOC); } ConsumerConfig(Map props) { super(config, props); } + public static void main(String[] args) { + System.out.println(config.toHtmlTable()); + } + } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java index e4cf7d1cfa01c..74dfdba0ecbca 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java @@ -9,7 +9,7 @@ * 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.clients.consumer; import java.util.Collection; @@ -17,34 +17,77 @@ import org.apache.kafka.common.TopicPartition; /** - * A callback interface that the user can implement to manage customized offsets on the start and end of - * every rebalance operation. This callback will execute in the user thread as part of the - * {@link Consumer#poll(long) poll(long)} API on every rebalance attempt. - * Default implementation of the callback will {@link Consumer#seek(java.util.Map) seek(offsets)} to the last committed offsets in the - * {@link #onPartitionsAssigned(Consumer, Collection) onPartitionsAssigned()} callback. And will commit offsets synchronously - * for the specified list of partitions to Kafka in the {@link #onPartitionsRevoked(Consumer, Collection) onPartitionsRevoked()} - * callback. + * A callback interface that the user can implement to trigger custom actions when the set of partitions assigned to the + * consumer changes. + *

      + * This is applicable when the consumer is having Kafka auto-manage group membership, if the consumer's directly subscribe to partitions + * those partitions will never be reassigned and this callback is not applicable. + *

      + * When Kafka is managing the group membership, a partition re-assignment will be triggered any time the members of the group changes or the subscription + * of the members changes. This can occur when processes die, new process instances are added or old instances come back to life after failure. + *

      + * There are many uses for this functionality. One common use is saving offsets in a custom store. By saving offsets in + * the {@link #onPartitionsRevoked(Consumer, Collection)} call we can ensure that any time partition assignment changes + * the offset gets saved. + *

      + * Another use is flushing out any kind of cache of intermediate results the consumer may be keeping. For example, + * consider a case where the consumer is subscribed to a topic containing user page views, and the goal is to count the + * number of page views per users for each five minute window. Let's say the topic is partitioned by the user id so that + * all events for a particular user will go to a single consumer instance. The consumer can keep in memory a running + * tally of actions per user and only flush these out to a remote data store when it's cache gets to big. However if a + * partition is reassigned it may want to automatically trigger a flush of this cache, before the new owner takes over + * consumption. + *

      + * This callback will execute in the user thread as part of the {@link Consumer#poll(long) poll(long)} call whenever partition assignment changes. + *

      + * It is guaranteed that all consumer processes will invoke {@link #onPartitionsRevoked(Consumer, Collection) onPartitionsRevoked} prior to + * any process invoking {@link #onPartitionsAssigned(Consumer, Collection) onPartitionsAssigned}. So if offsets or other state is saved in the + * {@link #onPartitionsRevoked(Consumer, Collection) onPartitionsRevoked} call it is guaranteed to be saved by the time the process taking over that + * partition has their {@link #onPartitionsAssigned(Consumer, Collection) onPartitionsAssigned} callback called to load the state. + *

      + * Here is pseudo-code for a callback implementation for saving offsets: + *

      + * {@code
      + *   public class SaveOffsetsOnRebalance implements ConsumerRebalanceCallback {
      + *       public void onPartitionsAssigned(Consumer consumer, Collection partitions) {
      + *           // read the offsets from an external store using some custom code not described here
      + *           for(TopicPartition partition: partitions)
      + *              consumer.position(partition, readOffsetFromExternalStore(partition));
      + *       }      
      + *       public void onPartitionsRevoked(Consumer consumer, Collection partitions) {
      + *           // save the offsets in an external store using some custom code not described here
      + *           for(TopicPartition partition: partitions)
      + *              saveOffsetInExternalStore(consumer.position(partition));
      + *       }
      + *   }
      + * }
      + * 
      */ public interface ConsumerRebalanceCallback { /** - * A callback method the user can implement to provide handling of customized offsets on completion of a successful - * rebalance operation. This method will be called after a rebalance operation completes and before the consumer - * starts fetching data. - *

      - * For examples on usage of this API, see Usage Examples section of {@link KafkaConsumer KafkaConsumer} - * @param partitions The list of partitions that are assigned to the consumer after rebalance + * A callback method the user can implement to provide handling of customized offsets on completion of a successful + * partition re-assignement. This method will be called after an offset re-assignement completes and before the + * consumer starts fetching data. + *

      + * It is guaranteed that all the processes in a consumer group will execute their + * {@link #onPartitionsRevoked(Consumer, Collection)} callback before any instance executes its + * {@link #onPartitionsAssigned(Consumer, Collection)} callback. + * + * @param partitions The list of partitions that are now assigned to the consumer (may include partitions previously + * assigned to the consumer) */ - public void onPartitionsAssigned(Consumer consumer, Collection partitions); - + public void onPartitionsAssigned(Consumer consumer, Collection partitions); + /** - * A callback method the user can implement to provide handling of offset commits to a customized store on the - * start of a rebalance operation. This method will be called before a rebalance operation starts and after the - * consumer stops fetching data. It is recommended that offsets should be committed in this callback to - * either Kafka or a custom offset store to prevent duplicate data - *

      - * For examples on usage of this API, see Usage Examples section of {@link KafkaConsumer KafkaConsumer} + * A callback method the user can implement to provide handling of offset commits to a customized store on the start + * of a rebalance operation. This method will be called before a rebalance operation starts and after the consumer + * stops fetching data. It is recommended that offsets should be committed in this callback to either Kafka or a + * custom offset store to prevent duplicate data + *

      + * For examples on usage of this API, see Usage Examples section of {@link KafkaConsumer KafkaConsumer} + * * @param partitions The list of partitions that were assigned to the consumer on the last rebalance */ - public void onPartitionsRevoked(Consumer consumer, Collection partitions); + public void onPartitionsRevoked(Consumer consumer, Collection partitions); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java index 16af70a5de52c..466254e81c323 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java @@ -9,119 +9,76 @@ * 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.clients.consumer; -import org.apache.kafka.common.TopicPartition; - /** - * A key/value pair to be received from Kafka. This consists of a topic name and a partition number, from which the - * record is being received and an offset that points to the record in a Kafka partition. + * A key/value pair to be received from Kafka. This consists of a topic name and a partition number, from which the + * record is being received and an offset that points to the record in a Kafka partition. */ -public final class ConsumerRecord { - private final TopicPartition partition; +public final class ConsumerRecord { + private final String topic; + private final int partition; + private final long offset; private final K key; private final V value; - private final long offset; - private volatile Exception error; - - /** - * Creates a record to be received from a specified topic and partition - * - * @param topic The topic this record is received from - * @param partitionId The partition of the topic this record is received from - * @param key The key of the record, if one exists - * @param value The record contents - * @param offset The offset of this record in the corresponding Kafka partition - */ - public ConsumerRecord(String topic, int partitionId, K key, V value, long offset) { - this(topic, partitionId, key, value, offset, null); - } /** * Create a record with no key * * @param topic The topic this record is received from - * @param partitionId The partition of the topic this record is received from - * @param value The record contents + * @param partition The partition of the topic this record is received from * @param offset The offset of this record in the corresponding Kafka partition + * @param value The record contents */ - public ConsumerRecord(String topic, int partitionId, V value, long offset) { - this(topic, partitionId, null, value, offset); - } - - /** - * Creates a record with an error code - * @param topic The topic this record is received from - * @param partitionId The partition of the topic this record is received from - * @param error The exception corresponding to the error code returned by the server for this topic partition - */ - public ConsumerRecord(String topic, int partitionId, Exception error) { - this(topic, partitionId, null, null, -1L, error); - } - - private ConsumerRecord(String topic, int partitionId, K key, V value, long offset, Exception error) { + public ConsumerRecord(String topic, int partition, long offset, K key, V value) { if (topic == null) throw new IllegalArgumentException("Topic cannot be null"); - this.partition = new TopicPartition(topic, partitionId); + this.topic = topic; + this.partition = partition; + this.offset = offset; this.key = key; this.value = value; - this.offset = offset; - this.error = error; } - + /** * The topic this record is received from */ public String topic() { - return partition.topic(); + return this.topic; } /** - * The partition from which this record is received + * The partition from which this record is received */ public int partition() { - return partition.partition(); + return this.partition; } - - /** - * The TopicPartition object containing the topic and partition - */ - public TopicPartition topicAndPartition() { - return partition; - } - + /** * The key (or null if no key is specified) - * @throws Exception The exception thrown while fetching this record. */ public K key() throws Exception { - if (this.error != null) - throw this.error; return key; } /** * The value - * @throws Exception The exception thrown while fetching this record. */ public V value() throws Exception { - if (this.error != null) - throw this.error; return value; } /** * The position of this record in the corresponding Kafka partition. - * @throws Exception The exception thrown while fetching this record. */ - public long offset() throws Exception { - if (this.error != null) - throw this.error; + public long offset() { return offset; } - public Exception error() { - return this.error; + @Override + public String toString() { + return "ConsumerRecord(topic = " + topic() + ", partition = " + partition() + ", offset = " + offset() + + ", key = " + key + ", value = " + value + ")"; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java index bdf4b26942d5a..416d703c3f59a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java @@ -9,53 +9,98 @@ * 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.clients.consumer; import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Map.Entry; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.AbstractIterator; /** - * A container that holds the list {@link ConsumerRecord} per partition for a particular topic. There is one for every topic returned by a - * {@link Consumer#poll(long)} operation. + * A container that holds the list {@link ConsumerRecord} per partition for a + * particular topic. There is one for every topic returned by a + * {@link Consumer#poll(long)} operation. */ -public class ConsumerRecords { +public class ConsumerRecords implements Iterable> { - private final String topic; - private final Map>> recordsPerPartition; - - public ConsumerRecords(String topic, Map>> records) { - this.topic = topic; - this.recordsPerPartition = records; + private final Map>> records; + + public ConsumerRecords(Map>> records) { + this.records = records; } - + /** - * @param partitions The input list of partitions for a particular topic. If no partitions are - * specified, returns records for all partitions - * @return The list of {@link ConsumerRecord}s associated with the given partitions. + * Get just the records for the given partition + * + * @param partition The partition to get records for */ - public List> records(int... partitions) { - List> recordsToReturn = new ArrayList>(); - if(partitions.length == 0) { - // return records for all partitions - for(Entry>> record : recordsPerPartition.entrySet()) { - recordsToReturn.addAll(record.getValue()); - } - } else { - for(int partition : partitions) { - List> recordsForThisPartition = recordsPerPartition.get(partition); - recordsToReturn.addAll(recordsForThisPartition); - } + public Iterable> records(TopicPartition partition) { + List> recs = this.records.get(partition); + if (recs == null) + return Collections.emptyList(); + else + return recs; + } + + /** + * Get just the records for the given topic + */ + public Iterable> records(String topic) { + if (topic == null) + throw new IllegalArgumentException("Topic must be non-null."); + List>> recs = new ArrayList>>(); + for (Map.Entry>> entry : records.entrySet()) { + if (entry.getKey().equals(topic)) + recs.add(entry.getValue()); } - return recordsToReturn; + return new ConcatenatedIterable(recs); } + @Override + public Iterator> iterator() { + return new ConcatenatedIterable(records.values()).iterator(); + } + /** - * @return The topic of all records associated with this instance + * The number of records for all topics */ - public String topic() { - return this.topic; + public int count() { + int count = 0; + for(List> recs: this.records.values()) + count += recs.size(); + return count; + } + + private static class ConcatenatedIterable implements Iterable> { + + private final Iterable>> iterables; + + public ConcatenatedIterable(Iterable>> iterables) { + this.iterables = iterables; + } + + @Override + public Iterator> iterator() { + return new AbstractIterator>() { + Iterator>> iters = iterables.iterator(); + Iterator> current; + + public ConsumerRecord makeNext() { + if (current == null || !current.hasNext()) { + if (iters.hasNext()) + current = iters.next().iterator(); + else + return allDone(); + } + return current.next(); + } + }; + } } + } 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 76efc216c9e6c..300c551f3d21a 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 @@ -9,380 +9,447 @@ * 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.clients.consumer; +import java.net.InetSocketAddress; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.kafka.clients.ClientRequest; +import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.ConnectionState; +import org.apache.kafka.clients.NetworkClient; +import org.apache.kafka.clients.RequestCompletionHandler; +import org.apache.kafka.clients.consumer.internals.Heartbeat; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.internals.Metadata; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Metric; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.JmxReporter; +import org.apache.kafka.common.metrics.Measurable; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.MetricsReporter; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.Avg; +import org.apache.kafka.common.metrics.stats.Count; +import org.apache.kafka.common.metrics.stats.Max; +import org.apache.kafka.common.metrics.stats.Rate; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.network.Selector; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.record.LogEntry; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.requests.ConsumerMetadataRequest; +import org.apache.kafka.common.requests.ConsumerMetadataResponse; +import org.apache.kafka.common.requests.FetchRequest; +import org.apache.kafka.common.requests.FetchRequest.PartitionData; +import org.apache.kafka.common.requests.FetchResponse; +import org.apache.kafka.common.requests.HeartbeatRequest; +import org.apache.kafka.common.requests.HeartbeatResponse; +import org.apache.kafka.common.requests.JoinGroupRequest; +import org.apache.kafka.common.requests.JoinGroupResponse; +import org.apache.kafka.common.requests.ListOffsetRequest; +import org.apache.kafka.common.requests.ListOffsetResponse; +import org.apache.kafka.common.requests.OffsetCommitRequest; +import org.apache.kafka.common.requests.OffsetCommitResponse; +import org.apache.kafka.common.requests.OffsetFetchRequest; +import org.apache.kafka.common.requests.OffsetFetchResponse; +import org.apache.kafka.common.requests.RequestHeader; +import org.apache.kafka.common.requests.RequestSend; import org.apache.kafka.common.utils.ClientUtils; import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.net.InetSocketAddress; -import java.util.*; - /** * A Kafka client that consumes records from a Kafka cluster. - *

      - * The consumer is thread safe and should generally be shared among all threads for best performance. *

      - * The consumer is single threaded and multiplexes I/O over TCP connections to each of the brokers it - * needs to communicate with. Failure to close the consumer after use will leak these resources. + * It will transparently handle the failure of servers in the Kafka cluster, and transparently adapt as partitions of + * data it subscribes to migrate within the cluster. This client also interacts with the server to allow groups of + * consumers to load balance consumption using consumer groups (as described below). + *

      + * The consumer maintains TCP connections to the necessary brokers to fetch data for the topics it subscribes to. + * Failure to close the consumer after use will leak these connections. + *

      + * The consumer is thread safe but generally will be used only from within a single thread. The consumer client has no + * threads of it's own, all work is done in the caller's thread when calls are made on the various methods exposed. + * + *

      Offsets and Consumer Position

      + * Kafka maintains a numerical offset for each record in a partition. This offset acts as a kind of unique identifier of + * a record within that partition, and also denotes the position of the consumer in the partition. That is, a consumer + * which has position 5 has consumed records with offsets 0 through 4 and will next receive record with offset 5. There + * are actually two notions of position relevant to the user of the consumer. + *

      + * The {@link #position(TopicPartition) position} of the consumer gives the offset of the next record that will be given + * out. It will be one larger than the highest offset the consumer has seen in that partition. It automatically advances + * every time the consumer receives data calls {@link #poll(long)} and receives messages. + *

      + * The {@link #commit(CommitType) committed position} is the last offset that has been saved securely. Should the + * process fail and restart, this is the offset that it will recover to. The consumer can either automatically commit + * offsets periodically, or it can choose to control this committed position manually by calling + * {@link #commit(CommitType) commit}. + *

      + * This distinction gives the consumer control over when a record is considered consumed. It is discussed in further + * detail below. + * + *

      Consumer Groups

      + * + * Kafka uses the concept of consumer groups to allow a pool of processes to divide up the work of consuming and + * processing records. These processes can either be running on the same machine or, as is more likely, they can be + * distributed over many machines to provide additional scalability and fault tolerance for processing. + *

      + * Each Kafka consumer must specify a consumer group that it belongs to. Kafka will deliver each message in the + * subscribed topics to one process in each consumer group. This is achieved by balancing the partitions in the topic + * over the consumer processes in each group. So if there is a topic with four partitions, and a consumer group with two + * processes, each process would consume from two partitions. This group membership is maintained dynamically: if a + * process fails the partitions assigned to it will be reassigned to other processes in the same group, and if a new + * process joins the group, partitions will be moved from existing consumers to this new process. + *

      + * So if two processes subscribe to a topic both specifying different groups they will each get all the records in that + * topic; if they both specify the same group they will each get about half the records. + *

      + * Conceptually you can think of a consumer group as being a single logical subscriber that happens to be made up of + * multiple processes. As a multi-subscriber system, Kafka naturally supports having any number of consumer groups for a + * given topic without duplicating data (additional consumers are actually quite cheap). + *

      + * This is a slight generalization of the functionality that is common in messaging systems. To get semantics similar to + * a queue in a traditional messaging system all processes would be part of a single consumer group and hence record + * delivery would be balanced over the group like with a queue. Unlike a traditional messaging system, though, you can + * have multiple such groups. To get semantics similar to pub-sub in a traditional messaging system each process would + * have it's own consumer group, so each process would subscribe to all the records published to the topic. + *

      + * In addition, when offsets are committed they are always committed for a given consumer group. + *

      + * It is also possible for the consumer to manually specify the partitions it subscribes to, which disables this dynamic + * partition balancing. + * *

      Usage Examples

      - * The consumer APIs offer flexibility to cover a variety of consumption use cases. Following are some examples to demonstrate the correct use of - * the available APIs. Each of the examples assumes the presence of a user implemented process() method that processes a given batch of messages - * and returns the offset of the latest processed message per partition. Note that process() is not part of the consumer API and is only used as - * a convenience method to demonstrate the different use cases of the consumer APIs. Here is a sample implementation of such a process() method. + * The consumer APIs offer flexibility to cover a variety of consumption use cases. Here are some examples to + * demonstrate how to use them. + * + *

      Simple Processing

      + * This example demonstrates the simplest usage of Kafka's consumer api. + * *
      - * {@code
      - * private Map process(Map records) {
      - *     Map processedOffsets = new HashMap();
      - *     for(Entry> recordMetadata : records.entrySet()) {
      - *          List> recordsPerTopic = recordMetadata.getValue().records();
      - *          for(int i = 0;i < recordsPerTopic.size();i++) {
      - *               ConsumerRecord record = recordsPerTopic.get(i);
      - *               // process record
      - *               try {
      - *               	processedOffsets.put(record.topicAndpartition(), record.offset());
      - *               } catch (Exception e) {
      - *               	e.printStackTrace();
      - *               }               
      - *          }
      + *     Properties props = new Properties();
      + *     props.put("metadata.broker.list", "localhost:9092");
      + *     props.put("group.id", "test");
      + *     props.put("enable.auto.commit", "true");
      + *     props.put("auto.commit.interval.ms", "1000");
      + *     props.put("session.timeout.ms", "30000");
      + *     props.put("key.serializer", "org.apache.kafka.common.serializers.StringSerializer");
      + *     props.put("value.serializer", "org.apache.kafka.common.serializers.StringSerializer");
      + *     KafkaConsumer<String, String> consumer = new KafkaConsumer<String, String>(props);
      + *     consumer.subscribe("foo", "bar");
      + *     while (true) {
      + *         ConsumerRecords<String, String> records = consumer.poll(100);
      + *         for (ConsumerRecord<String, String> record : records)
      + *             System.out.printf("offset = %d, key = %s, value = %s", record.offset(), record.key(), record.value());
        *     }
      - *     return processedOffsets; 
      - * }
      - * }
        * 
      + * + * Setting enable.auto.commit means that offsets are committed automatically with a frequency controlled by + * the config auto.commit.interval.ms. *

      - * This example demonstrates how the consumer can be used to leverage Kafka's group management functionality for automatic consumer load - * balancing and failover. This example assumes that the offsets are stored in Kafka and are automatically committed periodically, - * as controlled by the auto.commit.interval.ms config - *

      - * {@code  
      - * Properties props = new Properties();
      - * props.put("metadata.broker.list", "localhost:9092");
      - * props.put("group.id", "test");
      - * props.put("session.timeout.ms", "1000");
      - * props.put("enable.auto.commit", "true");
      - * props.put("auto.commit.interval.ms", "10000");
      - * KafkaConsumer consumer = new KafkaConsumer(props);
      - * consumer.subscribe("foo", "bar");
      - * boolean isRunning = true;
      - * while(isRunning) {
      - *   Map> records = consumer.poll(100);
      - *   process(records);
      - * }
      - * consumer.close();
      - * }
      - * 
      - * This example demonstrates how the consumer can be used to leverage Kafka's group management functionality for automatic consumer load - * balancing and failover. This example assumes that the offsets are stored in Kafka and are manually committed using - * the commit(boolean) API. This example also demonstrates rewinding the consumer's offsets if processing of the consumed - * messages fails. Note that this method of rewinding offsets using {@link #seek(Map) seek(offsets)} is only useful for rewinding the offsets - * of the current consumer instance. As such, this will not trigger a rebalance or affect the fetch offsets for the other consumer instances. + * The connection to the cluster is bootstrapped by specifying a list of one or more brokers to contact using the + * configuration metadata.broker.list. This list is just used to discover the rest of the brokers in the + * cluster and need not be an exhaustive list of servers in the cluster (though you may want to specify more than one in + * case there are servers down when the client is connecting). + *

      + * In this example the client is subscribing to the topics foo and bar as part of a group of consumers + * called test as described above. + *

      + * The broker will automatically detect failed processes in the test group by using a heartbeat mechanism. The + * consumer will automatically ping the cluster periodically, which let's the cluster know that it is alive. As long as + * the consumer is able to do this it is considered alive and retains the right to consume from the partitions assigned + * to it. If it stops heartbeating for a period of time longer than session.timeout.ms then it will be + * considered dead and it's partitions will be assigned to another process. + *

      + * The serializers settings specify how to turn the objects the user provides into bytes. By specifying the string + * serializers we are saying that our record's key and value will just be simple strings. + * + *

      Controlling When Messages Are Considered Consumed

      + * + * In this example we will consume a batch of records and batch them up in memory, when we have sufficient records + * batched we will insert them into a database. If we allowed offsets to auto commit as in the previous example messages + * would be considered consumed after they were given out by the consumer, and it would be possible that our process + * could fail after we have read messages into our in-memory buffer but before they had been inserted into the database. + * To avoid this we will manually commit the offsets only once the corresponding messages have been inserted into the + * database. This gives us exact control of when a message is considered consumed. This raises the opposite possibility: + * the process could fail in the interval after the insert into the database but before the commit (even though this + * would likely just be a few milliseconds, it is a possibility). In this case the process that took over consumption + * would consume from last committed offset and would repeat the insert of the last batch of data. Used in this way + * Kafka provides what is often called "at-least once delivery" guarantees, as each message will likely be delivered one + * time but in failure cases could be duplicated. + * *
      - * {@code  
      - * Properties props = new Properties();
      - * props.put("metadata.broker.list", "localhost:9092");
      - * props.put("group.id", "test");
      - * props.put("session.timeout.ms", "1000");
      - * props.put("enable.auto.commit", "false");
      - * KafkaConsumer consumer = new KafkaConsumer(props);
      - * consumer.subscribe("foo", "bar");
      - * int commitInterval = 100;
      - * int numRecords = 0;
      - * boolean isRunning = true;
      - * Map consumedOffsets = new HashMap();
      - * while(isRunning) {
      - *     Map> records = consumer.poll(100);
      - *     try {
      - *         Map lastConsumedOffsets = process(records);
      - *         consumedOffsets.putAll(lastConsumedOffsets);
      - *         numRecords += records.size();
      - *         // commit offsets for all partitions of topics foo, bar synchronously, owned by this consumer instance
      - *         if(numRecords % commitInterval == 0) 
      - *           consumer.commit(false);
      - *     } catch(Exception e) {
      - *         try {
      - *             // rewind consumer's offsets for failed partitions
      - *             // assume failedPartitions() returns the list of partitions for which the processing of the last batch of messages failed
      - *             List failedPartitions = failedPartitions();   
      - *             Map offsetsToRewindTo = new HashMap();
      - *             for(TopicPartition failedPartition : failedPartitions) {
      - *                 // rewind to the last consumed offset for the failed partition. Since process() failed for this partition, the consumed offset
      - *                 // should still be pointing to the last successfully processed offset and hence is the right offset to rewind consumption to.
      - *                 offsetsToRewindTo.put(failedPartition, consumedOffsets.get(failedPartition));
      + *     Properties props = new Properties();
      + *     props.put("metadata.broker.list", "localhost:9092");
      + *     props.put("group.id", "test");
      + *     props.put("enable.auto.commit", "false");
      + *     props.put("auto.commit.interval.ms", "1000");
      + *     props.put("session.timeout.ms", "30000");
      + *     props.put("key.serializer", "org.apache.kafka.common.serializers.StringSerializer");
      + *     props.put("value.serializer", "org.apache.kafka.common.serializers.StringSerializer");
      + *     KafkaConsumer<String, String> consumer = new KafkaConsumer<String, String>(props);
      + *     consumer.subscribe("foo", "bar");
      + *     int commitInterval = 200;
      + *     List<ConsumerRecord<String, String>> buffer = new ArrayList<ConsumerRecord<String, String>>();
      + *     while (true) {
      + *         ConsumerRecords<String, String> records = consumer.poll(100);
      + *         for (ConsumerRecord<String, String> record : records) {
      + *             buffer.add(record);
      + *             if (buffer.size() >= commitInterval) {
      + *                 insertIntoDb(buffer);
      + *                 consumer.commit(CommitType.SYNC);
      + *                 buffer.clear();
        *             }
      - *             // seek to new offsets only for partitions that failed the last process()
      - *             consumer.seek(offsetsToRewindTo);
      - *         } catch(Exception e) {  break; } // rewind failed
      + *         }
        *     }
      - * }         
      - * consumer.close();
      - * }
        * 
      + * + *

      Subscribing To Specific Partitions

      + * + * In the previous examples we subscribed to the topics we were interested in and let Kafka give our particular process + * a fair share of the partitions for those topics. This provides a simple load balancing mechanism so multiple + * instances of our program can divided up the work of processing records. *

      - * This example demonstrates how to rewind the offsets of the entire consumer group. It is assumed that the user has chosen to use Kafka's - * group management functionality for automatic consumer load balancing and failover. This example also assumes that the offsets are stored in - * Kafka. If group management is used, the right place to systematically rewind offsets for every consumer instance is inside the - * ConsumerRebalanceCallback. The onPartitionsAssigned callback is invoked after the consumer is assigned a new set of partitions on rebalance - * and before the consumption restarts post rebalance. This is the right place to supply the newly rewound offsets to the consumer. It - * is recommended that if you foresee the requirement to ever reset the consumer's offsets in the presence of group management, that you - * always configure the consumer to use the ConsumerRebalanceCallback with a flag that protects whether or not the offset rewind logic is used. - * This method of rewinding offsets is useful if you notice an issue with your message processing after successful consumption and offset commit. - * And you would like to rewind the offsets for the entire consumer group as part of rolling out a fix to your processing logic. In this case, - * you would configure each of your consumer instances with the offset rewind configuration flag turned on and bounce each consumer instance - * in a rolling restart fashion. Each restart will trigger a rebalance and eventually all consumer instances would have rewound the offsets for - * the partitions they own, effectively rewinding the offsets for the entire consumer group. - *

      - * {@code  
      - * Properties props = new Properties();
      - * props.put("metadata.broker.list", "localhost:9092");
      - * props.put("group.id", "test");
      - * props.put("session.timeout.ms", "1000");
      - * props.put("enable.auto.commit", "false");
      - * KafkaConsumer consumer = new KafkaConsumer(
      - *                                            props,
      - *                                            new ConsumerRebalanceCallback() {
      - *                                                boolean rewindOffsets = true;  // should be retrieved from external application config
      - *                                                public void onPartitionsAssigned(Consumer consumer, Collection partitions) {
      - *                                                    Map latestCommittedOffsets = consumer.committed(partitions);
      - *                                                    if(rewindOffsets)
      - *                                                        Map newOffsets = rewindOffsets(latestCommittedOffsets, 100);
      - *                                                    consumer.seek(newOffsets);
      - *                                                }
      - *                                                public void onPartitionsRevoked(Consumer consumer, Collection partitions) {
      - *                                                    consumer.commit(true);
      - *                                                }
      - *                                                // this API rewinds every partition back by numberOfMessagesToRewindBackTo messages 
      - *                                                private Map rewindOffsets(Map currentOffsets,
      - *                                                                                                long numberOfMessagesToRewindBackTo) {
      - *                                                    Map newOffsets = new HashMap();
      - *                                                    for(Map.Entry offset : currentOffsets.entrySet()) 
      - *                                                        newOffsets.put(offset.getKey(), offset.getValue() - numberOfMessagesToRewindBackTo);
      - *                                                    return newOffsets;
      - *                                                }
      - *                                            });
      - * consumer.subscribe("foo", "bar");
      - * int commitInterval = 100;
      - * int numRecords = 0;
      - * boolean isRunning = true;
      - * Map consumedOffsets = new HashMap();
      - * while(isRunning) {
      - *     Map> records = consumer.poll(100);
      - *     Map lastConsumedOffsets = process(records);
      - *     consumedOffsets.putAll(lastConsumedOffsets);
      - *     numRecords += records.size();
      - *     // commit offsets for all partitions of topics foo, bar synchronously, owned by this consumer instance
      - *     if(numRecords % commitInterval == 0) 
      - *         consumer.commit(consumedOffsets, true);
      - * }
      - * consumer.commit(true);
      - * consumer.close();
      - * }
      - * 
      - * This example demonstrates how the consumer can be used to leverage Kafka's group management functionality along with custom offset storage. - * In this example, the assumption made is that the user chooses to store the consumer offsets outside Kafka. This requires the user to - * plugin logic for retrieving the offsets from a custom store and provide the offsets to the consumer in the ConsumerRebalanceCallback - * callback. The onPartitionsAssigned callback is invoked after the consumer is assigned a new set of partitions on rebalance and - * before the consumption restarts post rebalance. This is the right place to supply offsets from a custom store to the consumer. + * In this mode the consumer will just get the partitions it subscribes to and if the consumer instance fails no attempt + * will be made to rebalance partitions to other instances. *

      - * Similarly, the user would also be required to plugin logic for storing the consumer's offsets to a custom store. The onPartitionsRevoked - * callback is invoked right after the consumer has stopped fetching data and before the partition ownership changes. This is the right place - * to commit the offsets for the current set of partitions owned by the consumer. - *

      - * {@code  
      - * Properties props = new Properties();
      - * props.put("metadata.broker.list", "localhost:9092");
      - * props.put("group.id", "test");
      - * props.put("session.timeout.ms", "1000");
      - * props.put("enable.auto.commit", "false"); // since enable.auto.commit only applies to Kafka based offset storage
      - * KafkaConsumer consumer = new KafkaConsumer(
      - *                                            props,
      - *                                            new ConsumerRebalanceCallback() {
      - *                                                public void onPartitionsAssigned(Consumer consumer, Collection partitions) {
      - *                                                    Map lastCommittedOffsets = getLastCommittedOffsetsFromCustomStore(partitions);
      - *                                                    consumer.seek(lastCommittedOffsets);
      - *                                                }
      - *                                                public void onPartitionsRevoked(Consumer consumer, Collection partitions) {
      - *                                                    Map offsets = getLastConsumedOffsets(partitions);
      - *                                                    commitOffsetsToCustomStore(offsets); 
      - *                                                }
      - *                                                // following APIs should be implemented by the user for custom offset management
      - *                                                private Map getLastCommittedOffsetsFromCustomStore(Collection partitions) {
      - *                                                    return null;
      - *                                                }
      - *                                                private Map getLastConsumedOffsets(Collection partitions) { return null; }
      - *                                                private void commitOffsetsToCustomStore(Map offsets) {}
      - *                                            });
      - * consumer.subscribe("foo", "bar");
      - * int commitInterval = 100;
      - * int numRecords = 0;
      - * boolean isRunning = true;
      - * Map consumedOffsets = new HashMap();
      - * while(isRunning) {
      - *     Map> records = consumer.poll(100);
      - *     Map lastConsumedOffsets = process(records);
      - *     consumedOffsets.putAll(lastConsumedOffsets);
      - *     numRecords += records.size();
      - *     // commit offsets for all partitions of topics foo, bar synchronously, owned by this consumer instance
      - *     if(numRecords % commitInterval == 0) 
      - *         commitOffsetsToCustomStore(consumedOffsets);
      - * }
      - * consumer.commit(true);
      - * consumer.close();
      - * }
      - * 
      - * This example demonstrates how the consumer can be used to subscribe to specific partitions of certain topics and consume upto the latest - * available message for each of those partitions before shutting down. When used to subscribe to specific partitions, the user foregoes - * the group management functionality and instead relies on manually configuring the consumer instances to subscribe to a set of partitions. - * This example assumes that the user chooses to use Kafka based offset storage. The user still has to specify a group.id to use Kafka - * based offset management. However, session.timeout.ms is not required since the Kafka consumer only does automatic failover when group - * management is used. - *
      - * {@code  
      - * Properties props = new Properties();
      - * props.put("metadata.broker.list", "localhost:9092");
      - * props.put("group.id", "test");
      - * props.put("enable.auto.commit", "true");
      - * props.put("auto.commit.interval.ms", "10000");
      - * KafkaConsumer consumer = new KafkaConsumer(props);
      - * // subscribe to some partitions of topic foo
      - * TopicPartition partition0 = new TopicPartition("foo", 0);
      - * TopicPartition partition1 = new TopicPartition("foo", 1);
      - * TopicPartition[] partitions = new TopicPartition[2];
      - * partitions[0] = partition0;
      - * partitions[1] = partition1;
      - * consumer.subscribe(partitions);
      - * // find the last committed offsets for partitions 0,1 of topic foo
      - * Map lastCommittedOffsets = consumer.committed(Arrays.asList(partitions));
      - * // seek to the last committed offsets to avoid duplicates
      - * consumer.seek(lastCommittedOffsets);        
      - * // find the offsets of the latest available messages to know where to stop consumption
      - * Map latestAvailableOffsets = consumer.offsetsBeforeTime(-2, Arrays.asList(partitions));
      - * boolean isRunning = true;
      - * Map consumedOffsets = new HashMap();
      - * while(isRunning) {
      - *     Map> records = consumer.poll(100);
      - *     Map lastConsumedOffsets = process(records);
      - *     consumedOffsets.putAll(lastConsumedOffsets);
      - *     for(TopicPartition partition : partitions) {
      - *         if(consumedOffsets.get(partition) >= latestAvailableOffsets.get(partition))
      - *             isRunning = false;
      - *         else
      - *             isRunning = true;
      - *     }
      - * }
      - * consumer.commit(true);
      - * consumer.close();
      - * }
      - * 
      - * This example demonstrates how the consumer can be used to subscribe to specific partitions of certain topics and consume upto the latest - * available message for each of those partitions before shutting down. When used to subscribe to specific partitions, the user foregoes - * the group management functionality and instead relies on manually configuring the consumer instances to subscribe to a set of partitions. - * This example assumes that the user chooses to use custom offset storage. + * There are several cases where this makes sense: + *
        + *
      • The first case is if the process is maintaining some kind of local state associated with that partition (like a + * local on-disk key-value store) and hence it should only get records for the partition it is maintaining on disk. + *
      • Another case is if the process itself is highly available and will be restarted if it fails (perhaps using a + * cluster management framework like YARN, Mesos, or AWS facilities, or as part of a stream processing framework). In + * this case there is no need for Kafka to detect the failure and reassign the partition, rather the consuming process + * will be restarted on another machine. + *
      + *

      + * This mode is easy to specify, rather than subscribing to the topic, the consumer just subscribes to particular + * partitions: + * *

      - * {@code  
      - * Properties props = new Properties();
      - * props.put("metadata.broker.list", "localhost:9092");
      - * KafkaConsumer consumer = new KafkaConsumer(props);
      - * // subscribe to some partitions of topic foo
      - * TopicPartition partition0 = new TopicPartition("foo", 0);
      - * TopicPartition partition1 = new TopicPartition("foo", 1);
      - * TopicPartition[] partitions = new TopicPartition[2];
      - * partitions[0] = partition0;
      - * partitions[1] = partition1;
      - * consumer.subscribe(partitions);
      - * Map lastCommittedOffsets = getLastCommittedOffsetsFromCustomStore();
      - * // seek to the last committed offsets to avoid duplicates
      - * consumer.seek(lastCommittedOffsets);        
      - * // find the offsets of the latest available messages to know where to stop consumption
      - * Map latestAvailableOffsets = consumer.offsetsBeforeTime(-2, Arrays.asList(partitions));
      - * boolean isRunning = true;
      - * Map consumedOffsets = new HashMap();
      - * while(isRunning) {
      - *     Map> records = consumer.poll(100);
      - *     Map lastConsumedOffsets = process(records);
      - *     consumedOffsets.putAll(lastConsumedOffsets);
      - *     // commit offsets for partitions 0,1 for topic foo to custom store
      - *     commitOffsetsToCustomStore(consumedOffsets);
      - *     for(TopicPartition partition : partitions) {
      - *         if(consumedOffsets.get(partition) >= latestAvailableOffsets.get(partition))
      - *             isRunning = false;
      - *         else
      - *             isRunning = true;
      - *     }            
      - * }      
      - * commitOffsetsToCustomStore(consumedOffsets);   
      - * consumer.close();
      - * }
      + *     String topic = "foo";
      + *     TopicPartition partition0 = new TopicPartition(topic, 0);
      + *     TopicPartition partition1 = new TopicPartition(topic, 1);
      + *     consumer.subscribe(partition0);
      + *     consumer.subscribe(partition1);
        * 
      + * + * The group that the consumer specifies is still used for committing offsets, but now the set of partitions will only + * be changed if the consumer specifies new partitions, and no attempt at failure detection will be made. + *

      + * It isn't possible to mix both subscription to specific partitions (with no load balancing) and to topics (with load + * balancing) using the same consumer instance. + * + *

      Managing Your Own Offsets

      + * + * The consumer application need not use Kafka's built-in offset storage, it can store offsets in a store of it's own + * choosing. The primary use case for this is allowing the application to store both the offset and the results of the + * consumption in the same system in a way that both the results and offsets are stored atomically. This is not always + * possible, but when it is it will make the consumption fully atomic and give "exactly once" semantics that are + * stronger than the default "at-least once" semantics you get with Kafka's offset commit functionality. + *

      + * Here are a couple of examples of this type of usage: + *

        + *
      • If the results of the consumption are being stored in a relational database, storing the offset in the database + * as well can allow committing both the results and offset in a single transaction. Thus either the transaction will + * succeed and the offset will be updated based on what was consumed or the result will not be stored and the offset + * won't be updated. + *
      • If the results are being stored in a local store it may be possible to store the offset there as well. For + * example a search index could be built by subscribing to a particular partition and storing both the offset and the + * indexed data together. If this is done in a way that is atomic, it is often possible to have it be the case that even + * if a crash occurs that causes unsync'd data to be lost, whatever is left has the corresponding offset stored as well. + * This means that in this case the indexing process that comes back having lost recent updates just resumes indexing + * from what it has ensuring that no updates are lost. + *
      + * + * Each record comes with it's own offset, so to manage your own offset you just need to do the following: + *
        + *
      1. Configure enable.auto.commit=false + *
      2. Use the offset provided with each {@link ConsumerRecord} to save your position. + *
      3. On restart restore the position of the consumer using {@link #seek(TopicPartition, long)}. + *
      + * + * This type of usage is simplest when the partition assignment is also done manually (this would be likely in the + * search index use case described above). If the partition assignment is done automatically special care will also be + * needed to handle the case where partition assignments change. This can be handled using a special callback specified + * using rebalance.callback.class, which specifies an implementation of the interface + * {@link ConsumerRebalanceCallback}. When partitions are taken from a consumer the consumer will want to commit its + * offset for those partitions by implementing + * {@link ConsumerRebalanceCallback#onPartitionsRevoked(Consumer, Collection)}. When partitions are assigned to a + * consumer, the consumer will want to look up the offset for those new partitions an correctly initialize the consumer + * to that position by implementing {@link ConsumerRebalanceCallback#onPartitionsAssigned(Consumer, Collection)}. + *

      + * Another common use for {@link ConsumerRebalanceCallback} is to flush any caches the application maintains for + * partitions that are moved elsewhere. + * + *

      Controlling The Consumer's Position

      + * + * In most use cases the consumer will simply consume records from beginning to end, periodically committing it's + * position (either automatically or manually). However Kafka allows the consumer to manually control it's position, + * moving forward or backwards in a partition at will. This means a consumer can re-consume older records, or skip to + * the most recent records without actually consuming the intermediate records. + *

      + * There are several instances where manually controlling the consumer's position can be useful. + *

      + * One case is for time-sensitive record processing it may make sense for a consumer that falls far enough behind to not + * attempt to catch up processing all records, but rather just skip to the most recent records. + *

      + * Another use case is for a system that maintains local state as described in the previous section. In such a system + * the consumer will want to initialize it's position on start-up to whatever is contained in the local store. Likewise + * if the local state is destroyed (say because the disk is lost) the state may be recreated on a new machine by + * reconsuming all the data and recreating the state (assuming that Kafka is retaining sufficient history). + * + * Kafka allows specifying the position using {@link #seek(TopicPartition, long)} to specify the new position. Special + * methods for seeking to the earliest and latest offset the server maintains are also available ( + * {@link #seekToBeginning(TopicPartition...)} and {@link #seekToEnd(TopicPartition...)} respectively). + * + *

      Multithreaded Processing

      + * + * The Kafka consumer is threadsafe but coarsely synchronized. All network I/O happens in the thread of the application + * making the call. We have intentionally avoided implementing a particular threading model for processing. + *

      + * This leaves several options for implementing multi-threaded processing of records. + * + *

      1. One Consumer Per Thread

      + * + * A simple option is to give each thread it's own consumer instance. Here are the pros and cons of this approach: + *
        + *
      • PRO: It is the easiest to implement + *
      • PRO: It is often the fastest as no inter-thread co-ordination is needed + *
      • PRO: It makes in-order processing on a per-partition basis very easy to implement (each thread just + * processes messages in the order it receives them). + *
      • CON: More consumers means more TCP connections to the cluster (one per thread). In general Kafka handles + * connections very efficiently so this is generally a small cost. + *
      • CON: Multiple consumers means more requests being sent to the server and slightly less batching of data + * which can cause some drop in I/O throughput. + *
      • CON: The number of total threads across all processes will be limited by the total number of partitions. + *
      + * + *

      2. Decouple Consumption and Processing

      + * + * Another alternative is to have one or more consumer threads that do all data consumption and hands off + * {@link ConsumerRecords} instances to a blocking queue consumed by a pool of processor threads that actually handle + * the record processing. + * + * This option likewise has pros and cons: + *
        + *
      • PRO: This option allows independently scaling the number of consumers and processors. This makes it + * possible to have a single consumer that feeds many processor threads, avoiding any limitation on partitions. + *
      • CON: Guaranteeing order across the processors requires particular care as the threads will execute + * independently an earlier chunk of data may actually be processed after a later chunk of data just due to the luck of + * thread execution timing. For processing that has no ordering requirements this is not a problem. + *
      • CON: Manually committing the position becomes harder as it requires that all threads co-ordinate to ensure + * that processing is complete for that partition. + *
      + * + * There are many possible variations on this approach. For example each processor thread can have it's own queue, and + * the consumer threads can hash into these queues using the TopicPartition to ensure in-order consumption and simplify + * commit. + * */ -public class KafkaConsumer implements Consumer { +public class KafkaConsumer implements Consumer { private static final Logger log = LoggerFactory.getLogger(KafkaConsumer.class); + private static final long EARLIEST_OFFSET_TIMESTAMP = -2L; + private static final long LATEST_OFFSET_TIMESTAMP = -1L; + private static final AtomicInteger consumerAutoId = new AtomicInteger(1); - private final long metadataFetchTimeoutMs; - private final long totalMemorySize; - private final Metrics metrics; - private final Set subscribedTopics; - private final Set subscribedPartitions; + private final Time time; + private final ConsumerMetrics metrics; private final Deserializer keyDeserializer; private final Deserializer valueDeserializer; + private final SubscriptionState subscriptions; + private final Metadata metadata; + private final Heartbeat heartbeat; + private final NetworkClient client; + private final int maxWaitMs; + private final int minBytes; + private final int fetchSize; + private final boolean autoCommit; + private final long autoCommitIntervalMs; + private final String group; + private final long sessionTimeoutMs; + private final long retryBackoffMs; + private final String partitionAssignmentStrategy; + private final AutoOffsetResetStrategy offsetResetStrategy; + private final ConsumerRebalanceCallback rebalanceCallback; + private final List> records; + private final boolean checkCrcs; + private long lastCommitAttemptMs; + private String consumerId; + private Node consumerCoordinator; + private boolean closed = false; + private int generation; /** * A consumer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings - * are documented here. Values can be - * either strings or Objects of the appropriate type (for example a numeric configuration would accept either the + * are documented here. Values can be + * either strings or objects of the appropriate type (for example a numeric configuration would accept either the * string "42" or the integer 42). *

      * Valid configuration strings are documented at {@link ConsumerConfig} - * @param configs The consumer configs + * + * @param configs The consumer configs */ public KafkaConsumer(Map configs) { - this(configs, null); + this(configs, null, null, null); } /** - * A consumer is instantiated by providing a set of key-value pairs as configuration and a {@link ConsumerRebalanceCallback} - * implementation + * A consumer is instantiated by providing a set of key-value pairs as configuration, a + * {@link ConsumerRebalanceCallback} implementation, a key and a value {@link Deserializer}. *

      * Valid configuration strings are documented at {@link ConsumerConfig} - * @param configs The consumer configs - * @param callback A callback interface that the user can implement to manage customized offsets on the start and end of - * every rebalance operation. + * + * @param configs The consumer configs + * @param callback A callback interface that the user can implement to manage customized offsets on the start and + * end of every rebalance operation. + * @param keyDeserializer The deserializer for key that implements {@link Deserializer}. The configure() method + * won't be called in the consumer when the deserializer is passed in directly. + * @param valueDeserializer The deserializer for value that implements {@link Deserializer}. The configure() method + * won't be called in the consumer when the deserializer is passed in directly. */ - public KafkaConsumer(Map configs, ConsumerRebalanceCallback callback) { - this(configs, callback, null, null); - } - - /** - * A consumer is instantiated by providing a set of key-value pairs as configuration, a {@link ConsumerRebalanceCallback} - * implementation, a key and a value {@link Deserializer}. - *

      - * Valid configuration strings are documented at {@link ConsumerConfig} - * @param configs The consumer configs - * @param callback A callback interface that the user can implement to manage customized offsets on the start and end of - * every rebalance operation. - * @param keyDeserializer The deserializer for key that implements {@link Deserializer}. The configure() method won't - * be called in the consumer when the deserializer is passed in directly. - * @param valueDeserializer The deserializer for value that implements {@link Deserializer}. The configure() method - * won't be called in the consumer when the deserializer is passed in directly. - */ - public KafkaConsumer(Map configs, ConsumerRebalanceCallback callback, Deserializer keyDeserializer, Deserializer valueDeserializer) { + public KafkaConsumer(Map configs, + ConsumerRebalanceCallback callback, + Deserializer keyDeserializer, + Deserializer valueDeserializer) { this(new ConsumerConfig(addDeserializerToConfig(configs, keyDeserializer, valueDeserializer)), - callback, keyDeserializer, valueDeserializer); + callback, + keyDeserializer, + valueDeserializer); } private static Map addDeserializerToConfig(Map configs, - Deserializer keyDeserializer, Deserializer valueDeserializer) { + Deserializer keyDeserializer, + Deserializer valueDeserializer) { Map newConfigs = new HashMap(); newConfigs.putAll(configs); if (keyDeserializer != null) @@ -393,24 +460,13 @@ private static Map addDeserializerToConfig(Map c } /** - * A consumer is instantiated by providing a {@link java.util.Properties} object as configuration. - * Valid configuration strings are documented at {@link ConsumerConfig} + * A consumer is instantiated by providing a {@link java.util.Properties} object as configuration. Valid + * configuration strings are documented at {@link ConsumerConfig} A consumer is instantiated by providing a + * {@link java.util.Properties} object as configuration. Valid configuration strings are documented at + * {@link ConsumerConfig} */ public KafkaConsumer(Properties properties) { - this(properties, null); - } - - /** - * A consumer is instantiated by providing a {@link java.util.Properties} object as configuration and a - * {@link ConsumerRebalanceCallback} implementation. - *

      - * Valid configuration strings are documented at {@link ConsumerConfig} - * @param properties The consumer configuration properties - * @param callback A callback interface that the user can implement to manage customized offsets on the start and end of - * every rebalance operation. - */ - public KafkaConsumer(Properties properties, ConsumerRebalanceCallback callback) { - this(properties, callback, null, null); + this(properties, null, null, null); } /** @@ -418,21 +474,28 @@ public KafkaConsumer(Properties properties, ConsumerRebalanceCallback callback) * {@link ConsumerRebalanceCallback} implementation, a key and a value {@link Deserializer}. *

      * Valid configuration strings are documented at {@link ConsumerConfig} + * * @param properties The consumer configuration properties - * @param callback A callback interface that the user can implement to manage customized offsets on the start and end of - * every rebalance operation. - * @param keyDeserializer The deserializer for key that implements {@link Deserializer}. The configure() method won't - * be called in the consumer when the deserializer is passed in directly. - * @param valueDeserializer The deserializer for value that implements {@link Deserializer}. The configure() method - * won't be called in the consumer when the deserializer is passed in directly. - */ - public KafkaConsumer(Properties properties, ConsumerRebalanceCallback callback, Deserializer keyDeserializer, Deserializer valueDeserializer) { + * @param callback A callback interface that the user can implement to manage customized offsets on the start and + * end of every rebalance operation. + * @param keyDeserializer The deserializer for key that implements {@link Deserializer}. The configure() method + * won't be called in the consumer when the deserializer is passed in directly. + * @param valueDeserializer The deserializer for value that implements {@link Deserializer}. The configure() method + * won't be called in the consumer when the deserializer is passed in directly. + */ + public KafkaConsumer(Properties properties, + ConsumerRebalanceCallback callback, + Deserializer keyDeserializer, + Deserializer valueDeserializer) { this(new ConsumerConfig(addDeserializerToConfig(properties, keyDeserializer, valueDeserializer)), - callback, keyDeserializer, valueDeserializer); + callback, + keyDeserializer, + valueDeserializer); } private static Properties addDeserializerToConfig(Properties properties, - Deserializer keyDeserializer, Deserializer valueDeserializer) { + Deserializer keyDeserializer, + Deserializer valueDeserializer) { Properties newProperties = new Properties(); newProperties.putAll(properties); if (keyDeserializer != null) @@ -442,17 +505,12 @@ private static Properties addDeserializerToConfig(Properties properties, return newProperties; } - private KafkaConsumer(ConsumerConfig config, ConsumerRebalanceCallback callback, Deserializer keyDeserializer, Deserializer valueDeserializer) { - log.trace("Starting the Kafka consumer"); - subscribedTopics = new HashSet(); - subscribedPartitions = new HashSet(); - this.metrics = new Metrics(new MetricConfig(), - Collections.singletonList((MetricsReporter) new JmxReporter("kafka.consumer.")), - new SystemTime()); - this.metadataFetchTimeoutMs = config.getLong(ConsumerConfig.METADATA_FETCH_TIMEOUT_CONFIG); - this.totalMemorySize = config.getLong(ConsumerConfig.TOTAL_BUFFER_MEMORY_CONFIG); - List addresses = ClientUtils.parseAndValidateAddresses(config.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); - + @SuppressWarnings("unchecked") + private KafkaConsumer(ConsumerConfig config, + ConsumerRebalanceCallback callback, + Deserializer keyDeserializer, + Deserializer valueDeserializer) { + log.debug("Starting the Kafka consumer"); if (keyDeserializer == null) this.keyDeserializer = config.getConfiguredInstance(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, Deserializer.class); @@ -463,181 +521,1072 @@ private KafkaConsumer(ConsumerConfig config, ConsumerRebalanceCallback callback, Deserializer.class); else this.valueDeserializer = valueDeserializer; + if (callback == null) + this.rebalanceCallback = config.getConfiguredInstance(ConsumerConfig.CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG, + ConsumerRebalanceCallback.class); + else + this.rebalanceCallback = callback; + this.time = new SystemTime(); + this.maxWaitMs = config.getInt(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG); + this.minBytes = config.getInt(ConsumerConfig.FETCH_MIN_BYTES_CONFIG); + this.fetchSize = config.getInt(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG); + this.group = config.getString(ConsumerConfig.GROUP_ID_CONFIG); + this.records = new LinkedList>(); + this.sessionTimeoutMs = config.getLong(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG); + this.heartbeat = new Heartbeat(this.sessionTimeoutMs, time.milliseconds()); + this.partitionAssignmentStrategy = config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG); + this.offsetResetStrategy = AutoOffsetResetStrategy.valueOf(config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG) + .toUpperCase()); + this.checkCrcs = config.getBoolean(ConsumerConfig.CHECK_CRCS_CONFIG); + this.autoCommit = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG); + this.autoCommitIntervalMs = config.getLong(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG); + this.lastCommitAttemptMs = time.milliseconds(); + + MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ConsumerConfig.METRICS_NUM_SAMPLES_CONFIG)) + .timeWindow(config.getLong(ConsumerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG), + TimeUnit.MILLISECONDS); + String clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG); + String jmxPrefix = "kafka.consumer"; + if(clientId .length() <= 0) + clientId = "consumer-" + consumerAutoId.getAndIncrement(); + List reporters = config.getConfiguredInstances(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG, + MetricsReporter.class); + reporters.add(new JmxReporter(jmxPrefix)); + Metrics metrics = new Metrics(metricConfig, reporters, time); + this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); + this.metadata = new Metadata(retryBackoffMs, config.getLong(ConsumerConfig.METADATA_MAX_AGE_CONFIG)); + List addresses = ClientUtils.parseAndValidateAddresses(config.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); + this.metadata.update(Cluster.bootstrap(addresses), 0); + + String metricsGroup = "consumer"; + Map metricsTags = new LinkedHashMap(); + metricsTags.put("client-id", clientId); + long reconnectBackoffMs = config.getLong(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG); + int sendBuffer = config.getInt(ConsumerConfig.SEND_BUFFER_CONFIG); + int receiveBuffer = config.getInt(ConsumerConfig.RECEIVE_BUFFER_CONFIG); + this.client = new NetworkClient(new Selector(metrics, time, metricsGroup, metricsTags), + this.metadata, + clientId, + 100, + reconnectBackoffMs, + sendBuffer, + receiveBuffer); + this.subscriptions = new SubscriptionState(); + this.metrics = new ConsumerMetrics(metrics, metricsGroup, metricsTags); config.logUnused(); - log.debug("Kafka consumer started"); + + this.consumerCoordinator = null; + this.consumerId = ""; + this.generation = -1; + log.debug("Kafka consumer created"); + } + + /** + * The set of partitions currently assigned to this consumer. If subscription happened by directly subscribing to + * partitions using {@link #subscribe(TopicPartition...)} then this will simply return the list of partitions that + * were subscribed to. If subscription was done by specifying only the topic using {@link #subscribe(String...)} + * then this will give the set of topics currently assigned to the consumer (which may be none if the assignment + * hasn't happened yet, or the partitions are in the process of getting reassigned). + */ + public synchronized Set subscriptions() { + return Collections.unmodifiableSet(this.subscriptions.assignedPartitions()); } /** * Incrementally subscribes to the given list of topics and uses the consumer's group management functionality *

      - * As part of group management, the consumer will keep track of the list of consumers that belong to a particular group and - * will trigger a rebalance operation if one of the following events trigger - + * As part of group management, the consumer will keep track of the list of consumers that belong to a particular + * group and will trigger a rebalance operation if one of the following events trigger - *

        - *
      • Number of partitions change for any of the subscribed list of topics - *
      • Topic is created or deleted - *
      • An existing member of the consumer group dies - *
      • A new member is added to an existing consumer group via the join API - *
      + *
    1. Number of partitions change for any of the subscribed list of topics + *
    2. Topic is created or deleted + *
    3. An existing member of the consumer group dies + *
    4. A new member is added to an existing consumer group via the join API + *
+ * * @param topics A variable list of topics that the consumer wants to subscribe to */ @Override - public void subscribe(String... topics) { - if(subscribedPartitions.size() > 0) - throw new IllegalStateException("Subcription to topics and partitions is mutually exclusive"); - for(String topic:topics) - subscribedTopics.add(topic); - // TODO: trigger a rebalance operation + public synchronized void subscribe(String... topics) { + ensureNotClosed(); + log.debug("Subscribed to topic(s): ", Utils.join(topics, ", ")); + for (String topic : topics) + this.subscriptions.subscribe(topic); + metadata.addTopics(topics); } /** - * Incrementally subscribes to a specific topic partition and does not use the consumer's group management functionality. As such, - * there will be no rebalance operation triggered when group membership or cluster and topic metadata change. + * Incrementally subscribes to a specific topic partition and does not use the consumer's group management + * functionality. As such, there will be no rebalance operation triggered when group membership or cluster and topic + * metadata change. *

+ * * @param partitions Partitions to incrementally subscribe to */ @Override - public void subscribe(TopicPartition... partitions) { - if(subscribedTopics.size() > 0) - throw new IllegalStateException("Subcription to topics and partitions is mutually exclusive"); - for(TopicPartition partition:partitions) - subscribedPartitions.add(partition); + public synchronized void subscribe(TopicPartition... partitions) { + ensureNotClosed(); + log.debug("Subscribed to partitions(s): ", Utils.join(partitions, ", ")); + for (TopicPartition tp : partitions) { + this.subscriptions.subscribe(tp); + metadata.addTopics(tp.topic()); + } } /** - * Unsubscribe from the specific topics. This will trigger a rebalance operation and messages for this topic will not be returned - * from the next {@link #poll(long) poll()} onwards + * Unsubscribe from the specific topics. This will trigger a rebalance operation and records for this topic will not + * be returned from the next {@link #poll(long) poll()} onwards + * * @param topics Topics to unsubscribe from */ - public void unsubscribe(String... topics) { + public synchronized void unsubscribe(String... topics) { + ensureNotClosed(); + log.debug("Unsubscribed from topic(s): ", Utils.join(topics, ", ")); // throw an exception if the topic was never subscribed to - for(String topic:topics) { - if(!subscribedTopics.contains(topic)) - throw new IllegalStateException("Topic " + topic + " was never subscribed to. subscribe(" + topic + ") should be called prior" + - " to unsubscribe(" + topic + ")"); - subscribedTopics.remove(topic); - } - // TODO trigger a rebalance operation + for (String topic : topics) + this.subscriptions.unsubscribe(topic); } /** - * Unsubscribe from the specific topic partitions. Messages for these partitions will not be returned from the next + * Unsubscribe from the specific topic partitions. records for these partitions will not be returned from the next * {@link #poll(long) poll()} onwards + * * @param partitions Partitions to unsubscribe from */ - public void unsubscribe(TopicPartition... partitions) { + public synchronized void unsubscribe(TopicPartition... partitions) { + ensureNotClosed(); + log.debug("Unsubscribed from partitions(s): ", Utils.join(partitions, ", ")); // throw an exception if the partition was never subscribed to - for(TopicPartition partition:partitions) { - if(!subscribedPartitions.contains(partition)) - throw new IllegalStateException("Partition " + partition + " was never subscribed to. subscribe(new TopicPartition(" + - partition.topic() + "," + partition.partition() + ") should be called prior" + - " to unsubscribe(new TopicPartition(" + partition.topic() + "," + partition.partition() + ")"); - subscribedPartitions.remove(partition); - } - // trigger a rebalance operation + for (TopicPartition partition : partitions) + this.subscriptions.unsubscribe(partition); } - + /** - * Fetches data for the topics or partitions specified using one of the subscribe APIs. It is an error to not have subscribed to - * any topics or partitions before polling for data. - *

- * The offset used for fetching the data is governed by whether or not {@link #seek(Map) seek(offsets)} - * is used. If {@link #seek(Map) seek(offsets)} is used, it will use the specified offsets on startup and - * on every rebalance, to consume data from that offset sequentially on every poll. If not, it will use the last checkpointed offset - * using {@link #commit(Map, boolean) commit(offsets, sync)} - * for the subscribed list of partitions. - * @param timeout The time, in milliseconds, spent waiting in poll if data is not available. If 0, waits indefinitely. Must not be negative + * Fetches data for the topics or partitions specified using one of the subscribe APIs. It is an error to not have + * subscribed to any topics or partitions before polling for data. + *

+ * The offset used for fetching the data is governed by whether or not {@link #seek(TopicPartition, long)} is used. + * If {@link #seek(TopicPartition, long)} is used, it will use the specified offsets on startup and on every + * rebalance, to consume data from that offset sequentially on every poll. If not, it will use the last checkpointed + * offset using {@link #commit(Map, CommitType) commit(offsets, sync)} for the subscribed list of partitions. + * + * @param timeout The time, in milliseconds, spent waiting in poll if data is not available. If 0, waits + * indefinitely. Must not be negative * @return map of topic to records since the last fetch for the subscribed list of topics and partitions + * + * @throws NoOffsetForPartitionException If there is no stored offset for a subscribed partition and no automatic + * offset reset policy has been configured. */ @Override - public Map> poll(long timeout) { - // TODO Auto-generated method stub - return null; + public synchronized ConsumerRecords poll(long timeout) { + ensureNotClosed(); + long now = time.milliseconds(); + + if (subscriptions.partitionsAutoAssigned()) { + // get partition assignment if needed + if (subscriptions.needsPartitionAssignment()) { + joinGroup(now); + } else if (!heartbeat.isAlive(now)) { + log.error("Failed heartbeat check."); + coordinatorDead(); + } else if (heartbeat.shouldHeartbeat(now)) { + initiateHeartbeat(now); + } + } + + // fetch positions if we have partitions we're subscribed to that we + // don't know the offset for + if (!subscriptions.hasAllFetchPositions()) + fetchMissingPositionsOrResetThem(this.subscriptions.missingFetchPositions(), now); + + // maybe autocommit position + if (shouldAutoCommit(now)) + commit(CommitType.ASYNC); + + /* + * initiate any needed fetches, then block for the timeout the user specified + */ + Cluster cluster = this.metadata.fetch(); + reinstateFetches(cluster, now); + client.poll(timeout, now); + + /* + * initiate a fetch request for any nodes that we just got a response from without blocking + */ + reinstateFetches(cluster, now); + client.poll(0, now); + + return new ConsumerRecords(consumeBufferedRecords()); } /** * Commits the specified offsets for the specified list of topics and partitions to Kafka. *

- * This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after every rebalance - * and also on startup. As such, if you need to store offsets in anything other than Kafka, this API should not be used. - * @param offsets The list of offsets per partition that should be committed to Kafka. - * @param sync If true, commit will block until the consumer receives an acknowledgment - * @return An {@link OffsetMetadata} object that contains the partition, offset and a corresponding error code. Returns null - * if the sync flag is set to false. + * This commits offsets to Kafka. The offsets committed using this API will be used on the first fetch after every + * rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API + * should not be used. + *

+ * A non-blocking commit will attempt to commit offsets asychronously. No error will be thrown if the commit fails. + * A blocking commit will wait for a response acknowledging the commit. In the event of an error it will retry until + * the commit succeeds. + * + * @param offsets The list of offsets per partition that should be committed to Kafka. + * @param commitType Control whether the commit is blocking */ @Override - public OffsetMetadata commit(Map offsets, boolean sync) { - throw new UnsupportedOperationException(); + public synchronized void commit(final Map offsets, CommitType commitType) { + ensureNotClosed(); + log.debug("Committing offsets ({}): {} ", commitType.toString().toLowerCase(), offsets); + long now = time.milliseconds(); + this.lastCommitAttemptMs = now; + if (!offsets.isEmpty()) { + Map offsetData = new HashMap(offsets.size()); + for (Map.Entry entry : offsets.entrySet()) + offsetData.put(entry.getKey(), new OffsetCommitRequest.PartitionData(entry.getValue(), now, "")); + OffsetCommitRequest req = new OffsetCommitRequest(this.group, this.generation, this.consumerId, offsetData); + + RequestCompletionHandler handler = new RequestCompletionHandler() { + public void onComplete(ClientResponse resp) { + if (resp.wasDisconnected()) { + handleDisconnect(resp, time.milliseconds()); + } else { + OffsetCommitResponse response = new OffsetCommitResponse(resp.responseBody()); + for (Map.Entry entry : response.responseData().entrySet()) { + TopicPartition tp = entry.getKey(); + short errorCode = entry.getValue(); + long offset = offsets.get(tp); + if (errorCode == Errors.NONE.code()) { + log.debug("Committed offset {} for partition {}", offset, tp); + subscriptions.committed(tp, offset); + } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() + || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { + coordinatorDead(); + } else { + log.error("Error committing partition {} at offset {}: {}", + tp, + offset, + Errors.forCode(errorCode).exception().getMessage()); + } + } + } + metrics.commitLatency.record(resp.requestLatencyMs()); + } + }; + + if (commitType == CommitType.ASYNC) { + this.initiateCoordinatorRequest(ApiKeys.OFFSET_COMMIT, req.toStruct(), handler, now); + return; + } else { + boolean done; + do { + ClientResponse response = blockingCoordinatorRequest(ApiKeys.OFFSET_COMMIT, + req.toStruct(), + handler, + now); + + // check for errors + done = true; + OffsetCommitResponse commitResponse = new OffsetCommitResponse(response.responseBody()); + for (short errorCode : commitResponse.responseData().values()) { + if (errorCode != Errors.NONE.code()) + done = false; + } + if (!done) { + log.debug("Error in offset commit, backing off for {} ms before retrying again.", + this.retryBackoffMs); + Utils.sleep(this.retryBackoffMs); + } + } while (!done); + } + } } /** - * Commits offsets returned on the last {@link #poll(long) poll()} for the subscribed list of topics and - * partitions. + * Commits offsets returned on the last {@link #poll(long) poll()} for the subscribed list of topics and partitions. *

- * This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after every rebalance - * and also on startup. As such, if you need to store offsets in anything other than Kafka, this API should not be used. - * @param sync If true, commit will block until the consumer receives an acknowledgment - * @return An {@link OffsetMetadata} object that contains the partition, offset and a corresponding error code. Returns null - * if the sync flag is set to false. + * This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after + * every rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API + * should not be used. + * + * @param commitType Whether or not the commit should block until it is acknowledged. */ @Override - public OffsetMetadata commit(boolean sync) { - throw new UnsupportedOperationException(); + public synchronized void commit(CommitType commitType) { + ensureNotClosed(); + commit(this.subscriptions.allConsumed(), commitType); } /** - * Overrides the fetch offsets that the consumer will use on the next {@link #poll(long) poll(timeout)}. If this API is invoked - * for the same partition more than once, the latest offset will be used on the next poll(). Note that you may lose data if this API is - * arbitrarily used in the middle of consumption, to reset the fetch offsets + * Overrides the fetch offsets that the consumer will use on the next {@link #poll(long) poll(timeout)}. If this API + * is invoked for the same partition more than once, the latest offset will be used on the next poll(). Note that + * you may lose data if this API is arbitrarily used in the middle of consumption, to reset the fetch offsets */ @Override - public void seek(Map offsets) { + public synchronized void seek(TopicPartition partition, long offset) { + ensureNotClosed(); + log.debug("Seeking to offset {} for partition {}", offset, partition); + this.subscriptions.seek(partition, offset); } /** - * Returns the fetch position of the next message for the specified topic partition to be used on the next {@link #poll(long) poll()} - * @param partitions Partitions for which the fetch position will be returned - * @return The position from which data will be fetched for the specified partition on the next {@link #poll(long) poll()} + * Seek to the first offset for each of the given partitions */ - public Map position(Collection partitions) { - return null; + public synchronized void seekToBeginning(TopicPartition... partitions) { + ensureNotClosed(); + Collection parts = partitions.length == 0 ? this.subscriptions.assignedPartitions() + : Arrays.asList(partitions); + for (TopicPartition tp : parts) { + // TODO: list offset call could be optimized by grouping by node + seek(tp, listOffset(tp, EARLIEST_OFFSET_TIMESTAMP)); + } } /** - * Fetches the last committed offsets of partitions that the consumer currently consumes. This API is only relevant if Kafka based offset - * storage is used. This API can be used in conjunction with {@link #seek(Map) seek(offsets)} to rewind consumption of data. - * @param partitions The list of partitions to return the last committed offset for - * @return The list of offsets committed on the last {@link #commit(boolean) commit(sync)} + * Seek to the last offset for each of the given partitions */ - @Override - public Map committed(Collection partitions) { - // TODO Auto-generated method stub - throw new UnsupportedOperationException(); + public synchronized void seekToEnd(TopicPartition... partitions) { + ensureNotClosed(); + Collection parts = partitions.length == 0 ? this.subscriptions.assignedPartitions() + : Arrays.asList(partitions); + for (TopicPartition tp : parts) { + // TODO: list offset call could be optimized by grouping by node + seek(tp, listOffset(tp, LATEST_OFFSET_TIMESTAMP)); + } } /** - * Fetches offsets before a certain timestamp. Note that the offsets returned are approximately computed and do not correspond to the exact - * message at the given timestamp. As such, if the consumer is rewound to offsets returned by this API, there may be duplicate messages - * returned by the consumer. - * @param partitions The list of partitions for which the offsets are returned - * @param timestamp The unix timestamp. Value -1 indicates earliest available timestamp. Value -2 indicates latest available timestamp. - * @return The offsets per partition before the specified timestamp. - */ - public Map offsetsBeforeTime(long timestamp, Collection partitions) { - return null; + * Returns the offset of the next record that will be fetched (if a record with that offset exists). + * + * @param partition The partition to get the position for + * @return The offset + * @throws NoOffsetForPartitionException If a position hasn't been set for a given partition, and no reset policy is + * available. + */ + public synchronized long position(TopicPartition partition) { + ensureNotClosed(); + if (!this.subscriptions.assignedPartitions().contains(partition)) + throw new IllegalArgumentException("You can only check the position for partitions assigned to this consumer."); + Long offset = this.subscriptions.consumed(partition); + if (offset == null) { + fetchMissingPositionsOrResetThem(Collections.singleton(partition), time.milliseconds()); + return this.subscriptions.consumed(partition); + } else { + return offset; + } } + /** + * Fetches the last committed offset for the given partition (whether the commit happened by this process or + * another). This offset will be used as the position for the consumer in the event of a failure. + *

+ * This call may block to do a remote call if the partition in question isn't assigned to this consumer or if the + * consumer hasn't yet initialized it's cache of committed offsets. + * + * @param partition The partition to check + * @return The last committed offset or null if no offset has been committed + * @throws NoOffsetForPartitionException If no offset has ever been committed by any process for the given + * partition. + */ + @Override + public synchronized long committed(TopicPartition partition) { + ensureNotClosed(); + Set partitionsToFetch; + if (subscriptions.assignedPartitions().contains(partition)) { + Long committed = this.subscriptions.committed(partition); + if (committed != null) + return committed; + partitionsToFetch = subscriptions.assignedPartitions(); + } else { + partitionsToFetch = Collections.singleton(partition); + } + this.refreshCommittedOffsets(time.milliseconds(), partitionsToFetch); + Long committed = this.subscriptions.committed(partition); + if (committed == null) + throw new NoOffsetForPartitionException("No offset has been committed for partition " + partition); + return committed; + } + + /** + * Get the metrics kept by the consumer + */ @Override public Map metrics() { - return Collections.unmodifiableMap(this.metrics.metrics()); + return Collections.unmodifiableMap(this.metrics.metrics.metrics()); } + /** + * Get metadata about the partitions for a given topic. This method will issue a remote call to the server if it + * does not already have any metadata about the given topic. + * + * @param topic The topic to get partition metadata for + * @return The list of partitions + */ @Override - public void close() { + public List partitionsFor(String topic) { + Cluster cluster = this.metadata.fetch(); + List parts = cluster.partitionsForTopic(topic); + if (parts == null) { + metadata.add(topic); + awaitMetadataUpdate(); + parts = metadata.fetch().partitionsForTopic(topic); + } + return parts; + } + + @Override + public synchronized void close() { log.trace("Closing the Kafka consumer."); - subscribedTopics.clear(); - subscribedPartitions.clear(); - this.metrics.close(); + this.closed = true; + this.metrics.metrics.close(); + this.client.close(); log.debug("The Kafka consumer has closed."); } + + private boolean shouldAutoCommit(long now) { + return this.autoCommit && this.lastCommitAttemptMs <= now - this.autoCommitIntervalMs; + } + + /* + * Request a metadata update and wait until it has occurred + */ + private void awaitMetadataUpdate() { + int version = this.metadata.requestUpdate(); + do { + long now = time.milliseconds(); + this.client.poll(this.retryBackoffMs, now); + } while (this.metadata.version() == version); + } + + /* + * Send a join group request to the controller + */ + private void joinGroup(long now) { + log.debug("Joining group {}", group); + + // execute the user's callback + try { + // TODO: Hmmm, is passing the full Consumer like this actually safe? + // Need to think about reentrancy... + this.rebalanceCallback.onPartitionsRevoked(this, this.subscriptions.assignedPartitions()); + } catch (Exception e) { + log.error("User provided callback " + this.rebalanceCallback.getClass().getName() + + " failed on partition revocation: ", e); + } + + // join the group + JoinGroupRequest jgr = new JoinGroupRequest(group, + (int) this.sessionTimeoutMs, + new ArrayList(this.subscriptions.subscribedTopics()), + this.consumerId, + this.partitionAssignmentStrategy); + ClientResponse resp = this.blockingCoordinatorRequest(ApiKeys.JOIN_GROUP, jgr.toStruct(), null, now); + // process the response + JoinGroupResponse response = new JoinGroupResponse(resp.responseBody()); + log.debug("Joined group: {}", response); + Errors.forCode(response.errorCode()).maybeThrow(); + this.consumerId = response.consumerId(); + this.subscriptions.changePartitionAssignment(response.assignedPartitions()); + this.heartbeat.receivedResponse(now); + + // execute the callback + try { + // TODO: Hmmm, is passing the full Consumer like this actually safe? + this.rebalanceCallback.onPartitionsAssigned(this, this.subscriptions.assignedPartitions()); + } catch (Exception e) { + log.error("User provided callback " + this.rebalanceCallback.getClass().getName() + + " failed on partition assignment: ", e); + } + + // record re-assignment time + this.metrics.partitionReassignments.record(time.milliseconds() - now); + } + + /* + * Empty the record buffer and update the consumed position. + */ + private Map>> consumeBufferedRecords() { + if (this.subscriptions.needsPartitionAssignment()) { + return Collections.emptyMap(); + } else { + Map>> drained = new HashMap>>(); + for (PartitionRecords part : this.records) { + Long consumed = subscriptions.consumed(part.partition); + if (this.subscriptions.assignedPartitions().contains(part.partition) + && (consumed == null || part.fetchOffset == consumed)) { + List> partRecs = drained.get(part.partition); + if (partRecs == null) { + partRecs = part.records; + drained.put(part.partition, partRecs); + } else { + partRecs.addAll(part.records); + } + subscriptions.consumed(part.partition, part.records.get(part.records.size() - 1).offset() + 1); + } else { + // these records aren't next in line based on the last consumed position, ignore them + // they must be from an obsolete request + log.debug("Ignoring fetched records for {} at offset {}", part.partition, part.fetchOffset); + } + } + this.records.clear(); + return drained; + } + } + + /* + * Set-up a fetch request for any node that we have assigned partitions for which doesn't have one + */ + private void reinstateFetches(Cluster cluster, long now) { + for (ClientRequest request : createFetchRequests(cluster)) { + Node node = cluster.nodeById(request.request().destination()); + if (client.ready(node, now)) { + log.trace("Initiating fetch to node {}: {}", node.id(), request); + client.send(request); + } + } + } + + /* + * Create fetch requests for all nodes for which we have assigned partitions that have no existing requests in + * flight + */ + private List createFetchRequests(Cluster cluster) { + Map> fetchable = new HashMap>(); + for (TopicPartition partition : subscriptions.assignedPartitions()) { + Node node = cluster.leaderFor(partition); + // if there is a leader and no in-flight requests, issue a new fetch + if (node != null && this.client.inFlightRequestCount(node.id()) == 0) { + Map fetch = fetchable.get(node); + if (fetch == null) { + fetch = new HashMap(); + fetchable.put(node.id(), fetch); + } + long offset = this.subscriptions.fetched(partition); + fetch.put(partition, new FetchRequest.PartitionData(offset, this.fetchSize)); + } + } + List requests = new ArrayList(fetchable.size()); + for (Map.Entry> entry : fetchable.entrySet()) { + int nodeId = entry.getKey(); + final FetchRequest fetch = new FetchRequest(this.maxWaitMs, minBytes, entry.getValue()); + RequestSend send = new RequestSend(nodeId, this.client.nextRequestHeader(ApiKeys.FETCH), fetch.toStruct()); + RequestCompletionHandler handler = new RequestCompletionHandler() { + public void onComplete(ClientResponse response) { + handleFetchResponse(response, fetch); + } + }; + requests.add(new ClientRequest(time.milliseconds(), true, send, handler)); + } + return requests; + } + + private void handleFetchResponse(ClientResponse resp, FetchRequest request) { + if (resp.wasDisconnected()) { + handleDisconnect(resp, time.milliseconds()); + } else { + int totalBytes = 0; + int totalCount = 0; + FetchResponse response = new FetchResponse(resp.responseBody()); + for (Map.Entry entry : response.responseData().entrySet()) { + TopicPartition tp = entry.getKey(); + FetchResponse.PartitionData partition = entry.getValue(); + if (!subscriptions.assignedPartitions().contains(tp)) { + log.debug("Ignoring fetched data for partition {} which is no longer assigned.", tp); + } else if (partition.errorCode == Errors.NONE.code()) { + ByteBuffer buffer = partition.recordSet; + buffer.position(buffer.limit()); // TODO: arguably we should not have to muck with the position here + MemoryRecords records = MemoryRecords.readableRecords(buffer); + long fetchOffset = request.fetchData().get(tp).offset; + int bytes = 0; + List> parsed = new ArrayList>(); + for (LogEntry logEntry : records) { + parsed.add(parseRecord(tp, logEntry)); + bytes += logEntry.size(); + } + if (parsed.size() > 0) { + ConsumerRecord record = parsed.get(parsed.size() - 1); + this.subscriptions.fetched(tp, record.offset() + 1); + this.metrics.lag.record(partition.highWatermark - record.offset()); + this.records.add(new PartitionRecords(fetchOffset, tp, parsed)); + } + this.metrics.recordTopicFetchMetrics(tp.topic(), bytes, parsed.size()); + totalBytes += bytes; + totalCount += parsed.size(); + } else if (partition.errorCode == Errors.NOT_LEADER_FOR_PARTITION.code() + || partition.errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code() + || partition.errorCode == Errors.LEADER_NOT_AVAILABLE.code()) { + this.metadata.requestUpdate(); + } else if (partition.errorCode == Errors.OFFSET_OUT_OF_RANGE.code()) { + // TODO: this could be optimized by grouping all out-of-range partitions + resetOffset(tp, time.milliseconds()); + } + } + this.metrics.bytesFetched.record(totalBytes); + this.metrics.recordsFetched.record(totalCount); + } + this.metrics.fetchLatency.record(resp.requestLatencyMs()); + } + + private ConsumerRecord parseRecord(TopicPartition partition, LogEntry logEntry) { + if (this.checkCrcs) + logEntry.record().ensureValid(); + long offset = logEntry.offset(); + ByteBuffer keyBytes = logEntry.record().key(); + K key = keyBytes == null ? null : this.keyDeserializer.deserialize(partition.topic(), Utils.toArray(keyBytes)); + ByteBuffer valueBytes = logEntry.record().value(); + V value = valueBytes == null ? null : this.valueDeserializer.deserialize(partition.topic(), + Utils.toArray(valueBytes)); + return new ConsumerRecord(partition.topic(), partition.partition(), offset, key, value); + } + + /* + * Begin sending a heartbeat to the controller but don't wait for the response + */ + private void initiateHeartbeat(long now) { + ensureCoordinatorReady(); + log.debug("Sending heartbeat to co-ordinator."); + HeartbeatRequest req = new HeartbeatRequest(this.group, this.generation, this.consumerId); + RequestSend send = new RequestSend(this.consumerCoordinator.id(), + this.client.nextRequestHeader(ApiKeys.HEARTBEAT), + req.toStruct()); + + RequestCompletionHandler handler = new RequestCompletionHandler() { + public void onComplete(ClientResponse resp) { + if (resp.wasDisconnected()) { + coordinatorDead(); + } else { + HeartbeatResponse response = new HeartbeatResponse(resp.responseBody()); + if (response.errorCode() == Errors.NONE.code()) { + log.debug("Received successful heartbeat response."); + heartbeat.receivedResponse(time.milliseconds()); + } else if (response.errorCode() == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code() + || response.errorCode() == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) { + coordinatorDead(); + } else { + throw new KafkaException("Unexpected error in hearbeat response: " + + Errors.forCode(response.errorCode()).exception().getMessage()); + } + } + metrics.heartbeatLatency.record(resp.requestLatencyMs()); + } + }; + this.client.send(new ClientRequest(now, true, send, handler)); + this.heartbeat.sentHeartbeat(now); + } + + private void coordinatorDead() { + log.info("Marking the co-ordinator dead."); + heartbeat.markDead(); + if (subscriptions.partitionsAutoAssigned()) + subscriptions.clearAssignment(); + this.consumerCoordinator = null; + } + + /* + * Initiate a request to the co-ordinator but don't wait for a response. + */ + private void initiateCoordinatorRequest(ApiKeys api, Struct request, RequestCompletionHandler handler, long now) { + log.debug("Issuing co-ordinator request: {}: {}", api, request); + ensureCoordinatorReady(); + RequestHeader header = this.client.nextRequestHeader(api); + RequestSend send = new RequestSend(this.consumerCoordinator.id(), header, request); + ClientRequest clientRequest = new ClientRequest(now, true, send, handler); + this.client.send(clientRequest); + } + + /* + * Repeatedly attempt to send a request to the co-ordinator until a response is received (retry if we are + * disconnected). Note that this means any requests sent this way must be idempotent. + * + * @return The response + */ + private ClientResponse blockingCoordinatorRequest(ApiKeys api, + Struct request, + RequestCompletionHandler handler, + long now) { + do { + initiateCoordinatorRequest(api, request, handler, now); + List responses = this.client.completeAll(consumerCoordinator.id(), now); + if (responses.size() == 0) { + throw new IllegalStateException("This should not happen."); + } else { + ClientResponse response = responses.get(responses.size() - 1); + if (response.wasDisconnected()) { + handleDisconnect(response, time.milliseconds()); + Utils.sleep(this.retryBackoffMs); + } else { + return response; + } + } + } while (true); + } + + /* + * update the current consumer co-ordinator if needed and ensure we have a ready connection to it + */ + private void ensureCoordinatorReady() { + while (true) { + if (this.consumerCoordinator == null) + discoverCoordinator(); + + while (true) { + boolean ready = this.client.ready(this.consumerCoordinator, time.milliseconds()); + if (ready) { + return; + } else { + log.debug("No connection to co-ordinator, attempting to connect."); + this.client.poll(this.retryBackoffMs, time.milliseconds()); + ConnectionState state = this.client.connectionState(this.consumerCoordinator.id()); + if (ConnectionState.DISCONNECTED.equals(state)) { + log.debug("Co-ordinator connection failed. Attempting to re-discover."); + coordinatorDead(); + break; + } + } + } + } + } + + private void discoverCoordinator() { + while (this.consumerCoordinator == null) { + log.debug("No consumer co-ordinator known, attempting to discover one."); + Node coordinator = fetchConsumerCoordinator(); + + if (coordinator == null) { + log.debug("No co-ordinator found, backing off."); + Utils.sleep(this.retryBackoffMs); + } else { + log.debug("Found consumer co-ordinator: " + coordinator); + this.consumerCoordinator = coordinator; + } + } + } + + private Node fetchConsumerCoordinator() { + // find a node to ask about the co-ordinator + Node node = this.client.leastLoadedNode(time.milliseconds()); + while (node == null || !this.client.ready(node, time.milliseconds())) { + long now = time.milliseconds(); + this.client.poll(this.retryBackoffMs, now); + node = this.client.leastLoadedNode(now); + } + + // send the metadata request and process all responses + long now = time.milliseconds(); + this.client.send(createConsumerMetadataRequest(now)); + List responses = this.client.completeAll(node.id(), now); + if (responses.isEmpty()) { + throw new IllegalStateException("This should not happen."); + } else { + ClientResponse resp = responses.get(responses.size() - 1); + if (!resp.wasDisconnected()) { + ConsumerMetadataResponse response = new ConsumerMetadataResponse(resp.responseBody()); + if (response.errorCode() == Errors.NONE.code()) + return new Node(Integer.MIN_VALUE, response.node().host(), response.node().port()); + } + } + return null; + } + + /** + * Update our cache of committed positions and then set the fetch position to the committed position (if there is + * one) or reset it using the offset reset policy the user has configured. + * + * @throws NoOffsetForPartitionException If no offset is stored for a given partition and no offset reset policy is + * defined + */ + private void fetchMissingPositionsOrResetThem(Set partitions, long now) { + // update the set of committed offsets + refreshCommittedOffsets(now, partitions); + + // reset the fetch position to the committed poisition + for (TopicPartition tp : partitions) { + if (subscriptions.fetched(tp) == null) { + if (subscriptions.committed(tp) == null) { + resetOffset(tp, now); + } else { + log.debug("Resetting offset for partition {} to committed offset"); + subscriptions.seek(tp, subscriptions.committed(tp)); + } + } + } + } + + /* + * Fetch the given set of partitions and update the cache of committed offsets using the result + */ + private void refreshCommittedOffsets(long now, Set partitions) { + log.debug("Fetching committed offsets for partitions: " + Utils.join(partitions, ", ")); + OffsetFetchRequest request = new OffsetFetchRequest(this.group, new ArrayList(partitions)); + ClientResponse resp = this.blockingCoordinatorRequest(ApiKeys.OFFSET_FETCH, request.toStruct(), null, now); + OffsetFetchResponse response = new OffsetFetchResponse(resp.responseBody()); + for (Map.Entry entry : response.responseData().entrySet()) { + TopicPartition tp = entry.getKey(); + OffsetFetchResponse.PartitionData data = entry.getValue(); + if (data.hasError()) { + log.debug("Error fetching offset for topic-partition {}: {}", tp, Errors.forCode(data.errorCode) + .exception() + .getMessage()); + } else if (data.offset >= 0) { + // update the position with the offset (-1 seems to indicate no + // such offset known) + this.subscriptions.committed(tp, data.offset); + } else { + log.debug("No committed offset for partition " + tp); + } + } + } + + /* + * Fetch a single offset before the given timestamp for the partition. + */ + private long listOffset(TopicPartition tp, long ts) { + log.debug("Fetching offsets for partition {}.", tp); + Map partitions = new HashMap(1); + partitions.put(tp, new ListOffsetRequest.PartitionData(ts, 1)); + while (true) { + long now = time.milliseconds(); + PartitionInfo info = metadata.fetch().partition(tp); + if (info == null) { + metadata.add(tp.topic()); + awaitMetadataUpdate(); + } else if (info.leader() == null) { + awaitMetadataUpdate(); + } else if (this.client.ready(info.leader(), now)) { + Node node = info.leader(); + ListOffsetRequest request = new ListOffsetRequest(-1, partitions); + RequestSend send = new RequestSend(node.id(), + this.client.nextRequestHeader(ApiKeys.LIST_OFFSETS), + request.toStruct()); + ClientRequest clientRequest = new ClientRequest(now, true, send, null); + this.client.send(clientRequest); + List responses = this.client.completeAll(node.id(), now); + if (responses.isEmpty()) + throw new IllegalStateException("This should not happen."); + ClientResponse response = responses.get(responses.size() - 1); + if (response.wasDisconnected()) { + awaitMetadataUpdate(); + } else { + ListOffsetResponse lor = new ListOffsetResponse(response.responseBody()); + short errorCode = lor.responseData().get(tp).errorCode; + if (errorCode == Errors.NONE.code()) { + List offsets = lor.responseData().get(tp).offsets; + if (offsets.size() != 1) + throw new IllegalStateException("This should not happen."); + return offsets.get(0); + } else if (errorCode == Errors.NOT_LEADER_FOR_PARTITION.code() + || errorCode == Errors.LEADER_NOT_AVAILABLE.code()) { + log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.", + tp); + awaitMetadataUpdate(); + continue; + } else { + Errors.forCode(errorCode).maybeThrow(); + } + } + } else { + client.poll(this.retryBackoffMs, now); + } + } + } + + /* + * Create a consumer metadata request for the given group + */ + private ClientRequest createConsumerMetadataRequest(long now) { + ConsumerMetadataRequest request = new ConsumerMetadataRequest(this.group); + Node destination = this.client.leastLoadedNode(now); + if (destination == null) // all nodes are blacked out + return null; + RequestSend send = new RequestSend(destination.id(), + this.client.nextRequestHeader(ApiKeys.CONSUMER_METADATA), + request.toStruct()); + ClientRequest consumerMetadataRequest = new ClientRequest(now, true, send, null); + return consumerMetadataRequest; + } + + /** + * Reset offsets for the given partition using the offset reset strategy + * + * @throws NoOffsetForPartitionException If no offset reset strategy is defined + */ + private void resetOffset(TopicPartition partition, long now) { + long timestamp; + if (this.offsetResetStrategy == AutoOffsetResetStrategy.EARLIEST) + timestamp = EARLIEST_OFFSET_TIMESTAMP; + else if (this.offsetResetStrategy == AutoOffsetResetStrategy.LATEST) + timestamp = LATEST_OFFSET_TIMESTAMP; + else + throw new NoOffsetForPartitionException("No offset is set and no reset policy is defined"); + + log.debug("Resetting offset for partition {} to {} offset.", partition, this.offsetResetStrategy.name() + .toLowerCase()); + this.subscriptions.seek(partition, listOffset(partition, timestamp)); + } + + private void handleDisconnect(ClientResponse response, long now) { + int correlation = response.request().request().header().correlationId(); + log.debug("Cancelled request {} with correlation id {} due to node {} being disconnected", + response.request(), + correlation, + response.request().request().destination()); + if (this.consumerCoordinator != null + && response.request().request().destination() == this.consumerCoordinator.id()) + coordinatorDead(); + } + + /* + * Check that the consumer hasn't been closed. + */ + private void ensureNotClosed() { + if (this.closed) + throw new IllegalStateException("This consumer has already been closed."); + } + + private static class PartitionRecords { + public long fetchOffset; + public TopicPartition partition; + public List> records; + + public PartitionRecords(long fetchOffset, TopicPartition partition, List> records) { + this.fetchOffset = fetchOffset; + this.partition = partition; + this.records = records; + } + } + + private static enum AutoOffsetResetStrategy { + LATEST, EARLIEST, NONE; + } + + private class ConsumerMetrics { + public final Metrics metrics; + public final Sensor bytesFetched; + public final Sensor recordsFetched; + public final Sensor fetchLatency; + public final Sensor commitLatency; + public final Sensor partitionReassignments; + public final Sensor heartbeatLatency; + public final Sensor lag; + + public ConsumerMetrics(Metrics metrics, String metricsGroup, Map tags) { + this.metrics = metrics; + + this.bytesFetched = metrics.sensor("bytes-fetched"); + this.bytesFetched.add(new MetricName("fetch-size-avg", + metricsGroup, + "The average number of bytes fetched per request", + tags), new Avg()); + this.bytesFetched.add(new MetricName("fetch-size-max", + metricsGroup, + "The maximum number of bytes fetched per request", + tags), new Max()); + this.bytesFetched.add(new MetricName("bytes-consumed-rate", + metricsGroup, + "The average number of bytes consumed per second", + tags), new Rate()); + + this.recordsFetched = metrics.sensor("records-fetched"); + this.recordsFetched.add(new MetricName("records-per-request-avg", + metricsGroup, + "The average number of records in each request", + tags), new Avg()); + this.recordsFetched.add(new MetricName("records-consumed-rate", + metricsGroup, + "The average number of records consumed per second", + tags), new Rate()); + + this.fetchLatency = metrics.sensor("fetch-latency"); + this.fetchLatency.add(new MetricName("fetch-latency-avg", + metricsGroup, + "The average time taken for a fetch request.", + tags), new Avg()); + this.fetchLatency.add(new MetricName("fetch-latency-max", + metricsGroup, + "The max time taken for any fetch request.", + tags), new Max()); + this.fetchLatency.add(new MetricName("fetch-rate", + metricsGroup, + "The number of fetch requests per second.", + tags), new Rate(new Count())); + + this.commitLatency = metrics.sensor("commit-latency"); + this.commitLatency.add(new MetricName("commit-latency-avg", + metricsGroup, + "The average time taken for a commit request", + tags), new Avg()); + this.commitLatency.add(new MetricName("commit-latency-max", + metricsGroup, + "The max time taken for a commit request", + tags), new Max()); + this.commitLatency.add(new MetricName("commit-rate", + metricsGroup, + "The number of commit calls per second", + tags), new Rate(new Count())); + + this.partitionReassignments = metrics.sensor("reassignment-latency"); + this.partitionReassignments.add(new MetricName("reassignment-time-avg", + metricsGroup, + "The average time taken for a partition reassignment", + tags), new Avg()); + this.partitionReassignments.add(new MetricName("reassignment-time-max", + metricsGroup, + "The max time taken for a partition reassignment", + tags), new Avg()); + this.partitionReassignments.add(new MetricName("reassignment-rate", + metricsGroup, + "The number of partition reassignments per second", + tags), new Rate(new Count())); + + this.heartbeatLatency = metrics.sensor("heartbeat-latency"); + this.heartbeatLatency.add(new MetricName("heartbeat-response-time-max", + metricsGroup, + "The max time taken to receive a response to a hearbeat request", + tags), new Max()); + this.heartbeatLatency.add(new MetricName("heartbeat-rate", + metricsGroup, + "The average number of heartbeats per second", + tags), new Rate(new Count())); + + this.lag = metrics.sensor("lag"); + this.lag.add(new MetricName("lag-max", + metricsGroup, + "The maximum lag for any partition in this window", + tags), new Max()); + + metrics.addMetric(new MetricName("assigned-partitions", + metricsGroup, + "The number of partitions currently assigned to this consumer", + tags), new Measurable() { + public double measure(MetricConfig config, long now) { + return subscriptions.assignedPartitions().size(); + } + }); + + metrics.addMetric(new MetricName("last-heartbeat-seconds-ago", + metricsGroup, + "The number of seconds since the last controller heartbeat", + tags), new Measurable() { + public double measure(MetricConfig config, long now) { + return TimeUnit.SECONDS.convert(now - heartbeat.lastHeartbeatSend(), TimeUnit.MILLISECONDS); + } + }); + } + + public void recordTopicFetchMetrics(String topic, int bytes, int records) { + // record bytes fetched + String name = "topic." + topic + ".bytes-fetched"; + Sensor bytesFetched = this.metrics.getSensor(name); + if (bytesFetched == null) + bytesFetched = this.metrics.sensor(name); + bytesFetched.record(bytes); + + // record records fetched + name = "topic." + topic + ".records-fetched"; + Sensor recordsFetched = this.metrics.getSensor(name); + if (recordsFetched == null) + recordsFetched = this.metrics.sensor(name); + recordsFetched.record(bytes); + } + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java index fa88ac1a8b19b..f50da82575693 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java @@ -9,185 +9,174 @@ * 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.clients.consumer; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectOutputStream; 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.Map.Entry; import java.util.Set; + +import org.apache.kafka.clients.consumer.internals.SubscriptionState; import org.apache.kafka.common.Metric; +import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.MetricName; /** - * A mock of the {@link Consumer} interface you can use for testing code that uses Kafka. - * This class is not threadsafe + * A mock of the {@link Consumer} interface you can use for testing code that uses Kafka. This class is not + * threadsafe *

- * The consumer runs in the user thread and multiplexes I/O over TCP connections to each of the brokers it - * needs to communicate with. Failure to close the consumer after use will leak these resources. + * The consumer runs in the user thread and multiplexes I/O over TCP connections to each of the brokers it needs to + * communicate with. Failure to close the consumer after use will leak these resources. */ -public class MockConsumer implements Consumer { +public class MockConsumer implements Consumer { + + private final Map> partitions; + private final SubscriptionState subscriptions; + private Map>> records; + private boolean closed; - private final Set subscribedPartitions; - private final Set subscribedTopics; - private final Map committedOffsets; - private final Map consumedOffsets; - public MockConsumer() { - subscribedPartitions = new HashSet(); - subscribedTopics = new HashSet(); - committedOffsets = new HashMap(); - consumedOffsets = new HashMap(); + this.subscriptions = new SubscriptionState(); + this.partitions = new HashMap>(); + this.records = new HashMap>>(); + this.closed = false; } @Override - public void subscribe(String... topics) { - if(subscribedPartitions.size() > 0) - throw new IllegalStateException("Subcription to topics and partitions is mutually exclusive"); - for(String topic : topics) { - subscribedTopics.add(topic); - } + public synchronized Set subscriptions() { + return this.subscriptions.assignedPartitions(); } @Override - public void subscribe(TopicPartition... partitions) { - if(subscribedTopics.size() > 0) - throw new IllegalStateException("Subcription to topics and partitions is mutually exclusive"); - for(TopicPartition partition : partitions) { - subscribedPartitions.add(partition); - consumedOffsets.put(partition, 0L); - } + public synchronized void subscribe(String... topics) { + ensureNotClosed(); + for (String topic : topics) + this.subscriptions.subscribe(topic); } - public void unsubscribe(String... topics) { - // throw an exception if the topic was never subscribed to - for(String topic:topics) { - if(!subscribedTopics.contains(topic)) - throw new IllegalStateException("Topic " + topic + " was never subscribed to. subscribe(" + topic + ") should be called prior" + - " to unsubscribe(" + topic + ")"); - subscribedTopics.remove(topic); - } + @Override + public synchronized void subscribe(TopicPartition... partitions) { + ensureNotClosed(); + for (TopicPartition partition : partitions) + this.subscriptions.subscribe(partition); } - public void unsubscribe(TopicPartition... partitions) { - // throw an exception if the partition was never subscribed to - for(TopicPartition partition:partitions) { - if(!subscribedPartitions.contains(partition)) - throw new IllegalStateException("Partition " + partition + " was never subscribed to. subscribe(new TopicPartition(" + - partition.topic() + "," + partition.partition() + ") should be called prior" + - " to unsubscribe(new TopicPartition(" + partition.topic() + "," + partition.partition() + ")"); - subscribedPartitions.remove(partition); - committedOffsets.remove(partition); - consumedOffsets.remove(partition); - } + public synchronized void unsubscribe(String... topics) { + ensureNotClosed(); + for (String topic : topics) + this.subscriptions.unsubscribe(topic); + } + + public synchronized void unsubscribe(TopicPartition... partitions) { + ensureNotClosed(); + for (TopicPartition partition : partitions) + this.subscriptions.unsubscribe(partition); } @Override - public Map> poll(long timeout) { - // hand out one dummy record, 1 per topic - Map> records = new HashMap>(); - Map> recordMetadata = new HashMap>(); - for(TopicPartition partition : subscribedPartitions) { - // get the last consumed offset - long messageSequence = consumedOffsets.get(partition); - ByteArrayOutputStream byteStream = new ByteArrayOutputStream(); - ObjectOutputStream outputStream; - try { - outputStream = new ObjectOutputStream(byteStream); - outputStream.writeLong(messageSequence++); - outputStream.close(); - } catch (IOException e) { - e.printStackTrace(); - } - List recordsForTopic = records.get(partition.topic()); - if(recordsForTopic == null) { - recordsForTopic = new ArrayList(); - records.put(partition.topic(), recordsForTopic); - } - recordsForTopic.add(new ConsumerRecord(partition.topic(), partition.partition(), null, byteStream.toByteArray(), messageSequence)); - consumedOffsets.put(partition, messageSequence); + public synchronized ConsumerRecords poll(long timeout) { + ensureNotClosed(); + // update the consumed offset + for (Map.Entry>> entry : this.records.entrySet()) { + List> recs = entry.getValue(); + if (!recs.isEmpty()) + this.subscriptions.consumed(entry.getKey(), recs.get(recs.size() - 1).offset()); } - for(Entry> recordsPerTopic : records.entrySet()) { - Map> recordsPerPartition = new HashMap>(); - for(ConsumerRecord record : recordsPerTopic.getValue()) { - List recordsForThisPartition = recordsPerPartition.get(record.partition()); - if(recordsForThisPartition == null) { - recordsForThisPartition = new ArrayList(); - recordsPerPartition.put(record.partition(), recordsForThisPartition); - } - recordsForThisPartition.add(record); - } - recordMetadata.put(recordsPerTopic.getKey(), new ConsumerRecords(recordsPerTopic.getKey(), recordsPerPartition)); + + ConsumerRecords copy = new ConsumerRecords(this.records); + this.records = new HashMap>>(); + return copy; + } + + public synchronized void addRecord(ConsumerRecord record) { + ensureNotClosed(); + TopicPartition tp = new TopicPartition(record.topic(), record.partition()); + this.subscriptions.assignedPartitions().add(tp); + List> recs = this.records.get(tp); + if (recs == null) { + recs = new ArrayList>(); + this.records.put(tp, recs); } - return recordMetadata; + recs.add(record); } @Override - public OffsetMetadata commit(Map offsets, boolean sync) { - if(!sync) - return null; - for(Entry partitionOffset : offsets.entrySet()) { - committedOffsets.put(partitionOffset.getKey(), partitionOffset.getValue()); - } - return new OffsetMetadata(committedOffsets, null); + public synchronized void commit(Map offsets, CommitType commitType) { + ensureNotClosed(); + for (Entry entry : offsets.entrySet()) + subscriptions.committed(entry.getKey(), entry.getValue()); } @Override - public OffsetMetadata commit(boolean sync) { - if(!sync) - return null; - return commit(consumedOffsets, sync); + public synchronized void commit(CommitType commitType) { + ensureNotClosed(); + commit(this.subscriptions.allConsumed(), commitType); } @Override - public void seek(Map offsets) { - // change the fetch offsets - for(Entry partitionOffset : offsets.entrySet()) { - consumedOffsets.put(partitionOffset.getKey(), partitionOffset.getValue()); - } + public synchronized void seek(TopicPartition partition, long offset) { + ensureNotClosed(); + subscriptions.seek(partition, offset); } @Override - public Map committed(Collection partitions) { - Map offsets = new HashMap(); - for(TopicPartition partition : partitions) { - offsets.put(new TopicPartition(partition.topic(), partition.partition()), committedOffsets.get(partition)); - } - return offsets; + public synchronized long committed(TopicPartition partition) { + ensureNotClosed(); + return subscriptions.committed(partition); } @Override - public Map position(Collection partitions) { - Map positions = new HashMap(); - for(TopicPartition partition : partitions) { - positions.put(partition, consumedOffsets.get(partition)); - } - return positions; + public synchronized long position(TopicPartition partition) { + ensureNotClosed(); + return subscriptions.consumed(partition); + } + + @Override + public synchronized void seekToBeginning(TopicPartition... partitions) { + ensureNotClosed(); + throw new UnsupportedOperationException(); } @Override - public Map offsetsBeforeTime(long timestamp, - Collection partitions) { + public synchronized void seekToEnd(TopicPartition... partitions) { + ensureNotClosed(); throw new UnsupportedOperationException(); } @Override public Map metrics() { - return null; + ensureNotClosed(); + return Collections.emptyMap(); } @Override - public void close() { - // unsubscribe from all partitions - TopicPartition[] allPartitions = new TopicPartition[subscribedPartitions.size()]; - unsubscribe(subscribedPartitions.toArray(allPartitions)); + public synchronized List partitionsFor(String topic) { + ensureNotClosed(); + List parts = this.partitions.get(topic); + if (parts == null) + return Collections.emptyList(); + else + return parts; + } + + public synchronized void updatePartitions(String topic, List partitions) { + ensureNotClosed(); + this.partitions.put(topic, partitions); + } + + @Override + public synchronized void close() { + ensureNotClosed(); + this.closed = true; + } + + private void ensureNotClosed() { + if (this.closed) + throw new IllegalStateException("This consumer has already been closed."); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java new file mode 100644 index 0000000000000..a21f97be5c2e4 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java @@ -0,0 +1,29 @@ +/** + * 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.clients.consumer; + +import org.apache.kafka.common.KafkaException; + +/** + * Indicates that there is no stored offset and no defined offset reset policy + */ +public class NoOffsetForPartitionException extends KafkaException { + + private static final long serialVersionUID = 1L; + + public NoOffsetForPartitionException(String message) { + super(message); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetMetadata.java b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetMetadata.java deleted file mode 100644 index ea423ad15eebd..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetMetadata.java +++ /dev/null @@ -1,59 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.clients.consumer; - -import java.util.Map; - -import org.apache.kafka.common.TopicPartition; - -/** - * The metadata for an offset commit that has been acknowledged by the server - */ -public final class OffsetMetadata { - - private final Map offsets; - private final Map errors; - - public OffsetMetadata(Map offsets, Map errors) { - super(); - this.offsets = offsets; - this.errors = errors; - } - - public OffsetMetadata(Map offsets) { - this(offsets, null); - } - - /** - * The offset of the record in the topic/partition. - */ - public long offset(TopicPartition partition) { - if(this.errors != null) - throw errors.get(partition); - return offsets.get(partition); - } - - /** - * @return The exception corresponding to the error code returned by the server - */ - public Exception error(TopicPartition partition) { - if(errors != null) - return errors.get(partition); - else - return null; - } -} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java new file mode 100644 index 0000000000000..d9483ecf6ae4a --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java @@ -0,0 +1,47 @@ +package org.apache.kafka.clients.consumer.internals; + +/** + * A helper class for managing the heartbeat to the co-ordinator + */ +public final class Heartbeat { + + /* The number of heartbeats to attempt to complete per session timeout interval. + * so, e.g., with a session timeout of 3 seconds we would attempt a heartbeat + * once per second. + */ + private final static int HEARTBEATS_PER_SESSION_INTERVAL = 3; + + private final long timeout; + private long lastHeartbeatSend; + private long lastHeartbeatResponse; + + public Heartbeat(long timeout, long now) { + this.timeout = timeout; + this.lastHeartbeatSend = now; + this.lastHeartbeatResponse = now; + } + + public void sentHeartbeat(long now) { + this.lastHeartbeatSend = now; + } + + public void receivedResponse(long now) { + this.lastHeartbeatResponse = now; + } + + public void markDead() { + this.lastHeartbeatResponse = -1; + } + + public boolean isAlive(long now) { + return now - lastHeartbeatResponse <= timeout; + } + + public boolean shouldHeartbeat(long now) { + return now - lastHeartbeatSend > (1.0 / HEARTBEATS_PER_SESSION_INTERVAL) * this.timeout; + } + + public long lastHeartbeatSend() { + return this.lastHeartbeatSend; + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.java new file mode 100644 index 0000000000000..7e57a39690d9b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.java @@ -0,0 +1,30 @@ +/** + * 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.clients.consumer.internals; + +import java.util.Collection; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback; +import org.apache.kafka.common.TopicPartition; + +public class NoOpConsumerRebalanceCallback implements ConsumerRebalanceCallback { + + @Override + public void onPartitionsAssigned(Consumer consumer, Collection partitions) {} + + @Override + public void onPartitionsRevoked(Consumer consumer, Collection partitions) {} + +} 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 new file mode 100644 index 0000000000000..71ce20db955bd --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java @@ -0,0 +1,166 @@ +package org.apache.kafka.clients.consumer.internals; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.kafka.common.TopicPartition; + +/** + * A class for tracking the topics, partitions, and offsets for the consumer + */ +public class SubscriptionState { + + /* the list of topics the user has requested */ + private final Set subscribedTopics; + + /* the list of partitions the user has requested */ + private final Set subscribedPartitions; + + /* the list of partitions currently assigned */ + private final Set assignedPartitions; + + /* the offset exposed to the user */ + private final Map consumed; + + /* the current point we have fetched up to */ + private final Map fetched; + + /* the last committed offset for each partition */ + private final Map committed; + + /* do we need to request a partition assignment from the co-ordinator? */ + private boolean needsPartitionAssignment; + + public SubscriptionState() { + this.subscribedTopics = new HashSet(); + this.subscribedPartitions = new HashSet(); + this.assignedPartitions = new HashSet(); + this.consumed = new HashMap(); + this.fetched = new HashMap(); + this.committed = new HashMap(); + this.needsPartitionAssignment = false; + } + + public void subscribe(String topic) { + if (this.subscribedPartitions.size() > 0) + throw new IllegalStateException("Subcription to topics and partitions are mutually exclusive"); + if (!this.subscribedTopics.contains(topic)) { + this.subscribedTopics.add(topic); + this.needsPartitionAssignment = true; + } + } + + public void unsubscribe(String topic) { + if (!this.subscribedTopics.contains(topic)) + throw new IllegalStateException("Topic " + topic + " was never subscribed to."); + this.subscribedTopics.remove(topic); + this.needsPartitionAssignment = true; + for(TopicPartition tp: assignedPartitions()) + if(topic.equals(tp.topic())) + clearPartition(tp); + } + + public void subscribe(TopicPartition tp) { + if (this.subscribedTopics.size() > 0) + throw new IllegalStateException("Subcription to topics and partitions are mutually exclusive"); + this.subscribedPartitions.add(tp); + this.assignedPartitions.add(tp); + } + + public void unsubscribe(TopicPartition partition) { + if (!subscribedPartitions.contains(partition)) + throw new IllegalStateException("Partition " + partition + " was never subscribed to."); + subscribedPartitions.remove(partition); + clearPartition(partition); + } + + private void clearPartition(TopicPartition tp) { + this.assignedPartitions.remove(tp); + this.committed.remove(tp); + this.fetched.remove(tp); + this.consumed.remove(tp); + } + + public void clearAssignment() { + this.assignedPartitions.clear(); + this.committed.clear(); + this.fetched.clear(); + this.needsPartitionAssignment = !subscribedTopics().isEmpty(); + } + + public Set subscribedTopics() { + return this.subscribedTopics; + } + + public Long fetched(TopicPartition tp) { + return this.fetched.get(tp); + } + + public void fetched(TopicPartition tp, long offset) { + if (!this.assignedPartitions.contains(tp)) + throw new IllegalArgumentException("Can't change the fetch position for a partition you are not currently subscribed to."); + this.fetched.put(tp, offset); + } + + public void committed(TopicPartition tp, long offset) { + this.committed.put(tp, offset); + } + + public Long committed(TopicPartition tp) { + return this.committed.get(tp); + } + + public void seek(TopicPartition tp, long offset) { + fetched(tp, offset); + consumed(tp, offset); + } + + public Set assignedPartitions() { + return this.assignedPartitions; + } + + public boolean partitionsAutoAssigned() { + return !this.subscribedTopics.isEmpty(); + } + + public void consumed(TopicPartition tp, long offset) { + if (!this.assignedPartitions.contains(tp)) + throw new IllegalArgumentException("Can't change the consumed position for a partition you are not currently subscribed to."); + this.consumed.put(tp, offset); + } + + public Long consumed(TopicPartition partition) { + return this.consumed.get(partition); + } + + public Map allConsumed() { + return this.consumed; + } + + public boolean hasAllFetchPositions() { + return this.fetched.size() >= this.assignedPartitions.size(); + } + + public Set missingFetchPositions() { + Set copy = new HashSet(this.assignedPartitions); + copy.removeAll(this.fetched.keySet()); + return copy; + } + + public boolean needsPartitionAssignment() { + return this.needsPartitionAssignment; + } + + public void changePartitionAssignment(List assignments) { + for (TopicPartition tp : assignments) + if (!this.subscribedTopics.contains(tp.topic())) + throw new IllegalArgumentException("Assigned partition " + tp + " for non-subscribed topic."); + this.clearAssignment(); + this.assignedPartitions.addAll(assignments); + this.needsPartitionAssignment = false; + } + +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index fc71710dd5997..ebc4c5315fb94 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -329,8 +329,7 @@ public Future send(ProducerRecord record, Callback callback " to class " + producerConfig.getClass(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG).getName() + " specified in value.serializer"); } - ProducerRecord serializedRecord = new ProducerRecord(record.topic(), record.partition(), serializedKey, serializedValue); - int partition = partitioner.partition(serializedRecord, metadata.fetch()); + int partition = partitioner.partition(record.topic(), serializedKey, record.partition(), metadata.fetch()); int serializedSize = Records.LOG_OVERHEAD + Record.recordSize(serializedKey, serializedValue); ensureValidRecordSize(serializedSize); TopicPartition tp = new TopicPartition(record.topic(), partition); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java index 904976fadf061..84530f2b948f9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java @@ -100,7 +100,7 @@ public synchronized Future send(ProducerRecord r public synchronized Future send(ProducerRecord record, Callback callback) { int partition = 0; if (this.cluster.partitionsForTopic(record.topic()) != null) - partition = partitioner.partition(record, this.cluster); + partition = partitioner.partition(record.topic(), record.key(), record.partition(), this.cluster); ProduceRequestResult result = new ProduceRequestResult(); FutureRecordMetadata future = new FutureRecordMetadata(result, 0); TopicPartition topicPartition = new TopicPartition(record.topic(), partition); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 8b3e565edd1ae..9a43d66837629 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -16,9 +16,9 @@ import static org.apache.kafka.common.config.ConfigDef.Range.between; import static org.apache.kafka.common.config.ConfigDef.ValidString.in; -import java.util.Arrays; import java.util.Map; +import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; @@ -38,23 +38,17 @@ public class ProducerConfig extends AbstractConfig { private static final ConfigDef config; /** bootstrap.servers */ - public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers"; - private static final String BOOSTRAP_SERVERS_DOC = "A list of host/port pairs to use for establishing the initial connection to the Kafka cluster. Data will be load " + "balanced over all servers irrespective of which servers are specified here for bootstrapping—this list only " - + "impacts the initial hosts used to discover the full set of servers. This list should be in the form " - + "host1:port1,host2:port2,.... Since these servers are just used for the initial connection to " - + "discover the full cluster membership (which may change dynamically), this list need not contain the full set of " - + "servers (you may want more than one, though, in case a server is down). If no server in this list is available sending " - + "data will fail until on becomes available."; + public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; /** metadata.fetch.timeout.ms */ public static final String METADATA_FETCH_TIMEOUT_CONFIG = "metadata.fetch.timeout.ms"; - private static final String METADATA_FETCH_TIMEOUT_DOC = "The first time data is sent to a topic we must fetch metadata about that topic to know which servers host the " + "topic's partitions. This configuration controls the maximum amount of time we will block waiting for the metadata " + private static final String METADATA_FETCH_TIMEOUT_DOC = "The first time data is sent to a topic we must fetch metadata about that topic to know which servers host the topic's partitions. This " + "fetch to succeed before throwing an exception back to the client."; /** metadata.max.age.ms */ - public static final String METADATA_MAX_AGE_CONFIG = "metadata.max.age.ms"; - private static final String METADATA_MAX_AGE_DOC = "The period of time in milliseconds after which we force a refresh of metadata even if we haven't seen any " + " partition leadership changes to proactively discover any new brokers or partitions."; - + public static final String METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG; + private static final String METADATA_MAX_AGE_DOC = CommonClientConfigs.METADATA_MAX_AGE_DOC; + /** batch.size */ public static final String BATCH_SIZE_CONFIG = "batch.size"; private static final String BATCH_SIZE_DOC = "The producer will attempt to batch records together into fewer requests whenever multiple records are being sent" + " to the same partition. This helps performance on both the client and the server. This configuration controls the " @@ -113,17 +107,13 @@ public class ProducerConfig extends AbstractConfig { + "for example, would have the effect of reducing the number of requests sent but would add up to 5ms of latency to records sent in the absense of load."; /** client.id */ - public static final String CLIENT_ID_CONFIG = "client.id"; - private static final String CLIENT_ID_DOC = "The id string to pass to the server when making requests. The purpose of this is to be able to track the source " + "of requests beyond just ip/port by allowing a logical application name to be included with the request. The " - + "application can set any string it wants as this has no functional purpose other than in logging and metrics."; + public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG; /** send.buffer.bytes */ - public static final String SEND_BUFFER_CONFIG = "send.buffer.bytes"; - private static final String SEND_BUFFER_DOC = "The size of the TCP send buffer to use when sending data"; + public static final String SEND_BUFFER_CONFIG = CommonClientConfigs.SEND_BUFFER_CONFIG; /** receive.buffer.bytes */ - public static final String RECEIVE_BUFFER_CONFIG = "receive.buffer.bytes"; - private static final String RECEIVE_BUFFER_DOC = "The size of the TCP receive buffer to use when reading data"; + public static final String RECEIVE_BUFFER_CONFIG = CommonClientConfigs.RECEIVE_BUFFER_CONFIG; /** max.request.size */ public static final String MAX_REQUEST_SIZE_CONFIG = "max.request.size"; @@ -131,8 +121,7 @@ public class ProducerConfig extends AbstractConfig { + "batches the producer will send in a single request to avoid sending huge requests."; /** reconnect.backoff.ms */ - public static final String RECONNECT_BACKOFF_MS_CONFIG = "reconnect.backoff.ms"; - private static final String RECONNECT_BACKOFF_MS_DOC = "The amount of time to wait before attempting to reconnect to a given host when a connection fails." + " This avoids a scenario where the client repeatedly attempts to connect to a host in a tight loop."; + public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG; /** block.on.buffer.full */ public static final String BLOCK_ON_BUFFER_FULL_CONFIG = "block.on.buffer.full"; @@ -147,8 +136,7 @@ public class ProducerConfig extends AbstractConfig { + "may appear first."; /** retry.backoff.ms */ - public static final String RETRY_BACKOFF_MS_CONFIG = "retry.backoff.ms"; - private static final String RETRY_BACKOFF_MS_DOC = "The amount of time to wait before attempting to retry a failed produce request to a given topic partition." + " This avoids repeated sending-and-failing in a tight loop."; + public static final String RETRY_BACKOFF_MS_CONFIG = CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG; /** compression.type */ public static final String COMPRESSION_TYPE_CONFIG = "compression.type"; @@ -156,17 +144,13 @@ public class ProducerConfig extends AbstractConfig { + "Compression is of full batches of data, so the efficacy of batching will also impact the compression ratio (more batching means better compression)."; /** metrics.sample.window.ms */ - public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = "metrics.sample.window.ms"; - private static final String METRICS_SAMPLE_WINDOW_MS_DOC = "The metrics system maintains a configurable number of samples over a fixed window size. This configuration " + "controls the size of the window. For example we might maintain two samples each measured over a 30 second period. " - + "When a window expires we erase and overwrite the oldest window."; + public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG; /** metrics.num.samples */ - public static final String METRICS_NUM_SAMPLES_CONFIG = "metrics.num.samples"; - private static final String METRICS_NUM_SAMPLES_DOC = "The number of samples maintained to compute metrics."; + public static final String METRICS_NUM_SAMPLES_CONFIG = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG; /** metric.reporters */ - public static final String METRIC_REPORTER_CLASSES_CONFIG = "metric.reporters"; - private static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters. Implementing the MetricReporter interface allows " + "plugging in classes that will be notified of new metric creation. The JmxReporter is always included to register JMX statistics."; + public static final String METRIC_REPORTER_CLASSES_CONFIG = CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG; /** max.in.flight.requests.per.connection */ public static final String MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION = "max.in.flight.requests.per.connection"; @@ -183,22 +167,22 @@ public class ProducerConfig extends AbstractConfig { private static final String VALUE_SERIALIZER_CLASS_DOC = "Serializer class for value that implements the Serializer interface."; static { - config = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, BOOSTRAP_SERVERS_DOC) + config = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, CommonClientConfigs.BOOSTRAP_SERVERS_DOC) .define(BUFFER_MEMORY_CONFIG, Type.LONG, 32 * 1024 * 1024L, atLeast(0L), Importance.HIGH, BUFFER_MEMORY_DOC) .define(RETRIES_CONFIG, Type.INT, 0, between(0, Integer.MAX_VALUE), Importance.HIGH, RETRIES_DOC) .define(ACKS_CONFIG, Type.STRING, "1", - in(Arrays.asList("all", "-1", "0", "1")), + in("all","-1", "0", "1"), Importance.HIGH, ACKS_DOC) .define(COMPRESSION_TYPE_CONFIG, Type.STRING, "none", Importance.HIGH, COMPRESSION_TYPE_DOC) .define(BATCH_SIZE_CONFIG, Type.INT, 16384, atLeast(0), Importance.MEDIUM, BATCH_SIZE_DOC) .define(TIMEOUT_CONFIG, Type.INT, 30 * 1000, atLeast(0), Importance.MEDIUM, TIMEOUT_DOC) .define(LINGER_MS_CONFIG, Type.LONG, 0, atLeast(0L), Importance.MEDIUM, LINGER_MS_DOC) - .define(CLIENT_ID_CONFIG, Type.STRING, "", Importance.MEDIUM, CLIENT_ID_DOC) - .define(SEND_BUFFER_CONFIG, Type.INT, 128 * 1024, atLeast(0), Importance.MEDIUM, SEND_BUFFER_DOC) - .define(RECEIVE_BUFFER_CONFIG, Type.INT, 32 * 1024, atLeast(0), Importance.MEDIUM, RECEIVE_BUFFER_DOC) + .define(CLIENT_ID_CONFIG, Type.STRING, "", Importance.MEDIUM, CommonClientConfigs.CLIENT_ID_DOC) + .define(SEND_BUFFER_CONFIG, Type.INT, 128 * 1024, atLeast(0), Importance.MEDIUM, CommonClientConfigs.SEND_BUFFER_DOC) + .define(RECEIVE_BUFFER_CONFIG, Type.INT, 32 * 1024, atLeast(0), Importance.MEDIUM, CommonClientConfigs.RECEIVE_BUFFER_DOC) .define(MAX_REQUEST_SIZE_CONFIG, Type.INT, 1 * 1024 * 1024, @@ -206,9 +190,9 @@ public class ProducerConfig extends AbstractConfig { Importance.MEDIUM, MAX_REQUEST_SIZE_DOC) .define(BLOCK_ON_BUFFER_FULL_CONFIG, Type.BOOLEAN, true, Importance.LOW, BLOCK_ON_BUFFER_FULL_DOC) - .define(RECONNECT_BACKOFF_MS_CONFIG, Type.LONG, 10L, atLeast(0L), Importance.LOW, RECONNECT_BACKOFF_MS_DOC) - .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, "", Importance.LOW, METRIC_REPORTER_CLASSES_DOC) - .define(RETRY_BACKOFF_MS_CONFIG, Type.LONG, 100L, atLeast(0L), Importance.LOW, RETRY_BACKOFF_MS_DOC) + .define(RECONNECT_BACKOFF_MS_CONFIG, Type.LONG, 50L, atLeast(0L), Importance.LOW, CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC) + .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, "", Importance.LOW, CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) + .define(RETRY_BACKOFF_MS_CONFIG, Type.LONG, 100L, atLeast(0L), Importance.LOW, CommonClientConfigs.RETRY_BACKOFF_MS_DOC) .define(METADATA_FETCH_TIMEOUT_CONFIG, Type.LONG, 60 * 1000, @@ -221,8 +205,8 @@ public class ProducerConfig extends AbstractConfig { 30000, atLeast(0), Importance.LOW, - METRICS_SAMPLE_WINDOW_MS_DOC) - .define(METRICS_NUM_SAMPLES_CONFIG, Type.INT, 2, atLeast(1), Importance.LOW, METRICS_NUM_SAMPLES_DOC) + CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC) + .define(METRICS_NUM_SAMPLES_CONFIG, Type.INT, 2, atLeast(1), Importance.LOW, CommonClientConfigs.METRICS_NUM_SAMPLES_DOC) .define(MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, Type.INT, 5, diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java index dcf46581b912c..3aff6242d9d74 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java @@ -78,9 +78,9 @@ public synchronized void add(String topic) { } /** - * The next time to update the cluster info is the maximum of the time the current info will expire - * and the time the current info can be updated (i.e. backoff time has elapsed); If an update has - * been request then the expiry time is now + * The next time to update the cluster info is the maximum of the time the current info will expire and the time the + * current info can be updated (i.e. backoff time has elapsed); If an update has been request then the expiry time + * is now */ public synchronized long timeToNextUpdate(long nowMs) { long timeToExpire = needUpdate ? 0 : Math.max(this.lastRefreshMs + this.metadataExpireMs - nowMs, 0); @@ -119,6 +119,15 @@ public synchronized void awaitUpdate(final int lastVersion, final long maxWaitMs } } + /** + * Add one or more topics to maintain metadata for + */ + public synchronized void addTopics(String... topics) { + for (String topic : topics) + this.topics.add(topic); + requestUpdate(); + } + /** * Get the list of topics we are currently maintaining metadata for */ @@ -137,6 +146,13 @@ public synchronized void update(Cluster cluster, long now) { notifyAll(); log.debug("Updated cluster metadata version {} to {}", this.version, this.cluster); } + + /** + * @return The current metadata version + */ + public synchronized int version() { + return this.version; + } /** * The last time metadata was updated. diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java index 483899d2e69b3..8112e6d89df78 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java @@ -20,7 +20,6 @@ import java.util.Random; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.utils.Utils; @@ -40,32 +39,34 @@ public class Partitioner { /** * Compute the partition for the given record. * - * @param record The record being sent + * @param topic The topic name + * @param key The key to partition on (or null if no key) + * @param partition The partition to use (or null if none) * @param cluster The current cluster metadata */ - public int partition(ProducerRecord record, Cluster cluster) { - List partitions = cluster.partitionsForTopic(record.topic()); + public int partition(String topic, byte[] key, Integer partition, Cluster cluster) { + List partitions = cluster.partitionsForTopic(topic); int numPartitions = partitions.size(); - if (record.partition() != null) { + if (partition != null) { // they have given us a partition, use it - if (record.partition() < 0 || record.partition() >= numPartitions) - throw new IllegalArgumentException("Invalid partition given with record: " + record.partition() + if (partition < 0 || partition >= numPartitions) + throw new IllegalArgumentException("Invalid partition given with record: " + partition + " is not in the range [0..." + numPartitions + "]."); - return record.partition(); - } else if (record.key() == null) { + return partition; + } else if (key == null) { // choose the next available node in a round-robin fashion for (int i = 0; i < numPartitions; i++) { - int partition = Utils.abs(counter.getAndIncrement()) % numPartitions; - if (partitions.get(partition).leader() != null) - return partition; + int part = Utils.abs(counter.getAndIncrement()) % numPartitions; + if (partitions.get(part).leader() != null) + return part; } // no partitions are available, give a non-available partition return Utils.abs(counter.getAndIncrement()) % numPartitions; } else { // hash the key to choose a partition - return Utils.abs(Utils.murmur2(record.key())) % numPartitions; + return Utils.abs(Utils.murmur2(key)) % numPartitions; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index ccc03d8447ebb..8726809f8ada6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -23,6 +23,7 @@ import org.apache.kafka.clients.ClientRequest; import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.KafkaClient; +import org.apache.kafka.clients.RequestCompletionHandler; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; @@ -146,7 +147,8 @@ public void run() { /** * Run a single iteration of sending * - * @param now The current POSIX time in milliseconds + * @param now + * The current POSIX time in milliseconds */ public void run(long now) { Cluster cluster = metadata.fetch(); @@ -169,9 +171,12 @@ public void run(long now) { } // create produce requests - Map> batches = this.accumulator.drain(cluster, result.readyNodes, this.maxRequestSize, now); + Map> batches = this.accumulator.drain(cluster, + result.readyNodes, + this.maxRequestSize, + now); + sensors.updateProduceRequestMetrics(batches); List requests = createProduceRequests(batches, now); - sensors.updateProduceRequestMetrics(requests); // If we have any nodes that are ready to send + have sendable data, poll with 0 timeout so this can immediately // loop and try sending more data. Otherwise, the timeout is determined by nodes that have partitions with data @@ -183,18 +188,14 @@ public void run(long now) { log.trace("Created {} produce requests: {}", requests.size(), requests); pollTimeout = 0; } + for (ClientRequest request : requests) + client.send(request); // if some partitions are already ready to be sent, the select time would be 0; // otherwise if some partition already has some data accumulated but not ready yet, // the select time will be the time difference between now and its linger expiry time; // otherwise the select time will be the time difference between now and the metadata expiry time; - List responses = this.client.poll(requests, pollTimeout, now); - for (ClientResponse response : responses) { - if (response.wasDisconnected()) - handleDisconnect(response, now); - else - handleResponse(response, now); - } + this.client.poll(pollTimeout, now); } /** @@ -206,45 +207,44 @@ public void initiateClose() { this.wakeup(); } - private void handleDisconnect(ClientResponse response, long now) { - log.trace("Cancelled request {} due to node {} being disconnected", response, response.request().request().destination()); - int correlation = response.request().request().header().correlationId(); - @SuppressWarnings("unchecked") - Map responseBatches = (Map) response.request().attachment(); - for (RecordBatch batch : responseBatches.values()) - completeBatch(batch, Errors.NETWORK_EXCEPTION, -1L, correlation, now); - } - /** * Handle a produce response */ - private void handleResponse(ClientResponse response, long now) { + private void handleProduceResponse(ClientResponse response, Map batches, long now) { int correlationId = response.request().request().header().correlationId(); - log.trace("Received produce response from node {} with correlation id {}", - response.request().request().destination(), - correlationId); - @SuppressWarnings("unchecked") - Map batches = (Map) response.request().attachment(); - // if we have a response, parse it - if (response.hasResponse()) { - ProduceResponse produceResponse = new ProduceResponse(response.responseBody()); - for (Map.Entry entry : produceResponse.responses().entrySet()) { - TopicPartition tp = entry.getKey(); - ProduceResponse.PartitionResponse partResp = entry.getValue(); - Errors error = Errors.forCode(partResp.errorCode); - RecordBatch batch = batches.get(tp); - completeBatch(batch, error, partResp.baseOffset, correlationId, now); - } - this.sensors.recordLatency(response.request().request().destination(), response.requestLatencyMs()); - } else { - // this is the acks = 0 case, just complete all requests + if (response.wasDisconnected()) { + log.trace("Cancelled request {} due to node {} being disconnected", response, response.request() + .request() + .destination()); for (RecordBatch batch : batches.values()) - completeBatch(batch, Errors.NONE, -1L, correlationId, now); + completeBatch(batch, Errors.NETWORK_EXCEPTION, -1L, correlationId, now); + } else { + log.trace("Received produce response from node {} with correlation id {}", + response.request().request().destination(), + correlationId); + // if we have a response, parse it + if (response.hasResponse()) { + ProduceResponse produceResponse = new ProduceResponse(response.responseBody()); + for (Map.Entry entry : produceResponse.responses() + .entrySet()) { + TopicPartition tp = entry.getKey(); + ProduceResponse.PartitionResponse partResp = entry.getValue(); + Errors error = Errors.forCode(partResp.errorCode); + RecordBatch batch = batches.get(tp); + completeBatch(batch, error, partResp.baseOffset, correlationId, now); + } + this.sensors.recordLatency(response.request().request().destination(), response.requestLatencyMs()); + } else { + // this is the acks = 0 case, just complete all requests + for (RecordBatch batch : batches.values()) + completeBatch(batch, Errors.NONE, -1L, correlationId, now); + } } } /** * Complete or retry the given batch of records. + * * @param batch The record batch * @param error The error (or null if none) * @param baseOffset The base offset assigned to the records if successful @@ -294,7 +294,7 @@ private List createProduceRequests(Map */ private ClientRequest produceRequest(long now, int destination, short acks, int timeout, List batches) { Map produceRecordsByPartition = new HashMap(batches.size()); - Map recordsByPartition = new HashMap(batches.size()); + final Map recordsByPartition = new HashMap(batches.size()); for (RecordBatch batch : batches) { TopicPartition tp = batch.topicPartition; ByteBuffer recordsBuffer = batch.records.buffer(); @@ -303,8 +303,15 @@ private ClientRequest produceRequest(long now, int destination, short acks, int recordsByPartition.put(tp, batch); } ProduceRequest request = new ProduceRequest(acks, timeout, produceRecordsByPartition); - RequestSend send = new RequestSend(destination, this.client.nextRequestHeader(ApiKeys.PRODUCE), request.toStruct()); - return new ClientRequest(now, acks != 0, send, recordsByPartition); + RequestSend send = new RequestSend(destination, + this.client.nextRequestHeader(ApiKeys.PRODUCE), + request.toStruct()); + RequestCompletionHandler callback = new RequestCompletionHandler() { + public void onComplete(ClientResponse response) { + handleProduceResponse(response, recordsByPartition, time.milliseconds()); + } + }; + return new ClientRequest(now, acks != 0, send, callback); } /** @@ -428,44 +435,38 @@ public void maybeRegisterTopicMetrics(String topic) { } } - public void updateProduceRequestMetrics(List requests) { + public void updateProduceRequestMetrics(Map> batches) { long now = time.milliseconds(); - for (int i = 0; i < requests.size(); i++) { - ClientRequest request = requests.get(i); + for (List nodeBatch : batches.values()) { int records = 0; - - if (request.attachment() != null) { - Map responseBatches = (Map) request.attachment(); - for (RecordBatch batch : responseBatches.values()) { - - // register all per-topic metrics at once - String topic = batch.topicPartition.topic(); - maybeRegisterTopicMetrics(topic); - - // per-topic record send rate - String topicRecordsCountName = "topic." + topic + ".records-per-batch"; - Sensor topicRecordCount = Utils.notNull(this.metrics.getSensor(topicRecordsCountName)); - topicRecordCount.record(batch.recordCount); - - // per-topic bytes send rate - String topicByteRateName = "topic." + topic + ".bytes"; - Sensor topicByteRate = Utils.notNull(this.metrics.getSensor(topicByteRateName)); - topicByteRate.record(batch.records.sizeInBytes()); - - // per-topic compression rate - String topicCompressionRateName = "topic." + topic + ".compression-rate"; - Sensor topicCompressionRate = Utils.notNull(this.metrics.getSensor(topicCompressionRateName)); - topicCompressionRate.record(batch.records.compressionRate()); - - // global metrics - this.batchSizeSensor.record(batch.records.sizeInBytes(), now); - this.queueTimeSensor.record(batch.drainedMs - batch.createdMs, now); - this.compressionRateSensor.record(batch.records.compressionRate()); - this.maxRecordSizeSensor.record(batch.maxRecordSize, now); - records += batch.recordCount; - } - this.recordsPerRequestSensor.record(records, now); + for (RecordBatch batch : nodeBatch) { + // register all per-topic metrics at once + String topic = batch.topicPartition.topic(); + maybeRegisterTopicMetrics(topic); + + // per-topic record send rate + String topicRecordsCountName = "topic." + topic + ".records-per-batch"; + Sensor topicRecordCount = Utils.notNull(this.metrics.getSensor(topicRecordsCountName)); + topicRecordCount.record(batch.recordCount); + + // per-topic bytes send rate + String topicByteRateName = "topic." + topic + ".bytes"; + Sensor topicByteRate = Utils.notNull(this.metrics.getSensor(topicByteRateName)); + topicByteRate.record(batch.records.sizeInBytes()); + + // per-topic compression rate + String topicCompressionRateName = "topic." + topic + ".compression-rate"; + Sensor topicCompressionRate = Utils.notNull(this.metrics.getSensor(topicCompressionRateName)); + topicCompressionRate.record(batch.records.compressionRate()); + + // global metrics + this.batchSizeSensor.record(batch.records.sizeInBytes(), now); + this.queueTimeSensor.record(batch.drainedMs - batch.createdMs, now); + this.compressionRateSensor.record(batch.records.compressionRate()); + this.maxRecordSizeSensor.record(batch.maxRecordSize, now); + records += batch.recordCount; } + this.recordsPerRequestSensor.record(records, now); } } diff --git a/clients/src/main/java/org/apache/kafka/common/Cluster.java b/clients/src/main/java/org/apache/kafka/common/Cluster.java index d3299b944062d..d7ccbcd91e657 100644 --- a/clients/src/main/java/org/apache/kafka/common/Cluster.java +++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java @@ -26,6 +26,7 @@ public final class Cluster { private final Map partitionsByTopicPartition; private final Map> partitionsByTopic; private final Map> partitionsByNode; + private final Map nodesById; /** * Create a new cluster with the given nodes and partitions @@ -37,6 +38,10 @@ public Cluster(Collection nodes, Collection partitions) { List copy = new ArrayList(nodes); Collections.shuffle(copy); this.nodes = Collections.unmodifiableList(copy); + + this.nodesById = new HashMap(); + for(Node node: nodes) + this.nodesById.put(node.id(), node); // index the partitions by topic/partition for quick lookup this.partitionsByTopicPartition = new HashMap(partitions.size()); @@ -97,6 +102,15 @@ public static Cluster bootstrap(List addresses) { public List nodes() { return this.nodes; } + + /** + * Get the node by the node id (or null if no such node exists) + * @param id The id of the node + * @return The node, or null if no such node exists + */ + public Node nodeById(int id) { + return this.nodesById.get(id); + } /** * Get the current leader for the given topic-partition diff --git a/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java index b15aa2c3ef2d7..28562f9019e1a 100644 --- a/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java +++ b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java @@ -69,10 +69,10 @@ public Node[] inSyncReplicas() { @Override public String toString() { - return String.format("Partition(topic = %s, partition = %d, leader = %d, replicas = %s, isr = %s", + return String.format("Partition(topic = %s, partition = %d, leader = %s, replicas = %s, isr = %s", topic, partition, - leader.id(), + leader == null? "none" : leader.id(), fmtNodeIds(replicas), fmtNodeIds(inSyncReplicas)); } diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java index 98cb79b701918..38ce10b312573 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java @@ -21,6 +21,8 @@ import java.util.Map; import java.util.Set; +import org.apache.kafka.common.utils.Utils; + /** * This class is used for specifying the set of expected configurations, their type, their defaults, their * documentation, and any special validation logic used for checking the correctness of the values the user provides. @@ -292,39 +294,23 @@ private ValidString(List validStrings) { this.validStrings = validStrings; } - public static ValidString in(List validStrings) { - return new ValidString(validStrings); + public static ValidString in(String... validStrings) { + return new ValidString(Arrays.asList(validStrings)); } @Override public void ensureValid(String name, Object o) { - String s = (String) o; - if (!validStrings.contains(s)) { - throw new ConfigException(name,o,"String must be one of:" +join(validStrings)); + throw new ConfigException(name,o,"String must be one of: " + Utils.join(validStrings, ", ")); } } public String toString() { - return "[" + join(validStrings) + "]"; + return "[" + Utils.join(validStrings, ", ") + "]"; } - private String join(List list) - { - StringBuilder sb = new StringBuilder(); - boolean first = true; - for (String item : list) - { - if (first) - first = false; - else - sb.append(","); - sb.append(item); - } - return sb.toString(); - } } private static class ConfigKey { diff --git a/clients/src/main/java/org/apache/kafka/common/errors/ApiException.java b/clients/src/main/java/org/apache/kafka/common/errors/ApiException.java index 7c948b166a8ac..a566b9006a63c 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/ApiException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/ApiException.java @@ -22,7 +22,7 @@ * Any API exception that is part of the public protocol and should be a subclass of this class and be part of this * package. */ -public abstract class ApiException extends KafkaException { +public class ApiException extends KafkaException { private static final long serialVersionUID = 1L; diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selectable.java b/clients/src/main/java/org/apache/kafka/common/network/Selectable.java index b68bbf00ab8eb..b5f8d83e89f90 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Selectable.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Selectable.java @@ -1,18 +1,14 @@ /** - * 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. + * 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.network; @@ -51,13 +47,17 @@ public interface Selectable { public void close(); /** - * Initiate any sends provided, and make progress on any other I/O operations in-flight (connections, - * disconnections, existing sends, and receives) + * Queue the given request for sending in the subsequent {@poll(long)} calls + * @param send The request to send + */ + public void send(NetworkSend send); + + /** + * Do I/O. Reads, writes, connection establishment, etc. * @param timeout The amount of time to block if there is nothing to do - * @param sends The new sends to initiate * @throws IOException */ - public void poll(long timeout, List sends) throws IOException; + public void poll(long timeout) throws IOException; /** * The list of sends that completed on the last {@link #poll(long, List) poll()} call. @@ -81,4 +81,26 @@ public interface Selectable { */ public List connected(); + /** + * Disable reads from the given connection + * @param id The id for the connection + */ + public void mute(int id); + + /** + * Re-enable reads from the given connection + * @param id The id for the connection + */ + public void unmute(int id); + + /** + * Disable reads from all connections + */ + public void muteAll(); + + /** + * Re-enable reads from all connections + */ + public void unmuteAll(); + } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java index 74d695ba39de4..e18a769a4b300 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java @@ -12,6 +12,7 @@ */ package org.apache.kafka.common.network; +import java.io.EOFException; import java.io.IOException; import java.net.InetAddress; import java.net.InetSocketAddress; @@ -81,6 +82,7 @@ public class Selector implements Selectable { private final List completedReceives; private final List disconnected; private final List connected; + private final List failedSends; private final Time time; private final SelectorMetrics sensors; private final String metricGrpPrefix; @@ -103,6 +105,7 @@ public Selector(Metrics metrics, Time time , String metricGrpPrefix , Map(); this.connected = new ArrayList(); this.disconnected = new ArrayList(); + this.failedSends = new ArrayList(); this.sensors = new SelectorMetrics(metrics); } @@ -178,11 +181,27 @@ public void close() { } } + /** + * Queue the given request for sending in the subsequent {@poll(long)} calls + * @param send The request to send + */ + public void send(NetworkSend send) { + SelectionKey key = keyForId(send.destination()); + Transmissions transmissions = transmissions(key); + if (transmissions.hasSend()) + throw new IllegalStateException("Attempt to begin a send operation with prior send operation still in progress."); + transmissions.send = send; + try { + key.interestOps(key.interestOps() | SelectionKey.OP_WRITE); + } catch (CancelledKeyException e) { + close(key); + this.failedSends.add(send.destination()); + } + } + /** * Do whatever I/O can be done on each connection without blocking. This includes completing connections, completing * disconnections, initiating new sends, or making progress on in-progress sends or receives. - *

- * The provided network sends will be started. * * When this call is completed the user can check for completed sends, receives, connections or disconnects using * {@link #completedSends()}, {@link #completedReceives()}, {@link #connected()}, {@link #disconnected()}. These @@ -190,29 +209,13 @@ public void close() { * completed I/O. * * @param timeout The amount of time to wait, in milliseconds. If negative, wait indefinitely. - * @param sends The list of new sends to begin - * * @throws IllegalStateException If a send is given for which we have no existing connection or for which there is * already an in-progress send */ @Override - public void poll(long timeout, List sends) throws IOException { + public void poll(long timeout) throws IOException { clear(); - /* register for write interest on any new sends */ - for (NetworkSend send : sends) { - SelectionKey key = keyForId(send.destination()); - Transmissions transmissions = transmissions(key); - if (transmissions.hasSend()) - throw new IllegalStateException("Attempt to begin a send operation with prior send operation still in progress."); - transmissions.send = send; - try { - key.interestOps(key.interestOps() | SelectionKey.OP_WRITE); - } catch (CancelledKeyException e) { - close(key); - } - } - /* check ready keys */ long startSelect = time.nanoseconds(); int readyKeys = select(timeout); @@ -266,21 +269,34 @@ public void poll(long timeout, List sends) throws IOException { } /* cancel any defunct sockets */ - if (!key.isValid()) + if (!key.isValid()) { close(key); + this.disconnected.add(transmissions.id); + } } catch (IOException e) { - InetAddress remoteAddress = null; - Socket socket = channel.socket(); - if (socket != null) - remoteAddress = socket.getInetAddress(); - log.warn("Error in I/O with {}", remoteAddress , e); + String desc = socketDescription(channel); + if(e instanceof EOFException) + log.info("Connection {} disconnected", desc); + else + log.warn("Error in I/O with connection to {}", desc, e); close(key); + this.disconnected.add(transmissions.id); } } } long endIo = time.nanoseconds(); this.sensors.ioTime.record(endIo - endSelect, time.milliseconds()); } + + private String socketDescription(SocketChannel channel) { + Socket socket = channel.socket(); + if(socket == null) + return "[unconnected socket]"; + else if(socket.getInetAddress() != null) + return socket.getInetAddress().toString(); + else + return socket.getLocalAddress().toString(); + } @Override public List completedSends() { @@ -302,6 +318,36 @@ public List connected() { return this.connected; } + @Override + public void mute(int id) { + mute(this.keyForId(id)); + } + + private void mute(SelectionKey key) { + key.interestOps(key.interestOps() & ~SelectionKey.OP_READ); + } + + @Override + public void unmute(int id) { + unmute(this.keyForId(id)); + } + + private void unmute(SelectionKey key) { + key.interestOps(key.interestOps() | SelectionKey.OP_READ); + } + + @Override + public void muteAll() { + for (SelectionKey key : this.keys.values()) + mute(key); + } + + @Override + public void unmuteAll() { + for (SelectionKey key : this.keys.values()) + unmute(key); + } + /** * Clear the results from the prior poll */ @@ -310,6 +356,8 @@ private void clear() { this.completedReceives.clear(); this.connected.clear(); this.disconnected.clear(); + this.disconnected.addAll(this.failedSends); + this.failedSends.clear(); } /** @@ -335,7 +383,6 @@ private void close(SelectionKey key) { SocketChannel channel = channel(key); Transmissions trans = transmissions(key); if (trans != null) { - this.disconnected.add(trans.id); this.keys.remove(trans.id); trans.clearReceive(); trans.clearSend(); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index 3316b6a109831..a8deac4ce5149 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -19,36 +19,62 @@ import java.util.HashMap; import java.util.Map; -import org.apache.kafka.common.errors.*; - +import org.apache.kafka.common.errors.ApiException; +import org.apache.kafka.common.errors.CorruptRecordException; +import org.apache.kafka.common.errors.InvalidTopicException; +import org.apache.kafka.common.errors.LeaderNotAvailableException; +import org.apache.kafka.common.errors.NetworkException; +import org.apache.kafka.common.errors.NotEnoughReplicasAfterAppendException; +import org.apache.kafka.common.errors.NotEnoughReplicasException; +import org.apache.kafka.common.errors.NotLeaderForPartitionException; +import org.apache.kafka.common.errors.OffsetMetadataTooLarge; +import org.apache.kafka.common.errors.OffsetOutOfRangeException; +import org.apache.kafka.common.errors.RecordBatchTooLargeException; +import org.apache.kafka.common.errors.RecordTooLargeException; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.errors.UnknownServerException; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; /** * This class contains all the client-server errors--those errors that must be sent from the server to the client. These * are thus part of the protocol. The names can be changed but the error code cannot. - * + * * Do not add exceptions that occur only on the client or only on the server here. */ public enum Errors { UNKNOWN(-1, new UnknownServerException("The server experienced an unexpected error when processing the request")), NONE(0, null), - OFFSET_OUT_OF_RANGE(1, new OffsetOutOfRangeException("The requested offset is not within the range of offsets maintained by the server.")), - CORRUPT_MESSAGE(2, new CorruptRecordException("The message contents does not match the message CRC or the message is otherwise corrupt.")), - UNKNOWN_TOPIC_OR_PARTITION(3, new UnknownTopicOrPartitionException("This server does not host this topic-partition.")), + OFFSET_OUT_OF_RANGE(1, + new OffsetOutOfRangeException("The requested offset is not within the range of offsets maintained by the server.")), + CORRUPT_MESSAGE(2, + new CorruptRecordException("The message contents does not match the message CRC or the message is otherwise corrupt.")), + UNKNOWN_TOPIC_OR_PARTITION(3, + new UnknownTopicOrPartitionException("This server does not host this topic-partition.")), // TODO: errorCode 4 for InvalidFetchSize - LEADER_NOT_AVAILABLE(5, new LeaderNotAvailableException("There is no leader for this topic-partition as we are in the middle of a leadership election.")), - NOT_LEADER_FOR_PARTITION(6, new NotLeaderForPartitionException("This server is not the leader for that topic-partition.")), + LEADER_NOT_AVAILABLE(5, + new LeaderNotAvailableException("There is no leader for this topic-partition as we are in the middle of a leadership election.")), + NOT_LEADER_FOR_PARTITION(6, + new NotLeaderForPartitionException("This server is not the leader for that topic-partition.")), REQUEST_TIMED_OUT(7, new TimeoutException("The request timed out.")), - // TODO: errorCode 8, 9, 11 - MESSAGE_TOO_LARGE(10, new RecordTooLargeException("The request included a message larger than the max message size the server will accept.")), + MESSAGE_TOO_LARGE(10, + new RecordTooLargeException("The request included a message larger than the max message size the server will accept.")), OFFSET_METADATA_TOO_LARGE(12, new OffsetMetadataTooLarge("The metadata field of the offset request was too large.")), NETWORK_EXCEPTION(13, new NetworkException("The server disconnected before a response was received.")), - // TODO: errorCode 14, 15, 16 - INVALID_TOPIC_EXCEPTION(17, new InvalidTopicException("The request attempted to perform an operation on an invalid topic.")), - RECORD_LIST_TOO_LARGE(18, new RecordBatchTooLargeException("The request included message batch larger than the configured segment size on the server.")), - NOT_ENOUGH_REPLICAS(19, new NotEnoughReplicasException("Messages are rejected since there are fewer in-sync replicas than required.")), - NOT_ENOUGH_REPLICAS_AFTER_APPEND(20, new NotEnoughReplicasAfterAppendException("Messages are written to the log, but to fewer in-sync replicas than required.")); + OFFSET_LOAD_IN_PROGRESS(14, new ApiException("The coordinator is loading offsets and can't process requests.")), + CONSUMER_COORDINATOR_NOT_AVAILABLE(15, new ApiException("The coordinator is not available.")), + NOT_COORDINATOR_FOR_CONSUMER(16, new ApiException("This is not the correct co-ordinator for this consumer.")), + INVALID_TOPIC_EXCEPTION(17, + new InvalidTopicException("The request attempted to perform an operation on an invalid topic.")), + RECORD_LIST_TOO_LARGE(18, + new RecordBatchTooLargeException("The request included message batch larger than the configured segment size on the server.")), + NOT_ENOUGH_REPLICAS(19, + new NotEnoughReplicasException("Messages are rejected since there are fewer in-sync replicas than required.")), + NOT_ENOUGH_REPLICAS_AFTER_APPEND(20, + new NotEnoughReplicasAfterAppendException("Messages are written to the log, but to fewer in-sync replicas than required.")); + private static Map, Errors> classToError = new HashMap, Errors>(); private static Map codeToError = new HashMap(); + static { for (Errors error : Errors.values()) { codeToError.put(error.code(), error); @@ -84,8 +110,9 @@ public short code() { * Throw the exception corresponding to this error if there is one */ public void maybeThrow() { - if (exception != null) + if (exception != null) { throw this.exception; + } } /** diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java index 121e880a941fc..ee1f78f06c19a 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java @@ -1,18 +1,14 @@ /** - * 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. + * 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.protocol.types; @@ -124,14 +120,6 @@ public Long getLong(String name) { return (Long) get(name); } - public ByteBuffer getBytes(Field field) { - return (ByteBuffer) get(field); - } - - public ByteBuffer getBytes(String name) { - return (ByteBuffer) get(name); - } - public Object[] getArray(Field field) { return (Object[]) get(field); } @@ -148,6 +136,14 @@ public String getString(String name) { return (String) get(name); } + public ByteBuffer getBytes(Field field) { + return (ByteBuffer) get(field); + } + + public ByteBuffer getBytes(String name) { + return (ByteBuffer) get(name); + } + /** * Set the given field to the specified value * @@ -175,9 +171,9 @@ public Struct set(String name, Object value) { } /** - * Create a struct for the schema of a container type (struct or array). - * Note that for array type, this method assumes that the type is an array of schema and creates a struct - * of that schema. Arrays of other types can't be instantiated with this method. + * Create a struct for the schema of a container type (struct or array). Note that for array type, this method + * assumes that the type is an array of schema and creates a struct of that schema. Arrays of other types can't be + * instantiated with this method. * * @param field The field to create an instance of * @return The struct diff --git a/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java b/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java index e4d688cbe0c61..2e54b560ed98b 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java +++ b/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java @@ -41,4 +41,8 @@ public Record record() { public String toString() { return "LogEntry(" + offset + ", " + record + ")"; } + + public int size() { + return record.size() + Records.LOG_OVERHEAD; + } } diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index 040e5b91005ed..cc4084faec15e 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -55,7 +55,7 @@ public static MemoryRecords emptyRecords(ByteBuffer buffer, CompressionType type return emptyRecords(buffer, type, buffer.capacity()); } - public static MemoryRecords iterableRecords(ByteBuffer buffer) { + public static MemoryRecords readableRecords(ByteBuffer buffer) { return new MemoryRecords(buffer, CompressionType.NONE, false, buffer.capacity()); } @@ -94,22 +94,21 @@ public void append(long offset, byte[] key, byte[] value) { * Note that the return value is based on the estimate of the bytes written to the compressor, which may not be * accurate if compression is really used. When this happens, the following append may cause dynamic buffer * re-allocation in the underlying byte buffer stream. - * + * * Also note that besides the records' capacity, there is also a size limit for the batch. This size limit may be * smaller than the capacity (e.g. when appending a single message whose size is larger than the batch size, the - * capacity will be the message size, but the size limit will still be the batch size), and when the records' size has - * exceed this limit we also mark this record as full. + * capacity will be the message size, but the size limit will still be the batch size), and when the records' size + * has exceed this limit we also mark this record as full. */ public boolean hasRoomFor(byte[] key, byte[] value) { - return this.writable && - this.capacity >= this.compressor.estimatedBytesWritten() + Records.LOG_OVERHEAD + Record.recordSize(key, value) && - this.sizeLimit >= this.compressor.estimatedBytesWritten(); + return this.writable && this.capacity >= this.compressor.estimatedBytesWritten() + Records.LOG_OVERHEAD + + Record.recordSize(key, value) && + this.sizeLimit >= this.compressor.estimatedBytesWritten(); } public boolean isFull() { - return !this.writable || - this.capacity <= this.compressor.estimatedBytesWritten() || - this.sizeLimit <= this.compressor.estimatedBytesWritten(); + return !this.writable || this.capacity <= this.compressor.estimatedBytesWritten() || + this.sizeLimit <= this.compressor.estimatedBytesWritten(); } /** @@ -132,7 +131,7 @@ public int writeTo(GatheringByteChannel channel) throws IOException { public int sizeInBytes() { return compressor.buffer().position(); } - + /** * The compression rate of this record set */ @@ -162,6 +161,25 @@ public Iterator iterator() { ByteBuffer copy = (ByteBuffer) this.buffer.duplicate().flip(); return new RecordsIterator(copy, CompressionType.NONE, false); } + + @Override + public String toString() { + Iterator iter = iterator(); + StringBuilder builder = new StringBuilder(); + builder.append('['); + while(iter.hasNext()) { + LogEntry entry = iter.next(); + builder.append('('); + builder.append("offset="); + builder.append(entry.offset()); + builder.append(","); + builder.append("record="); + builder.append(entry.record()); + builder.append(")"); + } + builder.append(']'); + return builder.toString(); + } public static class RecordsIterator extends AbstractIterator { private final ByteBuffer buffer; @@ -174,7 +192,7 @@ public RecordsIterator(ByteBuffer buffer, CompressionType type, boolean shallow) this.type = type; this.buffer = buffer; this.shallow = shallow; - stream = Compressor.wrapForInput(new ByteBufferInputStream(this.buffer), type); + this.stream = Compressor.wrapForInput(new ByteBufferInputStream(this.buffer), type); } /* @@ -199,7 +217,10 @@ protected LogEntry makeNext() { ByteBuffer rec; if (type == CompressionType.NONE) { rec = buffer.slice(); - buffer.position(buffer.position() + size); + int newPos = buffer.position() + size; + if(newPos > buffer.limit()) + return allDone(); + buffer.position(newPos); rec.limit(size); } else { byte[] recordBuffer = new byte[size]; @@ -207,7 +228,6 @@ protected LogEntry makeNext() { rec = ByteBuffer.wrap(recordBuffer); } LogEntry entry = new LogEntry(offset, new Record(rec)); - entry.record().ensureValid(); // decide whether to go shallow or deep iteration if it is compressed CompressionType compression = entry.record().compressionType(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java index 99b52c23d639d..4c99d4a3c423e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java @@ -20,13 +20,14 @@ import java.nio.ByteBuffer; public class ConsumerMetadataRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.CONSUMER_METADATA.id); - private static String GROUP_ID_KEY_NAME = "group_id"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.CONSUMER_METADATA.id); + private static final String GROUP_ID_KEY_NAME = "group_id"; private final String groupId; public ConsumerMetadataRequest(String groupId) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); struct.set(GROUP_ID_KEY_NAME, groupId); this.groupId = groupId; @@ -42,6 +43,6 @@ public String groupId() { } public static ConsumerMetadataRequest parse(ByteBuffer buffer) { - return new ConsumerMetadataRequest(((Struct) curSchema.read(buffer))); + return new ConsumerMetadataRequest(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java index 8b8f591c4b280..173333be3afc4 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java @@ -21,20 +21,21 @@ import java.nio.ByteBuffer; public class ConsumerMetadataResponse extends AbstractRequestResponse { - private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.CONSUMER_METADATA.id); - private static String ERROR_CODE_KEY_NAME = "error_code"; - private static String COORDINATOR_KEY_NAME = "coordinator"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.CONSUMER_METADATA.id); + private static final String ERROR_CODE_KEY_NAME = "error_code"; + private static final String COORDINATOR_KEY_NAME = "coordinator"; // coordinator level field names - private static String NODE_ID_KEY_NAME = "node_id"; - private static String HOST_KEY_NAME = "host"; - private static String PORT_KEY_NAME = "port"; + private static final String NODE_ID_KEY_NAME = "node_id"; + private static final String HOST_KEY_NAME = "host"; + private static final String PORT_KEY_NAME = "port"; private final short errorCode; private final Node node; public ConsumerMetadataResponse(short errorCode, Node node) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); struct.set(ERROR_CODE_KEY_NAME, errorCode); Struct coordinator = struct.instance(COORDINATOR_KEY_NAME); coordinator.set(NODE_ID_KEY_NAME, node.id()); @@ -64,6 +65,6 @@ public Node node() { } public static ConsumerMetadataResponse parse(ByteBuffer buffer) { - return new ConsumerMetadataResponse(((Struct) curSchema.read(buffer))); + return new ConsumerMetadataResponse(((Struct) CURRENT_SCHEMA.read(buffer))); } } \ No newline at end of file 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 2fc471f64f435..2529a09767934 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 @@ -1,21 +1,23 @@ /** - * 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. + * 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.requests; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ProtoUtils; @@ -23,27 +25,23 @@ import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.CollectionUtils; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - public class FetchRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.FETCH.id); - private static String REPLICA_ID_KEY_NAME = "replica_id"; - private static String MAX_WAIT_KEY_NAME = "max_wait_time"; - private static String MIN_BYTES_KEY_NAME = "min_bytes"; - private static String TOPICS_KEY_NAME = "topics"; + + public static final int CONSUMER_REPLICA_ID = -1; + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.FETCH.id); + 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 TOPICS_KEY_NAME = "topics"; // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partitions"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partitions"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String FETCH_OFFSET_KEY_NAME = "fetch_offset"; - private static String MAX_BYTES_KEY_NAME = "max_bytes"; + 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"; private final int replicaId; private final int maxWait; @@ -60,15 +58,25 @@ public PartitionData(long offset, int maxBytes) { } } + /** + * Create a non-replica fetch request + */ + public FetchRequest(int maxWait, int minBytes, Map fetchData) { + this(CONSUMER_REPLICA_ID, maxWait, minBytes, fetchData); + } + + /** + * Create a replica fetch request + */ public FetchRequest(int replicaId, int maxWait, int minBytes, Map fetchData) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> topicsData = CollectionUtils.groupDataByTopic(fetchData); struct.set(REPLICA_ID_KEY_NAME, replicaId); struct.set(MAX_WAIT_KEY_NAME, maxWait); struct.set(MIN_BYTES_KEY_NAME, minBytes); List topicArray = new ArrayList(); - for (Map.Entry> topicEntry: topicsData.entrySet()) { + for (Map.Entry> topicEntry : topicsData.entrySet()) { Struct topicData = struct.instance(TOPICS_KEY_NAME); topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); List partitionArray = new ArrayList(); @@ -127,6 +135,6 @@ public Map fetchData() { } public static FetchRequest parse(ByteBuffer buffer) { - return new FetchRequest(((Struct) curSchema.read(buffer))); + return new FetchRequest(((Struct) CURRENT_SCHEMA.read(buffer))); } } 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 f719010119951..c1e5f44beae05 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 @@ -30,18 +30,19 @@ import java.util.Map; public class FetchResponse extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.FETCH.id); - private static String RESPONSES_KEY_NAME = "responses"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.FETCH.id); + private static final String RESPONSES_KEY_NAME = "responses"; // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partition_responses"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partition_responses"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String ERROR_CODE_KEY_NAME = "error_code"; - private static String HIGH_WATERMARK_KEY_NAME = "high_watermark"; - private static String RECORD_SET_KEY_NAME = "record_set"; + private static final String PARTITION_KEY_NAME = "partition"; + private static final String ERROR_CODE_KEY_NAME = "error_code"; + private static final String HIGH_WATERMARK_KEY_NAME = "high_watermark"; + private static final String RECORD_SET_KEY_NAME = "record_set"; private final Map responseData; @@ -58,7 +59,7 @@ public PartitionData(short errorCode, long highWatermark, ByteBuffer recordSet) } public FetchResponse(Map responseData) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> topicsData = CollectionUtils.groupDataByTopic(responseData); List topicArray = new ArrayList(); @@ -105,6 +106,6 @@ public Map responseData() { } public static FetchResponse parse(ByteBuffer buffer) { - return new FetchResponse(((Struct) curSchema.read(buffer))); + return new FetchResponse(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java index 9512db2365d8d..cfdb5de523cc3 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java @@ -20,17 +20,18 @@ import java.nio.ByteBuffer; public class HeartbeatRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.HEARTBEAT.id); - private static String GROUP_ID_KEY_NAME = "group_id"; - private static String GROUP_GENERATION_ID_KEY_NAME = "group_generation_id"; - private static String CONSUMER_ID_KEY_NAME = "consumer_id"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.HEARTBEAT.id); + private static final String GROUP_ID_KEY_NAME = "group_id"; + private static final String GROUP_GENERATION_ID_KEY_NAME = "group_generation_id"; + private static final String CONSUMER_ID_KEY_NAME = "consumer_id"; private final String groupId; private final int groupGenerationId; private final String consumerId; public HeartbeatRequest(String groupId, int groupGenerationId, String consumerId) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); struct.set(GROUP_ID_KEY_NAME, groupId); struct.set(GROUP_GENERATION_ID_KEY_NAME, groupGenerationId); struct.set(CONSUMER_ID_KEY_NAME, consumerId); @@ -59,6 +60,6 @@ public String consumerId() { } public static HeartbeatRequest parse(ByteBuffer buffer) { - return new HeartbeatRequest(((Struct) curSchema.read(buffer))); + return new HeartbeatRequest(((Struct) CURRENT_SCHEMA.read(buffer))); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java index 8997ffc44c18b..ea964f7c7bd8a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java @@ -20,12 +20,13 @@ import java.nio.ByteBuffer; public class HeartbeatResponse extends AbstractRequestResponse { - private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.HEARTBEAT.id); - private static String ERROR_CODE_KEY_NAME = "error_code"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.HEARTBEAT.id); + private static final String ERROR_CODE_KEY_NAME = "error_code"; private final short errorCode; public HeartbeatResponse(short errorCode) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); struct.set(ERROR_CODE_KEY_NAME, errorCode); this.errorCode = errorCode; } @@ -40,6 +41,6 @@ public short errorCode() { } public static HeartbeatResponse parse(ByteBuffer buffer) { - return new HeartbeatResponse(((Struct) curSchema.read(buffer))); + return new HeartbeatResponse(((Struct) CURRENT_SCHEMA.read(buffer))); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java index d6e91f3931798..a1d48c9ab1867 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java @@ -22,12 +22,13 @@ import java.util.List; public class JoinGroupRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.JOIN_GROUP.id); - private static String GROUP_ID_KEY_NAME = "group_id"; - private static String SESSION_TIMEOUT_KEY_NAME = "session_timeout"; - private static String TOPICS_KEY_NAME = "topics"; - private static String CONSUMER_ID_KEY_NAME = "consumer_id"; - private static String STRATEGY_KEY_NAME = "partition_assignment_strategy"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.JOIN_GROUP.id); + private static final String GROUP_ID_KEY_NAME = "group_id"; + private static final String SESSION_TIMEOUT_KEY_NAME = "session_timeout"; + private static final String TOPICS_KEY_NAME = "topics"; + private static final String CONSUMER_ID_KEY_NAME = "consumer_id"; + private static final String STRATEGY_KEY_NAME = "partition_assignment_strategy"; private final String groupId; private final int sessionTimeout; @@ -36,7 +37,7 @@ public class JoinGroupRequest extends AbstractRequestResponse { private final String strategy; public JoinGroupRequest(String groupId, int sessionTimeout, List topics, String consumerId, String strategy) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); struct.set(GROUP_ID_KEY_NAME, groupId); struct.set(SESSION_TIMEOUT_KEY_NAME, sessionTimeout); struct.set(TOPICS_KEY_NAME, topics.toArray()); @@ -82,6 +83,6 @@ public String strategy() { } public static JoinGroupRequest parse(ByteBuffer buffer) { - return new JoinGroupRequest(((Struct) curSchema.read(buffer))); + return new JoinGroupRequest(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java index efe89796a7bd0..1e9f3494d6dff 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java @@ -23,16 +23,17 @@ import java.util.*; public class JoinGroupResponse extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.JOIN_GROUP.id); - private static String ERROR_CODE_KEY_NAME = "error_code"; - private static String GENERATION_ID_KEY_NAME = "group_generation_id"; - private static String CONSUMER_ID_KEY_NAME = "consumer_id"; - private static String ASSIGNED_PARTITIONS_KEY_NAME = "assigned_partitions"; - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partitions"; - - public static int UNKNOWN_GENERATION_ID = -1; - public static String UNKNOWN_CONSUMER_ID = ""; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.JOIN_GROUP.id); + private static final String ERROR_CODE_KEY_NAME = "error_code"; + private static final String GENERATION_ID_KEY_NAME = "group_generation_id"; + private static final String CONSUMER_ID_KEY_NAME = "consumer_id"; + private static final String ASSIGNED_PARTITIONS_KEY_NAME = "assigned_partitions"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partitions"; + + public static final int UNKNOWN_GENERATION_ID = -1; + public static final String UNKNOWN_CONSUMER_ID = ""; private final short errorCode; private final int generationId; @@ -40,7 +41,7 @@ public class JoinGroupResponse extends AbstractRequestResponse { private final List assignedPartitions; public JoinGroupResponse(short errorCode, int generationId, String consumerId, List assignedPartitions) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> partitionsByTopic = CollectionUtils.groupDataByTopic(assignedPartitions); @@ -97,6 +98,6 @@ public List assignedPartitions() { } public static JoinGroupResponse parse(ByteBuffer buffer) { - return new JoinGroupResponse(((Struct) curSchema.read(buffer))); + return new JoinGroupResponse(((Struct) CURRENT_SCHEMA.read(buffer))); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java index 99364c1ca464f..05c5fed6dd7dc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java @@ -30,18 +30,19 @@ import java.util.Map; public class ListOffsetRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.LIST_OFFSETS.id); - private static String REPLICA_ID_KEY_NAME = "replica_id"; - private static String TOPICS_KEY_NAME = "topics"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.LIST_OFFSETS.id); + private static final String REPLICA_ID_KEY_NAME = "replica_id"; + private static final String TOPICS_KEY_NAME = "topics"; // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partitions"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partitions"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String TIMESTAMP_KEY_NAME = "timestamp"; - private static String MAX_NUM_OFFSETS_KEY_NAME = "max_num_offsets"; + private static final String PARTITION_KEY_NAME = "partition"; + private static final String TIMESTAMP_KEY_NAME = "timestamp"; + private static final String MAX_NUM_OFFSETS_KEY_NAME = "max_num_offsets"; private final int replicaId; private final Map offsetData; @@ -55,9 +56,13 @@ public PartitionData(long timestamp, int maxNumOffsets) { this.maxNumOffsets = maxNumOffsets; } } + + public ListOffsetRequest(Map offsetData) { + this(-1, offsetData); + } public ListOffsetRequest(int replicaId, Map offsetData) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> topicsData = CollectionUtils.groupDataByTopic(offsetData); struct.set(REPLICA_ID_KEY_NAME, replicaId); @@ -109,6 +114,6 @@ public Map offsetData() { } public static ListOffsetRequest parse(ByteBuffer buffer) { - return new ListOffsetRequest(((Struct) curSchema.read(buffer))); + return new ListOffsetRequest(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java index ac239712f1184..b2e473e85b2b1 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java @@ -30,17 +30,18 @@ import java.util.Map; public class ListOffsetResponse extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.LIST_OFFSETS.id); - private static String RESPONSES_KEY_NAME = "responses"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.LIST_OFFSETS.id); + private static final String RESPONSES_KEY_NAME = "responses"; // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partition_responses"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partition_responses"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String ERROR_CODE_KEY_NAME = "error_code"; - private static String OFFSETS_KEY_NAME = "offsets"; + private static final String PARTITION_KEY_NAME = "partition"; + private static final String ERROR_CODE_KEY_NAME = "error_code"; + private static final String OFFSETS_KEY_NAME = "offsets"; private final Map responseData; @@ -55,7 +56,7 @@ public PartitionData(short errorCode, List offsets) { } public ListOffsetResponse(Map responseData) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> topicsData = CollectionUtils.groupDataByTopic(responseData); List topicArray = new ArrayList(); @@ -103,6 +104,6 @@ public Map responseData() { } public static ListOffsetResponse parse(ByteBuffer buffer) { - return new ListOffsetResponse(((Struct) curSchema.read(buffer))); + return new ListOffsetResponse(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java index b22ca1dce65f6..0186783e1abd0 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java @@ -22,13 +22,14 @@ import org.apache.kafka.common.protocol.types.Struct; public class MetadataRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id); - private static String TOPICS_KEY_NAME = "topics"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id); + private static final String TOPICS_KEY_NAME = "topics"; private final List topics; public MetadataRequest(List topics) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); struct.set(TOPICS_KEY_NAME, topics.toArray()); this.topics = topics; } @@ -47,6 +48,6 @@ public List topics() { } public static MetadataRequest parse(ByteBuffer buffer) { - return new MetadataRequest(((Struct) curSchema.read(buffer))); + return new MetadataRequest(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java index d97962d384017..13daf599635e8 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -28,32 +28,33 @@ import org.apache.kafka.common.protocol.types.Struct; public class MetadataResponse extends AbstractRequestResponse { - private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.METADATA.id); - private static String BROKERS_KEY_NAME = "brokers"; - private static String TOPIC_METATDATA_KEY_NAME = "topic_metadata"; + + private static Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.METADATA.id); + private static final String BROKERS_KEY_NAME = "brokers"; + private static final String TOPIC_METATDATA_KEY_NAME = "topic_metadata"; // broker level field names - private static String NODE_ID_KEY_NAME = "node_id"; - private static String HOST_KEY_NAME = "host"; - private static String PORT_KEY_NAME = "port"; + private static final String NODE_ID_KEY_NAME = "node_id"; + private static final String HOST_KEY_NAME = "host"; + private static final String PORT_KEY_NAME = "port"; // topic level field names - private static String TOPIC_ERROR_CODE_KEY_NAME = "topic_error_code"; - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITION_METADATA_KEY_NAME = "partition_metadata"; + private static final String TOPIC_ERROR_CODE_KEY_NAME = "topic_error_code"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITION_METADATA_KEY_NAME = "partition_metadata"; // partition level field names - private static String PARTITION_ERROR_CODE_KEY_NAME = "partition_error_code"; - private static String PARTITION_KEY_NAME = "partition_id"; - private static String LEADER_KEY_NAME = "leader"; - private static String REPLICAS_KEY_NAME = "replicas"; - private static String ISR_KEY_NAME = "isr"; + private static final String PARTITION_ERROR_CODE_KEY_NAME = "partition_error_code"; + private static final String PARTITION_KEY_NAME = "partition_id"; + private static final String LEADER_KEY_NAME = "leader"; + private static final String REPLICAS_KEY_NAME = "replicas"; + private static final String ISR_KEY_NAME = "isr"; private final Cluster cluster; private final Map errors; public MetadataResponse(Cluster cluster) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); List brokerArray = new ArrayList(); for (Node node: cluster.nodes()) { @@ -147,6 +148,6 @@ public Cluster cluster() { } public static MetadataResponse parse(ByteBuffer buffer) { - return new MetadataResponse(((Struct) curSchema.read(buffer))); + return new MetadataResponse(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java index 3ee5cbad55ce8..4fb48c8f3592d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java @@ -3,15 +3,21 @@ * 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.requests; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ProtoUtils; @@ -19,31 +25,26 @@ import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.CollectionUtils; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - /** * This wrapper supports both v0 and v1 of OffsetCommitRequest. */ public class OffsetCommitRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_COMMIT.id); - private static String GROUP_ID_KEY_NAME = "group_id"; - private static String GENERATION_ID_KEY_NAME = "group_generation_id"; - private static String CONSUMER_ID_KEY_NAME = "consumer_id"; - private static String TOPICS_KEY_NAME = "topics"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_COMMIT.id); + private static final String GROUP_ID_KEY_NAME = "group_id"; + private static final String GENERATION_ID_KEY_NAME = "group_generation_id"; + private static final String CONSUMER_ID_KEY_NAME = "consumer_id"; + private static final String TOPICS_KEY_NAME = "topics"; // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partitions"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partitions"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String COMMIT_OFFSET_KEY_NAME = "offset"; - private static String TIMESTAMP_KEY_NAME = "timestamp"; - private static String METADATA_KEY_NAME = "metadata"; + private static final String PARTITION_KEY_NAME = "partition"; + private static final String COMMIT_OFFSET_KEY_NAME = "offset"; + private static final String TIMESTAMP_KEY_NAME = "timestamp"; + private static final String METADATA_KEY_NAME = "metadata"; public static final int DEFAULT_GENERATION_ID = -1; public static final String DEFAULT_CONSUMER_ID = ""; @@ -88,7 +89,7 @@ public OffsetCommitRequest(String groupId, Map of * @param offsetData */ public OffsetCommitRequest(String groupId, int generationId, String consumerId, Map offsetData) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); initCommonFields(groupId, offsetData); struct.set(GENERATION_ID_KEY_NAME, generationId); @@ -104,7 +105,7 @@ private void initCommonFields(String groupId, Map struct.set(GROUP_ID_KEY_NAME, groupId); List topicArray = new ArrayList(); - for (Map.Entry> topicEntry: topicsData.entrySet()) { + for (Map.Entry> topicEntry : topicsData.entrySet()) { Struct topicData = struct.instance(TOPICS_KEY_NAME); topicData.set(TOPIC_KEY_NAME, topicEntry.getKey()); List partitionArray = new ArrayList(); @@ -175,6 +176,6 @@ public static OffsetCommitRequest parse(ByteBuffer buffer, int versionId) { } public static OffsetCommitRequest parse(ByteBuffer buffer) { - return new OffsetCommitRequest(((Struct) curSchema.read(buffer))); + return new OffsetCommitRequest(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java index 711232ac61378..2ab1dc6c516a1 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java @@ -26,21 +26,22 @@ import java.util.Map; public class OffsetCommitResponse extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_COMMIT.id); - private static String RESPONSES_KEY_NAME = "responses"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_COMMIT.id); + private static final String RESPONSES_KEY_NAME = "responses"; // topic level fields - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partition_responses"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partition_responses"; // partition level fields - private static String PARTITION_KEY_NAME = "partition"; - private static String ERROR_CODE_KEY_NAME = "error_code"; + private static final String PARTITION_KEY_NAME = "partition"; + private static final String ERROR_CODE_KEY_NAME = "error_code"; private final Map responseData; public OffsetCommitResponse(Map responseData) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> topicsData = CollectionUtils.groupDataByTopic(responseData); @@ -82,6 +83,6 @@ public Map responseData() { } public static OffsetCommitResponse parse(ByteBuffer buffer) { - return new OffsetCommitResponse(((Struct) curSchema.read(buffer))); + return new OffsetCommitResponse(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java index 90d5135b97a44..333483fd8b50e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java @@ -28,16 +28,17 @@ * This wrapper supports both v0 and v1 of OffsetFetchRequest. */ public class OffsetFetchRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_FETCH.id); - private static String GROUP_ID_KEY_NAME = "group_id"; - private static String TOPICS_KEY_NAME = "topics"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_FETCH.id); + private static final String GROUP_ID_KEY_NAME = "group_id"; + private static final String TOPICS_KEY_NAME = "topics"; // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partitions"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partitions"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; + private static final String PARTITION_KEY_NAME = "partition"; public static final int DEFAULT_GENERATION_ID = -1; public static final String DEFAULT_CONSUMER_ID = ""; @@ -46,7 +47,7 @@ public class OffsetFetchRequest extends AbstractRequestResponse { private final List partitions; public OffsetFetchRequest(String groupId, List partitions) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> topicsData = CollectionUtils.groupDataByTopic(partitions); @@ -93,6 +94,6 @@ public List partitions() { } public static OffsetFetchRequest parse(ByteBuffer buffer) { - return new OffsetFetchRequest(((Struct) curSchema.read(buffer))); + return new OffsetFetchRequest(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java index 6b7c269ad7679..04c88c0c057b9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java @@ -3,43 +3,45 @@ * 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.requests; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.utils.CollectionUtils; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - public class OffsetFetchResponse extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_FETCH.id); - private static String RESPONSES_KEY_NAME = "responses"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_FETCH.id); + private static final String RESPONSES_KEY_NAME = "responses"; // topic level fields - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITIONS_KEY_NAME = "partition_responses"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partition_responses"; // partition level fields - private static String PARTITION_KEY_NAME = "partition"; - private static String COMMIT_OFFSET_KEY_NAME = "offset"; - private static String METADATA_KEY_NAME = "metadata"; - private static String ERROR_CODE_KEY_NAME = "error_code"; + private static final String PARTITION_KEY_NAME = "partition"; + private static final String COMMIT_OFFSET_KEY_NAME = "offset"; + private static final String METADATA_KEY_NAME = "metadata"; + private static final String ERROR_CODE_KEY_NAME = "error_code"; - private final Map responseData; + private final Map responseData; public static final class PartitionData { public final long offset; @@ -51,15 +53,19 @@ public PartitionData(long offset, String metadata, short errorCode) { this.metadata = metadata; this.errorCode = errorCode; } + + public boolean hasError() { + return this.errorCode != Errors.NONE.code(); + } } public OffsetFetchResponse(Map responseData) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> topicsData = CollectionUtils.groupDataByTopic(responseData); List topicArray = new ArrayList(); - for (Map.Entry> entries: topicsData.entrySet()) { + for (Map.Entry> entries : topicsData.entrySet()) { Struct topicData = struct.instance(RESPONSES_KEY_NAME); topicData.set(TOPIC_KEY_NAME, entries.getKey()); List partitionArray = new ArrayList(); @@ -102,6 +108,6 @@ public Map responseData() { } public static OffsetFetchResponse parse(ByteBuffer buffer) { - return new OffsetFetchResponse(((Struct) curSchema.read(buffer))); + return new OffsetFetchResponse(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 3dbba8a360f11..03a0ab1afbb7d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -27,25 +27,26 @@ import java.util.Map; public class ProduceRequest extends AbstractRequestResponse { - public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.PRODUCE.id); - private static String ACKS_KEY_NAME = "acks"; - private static String TIMEOUT_KEY_NAME = "timeout"; - private static String TOPIC_DATA_KEY_NAME = "topic_data"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.PRODUCE.id); + private static final String ACKS_KEY_NAME = "acks"; + private static final String TIMEOUT_KEY_NAME = "timeout"; + private static final String TOPIC_DATA_KEY_NAME = "topic_data"; // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITION_DATA_KEY_NAME = "data"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITION_DATA_KEY_NAME = "data"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String RECORD_SET_KEY_NAME = "record_set"; + private static final String PARTITION_KEY_NAME = "partition"; + private static final String RECORD_SET_KEY_NAME = "record_set"; private final short acks; private final int timeout; private final Map partitionRecords; public ProduceRequest(short acks, int timeout, Map partitionRecords) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> recordsByTopic = CollectionUtils.groupDataByTopic(partitionRecords); struct.set(ACKS_KEY_NAME, acks); struct.set(TIMEOUT_KEY_NAME, timeout); @@ -100,6 +101,6 @@ public Map partitionRecords() { } public static ProduceRequest parse(ByteBuffer buffer) { - return new ProduceRequest(((Struct) curSchema.read(buffer))); + return new ProduceRequest(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 5220464913e6e..e42d7dbbe7cc5 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -26,22 +26,23 @@ import java.util.Map; public class ProduceResponse extends AbstractRequestResponse { - private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id); - private static String RESPONSES_KEY_NAME = "responses"; + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id); + private static final String RESPONSES_KEY_NAME = "responses"; // topic level field names - private static String TOPIC_KEY_NAME = "topic"; - private static String PARTITION_RESPONSES_KEY_NAME = "partition_responses"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITION_RESPONSES_KEY_NAME = "partition_responses"; // partition level field names - private static String PARTITION_KEY_NAME = "partition"; - private static String ERROR_CODE_KEY_NAME = "error_code"; - private static String BASE_OFFSET_KEY_NAME = "base_offset"; + private static final String PARTITION_KEY_NAME = "partition"; + private static final String ERROR_CODE_KEY_NAME = "error_code"; + private static final String BASE_OFFSET_KEY_NAME = "base_offset"; private final Map responses; public ProduceResponse(Map responses) { - super(new Struct(curSchema)); + super(new Struct(CURRENT_SCHEMA)); Map> responseByTopic = CollectionUtils.groupDataByTopic(responses); List topicDatas = new ArrayList(responseByTopic.size()); for (Map.Entry> entry : responseByTopic.entrySet()) { @@ -107,6 +108,6 @@ public String toString() { } public static ProduceResponse parse(ByteBuffer buffer) { - return new ProduceResponse(((Struct) curSchema.read(buffer))); + return new ProduceResponse(((Struct) CURRENT_SCHEMA.read(buffer))); } } diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java index 3c001d33091c0..13be6a38cb356 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java @@ -29,7 +29,7 @@ public interface Deserializer { * @param isKey whether is for key or value */ public void configure(Map configs, boolean isKey); - + /** * * @param topic topic associated with the data diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index 527dd0f9c47fc..8a305b0fb4656 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -17,6 +17,9 @@ import java.io.OutputStream; import java.io.UnsupportedEncodingException; import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collection; +import java.util.Iterator; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -223,6 +226,18 @@ public static T notNull(T t) { return t; } + /** + * Sleep for a bit + * @param ms The duration of the sleep + */ + public static void sleep(long ms) { + try { + Thread.sleep(ms); + } catch (InterruptedException e) { + // this is okay, we just wake up early + } + } + /** * Instantiate the class */ @@ -313,4 +328,31 @@ public static String formatAddress(String host, Integer port) { ? "[" + host + "]:" + port // IPv6 : host + ":" + port; } + + /** + * Create a string representation of an array joined by the given separator + * @param strs The array of items + * @param seperator The separator + * @return The string representation. + */ + public static String join(T[] strs, String seperator) { + return join(Arrays.asList(strs), seperator); + } + + /** + * Create a string representation of a list joined by the given separator + * @param list The list of items + * @param seperator The separator + * @return The string representation. + */ + public static String join(Collection list, String seperator) { + StringBuilder sb = new StringBuilder(); + Iterator iter = list.iterator(); + while(iter.hasNext()) { + sb.append(iter.next()); + if(iter.hasNext()) + sb.append(seperator); + } + return sb.toString(); + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/MockClient.java b/clients/src/test/java/org/apache/kafka/clients/MockClient.java index 47b5d4ac1f2a5..67bee40abb61f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java @@ -59,13 +59,33 @@ public void disconnect(Integer node) { } @Override - public List poll(List requests, long timeoutMs, long now) { - this.requests.addAll(requests); - List copy = new ArrayList(this.responses); + public void send(ClientRequest request) { + this.requests.add(request); + } + + @Override + public List poll(long timeoutMs, long now) { + for(ClientResponse response: this.responses) + if (response.request().hasCallback()) + response.request().callback().onComplete(response); + List copy = new ArrayList(); this.responses.clear(); return copy; } + @Override + public List completeAll(int node, long now) { + return completeAll(now); + } + + @Override + public List completeAll(long now) { + List responses = poll(0, now); + if (requests.size() > 0) + throw new IllegalStateException("Requests without responses remain."); + return responses; + } + public Queue requests() { return this.requests; } @@ -80,6 +100,11 @@ public int inFlightRequestCount() { return requests.size(); } + @Override + public int inFlightRequestCount(int nodeId) { + return requests.size(); + } + @Override public RequestHeader nextRequestHeader(ApiKeys key) { return new RequestHeader(key.id, "mock", correlation++); diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index 1a55242e9399f..5debcd6cc6ce3 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -5,7 +5,6 @@ import static org.junit.Assert.assertTrue; import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -46,14 +45,13 @@ public void setup() { @Test public void testReadyAndDisconnect() { - List reqs = new ArrayList(); assertFalse("Client begins unready as it has no connection.", client.ready(node, time.milliseconds())); assertEquals("The connection is established as a side-effect of the readiness check", 1, selector.connected().size()); - client.poll(reqs, 1, time.milliseconds()); + client.poll(1, time.milliseconds()); selector.clear(); assertTrue("Now the client is ready", client.ready(node, time.milliseconds())); selector.disconnect(node.id()); - client.poll(reqs, 1, time.milliseconds()); + client.poll(1, time.milliseconds()); selector.clear(); assertFalse("After we forced the disconnection the client is no longer ready.", client.ready(node, time.milliseconds())); assertTrue("Metadata should get updated.", metadata.timeToNextUpdate(time.milliseconds()) == 0); @@ -65,7 +63,8 @@ public void testSendToUnreadyNode() { client.nextRequestHeader(ApiKeys.METADATA), new MetadataRequest(Arrays.asList("test")).toStruct()); ClientRequest request = new ClientRequest(time.milliseconds(), false, send, null); - client.poll(Arrays.asList(request), 1, time.milliseconds()); + client.send(request); + client.poll(1, time.milliseconds()); } @Test @@ -73,9 +72,11 @@ public void testSimpleRequestResponse() { ProduceRequest produceRequest = new ProduceRequest((short) 1, 1000, Collections.emptyMap()); RequestHeader reqHeader = client.nextRequestHeader(ApiKeys.PRODUCE); RequestSend send = new RequestSend(node.id(), reqHeader, produceRequest.toStruct()); - ClientRequest request = new ClientRequest(time.milliseconds(), true, send, null); + TestCallbackHandler handler = new TestCallbackHandler(); + ClientRequest request = new ClientRequest(time.milliseconds(), true, send, handler); awaitReady(client, node); - client.poll(Arrays.asList(request), 1, time.milliseconds()); + client.send(request); + client.poll(1, time.milliseconds()); assertEquals(1, client.inFlightRequestCount()); ResponseHeader respHeader = new ResponseHeader(reqHeader.correlationId()); Struct resp = new Struct(ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id)); @@ -86,16 +87,26 @@ public void testSimpleRequestResponse() { resp.writeTo(buffer); buffer.flip(); selector.completeReceive(new NetworkReceive(node.id(), buffer)); - List responses = client.poll(new ArrayList(), 1, time.milliseconds()); + List responses = client.poll(1, time.milliseconds()); assertEquals(1, responses.size()); - ClientResponse response = responses.get(0); - assertTrue("Should have a response body.", response.hasResponse()); - assertEquals("Should be correlated to the original request", request, response.request()); + assertTrue("The handler should have executed.", handler.executed); + assertTrue("Should have a response body.", handler.response.hasResponse()); + assertEquals("Should be correlated to the original request", request, handler.response.request()); } private void awaitReady(NetworkClient client, Node node) { while (!client.ready(node, time.milliseconds())) - client.poll(new ArrayList(), 1, time.milliseconds()); + client.poll(1, time.milliseconds()); + } + + private static class TestCallbackHandler implements RequestCompletionHandler { + public boolean executed = false; + public ClientResponse response; + + public void onComplete(ClientResponse response) { + this.executed = true; + this.response = response; + } } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerExampleTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerExampleTest.java deleted file mode 100644 index 29ad25e90606f..0000000000000 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerExampleTest.java +++ /dev/null @@ -1,297 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. -*/ -package org.apache.kafka.clients.consumer; - -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Properties; - -import org.apache.kafka.common.TopicPartition; -import org.junit.Test; - -/** - * TODO: Clean this after the consumer implementation is complete. Until then, it is useful to write some sample test code using the new APIs - * - */ -public class ConsumerExampleTest { - /** - * This example demonstrates how to use the consumer to leverage Kafka's group management functionality for automatic consumer load - * balancing and failure detection. This example assumes that the offsets are stored in Kafka and are automatically committed periodically, - * as controlled by the auto.commit.interval.ms config - */ -// @Test -// public void testConsumerGroupManagementWithAutoOffsetCommits() { -// Properties props = new Properties(); -// props.put("metadata.broker.list", "localhost:9092"); -// props.put("group.id", "test"); -// props.put("session.timeout.ms", "1000"); -// props.put("auto.commit.enable", "true"); -// props.put("auto.commit.interval.ms", "10000"); -// KafkaConsumer consumer = new KafkaConsumer(props); -// // subscribe to some topics -// consumer.subscribe("foo", "bar"); -// boolean isRunning = true; -// while(isRunning) { -// Map records = consumer.poll(100); -// process(records); -// } -// consumer.close(); -// } - - /** - * This example demonstrates how to use the consumer to leverage Kafka's group management functionality for automatic consumer load - * balancing and failure detection. This example assumes that the offsets are stored in Kafka and are manually committed using the - * commit() API. This example also demonstrates rewinding the consumer's offsets if processing of consumed messages fails. - */ -// @Test -// public void testConsumerGroupManagementWithManualOffsetCommit() { -// Properties props = new Properties(); -// props.put("metadata.broker.list", "localhost:9092"); -// props.put("group.id", "test"); -// props.put("session.timeout.ms", "1000"); -// props.put("auto.commit.enable", "false"); -// KafkaConsumer consumer = new KafkaConsumer(props); -// // subscribe to some topics -// consumer.subscribe("foo", "bar"); -// int commitInterval = 100; -// int numRecords = 0; -// boolean isRunning = true; -// Map consumedOffsets = new HashMap(); -// while(isRunning) { -// Map records = consumer.poll(100); -// try { -// Map lastConsumedOffsets = process(records); -// consumedOffsets.putAll(lastConsumedOffsets); -// numRecords += records.size(); -// // commit offsets for all partitions of topics foo, bar synchronously, owned by this consumer instance -// if(numRecords % commitInterval == 0) -// consumer.commit(true); -// } catch(Exception e) { -// // rewind consumer's offsets for failed partitions -// List failedPartitions = getFailedPartitions(); -// Map offsetsToRewindTo = new HashMap(); -// for(TopicPartition failedPartition : failedPartitions) { -// // rewind to the last consumed offset for the failed partition. Since process() failed for this partition, the consumed offset -// // should still be pointing to the last successfully processed offset and hence is the right offset to rewind consumption to. -// offsetsToRewindTo.put(failedPartition, consumedOffsets.get(failedPartition)); -// } -// // seek to new offsets only for partitions that failed the last process() -// consumer.seek(offsetsToRewindTo); -// } -// } -// consumer.close(); -// } - - private List getFailedPartitions() { return null; } - - /** - * This example demonstrates the consumer can be used to leverage Kafka's group management functionality along with custom offset storage. - * In this example, the assumption made is that the user chooses to store the consumer offsets outside Kafka. This requires the user to - * plugin logic for retrieving the offsets from a custom store and provide the offsets to the consumer in the ConsumerRebalanceCallback - * callback. The onPartitionsAssigned callback is invoked after the consumer is assigned a new set of partitions on rebalance and - * before the consumption restarts post rebalance. This is the right place to supply offsets from a custom store to the consumer. - */ -// @Test -// public void testConsumerRebalanceWithCustomOffsetStore() { -// Properties props = new Properties(); -// props.put("metadata.broker.list", "localhost:9092"); -// props.put("group.id", "test"); -// props.put("session.timeout.ms", "1000"); -// props.put("auto.commit.enable", "true"); -// props.put("auto.commit.interval.ms", "10000"); -// KafkaConsumer consumer = new KafkaConsumer(props, -// new ConsumerRebalanceCallback() { -// public void onPartitionsAssigned(Consumer consumer, Collection partitions) { -// Map lastCommittedOffsets = getLastCommittedOffsetsFromCustomStore(partitions); -// consumer.seek(lastCommittedOffsets); -// } -// public void onPartitionsRevoked(Consumer consumer, Collection partitions) { -// Map offsets = getLastConsumedOffsets(partitions); // implemented by the user -// commitOffsetsToCustomStore(offsets); // implemented by the user -// } -// private Map getLastCommittedOffsetsFromCustomStore(Collection partitions) { -// return null; -// } -// private Map getLastConsumedOffsets(Collection partitions) { return null; } -// private void commitOffsetsToCustomStore(Map offsets) {} -// }); -// // subscribe to topics -// consumer.subscribe("foo", "bar"); -// int commitInterval = 100; -// int numRecords = 0; -// boolean isRunning = true; -// while(isRunning) { -// Map records = consumer.poll(100); -// Map consumedOffsets = process(records); -// numRecords += records.size(); -// // commit offsets for all partitions of topics foo, bar synchronously, owned by this consumer instance -// if(numRecords % commitInterval == 0) -// commitOffsetsToCustomStore(consumedOffsets); -// } -// consumer.close(); -// } - - /** - * This example demonstrates how the consumer can be used to leverage Kafka's group management functionality along with Kafka based offset storage. - * In this example, the assumption made is that the user chooses to use Kafka based offset management. - */ -// @Test -// public void testConsumerRewindWithGroupManagementAndKafkaOffsetStorage() { -// Properties props = new Properties(); -// props.put("metadata.broker.list", "localhost:9092"); -// props.put("group.id", "test"); -// props.put("session.timeout.ms", "1000"); -// props.put("auto.commit.enable", "false"); -// KafkaConsumer consumer = new KafkaConsumer(props, -// new ConsumerRebalanceCallback() { -// boolean rewindOffsets = true; -// public void onPartitionsAssigned(Consumer consumer, Collection partitions) { -// if(rewindOffsets) { -// Map latestCommittedOffsets = consumer.committed(null); -// Map newOffsets = rewindOffsets(latestCommittedOffsets, 100); -// consumer.seek(newOffsets); -// } -// } -// public void onPartitionsRevoked(Consumer consumer, Collection partitions) { -// consumer.commit(true); -// } -// // this API rewinds every partition back by numberOfMessagesToRewindBackTo messages -// private Map rewindOffsets(Map currentOffsets, -// long numberOfMessagesToRewindBackTo) { -// Map newOffsets = new HashMap(); -// for(Map.Entry offset : currentOffsets.entrySet()) { -// newOffsets.put(offset.getKey(), offset.getValue() - numberOfMessagesToRewindBackTo); -// } -// return newOffsets; -// } -// }); -// // subscribe to topics -// consumer.subscribe("foo", "bar"); -// int commitInterval = 100; -// int numRecords = 0; -// boolean isRunning = true; -// while(isRunning) { -// Map records = consumer.poll(100); -// Map consumedOffsets = process(records); -// numRecords += records.size(); -// // commit offsets for all partitions of topics foo, bar synchronously, owned by this consumer instance -// if(numRecords % commitInterval == 0) -// commitOffsetsToCustomStore(consumedOffsets); -// } -// consumer.close(); -// } - - /** - * This example demonstrates how the consumer can be used to subscribe to specific partitions of certain topics and consume upto the latest - * available message for each of those partitions before shutting down. When used to subscribe to specific partitions, the user foregoes - * the group management functionality and instead relies on manually configuring the consumer instances to subscribe to a set of partitions. - * This example assumes that the user chooses to use Kafka based offset storage. The user still has to specify a group.id to use Kafka - * based offset management. However, session.timeout.ms is not required since the Kafka consumer only does failure detection with group - * management. - */ -// @Test -// public void testConsumerWithKafkaBasedOffsetManagement() { -// Properties props = new Properties(); -// props.put("metadata.broker.list", "localhost:9092"); -// props.put("group.id", "test"); -// props.put("auto.commit.enable", "true"); -// props.put("auto.commit.interval.ms", "10000"); -// KafkaConsumer consumer = new KafkaConsumer(props); -// // subscribe to some partitions of topic foo -// TopicPartition partition0 = new TopicPartition("foo", 0); -// TopicPartition partition1 = new TopicPartition("foo", 1); -// TopicPartition[] partitions = new TopicPartition[2]; -// partitions[0] = partition0; -// partitions[1] = partition1; -// consumer.subscribe(partitions); -// // find the last committed offsets for partitions 0,1 of topic foo -// Map lastCommittedOffsets = consumer.committed(null); -// // seek to the last committed offsets to avoid duplicates -// consumer.seek(lastCommittedOffsets); -// // find the offsets of the latest available messages to know where to stop consumption -// Map latestAvailableOffsets = consumer.offsetsBeforeTime(-2, null); -// boolean isRunning = true; -// while(isRunning) { -// Map records = consumer.poll(100); -// Map consumedOffsets = process(records); -// for(TopicPartition partition : partitions) { -// if(consumedOffsets.get(partition) >= latestAvailableOffsets.get(partition)) -// isRunning = false; -// else -// isRunning = true; -// } -// } -// consumer.close(); -// } - - /** - * This example demonstrates how the consumer can be used to subscribe to specific partitions of certain topics and consume upto the latest - * available message for each of those partitions before shutting down. When used to subscribe to specific partitions, the user foregoes - * the group management functionality and instead relies on manually configuring the consumer instances to subscribe to a set of partitions. - * This example assumes that the user chooses to use custom offset storage. - */ - @Test - public void testConsumerWithCustomOffsetManagement() { -// Properties props = new Properties(); -// props.put("metadata.broker.list", "localhost:9092"); -// KafkaConsumer consumer = new KafkaConsumer(props); -// // subscribe to some partitions of topic foo -// TopicPartition partition0 = new TopicPartition("foo", 0); -// TopicPartition partition1 = new TopicPartition("foo", 1); -// TopicPartition[] partitions = new TopicPartition[2]; -// partitions[0] = partition0; -// partitions[1] = partition1; -// consumer.subscribe(partitions); -// Map lastCommittedOffsets = getLastCommittedOffsetsFromCustomStore(); -// // seek to the last committed offsets to avoid duplicates -// consumer.seek(lastCommittedOffsets); -// // find the offsets of the latest available messages to know where to stop consumption -// Map latestAvailableOffsets = consumer.offsetsBeforeTime(-2, null); -// boolean isRunning = true; -// while(isRunning) { -// Map records = consumer.poll(100); -// Map consumedOffsets = process(records); -// // commit offsets for partitions 0,1 for topic foo to custom store -// commitOffsetsToCustomStore(consumedOffsets); -// for(TopicPartition partition : partitions) { -// if(consumedOffsets.get(partition) >= latestAvailableOffsets.get(partition)) -// isRunning = false; -// else -// isRunning = true; -// } -// } -// consumer.close(); - } - - private Map getLastCommittedOffsetsFromCustomStore() { return null; } - private void commitOffsetsToCustomStore(Map consumedOffsets) {} - private Map process(Map records) { - Map processedOffsets = new HashMap(); - for(Entry recordMetadata : records.entrySet()) { - List recordsPerTopic = recordMetadata.getValue().records(); - for(int i = 0;i < recordsPerTopic.size();i++) { - ConsumerRecord record = recordsPerTopic.get(i); - // process record - try { - processedOffsets.put(record.topicAndPartition(), record.offset()); - } catch (Exception e) { - e.printStackTrace(); - } - } - } - return processedOffsets; - } -} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java new file mode 100644 index 0000000000000..e51d2dfdadee3 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java @@ -0,0 +1,32 @@ +package org.apache.kafka.clients.consumer; + +import static org.junit.Assert.*; + +import java.util.Iterator; + +import org.apache.kafka.common.TopicPartition; +import org.junit.Test; + +public class MockConsumerTest { + + private MockConsumer consumer = new MockConsumer(); + + @Test + public void testSimpleMock() { + consumer.subscribe("topic"); + assertEquals(0, consumer.poll(1000).count()); + ConsumerRecord rec1 = new ConsumerRecord("test", 0, 0, "key1", "value1"); + ConsumerRecord rec2 = new ConsumerRecord("test", 0, 1, "key2", "value2"); + consumer.addRecord(rec1); + consumer.addRecord(rec2); + ConsumerRecords recs = consumer.poll(1); + Iterator> iter = recs.iterator(); + assertEquals(rec1, iter.next()); + assertEquals(rec2, iter.next()); + assertFalse(iter.hasNext()); + assertEquals(1L, consumer.position(new TopicPartition("test", 0))); + consumer.commit(CommitType.SYNC); + assertEquals(1L, consumer.committed(new TopicPartition("test", 0))); + } + +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java new file mode 100644 index 0000000000000..864f1c736458d --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java @@ -0,0 +1,61 @@ +package org.apache.kafka.clients.consumer.internals; + +import static org.junit.Assert.*; +import static java.util.Arrays.asList; + +import java.util.Collections; + +import org.apache.kafka.common.TopicPartition; +import org.junit.Test; + +public class SubscriptionStateTest { + + private final SubscriptionState state = new SubscriptionState(); + private final TopicPartition tp0 = new TopicPartition("test", 0); + private final TopicPartition tp1 = new TopicPartition("test", 1); + + @Test + public void partitionSubscription() { + state.subscribe(tp0); + assertEquals(Collections.singleton(tp0), state.assignedPartitions()); + state.committed(tp0, 1); + state.fetched(tp0, 1); + state.consumed(tp0, 1); + assertAllPositions(tp0, 1L); + state.unsubscribe(tp0); + assertTrue(state.assignedPartitions().isEmpty()); + assertAllPositions(tp0, null); + } + + public void topicSubscription() { + state.subscribe("test"); + assertEquals(1, state.subscribedTopics().size()); + assertTrue(state.assignedPartitions().isEmpty()); + assertTrue(state.partitionsAutoAssigned()); + state.changePartitionAssignment(asList(tp0)); + state.committed(tp0, 1); + state.fetched(tp0, 1); + state.consumed(tp0, 1); + assertAllPositions(tp0, 1L); + state.changePartitionAssignment(asList(tp1)); + assertAllPositions(tp0, null); + assertEquals(Collections.singleton(tp1), state.assignedPartitions()); + } + + @Test(expected = IllegalArgumentException.class) + public void cantChangeFetchPositionForNonAssignedPartition() { + state.fetched(tp0, 1); + } + + @Test(expected = IllegalArgumentException.class) + public void cantChangeConsumedPositionForNonAssignedPartition() { + state.consumed(tp0, 1); + } + + public void assertAllPositions(TopicPartition tp, Long offset) { + assertEquals(offset, state.committed(tp)); + assertEquals(offset, state.fetched(tp)); + assertEquals(offset, state.consumed(tp)); + } + +} diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java index 12368038e1381..77b23e7aa8246 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java @@ -43,7 +43,7 @@ public class BufferPoolTest { */ @Test public void testSimple() throws Exception { - int totalMemory = 64 * 1024; + long totalMemory = 64 * 1024; int size = 1024; BufferPool pool = new BufferPool(totalMemory, size, false, metrics, time, metricGroup, metricTags); ByteBuffer buffer = pool.allocate(size); @@ -100,7 +100,7 @@ public void testDelayedAllocation() throws Exception { ByteBuffer buffer = pool.allocate(1024); CountDownLatch doDealloc = asyncDeallocate(pool, buffer); CountDownLatch allocation = asyncAllocate(pool, 5 * 1024); - assertEquals("Allocation shouldn't have happened yet, waiting on memory.", 1, allocation.getCount()); + assertEquals("Allocation shouldn't have happened yet, waiting on memory.", 1L, allocation.getCount()); doDealloc.countDown(); // return the memory allocation.await(); } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java index 3676b05eb8b83..d3377ef8dfffd 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java @@ -38,7 +38,7 @@ public void testAutoCompleteMock() throws Exception { Future metadata = producer.send(record); assertTrue("Send should be immediately complete", metadata.isDone()); assertFalse("Send should be successful", isError(metadata)); - assertEquals("Offset should be 0", 0, metadata.get().offset()); + assertEquals("Offset should be 0", 0L, metadata.get().offset()); assertEquals(topic, metadata.get().topic()); assertEquals("We should have the record in our history", asList(record), producer.history()); producer.clear(); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java index 1d077fd4c56bf..82d8083b6072b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java @@ -1,30 +1,23 @@ /** - * 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. + * 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.clients.producer; import static java.util.Arrays.asList; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertTrue; import java.util.List; - -import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.internals.Partitioner; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; @@ -34,7 +27,6 @@ public class PartitionerTest { private byte[] key = "key".getBytes(); - private byte[] value = "value".getBytes(); private Partitioner partitioner = new Partitioner(); private Node node0 = new Node(0, "localhost", 99); private Node node1 = new Node(1, "localhost", 100); @@ -48,33 +40,28 @@ public class PartitionerTest { @Test public void testUserSuppliedPartitioning() { - assertEquals("If the user supplies a partition we should use it.", - 0, - partitioner.partition(new ProducerRecord("test", 0, key, value), cluster)); + assertEquals("If the user supplies a partition we should use it.", 0, partitioner.partition("test", key, 0, cluster)); } @Test public void testKeyPartitionIsStable() { - int partition = partitioner.partition(new ProducerRecord("test", key, value), cluster); - assertEquals("Same key should yield same partition", - partition, - partitioner.partition(new ProducerRecord("test", key, "value2".getBytes()), cluster)); + int partition = partitioner.partition("test", key, null, cluster); + assertEquals("Same key should yield same partition", partition, partitioner.partition("test", key, null, cluster)); } @Test public void testRoundRobinIsStable() { - int startPart = partitioner.partition(new ProducerRecord("test", value), cluster); + int startPart = partitioner.partition("test", null, null, cluster); for (int i = 1; i <= 100; i++) { - int partition = partitioner.partition(new ProducerRecord("test", value), cluster); - assertEquals("Should yield a different partition each call with round-robin partitioner", - partition, (startPart + i) % 2); - } + int partition = partitioner.partition("test", null, null, cluster); + assertEquals("Should yield a different partition each call with round-robin partitioner", partition, (startPart + i) % 2); + } } @Test public void testRoundRobinWithDownNode() { for (int i = 0; i < partitions.size(); i++) { - int part = partitioner.partition(new ProducerRecord("test", value), cluster); + int part = partitioner.partition("test", null, null, cluster); assertTrue("We should never choose a leader-less node in round robin", part >= 0 && part < 2); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java index 66cbdf5babed3..888b9295d47fe 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java @@ -72,14 +72,14 @@ public void setup() { @Test public void testSimple() throws Exception { - int offset = 0; + long offset = 0; Future future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null).future; sender.run(time.milliseconds()); // connect sender.run(time.milliseconds()); // send produce request assertEquals("We should have a single produce request in flight.", 1, client.inFlightRequestCount()); client.respond(produceResponse(tp.topic(), tp.partition(), offset, Errors.NONE.code())); sender.run(time.milliseconds()); - assertEquals("All requests completed.", offset, client.inFlightRequestCount()); + assertEquals("All requests completed.", offset, (long) client.inFlightRequestCount()); sender.run(time.milliseconds()); assertTrue("Request should be completed", future.isDone()); assertEquals(offset, future.get().offset()); @@ -110,7 +110,7 @@ public void testRetries() throws Exception { sender.run(time.milliseconds()); // reconnect sender.run(time.milliseconds()); // resend assertEquals(1, client.inFlightRequestCount()); - int offset = 0; + long offset = 0; client.respond(produceResponse(tp.topic(), tp.partition(), offset, Errors.NONE.code())); sender.run(time.milliseconds()); assertTrue("Request should have retried and completed", future.isDone()); diff --git a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java index 3c442a27a7ba3..16d3fedef8cca 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java @@ -110,18 +110,18 @@ private void testBadInputs(Type type, Object... values) { @Test(expected = ConfigException.class) public void testInvalidDefaultRange() { - ConfigDef def = new ConfigDef().define("name", Type.INT, -1, Range.between(0,10), Importance.HIGH, "docs"); + new ConfigDef().define("name", Type.INT, -1, Range.between(0,10), Importance.HIGH, "docs"); } @Test(expected = ConfigException.class) public void testInvalidDefaultString() { - ConfigDef def = new ConfigDef().define("name", Type.STRING, "bad", ValidString.in(Arrays.asList("valid", "values")), Importance.HIGH, "docs"); + new ConfigDef().define("name", Type.STRING, "bad", ValidString.in("valid", "values"), Importance.HIGH, "docs"); } @Test public void testValidators() { testValidators(Type.INT, Range.between(0,10), 5, new Object[]{1, 5, 9}, new Object[]{-1, 11}); - testValidators(Type.STRING, ValidString.in(Arrays.asList("good", "values", "default")), "default", + testValidators(Type.STRING, ValidString.in("good", "values", "default"), "default", new Object[]{"good", "values", "default"}, new Object[]{"bad", "inputs"}); } diff --git a/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java index 74c19573a29b0..a14659a713795 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java @@ -12,7 +12,6 @@ */ package org.apache.kafka.common.network; -import static java.util.Arrays.asList; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -74,7 +73,7 @@ public void testServerDisconnect() throws Exception { // disconnect this.server.closeConnections(); while (!selector.disconnected().contains(node)) - selector.poll(1000L, EMPTY); + selector.poll(1000L); // reconnect and do another request blockingConnect(node); @@ -89,7 +88,8 @@ public void testClientDisconnect() throws Exception { int node = 0; blockingConnect(node); selector.disconnect(node); - selector.poll(10, asList(createSend(node, "hello1"))); + selector.send(createSend(node, "hello1")); + selector.poll(10); assertEquals("Request should not have succeeded", 0, selector.completedSends().size()); assertEquals("There should be a disconnect", 1, selector.disconnected().size()); assertTrue("The disconnect should be from our node", selector.disconnected().contains(node)); @@ -104,7 +104,9 @@ public void testClientDisconnect() throws Exception { public void testCantSendWithInProgress() throws Exception { int node = 0; blockingConnect(node); - selector.poll(1000L, asList(createSend(node, "test1"), createSend(node, "test2"))); + selector.send(createSend(node, "test1")); + selector.send(createSend(node, "test2")); + selector.poll(1000L); } /** @@ -112,7 +114,8 @@ public void testCantSendWithInProgress() throws Exception { */ @Test(expected = IllegalStateException.class) public void testCantSendWithoutConnecting() throws Exception { - selector.poll(1000L, asList(createSend(0, "test"))); + selector.send(createSend(0, "test")); + selector.poll(1000L); } /** @@ -131,7 +134,7 @@ public void testConnectionRefused() throws Exception { int node = 0; selector.connect(node, new InetSocketAddress("localhost", TestUtils.choosePort()), BUFFER_SIZE, BUFFER_SIZE); while (selector.disconnected().contains(node)) - selector.poll(1000L, EMPTY); + selector.poll(1000L); } /** @@ -152,14 +155,13 @@ public void testNormalOperation() throws Exception { int[] requests = new int[conns]; int[] responses = new int[conns]; int responseCount = 0; - List sends = new ArrayList(); for (int i = 0; i < conns; i++) - sends.add(createSend(i, i + "-" + 0)); + selector.send(createSend(i, i + "-" + 0)); // loop until we complete all requests while (responseCount < conns * reqs) { // do the i/o - selector.poll(0L, sends); + selector.poll(0L); assertEquals("No disconnects should have occurred.", 0, selector.disconnected().size()); @@ -175,12 +177,11 @@ public void testNormalOperation() throws Exception { } // prepare new sends for the next round - sends.clear(); for (NetworkSend send : selector.completedSends()) { int dest = send.destination(); requests[dest]++; if (requests[dest] < reqs) - sends.add(createSend(dest, dest + "-" + requests[dest])); + selector.send(createSend(dest, dest + "-" + requests[dest])); } } } @@ -212,10 +213,34 @@ public void testExistingConnectionId() throws IOException { blockingConnect(0); } + @Test + public void testMute() throws Exception { + blockingConnect(0); + blockingConnect(1); + + selector.send(createSend(0, "hello")); + selector.send(createSend(1, "hi")); + + selector.mute(1); + + while (selector.completedReceives().isEmpty()) + selector.poll(5); + assertEquals("We should have only one response", 1, selector.completedReceives().size()); + assertEquals("The response should not be from the muted node", 0, selector.completedReceives().get(0).source()); + + selector.unmute(1); + do { + selector.poll(5); + } while (selector.completedReceives().isEmpty()); + assertEquals("We should have only one response", 1, selector.completedReceives().size()); + assertEquals("The response should be from the previously muted node", 1, selector.completedReceives().get(0).source()); + } + private String blockingRequest(int node, String s) throws IOException { - selector.poll(1000L, asList(createSend(node, s))); + selector.send(createSend(node, s)); + selector.poll(1000L); while (true) { - selector.poll(1000L, EMPTY); + selector.poll(1000L); for (NetworkReceive receive : selector.completedReceives()) if (receive.source() == node) return asString(receive); @@ -226,7 +251,7 @@ private String blockingRequest(int node, String s) throws IOException { private void blockingConnect(int node) throws IOException { selector.connect(node, new InetSocketAddress("localhost", server.port), BUFFER_SIZE, BUFFER_SIZE); while (!selector.connected().contains(node)) - selector.poll(10000L, EMPTY); + selector.poll(10000L); } private NetworkSend createSend(int node, String s) { diff --git a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java index a39fab532f731..4c2ea34815b63 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java @@ -16,6 +16,9 @@ */ package org.apache.kafka.common.utils; +import java.util.Arrays; +import java.util.Collections; + import org.junit.Test; import static org.apache.kafka.common.utils.Utils.getHost; @@ -48,4 +51,11 @@ public void testFormatAddress() { assertEquals("[::1]:1234", formatAddress("::1", 1234)); assertEquals("[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678", formatAddress("2001:db8:85a3:8d3:1319:8a2e:370:7348", 5678)); } + + @Test + public void testJoin() { + assertEquals("", Utils.join(Collections.emptyList(), ",")); + assertEquals("1", Utils.join(Arrays.asList("1"), ",")); + assertEquals("1,2,3", Utils.join(Arrays.asList(1, 2, 3), ",")); + } } \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/test/MockSelector.java b/clients/src/test/java/org/apache/kafka/test/MockSelector.java index d61de52917331..ea89b06a4c9e5 100644 --- a/clients/src/test/java/org/apache/kafka/test/MockSelector.java +++ b/clients/src/test/java/org/apache/kafka/test/MockSelector.java @@ -1,18 +1,14 @@ /** - * 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. + * 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.test; @@ -26,13 +22,13 @@ import org.apache.kafka.common.network.Selectable; import org.apache.kafka.common.utils.Time; - /** * A fake selector to use for testing */ public class MockSelector implements Selectable { private final Time time; + private final List initiatedSends = new ArrayList(); private final List completedSends = new ArrayList(); private final List completedReceives = new ArrayList(); private final List disconnected = new ArrayList(); @@ -68,8 +64,14 @@ public void clear() { } @Override - public void poll(long timeout, List sends) throws IOException { - this.completedSends.addAll(sends); + public void send(NetworkSend send) { + this.initiatedSends.add(send); + } + + @Override + public void poll(long timeout) throws IOException { + this.completedSends.addAll(this.initiatedSends); + this.initiatedSends.clear(); time.sleep(timeout); } @@ -101,4 +103,20 @@ public List connected() { return connected; } + @Override + public void mute(int id) { + } + + @Override + public void unmute(int id) { + } + + @Override + public void muteAll() { + } + + @Override + public void unmuteAll() { + } + } diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala index 6d00ed090d76c..a3b1b78adb760 100644 --- a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala @@ -63,7 +63,7 @@ case class ConsumerMetadataRequest(group: String, override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { // return ConsumerCoordinatorNotAvailable for all uncaught errors - val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode) + val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, correlationId) requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) } diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala index 84f60178f6eba..24aaf954dc42e 100644 --- a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala +++ b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala @@ -40,7 +40,7 @@ object ConsumerMetadataResponse { } -case class ConsumerMetadataResponse (coordinatorOpt: Option[Broker], errorCode: Short, correlationId: Int = 0) +case class ConsumerMetadataResponse (coordinatorOpt: Option[Broker], errorCode: Short, correlationId: Int) extends RequestOrResponse() { def sizeInBytes = diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index b230e9a1fb1a3..e6ad8be5e33b6 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -90,7 +90,7 @@ class Partition(val topic: String, val checkpoint = replicaManager.highWatermarkCheckpoints(log.dir.getParentFile.getAbsolutePath) val offsetMap = checkpoint.read if (!offsetMap.contains(TopicAndPartition(topic, partitionId))) - warn("No checkpointed highwatermark is found for partition [%s,%d]".format(topic, partitionId)) + info("No checkpointed highwatermark is found for partition [%s,%d]".format(topic, partitionId)) val offset = offsetMap.getOrElse(TopicAndPartition(topic, partitionId), 0L).min(log.logEndOffset) val localReplica = new Replica(replicaId, this, time, offset, Some(log)) addReplicaIfNotExists(localReplica) diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index fbef34cad16af..14b22ab38cc14 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -141,7 +141,7 @@ class RequestSendThread(val controllerId: Int, connectToBroker(toBroker, channel) isSendSuccessful = false // backoff before retrying the connection and send - Utils.swallow(Thread.sleep(300)) + Utils.swallowTrace(Thread.sleep(300)) } } if (receive != null) { diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index 4631bc78106e6..8b67aee3a3776 100644 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -179,12 +179,12 @@ object LogConfig { .define(FileDeleteDelayMsProp, LONG, Defaults.FileDeleteDelayMs, atLeast(0), MEDIUM, FileDeleteDelayMsDoc) .define(MinCleanableDirtyRatioProp, DOUBLE, Defaults.MinCleanableDirtyRatio, between(0, 1), MEDIUM, MinCleanableRatioDoc) - .define(CleanupPolicyProp, STRING, if (Defaults.Compact) Compact else Delete, in(asList(Compact, Delete)), MEDIUM, + .define(CleanupPolicyProp, STRING, if (Defaults.Compact) Compact else Delete, in(Compact, Delete), MEDIUM, CompactDoc) .define(UncleanLeaderElectionEnableProp, BOOLEAN, Defaults.UncleanLeaderElectionEnable, MEDIUM, UncleanLeaderElectionEnableDoc) .define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), MEDIUM, MinInSyncReplicasDoc) - .define(CompressionTypeProp, STRING, Defaults.CompressionType, in(seqAsJavaList(BrokerCompressionCodec.brokerCompressionOptions)), MEDIUM, CompressionTypeDoc) + .define(CompressionTypeProp, STRING, Defaults.CompressionType, in(BrokerCompressionCodec.brokerCompressionOptions:_*), MEDIUM, CompressionTypeDoc) } def configNames() = { diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index ec8d9f7ba4474..48bc4359ff448 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -17,6 +17,12 @@ package kafka.server +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.JoinGroupResponse +import org.apache.kafka.common.requests.HeartbeatResponse +import org.apache.kafka.common.requests.ResponseHeader +import org.apache.kafka.common.protocol.types.Struct + import kafka.api._ import kafka.common._ import kafka.log._ @@ -26,6 +32,9 @@ import kafka.network.RequestChannel.Response import kafka.controller.KafkaController import kafka.utils.{SystemTime, Logging} +import java.nio.ByteBuffer +import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic._ import scala.collection._ import org.I0Itec.zkclient.ZkClient @@ -43,6 +52,7 @@ class KafkaApis(val requestChannel: RequestChannel, this.logIdent = "[KafkaApi-%d] ".format(brokerId) val metadataCache = new MetadataCache + private var consumerGroupGenerationId = 0 /** * Top-level method that handles all requests and multiplexes to the right api @@ -62,6 +72,8 @@ class KafkaApis(val requestChannel: RequestChannel, case RequestKeys.OffsetCommitKey => handleOffsetCommitRequest(request) case RequestKeys.OffsetFetchKey => handleOffsetFetchRequest(request) case RequestKeys.ConsumerMetadataKey => handleConsumerMetadataRequest(request) + case RequestKeys.JoinGroupKey => handleJoinGroupRequest(request) + case RequestKeys.HeartbeatKey => handleHeartbeatRequest(request) case requestId => throw new KafkaException("Unknown api code " + requestId) } } catch { @@ -442,6 +454,23 @@ class KafkaApis(val requestChannel: RequestChannel, requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } + def handleJoinGroupRequest(request: RequestChannel.Request) { + val joinGroupReq = request.requestObj.asInstanceOf[JoinGroupRequestAndHeader] + val topics = JavaConversions.asScalaIterable(joinGroupReq.body.topics()).toSet + val partitions = this.replicaManager.logManager.allLogs.filter(log => topics.contains(log.topicAndPartition.topic)) + val partitionList = partitions.map(_.topicAndPartition).map(tp => new org.apache.kafka.common.TopicPartition(tp.topic, tp.partition)).toBuffer + this.consumerGroupGenerationId += 1 + val response = new JoinGroupResponse(ErrorMapping.NoError, this.consumerGroupGenerationId, joinGroupReq.body.consumerId, JavaConversions.asJavaList(partitionList)) + val send = new BoundedByteBufferSend(new JoinGroupResponseAndHeader(joinGroupReq.correlationId, response)) + requestChannel.sendResponse(new RequestChannel.Response(request, send)) + } + + def handleHeartbeatRequest(request: RequestChannel.Request) { + val hbReq = request.requestObj.asInstanceOf[HeartbeatRequestAndHeader] + val send = new BoundedByteBufferSend(new HeartbeatResponseAndHeader(hbReq.correlationId, new HeartbeatResponse(Errors.NONE.code))) + requestChannel.sendResponse(new RequestChannel.Response(request, send)) + } + def close() { // TODO currently closing the API is an no-op since the API no longer maintain any modules // maybe removing the closing call in the end when KafkaAPI becomes a pure stateless layer diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index e58fbb922e93b..fb948b9ab28c5 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -327,7 +327,7 @@ class ReplicaManager(val config: KafkaConfig, BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(numAppendedMessages) trace("%d bytes written to log %s-%d beginning at offset %d and ending at offset %d" - .format(messages.size, topicAndPartition.topic, topicAndPartition.partition, info.firstOffset, info.lastOffset)) + .format(messages.sizeInBytes, topicAndPartition.topic, topicAndPartition.partition, info.firstOffset, info.lastOffset)) (topicAndPartition, LogAppendResult(info)) } catch { // NOTE: Failed produce requests metric is not incremented for known exceptions diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala index e455cb9a1de22..910691e88ccc6 100644 --- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala @@ -113,8 +113,6 @@ object ConsoleConsumer extends Logging { KafkaMetricsReporter.startReporters(verifiableProps) } - - val consumerProps = if (options.has(consumerConfigOpt)) Utils.loadProps(options.valueOf(consumerConfigOpt)) else diff --git a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala index 093c800ea7f8a..c39c0672ff8d0 100644 --- a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala +++ b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala @@ -17,14 +17,21 @@ package kafka.tools -import java.util.concurrent.CountDownLatch +import scala.collection.JavaConversions._ import java.util.concurrent.atomic.AtomicLong import java.nio.channels.ClosedByInterruptException import org.apache.log4j.Logger +import org.apache.kafka.clients.consumer.KafkaConsumer +import org.apache.kafka.common.record.Record +import org.apache.kafka.common.record.Records +import org.apache.kafka.common.serialization.ByteArrayDeserializer import kafka.message.Message import kafka.utils.{ZkUtils, CommandLineUtils} import java.util.{ Random, Properties } -import kafka.consumer._ +import kafka.consumer.Consumer +import kafka.consumer.ConsumerConnector +import kafka.consumer.KafkaStream +import kafka.consumer.ConsumerTimeoutException import java.text.SimpleDateFormat /** @@ -42,50 +49,98 @@ object ConsumerPerformance { if (!config.hideHeader) { if (!config.showDetailedStats) - println("start.time, end.time, fetch.size, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec") + println("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec") else - println("time, fetch.size, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec") + println("time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec") } - // clean up zookeeper state for this group id for every perf run - ZkUtils.maybeDeletePath(config.consumerConfig.zkConnect, "/consumers/" + config.consumerConfig.groupId) - - val consumerConnector: ConsumerConnector = Consumer.create(config.consumerConfig) - - val topicMessageStreams = consumerConnector.createMessageStreams(Map(config.topic -> config.numThreads)) - var threadList = List[ConsumerPerfThread]() - for ((topic, streamList) <- topicMessageStreams) - for (i <- 0 until streamList.length) - threadList ::= new ConsumerPerfThread(i, "kafka-zk-consumer-" + i, streamList(i), config, - totalMessagesRead, totalBytesRead) - - logger.info("Sleeping for 1 second.") - Thread.sleep(1000) - logger.info("starting threads") - val startMs = System.currentTimeMillis - for (thread <- threadList) - thread.start - - for (thread <- threadList) - thread.join - - val endMs = System.currentTimeMillis - val elapsedSecs = (endMs - startMs - config.consumerConfig.consumerTimeoutMs) / 1000.0 + var startMs, endMs = 0L + if(config.useNewConsumer) { + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](config.props) + consumer.subscribe(config.topic) + startMs = System.currentTimeMillis + consume(consumer, config.numMessages, 1000, config, totalMessagesRead, totalBytesRead) + endMs = System.currentTimeMillis + } else { + import kafka.consumer.ConsumerConfig + val consumerConfig = new ConsumerConfig(config.props) + val consumerConnector: ConsumerConnector = Consumer.create(consumerConfig) + val topicMessageStreams = consumerConnector.createMessageStreams(Map(config.topic -> config.numThreads)) + var threadList = List[ConsumerPerfThread]() + for ((topic, streamList) <- topicMessageStreams) + for (i <- 0 until streamList.length) + threadList ::= new ConsumerPerfThread(i, "kafka-zk-consumer-" + i, streamList(i), config, totalMessagesRead, totalBytesRead) + + logger.info("Sleeping for 1 second.") + Thread.sleep(1000) + logger.info("starting threads") + startMs = System.currentTimeMillis + for (thread <- threadList) + thread.start + for (thread <- threadList) + thread.join + endMs = System.currentTimeMillis - consumerConfig.consumerTimeoutMs + } + val elapsedSecs = (endMs - startMs) / 1000.0 if (!config.showDetailedStats) { val totalMBRead = (totalBytesRead.get * 1.0) / (1024 * 1024) - println(("%s, %s, %d, %.4f, %.4f, %d, %.4f").format(config.dateFormat.format(startMs), config.dateFormat.format(endMs), - config.consumerConfig.fetchMessageMaxBytes, totalMBRead, totalMBRead / elapsedSecs, totalMessagesRead.get, - totalMessagesRead.get / elapsedSecs)) + println(("%s, %s, %.4f, %.4f, %d, %.4f").format(config.dateFormat.format(startMs), config.dateFormat.format(endMs), + totalMBRead, totalMBRead / elapsedSecs, totalMessagesRead.get, totalMessagesRead.get / elapsedSecs)) } System.exit(0) } + + def consume(consumer: KafkaConsumer[Array[Byte], Array[Byte]], count: Long, timeout: Long, config: ConsumerPerfConfig, totalMessagesRead: AtomicLong, totalBytesRead: AtomicLong) { + var bytesRead = 0L + var messagesRead = 0L + val startMs = System.currentTimeMillis + var lastReportTime: Long = startMs + var lastBytesRead = 0L + var lastMessagesRead = 0L + var lastConsumed = System.currentTimeMillis + while(messagesRead < count && lastConsumed >= System.currentTimeMillis - timeout) { + val records = consumer.poll(100) + if(records.count() > 0) + lastConsumed = System.currentTimeMillis + for(record <- records) { + messagesRead += 1 + if(record.key != null) + bytesRead += record.key.size + if(record.value != null) + bytesRead += record.value.size + + if (messagesRead % config.reportingInterval == 0) { + if (config.showDetailedStats) + printProgressMessage(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, lastReportTime, System.currentTimeMillis, config.dateFormat) + lastReportTime = System.currentTimeMillis + lastMessagesRead = messagesRead + lastBytesRead = bytesRead + } + } + } + totalMessagesRead.set(messagesRead) + totalBytesRead.set(bytesRead) + } + + def printProgressMessage(id: Int, bytesRead: Long, lastBytesRead: Long, messagesRead: Long, lastMessagesRead: Long, + startMs: Long, endMs: Long, dateFormat: SimpleDateFormat) = { + val elapsedMs: Double = endMs - startMs + val totalMBRead = (bytesRead * 1.0) / (1024 * 1024) + val mbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024) + println(("%s, %d, %.4f, %.4f, %d, %.4f").format(dateFormat.format(endMs), id, totalMBRead, + 1000.0 * (mbRead / elapsedMs), messagesRead, ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0)) + } class ConsumerPerfConfig(args: Array[String]) extends PerfConfig(args) { - val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED: The connection string for the zookeeper connection in the form host:port. " + - "Multiple URLS can be given to allow fail-over.") + val zkConnectOpt = parser.accepts("zookeeper", "The connection string for the zookeeper connection in the form host:port. " + + "Multiple URLS can be given to allow fail-over. This option is only used with the old consumer.") .withRequiredArg .describedAs("urls") .ofType(classOf[String]) + val bootstrapServersOpt = parser.accepts("broker-list", "A broker list to use for connecting if using the new consumer.") + .withRequiredArg() + .describedAs("host") + .ofType(classOf[String]) val topicOpt = parser.accepts("topic", "REQUIRED: The topic to consume from.") .withRequiredArg .describedAs("topic") @@ -117,20 +172,35 @@ object ConsumerPerformance { .describedAs("count") .ofType(classOf[java.lang.Integer]) .defaultsTo(1) + val useNewConsumerOpt = parser.accepts("new-consumer", "Use the new consumer implementation.") val options = parser.parse(args: _*) - CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, zkConnectOpt) - + CommandLineUtils.checkRequiredArgs(parser, options, topicOpt) + + val useNewConsumer = options.has(useNewConsumerOpt) + val props = new Properties - props.put("group.id", options.valueOf(groupIdOpt)) - props.put("socket.receive.buffer.bytes", options.valueOf(socketBufferSizeOpt).toString) - props.put("fetch.message.max.bytes", options.valueOf(fetchSizeOpt).toString) - props.put("auto.offset.reset", if (options.has(resetBeginningOffsetOpt)) "largest" else "smallest") - props.put("zookeeper.connect", options.valueOf(zkConnectOpt)) - props.put("consumer.timeout.ms", "5000") - props.put("num.consumer.fetchers", options.valueOf(numFetchersOpt).toString) - val consumerConfig = new ConsumerConfig(props) + if(useNewConsumer) { + import org.apache.kafka.clients.consumer.ConsumerConfig + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, options.valueOf(bootstrapServersOpt)) + props.put(ConsumerConfig.GROUP_ID_CONFIG, options.valueOf(groupIdOpt)) + props.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, options.valueOf(socketBufferSizeOpt).toString) + props.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, options.valueOf(fetchSizeOpt).toString) + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, if (options.has(resetBeginningOffsetOpt)) "latest" else "earliest") + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[ByteArrayDeserializer]) + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[ByteArrayDeserializer]) + props.put(ConsumerConfig.CHECK_CRCS_CONFIG, "false") + } else { + CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt) + props.put("group.id", options.valueOf(groupIdOpt)) + props.put("socket.receive.buffer.bytes", options.valueOf(socketBufferSizeOpt).toString) + props.put("fetch.message.max.bytes", options.valueOf(fetchSizeOpt).toString) + props.put("auto.offset.reset", if (options.has(resetBeginningOffsetOpt)) "largest" else "smallest") + props.put("zookeeper.connect", options.valueOf(zkConnectOpt)) + props.put("consumer.timeout.ms", "1000") + props.put("num.consumer.fetchers", options.valueOf(numFetchersOpt).toString) + } val numThreads = options.valueOf(numThreadsOpt).intValue val topic = options.valueOf(topicOpt) val numMessages = options.valueOf(numMessagesOpt).longValue @@ -161,7 +231,7 @@ object ConsumerPerformance { if (messagesRead % config.reportingInterval == 0) { if (config.showDetailedStats) - printMessage(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, lastReportTime, System.currentTimeMillis) + printProgressMessage(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, lastReportTime, System.currentTimeMillis, config.dateFormat) lastReportTime = System.currentTimeMillis lastMessagesRead = messagesRead lastBytesRead = bytesRead @@ -176,18 +246,9 @@ object ConsumerPerformance { totalMessagesRead.addAndGet(messagesRead) totalBytesRead.addAndGet(bytesRead) if (config.showDetailedStats) - printMessage(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, System.currentTimeMillis) + printProgressMessage(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, System.currentTimeMillis, config.dateFormat) } - private def printMessage(id: Int, bytesRead: Long, lastBytesRead: Long, messagesRead: Long, lastMessagesRead: Long, - startMs: Long, endMs: Long) = { - val elapsedMs = endMs - startMs - val totalMBRead = (bytesRead * 1.0) / (1024 * 1024) - val mbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024) - println(("%s, %d, %d, %.4f, %.4f, %d, %.4f").format(config.dateFormat.format(endMs), id, - config.consumerConfig.fetchMessageMaxBytes, totalMBRead, - 1000.0 * (mbRead / elapsedMs), messagesRead, ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0)) - } } } diff --git a/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala b/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala index 7602b8d705970..900f7df9f5ce0 100644 --- a/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala +++ b/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala @@ -79,7 +79,7 @@ object SimpleConsumerPerformance { done = true else // we only did one fetch so we find the offset for the first (head) messageset - offset += messageSet.validBytes + offset = messageSet.last.nextOffset totalBytesRead += bytesRead totalMessagesRead += messagesRead diff --git a/core/src/main/scala/kafka/utils/KafkaScheduler.scala b/core/src/main/scala/kafka/utils/KafkaScheduler.scala index 9a16343d2ff71..7ceadccefe820 100644 --- a/core/src/main/scala/kafka/utils/KafkaScheduler.scala +++ b/core/src/main/scala/kafka/utils/KafkaScheduler.scala @@ -40,6 +40,11 @@ trait Scheduler { */ def shutdown() + /** + * Check if the scheduler has been started + */ + def isStarted: Boolean + /** * Schedule a task * @param name The name of this task @@ -63,13 +68,13 @@ trait Scheduler { class KafkaScheduler(val threads: Int, val threadNamePrefix: String = "kafka-scheduler-", daemon: Boolean = true) extends Scheduler with Logging { - @volatile private var executor: ScheduledThreadPoolExecutor = null + private var executor: ScheduledThreadPoolExecutor = null private val schedulerThreadId = new AtomicInteger(0) override def startup() { debug("Initializing task scheduler.") this synchronized { - if(executor != null) + if(isStarted) throw new IllegalStateException("This scheduler has already been started!") executor = new ScheduledThreadPoolExecutor(threads) executor.setContinueExistingPeriodicTasksAfterShutdownPolicy(false) @@ -83,34 +88,45 @@ class KafkaScheduler(val threads: Int, override def shutdown() { debug("Shutting down task scheduler.") - ensureStarted - executor.shutdown() - executor.awaitTermination(1, TimeUnit.DAYS) - this.executor = null + this synchronized { + if(isStarted) { + executor.shutdown() + executor.awaitTermination(1, TimeUnit.DAYS) + this.executor = null + } + } } def schedule(name: String, fun: ()=>Unit, delay: Long, period: Long, unit: TimeUnit) = { debug("Scheduling task %s with initial delay %d ms and period %d ms." .format(name, TimeUnit.MILLISECONDS.convert(delay, unit), TimeUnit.MILLISECONDS.convert(period, unit))) - ensureStarted - val runnable = Utils.runnable { - try { - trace("Begining execution of scheduled task '%s'.".format(name)) - fun() - } catch { - case t: Throwable => error("Uncaught exception in scheduled task '" + name +"'", t) - } finally { - trace("Completed execution of scheduled task '%s'.".format(name)) + this synchronized { + ensureStarted + val runnable = Utils.runnable { + try { + trace("Begining execution of scheduled task '%s'.".format(name)) + fun() + } catch { + case t: Throwable => error("Uncaught exception in scheduled task '" + name +"'", t) + } finally { + trace("Completed execution of scheduled task '%s'.".format(name)) + } } + if(period >= 0) + executor.scheduleAtFixedRate(runnable, delay, period, unit) + else + executor.schedule(runnable, delay, unit) + } + } + + def isStarted: Boolean = { + this synchronized { + executor != null } - if(period >= 0) - executor.scheduleAtFixedRate(runnable, delay, period, unit) - else - executor.schedule(runnable, delay, unit) } private def ensureStarted = { - if(executor == null) + if(!isStarted) throw new IllegalStateException("Kafka scheduler has not been started") } } diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala new file mode 100644 index 0000000000000..798f035df52e4 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -0,0 +1,286 @@ +/** + * 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.api + +import org.apache.kafka.common.KafkaException +import org.apache.kafka.clients.producer.ProducerConfig +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.clients.consumer.Consumer +import org.apache.kafka.clients.consumer.KafkaConsumer +import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback +import org.apache.kafka.clients.consumer.ConsumerRecord +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.clients.consumer.CommitType +import org.apache.kafka.common.serialization.ByteArrayDeserializer +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.clients.consumer.NoOffsetForPartitionException +import scala.collection.mutable.Buffer +import scala.collection.JavaConversions._ +import java.util.ArrayList +import java.util.Arrays +import org.junit.Assert._ +import kafka.utils.TestUtils +import kafka.utils.Logging +import kafka.server.OffsetManager + +/** + * Integration tests for the new consumer that cover basic usage as well as server failures + */ +class ConsumerTest extends IntegrationTestHarness with Logging { + + val producerCount = 1 + val consumerCount = 2 + val serverCount = 3 + + val topic = "topic" + val part = 0 + val tp = new TopicPartition(topic, part) + + // configure the servers and clients + this.serverConfig.setProperty("controlled.shutdown.enable", "false") // speed up shutdown + this.serverConfig.setProperty("offsets.topic.replication.factor", "3") // don't want to lose offset + this.serverConfig.setProperty("offsets.topic.num.partitions", "1") + this.producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "all") + this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "my-test") + this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 4096.toString) + this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + + override def setUp() { + super.setUp() + // this will trigger the creation of the consumer offsets topic + this.consumers(0).partitionsFor(OffsetManager.OffsetsTopicName) + } + + def testSimpleConsumption() { + val numRecords = 10000 + sendRecords(numRecords) + + assertEquals(0, this.consumers(0).subscriptions.size) + this.consumers(0).subscribe(tp) + assertEquals(1, this.consumers(0).subscriptions.size) + + this.consumers(0).seek(tp, 0) + consumeRecords(this.consumers(0), numRecords = numRecords, startingOffset = 0) + } + + def testAutoOffsetReset() { + sendRecords(1) + this.consumers(0).subscribe(tp) + consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0) + } + + def testSeek() { + val consumer = this.consumers(0) + val totalRecords = 50L + sendRecords(totalRecords.toInt) + consumer.subscribe(tp) + + consumer.seekToEnd(tp) + assertEquals(totalRecords, consumer.position(tp)) + assertFalse(consumer.poll(totalRecords).iterator().hasNext()) + + consumer.seekToBeginning(tp) + assertEquals(0, consumer.position(tp), 0) + consumeRecords(consumer, numRecords = 1, startingOffset = 0) + + val mid = totalRecords / 2 + consumer.seek(tp, mid) + assertEquals(mid, consumer.position(tp)) + consumeRecords(consumer, numRecords = 1, startingOffset = mid.toInt) + } + + def testGroupConsumption() { + // we need to do this test with only one server since we have the hack join group + // that just assigns the partition hosted on the local machine (with two we might get the wrong machine + this.servers.last.shutdown() + this.servers.head.shutdown() + sendRecords(10) + this.consumers(0).subscribe(topic) + consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0) + } + + def testPositionAndCommit() { + sendRecords(5) + + // committed() on a partition with no committed offset throws an exception + intercept[NoOffsetForPartitionException] { + this.consumers(0).committed(new TopicPartition(topic, 15)) + } + + // position() on a partition that we aren't subscribed to throws an exception + intercept[IllegalArgumentException] { + this.consumers(0).position(new TopicPartition(topic, 15)) + } + + this.consumers(0).subscribe(tp) + + assertEquals("position() on a partition that we are subscribed to should reset the offset", 0L, this.consumers(0).position(tp)) + this.consumers(0).commit(CommitType.SYNC) + assertEquals(0L, this.consumers(0).committed(tp)) + + consumeRecords(this.consumers(0), 5, 0) + assertEquals("After consuming 5 records, position should be 5", 5L, this.consumers(0).position(tp)) + this.consumers(0).commit(CommitType.SYNC) + assertEquals("Committed offset should be returned", 5L, this.consumers(0).committed(tp)); + + sendRecords(1) + + // another consumer in the same group should get the same position + this.consumers(1).subscribe(tp) + consumeRecords(this.consumers(1), 1, 5) + } + + def testPartitionsFor() { + val numParts = 2; + TestUtils.createTopic(this.zkClient, topic, numParts, 1, this.servers) + val parts = this.consumers(0).partitionsFor(topic) + assertNotNull(parts) + assertEquals(2, parts.length) + assertNull(this.consumers(0).partitionsFor("non-existant-topic")) + } + + def testConsumptionWithBrokerFailures() = consumeWithBrokerFailures(numRecords = 1000) + + /* + * 1. Produce a bunch of messages + * 2. Then consume the messages while killing and restarting brokers at random + */ + def consumeWithBrokerFailures(numRecords: Int) { + TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) + sendRecords(numRecords) + this.producers.map(_.close) + var consumed = 0 + val consumer = this.consumers(0) + consumer.subscribe(topic) + while (consumed < numRecords) { + // check that we are getting the messages in order + for (record <- consumer.poll(200)) { + assertEquals(consumed.toLong, record.offset()) + consumed += 1 + } + consumer.commit(CommitType.SYNC); + + /* restart any dead brokers, and kill a broker (with probability 1/3) */ + restartDeadBrokers() + if (TestUtils.random.nextInt(3) == 0) { + info("Killing broker") + killRandomBroker() + } + } + } + + def testSeekAndCommitWithBrokerFailures() = seekAndCommitWithBrokerFailures(20) + + def seekAndCommitWithBrokerFailures(numIters: Int) { + // create a topic and send it some data + val numRecords = 1000 + TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) + sendRecords(numRecords) + this.producers.map(_.close) + + val consumer = this.consumers(0) + consumer.subscribe(tp) + consumer.seek(tp, 0) + for (iter <- 0 until numIters) { + val coin = TestUtils.random.nextInt(4) + if (coin == 0) { + info("Seeking to end of log") + consumer.seekToEnd() + assertEquals(1000.toLong, consumer.position(tp)) + } else if (coin == 1) { + val pos = TestUtils.random.nextInt(numRecords).toLong + info("Seeking to " + pos) + consumer.seek(tp, pos) + assertEquals(pos, consumer.position(tp)) + } else if (coin == 2) { + info("Committing offset.") + consumer.commit(CommitType.SYNC) + assertEquals(consumer.position(tp), consumer.committed(tp)) + } else { + restartDeadBrokers() + killRandomBroker() + } + } + } + + def testPartitionReassignmentCallback() { + val callback = new TestConsumerReassignmentCallback() + this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "200"); // timeout quickly to avoid slow test + val consumer0 = new KafkaConsumer(this.consumerConfig, callback, new ByteArrayDeserializer(), new ByteArrayDeserializer()) + consumer0.subscribe("test") + + // the initial subscription should cause a callback execution + while(callback.callsToAssigned == 0) + consumer0.poll(50) + + // get metadata for the topic + var parts = consumer0.partitionsFor(OffsetManager.OffsetsTopicName) + while(parts == null) + parts = consumer0.partitionsFor(OffsetManager.OffsetsTopicName) + assertEquals(1, parts.size) + assertNotNull(parts(0).leader()) + + // shutdown the co-ordinator + val coordinator = parts(0).leader().id() + this.servers(coordinator).shutdown() + + // this should cause another callback execution + while(callback.callsToAssigned < 2) + consumer0.poll(50) + assertEquals(2, callback.callsToAssigned) + assertEquals(2, callback.callsToRevoked) + + consumer0.close() + } + + class TestConsumerReassignmentCallback extends ConsumerRebalanceCallback { + var callsToAssigned = 0 + var callsToRevoked = 0 + def onPartitionsAssigned(consumer: Consumer[_,_], partitions: java.util.Collection[TopicPartition]) { + info("onPartitionsAssigned called.") + callsToAssigned += 1 + } + def onPartitionsRevoked(consumer: Consumer[_,_], partitions: java.util.Collection[TopicPartition]) { + info("onPartitionsRevoked called.") + callsToRevoked += 1 + } + } + + private def sendRecords(numRecords: Int) { + val futures = (0 until numRecords).map { i => + this.producers(0).send(new ProducerRecord(topic, part, i.toString.getBytes, i.toString.getBytes)) + } + futures.map(_.get) + } + + private def consumeRecords(consumer: Consumer[Array[Byte], Array[Byte]], numRecords: Int, startingOffset: Int) { + val records = new ArrayList[ConsumerRecord[Array[Byte], Array[Byte]]]() + val maxIters = numRecords * 300 + var iters = 0 + while (records.size < numRecords) { + for (record <- consumer.poll(50)) + records.add(record) + if(iters > maxIters) + throw new IllegalStateException("Failed to consume the expected records after " + iters + " iterations."); + iters += 1 + } + for (i <- 0 until numRecords) { + val record = records.get(i) + val offset = startingOffset + i + assertEquals(topic, record.topic()) + assertEquals(part, record.partition()) + assertEquals(offset.toLong, record.offset()) + } + } + +} \ No newline at end of file diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala new file mode 100644 index 0000000000000..5650b4a7b950b --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -0,0 +1,73 @@ +/** + * 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.api + +import org.apache.kafka.clients.producer.ProducerConfig +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.scalatest.junit.JUnit3Suite +import collection._ +import kafka.utils.TestUtils +import java.util.Properties +import java.util.Arrays +import org.apache.kafka.clients.consumer.KafkaConsumer +import org.apache.kafka.clients.producer.KafkaProducer +import kafka.server.KafkaConfig +import kafka.integration.KafkaServerTestHarness +import scala.collection.mutable.Buffer + +/** + * A helper class for writing integration tests that involve producers, consumers, and servers + */ +trait IntegrationTestHarness extends KafkaServerTestHarness { + + val producerCount: Int + val consumerCount: Int + val serverCount: Int + lazy val producerConfig = new Properties + lazy val consumerConfig = new Properties + lazy val serverConfig = new Properties + override lazy val configs = { + val cfgs = TestUtils.createBrokerConfigs(serverCount) + cfgs.map(_.putAll(serverConfig)) + cfgs.map(new KafkaConfig(_)) + } + + var consumers = Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]() + var producers = Buffer[KafkaProducer[Array[Byte], Array[Byte]]]() + + override def setUp() { + super.setUp() + producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapUrl) + producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArraySerializer]) + producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArraySerializer]) + consumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapUrl) + consumerConfig.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArrayDeserializer]) + consumerConfig.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArrayDeserializer]) + for(i <- 0 until producerCount) + producers += new KafkaProducer(producerConfig) + for(i <- 0 until consumerCount) + consumers += new KafkaConsumer(consumerConfig) + } + + override def tearDown() { + producers.map(_.close()) + consumers.map(_.close()) + super.tearDown() + } + +} diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index cd16ced5465d0..a1f72f8c2042f 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -192,7 +192,7 @@ object SerializationTestUtils { } def createConsumerMetadataResponse: ConsumerMetadataResponse = { - ConsumerMetadataResponse(Some(brokers.head), ErrorMapping.NoError) + ConsumerMetadataResponse(Some(brokers.head), ErrorMapping.NoError, 0) } def createHeartbeatRequestAndHeader: HeartbeatRequestAndHeader = { @@ -237,7 +237,7 @@ class RequestResponseSerializationTest extends JUnitSuite { private val offsetFetchResponse = SerializationTestUtils.createTestOffsetFetchResponse private val consumerMetadataRequest = SerializationTestUtils.createConsumerMetadataRequest private val consumerMetadataResponse = SerializationTestUtils.createConsumerMetadataResponse - private val consumerMetadataResponseNoCoordinator = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode) + private val consumerMetadataResponseNoCoordinator = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, 0) private val heartbeatRequest = SerializationTestUtils.createHeartbeatRequestAndHeader private val heartbeatResponse = SerializationTestUtils.createHeartbeatResponseAndHeader private val joinGroupRequest = SerializationTestUtils.createJoinGroupRequestAndHeader diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index 3cf7c9bcd6449..ef4c9aeaa2711 100644 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -17,11 +17,14 @@ package kafka.integration +import java.util.Arrays +import scala.collection.mutable.Buffer import kafka.server._ import kafka.utils.{Utils, TestUtils} import org.scalatest.junit.JUnit3Suite import kafka.zk.ZooKeeperTestHarness import kafka.common.KafkaException +import kafka.utils.TestUtils /** * A test harness that brings up some number of broker nodes @@ -29,15 +32,22 @@ import kafka.common.KafkaException trait KafkaServerTestHarness extends JUnit3Suite with ZooKeeperTestHarness { val configs: List[KafkaConfig] - var servers: List[KafkaServer] = null + var servers: Buffer[KafkaServer] = null var brokerList: String = null - + var alive: Array[Boolean] = null + + def serverForId(id: Int) = servers.find(s => s.config.brokerId == id) + + def bootstrapUrl = configs.map(c => c.hostName + ":" + c.port).mkString(",") + override def setUp() { super.setUp if(configs.size <= 0) throw new KafkaException("Must suply at least one server config.") brokerList = TestUtils.getBrokerListStrFromConfigs(configs) - servers = configs.map(TestUtils.createServer(_)) + servers = configs.map(TestUtils.createServer(_)).toBuffer + alive = new Array[Boolean](servers.length) + Arrays.fill(alive, true) } override def tearDown() { @@ -45,4 +55,27 @@ trait KafkaServerTestHarness extends JUnit3Suite with ZooKeeperTestHarness { servers.map(server => server.config.logDirs.map(Utils.rm(_))) super.tearDown } + + /** + * Pick a broker at random and kill it if it isn't already dead + * Return the id of the broker killed + */ + def killRandomBroker(): Int = { + val index = TestUtils.random.nextInt(servers.length) + if(alive(index)) { + servers(index).shutdown() + alive(index) = false + } + index + } + + /** + * Restart any dead brokers + */ + def restartDeadBrokers() { + for(i <- 0 until servers.length if !alive(i)) { + servers(i) = TestUtils.createServer(configs(i)) + alive(i) = true + } + } } diff --git a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala index a5386a03b6295..aeb7a19acaefa 100644 --- a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala +++ b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala @@ -32,6 +32,7 @@ import kafka.common.{TopicAndPartition, ErrorMapping, UnknownTopicOrPartitionExc import kafka.utils.{StaticPartitioner, TestUtils, Utils} import kafka.serializer.StringEncoder import java.util.Properties +import TestUtils._ /** * End to end tests of the primitive apis against a local server @@ -113,7 +114,8 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with } private def produceAndMultiFetch(producer: Producer[String, String]) { - createSimpleTopicsAndAwaitLeader(zkClient, List("test1", "test2", "test3", "test4")) + for(topic <- List("test1", "test2", "test3", "test4")) + TestUtils.createTopic(zkClient, topic, servers = servers) // send some messages val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0)); @@ -181,7 +183,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with private def multiProduce(producer: Producer[String, String]) { val topics = Map("test4" -> 0, "test1" -> 0, "test2" -> 0, "test3" -> 0) - createSimpleTopicsAndAwaitLeader(zkClient, topics.keys) + topics.keys.map(topic => TestUtils.createTopic(zkClient, topic, servers = servers)) val messages = new mutable.HashMap[String, Seq[String]] val builder = new FetchRequestBuilder() @@ -215,7 +217,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with def testPipelinedProduceRequests() { val topics = Map("test4" -> 0, "test1" -> 0, "test2" -> 0, "test3" -> 0) - createSimpleTopicsAndAwaitLeader(zkClient, topics.keys) + topics.keys.map(topic => TestUtils.createTopic(zkClient, topic, servers = servers)) val props = new Properties() props.put("request.required.acks", "0") val pipelinedProducer: Producer[String, String] = @@ -265,15 +267,4 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with assertEquals(messages(topic), fetched.map(messageAndOffset => Utils.readString(messageAndOffset.message.payload))) } } - - /** - * For testing purposes, just create these topics each with one partition and one replica for - * which the provided broker should the leader for. Create and wait for broker to lead. Simple. - */ - private def createSimpleTopicsAndAwaitLeader(zkClient: ZkClient, topics: Iterable[String]) { - for( topic <- topics ) { - AdminUtils.createTopic(zkClient, topic, partitions = 1, replicationFactor = 1) - TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, partition = 0) - } - } } diff --git a/core/src/test/scala/unit/kafka/utils/MockScheduler.scala b/core/src/test/scala/unit/kafka/utils/MockScheduler.scala index d5896ed4d3b73..c6740782813cb 100644 --- a/core/src/test/scala/unit/kafka/utils/MockScheduler.scala +++ b/core/src/test/scala/unit/kafka/utils/MockScheduler.scala @@ -36,6 +36,8 @@ class MockScheduler(val time: Time) extends Scheduler { /* a priority queue of tasks ordered by next execution time */ var tasks = new PriorityQueue[MockTask]() + + def isStarted = true def startup() {} diff --git a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala index b364ac2d6d623..cfea63b88e259 100644 --- a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala +++ b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala @@ -90,4 +90,21 @@ class SchedulerTest { assertTrue("Should count to 20", counter1.get >= 20) } } + + @Test + def testRestart() { + // schedule a task to increment a counter + mockTime.scheduler.schedule("test1", counter1.getAndIncrement, delay=1) + mockTime.sleep(1) + assertEquals(1, counter1.get()) + + // restart the scheduler + mockTime.scheduler.shutdown() + mockTime.scheduler.startup() + + // schedule another task to increment the counter + mockTime.scheduler.schedule("test1", counter1.getAndIncrement, delay=1) + mockTime.sleep(1) + assertEquals(2, counter1.get()) + } } \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index ac15d34425795..54755e8dd3f23 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -45,6 +45,7 @@ import kafka.log._ import junit.framework.AssertionFailedError import junit.framework.Assert._ import org.apache.kafka.clients.producer.KafkaProducer +import collection.Iterable import scala.collection.Map @@ -709,24 +710,21 @@ object TestUtils extends Logging { /** * Create new LogManager instance with default configuration for testing */ - def createLogManager( - logDirs: Array[File] = Array.empty[File], - defaultConfig: LogConfig = LogConfig(), - cleanerConfig: CleanerConfig = CleanerConfig(enableCleaner = false), - time: MockTime = new MockTime()) = - { - new LogManager( - logDirs = logDirs, - topicConfigs = Map(), - defaultConfig = defaultConfig, - cleanerConfig = cleanerConfig, - ioThreads = 4, - flushCheckMs = 1000L, - flushCheckpointMs = 10000L, - retentionCheckMs = 1000L, - scheduler = time.scheduler, - time = time, - brokerState = new BrokerState()) + def createLogManager(logDirs: Array[File] = Array.empty[File], + defaultConfig: LogConfig = LogConfig(), + cleanerConfig: CleanerConfig = CleanerConfig(enableCleaner = false), + time: MockTime = new MockTime()): LogManager = { + new LogManager(logDirs = logDirs, + topicConfigs = Map(), + defaultConfig = defaultConfig, + cleanerConfig = cleanerConfig, + ioThreads = 4, + flushCheckMs = 1000L, + flushCheckpointMs = 10000L, + retentionCheckMs = 1000L, + scheduler = time.scheduler, + time = time, + brokerState = new BrokerState()) } def sendMessagesToPartition(configs: Seq[KafkaConfig], From 75a286e4308831203cc7e1d4748bdab504e8a1f3 Mon Sep 17 00:00:00 2001 From: Jay Kreps Date: Thu, 29 Jan 2015 20:09:36 -0800 Subject: [PATCH 113/491] KAFKA-1760 Follow-up: fix compilation issue with Scala 2.11 --- core/src/main/scala/kafka/server/KafkaApis.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 48bc4359ff448..f2b027bf944e7 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -455,12 +455,13 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleJoinGroupRequest(request: RequestChannel.Request) { + import JavaConversions._ val joinGroupReq = request.requestObj.asInstanceOf[JoinGroupRequestAndHeader] - val topics = JavaConversions.asScalaIterable(joinGroupReq.body.topics()).toSet + val topics = joinGroupReq.body.topics().toSet val partitions = this.replicaManager.logManager.allLogs.filter(log => topics.contains(log.topicAndPartition.topic)) val partitionList = partitions.map(_.topicAndPartition).map(tp => new org.apache.kafka.common.TopicPartition(tp.topic, tp.partition)).toBuffer this.consumerGroupGenerationId += 1 - val response = new JoinGroupResponse(ErrorMapping.NoError, this.consumerGroupGenerationId, joinGroupReq.body.consumerId, JavaConversions.asJavaList(partitionList)) + val response = new JoinGroupResponse(ErrorMapping.NoError, this.consumerGroupGenerationId, joinGroupReq.body.consumerId, partitionList) val send = new BoundedByteBufferSend(new JoinGroupResponseAndHeader(joinGroupReq.correlationId, response)) requestChannel.sendResponse(new RequestChannel.Response(request, send)) } From f1ba4ff87e60a6dc06fb3d16e84fa94f8c4a7d9e Mon Sep 17 00:00:00 2001 From: Joel Koshy Date: Mon, 2 Feb 2015 18:17:15 -0800 Subject: [PATCH 114/491] KAFKA-1729; Add constructor to javaapi to allow constructing explicitly versioned offset commit requests; reviewed by Jun Rao --- .../scala/kafka/api/OffsetCommitResponse.scala | 4 +++- .../kafka/javaapi/ConsumerMetadataResponse.scala | 6 ++++++ .../scala/kafka/javaapi/OffsetCommitRequest.scala | 14 ++++++++++++-- .../scala/kafka/javaapi/OffsetCommitResponse.scala | 9 +++++++++ .../scala/kafka/javaapi/OffsetFetchResponse.scala | 5 +++++ 5 files changed, 35 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/api/OffsetCommitResponse.scala b/core/src/main/scala/kafka/api/OffsetCommitResponse.scala index 624a1c1cc5406..116547ae1139e 100644 --- a/core/src/main/scala/kafka/api/OffsetCommitResponse.scala +++ b/core/src/main/scala/kafka/api/OffsetCommitResponse.scala @@ -20,7 +20,7 @@ package kafka.api import java.nio.ByteBuffer import kafka.utils.Logging -import kafka.common.TopicAndPartition +import kafka.common.{ErrorMapping, TopicAndPartition} object OffsetCommitResponse extends Logging { val CurrentVersion: Short = 0 @@ -47,6 +47,8 @@ case class OffsetCommitResponse(commitStatus: Map[TopicAndPartition, Short], lazy val commitStatusGroupedByTopic = commitStatus.groupBy(_._1.topic) + def hasError = commitStatus.exists{ case (topicAndPartition, errorCode) => errorCode != ErrorMapping.NoError } + def writeTo(buffer: ByteBuffer) { buffer.putInt(correlationId) buffer.putInt(commitStatusGroupedByTopic.size) diff --git a/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala index 1b28861cdf7df..d281bb31a66fd 100644 --- a/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala +++ b/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala @@ -17,6 +17,8 @@ package kafka.javaapi +import java.nio.ByteBuffer + import kafka.cluster.Broker class ConsumerMetadataResponse(private val underlying: kafka.api.ConsumerMetadataResponse) { @@ -40,3 +42,7 @@ class ConsumerMetadataResponse(private val underlying: kafka.api.ConsumerMetadat override def toString = underlying.toString } + +object ConsumerMetadataResponse { + def readFrom(buffer: ByteBuffer) = new ConsumerMetadataResponse(kafka.api.ConsumerMetadataResponse.readFrom(buffer)) +} diff --git a/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala b/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala index 873f575cc22af..456c3c423a1dd 100644 --- a/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala +++ b/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala @@ -22,7 +22,8 @@ import kafka.common.{OffsetAndMetadata, TopicAndPartition} class OffsetCommitRequest(groupId: String, requestInfo: java.util.Map[TopicAndPartition, OffsetAndMetadata], correlationId: Int, - clientId: String) { + clientId: String, + versionId: Short) { val underlying = { val scalaMap: collection.immutable.Map[TopicAndPartition, OffsetAndMetadata] = { import collection.JavaConversions._ @@ -32,12 +33,21 @@ class OffsetCommitRequest(groupId: String, kafka.api.OffsetCommitRequest( groupId = groupId, requestInfo = scalaMap, - versionId = 0, // binds to version 0 so that it commits to Zookeeper + versionId = versionId, correlationId = correlationId, clientId = clientId ) } + def this(groupId: String, + requestInfo: java.util.Map[TopicAndPartition, OffsetAndMetadata], + correlationId: Int, + clientId: String) { + + // by default bind to version 0 so that it commits to Zookeeper + this(groupId, requestInfo, correlationId, clientId, 0) + } + override def toString = underlying.toString diff --git a/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala b/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala index c2d3d114b82a8..b22232980e5ab 100644 --- a/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala +++ b/core/src/main/scala/kafka/javaapi/OffsetCommitResponse.scala @@ -17,6 +17,8 @@ package kafka.javaapi +import java.nio.ByteBuffer + import kafka.common.TopicAndPartition import collection.JavaConversions @@ -27,5 +29,12 @@ class OffsetCommitResponse(private val underlying: kafka.api.OffsetCommitRespons underlying.commitStatus } + def hasError = underlying.hasError + + def errorCode(topicAndPartition: TopicAndPartition) = underlying.commitStatus(topicAndPartition) + } +object OffsetCommitResponse { + def readFrom(buffer: ByteBuffer) = new OffsetCommitResponse(kafka.api.OffsetCommitResponse.readFrom(buffer)) +} diff --git a/core/src/main/scala/kafka/javaapi/OffsetFetchResponse.scala b/core/src/main/scala/kafka/javaapi/OffsetFetchResponse.scala index 60924d2fe543d..c4bdb128717ee 100644 --- a/core/src/main/scala/kafka/javaapi/OffsetFetchResponse.scala +++ b/core/src/main/scala/kafka/javaapi/OffsetFetchResponse.scala @@ -17,6 +17,8 @@ package kafka.javaapi +import java.nio.ByteBuffer + import kafka.common.{TopicAndPartition, OffsetMetadataAndError} import collection.JavaConversions @@ -29,3 +31,6 @@ class OffsetFetchResponse(private val underlying: kafka.api.OffsetFetchResponse) } +object OffsetFetchResponse { + def readFrom(buffer: ByteBuffer) = new OffsetFetchResponse(kafka.api.OffsetFetchResponse.readFrom(buffer)) +} From 1c6d5bbac672cbf49591aed0889510b10e3285fa Mon Sep 17 00:00:00 2001 From: Jay Kreps Date: Mon, 2 Feb 2015 21:36:21 -0800 Subject: [PATCH 115/491] KAFKA-1915: Add checkstyle for java code. --- README.md | 3 + build.gradle | 6 + checkstyle/checkstyle.xml | 83 + checkstyle/import-control.xml | 100 + .../utils => clients}/ClientUtils.java | 11 +- .../clients/ClusterConnectionStates.java | 9 +- .../{producer/internals => }/Metadata.java | 2 +- .../apache/kafka/clients/NetworkClient.java | 3 +- .../kafka/clients/consumer/CommitType.java | 12 + .../clients/consumer/ConsumerConfig.java | 8 +- .../clients/consumer/ConsumerRecords.java | 40 +- .../kafka/clients/consumer/KafkaConsumer.java | 43 +- .../clients/consumer/internals/Heartbeat.java | 12 + .../NoOpConsumerRebalanceCallback.java | 4 +- .../consumer/internals/SubscriptionState.java | 16 +- .../kafka/clients/producer/KafkaProducer.java | 25 +- .../kafka/clients/producer/Producer.java | 6 +- .../clients/producer/ProducerConfig.java | 10 +- .../producer/internals/BufferPool.java | 8 +- .../internals/ProduceRequestResult.java | 1 - .../producer/internals/RecordAccumulator.java | 40 +- .../clients/producer/internals/Sender.java | 1 + .../clients/tools/ProducerPerformance.java | 8 +- .../java/org/apache/kafka/common/Cluster.java | 4 +- .../org/apache/kafka/common/MetricName.java | 4 +- .../apache/kafka/common/PartitionInfo.java | 2 +- .../apache/kafka/common/config/ConfigDef.java | 82 +- ...NotEnoughReplicasAfterAppendException.java | 33 +- .../errors/NotEnoughReplicasException.java | 28 +- .../message/KafkaLZ4BlockInputStream.java | 233 -- .../message/KafkaLZ4BlockOutputStream.java | 387 --- .../kafka/common/metrics/JmxReporter.java | 57 +- .../apache/kafka/common/metrics/Sensor.java | 12 +- .../kafka/common/metrics/stats/Rate.java | 2 +- .../kafka/common/network/NetworkReceive.java | 2 +- .../apache/kafka/common/network/Selector.java | 9 +- .../apache/kafka/common/protocol/ApiKeys.java | 12 +- .../kafka/common/protocol/Protocol.java | 590 +++-- .../kafka/common/protocol/types/Struct.java | 2 +- .../common/record/ByteBufferOutputStream.java | 2 +- .../kafka/common/record/Compressor.java | 29 +- .../record/KafkaLZ4BlockInputStream.java | 234 ++ .../record/KafkaLZ4BlockOutputStream.java | 392 +++ .../kafka/common/record/MemoryRecords.java | 36 +- .../requests/ConsumerMetadataRequest.java | 2 +- .../requests/ConsumerMetadataResponse.java | 2 +- .../kafka/common/requests/FetchRequest.java | 2 +- .../kafka/common/requests/FetchResponse.java | 2 +- .../common/requests/HeartbeatRequest.java | 2 +- .../common/requests/HeartbeatResponse.java | 2 +- .../common/requests/JoinGroupRequest.java | 2 +- .../common/requests/JoinGroupResponse.java | 2 +- .../common/requests/ListOffsetRequest.java | 4 +- .../common/requests/ListOffsetResponse.java | 2 +- .../common/requests/MetadataRequest.java | 2 +- .../common/requests/MetadataResponse.java | 8 +- .../common/requests/OffsetCommitRequest.java | 4 +- .../common/requests/OffsetCommitResponse.java | 2 +- .../common/requests/OffsetFetchRequest.java | 4 +- .../common/requests/OffsetFetchResponse.java | 2 +- .../kafka/common/requests/ProduceRequest.java | 2 +- .../common/requests/ProduceResponse.java | 2 +- .../kafka/common/requests/RequestHeader.java | 8 +- .../kafka/common/requests/ResponseHeader.java | 4 +- .../org/apache/kafka/common/utils/Crc32.java | 2338 ++--------------- .../org/apache/kafka/common/utils/Utils.java | 44 +- .../utils => clients}/ClientUtilsTest.java | 2 +- .../org/apache/kafka/clients/MockClient.java | 18 +- .../kafka/clients/NetworkClientTest.java | 17 +- .../clients/consumer/MockConsumerTest.java | 16 + .../internals/SubscriptionStateTest.java | 16 + .../clients/producer/BufferPoolTest.java | 14 +- .../kafka/clients/producer/MetadataTest.java | 8 +- .../clients/producer/MockProducerTest.java | 1 + .../clients/producer/PartitionerTest.java | 2 +- .../producer/RecordAccumulatorTest.java | 3 +- .../clients/producer/RecordSendTest.java | 5 +- .../kafka/clients/producer/SenderTest.java | 6 +- .../common/config/AbstractConfigTest.java | 106 +- .../kafka/common/config/ConfigDefTest.java | 5 +- .../common/metrics/FakeMetricsReporter.java | 32 + .../kafka/common/metrics/MetricsTest.java | 4 +- .../common/metrics/stats/HistogramTest.java | 1 - .../kafka/common/network/SelectorTest.java | 1 - .../types/ProtocolSerializationTest.java | 14 +- .../common/record/MemoryRecordsTest.java | 2 +- .../kafka/common/record/RecordTest.java | 4 +- .../common/requests/RequestResponseTest.java | 23 +- .../serialization/SerializationTest.java | 4 +- .../apache/kafka/common/utils/CrcTest.java | 8 +- .../apache/kafka/test/Microbenchmarks.java | 1 - .../java/org/apache/kafka/test/TestUtils.java | 16 +- .../consumer/ConsumerRebalanceListener.java | 3 - .../kafka/message/CompressionFactory.scala | 2 +- .../scala/kafka/tools/KafkaMigrationTool.java | 12 +- core/src/main/scala/kafka/utils/Crc32.java | 40 +- .../kafka/examples/SimpleConsumerDemo.java | 9 +- 97 files changed, 1972 insertions(+), 3468 deletions(-) create mode 100644 checkstyle/checkstyle.xml create mode 100644 checkstyle/import-control.xml rename clients/src/main/java/org/apache/kafka/{common/utils => clients}/ClientUtils.java (86%) rename clients/src/main/java/org/apache/kafka/clients/{producer/internals => }/Metadata.java (99%) delete mode 100644 clients/src/main/java/org/apache/kafka/common/message/KafkaLZ4BlockInputStream.java delete mode 100644 clients/src/main/java/org/apache/kafka/common/message/KafkaLZ4BlockOutputStream.java create mode 100644 clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.java create mode 100644 clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream.java rename clients/src/test/java/org/apache/kafka/{common/utils => clients}/ClientUtilsTest.java (97%) create mode 100644 clients/src/test/java/org/apache/kafka/common/metrics/FakeMetricsReporter.java diff --git a/README.md b/README.md index 9bdcf7052434d..784daaf6c17c0 100644 --- a/README.md +++ b/README.md @@ -98,6 +98,9 @@ Please note for this to work you should create/update `~/.gradle/gradle.properti ### Determining how transitive dependencies are added ### ./gradlew core:dependencies --configuration runtime + +### Running checkstyle on the java code ### + ./gradlew checkstyleMain checkstyleTest ### Running in Vagrant ### diff --git a/build.gradle b/build.gradle index 68443725868c4..0f0fe60a74542 100644 --- a/build.gradle +++ b/build.gradle @@ -345,6 +345,7 @@ project(':examples') { } project(':clients') { + apply plugin: 'checkstyle' archivesBaseName = "kafka-clients" dependencies { @@ -379,4 +380,9 @@ project(':clients') { artifacts { archives testJar } + + checkstyle { + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + } + test.dependsOn('checkstyleMain', 'checkstyleTest') } diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml new file mode 100644 index 0000000000000..a215ff36e9252 --- /dev/null +++ b/checkstyle/checkstyle.xml @@ -0,0 +1,83 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml new file mode 100644 index 0000000000000..cca4b38ec7660 --- /dev/null +++ b/checkstyle/import-control.xml @@ -0,0 +1,100 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/utils/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java similarity index 86% rename from clients/src/main/java/org/apache/kafka/common/utils/ClientUtils.java rename to clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index b987e7f0434c6..d0da5d7a08a0c 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -10,13 +10,12 @@ * 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.utils; +package org.apache.kafka.clients; import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.List; -import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.config.ConfigException; import static org.apache.kafka.common.utils.Utils.getHost; @@ -31,19 +30,19 @@ public static List parseAndValidateAddresses(List url String host = getHost(url); Integer port = getPort(url); if (host == null || port == null) - throw new ConfigException("Invalid url in " + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + ": " + url); + throw new ConfigException("Invalid url in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + ": " + url); try { InetSocketAddress address = new InetSocketAddress(host, port); if (address.isUnresolved()) - throw new ConfigException("DNS resolution failed for url in " + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + ": " + url); + throw new ConfigException("DNS resolution failed for url in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + ": " + url); addresses.add(address); } catch (NumberFormatException e) { - throw new ConfigException("Invalid port in " + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG + ": " + url); + throw new ConfigException("Invalid port in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG + ": " + url); } } } if (addresses.size() < 1) - throw new ConfigException("No bootstrap urls given in " + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG); + throw new ConfigException("No bootstrap urls given in " + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG); return addresses; } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java index 574287d77f7d4..da76cc257b4cf 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java @@ -69,8 +69,7 @@ public long connectionDelay(int node, long now) { long timeWaited = now - state.lastConnectAttemptMs; if (state.state == ConnectionState.DISCONNECTED) { return Math.max(this.reconnectBackoffMs - timeWaited, 0); - } - else { + } else { // When connecting or connected, we should be able to delay indefinitely since other events (connection or // data acked) will cause a wakeup once data can be sent. return Long.MAX_VALUE; @@ -109,7 +108,8 @@ public boolean isConnecting(int node) { * @param node The node we have connected to */ public void connected(int node) { - nodeState(node).state = ConnectionState.CONNECTED; + NodeConnectionState nodeState = nodeState(node); + nodeState.state = ConnectionState.CONNECTED; } /** @@ -117,7 +117,8 @@ public void connected(int node) { * @param node The node we have disconnected from */ public void disconnected(int node) { - nodeState(node).state = ConnectionState.DISCONNECTED; + NodeConnectionState nodeState = nodeState(node); + nodeState.state = ConnectionState.DISCONNECTED; } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java similarity index 99% rename from clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java rename to clients/src/main/java/org/apache/kafka/clients/Metadata.java index 3aff6242d9d74..b8cdd145bfcc6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -10,7 +10,7 @@ * 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.clients.producer.internals; +package org.apache.kafka.clients; import java.util.HashSet; import java.util.Set; diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 5950191b240f3..fef90a03ed04d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -19,7 +19,6 @@ import java.util.Random; import java.util.Set; -import org.apache.kafka.clients.producer.internals.Metadata; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.network.NetworkReceive; @@ -199,7 +198,7 @@ public List poll(long timeout, long now) { // should we update our metadata? long timeToNextMetadataUpdate = metadata.timeToNextUpdate(now); long timeToNextReconnectAttempt = Math.max(this.lastNoNodeAvailableMs + metadata.refreshBackoff() - now, 0); - long waitForMetadataFetch = (this.metadataFetchInProgress ? Integer.MAX_VALUE : 0); + long waitForMetadataFetch = this.metadataFetchInProgress ? Integer.MAX_VALUE : 0; // if there is no node available to connect, back off refreshing metadata long metadataTimeout = Math.max(Math.max(timeToNextMetadataUpdate, timeToNextReconnectAttempt), waitForMetadataFetch); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java b/clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java index 072cc2e6f92db..7548a9ba0163a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java @@ -1,3 +1,15 @@ +/** + * 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.clients.consumer; public enum CommitType { 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 6d4ff7cd2a283..5fb21001abd77 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 @@ -28,7 +28,7 @@ * The consumer configuration keys */ public class ConsumerConfig extends AbstractConfig { - private static final ConfigDef config; + private static final ConfigDef CONFIG; /* * NOTE: DO NOT CHANGE EITHER CONFIG STRINGS OR THEIR JAVA VARIABLE NAMES AS @@ -154,7 +154,7 @@ public class ConsumerConfig extends AbstractConfig { private static final String VALUE_DESERIALIZER_CLASS_DOC = "Deserializer class for value that implements the Deserializer interface."; static { - config = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, + CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, CommonClientConfigs.BOOSTRAP_SERVERS_DOC) @@ -277,11 +277,11 @@ public class ConsumerConfig extends AbstractConfig { } ConsumerConfig(Map props) { - super(config, props); + super(CONFIG, props); } public static void main(String[] args) { - System.out.println(config.toHtmlTable()); + System.out.println(CONFIG.toHtmlTable()); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java index 416d703c3f59a..305ec8ee2b942 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java @@ -26,11 +26,11 @@ * particular topic. There is one for every topic returned by a * {@link Consumer#poll(long)} operation. */ -public class ConsumerRecords implements Iterable> { +public class ConsumerRecords implements Iterable> { - private final Map>> records; + private final Map>> records; - public ConsumerRecords(Map>> records) { + public ConsumerRecords(Map>> records) { this.records = records; } @@ -39,8 +39,8 @@ public ConsumerRecords(Map>> records) { * * @param partition The partition to get records for */ - public Iterable> records(TopicPartition partition) { - List> recs = this.records.get(partition); + public Iterable> records(TopicPartition partition) { + List> recs = this.records.get(partition); if (recs == null) return Collections.emptyList(); else @@ -50,20 +50,20 @@ public Iterable> records(TopicPartition partition) { /** * Get just the records for the given topic */ - public Iterable> records(String topic) { + public Iterable> records(String topic) { if (topic == null) throw new IllegalArgumentException("Topic must be non-null."); - List>> recs = new ArrayList>>(); - for (Map.Entry>> entry : records.entrySet()) { + List>> recs = new ArrayList>>(); + for (Map.Entry>> entry : records.entrySet()) { if (entry.getKey().equals(topic)) recs.add(entry.getValue()); } - return new ConcatenatedIterable(recs); + return new ConcatenatedIterable(recs); } @Override - public Iterator> iterator() { - return new ConcatenatedIterable(records.values()).iterator(); + public Iterator> iterator() { + return new ConcatenatedIterable(records.values()).iterator(); } /** @@ -71,26 +71,26 @@ public Iterator> iterator() { */ public int count() { int count = 0; - for(List> recs: this.records.values()) + for (List> recs: this.records.values()) count += recs.size(); return count; } - private static class ConcatenatedIterable implements Iterable> { + private static class ConcatenatedIterable implements Iterable> { - private final Iterable>> iterables; + private final Iterable>> iterables; - public ConcatenatedIterable(Iterable>> iterables) { + public ConcatenatedIterable(Iterable>> iterables) { this.iterables = iterables; } @Override - public Iterator> iterator() { - return new AbstractIterator>() { - Iterator>> iters = iterables.iterator(); - Iterator> current; + public Iterator> iterator() { + return new AbstractIterator>() { + Iterator>> iters = iterables.iterator(); + Iterator> current; - public ConsumerRecord makeNext() { + public ConsumerRecord makeNext() { if (current == null || !current.hasNext()) { if (iters.hasNext()) current = iters.next().iterator(); 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 300c551f3d21a..09a6f11163ecb 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 @@ -30,13 +30,13 @@ import org.apache.kafka.clients.ClientRequest; import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.ClientUtils; import org.apache.kafka.clients.ConnectionState; +import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.clients.RequestCompletionHandler; import org.apache.kafka.clients.consumer.internals.Heartbeat; import org.apache.kafka.clients.consumer.internals.SubscriptionState; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.clients.producer.internals.Metadata; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Metric; @@ -78,7 +78,6 @@ import org.apache.kafka.common.requests.OffsetFetchResponse; import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.requests.RequestSend; -import org.apache.kafka.common.utils.ClientUtils; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; @@ -380,7 +379,7 @@ public class KafkaConsumer implements Consumer { private static final Logger log = LoggerFactory.getLogger(KafkaConsumer.class); private static final long EARLIEST_OFFSET_TIMESTAMP = -2L; private static final long LATEST_OFFSET_TIMESTAMP = -1L; - private static final AtomicInteger consumerAutoId = new AtomicInteger(1); + private static final AtomicInteger CONSUMER_CLIENT_ID_SEQUENCE = new AtomicInteger(1); private final Time time; private final ConsumerMetrics metrics; @@ -547,15 +546,15 @@ private KafkaConsumer(ConsumerConfig config, TimeUnit.MILLISECONDS); String clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG); String jmxPrefix = "kafka.consumer"; - if(clientId .length() <= 0) - clientId = "consumer-" + consumerAutoId.getAndIncrement(); + if (clientId.length() <= 0) + clientId = "consumer-" + CONSUMER_CLIENT_ID_SEQUENCE.getAndIncrement(); List reporters = config.getConfiguredInstances(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG, MetricsReporter.class); reporters.add(new JmxReporter(jmxPrefix)); Metrics metrics = new Metrics(metricConfig, reporters, time); this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG); this.metadata = new Metadata(retryBackoffMs, config.getLong(ConsumerConfig.METADATA_MAX_AGE_CONFIG)); - List addresses = ClientUtils.parseAndValidateAddresses(config.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); + List addresses = ClientUtils.parseAndValidateAddresses(config.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); this.metadata.update(Cluster.bootstrap(addresses), 0); String metricsGroup = "consumer"; @@ -1554,23 +1553,31 @@ public ConsumerMetrics(Metrics metrics, String metricsGroup, Map "The maximum lag for any partition in this window", tags), new Max()); + Measurable numParts = + new Measurable() { + public double measure(MetricConfig config, long now) { + return subscriptions.assignedPartitions().size(); + } + }; metrics.addMetric(new MetricName("assigned-partitions", metricsGroup, "The number of partitions currently assigned to this consumer", - tags), new Measurable() { - public double measure(MetricConfig config, long now) { - return subscriptions.assignedPartitions().size(); - } - }); - + tags), + numParts); + + + Measurable lastHeartbeat = + new Measurable() { + public double measure(MetricConfig config, long now) { + return TimeUnit.SECONDS.convert(now - heartbeat.lastHeartbeatSend(), TimeUnit.MILLISECONDS); + } + }; metrics.addMetric(new MetricName("last-heartbeat-seconds-ago", metricsGroup, "The number of seconds since the last controller heartbeat", - tags), new Measurable() { - public double measure(MetricConfig config, long now) { - return TimeUnit.SECONDS.convert(now - heartbeat.lastHeartbeatSend(), TimeUnit.MILLISECONDS); - } - }); + tags), + + lastHeartbeat); } public void recordTopicFetchMetrics(String topic, int bytes, int records) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java index d9483ecf6ae4a..ee0751e494912 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java @@ -1,3 +1,15 @@ +/** + * 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.clients.consumer.internals; /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.java index 7e57a39690d9b..c06ab3a372a54 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.java @@ -22,9 +22,9 @@ public class NoOpConsumerRebalanceCallback implements ConsumerRebalanceCallback { @Override - public void onPartitionsAssigned(Consumer consumer, Collection partitions) {} + public void onPartitionsAssigned(Consumer consumer, Collection partitions) {} @Override - public void onPartitionsRevoked(Consumer consumer, Collection partitions) {} + public void onPartitionsRevoked(Consumer consumer, Collection partitions) {} } 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 71ce20db955bd..d41d3068c11d4 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 @@ -1,3 +1,15 @@ +/** + * 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.clients.consumer.internals; import java.util.HashMap; @@ -58,8 +70,8 @@ public void unsubscribe(String topic) { throw new IllegalStateException("Topic " + topic + " was never subscribed to."); this.subscribedTopics.remove(topic); this.needsPartitionAssignment = true; - for(TopicPartition tp: assignedPartitions()) - if(topic.equals(tp.topic())) + for (TopicPartition tp: assignedPartitions()) + if (topic.equals(tp.topic())) clearPartition(tp); } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index ebc4c5315fb94..1fd6917c8a513 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -19,8 +19,9 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.kafka.clients.ClientUtils; +import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.NetworkClient; -import org.apache.kafka.clients.producer.internals.Metadata; import org.apache.kafka.clients.producer.internals.Partitioner; import org.apache.kafka.clients.producer.internals.RecordAccumulator; import org.apache.kafka.clients.producer.internals.Sender; @@ -45,7 +46,6 @@ import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.Records; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.utils.ClientUtils; import org.apache.kafka.common.utils.KafkaThread; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; @@ -60,9 +60,10 @@ * The producer manages a single background thread that does I/O as well as a TCP connection to each of the brokers it * needs to communicate with. Failure to close the producer after use will leak these resources. */ -public class KafkaProducer implements Producer { +public class KafkaProducer implements Producer { private static final Logger log = LoggerFactory.getLogger(KafkaProducer.class); + private static final AtomicInteger PRODUCER_CLIENT_ID_SEQUENCE = new AtomicInteger(1); private final Partitioner partitioner; private final int maxRequestSize; @@ -79,7 +80,6 @@ public class KafkaProducer implements Producer { private final Serializer keySerializer; private final Serializer valueSerializer; private final ProducerConfig producerConfig; - private static final AtomicInteger producerAutoId = new AtomicInteger(1); /** * A producer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings @@ -154,6 +154,7 @@ private static Properties addSerializerToConfig(Properties properties, return newProperties; } + @SuppressWarnings("unchecked") private KafkaProducer(ProducerConfig config, Serializer keySerializer, Serializer valueSerializer) { log.trace("Starting the Kafka producer"); this.producerConfig = config; @@ -162,8 +163,8 @@ private KafkaProducer(ProducerConfig config, Serializer keySerializer, Serial .timeWindow(config.getLong(ProducerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG), TimeUnit.MILLISECONDS); String clientId = config.getString(ProducerConfig.CLIENT_ID_CONFIG); - if(clientId.length() <= 0) - clientId = "producer-" + producerAutoId.getAndIncrement(); + if (clientId.length() <= 0) + clientId = "producer-" + PRODUCER_CLIENT_ID_SEQUENCE.getAndIncrement(); String jmxPrefix = "kafka.producer"; List reporters = config.getConfiguredInstances(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG, MetricsReporter.class); @@ -216,16 +217,16 @@ private KafkaProducer(ProducerConfig config, Serializer keySerializer, Serial this.keySerializer = config.getConfiguredInstance(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, Serializer.class); this.keySerializer.configure(config.originals(), true); - } - else + } else { this.keySerializer = keySerializer; + } if (valueSerializer == null) { this.valueSerializer = config.getConfiguredInstance(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, Serializer.class); this.valueSerializer.configure(config.originals(), false); - } - else + } else { this.valueSerializer = valueSerializer; + } config.logUnused(); log.debug("Kafka producer started"); @@ -244,7 +245,7 @@ private static int parseAcks(String acksString) { * @param record The record to be sent */ @Override - public Future send(ProducerRecord record) { + public Future send(ProducerRecord record) { return send(record, null); } @@ -309,7 +310,7 @@ public Future send(ProducerRecord record) { * indicates no callback) */ @Override - public Future send(ProducerRecord record, Callback callback) { + public Future send(ProducerRecord record, Callback callback) { try { // first make sure the metadata for the topic is available waitOnMetadata(record.topic(), this.metadataFetchTimeoutMs); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java index 6b2471f878b7d..17fe541588d46 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java @@ -31,7 +31,7 @@ * @see KafkaProducer * @see MockProducer */ -public interface Producer extends Closeable { +public interface Producer extends Closeable { /** * Send the given record asynchronously and return a future which will eventually contain the response information. @@ -39,12 +39,12 @@ public interface Producer extends Closeable { * @param record The record to send * @return A future which will eventually contain the response information */ - public Future send(ProducerRecord record); + public Future send(ProducerRecord record); /** * Send a record and invoke the given callback when the record has been acknowledged by the server */ - public Future send(ProducerRecord record, Callback callback); + public Future send(ProducerRecord record, Callback callback); /** * Get a list of partitions for the given topic for custom partition assignment. The partition metadata will change diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 9a43d66837629..122375c473bf7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -35,7 +35,7 @@ public class ProducerConfig extends AbstractConfig { * CHANGE WILL BREAK USER CODE. */ - private static final ConfigDef config; + private static final ConfigDef CONFIG; /** bootstrap.servers */ public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; @@ -167,13 +167,13 @@ public class ProducerConfig extends AbstractConfig { private static final String VALUE_SERIALIZER_CLASS_DOC = "Serializer class for value that implements the Serializer interface."; static { - config = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, CommonClientConfigs.BOOSTRAP_SERVERS_DOC) + CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, CommonClientConfigs.BOOSTRAP_SERVERS_DOC) .define(BUFFER_MEMORY_CONFIG, Type.LONG, 32 * 1024 * 1024L, atLeast(0L), Importance.HIGH, BUFFER_MEMORY_DOC) .define(RETRIES_CONFIG, Type.INT, 0, between(0, Integer.MAX_VALUE), Importance.HIGH, RETRIES_DOC) .define(ACKS_CONFIG, Type.STRING, "1", - in("all","-1", "0", "1"), + in("all", "-1", "0", "1"), Importance.HIGH, ACKS_DOC) .define(COMPRESSION_TYPE_CONFIG, Type.STRING, "none", Importance.HIGH, COMPRESSION_TYPE_DOC) @@ -218,11 +218,11 @@ public class ProducerConfig extends AbstractConfig { } ProducerConfig(Map props) { - super(config, props); + super(CONFIG, props); } public static void main(String[] args) { - System.out.println(config.toHtmlTable()); + System.out.println(CONFIG.toHtmlTable()); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java index 8d4156d17e949..4cb1e50d6c4ed 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java @@ -80,11 +80,11 @@ public BufferPool(long memory, int poolableSize, boolean blockOnExhaustion, Metr this.time = time; this.waitTime = this.metrics.sensor("bufferpool-wait-time"); MetricName metricName = new MetricName("bufferpool-wait-ratio", - metricGrpName, - "The fraction of time an appender waits for space allocation.", - metricTags); + metricGrpName, + "The fraction of time an appender waits for space allocation.", + metricTags); this.waitTime.add(metricName, new Rate(TimeUnit.NANOSECONDS)); - } + } /** * Allocate a buffer of the given size. This method blocks if there is not enough memory and the buffer pool diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java index b70ece71fdbd1..8e5855d9f41e2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java @@ -19,7 +19,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.TopicPartition; diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index 50889e4ce4b6c..ecfe2144d778a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -102,26 +102,27 @@ public RecordAccumulator(int batchSize, private void registerMetrics(Metrics metrics, String metricGrpName, Map metricTags) { MetricName metricName = new MetricName("waiting-threads", metricGrpName, "The number of user threads blocked waiting for buffer memory to enqueue their records", metricTags); - metrics.addMetric(metricName, - new Measurable() { - public double measure(MetricConfig config, long now) { - return free.queued(); - } - }); + Measurable waitingThreads = new Measurable() { + public double measure(MetricConfig config, long now) { + return free.queued(); + } + }; + metrics.addMetric(metricName, waitingThreads); + metricName = new MetricName("buffer-total-bytes", metricGrpName, "The maximum amount of buffer memory the client can use (whether or not it is currently used).", metricTags); - metrics.addMetric(metricName, - new Measurable() { - public double measure(MetricConfig config, long now) { - return free.totalMemory(); - } - }); + Measurable totalBytes = new Measurable() { + public double measure(MetricConfig config, long now) { + return free.totalMemory(); + } + }; + metrics.addMetric(metricName, totalBytes); metricName = new MetricName("buffer-available-bytes", metricGrpName, "The total amount of buffer memory that is not being used (either unallocated or in the free list).", metricTags); - metrics.addMetric(metricName, - new Measurable() { - public double measure(MetricConfig config, long now) { - return free.availableMemory(); - } - }); + Measurable availableBytes = new Measurable() { + public double measure(MetricConfig config, long now) { + return free.availableMemory(); + } + }; + metrics.addMetric(metricName, availableBytes); } /** @@ -228,8 +229,7 @@ public ReadyCheckResult ready(Cluster cluster, long nowMs) { boolean sendable = full || expired || exhausted || closed; if (sendable && !backingOff) { readyNodes.add(leader); - } - else { + } else { // Note that this results in a conservative estimate since an un-sendable partition may have // a leader that will later be found to have sendable data. However, this is good enough // since we'll just wake up and then sleep again for the remaining time. diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 8726809f8ada6..ed9c63a6679e3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -23,6 +23,7 @@ import org.apache.kafka.clients.ClientRequest; import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.KafkaClient; +import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.RequestCompletionHandler; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; diff --git a/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java b/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java index 689bae9e6ba69..13f4d5958052a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java +++ b/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java @@ -45,12 +45,12 @@ public static void main(String[] args) throws Exception { } props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - KafkaProducer producer = new KafkaProducer(props); + KafkaProducer producer = new KafkaProducer(props); /* setup perf test */ byte[] payload = new byte[recordSize]; Arrays.fill(payload, (byte) 1); - ProducerRecord record = new ProducerRecord(topicName, payload); + ProducerRecord record = new ProducerRecord(topicName, payload); long sleepTime = NS_PER_SEC / throughput; long sleepDeficitNs = 0; Stats stats = new Stats(numRecords, 5000); @@ -66,8 +66,8 @@ public static void main(String[] args) throws Exception { * and then make up the whole deficit in one longer sleep. */ if (throughput > 0) { - float elapsed = (sendStart - start)/1000.f; - if (elapsed > 0 && i/elapsed > throughput) { + float elapsed = (sendStart - start) / 1000.f; + if (elapsed > 0 && i / elapsed > throughput) { sleepDeficitNs += sleepTime; if (sleepDeficitNs >= MIN_SLEEP_NS) { long sleepMs = sleepDeficitNs / 1000000; diff --git a/clients/src/main/java/org/apache/kafka/common/Cluster.java b/clients/src/main/java/org/apache/kafka/common/Cluster.java index d7ccbcd91e657..8fcd2916f1d55 100644 --- a/clients/src/main/java/org/apache/kafka/common/Cluster.java +++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java @@ -40,8 +40,8 @@ public Cluster(Collection nodes, Collection partitions) { this.nodes = Collections.unmodifiableList(copy); this.nodesById = new HashMap(); - for(Node node: nodes) - this.nodesById.put(node.id(), node); + for (Node node: nodes) + this.nodesById.put(node.id(), node); // index the partitions by topic/partition for quick lookup this.partitionsByTopicPartition = new HashMap(partitions.size()); diff --git a/clients/src/main/java/org/apache/kafka/common/MetricName.java b/clients/src/main/java/org/apache/kafka/common/MetricName.java index 7e977e94a8e0b..04b4a09badd51 100644 --- a/clients/src/main/java/org/apache/kafka/common/MetricName.java +++ b/clients/src/main/java/org/apache/kafka/common/MetricName.java @@ -90,8 +90,8 @@ private static Map getTags(String... keyValue) { throw new IllegalArgumentException("keyValue needs to be specified in paris"); Map tags = new HashMap(); - for (int i=0; i<(keyValue.length / 2); i++) - tags.put(keyValue[i], keyValue[i+1]); + for (int i = 0; i < keyValue.length / 2; i++) + tags.put(keyValue[i], keyValue[i + 1]); return tags; } diff --git a/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java index 28562f9019e1a..321da8afc7394 100644 --- a/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java +++ b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java @@ -72,7 +72,7 @@ public String toString() { return String.format("Partition(topic = %s, partition = %d, leader = %s, replicas = %s, isr = %s", topic, partition, - leader == null? "none" : leader.id(), + leader == null ? "none" : leader.id(), fmtNodeIds(replicas), fmtNodeIds(inSyncReplicas)); } diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java index 38ce10b312573..8523333a9b39a 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java @@ -53,6 +53,7 @@ public class ConfigDef { /** * Returns unmodifiable set of properties names defined in this {@linkplain ConfigDef} + * * @return new unmodifiable {@link Set} instance containing the keys */ public Set names() { @@ -61,6 +62,7 @@ public Set names() { /** * Define a new configuration + * * @param name The name of the config parameter * @param type The type of the config * @param defaultValue The default value to use if this config isn't present @@ -69,16 +71,23 @@ public Set names() { * @param documentation The documentation string for the config * @return This ConfigDef so you can chain calls */ - public ConfigDef define(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation) { + public ConfigDef define(String name, + Type type, + Object defaultValue, + Validator validator, + Importance importance, + String documentation) { if (configKeys.containsKey(name)) throw new ConfigException("Configuration " + name + " is defined twice."); - Object parsedDefault = defaultValue == NO_DEFAULT_VALUE ? NO_DEFAULT_VALUE : parseType(name, defaultValue, type); + Object parsedDefault = defaultValue == NO_DEFAULT_VALUE ? NO_DEFAULT_VALUE + : parseType(name, defaultValue, type); configKeys.put(name, new ConfigKey(name, type, parsedDefault, validator, importance, documentation)); return this; } /** * Define a new configuration with no special validation logic + * * @param name The name of the config parameter * @param type The type of the config * @param defaultValue The default value to use if this config isn't present @@ -92,6 +101,7 @@ public ConfigDef define(String name, Type type, Object defaultValue, Importance /** * Define a required parameter with no default value + * * @param name The name of the config parameter * @param type The type of the config * @param validator A validator to use in checking the correctness of the config @@ -105,6 +115,7 @@ public ConfigDef define(String name, Type type, Validator validator, Importance /** * Define a required parameter with no default value and no special validation logic + * * @param name The name of the config parameter * @param type The type of the config * @param importance The importance of this config: is this something you will likely need to change. @@ -120,6 +131,7 @@ public ConfigDef define(String name, Type type, Importance importance, String do * that the keys of the map are strings, but the values can either be strings or they may already be of the * appropriate type (int, string, etc). This will work equally well with either java.util.Properties instances or a * programmatically constructed map. + * * @param props The configs to parse and validate * @return Parsed and validated configs. The key will be the config name and the value will be the value parsed into * the appropriate type (int, string, etc) @@ -132,7 +144,8 @@ public Map parse(Map props) { if (props.containsKey(key.name)) value = parseType(key.name, props.get(key.name), key.type); else if (key.defaultValue == NO_DEFAULT_VALUE) - throw new ConfigException("Missing required configuration \"" + key.name + "\" which has no default value."); + throw new ConfigException("Missing required configuration \"" + key.name + + "\" which has no default value."); else value = key.defaultValue; if (key.validator != null) @@ -144,6 +157,7 @@ else if (key.defaultValue == NO_DEFAULT_VALUE) /** * Parse a value according to its expected type. + * * @param name The config name * @param value The config value * @param type The expected type @@ -157,14 +171,13 @@ private Object parseType(String name, Object value, Type type) { switch (type) { case BOOLEAN: if (value instanceof String) { - if (trimmed.equalsIgnoreCase("true")) - return true; - else if (trimmed.equalsIgnoreCase("false")) - return false; - else - throw new ConfigException(name, value, "Expected value to be either true or false"); - } - else if (value instanceof Boolean) + if (trimmed.equalsIgnoreCase("true")) + return true; + else if (trimmed.equalsIgnoreCase("false")) + return false; + else + throw new ConfigException(name, value, "Expected value to be either true or false"); + } else if (value instanceof Boolean) return value; else throw new ConfigException(name, value, "Expected value to be either true or false"); @@ -172,7 +185,8 @@ else if (value instanceof Boolean) if (value instanceof String) return trimmed; else - throw new ConfigException(name, value, "Expected value to be a string, but it was a " + value.getClass().getName()); + throw new ConfigException(name, value, "Expected value to be a string, but it was a " + + value.getClass().getName()); case INT: if (value instanceof Integer) { return (Integer) value; @@ -256,6 +270,7 @@ private Range(Number min, Number max) { /** * A numeric range that checks only the lower bound + * * @param min The minimum acceptable value */ public static Range atLeast(Number min) { @@ -287,32 +302,30 @@ else if (max == null) } } - public static class ValidString implements Validator { - List validStrings; + public static class ValidString implements Validator { + private final List validStrings; - private ValidString(List validStrings) { - this.validStrings = validStrings; - } + private ValidString(List validStrings) { + this.validStrings = validStrings; + } - public static ValidString in(String... validStrings) { - return new ValidString(Arrays.asList(validStrings)); - } + public static ValidString in(String... validStrings) { + return new ValidString(Arrays.asList(validStrings)); + } - @Override - public void ensureValid(String name, Object o) { - String s = (String) o; - if (!validStrings.contains(s)) { - throw new ConfigException(name,o,"String must be one of: " + Utils.join(validStrings, ", ")); - } + @Override + public void ensureValid(String name, Object o) { + String s = (String) o; + if (!validStrings.contains(s)) + throw new ConfigException(name, o, "String must be one of: " + Utils.join(validStrings, ", ")); + } - } + public String toString() { + return "[" + Utils.join(validStrings, ", ") + "]"; + } - public String toString() { - return "[" + Utils.join(validStrings, ", ") + "]"; } - } - private static class ConfigKey { public final String name; public final Type type; @@ -321,7 +334,12 @@ private static class ConfigKey { public final Validator validator; public final Importance importance; - public ConfigKey(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation) { + public ConfigKey(String name, + Type type, + Object defaultValue, + Validator validator, + Importance importance, + String documentation) { super(); this.name = name; this.type = type; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java b/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java index 75c80a97e4308..a6107b818947d 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java @@ -1,4 +1,4 @@ -/* +/** * 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. @@ -17,27 +17,26 @@ package org.apache.kafka.common.errors; /** - * Number of insync replicas for the partition is lower than min.insync.replicas - * This exception is raised when the low ISR size is discovered *after* the message - * was already appended to the log. Producer retries will cause duplicates. + * Number of insync replicas for the partition is lower than min.insync.replicas This exception is raised when the low + * ISR size is discovered *after* the message was already appended to the log. Producer retries will cause duplicates. */ public class NotEnoughReplicasAfterAppendException extends RetriableException { - private static final long serialVersionUID = 1L; + private static final long serialVersionUID = 1L; - public NotEnoughReplicasAfterAppendException() { - super(); - } + public NotEnoughReplicasAfterAppendException() { + super(); + } - public NotEnoughReplicasAfterAppendException(String message, Throwable cause) { - super(message,cause); - } + public NotEnoughReplicasAfterAppendException(String message, Throwable cause) { + super(message, cause); + } - public NotEnoughReplicasAfterAppendException(String message) { - super(message); - } + public NotEnoughReplicasAfterAppendException(String message) { + super(message); + } - public NotEnoughReplicasAfterAppendException(Throwable cause) { - super(cause); - } + public NotEnoughReplicasAfterAppendException(Throwable cause) { + super(cause); + } } diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasException.java b/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasException.java index 486d5155bbb1f..1573227acef53 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasException.java @@ -1,4 +1,4 @@ -/* +/** * 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. @@ -20,21 +20,21 @@ * Number of insync replicas for the partition is lower than min.insync.replicas */ public class NotEnoughReplicasException extends RetriableException { - private static final long serialVersionUID = 1L; + private static final long serialVersionUID = 1L; - public NotEnoughReplicasException() { - super(); - } + public NotEnoughReplicasException() { + super(); + } - public NotEnoughReplicasException(String message, Throwable cause) { - super(message, cause); - } + public NotEnoughReplicasException(String message, Throwable cause) { + super(message, cause); + } - public NotEnoughReplicasException(String message) { - super(message); - } + public NotEnoughReplicasException(String message) { + super(message); + } - public NotEnoughReplicasException(Throwable cause) { - super(cause); - } + public NotEnoughReplicasException(Throwable cause) { + super(cause); + } } diff --git a/clients/src/main/java/org/apache/kafka/common/message/KafkaLZ4BlockInputStream.java b/clients/src/main/java/org/apache/kafka/common/message/KafkaLZ4BlockInputStream.java deleted file mode 100644 index 5be72fef1f976..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/message/KafkaLZ4BlockInputStream.java +++ /dev/null @@ -1,233 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.common.message; - -import static org.apache.kafka.common.message.KafkaLZ4BlockOutputStream.LZ4_FRAME_INCOMPRESSIBLE_MASK; -import static org.apache.kafka.common.message.KafkaLZ4BlockOutputStream.LZ4_MAX_HEADER_LENGTH; -import static org.apache.kafka.common.message.KafkaLZ4BlockOutputStream.MAGIC; - -import java.io.FilterInputStream; -import java.io.IOException; -import java.io.InputStream; - -import org.apache.kafka.common.message.KafkaLZ4BlockOutputStream.BD; -import org.apache.kafka.common.message.KafkaLZ4BlockOutputStream.FLG; -import org.apache.kafka.common.utils.Utils; - -import net.jpountz.lz4.LZ4Exception; -import net.jpountz.lz4.LZ4Factory; -import net.jpountz.lz4.LZ4SafeDecompressor; -import net.jpountz.xxhash.XXHash32; -import net.jpountz.xxhash.XXHashFactory; - -/** - * A partial implementation of the v1.4.1 LZ4 Frame format. - * - * @see LZ4 Framing Format Spec - */ -public final class KafkaLZ4BlockInputStream extends FilterInputStream { - - public static final String PREMATURE_EOS = "Stream ended prematurely"; - public static final String NOT_SUPPORTED = "Stream unsupported"; - public static final String BLOCK_HASH_MISMATCH = "Block checksum mismatch"; - public static final String DESCRIPTOR_HASH_MISMATCH = "Stream frame descriptor corrupted"; - - private final LZ4SafeDecompressor decompressor; - private final XXHash32 checksum; - private final byte[] buffer; - private final byte[] compressedBuffer; - private final int maxBlockSize; - private FLG flg; - private BD bd; - private int bufferOffset; - private int bufferSize; - private boolean finished; - - /** - * Create a new {@link InputStream} that will decompress data using the LZ4 algorithm. - * - * @param in The stream to decompress - * @throws IOException - */ - public KafkaLZ4BlockInputStream(InputStream in) throws IOException { - super(in); - decompressor = LZ4Factory.fastestInstance().safeDecompressor(); - checksum = XXHashFactory.fastestInstance().hash32(); - readHeader(); - maxBlockSize = bd.getBlockMaximumSize(); - buffer = new byte[maxBlockSize]; - compressedBuffer = new byte[maxBlockSize]; - bufferOffset = 0; - bufferSize = 0; - finished = false; - } - - /** - * Reads the magic number and frame descriptor from the underlying {@link InputStream}. - * - * @throws IOException - */ - private void readHeader() throws IOException { - byte[] header = new byte[LZ4_MAX_HEADER_LENGTH]; - - // read first 6 bytes into buffer to check magic and FLG/BD descriptor flags - bufferOffset = 6; - if (in.read(header, 0, bufferOffset) != bufferOffset) { - throw new IOException(PREMATURE_EOS); - } - - if (MAGIC != Utils.readUnsignedIntLE(header, bufferOffset-6)) { - throw new IOException(NOT_SUPPORTED); - } - flg = FLG.fromByte(header[bufferOffset-2]); - bd = BD.fromByte(header[bufferOffset-1]); - // TODO read uncompressed content size, update flg.validate() - // TODO read dictionary id, update flg.validate() - - // check stream descriptor hash - byte hash = (byte) ((checksum.hash(header, 0, bufferOffset, 0) >> 8) & 0xFF); - header[bufferOffset++] = (byte) in.read(); - if (hash != header[bufferOffset-1]) { - throw new IOException(DESCRIPTOR_HASH_MISMATCH); - } - } - - /** - * Decompresses (if necessary) buffered data, optionally computes and validates a XXHash32 checksum, - * and writes the result to a buffer. - * - * @throws IOException - */ - private void readBlock() throws IOException { - int blockSize = Utils.readUnsignedIntLE(in); - - // Check for EndMark - if (blockSize == 0) { - finished = true; - // TODO implement content checksum, update flg.validate() - return; - } else if (blockSize > maxBlockSize) { - throw new IOException(String.format("Block size %s exceeded max: %s", blockSize, maxBlockSize)); - } - - boolean compressed = (blockSize & LZ4_FRAME_INCOMPRESSIBLE_MASK) == 0; - byte[] bufferToRead; - if (compressed) { - bufferToRead = compressedBuffer; - } else { - blockSize &= ~LZ4_FRAME_INCOMPRESSIBLE_MASK; - bufferToRead = buffer; - bufferSize = blockSize; - } - - if (in.read(bufferToRead, 0, blockSize) != blockSize) { - throw new IOException(PREMATURE_EOS); - } - - // verify checksum - if (flg.isBlockChecksumSet() && Utils.readUnsignedIntLE(in) != checksum.hash(bufferToRead, 0, blockSize, 0)) { - throw new IOException(BLOCK_HASH_MISMATCH); - } - - if (compressed) { - try { - bufferSize = decompressor.decompress(compressedBuffer, 0, blockSize, buffer, 0, maxBlockSize); - } catch (LZ4Exception e) { - throw new IOException(e); - } - } - - bufferOffset = 0; - } - - @Override - public int read() throws IOException { - if (finished) { - return -1; - } - if (available() == 0) { - readBlock(); - } - if (finished) { - return -1; - } - int value = buffer[bufferOffset++] & 0xFF; - - return value; - } - - @Override - public int read(byte b[], int off, int len) throws IOException { - net.jpountz.util.Utils.checkRange(b, off, len); - if (finished) { - return -1; - } - if (available() == 0) { - readBlock(); - } - if (finished) { - return -1; - } - len = Math.min(len, available()); - System.arraycopy(buffer, bufferOffset, b, off, len); - bufferOffset += len; - return len; - } - - @Override - public long skip(long n) throws IOException { - if (finished) { - return 0; - } - if (available() == 0) { - readBlock(); - } - if (finished) { - return 0; - } - n = Math.min(n, available()); - bufferOffset += n; - return n; - } - - @Override - public int available() throws IOException { - return bufferSize - bufferOffset; - } - - @Override - public void close() throws IOException { - in.close(); - } - - @Override - public synchronized void mark(int readlimit) { - throw new RuntimeException("mark not supported"); - } - - @Override - public synchronized void reset() throws IOException { - throw new RuntimeException("reset not supported"); - } - - @Override - public boolean markSupported() { - return false; - } - -} diff --git a/clients/src/main/java/org/apache/kafka/common/message/KafkaLZ4BlockOutputStream.java b/clients/src/main/java/org/apache/kafka/common/message/KafkaLZ4BlockOutputStream.java deleted file mode 100644 index e5b9e433e14ef..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/message/KafkaLZ4BlockOutputStream.java +++ /dev/null @@ -1,387 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.common.message; - -import java.io.FilterOutputStream; -import java.io.IOException; -import java.io.OutputStream; - -import org.apache.kafka.common.utils.Utils; - -import net.jpountz.lz4.LZ4Compressor; -import net.jpountz.lz4.LZ4Factory; -import net.jpountz.xxhash.XXHash32; -import net.jpountz.xxhash.XXHashFactory; - -/** - * A partial implementation of the v1.4.1 LZ4 Frame format. - * - * @see LZ4 Framing Format Spec - */ -public final class KafkaLZ4BlockOutputStream extends FilterOutputStream { - - public static final int MAGIC = 0x184D2204; - public static final int LZ4_MAX_HEADER_LENGTH = 19; - public static final int LZ4_FRAME_INCOMPRESSIBLE_MASK = 0x80000000; - - public static final String CLOSED_STREAM = "The stream is already closed"; - - public static final int BLOCKSIZE_64KB = 4; - public static final int BLOCKSIZE_256KB = 5; - public static final int BLOCKSIZE_1MB = 6; - public static final int BLOCKSIZE_4MB = 7; - - private final LZ4Compressor compressor; - private final XXHash32 checksum; - private final FLG flg; - private final BD bd; - private final byte[] buffer; - private final byte[] compressedBuffer; - private final int maxBlockSize; - private int bufferOffset; - private boolean finished; - - /** - * Create a new {@link OutputStream} that will compress data using the LZ4 algorithm. - * - * @param out The output stream to compress - * @param blockSize Default: 4. The block size used during compression. 4=64kb, 5=256kb, 6=1mb, 7=4mb. All other values will generate an exception - * @param blockChecksum Default: false. When true, a XXHash32 checksum is computed and appended to the stream for every block of data - * @throws IOException - */ - public KafkaLZ4BlockOutputStream(OutputStream out, int blockSize, boolean blockChecksum) throws IOException { - super(out); - compressor = LZ4Factory.fastestInstance().fastCompressor(); - checksum = XXHashFactory.fastestInstance().hash32(); - bd = new BD(blockSize); - flg = new FLG(blockChecksum); - bufferOffset = 0; - maxBlockSize = bd.getBlockMaximumSize(); - buffer = new byte[maxBlockSize]; - compressedBuffer = new byte[compressor.maxCompressedLength(maxBlockSize)]; - finished = false; - writeHeader(); - } - - /** - * Create a new {@link OutputStream} that will compress data using the LZ4 algorithm. - * - * @param out The stream to compress - * @param blockSize Default: 4. The block size used during compression. 4=64kb, 5=256kb, 6=1mb, 7=4mb. All other values will generate an exception - * @throws IOException - */ - public KafkaLZ4BlockOutputStream(OutputStream out, int blockSize) throws IOException { - this(out, blockSize, false); - } - - /** - * Create a new {@link OutputStream} that will compress data using the LZ4 algorithm. - * - * @param out The output stream to compress - * @throws IOException - */ - public KafkaLZ4BlockOutputStream(OutputStream out) throws IOException { - this(out, BLOCKSIZE_64KB); - } - - /** - * Writes the magic number and frame descriptor to the underlying {@link OutputStream}. - * - * @throws IOException - */ - private void writeHeader() throws IOException { - Utils.writeUnsignedIntLE(buffer, 0, MAGIC); - bufferOffset = 4; - buffer[bufferOffset++] = flg.toByte(); - buffer[bufferOffset++] = bd.toByte(); - // TODO write uncompressed content size, update flg.validate() - // TODO write dictionary id, update flg.validate() - // compute checksum on all descriptor fields - int hash = (checksum.hash(buffer, 0, bufferOffset, 0) >> 8) & 0xFF; - buffer[bufferOffset++] = (byte) hash; - // write out frame descriptor - out.write(buffer, 0, bufferOffset); - bufferOffset = 0; - } - - /** - * Compresses buffered data, optionally computes an XXHash32 checksum, and writes - * the result to the underlying {@link OutputStream}. - * - * @throws IOException - */ - private void writeBlock() throws IOException { - if (bufferOffset == 0) { - return; - } - - int compressedLength = compressor.compress(buffer, 0, bufferOffset, compressedBuffer, 0); - byte[] bufferToWrite = compressedBuffer; - int compressMethod = 0; - - // Store block uncompressed if compressed length is greater (incompressible) - if (compressedLength >= bufferOffset) { - bufferToWrite = buffer; - compressedLength = bufferOffset; - compressMethod = LZ4_FRAME_INCOMPRESSIBLE_MASK; - } - - // Write content - Utils.writeUnsignedIntLE(out, compressedLength | compressMethod); - out.write(bufferToWrite, 0, compressedLength); - - // Calculate and write block checksum - if (flg.isBlockChecksumSet()) { - int hash = checksum.hash(bufferToWrite, 0, compressedLength, 0); - Utils.writeUnsignedIntLE(out, hash); - } - bufferOffset = 0; - } - - /** - * Similar to the {@link #writeBlock()} method. Writes a 0-length block - * (without block checksum) to signal the end of the block stream. - * - * @throws IOException - */ - private void writeEndMark() throws IOException { - Utils.writeUnsignedIntLE(out, 0); - // TODO implement content checksum, update flg.validate() - finished = true; - } - - @Override - public void write(int b) throws IOException { - ensureNotFinished(); - if (bufferOffset == maxBlockSize) { - writeBlock(); - } - buffer[bufferOffset++] = (byte) b; - } - - @Override - public void write(byte[] b, int off, int len) throws IOException { - net.jpountz.util.Utils.checkRange(b, off, len); - ensureNotFinished(); - - int bufferRemainingLength = maxBlockSize - bufferOffset; - // while b will fill the buffer - while (len > bufferRemainingLength) { - // fill remaining space in buffer - System.arraycopy(b, off, buffer, bufferOffset, bufferRemainingLength); - bufferOffset = maxBlockSize; - writeBlock(); - // compute new offset and length - off += bufferRemainingLength; - len -= bufferRemainingLength; - bufferRemainingLength = maxBlockSize; - } - - System.arraycopy(b, off, buffer, bufferOffset, len); - bufferOffset += len; - } - - @Override - public void flush() throws IOException { - if (!finished) { - writeBlock(); - } - if (out != null) { - out.flush(); - } - } - - /** - * A simple state check to ensure the stream is still open. - */ - private void ensureNotFinished() { - if (finished) { - throw new IllegalStateException(CLOSED_STREAM); - } - } - - @Override - public void close() throws IOException { - if (!finished) { - writeEndMark(); - flush(); - finished = true; - } - if (out != null) { - out.close(); - out = null; - } - } - - public static class FLG { - - private static final int VERSION = 1; - - private final int presetDictionary; - private final int reserved1; - private final int contentChecksum; - private final int contentSize; - private final int blockChecksum; - private final int blockIndependence; - private final int version; - - public FLG() { - this(false); - } - - public FLG(boolean blockChecksum) { - this(0, 0, 0, 0, blockChecksum ? 1 : 0, 1, VERSION); - } - - private FLG(int presetDictionary, int reserved1, int contentChecksum, - int contentSize, int blockChecksum, int blockIndependence, int version) { - this.presetDictionary = presetDictionary; - this.reserved1 = reserved1; - this.contentChecksum = contentChecksum; - this.contentSize = contentSize; - this.blockChecksum = blockChecksum; - this.blockIndependence = blockIndependence; - this.version = version; - validate(); - } - - public static FLG fromByte(byte flg) { - int presetDictionary = (flg >>> 0) & 1; - int reserved1 = (flg >>> 1) & 1; - int contentChecksum = (flg >>> 2) & 1; - int contentSize = (flg >>> 3) & 1; - int blockChecksum = (flg >>> 4) & 1; - int blockIndependence = (flg >>> 5) & 1; - int version = (flg >>> 6) & 3; - - return new FLG(presetDictionary, reserved1, contentChecksum, - contentSize, blockChecksum, blockIndependence, version); - } - - public byte toByte() { - return (byte) ( - ((presetDictionary & 1) << 0) - | ((reserved1 & 1) << 1) - | ((contentChecksum & 1) << 2) - | ((contentSize & 1) << 3) - | ((blockChecksum & 1) << 4) - | ((blockIndependence & 1) << 5) - | ((version & 3) << 6) ); - } - - private void validate() { - if (presetDictionary != 0) { - throw new RuntimeException("Preset dictionary is unsupported"); - } - if (reserved1 != 0) { - throw new RuntimeException("Reserved1 field must be 0"); - } - if (contentChecksum != 0) { - throw new RuntimeException("Content checksum is unsupported"); - } - if (contentSize != 0) { - throw new RuntimeException("Content size is unsupported"); - } - if (blockIndependence != 1) { - throw new RuntimeException("Dependent block stream is unsupported"); - } - if (version != VERSION) { - throw new RuntimeException(String.format("Version %d is unsupported", version)); - } - } - - public boolean isPresetDictionarySet() { - return presetDictionary == 1; - } - - public boolean isContentChecksumSet() { - return contentChecksum == 1; - } - - public boolean isContentSizeSet() { - return contentSize == 1; - } - - public boolean isBlockChecksumSet() { - return blockChecksum == 1; - } - - public boolean isBlockIndependenceSet() { - return blockIndependence == 1; - } - - public int getVersion() { - return version; - } - } - - public static class BD { - - private final int reserved2; - private final int blockSizeValue; - private final int reserved3; - - public BD() { - this(0, BLOCKSIZE_64KB, 0); - } - - public BD(int blockSizeValue) { - this(0, blockSizeValue, 0); - } - - private BD(int reserved2, int blockSizeValue, int reserved3) { - this.reserved2 = reserved2; - this.blockSizeValue = blockSizeValue; - this.reserved3 = reserved3; - validate(); - } - - public static BD fromByte(byte bd) { - int reserved2 = (bd >>> 0) & 15; - int blockMaximumSize = (bd >>> 4) & 7; - int reserved3 = (bd >>> 7) & 1; - - return new BD(reserved2, blockMaximumSize, reserved3); - } - - private void validate() { - if (reserved2 != 0) { - throw new RuntimeException("Reserved2 field must be 0"); - } - if (blockSizeValue < 4 || blockSizeValue > 7) { - throw new RuntimeException("Block size value must be between 4 and 7"); - } - if (reserved3 != 0) { - throw new RuntimeException("Reserved3 field must be 0"); - } - } - - // 2^(2n+8) - public int getBlockMaximumSize() { - return (1 << ((2 * blockSizeValue) + 8)); - } - - public byte toByte() { - return (byte) ( - ((reserved2 & 15) << 0) - | ((blockSizeValue & 7) << 4) - | ((reserved3 & 1) << 7) ); - } - } - -} diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java b/clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java index 9c205387acc13..6b9590c418aed 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java @@ -42,7 +42,7 @@ public class JmxReporter implements MetricsReporter { private static final Logger log = LoggerFactory.getLogger(JmxReporter.class); - private static final Object lock = new Object(); + private static final Object LOCK = new Object(); private String prefix; private final Map mbeans = new HashMap(); @@ -58,12 +58,11 @@ public JmxReporter(String prefix) { } @Override - public void configure(Map configs) { - } + public void configure(Map configs) {} @Override public void init(List metrics) { - synchronized (lock) { + synchronized (LOCK) { for (KafkaMetric metric : metrics) addAttribute(metric); for (KafkaMbean mbean : mbeans.values()) @@ -73,7 +72,7 @@ public void init(List metrics) { @Override public void metricChange(KafkaMetric metric) { - synchronized (lock) { + synchronized (LOCK) { KafkaMbean mbean = addAttribute(metric); reregister(mbean); } @@ -86,36 +85,35 @@ private KafkaMbean addAttribute(KafkaMetric metric) { if (!this.mbeans.containsKey(mBeanName)) mbeans.put(mBeanName, new KafkaMbean(mBeanName)); KafkaMbean mbean = this.mbeans.get(mBeanName); - mbean.setAttribute(metricName.name() , metric); + mbean.setAttribute(metricName.name(), metric); return mbean; } catch (JMException e) { throw new KafkaException("Error creating mbean attribute for metricName :" + metric.metricName(), e); } } - /** - * @param metricName - * @return standard JMX MBean name in the following format - * domainName:type=metricType,key1=val1,key2=val2 - */ - private String getMBeanName(MetricName metricName) { - StringBuilder mBeanName = new StringBuilder(); - mBeanName.append(prefix); - mBeanName.append(":type="); - mBeanName.append(metricName.group()); - for (Map.Entry entry : metricName.tags().entrySet()) { - if(entry.getKey().length() <= 0 || entry.getValue().length() <= 0) - continue; - mBeanName.append(","); - mBeanName.append(entry.getKey()); - mBeanName.append("="); - mBeanName.append(entry.getValue()); + /** + * @param metricName + * @return standard JMX MBean name in the following format domainName:type=metricType,key1=val1,key2=val2 + */ + private String getMBeanName(MetricName metricName) { + StringBuilder mBeanName = new StringBuilder(); + mBeanName.append(prefix); + mBeanName.append(":type="); + mBeanName.append(metricName.group()); + for (Map.Entry entry : metricName.tags().entrySet()) { + if (entry.getKey().length() <= 0 || entry.getValue().length() <= 0) + continue; + mBeanName.append(","); + mBeanName.append(entry.getKey()); + mBeanName.append("="); + mBeanName.append(entry.getValue()); + } + return mBeanName.toString(); } - return mBeanName.toString(); - } public void close() { - synchronized (lock) { + synchronized (LOCK) { for (KafkaMbean mbean : this.mbeans.values()) unregister(mbean); } @@ -185,7 +183,12 @@ public MBeanInfo getMBeanInfo() { for (Map.Entry entry : this.metrics.entrySet()) { String attribute = entry.getKey(); KafkaMetric metric = entry.getValue(); - attrs[i] = new MBeanAttributeInfo(attribute, double.class.getName(), metric.metricName().description(), true, false, false); + attrs[i] = new MBeanAttributeInfo(attribute, + double.class.getName(), + metric.metricName().description(), + true, + false, + false); i += 1; } return new MBeanInfo(this.getClass().getName(), "", attrs, null, null, null); diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java index e53cfaa69f518..ca823fd463952 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java @@ -147,7 +147,7 @@ public synchronized void add(CompoundStat stat, MetricConfig config) { * @param stat The statistic to keep */ public void add(MetricName metricName, MeasurableStat stat) { - add(metricName, stat, null); + add(metricName, stat, null); } /** @@ -157,11 +157,11 @@ public void add(MetricName metricName, MeasurableStat stat) { * @param config A special configuration for this metric. If null use the sensor default configuration. */ public synchronized void add(MetricName metricName, MeasurableStat stat, MetricConfig config) { - KafkaMetric metric = new KafkaMetric(new Object(), - Utils.notNull(metricName), - Utils.notNull(stat), - config == null ? this.config : config, - time); + KafkaMetric metric = new KafkaMetric(new Object(), + Utils.notNull(metricName), + Utils.notNull(stat), + config == null ? this.config : config, + time); this.registry.registerMetric(metric); this.metrics.add(metric); this.stats.add(stat); diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java index a5838b3894906..98429da34418f 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java @@ -71,7 +71,7 @@ private double convert(long time) { case MILLISECONDS: return time; case SECONDS: - return time / (1000.0); + return time / 1000.0; case MINUTES: return time / (60.0 * 1000.0); case HOURS: diff --git a/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java b/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java index dcc639a4bb451..fc0d168324aae 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java +++ b/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java @@ -50,7 +50,7 @@ public boolean complete() { @Override public ByteBuffer[] reify() { - return new ByteBuffer[] { this.buffer }; + return new ByteBuffer[] {this.buffer}; } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java index e18a769a4b300..6baad9366a197 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java @@ -14,7 +14,6 @@ import java.io.EOFException; import java.io.IOException; -import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.Socket; import java.nio.channels.CancelledKeyException; @@ -275,7 +274,7 @@ public void poll(long timeout) throws IOException { } } catch (IOException e) { String desc = socketDescription(channel); - if(e instanceof EOFException) + if (e instanceof EOFException) log.info("Connection {} disconnected", desc); else log.warn("Error in I/O with connection to {}", desc, e); @@ -290,9 +289,9 @@ public void poll(long timeout) throws IOException { private String socketDescription(SocketChannel channel) { Socket socket = channel.socket(); - if(socket == null) + if (socket == null) return "[unconnected socket]"; - else if(socket.getInetAddress() != null) + else if (socket.getInetAddress() != null) return socket.getInetAddress().toString(); else return socket.getLocalAddress().toString(); @@ -525,7 +524,7 @@ public void maybeRegisterNodeMetrics(int node) { String metricGrpName = metricGrpPrefix + "-node-metrics"; Map tags = new LinkedHashMap(metricTags); - tags.put("node-id", "node-"+node); + tags.put("node-id", "node-" + node); nodeRequest = this.metrics.sensor(nodeRequestName); MetricName metricName = new MetricName("outgoing-byte-rate", metricGrpName, tags); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index 109fc965e09b2..07aba71303bc1 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -16,10 +16,6 @@ */ package org.apache.kafka.common.protocol; - -import java.util.ArrayList; -import java.util.List; - /** * Identifiers for all the Kafka APIs */ @@ -37,16 +33,18 @@ public enum ApiKeys { HEARTBEAT(12, "heartbeat"); private static ApiKeys[] codeToType; - public static int MAX_API_KEY = -1; + public static final int MAX_API_KEY; static { + int maxKey = -1; for (ApiKeys key : ApiKeys.values()) { - MAX_API_KEY = Math.max(MAX_API_KEY, key.id); + maxKey = Math.max(maxKey, key.id); } - codeToType = new ApiKeys[MAX_API_KEY+1]; + codeToType = new ApiKeys[maxKey + 1]; for (ApiKeys key : ApiKeys.values()) { codeToType[key.id] = key; } + MAX_API_KEY = maxKey; } /** the perminant and immutable id of an API--this can't change ever */ 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 7517b879866fc..101f382170ad6 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 @@ -28,346 +28,347 @@ public class Protocol { - public static Schema REQUEST_HEADER = new Schema(new Field("api_key", INT16, "The id of the request type."), - new Field("api_version", INT16, "The version of the API."), - new Field("correlation_id", - INT32, - "A user-supplied integer value that will be passed back with the response"), - new Field("client_id", - STRING, - "A user specified identifier for the client making the request.")); - - public static Schema RESPONSE_HEADER = new Schema(new Field("correlation_id", - INT32, - "The user-supplied value passed in with the request")); + public static final Schema REQUEST_HEADER = new Schema(new Field("api_key", INT16, "The id of the request type."), + new Field("api_version", INT16, "The version of the API."), + new Field("correlation_id", + INT32, + "A user-supplied integer value that will be passed back with the response"), + new Field("client_id", + STRING, + "A user specified identifier for the client making the request.")); + + public static final Schema RESPONSE_HEADER = new Schema(new Field("correlation_id", + INT32, + "The user-supplied value passed in with the request")); /* Metadata api */ - public static Schema METADATA_REQUEST_V0 = new Schema(new Field("topics", - new ArrayOf(STRING), - "An array of topics to fetch metadata for. If no topics are specified fetch metadtata for all topics.")); + public static final Schema METADATA_REQUEST_V0 = new Schema(new Field("topics", + new ArrayOf(STRING), + "An array of topics to fetch metadata for. If no topics are specified fetch metadtata for all topics.")); - public static Schema BROKER = new Schema(new Field("node_id", INT32, "The broker id."), - new Field("host", STRING, "The hostname of the broker."), - new Field("port", INT32, "The port on which the broker accepts requests.")); + public static final Schema BROKER = new Schema(new Field("node_id", INT32, "The broker id."), + new Field("host", STRING, "The hostname of the broker."), + new Field("port", + INT32, + "The port on which the broker accepts requests.")); - public static Schema PARTITION_METADATA_V0 = new Schema(new Field("partition_error_code", - INT16, - "The error code for the partition, if any."), - new Field("partition_id", INT32, "The id of the partition."), - new Field("leader", - INT32, - "The id of the broker acting as leader for this partition."), - new Field("replicas", - new ArrayOf(INT32), - "The set of all nodes that host this partition."), - new Field("isr", - new ArrayOf(INT32), - "The set of nodes that are in sync with the leader for this partition.")); - - public static Schema TOPIC_METADATA_V0 = new Schema(new Field("topic_error_code", INT16, "The error code for the given topic."), - new Field("topic", STRING, "The name of the topic"), - new Field("partition_metadata", - new ArrayOf(PARTITION_METADATA_V0), - "Metadata for each partition of the topic.")); - - public static Schema METADATA_RESPONSE_V0 = new Schema(new Field("brokers", - new ArrayOf(BROKER), - "Host and port information for all brokers."), - new Field("topic_metadata", new ArrayOf(TOPIC_METADATA_V0))); - - public static Schema[] METADATA_REQUEST = new Schema[] { METADATA_REQUEST_V0 }; - public static Schema[] METADATA_RESPONSE = new Schema[] { METADATA_RESPONSE_V0 }; + public static final Schema PARTITION_METADATA_V0 = new Schema(new Field("partition_error_code", + INT16, + "The error code for the partition, if any."), + new Field("partition_id", + INT32, + "The id of the partition."), + new Field("leader", + INT32, + "The id of the broker acting as leader for this partition."), + new Field("replicas", + new ArrayOf(INT32), + "The set of all nodes that host this partition."), + new Field("isr", + new ArrayOf(INT32), + "The set of nodes that are in sync with the leader for this partition.")); + + public static final Schema TOPIC_METADATA_V0 = new Schema(new Field("topic_error_code", + INT16, + "The error code for the given topic."), + new Field("topic", STRING, "The name of the topic"), + new Field("partition_metadata", + new ArrayOf(PARTITION_METADATA_V0), + "Metadata for each partition of the topic.")); + + public static final Schema METADATA_RESPONSE_V0 = new Schema(new Field("brokers", + new ArrayOf(BROKER), + "Host and port information for all brokers."), + new Field("topic_metadata", + new ArrayOf(TOPIC_METADATA_V0))); + + public static final Schema[] METADATA_REQUEST = new Schema[] {METADATA_REQUEST_V0}; + public static final Schema[] METADATA_RESPONSE = new Schema[] {METADATA_RESPONSE_V0}; /* Produce api */ - public static Schema TOPIC_PRODUCE_DATA_V0 = new Schema(new Field("topic", STRING), - new Field("data", new ArrayOf(new Schema(new Field("partition", INT32), - new Field("record_set", BYTES))))); - - public static Schema PRODUCE_REQUEST_V0 = new Schema(new Field("acks", - INT16, - "The number of nodes that should replicate the produce before returning. -1 indicates the full ISR."), - new Field("timeout", INT32, "The time to await a response in ms."), - new Field("topic_data", new ArrayOf(TOPIC_PRODUCE_DATA_V0))); - - public static Schema PRODUCE_RESPONSE_V0 = new Schema(new Field("responses", - new ArrayOf(new Schema(new Field("topic", STRING), - new Field("partition_responses", - new ArrayOf(new Schema(new Field("partition", - INT32), - new Field("error_code", - INT16), - new Field("base_offset", - INT64)))))))); - - public static Schema[] PRODUCE_REQUEST = new Schema[] { PRODUCE_REQUEST_V0 }; - public static Schema[] PRODUCE_RESPONSE = new Schema[] { PRODUCE_RESPONSE_V0 }; - - /* Offset commit api */ - public static Schema OFFSET_COMMIT_REQUEST_PARTITION_V0 = new Schema(new Field("partition", - INT32, - "Topic partition id."), - new Field("offset", - INT64, - "Message offset to be committed."), - new Field("timestamp", - INT64, - "Timestamp of the commit"), - new Field("metadata", - STRING, - "Any associated metadata the client wants to keep.")); - - public static Schema OFFSET_COMMIT_REQUEST_TOPIC_V0 = new Schema(new Field("topic", - STRING, - "Topic to commit."), - new Field("partitions", - new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V0), - "Partitions to commit offsets.")); - - public static Schema OFFSET_COMMIT_REQUEST_V0 = new Schema(new Field("group_id", - STRING, - "The consumer group id."), - new Field("topics", - new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0), - "Topics to commit offsets.")); - - public static Schema OFFSET_COMMIT_REQUEST_V1 = new Schema(new Field("group_id", - STRING, - "The consumer group id."), - new Field("group_generation_id", + public static final Schema TOPIC_PRODUCE_DATA_V0 = new Schema(new Field("topic", STRING), + new Field("data", + new ArrayOf(new Schema(new Field("partition", + INT32), + new Field("record_set", + BYTES))))); + + public static final Schema PRODUCE_REQUEST_V0 = new Schema(new Field("acks", + INT16, + "The number of nodes that should replicate the produce before returning. -1 indicates the full ISR."), + new Field("timeout", INT32, - "The generation of the consumer group."), - new Field("consumer_id", - STRING, - "The consumer id assigned by the group coordinator."), - new Field("topics", - new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0), - "Topics to commit offsets.")); - - public static Schema OFFSET_COMMIT_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", - INT32, - "Topic partition id."), - new Field("error_code", - INT16)); - - public static Schema OFFSET_COMMIT_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), - new Field("partition_responses", - new ArrayOf(OFFSET_COMMIT_RESPONSE_PARTITION_V0))); - - public static Schema OFFSET_COMMIT_RESPONSE_V0 = new Schema(new Field("responses", - new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0))); - - public static Schema[] OFFSET_COMMIT_REQUEST = new Schema[] { OFFSET_COMMIT_REQUEST_V0, OFFSET_COMMIT_REQUEST_V1 }; - /* The response types for both V0 and V1 of OFFSET_COMMIT_REQUEST are the same. */ - public static Schema[] OFFSET_COMMIT_RESPONSE = new Schema[] { OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V0}; + "The time to await a response in ms."), + new Field("topic_data", + new ArrayOf(TOPIC_PRODUCE_DATA_V0))); + + public static final Schema PRODUCE_RESPONSE_V0 = new Schema(new Field("responses", + new ArrayOf(new Schema(new Field("topic", + STRING), + new Field("partition_responses", + new ArrayOf(new Schema(new Field("partition", + INT32), + new Field("error_code", + INT16), + new Field("base_offset", + INT64)))))))); + + public static final Schema[] PRODUCE_REQUEST = new Schema[] {PRODUCE_REQUEST_V0}; + public static final Schema[] PRODUCE_RESPONSE = new Schema[] {PRODUCE_RESPONSE_V0}; - /* Offset fetch api */ - public static Schema OFFSET_FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition", - INT32, - "Topic partition id.")); + /* Offset commit api */ + public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V0 = new Schema(new Field("partition", + INT32, + "Topic partition id."), + new Field("offset", + INT64, + "Message offset to be committed."), + new Field("timestamp", + INT64, + "Timestamp of the commit"), + new Field("metadata", + STRING, + "Any associated metadata the client wants to keep.")); + + public static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V0 = new Schema(new Field("topic", + STRING, + "Topic to commit."), + new Field("partitions", + new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V0), + "Partitions to commit offsets.")); + + public static final Schema OFFSET_COMMIT_REQUEST_V0 = new Schema(new Field("group_id", + STRING, + "The consumer group id."), + new Field("topics", + new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0), + "Topics to commit offsets.")); - public static Schema OFFSET_FETCH_REQUEST_TOPIC_V0 = new Schema(new Field("topic", + public static final Schema OFFSET_COMMIT_REQUEST_V1 = new Schema(new Field("group_id", STRING, - "Topic to fetch offset."), - new Field("partitions", - new ArrayOf(OFFSET_FETCH_REQUEST_PARTITION_V0), - "Partitions to fetch offsets.")); - - public static Schema OFFSET_FETCH_REQUEST_V0 = new Schema(new Field("group_id", - STRING, - "The consumer group id."), - new Field("topics", - new ArrayOf(OFFSET_FETCH_REQUEST_TOPIC_V0), - "Topics to fetch offsets.")); - - public static Schema OFFSET_FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", - INT32, - "Topic partition id."), - new Field("offset", - INT64, - "Last committed message offset."), - new Field("metadata", - STRING, - "Any associated metadata the client wants to keep."), - new Field("error_code", - INT16)); + "The consumer group id."), + new Field("group_generation_id", + INT32, + "The generation of the consumer group."), + new Field("consumer_id", + STRING, + "The consumer id assigned by the group coordinator."), + new Field("topics", + new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0), + "Topics to commit offsets.")); + + public static final Schema OFFSET_COMMIT_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", + INT32, + "Topic partition id."), + new Field("error_code", INT16)); - public static Schema OFFSET_FETCH_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), - new Field("partition_responses", - new ArrayOf(OFFSET_FETCH_RESPONSE_PARTITION_V0))); + public static final Schema OFFSET_COMMIT_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), + new Field("partition_responses", + new ArrayOf(OFFSET_COMMIT_RESPONSE_PARTITION_V0))); - public static Schema OFFSET_FETCH_RESPONSE_V0 = new Schema(new Field("responses", - new ArrayOf(OFFSET_FETCH_RESPONSE_TOPIC_V0))); + public static final Schema OFFSET_COMMIT_RESPONSE_V0 = new Schema(new Field("responses", + new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0))); - public static Schema[] OFFSET_FETCH_REQUEST = new Schema[] { OFFSET_FETCH_REQUEST_V0 }; - public static Schema[] OFFSET_FETCH_RESPONSE = new Schema[] { OFFSET_FETCH_RESPONSE_V0 }; + public static final Schema[] OFFSET_COMMIT_REQUEST = new Schema[] {OFFSET_COMMIT_REQUEST_V0, OFFSET_COMMIT_REQUEST_V1}; + /* The response types for both V0 and V1 of OFFSET_COMMIT_REQUEST are the same. */ + public static final Schema[] OFFSET_COMMIT_RESPONSE = new Schema[] {OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V0}; + + /* Offset fetch api */ + public static final Schema OFFSET_FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition", + INT32, + "Topic partition id.")); + + public static final Schema OFFSET_FETCH_REQUEST_TOPIC_V0 = new Schema(new Field("topic", + STRING, + "Topic to fetch offset."), + new Field("partitions", + new ArrayOf(OFFSET_FETCH_REQUEST_PARTITION_V0), + "Partitions to fetch offsets.")); + + public static final Schema OFFSET_FETCH_REQUEST_V0 = new Schema(new Field("group_id", + STRING, + "The consumer group id."), + new Field("topics", + new ArrayOf(OFFSET_FETCH_REQUEST_TOPIC_V0), + "Topics to fetch offsets.")); + + public static final Schema OFFSET_FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", + INT32, + "Topic partition id."), + new Field("offset", + INT64, + "Last committed message offset."), + new Field("metadata", + STRING, + "Any associated metadata the client wants to keep."), + new Field("error_code", INT16)); + + public static final Schema OFFSET_FETCH_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), + new Field("partition_responses", + new ArrayOf(OFFSET_FETCH_RESPONSE_PARTITION_V0))); + + public static final Schema OFFSET_FETCH_RESPONSE_V0 = new Schema(new Field("responses", + new ArrayOf(OFFSET_FETCH_RESPONSE_TOPIC_V0))); + + public static final Schema[] OFFSET_FETCH_REQUEST = new Schema[] {OFFSET_FETCH_REQUEST_V0}; + public static final Schema[] OFFSET_FETCH_RESPONSE = new Schema[] {OFFSET_FETCH_RESPONSE_V0}; /* List offset api */ - public static Schema LIST_OFFSET_REQUEST_PARTITION_V0 = new Schema(new Field("partition", + public static final Schema LIST_OFFSET_REQUEST_PARTITION_V0 = new Schema(new Field("partition", + INT32, + "Topic partition id."), + new Field("timestamp", INT64, "Timestamp."), + new Field("max_num_offsets", + INT32, + "Maximum offsets to return.")); + + public static final Schema LIST_OFFSET_REQUEST_TOPIC_V0 = new Schema(new Field("topic", + STRING, + "Topic to list offset."), + new Field("partitions", + new ArrayOf(LIST_OFFSET_REQUEST_PARTITION_V0), + "Partitions to list offset.")); + + public static final Schema LIST_OFFSET_REQUEST_V0 = new Schema(new Field("replica_id", + INT32, + "Broker id of the follower. For normal consumers, use -1."), + new Field("topics", + new ArrayOf(LIST_OFFSET_REQUEST_TOPIC_V0), + "Topics to list offsets.")); + + public static final Schema LIST_OFFSET_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", + INT32, + "Topic partition id."), + new Field("error_code", INT16), + new Field("offsets", + new ArrayOf(INT64), + "A list of offsets.")); + + public static final Schema LIST_OFFSET_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), + new Field("partition_responses", + new ArrayOf(LIST_OFFSET_RESPONSE_PARTITION_V0))); + + public static final Schema LIST_OFFSET_RESPONSE_V0 = new Schema(new Field("responses", + new ArrayOf(LIST_OFFSET_RESPONSE_TOPIC_V0))); + + public static final Schema[] LIST_OFFSET_REQUEST = new Schema[] {LIST_OFFSET_REQUEST_V0}; + public static final Schema[] LIST_OFFSET_RESPONSE = new Schema[] {LIST_OFFSET_RESPONSE_V0}; + + /* Fetch api */ + public static final Schema FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition", INT32, "Topic partition id."), - new Field("timestamp", + new Field("fetch_offset", INT64, - "Timestamp."), - new Field("max_num_offsets", + "Message offset."), + new Field("max_bytes", INT32, - "Maximum offsets to return.")); + "Maximum bytes to fetch.")); - public static Schema LIST_OFFSET_REQUEST_TOPIC_V0 = new Schema(new Field("topic", - STRING, - "Topic to list offset."), + public static final Schema FETCH_REQUEST_TOPIC_V0 = new Schema(new Field("topic", STRING, "Topic to fetch."), new Field("partitions", - new ArrayOf(LIST_OFFSET_REQUEST_PARTITION_V0), - "Partitions to list offset.")); + new ArrayOf(FETCH_REQUEST_PARTITION_V0), + "Partitions to fetch.")); - public static Schema LIST_OFFSET_REQUEST_V0 = new Schema(new Field("replica_id", + public static final Schema FETCH_REQUEST_V0 = 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("topics", - new ArrayOf(LIST_OFFSET_REQUEST_TOPIC_V0), - "Topics to list offsets.")); + new ArrayOf(FETCH_REQUEST_TOPIC_V0), + "Topics to fetch.")); - public static Schema LIST_OFFSET_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", + public static final Schema FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", INT32, "Topic partition id."), - new Field("error_code", - INT16), - new Field("offsets", - new ArrayOf(INT64), - "A list of offsets.")); + new Field("error_code", INT16), + new Field("high_watermark", + INT64, + "Last committed offset."), + new Field("record_set", BYTES)); - public static Schema LIST_OFFSET_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), + public static final Schema FETCH_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), new Field("partition_responses", - new ArrayOf(LIST_OFFSET_RESPONSE_PARTITION_V0))); - - public static Schema LIST_OFFSET_RESPONSE_V0 = new Schema(new Field("responses", - new ArrayOf(LIST_OFFSET_RESPONSE_TOPIC_V0))); - - public static Schema[] LIST_OFFSET_REQUEST = new Schema[] { LIST_OFFSET_REQUEST_V0 }; - public static Schema[] LIST_OFFSET_RESPONSE = new Schema[] { LIST_OFFSET_RESPONSE_V0 }; - - /* Fetch api */ - public static Schema FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition", - INT32, - "Topic partition id."), - new Field("fetch_offset", - INT64, - "Message offset."), - new Field("max_bytes", - INT32, - "Maximum bytes to fetch.")); - - public static Schema FETCH_REQUEST_TOPIC_V0 = new Schema(new Field("topic", - STRING, - "Topic to fetch."), - new Field("partitions", - new ArrayOf(FETCH_REQUEST_PARTITION_V0), - "Partitions to fetch.")); - - public static Schema FETCH_REQUEST_V0 = 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("topics", - new ArrayOf(FETCH_REQUEST_TOPIC_V0), - "Topics to fetch.")); - - public static Schema FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition", - INT32, - "Topic partition id."), - new Field("error_code", - INT16), - new Field("high_watermark", - INT64, - "Last committed offset."), - new Field("record_set", BYTES)); - - public static Schema FETCH_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), - new Field("partition_responses", - new ArrayOf(FETCH_RESPONSE_PARTITION_V0))); + new ArrayOf(FETCH_RESPONSE_PARTITION_V0))); - public static Schema FETCH_RESPONSE_V0 = new Schema(new Field("responses", - new ArrayOf(FETCH_RESPONSE_TOPIC_V0))); + public static final Schema FETCH_RESPONSE_V0 = new Schema(new Field("responses", + new ArrayOf(FETCH_RESPONSE_TOPIC_V0))); - public static Schema[] FETCH_REQUEST = new Schema[] { FETCH_REQUEST_V0 }; - public static Schema[] FETCH_RESPONSE = new Schema[] { FETCH_RESPONSE_V0 }; + public static final Schema[] FETCH_REQUEST = new Schema[] {FETCH_REQUEST_V0}; + public static final Schema[] FETCH_RESPONSE = new Schema[] {FETCH_RESPONSE_V0}; /* Consumer metadata api */ - public static Schema CONSUMER_METADATA_REQUEST_V0 = new Schema(new Field("group_id", - STRING, - "The consumer group id.")); + public static final Schema CONSUMER_METADATA_REQUEST_V0 = new Schema(new Field("group_id", + STRING, + "The consumer group id.")); - public static Schema CONSUMER_METADATA_RESPONSE_V0 = new Schema(new Field("error_code", - INT16), - new Field("coordinator", - BROKER, - "Host and port information for the coordinator for a consumer group.")); + public static final Schema CONSUMER_METADATA_RESPONSE_V0 = new Schema(new Field("error_code", INT16), + new Field("coordinator", + BROKER, + "Host and port information for the coordinator for a consumer group.")); - public static Schema[] CONSUMER_METADATA_REQUEST = new Schema[] { CONSUMER_METADATA_REQUEST_V0 }; - public static Schema[] CONSUMER_METADATA_RESPONSE = new Schema[] { CONSUMER_METADATA_RESPONSE_V0 }; + public static final Schema[] CONSUMER_METADATA_REQUEST = new Schema[] {CONSUMER_METADATA_REQUEST_V0}; + public static final Schema[] CONSUMER_METADATA_RESPONSE = new Schema[] {CONSUMER_METADATA_RESPONSE_V0}; /* Join group api */ - public static Schema JOIN_GROUP_REQUEST_V0 = new Schema(new Field("group_id", - STRING, - "The consumer group id."), - new Field("session_timeout", - INT32, - "The coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms."), - new Field("topics", - new ArrayOf(STRING), - "An array of topics to subscribe to."), - new Field("consumer_id", - STRING, - "The assigned consumer id or an empty string for a new consumer."), - new Field("partition_assignment_strategy", - STRING, - "The strategy for the coordinator to assign partitions.")); - - public static Schema JOIN_GROUP_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), - new Field("partitions", new ArrayOf(INT32))); - public static Schema JOIN_GROUP_RESPONSE_V0 = new Schema(new Field("error_code", - INT16), - new Field("group_generation_id", - INT32, - "The generation of the consumer group."), - new Field("consumer_id", - STRING, - "The consumer id assigned by the group coordinator."), - new Field("assigned_partitions", - new ArrayOf(JOIN_GROUP_RESPONSE_TOPIC_V0))); + public static final Schema JOIN_GROUP_REQUEST_V0 = new Schema(new Field("group_id", + STRING, + "The consumer group id."), + new Field("session_timeout", + INT32, + "The coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms."), + new Field("topics", + new ArrayOf(STRING), + "An array of topics to subscribe to."), + new Field("consumer_id", + STRING, + "The assigned consumer id or an empty string for a new consumer."), + new Field("partition_assignment_strategy", + STRING, + "The strategy for the coordinator to assign partitions.")); + + public static final Schema JOIN_GROUP_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING), + new Field("partitions", new ArrayOf(INT32))); + public static final Schema JOIN_GROUP_RESPONSE_V0 = new Schema(new Field("error_code", INT16), + new Field("group_generation_id", + INT32, + "The generation of the consumer group."), + new Field("consumer_id", + STRING, + "The consumer id assigned by the group coordinator."), + new Field("assigned_partitions", + new ArrayOf(JOIN_GROUP_RESPONSE_TOPIC_V0))); - public static Schema[] JOIN_GROUP_REQUEST = new Schema[] { JOIN_GROUP_REQUEST_V0 }; - public static Schema[] JOIN_GROUP_RESPONSE = new Schema[] { JOIN_GROUP_RESPONSE_V0 }; + public static final Schema[] JOIN_GROUP_REQUEST = new Schema[] {JOIN_GROUP_REQUEST_V0}; + public static final Schema[] JOIN_GROUP_RESPONSE = new Schema[] {JOIN_GROUP_RESPONSE_V0}; /* Heartbeat api */ - public static Schema HEARTBEAT_REQUEST_V0 = new Schema(new Field("group_id", - STRING, - "The consumer group id."), - new Field("group_generation_id", - INT32, - "The generation of the consumer group."), - new Field("consumer_id", - STRING, - "The consumer id assigned by the group coordinator.")); + public static final Schema HEARTBEAT_REQUEST_V0 = new Schema(new Field("group_id", STRING, "The consumer group id."), + new Field("group_generation_id", + INT32, + "The generation of the consumer group."), + new Field("consumer_id", + STRING, + "The consumer id assigned by the group coordinator.")); - public static Schema HEARTBEAT_RESPONSE_V0 = new Schema(new Field("error_code", - INT16)); + public static final Schema HEARTBEAT_RESPONSE_V0 = new Schema(new Field("error_code", INT16)); - public static Schema[] HEARTBEAT_REQUEST = new Schema[] {HEARTBEAT_REQUEST_V0}; - public static Schema[] HEARTBEAT_RESPONSE = new Schema[] {HEARTBEAT_RESPONSE_V0}; + public static final Schema[] HEARTBEAT_REQUEST = new Schema[] {HEARTBEAT_REQUEST_V0}; + public static final Schema[] HEARTBEAT_RESPONSE = new Schema[] {HEARTBEAT_RESPONSE_V0}; /* an array of all requests and responses with all schema versions */ - public static Schema[][] REQUESTS = new Schema[ApiKeys.MAX_API_KEY + 1][]; - public static Schema[][] RESPONSES = new Schema[ApiKeys.MAX_API_KEY + 1][]; + public static final Schema[][] REQUESTS = new Schema[ApiKeys.MAX_API_KEY + 1][]; + public static final Schema[][] RESPONSES = new Schema[ApiKeys.MAX_API_KEY + 1][]; /* the latest version of each api */ - public static short[] CURR_VERSION = new short[ApiKeys.MAX_API_KEY + 1]; + public static final short[] CURR_VERSION = new short[ApiKeys.MAX_API_KEY + 1]; static { REQUESTS[ApiKeys.PRODUCE.id] = PRODUCE_REQUEST; @@ -401,11 +402,8 @@ public class Protocol { /* sanity check that we have the same number of request and response versions for each api */ for (ApiKeys api : ApiKeys.values()) if (REQUESTS[api.id].length != RESPONSES[api.id].length) - throw new IllegalStateException(REQUESTS[api.id].length + " request versions for api " - + api.name - + " but " - + RESPONSES[api.id].length - + " response versions."); + throw new IllegalStateException(REQUESTS[api.id].length + " request versions for api " + api.name + + " but " + RESPONSES[api.id].length + " response versions."); } } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java index ee1f78f06c19a..ff89f0e37d5fa 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java @@ -245,7 +245,7 @@ public void validate() { public ByteBuffer[] toBytes() { ByteBuffer buffer = ByteBuffer.allocate(sizeOf()); writeTo(buffer); - return new ByteBuffer[] { buffer }; + return new ByteBuffer[] {buffer}; } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java index c7bd2f8852bd9..1c9fbaa958423 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java +++ b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java @@ -24,7 +24,7 @@ */ public class ByteBufferOutputStream extends OutputStream { - private static float REALLOCATION_FACTOR = 1.1f; + private static final float REALLOCATION_FACTOR = 1.1f; private ByteBuffer buffer; diff --git a/clients/src/main/java/org/apache/kafka/common/record/Compressor.java b/clients/src/main/java/org/apache/kafka/common/record/Compressor.java index d684e6833bd81..e570b29d5ffba 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/Compressor.java +++ b/clients/src/main/java/org/apache/kafka/common/record/Compressor.java @@ -34,16 +34,15 @@ public class Compressor { static private final float COMPRESSION_RATE_ESTIMATION_FACTOR = 1.05f; static private final int COMPRESSION_DEFAULT_BUFFER_SIZE = 1024; - private static float[] typeToRate; - private static int MAX_TYPE_ID = -1; + private static final float[] TYPE_TO_RATE; static { + int maxTypeId = -1; + for (CompressionType type : CompressionType.values()) + maxTypeId = Math.max(maxTypeId, type.id); + TYPE_TO_RATE = new float[maxTypeId + 1]; for (CompressionType type : CompressionType.values()) { - MAX_TYPE_ID = Math.max(MAX_TYPE_ID, type.id); - } - typeToRate = new float[MAX_TYPE_ID+1]; - for (CompressionType type : CompressionType.values()) { - typeToRate[type.id] = type.rate; + TYPE_TO_RATE[type.id] = type.rate; } } @@ -118,7 +117,7 @@ public void close() { // update the compression ratio float compressionRate = (float) buffer.position() / this.writtenUncompressed; - typeToRate[type.id] = typeToRate[type.id] * COMPRESSION_RATE_DAMPING_FACTOR + + TYPE_TO_RATE[type.id] = TYPE_TO_RATE[type.id] * COMPRESSION_RATE_DAMPING_FACTOR + compressionRate * (1 - COMPRESSION_RATE_DAMPING_FACTOR); } } @@ -192,7 +191,7 @@ public long estimatedBytesWritten() { return bufferStream.buffer().position(); } else { // estimate the written bytes to the underlying byte buffer based on uncompressed written bytes - return (long) (writtenUncompressed * typeToRate[type.id] * COMPRESSION_RATE_ESTIMATION_FACTOR); + return (long) (writtenUncompressed * TYPE_TO_RATE[type.id] * COMPRESSION_RATE_ESTIMATION_FACTOR); } } @@ -209,8 +208,8 @@ static public DataOutputStream wrapForOutput(ByteBufferOutputStream buffer, Comp // dynamically load the snappy class to avoid runtime dependency // on snappy if we are not using it try { - Class SnappyOutputStream = Class.forName("org.xerial.snappy.SnappyOutputStream"); - OutputStream stream = (OutputStream) SnappyOutputStream.getConstructor(OutputStream.class, Integer.TYPE) + Class outputStreamClass = Class.forName("org.xerial.snappy.SnappyOutputStream"); + OutputStream stream = (OutputStream) outputStreamClass.getConstructor(OutputStream.class, Integer.TYPE) .newInstance(buffer, bufferSize); return new DataOutputStream(stream); } catch (Exception e) { @@ -218,7 +217,7 @@ static public DataOutputStream wrapForOutput(ByteBufferOutputStream buffer, Comp } case LZ4: try { - Class outputStreamClass = Class.forName("org.apache.kafka.common.message.KafkaLZ4BlockOutputStream"); + Class outputStreamClass = Class.forName("org.apache.kafka.common.record.KafkaLZ4BlockOutputStream"); OutputStream stream = (OutputStream) outputStreamClass.getConstructor(OutputStream.class) .newInstance(buffer); return new DataOutputStream(stream); @@ -244,8 +243,8 @@ static public DataInputStream wrapForInput(ByteBufferInputStream buffer, Compres // dynamically load the snappy class to avoid runtime dependency // on snappy if we are not using it try { - Class SnappyInputStream = Class.forName("org.xerial.snappy.SnappyInputStream"); - InputStream stream = (InputStream) SnappyInputStream.getConstructor(InputStream.class) + Class inputStreamClass = Class.forName("org.xerial.snappy.SnappyInputStream"); + InputStream stream = (InputStream) inputStreamClass.getConstructor(InputStream.class) .newInstance(buffer); return new DataInputStream(stream); } catch (Exception e) { @@ -254,7 +253,7 @@ static public DataInputStream wrapForInput(ByteBufferInputStream buffer, Compres case LZ4: // dynamically load LZ4 class to avoid runtime dependency try { - Class inputStreamClass = Class.forName("org.apache.kafka.common.message.KafkaLZ4BlockInputStream"); + Class inputStreamClass = Class.forName("org.apache.kafka.common.record.KafkaLZ4BlockInputStream"); InputStream stream = (InputStream) inputStreamClass.getConstructor(InputStream.class) .newInstance(buffer); return new DataInputStream(stream); diff --git a/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.java new file mode 100644 index 0000000000000..f480da2ae0992 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockInputStream.java @@ -0,0 +1,234 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.common.record; + +import static org.apache.kafka.common.record.KafkaLZ4BlockOutputStream.LZ4_FRAME_INCOMPRESSIBLE_MASK; +import static org.apache.kafka.common.record.KafkaLZ4BlockOutputStream.LZ4_MAX_HEADER_LENGTH; +import static org.apache.kafka.common.record.KafkaLZ4BlockOutputStream.MAGIC; + +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; + +import org.apache.kafka.common.record.KafkaLZ4BlockOutputStream.BD; +import org.apache.kafka.common.record.KafkaLZ4BlockOutputStream.FLG; +import org.apache.kafka.common.utils.Utils; + +import net.jpountz.lz4.LZ4Exception; +import net.jpountz.lz4.LZ4Factory; +import net.jpountz.lz4.LZ4SafeDecompressor; +import net.jpountz.xxhash.XXHash32; +import net.jpountz.xxhash.XXHashFactory; + +/** + * A partial implementation of the v1.4.1 LZ4 Frame format. + * + * @see LZ4 Framing + * Format Spec + */ +public final class KafkaLZ4BlockInputStream extends FilterInputStream { + + public static final String PREMATURE_EOS = "Stream ended prematurely"; + public static final String NOT_SUPPORTED = "Stream unsupported"; + public static final String BLOCK_HASH_MISMATCH = "Block checksum mismatch"; + public static final String DESCRIPTOR_HASH_MISMATCH = "Stream frame descriptor corrupted"; + + private final LZ4SafeDecompressor decompressor; + private final XXHash32 checksum; + private final byte[] buffer; + private final byte[] compressedBuffer; + private final int maxBlockSize; + private FLG flg; + private BD bd; + private int bufferOffset; + private int bufferSize; + private boolean finished; + + /** + * Create a new {@link InputStream} that will decompress data using the LZ4 algorithm. + * + * @param in The stream to decompress + * @throws IOException + */ + public KafkaLZ4BlockInputStream(InputStream in) throws IOException { + super(in); + decompressor = LZ4Factory.fastestInstance().safeDecompressor(); + checksum = XXHashFactory.fastestInstance().hash32(); + readHeader(); + maxBlockSize = bd.getBlockMaximumSize(); + buffer = new byte[maxBlockSize]; + compressedBuffer = new byte[maxBlockSize]; + bufferOffset = 0; + bufferSize = 0; + finished = false; + } + + /** + * Reads the magic number and frame descriptor from the underlying {@link InputStream}. + * + * @throws IOException + */ + private void readHeader() throws IOException { + byte[] header = new byte[LZ4_MAX_HEADER_LENGTH]; + + // read first 6 bytes into buffer to check magic and FLG/BD descriptor flags + bufferOffset = 6; + if (in.read(header, 0, bufferOffset) != bufferOffset) { + throw new IOException(PREMATURE_EOS); + } + + if (MAGIC != Utils.readUnsignedIntLE(header, bufferOffset - 6)) { + throw new IOException(NOT_SUPPORTED); + } + flg = FLG.fromByte(header[bufferOffset - 2]); + bd = BD.fromByte(header[bufferOffset - 1]); + // TODO read uncompressed content size, update flg.validate() + // TODO read dictionary id, update flg.validate() + + // check stream descriptor hash + byte hash = (byte) ((checksum.hash(header, 0, bufferOffset, 0) >> 8) & 0xFF); + header[bufferOffset++] = (byte) in.read(); + if (hash != header[bufferOffset - 1]) { + throw new IOException(DESCRIPTOR_HASH_MISMATCH); + } + } + + /** + * Decompresses (if necessary) buffered data, optionally computes and validates a XXHash32 checksum, and writes the + * result to a buffer. + * + * @throws IOException + */ + private void readBlock() throws IOException { + int blockSize = Utils.readUnsignedIntLE(in); + + // Check for EndMark + if (blockSize == 0) { + finished = true; + // TODO implement content checksum, update flg.validate() + return; + } else if (blockSize > maxBlockSize) { + throw new IOException(String.format("Block size %s exceeded max: %s", blockSize, maxBlockSize)); + } + + boolean compressed = (blockSize & LZ4_FRAME_INCOMPRESSIBLE_MASK) == 0; + byte[] bufferToRead; + if (compressed) { + bufferToRead = compressedBuffer; + } else { + blockSize &= ~LZ4_FRAME_INCOMPRESSIBLE_MASK; + bufferToRead = buffer; + bufferSize = blockSize; + } + + if (in.read(bufferToRead, 0, blockSize) != blockSize) { + throw new IOException(PREMATURE_EOS); + } + + // verify checksum + if (flg.isBlockChecksumSet() && Utils.readUnsignedIntLE(in) != checksum.hash(bufferToRead, 0, blockSize, 0)) { + throw new IOException(BLOCK_HASH_MISMATCH); + } + + if (compressed) { + try { + bufferSize = decompressor.decompress(compressedBuffer, 0, blockSize, buffer, 0, maxBlockSize); + } catch (LZ4Exception e) { + throw new IOException(e); + } + } + + bufferOffset = 0; + } + + @Override + public int read() throws IOException { + if (finished) { + return -1; + } + if (available() == 0) { + readBlock(); + } + if (finished) { + return -1; + } + int value = buffer[bufferOffset++] & 0xFF; + + return value; + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + net.jpountz.util.Utils.checkRange(b, off, len); + if (finished) { + return -1; + } + if (available() == 0) { + readBlock(); + } + if (finished) { + return -1; + } + len = Math.min(len, available()); + System.arraycopy(buffer, bufferOffset, b, off, len); + bufferOffset += len; + return len; + } + + @Override + public long skip(long n) throws IOException { + if (finished) { + return 0; + } + if (available() == 0) { + readBlock(); + } + if (finished) { + return 0; + } + n = Math.min(n, available()); + bufferOffset += n; + return n; + } + + @Override + public int available() throws IOException { + return bufferSize - bufferOffset; + } + + @Override + public void close() throws IOException { + in.close(); + } + + @Override + public synchronized void mark(int readlimit) { + throw new RuntimeException("mark not supported"); + } + + @Override + public synchronized void reset() throws IOException { + throw new RuntimeException("reset not supported"); + } + + @Override + public boolean markSupported() { + return false; + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream.java b/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream.java new file mode 100644 index 0000000000000..6a2231f477577 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/record/KafkaLZ4BlockOutputStream.java @@ -0,0 +1,392 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.common.record; + +import java.io.FilterOutputStream; +import java.io.IOException; +import java.io.OutputStream; + +import org.apache.kafka.common.utils.Utils; + +import net.jpountz.lz4.LZ4Compressor; +import net.jpountz.lz4.LZ4Factory; +import net.jpountz.xxhash.XXHash32; +import net.jpountz.xxhash.XXHashFactory; + +/** + * A partial implementation of the v1.4.1 LZ4 Frame format. + * + * @see LZ4 Framing + * Format Spec + */ +public final class KafkaLZ4BlockOutputStream extends FilterOutputStream { + + public static final int MAGIC = 0x184D2204; + public static final int LZ4_MAX_HEADER_LENGTH = 19; + public static final int LZ4_FRAME_INCOMPRESSIBLE_MASK = 0x80000000; + + public static final String CLOSED_STREAM = "The stream is already closed"; + + public static final int BLOCKSIZE_64KB = 4; + public static final int BLOCKSIZE_256KB = 5; + public static final int BLOCKSIZE_1MB = 6; + public static final int BLOCKSIZE_4MB = 7; + + private final LZ4Compressor compressor; + private final XXHash32 checksum; + private final FLG flg; + private final BD bd; + private final byte[] buffer; + private final byte[] compressedBuffer; + private final int maxBlockSize; + private int bufferOffset; + private boolean finished; + + /** + * Create a new {@link OutputStream} that will compress data using the LZ4 algorithm. + * + * @param out The output stream to compress + * @param blockSize Default: 4. The block size used during compression. 4=64kb, 5=256kb, 6=1mb, 7=4mb. All other + * values will generate an exception + * @param blockChecksum Default: false. When true, a XXHash32 checksum is computed and appended to the stream for + * every block of data + * @throws IOException + */ + public KafkaLZ4BlockOutputStream(OutputStream out, int blockSize, boolean blockChecksum) throws IOException { + super(out); + compressor = LZ4Factory.fastestInstance().fastCompressor(); + checksum = XXHashFactory.fastestInstance().hash32(); + bd = new BD(blockSize); + flg = new FLG(blockChecksum); + bufferOffset = 0; + maxBlockSize = bd.getBlockMaximumSize(); + buffer = new byte[maxBlockSize]; + compressedBuffer = new byte[compressor.maxCompressedLength(maxBlockSize)]; + finished = false; + writeHeader(); + } + + /** + * Create a new {@link OutputStream} that will compress data using the LZ4 algorithm. + * + * @param out The stream to compress + * @param blockSize Default: 4. The block size used during compression. 4=64kb, 5=256kb, 6=1mb, 7=4mb. All other + * values will generate an exception + * @throws IOException + */ + public KafkaLZ4BlockOutputStream(OutputStream out, int blockSize) throws IOException { + this(out, blockSize, false); + } + + /** + * Create a new {@link OutputStream} that will compress data using the LZ4 algorithm. + * + * @param out The output stream to compress + * @throws IOException + */ + public KafkaLZ4BlockOutputStream(OutputStream out) throws IOException { + this(out, BLOCKSIZE_64KB); + } + + /** + * Writes the magic number and frame descriptor to the underlying {@link OutputStream}. + * + * @throws IOException + */ + private void writeHeader() throws IOException { + Utils.writeUnsignedIntLE(buffer, 0, MAGIC); + bufferOffset = 4; + buffer[bufferOffset++] = flg.toByte(); + buffer[bufferOffset++] = bd.toByte(); + // TODO write uncompressed content size, update flg.validate() + // TODO write dictionary id, update flg.validate() + // compute checksum on all descriptor fields + int hash = (checksum.hash(buffer, 0, bufferOffset, 0) >> 8) & 0xFF; + buffer[bufferOffset++] = (byte) hash; + // write out frame descriptor + out.write(buffer, 0, bufferOffset); + bufferOffset = 0; + } + + /** + * Compresses buffered data, optionally computes an XXHash32 checksum, and writes the result to the underlying + * {@link OutputStream}. + * + * @throws IOException + */ + private void writeBlock() throws IOException { + if (bufferOffset == 0) { + return; + } + + int compressedLength = compressor.compress(buffer, 0, bufferOffset, compressedBuffer, 0); + byte[] bufferToWrite = compressedBuffer; + int compressMethod = 0; + + // Store block uncompressed if compressed length is greater (incompressible) + if (compressedLength >= bufferOffset) { + bufferToWrite = buffer; + compressedLength = bufferOffset; + compressMethod = LZ4_FRAME_INCOMPRESSIBLE_MASK; + } + + // Write content + Utils.writeUnsignedIntLE(out, compressedLength | compressMethod); + out.write(bufferToWrite, 0, compressedLength); + + // Calculate and write block checksum + if (flg.isBlockChecksumSet()) { + int hash = checksum.hash(bufferToWrite, 0, compressedLength, 0); + Utils.writeUnsignedIntLE(out, hash); + } + bufferOffset = 0; + } + + /** + * Similar to the {@link #writeBlock()} method. Writes a 0-length block (without block checksum) to signal the end + * of the block stream. + * + * @throws IOException + */ + private void writeEndMark() throws IOException { + Utils.writeUnsignedIntLE(out, 0); + // TODO implement content checksum, update flg.validate() + finished = true; + } + + @Override + public void write(int b) throws IOException { + ensureNotFinished(); + if (bufferOffset == maxBlockSize) { + writeBlock(); + } + buffer[bufferOffset++] = (byte) b; + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + net.jpountz.util.Utils.checkRange(b, off, len); + ensureNotFinished(); + + int bufferRemainingLength = maxBlockSize - bufferOffset; + // while b will fill the buffer + while (len > bufferRemainingLength) { + // fill remaining space in buffer + System.arraycopy(b, off, buffer, bufferOffset, bufferRemainingLength); + bufferOffset = maxBlockSize; + writeBlock(); + // compute new offset and length + off += bufferRemainingLength; + len -= bufferRemainingLength; + bufferRemainingLength = maxBlockSize; + } + + System.arraycopy(b, off, buffer, bufferOffset, len); + bufferOffset += len; + } + + @Override + public void flush() throws IOException { + if (!finished) { + writeBlock(); + } + if (out != null) { + out.flush(); + } + } + + /** + * A simple state check to ensure the stream is still open. + */ + private void ensureNotFinished() { + if (finished) { + throw new IllegalStateException(CLOSED_STREAM); + } + } + + @Override + public void close() throws IOException { + if (!finished) { + writeEndMark(); + flush(); + finished = true; + } + if (out != null) { + out.close(); + out = null; + } + } + + public static class FLG { + + private static final int VERSION = 1; + + private final int presetDictionary; + private final int reserved1; + private final int contentChecksum; + private final int contentSize; + private final int blockChecksum; + private final int blockIndependence; + private final int version; + + public FLG() { + this(false); + } + + public FLG(boolean blockChecksum) { + this(0, 0, 0, 0, blockChecksum ? 1 : 0, 1, VERSION); + } + + private FLG(int presetDictionary, + int reserved1, + int contentChecksum, + int contentSize, + int blockChecksum, + int blockIndependence, + int version) { + this.presetDictionary = presetDictionary; + this.reserved1 = reserved1; + this.contentChecksum = contentChecksum; + this.contentSize = contentSize; + this.blockChecksum = blockChecksum; + this.blockIndependence = blockIndependence; + this.version = version; + validate(); + } + + public static FLG fromByte(byte flg) { + int presetDictionary = (flg >>> 0) & 1; + int reserved1 = (flg >>> 1) & 1; + int contentChecksum = (flg >>> 2) & 1; + int contentSize = (flg >>> 3) & 1; + int blockChecksum = (flg >>> 4) & 1; + int blockIndependence = (flg >>> 5) & 1; + int version = (flg >>> 6) & 3; + + return new FLG(presetDictionary, + reserved1, + contentChecksum, + contentSize, + blockChecksum, + blockIndependence, + version); + } + + public byte toByte() { + return (byte) (((presetDictionary & 1) << 0) | ((reserved1 & 1) << 1) | ((contentChecksum & 1) << 2) + | ((contentSize & 1) << 3) | ((blockChecksum & 1) << 4) | ((blockIndependence & 1) << 5) | ((version & 3) << 6)); + } + + private void validate() { + if (presetDictionary != 0) { + throw new RuntimeException("Preset dictionary is unsupported"); + } + if (reserved1 != 0) { + throw new RuntimeException("Reserved1 field must be 0"); + } + if (contentChecksum != 0) { + throw new RuntimeException("Content checksum is unsupported"); + } + if (contentSize != 0) { + throw new RuntimeException("Content size is unsupported"); + } + if (blockIndependence != 1) { + throw new RuntimeException("Dependent block stream is unsupported"); + } + if (version != VERSION) { + throw new RuntimeException(String.format("Version %d is unsupported", version)); + } + } + + public boolean isPresetDictionarySet() { + return presetDictionary == 1; + } + + public boolean isContentChecksumSet() { + return contentChecksum == 1; + } + + public boolean isContentSizeSet() { + return contentSize == 1; + } + + public boolean isBlockChecksumSet() { + return blockChecksum == 1; + } + + public boolean isBlockIndependenceSet() { + return blockIndependence == 1; + } + + public int getVersion() { + return version; + } + } + + public static class BD { + + private final int reserved2; + private final int blockSizeValue; + private final int reserved3; + + public BD() { + this(0, BLOCKSIZE_64KB, 0); + } + + public BD(int blockSizeValue) { + this(0, blockSizeValue, 0); + } + + private BD(int reserved2, int blockSizeValue, int reserved3) { + this.reserved2 = reserved2; + this.blockSizeValue = blockSizeValue; + this.reserved3 = reserved3; + validate(); + } + + public static BD fromByte(byte bd) { + int reserved2 = (bd >>> 0) & 15; + int blockMaximumSize = (bd >>> 4) & 7; + int reserved3 = (bd >>> 7) & 1; + + return new BD(reserved2, blockMaximumSize, reserved3); + } + + private void validate() { + if (reserved2 != 0) { + throw new RuntimeException("Reserved2 field must be 0"); + } + if (blockSizeValue < 4 || blockSizeValue > 7) { + throw new RuntimeException("Block size value must be between 4 and 7"); + } + if (reserved3 != 0) { + throw new RuntimeException("Reserved3 field must be 0"); + } + } + + // 2^(2n+8) + public int getBlockMaximumSize() { + return 1 << ((2 * blockSizeValue) + 8); + } + + public byte toByte() { + return (byte) (((reserved2 & 15) << 0) | ((blockSizeValue & 7) << 4) | ((reserved3 & 1) << 7)); + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index cc4084faec15e..083e7a39249ab 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -164,21 +164,21 @@ public Iterator iterator() { @Override public String toString() { - Iterator iter = iterator(); - StringBuilder builder = new StringBuilder(); - builder.append('['); - while(iter.hasNext()) { - LogEntry entry = iter.next(); - builder.append('('); - builder.append("offset="); - builder.append(entry.offset()); - builder.append(","); - builder.append("record="); - builder.append(entry.record()); - builder.append(")"); - } - builder.append(']'); - return builder.toString(); + Iterator iter = iterator(); + StringBuilder builder = new StringBuilder(); + builder.append('['); + while (iter.hasNext()) { + LogEntry entry = iter.next(); + builder.append('('); + builder.append("offset="); + builder.append(entry.offset()); + builder.append(","); + builder.append("record="); + builder.append(entry.record()); + builder.append(")"); + } + builder.append(']'); + return builder.toString(); } public static class RecordsIterator extends AbstractIterator { @@ -218,8 +218,8 @@ protected LogEntry makeNext() { if (type == CompressionType.NONE) { rec = buffer.slice(); int newPos = buffer.position() + size; - if(newPos > buffer.limit()) - return allDone(); + if (newPos > buffer.limit()) + return allDone(); buffer.position(newPos); rec.limit(size); } else { @@ -251,7 +251,7 @@ protected LogEntry makeNext() { } private boolean innerDone() { - return (innerIter == null || !innerIter.hasNext()); + return innerIter == null || !innerIter.hasNext(); } } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java index 4c99d4a3c423e..1651e75dedf32 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java @@ -43,6 +43,6 @@ public String groupId() { } public static ConsumerMetadataRequest parse(ByteBuffer buffer) { - return new ConsumerMetadataRequest(((Struct) CURRENT_SCHEMA.read(buffer))); + return new ConsumerMetadataRequest((Struct) CURRENT_SCHEMA.read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java index 173333be3afc4..0c250c389a445 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java @@ -65,6 +65,6 @@ public Node node() { } public static ConsumerMetadataResponse parse(ByteBuffer buffer) { - return new ConsumerMetadataResponse(((Struct) CURRENT_SCHEMA.read(buffer))); + return new ConsumerMetadataResponse((Struct) CURRENT_SCHEMA.read(buffer)); } } \ No newline at end of file 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 2529a09767934..721e7d3f53247 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 @@ -135,6 +135,6 @@ public Map fetchData() { } public static FetchRequest parse(ByteBuffer buffer) { - return new FetchRequest(((Struct) CURRENT_SCHEMA.read(buffer))); + return new FetchRequest((Struct) CURRENT_SCHEMA.read(buffer)); } } 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 c1e5f44beae05..e67c4c8332cb1 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 @@ -106,6 +106,6 @@ public Map responseData() { } public static FetchResponse parse(ByteBuffer buffer) { - return new FetchResponse(((Struct) CURRENT_SCHEMA.read(buffer))); + return new FetchResponse((Struct) CURRENT_SCHEMA.read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java index cfdb5de523cc3..6943878116a97 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java @@ -60,6 +60,6 @@ public String consumerId() { } public static HeartbeatRequest parse(ByteBuffer buffer) { - return new HeartbeatRequest(((Struct) CURRENT_SCHEMA.read(buffer))); + return new HeartbeatRequest((Struct) CURRENT_SCHEMA.read(buffer)); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java index ea964f7c7bd8a..0057496228fee 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java @@ -41,6 +41,6 @@ public short errorCode() { } public static HeartbeatResponse parse(ByteBuffer buffer) { - return new HeartbeatResponse(((Struct) CURRENT_SCHEMA.read(buffer))); + return new HeartbeatResponse((Struct) CURRENT_SCHEMA.read(buffer)); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java index a1d48c9ab1867..8c50e9be534c6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java @@ -83,6 +83,6 @@ public String strategy() { } public static JoinGroupRequest parse(ByteBuffer buffer) { - return new JoinGroupRequest(((Struct) CURRENT_SCHEMA.read(buffer))); + return new JoinGroupRequest((Struct) CURRENT_SCHEMA.read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java index 1e9f3494d6dff..52b1803d8b558 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java @@ -98,6 +98,6 @@ public List assignedPartitions() { } public static JoinGroupResponse parse(ByteBuffer buffer) { - return new JoinGroupResponse(((Struct) CURRENT_SCHEMA.read(buffer))); + return new JoinGroupResponse((Struct) CURRENT_SCHEMA.read(buffer)); } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java index 05c5fed6dd7dc..e5dc92e9bb2aa 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java @@ -58,7 +58,7 @@ public PartitionData(long timestamp, int maxNumOffsets) { } public ListOffsetRequest(Map offsetData) { - this(-1, offsetData); + this(-1, offsetData); } public ListOffsetRequest(int replicaId, Map offsetData) { @@ -114,6 +114,6 @@ public Map offsetData() { } public static ListOffsetRequest parse(ByteBuffer buffer) { - return new ListOffsetRequest(((Struct) CURRENT_SCHEMA.read(buffer))); + return new ListOffsetRequest((Struct) CURRENT_SCHEMA.read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java index b2e473e85b2b1..cfac47a4a05dc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java @@ -104,6 +104,6 @@ public Map responseData() { } public static ListOffsetResponse parse(ByteBuffer buffer) { - return new ListOffsetResponse(((Struct) CURRENT_SCHEMA.read(buffer))); + return new ListOffsetResponse((Struct) CURRENT_SCHEMA.read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java index 0186783e1abd0..5d5f52c644e9b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java @@ -48,6 +48,6 @@ public List topics() { } public static MetadataRequest parse(ByteBuffer buffer) { - return new MetadataRequest(((Struct) CURRENT_SCHEMA.read(buffer))); + return new MetadataRequest((Struct) CURRENT_SCHEMA.read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java index 13daf599635e8..90f31413d7d80 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -29,7 +29,7 @@ public class MetadataResponse extends AbstractRequestResponse { - private static Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.METADATA.id); + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.METADATA.id); private static final String BROKERS_KEY_NAME = "brokers"; private static final String TOPIC_METATDATA_KEY_NAME = "topic_metadata"; @@ -69,12 +69,12 @@ public MetadataResponse(Cluster cluster) { List topicArray = new ArrayList(); for (String topic: cluster.topics()) { Struct topicData = struct.instance(TOPIC_METATDATA_KEY_NAME); - topicData.set(TOPIC_ERROR_CODE_KEY_NAME, (short)0); // no error + topicData.set(TOPIC_ERROR_CODE_KEY_NAME, Errors.NONE.code()); topicData.set(TOPIC_KEY_NAME, topic); List partitionArray = new ArrayList(); for (PartitionInfo fetchPartitionData : cluster.partitionsForTopic(topic)) { Struct partitionData = topicData.instance(PARTITION_METADATA_KEY_NAME); - partitionData.set(PARTITION_ERROR_CODE_KEY_NAME, (short)0); // no error + partitionData.set(PARTITION_ERROR_CODE_KEY_NAME, Errors.NONE.code()); partitionData.set(PARTITION_KEY_NAME, fetchPartitionData.partition()); partitionData.set(LEADER_KEY_NAME, fetchPartitionData.leader().id()); ArrayList replicas = new ArrayList(); @@ -148,6 +148,6 @@ public Cluster cluster() { } public static MetadataResponse parse(ByteBuffer buffer) { - return new MetadataResponse(((Struct) CURRENT_SCHEMA.read(buffer))); + return new MetadataResponse((Struct) CURRENT_SCHEMA.read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java index 4fb48c8f3592d..94e9d376235b3 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java @@ -172,10 +172,10 @@ public Map offsetData() { public static OffsetCommitRequest parse(ByteBuffer buffer, int versionId) { Schema schema = ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, versionId); - return new OffsetCommitRequest(((Struct) schema.read(buffer))); + return new OffsetCommitRequest((Struct) schema.read(buffer)); } public static OffsetCommitRequest parse(ByteBuffer buffer) { - return new OffsetCommitRequest(((Struct) CURRENT_SCHEMA.read(buffer))); + return new OffsetCommitRequest((Struct) CURRENT_SCHEMA.read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java index 2ab1dc6c516a1..4d3b9ececee4b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java @@ -83,6 +83,6 @@ public Map responseData() { } public static OffsetCommitResponse parse(ByteBuffer buffer) { - return new OffsetCommitResponse(((Struct) CURRENT_SCHEMA.read(buffer))); + return new OffsetCommitResponse((Struct) CURRENT_SCHEMA.read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java index 333483fd8b50e..16c807c01628b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java @@ -83,7 +83,7 @@ public OffsetFetchRequest(Struct struct) { } } groupId = struct.getString(GROUP_ID_KEY_NAME); - } + } public String groupId() { return groupId; @@ -94,6 +94,6 @@ public List partitions() { } public static OffsetFetchRequest parse(ByteBuffer buffer) { - return new OffsetFetchRequest(((Struct) CURRENT_SCHEMA.read(buffer))); + return new OffsetFetchRequest((Struct) CURRENT_SCHEMA.read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java index 04c88c0c057b9..edbed5880dc44 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java @@ -108,6 +108,6 @@ public Map responseData() { } public static OffsetFetchResponse parse(ByteBuffer buffer) { - return new OffsetFetchResponse(((Struct) CURRENT_SCHEMA.read(buffer))); + return new OffsetFetchResponse((Struct) CURRENT_SCHEMA.read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 03a0ab1afbb7d..995f89f25b621 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -101,6 +101,6 @@ public Map partitionRecords() { } public static ProduceRequest parse(ByteBuffer buffer) { - return new ProduceRequest(((Struct) CURRENT_SCHEMA.read(buffer))); + return new ProduceRequest((Struct) CURRENT_SCHEMA.read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index e42d7dbbe7cc5..a00dcdf15d1c7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -108,6 +108,6 @@ public String toString() { } public static ProduceResponse parse(ByteBuffer buffer) { - return new ProduceResponse(((Struct) CURRENT_SCHEMA.read(buffer))); + return new ProduceResponse((Struct) CURRENT_SCHEMA.read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java index f459a2a62f7b9..14bcde7d7b336 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java @@ -26,10 +26,10 @@ */ public class RequestHeader extends AbstractRequestResponse { - private static Field API_KEY_FIELD = REQUEST_HEADER.get("api_key"); - private static Field API_VERSION_FIELD = REQUEST_HEADER.get("api_version"); - private static Field CLIENT_ID_FIELD = REQUEST_HEADER.get("client_id"); - private static Field CORRELATION_ID_FIELD = REQUEST_HEADER.get("correlation_id"); + private static final Field API_KEY_FIELD = REQUEST_HEADER.get("api_key"); + private static final Field API_VERSION_FIELD = REQUEST_HEADER.get("api_version"); + private static final Field CLIENT_ID_FIELD = REQUEST_HEADER.get("client_id"); + private static final Field CORRELATION_ID_FIELD = REQUEST_HEADER.get("correlation_id"); private final short apiKey; private final short apiVersion; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java b/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java index dd63853e15f50..e8a7ef9a0b464 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java @@ -30,7 +30,7 @@ */ public class ResponseHeader extends AbstractRequestResponse { - private static Field CORRELATION_KEY_FIELD = RESPONSE_HEADER.get("correlation_id"); + private static final Field CORRELATION_KEY_FIELD = RESPONSE_HEADER.get("correlation_id"); private final int correlationId; @@ -50,7 +50,7 @@ public int correlationId() { } public static ResponseHeader parse(ByteBuffer buffer) { - return new ResponseHeader(((Struct) Protocol.RESPONSE_HEADER.read(buffer))); + return new ResponseHeader((Struct) Protocol.RESPONSE_HEADER.read(buffer)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java b/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java index 047ca98ef6dd0..5b867001065df 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java @@ -30,7 +30,7 @@ public class Crc32 implements Checksum { /** * Compute the CRC32 of the byte array - * + * * @param bytes The array to compute the checksum for * @return The CRC32 */ @@ -40,7 +40,7 @@ public static long crc32(byte[] bytes) { /** * Compute the CRC32 of the segment of the byte array given by the specified size and offset - * + * * @param bytes The bytes to checksum * @param offset the offset at which to begin checksumming * @param size the number of bytes to checksum @@ -79,14 +79,14 @@ public void update(byte[] b, int off, int len) { final int c1 = (b[off + 1] ^ (localCrc >>>= 8)) & 0xff; final int c2 = (b[off + 2] ^ (localCrc >>>= 8)) & 0xff; final int c3 = (b[off + 3] ^ (localCrc >>>= 8)) & 0xff; - localCrc = (T[T8_7_start + c0] ^ T[T8_6_start + c1]) ^ (T[T8_5_start + c2] ^ T[T8_4_start + c3]); + localCrc = (T[T8_7_START + c0] ^ T[T8_6_START + c1]) ^ (T[T8_5_START + c2] ^ T[T8_4_START + c3]); final int c4 = b[off + 4] & 0xff; final int c5 = b[off + 5] & 0xff; final int c6 = b[off + 6] & 0xff; final int c7 = b[off + 7] & 0xff; - localCrc ^= (T[T8_3_start + c4] ^ T[T8_2_start + c5]) ^ (T[T8_1_start + c6] ^ T[T8_0_start + c7]); + localCrc ^= (T[T8_3_START + c4] ^ T[T8_2_START + c5]) ^ (T[T8_1_START + c6] ^ T[T8_0_START + c7]); off += 8; len -= 8; @@ -95,19 +95,19 @@ public void update(byte[] b, int off, int len) { /* loop unroll - duff's device style */ switch (len) { case 7: - localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; + localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)]; case 6: - localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; + localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)]; case 5: - localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; + localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)]; case 4: - localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; + localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)]; case 3: - localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; + localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)]; case 2: - localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; + localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)]; case 1: - localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; + localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)]; default: /* nothing */ } @@ -118,7 +118,7 @@ public void update(byte[] b, int off, int len) { @Override final public void update(int b) { - crc = (crc >>> 8) ^ T[T8_0_start + ((crc ^ b) & 0xff)]; + crc = (crc >>> 8) ^ T[T8_0_START + ((crc ^ b) & 0xff)]; } /** @@ -131,2075 +131,257 @@ final public void updateInt(int input) { update((byte) input /* >> 0 */); } - - /* * CRC-32 lookup tables generated by the polynomial 0xEDB88320. See also TestPureJavaCrc32.Table. */ - private static final int T8_0_start = 0 * 256; - private static final int T8_1_start = 1 * 256; - private static final int T8_2_start = 2 * 256; - private static final int T8_3_start = 3 * 256; - private static final int T8_4_start = 4 * 256; - private static final int T8_5_start = 5 * 256; - private static final int T8_6_start = 6 * 256; - private static final int T8_7_start = 7 * 256; + private static final int T8_0_START = 0 * 256; + private static final int T8_1_START = 1 * 256; + private static final int T8_2_START = 2 * 256; + private static final int T8_3_START = 3 * 256; + private static final int T8_4_START = 4 * 256; + private static final int T8_5_START = 5 * 256; + private static final int T8_6_START = 6 * 256; + private static final int T8_7_START = 7 * 256; private static final int[] T = new int[] { - /* T8_0 */ - 0x00000000, - 0x77073096, - 0xEE0E612C, - 0x990951BA, - 0x076DC419, - 0x706AF48F, - 0xE963A535, - 0x9E6495A3, - 0x0EDB8832, - 0x79DCB8A4, - 0xE0D5E91E, - 0x97D2D988, - 0x09B64C2B, - 0x7EB17CBD, - 0xE7B82D07, - 0x90BF1D91, - 0x1DB71064, - 0x6AB020F2, - 0xF3B97148, - 0x84BE41DE, - 0x1ADAD47D, - 0x6DDDE4EB, - 0xF4D4B551, - 0x83D385C7, - 0x136C9856, - 0x646BA8C0, - 0xFD62F97A, - 0x8A65C9EC, - 0x14015C4F, - 0x63066CD9, - 0xFA0F3D63, - 0x8D080DF5, - 0x3B6E20C8, - 0x4C69105E, - 0xD56041E4, - 0xA2677172, - 0x3C03E4D1, - 0x4B04D447, - 0xD20D85FD, - 0xA50AB56B, - 0x35B5A8FA, - 0x42B2986C, - 0xDBBBC9D6, - 0xACBCF940, - 0x32D86CE3, - 0x45DF5C75, - 0xDCD60DCF, - 0xABD13D59, - 0x26D930AC, - 0x51DE003A, - 0xC8D75180, - 0xBFD06116, - 0x21B4F4B5, - 0x56B3C423, - 0xCFBA9599, - 0xB8BDA50F, - 0x2802B89E, - 0x5F058808, - 0xC60CD9B2, - 0xB10BE924, - 0x2F6F7C87, - 0x58684C11, - 0xC1611DAB, - 0xB6662D3D, - 0x76DC4190, - 0x01DB7106, - 0x98D220BC, - 0xEFD5102A, - 0x71B18589, - 0x06B6B51F, - 0x9FBFE4A5, - 0xE8B8D433, - 0x7807C9A2, - 0x0F00F934, - 0x9609A88E, - 0xE10E9818, - 0x7F6A0DBB, - 0x086D3D2D, - 0x91646C97, - 0xE6635C01, - 0x6B6B51F4, - 0x1C6C6162, - 0x856530D8, - 0xF262004E, - 0x6C0695ED, - 0x1B01A57B, - 0x8208F4C1, - 0xF50FC457, - 0x65B0D9C6, - 0x12B7E950, - 0x8BBEB8EA, - 0xFCB9887C, - 0x62DD1DDF, - 0x15DA2D49, - 0x8CD37CF3, - 0xFBD44C65, - 0x4DB26158, - 0x3AB551CE, - 0xA3BC0074, - 0xD4BB30E2, - 0x4ADFA541, - 0x3DD895D7, - 0xA4D1C46D, - 0xD3D6F4FB, - 0x4369E96A, - 0x346ED9FC, - 0xAD678846, - 0xDA60B8D0, - 0x44042D73, - 0x33031DE5, - 0xAA0A4C5F, - 0xDD0D7CC9, - 0x5005713C, - 0x270241AA, - 0xBE0B1010, - 0xC90C2086, - 0x5768B525, - 0x206F85B3, - 0xB966D409, - 0xCE61E49F, - 0x5EDEF90E, - 0x29D9C998, - 0xB0D09822, - 0xC7D7A8B4, - 0x59B33D17, - 0x2EB40D81, - 0xB7BD5C3B, - 0xC0BA6CAD, - 0xEDB88320, - 0x9ABFB3B6, - 0x03B6E20C, - 0x74B1D29A, - 0xEAD54739, - 0x9DD277AF, - 0x04DB2615, - 0x73DC1683, - 0xE3630B12, - 0x94643B84, - 0x0D6D6A3E, - 0x7A6A5AA8, - 0xE40ECF0B, - 0x9309FF9D, - 0x0A00AE27, - 0x7D079EB1, - 0xF00F9344, - 0x8708A3D2, - 0x1E01F268, - 0x6906C2FE, - 0xF762575D, - 0x806567CB, - 0x196C3671, - 0x6E6B06E7, - 0xFED41B76, - 0x89D32BE0, - 0x10DA7A5A, - 0x67DD4ACC, - 0xF9B9DF6F, - 0x8EBEEFF9, - 0x17B7BE43, - 0x60B08ED5, - 0xD6D6A3E8, - 0xA1D1937E, - 0x38D8C2C4, - 0x4FDFF252, - 0xD1BB67F1, - 0xA6BC5767, - 0x3FB506DD, - 0x48B2364B, - 0xD80D2BDA, - 0xAF0A1B4C, - 0x36034AF6, - 0x41047A60, - 0xDF60EFC3, - 0xA867DF55, - 0x316E8EEF, - 0x4669BE79, - 0xCB61B38C, - 0xBC66831A, - 0x256FD2A0, - 0x5268E236, - 0xCC0C7795, - 0xBB0B4703, - 0x220216B9, - 0x5505262F, - 0xC5BA3BBE, - 0xB2BD0B28, - 0x2BB45A92, - 0x5CB36A04, - 0xC2D7FFA7, - 0xB5D0CF31, - 0x2CD99E8B, - 0x5BDEAE1D, - 0x9B64C2B0, - 0xEC63F226, - 0x756AA39C, - 0x026D930A, - 0x9C0906A9, - 0xEB0E363F, - 0x72076785, - 0x05005713, - 0x95BF4A82, - 0xE2B87A14, - 0x7BB12BAE, - 0x0CB61B38, - 0x92D28E9B, - 0xE5D5BE0D, - 0x7CDCEFB7, - 0x0BDBDF21, - 0x86D3D2D4, - 0xF1D4E242, - 0x68DDB3F8, - 0x1FDA836E, - 0x81BE16CD, - 0xF6B9265B, - 0x6FB077E1, - 0x18B74777, - 0x88085AE6, - 0xFF0F6A70, - 0x66063BCA, - 0x11010B5C, - 0x8F659EFF, - 0xF862AE69, - 0x616BFFD3, - 0x166CCF45, - 0xA00AE278, - 0xD70DD2EE, - 0x4E048354, - 0x3903B3C2, - 0xA7672661, - 0xD06016F7, - 0x4969474D, - 0x3E6E77DB, - 0xAED16A4A, - 0xD9D65ADC, - 0x40DF0B66, - 0x37D83BF0, - 0xA9BCAE53, - 0xDEBB9EC5, - 0x47B2CF7F, - 0x30B5FFE9, - 0xBDBDF21C, - 0xCABAC28A, - 0x53B39330, - 0x24B4A3A6, - 0xBAD03605, - 0xCDD70693, - 0x54DE5729, - 0x23D967BF, - 0xB3667A2E, - 0xC4614AB8, - 0x5D681B02, - 0x2A6F2B94, - 0xB40BBE37, - 0xC30C8EA1, - 0x5A05DF1B, - 0x2D02EF8D, - /* T8_1 */ - 0x00000000, - 0x191B3141, - 0x32366282, - 0x2B2D53C3, - 0x646CC504, - 0x7D77F445, - 0x565AA786, - 0x4F4196C7, - 0xC8D98A08, - 0xD1C2BB49, - 0xFAEFE88A, - 0xE3F4D9CB, - 0xACB54F0C, - 0xB5AE7E4D, - 0x9E832D8E, - 0x87981CCF, - 0x4AC21251, - 0x53D92310, - 0x78F470D3, - 0x61EF4192, - 0x2EAED755, - 0x37B5E614, - 0x1C98B5D7, - 0x05838496, - 0x821B9859, - 0x9B00A918, - 0xB02DFADB, - 0xA936CB9A, - 0xE6775D5D, - 0xFF6C6C1C, - 0xD4413FDF, - 0xCD5A0E9E, - 0x958424A2, - 0x8C9F15E3, - 0xA7B24620, - 0xBEA97761, - 0xF1E8E1A6, - 0xE8F3D0E7, - 0xC3DE8324, - 0xDAC5B265, - 0x5D5DAEAA, - 0x44469FEB, - 0x6F6BCC28, - 0x7670FD69, - 0x39316BAE, - 0x202A5AEF, - 0x0B07092C, - 0x121C386D, - 0xDF4636F3, - 0xC65D07B2, - 0xED705471, - 0xF46B6530, - 0xBB2AF3F7, - 0xA231C2B6, - 0x891C9175, - 0x9007A034, - 0x179FBCFB, - 0x0E848DBA, - 0x25A9DE79, - 0x3CB2EF38, - 0x73F379FF, - 0x6AE848BE, - 0x41C51B7D, - 0x58DE2A3C, - 0xF0794F05, - 0xE9627E44, - 0xC24F2D87, - 0xDB541CC6, - 0x94158A01, - 0x8D0EBB40, - 0xA623E883, - 0xBF38D9C2, - 0x38A0C50D, - 0x21BBF44C, - 0x0A96A78F, - 0x138D96CE, - 0x5CCC0009, - 0x45D73148, - 0x6EFA628B, - 0x77E153CA, - 0xBABB5D54, - 0xA3A06C15, - 0x888D3FD6, - 0x91960E97, - 0xDED79850, - 0xC7CCA911, - 0xECE1FAD2, - 0xF5FACB93, - 0x7262D75C, - 0x6B79E61D, - 0x4054B5DE, - 0x594F849F, - 0x160E1258, - 0x0F152319, - 0x243870DA, - 0x3D23419B, - 0x65FD6BA7, - 0x7CE65AE6, - 0x57CB0925, - 0x4ED03864, - 0x0191AEA3, - 0x188A9FE2, - 0x33A7CC21, - 0x2ABCFD60, - 0xAD24E1AF, - 0xB43FD0EE, - 0x9F12832D, - 0x8609B26C, - 0xC94824AB, - 0xD05315EA, - 0xFB7E4629, - 0xE2657768, - 0x2F3F79F6, - 0x362448B7, - 0x1D091B74, - 0x04122A35, - 0x4B53BCF2, - 0x52488DB3, - 0x7965DE70, - 0x607EEF31, - 0xE7E6F3FE, - 0xFEFDC2BF, - 0xD5D0917C, - 0xCCCBA03D, - 0x838A36FA, - 0x9A9107BB, - 0xB1BC5478, - 0xA8A76539, - 0x3B83984B, - 0x2298A90A, - 0x09B5FAC9, - 0x10AECB88, - 0x5FEF5D4F, - 0x46F46C0E, - 0x6DD93FCD, - 0x74C20E8C, - 0xF35A1243, - 0xEA412302, - 0xC16C70C1, - 0xD8774180, - 0x9736D747, - 0x8E2DE606, - 0xA500B5C5, - 0xBC1B8484, - 0x71418A1A, - 0x685ABB5B, - 0x4377E898, - 0x5A6CD9D9, - 0x152D4F1E, - 0x0C367E5F, - 0x271B2D9C, - 0x3E001CDD, - 0xB9980012, - 0xA0833153, - 0x8BAE6290, - 0x92B553D1, - 0xDDF4C516, - 0xC4EFF457, - 0xEFC2A794, - 0xF6D996D5, - 0xAE07BCE9, - 0xB71C8DA8, - 0x9C31DE6B, - 0x852AEF2A, - 0xCA6B79ED, - 0xD37048AC, - 0xF85D1B6F, - 0xE1462A2E, - 0x66DE36E1, - 0x7FC507A0, - 0x54E85463, - 0x4DF36522, - 0x02B2F3E5, - 0x1BA9C2A4, - 0x30849167, - 0x299FA026, - 0xE4C5AEB8, - 0xFDDE9FF9, - 0xD6F3CC3A, - 0xCFE8FD7B, - 0x80A96BBC, - 0x99B25AFD, - 0xB29F093E, - 0xAB84387F, - 0x2C1C24B0, - 0x350715F1, - 0x1E2A4632, - 0x07317773, - 0x4870E1B4, - 0x516BD0F5, - 0x7A468336, - 0x635DB277, - 0xCBFAD74E, - 0xD2E1E60F, - 0xF9CCB5CC, - 0xE0D7848D, - 0xAF96124A, - 0xB68D230B, - 0x9DA070C8, - 0x84BB4189, - 0x03235D46, - 0x1A386C07, - 0x31153FC4, - 0x280E0E85, - 0x674F9842, - 0x7E54A903, - 0x5579FAC0, - 0x4C62CB81, - 0x8138C51F, - 0x9823F45E, - 0xB30EA79D, - 0xAA1596DC, - 0xE554001B, - 0xFC4F315A, - 0xD7626299, - 0xCE7953D8, - 0x49E14F17, - 0x50FA7E56, - 0x7BD72D95, - 0x62CC1CD4, - 0x2D8D8A13, - 0x3496BB52, - 0x1FBBE891, - 0x06A0D9D0, - 0x5E7EF3EC, - 0x4765C2AD, - 0x6C48916E, - 0x7553A02F, - 0x3A1236E8, - 0x230907A9, - 0x0824546A, - 0x113F652B, - 0x96A779E4, - 0x8FBC48A5, - 0xA4911B66, - 0xBD8A2A27, - 0xF2CBBCE0, - 0xEBD08DA1, - 0xC0FDDE62, - 0xD9E6EF23, - 0x14BCE1BD, - 0x0DA7D0FC, - 0x268A833F, - 0x3F91B27E, - 0x70D024B9, - 0x69CB15F8, - 0x42E6463B, - 0x5BFD777A, - 0xDC656BB5, - 0xC57E5AF4, - 0xEE530937, - 0xF7483876, - 0xB809AEB1, - 0xA1129FF0, - 0x8A3FCC33, - 0x9324FD72, - /* T8_2 */ - 0x00000000, - 0x01C26A37, - 0x0384D46E, - 0x0246BE59, - 0x0709A8DC, - 0x06CBC2EB, - 0x048D7CB2, - 0x054F1685, - 0x0E1351B8, - 0x0FD13B8F, - 0x0D9785D6, - 0x0C55EFE1, - 0x091AF964, - 0x08D89353, - 0x0A9E2D0A, - 0x0B5C473D, - 0x1C26A370, - 0x1DE4C947, - 0x1FA2771E, - 0x1E601D29, - 0x1B2F0BAC, - 0x1AED619B, - 0x18ABDFC2, - 0x1969B5F5, - 0x1235F2C8, - 0x13F798FF, - 0x11B126A6, - 0x10734C91, - 0x153C5A14, - 0x14FE3023, - 0x16B88E7A, - 0x177AE44D, - 0x384D46E0, - 0x398F2CD7, - 0x3BC9928E, - 0x3A0BF8B9, - 0x3F44EE3C, - 0x3E86840B, - 0x3CC03A52, - 0x3D025065, - 0x365E1758, - 0x379C7D6F, - 0x35DAC336, - 0x3418A901, - 0x3157BF84, - 0x3095D5B3, - 0x32D36BEA, - 0x331101DD, - 0x246BE590, - 0x25A98FA7, - 0x27EF31FE, - 0x262D5BC9, - 0x23624D4C, - 0x22A0277B, - 0x20E69922, - 0x2124F315, - 0x2A78B428, - 0x2BBADE1F, - 0x29FC6046, - 0x283E0A71, - 0x2D711CF4, - 0x2CB376C3, - 0x2EF5C89A, - 0x2F37A2AD, - 0x709A8DC0, - 0x7158E7F7, - 0x731E59AE, - 0x72DC3399, - 0x7793251C, - 0x76514F2B, - 0x7417F172, - 0x75D59B45, - 0x7E89DC78, - 0x7F4BB64F, - 0x7D0D0816, - 0x7CCF6221, - 0x798074A4, - 0x78421E93, - 0x7A04A0CA, - 0x7BC6CAFD, - 0x6CBC2EB0, - 0x6D7E4487, - 0x6F38FADE, - 0x6EFA90E9, - 0x6BB5866C, - 0x6A77EC5B, - 0x68315202, - 0x69F33835, - 0x62AF7F08, - 0x636D153F, - 0x612BAB66, - 0x60E9C151, - 0x65A6D7D4, - 0x6464BDE3, - 0x662203BA, - 0x67E0698D, - 0x48D7CB20, - 0x4915A117, - 0x4B531F4E, - 0x4A917579, - 0x4FDE63FC, - 0x4E1C09CB, - 0x4C5AB792, - 0x4D98DDA5, - 0x46C49A98, - 0x4706F0AF, - 0x45404EF6, - 0x448224C1, - 0x41CD3244, - 0x400F5873, - 0x4249E62A, - 0x438B8C1D, - 0x54F16850, - 0x55330267, - 0x5775BC3E, - 0x56B7D609, - 0x53F8C08C, - 0x523AAABB, - 0x507C14E2, - 0x51BE7ED5, - 0x5AE239E8, - 0x5B2053DF, - 0x5966ED86, - 0x58A487B1, - 0x5DEB9134, - 0x5C29FB03, - 0x5E6F455A, - 0x5FAD2F6D, - 0xE1351B80, - 0xE0F771B7, - 0xE2B1CFEE, - 0xE373A5D9, - 0xE63CB35C, - 0xE7FED96B, - 0xE5B86732, - 0xE47A0D05, - 0xEF264A38, - 0xEEE4200F, - 0xECA29E56, - 0xED60F461, - 0xE82FE2E4, - 0xE9ED88D3, - 0xEBAB368A, - 0xEA695CBD, - 0xFD13B8F0, - 0xFCD1D2C7, - 0xFE976C9E, - 0xFF5506A9, - 0xFA1A102C, - 0xFBD87A1B, - 0xF99EC442, - 0xF85CAE75, - 0xF300E948, - 0xF2C2837F, - 0xF0843D26, - 0xF1465711, - 0xF4094194, - 0xF5CB2BA3, - 0xF78D95FA, - 0xF64FFFCD, - 0xD9785D60, - 0xD8BA3757, - 0xDAFC890E, - 0xDB3EE339, - 0xDE71F5BC, - 0xDFB39F8B, - 0xDDF521D2, - 0xDC374BE5, - 0xD76B0CD8, - 0xD6A966EF, - 0xD4EFD8B6, - 0xD52DB281, - 0xD062A404, - 0xD1A0CE33, - 0xD3E6706A, - 0xD2241A5D, - 0xC55EFE10, - 0xC49C9427, - 0xC6DA2A7E, - 0xC7184049, - 0xC25756CC, - 0xC3953CFB, - 0xC1D382A2, - 0xC011E895, - 0xCB4DAFA8, - 0xCA8FC59F, - 0xC8C97BC6, - 0xC90B11F1, - 0xCC440774, - 0xCD866D43, - 0xCFC0D31A, - 0xCE02B92D, - 0x91AF9640, - 0x906DFC77, - 0x922B422E, - 0x93E92819, - 0x96A63E9C, - 0x976454AB, - 0x9522EAF2, - 0x94E080C5, - 0x9FBCC7F8, - 0x9E7EADCF, - 0x9C381396, - 0x9DFA79A1, - 0x98B56F24, - 0x99770513, - 0x9B31BB4A, - 0x9AF3D17D, - 0x8D893530, - 0x8C4B5F07, - 0x8E0DE15E, - 0x8FCF8B69, - 0x8A809DEC, - 0x8B42F7DB, - 0x89044982, - 0x88C623B5, - 0x839A6488, - 0x82580EBF, - 0x801EB0E6, - 0x81DCDAD1, - 0x8493CC54, - 0x8551A663, - 0x8717183A, - 0x86D5720D, - 0xA9E2D0A0, - 0xA820BA97, - 0xAA6604CE, - 0xABA46EF9, - 0xAEEB787C, - 0xAF29124B, - 0xAD6FAC12, - 0xACADC625, - 0xA7F18118, - 0xA633EB2F, - 0xA4755576, - 0xA5B73F41, - 0xA0F829C4, - 0xA13A43F3, - 0xA37CFDAA, - 0xA2BE979D, - 0xB5C473D0, - 0xB40619E7, - 0xB640A7BE, - 0xB782CD89, - 0xB2CDDB0C, - 0xB30FB13B, - 0xB1490F62, - 0xB08B6555, - 0xBBD72268, - 0xBA15485F, - 0xB853F606, - 0xB9919C31, - 0xBCDE8AB4, - 0xBD1CE083, - 0xBF5A5EDA, - 0xBE9834ED, - /* T8_3 */ - 0x00000000, - 0xB8BC6765, - 0xAA09C88B, - 0x12B5AFEE, - 0x8F629757, - 0x37DEF032, - 0x256B5FDC, - 0x9DD738B9, - 0xC5B428EF, - 0x7D084F8A, - 0x6FBDE064, - 0xD7018701, - 0x4AD6BFB8, - 0xF26AD8DD, - 0xE0DF7733, - 0x58631056, - 0x5019579F, - 0xE8A530FA, - 0xFA109F14, - 0x42ACF871, - 0xDF7BC0C8, - 0x67C7A7AD, - 0x75720843, - 0xCDCE6F26, - 0x95AD7F70, - 0x2D111815, - 0x3FA4B7FB, - 0x8718D09E, - 0x1ACFE827, - 0xA2738F42, - 0xB0C620AC, - 0x087A47C9, - 0xA032AF3E, - 0x188EC85B, - 0x0A3B67B5, - 0xB28700D0, - 0x2F503869, - 0x97EC5F0C, - 0x8559F0E2, - 0x3DE59787, - 0x658687D1, - 0xDD3AE0B4, - 0xCF8F4F5A, - 0x7733283F, - 0xEAE41086, - 0x525877E3, - 0x40EDD80D, - 0xF851BF68, - 0xF02BF8A1, - 0x48979FC4, - 0x5A22302A, - 0xE29E574F, - 0x7F496FF6, - 0xC7F50893, - 0xD540A77D, - 0x6DFCC018, - 0x359FD04E, - 0x8D23B72B, - 0x9F9618C5, - 0x272A7FA0, - 0xBAFD4719, - 0x0241207C, - 0x10F48F92, - 0xA848E8F7, - 0x9B14583D, - 0x23A83F58, - 0x311D90B6, - 0x89A1F7D3, - 0x1476CF6A, - 0xACCAA80F, - 0xBE7F07E1, - 0x06C36084, - 0x5EA070D2, - 0xE61C17B7, - 0xF4A9B859, - 0x4C15DF3C, - 0xD1C2E785, - 0x697E80E0, - 0x7BCB2F0E, - 0xC377486B, - 0xCB0D0FA2, - 0x73B168C7, - 0x6104C729, - 0xD9B8A04C, - 0x446F98F5, - 0xFCD3FF90, - 0xEE66507E, - 0x56DA371B, - 0x0EB9274D, - 0xB6054028, - 0xA4B0EFC6, - 0x1C0C88A3, - 0x81DBB01A, - 0x3967D77F, - 0x2BD27891, - 0x936E1FF4, - 0x3B26F703, - 0x839A9066, - 0x912F3F88, - 0x299358ED, - 0xB4446054, - 0x0CF80731, - 0x1E4DA8DF, - 0xA6F1CFBA, - 0xFE92DFEC, - 0x462EB889, - 0x549B1767, - 0xEC277002, - 0x71F048BB, - 0xC94C2FDE, - 0xDBF98030, - 0x6345E755, - 0x6B3FA09C, - 0xD383C7F9, - 0xC1366817, - 0x798A0F72, - 0xE45D37CB, - 0x5CE150AE, - 0x4E54FF40, - 0xF6E89825, - 0xAE8B8873, - 0x1637EF16, - 0x048240F8, - 0xBC3E279D, - 0x21E91F24, - 0x99557841, - 0x8BE0D7AF, - 0x335CB0CA, - 0xED59B63B, - 0x55E5D15E, - 0x47507EB0, - 0xFFEC19D5, - 0x623B216C, - 0xDA874609, - 0xC832E9E7, - 0x708E8E82, - 0x28ED9ED4, - 0x9051F9B1, - 0x82E4565F, - 0x3A58313A, - 0xA78F0983, - 0x1F336EE6, - 0x0D86C108, - 0xB53AA66D, - 0xBD40E1A4, - 0x05FC86C1, - 0x1749292F, - 0xAFF54E4A, - 0x322276F3, - 0x8A9E1196, - 0x982BBE78, - 0x2097D91D, - 0x78F4C94B, - 0xC048AE2E, - 0xD2FD01C0, - 0x6A4166A5, - 0xF7965E1C, - 0x4F2A3979, - 0x5D9F9697, - 0xE523F1F2, - 0x4D6B1905, - 0xF5D77E60, - 0xE762D18E, - 0x5FDEB6EB, - 0xC2098E52, - 0x7AB5E937, - 0x680046D9, - 0xD0BC21BC, - 0x88DF31EA, - 0x3063568F, - 0x22D6F961, - 0x9A6A9E04, - 0x07BDA6BD, - 0xBF01C1D8, - 0xADB46E36, - 0x15080953, - 0x1D724E9A, - 0xA5CE29FF, - 0xB77B8611, - 0x0FC7E174, - 0x9210D9CD, - 0x2AACBEA8, - 0x38191146, - 0x80A57623, - 0xD8C66675, - 0x607A0110, - 0x72CFAEFE, - 0xCA73C99B, - 0x57A4F122, - 0xEF189647, - 0xFDAD39A9, - 0x45115ECC, - 0x764DEE06, - 0xCEF18963, - 0xDC44268D, - 0x64F841E8, - 0xF92F7951, - 0x41931E34, - 0x5326B1DA, - 0xEB9AD6BF, - 0xB3F9C6E9, - 0x0B45A18C, - 0x19F00E62, - 0xA14C6907, - 0x3C9B51BE, - 0x842736DB, - 0x96929935, - 0x2E2EFE50, - 0x2654B999, - 0x9EE8DEFC, - 0x8C5D7112, - 0x34E11677, - 0xA9362ECE, - 0x118A49AB, - 0x033FE645, - 0xBB838120, - 0xE3E09176, - 0x5B5CF613, - 0x49E959FD, - 0xF1553E98, - 0x6C820621, - 0xD43E6144, - 0xC68BCEAA, - 0x7E37A9CF, - 0xD67F4138, - 0x6EC3265D, - 0x7C7689B3, - 0xC4CAEED6, - 0x591DD66F, - 0xE1A1B10A, - 0xF3141EE4, - 0x4BA87981, - 0x13CB69D7, - 0xAB770EB2, - 0xB9C2A15C, - 0x017EC639, - 0x9CA9FE80, - 0x241599E5, - 0x36A0360B, - 0x8E1C516E, - 0x866616A7, - 0x3EDA71C2, - 0x2C6FDE2C, - 0x94D3B949, - 0x090481F0, - 0xB1B8E695, - 0xA30D497B, - 0x1BB12E1E, - 0x43D23E48, - 0xFB6E592D, - 0xE9DBF6C3, - 0x516791A6, - 0xCCB0A91F, - 0x740CCE7A, - 0x66B96194, - 0xDE0506F1, - /* T8_4 */ - 0x00000000, - 0x3D6029B0, - 0x7AC05360, - 0x47A07AD0, - 0xF580A6C0, - 0xC8E08F70, - 0x8F40F5A0, - 0xB220DC10, - 0x30704BC1, - 0x0D106271, - 0x4AB018A1, - 0x77D03111, - 0xC5F0ED01, - 0xF890C4B1, - 0xBF30BE61, - 0x825097D1, - 0x60E09782, - 0x5D80BE32, - 0x1A20C4E2, - 0x2740ED52, - 0x95603142, - 0xA80018F2, - 0xEFA06222, - 0xD2C04B92, - 0x5090DC43, - 0x6DF0F5F3, - 0x2A508F23, - 0x1730A693, - 0xA5107A83, - 0x98705333, - 0xDFD029E3, - 0xE2B00053, - 0xC1C12F04, - 0xFCA106B4, - 0xBB017C64, - 0x866155D4, - 0x344189C4, - 0x0921A074, - 0x4E81DAA4, - 0x73E1F314, - 0xF1B164C5, - 0xCCD14D75, - 0x8B7137A5, - 0xB6111E15, - 0x0431C205, - 0x3951EBB5, - 0x7EF19165, - 0x4391B8D5, - 0xA121B886, - 0x9C419136, - 0xDBE1EBE6, - 0xE681C256, - 0x54A11E46, - 0x69C137F6, - 0x2E614D26, - 0x13016496, - 0x9151F347, - 0xAC31DAF7, - 0xEB91A027, - 0xD6F18997, - 0x64D15587, - 0x59B17C37, - 0x1E1106E7, - 0x23712F57, - 0x58F35849, - 0x659371F9, - 0x22330B29, - 0x1F532299, - 0xAD73FE89, - 0x9013D739, - 0xD7B3ADE9, - 0xEAD38459, - 0x68831388, - 0x55E33A38, - 0x124340E8, - 0x2F236958, - 0x9D03B548, - 0xA0639CF8, - 0xE7C3E628, - 0xDAA3CF98, - 0x3813CFCB, - 0x0573E67B, - 0x42D39CAB, - 0x7FB3B51B, - 0xCD93690B, - 0xF0F340BB, - 0xB7533A6B, - 0x8A3313DB, - 0x0863840A, - 0x3503ADBA, - 0x72A3D76A, - 0x4FC3FEDA, - 0xFDE322CA, - 0xC0830B7A, - 0x872371AA, - 0xBA43581A, - 0x9932774D, - 0xA4525EFD, - 0xE3F2242D, - 0xDE920D9D, - 0x6CB2D18D, - 0x51D2F83D, - 0x167282ED, - 0x2B12AB5D, - 0xA9423C8C, - 0x9422153C, - 0xD3826FEC, - 0xEEE2465C, - 0x5CC29A4C, - 0x61A2B3FC, - 0x2602C92C, - 0x1B62E09C, - 0xF9D2E0CF, - 0xC4B2C97F, - 0x8312B3AF, - 0xBE729A1F, - 0x0C52460F, - 0x31326FBF, - 0x7692156F, - 0x4BF23CDF, - 0xC9A2AB0E, - 0xF4C282BE, - 0xB362F86E, - 0x8E02D1DE, - 0x3C220DCE, - 0x0142247E, - 0x46E25EAE, - 0x7B82771E, - 0xB1E6B092, - 0x8C869922, - 0xCB26E3F2, - 0xF646CA42, - 0x44661652, - 0x79063FE2, - 0x3EA64532, - 0x03C66C82, - 0x8196FB53, - 0xBCF6D2E3, - 0xFB56A833, - 0xC6368183, - 0x74165D93, - 0x49767423, - 0x0ED60EF3, - 0x33B62743, - 0xD1062710, - 0xEC660EA0, - 0xABC67470, - 0x96A65DC0, - 0x248681D0, - 0x19E6A860, - 0x5E46D2B0, - 0x6326FB00, - 0xE1766CD1, - 0xDC164561, - 0x9BB63FB1, - 0xA6D61601, - 0x14F6CA11, - 0x2996E3A1, - 0x6E369971, - 0x5356B0C1, - 0x70279F96, - 0x4D47B626, - 0x0AE7CCF6, - 0x3787E546, - 0x85A73956, - 0xB8C710E6, - 0xFF676A36, - 0xC2074386, - 0x4057D457, - 0x7D37FDE7, - 0x3A978737, - 0x07F7AE87, - 0xB5D77297, - 0x88B75B27, - 0xCF1721F7, - 0xF2770847, - 0x10C70814, - 0x2DA721A4, - 0x6A075B74, - 0x576772C4, - 0xE547AED4, - 0xD8278764, - 0x9F87FDB4, - 0xA2E7D404, - 0x20B743D5, - 0x1DD76A65, - 0x5A7710B5, - 0x67173905, - 0xD537E515, - 0xE857CCA5, - 0xAFF7B675, - 0x92979FC5, - 0xE915E8DB, - 0xD475C16B, - 0x93D5BBBB, - 0xAEB5920B, - 0x1C954E1B, - 0x21F567AB, - 0x66551D7B, - 0x5B3534CB, - 0xD965A31A, - 0xE4058AAA, - 0xA3A5F07A, - 0x9EC5D9CA, - 0x2CE505DA, - 0x11852C6A, - 0x562556BA, - 0x6B457F0A, - 0x89F57F59, - 0xB49556E9, - 0xF3352C39, - 0xCE550589, - 0x7C75D999, - 0x4115F029, - 0x06B58AF9, - 0x3BD5A349, - 0xB9853498, - 0x84E51D28, - 0xC34567F8, - 0xFE254E48, - 0x4C059258, - 0x7165BBE8, - 0x36C5C138, - 0x0BA5E888, - 0x28D4C7DF, - 0x15B4EE6F, - 0x521494BF, - 0x6F74BD0F, - 0xDD54611F, - 0xE03448AF, - 0xA794327F, - 0x9AF41BCF, - 0x18A48C1E, - 0x25C4A5AE, - 0x6264DF7E, - 0x5F04F6CE, - 0xED242ADE, - 0xD044036E, - 0x97E479BE, - 0xAA84500E, - 0x4834505D, - 0x755479ED, - 0x32F4033D, - 0x0F942A8D, - 0xBDB4F69D, - 0x80D4DF2D, - 0xC774A5FD, - 0xFA148C4D, - 0x78441B9C, - 0x4524322C, - 0x028448FC, - 0x3FE4614C, - 0x8DC4BD5C, - 0xB0A494EC, - 0xF704EE3C, - 0xCA64C78C, - /* T8_5 */ - 0x00000000, - 0xCB5CD3A5, - 0x4DC8A10B, - 0x869472AE, - 0x9B914216, - 0x50CD91B3, - 0xD659E31D, - 0x1D0530B8, - 0xEC53826D, - 0x270F51C8, - 0xA19B2366, - 0x6AC7F0C3, - 0x77C2C07B, - 0xBC9E13DE, - 0x3A0A6170, - 0xF156B2D5, - 0x03D6029B, - 0xC88AD13E, - 0x4E1EA390, - 0x85427035, - 0x9847408D, - 0x531B9328, - 0xD58FE186, - 0x1ED33223, - 0xEF8580F6, - 0x24D95353, - 0xA24D21FD, - 0x6911F258, - 0x7414C2E0, - 0xBF481145, - 0x39DC63EB, - 0xF280B04E, - 0x07AC0536, - 0xCCF0D693, - 0x4A64A43D, - 0x81387798, - 0x9C3D4720, - 0x57619485, - 0xD1F5E62B, - 0x1AA9358E, - 0xEBFF875B, - 0x20A354FE, - 0xA6372650, - 0x6D6BF5F5, - 0x706EC54D, - 0xBB3216E8, - 0x3DA66446, - 0xF6FAB7E3, - 0x047A07AD, - 0xCF26D408, - 0x49B2A6A6, - 0x82EE7503, - 0x9FEB45BB, - 0x54B7961E, - 0xD223E4B0, - 0x197F3715, - 0xE82985C0, - 0x23755665, - 0xA5E124CB, - 0x6EBDF76E, - 0x73B8C7D6, - 0xB8E41473, - 0x3E7066DD, - 0xF52CB578, - 0x0F580A6C, - 0xC404D9C9, - 0x4290AB67, - 0x89CC78C2, - 0x94C9487A, - 0x5F959BDF, - 0xD901E971, - 0x125D3AD4, - 0xE30B8801, - 0x28575BA4, - 0xAEC3290A, - 0x659FFAAF, - 0x789ACA17, - 0xB3C619B2, - 0x35526B1C, - 0xFE0EB8B9, - 0x0C8E08F7, - 0xC7D2DB52, - 0x4146A9FC, - 0x8A1A7A59, - 0x971F4AE1, - 0x5C439944, - 0xDAD7EBEA, - 0x118B384F, - 0xE0DD8A9A, - 0x2B81593F, - 0xAD152B91, - 0x6649F834, - 0x7B4CC88C, - 0xB0101B29, - 0x36846987, - 0xFDD8BA22, - 0x08F40F5A, - 0xC3A8DCFF, - 0x453CAE51, - 0x8E607DF4, - 0x93654D4C, - 0x58399EE9, - 0xDEADEC47, - 0x15F13FE2, - 0xE4A78D37, - 0x2FFB5E92, - 0xA96F2C3C, - 0x6233FF99, - 0x7F36CF21, - 0xB46A1C84, - 0x32FE6E2A, - 0xF9A2BD8F, - 0x0B220DC1, - 0xC07EDE64, - 0x46EAACCA, - 0x8DB67F6F, - 0x90B34FD7, - 0x5BEF9C72, - 0xDD7BEEDC, - 0x16273D79, - 0xE7718FAC, - 0x2C2D5C09, - 0xAAB92EA7, - 0x61E5FD02, - 0x7CE0CDBA, - 0xB7BC1E1F, - 0x31286CB1, - 0xFA74BF14, - 0x1EB014D8, - 0xD5ECC77D, - 0x5378B5D3, - 0x98246676, - 0x852156CE, - 0x4E7D856B, - 0xC8E9F7C5, - 0x03B52460, - 0xF2E396B5, - 0x39BF4510, - 0xBF2B37BE, - 0x7477E41B, - 0x6972D4A3, - 0xA22E0706, - 0x24BA75A8, - 0xEFE6A60D, - 0x1D661643, - 0xD63AC5E6, - 0x50AEB748, - 0x9BF264ED, - 0x86F75455, - 0x4DAB87F0, - 0xCB3FF55E, - 0x006326FB, - 0xF135942E, - 0x3A69478B, - 0xBCFD3525, - 0x77A1E680, - 0x6AA4D638, - 0xA1F8059D, - 0x276C7733, - 0xEC30A496, - 0x191C11EE, - 0xD240C24B, - 0x54D4B0E5, - 0x9F886340, - 0x828D53F8, - 0x49D1805D, - 0xCF45F2F3, - 0x04192156, - 0xF54F9383, - 0x3E134026, - 0xB8873288, - 0x73DBE12D, - 0x6EDED195, - 0xA5820230, - 0x2316709E, - 0xE84AA33B, - 0x1ACA1375, - 0xD196C0D0, - 0x5702B27E, - 0x9C5E61DB, - 0x815B5163, - 0x4A0782C6, - 0xCC93F068, - 0x07CF23CD, - 0xF6999118, - 0x3DC542BD, - 0xBB513013, - 0x700DE3B6, - 0x6D08D30E, - 0xA65400AB, - 0x20C07205, - 0xEB9CA1A0, - 0x11E81EB4, - 0xDAB4CD11, - 0x5C20BFBF, - 0x977C6C1A, - 0x8A795CA2, - 0x41258F07, - 0xC7B1FDA9, - 0x0CED2E0C, - 0xFDBB9CD9, - 0x36E74F7C, - 0xB0733DD2, - 0x7B2FEE77, - 0x662ADECF, - 0xAD760D6A, - 0x2BE27FC4, - 0xE0BEAC61, - 0x123E1C2F, - 0xD962CF8A, - 0x5FF6BD24, - 0x94AA6E81, - 0x89AF5E39, - 0x42F38D9C, - 0xC467FF32, - 0x0F3B2C97, - 0xFE6D9E42, - 0x35314DE7, - 0xB3A53F49, - 0x78F9ECEC, - 0x65FCDC54, - 0xAEA00FF1, - 0x28347D5F, - 0xE368AEFA, - 0x16441B82, - 0xDD18C827, - 0x5B8CBA89, - 0x90D0692C, - 0x8DD55994, - 0x46898A31, - 0xC01DF89F, - 0x0B412B3A, - 0xFA1799EF, - 0x314B4A4A, - 0xB7DF38E4, - 0x7C83EB41, - 0x6186DBF9, - 0xAADA085C, - 0x2C4E7AF2, - 0xE712A957, - 0x15921919, - 0xDECECABC, - 0x585AB812, - 0x93066BB7, - 0x8E035B0F, - 0x455F88AA, - 0xC3CBFA04, - 0x089729A1, - 0xF9C19B74, - 0x329D48D1, - 0xB4093A7F, - 0x7F55E9DA, - 0x6250D962, - 0xA90C0AC7, - 0x2F987869, - 0xE4C4ABCC, - /* T8_6 */ - 0x00000000, - 0xA6770BB4, - 0x979F1129, - 0x31E81A9D, - 0xF44F2413, - 0x52382FA7, - 0x63D0353A, - 0xC5A73E8E, - 0x33EF4E67, - 0x959845D3, - 0xA4705F4E, - 0x020754FA, - 0xC7A06A74, - 0x61D761C0, - 0x503F7B5D, - 0xF64870E9, - 0x67DE9CCE, - 0xC1A9977A, - 0xF0418DE7, - 0x56368653, - 0x9391B8DD, - 0x35E6B369, - 0x040EA9F4, - 0xA279A240, - 0x5431D2A9, - 0xF246D91D, - 0xC3AEC380, - 0x65D9C834, - 0xA07EF6BA, - 0x0609FD0E, - 0x37E1E793, - 0x9196EC27, - 0xCFBD399C, - 0x69CA3228, - 0x582228B5, - 0xFE552301, - 0x3BF21D8F, - 0x9D85163B, - 0xAC6D0CA6, - 0x0A1A0712, - 0xFC5277FB, - 0x5A257C4F, - 0x6BCD66D2, - 0xCDBA6D66, - 0x081D53E8, - 0xAE6A585C, - 0x9F8242C1, - 0x39F54975, - 0xA863A552, - 0x0E14AEE6, - 0x3FFCB47B, - 0x998BBFCF, - 0x5C2C8141, - 0xFA5B8AF5, - 0xCBB39068, - 0x6DC49BDC, - 0x9B8CEB35, - 0x3DFBE081, - 0x0C13FA1C, - 0xAA64F1A8, - 0x6FC3CF26, - 0xC9B4C492, - 0xF85CDE0F, - 0x5E2BD5BB, - 0x440B7579, - 0xE27C7ECD, - 0xD3946450, - 0x75E36FE4, - 0xB044516A, - 0x16335ADE, - 0x27DB4043, - 0x81AC4BF7, - 0x77E43B1E, - 0xD19330AA, - 0xE07B2A37, - 0x460C2183, - 0x83AB1F0D, - 0x25DC14B9, - 0x14340E24, - 0xB2430590, - 0x23D5E9B7, - 0x85A2E203, - 0xB44AF89E, - 0x123DF32A, - 0xD79ACDA4, - 0x71EDC610, - 0x4005DC8D, - 0xE672D739, - 0x103AA7D0, - 0xB64DAC64, - 0x87A5B6F9, - 0x21D2BD4D, - 0xE47583C3, - 0x42028877, - 0x73EA92EA, - 0xD59D995E, - 0x8BB64CE5, - 0x2DC14751, - 0x1C295DCC, - 0xBA5E5678, - 0x7FF968F6, - 0xD98E6342, - 0xE86679DF, - 0x4E11726B, - 0xB8590282, - 0x1E2E0936, - 0x2FC613AB, - 0x89B1181F, - 0x4C162691, - 0xEA612D25, - 0xDB8937B8, - 0x7DFE3C0C, - 0xEC68D02B, - 0x4A1FDB9F, - 0x7BF7C102, - 0xDD80CAB6, - 0x1827F438, - 0xBE50FF8C, - 0x8FB8E511, - 0x29CFEEA5, - 0xDF879E4C, - 0x79F095F8, - 0x48188F65, - 0xEE6F84D1, - 0x2BC8BA5F, - 0x8DBFB1EB, - 0xBC57AB76, - 0x1A20A0C2, - 0x8816EAF2, - 0x2E61E146, - 0x1F89FBDB, - 0xB9FEF06F, - 0x7C59CEE1, - 0xDA2EC555, - 0xEBC6DFC8, - 0x4DB1D47C, - 0xBBF9A495, - 0x1D8EAF21, - 0x2C66B5BC, - 0x8A11BE08, - 0x4FB68086, - 0xE9C18B32, - 0xD82991AF, - 0x7E5E9A1B, - 0xEFC8763C, - 0x49BF7D88, - 0x78576715, - 0xDE206CA1, - 0x1B87522F, - 0xBDF0599B, - 0x8C184306, - 0x2A6F48B2, - 0xDC27385B, - 0x7A5033EF, - 0x4BB82972, - 0xEDCF22C6, - 0x28681C48, - 0x8E1F17FC, - 0xBFF70D61, - 0x198006D5, - 0x47ABD36E, - 0xE1DCD8DA, - 0xD034C247, - 0x7643C9F3, - 0xB3E4F77D, - 0x1593FCC9, - 0x247BE654, - 0x820CEDE0, - 0x74449D09, - 0xD23396BD, - 0xE3DB8C20, - 0x45AC8794, - 0x800BB91A, - 0x267CB2AE, - 0x1794A833, - 0xB1E3A387, - 0x20754FA0, - 0x86024414, - 0xB7EA5E89, - 0x119D553D, - 0xD43A6BB3, - 0x724D6007, - 0x43A57A9A, - 0xE5D2712E, - 0x139A01C7, - 0xB5ED0A73, - 0x840510EE, - 0x22721B5A, - 0xE7D525D4, - 0x41A22E60, - 0x704A34FD, - 0xD63D3F49, - 0xCC1D9F8B, - 0x6A6A943F, - 0x5B828EA2, - 0xFDF58516, - 0x3852BB98, - 0x9E25B02C, - 0xAFCDAAB1, - 0x09BAA105, - 0xFFF2D1EC, - 0x5985DA58, - 0x686DC0C5, - 0xCE1ACB71, - 0x0BBDF5FF, - 0xADCAFE4B, - 0x9C22E4D6, - 0x3A55EF62, - 0xABC30345, - 0x0DB408F1, - 0x3C5C126C, - 0x9A2B19D8, - 0x5F8C2756, - 0xF9FB2CE2, - 0xC813367F, - 0x6E643DCB, - 0x982C4D22, - 0x3E5B4696, - 0x0FB35C0B, - 0xA9C457BF, - 0x6C636931, - 0xCA146285, - 0xFBFC7818, - 0x5D8B73AC, - 0x03A0A617, - 0xA5D7ADA3, - 0x943FB73E, - 0x3248BC8A, - 0xF7EF8204, - 0x519889B0, - 0x6070932D, - 0xC6079899, - 0x304FE870, - 0x9638E3C4, - 0xA7D0F959, - 0x01A7F2ED, - 0xC400CC63, - 0x6277C7D7, - 0x539FDD4A, - 0xF5E8D6FE, - 0x647E3AD9, - 0xC209316D, - 0xF3E12BF0, - 0x55962044, - 0x90311ECA, - 0x3646157E, - 0x07AE0FE3, - 0xA1D90457, - 0x579174BE, - 0xF1E67F0A, - 0xC00E6597, - 0x66796E23, - 0xA3DE50AD, - 0x05A95B19, - 0x34414184, - 0x92364A30, - /* T8_7 */ - 0x00000000, - 0xCCAA009E, - 0x4225077D, - 0x8E8F07E3, - 0x844A0EFA, - 0x48E00E64, - 0xC66F0987, - 0x0AC50919, - 0xD3E51BB5, - 0x1F4F1B2B, - 0x91C01CC8, - 0x5D6A1C56, - 0x57AF154F, - 0x9B0515D1, - 0x158A1232, - 0xD92012AC, - 0x7CBB312B, - 0xB01131B5, - 0x3E9E3656, - 0xF23436C8, - 0xF8F13FD1, - 0x345B3F4F, - 0xBAD438AC, - 0x767E3832, - 0xAF5E2A9E, - 0x63F42A00, - 0xED7B2DE3, - 0x21D12D7D, - 0x2B142464, - 0xE7BE24FA, - 0x69312319, - 0xA59B2387, - 0xF9766256, - 0x35DC62C8, - 0xBB53652B, - 0x77F965B5, - 0x7D3C6CAC, - 0xB1966C32, - 0x3F196BD1, - 0xF3B36B4F, - 0x2A9379E3, - 0xE639797D, - 0x68B67E9E, - 0xA41C7E00, - 0xAED97719, - 0x62737787, - 0xECFC7064, - 0x205670FA, - 0x85CD537D, - 0x496753E3, - 0xC7E85400, - 0x0B42549E, - 0x01875D87, - 0xCD2D5D19, - 0x43A25AFA, - 0x8F085A64, - 0x562848C8, - 0x9A824856, - 0x140D4FB5, - 0xD8A74F2B, - 0xD2624632, - 0x1EC846AC, - 0x9047414F, - 0x5CED41D1, - 0x299DC2ED, - 0xE537C273, - 0x6BB8C590, - 0xA712C50E, - 0xADD7CC17, - 0x617DCC89, - 0xEFF2CB6A, - 0x2358CBF4, - 0xFA78D958, - 0x36D2D9C6, - 0xB85DDE25, - 0x74F7DEBB, - 0x7E32D7A2, - 0xB298D73C, - 0x3C17D0DF, - 0xF0BDD041, - 0x5526F3C6, - 0x998CF358, - 0x1703F4BB, - 0xDBA9F425, - 0xD16CFD3C, - 0x1DC6FDA2, - 0x9349FA41, - 0x5FE3FADF, - 0x86C3E873, - 0x4A69E8ED, - 0xC4E6EF0E, - 0x084CEF90, - 0x0289E689, - 0xCE23E617, - 0x40ACE1F4, - 0x8C06E16A, - 0xD0EBA0BB, - 0x1C41A025, - 0x92CEA7C6, - 0x5E64A758, - 0x54A1AE41, - 0x980BAEDF, - 0x1684A93C, - 0xDA2EA9A2, - 0x030EBB0E, - 0xCFA4BB90, - 0x412BBC73, - 0x8D81BCED, - 0x8744B5F4, - 0x4BEEB56A, - 0xC561B289, - 0x09CBB217, - 0xAC509190, - 0x60FA910E, - 0xEE7596ED, - 0x22DF9673, - 0x281A9F6A, - 0xE4B09FF4, - 0x6A3F9817, - 0xA6959889, - 0x7FB58A25, - 0xB31F8ABB, - 0x3D908D58, - 0xF13A8DC6, - 0xFBFF84DF, - 0x37558441, - 0xB9DA83A2, - 0x7570833C, - 0x533B85DA, - 0x9F918544, - 0x111E82A7, - 0xDDB48239, - 0xD7718B20, - 0x1BDB8BBE, - 0x95548C5D, - 0x59FE8CC3, - 0x80DE9E6F, - 0x4C749EF1, - 0xC2FB9912, - 0x0E51998C, - 0x04949095, - 0xC83E900B, - 0x46B197E8, - 0x8A1B9776, - 0x2F80B4F1, - 0xE32AB46F, - 0x6DA5B38C, - 0xA10FB312, - 0xABCABA0B, - 0x6760BA95, - 0xE9EFBD76, - 0x2545BDE8, - 0xFC65AF44, - 0x30CFAFDA, - 0xBE40A839, - 0x72EAA8A7, - 0x782FA1BE, - 0xB485A120, - 0x3A0AA6C3, - 0xF6A0A65D, - 0xAA4DE78C, - 0x66E7E712, - 0xE868E0F1, - 0x24C2E06F, - 0x2E07E976, - 0xE2ADE9E8, - 0x6C22EE0B, - 0xA088EE95, - 0x79A8FC39, - 0xB502FCA7, - 0x3B8DFB44, - 0xF727FBDA, - 0xFDE2F2C3, - 0x3148F25D, - 0xBFC7F5BE, - 0x736DF520, - 0xD6F6D6A7, - 0x1A5CD639, - 0x94D3D1DA, - 0x5879D144, - 0x52BCD85D, - 0x9E16D8C3, - 0x1099DF20, - 0xDC33DFBE, - 0x0513CD12, - 0xC9B9CD8C, - 0x4736CA6F, - 0x8B9CCAF1, - 0x8159C3E8, - 0x4DF3C376, - 0xC37CC495, - 0x0FD6C40B, - 0x7AA64737, - 0xB60C47A9, - 0x3883404A, - 0xF42940D4, - 0xFEEC49CD, - 0x32464953, - 0xBCC94EB0, - 0x70634E2E, - 0xA9435C82, - 0x65E95C1C, - 0xEB665BFF, - 0x27CC5B61, - 0x2D095278, - 0xE1A352E6, - 0x6F2C5505, - 0xA386559B, - 0x061D761C, - 0xCAB77682, - 0x44387161, - 0x889271FF, - 0x825778E6, - 0x4EFD7878, - 0xC0727F9B, - 0x0CD87F05, - 0xD5F86DA9, - 0x19526D37, - 0x97DD6AD4, - 0x5B776A4A, - 0x51B26353, - 0x9D1863CD, - 0x1397642E, - 0xDF3D64B0, - 0x83D02561, - 0x4F7A25FF, - 0xC1F5221C, - 0x0D5F2282, - 0x079A2B9B, - 0xCB302B05, - 0x45BF2CE6, - 0x89152C78, - 0x50353ED4, - 0x9C9F3E4A, - 0x121039A9, - 0xDEBA3937, - 0xD47F302E, - 0x18D530B0, - 0x965A3753, - 0x5AF037CD, - 0xFF6B144A, - 0x33C114D4, - 0xBD4E1337, - 0x71E413A9, - 0x7B211AB0, - 0xB78B1A2E, - 0x39041DCD, - 0xF5AE1D53, - 0x2C8E0FFF, - 0xE0240F61, - 0x6EAB0882, - 0xA201081C, - 0xA8C40105, - 0x646E019B, - 0xEAE10678, - 0x264B06E6 }; + /* T8_0 */ + 0x00000000, 0x77073096, 0xEE0E612C, 0x990951BA, 0x076DC419, 0x706AF48F, 0xE963A535, 0x9E6495A3, 0x0EDB8832, + 0x79DCB8A4, 0xE0D5E91E, 0x97D2D988, 0x09B64C2B, 0x7EB17CBD, 0xE7B82D07, 0x90BF1D91, 0x1DB71064, 0x6AB020F2, + 0xF3B97148, 0x84BE41DE, 0x1ADAD47D, 0x6DDDE4EB, 0xF4D4B551, 0x83D385C7, 0x136C9856, 0x646BA8C0, 0xFD62F97A, + 0x8A65C9EC, 0x14015C4F, 0x63066CD9, 0xFA0F3D63, 0x8D080DF5, 0x3B6E20C8, 0x4C69105E, 0xD56041E4, 0xA2677172, + 0x3C03E4D1, 0x4B04D447, 0xD20D85FD, 0xA50AB56B, 0x35B5A8FA, 0x42B2986C, 0xDBBBC9D6, 0xACBCF940, 0x32D86CE3, + 0x45DF5C75, 0xDCD60DCF, 0xABD13D59, 0x26D930AC, 0x51DE003A, 0xC8D75180, 0xBFD06116, 0x21B4F4B5, 0x56B3C423, + 0xCFBA9599, 0xB8BDA50F, 0x2802B89E, 0x5F058808, 0xC60CD9B2, 0xB10BE924, 0x2F6F7C87, 0x58684C11, 0xC1611DAB, + 0xB6662D3D, 0x76DC4190, 0x01DB7106, 0x98D220BC, 0xEFD5102A, 0x71B18589, 0x06B6B51F, 0x9FBFE4A5, 0xE8B8D433, + 0x7807C9A2, 0x0F00F934, 0x9609A88E, 0xE10E9818, 0x7F6A0DBB, 0x086D3D2D, 0x91646C97, 0xE6635C01, 0x6B6B51F4, + 0x1C6C6162, 0x856530D8, 0xF262004E, 0x6C0695ED, 0x1B01A57B, 0x8208F4C1, 0xF50FC457, 0x65B0D9C6, 0x12B7E950, + 0x8BBEB8EA, 0xFCB9887C, 0x62DD1DDF, 0x15DA2D49, 0x8CD37CF3, 0xFBD44C65, 0x4DB26158, 0x3AB551CE, 0xA3BC0074, + 0xD4BB30E2, 0x4ADFA541, 0x3DD895D7, 0xA4D1C46D, 0xD3D6F4FB, 0x4369E96A, 0x346ED9FC, 0xAD678846, 0xDA60B8D0, + 0x44042D73, 0x33031DE5, 0xAA0A4C5F, 0xDD0D7CC9, 0x5005713C, 0x270241AA, 0xBE0B1010, 0xC90C2086, 0x5768B525, + 0x206F85B3, 0xB966D409, 0xCE61E49F, 0x5EDEF90E, 0x29D9C998, 0xB0D09822, 0xC7D7A8B4, 0x59B33D17, 0x2EB40D81, + 0xB7BD5C3B, 0xC0BA6CAD, 0xEDB88320, 0x9ABFB3B6, 0x03B6E20C, 0x74B1D29A, 0xEAD54739, 0x9DD277AF, 0x04DB2615, + 0x73DC1683, 0xE3630B12, 0x94643B84, 0x0D6D6A3E, 0x7A6A5AA8, 0xE40ECF0B, 0x9309FF9D, 0x0A00AE27, 0x7D079EB1, + 0xF00F9344, 0x8708A3D2, 0x1E01F268, 0x6906C2FE, 0xF762575D, 0x806567CB, 0x196C3671, 0x6E6B06E7, 0xFED41B76, + 0x89D32BE0, 0x10DA7A5A, 0x67DD4ACC, 0xF9B9DF6F, 0x8EBEEFF9, 0x17B7BE43, 0x60B08ED5, 0xD6D6A3E8, 0xA1D1937E, + 0x38D8C2C4, 0x4FDFF252, 0xD1BB67F1, 0xA6BC5767, 0x3FB506DD, 0x48B2364B, 0xD80D2BDA, 0xAF0A1B4C, 0x36034AF6, + 0x41047A60, 0xDF60EFC3, 0xA867DF55, 0x316E8EEF, 0x4669BE79, 0xCB61B38C, 0xBC66831A, 0x256FD2A0, 0x5268E236, + 0xCC0C7795, 0xBB0B4703, 0x220216B9, 0x5505262F, 0xC5BA3BBE, 0xB2BD0B28, 0x2BB45A92, 0x5CB36A04, 0xC2D7FFA7, + 0xB5D0CF31, 0x2CD99E8B, 0x5BDEAE1D, 0x9B64C2B0, 0xEC63F226, 0x756AA39C, 0x026D930A, 0x9C0906A9, 0xEB0E363F, + 0x72076785, 0x05005713, 0x95BF4A82, 0xE2B87A14, 0x7BB12BAE, 0x0CB61B38, 0x92D28E9B, 0xE5D5BE0D, 0x7CDCEFB7, + 0x0BDBDF21, 0x86D3D2D4, 0xF1D4E242, 0x68DDB3F8, 0x1FDA836E, 0x81BE16CD, 0xF6B9265B, 0x6FB077E1, 0x18B74777, + 0x88085AE6, 0xFF0F6A70, 0x66063BCA, 0x11010B5C, 0x8F659EFF, 0xF862AE69, 0x616BFFD3, 0x166CCF45, 0xA00AE278, + 0xD70DD2EE, 0x4E048354, 0x3903B3C2, 0xA7672661, 0xD06016F7, 0x4969474D, 0x3E6E77DB, 0xAED16A4A, 0xD9D65ADC, + 0x40DF0B66, 0x37D83BF0, 0xA9BCAE53, 0xDEBB9EC5, 0x47B2CF7F, 0x30B5FFE9, 0xBDBDF21C, 0xCABAC28A, 0x53B39330, + 0x24B4A3A6, 0xBAD03605, 0xCDD70693, 0x54DE5729, 0x23D967BF, 0xB3667A2E, 0xC4614AB8, 0x5D681B02, 0x2A6F2B94, + 0xB40BBE37, 0xC30C8EA1, 0x5A05DF1B, 0x2D02EF8D, + /* T8_1 */ + 0x00000000, 0x191B3141, 0x32366282, 0x2B2D53C3, 0x646CC504, 0x7D77F445, 0x565AA786, 0x4F4196C7, 0xC8D98A08, + 0xD1C2BB49, 0xFAEFE88A, 0xE3F4D9CB, 0xACB54F0C, 0xB5AE7E4D, 0x9E832D8E, 0x87981CCF, 0x4AC21251, 0x53D92310, + 0x78F470D3, 0x61EF4192, 0x2EAED755, 0x37B5E614, 0x1C98B5D7, 0x05838496, 0x821B9859, 0x9B00A918, 0xB02DFADB, + 0xA936CB9A, 0xE6775D5D, 0xFF6C6C1C, 0xD4413FDF, 0xCD5A0E9E, 0x958424A2, 0x8C9F15E3, 0xA7B24620, 0xBEA97761, + 0xF1E8E1A6, 0xE8F3D0E7, 0xC3DE8324, 0xDAC5B265, 0x5D5DAEAA, 0x44469FEB, 0x6F6BCC28, 0x7670FD69, 0x39316BAE, + 0x202A5AEF, 0x0B07092C, 0x121C386D, 0xDF4636F3, 0xC65D07B2, 0xED705471, 0xF46B6530, 0xBB2AF3F7, 0xA231C2B6, + 0x891C9175, 0x9007A034, 0x179FBCFB, 0x0E848DBA, 0x25A9DE79, 0x3CB2EF38, 0x73F379FF, 0x6AE848BE, 0x41C51B7D, + 0x58DE2A3C, 0xF0794F05, 0xE9627E44, 0xC24F2D87, 0xDB541CC6, 0x94158A01, 0x8D0EBB40, 0xA623E883, 0xBF38D9C2, + 0x38A0C50D, 0x21BBF44C, 0x0A96A78F, 0x138D96CE, 0x5CCC0009, 0x45D73148, 0x6EFA628B, 0x77E153CA, 0xBABB5D54, + 0xA3A06C15, 0x888D3FD6, 0x91960E97, 0xDED79850, 0xC7CCA911, 0xECE1FAD2, 0xF5FACB93, 0x7262D75C, 0x6B79E61D, + 0x4054B5DE, 0x594F849F, 0x160E1258, 0x0F152319, 0x243870DA, 0x3D23419B, 0x65FD6BA7, 0x7CE65AE6, 0x57CB0925, + 0x4ED03864, 0x0191AEA3, 0x188A9FE2, 0x33A7CC21, 0x2ABCFD60, 0xAD24E1AF, 0xB43FD0EE, 0x9F12832D, 0x8609B26C, + 0xC94824AB, 0xD05315EA, 0xFB7E4629, 0xE2657768, 0x2F3F79F6, 0x362448B7, 0x1D091B74, 0x04122A35, 0x4B53BCF2, + 0x52488DB3, 0x7965DE70, 0x607EEF31, 0xE7E6F3FE, 0xFEFDC2BF, 0xD5D0917C, 0xCCCBA03D, 0x838A36FA, 0x9A9107BB, + 0xB1BC5478, 0xA8A76539, 0x3B83984B, 0x2298A90A, 0x09B5FAC9, 0x10AECB88, 0x5FEF5D4F, 0x46F46C0E, 0x6DD93FCD, + 0x74C20E8C, 0xF35A1243, 0xEA412302, 0xC16C70C1, 0xD8774180, 0x9736D747, 0x8E2DE606, 0xA500B5C5, 0xBC1B8484, + 0x71418A1A, 0x685ABB5B, 0x4377E898, 0x5A6CD9D9, 0x152D4F1E, 0x0C367E5F, 0x271B2D9C, 0x3E001CDD, 0xB9980012, + 0xA0833153, 0x8BAE6290, 0x92B553D1, 0xDDF4C516, 0xC4EFF457, 0xEFC2A794, 0xF6D996D5, 0xAE07BCE9, 0xB71C8DA8, + 0x9C31DE6B, 0x852AEF2A, 0xCA6B79ED, 0xD37048AC, 0xF85D1B6F, 0xE1462A2E, 0x66DE36E1, 0x7FC507A0, 0x54E85463, + 0x4DF36522, 0x02B2F3E5, 0x1BA9C2A4, 0x30849167, 0x299FA026, 0xE4C5AEB8, 0xFDDE9FF9, 0xD6F3CC3A, 0xCFE8FD7B, + 0x80A96BBC, 0x99B25AFD, 0xB29F093E, 0xAB84387F, 0x2C1C24B0, 0x350715F1, 0x1E2A4632, 0x07317773, 0x4870E1B4, + 0x516BD0F5, 0x7A468336, 0x635DB277, 0xCBFAD74E, 0xD2E1E60F, 0xF9CCB5CC, 0xE0D7848D, 0xAF96124A, 0xB68D230B, + 0x9DA070C8, 0x84BB4189, 0x03235D46, 0x1A386C07, 0x31153FC4, 0x280E0E85, 0x674F9842, 0x7E54A903, 0x5579FAC0, + 0x4C62CB81, 0x8138C51F, 0x9823F45E, 0xB30EA79D, 0xAA1596DC, 0xE554001B, 0xFC4F315A, 0xD7626299, 0xCE7953D8, + 0x49E14F17, 0x50FA7E56, 0x7BD72D95, 0x62CC1CD4, 0x2D8D8A13, 0x3496BB52, 0x1FBBE891, 0x06A0D9D0, 0x5E7EF3EC, + 0x4765C2AD, 0x6C48916E, 0x7553A02F, 0x3A1236E8, 0x230907A9, 0x0824546A, 0x113F652B, 0x96A779E4, 0x8FBC48A5, + 0xA4911B66, 0xBD8A2A27, 0xF2CBBCE0, 0xEBD08DA1, 0xC0FDDE62, 0xD9E6EF23, 0x14BCE1BD, 0x0DA7D0FC, 0x268A833F, + 0x3F91B27E, 0x70D024B9, 0x69CB15F8, 0x42E6463B, 0x5BFD777A, 0xDC656BB5, 0xC57E5AF4, 0xEE530937, 0xF7483876, + 0xB809AEB1, 0xA1129FF0, 0x8A3FCC33, 0x9324FD72, + /* T8_2 */ + 0x00000000, 0x01C26A37, 0x0384D46E, 0x0246BE59, 0x0709A8DC, 0x06CBC2EB, 0x048D7CB2, 0x054F1685, 0x0E1351B8, + 0x0FD13B8F, 0x0D9785D6, 0x0C55EFE1, 0x091AF964, 0x08D89353, 0x0A9E2D0A, 0x0B5C473D, 0x1C26A370, 0x1DE4C947, + 0x1FA2771E, 0x1E601D29, 0x1B2F0BAC, 0x1AED619B, 0x18ABDFC2, 0x1969B5F5, 0x1235F2C8, 0x13F798FF, 0x11B126A6, + 0x10734C91, 0x153C5A14, 0x14FE3023, 0x16B88E7A, 0x177AE44D, 0x384D46E0, 0x398F2CD7, 0x3BC9928E, 0x3A0BF8B9, + 0x3F44EE3C, 0x3E86840B, 0x3CC03A52, 0x3D025065, 0x365E1758, 0x379C7D6F, 0x35DAC336, 0x3418A901, 0x3157BF84, + 0x3095D5B3, 0x32D36BEA, 0x331101DD, 0x246BE590, 0x25A98FA7, 0x27EF31FE, 0x262D5BC9, 0x23624D4C, 0x22A0277B, + 0x20E69922, 0x2124F315, 0x2A78B428, 0x2BBADE1F, 0x29FC6046, 0x283E0A71, 0x2D711CF4, 0x2CB376C3, 0x2EF5C89A, + 0x2F37A2AD, 0x709A8DC0, 0x7158E7F7, 0x731E59AE, 0x72DC3399, 0x7793251C, 0x76514F2B, 0x7417F172, 0x75D59B45, + 0x7E89DC78, 0x7F4BB64F, 0x7D0D0816, 0x7CCF6221, 0x798074A4, 0x78421E93, 0x7A04A0CA, 0x7BC6CAFD, 0x6CBC2EB0, + 0x6D7E4487, 0x6F38FADE, 0x6EFA90E9, 0x6BB5866C, 0x6A77EC5B, 0x68315202, 0x69F33835, 0x62AF7F08, 0x636D153F, + 0x612BAB66, 0x60E9C151, 0x65A6D7D4, 0x6464BDE3, 0x662203BA, 0x67E0698D, 0x48D7CB20, 0x4915A117, 0x4B531F4E, + 0x4A917579, 0x4FDE63FC, 0x4E1C09CB, 0x4C5AB792, 0x4D98DDA5, 0x46C49A98, 0x4706F0AF, 0x45404EF6, 0x448224C1, + 0x41CD3244, 0x400F5873, 0x4249E62A, 0x438B8C1D, 0x54F16850, 0x55330267, 0x5775BC3E, 0x56B7D609, 0x53F8C08C, + 0x523AAABB, 0x507C14E2, 0x51BE7ED5, 0x5AE239E8, 0x5B2053DF, 0x5966ED86, 0x58A487B1, 0x5DEB9134, 0x5C29FB03, + 0x5E6F455A, 0x5FAD2F6D, 0xE1351B80, 0xE0F771B7, 0xE2B1CFEE, 0xE373A5D9, 0xE63CB35C, 0xE7FED96B, 0xE5B86732, + 0xE47A0D05, 0xEF264A38, 0xEEE4200F, 0xECA29E56, 0xED60F461, 0xE82FE2E4, 0xE9ED88D3, 0xEBAB368A, 0xEA695CBD, + 0xFD13B8F0, 0xFCD1D2C7, 0xFE976C9E, 0xFF5506A9, 0xFA1A102C, 0xFBD87A1B, 0xF99EC442, 0xF85CAE75, 0xF300E948, + 0xF2C2837F, 0xF0843D26, 0xF1465711, 0xF4094194, 0xF5CB2BA3, 0xF78D95FA, 0xF64FFFCD, 0xD9785D60, 0xD8BA3757, + 0xDAFC890E, 0xDB3EE339, 0xDE71F5BC, 0xDFB39F8B, 0xDDF521D2, 0xDC374BE5, 0xD76B0CD8, 0xD6A966EF, 0xD4EFD8B6, + 0xD52DB281, 0xD062A404, 0xD1A0CE33, 0xD3E6706A, 0xD2241A5D, 0xC55EFE10, 0xC49C9427, 0xC6DA2A7E, 0xC7184049, + 0xC25756CC, 0xC3953CFB, 0xC1D382A2, 0xC011E895, 0xCB4DAFA8, 0xCA8FC59F, 0xC8C97BC6, 0xC90B11F1, 0xCC440774, + 0xCD866D43, 0xCFC0D31A, 0xCE02B92D, 0x91AF9640, 0x906DFC77, 0x922B422E, 0x93E92819, 0x96A63E9C, 0x976454AB, + 0x9522EAF2, 0x94E080C5, 0x9FBCC7F8, 0x9E7EADCF, 0x9C381396, 0x9DFA79A1, 0x98B56F24, 0x99770513, 0x9B31BB4A, + 0x9AF3D17D, 0x8D893530, 0x8C4B5F07, 0x8E0DE15E, 0x8FCF8B69, 0x8A809DEC, 0x8B42F7DB, 0x89044982, 0x88C623B5, + 0x839A6488, 0x82580EBF, 0x801EB0E6, 0x81DCDAD1, 0x8493CC54, 0x8551A663, 0x8717183A, 0x86D5720D, 0xA9E2D0A0, + 0xA820BA97, 0xAA6604CE, 0xABA46EF9, 0xAEEB787C, 0xAF29124B, 0xAD6FAC12, 0xACADC625, 0xA7F18118, 0xA633EB2F, + 0xA4755576, 0xA5B73F41, 0xA0F829C4, 0xA13A43F3, 0xA37CFDAA, 0xA2BE979D, 0xB5C473D0, 0xB40619E7, 0xB640A7BE, + 0xB782CD89, 0xB2CDDB0C, 0xB30FB13B, 0xB1490F62, 0xB08B6555, 0xBBD72268, 0xBA15485F, 0xB853F606, 0xB9919C31, + 0xBCDE8AB4, 0xBD1CE083, 0xBF5A5EDA, 0xBE9834ED, + /* T8_3 */ + 0x00000000, 0xB8BC6765, 0xAA09C88B, 0x12B5AFEE, 0x8F629757, 0x37DEF032, 0x256B5FDC, 0x9DD738B9, 0xC5B428EF, + 0x7D084F8A, 0x6FBDE064, 0xD7018701, 0x4AD6BFB8, 0xF26AD8DD, 0xE0DF7733, 0x58631056, 0x5019579F, 0xE8A530FA, + 0xFA109F14, 0x42ACF871, 0xDF7BC0C8, 0x67C7A7AD, 0x75720843, 0xCDCE6F26, 0x95AD7F70, 0x2D111815, 0x3FA4B7FB, + 0x8718D09E, 0x1ACFE827, 0xA2738F42, 0xB0C620AC, 0x087A47C9, 0xA032AF3E, 0x188EC85B, 0x0A3B67B5, 0xB28700D0, + 0x2F503869, 0x97EC5F0C, 0x8559F0E2, 0x3DE59787, 0x658687D1, 0xDD3AE0B4, 0xCF8F4F5A, 0x7733283F, 0xEAE41086, + 0x525877E3, 0x40EDD80D, 0xF851BF68, 0xF02BF8A1, 0x48979FC4, 0x5A22302A, 0xE29E574F, 0x7F496FF6, 0xC7F50893, + 0xD540A77D, 0x6DFCC018, 0x359FD04E, 0x8D23B72B, 0x9F9618C5, 0x272A7FA0, 0xBAFD4719, 0x0241207C, 0x10F48F92, + 0xA848E8F7, 0x9B14583D, 0x23A83F58, 0x311D90B6, 0x89A1F7D3, 0x1476CF6A, 0xACCAA80F, 0xBE7F07E1, 0x06C36084, + 0x5EA070D2, 0xE61C17B7, 0xF4A9B859, 0x4C15DF3C, 0xD1C2E785, 0x697E80E0, 0x7BCB2F0E, 0xC377486B, 0xCB0D0FA2, + 0x73B168C7, 0x6104C729, 0xD9B8A04C, 0x446F98F5, 0xFCD3FF90, 0xEE66507E, 0x56DA371B, 0x0EB9274D, 0xB6054028, + 0xA4B0EFC6, 0x1C0C88A3, 0x81DBB01A, 0x3967D77F, 0x2BD27891, 0x936E1FF4, 0x3B26F703, 0x839A9066, 0x912F3F88, + 0x299358ED, 0xB4446054, 0x0CF80731, 0x1E4DA8DF, 0xA6F1CFBA, 0xFE92DFEC, 0x462EB889, 0x549B1767, 0xEC277002, + 0x71F048BB, 0xC94C2FDE, 0xDBF98030, 0x6345E755, 0x6B3FA09C, 0xD383C7F9, 0xC1366817, 0x798A0F72, 0xE45D37CB, + 0x5CE150AE, 0x4E54FF40, 0xF6E89825, 0xAE8B8873, 0x1637EF16, 0x048240F8, 0xBC3E279D, 0x21E91F24, 0x99557841, + 0x8BE0D7AF, 0x335CB0CA, 0xED59B63B, 0x55E5D15E, 0x47507EB0, 0xFFEC19D5, 0x623B216C, 0xDA874609, 0xC832E9E7, + 0x708E8E82, 0x28ED9ED4, 0x9051F9B1, 0x82E4565F, 0x3A58313A, 0xA78F0983, 0x1F336EE6, 0x0D86C108, 0xB53AA66D, + 0xBD40E1A4, 0x05FC86C1, 0x1749292F, 0xAFF54E4A, 0x322276F3, 0x8A9E1196, 0x982BBE78, 0x2097D91D, 0x78F4C94B, + 0xC048AE2E, 0xD2FD01C0, 0x6A4166A5, 0xF7965E1C, 0x4F2A3979, 0x5D9F9697, 0xE523F1F2, 0x4D6B1905, 0xF5D77E60, + 0xE762D18E, 0x5FDEB6EB, 0xC2098E52, 0x7AB5E937, 0x680046D9, 0xD0BC21BC, 0x88DF31EA, 0x3063568F, 0x22D6F961, + 0x9A6A9E04, 0x07BDA6BD, 0xBF01C1D8, 0xADB46E36, 0x15080953, 0x1D724E9A, 0xA5CE29FF, 0xB77B8611, 0x0FC7E174, + 0x9210D9CD, 0x2AACBEA8, 0x38191146, 0x80A57623, 0xD8C66675, 0x607A0110, 0x72CFAEFE, 0xCA73C99B, 0x57A4F122, + 0xEF189647, 0xFDAD39A9, 0x45115ECC, 0x764DEE06, 0xCEF18963, 0xDC44268D, 0x64F841E8, 0xF92F7951, 0x41931E34, + 0x5326B1DA, 0xEB9AD6BF, 0xB3F9C6E9, 0x0B45A18C, 0x19F00E62, 0xA14C6907, 0x3C9B51BE, 0x842736DB, 0x96929935, + 0x2E2EFE50, 0x2654B999, 0x9EE8DEFC, 0x8C5D7112, 0x34E11677, 0xA9362ECE, 0x118A49AB, 0x033FE645, 0xBB838120, + 0xE3E09176, 0x5B5CF613, 0x49E959FD, 0xF1553E98, 0x6C820621, 0xD43E6144, 0xC68BCEAA, 0x7E37A9CF, 0xD67F4138, + 0x6EC3265D, 0x7C7689B3, 0xC4CAEED6, 0x591DD66F, 0xE1A1B10A, 0xF3141EE4, 0x4BA87981, 0x13CB69D7, 0xAB770EB2, + 0xB9C2A15C, 0x017EC639, 0x9CA9FE80, 0x241599E5, 0x36A0360B, 0x8E1C516E, 0x866616A7, 0x3EDA71C2, 0x2C6FDE2C, + 0x94D3B949, 0x090481F0, 0xB1B8E695, 0xA30D497B, 0x1BB12E1E, 0x43D23E48, 0xFB6E592D, 0xE9DBF6C3, 0x516791A6, + 0xCCB0A91F, 0x740CCE7A, 0x66B96194, 0xDE0506F1, + /* T8_4 */ + 0x00000000, 0x3D6029B0, 0x7AC05360, 0x47A07AD0, 0xF580A6C0, 0xC8E08F70, 0x8F40F5A0, 0xB220DC10, 0x30704BC1, + 0x0D106271, 0x4AB018A1, 0x77D03111, 0xC5F0ED01, 0xF890C4B1, 0xBF30BE61, 0x825097D1, 0x60E09782, 0x5D80BE32, + 0x1A20C4E2, 0x2740ED52, 0x95603142, 0xA80018F2, 0xEFA06222, 0xD2C04B92, 0x5090DC43, 0x6DF0F5F3, 0x2A508F23, + 0x1730A693, 0xA5107A83, 0x98705333, 0xDFD029E3, 0xE2B00053, 0xC1C12F04, 0xFCA106B4, 0xBB017C64, 0x866155D4, + 0x344189C4, 0x0921A074, 0x4E81DAA4, 0x73E1F314, 0xF1B164C5, 0xCCD14D75, 0x8B7137A5, 0xB6111E15, 0x0431C205, + 0x3951EBB5, 0x7EF19165, 0x4391B8D5, 0xA121B886, 0x9C419136, 0xDBE1EBE6, 0xE681C256, 0x54A11E46, 0x69C137F6, + 0x2E614D26, 0x13016496, 0x9151F347, 0xAC31DAF7, 0xEB91A027, 0xD6F18997, 0x64D15587, 0x59B17C37, 0x1E1106E7, + 0x23712F57, 0x58F35849, 0x659371F9, 0x22330B29, 0x1F532299, 0xAD73FE89, 0x9013D739, 0xD7B3ADE9, 0xEAD38459, + 0x68831388, 0x55E33A38, 0x124340E8, 0x2F236958, 0x9D03B548, 0xA0639CF8, 0xE7C3E628, 0xDAA3CF98, 0x3813CFCB, + 0x0573E67B, 0x42D39CAB, 0x7FB3B51B, 0xCD93690B, 0xF0F340BB, 0xB7533A6B, 0x8A3313DB, 0x0863840A, 0x3503ADBA, + 0x72A3D76A, 0x4FC3FEDA, 0xFDE322CA, 0xC0830B7A, 0x872371AA, 0xBA43581A, 0x9932774D, 0xA4525EFD, 0xE3F2242D, + 0xDE920D9D, 0x6CB2D18D, 0x51D2F83D, 0x167282ED, 0x2B12AB5D, 0xA9423C8C, 0x9422153C, 0xD3826FEC, 0xEEE2465C, + 0x5CC29A4C, 0x61A2B3FC, 0x2602C92C, 0x1B62E09C, 0xF9D2E0CF, 0xC4B2C97F, 0x8312B3AF, 0xBE729A1F, 0x0C52460F, + 0x31326FBF, 0x7692156F, 0x4BF23CDF, 0xC9A2AB0E, 0xF4C282BE, 0xB362F86E, 0x8E02D1DE, 0x3C220DCE, 0x0142247E, + 0x46E25EAE, 0x7B82771E, 0xB1E6B092, 0x8C869922, 0xCB26E3F2, 0xF646CA42, 0x44661652, 0x79063FE2, 0x3EA64532, + 0x03C66C82, 0x8196FB53, 0xBCF6D2E3, 0xFB56A833, 0xC6368183, 0x74165D93, 0x49767423, 0x0ED60EF3, 0x33B62743, + 0xD1062710, 0xEC660EA0, 0xABC67470, 0x96A65DC0, 0x248681D0, 0x19E6A860, 0x5E46D2B0, 0x6326FB00, 0xE1766CD1, + 0xDC164561, 0x9BB63FB1, 0xA6D61601, 0x14F6CA11, 0x2996E3A1, 0x6E369971, 0x5356B0C1, 0x70279F96, 0x4D47B626, + 0x0AE7CCF6, 0x3787E546, 0x85A73956, 0xB8C710E6, 0xFF676A36, 0xC2074386, 0x4057D457, 0x7D37FDE7, 0x3A978737, + 0x07F7AE87, 0xB5D77297, 0x88B75B27, 0xCF1721F7, 0xF2770847, 0x10C70814, 0x2DA721A4, 0x6A075B74, 0x576772C4, + 0xE547AED4, 0xD8278764, 0x9F87FDB4, 0xA2E7D404, 0x20B743D5, 0x1DD76A65, 0x5A7710B5, 0x67173905, 0xD537E515, + 0xE857CCA5, 0xAFF7B675, 0x92979FC5, 0xE915E8DB, 0xD475C16B, 0x93D5BBBB, 0xAEB5920B, 0x1C954E1B, 0x21F567AB, + 0x66551D7B, 0x5B3534CB, 0xD965A31A, 0xE4058AAA, 0xA3A5F07A, 0x9EC5D9CA, 0x2CE505DA, 0x11852C6A, 0x562556BA, + 0x6B457F0A, 0x89F57F59, 0xB49556E9, 0xF3352C39, 0xCE550589, 0x7C75D999, 0x4115F029, 0x06B58AF9, 0x3BD5A349, + 0xB9853498, 0x84E51D28, 0xC34567F8, 0xFE254E48, 0x4C059258, 0x7165BBE8, 0x36C5C138, 0x0BA5E888, 0x28D4C7DF, + 0x15B4EE6F, 0x521494BF, 0x6F74BD0F, 0xDD54611F, 0xE03448AF, 0xA794327F, 0x9AF41BCF, 0x18A48C1E, 0x25C4A5AE, + 0x6264DF7E, 0x5F04F6CE, 0xED242ADE, 0xD044036E, 0x97E479BE, 0xAA84500E, 0x4834505D, 0x755479ED, 0x32F4033D, + 0x0F942A8D, 0xBDB4F69D, 0x80D4DF2D, 0xC774A5FD, 0xFA148C4D, 0x78441B9C, 0x4524322C, 0x028448FC, 0x3FE4614C, + 0x8DC4BD5C, 0xB0A494EC, 0xF704EE3C, 0xCA64C78C, + /* T8_5 */ + 0x00000000, 0xCB5CD3A5, 0x4DC8A10B, 0x869472AE, 0x9B914216, 0x50CD91B3, 0xD659E31D, 0x1D0530B8, 0xEC53826D, + 0x270F51C8, 0xA19B2366, 0x6AC7F0C3, 0x77C2C07B, 0xBC9E13DE, 0x3A0A6170, 0xF156B2D5, 0x03D6029B, 0xC88AD13E, + 0x4E1EA390, 0x85427035, 0x9847408D, 0x531B9328, 0xD58FE186, 0x1ED33223, 0xEF8580F6, 0x24D95353, 0xA24D21FD, + 0x6911F258, 0x7414C2E0, 0xBF481145, 0x39DC63EB, 0xF280B04E, 0x07AC0536, 0xCCF0D693, 0x4A64A43D, 0x81387798, + 0x9C3D4720, 0x57619485, 0xD1F5E62B, 0x1AA9358E, 0xEBFF875B, 0x20A354FE, 0xA6372650, 0x6D6BF5F5, 0x706EC54D, + 0xBB3216E8, 0x3DA66446, 0xF6FAB7E3, 0x047A07AD, 0xCF26D408, 0x49B2A6A6, 0x82EE7503, 0x9FEB45BB, 0x54B7961E, + 0xD223E4B0, 0x197F3715, 0xE82985C0, 0x23755665, 0xA5E124CB, 0x6EBDF76E, 0x73B8C7D6, 0xB8E41473, 0x3E7066DD, + 0xF52CB578, 0x0F580A6C, 0xC404D9C9, 0x4290AB67, 0x89CC78C2, 0x94C9487A, 0x5F959BDF, 0xD901E971, 0x125D3AD4, + 0xE30B8801, 0x28575BA4, 0xAEC3290A, 0x659FFAAF, 0x789ACA17, 0xB3C619B2, 0x35526B1C, 0xFE0EB8B9, 0x0C8E08F7, + 0xC7D2DB52, 0x4146A9FC, 0x8A1A7A59, 0x971F4AE1, 0x5C439944, 0xDAD7EBEA, 0x118B384F, 0xE0DD8A9A, 0x2B81593F, + 0xAD152B91, 0x6649F834, 0x7B4CC88C, 0xB0101B29, 0x36846987, 0xFDD8BA22, 0x08F40F5A, 0xC3A8DCFF, 0x453CAE51, + 0x8E607DF4, 0x93654D4C, 0x58399EE9, 0xDEADEC47, 0x15F13FE2, 0xE4A78D37, 0x2FFB5E92, 0xA96F2C3C, 0x6233FF99, + 0x7F36CF21, 0xB46A1C84, 0x32FE6E2A, 0xF9A2BD8F, 0x0B220DC1, 0xC07EDE64, 0x46EAACCA, 0x8DB67F6F, 0x90B34FD7, + 0x5BEF9C72, 0xDD7BEEDC, 0x16273D79, 0xE7718FAC, 0x2C2D5C09, 0xAAB92EA7, 0x61E5FD02, 0x7CE0CDBA, 0xB7BC1E1F, + 0x31286CB1, 0xFA74BF14, 0x1EB014D8, 0xD5ECC77D, 0x5378B5D3, 0x98246676, 0x852156CE, 0x4E7D856B, 0xC8E9F7C5, + 0x03B52460, 0xF2E396B5, 0x39BF4510, 0xBF2B37BE, 0x7477E41B, 0x6972D4A3, 0xA22E0706, 0x24BA75A8, 0xEFE6A60D, + 0x1D661643, 0xD63AC5E6, 0x50AEB748, 0x9BF264ED, 0x86F75455, 0x4DAB87F0, 0xCB3FF55E, 0x006326FB, 0xF135942E, + 0x3A69478B, 0xBCFD3525, 0x77A1E680, 0x6AA4D638, 0xA1F8059D, 0x276C7733, 0xEC30A496, 0x191C11EE, 0xD240C24B, + 0x54D4B0E5, 0x9F886340, 0x828D53F8, 0x49D1805D, 0xCF45F2F3, 0x04192156, 0xF54F9383, 0x3E134026, 0xB8873288, + 0x73DBE12D, 0x6EDED195, 0xA5820230, 0x2316709E, 0xE84AA33B, 0x1ACA1375, 0xD196C0D0, 0x5702B27E, 0x9C5E61DB, + 0x815B5163, 0x4A0782C6, 0xCC93F068, 0x07CF23CD, 0xF6999118, 0x3DC542BD, 0xBB513013, 0x700DE3B6, 0x6D08D30E, + 0xA65400AB, 0x20C07205, 0xEB9CA1A0, 0x11E81EB4, 0xDAB4CD11, 0x5C20BFBF, 0x977C6C1A, 0x8A795CA2, 0x41258F07, + 0xC7B1FDA9, 0x0CED2E0C, 0xFDBB9CD9, 0x36E74F7C, 0xB0733DD2, 0x7B2FEE77, 0x662ADECF, 0xAD760D6A, 0x2BE27FC4, + 0xE0BEAC61, 0x123E1C2F, 0xD962CF8A, 0x5FF6BD24, 0x94AA6E81, 0x89AF5E39, 0x42F38D9C, 0xC467FF32, 0x0F3B2C97, + 0xFE6D9E42, 0x35314DE7, 0xB3A53F49, 0x78F9ECEC, 0x65FCDC54, 0xAEA00FF1, 0x28347D5F, 0xE368AEFA, 0x16441B82, + 0xDD18C827, 0x5B8CBA89, 0x90D0692C, 0x8DD55994, 0x46898A31, 0xC01DF89F, 0x0B412B3A, 0xFA1799EF, 0x314B4A4A, + 0xB7DF38E4, 0x7C83EB41, 0x6186DBF9, 0xAADA085C, 0x2C4E7AF2, 0xE712A957, 0x15921919, 0xDECECABC, 0x585AB812, + 0x93066BB7, 0x8E035B0F, 0x455F88AA, 0xC3CBFA04, 0x089729A1, 0xF9C19B74, 0x329D48D1, 0xB4093A7F, 0x7F55E9DA, + 0x6250D962, 0xA90C0AC7, 0x2F987869, 0xE4C4ABCC, + /* T8_6 */ + 0x00000000, 0xA6770BB4, 0x979F1129, 0x31E81A9D, 0xF44F2413, 0x52382FA7, 0x63D0353A, 0xC5A73E8E, 0x33EF4E67, + 0x959845D3, 0xA4705F4E, 0x020754FA, 0xC7A06A74, 0x61D761C0, 0x503F7B5D, 0xF64870E9, 0x67DE9CCE, 0xC1A9977A, + 0xF0418DE7, 0x56368653, 0x9391B8DD, 0x35E6B369, 0x040EA9F4, 0xA279A240, 0x5431D2A9, 0xF246D91D, 0xC3AEC380, + 0x65D9C834, 0xA07EF6BA, 0x0609FD0E, 0x37E1E793, 0x9196EC27, 0xCFBD399C, 0x69CA3228, 0x582228B5, 0xFE552301, + 0x3BF21D8F, 0x9D85163B, 0xAC6D0CA6, 0x0A1A0712, 0xFC5277FB, 0x5A257C4F, 0x6BCD66D2, 0xCDBA6D66, 0x081D53E8, + 0xAE6A585C, 0x9F8242C1, 0x39F54975, 0xA863A552, 0x0E14AEE6, 0x3FFCB47B, 0x998BBFCF, 0x5C2C8141, 0xFA5B8AF5, + 0xCBB39068, 0x6DC49BDC, 0x9B8CEB35, 0x3DFBE081, 0x0C13FA1C, 0xAA64F1A8, 0x6FC3CF26, 0xC9B4C492, 0xF85CDE0F, + 0x5E2BD5BB, 0x440B7579, 0xE27C7ECD, 0xD3946450, 0x75E36FE4, 0xB044516A, 0x16335ADE, 0x27DB4043, 0x81AC4BF7, + 0x77E43B1E, 0xD19330AA, 0xE07B2A37, 0x460C2183, 0x83AB1F0D, 0x25DC14B9, 0x14340E24, 0xB2430590, 0x23D5E9B7, + 0x85A2E203, 0xB44AF89E, 0x123DF32A, 0xD79ACDA4, 0x71EDC610, 0x4005DC8D, 0xE672D739, 0x103AA7D0, 0xB64DAC64, + 0x87A5B6F9, 0x21D2BD4D, 0xE47583C3, 0x42028877, 0x73EA92EA, 0xD59D995E, 0x8BB64CE5, 0x2DC14751, 0x1C295DCC, + 0xBA5E5678, 0x7FF968F6, 0xD98E6342, 0xE86679DF, 0x4E11726B, 0xB8590282, 0x1E2E0936, 0x2FC613AB, 0x89B1181F, + 0x4C162691, 0xEA612D25, 0xDB8937B8, 0x7DFE3C0C, 0xEC68D02B, 0x4A1FDB9F, 0x7BF7C102, 0xDD80CAB6, 0x1827F438, + 0xBE50FF8C, 0x8FB8E511, 0x29CFEEA5, 0xDF879E4C, 0x79F095F8, 0x48188F65, 0xEE6F84D1, 0x2BC8BA5F, 0x8DBFB1EB, + 0xBC57AB76, 0x1A20A0C2, 0x8816EAF2, 0x2E61E146, 0x1F89FBDB, 0xB9FEF06F, 0x7C59CEE1, 0xDA2EC555, 0xEBC6DFC8, + 0x4DB1D47C, 0xBBF9A495, 0x1D8EAF21, 0x2C66B5BC, 0x8A11BE08, 0x4FB68086, 0xE9C18B32, 0xD82991AF, 0x7E5E9A1B, + 0xEFC8763C, 0x49BF7D88, 0x78576715, 0xDE206CA1, 0x1B87522F, 0xBDF0599B, 0x8C184306, 0x2A6F48B2, 0xDC27385B, + 0x7A5033EF, 0x4BB82972, 0xEDCF22C6, 0x28681C48, 0x8E1F17FC, 0xBFF70D61, 0x198006D5, 0x47ABD36E, 0xE1DCD8DA, + 0xD034C247, 0x7643C9F3, 0xB3E4F77D, 0x1593FCC9, 0x247BE654, 0x820CEDE0, 0x74449D09, 0xD23396BD, 0xE3DB8C20, + 0x45AC8794, 0x800BB91A, 0x267CB2AE, 0x1794A833, 0xB1E3A387, 0x20754FA0, 0x86024414, 0xB7EA5E89, 0x119D553D, + 0xD43A6BB3, 0x724D6007, 0x43A57A9A, 0xE5D2712E, 0x139A01C7, 0xB5ED0A73, 0x840510EE, 0x22721B5A, 0xE7D525D4, + 0x41A22E60, 0x704A34FD, 0xD63D3F49, 0xCC1D9F8B, 0x6A6A943F, 0x5B828EA2, 0xFDF58516, 0x3852BB98, 0x9E25B02C, + 0xAFCDAAB1, 0x09BAA105, 0xFFF2D1EC, 0x5985DA58, 0x686DC0C5, 0xCE1ACB71, 0x0BBDF5FF, 0xADCAFE4B, 0x9C22E4D6, + 0x3A55EF62, 0xABC30345, 0x0DB408F1, 0x3C5C126C, 0x9A2B19D8, 0x5F8C2756, 0xF9FB2CE2, 0xC813367F, 0x6E643DCB, + 0x982C4D22, 0x3E5B4696, 0x0FB35C0B, 0xA9C457BF, 0x6C636931, 0xCA146285, 0xFBFC7818, 0x5D8B73AC, 0x03A0A617, + 0xA5D7ADA3, 0x943FB73E, 0x3248BC8A, 0xF7EF8204, 0x519889B0, 0x6070932D, 0xC6079899, 0x304FE870, 0x9638E3C4, + 0xA7D0F959, 0x01A7F2ED, 0xC400CC63, 0x6277C7D7, 0x539FDD4A, 0xF5E8D6FE, 0x647E3AD9, 0xC209316D, 0xF3E12BF0, + 0x55962044, 0x90311ECA, 0x3646157E, 0x07AE0FE3, 0xA1D90457, 0x579174BE, 0xF1E67F0A, 0xC00E6597, 0x66796E23, + 0xA3DE50AD, 0x05A95B19, 0x34414184, 0x92364A30, + /* T8_7 */ + 0x00000000, 0xCCAA009E, 0x4225077D, 0x8E8F07E3, 0x844A0EFA, 0x48E00E64, 0xC66F0987, 0x0AC50919, 0xD3E51BB5, + 0x1F4F1B2B, 0x91C01CC8, 0x5D6A1C56, 0x57AF154F, 0x9B0515D1, 0x158A1232, 0xD92012AC, 0x7CBB312B, 0xB01131B5, + 0x3E9E3656, 0xF23436C8, 0xF8F13FD1, 0x345B3F4F, 0xBAD438AC, 0x767E3832, 0xAF5E2A9E, 0x63F42A00, 0xED7B2DE3, + 0x21D12D7D, 0x2B142464, 0xE7BE24FA, 0x69312319, 0xA59B2387, 0xF9766256, 0x35DC62C8, 0xBB53652B, 0x77F965B5, + 0x7D3C6CAC, 0xB1966C32, 0x3F196BD1, 0xF3B36B4F, 0x2A9379E3, 0xE639797D, 0x68B67E9E, 0xA41C7E00, 0xAED97719, + 0x62737787, 0xECFC7064, 0x205670FA, 0x85CD537D, 0x496753E3, 0xC7E85400, 0x0B42549E, 0x01875D87, 0xCD2D5D19, + 0x43A25AFA, 0x8F085A64, 0x562848C8, 0x9A824856, 0x140D4FB5, 0xD8A74F2B, 0xD2624632, 0x1EC846AC, 0x9047414F, + 0x5CED41D1, 0x299DC2ED, 0xE537C273, 0x6BB8C590, 0xA712C50E, 0xADD7CC17, 0x617DCC89, 0xEFF2CB6A, 0x2358CBF4, + 0xFA78D958, 0x36D2D9C6, 0xB85DDE25, 0x74F7DEBB, 0x7E32D7A2, 0xB298D73C, 0x3C17D0DF, 0xF0BDD041, 0x5526F3C6, + 0x998CF358, 0x1703F4BB, 0xDBA9F425, 0xD16CFD3C, 0x1DC6FDA2, 0x9349FA41, 0x5FE3FADF, 0x86C3E873, 0x4A69E8ED, + 0xC4E6EF0E, 0x084CEF90, 0x0289E689, 0xCE23E617, 0x40ACE1F4, 0x8C06E16A, 0xD0EBA0BB, 0x1C41A025, 0x92CEA7C6, + 0x5E64A758, 0x54A1AE41, 0x980BAEDF, 0x1684A93C, 0xDA2EA9A2, 0x030EBB0E, 0xCFA4BB90, 0x412BBC73, 0x8D81BCED, + 0x8744B5F4, 0x4BEEB56A, 0xC561B289, 0x09CBB217, 0xAC509190, 0x60FA910E, 0xEE7596ED, 0x22DF9673, 0x281A9F6A, + 0xE4B09FF4, 0x6A3F9817, 0xA6959889, 0x7FB58A25, 0xB31F8ABB, 0x3D908D58, 0xF13A8DC6, 0xFBFF84DF, 0x37558441, + 0xB9DA83A2, 0x7570833C, 0x533B85DA, 0x9F918544, 0x111E82A7, 0xDDB48239, 0xD7718B20, 0x1BDB8BBE, 0x95548C5D, + 0x59FE8CC3, 0x80DE9E6F, 0x4C749EF1, 0xC2FB9912, 0x0E51998C, 0x04949095, 0xC83E900B, 0x46B197E8, 0x8A1B9776, + 0x2F80B4F1, 0xE32AB46F, 0x6DA5B38C, 0xA10FB312, 0xABCABA0B, 0x6760BA95, 0xE9EFBD76, 0x2545BDE8, 0xFC65AF44, + 0x30CFAFDA, 0xBE40A839, 0x72EAA8A7, 0x782FA1BE, 0xB485A120, 0x3A0AA6C3, 0xF6A0A65D, 0xAA4DE78C, 0x66E7E712, + 0xE868E0F1, 0x24C2E06F, 0x2E07E976, 0xE2ADE9E8, 0x6C22EE0B, 0xA088EE95, 0x79A8FC39, 0xB502FCA7, 0x3B8DFB44, + 0xF727FBDA, 0xFDE2F2C3, 0x3148F25D, 0xBFC7F5BE, 0x736DF520, 0xD6F6D6A7, 0x1A5CD639, 0x94D3D1DA, 0x5879D144, + 0x52BCD85D, 0x9E16D8C3, 0x1099DF20, 0xDC33DFBE, 0x0513CD12, 0xC9B9CD8C, 0x4736CA6F, 0x8B9CCAF1, 0x8159C3E8, + 0x4DF3C376, 0xC37CC495, 0x0FD6C40B, 0x7AA64737, 0xB60C47A9, 0x3883404A, 0xF42940D4, 0xFEEC49CD, 0x32464953, + 0xBCC94EB0, 0x70634E2E, 0xA9435C82, 0x65E95C1C, 0xEB665BFF, 0x27CC5B61, 0x2D095278, 0xE1A352E6, 0x6F2C5505, + 0xA386559B, 0x061D761C, 0xCAB77682, 0x44387161, 0x889271FF, 0x825778E6, 0x4EFD7878, 0xC0727F9B, 0x0CD87F05, + 0xD5F86DA9, 0x19526D37, 0x97DD6AD4, 0x5B776A4A, 0x51B26353, 0x9D1863CD, 0x1397642E, 0xDF3D64B0, 0x83D02561, + 0x4F7A25FF, 0xC1F5221C, 0x0D5F2282, 0x079A2B9B, 0xCB302B05, 0x45BF2CE6, 0x89152C78, 0x50353ED4, 0x9C9F3E4A, + 0x121039A9, 0xDEBA3937, 0xD47F302E, 0x18D530B0, 0x965A3753, 0x5AF037CD, 0xFF6B144A, 0x33C114D4, 0xBD4E1337, + 0x71E413A9, 0x7B211AB0, 0xB78B1A2E, 0x39041DCD, 0xF5AE1D53, 0x2C8E0FFF, 0xE0240F61, 0x6EAB0882, 0xA201081C, + 0xA8C40105, 0x646E019B, 0xEAE10678, 0x264B06E6 }; } diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index 8a305b0fb4656..69530c187cd1c 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -29,7 +29,7 @@ public class Utils { private static final Pattern HOST_PORT_PATTERN = Pattern.compile("\\[?(.+?)\\]?:(\\d+)"); - public static String NL = System.getProperty("line.separator"); + public static final String NL = System.getProperty("line.separator"); /** * Turn the given UTF8 byte array into a string @@ -87,10 +87,10 @@ public static long readUnsignedInt(ByteBuffer buffer, int index) { * @return The integer read (MUST BE TREATED WITH SPECIAL CARE TO AVOID SIGNEDNESS) */ public static int readUnsignedIntLE(InputStream in) throws IOException { - return (in.read() << 8*0) - | (in.read() << 8*1) - | (in.read() << 8*2) - | (in.read() << 8*3); + return (in.read() << 8 * 0) + | (in.read() << 8 * 1) + | (in.read() << 8 * 2) + | (in.read() << 8 * 3); } /** @@ -102,10 +102,10 @@ public static int readUnsignedIntLE(InputStream in) throws IOException { * @return The integer read (MUST BE TREATED WITH SPECIAL CARE TO AVOID SIGNEDNESS) */ public static int readUnsignedIntLE(byte[] buffer, int offset) { - return (buffer[offset++] << 8*0) - | (buffer[offset++] << 8*1) - | (buffer[offset++] << 8*2) - | (buffer[offset] << 8*3); + return (buffer[offset++] << 8 * 0) + | (buffer[offset++] << 8 * 1) + | (buffer[offset++] << 8 * 2) + | (buffer[offset] << 8 * 3); } /** @@ -136,10 +136,10 @@ public static void writeUnsignedInt(ByteBuffer buffer, int index, long value) { * @param value The value to write */ public static void writeUnsignedIntLE(OutputStream out, int value) throws IOException { - out.write(value >>> 8*0); - out.write(value >>> 8*1); - out.write(value >>> 8*2); - out.write(value >>> 8*3); + out.write(value >>> 8 * 0); + out.write(value >>> 8 * 1); + out.write(value >>> 8 * 2); + out.write(value >>> 8 * 3); } /** @@ -151,10 +151,10 @@ public static void writeUnsignedIntLE(OutputStream out, int value) throws IOExce * @param value The value to write */ public static void writeUnsignedIntLE(byte[] buffer, int offset, int value) { - buffer[offset++] = (byte) (value >>> 8*0); - buffer[offset++] = (byte) (value >>> 8*1); - buffer[offset++] = (byte) (value >>> 8*2); - buffer[offset] = (byte) (value >>> 8*3); + buffer[offset++] = (byte) (value >>> 8 * 0); + buffer[offset++] = (byte) (value >>> 8 * 1); + buffer[offset++] = (byte) (value >>> 8 * 2); + buffer[offset] = (byte) (value >>> 8 * 3); } @@ -285,7 +285,7 @@ public static int murmur2(final byte[] data) { case 2: h ^= (data[(length & ~3) + 1] & 0xff) << 8; case 1: - h ^= (data[length & ~3] & 0xff); + h ^= data[length & ~3] & 0xff; h *= m; } @@ -348,11 +348,11 @@ public static String join(T[] strs, String seperator) { public static String join(Collection list, String seperator) { StringBuilder sb = new StringBuilder(); Iterator iter = list.iterator(); - while(iter.hasNext()) { + while (iter.hasNext()) { sb.append(iter.next()); - if(iter.hasNext()) - sb.append(seperator); + if (iter.hasNext()) + sb.append(seperator); } - return sb.toString(); + return sb.toString(); } } diff --git a/clients/src/test/java/org/apache/kafka/common/utils/ClientUtilsTest.java b/clients/src/test/java/org/apache/kafka/clients/ClientUtilsTest.java similarity index 97% rename from clients/src/test/java/org/apache/kafka/common/utils/ClientUtilsTest.java rename to clients/src/test/java/org/apache/kafka/clients/ClientUtilsTest.java index 6e37ea553f73d..13ce519f03d13 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/ClientUtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/ClientUtilsTest.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.common.utils; +package org.apache.kafka.clients; import org.apache.kafka.common.config.ConfigException; import org.junit.Test; diff --git a/clients/src/test/java/org/apache/kafka/clients/MockClient.java b/clients/src/test/java/org/apache/kafka/clients/MockClient.java index 67bee40abb61f..8f1a7a625e4ee 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java @@ -1,3 +1,19 @@ +/** + * 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.clients; import java.util.ArrayDeque; @@ -65,7 +81,7 @@ public void send(ClientRequest request) { @Override public List poll(long timeoutMs, long now) { - for(ClientResponse response: this.responses) + for (ClientResponse response: this.responses) if (response.request().hasCallback()) response.request().callback().onComplete(response); List copy = new ArrayList(); diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index 5debcd6cc6ce3..8b278892883e6 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -1,3 +1,19 @@ +/** + * 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.clients; import static org.junit.Assert.assertEquals; @@ -9,7 +25,6 @@ import java.util.Collections; import java.util.List; -import org.apache.kafka.clients.producer.internals.Metadata; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java index e51d2dfdadee3..677edd385f35d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java @@ -1,3 +1,19 @@ +/** + * 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.clients.consumer; import static org.junit.Assert.*; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java index 864f1c736458d..090087a319e26 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java @@ -1,3 +1,19 @@ +/** + * 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.clients.consumer.internals; import static org.junit.Assert.*; diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java index 77b23e7aa8246..4ae43ed47e31a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java @@ -107,7 +107,7 @@ public void testDelayedAllocation() throws Exception { private CountDownLatch asyncDeallocate(final BufferPool pool, final ByteBuffer buffer) { final CountDownLatch latch = new CountDownLatch(1); - new Thread() { + Thread thread = new Thread() { public void run() { try { latch.await(); @@ -116,13 +116,14 @@ public void run() { } pool.deallocate(buffer); } - }.start(); + }; + thread.start(); return latch; } private CountDownLatch asyncAllocate(final BufferPool pool, final int size) { final CountDownLatch completed = new CountDownLatch(1); - new Thread() { + Thread thread = new Thread() { public void run() { try { pool.allocate(size); @@ -132,7 +133,8 @@ public void run() { completed.countDown(); } } - }.start(); + }; + thread.start(); return completed; } @@ -172,12 +174,12 @@ public void run() { try { for (int i = 0; i < iterations; i++) { int size; - if (TestUtils.random.nextBoolean()) + if (TestUtils.RANDOM.nextBoolean()) // allocate poolable size size = pool.poolableSize(); else // allocate a random size - size = TestUtils.random.nextInt((int) pool.totalMemory()); + size = TestUtils.RANDOM.nextInt((int) pool.totalMemory()); ByteBuffer buffer = pool.allocate(size); pool.deallocate(buffer); } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java index 74605c38cfd7e..743aa7e523dd4 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java @@ -12,7 +12,7 @@ */ package org.apache.kafka.clients.producer; -import org.apache.kafka.clients.producer.internals.Metadata; +import org.apache.kafka.clients.Metadata; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.test.TestUtils; @@ -49,7 +49,7 @@ public void testMetadata() throws Exception { } /** - * Tests that {@link org.apache.kafka.clients.producer.internals.Metadata#awaitUpdate(int, long)} doesn't + * Tests that {@link org.apache.kafka.clients.Metadata#awaitUpdate(int, long)} doesn't * wait forever with a max timeout value of 0 * * @throws Exception @@ -68,9 +68,9 @@ public void testMetadataUpdateWaitTime() throws Exception { // expected } // now try with a higher timeout value once - final long TWO_SECOND_WAIT = 2000; + final long twoSecondWait = 2000; try { - metadata.awaitUpdate(metadata.requestUpdate(), TWO_SECOND_WAIT); + metadata.awaitUpdate(metadata.requestUpdate(), twoSecondWait); fail("Wait on metadata update was expected to timeout, but it didn't"); } catch (TimeoutException te) { // expected diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java index d3377ef8dfffd..75513b0bdd439 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java @@ -32,6 +32,7 @@ public class MockProducerTest { private String topic = "topic"; @Test + @SuppressWarnings("unchecked") public void testAutoCompleteMock() throws Exception { MockProducer producer = new MockProducer(true); ProducerRecord record = new ProducerRecord(topic, "key".getBytes(), "value".getBytes()); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java index 82d8083b6072b..29c8417422c0c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java @@ -31,7 +31,7 @@ public class PartitionerTest { private Node node0 = new Node(0, "localhost", 99); private Node node1 = new Node(1, "localhost", 100); private Node node2 = new Node(2, "localhost", 101); - private Node[] nodes = new Node[] { node0, node1, node2 }; + private Node[] nodes = new Node[] {node0, node1, node2}; private String topic = "test"; private List partitions = asList(new PartitionInfo(topic, 0, node0, nodes, nodes), new PartitionInfo(topic, 1, node1, nodes, nodes), diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java index e2bb8da7154ee..83338633717cf 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java @@ -128,6 +128,7 @@ public void testPartialDrain() throws Exception { assertEquals("But due to size bound only one partition should have been retrieved", 1, batches.size()); } + @SuppressWarnings("unused") @Test public void testStressfulSituation() throws Exception { final int numThreads = 5; @@ -194,7 +195,7 @@ public void testNextReadyCheckDelay() throws Exception { assertEquals("Next check time should be defined by node1, half remaining linger time", lingerMs / 2, result.nextReadyCheckDelayMs); // Add data for another partition on node1, enough to make data sendable immediately - for (int i = 0; i < appends+1; i++) + for (int i = 0; i < appends + 1; i++) accum.append(tp2, key, value, CompressionType.NONE, null); result = accum.ready(cluster, time.milliseconds()); assertEquals("Node1 should be ready", Collections.singleton(node1), result.readyNodes); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java index a3700a6a10a9a..1e5d1c2d6159b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java @@ -78,7 +78,7 @@ public void testBlocking() throws Exception { /* create a new request result that will be completed after the given timeout */ public ProduceRequestResult asyncRequest(final long baseOffset, final RuntimeException error, final long timeout) { final ProduceRequestResult request = new ProduceRequestResult(); - new Thread() { + Thread thread = new Thread() { public void run() { try { sleep(timeout); @@ -87,7 +87,8 @@ public void run() { e.printStackTrace(); } } - }.start(); + }; + thread.start(); return request; } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java index 888b9295d47fe..558942aaecd1b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java @@ -21,8 +21,8 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; +import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.MockClient; -import org.apache.kafka.clients.producer.internals.Metadata; import org.apache.kafka.clients.producer.internals.RecordAccumulator; import org.apache.kafka.clients.producer.internals.Sender; import org.apache.kafka.common.Cluster; @@ -147,8 +147,8 @@ private Struct produceResponse(String topic, int part, long offset, int error) { partResp.set("partition", part); partResp.set("error_code", (short) error); partResp.set("base_offset", offset); - response.set("partition_responses", new Object[] { partResp }); - struct.set("responses", new Object[] { response }); + response.set("partition_responses", new Object[] {partResp}); + struct.set("responses", new Object[] {response}); return struct; } diff --git a/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java b/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java index 3cfd36d992f5f..66442ed38533a 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/AbstractConfigTest.java @@ -14,87 +14,67 @@ import static org.junit.Assert.fail; -import java.util.List; import java.util.Map; import java.util.Properties; import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Type; -import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.MetricsReporter; import org.junit.Test; public class AbstractConfigTest { - @Test - public void testConfiguredInstances() { - testValidInputs(""); - testValidInputs("org.apache.kafka.common.config.AbstractConfigTest$TestMetricsReporter"); - testValidInputs("org.apache.kafka.common.config.AbstractConfigTest$TestMetricsReporter,org.apache.kafka.common.config.AbstractConfigTest$TestMetricsReporter"); - testInvalidInputs(","); - testInvalidInputs("org.apache.kafka.clients.producer.unknown-metrics-reporter"); - testInvalidInputs("test1,test2"); - testInvalidInputs("org.apache.kafka.common.config.AbstractConfigTest$TestMetricsReporter,"); - } - - private void testValidInputs(String configValue) { - Properties props = new Properties(); - props.put(TestConfig.METRIC_REPORTER_CLASSES_CONFIG, configValue); - TestConfig config = new TestConfig(props); - try { - config.getConfiguredInstances(TestConfig.METRIC_REPORTER_CLASSES_CONFIG, - MetricsReporter.class); - } catch (ConfigException e) { - fail("No exceptions are expected here, valid props are :" + props); - } - } - - private void testInvalidInputs(String configValue) { - Properties props = new Properties(); - props.put(TestConfig.METRIC_REPORTER_CLASSES_CONFIG, configValue); - TestConfig config = new TestConfig(props); - try { - config.getConfiguredInstances(TestConfig.METRIC_REPORTER_CLASSES_CONFIG, - MetricsReporter.class); - fail("Expected a config exception due to invalid props :" + props); - } catch (ConfigException e) { - // this is good + @Test + public void testConfiguredInstances() { + testValidInputs(""); + testValidInputs("org.apache.kafka.common.metrics.FakeMetricsReporter"); + testValidInputs("org.apache.kafka.common.metrics.FakeMetricsReporter, org.apache.kafka.common.metrics.FakeMetricsReporter"); + testInvalidInputs(","); + testInvalidInputs("org.apache.kafka.clients.producer.unknown-metrics-reporter"); + testInvalidInputs("test1,test2"); + testInvalidInputs("org.apache.kafka.common.metrics.FakeMetricsReporter,"); } - } - - private static class TestConfig extends AbstractConfig { - - private static final ConfigDef config; - - public static final String METRIC_REPORTER_CLASSES_CONFIG = "metric.reporters"; - private static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters."; - static { - config = new ConfigDef().define(METRIC_REPORTER_CLASSES_CONFIG, - Type.LIST, "", Importance.LOW, METRIC_REPORTER_CLASSES_DOC); + private void testValidInputs(String configValue) { + Properties props = new Properties(); + props.put(TestConfig.METRIC_REPORTER_CLASSES_CONFIG, configValue); + TestConfig config = new TestConfig(props); + try { + config.getConfiguredInstances(TestConfig.METRIC_REPORTER_CLASSES_CONFIG, MetricsReporter.class); + } catch (ConfigException e) { + fail("No exceptions are expected here, valid props are :" + props); + } } - public TestConfig(Map props) { - super(config, props); + private void testInvalidInputs(String configValue) { + Properties props = new Properties(); + props.put(TestConfig.METRIC_REPORTER_CLASSES_CONFIG, configValue); + TestConfig config = new TestConfig(props); + try { + config.getConfiguredInstances(TestConfig.METRIC_REPORTER_CLASSES_CONFIG, MetricsReporter.class); + fail("Expected a config exception due to invalid props :" + props); + } catch (ConfigException e) { + // this is good + } } - } - - public static class TestMetricsReporter implements MetricsReporter { - @Override - public void configure(Map configs) { - } + private static class TestConfig extends AbstractConfig { - @Override - public void init(List metrics) { -} + private static final ConfigDef CONFIG; - @Override - public void metricChange(KafkaMetric metric) { - } + public static final String METRIC_REPORTER_CLASSES_CONFIG = "metric.reporters"; + private static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters."; + + static { + CONFIG = new ConfigDef().define(METRIC_REPORTER_CLASSES_CONFIG, + Type.LIST, + "", + Importance.LOW, + METRIC_REPORTER_CLASSES_DOC); + } - @Override - public void close() { + public TestConfig(Map props) { + super(CONFIG, props); + } } - } } diff --git a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java index 16d3fedef8cca..44c2ef0a859de 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java @@ -16,7 +16,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; -import java.util.Arrays; import java.util.HashMap; import java.util.Map; import java.util.Properties; @@ -110,7 +109,7 @@ private void testBadInputs(Type type, Object... values) { @Test(expected = ConfigException.class) public void testInvalidDefaultRange() { - new ConfigDef().define("name", Type.INT, -1, Range.between(0,10), Importance.HIGH, "docs"); + new ConfigDef().define("name", Type.INT, -1, Range.between(0, 10), Importance.HIGH, "docs"); } @Test(expected = ConfigException.class) @@ -120,7 +119,7 @@ public void testInvalidDefaultString() { @Test public void testValidators() { - testValidators(Type.INT, Range.between(0,10), 5, new Object[]{1, 5, 9}, new Object[]{-1, 11}); + testValidators(Type.INT, Range.between(0, 10), 5, new Object[]{1, 5, 9}, new Object[]{-1, 11}); testValidators(Type.STRING, ValidString.in("good", "values", "default"), "default", new Object[]{"good", "values", "default"}, new Object[]{"bad", "inputs"}); } diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/FakeMetricsReporter.java b/clients/src/test/java/org/apache/kafka/common/metrics/FakeMetricsReporter.java new file mode 100644 index 0000000000000..7c7ead11d211e --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/metrics/FakeMetricsReporter.java @@ -0,0 +1,32 @@ +/** + * 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.metrics; + +import java.util.List; +import java.util.Map; + +public class FakeMetricsReporter implements MetricsReporter { + + @Override + public void configure(Map configs) {} + + @Override + public void init(List metrics) {} + + @Override + public void metricChange(KafkaMetric metric) {} + + @Override + public void close() {} + +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java index 998a57c492a51..544e120594de7 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java @@ -36,7 +36,7 @@ public class MetricsTest { - private static double EPS = 0.000001; + private static final double EPS = 0.000001; MockTime time = new MockTime(); Metrics metrics = new Metrics(new MetricConfig(), Arrays.asList((MetricsReporter) new JmxReporter()), time); @@ -71,7 +71,7 @@ public void testSimpleStats() throws Exception { s.add(new MetricName("test.count", "grp1"), new Count()); s.add(new Percentiles(100, -100, 100, BucketSizing.CONSTANT, new Percentile(new MetricName("test.median", "grp1"), 50.0), - new Percentile(new MetricName("test.perc99_9", "grp1"),99.9))); + new Percentile(new MetricName("test.perc99_9", "grp1"), 99.9))); Sensor s2 = metrics.sensor("test.sensor2"); s2.add(new MetricName("s2.total", "grp1"), new Total()); diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/stats/HistogramTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/stats/HistogramTest.java index 3be6b2d5c718f..a55cc3226c68e 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/stats/HistogramTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/stats/HistogramTest.java @@ -22,7 +22,6 @@ import java.util.Random; -import org.apache.kafka.common.metrics.stats.Histogram; import org.apache.kafka.common.metrics.stats.Histogram.BinScheme; import org.apache.kafka.common.metrics.stats.Histogram.ConstantBinScheme; import org.apache.kafka.common.metrics.stats.Histogram.LinearBinScheme; diff --git a/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java index a14659a713795..0d030bc9becfa 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java @@ -40,7 +40,6 @@ */ public class SelectorTest { - private static final List EMPTY = new ArrayList(); private static final int BUFFER_SIZE = 4 * 1024; private EchoServer server; diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java index 4480e9b2aafe6..8b926340de49b 100644 --- a/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java @@ -23,12 +23,6 @@ import java.nio.ByteBuffer; import java.util.Arrays; -import org.apache.kafka.common.protocol.types.ArrayOf; -import org.apache.kafka.common.protocol.types.Field; -import org.apache.kafka.common.protocol.types.Schema; -import org.apache.kafka.common.protocol.types.SchemaException; -import org.apache.kafka.common.protocol.types.Struct; -import org.apache.kafka.common.protocol.types.Type; import org.junit.Before; import org.junit.Test; @@ -53,8 +47,8 @@ public void setup() { .set("int64", (long) 1) .set("string", "1") .set("bytes", "1".getBytes()) - .set("array", new Object[] { 1 }); - this.struct.set("struct", this.struct.instance("struct").set("field", new Object[] { 1, 2, 3 })); + .set("array", new Object[] {1}); + this.struct.set("struct", this.struct.instance("struct").set("field", new Object[] {1, 2, 3})); } @Test @@ -68,9 +62,9 @@ public void testSimple() { check(Type.STRING, "A\u00ea\u00f1\u00fcC"); check(Type.BYTES, ByteBuffer.allocate(0)); check(Type.BYTES, ByteBuffer.wrap("abcd".getBytes())); - check(new ArrayOf(Type.INT32), new Object[] { 1, 2, 3, 4 }); + check(new ArrayOf(Type.INT32), new Object[] {1, 2, 3, 4}); check(new ArrayOf(Type.STRING), new Object[] {}); - check(new ArrayOf(Type.STRING), new Object[] { "hello", "there", "beautiful" }); + check(new ArrayOf(Type.STRING), new Object[] {"hello", "there", "beautiful"}); } @Test diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java index 94a11121e207d..e343327faf115 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java @@ -71,7 +71,7 @@ public void testIterator() { public static Collection data() { List values = new ArrayList(); for (CompressionType type: CompressionType.values()) - values.add(new Object[] { type }); + values.add(new Object[] {type}); return values; } } diff --git a/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java b/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java index 2765913d5bfd4..957fc8fa3999c 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java @@ -63,7 +63,7 @@ public void testFields() { @Test public void testChecksum() { assertEquals(record.checksum(), record.computeChecksum()); - assertEquals(record.checksum(), record.computeChecksum( + assertEquals(record.checksum(), Record.computeChecksum( this.key == null ? null : this.key.array(), this.value == null ? null : this.value.array(), this.compression, 0, -1)); @@ -102,7 +102,7 @@ public static Collection data() { for (byte[] key : Arrays.asList(null, "".getBytes(), "key".getBytes(), payload)) for (byte[] value : Arrays.asList(null, "".getBytes(), "value".getBytes(), payload)) for (CompressionType compression : CompressionType.values()) - values.add(new Object[] { key, value, compression }); + values.add(new Object[] {key, value, compression}); return values; } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index df37fc6d8f0db..13237fd72da54 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 @@ -17,6 +17,7 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.Errors; import org.junit.Test; import java.lang.reflect.Method; @@ -31,7 +32,7 @@ public class RequestResponseTest { @Test - public void testSerialization() throws Exception{ + public void testSerialization() throws Exception { List requestList = Arrays.asList( createRequestHeader(), createResponseHeader(), @@ -67,7 +68,7 @@ public void testSerialization() throws Exception{ } private AbstractRequestResponse createRequestHeader() { - return new RequestHeader((short)10, (short)1, "", 10); + return new RequestHeader((short) 10, (short) 1, "", 10); } private AbstractRequestResponse createResponseHeader() { @@ -79,7 +80,7 @@ private AbstractRequestResponse createConsumerMetadataRequest() { } private AbstractRequestResponse createConsumerMetadataResponse() { - return new ConsumerMetadataResponse((short)1, new Node(10, "host1", 2014)); + return new ConsumerMetadataResponse((short) 1, new Node(10, "host1", 2014)); } private AbstractRequestResponse createFetchRequest() { @@ -91,7 +92,7 @@ private AbstractRequestResponse createFetchRequest() { private AbstractRequestResponse createFetchResponse() { Map responseData = new HashMap(); - responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData((short)0, 1000000, ByteBuffer.allocate(10))); + responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData(Errors.NONE.code(), 1000000, ByteBuffer.allocate(10))); return new FetchResponse(responseData); } @@ -100,7 +101,7 @@ private AbstractRequestResponse createHeartBeatRequest() { } private AbstractRequestResponse createHeartBeatResponse() { - return new HeartbeatResponse((short)0); + return new HeartbeatResponse(Errors.NONE.code()); } private AbstractRequestResponse createJoinGroupRequest() { @@ -108,7 +109,7 @@ private AbstractRequestResponse createJoinGroupRequest() { } private AbstractRequestResponse createJoinGroupResponse() { - return new JoinGroupResponse((short)0, 1, "consumer1", Arrays.asList(new TopicPartition("test11", 1), new TopicPartition("test2", 1))); + return new JoinGroupResponse(Errors.NONE.code(), 1, "consumer1", Arrays.asList(new TopicPartition("test11", 1), new TopicPartition("test2", 1))); } private AbstractRequestResponse createListOffsetRequest() { @@ -119,7 +120,7 @@ private AbstractRequestResponse createListOffsetRequest() { private AbstractRequestResponse createListOffsetResponse() { Map responseData = new HashMap(); - responseData.put(new TopicPartition("test", 0), new ListOffsetResponse.PartitionData((short)0, Arrays.asList(100L))); + responseData.put(new TopicPartition("test", 0), new ListOffsetResponse.PartitionData(Errors.NONE.code(), Arrays.asList(100L))); return new ListOffsetResponse(responseData); } @@ -145,7 +146,7 @@ private AbstractRequestResponse createOffsetCommitRequest() { private AbstractRequestResponse createOffsetCommitResponse() { Map responseData = new HashMap(); - responseData.put(new TopicPartition("test", 0), (short)0); + responseData.put(new TopicPartition("test", 0), Errors.NONE.code()); return new OffsetCommitResponse(responseData); } @@ -155,19 +156,19 @@ private AbstractRequestResponse createOffsetFetchRequest() { private AbstractRequestResponse createOffsetFetchResponse() { Map responseData = new HashMap(); - responseData.put(new TopicPartition("test", 0), new OffsetFetchResponse.PartitionData(100L, "", (short)0)); + responseData.put(new TopicPartition("test", 0), new OffsetFetchResponse.PartitionData(100L, "", Errors.NONE.code())); return new OffsetFetchResponse(responseData); } private AbstractRequestResponse createProduceRequest() { Map produceData = new HashMap(); produceData.put(new TopicPartition("test", 0), ByteBuffer.allocate(10)); - return new ProduceRequest((short)0, 5000, produceData); + return new ProduceRequest(Errors.NONE.code(), 5000, produceData); } private AbstractRequestResponse createProduceResponse() { Map responseData = new HashMap(); - responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse((short) 0, 10000)); + responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE.code(), 10000)); return new ProduceResponse(responseData); } } diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java index b6e1497506675..f5cd61c1aa943 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java @@ -35,13 +35,13 @@ public SerDeser(Serializer serializer, Deserializer deserializer) { @Test public void testStringSerializer() { - String str = "my string"; + String str = "my string"; String mytopic = "testTopic"; List encodings = new ArrayList(); encodings.add("UTF8"); encodings.add("UTF-16"); - for ( String encoding : encodings) { + for (String encoding : encodings) { SerDeser serDeser = getStringSerDeser(encoding); Serializer serializer = serDeser.serializer; Deserializer deserializer = serDeser.deserializer; diff --git a/clients/src/test/java/org/apache/kafka/common/utils/CrcTest.java b/clients/src/test/java/org/apache/kafka/common/utils/CrcTest.java index 6b323819390b5..c39c3cff64d4e 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/CrcTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/CrcTest.java @@ -25,7 +25,7 @@ public class CrcTest { @Test public void testUpdate() { - final byte bytes[] = "Any String you want".getBytes(); + final byte[] bytes = "Any String you want".getBytes(); final int len = bytes.length; Crc32 crc1 = new Crc32(); @@ -33,10 +33,10 @@ public void testUpdate() { Crc32 crc3 = new Crc32(); crc1.update(bytes, 0, len); - for(int i = 0; i < len; i++) + for (int i = 0; i < len; i++) crc2.update(bytes[i]); - crc3.update(bytes, 0, len/2); - crc3.update(bytes, len/2, len-len/2); + crc3.update(bytes, 0, len / 2); + crc3.update(bytes, len / 2, len - len / 2); assertEquals("Crc values should be the same", crc1.getValue(), crc2.getValue()); assertEquals("Crc values should be the same", crc1.getValue(), crc3.getValue()); diff --git a/clients/src/test/java/org/apache/kafka/test/Microbenchmarks.java b/clients/src/test/java/org/apache/kafka/test/Microbenchmarks.java index b24d4de21bfea..8cd19b2edfa75 100644 --- a/clients/src/test/java/org/apache/kafka/test/Microbenchmarks.java +++ b/clients/src/test/java/org/apache/kafka/test/Microbenchmarks.java @@ -162,7 +162,6 @@ private static void benchMap(int numThreads, final int iters, final Map data = producerDataChannel.receiveRequest(); if(!data.equals(shutdownMessage)) { producer.send(data); - if(logger.isDebugEnabled()) logger.debug("Sending message %s".format(new String(data.message()))); + if(logger.isDebugEnabled()) logger.debug(String.format("Sending message %s", new String(data.message()))); } else break; diff --git a/core/src/main/scala/kafka/utils/Crc32.java b/core/src/main/scala/kafka/utils/Crc32.java index af9fe0d7d4ab2..0e0e7bcb33886 100644 --- a/core/src/main/scala/kafka/utils/Crc32.java +++ b/core/src/main/scala/kafka/utils/Crc32.java @@ -62,16 +62,16 @@ public void update(byte[] b, int off, int len) { final int c1 =(b[off+1] ^ (localCrc >>>= 8)) & 0xff; final int c2 =(b[off+2] ^ (localCrc >>>= 8)) & 0xff; final int c3 =(b[off+3] ^ (localCrc >>>= 8)) & 0xff; - localCrc = (T[T8_7_start + c0] ^ T[T8_6_start + c1]) - ^ (T[T8_5_start + c2] ^ T[T8_4_start + c3]); + localCrc = (T[T8_7_START + c0] ^ T[T8_6_START + c1]) + ^ (T[T8_5_START + c2] ^ T[T8_4_START + c3]); final int c4 = b[off+4] & 0xff; final int c5 = b[off+5] & 0xff; final int c6 = b[off+6] & 0xff; final int c7 = b[off+7] & 0xff; - localCrc ^= (T[T8_3_start + c4] ^ T[T8_2_start + c5]) - ^ (T[T8_1_start + c6] ^ T[T8_0_start + c7]); + localCrc ^= (T[T8_3_START + c4] ^ T[T8_2_START + c5]) + ^ (T[T8_1_START + c6] ^ T[T8_0_START + c7]); off += 8; len -= 8; @@ -79,13 +79,13 @@ public void update(byte[] b, int off, int len) { /* loop unroll - duff's device style */ switch(len) { - case 7: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; - case 6: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; - case 5: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; - case 4: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; - case 3: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; - case 2: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; - case 1: localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)]; + case 7: localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)]; + case 6: localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)]; + case 5: localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)]; + case 4: localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)]; + case 3: localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)]; + case 2: localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)]; + case 1: localCrc = (localCrc >>> 8) ^ T[T8_0_START + ((localCrc ^ b[off++]) & 0xff)]; default: /* nothing */ } @@ -96,21 +96,21 @@ public void update(byte[] b, int off, int len) { @Override final public void update(int b) { - crc = (crc >>> 8) ^ T[T8_0_start + ((crc ^ b) & 0xff)]; + crc = (crc >>> 8) ^ T[T8_0_START + ((crc ^ b) & 0xff)]; } /* * CRC-32 lookup tables generated by the polynomial 0xEDB88320. * See also TestPureJavaCrc32.Table. */ - private static final int T8_0_start = 0*256; - private static final int T8_1_start = 1*256; - private static final int T8_2_start = 2*256; - private static final int T8_3_start = 3*256; - private static final int T8_4_start = 4*256; - private static final int T8_5_start = 5*256; - private static final int T8_6_start = 6*256; - private static final int T8_7_start = 7*256; + private static final int T8_0_START = 0*256; + private static final int T8_1_START = 1*256; + private static final int T8_2_START = 2*256; + private static final int T8_3_START = 3*256; + private static final int T8_4_START = 4*256; + private static final int T8_5_START = 5*256; + private static final int T8_6_START = 6*256; + private static final int T8_7_START = 7*256; private static final int[] T = new int[] { /* T8_0 */ diff --git a/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java b/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java index c79192c5c195d..0d66fe5f88191 100644 --- a/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java +++ b/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java @@ -22,7 +22,7 @@ import java.io.UnsupportedEncodingException; import java.nio.ByteBuffer; -import java.util.ArrayList; +import java.util.Collections; import java.util.List; import kafka.javaapi.consumer.SimpleConsumer; import kafka.javaapi.message.ByteBufferMessageSet; @@ -71,10 +71,9 @@ public static void main(String[] args) throws Exception { printMessages((ByteBufferMessageSet) fetchResponse.messageSet(KafkaProperties.topic2, 0)); System.out.println("Testing single multi-fetch"); - Map> topicMap = new HashMap>() {{ - put(KafkaProperties.topic2, new ArrayList(){{ add(0); }}); - put(KafkaProperties.topic3, new ArrayList(){{ add(0); }}); - }}; + Map> topicMap = new HashMap>(); + topicMap.put(KafkaProperties.topic2, Collections.singletonList(0)); + topicMap.put(KafkaProperties.topic3, Collections.singletonList(0)); req = new FetchRequestBuilder() .clientId(KafkaProperties.clientId) .addFetch(KafkaProperties.topic2, 0, 0L, 100) From 0839def4bdbfd9f5939e463f42cfbc81a98a8eff Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 9 Feb 2015 09:33:16 -0800 Subject: [PATCH 116/491] KAFKA-1925; Fix coordinator broker id stuck with INT_MIN; reviewed by Jay Kreps --- .../java/org/apache/kafka/clients/consumer/KafkaConsumer.java | 2 +- clients/src/main/java/org/apache/kafka/common/Node.java | 2 +- .../scala/unit/kafka/integration/KafkaServerTestHarness.scala | 1 + 3 files changed, 3 insertions(+), 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 09a6f11163ecb..67ceb754a52c0 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 @@ -1281,7 +1281,7 @@ private Node fetchConsumerCoordinator() { if (!resp.wasDisconnected()) { ConsumerMetadataResponse response = new ConsumerMetadataResponse(resp.responseBody()); if (response.errorCode() == Errors.NONE.code()) - return new Node(Integer.MIN_VALUE, response.node().host(), response.node().port()); + return new Node(Integer.MAX_VALUE - response.node().id(), response.node().host(), response.node().port()); } } return null; diff --git a/clients/src/main/java/org/apache/kafka/common/Node.java b/clients/src/main/java/org/apache/kafka/common/Node.java index 0e47ff3ff0e05..88c3b2425e42d 100644 --- a/clients/src/main/java/org/apache/kafka/common/Node.java +++ b/clients/src/main/java/org/apache/kafka/common/Node.java @@ -82,7 +82,7 @@ public boolean equals(Object obj) { @Override public String toString() { - return "Node(" + (id < 0 ? "" : id + ", ") + host + ", " + port + ")"; + return "Node(" + id + ", " + host + ", " + port + ")"; } } diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index ef4c9aeaa2711..dc0512b526e91 100644 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -64,6 +64,7 @@ trait KafkaServerTestHarness extends JUnit3Suite with ZooKeeperTestHarness { val index = TestUtils.random.nextInt(servers.length) if(alive(index)) { servers(index).shutdown() + servers(index).awaitShutdown() alive(index) = false } index From 39cd48de321585ac55c94ec407fede5858f38962 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 9 Feb 2015 15:13:10 -0800 Subject: [PATCH 117/491] KAFKA-1333; Add the consumer coordinator to server; reviewed by Onur Karaman and Jay Kreps --- .../scala/kafka/network/SocketServer.scala | 2 +- .../kafka/server/DelayedOperationKey.scala | 19 +++ .../main/scala/kafka/server/KafkaApis.scala | 70 +++++---- .../main/scala/kafka/server/KafkaServer.scala | 144 +++++++++++------- .../scala/kafka/server/MetadataCache.scala | 17 ++- .../scala/kafka/server/OffsetManager.scala | 2 + .../main/scala/kafka/tools/MirrorMaker.scala | 6 +- .../api/ProducerFailureHandlingTest.scala | 2 +- .../kafka/server/ServerShutdownTest.scala | 13 +- .../unit/kafka/server/ServerStartupTest.scala | 15 -- 10 files changed, 179 insertions(+), 111 deletions(-) diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index 39b1651b680b2..76ce41aed6e04 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -107,7 +107,7 @@ class SocketServer(val brokerId: Int, */ private[kafka] abstract class AbstractServerThread(connectionQuotas: ConnectionQuotas) extends Runnable with Logging { - protected val selector = Selector.open(); + protected val selector = Selector.open() private val startupLatch = new CountDownLatch(1) private val shutdownLatch = new CountDownLatch(1) private val alive = new AtomicBoolean(true) diff --git a/core/src/main/scala/kafka/server/DelayedOperationKey.scala b/core/src/main/scala/kafka/server/DelayedOperationKey.scala index fb7e9ed5c16dd..b673e43b0ba40 100644 --- a/core/src/main/scala/kafka/server/DelayedOperationKey.scala +++ b/core/src/main/scala/kafka/server/DelayedOperationKey.scala @@ -30,9 +30,28 @@ object DelayedOperationKey { val globalLabel = "All" } +/* used by delayed-produce and delayed-fetch operations */ case class TopicPartitionOperationKey(topic: String, partition: Int) extends DelayedOperationKey { def this(topicAndPartition: TopicAndPartition) = this(topicAndPartition.topic, topicAndPartition.partition) override def keyLabel = "%s-%d".format(topic, partition) } + +/* used by bucketized delayed-heartbeat operations */ +case class TTimeMsKey(time: Long) extends DelayedOperationKey { + + override def keyLabel = "%d".format(time) +} + +/* used by delayed-join-group operations */ +case class ConsumerKey(groupId: String, consumerId: String) extends DelayedOperationKey { + + override def keyLabel = "%s-%s".format(groupId, consumerId) +} + +/* used by delayed-rebalance operations */ +case class ConsumerGroupKey(groupId: String) extends DelayedOperationKey { + + override def keyLabel = groupId +} diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index f2b027bf944e7..6ee7d8819a9ef 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -17,24 +17,20 @@ package kafka.server -import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.JoinGroupResponse import org.apache.kafka.common.requests.HeartbeatResponse -import org.apache.kafka.common.requests.ResponseHeader -import org.apache.kafka.common.protocol.types.Struct +import org.apache.kafka.common.TopicPartition import kafka.api._ +import kafka.admin.AdminUtils import kafka.common._ +import kafka.controller.KafkaController +import kafka.coordinator.ConsumerCoordinator import kafka.log._ import kafka.network._ -import kafka.admin.AdminUtils import kafka.network.RequestChannel.Response -import kafka.controller.KafkaController import kafka.utils.{SystemTime, Logging} -import java.nio.ByteBuffer -import java.util.concurrent.TimeUnit -import java.util.concurrent.atomic._ import scala.collection._ import org.I0Itec.zkclient.ZkClient @@ -45,14 +41,14 @@ import org.I0Itec.zkclient.ZkClient class KafkaApis(val requestChannel: RequestChannel, val replicaManager: ReplicaManager, val offsetManager: OffsetManager, + val coordinator: ConsumerCoordinator, + val controller: KafkaController, val zkClient: ZkClient, val brokerId: Int, - val config: KafkaConfig, - val controller: KafkaController) extends Logging { + val config: KafkaConfig) extends Logging { this.logIdent = "[KafkaApi-%d] ".format(brokerId) - val metadataCache = new MetadataCache - private var consumerGroupGenerationId = 0 + val metadataCache = new MetadataCache(brokerId) /** * Top-level method that handles all requests and multiplexes to the right api @@ -137,7 +133,7 @@ class KafkaApis(val requestChannel: RequestChannel, def handleOffsetCommitRequest(request: RequestChannel.Request) { val offsetCommitRequest = request.requestObj.asInstanceOf[OffsetCommitRequest] - // the callback for sending the response + // the callback for sending an offset commit response def sendResponseCallback(commitStatus: immutable.Map[TopicAndPartition, Short]) { commitStatus.foreach { case (topicAndPartition, errorCode) => // we only print warnings for known errors here; only replica manager could see an unknown @@ -169,7 +165,7 @@ class KafkaApis(val requestChannel: RequestChannel, def handleProducerRequest(request: RequestChannel.Request) { val produceRequest = request.requestObj.asInstanceOf[ProducerRequest] - // the callback for sending the response + // the callback for sending a produce response def sendResponseCallback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) { var errorInResponse = false responseStatus.foreach { case (topicAndPartition, status) => @@ -224,7 +220,7 @@ class KafkaApis(val requestChannel: RequestChannel, def handleFetchRequest(request: RequestChannel.Request) { val fetchRequest = request.requestObj.asInstanceOf[FetchRequest] - // the callback for sending the response + // the callback for sending a fetch response def sendResponseCallback(responsePartitionData: Map[TopicAndPartition, FetchResponsePartitionData]) { responsePartitionData.foreach { case (topicAndPartition, data) => // we only print warnings for known errors here; if it is unknown, it will cause @@ -456,20 +452,42 @@ class KafkaApis(val requestChannel: RequestChannel, def handleJoinGroupRequest(request: RequestChannel.Request) { import JavaConversions._ - val joinGroupReq = request.requestObj.asInstanceOf[JoinGroupRequestAndHeader] - val topics = joinGroupReq.body.topics().toSet - val partitions = this.replicaManager.logManager.allLogs.filter(log => topics.contains(log.topicAndPartition.topic)) - val partitionList = partitions.map(_.topicAndPartition).map(tp => new org.apache.kafka.common.TopicPartition(tp.topic, tp.partition)).toBuffer - this.consumerGroupGenerationId += 1 - val response = new JoinGroupResponse(ErrorMapping.NoError, this.consumerGroupGenerationId, joinGroupReq.body.consumerId, partitionList) - val send = new BoundedByteBufferSend(new JoinGroupResponseAndHeader(joinGroupReq.correlationId, response)) - requestChannel.sendResponse(new RequestChannel.Response(request, send)) + + val joinGroupRequest = request.requestObj.asInstanceOf[JoinGroupRequestAndHeader] + + // the callback for sending a join-group response + def sendResponseCallback(partitions: List[TopicAndPartition], generationId: Int, errorCode: Short) { + val partitionList = partitions.map(tp => new TopicPartition(tp.topic, tp.partition)).toBuffer + val responseBody = new JoinGroupResponse(errorCode, generationId, joinGroupRequest.body.consumerId, partitionList) + val response = new JoinGroupResponseAndHeader(joinGroupRequest.correlationId, responseBody) + requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + } + + // let the coordinator to handle join-group + coordinator.consumerJoinGroup( + joinGroupRequest.body.groupId(), + joinGroupRequest.body.consumerId(), + joinGroupRequest.body.topics().toList, + joinGroupRequest.body.sessionTimeout(), + joinGroupRequest.body.strategy(), + sendResponseCallback) } def handleHeartbeatRequest(request: RequestChannel.Request) { - val hbReq = request.requestObj.asInstanceOf[HeartbeatRequestAndHeader] - val send = new BoundedByteBufferSend(new HeartbeatResponseAndHeader(hbReq.correlationId, new HeartbeatResponse(Errors.NONE.code))) - requestChannel.sendResponse(new RequestChannel.Response(request, send)) + val heartbeatRequest = request.requestObj.asInstanceOf[HeartbeatRequestAndHeader] + + // the callback for sending a heartbeat response + def sendResponseCallback(errorCode: Short) { + val response = new HeartbeatResponseAndHeader(heartbeatRequest.correlationId, new HeartbeatResponse(errorCode)) + requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + } + + // let the coordinator to handle heartbeat + coordinator.consumerHeartbeat( + heartbeatRequest.body.groupId(), + heartbeatRequest.body.consumerId(), + heartbeatRequest.body.groupGenerationId(), + sendResponseCallback) } def close() { diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 89200da30a049..7e5ddcb9be8fc 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -34,30 +34,43 @@ import kafka.common.{ErrorMapping, InconsistentBrokerIdException, GenerateBroker import kafka.network.{Receive, BlockingChannel, SocketServer} import kafka.metrics.KafkaMetricsGroup import com.yammer.metrics.core.Gauge +import kafka.coordinator.ConsumerCoordinator /** * Represents the lifecycle of a single Kafka broker. Handles all functionality required * to start up and shutdown a single Kafka node. */ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logging with KafkaMetricsGroup { - private var isShuttingDown = new AtomicBoolean(false) + private val startupComplete = new AtomicBoolean(false) + private val isShuttingDown = new AtomicBoolean(false) + private val isStartingUp = new AtomicBoolean(false) + private var shutdownLatch = new CountDownLatch(1) - private var startupComplete = new AtomicBoolean(false) - private var brokerId: Int = -1 val brokerState: BrokerState = new BrokerState - val correlationId: AtomicInteger = new AtomicInteger(0) + + var apis: KafkaApis = null var socketServer: SocketServer = null var requestHandlerPool: KafkaRequestHandlerPool = null + var logManager: LogManager = null + var offsetManager: OffsetManager = null - var kafkaHealthcheck: KafkaHealthcheck = null - var topicConfigManager: TopicConfigManager = null + var replicaManager: ReplicaManager = null - var apis: KafkaApis = null + + var topicConfigManager: TopicConfigManager = null + + var consumerCoordinator: ConsumerCoordinator = null + var kafkaController: KafkaController = null + val kafkaScheduler = new KafkaScheduler(config.backgroundThreads) + + var kafkaHealthcheck: KafkaHealthcheck = null + var zkClient: ZkClient = null + val correlationId: AtomicInteger = new AtomicInteger(0) val brokerMetaPropsFile = "meta.properties" val brokerMetadataCheckpoints = config.logDirs.map(logDir => (logDir, new BrokerMetadataCheckpoint(new File(logDir + File.separator +brokerMetaPropsFile)))).toMap @@ -75,69 +88,87 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg def startup() { try { info("starting") - brokerState.newState(Starting) - isShuttingDown = new AtomicBoolean(false) - shutdownLatch = new CountDownLatch(1) - /* start scheduler */ - kafkaScheduler.startup() + if(isShuttingDown.get) + throw new IllegalStateException("Kafka server is still shutting down, cannot re-start!") + + if(startupComplete.get) + return + + val canStartup = isStartingUp.compareAndSet(false, true) + if (canStartup) { + brokerState.newState(Starting) - /* setup zookeeper */ - zkClient = initZk() + /* start scheduler */ + kafkaScheduler.startup() - /* start log manager */ - logManager = createLogManager(zkClient, brokerState) - logManager.startup() + /* setup zookeeper */ + zkClient = initZk() - /* generate brokerId */ - config.brokerId = getBrokerId - this.logIdent = "[Kafka Server " + config.brokerId + "], " + /* start log manager */ + logManager = createLogManager(zkClient, brokerState) + logManager.startup() - socketServer = new SocketServer(config.brokerId, - config.hostName, - config.port, - config.numNetworkThreads, - config.queuedMaxRequests, - config.socketSendBufferBytes, - config.socketReceiveBufferBytes, - config.socketRequestMaxBytes, - config.maxConnectionsPerIp, - config.connectionsMaxIdleMs, - config.maxConnectionsPerIpOverrides) - socketServer.startup() + /* generate brokerId */ + config.brokerId = getBrokerId + this.logIdent = "[Kafka Server " + config.brokerId + "], " - replicaManager = new ReplicaManager(config, time, zkClient, kafkaScheduler, logManager, isShuttingDown) + socketServer = new SocketServer(config.brokerId, + config.hostName, + config.port, + config.numNetworkThreads, + config.queuedMaxRequests, + config.socketSendBufferBytes, + config.socketReceiveBufferBytes, + config.socketRequestMaxBytes, + config.maxConnectionsPerIp, + config.connectionsMaxIdleMs, + config.maxConnectionsPerIpOverrides) + socketServer.startup() - /* start offset manager */ - offsetManager = createOffsetManager() + /* start replica manager */ + replicaManager = new ReplicaManager(config, time, zkClient, kafkaScheduler, logManager, isShuttingDown) + replicaManager.startup() - kafkaController = new KafkaController(config, zkClient, brokerState) + /* start offset manager */ + offsetManager = createOffsetManager() - /* start processing requests */ - apis = new KafkaApis(socketServer.requestChannel, replicaManager, offsetManager, zkClient, config.brokerId, config, kafkaController) - requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) - brokerState.newState(RunningAsBroker) + /* start kafka controller */ + kafkaController = new KafkaController(config, zkClient, brokerState) + kafkaController.startup() - Mx4jLoader.maybeLoad() + /* start kafka coordinator */ + consumerCoordinator = new ConsumerCoordinator(config, zkClient) + consumerCoordinator.startup() - replicaManager.startup() + /* start processing requests */ + apis = new KafkaApis(socketServer.requestChannel, replicaManager, offsetManager, consumerCoordinator, kafkaController, zkClient, config.brokerId, config) + requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) + brokerState.newState(RunningAsBroker) - kafkaController.startup() + Mx4jLoader.maybeLoad() - topicConfigManager = new TopicConfigManager(zkClient, logManager) - topicConfigManager.startup() + /* start topic config manager */ + topicConfigManager = new TopicConfigManager(zkClient, logManager) + topicConfigManager.startup() - /* tell everyone we are alive */ - kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, config.advertisedHostName, config.advertisedPort, config.zkSessionTimeoutMs, zkClient) - kafkaHealthcheck.startup() + /* tell everyone we are alive */ + kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, config.advertisedHostName, config.advertisedPort, config.zkSessionTimeoutMs, zkClient) + kafkaHealthcheck.startup() - registerStats() - startupComplete.set(true) - info("started") + /* register broker metrics */ + registerStats() + + shutdownLatch = new CountDownLatch(1) + startupComplete.set(true) + isStartingUp.set(false) + info("started") + } } catch { case e: Throwable => fatal("Fatal error during KafkaServer startup. Prepare to shutdown", e) + isStartingUp.set(false) shutdown() throw e } @@ -271,6 +302,10 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg def shutdown() { try { info("shutting down") + + if(isStartingUp.get) + throw new IllegalStateException("Kafka server is still starting up, cannot shut down!") + val canShutdown = isShuttingDown.compareAndSet(false, true) if (canShutdown) { Utils.swallow(controlledShutdown()) @@ -290,20 +325,25 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg Utils.swallow(replicaManager.shutdown()) if(logManager != null) Utils.swallow(logManager.shutdown()) + if(consumerCoordinator != null) + Utils.swallow(consumerCoordinator.shutdown()) if(kafkaController != null) Utils.swallow(kafkaController.shutdown()) if(zkClient != null) Utils.swallow(zkClient.close()) brokerState.newState(NotRunning) - shutdownLatch.countDown() + startupComplete.set(false) + isShuttingDown.set(false) + shutdownLatch.countDown() info("shut down completed") } } catch { case e: Throwable => fatal("Fatal error during KafkaServer shutdown.", e) + isShuttingDown.set(false) throw e } } diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index bf81a1ab88c14..4c70aa7e0157b 100644 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -17,25 +17,28 @@ package kafka.server -import scala.collection.{Seq, Set, mutable} import kafka.api._ +import kafka.common._ import kafka.cluster.Broker -import java.util.concurrent.locks.ReentrantReadWriteLock -import kafka.utils.Utils._ -import kafka.common.{ErrorMapping, ReplicaNotAvailableException, LeaderNotAvailableException} -import kafka.common.TopicAndPartition import kafka.controller.KafkaController.StateChangeLogger +import scala.collection.{Seq, Set, mutable} +import kafka.utils.Logging +import kafka.utils.Utils._ + +import java.util.concurrent.locks.ReentrantReadWriteLock /** * A cache for the state (e.g., current leader) of each partition. This cache is updated through * UpdateMetadataRequest from the controller. Every broker maintains the same cache, asynchronously. */ -private[server] class MetadataCache { +private[server] class MetadataCache(brokerId: Int) extends Logging { private val cache: mutable.Map[String, mutable.Map[Int, PartitionStateInfo]] = new mutable.HashMap[String, mutable.Map[Int, PartitionStateInfo]]() private var aliveBrokers: Map[Int, Broker] = Map() private val partitionMetadataLock = new ReentrantReadWriteLock() + this.logIdent = "[Kafka Metadata Cache on broker %d] ".format(brokerId) + def getTopicMetadata(topics: Set[String]) = { val isAllTopics = topics.isEmpty val topicsRequested = if(isAllTopics) cache.keySet else topics @@ -68,7 +71,7 @@ private[server] class MetadataCache { new PartitionMetadata(partitionId, leaderInfo, replicaInfo, isrInfo, ErrorMapping.NoError) } catch { case e: Throwable => - debug("Error while fetching metadata for %s. Possible cause: %s".format(topicPartition, e.getMessage)) + debug("Error while fetching metadata for %s: %s".format(topicPartition, e.toString)) new PartitionMetadata(partitionId, leaderInfo, replicaInfo, isrInfo, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) } diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 0bdd42fea931c..83d52643028c5 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -96,6 +96,8 @@ class OffsetManager(val config: OffsetManagerConfig, private val shuttingDown = new AtomicBoolean(false) + this.logIdent = "[Offset Manager on Broker " + replicaManager.config.brokerId + "]: " + scheduler.schedule(name = "offsets-cache-compactor", fun = compact, period = config.offsetsRetentionCheckIntervalMs, diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index 81ae205ef7b20..5374280dc97dc 100644 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -442,7 +442,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { val producer: MirrorMakerBaseProducer, val threadId: Int) extends Thread with Logging with KafkaMetricsGroup { private val threadName = "mirrormaker-producer-" + threadId - private val shutdownComplete: CountDownLatch = new CountDownLatch(1) + private val shutdownLatch: CountDownLatch = new CountDownLatch(1) this.logIdent = "[%s] ".format(threadName) setName(threadName) @@ -466,7 +466,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { case t: Throwable => fatal("Producer thread failure due to ", t) } finally { - shutdownComplete.countDown() + shutdownLatch.countDown() info("Producer thread stopped") // if it exits accidentally, stop the entire mirror maker if (!isShuttingdown.get()) { @@ -490,7 +490,7 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { def awaitShutdown() { try { - shutdownComplete.await() + shutdownLatch.await() producer.close() info("Producer thread shutdown complete") } catch { diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index 90c0b7a19c7af..11d6a9751a831 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -272,7 +272,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { for (server <- servers) { server.shutdown() server.awaitShutdown() - server.startup + server.startup() Thread.sleep(2000) } diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index ba1e48e4300c9..82fa4cff45043 100644 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -119,7 +119,7 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { val newProps = TestUtils.createBrokerConfig(0, port) newProps.setProperty("zookeeper.connect", "fakehostthatwontresolve:65535") val newConfig = new KafkaConfig(newProps) - var server = new KafkaServer(newConfig) + val server = new KafkaServer(newConfig) try { server.startup() fail("Expected KafkaServer setup to fail, throw exception") @@ -129,14 +129,15 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { // identify the correct exception, making sure the server was shutdown, and cleaning up if anything // goes wrong so that awaitShutdown doesn't hang case e: org.I0Itec.zkclient.exception.ZkException => - assertEquals(server.brokerState.currentState, NotRunning.state) - if (server.brokerState.currentState != NotRunning.state) - server.shutdown() + assertEquals(NotRunning.state, server.brokerState.currentState) case e: Throwable => - fail("Expected KafkaServer setup to fail with connection exception but caught a different exception.") + fail("Expected ZkException during Kafka server starting up but caught a different exception %s".format(e.toString)) + } + finally { + if (server.brokerState.currentState != NotRunning.state) server.shutdown() + server.awaitShutdown() } - server.awaitShutdown() Utils.rm(server.config.logDirs) verifyNonDaemonThreadsStatus } diff --git a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala index 8fe7cd496f74a..764655a6883f9 100644 --- a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala @@ -48,19 +48,4 @@ class ServerStartupTest extends JUnit3Suite with ZooKeeperTestHarness { val pathExists = ZkUtils.pathExists(zkClient, zookeeperChroot) assertTrue(pathExists) } - - def testServerStartupConsecutively() { - server.shutdown() - try { - intercept[IllegalStateException]{ - server.startup() - server.startup() - } - } - finally { - server.shutdown() - } - - } - } \ No newline at end of file From 71602de0bbf7727f498a812033027f6cbfe34eb8 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 9 Feb 2015 15:24:44 -0800 Subject: [PATCH 118/491] KAFKA-1333 follow-up; Add missing files for the coordinator folder --- .../coordinator/ConsumerCoordinator.scala | 333 ++++++++++++++++++ .../kafka/coordinator/ConsumerRegistry.scala | 52 +++ .../kafka/coordinator/DelayedHeartbeat.scala | 44 +++ .../kafka/coordinator/DelayedJoinGroup.scala | 44 +++ .../kafka/coordinator/DelayedRebalance.scala | 62 ++++ .../kafka/coordinator/GroupRegistry.scala | 74 ++++ .../kafka/coordinator/HeartbeatBucket.scala | 36 ++ 7 files changed, 645 insertions(+) create mode 100644 core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala create mode 100644 core/src/main/scala/kafka/coordinator/ConsumerRegistry.scala create mode 100644 core/src/main/scala/kafka/coordinator/DelayedHeartbeat.scala create mode 100644 core/src/main/scala/kafka/coordinator/DelayedJoinGroup.scala create mode 100644 core/src/main/scala/kafka/coordinator/DelayedRebalance.scala create mode 100644 core/src/main/scala/kafka/coordinator/GroupRegistry.scala create mode 100644 core/src/main/scala/kafka/coordinator/HeartbeatBucket.scala diff --git a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala new file mode 100644 index 0000000000000..01cf1d91b7056 --- /dev/null +++ b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala @@ -0,0 +1,333 @@ +/** + * 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.coordinator + +import org.apache.kafka.common.protocol.Errors + +import kafka.common.TopicAndPartition +import kafka.server._ +import kafka.utils._ + +import scala.collection.mutable.HashMap + +import org.I0Itec.zkclient.{IZkChildListener, ZkClient} + + +/** + * Kafka coordinator handles consumer group and consumer offset management. + * + * Each Kafka server instantiates a coordinator, which is responsible for a set of + * consumer groups; the consumer groups are assigned to coordinators based on their + * group names. + */ +class ConsumerCoordinator(val config: KafkaConfig, + val zkClient: ZkClient) extends Logging { + + this.logIdent = "[Kafka Coordinator " + config.brokerId + "]: " + + /* zookeeper listener for topic-partition changes */ + private val topicPartitionChangeListeners = new HashMap[String, TopicPartitionChangeListener] + + /* the consumer group registry cache */ + // TODO: access to this map needs to be synchronized + private val consumerGroupRegistries = new HashMap[String, GroupRegistry] + + /* the list of subscribed groups per topic */ + // TODO: access to this map needs to be synchronized + private val consumerGroupsPerTopic = new HashMap[String, List[String]] + + /* the delayed operation purgatory for heartbeat-based failure detection */ + private var heartbeatPurgatory: DelayedOperationPurgatory[DelayedHeartbeat] = null + + /* the delayed operation purgatory for handling join-group requests */ + private var joinGroupPurgatory: DelayedOperationPurgatory[DelayedJoinGroup] = null + + /* the delayed operation purgatory for preparing rebalance process */ + private var rebalancePurgatory: DelayedOperationPurgatory[DelayedRebalance] = null + + /* latest consumer heartbeat bucket's end timestamp in milliseconds */ + private var latestHeartbeatBucketEndMs: Long = SystemTime.milliseconds + + /** + * Start-up logic executed at the same time when the server starts up. + */ + def startup() { + + // Initialize consumer group registries and heartbeat bucket metadata + latestHeartbeatBucketEndMs = SystemTime.milliseconds + + // Initialize purgatories for delayed heartbeat, join-group and rebalance operations + heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat](config.brokerId) + joinGroupPurgatory = new DelayedOperationPurgatory[DelayedJoinGroup](config.brokerId) + rebalancePurgatory = new DelayedOperationPurgatory[DelayedRebalance](config.brokerId) + + } + + /** + * Shut-down logic executed at the same time when server shuts down, + * ordering of actions should be reversed from the start-up process + * + */ + def shutdown() { + + // De-register all Zookeeper listeners for topic-partition changes + for (topic <- topicPartitionChangeListeners.keys) { + deregisterTopicChangeListener(topic) + } + topicPartitionChangeListeners.clear() + + // Shutdown purgatories for delayed heartbeat, join-group and rebalance operations + heartbeatPurgatory.shutdown() + joinGroupPurgatory.shutdown() + rebalancePurgatory.shutdown() + + // Clean up consumer group registries metadata + consumerGroupRegistries.clear() + consumerGroupsPerTopic.clear() + } + + /** + * Process a join-group request from a consumer to join as a new group member + */ + def consumerJoinGroup(groupId: String, + consumerId: String, + topics: List[String], + sessionTimeoutMs: Int, + partitionAssignmentStrategy: String, + responseCallback:(List[TopicAndPartition], Int, Short) => Unit ) { + + // if the group does not exist yet, create one + if (!consumerGroupRegistries.contains(groupId)) + createNewGroup(groupId, partitionAssignmentStrategy) + + // if the consumer id is unknown or it does exists in + // the group yet, register this consumer to the group + // TODO + + // add a delayed join-group operation to the purgatory + // TODO + + // if the current group is under rebalance process, + // check if the delayed rebalance operation can be finished + // TODO + + // TODO -------------------------------------------------------------- + // TODO: this is just a stub for new consumer testing, + // TODO: needs to be replaced with the logic above + // TODO -------------------------------------------------------------- + // just return all the partitions of the subscribed topics + val partitionIdsPerTopic = ZkUtils.getPartitionsForTopics(zkClient, topics) + val partitions = partitionIdsPerTopic.flatMap{ case (topic, partitionIds) => + partitionIds.map(partition => { + TopicAndPartition(topic, partition) + }) + }.toList + + responseCallback(partitions, 1 /* generation id */, Errors.NONE.code) + + info("Handled join-group from consumer " + consumerId + " to group " + groupId) + } + + /** + * Process a heartbeat request from a consumer + */ + def consumerHeartbeat(groupId: String, + consumerId: String, + generationId: Int, + responseCallback: Short => Unit) { + + // check that the group already exists + // TODO + + // check that the consumer has already registered for the group + // TODO + + // check if the consumer generation id is correct + // TODO + + // remove the consumer from its current heartbeat bucket, and add it back to the corresponding bucket + // TODO + + // create the heartbeat response, if partition rebalance is triggered set the corresponding error code + // TODO + + info("Handled heartbeat of consumer " + consumerId + " from group " + groupId) + + // TODO -------------------------------------------------------------- + // TODO: this is just a stub for new consumer testing, + // TODO: needs to be replaced with the logic above + // TODO -------------------------------------------------------------- + // always return OK for heartbeat immediately + responseCallback(Errors.NONE.code) + } + + /** + * Create a new consumer + */ + private def createNewConsumer(consumerId: String, + topics: List[String], + sessionTimeoutMs: Int, + groupRegistry: GroupRegistry) { + debug("Registering consumer " + consumerId + " for group " + groupRegistry.groupId) + + // create the new consumer registry entry + // TODO: specify consumerId as unknown and update at the end of the prepare-rebalance phase + + // check if the partition assignment strategy is consistent with the group + // TODO + + // add the group to the subscribed topics + // TODO + + // schedule heartbeat tasks for the consumer + // TODO + + // add the member registry entry to the group + // TODO + + // start preparing group partition rebalance + // TODO + + info("Registered consumer " + consumerId + " for group " + groupRegistry.groupId) + } + + /** + * Create a new consumer group in the registry + */ + private def createNewGroup(groupId: String, partitionAssignmentStrategy: String) { + debug("Creating new group " + groupId) + + val groupRegistry = new GroupRegistry(groupId, partitionAssignmentStrategy) + + consumerGroupRegistries.put(groupId, groupRegistry) + + info("Created new group registry " + groupId) + } + + /** + * Callback invoked when a consumer's heartbeat has expired + */ + private def onConsumerHeartbeatExpired(groupId: String, consumerId: String) { + + // if the consumer does not exist in group registry anymore, do nothing + // TODO + + // record heartbeat failure + // TODO + + // if the maximum failures has been reached, mark consumer as failed + // TODO + } + + /** + * Callback invoked when a consumer is marked as failed + */ + private def onConsumerFailure(groupId: String, consumerId: String) { + + // remove the consumer from its group registry metadata + // TODO + + // cut the socket connection to the consumer + // TODO: howto ?? + + // if the group has no consumer members any more, remove the group + // otherwise start preparing group partition rebalance + // TODO + + } + + /** + * Prepare partition rebalance for the group + */ + private def prepareRebalance(groupId: String) { + + // try to change the group state to PrepareRebalance + + // add a task to the delayed rebalance purgatory + + // TODO + } + + /** + * Start partition rebalance for the group + */ + private def startRebalance(groupId: String) { + + // try to change the group state to UnderRebalance + + // compute new assignment based on the strategy + + // send back the join-group response + + // TODO + } + + /** + * Fail current partition rebalance for the group + */ + + /** + * Register ZK listeners for topic-partition changes + */ + private def registerTopicChangeListener(topic: String) = { + if (!topicPartitionChangeListeners.contains(topic)) { + val listener = new TopicPartitionChangeListener(config) + topicPartitionChangeListeners.put(topic, listener) + ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.getTopicPath(topic)) + zkClient.subscribeChildChanges(ZkUtils.getTopicPath(topic), listener) + } + } + + /** + * De-register ZK listeners for topic-partition changes + */ + private def deregisterTopicChangeListener(topic: String) = { + val listener = topicPartitionChangeListeners.get(topic).get + zkClient.unsubscribeChildChanges(ZkUtils.getTopicPath(topic), listener) + topicPartitionChangeListeners.remove(topic) + } + + /** + * Zookeeper listener that catch topic-partition changes + */ + class TopicPartitionChangeListener(val config: KafkaConfig) extends IZkChildListener with Logging { + + this.logIdent = "[TopicChangeListener on coordinator " + config.brokerId + "]: " + + /** + * Try to trigger a rebalance for each group subscribed in the changed topic + * + * @throws Exception + * On any error. + */ + def handleChildChange(parentPath: String , curChilds: java.util.List[String]) { + debug("Fired for path %s with children %s".format(parentPath, curChilds)) + + // get the topic + val topic = parentPath.split("/").last + + // get groups that subscribed to this topic + val groups = consumerGroupsPerTopic.get(topic).get + + for (groupId <- groups) { + prepareRebalance(groupId) + } + } + } +} + + diff --git a/core/src/main/scala/kafka/coordinator/ConsumerRegistry.scala b/core/src/main/scala/kafka/coordinator/ConsumerRegistry.scala new file mode 100644 index 0000000000000..b65c04d0a5d53 --- /dev/null +++ b/core/src/main/scala/kafka/coordinator/ConsumerRegistry.scala @@ -0,0 +1,52 @@ +/** + * 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.coordinator + +import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} +import java.util.HashMap + +/** + * Consumer registry metadata contains the following metadata: + * + * Heartbeat metadata: + * 1. negotiated heartbeat session timeout. + * 2. recorded number of timed-out heartbeats. + * 3. associated heartbeat bucket in the purgatory. + * + * Subscription metadata: + * 1. subscribed topic list + * 2. assigned partitions for the subscribed topics. + */ +class ConsumerRegistry(val consumerId: String, + val subscribedTopics: List[String], + val sessionTimeoutMs: Int, + val groupRegistry: GroupRegistry) { + + /* number of expired heartbeat recorded */ + val numExpiredHeartbeat = new AtomicInteger(0) + + /* flag indicating if join group request is received */ + val joinGroupReceived = new AtomicBoolean(false) + + /* assigned partitions per subscribed topic */ + val assignedPartitions = new HashMap[String, List[Int]] + + /* associated heartbeat bucket */ + var currentHeartbeatBucket = null + +} diff --git a/core/src/main/scala/kafka/coordinator/DelayedHeartbeat.scala b/core/src/main/scala/kafka/coordinator/DelayedHeartbeat.scala new file mode 100644 index 0000000000000..894d6edb4077c --- /dev/null +++ b/core/src/main/scala/kafka/coordinator/DelayedHeartbeat.scala @@ -0,0 +1,44 @@ +/** + * 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.coordinator + +import kafka.server.DelayedOperation + +/** + * Delayed heartbeat operations that are added to the purgatory for session-timeout checking + * + * These operations will always be expired. Once it has expired, all its + * currently contained consumers are marked as heartbeat timed out. + */ +class DelayedHeartbeat(sessionTimeout: Long, + bucket: HeartbeatBucket, + expireCallback: (String, String) => Unit) + extends DelayedOperation(sessionTimeout) { + + /* this function should never be called */ + override def tryComplete(): Boolean = { + + throw new IllegalStateException("Delayed heartbeat purgatory should never try to complete any bucket") + } + + /* mark all consumers within the heartbeat as heartbeat timed out */ + override def onComplete() { + for (registry <- bucket.consumerRegistryList) + expireCallback(registry.groupRegistry.groupId, registry.consumerId) + } +} diff --git a/core/src/main/scala/kafka/coordinator/DelayedJoinGroup.scala b/core/src/main/scala/kafka/coordinator/DelayedJoinGroup.scala new file mode 100644 index 0000000000000..445bfa1bf8840 --- /dev/null +++ b/core/src/main/scala/kafka/coordinator/DelayedJoinGroup.scala @@ -0,0 +1,44 @@ +/** + * 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.coordinator + +import kafka.server.DelayedOperation + +/** + * Delayed join-group operations that are kept in the purgatory before the partition assignment completed + * + * These operation should never expire; when the rebalance has completed, all consumer's + * join-group operations will be completed by sending back the response with the + * calculated partition assignment. + */ +class DelayedJoinGroup(sessionTimeout: Long, + consumerRegistry: ConsumerRegistry, + responseCallback: () => Unit) extends DelayedOperation(sessionTimeout) { + + /* always successfully complete the operation once called */ + override def tryComplete(): Boolean = { + forceComplete() + } + + /* always assume the partition is already assigned as this delayed operation should never time-out */ + override def onComplete() { + + // TODO + responseCallback + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/coordinator/DelayedRebalance.scala b/core/src/main/scala/kafka/coordinator/DelayedRebalance.scala new file mode 100644 index 0000000000000..b3b3749a21d35 --- /dev/null +++ b/core/src/main/scala/kafka/coordinator/DelayedRebalance.scala @@ -0,0 +1,62 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.coordinator + +import kafka.server.DelayedOperation +import java.util.concurrent.atomic.AtomicBoolean + + +/** + * Delayed rebalance operations that are added to the purgatory when group is preparing for rebalance + * + * Whenever a join-group request is received, check if all known consumers have requested + * to re-join the group; if yes, complete this operation to proceed rebalance. + * + * When the operation has expired, any known consumers that have not requested to re-join + * the group are marked as failed, and complete this operation to proceed rebalance with + * the rest of the group. + */ +class DelayedRebalance(sessionTimeout: Long, + groupRegistry: GroupRegistry, + rebalanceCallback: String => Unit, + failureCallback: (String, String) => Unit) + extends DelayedOperation(sessionTimeout) { + + val allConsumersJoinedGroup = new AtomicBoolean(false) + + /* check if all known consumers have requested to re-join group */ + override def tryComplete(): Boolean = { + allConsumersJoinedGroup.set(groupRegistry.memberRegistries.values.foldLeft + (true) ((agg, cur) => agg && cur.joinGroupReceived.get())) + + if (allConsumersJoinedGroup.get()) + forceComplete() + else + false + } + + /* mark consumers that have not re-joined group as failed and proceed to rebalance the rest of the group */ + override def onComplete() { + groupRegistry.memberRegistries.values.foreach(consumerRegistry => + if (!consumerRegistry.joinGroupReceived.get()) + failureCallback(groupRegistry.groupId, consumerRegistry.consumerId) + ) + + rebalanceCallback(groupRegistry.groupId) + } +} diff --git a/core/src/main/scala/kafka/coordinator/GroupRegistry.scala b/core/src/main/scala/kafka/coordinator/GroupRegistry.scala new file mode 100644 index 0000000000000..7d17e10223513 --- /dev/null +++ b/core/src/main/scala/kafka/coordinator/GroupRegistry.scala @@ -0,0 +1,74 @@ +/** + * 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.coordinator + +import scala.collection.mutable + +sealed trait GroupStates { def state: Byte } + +/** + * Consumer group is preparing start rebalance + * + * action: respond consumer heartbeat with error code, + * transition: all known consumers has re-joined group => UnderRebalance + */ +case object PrepareRebalance extends GroupStates { val state: Byte = 1 } + +/** + * Consumer group is under rebalance + * + * action: send the join-group response with new assignment + * transition: all consumers has heartbeat with the new generation id => Fetching + * new consumer join-group received => PrepareRebalance + */ +case object UnderRebalance extends GroupStates { val state: Byte = 2 } + +/** + * Consumer group is fetching data + * + * action: respond consumer heartbeat normally + * transition: consumer failure detected via heartbeat => PrepareRebalance + * consumer join-group received => PrepareRebalance + * zookeeper watcher fired => PrepareRebalance + */ +case object Fetching extends GroupStates { val state: Byte = 3 } + +case class GroupState() { + @volatile var currentState: Byte = PrepareRebalance.state +} + +/* Group registry contains the following metadata of a registered group in the coordinator: + * + * Membership metadata: + * 1. List of consumers registered in this group + * 2. Partition assignment strategy for this group + * + * State metadata: + * 1. Current group state + * 2. Current group generation id + */ +class GroupRegistry(val groupId: String, + val partitionAssignmentStrategy: String) { + + val memberRegistries = new mutable.HashMap[String, ConsumerRegistry]() + + val state: GroupState = new GroupState() + + var generationId: Int = 1 +} + diff --git a/core/src/main/scala/kafka/coordinator/HeartbeatBucket.scala b/core/src/main/scala/kafka/coordinator/HeartbeatBucket.scala new file mode 100644 index 0000000000000..821e26e97eaa9 --- /dev/null +++ b/core/src/main/scala/kafka/coordinator/HeartbeatBucket.scala @@ -0,0 +1,36 @@ +/** + * 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.coordinator + +import scala.collection.mutable + +/** + * A bucket of consumers that are scheduled for heartbeat expiration. + * + * The motivation behind this is to avoid expensive fine-grained per-consumer + * heartbeat expiration but use coarsen-grained methods that group consumers + * with similar deadline together. This will result in some consumers not + * being expired for heartbeats in time but is tolerable. + */ +class HeartbeatBucket(val startMs: Long, endMs: Long) { + + /* The list of consumers that are contained in this bucket */ + val consumerRegistryList = new mutable.HashSet[ConsumerRegistry] + + // TODO +} From 9fe9913e95e1d3e114c74620d8da40f804f71b18 Mon Sep 17 00:00:00 2001 From: Onur Karaman Date: Tue, 10 Feb 2015 11:58:54 -0800 Subject: [PATCH 119/491] KAFKA-1476 Added a ConsumerCommand tool that will replace other consumer related tools in the future; reviewed by Neha Narkhede --- .../main/scala/kafka/admin/AdminUtils.scala | 56 +++- .../kafka/admin/ConsumerGroupCommand.scala | 310 ++++++++++++++++++ core/src/main/scala/kafka/utils/ZkUtils.scala | 26 +- .../kafka/admin/DeleteConsumerGroupTest.scala | 212 ++++++++++++ .../unit/kafka/admin/DeleteTopicTest.scala | 52 +-- .../scala/unit/kafka/utils/TestUtils.scala | 33 +- 6 files changed, 641 insertions(+), 48 deletions(-) create mode 100644 core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala create mode 100644 core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 28b12c7b89a56..b700110f2d7f1 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -20,7 +20,7 @@ package kafka.admin import kafka.common._ import kafka.cluster.Broker import kafka.log.LogConfig -import kafka.utils.{Logging, ZkUtils, Json} +import kafka.utils._ import kafka.api.{TopicMetadata, PartitionMetadata} import java.util.Random @@ -164,6 +164,60 @@ object AdminUtils extends Logging { ZkUtils.createPersistentPath(zkClient, ZkUtils.getDeleteTopicPath(topic)) } + def isConsumerGroupActive(zkClient: ZkClient, group: String) = { + ZkUtils.getConsumersInGroup(zkClient, group).nonEmpty + } + + /** + * Delete the whole directory of the given consumer group if the group is inactive. + * + * @param zkClient Zookeeper client + * @param group Consumer group + * @return whether or not we deleted the consumer group information + */ + def deleteConsumerGroupInZK(zkClient: ZkClient, group: String) = { + if (!isConsumerGroupActive(zkClient, group)) { + val dir = new ZKGroupDirs(group) + ZkUtils.deletePathRecursive(zkClient, dir.consumerGroupDir) + true + } + else false + } + + /** + * Delete the given consumer group's information for the given topic in Zookeeper if the group is inactive. + * If the consumer group consumes no other topics, delete the whole consumer group directory. + * + * @param zkClient Zookeeper client + * @param group Consumer group + * @param topic Topic of the consumer group information we wish to delete + * @return whether or not we deleted the consumer group information for the given topic + */ + def deleteConsumerGroupInfoForTopicInZK(zkClient: ZkClient, group: String, topic: String) = { + val topics = ZkUtils.getTopicsByConsumerGroup(zkClient, group) + if (topics == Seq(topic)) { + deleteConsumerGroupInZK(zkClient, group) + } + else if (!isConsumerGroupActive(zkClient, group)) { + val dir = new ZKGroupTopicDirs(group, topic) + ZkUtils.deletePathRecursive(zkClient, dir.consumerOwnerDir) + ZkUtils.deletePathRecursive(zkClient, dir.consumerOffsetDir) + true + } + else false + } + + /** + * Delete every inactive consumer group's information about the given topic in Zookeeper. + * + * @param zkClient Zookeeper client + * @param topic Topic of the consumer group information we wish to delete + */ + def deleteAllConsumerGroupInfoForTopicInZK(zkClient: ZkClient, topic: String) { + val groups = ZkUtils.getAllConsumerGroupsForTopic(zkClient, topic) + groups.foreach(group => deleteConsumerGroupInfoForTopicInZK(zkClient, group, topic)) + } + def topicExists(zkClient: ZkClient, topic: String): Boolean = zkClient.exists(ZkUtils.getTopicPath(topic)) diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala new file mode 100644 index 0000000000000..89fa29a882ae1 --- /dev/null +++ b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala @@ -0,0 +1,310 @@ +/** + * 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.admin + + +import kafka.utils._ +import org.I0Itec.zkclient.ZkClient +import kafka.common._ +import java.util.Properties +import kafka.client.ClientUtils +import kafka.api.{OffsetRequest, PartitionOffsetRequestInfo, OffsetFetchResponse, OffsetFetchRequest} +import org.I0Itec.zkclient.exception.ZkNoNodeException +import kafka.common.TopicAndPartition +import joptsimple.{OptionSpec, OptionParser} +import scala.collection.{Set, mutable} +import kafka.consumer.SimpleConsumer +import collection.JavaConversions._ + + +object ConsumerGroupCommand { + + def main(args: Array[String]) { + val opts = new ConsumerGroupCommandOptions(args) + + if(args.length == 0) + CommandLineUtils.printUsageAndDie(opts.parser, "List all consumer groups, describe a consumer group, or delete consumer group info.") + + // should have exactly one action + val actions = Seq(opts.listOpt, opts.describeOpt, opts.deleteOpt).count(opts.options.has _) + if(actions != 1) + CommandLineUtils.printUsageAndDie(opts.parser, "Command must include exactly one action: --list, --describe, --delete") + + opts.checkArgs() + + val zkClient = new ZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000, ZKStringSerializer) + + try { + if (opts.options.has(opts.listOpt)) + list(zkClient) + else if (opts.options.has(opts.describeOpt)) + describe(zkClient, opts) + else if (opts.options.has(opts.deleteOpt)) + delete(zkClient, opts) + } catch { + case e: Throwable => + println("Error while executing consumer group command " + e.getMessage) + println(Utils.stackTrace(e)) + } finally { + zkClient.close() + } + } + + def list(zkClient: ZkClient) { + ZkUtils.getConsumerGroups(zkClient).foreach(println) + } + + def describe(zkClient: ZkClient, opts: ConsumerGroupCommandOptions) { + val configs = parseConfigs(opts) + val channelSocketTimeoutMs = configs.getProperty("channelSocketTimeoutMs", "600").toInt + val channelRetryBackoffMs = configs.getProperty("channelRetryBackoffMsOpt", "300").toInt + val group = opts.options.valueOf(opts.groupOpt) + val topics = ZkUtils.getTopicsByConsumerGroup(zkClient, group) + if (topics.isEmpty) { + println("No topic available for consumer group provided") + } + topics.foreach(topic => describeTopic(zkClient, group, topic, channelSocketTimeoutMs, channelRetryBackoffMs)) + } + + def delete(zkClient: ZkClient, opts: ConsumerGroupCommandOptions) { + if (opts.options.has(opts.groupOpt) && opts.options.has(opts.topicOpt)) { + deleteForTopic(zkClient, opts) + } + else if (opts.options.has(opts.groupOpt)) { + deleteForGroup(zkClient, opts) + } + else if (opts.options.has(opts.topicOpt)) { + deleteAllForTopic(zkClient, opts) + } + } + + private def deleteForGroup(zkClient: ZkClient, opts: ConsumerGroupCommandOptions) { + val groups = opts.options.valuesOf(opts.groupOpt) + groups.foreach { group => + try { + if (AdminUtils.deleteConsumerGroupInZK(zkClient, group)) + println("Deleted all consumer group information for group %s in zookeeper.".format(group)) + else + println("Delete for group %s failed because its consumers are still active.".format(group)) + } + catch { + case e: ZkNoNodeException => + println("Delete for group %s failed because group does not exist.".format(group)) + } + } + } + + private def deleteForTopic(zkClient: ZkClient, opts: ConsumerGroupCommandOptions) { + val groups = opts.options.valuesOf(opts.groupOpt) + val topic = opts.options.valueOf(opts.topicOpt) + Topic.validate(topic) + groups.foreach { group => + try { + if (AdminUtils.deleteConsumerGroupInfoForTopicInZK(zkClient, group, topic)) + println("Deleted consumer group information for group %s topic %s in zookeeper.".format(group, topic)) + else + println("Delete for group %s topic %s failed because its consumers are still active.".format(group, topic)) + } + catch { + case e: ZkNoNodeException => + println("Delete for group %s topic %s failed because group does not exist.".format(group, topic)) + } + } + } + + private def deleteAllForTopic(zkClient: ZkClient, opts: ConsumerGroupCommandOptions) { + val topic = opts.options.valueOf(opts.topicOpt) + Topic.validate(topic) + AdminUtils.deleteAllConsumerGroupInfoForTopicInZK(zkClient, topic) + println("Deleted consumer group information for all inactive consumer groups for topic %s in zookeeper.".format(topic)) + } + + private def parseConfigs(opts: ConsumerGroupCommandOptions): Properties = { + val configsToBeAdded = opts.options.valuesOf(opts.configOpt).map(_.split("""\s*=\s*""")) + require(configsToBeAdded.forall(config => config.length == 2), + "Invalid config: all configs to be added must be in the format \"key=val\".") + val props = new Properties + configsToBeAdded.foreach(pair => props.setProperty(pair(0).trim, pair(1).trim)) + props + } + + private def describeTopic(zkClient: ZkClient, + group: String, + topic: String, + channelSocketTimeoutMs: Int, + channelRetryBackoffMs: Int) { + val topicPartitions = getTopicPartitions(zkClient, topic) + val partitionOffsets = getPartitionOffsets(zkClient, group, topicPartitions, channelSocketTimeoutMs, channelRetryBackoffMs) + println("%s, %s, %s, %s, %s, %s, %s" + .format("GROUP", "TOPIC", "PARTITION", "CURRENT OFFSET", "LOG END OFFSET", "LAG", "OWNER")) + topicPartitions + .sortBy { case topicPartition => topicPartition.partition } + .foreach { topicPartition => + describePartition(zkClient, group, topicPartition.topic, topicPartition.partition, partitionOffsets.get(topicPartition)) + } + } + + private def getTopicPartitions(zkClient: ZkClient, topic: String) = { + val topicPartitionMap = ZkUtils.getPartitionsForTopics(zkClient, Seq(topic)) + val partitions = topicPartitionMap.getOrElse(topic, Seq.empty) + partitions.map(TopicAndPartition(topic, _)) + } + + private def getPartitionOffsets(zkClient: ZkClient, + group: String, + topicPartitions: Seq[TopicAndPartition], + channelSocketTimeoutMs: Int, + channelRetryBackoffMs: Int): Map[TopicAndPartition, Long] = { + val offsetMap = mutable.Map[TopicAndPartition, Long]() + val channel = ClientUtils.channelToOffsetManager(group, zkClient, channelSocketTimeoutMs, channelRetryBackoffMs) + channel.send(OffsetFetchRequest(group, topicPartitions)) + val offsetFetchResponse = OffsetFetchResponse.readFrom(channel.receive().buffer) + + offsetFetchResponse.requestInfo.foreach { case (topicAndPartition, offsetAndMetadata) => + if (offsetAndMetadata == OffsetMetadataAndError.NoOffset) { + val topicDirs = new ZKGroupTopicDirs(group, topicAndPartition.topic) + // this group may not have migrated off zookeeper for offsets storage (we don't expose the dual-commit option in this tool + // (meaning the lag may be off until all the consumers in the group have the same setting for offsets storage) + try { + val offset = ZkUtils.readData(zkClient, topicDirs.consumerOffsetDir + "/" + topicAndPartition.partition)._1.toLong + offsetMap.put(topicAndPartition, offset) + } catch { + case z: ZkNoNodeException => + println("Could not fetch offset from zookeeper for group %s partition %s due to missing offset data in zookeeper." + .format(group, topicAndPartition)) + } + } + else if (offsetAndMetadata.error == ErrorMapping.NoError) + offsetMap.put(topicAndPartition, offsetAndMetadata.offset) + else + println("Could not fetch offset from kafka for group %s partition %s due to %s." + .format(group, topicAndPartition, ErrorMapping.exceptionFor(offsetAndMetadata.error))) + } + channel.disconnect() + offsetMap.toMap + } + + private def describePartition(zkClient: ZkClient, + group: String, + topic: String, + partition: Int, + offsetOpt: Option[Long]) { + val topicAndPartition = TopicAndPartition(topic, partition) + val groupDirs = new ZKGroupTopicDirs(group, topic) + val owner = ZkUtils.readDataMaybeNull(zkClient, groupDirs.consumerOwnerDir + "/" + partition)._1 + ZkUtils.getLeaderForPartition(zkClient, topic, partition) match { + case Some(-1) => + println("%s, %s, %s, %s, %s, %s, %s" + .format(group, topic, partition, offsetOpt.getOrElse("unknown"), "unknown", "unknown", owner.getOrElse("none"))) + case Some(brokerId) => + val consumerOpt = getConsumer(zkClient, brokerId) + consumerOpt match { + case Some(consumer) => + val request = + OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(OffsetRequest.LatestTime, 1))) + val logEndOffset = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets.head + consumer.close() + + val lag = offsetOpt.filter(_ != -1).map(logEndOffset - _) + println("%s, %s, %s, %s, %s, %s, %s" + .format(group, topic, partition, offsetOpt.getOrElse("unknown"), logEndOffset, lag.getOrElse("unknown"), owner.getOrElse("none"))) + case None => // ignore + } + case None => + println("No broker for partition %s".format(topicAndPartition)) + } + } + + private def getConsumer(zkClient: ZkClient, brokerId: Int): Option[SimpleConsumer] = { + try { + ZkUtils.readDataMaybeNull(zkClient, ZkUtils.BrokerIdsPath + "/" + brokerId)._1 match { + case Some(brokerInfoString) => + Json.parseFull(brokerInfoString) match { + case Some(m) => + val brokerInfo = m.asInstanceOf[Map[String, Any]] + val host = brokerInfo.get("host").get.asInstanceOf[String] + val port = brokerInfo.get("port").get.asInstanceOf[Int] + Some(new SimpleConsumer(host, port, 10000, 100000, "ConsumerGroupCommand")) + case None => + throw new BrokerNotAvailableException("Broker id %d does not exist".format(brokerId)) + } + case None => + throw new BrokerNotAvailableException("Broker id %d does not exist".format(brokerId)) + } + } catch { + case t: Throwable => + println("Could not parse broker info due to " + t.getMessage) + None + } + } + + class ConsumerGroupCommandOptions(args: Array[String]) { + val ZkConnectDoc = "REQUIRED: The connection string for the zookeeper connection in the form host:port. " + + "Multiple URLS can be given to allow fail-over." + val GroupDoc = "The consumer group we wish to act on." + val TopicDoc = "The topic whose consumer group information should be deleted." + val ConfigDoc = "Configuration for timeouts. For instance --config channelSocketTimeoutMs=600" + val ListDoc = "List all consumer groups." + val DescribeDoc = "Describe consumer group and list offset lag related to given group." + val nl = System.getProperty("line.separator") + val DeleteDoc = "Pass in groups to delete topic partition offsets and ownership information " + + "over the entire consumer group. For instance --group g1 --group g2" + nl + + "Pass in groups with a single topic to just delete the given topic's partition offsets and ownership " + + "information for the given consumer groups. For instance --group g1 --group g2 --topic t1" + nl + + "Pass in just a topic to delete the given topic's partition offsets and ownership information " + + "for every consumer group. For instance --topic t1" + nl + + "WARNING: Only does deletions on consumer groups that are not active." + val parser = new OptionParser + val zkConnectOpt = parser.accepts("zookeeper", ZkConnectDoc) + .withRequiredArg + .describedAs("urls") + .ofType(classOf[String]) + val groupOpt = parser.accepts("group", GroupDoc) + .withRequiredArg + .describedAs("consumer group") + .ofType(classOf[String]) + val topicOpt = parser.accepts("topic", TopicDoc) + .withRequiredArg + .describedAs("topic") + .ofType(classOf[String]) + val configOpt = parser.accepts("config", ConfigDoc) + .withRequiredArg + .describedAs("name=value") + .ofType(classOf[String]) + val listOpt = parser.accepts("list", ListDoc) + val describeOpt = parser.accepts("describe", DescribeDoc) + val deleteOpt = parser.accepts("delete", DeleteDoc) + val options = parser.parse(args : _*) + + val allConsumerGroupLevelOpts: Set[OptionSpec[_]] = Set(listOpt, describeOpt, deleteOpt) + + def checkArgs() { + // check required args + CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt) + if (options.has(describeOpt)) + CommandLineUtils.checkRequiredArgs(parser, options, groupOpt) + if (options.has(deleteOpt) && !options.has(groupOpt) && !options.has(topicOpt)) + CommandLineUtils.printUsageAndDie(parser, "Option %s either takes %s, %s, or both".format(deleteOpt, groupOpt, topicOpt)) + + // check invalid args + CommandLineUtils.checkInvalidArgs(parser, options, groupOpt, allConsumerGroupLevelOpts - describeOpt - deleteOpt) + CommandLineUtils.checkInvalidArgs(parser, options, topicOpt, allConsumerGroupLevelOpts - deleteOpt) + CommandLineUtils.checkInvalidArgs(parser, options, configOpt, allConsumerGroupLevelOpts - describeOpt) + } + } +} diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index c14bd455b6642..c78a1b6ff4213 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -749,6 +749,26 @@ object ZkUtils extends Logging { }.flatten.toSet } } + + def getConsumerGroups(zkClient: ZkClient) = { + ZkUtils.getChildren(zkClient, ConsumersPath) + } + + def getTopicsByConsumerGroup(zkClient: ZkClient,consumerGroup:String) = { + ZkUtils.getChildrenParentMayNotExist(zkClient, new ZKGroupDirs(consumerGroup).consumerGroupOwnersDir) + } + + def getAllConsumerGroupsForTopic(zkClient: ZkClient, topic: String): Set[String] = { + val groups = ZkUtils.getChildrenParentMayNotExist(zkClient, ConsumersPath) + if (groups == null) Set.empty + else { + groups.foldLeft(Set.empty[String]) {(consumerGroupsForTopic, group) => + val topics = getChildren(zkClient, new ZKGroupDirs(group).consumerGroupOffsetsDir) + if (topics.contains(topic)) consumerGroupsForTopic + group + else consumerGroupsForTopic + } + } + } } object ZKStringSerializer extends ZkSerializer { @@ -769,11 +789,13 @@ class ZKGroupDirs(val group: String) { def consumerDir = ZkUtils.ConsumersPath def consumerGroupDir = consumerDir + "/" + group def consumerRegistryDir = consumerGroupDir + "/ids" + def consumerGroupOffsetsDir = consumerGroupDir + "/offsets" + def consumerGroupOwnersDir = consumerGroupDir + "/owners" } class ZKGroupTopicDirs(group: String, topic: String) extends ZKGroupDirs(group) { - def consumerOffsetDir = consumerGroupDir + "/offsets/" + topic - def consumerOwnerDir = consumerGroupDir + "/owners/" + topic + def consumerOffsetDir = consumerGroupOffsetsDir + "/" + topic + def consumerOwnerDir = consumerGroupOwnersDir + "/" + topic } diff --git a/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala new file mode 100644 index 0000000000000..d530338728be4 --- /dev/null +++ b/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala @@ -0,0 +1,212 @@ +/** + * 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.admin + +import org.scalatest.junit.JUnit3Suite +import kafka.utils.{ZKGroupDirs, ZKGroupTopicDirs, ZkUtils, TestUtils} +import kafka.server.KafkaConfig +import org.junit.Test +import kafka.consumer._ +import org.apache.kafka.clients.producer.{ProducerRecord, KafkaProducer} +import kafka.integration.KafkaServerTestHarness + + +class DeleteConsumerGroupTest extends JUnit3Suite with KafkaServerTestHarness { + val configs = TestUtils.createBrokerConfigs(3, false, true).map(new KafkaConfig(_)) + + @Test + def testGroupWideDeleteInZK() { + val topic = "test" + val groupToDelete = "groupToDelete" + val otherGroup = "otherGroup" + + TestUtils.createTopic(zkClient, topic, 1, 3, servers) + fillInConsumerGroupInfo(topic, groupToDelete, "consumer", 0, 10, false) + fillInConsumerGroupInfo(topic, otherGroup, "consumer", 0, 10, false) + + AdminUtils.deleteConsumerGroupInZK(zkClient, groupToDelete) + + TestUtils.waitUntilTrue(() => !groupDirExists(new ZKGroupDirs(groupToDelete)), + "DeleteConsumerGroupInZK should delete the provided consumer group's directory") + TestUtils.waitUntilTrue(() => groupDirExists(new ZKGroupDirs(otherGroup)), + "DeleteConsumerGroupInZK should not delete unrelated consumer group directories") + } + + @Test + def testGroupWideDeleteInZKDoesNothingForActiveConsumerGroup() { + val topic = "test" + val groupToDelete = "groupToDelete" + val otherGroup = "otherGroup" + + TestUtils.createTopic(zkClient, topic, 1, 3, servers) + fillInConsumerGroupInfo(topic, groupToDelete, "consumer", 0, 10, true) + fillInConsumerGroupInfo(topic, otherGroup, "consumer", 0, 10, false) + + AdminUtils.deleteConsumerGroupInZK(zkClient, groupToDelete) + + TestUtils.waitUntilTrue(() => groupDirExists(new ZKGroupDirs(groupToDelete)), + "DeleteConsumerGroupInZK should not delete the provided consumer group's directory if the consumer group is still active") + TestUtils.waitUntilTrue(() => groupDirExists(new ZKGroupDirs(otherGroup)), + "DeleteConsumerGroupInZK should not delete unrelated consumer group directories") + } + + @Test + def testGroupTopicWideDeleteInZKForGroupConsumingOneTopic() { + val topic = "test" + val groupToDelete = "groupToDelete" + val otherGroup = "otherGroup" + TestUtils.createTopic(zkClient, topic, 1, 3, servers) + fillInConsumerGroupInfo(topic, groupToDelete, "consumer", 0, 10, false) + fillInConsumerGroupInfo(topic, otherGroup, "consumer", 0, 10, false) + + AdminUtils.deleteConsumerGroupInfoForTopicInZK(zkClient, groupToDelete, topic) + + TestUtils.waitUntilTrue(() => !groupDirExists(new ZKGroupDirs(groupToDelete)), + "DeleteConsumerGroupInfoForTopicInZK should delete the provided consumer group's directory if it just consumes from one topic") + TestUtils.waitUntilTrue(() => groupTopicOffsetAndOwnerDirsExist(new ZKGroupTopicDirs(otherGroup, topic)), + "DeleteConsumerGroupInfoForTopicInZK should not delete unrelated consumer group owner and offset directories") + } + + @Test + def testGroupTopicWideDeleteInZKForGroupConsumingMultipleTopics() { + val topicToDelete = "topicToDelete" + val otherTopic = "otherTopic" + val groupToDelete = "groupToDelete" + val otherGroup = "otherGroup" + TestUtils.createTopic(zkClient, topicToDelete, 1, 3, servers) + TestUtils.createTopic(zkClient, otherTopic, 1, 3, servers) + + fillInConsumerGroupInfo(topicToDelete, groupToDelete, "consumer", 0, 10, false) + fillInConsumerGroupInfo(otherTopic, groupToDelete, "consumer", 0, 10, false) + fillInConsumerGroupInfo(topicToDelete, otherGroup, "consumer", 0, 10, false) + + AdminUtils.deleteConsumerGroupInfoForTopicInZK(zkClient, groupToDelete, topicToDelete) + + TestUtils.waitUntilTrue(() => !groupTopicOffsetAndOwnerDirsExist(new ZKGroupTopicDirs(groupToDelete, topicToDelete)), + "DeleteConsumerGroupInfoForTopicInZK should delete the provided consumer group's owner and offset directories for the given topic") + TestUtils.waitUntilTrue(() => groupTopicOffsetAndOwnerDirsExist(new ZKGroupTopicDirs(groupToDelete, otherTopic)), + "DeleteConsumerGroupInfoForTopicInZK should not delete the provided consumer group's owner and offset directories for unrelated topics") + TestUtils.waitUntilTrue(() => groupTopicOffsetAndOwnerDirsExist(new ZKGroupTopicDirs(otherGroup, topicToDelete)), + "DeleteConsumerGroupInfoForTopicInZK should not delete unrelated consumer group owner and offset directories") + } + + @Test + def testGroupTopicWideDeleteInZKDoesNothingForActiveGroupConsumingMultipleTopics() { + val topicToDelete = "topicToDelete" + val otherTopic = "otherTopic" + val group = "group" + TestUtils.createTopic(zkClient, topicToDelete, 1, 3, servers) + TestUtils.createTopic(zkClient, otherTopic, 1, 3, servers) + + fillInConsumerGroupInfo(topicToDelete, group, "consumer", 0, 10, true) + fillInConsumerGroupInfo(otherTopic, group, "consumer", 0, 10, true) + + AdminUtils.deleteConsumerGroupInfoForTopicInZK(zkClient, group, topicToDelete) + + TestUtils.waitUntilTrue(() => groupTopicOffsetAndOwnerDirsExist(new ZKGroupTopicDirs(group, topicToDelete)), + "DeleteConsumerGroupInfoForTopicInZK should not delete the provided consumer group's owner and offset directories for the given topic if the consumer group is still active") + TestUtils.waitUntilTrue(() => groupTopicOffsetAndOwnerDirsExist(new ZKGroupTopicDirs(group, otherTopic)), + "DeleteConsumerGroupInfoForTopicInZK should not delete the provided consumer group's owner and offset directories for unrelated topics") + } + + @Test + def testTopicWideDeleteInZK() { + val topicToDelete = "topicToDelete" + val otherTopic = "otherTopic" + val groups = Seq("group1", "group2") + + TestUtils.createTopic(zkClient, topicToDelete, 1, 3, servers) + TestUtils.createTopic(zkClient, otherTopic, 1, 3, servers) + val groupTopicDirsForTopicToDelete = groups.map(group => new ZKGroupTopicDirs(group, topicToDelete)) + val groupTopicDirsForOtherTopic = groups.map(group => new ZKGroupTopicDirs(group, otherTopic)) + groupTopicDirsForTopicToDelete.foreach(dir => fillInConsumerGroupInfo(topicToDelete, dir.group, "consumer", 0, 10, false)) + groupTopicDirsForOtherTopic.foreach(dir => fillInConsumerGroupInfo(otherTopic, dir.group, "consumer", 0, 10, false)) + + AdminUtils.deleteAllConsumerGroupInfoForTopicInZK(zkClient, topicToDelete) + + TestUtils.waitUntilTrue(() => !groupTopicDirsForTopicToDelete.exists(groupTopicOffsetAndOwnerDirsExist), + "Consumer group info on deleted topic topic should be deleted by DeleteAllConsumerGroupInfoForTopicInZK") + TestUtils.waitUntilTrue(() => groupTopicDirsForOtherTopic.forall(groupTopicOffsetAndOwnerDirsExist), + "Consumer group info on unrelated topics should not be deleted by DeleteAllConsumerGroupInfoForTopicInZK") + } + + @Test + def testConsumptionOnRecreatedTopicAfterTopicWideDeleteInZK() { + val topic = "topic" + val group = "group" + + TestUtils.createTopic(zkClient, topic, 1, 3, servers) + val dir = new ZKGroupTopicDirs(group, topic) + fillInConsumerGroupInfo(topic, dir.group, "consumer", 0, 10, false) + + AdminUtils.deleteTopic(zkClient, topic) + TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) + AdminUtils.deleteAllConsumerGroupInfoForTopicInZK(zkClient, topic) + + TestUtils.waitUntilTrue(() => !groupDirExists(dir), + "Consumer group info on related topics should be deleted by DeleteAllConsumerGroupInfoForTopicInZK") + //produce events + val producer = TestUtils.createNewProducer(brokerList) + produceEvents(producer, topic, List.fill(10)("test")) + + //consume events + val consumerProps = TestUtils.createConsumerProperties(zkConnect, group, "consumer") + consumerProps.put("auto.commit.enable", "false") + consumerProps.put("auto.offset.reset", "smallest") + consumerProps.put("consumer.timeout.ms", "2000") + consumerProps.put("fetch.wait.max.ms", "0") + val consumerConfig = new ConsumerConfig(consumerProps) + val consumerConnector: ConsumerConnector = Consumer.create(consumerConfig) + val messageStream = consumerConnector.createMessageStreams(Map(topic -> 1))(topic).head + consumeEvents(messageStream, 5) + consumerConnector.commitOffsets(false) + producer.close() + consumerConnector.shutdown() + + TestUtils.waitUntilTrue(() => groupTopicOffsetAndOwnerDirsExist(dir), + "Consumer group info should exist after consuming from a recreated topic") + } + + private def fillInConsumerGroupInfo(topic: String, group: String, consumerId: String, partition: Int, offset: Int, registerConsumer: Boolean) { + val consumerProps = TestUtils.createConsumerProperties(zkConnect, group, consumerId) + val consumerConfig = new ConsumerConfig(consumerProps) + val dir = new ZKGroupTopicDirs(group, topic) + TestUtils.updateConsumerOffset(consumerConfig, dir.consumerOffsetDir + "/" + partition, offset) + ZkUtils.createEphemeralPathExpectConflict(zkClient, ZkUtils.getConsumerPartitionOwnerPath(group, topic, partition), "") + ZkUtils.makeSurePersistentPathExists(zkClient, dir.consumerRegistryDir) + if (registerConsumer) { + ZkUtils.createEphemeralPathExpectConflict(zkClient, dir.consumerRegistryDir + "/" + consumerId, "") + } + } + + private def groupDirExists(dir: ZKGroupDirs) = { + ZkUtils.pathExists(zkClient, dir.consumerGroupDir) + } + + private def groupTopicOffsetAndOwnerDirsExist(dir: ZKGroupTopicDirs) = { + ZkUtils.pathExists(zkClient, dir.consumerOffsetDir) && ZkUtils.pathExists(zkClient, dir.consumerOwnerDir) + } + + private def produceEvents(producer: KafkaProducer[Array[Byte], Array[Byte]], topic: String, messages: List[String]) { + messages.foreach(message => producer.send(new ProducerRecord(topic, message.getBytes))) + } + + private def consumeEvents(messageStream: KafkaStream[Array[Byte], Array[Byte]], n: Int) { + val iter = messageStream.iterator + (0 until n).foreach(_ => iter.next) + } +} diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala index 33c27678bf8ae..0cbd72684a0cc 100644 --- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala @@ -16,23 +16,15 @@ */ package kafka.admin -import java.io.File - import kafka.log.Log import org.scalatest.junit.JUnit3Suite import kafka.zk.ZooKeeperTestHarness import junit.framework.Assert._ import kafka.utils.{ZkUtils, TestUtils} -import kafka.server.{OffsetCheckpoint, KafkaServer, KafkaConfig} +import kafka.server.{KafkaServer, KafkaConfig} import org.junit.Test -import kafka.common._ -import kafka.producer.{ProducerConfig, Producer} import java.util.Properties -import kafka.api._ -import kafka.consumer.SimpleConsumer -import kafka.producer.KeyedMessage import kafka.common.TopicAndPartition -import kafka.api.PartitionOffsetRequestInfo class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { @@ -43,7 +35,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { val servers = createTestTopicAndCluster(topic) // start topic deletion AdminUtils.deleteTopic(zkClient, topic) - verifyTopicDeletion(topic, servers) + TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) servers.foreach(_.shutdown()) } @@ -68,7 +60,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { "Admin path /admin/delete_topic/test path deleted even when a follower replica is down") // restart follower replica follower.startup() - verifyTopicDeletion(topic, servers) + TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) servers.foreach(_.shutdown()) } @@ -95,7 +87,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { controller.startup() follower.startup() - verifyTopicDeletion(topic, servers) + TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) servers.foreach(_.shutdown()) } @@ -141,7 +133,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, 0) assertEquals("Partition should not be reassigned to 0, 1, 2", oldAssignedReplicas, assignedReplicas) follower.startup() - verifyTopicDeletion(topic, servers) + TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) allServers.foreach(_.shutdown()) } @@ -160,7 +152,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { AdminUtils.deleteTopic(zkClient, topic) follower.startup() // test if topic deletion is resumed - verifyTopicDeletion(topic, servers) + TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) // verify that new partition doesn't exist on any broker either TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res, server) => res && server.getLogManager().getLog(newPartition).isEmpty), @@ -178,7 +170,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { // add partitions to topic val newPartition = TopicAndPartition(topic, 1) AdminUtils.addPartitions(zkClient, topic, 2, "0:1:2,0:1:2") - verifyTopicDeletion(topic, servers) + TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) // verify that new partition doesn't exist on any broker either assertTrue("Replica logs not deleted after delete topic is complete", servers.foldLeft(true)((res, server) => res && server.getLogManager().getLog(newPartition).isEmpty)) @@ -193,7 +185,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { val servers = createTestTopicAndCluster(topic) // start topic deletion AdminUtils.deleteTopic(zkClient, topic) - verifyTopicDeletion(topic, servers) + TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) // re-create topic on same replicas AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment) // wait until leader is elected @@ -213,7 +205,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { // start topic deletion AdminUtils.deleteTopic(zkClient, "test2") // verify delete topic path for test2 is removed from zookeeper - verifyTopicDeletion("test2", servers) + TestUtils.verifyTopicDeletion(zkClient, "test2", 1, servers) // verify that topic test is untouched TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res, server) => res && server.getLogManager().getLog(topicAndPartition).isDefined), @@ -252,7 +244,7 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { // delete topic AdminUtils.deleteTopic(zkClient, "test") - verifyTopicDeletion("test", servers) + TestUtils.verifyTopicDeletion(zkClient, "test", 1, servers) servers.foreach(_.shutdown()) } @@ -279,30 +271,6 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { servers } - private def verifyTopicDeletion(topic: String, servers: Seq[KafkaServer]) { - val topicAndPartition = TopicAndPartition(topic, 0) - // wait until admin path for delete topic is deleted, signaling completion of topic deletion - TestUtils.waitUntilTrue(() => !ZkUtils.pathExists(zkClient, ZkUtils.getDeleteTopicPath(topic)), - "Admin path /admin/delete_topic/test path not deleted even after a replica is restarted") - TestUtils.waitUntilTrue(() => !ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic)), - "Topic path /brokers/topics/test not deleted after /admin/delete_topic/test path is deleted") - // ensure that the topic-partition has been deleted from all brokers' replica managers - TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res, server) => res && server.replicaManager.getPartition(topic, 0) == None), - "Replica manager's should have deleted all of this topic's partitions") - // ensure that logs from all replicas are deleted if delete topic is marked successful in zookeeper - assertTrue("Replica logs not deleted after delete topic is complete", - servers.foldLeft(true)((res, server) => res && server.getLogManager().getLog(topicAndPartition).isEmpty)) - // ensure that topic is removed from all cleaner offsets - TestUtils.waitUntilTrue(() => servers.foldLeft(true)((res,server) => res && - { - val topicAndPartition = TopicAndPartition(topic,0) - val logdir = server.getLogManager().logDirs(0) - val checkpoints = new OffsetCheckpoint(new File(logdir,"cleaner-offset-checkpoint")).read() - !checkpoints.contains(topicAndPartition) - }), - "Cleaner offset for deleted partition should have been removed") - } - private def writeDups(numKeys: Int, numDups: Int, log: Log): Seq[(Int, Int)] = { var counter = 0 for(dup <- 0 until numDups; key <- 0 until numKeys) yield { diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 54755e8dd3f23..21d0ed2cb7c94 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -140,9 +140,10 @@ object TestUtils extends Logging { * Create a test config for the given node id */ def createBrokerConfigs(numConfigs: Int, - enableControlledShutdown: Boolean = true): List[Properties] = { + enableControlledShutdown: Boolean = true, + enableDeleteTopic: Boolean = false): List[Properties] = { for((port, node) <- choosePorts(numConfigs).zipWithIndex) - yield createBrokerConfig(node, port, enableControlledShutdown) + yield createBrokerConfig(node, port, enableControlledShutdown, enableDeleteTopic) } def getBrokerListStrFromConfigs(configs: Seq[KafkaConfig]): String = { @@ -153,7 +154,8 @@ object TestUtils extends Logging { * Create a test config for the given node id */ def createBrokerConfig(nodeId: Int, port: Int = choosePort(), - enableControlledShutdown: Boolean = true): Properties = { + enableControlledShutdown: Boolean = true, + enableDeleteTopic: Boolean = false): Properties = { val props = new Properties if (nodeId >= 0) props.put("broker.id", nodeId.toString) props.put("host.name", "localhost") @@ -162,6 +164,7 @@ object TestUtils extends Logging { props.put("zookeeper.connect", TestZKUtils.zookeeperConnect) props.put("replica.socket.timeout.ms", "1500") props.put("controlled.shutdown.enable", enableControlledShutdown.toString) + props.put("delete.topic.enable", enableDeleteTopic.toString) props } @@ -793,6 +796,30 @@ object TestUtils extends Logging { } messages.reverse } + + def verifyTopicDeletion(zkClient: ZkClient, topic: String, numPartitions: Int, servers: Seq[KafkaServer]) { + val topicAndPartitions = (0 until numPartitions).map(TopicAndPartition(topic, _)) + // wait until admin path for delete topic is deleted, signaling completion of topic deletion + TestUtils.waitUntilTrue(() => !ZkUtils.pathExists(zkClient, ZkUtils.getDeleteTopicPath(topic)), + "Admin path /admin/delete_topic/test path not deleted even after a replica is restarted") + TestUtils.waitUntilTrue(() => !ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic)), + "Topic path /brokers/topics/test not deleted after /admin/delete_topic/test path is deleted") + // ensure that the topic-partition has been deleted from all brokers' replica managers + TestUtils.waitUntilTrue(() => + servers.forall(server => topicAndPartitions.forall(tp => server.replicaManager.getPartition(tp.topic, tp.partition) == None)), + "Replica manager's should have deleted all of this topic's partitions") + // ensure that logs from all replicas are deleted if delete topic is marked successful in zookeeper + assertTrue("Replica logs not deleted after delete topic is complete", + servers.forall(server => topicAndPartitions.forall(tp => server.getLogManager().getLog(tp).isEmpty))) + // ensure that topic is removed from all cleaner offsets + TestUtils.waitUntilTrue(() => servers.forall(server => topicAndPartitions.forall { tp => + val checkpoints = server.getLogManager().logDirs.map { logDir => + new OffsetCheckpoint(new File(logDir, "cleaner-offset-checkpoint")).read() + } + checkpoints.forall(checkpointsPerLogDir => !checkpointsPerLogDir.contains(tp)) + }), "Cleaner offset for deleted partition should have been removed") + } + } object TestZKUtils { From a3d6dcaf1bcc6189c3ef17b63719ae4d12dcb789 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 12 Feb 2015 17:09:42 -0800 Subject: [PATCH 120/491] KAFKA-1948; Fix ConsumerTest.testPartitionReassignmentCallback handling issue; reviewed by Gwen Shapira --- core/src/main/scala/kafka/controller/KafkaController.scala | 2 ++ core/src/test/scala/integration/kafka/api/ConsumerTest.scala | 5 ++++- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 66df6d2fbdbdd..e9b4dc62df3f1 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -365,6 +365,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt controllerContext.epoch=0 controllerContext.epochZkVersion=0 brokerState.newState(RunningAsBroker) + + info("Broker %d resigned as the controller".format(config.brokerId)) } } diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala index 798f035df52e4..2802a399bf599 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala @@ -230,7 +230,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging { assertEquals(1, parts.size) assertNotNull(parts(0).leader()) - // shutdown the co-ordinator + // shutdown the coordinator val coordinator = parts(0).leader().id() this.servers(coordinator).shutdown() @@ -239,6 +239,9 @@ class ConsumerTest extends IntegrationTestHarness with Logging { consumer0.poll(50) assertEquals(2, callback.callsToAssigned) assertEquals(2, callback.callsToRevoked) + + // restart the coordinator since it may also be hosting "test" topic + this.servers(coordinator).startup() consumer0.close() } From eab4f4c9f43e20b3d91bf0e2e9be6b6fd72f0acf Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Fri, 13 Feb 2015 13:08:10 -0800 Subject: [PATCH 121/491] KAFKA-1697; Remove support for producer ack > 1 on the broker; reviewed by Joel Koshy --- .../errors/InvalidRequiredAcksException.java | 25 ++++++ ...NotEnoughReplicasAfterAppendException.java | 12 --- .../apache/kafka/common/protocol/Errors.java | 19 +---- .../main/scala/kafka/cluster/Partition.scala | 2 - .../main/scala/kafka/server/KafkaApis.scala | 2 +- .../scala/kafka/server/ReplicaManager.scala | 85 ++++++++++++------- .../RequestResponseSerializationTest.scala | 2 +- .../kafka/server/ReplicaManagerTest.scala | 26 ++++++ 8 files changed, 111 insertions(+), 62 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/InvalidRequiredAcksException.java diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InvalidRequiredAcksException.java b/clients/src/main/java/org/apache/kafka/common/errors/InvalidRequiredAcksException.java new file mode 100644 index 0000000000000..9d19b2844aec7 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/InvalidRequiredAcksException.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.errors; + +public class InvalidRequiredAcksException extends ApiException { + private static final long serialVersionUID = 1L; + + public InvalidRequiredAcksException(String message) { + super(message); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java b/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java index a6107b818947d..fd7f6d8f5c69b 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/NotEnoughReplicasAfterAppendException.java @@ -23,20 +23,8 @@ public class NotEnoughReplicasAfterAppendException extends RetriableException { private static final long serialVersionUID = 1L; - public NotEnoughReplicasAfterAppendException() { - super(); - } - - public NotEnoughReplicasAfterAppendException(String message, Throwable cause) { - super(message, cause); - } - public NotEnoughReplicasAfterAppendException(String message) { super(message); } - public NotEnoughReplicasAfterAppendException(Throwable cause) { - super(cause); - } - } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index a8deac4ce5149..ad2171f5417c9 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -19,21 +19,7 @@ import java.util.HashMap; import java.util.Map; -import org.apache.kafka.common.errors.ApiException; -import org.apache.kafka.common.errors.CorruptRecordException; -import org.apache.kafka.common.errors.InvalidTopicException; -import org.apache.kafka.common.errors.LeaderNotAvailableException; -import org.apache.kafka.common.errors.NetworkException; -import org.apache.kafka.common.errors.NotEnoughReplicasAfterAppendException; -import org.apache.kafka.common.errors.NotEnoughReplicasException; -import org.apache.kafka.common.errors.NotLeaderForPartitionException; -import org.apache.kafka.common.errors.OffsetMetadataTooLarge; -import org.apache.kafka.common.errors.OffsetOutOfRangeException; -import org.apache.kafka.common.errors.RecordBatchTooLargeException; -import org.apache.kafka.common.errors.RecordTooLargeException; -import org.apache.kafka.common.errors.TimeoutException; -import org.apache.kafka.common.errors.UnknownServerException; -import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; +import org.apache.kafka.common.errors.*; /** * This class contains all the client-server errors--those errors that must be sent from the server to the client. These @@ -70,7 +56,8 @@ public enum Errors { NOT_ENOUGH_REPLICAS(19, new NotEnoughReplicasException("Messages are rejected since there are fewer in-sync replicas than required.")), NOT_ENOUGH_REPLICAS_AFTER_APPEND(20, - new NotEnoughReplicasAfterAppendException("Messages are written to the log, but to fewer in-sync replicas than required.")); + new NotEnoughReplicasAfterAppendException("Messages are written to the log, but to fewer in-sync replicas than required.")), + INVALID_REQUIRED_ACKS(21, new InvalidRequiredAcksException("Produce request specified an invalid value for required acks.")); private static Map, Errors> classToError = new HashMap, Errors>(); private static Map codeToError = new HashMap(); diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index e6ad8be5e33b6..bfe4f45486b19 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -317,8 +317,6 @@ class Partition(val topic: String, } else { (true, ErrorMapping.NotEnoughReplicasAfterAppendCode) } - } else if (requiredAcks > 0 && numAcks >= requiredAcks) { - (true, ErrorMapping.NoError) } else (false, ErrorMapping.NoError) case None => diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 6ee7d8819a9ef..703886a1d48e6 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -185,7 +185,7 @@ class KafkaApis(val requestChannel: RequestChannel, // the producer client will know that some error has happened and will refresh its metadata if (errorInResponse) { info("Close connection due to error handling produce request with correlation id %d from client id %s with ack=0" - .format(produceRequest.correlationId, produceRequest.clientId)) + .format(produceRequest.correlationId, produceRequest.clientId)) requestChannel.closeConnection(request.processor, request) } else { requestChannel.noOperation(request.processor, request) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index fb948b9ab28c5..ce36cc72606fb 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -29,6 +29,8 @@ import kafka.message.{ByteBufferMessageSet, MessageSet} import java.util.concurrent.atomic.AtomicBoolean import java.io.{IOException, File} import java.util.concurrent.TimeUnit +import org.apache.kafka.common.protocol.Errors + import scala.Predef._ import scala.collection._ import scala.collection.mutable.HashMap @@ -253,43 +255,66 @@ class ReplicaManager(val config: KafkaConfig, messagesPerPartition: Map[TopicAndPartition, MessageSet], responseCallback: Map[TopicAndPartition, ProducerResponseStatus] => Unit) { - val sTime = SystemTime.milliseconds - val localProduceResults = appendToLocalLog(internalTopicsAllowed, messagesPerPartition, requiredAcks) - debug("Produce to local log in %d ms".format(SystemTime.milliseconds - sTime)) + if (isValidRequiredAcks(requiredAcks)) { - val produceStatus = localProduceResults.map{ case (topicAndPartition, result) => - topicAndPartition -> - ProducePartitionStatus( - result.info.lastOffset + 1, // required offset - ProducerResponseStatus(result.errorCode, result.info.firstOffset)) // response status - } + val sTime = SystemTime.milliseconds + val localProduceResults = appendToLocalLog(internalTopicsAllowed, messagesPerPartition, requiredAcks) + debug("Produce to local log in %d ms".format(SystemTime.milliseconds - sTime)) - if(requiredAcks == 0 || - requiredAcks == 1 || - messagesPerPartition.size <= 0 || - localProduceResults.values.count(_.error.isDefined) == messagesPerPartition.size) { - // in case of the following we can respond immediately: - // - // 1. required acks = 0 or 1 - // 2. there is no data to append - // 3. all partition appends have failed - val produceResponseStatus = produceStatus.mapValues(status => status.responseStatus) - responseCallback(produceResponseStatus) - } else { - // create delayed produce operation - val produceMetadata = ProduceMetadata(requiredAcks, produceStatus) - val delayedProduce = new DelayedProduce(timeout, produceMetadata, this, responseCallback) + val produceStatus = localProduceResults.map { case (topicAndPartition, result) => + topicAndPartition -> + ProducePartitionStatus( + result.info.lastOffset + 1, // required offset + ProducerResponseStatus(result.errorCode, result.info.firstOffset)) // response status + } - // create a list of (topic, partition) pairs to use as keys for this delayed produce operation - val producerRequestKeys = messagesPerPartition.keys.map(new TopicPartitionOperationKey(_)).toSeq + if (delayedRequestRequired(requiredAcks, messagesPerPartition, localProduceResults)) { + // create delayed produce operation + val produceMetadata = ProduceMetadata(requiredAcks, produceStatus) + val delayedProduce = new DelayedProduce(timeout, produceMetadata, this, responseCallback) - // try to complete the request immediately, otherwise put it into the purgatory - // this is because while the delayed produce operation is being created, new - // requests may arrive and hence make this operation completable. - delayedProducePurgatory.tryCompleteElseWatch(delayedProduce, producerRequestKeys) + // create a list of (topic, partition) pairs to use as keys for this delayed produce operation + val producerRequestKeys = messagesPerPartition.keys.map(new TopicPartitionOperationKey(_)).toSeq + + // try to complete the request immediately, otherwise put it into the purgatory + // this is because while the delayed produce operation is being created, new + // requests may arrive and hence make this operation completable. + delayedProducePurgatory.tryCompleteElseWatch(delayedProduce, producerRequestKeys) + + } else { + // we can respond immediately + val produceResponseStatus = produceStatus.mapValues(status => status.responseStatus) + responseCallback(produceResponseStatus) + } + } else { + // If required.acks is outside accepted range, something is wrong with the client + // Just return an error and don't handle the request at all + val responseStatus = messagesPerPartition.map { + case (topicAndPartition, messageSet) => + (topicAndPartition -> + ProducerResponseStatus(Errors.INVALID_REQUIRED_ACKS.code, + LogAppendInfo.UnknownLogAppendInfo.firstOffset)) + } + responseCallback(responseStatus) } } + // If all the following conditions are true, we need to put a delayed produce request and wait for replication to complete + // + // 1. required acks = -1 + // 2. there is data to append + // 3. at least one partition append was successful (fewer errors than partitions) + private def delayedRequestRequired(requiredAcks: Short, messagesPerPartition: Map[TopicAndPartition, MessageSet], + localProduceResults: Map[TopicAndPartition, LogAppendResult]): Boolean = { + requiredAcks == -1 && + messagesPerPartition.size > 0 && + localProduceResults.values.count(_.error.isDefined) < messagesPerPartition.size + } + + private def isValidRequiredAcks(requiredAcks: Short): Boolean = { + requiredAcks == -1 || requiredAcks == 1 || requiredAcks == 0 + } + /** * Append the messages to the local replica logs */ diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index a1f72f8c2042f..fba852afa1b2f 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -59,7 +59,7 @@ object SerializationTestUtils { private val partitionDataMessage3 = new ByteBufferMessageSet(new Message("fourth message".getBytes)) private val partitionDataProducerRequestArray = Array(partitionDataMessage0, partitionDataMessage1, partitionDataMessage2, partitionDataMessage3) - private val topicDataProducerRequest = { + val topicDataProducerRequest = { val groupedData = Array(topic1, topic2).flatMap(topic => partitionDataProducerRequestArray.zipWithIndex.map { diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index faa907131ed0a..d1ed5c2c50689 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -17,16 +17,21 @@ package kafka.server +import kafka.api.{ProducerResponseStatus, SerializationTestUtils, ProducerRequest} +import kafka.common.TopicAndPartition import kafka.utils.{MockScheduler, MockTime, TestUtils} import java.util.concurrent.atomic.AtomicBoolean import java.io.File +import org.apache.kafka.common.protocol.Errors import org.easymock.EasyMock import org.I0Itec.zkclient.ZkClient import org.scalatest.junit.JUnit3Suite import org.junit.Test +import scala.collection.Map + class ReplicaManagerTest extends JUnit3Suite { val topic = "test-topic" @@ -63,4 +68,25 @@ class ReplicaManagerTest extends JUnit3Suite { // shutdown the replica manager upon test completion rm.shutdown(false) } + + @Test + def testIllegalRequiredAcks() { + val props = TestUtils.createBrokerConfig(1) + val config = new KafkaConfig(props) + val zkClient = EasyMock.createMock(classOf[ZkClient]) + val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) + val time: MockTime = new MockTime() + val rm = new ReplicaManager(config, time, zkClient, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false)) + val produceRequest = new ProducerRequest(1, "client 1", 3, 1000, SerializationTestUtils.topicDataProducerRequest) + def callback(responseStatus: Map[TopicAndPartition, ProducerResponseStatus]) = { + assert(responseStatus.values.head.error == Errors.INVALID_REQUIRED_ACKS.code) + } + + rm.appendMessages(timeout = 0, requiredAcks = 3, internalTopicsAllowed = false, messagesPerPartition = produceRequest.data, responseCallback = callback) + + rm.shutdown(false); + + TestUtils.verifyNonDaemonThreadsStatus + + } } From 5b949c7b1ad5ab3f71cae6024eea10b59a7ad9b6 Mon Sep 17 00:00:00 2001 From: Parth Brahmbhatt Date: Tue, 17 Feb 2015 16:18:35 -0800 Subject: [PATCH 122/491] KAFKA-1805; ProducerRecord should implement equals and hashCode; reviewed by Guozhang Wang --- .../clients/producer/ProducerRecord.java | 24 +++++++++ .../clients/producer/ProducerRecordTest.java | 52 +++++++++++++++++++ 2 files changed, 76 insertions(+) create mode 100644 clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java index 065d4e6c6a496..4990692efa6f0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java @@ -99,4 +99,28 @@ public String toString() { String value = this.value == null ? "null" : this.value.toString(); return "ProducerRecord(topic=" + topic + ", partition=" + partition + ", key=" + key + ", value=" + value; } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (!(o instanceof ProducerRecord)) return false; + + ProducerRecord that = (ProducerRecord) o; + + if (key != null ? !key.equals(that.key) : that.key != null) return false; + if (partition != null ? !partition.equals(that.partition) : that.partition != null) return false; + if (topic != null ? !topic.equals(that.topic) : that.topic != null) return false; + if (value != null ? !value.equals(that.value) : that.value != null) return false; + + return true; + } + + @Override + public int hashCode() { + int result = topic != null ? topic.hashCode() : 0; + result = 31 * result + (partition != null ? partition.hashCode() : 0); + result = 31 * result + (key != null ? key.hashCode() : 0); + result = 31 * result + (value != null ? value.hashCode() : 0); + return result; + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java new file mode 100644 index 0000000000000..7bb181e46c474 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/producer/ProducerRecordTest.java @@ -0,0 +1,52 @@ +/** + * 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.clients.producer; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +public class ProducerRecordTest { + + @Test + public void testEqualsAndHashCode() { + ProducerRecord producerRecord = new ProducerRecord("test", 1 , "key", 1); + assertEquals(producerRecord, producerRecord); + assertEquals(producerRecord.hashCode(), producerRecord.hashCode()); + + ProducerRecord equalRecord = new ProducerRecord("test", 1 , "key", 1); + assertEquals(producerRecord, equalRecord); + assertEquals(producerRecord.hashCode(), equalRecord.hashCode()); + + ProducerRecord topicMisMatch = new ProducerRecord("test-1", 1 , "key", 1); + assertFalse(producerRecord.equals(topicMisMatch)); + + ProducerRecord partitionMismatch = new ProducerRecord("test", 2 , "key", 1); + assertFalse(producerRecord.equals(partitionMismatch)); + + ProducerRecord keyMisMatch = new ProducerRecord("test", 1 , "key-1", 1); + assertFalse(producerRecord.equals(keyMisMatch)); + + ProducerRecord valueMisMatch = new ProducerRecord("test", 1 , "key", 2); + assertFalse(producerRecord.equals(valueMisMatch)); + + ProducerRecord nullFieldsRecord = new ProducerRecord("topic", null, null, null); + assertEquals(nullFieldsRecord, nullFieldsRecord); + assertEquals(nullFieldsRecord.hashCode(), nullFieldsRecord.hashCode()); + } +} From b8be3140b68cb62d908ca78fe280b089ded851b8 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Tue, 17 Feb 2015 17:33:04 -0800 Subject: [PATCH 123/491] KAFKA-1943; MessageSizeTooLarge and MessageSetSizeTooLarge should not be counted toward broker-side producer failure rate --- core/src/main/scala/kafka/server/ReplicaManager.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index ce36cc72606fb..399f7c48400c9 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -365,11 +365,15 @@ class ReplicaManager(val config: KafkaConfig, (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(utpe))) case nle: NotLeaderForPartitionException => (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(nle))) - case e: Throwable => + case mtl: MessageSizeTooLargeException => + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(mtl))) + case mstl: MessageSetSizeTooLargeException => + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(mstl))) + case t: Throwable => BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).failedProduceRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats.failedProduceRequestRate.mark() - error("Error processing append operation on partition %s".format(topicAndPartition), e) - (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(e))) + error("Error processing append operation on partition %s".format(topicAndPartition), t) + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(t))) } } } From cb40ec2e7a73bdecdfea6c88ffd8e8717b630d0f Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Tue, 17 Feb 2015 17:36:12 -0800 Subject: [PATCH 124/491] KAFKA-1914; Include total produce/fetch stats in broker topic metrics. --- .../src/main/scala/kafka/server/KafkaRequestHandler.scala | 2 ++ core/src/main/scala/kafka/server/ReplicaManager.scala | 6 ++++++ .../test/scala/unit/kafka/server/SimpleFetchTest.scala | 8 +++++++- 3 files changed, 15 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala index e4053fbe8ef78..4d86bdf5d146d 100644 --- a/core/src/main/scala/kafka/server/KafkaRequestHandler.scala +++ b/core/src/main/scala/kafka/server/KafkaRequestHandler.scala @@ -105,6 +105,8 @@ class BrokerTopicMetrics(name: Option[String]) extends KafkaMetricsGroup { val bytesRejectedRate = newMeter("BytesRejectedPerSec", "bytes", TimeUnit.SECONDS, tags) val failedProduceRequestRate = newMeter("FailedProduceRequestsPerSec", "requests", TimeUnit.SECONDS, tags) val failedFetchRequestRate = newMeter("FailedFetchRequestsPerSec", "requests", TimeUnit.SECONDS, tags) + val totalProduceRequestRate = newMeter("TotalProduceRequestsPerSec", "requests", TimeUnit.SECONDS, tags) + val totalFetchRequestRate = newMeter("TotalFetchRequestsPerSec", "requests", TimeUnit.SECONDS, tags) } object BrokerTopicStats extends Logging { diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 399f7c48400c9..b82ff55e1dd1f 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -323,6 +323,9 @@ class ReplicaManager(val config: KafkaConfig, requiredAcks: Short): Map[TopicAndPartition, LogAppendResult] = { trace("Append [%s] to local log ".format(messagesPerPartition)) messagesPerPartition.map { case (topicAndPartition, messages) => + BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).totalProduceRequestRate.mark() + BrokerTopicStats.getBrokerAllTopicsStats().totalProduceRequestRate.mark() + // reject appending to internal topics if it is not allowed if (Topic.InternalTopics.contains(topicAndPartition.topic) && !internalTopicsAllowed) { @@ -439,6 +442,9 @@ class ReplicaManager(val config: KafkaConfig, readPartitionInfo: Map[TopicAndPartition, PartitionFetchInfo]): Map[TopicAndPartition, LogReadResult] = { 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)) diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index ccf5e2e36260b..292a0427994fc 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -129,13 +129,19 @@ class SimpleFetchTest extends JUnit3Suite { * should only return data up to the HW of the partition; when a fetch operation with read * committed data turned off is received, the replica manager could return data up to the LEO * of the local leader replica's log. + * + * This test also verifies counts of fetch requests recorded by the ReplicaManager */ def testReadFromLog() { + val initialTopicCount = BrokerTopicStats.getBrokerTopicStats(topic).totalFetchRequestRate.count(); + 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) - 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) + + assertEquals("Counts should increment after fetch", initialTopicCount+2, BrokerTopicStats.getBrokerTopicStats(topic).totalFetchRequestRate.count()); + assertEquals("Counts should increment after fetch", initialTopicCount+2, BrokerTopicStats.getBrokerAllTopicsStats().totalFetchRequestRate.count()); } } From cdf06564458ac384119d21ac8b25a3a7ebe27014 Mon Sep 17 00:00:00 2001 From: Tong Li Date: Tue, 17 Feb 2015 23:48:21 -0800 Subject: [PATCH 125/491] KAFKA-1959; Rename group to groupId in TestOffsetManager due to collision with Thread.group in IBM's JDK; reviewed by Joel Koshy and Gwen Shapira --- .../src/test/scala/other/kafka/TestOffsetManager.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/test/scala/other/kafka/TestOffsetManager.scala b/core/src/test/scala/other/kafka/TestOffsetManager.scala index 41f334d48897b..a106379d96ab3 100644 --- a/core/src/test/scala/other/kafka/TestOffsetManager.scala +++ b/core/src/test/scala/other/kafka/TestOffsetManager.scala @@ -50,9 +50,9 @@ object TestOffsetManager { extends ShutdownableThread("commit-thread") with KafkaMetricsGroup { - private val group = "group-" + id + private val groupId = "group-" + id private val metadata = "Metadata from commit thread " + id - private var offsetsChannel = ClientUtils.channelToOffsetManager(group, zkClient, SocketTimeoutMs) + private var offsetsChannel = ClientUtils.channelToOffsetManager(groupId, zkClient, SocketTimeoutMs) private var offset = 0L val numErrors = new AtomicInteger(0) val numCommits = new AtomicInteger(0) @@ -62,11 +62,11 @@ object TestOffsetManager { private def ensureConnected() { if (!offsetsChannel.isConnected) - offsetsChannel = ClientUtils.channelToOffsetManager(group, zkClient, SocketTimeoutMs) + offsetsChannel = ClientUtils.channelToOffsetManager(groupId, zkClient, SocketTimeoutMs) } override def doWork() { - val commitRequest = OffsetCommitRequest(group, immutable.Map((1 to partitionCount).map(TopicAndPartition("topic-" + id, _) -> OffsetAndMetadata(offset, metadata)):_*)) + val commitRequest = OffsetCommitRequest(groupId, immutable.Map((1 to partitionCount).map(TopicAndPartition("topic-" + id, _) -> OffsetAndMetadata(offset, metadata)):_*)) try { ensureConnected() offsetsChannel.send(commitRequest) @@ -81,7 +81,7 @@ object TestOffsetManager { case e1: ClosedByInterruptException => offsetsChannel.disconnect() case e2: IOException => - println("Commit thread %d: Error while committing offsets to %s:%d for group %s due to %s.".format(id, offsetsChannel.host, offsetsChannel.port, group, e2)) + println("Commit thread %d: Error while committing offsets to %s:%d for group %s due to %s.".format(id, offsetsChannel.host, offsetsChannel.port, groupId, e2)) offsetsChannel.disconnect() } finally { From d5fbba633612fd08f15c523362a0c1ff5a121ff2 Mon Sep 17 00:00:00 2001 From: Tong Li Date: Tue, 17 Feb 2015 23:49:57 -0800 Subject: [PATCH 126/491] KAFKA-1960; .gitignore does not exclude test generated files and folders; reviewed by Joel Koshy and Gwen Shapira --- .gitignore | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.gitignore b/.gitignore index 06a64184eaa53..4c6c29e2575e0 100644 --- a/.gitignore +++ b/.gitignore @@ -26,3 +26,5 @@ Vagrantfile.local config/server-* config/zookeeper-* +core/data/* +gradle/wrapper/* From f5684366ef60125c4d799121a6c0adca4744e8ab Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Wed, 18 Feb 2015 13:39:05 -0800 Subject: [PATCH 127/491] kafka-1952; High CPU Usage in 0.8.2 release; patched by Jun Rao; reviewed by Guozhang Wang, Ewen Cheslack-Postava and Neha Narkhede --- .../scala/kafka/server/DelayedOperation.scala | 34 +++++++++++++------ 1 file changed, 23 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/kafka/server/DelayedOperation.scala b/core/src/main/scala/kafka/server/DelayedOperation.scala index fc06b01cad3a0..1d11099a72d16 100644 --- a/core/src/main/scala/kafka/server/DelayedOperation.scala +++ b/core/src/main/scala/kafka/server/DelayedOperation.scala @@ -128,25 +128,37 @@ class DelayedOperationPurgatory[T <: DelayedOperation](brokerId: Int = 0, purgeI * @return true iff the delayed operations can be completed by the caller */ def tryCompleteElseWatch(operation: T, watchKeys: Seq[Any]): Boolean = { + assert(watchKeys.size > 0, "The watch key list can't be empty") + + // The cost of tryComplete() is typically proportional to the number of keys. Calling + // tryComplete() for each key is going to be expensive if there are many keys. Instead, + // we do the check in the following way. Call tryComplete(). If the operation is not completed, + // we just add the operation to all keys. Then we call tryComplete() again. At this time, if + // the operation is still not completed, we are guaranteed that it won't miss any future triggering + // event since the operation is already on the watcher list for all keys. This does mean that + // if the operation is completed (by another thread) between the two tryComplete() calls, the + // operation is unnecessarily added for watch. However, this is a less severe issue since the + // expire reaper will clean it up periodically. + + var isCompletedByMe = operation synchronized operation.tryComplete() + if (isCompletedByMe) + return true + for(key <- watchKeys) { - // if the operation is already completed, stopping adding it to - // any further lists and return false + // If the operation is already completed, stop adding it to the rest of the watcher list. if (operation.isCompleted()) return false val watchers = watchersFor(key) - // if the operation can by completed by myself, stop adding it to - // any further lists and return true immediately - if(operation synchronized operation.tryComplete()) { - return true - } else { - watchers.watch(operation) - } + watchers.watch(operation) } + isCompletedByMe = operation synchronized operation.tryComplete() + if (isCompletedByMe) + return true + // if it cannot be completed by now and hence is watched, add to the expire queue also - if (! operation.isCompleted()) { + if (! operation.isCompleted()) expirationReaper.enqueue(operation) - } false } From 8d72816f48d18d61e8ef8b174896c6d177a09a92 Mon Sep 17 00:00:00 2001 From: Joel Koshy Date: Wed, 18 Feb 2015 14:38:03 -0800 Subject: [PATCH 128/491] KAFKA-1914; follow-up to address unit test failure --- core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index 292a0427994fc..525c835803b04 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -140,8 +140,5 @@ class SimpleFetchTest extends JUnit3Suite { replicaManager.readFromLocalLog(true, true, 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) - - assertEquals("Counts should increment after fetch", initialTopicCount+2, BrokerTopicStats.getBrokerTopicStats(topic).totalFetchRequestRate.count()); - assertEquals("Counts should increment after fetch", initialTopicCount+2, BrokerTopicStats.getBrokerAllTopicsStats().totalFetchRequestRate.count()); } } From f7f8ac1a8864acdbfc52326edc7cbeb9e35f4adf Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Wed, 18 Feb 2015 16:36:58 -0800 Subject: [PATCH 129/491] KAFKA-1914; follow-up to fix SimpleFetchTest; reviewed by Joel Koshy --- core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index 525c835803b04..fd8f32c4158d4 100644 --- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -140,5 +140,8 @@ class SimpleFetchTest extends JUnit3Suite { replicaManager.readFromLocalLog(true, true, 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) + + 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 4c8817e3d156416240906dd06a4494386949ba0a Mon Sep 17 00:00:00 2001 From: Joel Koshy Date: Wed, 18 Feb 2015 17:53:45 -0800 Subject: [PATCH 130/491] KAFKA-1729; (followup patch)Add constructor to javaapi to allow constructing explicitly versioned offset fetch requests; reviewed by Jun Rao --- .../main/scala/kafka/javaapi/OffsetFetchRequest.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/javaapi/OffsetFetchRequest.scala b/core/src/main/scala/kafka/javaapi/OffsetFetchRequest.scala index 1c25aa3332f9e..818ae33034312 100644 --- a/core/src/main/scala/kafka/javaapi/OffsetFetchRequest.scala +++ b/core/src/main/scala/kafka/javaapi/OffsetFetchRequest.scala @@ -28,6 +28,14 @@ class OffsetFetchRequest(groupId: String, correlationId: Int, clientId: String) { + def this(groupId: String, + requestInfo: java.util.List[TopicAndPartition], + correlationId: Int, + clientId: String) { + // by default bind to version 0 so that it fetches from ZooKeeper + this(groupId, requestInfo, 0, correlationId, clientId) + } + val underlying = { val scalaSeq = { import JavaConversions._ @@ -36,7 +44,7 @@ class OffsetFetchRequest(groupId: String, kafka.api.OffsetFetchRequest( groupId = groupId, requestInfo = scalaSeq, - versionId = 0, // binds to version 0 so that it commits to Zookeeper + versionId = versionId, correlationId = correlationId, clientId = clientId ) From 38346fb106064a10047ecc7cefa634c6f1b09960 Mon Sep 17 00:00:00 2001 From: Yaguo Zhou Date: Fri, 20 Feb 2015 14:36:20 -0800 Subject: [PATCH 131/491] Minor typo in KafkaConfig; reviewed by Gwen Shapira --- core/src/main/scala/kafka/server/KafkaConfig.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 6d74983472249..14bf3216bae03 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -117,10 +117,10 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro * it will publish the same port that the broker binds to. */ val advertisedPort: Int = props.getInt("advertised.port", port) - /* the SO_SNDBUFF buffer of the socket sever sockets */ + /* the SO_SNDBUF buffer of the socket sever sockets */ val socketSendBufferBytes: Int = props.getInt("socket.send.buffer.bytes", 100*1024) - /* the SO_RCVBUFF buffer of the socket sever sockets */ + /* the SO_RCVBUF buffer of the socket sever sockets */ val socketReceiveBufferBytes: Int = props.getInt("socket.receive.buffer.bytes", 100*1024) /* the maximum number of bytes in a socket request */ From 8c1b9325be4adc5065c6dbe3dcdbdccb1887d604 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Fri, 20 Feb 2015 16:22:15 -0800 Subject: [PATCH 132/491] KAFKA-1867 liveBroker list not updated on a cluster with no topics; reviewed by Neha Narkhede --- .../kafka/controller/ControllerChannelManager.scala | 8 +++++++- .../kafka/api/ProducerFailureHandlingTest.scala | 7 ------- .../test/scala/unit/kafka/server/OffsetCommitTest.scala | 9 +-------- 3 files changed, 8 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index 14b22ab38cc14..c582191636f61 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -269,7 +269,13 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging else givenPartitions -- controller.deleteTopicManager.partitionsToBeDeleted } - filteredPartitions.foreach(partition => updateMetadataRequestMapFor(partition, beingDeleted = false)) + if(filteredPartitions.isEmpty) + brokerIds.filter(b => b >= 0).foreach { brokerId => + updateMetadataRequestMap.getOrElseUpdate(brokerId, new mutable.HashMap[TopicAndPartition, PartitionStateInfo]) + } + else + filteredPartitions.foreach(partition => updateMetadataRequestMapFor(partition, beingDeleted = false)) + controller.deleteTopicManager.partitionsToBeDeleted.foreach(partition => updateMetadataRequestMapFor(partition, beingDeleted = true)) } diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index 11d6a9751a831..ba48a636dd0b0 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -46,13 +46,6 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { override val zkConnect = TestZKUtils.zookeeperConnect override val autoCreateTopicsEnable = false override val messageMaxBytes = serverMessageMaxBytes - // TODO: Currently, when there is no topic in a cluster, the controller doesn't send any UpdateMetadataRequest to - // the broker. As a result, the live broker list in metadataCache is empty. If the number of live brokers is 0, we - // try to create the offset topic with the default offsets.topic.replication.factor of 3. The creation will fail - // since there is not enough live brokers. This causes testCannotSendToInternalTopic() to fail. Temporarily fixing - // the issue by overriding offsets.topic.replication.factor to 1 for now. When we fix KAFKA-1867, we need to - // remove the following config override. - override val offsetsTopicReplicationFactor = 1.asInstanceOf[Short] // 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 override val offsetsTopicPartitions = 1 diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index 5b93239cdc26b..a2bb8855c3c05 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -5,7 +5,7 @@ * 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 @@ -46,13 +46,6 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { override def setUp() { super.setUp() val config: Properties = createBrokerConfig(1, brokerPort) - // TODO: Currently, when there is no topic in a cluster, the controller doesn't send any UpdateMetadataRequest to - // the broker. As a result, the live broker list in metadataCache is empty. This causes the ConsumerMetadataRequest - // to fail since if the number of live brokers is 0, we try to create the offset topic with the default - // offsets.topic.replication.factor of 3. The creation will fail since there is not enough live brokers. In order - // for the unit test to pass, overriding offsets.topic.replication.factor to 1 for now. When we fix KAFKA-1867, we - // need to remove the following config override. - config.put("offsets.topic.replication.factor", "1") val logDirPath = config.getProperty("log.dir") logDir = new File(logDirPath) time = new MockTime() From fbb11549743aa00420e40696f11b48a705dce019 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 20 Feb 2015 16:33:01 -0800 Subject: [PATCH 133/491] Leaner DelayedItem; reviewed by Neha Narkhede and Joel Koshy --- .../main/scala/kafka/utils/DelayedItem.scala | 29 +++++++------------ 1 file changed, 10 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/kafka/utils/DelayedItem.scala b/core/src/main/scala/kafka/utils/DelayedItem.scala index a4e0dabc858bc..cbab2a01bfbfb 100644 --- a/core/src/main/scala/kafka/utils/DelayedItem.scala +++ b/core/src/main/scala/kafka/utils/DelayedItem.scala @@ -5,7 +5,7 @@ * 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 @@ -20,34 +20,25 @@ package kafka.utils import java.util.concurrent._ import scala.math._ -class DelayedItem(delay: Long, unit: TimeUnit) extends Delayed with Logging { +class DelayedItem(delayMs: Long) extends Delayed with Logging { - val createdMs = SystemTime.milliseconds - val delayMs = { - val given = unit.toMillis(delay) - if (given < 0 || (createdMs + given) < 0) (Long.MaxValue - createdMs) - else given - } + private val dueMs = SystemTime.milliseconds + delayMs - def this(delayMs: Long) = - this(delayMs, TimeUnit.MILLISECONDS) + def this(delay: Long, unit: TimeUnit) = this(unit.toMillis(delay)) /** * The remaining delay time */ def getDelay(unit: TimeUnit): Long = { - val elapsedMs = (SystemTime.milliseconds - createdMs) - unit.convert(max(delayMs - elapsedMs, 0), TimeUnit.MILLISECONDS) + unit.convert(max(dueMs - SystemTime.milliseconds, 0), TimeUnit.MILLISECONDS) } - + def compareTo(d: Delayed): Int = { - val delayed = d.asInstanceOf[DelayedItem] - val myEnd = createdMs + delayMs - val yourEnd = delayed.createdMs + delayed.delayMs + val other = d.asInstanceOf[DelayedItem] - if(myEnd < yourEnd) -1 - else if(myEnd > yourEnd) 1 + if(dueMs < other.dueMs) -1 + else if(dueMs > other.dueMs) 1 else 0 } - + } From 19031823c15cb3a0e5fcecaaf5de79c09b5300ed Mon Sep 17 00:00:00 2001 From: Jay Kreps Date: Wed, 4 Feb 2015 17:01:51 -0800 Subject: [PATCH 134/491] KAFKA-1919: Always update the metadata, when a metadata response is received to ensure we back off. --- .../src/main/java/org/apache/kafka/clients/Metadata.java | 8 ++++++++ .../main/java/org/apache/kafka/clients/NetworkClient.java | 6 ++++-- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java index b8cdd145bfcc6..e8afecda95630 100644 --- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -147,6 +147,14 @@ public synchronized void update(Cluster cluster, long now) { log.debug("Updated cluster metadata version {} to {}", this.version, this.cluster); } + /** + * Record an attempt to update the metadata that failed. We need to keep track of this + * to avoid retrying immediately. + */ + public synchronized void failedUpdate(long now) { + this.lastRefreshMs = now; + } + /** * @return The current metadata version */ diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index fef90a03ed04d..a7fa4a9dfbcfb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -381,10 +381,12 @@ private void handleMetadataResponse(RequestHeader header, Struct body, long now) Cluster cluster = response.cluster(); // don't update the cluster if there are no valid nodes...the topic we want may still be in the process of being // created which means we will get errors and no nodes until it exists - if (cluster.nodes().size() > 0) + if (cluster.nodes().size() > 0) { this.metadata.update(cluster, now); - else + } else { log.trace("Ignoring empty metadata response with correlation id {}.", header.correlationId()); + this.metadata.failedUpdate(now); + } } /** From 62c7e0e485b8b7693354c3a0a0bf95af671c93b6 Mon Sep 17 00:00:00 2001 From: Honghai Chen Date: Sat, 21 Feb 2015 20:26:57 -0800 Subject: [PATCH 135/491] KAFKA-1947 can't explicitly set replica-assignment when add partitions; reviewed by Neha Narkhede --- core/src/main/scala/kafka/admin/TopicCommand.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index 285c0333ff435..f400b71f8444f 100644 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -301,8 +301,9 @@ object TopicCommand { CommandLineUtils.checkInvalidArgs(parser, options, deleteConfigOpt, allTopicLevelOpts -- Set(alterOpt)) CommandLineUtils.checkInvalidArgs(parser, options, partitionsOpt, allTopicLevelOpts -- Set(alterOpt, createOpt)) CommandLineUtils.checkInvalidArgs(parser, options, replicationFactorOpt, allTopicLevelOpts -- Set(createOpt)) - CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, - allTopicLevelOpts -- Set(alterOpt, createOpt) + partitionsOpt + replicationFactorOpt) + CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, allTopicLevelOpts -- Set(createOpt,alterOpt)) + if(options.has(createOpt)) + CommandLineUtils.checkInvalidArgs(parser, options, replicaAssignmentOpt, Set(partitionsOpt, replicationFactorOpt)) CommandLineUtils.checkInvalidArgs(parser, options, reportUnderReplicatedPartitionsOpt, allTopicLevelOpts -- Set(describeOpt) + reportUnavailablePartitionsOpt + topicsWithOverridesOpt) CommandLineUtils.checkInvalidArgs(parser, options, reportUnavailablePartitionsOpt, From 3f1e0882297a102843dcd17e05f54ddbedb4882a Mon Sep 17 00:00:00 2001 From: Jay Kreps Date: Sat, 21 Feb 2015 21:10:28 -0800 Subject: [PATCH 136/491] KAFKA-1757 Unmap file prior to delete on windows. --- core/src/main/scala/kafka/log/OffsetIndex.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/kafka/log/OffsetIndex.scala b/core/src/main/scala/kafka/log/OffsetIndex.scala index 1c4c7bd89e19e..ca82c043bb841 100644 --- a/core/src/main/scala/kafka/log/OffsetIndex.scala +++ b/core/src/main/scala/kafka/log/OffsetIndex.scala @@ -318,6 +318,8 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi */ def delete(): Boolean = { info("Deleting index " + this.file.getAbsolutePath) + if(Os.isWindows) + Utils.swallow(forceUnmap(this.mmap)) this.file.delete() } From 41189ea5601837bdb697ade31f55e244abbe6d1c Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Mon, 23 Feb 2015 11:51:32 -0800 Subject: [PATCH 137/491] kafka-1971; starting a broker with a conflicting id will delete the previous broker registration; patched by Jun Rao; reviewed by Neha Narkhede --- .../scala/kafka/server/KafkaHealthcheck.scala | 7 +--- .../main/scala/kafka/server/KafkaServer.scala | 2 - core/src/main/scala/kafka/utils/ZkUtils.scala | 6 --- .../unit/kafka/server/ServerStartupTest.scala | 42 +++++++++++++------ 4 files changed, 31 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala index 4acdd70fe9c1e..7907987e43404 100644 --- a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala +++ b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala @@ -39,17 +39,12 @@ class KafkaHealthcheck(private val brokerId: Int, val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + brokerId val sessionExpireListener = new SessionExpireListener - + def startup() { zkClient.subscribeStateChanges(sessionExpireListener) register() } - def shutdown() { - zkClient.unsubscribeStateChanges(sessionExpireListener) - ZkUtils.deregisterBrokerInZk(zkClient, brokerId) - } - /** * Register this broker as "alive" in zookeeper */ diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 7e5ddcb9be8fc..426e522fc9819 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -310,8 +310,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg if (canShutdown) { Utils.swallow(controlledShutdown()) brokerState.newState(BrokerShuttingDown) - if(kafkaHealthcheck != null) - Utils.swallow(kafkaHealthcheck.shutdown()) if(socketServer != null) Utils.swallow(socketServer.shutdown()) if(requestHandlerPool != null) diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index c78a1b6ff4213..8a2fb2d9a42ae 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -189,12 +189,6 @@ object ZkUtils extends Logging { info("Registered broker %d at path %s with address %s:%d.".format(id, brokerIdPath, host, port)) } - def deregisterBrokerInZk(zkClient: ZkClient, id: Int) { - val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id - deletePath(zkClient, brokerIdPath) - info("Deregistered broker %d at path %s.".format(id, brokerIdPath)) - } - def getConsumerPartitionOwnerPath(group: String, topic: String, partition: Int): String = { val topicDirs = new ZKGroupTopicDirs(group, topic) topicDirs.consumerOwnerDir + "/" + partition diff --git a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala index 764655a6883f9..93af7dfcec7a6 100644 --- a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala @@ -26,26 +26,44 @@ import kafka.zk.ZooKeeperTestHarness import junit.framework.Assert._ class ServerStartupTest extends JUnit3Suite with ZooKeeperTestHarness { - var server : KafkaServer = null - val brokerId = 0 - val zookeeperChroot = "/kafka-chroot-for-unittest" - override def setUp() { - super.setUp() + def testBrokerCreatesZKChroot { + val brokerId = 0 + val zookeeperChroot = "/kafka-chroot-for-unittest" val props = TestUtils.createBrokerConfig(brokerId, TestUtils.choosePort()) val zooKeeperConnect = props.get("zookeeper.connect") props.put("zookeeper.connect", zooKeeperConnect + zookeeperChroot) - server = TestUtils.createServer(new KafkaConfig(props)) - } + val server = TestUtils.createServer(new KafkaConfig(props)) + + val pathExists = ZkUtils.pathExists(zkClient, zookeeperChroot) + assertTrue(pathExists) - override def tearDown() { server.shutdown() Utils.rm(server.config.logDirs) - super.tearDown() } - def testBrokerCreatesZKChroot { - val pathExists = ZkUtils.pathExists(zkClient, zookeeperChroot) - assertTrue(pathExists) + def testConflictBrokerRegistration { + // Try starting a broker with the a conflicting broker id. + // This shouldn't affect the existing broker registration. + + val brokerId = 0 + val props1 = TestUtils.createBrokerConfig(brokerId) + val server1 = TestUtils.createServer(new KafkaConfig(props1)) + val brokerRegistration = ZkUtils.readData(zkClient, ZkUtils.BrokerIdsPath + "/" + brokerId)._1 + + val props2 = TestUtils.createBrokerConfig(brokerId) + try { + TestUtils.createServer(new KafkaConfig(props2)) + fail("Registering a broker with a conflicting id should fail") + } catch { + case e : RuntimeException => + // this is expected + } + + // broker registration shouldn't change + assertEquals(brokerRegistration, ZkUtils.readData(zkClient, ZkUtils.BrokerIdsPath + "/" + brokerId)._1) + + server1.shutdown() + Utils.rm(server1.config.logDirs) } } \ No newline at end of file From 7c82afddc812908f9289ded4e2cb0733b8116a77 Mon Sep 17 00:00:00 2001 From: Onur Karaman Date: Tue, 24 Feb 2015 08:25:27 -0800 Subject: [PATCH 138/491] Added script for bin/kafka-consumer-groups.sh that was mistakenly dropped from the original patch for KAFKA-1476 --- bin/kafka-consumer-groups.sh | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) create mode 100755 bin/kafka-consumer-groups.sh diff --git a/bin/kafka-consumer-groups.sh b/bin/kafka-consumer-groups.sh new file mode 100755 index 0000000000000..f4786dbda01f2 --- /dev/null +++ b/bin/kafka-consumer-groups.sh @@ -0,0 +1,17 @@ +#!/bin/bash +# 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. + +exec $(dirname $0)/kafka-run-class.sh kafka.admin.ConsumerGroupCommand $@ From 10311c138923dcded5d8f57f587c7f7ef1119dac Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Tue, 24 Feb 2015 14:07:27 -0800 Subject: [PATCH 139/491] kafka-1984; java producer may miss an available partition; patched by Jun Rao; reviewed by Ewen Cheslack-Postava, Jay Kreps, and Guozhang Wang --- .../producer/internals/Partitioner.java | 15 +++++----- .../java/org/apache/kafka/common/Cluster.java | 24 +++++++++++++-- .../clients/producer/PartitionerTest.java | 29 ++++++++++--------- 3 files changed, 45 insertions(+), 23 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java index 8112e6d89df78..dfb936d8f0d58 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java @@ -56,14 +56,15 @@ public int partition(String topic, byte[] key, Integer partition, Cluster cluste + "]."); return partition; } else if (key == null) { - // choose the next available node in a round-robin fashion - for (int i = 0; i < numPartitions; i++) { - int part = Utils.abs(counter.getAndIncrement()) % numPartitions; - if (partitions.get(part).leader() != null) - return part; + int nextValue = counter.getAndIncrement(); + List availablePartitions = cluster.availablePartitionsForTopic(topic); + if (availablePartitions.size() > 0) { + int part = Utils.abs(nextValue) % availablePartitions.size(); + return availablePartitions.get(part).partition(); + } else { + // no partitions are available, give a non-available partition + return Utils.abs(nextValue) % numPartitions; } - // no partitions are available, give a non-available partition - return Utils.abs(counter.getAndIncrement()) % numPartitions; } else { // hash the key to choose a partition return Utils.abs(Utils.murmur2(key)) % numPartitions; diff --git a/clients/src/main/java/org/apache/kafka/common/Cluster.java b/clients/src/main/java/org/apache/kafka/common/Cluster.java index 8fcd2916f1d55..60594a7dce901 100644 --- a/clients/src/main/java/org/apache/kafka/common/Cluster.java +++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java @@ -25,6 +25,7 @@ public final class Cluster { private final List nodes; private final Map partitionsByTopicPartition; private final Map> partitionsByTopic; + private final Map> availablePartitionsByTopic; private final Map> partitionsByNode; private final Map nodesById; @@ -68,8 +69,18 @@ public Cluster(Collection nodes, Collection partitions) { } } this.partitionsByTopic = new HashMap>(partsForTopic.size()); - for (Map.Entry> entry : partsForTopic.entrySet()) - this.partitionsByTopic.put(entry.getKey(), Collections.unmodifiableList(entry.getValue())); + this.availablePartitionsByTopic = new HashMap>(partsForTopic.size()); + for (Map.Entry> entry : partsForTopic.entrySet()) { + String topic = entry.getKey(); + List partitionList = entry.getValue(); + this.partitionsByTopic.put(topic, Collections.unmodifiableList(partitionList)); + List availablePartitions = new ArrayList(); + for (PartitionInfo part : partitionList) { + if (part.leader() != null) + availablePartitions.add(part); + } + this.availablePartitionsByTopic.put(topic, Collections.unmodifiableList(availablePartitions)); + } this.partitionsByNode = new HashMap>(partsForNode.size()); for (Map.Entry> entry : partsForNode.entrySet()) this.partitionsByNode.put(entry.getKey(), Collections.unmodifiableList(entry.getValue())); @@ -143,6 +154,15 @@ public List partitionsForTopic(String topic) { return this.partitionsByTopic.get(topic); } + /** + * Get the list of available partitions for this topic + * @param topic The topic name + * @return A list of partitions + */ + public List availablePartitionsForTopic(String topic) { + return this.availablePartitionsByTopic.get(topic); + } + /** * Get the list of partitions whose leader is this node * @param nodeId The node id diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java index 29c8417422c0c..404bedb3dc4e4 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java @@ -33,9 +33,10 @@ public class PartitionerTest { private Node node2 = new Node(2, "localhost", 101); private Node[] nodes = new Node[] {node0, node1, node2}; private String topic = "test"; - private List partitions = asList(new PartitionInfo(topic, 0, node0, nodes, nodes), - new PartitionInfo(topic, 1, node1, nodes, nodes), - new PartitionInfo(topic, 2, null, nodes, nodes)); + // Intentionally make the partition list not in partition order to test the edge cases. + private List partitions = asList(new PartitionInfo(topic, 1, null, nodes, nodes), + new PartitionInfo(topic, 2, node1, nodes, nodes), + new PartitionInfo(topic, 0, node0, nodes, nodes)); private Cluster cluster = new Cluster(asList(node0, node1, node2), partitions); @Test @@ -50,19 +51,19 @@ public void testKeyPartitionIsStable() { } @Test - public void testRoundRobinIsStable() { - int startPart = partitioner.partition("test", null, null, cluster); + public void testRoundRobinWithUnavailablePartitions() { + // When there are some unavailable partitions, we want to make sure that (1) we always pick an available partition, + // and (2) the available partitions are selected in a round robin way. + int countForPart0 = 0; + int countForPart2 = 0; for (int i = 1; i <= 100; i++) { - int partition = partitioner.partition("test", null, null, cluster); - assertEquals("Should yield a different partition each call with round-robin partitioner", partition, (startPart + i) % 2); - } - } - - @Test - public void testRoundRobinWithDownNode() { - for (int i = 0; i < partitions.size(); i++) { int part = partitioner.partition("test", null, null, cluster); - assertTrue("We should never choose a leader-less node in round robin", part >= 0 && part < 2); + assertTrue("We should never choose a leader-less node in round robin", part == 0 || part == 2); + if (part == 0) + countForPart0++; + else + countForPart2++; } + assertEquals("The distribution between two available partitions should be even", countForPart0, countForPart2); } } From b8904e9614b4c1f11b8487e7a4b88b1e37e1f20b Mon Sep 17 00:00:00 2001 From: Joel Koshy Date: Wed, 25 Feb 2015 10:59:28 -0800 Subject: [PATCH 140/491] KAFKA-1953; KAFKA-1962; Disambiguate purgatory metrics; restore delayed request metrics; reviewed by Guozhang Wang --- .../coordinator/ConsumerCoordinator.scala | 6 ++-- .../kafka/coordinator/DelayedHeartbeat.scala | 4 +++ .../kafka/coordinator/DelayedJoinGroup.scala | 4 +++ .../kafka/coordinator/DelayedRebalance.scala | 4 +++ .../scala/kafka/server/DelayedFetch.scala | 20 +++++++++++- .../scala/kafka/server/DelayedOperation.scala | 22 ++++++++++--- .../scala/kafka/server/DelayedProduce.scala | 31 +++++++++++++++++++ .../scala/kafka/server/ReplicaManager.scala | 9 ++++-- .../kafka/server/DelayedOperationTest.scala | 6 +++- 9 files changed, 93 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala index 01cf1d91b7056..21790a5059ee0 100644 --- a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala @@ -71,9 +71,9 @@ class ConsumerCoordinator(val config: KafkaConfig, latestHeartbeatBucketEndMs = SystemTime.milliseconds // Initialize purgatories for delayed heartbeat, join-group and rebalance operations - heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat](config.brokerId) - joinGroupPurgatory = new DelayedOperationPurgatory[DelayedJoinGroup](config.brokerId) - rebalancePurgatory = new DelayedOperationPurgatory[DelayedRebalance](config.brokerId) + heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat](purgatoryName = "Heartbeat", brokerId = config.brokerId) + joinGroupPurgatory = new DelayedOperationPurgatory[DelayedJoinGroup](purgatoryName = "JoinGroup", brokerId = config.brokerId) + rebalancePurgatory = new DelayedOperationPurgatory[DelayedRebalance](purgatoryName = "Rebalance", brokerId = config.brokerId) } diff --git a/core/src/main/scala/kafka/coordinator/DelayedHeartbeat.scala b/core/src/main/scala/kafka/coordinator/DelayedHeartbeat.scala index 894d6edb4077c..b1248e95d8a64 100644 --- a/core/src/main/scala/kafka/coordinator/DelayedHeartbeat.scala +++ b/core/src/main/scala/kafka/coordinator/DelayedHeartbeat.scala @@ -36,6 +36,10 @@ class DelayedHeartbeat(sessionTimeout: Long, throw new IllegalStateException("Delayed heartbeat purgatory should never try to complete any bucket") } + override def onExpiration() { + // TODO + } + /* mark all consumers within the heartbeat as heartbeat timed out */ override def onComplete() { for (registry <- bucket.consumerRegistryList) diff --git a/core/src/main/scala/kafka/coordinator/DelayedJoinGroup.scala b/core/src/main/scala/kafka/coordinator/DelayedJoinGroup.scala index 445bfa1bf8840..df60cbc35d099 100644 --- a/core/src/main/scala/kafka/coordinator/DelayedJoinGroup.scala +++ b/core/src/main/scala/kafka/coordinator/DelayedJoinGroup.scala @@ -35,6 +35,10 @@ class DelayedJoinGroup(sessionTimeout: Long, forceComplete() } + override def onExpiration() { + // TODO + } + /* always assume the partition is already assigned as this delayed operation should never time-out */ override def onComplete() { diff --git a/core/src/main/scala/kafka/coordinator/DelayedRebalance.scala b/core/src/main/scala/kafka/coordinator/DelayedRebalance.scala index b3b3749a21d35..8defa2e41c92f 100644 --- a/core/src/main/scala/kafka/coordinator/DelayedRebalance.scala +++ b/core/src/main/scala/kafka/coordinator/DelayedRebalance.scala @@ -50,6 +50,10 @@ class DelayedRebalance(sessionTimeout: Long, false } + override def onExpiration() { + // TODO + } + /* mark consumers that have not re-joined group as failed and proceed to rebalance the rest of the group */ override def onComplete() { groupRegistry.memberRegistries.values.foreach(consumerRegistry => diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index dd602ee2e65c2..de6cf5bdaa0e7 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -17,11 +17,14 @@ package kafka.server +import java.util.concurrent.TimeUnit + import kafka.api.FetchResponsePartitionData import kafka.api.PartitionFetchInfo import kafka.common.UnknownTopicOrPartitionException import kafka.common.NotLeaderForPartitionException import kafka.common.TopicAndPartition +import kafka.metrics.KafkaMetricsGroup import scala.collection._ @@ -37,6 +40,7 @@ case class FetchPartitionStatus(startOffsetMetadata: LogOffsetMetadata, fetchInf case class FetchMetadata(fetchMinBytes: Int, fetchOnlyLeader: Boolean, fetchOnlyCommitted: Boolean, + isFromFollower: Boolean, fetchPartitionStatus: Map[TopicAndPartition, FetchPartitionStatus]) { override def toString = "[minBytes: " + fetchMinBytes + ", " + @@ -109,6 +113,13 @@ class DelayedFetch(delayMs: Long, false } + override def onExpiration() { + if (fetchMetadata.isFromFollower) + DelayedFetchMetrics.followerExpiredRequestMeter.mark() + else + DelayedFetchMetrics.consumerExpiredRequestMeter.mark() + } + /** * Upon completion, read whatever data is available and pass to the complete callback */ @@ -122,4 +133,11 @@ class DelayedFetch(delayMs: Long, responseCallback(fetchPartitionData) } -} \ No newline at end of file +} + +object DelayedFetchMetrics extends KafkaMetricsGroup { + private val FetcherTypeKey = "fetcherType" + val followerExpiredRequestMeter = newMeter("ExpiresPerSec", "requests", TimeUnit.SECONDS, tags = Map(FetcherTypeKey -> "follower")) + val consumerExpiredRequestMeter = newMeter("ExpiresPerSec", "requests", TimeUnit.SECONDS, tags = Map(FetcherTypeKey -> "consumer")) +} + diff --git a/core/src/main/scala/kafka/server/DelayedOperation.scala b/core/src/main/scala/kafka/server/DelayedOperation.scala index 1d11099a72d16..e317676b4dd5b 100644 --- a/core/src/main/scala/kafka/server/DelayedOperation.scala +++ b/core/src/main/scala/kafka/server/DelayedOperation.scala @@ -70,6 +70,11 @@ abstract class DelayedOperation(delayMs: Long) extends DelayedItem(delayMs) { */ def isCompleted(): Boolean = completed.get() + /** + * Call-back to execute when a delayed operation expires, but before completion. + */ + def onExpiration(): Unit + /** * Process for completing an operation; This function needs to be defined * in subclasses and will be called exactly once in forceComplete() @@ -89,7 +94,7 @@ abstract class DelayedOperation(delayMs: Long) extends DelayedItem(delayMs) { /** * A helper purgatory class for bookkeeping delayed operations with a timeout, and expiring timed out operations. */ -class DelayedOperationPurgatory[T <: DelayedOperation](brokerId: Int = 0, purgeInterval: Int = 1000) +class DelayedOperationPurgatory[T <: DelayedOperation](purgatoryName: String, brokerId: Int = 0, purgeInterval: Int = 1000) extends Logging with KafkaMetricsGroup { /* a list of operation watching keys */ @@ -98,18 +103,22 @@ class DelayedOperationPurgatory[T <: DelayedOperation](brokerId: Int = 0, purgeI /* background thread expiring operations that have timed out */ private val expirationReaper = new ExpiredOperationReaper + private val metricsTags = Map("delayedOperation" -> purgatoryName) + newGauge( "PurgatorySize", new Gauge[Int] { def value = watched() - } + }, + metricsTags ) newGauge( "NumDelayedOperations", new Gauge[Int] { def value = delayed() - } + }, + metricsTags ) expirationReaper.start() @@ -283,9 +292,12 @@ class DelayedOperationPurgatory[T <: DelayedOperation](brokerId: Int = 0, purgeI val curr = delayedQueue.poll(200L, TimeUnit.MILLISECONDS) if (curr != null.asInstanceOf[T]) { // if there is an expired operation, try to force complete it - if (curr synchronized curr.forceComplete()) { - debug("Force complete expired delayed operation %s".format(curr)) + val completedByMe: Boolean = curr synchronized { + curr.onExpiration() + curr.forceComplete() } + if (completedByMe) + debug("Force complete expired delayed operation %s".format(curr)) } } diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index c229088eb4f3d..4d763bf05efb2 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -18,9 +18,14 @@ package kafka.server +import java.util.concurrent.TimeUnit + +import com.yammer.metrics.core.Meter import kafka.api.ProducerResponseStatus import kafka.common.ErrorMapping import kafka.common.TopicAndPartition +import kafka.metrics.KafkaMetricsGroup +import kafka.utils.Pool import scala.collection._ @@ -110,6 +115,14 @@ class DelayedProduce(delayMs: Long, false } + override def onExpiration() { + produceMetadata.produceStatus.foreach { case (topicPartition, status) => + if (status.acksPending) { + DelayedProduceMetrics.recordExpiration(topicPartition) + } + } + } + /** * Upon completion, return the current response status along with the error code per partition */ @@ -118,3 +131,21 @@ class DelayedProduce(delayMs: Long, responseCallback(responseStatus) } } + +object DelayedProduceMetrics extends KafkaMetricsGroup { + + private val aggregateExpirationMeter = newMeter("ExpiresPerSec", "requests", TimeUnit.SECONDS) + + private val partitionExpirationMeterFactory = (key: TopicAndPartition) => + newMeter("ExpiresPerSec", + "requests", + TimeUnit.SECONDS, + tags = Map("topic" -> key.topic, "partition" -> key.partition.toString)) + private val partitionExpirationMeters = new Pool[TopicAndPartition, Meter](valueFactory = Some(partitionExpirationMeterFactory)) + + def recordExpiration(partition: TopicAndPartition) { + aggregateExpirationMeter.mark() + partitionExpirationMeters.getAndMaybePut(partition).mark() + } +} + diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index b82ff55e1dd1f..586cf4caa95f5 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -82,8 +82,10 @@ class ReplicaManager(val config: KafkaConfig, this.logIdent = "[Replica Manager on Broker " + localBrokerId + "]: " val stateChangeLogger = KafkaController.stateChangeLogger - val delayedProducePurgatory = new DelayedOperationPurgatory[DelayedProduce](config.brokerId, config.producerPurgatoryPurgeIntervalRequests) - val delayedFetchPurgatory = new DelayedOperationPurgatory[DelayedFetch](config.brokerId, config.fetchPurgatoryPurgeIntervalRequests) + val delayedProducePurgatory = new DelayedOperationPurgatory[DelayedProduce]( + purgatoryName = "Produce", config.brokerId, config.producerPurgatoryPurgeIntervalRequests) + val delayedFetchPurgatory = new DelayedOperationPurgatory[DelayedFetch]( + purgatoryName = "Fetch", config.brokerId, config.fetchPurgatoryPurgeIntervalRequests) newGauge( @@ -392,6 +394,7 @@ class ReplicaManager(val config: KafkaConfig, fetchInfo: Map[TopicAndPartition, PartitionFetchInfo], responseCallback: Map[TopicAndPartition, FetchResponsePartitionData] => Unit) { + val isFromFollower = replicaId >= 0 val fetchOnlyFromLeader: Boolean = replicaId != Request.DebuggingConsumerId val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(replicaId) @@ -421,7 +424,7 @@ class ReplicaManager(val config: KafkaConfig, val fetchPartitionStatus = logReadResults.map { case (topicAndPartition, result) => (topicAndPartition, FetchPartitionStatus(result.info.fetchOffset, fetchInfo.get(topicAndPartition).get)) } - val fetchMetadata = FetchMetadata(fetchMinBytes, fetchOnlyFromLeader, fetchOnlyCommitted, fetchPartitionStatus) + val fetchMetadata = FetchMetadata(fetchMinBytes, 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 diff --git a/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala b/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala index 93f52d3222fc1..7a37617395b9e 100644 --- a/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala +++ b/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala @@ -28,7 +28,7 @@ class DelayedOperationTest extends JUnit3Suite { override def setUp() { super.setUp() - purgatory = new DelayedOperationPurgatory[MockDelayedOperation](0, 5) + purgatory = new DelayedOperationPurgatory[MockDelayedOperation](purgatoryName = "mock", 0, 5) } override def tearDown() { @@ -114,6 +114,10 @@ class DelayedOperationTest extends JUnit3Suite { false } + override def onExpiration() { + + } + override def onComplete() { synchronized { notify() From ef252dea704192a8055367d22e45f673f5b06e94 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Thu, 26 Feb 2015 15:03:12 -0800 Subject: [PATCH 141/491] kafka-1881; transient unit test failure in testDeleteTopicWithCleaner due to OOME; patched by Ewen Cheslack-Postava; reviewed by Jun Rao --- core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala index 0cbd72684a0cc..c8f336aa034ab 100644 --- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala @@ -230,6 +230,8 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness { brokerConfigs(0).setProperty("log.cleanup.policy","compact") brokerConfigs(0).setProperty("log.segment.bytes","100") brokerConfigs(0).setProperty("log.segment.delete.delay.ms","1000") + brokerConfigs(0).setProperty("log.cleaner.dedupe.buffer.size","1048577") + val servers = createTestTopicAndCluster(topic,brokerConfigs) // for simplicity, we are validating cleaner offsets on a single broker From e39373d16bc208128d5d6cf0e0672f27f35e3b20 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Thu, 26 Feb 2015 18:02:19 -0800 Subject: [PATCH 142/491] kafka-1400; transient unit test failure in SocketServerTest; patched by Jun Rao; reviewed by Ewen Cheslack-Postava and Jiangjie Qin --- .../src/test/scala/unit/kafka/network/SocketServerTest.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 78b431f9c88cc..0af23abf146d9 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -139,8 +139,11 @@ class SocketServerTest extends JUnitSuite { processRequest(server.requestChannel) // then shutdown the server server.shutdown() + + val largeChunkOfBytes = new Array[Byte](1000000) // doing a subsequent send should throw an exception as the connection should be closed. - sendRequest(socket, 0, bytes) + // send a large chunk of bytes to trigger a socket flush + sendRequest(socket, 0, largeChunkOfBytes) } @Test From a864172da53db009607989e2d7a6c0ff1c600995 Mon Sep 17 00:00:00 2001 From: Neha Narkhede Date: Fri, 27 Feb 2015 14:11:21 -0800 Subject: [PATCH 143/491] Deleting the ConsumerTest until the issue with the hanging test is resolved; discussed on the mailing list and got several +1s --- .../integration/kafka/api/ConsumerTest.scala | 289 ------------------ 1 file changed, 289 deletions(-) delete mode 100644 core/src/test/scala/integration/kafka/api/ConsumerTest.scala diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala deleted file mode 100644 index 2802a399bf599..0000000000000 --- a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala +++ /dev/null @@ -1,289 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ -package kafka.api - -import org.apache.kafka.common.KafkaException -import org.apache.kafka.clients.producer.ProducerConfig -import org.apache.kafka.clients.producer.ProducerRecord -import org.apache.kafka.clients.consumer.Consumer -import org.apache.kafka.clients.consumer.KafkaConsumer -import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback -import org.apache.kafka.clients.consumer.ConsumerRecord -import org.apache.kafka.clients.consumer.ConsumerConfig -import org.apache.kafka.clients.consumer.CommitType -import org.apache.kafka.common.serialization.ByteArrayDeserializer -import org.apache.kafka.common.TopicPartition -import org.apache.kafka.clients.consumer.NoOffsetForPartitionException -import scala.collection.mutable.Buffer -import scala.collection.JavaConversions._ -import java.util.ArrayList -import java.util.Arrays -import org.junit.Assert._ -import kafka.utils.TestUtils -import kafka.utils.Logging -import kafka.server.OffsetManager - -/** - * Integration tests for the new consumer that cover basic usage as well as server failures - */ -class ConsumerTest extends IntegrationTestHarness with Logging { - - val producerCount = 1 - val consumerCount = 2 - val serverCount = 3 - - val topic = "topic" - val part = 0 - val tp = new TopicPartition(topic, part) - - // configure the servers and clients - this.serverConfig.setProperty("controlled.shutdown.enable", "false") // speed up shutdown - this.serverConfig.setProperty("offsets.topic.replication.factor", "3") // don't want to lose offset - this.serverConfig.setProperty("offsets.topic.num.partitions", "1") - this.producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "all") - this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "my-test") - this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 4096.toString) - this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") - - override def setUp() { - super.setUp() - // this will trigger the creation of the consumer offsets topic - this.consumers(0).partitionsFor(OffsetManager.OffsetsTopicName) - } - - def testSimpleConsumption() { - val numRecords = 10000 - sendRecords(numRecords) - - assertEquals(0, this.consumers(0).subscriptions.size) - this.consumers(0).subscribe(tp) - assertEquals(1, this.consumers(0).subscriptions.size) - - this.consumers(0).seek(tp, 0) - consumeRecords(this.consumers(0), numRecords = numRecords, startingOffset = 0) - } - - def testAutoOffsetReset() { - sendRecords(1) - this.consumers(0).subscribe(tp) - consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0) - } - - def testSeek() { - val consumer = this.consumers(0) - val totalRecords = 50L - sendRecords(totalRecords.toInt) - consumer.subscribe(tp) - - consumer.seekToEnd(tp) - assertEquals(totalRecords, consumer.position(tp)) - assertFalse(consumer.poll(totalRecords).iterator().hasNext()) - - consumer.seekToBeginning(tp) - assertEquals(0, consumer.position(tp), 0) - consumeRecords(consumer, numRecords = 1, startingOffset = 0) - - val mid = totalRecords / 2 - consumer.seek(tp, mid) - assertEquals(mid, consumer.position(tp)) - consumeRecords(consumer, numRecords = 1, startingOffset = mid.toInt) - } - - def testGroupConsumption() { - // we need to do this test with only one server since we have the hack join group - // that just assigns the partition hosted on the local machine (with two we might get the wrong machine - this.servers.last.shutdown() - this.servers.head.shutdown() - sendRecords(10) - this.consumers(0).subscribe(topic) - consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0) - } - - def testPositionAndCommit() { - sendRecords(5) - - // committed() on a partition with no committed offset throws an exception - intercept[NoOffsetForPartitionException] { - this.consumers(0).committed(new TopicPartition(topic, 15)) - } - - // position() on a partition that we aren't subscribed to throws an exception - intercept[IllegalArgumentException] { - this.consumers(0).position(new TopicPartition(topic, 15)) - } - - this.consumers(0).subscribe(tp) - - assertEquals("position() on a partition that we are subscribed to should reset the offset", 0L, this.consumers(0).position(tp)) - this.consumers(0).commit(CommitType.SYNC) - assertEquals(0L, this.consumers(0).committed(tp)) - - consumeRecords(this.consumers(0), 5, 0) - assertEquals("After consuming 5 records, position should be 5", 5L, this.consumers(0).position(tp)) - this.consumers(0).commit(CommitType.SYNC) - assertEquals("Committed offset should be returned", 5L, this.consumers(0).committed(tp)); - - sendRecords(1) - - // another consumer in the same group should get the same position - this.consumers(1).subscribe(tp) - consumeRecords(this.consumers(1), 1, 5) - } - - def testPartitionsFor() { - val numParts = 2; - TestUtils.createTopic(this.zkClient, topic, numParts, 1, this.servers) - val parts = this.consumers(0).partitionsFor(topic) - assertNotNull(parts) - assertEquals(2, parts.length) - assertNull(this.consumers(0).partitionsFor("non-existant-topic")) - } - - def testConsumptionWithBrokerFailures() = consumeWithBrokerFailures(numRecords = 1000) - - /* - * 1. Produce a bunch of messages - * 2. Then consume the messages while killing and restarting brokers at random - */ - def consumeWithBrokerFailures(numRecords: Int) { - TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) - sendRecords(numRecords) - this.producers.map(_.close) - var consumed = 0 - val consumer = this.consumers(0) - consumer.subscribe(topic) - while (consumed < numRecords) { - // check that we are getting the messages in order - for (record <- consumer.poll(200)) { - assertEquals(consumed.toLong, record.offset()) - consumed += 1 - } - consumer.commit(CommitType.SYNC); - - /* restart any dead brokers, and kill a broker (with probability 1/3) */ - restartDeadBrokers() - if (TestUtils.random.nextInt(3) == 0) { - info("Killing broker") - killRandomBroker() - } - } - } - - def testSeekAndCommitWithBrokerFailures() = seekAndCommitWithBrokerFailures(20) - - def seekAndCommitWithBrokerFailures(numIters: Int) { - // create a topic and send it some data - val numRecords = 1000 - TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers) - sendRecords(numRecords) - this.producers.map(_.close) - - val consumer = this.consumers(0) - consumer.subscribe(tp) - consumer.seek(tp, 0) - for (iter <- 0 until numIters) { - val coin = TestUtils.random.nextInt(4) - if (coin == 0) { - info("Seeking to end of log") - consumer.seekToEnd() - assertEquals(1000.toLong, consumer.position(tp)) - } else if (coin == 1) { - val pos = TestUtils.random.nextInt(numRecords).toLong - info("Seeking to " + pos) - consumer.seek(tp, pos) - assertEquals(pos, consumer.position(tp)) - } else if (coin == 2) { - info("Committing offset.") - consumer.commit(CommitType.SYNC) - assertEquals(consumer.position(tp), consumer.committed(tp)) - } else { - restartDeadBrokers() - killRandomBroker() - } - } - } - - def testPartitionReassignmentCallback() { - val callback = new TestConsumerReassignmentCallback() - this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "200"); // timeout quickly to avoid slow test - val consumer0 = new KafkaConsumer(this.consumerConfig, callback, new ByteArrayDeserializer(), new ByteArrayDeserializer()) - consumer0.subscribe("test") - - // the initial subscription should cause a callback execution - while(callback.callsToAssigned == 0) - consumer0.poll(50) - - // get metadata for the topic - var parts = consumer0.partitionsFor(OffsetManager.OffsetsTopicName) - while(parts == null) - parts = consumer0.partitionsFor(OffsetManager.OffsetsTopicName) - assertEquals(1, parts.size) - assertNotNull(parts(0).leader()) - - // shutdown the coordinator - val coordinator = parts(0).leader().id() - this.servers(coordinator).shutdown() - - // this should cause another callback execution - while(callback.callsToAssigned < 2) - consumer0.poll(50) - assertEquals(2, callback.callsToAssigned) - assertEquals(2, callback.callsToRevoked) - - // restart the coordinator since it may also be hosting "test" topic - this.servers(coordinator).startup() - - consumer0.close() - } - - class TestConsumerReassignmentCallback extends ConsumerRebalanceCallback { - var callsToAssigned = 0 - var callsToRevoked = 0 - def onPartitionsAssigned(consumer: Consumer[_,_], partitions: java.util.Collection[TopicPartition]) { - info("onPartitionsAssigned called.") - callsToAssigned += 1 - } - def onPartitionsRevoked(consumer: Consumer[_,_], partitions: java.util.Collection[TopicPartition]) { - info("onPartitionsRevoked called.") - callsToRevoked += 1 - } - } - - private def sendRecords(numRecords: Int) { - val futures = (0 until numRecords).map { i => - this.producers(0).send(new ProducerRecord(topic, part, i.toString.getBytes, i.toString.getBytes)) - } - futures.map(_.get) - } - - private def consumeRecords(consumer: Consumer[Array[Byte], Array[Byte]], numRecords: Int, startingOffset: Int) { - val records = new ArrayList[ConsumerRecord[Array[Byte], Array[Byte]]]() - val maxIters = numRecords * 300 - var iters = 0 - while (records.size < numRecords) { - for (record <- consumer.poll(50)) - records.add(record) - if(iters > maxIters) - throw new IllegalStateException("Failed to consume the expected records after " + iters + " iterations."); - iters += 1 - } - for (i <- 0 until numRecords) { - val record = records.get(i) - val offset = startingOffset + i - assertEquals(topic, record.topic()) - assertEquals(part, record.partition()) - assertEquals(offset.toLong, record.offset()) - } - } - -} \ No newline at end of file From b56f5973c739072350f3f6bf6efa4eb05bc692bf Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Fri, 27 Feb 2015 15:09:41 -0800 Subject: [PATCH 144/491] KAFKA-1664 Kafka does not properly parse multiple ZK nodes with non-root chroot; reviewed by Neha Narkhede and Jun Rao --- core/src/main/scala/kafka/utils/ZkUtils.scala | 46 ++++-- .../test/scala/unit/kafka/zk/ZKPathTest.scala | 147 ++++++++++++++++++ 2 files changed, 183 insertions(+), 10 deletions(-) create mode 100644 core/src/test/scala/unit/kafka/zk/ZKPathTest.scala diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 8a2fb2d9a42ae..7ae999ec61944 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -23,6 +23,7 @@ import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException, ZkMarshallingError, ZkBadVersionException} import org.I0Itec.zkclient.serialize.ZkSerializer +import org.apache.kafka.common.config.ConfigException import collection._ import kafka.api.LeaderAndIsr import org.apache.zookeeper.data.Stat @@ -212,7 +213,7 @@ object ZkUtils extends Logging { */ def makeSurePersistentPathExists(client: ZkClient, path: String) { if (!client.exists(path)) - client.createPersistent(path, true) // won't throw NoNodeException or NodeExistsException + new ZkPath(client).createPersistent(path, true) // won't throw NoNodeException or NodeExistsException } /** @@ -220,20 +221,22 @@ object ZkUtils extends Logging { */ private def createParentPath(client: ZkClient, path: String): Unit = { val parentDir = path.substring(0, path.lastIndexOf('/')) - if (parentDir.length != 0) - client.createPersistent(parentDir, true) + if (parentDir.length != 0) { + new ZkPath(client).createPersistent(parentDir, true) + } } /** * Create an ephemeral node with the given path and data. Create parents if necessary. */ private def createEphemeralPath(client: ZkClient, path: String, data: String): Unit = { + val zkPath = new ZkPath(client) try { - client.createEphemeral(path, data) + zkPath.createEphemeral(path, data) } catch { case e: ZkNoNodeException => { createParentPath(client, path) - client.createEphemeral(path, data) + zkPath.createEphemeral(path, data) } } } @@ -312,18 +315,19 @@ object ZkUtils extends Logging { * Create an persistent node with the given path and data. Create parents if necessary. */ def createPersistentPath(client: ZkClient, path: String, data: String = ""): Unit = { + val zkPath = new ZkPath(client) try { - client.createPersistent(path, data) + zkPath.createPersistent(path, data) } catch { case e: ZkNoNodeException => { createParentPath(client, path) - client.createPersistent(path, data) + zkPath.createPersistent(path, data) } } } def createSequentialPersistentPath(client: ZkClient, path: String, data: String = ""): String = { - client.createPersistentSequential(path, data) + new ZkPath(client).createPersistentSequential(path, data) } /** @@ -338,7 +342,7 @@ object ZkUtils extends Logging { case e: ZkNoNodeException => { createParentPath(client, path) try { - client.createPersistent(path, data) + new ZkPath(client).createPersistent(path, data) } catch { case e: ZkNodeExistsException => client.writeData(path, data) @@ -409,7 +413,7 @@ object ZkUtils extends Logging { } catch { case e: ZkNoNodeException => { createParentPath(client, path) - client.createEphemeral(path, data) + new ZkPath(client).createEphemeral(path, data) } case e2: Throwable => throw e2 } @@ -806,3 +810,25 @@ class ZKConfig(props: VerifiableProperties) { /** how far a ZK follower can be behind a ZK leader */ val zkSyncTimeMs = props.getInt("zookeeper.sync.time.ms", 2000) } + +class ZkPath(client: ZkClient) { + if (!client.exists("/")) { + throw new ConfigException("Zookeeper namespace does not exist") + } + + def createPersistent(path: String, data: Object) { + client.createPersistent(path, data) + } + + def createPersistent(path: String, createParents: Boolean) { + client.createPersistent(path, createParents) + } + + def createEphemeral(path: String, data: Object) { + client.createEphemeral(path, data) + } + + def createPersistentSequential(path: String, data: Object): String = { + client.createPersistentSequential(path, data) + } +} diff --git a/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala b/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala new file mode 100644 index 0000000000000..9897b2fa8f826 --- /dev/null +++ b/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala @@ -0,0 +1,147 @@ +/** + * 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 unit.kafka.zk + +import junit.framework.Assert +import kafka.consumer.ConsumerConfig +import kafka.utils.{TestUtils, ZKStringSerializer, ZkUtils} +import kafka.zk.ZooKeeperTestHarness +import org.I0Itec.zkclient.ZkClient +import org.apache.kafka.common.config.ConfigException +import org.scalatest.junit.JUnit3Suite + +class ZKPathTest extends JUnit3Suite with ZooKeeperTestHarness { + + val path: String = "/some_dir" + val zkSessionTimeoutMs = 1000 + val zkConnectWithInvalidRoot: String = zkConnect + "/ghost" + + def testCreatePersistentPathThrowsException { + val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot, + "test", "1")) + var zkClient = new ZkClient(zkConnectWithInvalidRoot, zkSessionTimeoutMs, + config.zkConnectionTimeoutMs, + ZKStringSerializer) + try { + ZkUtils.createPersistentPath(zkClient, path) + fail("Failed to throw ConfigException for missing zookeeper root node") + } catch { + case configException: ConfigException => + case exception: Throwable => fail("Should have thrown ConfigException") + } + } + + def testCreatePersistentPath { + val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) + var zkClient = new ZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, + ZKStringSerializer) + try { + ZkUtils.createPersistentPath(zkClient, path) + } catch { + case exception: Throwable => fail("Failed to create persistent path") + } + + Assert.assertTrue("Failed to create persistent path", ZkUtils.pathExists(zkClient, path)); + } + + def testMakeSurePersistsPathExistsThrowsException { + val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot, + "test", "1")) + var zkClient = new ZkClient(zkConnectWithInvalidRoot, zkSessionTimeoutMs, + config.zkConnectionTimeoutMs, + ZKStringSerializer) + try { + ZkUtils.makeSurePersistentPathExists(zkClient, path) + fail("Failed to throw ConfigException for missing zookeeper root node") + } catch { + case configException: ConfigException => + case exception: Throwable => fail("Should have thrown ConfigException") + } + } + + def testMakeSurePersistsPathExists { + val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) + var zkClient = new ZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, + ZKStringSerializer) + try { + ZkUtils.makeSurePersistentPathExists(zkClient, path) + } catch { + case exception: Throwable => fail("Failed to create persistent path") + } + + Assert.assertTrue("Failed to create persistent path", ZkUtils.pathExists(zkClient, path)); + } + + def testCreateEphemeralPathThrowsException { + val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot, + "test", "1")) + var zkClient = new ZkClient(zkConnectWithInvalidRoot, zkSessionTimeoutMs, + config.zkConnectionTimeoutMs, + ZKStringSerializer) + try { + ZkUtils.createEphemeralPathExpectConflict(zkClient, path, "somedata") + fail("Failed to throw ConfigException for missing zookeeper root node") + } catch { + case configException: ConfigException => + case exception: Throwable => fail("Should have thrown ConfigException") + } + } + + def testCreateEphemeralPathExists { + val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) + var zkClient = new ZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, + ZKStringSerializer) + try { + ZkUtils.createEphemeralPathExpectConflict(zkClient, path, "somedata") + } catch { + case exception: Throwable => fail("Failed to create ephemeral path") + } + + Assert.assertTrue("Failed to create ephemeral path", ZkUtils.pathExists(zkClient, path)); + } + + def testCreatePersistentSequentialThrowsException { + val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot, + "test", "1")) + var zkClient = new ZkClient(zkConnectWithInvalidRoot, zkSessionTimeoutMs, + config.zkConnectionTimeoutMs, + ZKStringSerializer) + try { + ZkUtils.createSequentialPersistentPath(zkClient, path) + fail("Failed to throw ConfigException for missing zookeeper root node") + } catch { + case configException: ConfigException => + case exception: Throwable => fail("Should have thrown ConfigException") + } + } + + def testCreatePersistentSequentialExists { + val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1")) + var zkClient = new ZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, + ZKStringSerializer) + + var actualPath: String = "" + try { + actualPath = ZkUtils.createSequentialPersistentPath(zkClient, path) + } catch { + case exception: Throwable => fail("Failed to create persistent path") + } + + Assert.assertTrue("Failed to create persistent path", ZkUtils.pathExists(zkClient, actualPath)); + } +} From 687abc98a4600bf90ed7a7acb6fb2a5e6eac2055 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Fri, 27 Feb 2015 16:10:37 -0800 Subject: [PATCH 145/491] KAFKA-1866 LogStartOffset gauge throws exceptions after log.delete(); reviewed by Neha Narkhede --- .../main/scala/kafka/cluster/Partition.scala | 11 ++++++- core/src/main/scala/kafka/log/Log.scala | 12 ++++++- .../unit/kafka/metrics/MetricsTest.scala | 33 ++++++++++++++++--- .../scala/unit/kafka/utils/TestUtils.scala | 4 +-- 4 files changed, 52 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index bfe4f45486b19..c4bf48a801007 100644 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -60,6 +60,7 @@ class Partition(val topic: String, this.logIdent = "Partition [%s,%d] on broker %d: ".format(topic, partitionId, localBrokerId) private def isReplicaLocal(replicaId: Int) : Boolean = (replicaId == localBrokerId) + val tags = Map("topic" -> topic, "partition" -> partitionId.toString) newGauge("UnderReplicated", new Gauge[Int] { @@ -67,7 +68,7 @@ class Partition(val topic: String, if (isUnderReplicated) 1 else 0 } }, - Map("topic" -> topic, "partition" -> partitionId.toString) + tags ) def isUnderReplicated(): Boolean = { @@ -141,6 +142,7 @@ class Partition(val topic: String, leaderReplicaIdOpt = None try { logManager.deleteLog(TopicAndPartition(topic, partitionId)) + removePartitionMetrics() } catch { case e: IOException => fatal("Error deleting the log for partition [%s,%d]".format(topic, partitionId), e) @@ -436,6 +438,13 @@ class Partition(val topic: String, } } + /** + * remove deleted log metrics + */ + private def removePartitionMetrics() { + removeMetric("UnderReplicated", tags) + } + override def equals(that: Any): Boolean = { if(!(that.isInstanceOf[Partition])) return false diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 846023bb98d0f..0c4efa8399d70 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -624,6 +624,7 @@ class Log(val dir: File, */ private[log] def delete() { lock synchronized { + removeLogMetrics() logSegments.foreach(_.delete()) segments.clear() Utils.rm(dir) @@ -769,7 +770,16 @@ class Log(val dir: File, newSegment.changeFileSuffixes(Log.SwapFileSuffix, "") } } - + + /** + * remove deleted log metrics + */ + private[log] def removeLogMetrics(): Unit = { + removeMetric("NumLogSegments", tags) + removeMetric("LogStartOffset", tags) + removeMetric("LogEndOffset", tags) + removeMetric("Size", tags) + } /** * Add the given segment to the segments in this log. If this segment replaces an existing segment, delete it. * @param segment The segment to add diff --git a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala index 3cf23b3d6d446..111e4a26c1efb 100644 --- a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala +++ b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala @@ -18,15 +18,20 @@ package kafka.consumer import com.yammer.metrics.Metrics +import com.yammer.metrics.core.MetricPredicate +import org.junit.Test import junit.framework.Assert._ import kafka.integration.KafkaServerTestHarness import kafka.server._ -import scala.collection._ -import org.scalatest.junit.JUnit3Suite import kafka.message._ import kafka.serializer._ import kafka.utils._ +import kafka.admin.AdminUtils import kafka.utils.TestUtils._ +import scala.collection._ +import scala.collection.JavaConversions._ +import scala.util.matching.Regex +import org.scalatest.junit.JUnit3Suite class MetricsTest extends JUnit3Suite with KafkaServerTestHarness with Logging { val zookeeperConnect = TestZKUtils.zookeeperConnect @@ -34,7 +39,7 @@ class MetricsTest extends JUnit3Suite with KafkaServerTestHarness with Logging { val numParts = 2 val topic = "topic1" val configs = - for (props <- TestUtils.createBrokerConfigs(numNodes)) + for (props <- TestUtils.createBrokerConfigs(numNodes, enableDeleteTopic=true)) yield new KafkaConfig(props) { override val zkConnect = zookeeperConnect override val numPartitions = numParts @@ -45,6 +50,7 @@ class MetricsTest extends JUnit3Suite with KafkaServerTestHarness with Logging { super.tearDown() } + @Test def testMetricsLeak() { // create topic topic1 with 1 partition on broker 0 createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = servers) @@ -59,6 +65,15 @@ class MetricsTest extends JUnit3Suite with KafkaServerTestHarness with Logging { } } + @Test + def testMetricsReporterAfterDeletingTopic() { + val topic = "test-topic-metric" + AdminUtils.createTopic(zkClient, topic, 1, 1) + AdminUtils.deleteTopic(zkClient, topic) + TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers) + assertFalse("Topic metrics exists after deleteTopic", checkTopicMetricsExists(topic)) + } + def createAndShutdownStep(group: String, consumerId: String, producerId: String): Unit = { val sentMessages1 = sendMessages(configs, topic, producerId, nMessages, "batch1", NoCompressionCodec, 1) // create a consumer @@ -69,4 +84,14 @@ class MetricsTest extends JUnit3Suite with KafkaServerTestHarness with Logging { zkConsumerConnector1.shutdown() } -} \ No newline at end of file + + private def checkTopicMetricsExists(topic: String): Boolean = { + val topicMetricRegex = new Regex(".*("+topic+")$") + val metricGroups = Metrics.defaultRegistry().groupedMetrics(MetricPredicate.ALL).entrySet() + for(metricGroup <- metricGroups) { + if (topicMetricRegex.pattern.matcher(metricGroup.getKey()).matches) + return true + } + false + } +} diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 21d0ed2cb7c94..32b2899010bac 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -801,9 +801,9 @@ object TestUtils extends Logging { val topicAndPartitions = (0 until numPartitions).map(TopicAndPartition(topic, _)) // wait until admin path for delete topic is deleted, signaling completion of topic deletion TestUtils.waitUntilTrue(() => !ZkUtils.pathExists(zkClient, ZkUtils.getDeleteTopicPath(topic)), - "Admin path /admin/delete_topic/test path not deleted even after a replica is restarted") + "Admin path /admin/delete_topic/%s path not deleted even after a replica is restarted".format(topic)) TestUtils.waitUntilTrue(() => !ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic)), - "Topic path /brokers/topics/test not deleted after /admin/delete_topic/test path is deleted") + "Topic path /brokers/topics/%s not deleted after /admin/delete_topic/%s path is deleted".format(topic, topic)) // ensure that the topic-partition has been deleted from all brokers' replica managers TestUtils.waitUntilTrue(() => servers.forall(server => topicAndPartitions.forall(tp => server.replicaManager.getPartition(tp.topic, tp.partition) == None)), From 22ff9e943a960a73d2f007b2c52b90e675a61299 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Sat, 28 Feb 2015 07:41:27 -0800 Subject: [PATCH 146/491] KAFKA-1824 ConsoleProducer - properties key.separator and parse.key no longer work; reviewed by Neha Narkhede --- .../scala/kafka/tools/ConsoleProducer.scala | 112 +++++++++++------- .../kafka/tools/ConsoleProducerTest.scala | 79 ++++++++++++ 2 files changed, 148 insertions(+), 43 deletions(-) create mode 100644 core/src/test/scala/kafka/tools/ConsoleProducerTest.scala diff --git a/core/src/main/scala/kafka/tools/ConsoleProducer.scala b/core/src/main/scala/kafka/tools/ConsoleProducer.scala index 7fefc2ed6fed1..00265f9f4a4b6 100644 --- a/core/src/main/scala/kafka/tools/ConsoleProducer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleProducer.scala @@ -27,6 +27,7 @@ import java.util.Properties import java.io._ import joptsimple._ +import org.apache.kafka.clients.producer.ProducerConfig object ConsoleProducer { @@ -34,54 +35,14 @@ object ConsoleProducer { val config = new ProducerConfig(args) val reader = Class.forName(config.readerClass).newInstance().asInstanceOf[MessageReader] - val props = new Properties - props.putAll(config.cmdLineProps) - props.put("topic", config.topic) - reader.init(System.in, props) + reader.init(System.in, getReaderProps(config)) try { val producer = if(config.useNewProducer) { - import org.apache.kafka.clients.producer.ProducerConfig - - props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, config.brokerList) - props.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, config.compressionCodec) - props.put(ProducerConfig.SEND_BUFFER_CONFIG, config.socketBuffer.toString) - props.put(ProducerConfig.RETRY_BACKOFF_MS_CONFIG, config.retryBackoffMs.toString) - props.put(ProducerConfig.METADATA_MAX_AGE_CONFIG, config.metadataExpiryMs.toString) - props.put(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG, config.metadataFetchTimeoutMs.toString) - props.put(ProducerConfig.ACKS_CONFIG, config.requestRequiredAcks.toString) - props.put(ProducerConfig.TIMEOUT_CONFIG, config.requestTimeoutMs.toString) - props.put(ProducerConfig.RETRIES_CONFIG, config.messageSendMaxRetries.toString) - props.put(ProducerConfig.LINGER_MS_CONFIG, config.sendTimeout.toString) - if(config.queueEnqueueTimeoutMs != -1) - props.put(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "false") - props.put(ProducerConfig.BUFFER_MEMORY_CONFIG, config.maxMemoryBytes.toString) - props.put(ProducerConfig.BATCH_SIZE_CONFIG, config.maxPartitionMemoryBytes.toString) - props.put(ProducerConfig.CLIENT_ID_CONFIG, "console-producer") - props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") - props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") - - new NewShinyProducer(props) + new NewShinyProducer(getNewProducerProps(config)) } else { - props.put("metadata.broker.list", config.brokerList) - props.put("compression.codec", config.compressionCodec) - props.put("producer.type", if(config.sync) "sync" else "async") - props.put("batch.num.messages", config.batchSize.toString) - props.put("message.send.max.retries", config.messageSendMaxRetries.toString) - props.put("retry.backoff.ms", config.retryBackoffMs.toString) - props.put("queue.buffering.max.ms", config.sendTimeout.toString) - props.put("queue.buffering.max.messages", config.queueSize.toString) - props.put("queue.enqueue.timeout.ms", config.queueEnqueueTimeoutMs.toString) - props.put("request.required.acks", config.requestRequiredAcks.toString) - props.put("request.timeout.ms", config.requestTimeoutMs.toString) - props.put("key.serializer.class", config.keyEncoderClass) - props.put("serializer.class", config.valueEncoderClass) - props.put("send.buffer.bytes", config.socketBuffer.toString) - props.put("topic.metadata.refresh.interval.ms", config.metadataExpiryMs.toString) - props.put("client.id", "console-producer") - - new OldProducer(props) + new OldProducer(getOldProducerProps(config)) } Runtime.getRuntime.addShutdownHook(new Thread() { @@ -104,6 +65,66 @@ object ConsoleProducer { System.exit(0) } + def getReaderProps(config: ProducerConfig): Properties = { + val props = new Properties + props.put("topic",config.topic) + props.putAll(config.cmdLineProps) + props + } + + def getOldProducerProps(config: ProducerConfig): Properties = { + + val props = new Properties; + + props.putAll(config.extraProducerProps) + + props.put("metadata.broker.list", config.brokerList) + props.put("compression.codec", config.compressionCodec) + props.put("producer.type", if(config.sync) "sync" else "async") + props.put("batch.num.messages", config.batchSize.toString) + props.put("message.send.max.retries", config.messageSendMaxRetries.toString) + props.put("retry.backoff.ms", config.retryBackoffMs.toString) + props.put("queue.buffering.max.ms", config.sendTimeout.toString) + props.put("queue.buffering.max.messages", config.queueSize.toString) + props.put("queue.enqueue.timeout.ms", config.queueEnqueueTimeoutMs.toString) + props.put("request.required.acks", config.requestRequiredAcks.toString) + props.put("request.timeout.ms", config.requestTimeoutMs.toString) + props.put("key.serializer.class", config.keyEncoderClass) + props.put("serializer.class", config.valueEncoderClass) + props.put("send.buffer.bytes", config.socketBuffer.toString) + props.put("topic.metadata.refresh.interval.ms", config.metadataExpiryMs.toString) + props.put("client.id", "console-producer") + + props + } + + def getNewProducerProps(config: ProducerConfig): Properties = { + + val props = new Properties; + + props.putAll(config.extraProducerProps) + + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, config.brokerList) + props.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, config.compressionCodec) + props.put(ProducerConfig.SEND_BUFFER_CONFIG, config.socketBuffer.toString) + props.put(ProducerConfig.RETRY_BACKOFF_MS_CONFIG, config.retryBackoffMs.toString) + props.put(ProducerConfig.METADATA_MAX_AGE_CONFIG, config.metadataExpiryMs.toString) + props.put(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG, config.metadataFetchTimeoutMs.toString) + props.put(ProducerConfig.ACKS_CONFIG, config.requestRequiredAcks.toString) + props.put(ProducerConfig.TIMEOUT_CONFIG, config.requestTimeoutMs.toString) + props.put(ProducerConfig.RETRIES_CONFIG, config.messageSendMaxRetries.toString) + props.put(ProducerConfig.LINGER_MS_CONFIG, config.sendTimeout.toString) + if(config.queueEnqueueTimeoutMs != -1) + props.put(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "false") + props.put(ProducerConfig.BUFFER_MEMORY_CONFIG, config.maxMemoryBytes.toString) + props.put(ProducerConfig.BATCH_SIZE_CONFIG, config.maxPartitionMemoryBytes.toString) + props.put(ProducerConfig.CLIENT_ID_CONFIG, "console-producer") + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") + + props + } + class ProducerConfig(args: Array[String]) { val parser = new OptionParser val topicOpt = parser.accepts("topic", "REQUIRED: The topic id to produce messages to.") @@ -211,6 +232,10 @@ object ConsoleProducer { .withRequiredArg .describedAs("prop") .ofType(classOf[String]) + val producerPropertyOpt = parser.accepts("producer-property", "A mechanism to pass user-defined properties in the form key=value to the producer. ") + .withRequiredArg + .describedAs("producer_prop") + .ofType(classOf[String]) val useNewProducerOpt = parser.accepts("new-producer", "Use the new producer implementation.") val options = parser.parse(args : _*) @@ -243,6 +268,7 @@ object ConsoleProducer { val readerClass = options.valueOf(messageReaderOpt) val socketBuffer = options.valueOf(socketBufferSizeOpt) val cmdLineProps = CommandLineUtils.parseKeyValueArgs(options.valuesOf(propertyOpt)) + val extraProducerProps = CommandLineUtils.parseKeyValueArgs(options.valuesOf(producerPropertyOpt)) /* new producer related configs */ val maxMemoryBytes = options.valueOf(maxMemoryBytesOpt) val maxPartitionMemoryBytes = options.valueOf(maxPartitionMemoryBytesOpt) diff --git a/core/src/test/scala/kafka/tools/ConsoleProducerTest.scala b/core/src/test/scala/kafka/tools/ConsoleProducerTest.scala new file mode 100644 index 0000000000000..6d1f51c866e23 --- /dev/null +++ b/core/src/test/scala/kafka/tools/ConsoleProducerTest.scala @@ -0,0 +1,79 @@ +/* + * 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.tools + +import kafka.producer +import kafka.tools.ConsoleProducer.{LineMessageReader, MessageReader,ProducerConfig} +import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig} +import joptsimple.UnrecognizedOptionException +import org.junit.Assert +import org.junit.Test + + +class ConsoleProducerTest { + + val validArgs: Array[String] = Array( + "--broker-list", + "localhost:1001,localhost:1002", + "--topic", + "t3", + "--property", + "parse.key=true", + "--property", + "key.separator=#" + ) + + val invalidArgs: Array[String] = Array( + "--t", // not a valid argument + "t3" + ) + + @Test + def testValidConfigsNewProducer() { + val config = new ConsoleProducer.ProducerConfig(validArgs) + // New ProducerConfig constructor is package private, so we can't call it directly + // Creating new Producer to validate instead + new KafkaProducer[Array[Byte],Array[Byte]](ConsoleProducer.getNewProducerProps(config)) + } + + @Test + def testValidConfigsOldProducer() { + val config = new ConsoleProducer.ProducerConfig(validArgs) + new producer.ProducerConfig(ConsoleProducer.getOldProducerProps(config)); + } + + @Test + def testInvalidConfigs() { + try { + val config = new ConsoleProducer.ProducerConfig(invalidArgs) + Assert.fail("Should have thrown an UnrecognizedOptionException") + } catch { + case e: joptsimple.OptionException => // expected exception + } + } + + @Test + def testParseKeyProp(): Unit = { + val config = new ConsoleProducer.ProducerConfig(validArgs) + val reader = Class.forName(config.readerClass).newInstance().asInstanceOf[LineMessageReader]; + reader.init(System.in,ConsoleProducer.getReaderProps(config)) + assert(reader.keySeparator == "#") + assert(reader.parseKey == true) + } + +} From 0636928d961a6ceaab46d908f9372d913c3e5faf Mon Sep 17 00:00:00 2001 From: Jay Kreps Date: Sat, 7 Feb 2015 12:01:51 -0800 Subject: [PATCH 147/491] KAFKA-1865 Add a flush() method to the producer. --- .../org/apache/kafka/clients/Metadata.java | 10 +- .../kafka/clients/producer/KafkaProducer.java | 187 ++++++++++++++---- .../kafka/clients/producer/MockProducer.java | 5 + .../kafka/clients/producer/Producer.java | 5 + .../clients/producer/ProducerRecord.java | 20 +- .../internals/FutureRecordMetadata.java | 10 +- .../producer/internals/RecordAccumulator.java | 77 +++++++- .../producer/internals/RecordBatch.java | 13 +- .../common/errors/InterruptException.java | 34 ++++ .../apache/kafka/common/utils/SystemTime.java | 2 +- .../clients/{producer => }/MetadataTest.java | 16 +- .../clients/producer/MockProducerTest.java | 6 + .../{ => internals}/BufferPoolTest.java | 6 +- .../{ => internals}/PartitionerTest.java | 3 +- .../RecordAccumulatorTest.java | 27 ++- .../producer/{ => internals}/SenderTest.java | 5 +- .../kafka/api/ProducerSendTest.scala | 62 ++++-- .../scala/unit/kafka/utils/TestUtils.scala | 4 +- 18 files changed, 393 insertions(+), 99 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/InterruptException.java rename clients/src/test/java/org/apache/kafka/clients/{producer => }/MetadataTest.java (91%) rename clients/src/test/java/org/apache/kafka/clients/producer/{ => internals}/BufferPoolTest.java (97%) rename clients/src/test/java/org/apache/kafka/clients/producer/{ => internals}/PartitionerTest.java (96%) rename clients/src/test/java/org/apache/kafka/clients/producer/{ => internals}/RecordAccumulatorTest.java (89%) rename clients/src/test/java/org/apache/kafka/clients/producer/{ => internals}/SenderTest.java (97%) diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java index e8afecda95630..c8bde8b732def 100644 --- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -99,19 +99,15 @@ public synchronized int requestUpdate() { /** * Wait for metadata update until the current version is larger than the last version we know of */ - public synchronized void awaitUpdate(final int lastVersion, final long maxWaitMs) { + public synchronized void awaitUpdate(final int lastVersion, final long maxWaitMs) throws InterruptedException { if (maxWaitMs < 0) { throw new IllegalArgumentException("Max time to wait for metadata updates should not be < 0 milli seconds"); } long begin = System.currentTimeMillis(); long remainingWaitMs = maxWaitMs; while (this.version <= lastVersion) { - try { - if (remainingWaitMs != 0) { - wait(remainingWaitMs); - } - } catch (InterruptedException e) { /* this is fine */ - } + if (remainingWaitMs != 0) + wait(remainingWaitMs); long elapsed = System.currentTimeMillis() - begin; if (elapsed >= maxWaitMs) throw new TimeoutException("Failed to update metadata after " + maxWaitMs + " ms."); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 1fd6917c8a513..7397e565fd865 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -32,6 +32,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.errors.ApiException; +import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.errors.RecordTooLargeException; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.errors.TimeoutException; @@ -55,10 +56,66 @@ /** * A Kafka client that publishes records to the Kafka cluster. *

- * The producer is thread safe and should generally be shared among all threads for best performance. + * The producer is thread safe and sharing a single producer instance across threads will generally be faster than + * having multiple instances. *

- * The producer manages a single background thread that does I/O as well as a TCP connection to each of the brokers it - * needs to communicate with. Failure to close the producer after use will leak these resources. + * Here is a simple example of using the producer to send records with strings containing sequential numbers as the key/value + * pairs. + *

+ * {@code
+ * Properties props = new Properties();
+ * props.put("bootstrap.servers", "localhost:4242");
+ * props.put("acks", "all");
+ * props.put("retries", 0);
+ * props.put("batch.size", 16384);
+ * props.put("linger.ms", 1);
+ * props.put("buffer.memory", 33554432);
+ * props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
+ * props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
+ * 
+ * Producer producer = new KafkaProducer(props);
+ * for(int i = 0; i < 100; i++)
+ *     producer.send(new ProducerRecord("my-topic", Integer.toString(i), Integer.toString(i)));
+ * 
+ * producer.close();
+ * }
+ *

+ * The producer consists of a pool of buffer space that holds records that haven't yet been transmitted to the server + * as well as a background I/O thread that is responsible for turning these records into requests and transmitting them + * to the cluster. Failure to close the producer after use will leak these resources. + *

+ * The {@link #send(ProducerRecord) send()} method is asynchronous. When called it adds the record to a buffer of pending record sends + * and immediately returns. This allows the producer to batch together individual records for efficiency. + *

+ * The acks config controls the criteria under which requests are considered complete. The "all" setting + * we have specified will result in blocking on the full commit of the record, the slowest but most durable setting. + *

+ * If the request fails, the producer can automatically retry, though since we have specified retries + * as 0 it won't. Enabling retries also opens up the possibility of duplicates (see the documentation on + * message delivery semantics for details). + *

+ * The producer maintains buffers of unsent records for each partition. These buffers are of a size specified by + * the batch.size config. Making this larger can result in more batching, but requires more memory (since we will + * generally have one of these buffers for each active partition). + *

+ * By default a buffer is available to send immediately even if there is additional unused space in the buffer. However if you + * want to reduce the number of requests you can set linger.ms to something greater than 0. This will + * instruct the producer to wait up to that number of milliseconds before sending a request in hope that more records will + * arrive to fill up the same batch. This is analogous to Nagle's algorithm in TCP. For example, in the code snippet above, + * likely all 100 records would be sent in a single request since we set our linger time to 1 millisecond. However this setting + * would add 1 millisecond of latency to our request waiting for more records to arrive if we didn't fill up the buffer. Note that + * records that arrive close together in time will generally batch together even with linger.ms=0 so under heavy load + * batching will occur regardless of the linger configuration; however setting this to something larger than 0 can lead to fewer, more + * efficient requests when not under maximal load at the cost of a small amount of latency. + *

+ * The buffer.memory controls the total amount of memory available to the producer for buffering. If records + * are sent faster than they can be transmitted to the server then this buffer space will be exhausted. When the buffer space is + * exhausted additional send calls will block. For uses where you want to avoid any blocking you can set block.on.buffer.full=false which + * will cause the send call to result in an exception. + *

+ * The key.serializer and value.serializer instruct how to turn the key and value objects the user provides with + * their ProducerRecord into bytes. You can use the included {@link org.apache.kafka.common.serialization.ByteArraySerializer} or + * {@link org.apache.kafka.common.serialization.StringSerializer} for simple string or byte types. */ public class KafkaProducer implements Producer { @@ -241,8 +298,8 @@ private static int parseAcks(String acksString) { } /** - * Asynchronously send a record to a topic. Equivalent to {@link #send(ProducerRecord, Callback) send(record, null)} - * @param record The record to be sent + * Asynchronously send a record to a topic. Equivalent to send(record, null). + * See {@link #send(ProducerRecord, Callback)} for details. */ @Override public Future send(ProducerRecord record) { @@ -261,53 +318,59 @@ public Future send(ProducerRecord record) { *

* Since the send call is asynchronous it returns a {@link java.util.concurrent.Future Future} for the * {@link RecordMetadata} that will be assigned to this record. Invoking {@link java.util.concurrent.Future#get() - * get()} on this future will result in the metadata for the record or throw any exception that occurred while - * sending the record. + * get()} on this future will block until the associated request completes and then return the metadata for the record + * or throw any exception that occurred while sending the record. *

- * If you want to simulate a simple blocking call you can do the following: + * If you want to simulate a simple blocking call you can call the get() method immediately: * - *

{@code
-     * producer.send(new ProducerRecord("the-topic", "key".getBytes(), "value".getBytes())).get();
+     * 
+     * {@code
+     * byte[] key = "key".getBytes();
+     * byte[] value = "value".getBytes();
+     * ProducerRecord record = new ProducerRecord("my-topic", key, value)
+     * producer.send(record).get();
      * }
*

- * Those desiring fully non-blocking usage can make use of the {@link Callback} parameter to provide a callback that + * Fully non-blocking usage can make use of the {@link Callback} parameter to provide a callback that * will be invoked when the request is complete. * - *

{@code
-     * ProducerRecord record = new ProducerRecord("the-topic", "key".getBytes(), "value".getBytes());
-     *   producer.send(myRecord,
-     *                new Callback() {
-     *                     public void onCompletion(RecordMetadata metadata, Exception e) {
-     *                         if(e != null)
-     *                             e.printStackTrace();
-     *                         System.out.println("The offset of the record we just sent is: " + metadata.offset());
-     *                     }
-     *                });
-     * }
+ *
+     * {@code
+     * ProducerRecord record = new ProducerRecord("the-topic", key, value);
+     * producer.send(myRecord,
+     *               new Callback() {
+     *                   public void onCompletion(RecordMetadata metadata, Exception e) {
+     *                       if(e != null)
+     *                           e.printStackTrace();
+     *                       System.out.println("The offset of the record we just sent is: " + metadata.offset());
+     *                   }
+     *               });
+     * }
+     * 
* * Callbacks for records being sent to the same partition are guaranteed to execute in order. That is, in the * following example callback1 is guaranteed to execute before callback2: * - *
{@code
+     * 
+     * {@code
      * producer.send(new ProducerRecord(topic, partition, key1, value1), callback1);
      * producer.send(new ProducerRecord(topic, partition, key2, value2), callback2);
-     * }
+ * } + *
*

* Note that callbacks will generally execute in the I/O thread of the producer and so should be reasonably fast or * they will delay the sending of messages from other threads. If you want to execute blocking or computationally * expensive callbacks it is recommended to use your own {@link java.util.concurrent.Executor} in the callback body * to parallelize processing. - *

- * The producer manages a buffer of records waiting to be sent. This buffer has a hard limit on it's size, which is - * controlled by the configuration total.memory.bytes. If send() is called faster than the - * I/O thread can transfer data to the brokers the buffer will eventually run out of space. The default behavior in - * this case is to block the send call until the I/O thread catches up and more buffer space is available. However - * in cases where non-blocking usage is desired the setting block.on.buffer.full=false will cause the - * producer to instead throw an exception when buffer memory is exhausted. * * @param record The record to send * @param callback A user-supplied callback to execute when the record has been acknowledged by the server (null * indicates no callback) + * + * @throws InterruptException If the thread is interrupted while blocked + * @throws SerializationException If the key or value are not valid objects given the configured serializers + * @throws BufferExhaustedException If block.on.buffer.full=false and the buffer is full. + * */ @Override public Future send(ProducerRecord record, Callback callback) { @@ -352,7 +415,7 @@ public Future send(ProducerRecord record, Callback callbac return new FutureFailure(e); } catch (InterruptedException e) { this.errors.record(); - throw new KafkaException(e); + throw new InterruptException(e); } catch (KafkaException e) { this.errors.record(); throw e; @@ -364,7 +427,7 @@ public Future send(ProducerRecord record, Callback callbac * @param topic The topic we want metadata for * @param maxWaitMs The maximum time in ms for waiting on the metadata */ - private void waitOnMetadata(String topic, long maxWaitMs) { + private void waitOnMetadata(String topic, long maxWaitMs) throws InterruptedException { if (metadata.fetch().partitionsForTopic(topic) != null) { return; } else { @@ -399,20 +462,72 @@ private void ensureValidRecordSize(int size) { ProducerConfig.BUFFER_MEMORY_CONFIG + " configuration."); } + + /** + * Invoking this method makes all buffered records immediately available to send (even if linger.ms is + * greater than 0) and blocks on the completion of the requests associated with these records. The post-condition + * of flush() is that any previously sent record will have completed (e.g. Future.isDone() == true). + * A request is considered completed when it is successfully acknowledged + * according to the acks configuration you have specified or else it results in an error. + *

+ * Other threads can continue sending records while one thread is blocked waiting for a flush call to complete, + * however no guarantee is made about the completion of records sent after the flush call begins. + *

+ * This method can be useful when consuming from some input system and producing into Kafka. The flush() call + * gives a convenient way to ensure all previously sent messages have actually completed. + *

+ * This example shows how to consume from one Kafka topic and produce to another Kafka topic: + *

+     * {@code
+     * for(ConsumerRecord record: consumer.poll(100))
+     *     producer.send(new ProducerRecord("my-topic", record.key(), record.value());
+     * producer.flush();
+     * consumer.commit();
+     * }
+     * 
+ * + * Note that the above example may drop records if the produce request fails. If we want to ensure that this does not occur + * we need to set retries=<large_number> in our config. + * + * @throws InterruptException If the thread is interrupted while blocked + */ + @Override + public void flush() { + log.trace("Flushing accumulated records in producer."); + this.accumulator.beginFlush(); + this.sender.wakeup(); + try { + this.accumulator.awaitFlushCompletion(); + } catch (InterruptedException e) { + throw new InterruptException("Flush interrupted.", e); + } + } + /** + * Get the partition metadata for the give topic. This can be used for custom partitioning. + * @throws InterruptException If the thread is interrupted while blocked + */ @Override public List partitionsFor(String topic) { - waitOnMetadata(topic, this.metadataFetchTimeoutMs); + try { + waitOnMetadata(topic, this.metadataFetchTimeoutMs); + } catch (InterruptedException e) { + throw new InterruptException(e); + } return this.metadata.fetch().partitionsForTopic(topic); } + /** + * Get the full set of internal metrics maintained by the producer. + */ @Override public Map metrics() { return Collections.unmodifiableMap(this.metrics.metrics()); } /** - * Close this producer. This method blocks until all in-flight requests complete. + * Close this producer. This method blocks until all previously sent requests complete. + * @throws InterruptException If the thread is interrupted while blocked */ @Override public void close() { @@ -421,7 +536,7 @@ public void close() { try { this.ioThread.join(); } catch (InterruptedException e) { - throw new KafkaException(e); + throw new InterruptException(e); } this.metrics.close(); this.keySerializer.close(); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java index 84530f2b948f9..6913090af03a4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java @@ -128,6 +128,11 @@ private long nextOffset(TopicPartition tp) { return offset; } } + + public synchronized void flush() { + while (!this.completions.isEmpty()) + completeNext(); + } public List partitionsFor(String topic) { return this.cluster.partitionsForTopic(topic); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java index 17fe541588d46..5b3e75ed83a94 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java @@ -45,6 +45,11 @@ public interface Producer extends Closeable { * Send a record and invoke the given callback when the record has been acknowledged by the server */ public Future send(ProducerRecord record, Callback callback); + + /** + * Flush any accumulated records from the producer. Blocks until all sends are complete. + */ + public void flush(); /** * Get a list of partitions for the given topic for custom partition assignment. The partition metadata will change diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java index 4990692efa6f0..75cd51e6d9a8a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java @@ -102,15 +102,21 @@ public String toString() { @Override public boolean equals(Object o) { - if (this == o) return true; - if (!(o instanceof ProducerRecord)) return false; + if (this == o) + return true; + else if (!(o instanceof ProducerRecord)) + return false; - ProducerRecord that = (ProducerRecord) o; + ProducerRecord that = (ProducerRecord) o; - if (key != null ? !key.equals(that.key) : that.key != null) return false; - if (partition != null ? !partition.equals(that.partition) : that.partition != null) return false; - if (topic != null ? !topic.equals(that.topic) : that.topic != null) return false; - if (value != null ? !value.equals(that.value) : that.value != null) return false; + if (key != null ? !key.equals(that.key) : that.key != null) + return false; + else if (partition != null ? !partition.equals(that.partition) : that.partition != null) + return false; + else if (topic != null ? !topic.equals(that.topic) : that.topic != null) + return false; + else if (value != null ? !value.equals(that.value) : that.value != null) + return false; return true; } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java index 4a2da41f47994..e2d9ca87ad01b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java @@ -51,13 +51,17 @@ public RecordMetadata get(long timeout, TimeUnit unit) throws InterruptedExcepti return valueOrError(); } - private RecordMetadata valueOrError() throws ExecutionException { + RecordMetadata valueOrError() throws ExecutionException { if (this.result.error() != null) throw new ExecutionException(this.result.error()); else - return new RecordMetadata(result.topicPartition(), this.result.baseOffset(), this.relativeOffset); + return value(); } - + + RecordMetadata value() { + return new RecordMetadata(result.topicPartition(), this.result.baseOffset(), this.relativeOffset); + } + public long relativeOffset() { return this.relativeOffset; } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index ecfe2144d778a..d5c79e2481d5e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -23,6 +23,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.common.Cluster; @@ -55,6 +56,7 @@ public final class RecordAccumulator { private static final Logger log = LoggerFactory.getLogger(RecordAccumulator.class); private volatile boolean closed; + private volatile AtomicInteger flushesInProgress; private int drainIndex; private final int batchSize; private final long lingerMs; @@ -62,6 +64,7 @@ public final class RecordAccumulator { private final BufferPool free; private final Time time; private final ConcurrentMap> batches; + private final IncompleteRecordBatches incomplete; /** * Create a new record accumulator @@ -89,12 +92,14 @@ public RecordAccumulator(int batchSize, Map metricTags) { this.drainIndex = 0; this.closed = false; + this.flushesInProgress = new AtomicInteger(0); this.batchSize = batchSize; this.lingerMs = lingerMs; this.retryBackoffMs = retryBackoffMs; this.batches = new CopyOnWriteMap>(); String metricGrpName = "producer-metrics"; this.free = new BufferPool(totalSize, batchSize, blockOnBufferFull, metrics, time , metricGrpName , metricTags); + this.incomplete = new IncompleteRecordBatches(); this.time = time; registerMetrics(metrics, metricGrpName, metricTags); } @@ -146,9 +151,8 @@ public RecordAppendResult append(TopicPartition tp, byte[] key, byte[] value, Co RecordBatch last = dq.peekLast(); if (last != null) { FutureRecordMetadata future = last.tryAppend(key, value, callback); - if (future != null) { + if (future != null) return new RecordAppendResult(future, dq.size() > 1 || last.records.isFull(), false); - } } } @@ -161,8 +165,7 @@ public RecordAppendResult append(TopicPartition tp, byte[] key, byte[] value, Co if (last != null) { FutureRecordMetadata future = last.tryAppend(key, value, callback); if (future != null) { - // Somebody else found us a batch, return the one we waited for! Hopefully this doesn't happen - // often... + // Somebody else found us a batch, return the one we waited for! Hopefully this doesn't happen often... free.deallocate(buffer); return new RecordAppendResult(future, dq.size() > 1 || last.records.isFull(), false); } @@ -172,6 +175,7 @@ public RecordAppendResult append(TopicPartition tp, byte[] key, byte[] value, Co FutureRecordMetadata future = Utils.notNull(batch.tryAppend(key, value, callback)); dq.addLast(batch); + incomplete.add(batch); return new RecordAppendResult(future, dq.size() > 1 || batch.records.isFull(), true); } } @@ -226,7 +230,7 @@ public ReadyCheckResult ready(Cluster cluster, long nowMs) { long timeLeftMs = Math.max(timeToWaitMs - waitedTimeMs, 0); boolean full = deque.size() > 1 || batch.records.isFull(); boolean expired = waitedTimeMs >= timeToWaitMs; - boolean sendable = full || expired || exhausted || closed; + boolean sendable = full || expired || exhausted || closed || flushInProgress(); if (sendable && !backingOff) { readyNodes.add(leader); } else { @@ -266,7 +270,6 @@ public boolean hasUnsent() { * @param maxSize The maximum number of bytes to drain * @param now The current unix time in milliseconds * @return A list of {@link RecordBatch} for each node specified with total size less than the requested maxSize. - * TODO: There may be a starvation issue due to iteration order */ public Map> drain(Cluster cluster, Set nodes, int maxSize, long now) { if (nodes.isEmpty()) @@ -324,8 +327,32 @@ private Deque dequeFor(TopicPartition tp) { * Deallocate the record batch */ public void deallocate(RecordBatch batch) { + incomplete.remove(batch); free.deallocate(batch.records.buffer(), batch.records.capacity()); } + + /** + * Are there any threads currently waiting on a flush? + */ + private boolean flushInProgress() { + return flushesInProgress.get() > 0; + } + + /** + * Initiate the flushing of data from the accumulator...this makes all requests immediately ready + */ + public void beginFlush() { + this.flushesInProgress.getAndIncrement(); + } + + /** + * Mark all partitions as ready to send and block until the send is complete + */ + public void awaitFlushCompletion() throws InterruptedException { + for (RecordBatch batch: this.incomplete.all()) + batch.produceFuture.await(); + this.flushesInProgress.decrementAndGet(); + } /** * Close this accumulator and force all the record buffers to be drained @@ -334,7 +361,9 @@ public void close() { this.closed = true; } - + /* + * Metadata about a record just appended to the record accumulator + */ public final static class RecordAppendResult { public final FutureRecordMetadata future; public final boolean batchIsFull; @@ -347,6 +376,9 @@ public RecordAppendResult(FutureRecordMetadata future, boolean batchIsFull, bool } } + /* + * The set of nodes that have at least one complete record batch in the accumulator + */ public final static class ReadyCheckResult { public final Set readyNodes; public final long nextReadyCheckDelayMs; @@ -358,4 +390,35 @@ public ReadyCheckResult(Set readyNodes, long nextReadyCheckDelayMs, boolea this.unknownLeadersExist = unknownLeadersExist; } } + + /* + * A threadsafe helper class to hold RecordBatches that haven't been ack'd yet + */ + private final static class IncompleteRecordBatches { + private final Set incomplete; + + public IncompleteRecordBatches() { + this.incomplete = new HashSet(); + } + + public void add(RecordBatch batch) { + synchronized (incomplete) { + this.incomplete.add(batch); + } + } + + public void remove(RecordBatch batch) { + synchronized (incomplete) { + boolean removed = this.incomplete.remove(batch); + if (!removed) + throw new IllegalStateException("Remove from the incomplete set failed. This should be impossible."); + } + } + + public Iterable all() { + synchronized (incomplete) { + return new ArrayList(this.incomplete); + } + } + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java index dd0af8aee98ab..06182db1c3a5d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java @@ -16,6 +16,7 @@ import java.util.List; import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Record; @@ -39,7 +40,7 @@ public final class RecordBatch { public long lastAttemptMs; public final MemoryRecords records; public final TopicPartition topicPartition; - private final ProduceRequestResult produceFuture; + public final ProduceRequestResult produceFuture; private final List thunks; public RecordBatch(TopicPartition tp, MemoryRecords records, long now) { @@ -77,7 +78,6 @@ public FutureRecordMetadata tryAppend(byte[] key, byte[] value, Callback callbac * @param exception The exception that occurred (or null if the request was successful) */ public void done(long baseOffset, RuntimeException exception) { - this.produceFuture.done(topicPartition, baseOffset, exception); log.trace("Produced messages to topic-partition {} with base offset offset {} and error: {}.", topicPartition, baseOffset, @@ -86,14 +86,17 @@ public void done(long baseOffset, RuntimeException exception) { for (int i = 0; i < this.thunks.size(); i++) { try { Thunk thunk = this.thunks.get(i); - if (exception == null) - thunk.callback.onCompletion(thunk.future.get(), null); - else + if (exception == null) { + RecordMetadata metadata = new RecordMetadata(this.topicPartition, baseOffset, thunk.future.relativeOffset()); + thunk.callback.onCompletion(metadata, null); + } else { thunk.callback.onCompletion(null, exception); + } } catch (Exception e) { log.error("Error executing user-provided callback on message for topic-partition {}:", topicPartition, e); } } + this.produceFuture.done(topicPartition, baseOffset, exception); } /** diff --git a/clients/src/main/java/org/apache/kafka/common/errors/InterruptException.java b/clients/src/main/java/org/apache/kafka/common/errors/InterruptException.java new file mode 100644 index 0000000000000..fee322fa0dd97 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/InterruptException.java @@ -0,0 +1,34 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.common.errors; + +import org.apache.kafka.common.KafkaException; + +/** + * An unchecked wrapper for InterruptedException + */ +public class InterruptException extends KafkaException { + + private static final long serialVersionUID = 1L; + + public InterruptException(InterruptedException cause) { + super(cause); + Thread.currentThread().interrupt(); + } + + public InterruptException(String message, InterruptedException cause) { + super(message, cause); + Thread.currentThread().interrupt(); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java b/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java index d682bd46ec382..18725de836838 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java @@ -36,7 +36,7 @@ public void sleep(long ms) { try { Thread.sleep(ms); } catch (InterruptedException e) { - // no stress + // just wake up early } } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java similarity index 91% rename from clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java rename to clients/src/test/java/org/apache/kafka/clients/MetadataTest.java index 743aa7e523dd4..928087d29deb8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java @@ -10,12 +10,14 @@ * 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.clients.producer; +package org.apache.kafka.clients; + +import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.kafka.clients.Metadata; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.test.TestUtils; +import org.junit.After; import org.junit.Test; import static org.junit.Assert.*; @@ -25,6 +27,12 @@ public class MetadataTest { private long refreshBackoffMs = 100; private long metadataExpireMs = 1000; private Metadata metadata = new Metadata(refreshBackoffMs, metadataExpireMs); + private AtomicBoolean backgroundError = new AtomicBoolean(false); + + @After + public void tearDown() { + assertFalse(backgroundError.get()); + } @Test public void testMetadata() throws Exception { @@ -83,8 +91,8 @@ public void run() { while (metadata.fetch().partitionsForTopic(topic) == null) { try { metadata.awaitUpdate(metadata.requestUpdate(), refreshBackoffMs); - } catch (TimeoutException e) { - // let it go + } catch (Exception e) { + backgroundError.set(true); } } } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java index 75513b0bdd439..6372f1a7f7f77 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java @@ -67,6 +67,12 @@ public void testManualCompletion() throws Exception { assertEquals(e, err.getCause()); } assertFalse("No more requests to complete", producer.completeNext()); + + Future md3 = producer.send(record1); + Future md4 = producer.send(record2); + assertTrue("Requests should not be completed.", !md3.isDone() && !md4.isDone()); + producer.flush(); + assertTrue("Requests should be completed.", md3.isDone() && md4.isDone()); } private boolean isError(Future future) { diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java similarity index 97% rename from clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java rename to clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java index 4ae43ed47e31a..2c693824fa53d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/BufferPoolTest.java @@ -14,9 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.clients.producer; +package org.apache.kafka.clients.producer.internals; -import org.apache.kafka.clients.producer.internals.BufferPool; +import org.apache.kafka.clients.producer.BufferExhaustedException; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.test.TestUtils; @@ -146,7 +146,7 @@ public void testStressfulSituation() throws Exception { int numThreads = 10; final int iterations = 50000; final int poolableSize = 1024; - final int totalMemory = numThreads / 2 * poolableSize; + final long totalMemory = numThreads / 2 * poolableSize; final BufferPool pool = new BufferPool(totalMemory, poolableSize, true, metrics, time, metricGroup, metricTags); List threads = new ArrayList(); for (int i = 0; i < numThreads; i++) diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/PartitionerTest.java similarity index 96% rename from clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java rename to clients/src/test/java/org/apache/kafka/clients/producer/internals/PartitionerTest.java index 404bedb3dc4e4..5dadd0e355457 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/PartitionerTest.java @@ -10,7 +10,7 @@ * 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.clients.producer; +package org.apache.kafka.clients.producer.internals; import static java.util.Arrays.asList; import static org.junit.Assert.assertEquals; @@ -18,7 +18,6 @@ import java.util.List; -import org.apache.kafka.clients.producer.internals.Partitioner; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java similarity index 89% rename from clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java rename to clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java index 83338633717cf..c1bc40648479d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java @@ -10,7 +10,7 @@ * 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.clients.producer; +package org.apache.kafka.clients.producer.internals; import static java.util.Arrays.asList; import static org.junit.Assert.assertEquals; @@ -27,8 +27,6 @@ import java.util.Map; import java.util.Set; -import org.apache.kafka.clients.producer.internals.RecordAccumulator; -import org.apache.kafka.clients.producer.internals.RecordBatch; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; @@ -203,5 +201,28 @@ public void testNextReadyCheckDelay() throws Exception { // but have leaders with other sendable data. assertTrue("Next check time should be defined by node2, at most linger time", result.nextReadyCheckDelayMs <= lingerMs); } + + @Test + public void testFlush() throws Exception { + long lingerMs = Long.MAX_VALUE; + final RecordAccumulator accum = new RecordAccumulator(4 * 1024, 64 * 1024, lingerMs, 100L, false, metrics, time, metricTags); + for (int i = 0; i < 100; i++) + accum.append(new TopicPartition(topic, i % 3), key, value, CompressionType.NONE, null); + RecordAccumulator.ReadyCheckResult result = accum.ready(cluster, time.milliseconds()); + assertEquals("No nodes should be ready.", 0, result.readyNodes.size()); + + accum.beginFlush(); + result = accum.ready(cluster, time.milliseconds()); + + // drain and deallocate all batches + Map> results = accum.drain(cluster, result.readyNodes, Integer.MAX_VALUE, time.milliseconds()); + for (List batches: results.values()) + for (RecordBatch batch: batches) + accum.deallocate(batch); + + // should be complete with no unsent records. + accum.awaitFlushCompletion(); + assertFalse(accum.hasUnsent()); + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java similarity index 97% rename from clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java rename to clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index 558942aaecd1b..ea56c997cb90d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -10,7 +10,7 @@ * 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.clients.producer; +package org.apache.kafka.clients.producer.internals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -23,8 +23,7 @@ import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.MockClient; -import org.apache.kafka.clients.producer.internals.RecordAccumulator; -import org.apache.kafka.clients.producer.internals.Sender; +import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.Metrics; diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala index b15237b76def3..8154a4210dc8d 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package kafka.api.test +package kafka.api import java.lang.{Integer, IllegalArgumentException} @@ -27,7 +27,6 @@ import org.junit.Assert._ import kafka.server.KafkaConfig import kafka.utils.{TestZKUtils, TestUtils} import kafka.consumer.SimpleConsumer -import kafka.api.FetchRequestBuilder import kafka.message.Message import kafka.integration.KafkaServerTestHarness import org.apache.kafka.common.errors.SerializationException @@ -66,13 +65,6 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { super.tearDown() } - class CheckErrorCallback extends Callback { - def onCompletion(metadata: RecordMetadata, exception: Exception) { - if (exception != null) - fail("Send callback returns the following exception", exception) - } - } - /** * testSendOffset checks the basic send API behavior * @@ -82,23 +74,36 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { @Test def testSendOffset() { var producer = TestUtils.createNewProducer(brokerList) - - val callback = new CheckErrorCallback + val partition = new Integer(0) + + object callback extends Callback { + var offset = 0L + def onCompletion(metadata: RecordMetadata, exception: Exception) { + if (exception == null) { + assertEquals(offset, metadata.offset()) + assertEquals(topic, metadata.topic()) + assertEquals(partition, metadata.partition()) + offset += 1 + } else { + fail("Send callback returns the following exception", exception) + } + } + } try { // create topic TestUtils.createTopic(zkClient, topic, 1, 2, servers) // send a normal record - val record0 = new ProducerRecord[Array[Byte],Array[Byte]](topic, new Integer(0), "key".getBytes, "value".getBytes) + val record0 = new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, "key".getBytes, "value".getBytes) assertEquals("Should have offset 0", 0L, producer.send(record0, callback).get.offset) // send a record with null value should be ok - val record1 = new ProducerRecord[Array[Byte],Array[Byte]](topic, new Integer(0), "key".getBytes, null) + val record1 = new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, "key".getBytes, null) assertEquals("Should have offset 1", 1L, producer.send(record1, callback).get.offset) // send a record with null key should be ok - val record2 = new ProducerRecord[Array[Byte],Array[Byte]](topic, new Integer(0), null, "value".getBytes) + val record2 = new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, null, "value".getBytes) assertEquals("Should have offset 2", 2L, producer.send(record2, callback).get.offset) // send a record with null part id should be ok @@ -107,7 +112,7 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { // send a record with null topic should fail try { - val record4 = new ProducerRecord[Array[Byte],Array[Byte]](null, new Integer(0), "key".getBytes, "value".getBytes) + val record4 = new ProducerRecord[Array[Byte],Array[Byte]](null, partition, "key".getBytes, "value".getBytes) producer.send(record4, callback) fail("Should not allow sending a record without topic") } catch { @@ -117,7 +122,7 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { // non-blocking send a list of records for (i <- 1 to numRecords) - producer.send(record0) + producer.send(record0, callback) // check that all messages have been acked via offset assertEquals("Should have offset " + (numRecords + 4), numRecords + 4L, producer.send(record0, callback).get.offset) @@ -235,7 +240,7 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { val responses = for (i <- 1 to numRecords) - yield producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, null, ("value" + i).getBytes)) + yield producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, null, ("value" + i).getBytes)) val futures = responses.toList futures.map(_.get) for (future <- futures) @@ -294,4 +299,27 @@ class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness { } } } + + /** + * Test that flush immediately sends all accumulated requests. + */ + @Test + def testFlush() { + var producer = TestUtils.createNewProducer(brokerList, lingerMs = Long.MaxValue) + try { + TestUtils.createTopic(zkClient, topic, 2, 2, servers) + val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, "value".getBytes) + for(i <- 0 until 50) { + val responses = (0 until numRecords) map (i => producer.send(record)) + assertTrue("No request is complete.", responses.forall(!_.isDone())) + producer.flush() + assertTrue("All requests are complete.", responses.forall(_.isDone())) + } + } finally { + if (producer != null) + producer.close() + } + } + + } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 32b2899010bac..6ce18076f6b5d 100644 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -387,7 +387,8 @@ object TestUtils extends Logging { metadataFetchTimeout: Long = 3000L, blockOnBufferFull: Boolean = true, bufferSize: Long = 1024L * 1024L, - retries: Int = 0) : KafkaProducer[Array[Byte],Array[Byte]] = { + retries: Int = 0, + lingerMs: Long = 0) : KafkaProducer[Array[Byte],Array[Byte]] = { import org.apache.kafka.clients.producer.ProducerConfig val producerProps = new Properties() @@ -399,6 +400,7 @@ object TestUtils extends Logging { producerProps.put(ProducerConfig.RETRIES_CONFIG, retries.toString) producerProps.put(ProducerConfig.RETRY_BACKOFF_MS_CONFIG, "100") producerProps.put(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG, "200") + producerProps.put(ProducerConfig.LINGER_MS_CONFIG, lingerMs.toString) producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") return new KafkaProducer[Array[Byte],Array[Byte]](producerProps) From 1cd6ed9e2c07a63474ed80a8224bd431d5d4243c Mon Sep 17 00:00:00 2001 From: Joel Koshy Date: Tue, 3 Mar 2015 10:44:04 -0800 Subject: [PATCH 148/491] KAFKA-1755; Reject compressed and unkeyed messages sent to compacted topics; reviewed by Mayuresh Gharat, Neha Narkhede and Guozhang Wang --- core/src/main/scala/kafka/log/Log.scala | 2 +- .../src/main/scala/kafka/log/LogCleaner.scala | 49 +++-- .../scala/kafka/log/LogCleanerManager.scala | 12 +- .../kafka/message/ByteBufferMessageSet.scala | 44 ++++- .../scala/kafka/server/OffsetManager.scala | 1 + .../scala/unit/kafka/log/CleanerTest.scala | 43 +++- .../test/scala/unit/kafka/log/LogTest.scala | 186 ++++++++++++------ .../message/ByteBufferMessageSetTest.scala | 4 +- 8 files changed, 235 insertions(+), 106 deletions(-) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 0c4efa8399d70..06b8ecc5d11a1 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -288,7 +288,7 @@ class Log(val dir: File, // assign offsets to the message set val offset = new AtomicLong(nextOffsetMetadata.messageOffset) try { - validMessages = validMessages.assignOffsets(offset, appendInfo.sourceCodec, appendInfo.targetCodec) + validMessages = validMessages.validateMessagesAndAssignOffsets(offset, appendInfo.sourceCodec, appendInfo.targetCodec, config.compact) } catch { case e: IOException => throw new KafkaException("Error in validating messages while appending to log '%s'".format(name), e) } diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index f8e7cd5fabce7..59914281594d5 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -133,7 +133,7 @@ class LogCleaner(val config: CleanerConfig, * Update checkpoint file, removing topics and partitions that no longer exist */ def updateCheckpoints(dataDir: File) { - cleanerManager.updateCheckpoints(dataDir, update=None); + cleanerManager.updateCheckpoints(dataDir, update=None) } /** @@ -152,8 +152,7 @@ class LogCleaner(val config: CleanerConfig, } /** - * TODO: - * For testing, a way to know when work has completed. This method blocks until the + * For testing, a way to know when work has completed. This method blocks until the * cleaner has processed up to the given offset on the specified topic/partition */ def awaitCleaned(topic: String, part: Int, offset: Long, timeout: Long = 30000L): Unit = { @@ -243,7 +242,7 @@ class LogCleaner(val config: CleanerConfig, "\tIndexed %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.mapBytesRead), stats.elapsedIndexSecs, mb(stats.mapBytesRead)/stats.elapsedIndexSecs, - 100 * stats.elapsedIndexSecs.toDouble/stats.elapsedSecs) + + 100 * stats.elapsedIndexSecs/stats.elapsedSecs) + "\tBuffer utilization: %.1f%%%n".format(100 * stats.bufferUtilization) + "\tCleaned %,.1f MB in %.1f seconds (%,.1f Mb/sec, %.1f%% of total time)%n".format(mb(stats.bytesRead), stats.elapsedSecs - stats.elapsedIndexSecs, @@ -253,6 +252,9 @@ class LogCleaner(val config: CleanerConfig, "\t%.1f%% size reduction (%.1f%% fewer messages)%n".format(100.0 * (1.0 - stats.bytesWritten.toDouble/stats.bytesRead), 100.0 * (1.0 - stats.messagesWritten.toDouble/stats.messagesRead)) info(message) + if (stats.invalidMessagesRead > 0) { + warn("\tFound %d invalid messages during compaction.".format(stats.invalidMessagesRead)) + } } } @@ -374,7 +376,7 @@ private[log] class Cleaner(val id: Int, } catch { case e: LogCleaningAbortedException => cleaned.delete() - throw e + throw e } } @@ -407,17 +409,20 @@ private[log] class Cleaner(val id: Int, position += size stats.readMessage(size) val key = entry.message.key - require(key != null, "Found null key in log segment %s which is marked as dedupe.".format(source.log.file.getAbsolutePath)) - val foundOffset = map.get(key) - /* two cases in which we can get rid of a message: - * 1) if there exists a message with the same key but higher offset - * 2) if the message is a delete "tombstone" marker and enough time has passed - */ - val redundant = foundOffset >= 0 && entry.offset < foundOffset - val obsoleteDelete = !retainDeletes && entry.message.isNull - if (!redundant && !obsoleteDelete) { - ByteBufferMessageSet.writeMessage(writeBuffer, entry.message, entry.offset) - stats.recopyMessage(size) + if (key != null) { + val foundOffset = map.get(key) + /* two cases in which we can get rid of a message: + * 1) if there exists a message with the same key but higher offset + * 2) if the message is a delete "tombstone" marker and enough time has passed + */ + val redundant = foundOffset >= 0 && entry.offset < foundOffset + val obsoleteDelete = !retainDeletes && entry.message.isNull + if (!redundant && !obsoleteDelete) { + ByteBufferMessageSet.writeMessage(writeBuffer, entry.message, entry.offset) + stats.recopyMessage(size) + } + } else { + stats.invalidMessage() } } // if any messages are to be retained, write them out @@ -536,10 +541,10 @@ private[log] class Cleaner(val id: Int, val startPosition = position for (entry <- messages) { val message = entry.message - require(message.hasKey) val size = MessageSet.entrySize(message) position += size - map.put(message.key, entry.offset) + if (message.hasKey) + map.put(message.key, entry.offset) offset = entry.offset stats.indexMessage(size) } @@ -556,7 +561,8 @@ private[log] class Cleaner(val id: Int, * A simple struct for collecting stats about log cleaning */ private case class CleanerStats(time: Time = SystemTime) { - var startTime, mapCompleteTime, endTime, bytesRead, bytesWritten, mapBytesRead, mapMessagesRead, messagesRead, messagesWritten = 0L + var startTime, mapCompleteTime, endTime, bytesRead, bytesWritten, mapBytesRead, mapMessagesRead, messagesRead, + messagesWritten, invalidMessagesRead = 0L var bufferUtilization = 0.0d clear() @@ -564,6 +570,10 @@ private case class CleanerStats(time: Time = SystemTime) { messagesRead += 1 bytesRead += size } + + def invalidMessage() { + invalidMessagesRead += 1 + } def recopyMessage(size: Int) { messagesWritten += 1 @@ -596,6 +606,7 @@ private case class CleanerStats(time: Time = SystemTime) { mapBytesRead = 0L mapMessagesRead = 0L messagesRead = 0L + invalidMessagesRead = 0L messagesWritten = 0L bufferUtilization = 0.0d } diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala index fd87d90597981..351824b3c9fab 100644 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -122,8 +122,8 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To inLock(lock) { abortAndPauseCleaning(topicAndPartition) resumeCleaning(topicAndPartition) - info("The cleaning for partition %s is aborted".format(topicAndPartition)) } + info("The cleaning for partition %s is aborted".format(topicAndPartition)) } /** @@ -152,8 +152,8 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To } while (!isCleaningInState(topicAndPartition, LogCleaningPaused)) pausedCleaningCond.await(100, TimeUnit.MILLISECONDS) - info("The cleaning for partition %s is aborted and paused".format(topicAndPartition)) } + info("The cleaning for partition %s is aborted and paused".format(topicAndPartition)) } /** @@ -181,14 +181,14 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To /** * Check if the cleaning for a partition is in a particular state. The caller is expected to hold lock while making the call. */ - def isCleaningInState(topicAndPartition: TopicAndPartition, expectedState: LogCleaningState): Boolean = { + private def isCleaningInState(topicAndPartition: TopicAndPartition, expectedState: LogCleaningState): Boolean = { inProgress.get(topicAndPartition) match { - case None => return false + case None => false case Some(state) => if (state == expectedState) - return true + true else - return false + false } } diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index f46ad5cbbbad7..9c694719dc9b5 100644 --- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -199,24 +199,48 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi } /** - * Update the offsets for this message set. This method attempts to do an in-place conversion - * if there is no compression, but otherwise recopies the messages + * Update the offsets for this message set and do further validation on messages. This method attempts to do an + * in-place conversion if there is no compression, but otherwise recopies the messages */ - private[kafka] def assignOffsets(offsetCounter: AtomicLong, sourceCodec: CompressionCodec, targetCodec: CompressionCodec): ByteBufferMessageSet = { + private[kafka] def validateMessagesAndAssignOffsets(offsetCounter: AtomicLong, + sourceCodec: CompressionCodec, + targetCodec: CompressionCodec, + compactedTopic: Boolean = false): ByteBufferMessageSet = { if(sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) { - // do an in-place conversion - var position = 0 + // do in-place validation and offset assignment + var messagePosition = 0 buffer.mark() - while(position < sizeInBytes - MessageSet.LogOverhead) { - buffer.position(position) + while(messagePosition < sizeInBytes - MessageSet.LogOverhead) { + buffer.position(messagePosition) buffer.putLong(offsetCounter.getAndIncrement()) - position += MessageSet.LogOverhead + buffer.getInt() + val messageSize = buffer.getInt() + val positionAfterKeySize = buffer.position + Message.KeySizeOffset + Message.KeySizeLength + if (compactedTopic && positionAfterKeySize < sizeInBytes) { + buffer.position(buffer.position() + Message.KeySizeOffset) + val keySize = buffer.getInt() + if (keySize <= 0) { + buffer.reset() + throw new InvalidMessageException("Compacted topic cannot accept message without key.") + } + } + messagePosition += MessageSet.LogOverhead + messageSize } buffer.reset() this } else { - // messages are compressed, crack open the messageset and recompress with correct offset - val messages = this.internalIterator(isShallow = false).map(_.message) + if (compactedTopic && targetCodec != NoCompressionCodec) + throw new InvalidMessageException("Compacted topic cannot accept compressed messages. " + + "Either the producer sent a compressed message or the topic has been configured with a broker-side compression codec.") + // We need to crack open the message-set if any of these are true: + // (i) messages are compressed, + // (ii) this message-set is sent to a compacted topic (and so we need to verify that each message has a key) + // If the broker is configured with a target compression codec then we need to recompress regardless of original codec + val messages = this.internalIterator(isShallow = false).map(messageAndOffset => { + if (compactedTopic && !messageAndOffset.message.hasKey) + throw new InvalidMessageException("Compacted topic cannot accept message without key.") + + messageAndOffset.message + }) new ByteBufferMessageSet(compressionCodec = targetCodec, offsetCounter = offsetCounter, messages = messages.toBuffer:_*) } } diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 83d52643028c5..c602a802858e2 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -168,6 +168,7 @@ class OffsetManager(val config: OffsetManagerConfig, val props = new Properties props.put(LogConfig.SegmentBytesProp, config.offsetsTopicSegmentBytes.toString) props.put(LogConfig.CleanupPolicyProp, "compact") + props.put(LogConfig.CompressionTypeProp, "none") props } diff --git a/core/src/test/scala/unit/kafka/log/CleanerTest.scala b/core/src/test/scala/unit/kafka/log/CleanerTest.scala index d10e4f4ccbca5..70f0488baa685 100644 --- a/core/src/test/scala/unit/kafka/log/CleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/CleanerTest.scala @@ -90,10 +90,42 @@ class CleanerTest extends JUnitSuite { assertTrue("None of the keys we deleted should still exist.", (0 until leo.toInt by 2).forall(!keys.contains(_))) } + + @Test + def testCleaningWithUnkeyedMessages { + val cleaner = makeCleaner(Int.MaxValue) + + // create a log with compaction turned off so we can append unkeyed messages + val log = makeLog(config = logConfig.copy(segmentSize = 1024, compact = false)) + + // append messages with unkeyed messages + while(log.numberOfSegments < 2) + log.append(unkeyedMessage(log.logEndOffset.toInt)) + val numInvalidMessages = unkeyedMessageCountInLog(log) + + val sizeWithUnkeyedMessages = log.size + + // append messages with unkeyed messages + while(log.numberOfSegments < 3) + log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt)) + + val expectedSizeAfterCleaning = log.size - sizeWithUnkeyedMessages + + // turn on compaction and compact the log + val compactedLog = makeLog(config = logConfig.copy(segmentSize = 1024)) + cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0)) + + assertEquals("Log should only contain keyed messages after cleaning.", 0, unkeyedMessageCountInLog(log)) + assertEquals("Log should only contain keyed messages after cleaning.", expectedSizeAfterCleaning, log.size) + assertEquals("Cleaner should have seen %d invalid messages.", numInvalidMessages, cleaner.stats.invalidMessagesRead) + } /* extract all the keys from a log */ - def keysInLog(log: Log): Iterable[Int] = - log.logSegments.flatMap(s => s.log.filter(!_.message.isNull).map(m => Utils.readString(m.message.key).toInt)) + def keysInLog(log: Log): Iterable[Int] = + log.logSegments.flatMap(s => s.log.filter(!_.message.isNull).filter(_.message.hasKey).map(m => Utils.readString(m.message.key).toInt)) + + def unkeyedMessageCountInLog(log: Log) = + log.logSegments.map(s => s.log.filter(!_.message.isNull).count(m => !m.message.hasKey)).sum def abortCheckDone(topicAndPartition: TopicAndPartition) { throw new LogCleaningAbortedException() @@ -130,7 +162,7 @@ class CleanerTest extends JUnitSuite { // append some messages to the log var i = 0 while(log.numberOfSegments < 10) { - log.append(TestUtils.singleMessageSet("hello".getBytes)) + log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes)) i += 1 } @@ -220,7 +252,10 @@ class CleanerTest extends JUnitSuite { def message(key: Int, value: Int) = new ByteBufferMessageSet(new Message(key=key.toString.getBytes, bytes=value.toString.getBytes)) - + + def unkeyedMessage(value: Int) = + new ByteBufferMessageSet(new Message(bytes=value.toString.getBytes)) + def deleteMessage(key: Int) = new ByteBufferMessageSet(new Message(key=key.toString.getBytes, bytes=null)) diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index c2dd8eb69da8c..1a4be70a21fe7 100644 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -62,10 +62,10 @@ class LogTest extends JUnitSuite { val set = TestUtils.singleMessageSet("test".getBytes()) // create a log - val log = new Log(logDir, - logConfig.copy(segmentMs = 1 * 60 * 60L), - recoveryPoint = 0L, - scheduler = time.scheduler, + val log = new Log(logDir, + logConfig.copy(segmentMs = 1 * 60 * 60L), + recoveryPoint = 0L, + scheduler = time.scheduler, time = time) assertEquals("Log begins with a single empty segment.", 1, log.numberOfSegments) time.sleep(log.config.segmentMs + 1) @@ -151,7 +151,7 @@ class LogTest extends JUnitSuite { def testAppendAndReadWithSequentialOffsets() { val log = new Log(logDir, logConfig.copy(segmentSize = 71), recoveryPoint = 0L, time.scheduler, time = time) val messages = (0 until 100 by 2).map(id => new Message(id.toString.getBytes)).toArray - + for(i <- 0 until messages.length) log.append(new ByteBufferMessageSet(NoCompressionCodec, messages = messages(i))) for(i <- 0 until messages.length) { @@ -161,7 +161,7 @@ class LogTest extends JUnitSuite { } assertEquals("Reading beyond the last message returns nothing.", 0, log.read(messages.length, 100, None).messageSet.size) } - + /** * This test appends a bunch of messages with non-sequential offsets and checks that we can read the correct message * from any offset less than the logEndOffset including offsets not appended. @@ -171,7 +171,7 @@ class LogTest extends JUnitSuite { val log = new Log(logDir, logConfig.copy(segmentSize = 71), 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 AtomicLong(messageIds(i)), messages = messages(i)), assignOffsets = false) @@ -182,27 +182,27 @@ class LogTest extends JUnitSuite { assertEquals("Message should match appended.", messages(idx), read.message) } } - + /** * This test covers an odd case where we have a gap in the offsets that falls at the end of a log segment. * Specifically we create a log where the last message in the first segment has offset 0. If we - * then read offset 1, we should expect this read to come from the second segment, even though the + * then read offset 1, we should expect this read to come from the second segment, even though the * first segment has the greatest lower bound on the offset. */ @Test def testReadAtLogGap() { val log = new Log(logDir, logConfig.copy(segmentSize = 300), recoveryPoint = 0L, time.scheduler, time = time) - + // keep appending until we have two segments with only a single message in the second segment while(log.numberOfSegments == 1) - log.append(new ByteBufferMessageSet(NoCompressionCodec, messages = new Message("42".getBytes))) - + log.append(new ByteBufferMessageSet(NoCompressionCodec, messages = new Message("42".getBytes))) + // now manually truncate off all but one message from the first segment to create a gap in the messages log.logSegments.head.truncateTo(1) - + assertEquals("A read should now return the last message in the log", log.logEndOffset-1, log.read(1, 200, None).messageSet.head.offset) } - + /** * Test reading at the boundary of the log, specifically * - reading from the logEndOffset should give an empty message set @@ -250,13 +250,13 @@ class LogTest extends JUnitSuite { } val lastRead = log.read(startOffset = numMessages, maxLength = 1024*1024, maxOffset = Some(numMessages + 1)).messageSet assertEquals("Should be no more messages", 0, lastRead.size) - + // check that rolling the log forced a flushed the log--the flush is asyn so retry in case of failure TestUtils.retry(1000L){ assertTrue("Log role should have forced flush", log.recoveryPoint >= log.activeSegment.baseOffset) } } - + /** * Test reads at offsets that fall within compressed message set boundaries. */ @@ -264,20 +264,20 @@ class LogTest extends JUnitSuite { def testCompressedMessages() { /* this log should roll after every messageset */ val log = new Log(logDir, logConfig.copy(segmentSize = 100), recoveryPoint = 0L, time.scheduler, time = time) - + /* append 2 compressed message sets, each with two messages giving offsets 0, 1, 2, 3 */ log.append(new ByteBufferMessageSet(DefaultCompressionCodec, new Message("hello".getBytes), new Message("there".getBytes))) log.append(new ByteBufferMessageSet(DefaultCompressionCodec, new Message("alpha".getBytes), new Message("beta".getBytes))) - + def read(offset: Int) = ByteBufferMessageSet.decompress(log.read(offset, 4096).messageSet.head.message) - + /* we should always get the first message in the compressed set when reading any offset in the set */ assertEquals("Read at offset 0 should produce 0", 0, read(0).head.offset) assertEquals("Read at offset 1 should produce 0", 0, read(1).head.offset) assertEquals("Read at offset 2 should produce 2", 2, read(2).head.offset) assertEquals("Read at offset 3 should produce 2", 2, read(3).head.offset) } - + /** * Test garbage collecting old segments */ @@ -289,7 +289,7 @@ class LogTest extends JUnitSuite { val log = new Log(logDir, logConfig.copy(segmentSize = 100), recoveryPoint = 0L, time.scheduler, time = time) for(i <- 0 until messagesToAppend) log.append(TestUtils.singleMessageSet(i.toString.getBytes)) - + var currOffset = log.logEndOffset assertEquals(currOffset, messagesToAppend) @@ -300,10 +300,10 @@ class LogTest extends JUnitSuite { assertEquals("We should still have one segment left", 1, log.numberOfSegments) assertEquals("Further collection shouldn't delete anything", 0, log.deleteOldSegments(_ => true)) assertEquals("Still no change in the logEndOffset", currOffset, log.logEndOffset) - assertEquals("Should still be able to append and should get the logEndOffset assigned to the new append", + assertEquals("Should still be able to append and should get the logEndOffset assigned to the new append", currOffset, log.append(TestUtils.singleMessageSet("hello".toString.getBytes)).firstOffset) - + // cleanup the log log.delete() } @@ -328,6 +328,64 @@ class LogTest extends JUnitSuite { } } + @Test + def testCompactedTopicConstraints() { + val keyedMessage = new Message(bytes = "this message has a key".getBytes, key = "and here it is".getBytes) + val anotherKeyedMessage = new Message(bytes = "this message also has a key".getBytes, key ="another key".getBytes) + val unkeyedMessage = new Message(bytes = "this message does not have a key".getBytes) + + val messageSetWithUnkeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, unkeyedMessage, keyedMessage) + val messageSetWithOneUnkeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, unkeyedMessage) + val messageSetWithCompressedKeyedMessage = new ByteBufferMessageSet(GZIPCompressionCodec, keyedMessage) + + val messageSetWithKeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, keyedMessage) + val messageSetWithKeyedMessages = new ByteBufferMessageSet(NoCompressionCodec, keyedMessage, anotherKeyedMessage) + + val log = new Log(logDir, logConfig.copy(compact = true), recoveryPoint = 0L, time.scheduler, time) + + try { + log.append(messageSetWithUnkeyedMessage) + fail("Compacted topics cannot accept a message without a key.") + } catch { + case e: InvalidMessageException => // this is good + } + try { + log.append(messageSetWithOneUnkeyedMessage) + fail("Compacted topics cannot accept a message without a key.") + } catch { + case e: InvalidMessageException => // this is good + } + try { + log.append(messageSetWithCompressedKeyedMessage) + fail("Compacted topics cannot accept compressed messages.") + } catch { + case e: InvalidMessageException => // this is good + } + + // the following should succeed without any InvalidMessageException + log.append(messageSetWithKeyedMessage) + log.append(messageSetWithKeyedMessages) + + // test that a compacted topic with broker-side compression type set to uncompressed can accept compressed messages + val uncompressedLog = new Log(logDir, logConfig.copy(compact = true, compressionType = "uncompressed"), + recoveryPoint = 0L, time.scheduler, time) + uncompressedLog.append(messageSetWithCompressedKeyedMessage) + uncompressedLog.append(messageSetWithKeyedMessage) + uncompressedLog.append(messageSetWithKeyedMessages) + try { + uncompressedLog.append(messageSetWithUnkeyedMessage) + fail("Compacted topics cannot accept a message without a key.") + } catch { + case e: InvalidMessageException => // this is good + } + try { + uncompressedLog.append(messageSetWithOneUnkeyedMessage) + fail("Compacted topics cannot accept a message without a key.") + } catch { + case e: InvalidMessageException => // this is good + } + } + /** * We have a max size limit on message appends, check that it is properly enforced by appending a message larger than the * setting and checking that an exception is thrown. @@ -369,13 +427,13 @@ class LogTest extends JUnitSuite { val numIndexEntries = log.activeSegment.index.entries val lastOffset = log.logEndOffset log.close() - + log = new Log(logDir, config, recoveryPoint = lastOffset, time.scheduler, time) assertEquals("Should have %d messages when log is reopened w/o recovery".format(numMessages), numMessages, log.logEndOffset) assertEquals("Should have same last index offset as before.", lastIndexOffset, log.activeSegment.index.lastOffset) assertEquals("Should have same number of index entries as before.", numIndexEntries, log.activeSegment.index.entries) log.close() - + // test recovery case log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) assertEquals("Should have %d messages when log is reopened with recovery".format(numMessages), numMessages, log.logEndOffset) @@ -383,7 +441,7 @@ class LogTest extends JUnitSuite { assertEquals("Should have same number of index entries as before.", numIndexEntries, log.activeSegment.index.entries) log.close() } - + /** * Test that if we manually delete an index segment it is rebuilt when the log is re-opened */ @@ -397,12 +455,12 @@ class LogTest extends JUnitSuite { log.append(TestUtils.singleMessageSet(TestUtils.randomBytes(10))) val indexFiles = log.logSegments.map(_.index.file) log.close() - + // delete all the index files indexFiles.foreach(_.delete()) - + // reopen the log - log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) + log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) assertEquals("Should have %d messages when log is reopened".format(numMessages), numMessages, log.logEndOffset) for(i <- 0 until numMessages) assertEquals(i, log.read(i, 100, None).messageSet.head.offset) @@ -425,10 +483,10 @@ class LogTest extends JUnitSuite { for (i<- 1 to msgPerSeg) log.append(set) - + assertEquals("There should be exactly 1 segments.", 1, log.numberOfSegments) assertEquals("Log end offset should be equal to number of messages", msgPerSeg, log.logEndOffset) - + val lastOffset = log.logEndOffset val size = log.size log.truncateTo(log.logEndOffset) // keep the entire log @@ -446,7 +504,7 @@ class LogTest extends JUnitSuite { for (i<- 1 to msgPerSeg) log.append(set) - + assertEquals("Should be back to original offset", log.logEndOffset, lastOffset) assertEquals("Should be back to original size", log.size, size) log.truncateFullyAndStartAt(log.logEndOffset - (msgPerSeg - 1)) @@ -497,23 +555,23 @@ class LogTest extends JUnitSuite { def testBogusIndexSegmentsAreRemoved() { val bogusIndex1 = Log.indexFilename(logDir, 0) val bogusIndex2 = Log.indexFilename(logDir, 5) - + val set = TestUtils.singleMessageSet("test".getBytes()) - val log = new Log(logDir, - logConfig.copy(segmentSize = set.sizeInBytes * 5, - maxIndexSize = 1000, + val log = new Log(logDir, + logConfig.copy(segmentSize = set.sizeInBytes * 5, + maxIndexSize = 1000, indexInterval = 1), recoveryPoint = 0L, time.scheduler, time) - + assertTrue("The first index file should have been replaced with a larger file", bogusIndex1.length > 0) assertFalse("The second index file should have been deleted.", bogusIndex2.exists) - + // check that we can append to the log for(i <- 0 until 10) log.append(set) - + log.delete() } @@ -523,22 +581,22 @@ class LogTest extends JUnitSuite { @Test def testReopenThenTruncate() { val set = TestUtils.singleMessageSet("test".getBytes()) - val config = logConfig.copy(segmentSize = set.sizeInBytes * 5, - maxIndexSize = 1000, + val config = logConfig.copy(segmentSize = set.sizeInBytes * 5, + maxIndexSize = 1000, indexInterval = 10000) // create a log - var log = new Log(logDir, + var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) - + // add enough messages to roll over several segments then close and re-open and attempt to truncate for(i <- 0 until 100) log.append(set) log.close() - log = new Log(logDir, + log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, @@ -547,7 +605,7 @@ class LogTest extends JUnitSuite { assertEquals("All but one segment should be deleted.", 1, log.numberOfSegments) assertEquals("Log end offset should be 3.", 3, log.logEndOffset) } - + /** * Test that deleted files are deleted after the appropriate time. */ @@ -555,38 +613,38 @@ class LogTest extends JUnitSuite { def testAsyncDelete() { val set = TestUtils.singleMessageSet("test".getBytes()) val asyncDeleteMs = 1000 - val config = logConfig.copy(segmentSize = set.sizeInBytes * 5, - fileDeleteDelayMs = asyncDeleteMs, - maxIndexSize = 1000, + val config = logConfig.copy(segmentSize = set.sizeInBytes * 5, + fileDeleteDelayMs = asyncDeleteMs, + maxIndexSize = 1000, indexInterval = 10000) val log = new Log(logDir, config, - recoveryPoint = 0L, + recoveryPoint = 0L, time.scheduler, time) - + // append some messages to create some segments for(i <- 0 until 100) log.append(set) - + // files should be renamed val segments = log.logSegments.toArray val oldFiles = segments.map(_.log.file) ++ segments.map(_.index.file) log.deleteOldSegments((s) => true) - + assertEquals("Only one segment should remain.", 1, log.numberOfSegments) - assertTrue("All log and index files should end in .deleted", segments.forall(_.log.file.getName.endsWith(Log.DeletedFileSuffix)) && + assertTrue("All log and index files should end in .deleted", segments.forall(_.log.file.getName.endsWith(Log.DeletedFileSuffix)) && segments.forall(_.index.file.getName.endsWith(Log.DeletedFileSuffix))) assertTrue("The .deleted files should still be there.", segments.forall(_.log.file.exists) && segments.forall(_.index.file.exists)) assertTrue("The original file should be gone.", oldFiles.forall(!_.exists)) - + // when enough time passes the files should be deleted val deletedFiles = segments.map(_.log.file) ++ segments.map(_.index.file) time.sleep(asyncDeleteMs + 1) assertTrue("Files should all be gone.", deletedFiles.forall(!_.exists)) } - + /** * Any files ending in .deleted should be removed when the log is re-opened. */ @@ -599,22 +657,22 @@ class LogTest extends JUnitSuite { recoveryPoint = 0L, time.scheduler, time) - + // append some messages to create some segments for(i <- 0 until 100) log.append(set) - + log.deleteOldSegments((s) => true) log.close() - - log = new Log(logDir, + + log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) assertEquals("The deleted segments should be gone.", 1, log.numberOfSegments) } - + @Test def testAppendMessageWithNullPayload() { val log = new Log(logDir, @@ -627,9 +685,9 @@ class LogTest extends JUnitSuite { assertEquals(0, messageSet.head.offset) assertTrue("Message payload should be null.", messageSet.head.message.isNull) } - + @Test - def testCorruptLog() { + def testCorruptLog() { // append some messages to create some segments val config = logConfig.copy(indexInterval = 1, maxMessageSize = 64*1024, segmentSize = 1000) val set = TestUtils.singleMessageSet("test".getBytes()) @@ -647,11 +705,11 @@ class LogTest extends JUnitSuite { log.append(set) val messages = log.logSegments.flatMap(_.log.iterator.toList) log.close() - + // corrupt index and log by appending random bytes TestUtils.appendNonsenseToFile(log.activeSegment.index.file, TestUtils.random.nextInt(1024) + 1) TestUtils.appendNonsenseToFile(log.activeSegment.log.file, TestUtils.random.nextInt(1024) + 1) - + // attempt recovery log = new Log(logDir, config, recoveryPoint, time.scheduler, time) assertEquals(numMessages, log.logEndOffset) diff --git a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala index 73a26377eb63a..07bc317bcd40c 100644 --- a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala @@ -147,11 +147,11 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { // check uncompressed offsets checkOffsets(messages, 0) var offset = 1234567 - checkOffsets(messages.assignOffsets(new AtomicLong(offset), NoCompressionCodec, NoCompressionCodec), offset) + checkOffsets(messages.validateMessagesAndAssignOffsets(new AtomicLong(offset), NoCompressionCodec, NoCompressionCodec), offset) // check compressed messages checkOffsets(compressedMessages, 0) - checkOffsets(compressedMessages.assignOffsets(new AtomicLong(offset), DefaultCompressionCodec, DefaultCompressionCodec), offset) + checkOffsets(compressedMessages.validateMessagesAndAssignOffsets(new AtomicLong(offset), DefaultCompressionCodec, DefaultCompressionCodec), offset) } /* check that offsets are assigned based on byte offset from the given base offset */ From 616987d196b654486a1261f4eed50e48560e3041 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Tue, 3 Mar 2015 11:16:38 -0800 Subject: [PATCH 149/491] KAFKA-1852; Reject offset commits to unknown topics; reviewed by Joel Koshy --- .../main/scala/kafka/server/KafkaApis.scala | 22 +++++++++---------- .../main/scala/kafka/server/KafkaServer.scala | 8 +++++-- .../scala/kafka/server/MetadataCache.scala | 7 +++++- .../scala/kafka/server/OffsetManager.scala | 19 +++++++++++----- .../unit/kafka/server/OffsetCommitTest.scala | 19 ++++++++++++++++ 5 files changed, 55 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 703886a1d48e6..35af98f0bc1b6 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -45,10 +45,10 @@ class KafkaApis(val requestChannel: RequestChannel, val controller: KafkaController, val zkClient: ZkClient, val brokerId: Int, - val config: KafkaConfig) extends Logging { + val config: KafkaConfig, + val metadataCache: MetadataCache) extends Logging { this.logIdent = "[KafkaApi-%d] ".format(brokerId) - val metadataCache = new MetadataCache(brokerId) /** * Top-level method that handles all requests and multiplexes to the right api @@ -149,7 +149,6 @@ class KafkaApis(val requestChannel: RequestChannel, val response = OffsetCommitResponse(commitStatus, offsetCommitRequest.correlationId) requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } - // call offset manager to store offsets offsetManager.storeOffsets( offsetCommitRequest.groupId, @@ -273,7 +272,7 @@ class KafkaApis(val requestChannel: RequestChannel, val hw = localReplica.highWatermark.messageOffset if (allOffsets.exists(_ > hw)) hw +: allOffsets.dropWhile(_ > hw) - else + else allOffsets } } @@ -297,19 +296,19 @@ class KafkaApis(val requestChannel: RequestChannel, val response = OffsetResponse(offsetRequest.correlationId, responseMap) requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } - + def fetchOffsets(logManager: LogManager, topicAndPartition: TopicAndPartition, timestamp: Long, maxNumOffsets: Int): Seq[Long] = { logManager.getLog(topicAndPartition) match { - case Some(log) => + case Some(log) => fetchOffsetsBefore(log, timestamp, maxNumOffsets) - case None => + case None => if (timestamp == OffsetRequest.LatestTime || timestamp == OffsetRequest.EarliestTime) Seq(0L) else Nil } } - + private def fetchOffsetsBefore(log: Log, timestamp: Long, maxNumOffsets: Int): Seq[Long] = { val segsArray = log.logSegments.toArray var offsetTimeArray: Array[(Long, Long)] = null @@ -454,7 +453,7 @@ class KafkaApis(val requestChannel: RequestChannel, import JavaConversions._ val joinGroupRequest = request.requestObj.asInstanceOf[JoinGroupRequestAndHeader] - + // the callback for sending a join-group response def sendResponseCallback(partitions: List[TopicAndPartition], generationId: Int, errorCode: Short) { val partitionList = partitions.map(tp => new TopicPartition(tp.topic, tp.partition)).toBuffer @@ -472,7 +471,7 @@ class KafkaApis(val requestChannel: RequestChannel, joinGroupRequest.body.strategy(), sendResponseCallback) } - + def handleHeartbeatRequest(request: RequestChannel.Request) { val heartbeatRequest = request.requestObj.asInstanceOf[HeartbeatRequestAndHeader] @@ -489,11 +488,10 @@ class KafkaApis(val requestChannel: RequestChannel, heartbeatRequest.body.groupGenerationId(), sendResponseCallback) } - + def close() { // TODO currently closing the API is an no-op since the API no longer maintain any modules // maybe removing the closing call in the end when KafkaAPI becomes a pure stateless layer debug("Shut down complete.") } } - diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 426e522fc9819..8e3def9e9edaf 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -68,6 +68,9 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg val kafkaScheduler = new KafkaScheduler(config.backgroundThreads) var kafkaHealthcheck: KafkaHealthcheck = null + val metadataCache: MetadataCache = new MetadataCache(config.brokerId) + + var zkClient: ZkClient = null val correlationId: AtomicInteger = new AtomicInteger(0) @@ -142,7 +145,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg consumerCoordinator.startup() /* start processing requests */ - apis = new KafkaApis(socketServer.requestChannel, replicaManager, offsetManager, consumerCoordinator, kafkaController, zkClient, config.brokerId, config) + apis = new KafkaApis(socketServer.requestChannel, replicaManager, offsetManager, consumerCoordinator, + kafkaController, zkClient, config.brokerId, config, metadataCache) requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads) brokerState.newState(RunningAsBroker) @@ -402,7 +406,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg offsetsTopicReplicationFactor = config.offsetsTopicReplicationFactor, offsetCommitTimeoutMs = config.offsetCommitTimeoutMs, offsetCommitRequiredAcks = config.offsetCommitRequiredAcks) - new OffsetManager(offsetManagerConfig, replicaManager, zkClient, kafkaScheduler) + new OffsetManager(offsetManagerConfig, replicaManager, zkClient, kafkaScheduler, metadataCache) } /** diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index 4c70aa7e0157b..6aef6e4508eca 100644 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -136,6 +136,12 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { } } + def contains(topic: String): Boolean = { + inReadLock(partitionMetadataLock) { + cache.contains(topic) + } + } + private def removePartitionInfo(topic: String, partitionId: Int) = { cache.get(topic) match { case Some(infos) => { @@ -149,4 +155,3 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { } } } - diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index c602a802858e2..d2d596222ac2f 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -86,7 +86,8 @@ object OffsetManagerConfig { class OffsetManager(val config: OffsetManagerConfig, replicaManager: ReplicaManager, zkClient: ZkClient, - scheduler: Scheduler) extends Logging with KafkaMetricsGroup { + scheduler: Scheduler, + metadataCache: MetadataCache) extends Logging with KafkaMetricsGroup { /* offsets and metadata cache */ private val offsetsCache = new Pool[GroupTopicPartition, OffsetAndMetadata] @@ -164,6 +165,7 @@ class OffsetManager(val config: OffsetManagerConfig, debug("Removed %d stale offsets in %d milliseconds.".format(numRemoved, SystemTime.milliseconds - startMs)) } + def offsetsTopicConfig: Properties = { val props = new Properties props.put(LogConfig.SegmentBytesProp, config.offsetsTopicSegmentBytes.toString) @@ -214,11 +216,16 @@ class OffsetManager(val config: OffsetManagerConfig, generationId: Int, offsetMetadata: immutable.Map[TopicAndPartition, OffsetAndMetadata], responseCallback: immutable.Map[TopicAndPartition, Short] => Unit) { + // check if there are any non-existent topics + val nonExistentTopics = offsetMetadata.filter { case (topicAndPartition, offsetMetadata) => + !metadataCache.contains(topicAndPartition.topic) + } - // first filter out partitions with offset metadata size exceeding limit + // first filter out partitions with offset metadata size exceeding limit or + // if its a non existing topic // TODO: in the future we may want to only support atomic commit and hence fail the whole commit val filteredOffsetMetadata = offsetMetadata.filter { case (topicAndPartition, offsetAndMetadata) => - validateOffsetMetadataLength(offsetAndMetadata.metadata) + validateOffsetMetadataLength(offsetAndMetadata.metadata) || nonExistentTopics.contains(topicAndPartition) } // construct the message set to append @@ -242,7 +249,7 @@ class OffsetManager(val config: OffsetManagerConfig, .format(responseStatus, offsetTopicPartition)) // construct the commit response status and insert - // the offset and metadata to cache iff the append status has no error + // the offset and metadata to cache if the append status has no error val status = responseStatus(offsetTopicPartition) val responseCode = @@ -267,7 +274,9 @@ class OffsetManager(val config: OffsetManagerConfig, // compute the final error codes for the commit response val commitStatus = offsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => - if (validateOffsetMetadataLength(offsetAndMetadata.metadata)) + if (nonExistentTopics.contains(topicAndPartition)) + (topicAndPartition, ErrorMapping.UnknownTopicOrPartitionCode) + else if (validateOffsetMetadataLength(offsetAndMetadata.metadata)) (topicAndPartition, responseCode) else (topicAndPartition, ErrorMapping.OffsetMetadataTooLargeCode) diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index a2bb8855c3c05..a37a74dc89edc 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -206,4 +206,23 @@ class OffsetCommitTest extends JUnit3Suite with ZooKeeperTestHarness { assertEquals(ErrorMapping.OffsetMetadataTooLargeCode, commitResponse1.commitStatus.get(topicAndPartition).get) } + + @Test + def testNonExistingTopicOffsetCommit() { + val topic1 = "topicDoesNotExists" + val topic2 = "topic-2" + + createTopic(zkClient, topic2, servers = Seq(server), numPartitions = 1) + + // Commit an offset + val expectedReplicaAssignment = Map(0 -> List(1)) + val commitRequest = OffsetCommitRequest(group, immutable.Map( + TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L), + TopicAndPartition(topic2, 0) -> OffsetAndMetadata(offset=42L) + )) + val commitResponse = simpleConsumer.commitOffsets(commitRequest) + + assertEquals(ErrorMapping.UnknownTopicOrPartitionCode, commitResponse.commitStatus.get(TopicAndPartition(topic1, 0)).get) + assertEquals(ErrorMapping.NoError, commitResponse.commitStatus.get(TopicAndPartition(topic2, 0)).get) + } } From c5d654acb2097eabb1784dcc88145e111a3d037b Mon Sep 17 00:00:00 2001 From: Joel Koshy Date: Tue, 3 Mar 2015 11:18:07 -0800 Subject: [PATCH 150/491] KAFKA-1499; trivial follow-up (remove unnecessary parentheses) --- core/src/main/scala/kafka/message/CompressionCodec.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/message/CompressionCodec.scala b/core/src/main/scala/kafka/message/CompressionCodec.scala index c4aa8ce0897d1..4d7ce1774f0e5 100644 --- a/core/src/main/scala/kafka/message/CompressionCodec.scala +++ b/core/src/main/scala/kafka/message/CompressionCodec.scala @@ -43,7 +43,7 @@ object BrokerCompressionCodec { val brokerCompressionCodecs = List(UncompressedCodec, SnappyCompressionCodec, LZ4CompressionCodec, GZIPCompressionCodec, ProducerCompressionCodec) val brokerCompressionOptions = brokerCompressionCodecs.map(codec => codec.name) - def isValid(compressionType: String): Boolean = brokerCompressionOptions.contains((compressionType.toLowerCase())) + def isValid(compressionType: String): Boolean = brokerCompressionOptions.contains(compressionType.toLowerCase()) def getCompressionCodec(compressionType: String): CompressionCodec = { compressionType.toLowerCase match { @@ -94,4 +94,4 @@ case object UncompressedCodec extends BrokerCompressionCodec { case object ProducerCompressionCodec extends BrokerCompressionCodec { val name = "producer" -} \ No newline at end of file +} From 57d38f672bcb85fdb20d8ca3fab9bd60d1bc8965 Mon Sep 17 00:00:00 2001 From: Aditya Auradkar Date: Tue, 3 Mar 2015 11:21:04 -0800 Subject: [PATCH 151/491] KAFKA-1986; Request failure rate should not include invalid message size and offset out of range; reviewed by Joel Koshy --- core/src/main/scala/kafka/server/ReplicaManager.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 586cf4caa95f5..c5274822c57bf 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -374,6 +374,8 @@ class ReplicaManager(val config: KafkaConfig, (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(mtl))) case mstl: MessageSetSizeTooLargeException => (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(mstl))) + case imse : InvalidMessageSizeException => + (topicAndPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(imse))) case t: Throwable => BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).failedProduceRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats.failedProduceRequestRate.mark() @@ -483,6 +485,8 @@ class ReplicaManager(val config: KafkaConfig, LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, Some(nle)) case rnae: ReplicaNotAvailableException => LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, Some(rnae)) + case oor : OffsetOutOfRangeException => + LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, fetchSize, Some(oor)) case e: Throwable => BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats().failedFetchRequestRate.mark() From 61f02fa542380e70c837fbc52deac74daec62505 Mon Sep 17 00:00:00 2001 From: Joel Koshy Date: Tue, 3 Mar 2015 16:03:24 -0800 Subject: [PATCH 152/491] KAFKA-2001; Trivial commit to prevent OffsetCommitTest from hanging --- core/src/main/scala/kafka/server/OffsetManager.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index d2d596222ac2f..8d415206152f6 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -170,7 +170,6 @@ class OffsetManager(val config: OffsetManagerConfig, val props = new Properties props.put(LogConfig.SegmentBytesProp, config.offsetsTopicSegmentBytes.toString) props.put(LogConfig.CleanupPolicyProp, "compact") - props.put(LogConfig.CompressionTypeProp, "none") props } From 4c9ad9b8a23e5487fc17579c46472c3693114d8b Mon Sep 17 00:00:00 2001 From: Joel Koshy Date: Tue, 3 Mar 2015 16:14:03 -0800 Subject: [PATCH 153/491] KAFKA-2001; Trivial commit to fix OffsetCommitTest --- core/src/main/scala/kafka/server/OffsetManager.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala index 8d415206152f6..d05e14d2018c0 100644 --- a/core/src/main/scala/kafka/server/OffsetManager.scala +++ b/core/src/main/scala/kafka/server/OffsetManager.scala @@ -170,6 +170,7 @@ class OffsetManager(val config: OffsetManagerConfig, val props = new Properties props.put(LogConfig.SegmentBytesProp, config.offsetsTopicSegmentBytes.toString) props.put(LogConfig.CleanupPolicyProp, "compact") + props.put(LogConfig.CompressionTypeProp, "uncompressed") props } From 151ae42cdf2f54d83563da7c90cd087b99300408 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Tue, 3 Mar 2015 16:53:50 -0800 Subject: [PATCH 154/491] kafka-1999; Fix failing unit-test: kafka.api.ProducerFailureHandlingTest > testNotEnoughReplicasAfterBrokerShutdown; patched by Gwen Shapira; reviewed by Jun Rao --- .../kafka/api/ProducerFailureHandlingTest.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index ba48a636dd0b0..d34ee3a40dcc8 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -32,7 +32,7 @@ import kafka.integration.KafkaServerTestHarness import kafka.utils.{TestZKUtils, ShutdownableThread, TestUtils} import org.apache.kafka.common.KafkaException -import org.apache.kafka.common.errors.{InvalidTopicException, NotEnoughReplicasException} +import org.apache.kafka.common.errors.{InvalidTopicException, NotEnoughReplicasException, NotEnoughReplicasAfterAppendException} import org.apache.kafka.clients.producer._ class ProducerFailureHandlingTest extends KafkaServerTestHarness { @@ -348,8 +348,10 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { fail("Expected exception when producing to topic with fewer brokers than min.insync.replicas") } catch { case e: ExecutionException => - if (!e.getCause.isInstanceOf[NotEnoughReplicasException]) { - fail("Expected NotEnoughReplicasException when producing to topic with fewer brokers than min.insync.replicas") + if (!e.getCause.isInstanceOf[NotEnoughReplicasException] && + !e.getCause.isInstanceOf[NotEnoughReplicasAfterAppendException]) { + fail("Expected NotEnoughReplicasException or NotEnoughReplicasAfterAppendException when producing to topic " + + "with fewer brokers than min.insync.replicas, but saw " + e.getCause) } } From 00c643e6fd6b54c7270e2b5a27441d738c7bcbd5 Mon Sep 17 00:00:00 2001 From: Joel Koshy Date: Tue, 3 Mar 2015 16:57:07 -0800 Subject: [PATCH 155/491] KAFKA-1755; trivial follow-up to fix comment in CleanerTest --- core/src/test/scala/unit/kafka/log/CleanerTest.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/CleanerTest.scala b/core/src/test/scala/unit/kafka/log/CleanerTest.scala index 70f0488baa685..a4da95f765e57 100644 --- a/core/src/test/scala/unit/kafka/log/CleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/CleanerTest.scala @@ -98,14 +98,14 @@ class CleanerTest extends JUnitSuite { // create a log with compaction turned off so we can append unkeyed messages val log = makeLog(config = logConfig.copy(segmentSize = 1024, compact = false)) - // append messages with unkeyed messages + // append unkeyed messages while(log.numberOfSegments < 2) log.append(unkeyedMessage(log.logEndOffset.toInt)) val numInvalidMessages = unkeyedMessageCountInLog(log) val sizeWithUnkeyedMessages = log.size - // append messages with unkeyed messages + // append keyed messages while(log.numberOfSegments < 3) log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt)) From 3a9f4b833bb7e86dc759361c33f4321ab043db05 Mon Sep 17 00:00:00 2001 From: Tong Li Date: Tue, 3 Mar 2015 17:15:30 -0800 Subject: [PATCH 156/491] KAFKA-1988; Fix org.apache.kafka.common.utils.Utils.abs and add Partitioner.toPositive; reviewed by Jun Rao and Guozhang Wang --- .../producer/internals/Partitioner.java | 23 +++++++++++++++---- .../org/apache/kafka/common/utils/Utils.java | 2 +- .../apache/kafka/common/utils/UtilsTest.java | 9 ++++++++ 3 files changed, 29 insertions(+), 5 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java index dfb936d8f0d58..93e799105fb6c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java @@ -24,7 +24,6 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.utils.Utils; - /** * The default partitioning strategy: *